From a2cedc58db982b9ed2bf0bf6e517cffacfa0df8c Mon Sep 17 00:00:00 2001 From: ekexium Date: Mon, 16 Jan 2023 22:01:16 +0800 Subject: [PATCH 1/4] let handleAssertionFailure work for partitioned table Signed-off-by: ekexium --- session/session.go | 51 +++++++++++++++------------- session/session_test/session_test.go | 12 +++++++ 2 files changed, 40 insertions(+), 23 deletions(-) diff --git a/session/session.go b/session/session.go index 641af7164e616..73445d4c6e6c7 100644 --- a/session/session.go +++ b/session/session.go @@ -789,31 +789,36 @@ func (s *session) handleAssertionFailure(ctx context.Context, err error) error { // if it's a record key or an index key, decode it if infoSchema, ok := s.sessionVars.TxnCtx.InfoSchema.(infoschema.InfoSchema); ok && infoSchema != nil && (tablecodec.IsRecordKey(key) || tablecodec.IsIndexKey(key)) { - tableID := tablecodec.DecodeTableID(key) - if table, ok := infoSchema.TableByID(tableID); ok { - if tablecodec.IsRecordKey(key) { - decodeFunc = consistency.DecodeRowMvccData(table.Meta()) - } else { - tableInfo := table.Meta() - _, indexID, _, e := tablecodec.DecodeIndexKey(key) - if e != nil { - logutil.Logger(ctx).Error("assertion failed but cannot decode index key", zap.Error(e)) - return err - } - var indexInfo *model.IndexInfo - for _, idx := range tableInfo.Indices { - if idx.ID == indexID { - indexInfo = idx - break - } - } - if indexInfo == nil { - return err + tableOrPartitionID := tablecodec.DecodeTableID(key) + tbl, ok := infoSchema.TableByID(tableOrPartitionID) + if !ok { + tbl, _, _ = infoSchema.FindTableByPartitionID(tableOrPartitionID) + } + if tbl == nil { + logutil.Logger(ctx).Warn("cannot find table by id", zap.Int64("tableID", tableOrPartitionID), zap.String("key", hex.EncodeToString(key))) + return newErr + } + + if tablecodec.IsRecordKey(key) { + decodeFunc = consistency.DecodeRowMvccData(tbl.Meta()) + } else { + tableInfo := tbl.Meta() + _, indexID, _, e := tablecodec.DecodeIndexKey(key) + if e != nil { + logutil.Logger(ctx).Error("assertion failed but cannot decode index key", zap.Error(e)) + return newErr + } + var indexInfo *model.IndexInfo + for _, idx := range tableInfo.Indices { + if idx.ID == indexID { + indexInfo = idx + break } - decodeFunc = consistency.DecodeIndexMvccData(indexInfo) } - } else { - logutil.Logger(ctx).Warn("assertion failed but table not found in infoschema", zap.Int64("tableID", tableID)) + if indexInfo == nil { + return newErr + } + decodeFunc = consistency.DecodeIndexMvccData(indexInfo) } } if store, ok := s.store.(helper.Storage); ok { diff --git a/session/session_test/session_test.go b/session/session_test/session_test.go index 3f8a57be18f75..f49a6910c386d 100644 --- a/session/session_test/session_test.go +++ b/session/session_test/session_test.go @@ -4091,3 +4091,15 @@ func TestIndexMergeRuntimeStats(t *testing.T) { tk.MustExec("set @@tidb_enable_collect_execution_info=0;") tk.MustQuery("select /*+ use_index_merge(t1, primary, t1a) */ * from t1 where id < 2 or a > 4 order by a").Check(testkit.Rows("1 1 1 1 1", "5 5 5 5 5")) } + +func TestHandleAssertionFailureForPartitionedTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + se := tk.Session() + se.SetConnectionID(1) + tk.MustExec("use test") + tk.MustExec("create table t (a int, b int, c int, primary key(a, b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))") + failpoint.Enable("github.com/pingcap/tidb/table/tables/addRecordForceAssertExist", "return") + defer failpoint.Disable("github.com/pingcap/tidb/table/tables/addRecordForceAssertExist") + tk.MustExec("insert into t values (1, 1, 1)") +} From 32be4f52e356152ec1aa49d9ecdc5525c2940dcc Mon Sep 17 00:00:00 2001 From: ekexium Date: Mon, 16 Jan 2023 22:58:09 +0800 Subject: [PATCH 2/4] test: make logHook a public utility and use it in TestHandleAssertionFailureForPartitionedTable Signed-off-by: ekexium --- executor/admin_test.go | 203 ++++++++------------------- session/session_test/session_test.go | 7 +- testkit/testutil/loghook.go | 111 +++++++++++++++ 3 files changed, 174 insertions(+), 147 deletions(-) create mode 100644 testkit/testutil/loghook.go diff --git a/executor/admin_test.go b/executor/admin_test.go index cd5c0664d031a..f823d4ad7bbb9 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -17,14 +17,10 @@ package executor_test import ( "context" "fmt" - "os" - "strings" "testing" "time" "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" @@ -39,12 +35,10 @@ import ( "github.com/pingcap/tidb/testkit/testutil" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" "go.uber.org/zap" - "go.uber.org/zap/zapcore" ) func TestAdminCheckIndexRange(t *testing.T) { @@ -1056,89 +1050,6 @@ func (tk *inconsistencyTestKit) rebuild() { tk.plainIndex = tables.NewIndex(tbl.Meta().ID, tbl.Meta(), tbl.Meta().Indices[1]) } -type logEntry struct { - entry zapcore.Entry - fields []zapcore.Field -} - -func (l *logEntry) checkMsg(t *testing.T, msg string) { - require.Equal(t, msg, l.entry.Message) -} - -func (l *logEntry) checkField(t *testing.T, requireFields ...zapcore.Field) { - for _, rf := range requireFields { - var f *zapcore.Field - for i, field := range l.fields { - if field.Equals(rf) { - f = &l.fields[i] - break - } - } - require.NotNilf(t, f, "matched log fields %s:%s not found in log", rf.Key, rf) - } -} - -func (l *logEntry) checkFieldNotEmpty(t *testing.T, fieldName string) { - var f *zapcore.Field - for i, field := range l.fields { - if field.Key == fieldName { - f = &l.fields[i] - break - } - } - require.NotNilf(t, f, "log field %s not found in log", fieldName) - require.NotEmpty(t, f.String) -} - -type logHook struct { - zapcore.Core - logs []logEntry - enc zapcore.Encoder - messageFilter string -} - -func (h *logHook) Write(entry zapcore.Entry, fields []zapcore.Field) error { - h.logs = append(h.logs, logEntry{entry: entry, fields: fields}) - return nil -} - -func (h *logHook) Check(entry zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { - if len(h.messageFilter) > 0 && !strings.Contains(entry.Message, h.messageFilter) { - return nil - } - return ce.AddCore(entry, h) -} - -func (h *logHook) encode(entry *logEntry) (string, error) { - buffer, err := h.enc.EncodeEntry(entry.entry, entry.fields) - if err != nil { - return "", err - } - return buffer.String(), nil -} - -func (h *logHook) checkLogCount(t *testing.T, expected int) { - logsStr := make([]string, len(h.logs)) - for i, item := range h.logs { - var err error - logsStr[i], err = h.encode(&item) - require.NoError(t, err) - } - // Check the length of strings, so that in case the test fails, the error message will be printed. - require.Len(t, logsStr, expected) -} - -func withLogHook(ctx context.Context, t *testing.T, msgFilter string) (newCtx context.Context, hook *logHook) { - conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} - _, r, _ := log.InitLogger(conf) - enc, err := log.NewTextEncoder(&config.GetGlobalConfig().Log.ToLogConfig().Config) - require.NoError(t, err) - hook = &logHook{r.Core, nil, enc, msgFilter} - logger := zap.New(hook) - newCtx = context.WithValue(ctx, logutil.CtxLogKey, logger) - return -} - func TestCheckFailReport(t *testing.T) { store := testkit.CreateMockStore(t) tk := newInconsistencyKit(t, testkit.NewAsyncTestKit(t, store), newDefaultOpt()) @@ -1153,16 +1064,16 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, tk.uniqueIndex.Delete(tk.sctx, txn, types.MakeDatums(1), kv.IntHandle(1))) require.NoError(t, txn.Commit(tk.ctx)) - ctx, hook := withLogHook(tk.ctx, t, "inconsistency") + ctx, hook := testutil.WithLogHook(tk.ctx, t, "inconsistency") tk.MustGetErrMsg(ctx, "admin check table admin_test", "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindInt64 1]\"") - hook.checkLogCount(t, 1) - hook.logs[0].checkMsg(t, "admin check found data inconsistency") - hook.logs[0].checkField(t, + hook.CheckLogCount(t, 1) + hook.Logs[0].CheckMsg(t, "admin check found data inconsistency") + hook.Logs[0].CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "uk1"), zap.Stringer("row_id", kv.IntHandle(1)), ) - hook.logs[0].checkFieldNotEmpty(t, "row_mvcc") + hook.Logs[0].CheckFieldNotEmpty(t, "row_mvcc") }() // row more than plain index @@ -1175,16 +1086,16 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, tk.plainIndex.Delete(tk.sctx, txn, []types.Datum{types.NewStringDatum("10")}, kv.IntHandle(1))) require.NoError(t, txn.Commit(tk.ctx)) - ctx, hook := withLogHook(tk.ctx, t, "inconsistency") + ctx, hook := testutil.WithLogHook(tk.ctx, t, "inconsistency") tk.MustGetErrMsg(ctx, "admin check table admin_test", "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindString 10]\"") - hook.checkLogCount(t, 1) - hook.logs[0].checkMsg(t, "admin check found data inconsistency") - hook.logs[0].checkField(t, + hook.CheckLogCount(t, 1) + hook.Logs[0].CheckMsg(t, "admin check found data inconsistency") + hook.Logs[0].CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "k2"), zap.Stringer("row_id", kv.IntHandle(1)), ) - hook.logs[0].checkFieldNotEmpty(t, "row_mvcc") + hook.Logs[0].CheckFieldNotEmpty(t, "row_mvcc") }() // row is missed for plain key @@ -1197,38 +1108,38 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, err) require.NoError(t, txn.Commit(tk.ctx)) - ctx, hook := withLogHook(tk.ctx, t, "inconsistency") + ctx, hook := testutil.WithLogHook(tk.ctx, t, "inconsistency") tk.MustGetErrMsg(ctx, "admin check table admin_test", "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"handle: 1, values: [KindString 100 KindInt64 1]\" != record-values:\"\"") - hook.checkLogCount(t, 1) - logEntry := hook.logs[0] - logEntry.checkMsg(t, "admin check found data inconsistency") - logEntry.checkField(t, + hook.CheckLogCount(t, 1) + logEntry := hook.Logs[0] + logEntry.CheckMsg(t, "admin check found data inconsistency") + logEntry.CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "k2"), zap.Stringer("row_id", kv.IntHandle(1)), ) - logEntry.checkFieldNotEmpty(t, "row_mvcc") - logEntry.checkFieldNotEmpty(t, "index_mvcc") + logEntry.CheckFieldNotEmpty(t, "row_mvcc") + logEntry.CheckFieldNotEmpty(t, "index_mvcc") // test inconsistency check in index lookup - ctx, hook = withLogHook(tk.ctx, t, "") + ctx, hook = testutil.WithLogHook(tk.ctx, t, "") rs, err := tk.Exec(ctx, "select * from admin_test use index(k2) where c3 = '100'") require.NoError(t, err) _, err = session.GetRows4Test(ctx, testkit.TryRetrieveSession(ctx), rs) require.Error(t, err) require.Equal(t, "[executor:8133]data inconsistency in table: admin_test, index: k2, index-count:1 != record-count:0", err.Error()) - hook.checkLogCount(t, 1) - logEntry = hook.logs[0] - logEntry.checkMsg(t, "indexLookup found data inconsistency") - logEntry.checkField(t, + hook.CheckLogCount(t, 1) + logEntry = hook.Logs[0] + logEntry.CheckMsg(t, "indexLookup found data inconsistency") + logEntry.CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "k2"), zap.Int64("table_cnt", 0), zap.Int64("index_cnt", 1), zap.String("missing_handles", `[1]`), ) - logEntry.checkFieldNotEmpty(t, "row_mvcc_0") + logEntry.CheckFieldNotEmpty(t, "row_mvcc_0") }() // row is missed for unique key @@ -1241,37 +1152,37 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, err) require.NoError(t, txn.Commit(tk.ctx)) - ctx, hook := withLogHook(tk.ctx, t, "inconsistency") + ctx, hook := testutil.WithLogHook(tk.ctx, t, "inconsistency") tk.MustGetErrMsg(ctx, "admin check table admin_test", "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"handle: 1, values: [KindInt64 10 KindInt64 1]\" != record-values:\"\"") - hook.checkLogCount(t, 1) - logEntry := hook.logs[0] - logEntry.checkMsg(t, "admin check found data inconsistency") - logEntry.checkField(t, + hook.CheckLogCount(t, 1) + logEntry := hook.Logs[0] + logEntry.CheckMsg(t, "admin check found data inconsistency") + logEntry.CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "uk1"), zap.Stringer("row_id", kv.IntHandle(1)), ) - logEntry.checkFieldNotEmpty(t, "row_mvcc") - logEntry.checkFieldNotEmpty(t, "index_mvcc") + logEntry.CheckFieldNotEmpty(t, "row_mvcc") + logEntry.CheckFieldNotEmpty(t, "index_mvcc") // test inconsistency check in point-get - ctx, hook = withLogHook(tk.ctx, t, "") + ctx, hook = testutil.WithLogHook(tk.ctx, t, "") rs, err := tk.Exec(ctx, "select * from admin_test use index(uk1) where c2 = 10") require.NoError(t, err) _, err = session.GetRows4Test(ctx, testkit.TryRetrieveSession(ctx), rs) require.Error(t, err) - hook.checkLogCount(t, 1) - logEntry = hook.logs[0] - logEntry.checkMsg(t, "indexLookup found data inconsistency") - logEntry.checkField(t, + hook.CheckLogCount(t, 1) + logEntry = hook.Logs[0] + logEntry.CheckMsg(t, "indexLookup found data inconsistency") + logEntry.CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "uk1"), zap.Int64("table_cnt", 0), zap.Int64("index_cnt", 1), zap.String("missing_handles", `[1]`), ) - logEntry.checkFieldNotEmpty(t, "row_mvcc_0") + logEntry.CheckFieldNotEmpty(t, "row_mvcc_0") }() // handle match but value is different for uk @@ -1285,20 +1196,20 @@ func TestCheckFailReport(t *testing.T) { _, err = tk.uniqueIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewIntDatum(20)}, kv.IntHandle(1), nil) require.NoError(t, err) require.NoError(t, txn.Commit(tk.ctx)) - ctx, hook := withLogHook(tk.ctx, t, "inconsistency") + ctx, hook := testutil.WithLogHook(tk.ctx, t, "inconsistency") tk.MustGetErrMsg(ctx, "admin check table admin_test", "[executor:8134]data inconsistency in table: admin_test, index: uk1, col: c2, handle: \"1\", index-values:\"KindInt64 20\" != record-values:\"KindInt64 10\", compare err:") - hook.checkLogCount(t, 1) - logEntry := hook.logs[0] - logEntry.checkMsg(t, "admin check found data inconsistency") - logEntry.checkField(t, + hook.CheckLogCount(t, 1) + logEntry := hook.Logs[0] + logEntry.CheckMsg(t, "admin check found data inconsistency") + logEntry.CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "uk1"), zap.Stringer("row_id", kv.IntHandle(1)), zap.String("col", "c2"), ) - logEntry.checkFieldNotEmpty(t, "row_mvcc") - logEntry.checkFieldNotEmpty(t, "index_mvcc") + logEntry.CheckFieldNotEmpty(t, "row_mvcc") + logEntry.CheckFieldNotEmpty(t, "index_mvcc") }() // handle match but value is different for plain key @@ -1312,20 +1223,20 @@ func TestCheckFailReport(t *testing.T) { _, err = tk.plainIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewStringDatum("200")}, kv.IntHandle(1), nil) require.NoError(t, err) require.NoError(t, txn.Commit(tk.ctx)) - ctx, hook := withLogHook(tk.ctx, t, "inconsistency") + ctx, hook := testutil.WithLogHook(tk.ctx, t, "inconsistency") tk.MustGetErrMsg(ctx, "admin check table admin_test", "[executor:8134]data inconsistency in table: admin_test, index: k2, col: c3, handle: \"1\", index-values:\"KindString 200\" != record-values:\"KindString 100\", compare err:") - hook.checkLogCount(t, 1) - logEntry := hook.logs[0] - logEntry.checkMsg(t, "admin check found data inconsistency") - logEntry.checkField(t, + hook.CheckLogCount(t, 1) + logEntry := hook.Logs[0] + logEntry.CheckMsg(t, "admin check found data inconsistency") + logEntry.CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "k2"), zap.Stringer("row_id", kv.IntHandle(1)), zap.String("col", "c3"), ) - logEntry.checkFieldNotEmpty(t, "row_mvcc") - logEntry.checkFieldNotEmpty(t, "index_mvcc") + logEntry.CheckFieldNotEmpty(t, "row_mvcc") + logEntry.CheckFieldNotEmpty(t, "index_mvcc") }() // test binary column. @@ -1348,7 +1259,7 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, err) require.NoError(t, txn.Commit(tk.ctx)) - ctx, hook := withLogHook(tk.ctx, t, "inconsistency") + ctx, hook := testutil.WithLogHook(tk.ctx, t, "inconsistency") // TODO(tiancaiamao): admin check doesn't support the chunk protocol. // Remove this after https://github.com/pingcap/tidb/issues/35156 @@ -1356,16 +1267,16 @@ func TestCheckFailReport(t *testing.T) { tk.MustGetErrMsg(ctx, "admin check table admin_test", `[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 282574488403969, index-values:"handle: 282574488403969, values: [KindInt64 282578800083201 KindInt64 282574488403969]" != record-values:""`) - hook.checkLogCount(t, 1) - logEntry := hook.logs[0] - logEntry.checkMsg(t, "admin check found data inconsistency") - logEntry.checkField(t, + hook.CheckLogCount(t, 1) + logEntry := hook.Logs[0] + logEntry.CheckMsg(t, "admin check found data inconsistency") + logEntry.CheckField(t, zap.String("table_name", "admin_test"), zap.String("index_name", "uk1"), zap.Stringer("row_id", kv.IntHandle(282574488403969)), ) - logEntry.checkFieldNotEmpty(t, "row_mvcc") - logEntry.checkFieldNotEmpty(t, "index_mvcc") + logEntry.CheckFieldNotEmpty(t, "row_mvcc") + logEntry.CheckFieldNotEmpty(t, "index_mvcc") }() } diff --git a/session/session_test/session_test.go b/session/session_test/session_test.go index f49a6910c386d..3a0a35b5b9332 100644 --- a/session/session_test/session_test.go +++ b/session/session_test/session_test.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testutil" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/memory" @@ -4101,5 +4102,9 @@ func TestHandleAssertionFailureForPartitionedTable(t *testing.T) { tk.MustExec("create table t (a int, b int, c int, primary key(a, b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))") failpoint.Enable("github.com/pingcap/tidb/table/tables/addRecordForceAssertExist", "return") defer failpoint.Disable("github.com/pingcap/tidb/table/tables/addRecordForceAssertExist") - tk.MustExec("insert into t values (1, 1, 1)") + + ctx, hook := testutil.WithLogHook(context.TODO(), t, "table") + _, err := tk.ExecWithContext(ctx, "insert into t values (1, 1, 1)") + require.ErrorContains(t, err, "assertion") + hook.CheckLogCount(t, 0) } diff --git a/testkit/testutil/loghook.go b/testkit/testutil/loghook.go new file mode 100644 index 0000000000000..71a0e516f4858 --- /dev/null +++ b/testkit/testutil/loghook.go @@ -0,0 +1,111 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testutil + +import ( + "context" + "os" + "strings" + "testing" + + "github.com/pingcap/log" + "github.com/pingcap/tidb/util/logutil" + "github.com/stretchr/testify/require" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" +) + +type logEntry struct { + entry zapcore.Entry + fields []zapcore.Field +} + +func (l *logEntry) CheckMsg(t *testing.T, msg string) { + require.Equal(t, msg, l.entry.Message) +} + +func (l *logEntry) CheckField(t *testing.T, requireFields ...zapcore.Field) { + for _, rf := range requireFields { + var f *zapcore.Field + for i, field := range l.fields { + if field.Equals(rf) { + f = &l.fields[i] + break + } + } + require.NotNilf(t, f, "matched log fields %s:%s not found in log", rf.Key, rf) + } +} + +func (l *logEntry) CheckFieldNotEmpty(t *testing.T, fieldName string) { + var f *zapcore.Field + for i, field := range l.fields { + if field.Key == fieldName { + f = &l.fields[i] + break + } + } + require.NotNilf(t, f, "log field %s not found in log", fieldName) + require.NotEmpty(t, f.String) +} + +type LogHook struct { + zapcore.Core + Logs []logEntry + enc zapcore.Encoder + messageFilter string +} + +func (h *LogHook) Write(entry zapcore.Entry, fields []zapcore.Field) error { + h.Logs = append(h.Logs, logEntry{entry: entry, fields: fields}) + return nil +} + +func (h *LogHook) Check(entry zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { + if len(h.messageFilter) > 0 && !strings.Contains(entry.Message, h.messageFilter) { + return nil + } + return ce.AddCore(entry, h) +} + +func (h *LogHook) encode(entry *logEntry) (string, error) { + buffer, err := h.enc.EncodeEntry(entry.entry, entry.fields) + if err != nil { + return "", err + } + return buffer.String(), nil +} + +func (h *LogHook) CheckLogCount(t *testing.T, expected int) { + logsStr := make([]string, len(h.Logs)) + for i, item := range h.Logs { + var err error + logsStr[i], err = h.encode(&item) + require.NoError(t, err) + } + // Check the length of strings, so that in case the test fails, the error message will be printed. + require.Len(t, logsStr, expected) +} + +func WithLogHook(ctx context.Context, t *testing.T, msgFilter string) (newCtx context.Context, hook *LogHook) { + conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} + _, r, _ := log.InitLogger(conf) + enc, err := log.NewTextEncoder(&log.Config{Format: "text"}) + require.NoError(t, err) + hook = &LogHook{r.Core, nil, enc, msgFilter} + logger := zap.New(hook) + newCtx = context.WithValue(ctx, logutil.CtxLogKey, logger) + return +} From 5638294cbde188774ac13fbdde5e69caa280a816 Mon Sep 17 00:00:00 2001 From: ekexium Date: Tue, 17 Jan 2023 11:16:48 +0800 Subject: [PATCH 3/4] comments for pub functions Signed-off-by: ekexium --- testkit/testutil/loghook.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/testkit/testutil/loghook.go b/testkit/testutil/loghook.go index 71a0e516f4858..ff388d671ca71 100644 --- a/testkit/testutil/loghook.go +++ b/testkit/testutil/loghook.go @@ -61,6 +61,7 @@ func (l *logEntry) CheckFieldNotEmpty(t *testing.T, fieldName string) { require.NotEmpty(t, f.String) } +// LogHook captures logs, mainly for testing type LogHook struct { zapcore.Core Logs []logEntry @@ -68,11 +69,13 @@ type LogHook struct { messageFilter string } +// Write captures the log and save it func (h *LogHook) Write(entry zapcore.Entry, fields []zapcore.Field) error { h.Logs = append(h.Logs, logEntry{entry: entry, fields: fields}) return nil } +// Check implements the string filter func (h *LogHook) Check(entry zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { if len(h.messageFilter) > 0 && !strings.Contains(entry.Message, h.messageFilter) { return nil @@ -88,6 +91,7 @@ func (h *LogHook) encode(entry *logEntry) (string, error) { return buffer.String(), nil } +// CheckLogCount is a helper function to assert the number of logs captured func (h *LogHook) CheckLogCount(t *testing.T, expected int) { logsStr := make([]string, len(h.Logs)) for i, item := range h.Logs { @@ -99,6 +103,7 @@ func (h *LogHook) CheckLogCount(t *testing.T, expected int) { require.Len(t, logsStr, expected) } +// WithLogHook is a helper function to use with LogHook. It returns a context whose logger is replaced with the hook. func WithLogHook(ctx context.Context, t *testing.T, msgFilter string) (newCtx context.Context, hook *LogHook) { conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} _, r, _ := log.InitLogger(conf) From 55d7c0e9e177abd8e550ef7c1c2c53388e2f01be Mon Sep 17 00:00:00 2001 From: ekexium Date: Tue, 17 Jan 2023 13:27:01 +0800 Subject: [PATCH 4/4] update bazel Signed-off-by: ekexium --- session/session_test/BUILD.bazel | 1 + testkit/testutil/BUILD.bazel | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/session/session_test/BUILD.bazel b/session/session_test/BUILD.bazel index f0fa774e9f9e3..597bb73b6f221 100644 --- a/session/session_test/BUILD.bazel +++ b/session/session_test/BUILD.bazel @@ -33,6 +33,7 @@ go_test( "//testkit", "//testkit/testmain", "//testkit/testsetup", + "//testkit/testutil", "//types", "//util", "//util/memory", diff --git a/testkit/testutil/BUILD.bazel b/testkit/testutil/BUILD.bazel index dd662dc0aa80f..3f4ed7312c01c 100644 --- a/testkit/testutil/BUILD.bazel +++ b/testkit/testutil/BUILD.bazel @@ -4,6 +4,7 @@ go_library( name = "testutil", srcs = [ "handle.go", + "loghook.go", "require.go", ], importpath = "github.com/pingcap/tidb/testkit/testutil", @@ -15,8 +16,12 @@ go_library( "//types", "//util/codec", "//util/collate", + "//util/logutil", + "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", "@org_golang_x_exp//slices", + "@org_uber_go_zap//:zap", + "@org_uber_go_zap//zapcore", ], )