diff --git a/errno/errcode.go b/errno/errcode.go index 638e1b889a902..f9638af7a2286 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1005,8 +1005,8 @@ const ( ErrMultiStatementDisabled = 8130 ErrPartitionStatsMissing = 8131 ErrNotSupportedWithSem = 8132 - ErrDataInConsistentExtraIndex = 8133 - ErrDataInConsistentMisMatchIndex = 8134 + ErrDataInconsistentMismatchCount = 8133 + ErrDataInconsistentMismatchIndex = 8134 ErrAsOf = 8135 // Error codes used by TiDB ddl package @@ -1033,7 +1033,7 @@ const ( ErrWriteOnSnapshot = 8220 ErrInvalidKey = 8221 ErrInvalidIndexKey = 8222 - ErrDataInConsistent = 8223 + ErrDataInconsistent = 8223 ErrDDLJobNotFound = 8224 ErrCancelFinishedDDLJob = 8225 ErrCannotCancelDDLJob = 8226 diff --git a/errno/errname.go b/errno/errname.go index 5ee1e470357bf..731cdf551eb12 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -998,7 +998,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrWriteOnSnapshot: mysql.Message("write on snapshot", nil), ErrInvalidKey: mysql.Message("invalid key", nil), ErrInvalidIndexKey: mysql.Message("invalid index key", nil), - ErrDataInConsistent: mysql.Message("index:%#v != record:%#v", []int{0, 1}), + ErrDataInconsistent: mysql.Message("data inconsistency in table: %s, index: %s, handle: %s, index-values:%#v != record-values:%#v", []int{2, 3, 4}), ErrDDLReorgElementNotExist: mysql.Message("DDL reorg element does not exist", nil), ErrDDLJobNotFound: mysql.Message("DDL Job:%v not found", nil), ErrCancelFinishedDDLJob: mysql.Message("This job:%v is finished, so can't be cancelled", nil), @@ -1006,8 +1006,8 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrUnknownAllocatorType: mysql.Message("Invalid allocator type", nil), ErrAutoRandReadFailed: mysql.Message("Failed to read auto-random value from storage engine", nil), ErrInvalidIncrementAndOffset: mysql.Message("Invalid auto_increment settings: auto_increment_increment: %d, auto_increment_offset: %d, both of them must be in range [1..65535]", nil), - ErrDataInConsistentExtraIndex: mysql.Message("handle %#v, index:%#v != record:%#v", []int{0, 1, 2}), - ErrDataInConsistentMisMatchIndex: mysql.Message("col %s, handle %#v, index:%#v != record:%#v, compare err:%#v", []int{1, 2, 3, 4}), + ErrDataInconsistentMismatchCount: mysql.Message("data inconsistency in table: %s, index: %s, index-count:%d != record-count:%d", nil), + ErrDataInconsistentMismatchIndex: mysql.Message("data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v", []int{3, 4, 5, 6}), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index c9dfc9c1eda91..094d0c5b3e04e 100644 --- a/errors.toml +++ b/errors.toml @@ -223,7 +223,7 @@ TiDB admin check table failed. ["admin:8223"] error = ''' -index:%#v != record:%#v +data inconsistency in table: %s, index: %s, handle: %s, index-values:%#v != record-values:%#v ''' ["admin:8224"] @@ -873,12 +873,12 @@ Export failed: %s ["executor:8133"] error = ''' -handle %#v, index:%#v != record:%#v +data inconsistency in table: %s, index: %s, index-count:%d != record-count:%d ''' ["executor:8134"] error = ''' -col %s, handle %#v, index:%#v != record:%#v, compare err:%#v +data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v ''' ["executor:8212"] diff --git a/executor/admin_test.go b/executor/admin_test.go index 1ed5edbbf974f..091d6fcf49216 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -17,22 +17,35 @@ package executor_test import ( "context" "fmt" + "os" + "strings" + "testing" "time" . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/domain" mysql "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" + testkit2 "github.com/pingcap/tidb/testkit" "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/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) func (s *testSuite1) TestAdminCheckIndexRange(c *C) { @@ -179,7 +192,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + c.Assert(consistency.ErrAdminCheckInconsistent.Equal(err), IsTrue) err = tk.ExecToErr("admin check index admin_test c2") c.Assert(err, NotNil) @@ -278,7 +291,7 @@ func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) { c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) - tk.MustGetErrCode("admin check table t", mysql.ErrAdminCheckTable) + tk.MustGetErrCode("admin check table t", mysql.ErrDataInconsistent) tk.MustGetErrCode("admin check index t idx", mysql.ErrAdminCheckTable) tk.MustQuery("SELECT COUNT(*) FROM t USE INDEX(idx)").Check(testkit.Rows("2")) @@ -313,7 +326,7 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + c.Assert(consistency.ErrAdminCheckInconsistent.Equal(err), IsTrue) r := tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") r.Check(testkit.Rows("2")) @@ -789,8 +802,8 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test_p") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("[executor:8003]admin_test_p err:[admin:8223]index: != record:&admin.RecordData{Handle:%d, Values:[]types.Datum{types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}}}", i, i)) - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + c.Assert(err.Error(), Equals, fmt.Sprintf("[admin:8223]data inconsistency in table: admin_test_p, index: idx, handle: %d, index-values:\"\" != record-values:\"handle: %d, values: [KindInt64 %d]\"", i, i, i)) + c.Assert(consistency.ErrAdminCheckInconsistent.Equal(err), IsTrue) // TODO: fix admin recover for partition table. // r := tk.MustQuery("admin recover index admin_test_p idx") // r.Check(testkit.Rows("0 0")) @@ -818,7 +831,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test_p") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("[executor:8133]handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:", i+8, i+8)) + c.Assert(err.Error(), Equals, fmt.Sprintf("[admin:8223]data inconsistency in table: admin_test_p, index: idx, handle: %d, index-values:\"handle: %d, values: [KindInt64 %d KindInt64 %d]\" != record-values:\"\"", i+8, i+8, i+8, i+8)) // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) @@ -841,7 +854,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test_p") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("[executor:8134]col c2, handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:", i, i+8, i)) + c.Assert(err.Error(), Equals, fmt.Sprintf("[executor:8134]data inconsistency in table: admin_test_p, index: idx, col: c2, handle: \"%d\", index-values:\"KindInt64 %d\" != record-values:\"KindInt64 %d\", compare err:", i, i+8, i)) // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) @@ -853,6 +866,358 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { } } +const dbName, tblName = "test", "admin_test" + +type inconsistencyTestKit struct { + *testkit2.AsyncTestKit + uniqueIndex table.Index + plainIndex table.Index + ctx context.Context + sctx *stmtctx.StatementContext + t *testing.T +} + +type kitOpt struct { + pkColType string + idxColType string + ukColType string + clustered string +} + +func newDefaultOpt() *kitOpt { + return &kitOpt{ + pkColType: "int", + idxColType: "int", + ukColType: "varchar(255)", + } +} + +func newInconsistencyKit(t *testing.T, tk *testkit2.AsyncTestKit, opt *kitOpt) *inconsistencyTestKit { + ctx := tk.OpenSession(context.Background(), dbName) + se := testkit2.TryRetrieveSession(ctx) + i := &inconsistencyTestKit{ + AsyncTestKit: tk, + ctx: ctx, + sctx: se.GetSessionVars().StmtCtx, + t: t, + } + tk.MustExec(i.ctx, "drop table if exists "+tblName) + tk.MustExec(i.ctx, + fmt.Sprintf("create table %s (c1 %s, c2 %s, c3 %s, primary key(c1) %s, index uk1(c2), index k2(c3))", + tblName, opt.pkColType, opt.idxColType, opt.ukColType, opt.clustered), + ) + i.rebuild() + return i +} + +func (tk *inconsistencyTestKit) rebuild() { + tk.MustExec(tk.ctx, "truncate table "+tblName) + is := domain.GetDomain(testkit2.TryRetrieveSession(tk.ctx)).InfoSchema() + tbl, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) + require.NoError(tk.t, err) + tk.uniqueIndex = tables.NewIndex(tbl.Meta().ID, tbl.Meta(), tbl.Meta().Indices[0]) + 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 _, field := range l.fields { + if field.Equals(rf) { + f = &field + 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 _, field := range l.fields { + if field.Key == fieldName { + f = &field + 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 + 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 withLogHook(ctx context.Context, msgFilter string) (newCtx context.Context, hook *logHook) { + conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} + _, r, _ := log.InitLogger(conf) + hook = &logHook{r.Core, nil, msgFilter} + logger := zap.New(hook) + newCtx = context.WithValue(ctx, logutil.CtxLogKey, logger) + return +} + +func TestCheckFailReport(t *testing.T) { + t.Parallel() + + store, clean := testkit2.CreateMockStore(t) + defer clean() + tk := newInconsistencyKit(t, testkit2.NewAsyncTestKit(t, store), newDefaultOpt()) + + // row more than unique index + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 1, '10')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + 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, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindInt64 1]\"", err.Error()) + require.Len(t, hook.logs, 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") + }() + + // row more than plain index + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 1, '10')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + 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, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindString 10]\"", err.Error()) + require.Len(t, hook.logs, 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") + }() + + // row is missed for plain key + func() { + defer tk.rebuild() + + txn, err := store.Begin() + require.NoError(t, err) + _, err = tk.plainIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewStringDatum("100")}, kv.IntHandle(1), nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"handle: 1, values: [KindString 100 KindInt64 1]\" != record-values:\"\"", err.Error()) + require.Len(t, hook.logs, 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") + + // test inconsistency check in index lookup + ctx, hook = withLogHook(tk.ctx, "") + rs, err := tk.Exec(ctx, "select * from admin_test use index(k2) where c3 = '100'") + require.NoError(t, err) + _, err = session.GetRows4Test(ctx, testkit2.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()) + require.Len(t, hook.logs, 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") + }() + + // row is missed for unique key + func() { + defer tk.rebuild() + + txn, err := store.Begin() + require.NoError(t, err) + _, err = tk.uniqueIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewIntDatum(10)}, kv.IntHandle(1), nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"handle: 1, values: [KindInt64 10 KindInt64 1]\" != record-values:\"\"", err.Error()) + require.Len(t, hook.logs, 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") + + // test inconsistency check in point-get + ctx, hook = withLogHook(tk.ctx, "") + rs, err := tk.Exec(ctx, "select * from admin_test use index(uk1) where c2 = 10") + require.NoError(t, err) + _, err = session.GetRows4Test(ctx, testkit2.TryRetrieveSession(ctx), rs) + require.Error(t, err) + require.Len(t, hook.logs, 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") + }() + + // handle match but value is different for uk + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 10, '100')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + require.NoError(t, tk.uniqueIndex.Delete(tk.sctx, txn, []types.Datum{types.NewIntDatum(10)}, kv.IntHandle(1))) + _, 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, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[executor:8134]data inconsistency in table: admin_test, index: uk1, col: c2, handle: \"1\", index-values:\"KindInt64 20\" != record-values:\"KindInt64 10\", compare err:", err.Error()) + require.Len(t, hook.logs, 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") + }() + + // handle match but value is different for plain key + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 10, '100')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + require.NoError(t, tk.plainIndex.Delete(tk.sctx, txn, []types.Datum{types.NewStringDatum("100")}, kv.IntHandle(1))) + _, 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, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[executor:8134]data inconsistency in table: admin_test, index: k2, col: c3, handle: \"1\", index-values:\"KindString 200\" != record-values:\"KindString 100\", compare err:", err.Error()) + require.Len(t, hook.logs, 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") + }() + + // test binary column. + opt := newDefaultOpt() + opt.clustered = "clustered" + opt.pkColType = "varbinary(300)" + opt.idxColType = "varbinary(300)" + opt.ukColType = "varbinary(300)" + tk = newInconsistencyKit(t, testkit2.NewAsyncTestKit(t, store), newDefaultOpt()) + func() { + defer tk.rebuild() + + txn, err := store.Begin() + require.NoError(t, err) + encoded, err := codec.EncodeKey(new(stmtctx.StatementContext), nil, types.NewBytesDatum([]byte{1, 0, 1, 0, 0, 1, 1})) + require.Nil(t, err) + hd, err := kv.NewCommonHandle(encoded) + require.NoError(t, err) + _, err = tk.uniqueIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewBytesDatum([]byte{1, 1, 0, 1, 1, 1, 1, 0})}, hd, nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, `[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 282574488403969, index-values:"handle: 282574488403969, values: [KindInt64 282578800083201 KindInt64 282574488403969]" != record-values:""`, err.Error()) + require.Len(t, hook.logs, 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") + }() +} + func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -886,13 +1251,12 @@ func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, - "[executor:8003]admin_test err:[admin:8223]index: != record:&admin.RecordData{Handle:-1, Values:[]types.Datum{types.Datum{k:0x1, decimal:0x0, length:0x0, i:-10, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}}}") - c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + c.Assert(err.Error(), Equals, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: -1, index-values:\"\" != record-values:\"handle: -1, values: [KindInt64 -10]\"") + c.Assert(consistency.ErrAdminCheckInconsistent.Equal(err), IsTrue) tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8003]admin_test err:[admin:8223]index:\"?\" != record:\"?\"") + c.Assert(err.Error(), Equals, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: ?, index-values:\"?\" != record-values:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") r := tk.MustQuery("admin recover index admin_test c2") r.Check(testkit.Rows("1 7")) @@ -909,11 +1273,11 @@ func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8133]handle 0, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:0, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") + c.Assert(err.Error(), Equals, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: 0, index-values:\"handle: 0, values: [KindInt64 0 KindInt64 0]\" != record-values:\"\"") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8133]handle \"?\", index:\"?\" != record:\"?\"") + c.Assert(err.Error(), Equals, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: ?, index-values:\"?\" != record-values:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Add one row of index. @@ -932,11 +1296,11 @@ func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle 2, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:13, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:12, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + c.Assert(err.Error(), Equals, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"2\", index-values:\"KindInt64 13\" != record-values:\"KindInt64 12\", compare err:") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + c.Assert(err.Error(), Equals, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"?\", index-values:\"?\" != record-values:\"?\", compare err:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Table count = index count. @@ -951,11 +1315,11 @@ func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + c.Assert(err.Error(), Equals, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"10\", index-values:\"KindInt64 19\" != record-values:\"KindInt64 20\", compare err:") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + c.Assert(err.Error(), Equals, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"?\", index-values:\"?\" != record-values:\"?\", compare err:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Table count = index count. @@ -970,11 +1334,11 @@ func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + c.Assert(err.Error(), Equals, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"10\", index-values:\"KindInt64 19\" != record-values:\"KindInt64 20\", compare err:") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + c.Assert(err.Error(), Equals, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"?\", index-values:\"?\" != record-values:\"?\", compare err:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Recover records. diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 0b6c0073c946c..62ab945a3b185 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/math" "github.com/pingcap/tidb/util/rowcodec" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" @@ -469,8 +470,22 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { val := values[string(key)] if len(val) == 0 { if e.idxInfo != nil && (!e.tblInfo.IsCommonHandle || !e.idxInfo.Primary) { - return kv.ErrNotExist.GenWithStack("inconsistent extra index %s, handle %d not found in table", - e.idxInfo.Name.O, e.handles[i]) + return (&consistency.Reporter{ + HandleEncode: func(_ kv.Handle) kv.Key { + return key + }, + IndexEncode: func(_ *consistency.RecordData) kv.Key { + return indexKeys[i] + }, + Tbl: e.tblInfo, + Idx: e.idxInfo, + Sctx: e.ctx, + }).ReportLookupInconsistent(ctx, + 1, 0, + e.handles[i:i+1], + e.handles, + []consistency.RecordData{{}}, + ) } continue } diff --git a/executor/distsql.go b/executor/distsql.go index c31d023317a80..cf4f16cbd554c 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -21,6 +21,7 @@ import ( "runtime" "runtime/trace" "sort" + "strings" "sync" "sync/atomic" "time" @@ -48,6 +49,7 @@ import ( "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" @@ -1120,6 +1122,35 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta chk := newFirstChunk(tableReader) tblInfo := w.idxLookup.table.Meta() vals := make([]types.Datum, 0, len(w.idxTblCols)) + + ir := func() *consistency.Reporter { + return &consistency.Reporter{ + HandleEncode: func(handle kv.Handle) kv.Key { + return tablecodec.EncodeRecordKey(w.idxLookup.table.RecordPrefix(), handle) + }, + IndexEncode: func(idxRow *consistency.RecordData) kv.Key { + var idx table.Index + for _, v := range w.idxLookup.table.Indices() { + if strings.EqualFold(v.Meta().Name.String(), w.idxLookup.index.Name.O) { + idx = v + break + } + } + if idx == nil { + return nil + } + k, _, err := idx.GenIndexKey(w.idxLookup.ctx.GetSessionVars().StmtCtx, idxRow.Values[:len(idx.Meta().Columns)], idxRow.Handle, nil) + if err != nil { + return nil + } + return k + }, + Tbl: tblInfo, + Idx: w.idxLookup.index, + Sctx: w.idxLookup.ctx, + } + } + for { err := Next(ctx, tableReader, chk) if err != nil { @@ -1128,7 +1159,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta if chk.NumRows() == 0 { task.indexOrder.Range(func(h kv.Handle, val interface{}) bool { idxRow := task.idxRows.GetRow(val.(int)) - err = ErrDataInConsistentExtraIndex.GenWithStackByArgs(h, idxRow.GetDatum(0, w.idxColTps[0]), nil) + err = ir().ReportAdminCheckInconsistent(ctx, h, &consistency.RecordData{Handle: h, Values: getDatumRow(&idxRow, w.idxColTps)}, nil) return false }) if err != nil { @@ -1163,12 +1194,32 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta tablecodec.TruncateIndexValue(&idxVal, w.idxLookup.index.Columns[i], col.ColumnInfo) cmpRes, err := idxVal.CompareDatum(sctx, &val) if err != nil { - return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, - handle, idxRow.GetDatum(i, tp), val, err) + fts := make([]*types.FieldType, 0, len(w.idxTblCols)) + for _, c := range w.idxTblCols { + fts = append(fts, &c.FieldType) + } + return ir().ReportAdminCheckInconsistentWithColInfo(ctx, + handle, + col.Name.O, + idxRow.GetDatum(i, tp), + val, + err, + &consistency.RecordData{Handle: handle, Values: getDatumRow(&idxRow, fts)}, + ) } if cmpRes != 0 { - return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, - handle, idxRow.GetDatum(i, tp), val, err) + fts := make([]*types.FieldType, 0, len(w.idxTblCols)) + for _, c := range w.idxTblCols { + fts = append(fts, &c.FieldType) + } + return ir().ReportAdminCheckInconsistentWithColInfo(ctx, + handle, + col.Name.O, + idxRow.GetDatum(i, tp), + val, + err, + &consistency.RecordData{Handle: handle, Values: getDatumRow(&idxRow, fts)}, + ) } } } @@ -1176,6 +1227,18 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta return nil } +func getDatumRow(r *chunk.Row, fields []*types.FieldType) []types.Datum { + datumRow := make([]types.Datum, 0, r.Chunk().NumCols()) + for colIdx := 0; colIdx < r.Chunk().NumCols(); colIdx++ { + if colIdx >= len(fields) { + break + } + datum := r.GetDatum(colIdx, fields[colIdx]) + datumRow = append(datumRow, datum) + } + return datumRow +} + // executeTask executes the table look up tasks. We will construct a table reader and send request by handles. // Then we hold the returning rows and finish this task. func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) error { @@ -1245,26 +1308,22 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er } obtainedHandlesMap.Set(handle, true) } - - if w.idxLookup.ctx.GetSessionVars().EnableRedactLog { - logutil.Logger(ctx).Error("inconsistent index handles", - zap.String("table_name", w.idxLookup.index.Table.O), - zap.String("index", w.idxLookup.index.Name.O), - zap.Int("index_cnt", handleCnt), - zap.Int("table_cnt", len(task.rows))) - } else { - logutil.Logger(ctx).Error("inconsistent index handles", - zap.String("table_name", w.idxLookup.index.Table.O), - zap.String("index", w.idxLookup.index.Name.O), - zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), - zap.String("missing_handles", fmt.Sprint(GetLackHandles(task.handles, obtainedHandlesMap))), - zap.String("total_handles", fmt.Sprint(task.handles))) - } - - // table scan in double read can never has conditions according to convertToIndexScan. - // if this table scan has no condition, the number of rows it returns must equal to the length of handles. - return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d", - w.idxLookup.index.Name.O, handleCnt, len(task.rows)) + missHds := GetLackHandles(task.handles, obtainedHandlesMap) + return (&consistency.Reporter{ + HandleEncode: func(hd kv.Handle) kv.Key { + return tablecodec.EncodeRecordKey(w.idxLookup.table.RecordPrefix(), hd) + }, + Tbl: w.idxLookup.table.Meta(), + Idx: w.idxLookup.index, + Sctx: w.idxLookup.ctx, + }).ReportLookupInconsistent(ctx, + handleCnt, + len(task.rows), + missHds, + task.handles, + nil, + //missRecords, + ) } } diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 8a5120e593717..aff9839a9238d 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -232,7 +232,7 @@ func (s *testSuite3) TestInconsistentIndex(c *C) { c.Assert(err, IsNil) err = tk.QueryToErr("select * from t use index(idx_a) where a >= 0") - c.Assert(err.Error(), Equals, fmt.Sprintf("inconsistent index idx_a handle count %d isn't equal to value count 10", i+11)) + c.Assert(err.Error(), Equals, fmt.Sprintf("[executor:8133]data inconsistency in table: t, index: idx_a, index-count:%d != record-count:10", i+11)) // if has other conditions, the inconsistent index check doesn't work. err = tk.QueryToErr("select * from t where a>=0 and b<10") diff --git a/executor/errors.go b/executor/errors.go index f592921122d9b..e466c238451c4 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -54,13 +54,11 @@ var ( ErrInvalidSplitRegionRanges = dbterror.ClassExecutor.NewStd(mysql.ErrInvalidSplitRegionRanges) ErrViewInvalid = dbterror.ClassExecutor.NewStd(mysql.ErrViewInvalid) - ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) - ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) - ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) - ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) - ErrCTEMaxRecursionDepth = dbterror.ClassExecutor.NewStd(mysql.ErrCTEMaxRecursionDepth) - ErrDataInConsistentExtraIndex = dbterror.ClassExecutor.NewStd(mysql.ErrDataInConsistentExtraIndex) - ErrDataInConsistentMisMatchIndex = dbterror.ClassExecutor.NewStd(mysql.ErrDataInConsistentMisMatchIndex) + ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) + ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) + ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) + ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) + ErrCTEMaxRecursionDepth = dbterror.ClassExecutor.NewStd(mysql.ErrCTEMaxRecursionDepth) errUnsupportedFlashbackTmpTable = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Recover/flashback table is not supported on temporary tables", nil)) errTruncateWrongInsertValue = dbterror.ClassTable.NewStdErr(mysql.ErrTruncatedWrongValue, parser_mysql.Message("Incorrect %-.32s value: '%-.128s' for column '%.192s' at row %d", nil)) diff --git a/executor/executor.go b/executor/executor.go index 3a1fa3beb9c48..1f0209a42763d 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -759,10 +759,7 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { if greater == admin.IdxCntGreater { err = e.checkTableIndexHandle(ctx, e.indexInfos[idxOffset]) } else if greater == admin.TblCntGreater { - err = e.checkTableRecord(idxOffset) - } - if err != nil && admin.ErrDataInConsistent.Equal(err) { - return ErrAdminCheckTable.GenWithStack("%v err:%v", e.table.Meta().Name, err) + err = e.checkTableRecord(ctx, idxOffset) } return errors.Trace(err) } @@ -797,7 +794,7 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } -func (e *CheckTableExec) checkTableRecord(idxOffset int) error { +func (e *CheckTableExec) checkTableRecord(ctx context.Context, idxOffset int) error { idxInfo := e.indexInfos[idxOffset] txn, err := e.ctx.Txn(true) if err != nil { @@ -805,7 +802,7 @@ func (e *CheckTableExec) checkTableRecord(idxOffset int) error { } if e.table.Meta().GetPartitionInfo() == nil { idx := tables.NewIndex(e.table.Meta().ID, e.table.Meta(), idxInfo) - return admin.CheckRecordAndIndex(e.ctx, txn, e.table, idx) + return admin.CheckRecordAndIndex(ctx, e.ctx, txn, e.table, idx) } info := e.table.Meta().GetPartitionInfo() @@ -813,7 +810,7 @@ func (e *CheckTableExec) checkTableRecord(idxOffset int) error { pid := def.ID partition := e.table.(table.PartitionedTable).GetPartition(pid) idx := tables.NewIndex(def.ID, e.table.Meta(), idxInfo) - if err := admin.CheckRecordAndIndex(e.ctx, txn, partition, idx); err != nil { + if err := admin.CheckRecordAndIndex(ctx, e.ctx, txn, partition, idx); err != nil { return errors.Trace(err) } } diff --git a/executor/executor_test.go b/executor/executor_test.go index 55b68448c5cbd..91e3ab267755b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3883,7 +3883,7 @@ func (s *testSuite) TestCheckIndex(c *C) { c.Assert(err, IsNil) _, err = se.Execute(context.Background(), "admin check index t c") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8133]handle 3, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:30, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") + c.Assert(err.Error(), Equals, "[admin:8223]data inconsistency in table: t, index: c, handle: 3, index-values:\"handle: 3, values: [KindInt64 30 KindInt64 3]\" != record-values:\"\"") // set data to: // index data (handle, data): (1, 10), (2, 20), (3, 30), (4, 40) diff --git a/executor/point_get.go b/executor/point_get.go index bcdedb260fb06..16769b3ff0013 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/rowcodec" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" ) @@ -300,8 +301,22 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { } if len(val) == 0 { if e.idxInfo != nil && !isCommonHandleRead(e.tblInfo, e.idxInfo) { - return kv.ErrNotExist.GenWithStack("inconsistent extra index %s, handle %d not found in table", - e.idxInfo.Name.O, e.handle) + return (&consistency.Reporter{ + HandleEncode: func(handle kv.Handle) kv.Key { + return key + }, + IndexEncode: func(idxRow *consistency.RecordData) kv.Key { + return e.idxKey + }, + Tbl: e.tblInfo, + Idx: e.idxInfo, + Sctx: e.ctx, + }).ReportLookupInconsistent(ctx, + 1, 0, + []kv.Handle{e.handle}, + []kv.Handle{e.handle}, + []consistency.RecordData{{}}, + ) } return nil } diff --git a/server/http_handler.go b/server/http_handler.go index c2788788b311f..fbb0c212d81f4 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -1739,7 +1739,7 @@ func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url. respValue := resp.Value var result interface{} = resp if respValue.Info != nil { - datas := make(map[string][]map[string]string) + datas := make(map[string]map[string]string) for _, w := range respValue.Info.Writes { if len(w.ShortValue) > 0 { datas[strconv.FormatUint(w.StartTs, 10)], err = h.decodeMvccData(w.ShortValue, colMap, tb.Meta()) @@ -1768,16 +1768,16 @@ func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url. return result, nil } -func (h mvccTxnHandler) decodeMvccData(bs []byte, colMap map[int64]*types.FieldType, tb *model.TableInfo) ([]map[string]string, error) { +func (h mvccTxnHandler) decodeMvccData(bs []byte, colMap map[int64]*types.FieldType, tb *model.TableInfo) (map[string]string, error) { rs, err := tablecodec.DecodeRowToDatumMap(bs, colMap, time.UTC) - var record []map[string]string + record := make(map[string]string, len(tb.Columns)) for _, col := range tb.Columns { if c, ok := rs[col.ID]; ok { data := "nil" if !c.IsNull() { data, err = c.ToString() } - record = append(record, map[string]string{col.Name.O: data}) + record[col.Name.O] = data } } return record, err diff --git a/testkit/asynctestkit.go b/testkit/asynctestkit.go index 907b940c4b378..1c8b8ef8329fc 100644 --- a/testkit/asynctestkit.go +++ b/testkit/asynctestkit.go @@ -52,7 +52,7 @@ func NewAsyncTestKit(t *testing.T, store kv.Storage) *AsyncTestKit { // OpenSession opens new session ctx if no exists one and use db. func (tk *AsyncTestKit) OpenSession(ctx context.Context, db string) context.Context { - if tryRetrieveSession(ctx) == nil { + if TryRetrieveSession(ctx) == nil { se, err := session.CreateSession4Test(tk.store) tk.require.NoError(err) se.SetConnectionID(asyncTestKitIDGenerator.Inc()) @@ -64,7 +64,7 @@ func (tk *AsyncTestKit) OpenSession(ctx context.Context, db string) context.Cont // CloseSession closes exists session from ctx. func (tk *AsyncTestKit) CloseSession(ctx context.Context) { - se := tryRetrieveSession(ctx) + se := TryRetrieveSession(ctx) tk.require.NotNil(se) se.Close() } @@ -134,7 +134,7 @@ func (tk *AsyncTestKit) ConcurrentRun( // Exec executes a sql statement. func (tk *AsyncTestKit) Exec(ctx context.Context, sql string, args ...interface{}) (sqlexec.RecordSet, error) { - se := tryRetrieveSession(ctx) + se := TryRetrieveSession(ctx) tk.require.NotNil(se) if len(args) == 0 { @@ -190,7 +190,7 @@ func (tk *AsyncTestKit) MustQuery(ctx context.Context, sql string, args ...inter // resultSetToResult converts ast.RecordSet to testkit.Result. // It is used to check results of execute statement in binary mode. func (tk *AsyncTestKit) resultSetToResult(ctx context.Context, rs sqlexec.RecordSet, comment string) *Result { - rows, err := session.GetRows4Test(context.Background(), tryRetrieveSession(ctx), rs) + rows, err := session.GetRows4Test(context.Background(), TryRetrieveSession(ctx), rs) tk.require.NoError(err, comment) err = rs.Close() @@ -218,7 +218,8 @@ type sessionCtxKeyType struct{} var sessionKey = sessionCtxKeyType{} -func tryRetrieveSession(ctx context.Context) session.Session { +// TryRetrieveSession tries retrieve session from context. +func TryRetrieveSession(ctx context.Context) session.Session { s := ctx.Value(sessionKey) if s == nil { return nil diff --git a/util/admin/admin.go b/util/admin/admin.go index a8d78fec976e0..5b5bfb995f408 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -19,6 +19,7 @@ import ( "encoding/json" "math" "sort" + "strings" "time" "github.com/pingcap/errors" @@ -36,6 +37,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/logutil/consistency" decoder "github.com/pingcap/tidb/util/rowDecoder" "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" @@ -285,12 +287,6 @@ func IterAllDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error return IterHistoryDDLJobs(txn, finishFn) } -// RecordData is the record data composed of a handle and values. -type RecordData struct { - Handle kv.Handle - Values []types.Datum -} - func getCount(exec sqlexec.RestrictedSQLExecutor, stmt ast.StmtNode, snapshot uint64) (int64, error) { rows, _, err := exec.ExecRestrictedStmt(context.Background(), stmt, sqlexec.ExecOptionWithSnapshot(snapshot)) if err != nil { @@ -370,13 +366,41 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices } // CheckRecordAndIndex is exported for testing. -func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { +func CheckRecordAndIndex(ctx context.Context, sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { sc := sessCtx.GetSessionVars().StmtCtx cols := make([]*table.Column, len(idx.Meta().Columns)) for i, col := range idx.Meta().Columns { cols[i] = t.Cols()[col.Offset] } + ir := func() *consistency.Reporter { + return &consistency.Reporter{ + HandleEncode: func(handle kv.Handle) kv.Key { + return tablecodec.EncodeRecordKey(t.RecordPrefix(), handle) + }, + IndexEncode: func(idxRow *consistency.RecordData) kv.Key { + var idx table.Index + for _, v := range t.Indices() { + if strings.EqualFold(v.Meta().Name.String(), idx.Meta().Name.O) { + idx = v + break + } + } + if idx == nil { + return nil + } + k, _, err := idx.GenIndexKey(sessCtx.GetSessionVars().StmtCtx, idxRow.Values, idxRow.Handle, nil) + if err != nil { + return nil + } + return k + }, + Tbl: t.Meta(), + Idx: idx.Meta(), + Sctx: sessCtx, + } + } + startKey := tablecodec.EncodeRecordKey(t.RecordPrefix(), kv.IntHandle(math.MinInt64)) filterFunc := func(h1 kv.Handle, vals1 []types.Datum, cols []*table.Column) (bool, error) { for i, val := range vals1 { @@ -395,16 +419,16 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table } isExist, h2, err := idx.Exist(sc, txn, vals1, h1) if kv.ErrKeyExists.Equal(err) { - record1 := &RecordData{Handle: h1, Values: vals1} - record2 := &RecordData{Handle: h2, Values: vals1} - return false, ErrDataInConsistent.GenWithStackByArgs(record2, record1) + record1 := &consistency.RecordData{Handle: h1, Values: vals1} + record2 := &consistency.RecordData{Handle: h2, Values: vals1} + return false, ir().ReportAdminCheckInconsistent(ctx, h1, record2, record1) } if err != nil { return false, errors.Trace(err) } if !isExist { - record := &RecordData{Handle: h1, Values: vals1} - return false, ErrDataInConsistent.GenWithStackByArgs(nil, record) + record := &consistency.RecordData{Handle: h1, Values: vals1} + return false, ir().ReportAdminCheckInconsistent(ctx, h1, nil, record) } return true, nil @@ -484,8 +508,6 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab } var ( - // ErrDataInConsistent indicate that meets inconsistent data. - ErrDataInConsistent = dbterror.ClassAdmin.NewStd(errno.ErrDataInConsistent) // ErrDDLJobNotFound indicates the job id was not found. ErrDDLJobNotFound = dbterror.ClassAdmin.NewStd(errno.ErrDDLJobNotFound) // ErrCancelFinishedDDLJob returns when cancel a finished ddl job. diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index be3f12812b0a4..cc521db408b27 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -367,7 +367,6 @@ func TestError(t *testing.T) { t.Parallel() kvErrs := []*terror.Error{ - ErrDataInConsistent, ErrDDLJobNotFound, ErrCancelFinishedDDLJob, ErrCannotCancelDDLJob, diff --git a/util/logutil/consistency/reporter.go b/util/logutil/consistency/reporter.go new file mode 100644 index 0000000000000..ce79db887c108 --- /dev/null +++ b/util/logutil/consistency/reporter.go @@ -0,0 +1,275 @@ +// Copyright 2021 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 consistency + +import ( + "context" + "encoding/hex" + "encoding/json" + "fmt" + "strconv" + "strings" + "time" + + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/helper" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/util/logutil" + "github.com/tikv/client-go/v2/tikv" + "go.uber.org/zap" +) + +var ( + // ErrAdminCheckInconsistent returns for data inconsistency for admin check. + ErrAdminCheckInconsistent = dbterror.ClassAdmin.NewStd(errno.ErrDataInconsistent) + // ErrLookupInconsistent returns for data inconsistency for index lookup. + ErrLookupInconsistent = dbterror.ClassExecutor.NewStd(errno.ErrDataInconsistentMismatchCount) + // ErrAdminCheckInconsistentWithColInfo returns for data inconsistency for admin check but with column info. + ErrAdminCheckInconsistentWithColInfo = dbterror.ClassExecutor.NewStd(errno.ErrDataInconsistentMismatchIndex) +) + +func getMvccByKey(sctx sessionctx.Context, key kv.Key, decodeMvccFn func(kv.Key, *kvrpcpb.MvccGetByKeyResponse, map[string]interface{})) string { + if key == nil { + return "" + } + tikvStore, ok := sctx.GetStore().(helper.Storage) + if !ok { + return "" + } + h := helper.NewHelper(tikvStore) + data, err := h.GetMvccByEncodedKey(key) + if err != nil { + return "" + } + regionID := getRegionIDByKey(tikvStore, key) + + decodeKey := strings.ToUpper(hex.EncodeToString(key)) + + resp := map[string]interface{}{ + "key": decodeKey, + "regionID": regionID, + "mvcc": data, + } + + if decodeMvccFn != nil { + decodeMvccFn(key, data, resp) + } + + rj, err := json.Marshal(resp) + if err != nil { + return "" + } + const maxMvccInfoLen = 5000 + s := string(rj) + if len(s) > maxMvccInfoLen { + s = s[:maxMvccInfoLen] + "[truncated]..." + } + + return s +} + +func getRegionIDByKey(tikvStore helper.Storage, encodedKey []byte) uint64 { + keyLocation, err := tikvStore.GetRegionCache().LocateKey(tikv.NewBackofferWithVars(context.Background(), 500, nil), encodedKey) + if err != nil { + return 0 + } + return keyLocation.Region.GetID() +} + +// Reporter is a helper to generate report. +type Reporter struct { + HandleEncode func(handle kv.Handle) kv.Key + IndexEncode func(idxRow *RecordData) kv.Key + Tbl *model.TableInfo + Idx *model.IndexInfo + Sctx sessionctx.Context +} + +func (r *Reporter) decodeRowMvccData(_ kv.Key, respValue *kvrpcpb.MvccGetByKeyResponse, outMap map[string]interface{}) { + colMap := make(map[int64]*types.FieldType, 3) + for _, col := range r.Tbl.Columns { + colMap[col.ID] = &col.FieldType + } + + if respValue.Info != nil { + var err error + datas := make(map[string]map[string]string) + for _, w := range respValue.Info.Writes { + if len(w.ShortValue) > 0 { + datas[strconv.FormatUint(w.StartTs, 10)], err = decodeMvccData(w.ShortValue, colMap, r.Tbl) + } + } + + for _, v := range respValue.Info.Values { + if len(v.Value) > 0 { + datas[strconv.FormatUint(v.StartTs, 10)], err = decodeMvccData(v.Value, colMap, r.Tbl) + } + } + if len(datas) > 0 { + outMap["decoded"] = datas + if err != nil { + outMap["decode_error"] = err.Error() + } + } + } +} + +func (r *Reporter) decodeIndexMvccData(key kv.Key, respValue *kvrpcpb.MvccGetByKeyResponse, outMap map[string]interface{}) { + if respValue.Info != nil { + var ( + hd kv.Handle + err error + datas = make(map[string]map[string]string) + ) + for _, w := range respValue.Info.Writes { + if len(w.ShortValue) > 0 { + hd, err = tablecodec.DecodeIndexHandle(key, w.ShortValue, len(r.Idx.Columns)) + if err == nil { + datas[strconv.FormatUint(w.StartTs, 10)] = map[string]string{"handle": hd.String()} + } + } + } + for _, v := range respValue.Info.Values { + if len(v.Value) > 0 { + hd, err = tablecodec.DecodeIndexHandle(key, v.Value, len(r.Idx.Columns)) + if err == nil { + datas[strconv.FormatUint(v.StartTs, 10)] = map[string]string{"handle": hd.String()} + } + } + } + if len(datas) > 0 { + outMap["decoded"] = datas + if err != nil { + outMap["decode_error"] = err.Error() + } + } + } +} + +func decodeMvccData(bs []byte, colMap map[int64]*types.FieldType, tb *model.TableInfo) (map[string]string, error) { + rs, err := tablecodec.DecodeRowToDatumMap(bs, colMap, time.UTC) + record := make(map[string]string, len(tb.Columns)) + for _, col := range tb.Columns { + if c, ok := rs[col.ID]; ok { + data := "nil" + if !c.IsNull() { + data, err = c.ToString() + } + record[col.Name.O] = data + } + } + return record, err +} + +// ReportLookupInconsistent reports inconsistent when index rows is more than record rows. +func (r *Reporter) ReportLookupInconsistent(ctx context.Context, idxCnt, tblCnt int, missHd, fullHd []kv.Handle, missRowIdx []RecordData) error { + if r.Sctx.GetSessionVars().EnableRedactLog { + logutil.Logger(ctx).Error("indexLookup found data inconsistency", + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.Int("index_cnt", idxCnt), + zap.Int("table_cnt", tblCnt)) + } else { + const maxFullHandleCnt = 50 + displayFullHdCnt := len(fullHd) + if displayFullHdCnt > maxFullHandleCnt { + displayFullHdCnt = maxFullHandleCnt + } + fs := []zap.Field{ + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.Int("index_cnt", idxCnt), zap.Int("table_cnt", tblCnt), + zap.String("missing_handles", fmt.Sprint(missHd)), + zap.String("total_handles", fmt.Sprint(fullHd[:displayFullHdCnt])), + } + for i, hd := range missHd { + fs = append(fs, zap.String("row_mvcc_"+strconv.Itoa(i), getMvccByKey(r.Sctx, r.HandleEncode(hd), r.decodeRowMvccData))) + } + for i, rowIdx := range missRowIdx { + fs = append(fs, zap.String("index_mvcc_"+strconv.Itoa(i), getMvccByKey(r.Sctx, r.IndexEncode(&rowIdx), r.decodeIndexMvccData))) + } + logutil.Logger(ctx).Error("indexLookup found data inconsistency", fs...) + } + return ErrLookupInconsistent.GenWithStackByArgs(r.Tbl.Name.O, r.Idx.Name.O, idxCnt, tblCnt) +} + +// ReportAdminCheckInconsistentWithColInfo reports inconsistent when the value of index row is different from record row. +func (r *Reporter) ReportAdminCheckInconsistentWithColInfo(ctx context.Context, handle kv.Handle, colName string, idxDat, tblDat fmt.Stringer, err error, idxRow *RecordData) error { + if r.Sctx.GetSessionVars().EnableRedactLog { + logutil.Logger(ctx).Error("admin check found data inconsistency", + zap.String("table_name", r.Tbl.Name.O), + zap.String("index", r.Idx.Name.O), + zap.String("col", colName), + zap.Error(err), + ) + } else { + fs := []zap.Field{ + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.String("col", colName), + zap.Stringer("row_id", handle), + zap.Stringer("idxDatum", idxDat), + zap.Stringer("rowDatum", tblDat), + } + fs = append(fs, zap.String("row_mvcc", getMvccByKey(r.Sctx, r.HandleEncode(handle), r.decodeRowMvccData))) + fs = append(fs, zap.String("index_mvcc", getMvccByKey(r.Sctx, r.IndexEncode(idxRow), r.decodeIndexMvccData))) + fs = append(fs, zap.Error(err)) + logutil.Logger(ctx).Error("admin check found data inconsistency", fs...) + } + return ErrAdminCheckInconsistentWithColInfo.GenWithStackByArgs(r.Tbl.Name.O, r.Idx.Name.O, colName, fmt.Sprint(handle), fmt.Sprint(idxDat), fmt.Sprint(tblDat), err) +} + +// RecordData is the record data composed of a handle and values. +type RecordData struct { + Handle kv.Handle + Values []types.Datum +} + +func (r *RecordData) String() string { + if r == nil { + return "" + } + return fmt.Sprintf("handle: %s, values: %s", fmt.Sprint(r.Handle), fmt.Sprint(r.Values)) +} + +// ReportAdminCheckInconsistent reports inconsistent when single index row not found in record rows. +func (r *Reporter) ReportAdminCheckInconsistent(ctx context.Context, handle kv.Handle, idxRow, tblRow *RecordData) error { + if r.Sctx.GetSessionVars().EnableRedactLog { + logutil.Logger(ctx).Error("admin check found data inconsistency", + zap.String("table_name", r.Tbl.Name.O), + zap.String("index", r.Idx.Name.O), + ) + } else { + fs := []zap.Field{ + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.Stringer("row_id", handle), + zap.Stringer("index", idxRow), + zap.Stringer("row", tblRow), + } + fs = append(fs, zap.String("row_mvcc", getMvccByKey(r.Sctx, r.HandleEncode(handle), r.decodeRowMvccData))) + if idxRow != nil { + fs = append(fs, zap.String("index_mvcc", getMvccByKey(r.Sctx, r.IndexEncode(idxRow), r.decodeIndexMvccData))) + } + logutil.Logger(ctx).Error("admin check found data inconsistency", fs...) + } + return ErrAdminCheckInconsistent.GenWithStackByArgs(r.Tbl.Name.O, r.Idx.Name.O, fmt.Sprint(handle), fmt.Sprint(idxRow), fmt.Sprint(tblRow)) +} diff --git a/util/logutil/log.go b/util/logutil/log.go index 5f522ee098e6b..795e026434c19 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -155,12 +155,14 @@ func SetLevel(level string) error { type ctxLogKeyType struct{} -var ctxLogKey = ctxLogKeyType{} +// CtxLogKey indicates the context key for logger +// public for test usage. +var CtxLogKey = ctxLogKeyType{} // Logger gets a contextual logger from current context. // contextual logger will output common fields from context. func Logger(ctx context.Context) *zap.Logger { - if ctxlogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxlogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { return ctxlogger } return log.L() @@ -174,23 +176,23 @@ func BgLogger() *zap.Logger { // WithConnID attaches connId to context. func WithConnID(ctx context.Context, connID uint64) context.Context { var logger *zap.Logger - if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, ctxLogKey, logger.With(zap.Uint64("conn", connID))) + return context.WithValue(ctx, CtxLogKey, logger.With(zap.Uint64("conn", connID))) } // WithTraceLogger attaches trace identifier to context func WithTraceLogger(ctx context.Context, connID uint64) context.Context { var logger *zap.Logger - if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, ctxLogKey, wrapTraceLogger(ctx, connID, logger)) + return context.WithValue(ctx, CtxLogKey, wrapTraceLogger(ctx, connID, logger)) } func wrapTraceLogger(ctx context.Context, connID uint64, logger *zap.Logger) *zap.Logger { @@ -222,12 +224,12 @@ func (t *traceLog) Sync() error { // WithKeyValue attaches key/value to context. func WithKeyValue(ctx context.Context, key, value string) context.Context { var logger *zap.Logger - if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, ctxLogKey, logger.With(zap.String(key, value))) + return context.WithValue(ctx, CtxLogKey, logger.With(zap.String(key, value))) } // TraceEventKey presents the TraceEventKey in span log.