diff --git a/pkg/executor/batch_point_get.go b/pkg/executor/batch_point_get.go index 4b0262b44014d..88a2a442f158d 100644 --- a/pkg/executor/batch_point_get.go +++ b/pkg/executor/batch_point_get.go @@ -424,9 +424,10 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { IndexEncode: func(_ *consistency.RecordData) kv.Key { return indexKeys[i] }, - Tbl: e.tblInfo, - Idx: e.idxInfo, - Sctx: e.Ctx(), + Tbl: e.tblInfo, + Idx: e.idxInfo, + EnableRedactLog: e.Ctx().GetSessionVars().EnableRedactLog, + Storage: e.Ctx().GetStore(), }).ReportLookupInconsistent(ctx, 1, 0, e.handles[i:i+1], diff --git a/pkg/executor/distsql.go b/pkg/executor/distsql.go index dec795a1ffa26..aaceb21694709 100644 --- a/pkg/executor/distsql.go +++ b/pkg/executor/distsql.go @@ -1365,9 +1365,10 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta } return k }, - Tbl: tblInfo, - Idx: w.idxLookup.index, - Sctx: w.idxLookup.Ctx(), + Tbl: tblInfo, + Idx: w.idxLookup.index, + EnableRedactLog: w.idxLookup.Ctx().GetSessionVars().EnableRedactLog, + Storage: w.idxLookup.Ctx().GetStore(), } } @@ -1553,9 +1554,10 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er 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(), + Tbl: w.idxLookup.table.Meta(), + Idx: w.idxLookup.index, + EnableRedactLog: w.idxLookup.Ctx().GetSessionVars().EnableRedactLog, + Storage: w.idxLookup.Ctx().GetStore(), }).ReportLookupInconsistent(ctx, handleCnt, len(task.rows), diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index 8f48bcbc94d57..8aa8c79384c95 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -2493,9 +2493,10 @@ func (w *checkIndexWorker) HandleTask(task checkIndexTask, _ func(workerpool.Non } return k }, - Tbl: w.table.Meta(), - Idx: idxInfo, - Sctx: w.sctx, + Tbl: w.table.Meta(), + Idx: idxInfo, + EnableRedactLog: w.sctx.GetSessionVars().EnableRedactLog, + Storage: w.sctx.GetStore(), } } diff --git a/pkg/executor/point_get.go b/pkg/executor/point_get.go index c26b430a09ea8..ee3b7047aa89e 100644 --- a/pkg/executor/point_get.go +++ b/pkg/executor/point_get.go @@ -376,9 +376,10 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { IndexEncode: func(*consistency.RecordData) kv.Key { return e.idxKey }, - Tbl: e.tblInfo, - Idx: e.idxInfo, - Sctx: e.Ctx(), + Tbl: e.tblInfo, + Idx: e.idxInfo, + EnableRedactLog: e.Ctx().GetSessionVars().EnableRedactLog, + Storage: e.Ctx().GetStore(), }).ReportLookupInconsistent(ctx, 1, 0, []kv.Handle{e.handle}, diff --git a/pkg/util/admin/admin.go b/pkg/util/admin/admin.go index 47e0f14436c8f..2d7f870549a34 100644 --- a/pkg/util/admin/admin.go +++ b/pkg/util/admin/admin.go @@ -146,9 +146,10 @@ func CheckRecordAndIndex(ctx context.Context, sessCtx sessionctx.Context, txn kv } return k }, - Tbl: t.Meta(), - Idx: idx.Meta(), - Sctx: sessCtx, + Tbl: t.Meta(), + Idx: idx.Meta(), + EnableRedactLog: sessCtx.GetSessionVars().EnableRedactLog, + Storage: sessCtx.GetStore(), } } diff --git a/pkg/util/logutil/consistency/reporter.go b/pkg/util/logutil/consistency/reporter.go index 4271a1dfc0b37..0727bc0ce4579 100644 --- a/pkg/util/logutil/consistency/reporter.go +++ b/pkg/util/logutil/consistency/reporter.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -95,11 +94,12 @@ func getRegionIDByKey(tikvStore helper.Storage, encodedKey []byte) uint64 { // 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 + HandleEncode func(handle kv.Handle) kv.Key + IndexEncode func(idxRow *RecordData) kv.Key + Tbl *model.TableInfo + Idx *model.IndexInfo + EnableRedactLog string + Storage any } // DecodeRowMvccData creates a closure that captures the tableInfo to be used a decode function in GetMvccByKey. @@ -187,7 +187,7 @@ func decodeMvccRecordValue(bs []byte, colMap map[int64]*types.FieldType, tb *mod // 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 { - rmode := r.Sctx.GetSessionVars().EnableRedactLog + rmode := r.EnableRedactLog const maxFullHandleCnt = 50 displayFullHdCnt := min(len(fullHd), maxFullHandleCnt) @@ -199,7 +199,7 @@ func (r *Reporter) ReportLookupInconsistent(ctx context.Context, idxCnt, tblCnt zap.String("total_handles", redact.String(rmode, fmt.Sprint(fullHd[:displayFullHdCnt]))), } if rmode != errors.RedactLogEnable { - store, ok := r.Sctx.GetStore().(helper.Storage) + store, ok := r.Storage.(helper.Storage) if ok { for i, hd := range missHd { fs = append(fs, zap.String("row_mvcc_"+strconv.Itoa(i), redact.String(rmode, GetMvccByKey(store, r.HandleEncode(hd), DecodeRowMvccData(r.Tbl))))) @@ -216,7 +216,7 @@ func (r *Reporter) ReportLookupInconsistent(ctx context.Context, 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 { - rmode := r.Sctx.GetSessionVars().EnableRedactLog + rmode := r.EnableRedactLog fs := []zap.Field{ zap.String("table_name", r.Tbl.Name.O), zap.String("index_name", r.Idx.Name.O), @@ -226,7 +226,7 @@ func (r *Reporter) ReportAdminCheckInconsistentWithColInfo(ctx context.Context, zap.Stringer("rowDatum", redact.Stringer(rmode, tblDat)), } if rmode != errors.RedactLogEnable { - store, ok := r.Sctx.GetStore().(helper.Storage) + store, ok := r.Storage.(helper.Storage) if ok { fs = append(fs, zap.String("row_mvcc", redact.String(rmode, GetMvccByKey(store, r.HandleEncode(handle), DecodeRowMvccData(r.Tbl))))) fs = append(fs, zap.String("index_mvcc", redact.String(rmode, GetMvccByKey(store, r.IndexEncode(idxRow), DecodeIndexMvccData(r.Idx))))) @@ -253,7 +253,7 @@ func (r *RecordData) String() string { // 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 { - rmode := r.Sctx.GetSessionVars().EnableRedactLog + rmode := r.EnableRedactLog fs := []zap.Field{ zap.String("table_name", r.Tbl.Name.O), zap.String("index_name", r.Idx.Name.O), @@ -262,7 +262,7 @@ func (r *Reporter) ReportAdminCheckInconsistent(ctx context.Context, handle kv.H zap.Stringer("row", redact.Stringer(rmode, tblRow)), } if rmode != errors.RedactLogEnable { - store, ok := r.Sctx.GetStore().(helper.Storage) + store, ok := r.Storage.(helper.Storage) if ok { fs = append(fs, zap.String("row_mvcc", redact.String(rmode, GetMvccByKey(store, r.HandleEncode(handle), DecodeRowMvccData(r.Tbl))))) if idxRow != nil {