diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index 05c26ebf41530..657b0520f2d85 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -1384,7 +1384,7 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra newColumnIDs = append(newColumnIDs, colID) newRow = append(newRow, val) } - rd := &w.tblCtx.GetSessionVars().RowEncoder + rd := w.tblCtx.GetRowEncodingConfig().RowEncoder ec := w.exprCtx.GetEvalCtx().ErrCtx() var checksum rowcodec.Checksum if w.checksumNeeded { diff --git a/pkg/errctx/context.go b/pkg/errctx/context.go index 209de1b273864..a23f3f6bd01f4 100644 --- a/pkg/errctx/context.go +++ b/pkg/errctx/context.go @@ -94,6 +94,9 @@ func (ctx *Context) AppendWarning(err error) { // It also allows using `errors.ErrorGroup`, in this case, it'll handle each error in order, and return the first error // it founds. func (ctx *Context) HandleError(err error) error { + if err == nil { + return nil + } // The function of handling `errors.ErrorGroup` is placed in `HandleError` but not in `HandleErrorWithAlias`, because // it's hard to give a proper error and warn alias for an error group. if errs, ok := err.(errors.ErrorGroup); ok { diff --git a/pkg/executor/distsql.go b/pkg/executor/distsql.go index dec795a1ffa26..b23390cd34f0a 100644 --- a/pkg/executor/distsql.go +++ b/pkg/executor/distsql.go @@ -1415,12 +1415,12 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta vals = append(vals, row.GetDatum(i, &col.FieldType)) } tablecodec.TruncateIndexValues(tblInfo, w.idxLookup.index, vals) - sctx := w.idxLookup.Ctx().GetSessionVars().StmtCtx + tc := w.idxLookup.Ctx().GetSessionVars().StmtCtx.TypeCtx() for i := range vals { col := w.idxTblCols[i] idxVal := idxRow.GetDatum(i, w.idxColTps[i]) tablecodec.TruncateIndexValue(&idxVal, w.idxLookup.index.Columns[i], col.ColumnInfo) - cmpRes, err := tables.CompareIndexAndVal(sctx, vals[i], idxVal, collators[i], col.FieldType.IsArray() && vals[i].Kind() == types.KindMysqlJSON) + cmpRes, err := tables.CompareIndexAndVal(tc, vals[i], idxVal, collators[i], col.FieldType.IsArray() && vals[i].Kind() == types.KindMysqlJSON) if err != nil { return ir().ReportAdminCheckInconsistentWithColInfo(ctx, handle, diff --git a/pkg/executor/test/admintest/admin_test.go b/pkg/executor/test/admintest/admin_test.go index f0293e8a171e4..736aa9b9edf5c 100644 --- a/pkg/executor/test/admintest/admin_test.go +++ b/pkg/executor/test/admintest/admin_test.go @@ -1666,13 +1666,12 @@ func TestAdminCheckTableErrorLocateForClusterIndex(t *testing.T) { tblInfo := tbl.Meta() idxInfo := tblInfo.Indices[0] indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - sc := ctx.GetSessionVars().StmtCtx pattern := "handle:\\s(\\d+)" r := regexp.MustCompile(pattern) getCommonHandle := func(randomRow int) *kv.CommonHandle { - h, err := codec.EncodeKey(sc.TimeZone(), nil, types.MakeDatums(randomRow)...) + h, err := codec.EncodeKey(ctx.GetExprCtx().GetEvalCtx().Location(), nil, types.MakeDatums(randomRow)...) require.NoError(t, err) ch, err := kv.NewCommonHandle(h) require.NoError(t, err) diff --git a/pkg/table/tables/BUILD.bazel b/pkg/table/tables/BUILD.bazel index 61069d42ee6ec..2e653027c6c09 100644 --- a/pkg/table/tables/BUILD.bazel +++ b/pkg/table/tables/BUILD.bazel @@ -30,7 +30,6 @@ go_library( "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/sessionctx/binloginfo", - "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/table", @@ -95,7 +94,6 @@ go_test( "//pkg/session", "//pkg/session/types", "//pkg/sessionctx", - "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/table", diff --git a/pkg/table/tables/index.go b/pkg/table/tables/index.go index 0b160359da0eb..68ad4a537ae83 100644 --- a/pkg/table/tables/index.go +++ b/pkg/table/tables/index.go @@ -179,9 +179,10 @@ func (c *index) Create(sctx table.MutateContext, txn kv.Transaction, indexedValu vars := sctx.GetSessionVars() writeBufs := vars.GetWriteStmtBufs() skipCheck := vars.StmtCtx.BatchCheck - sc := sctx.GetSessionVars().StmtCtx + evalCtx := sctx.GetExprCtx().GetEvalCtx() + loc, ec := evalCtx.Location(), evalCtx.ErrCtx() for _, value := range indexedValues { - key, distinct, err := c.GenIndexKey(sc.ErrCtx(), sc.TimeZone(), value, h, writeBufs.IndexKeyBuf) + key, distinct, err := c.GenIndexKey(ec, loc, value, h, writeBufs.IndexKeyBuf) if err != nil { return nil, err } @@ -238,9 +239,9 @@ func (c *index) Create(sctx table.MutateContext, txn kv.Transaction, indexedValu c.initNeedRestoreData.Do(func() { c.needRestoredData = NeedRestoredData(c.idxInfo.Columns, c.tblInfo.Columns) }) - idxVal, err := tablecodec.GenIndexValuePortal(sctx.GetSessionVars().StmtCtx.TimeZone(), c.tblInfo, c.idxInfo, + idxVal, err := tablecodec.GenIndexValuePortal(loc, c.tblInfo, c.idxInfo, c.needRestoredData, distinct, opt.Untouched, value, h, c.phyTblID, handleRestoreData, nil) - err = sctx.GetSessionVars().StmtCtx.HandleError(err) + err = ec.HandleError(err) if err != nil { return nil, err } @@ -393,9 +394,10 @@ func needPresumeKeyNotExistsFlag(ctx context.Context, txn kv.Transaction, key, t // Delete removes the entry for handle h and indexedValues from KV index. func (c *index) Delete(ctx table.MutateContext, txn kv.Transaction, indexedValue []types.Datum, h kv.Handle) error { indexedValues := c.getIndexedValue(indexedValue) - sc := ctx.GetSessionVars().StmtCtx + evalCtx := ctx.GetExprCtx().GetEvalCtx() + loc, ec := evalCtx.Location(), evalCtx.ErrCtx() for _, value := range indexedValues { - key, distinct, err := c.GenIndexKey(sc.ErrCtx(), sc.TimeZone(), value, h, nil) + key, distinct, err := c.GenIndexKey(ec, loc, value, h, nil) if err != nil { return err } diff --git a/pkg/table/tables/mutation_checker.go b/pkg/table/tables/mutation_checker.go index 3b9bbd6d4a2e2..33f18ea37cb95 100644 --- a/pkg/table/tables/mutation_checker.go +++ b/pkg/table/tables/mutation_checker.go @@ -23,7 +23,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/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" @@ -77,7 +76,7 @@ type columnMaps struct { // The time complexity is O(M * C + I) // The space complexity is O(M + C + I) func CheckDataConsistency( - txn kv.Transaction, sessVars *variable.SessionVars, t *TableCommon, + txn kv.Transaction, tc types.Context, t *TableCommon, rowToInsert, rowToRemove []types.Datum, memBuffer kv.MemBuffer, sh kv.StagingHandle, ) error { if t.Meta().GetPartitionInfo() != nil { @@ -115,7 +114,7 @@ func CheckDataConsistency( } if err := checkIndexKeys( - sessVars, t, rowToInsert, rowToRemove, indexMutations, columnMaps.IndexIDToInfo, columnMaps.IndexIDToRowColInfos, + tc, t, rowToInsert, rowToRemove, indexMutations, columnMaps.IndexIDToInfo, columnMaps.IndexIDToRowColInfos, ); err != nil { return errors.Trace(err) } @@ -205,7 +204,7 @@ func checkHandleConsistency(rowInsertion mutation, indexMutations []mutation, in // (a) {added indices} is a subset of {needed indices} => each index mutation is consistent with the input/row key/value // (b) {needed indices} is a subset of {added indices}. The check process would be exactly the same with how we generate the mutations, thus ignored. func checkIndexKeys( - sessVars *variable.SessionVars, t *TableCommon, rowToInsert, rowToRemove []types.Datum, + tc types.Context, t *TableCommon, rowToInsert, rowToRemove []types.Datum, indexMutations []mutation, indexIDToInfo map[int64]*model.IndexInfo, indexIDToRowColInfos map[int64][]rowcodec.ColInfo, ) error { @@ -260,9 +259,10 @@ func checkIndexKeys( indexData = indexData[:0] } + loc := tc.Location() for i, v := range decodedIndexValues { fieldType := t.Columns[indexInfo.Columns[i].Offset].FieldType.ArrayType() - datum, err := tablecodec.DecodeColumnValue(v, fieldType, sessVars.Location()) + datum, err := tablecodec.DecodeColumnValue(v, fieldType, loc) if err != nil { return errors.Trace(err) } @@ -271,9 +271,9 @@ func checkIndexKeys( // When it is in add index new backfill state. if len(value) == 0 || isTmpIdxValAndDeleted { - err = compareIndexData(sessVars.StmtCtx, t.Columns, indexData, rowToRemove, indexInfo, t.Meta()) + err = compareIndexData(tc, t.Columns, indexData, rowToRemove, indexInfo, t.Meta()) } else { - err = compareIndexData(sessVars.StmtCtx, t.Columns, indexData, rowToInsert, indexInfo, t.Meta()) + err = compareIndexData(tc, t.Columns, indexData, rowToInsert, indexInfo, t.Meta()) } if err != nil { return errors.Trace(err) @@ -356,7 +356,7 @@ func collectTableMutationsFromBufferStage(t *TableCommon, memBuffer kv.MemBuffer // compareIndexData compares the decoded index data with the input data. // Returns error if the index data is not a subset of the input data. func compareIndexData( - sc *stmtctx.StatementContext, cols []*table.Column, indexData, input []types.Datum, indexInfo *model.IndexInfo, + tc types.Context, cols []*table.Column, indexData, input []types.Datum, indexInfo *model.IndexInfo, tableInfo *model.TableInfo, ) error { for i := range indexData { @@ -372,7 +372,7 @@ func compareIndexData( cols[indexInfo.Columns[i].Offset].ColumnInfo, ) - comparison, err := CompareIndexAndVal(sc, expectedDatum, decodedMutationDatum, + comparison, err := CompareIndexAndVal(tc, expectedDatum, decodedMutationDatum, collate.GetCollator(decodedMutationDatum.Collation()), cols[indexInfo.Columns[i].Offset].ColumnInfo.FieldType.IsArray() && expectedDatum.Kind() == types.KindMysqlJSON) if err != nil { @@ -392,7 +392,7 @@ func compareIndexData( } // CompareIndexAndVal compare index valued and row value. -func CompareIndexAndVal(sctx *stmtctx.StatementContext, rowVal types.Datum, idxVal types.Datum, collator collate.Collator, cmpMVIndex bool) (int, error) { +func CompareIndexAndVal(tc types.Context, rowVal types.Datum, idxVal types.Datum, collator collate.Collator, cmpMVIndex bool) (int, error) { var cmpRes int var err error if cmpMVIndex { @@ -401,7 +401,7 @@ func CompareIndexAndVal(sctx *stmtctx.StatementContext, rowVal types.Datum, idxV count := bj.GetElemCount() for elemIdx := 0; elemIdx < count; elemIdx++ { jsonDatum := types.NewJSONDatum(bj.ArrayGetElem(elemIdx)) - cmpRes, err = jsonDatum.Compare(sctx.TypeCtx(), &idxVal, collate.GetBinaryCollator()) + cmpRes, err = jsonDatum.Compare(tc, &idxVal, collate.GetBinaryCollator()) if err != nil { return 0, errors.Trace(err) } @@ -410,7 +410,7 @@ func CompareIndexAndVal(sctx *stmtctx.StatementContext, rowVal types.Datum, idxV } } } else { - cmpRes, err = idxVal.Compare(sctx.TypeCtx(), &rowVal, collator) + cmpRes, err = idxVal.Compare(tc, &rowVal, collator) } return cmpRes, err } diff --git a/pkg/table/tables/mutation_checker_test.go b/pkg/table/tables/mutation_checker_test.go index ccad7f9a826eb..a9407ee173367 100644 --- a/pkg/table/tables/mutation_checker_test.go +++ b/pkg/table/tables/mutation_checker_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" @@ -72,7 +71,7 @@ func TestCompareIndexData(t *testing.T) { } for caseID, data := range testData { - sc := stmtctx.NewStmtCtx() + tc := types.DefaultStmtNoWarningContext cols := make([]*table.Column, 0) indexCols := make([]*model.IndexColumn, 0) for i, ft := range data.fts { @@ -81,7 +80,7 @@ func TestCompareIndexData(t *testing.T) { } indexInfo := &model.IndexInfo{Name: model.NewCIStr("i0"), Columns: indexCols} - err := compareIndexData(sc, cols, data.indexData, data.inputData, indexInfo, &model.TableInfo{Name: model.NewCIStr("t")}) + err := compareIndexData(tc, cols, data.indexData, data.inputData, indexInfo, &model.TableInfo{Name: model.NewCIStr("t")}) require.Equal(t, data.correct, err == nil, "case id = %v", caseID) } } @@ -228,7 +227,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { {ID: 2, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeDatetime)}, }, } - sessVars := variable.NewSessionVars(nil) + tc := types.DefaultStmtNoWarningContext rd := rowcodec.Encoder{Enable: true} now := types.CurrentTime(mysql.TypeDatetime) @@ -236,7 +235,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { types.NewStringDatum("some string"), types.NewTimeDatum(now), } - anotherTime, err := now.Add(sessVars.StmtCtx.TypeCtx(), types.NewDuration(24, 0, 0, 0, 0)) + anotherTime, err := now.Add(tc, types.NewDuration(24, 0, 0, 0, 0)) require.Nil(t, err) rowToRemove := []types.Datum{ types.NewStringDatum("old string"), @@ -254,7 +253,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { for _, isCommonHandle := range []bool{true, false} { for _, lc := range locations { for _, columnInfos := range columnInfoSets { - sessVars.StmtCtx.SetTimeZone(lc) + tc = tc.WithLocation(lc) tableInfo := model.TableInfo{ ID: 1, Name: model.NewCIStr("t"), @@ -266,7 +265,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { table := MockTableFromMeta(&tableInfo).(*TableCommon) var handle, corruptedHandle kv.Handle if isCommonHandle { - encoded, err := codec.EncodeKey(sessVars.StmtCtx.TimeZone(), nil, rowToInsert[0]) + encoded, err := codec.EncodeKey(tc.Location(), nil, rowToInsert[0]) require.Nil(t, err) corrupted := make([]byte, len(encoded)) copy(corrupted, encoded) @@ -285,10 +284,10 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { maps := getOrBuildColumnMaps(getter, setter, table) // test checkIndexKeys - insertionKey, insertionValue, err := buildIndexKeyValue(index, rowToInsert, sessVars, tableInfo, + insertionKey, insertionValue, err := buildIndexKeyValue(index, rowToInsert, tc.Location(), tableInfo, indexInfo, table, handle) require.Nil(t, err) - deletionKey, _, err := buildIndexKeyValue(index, rowToRemove, sessVars, tableInfo, indexInfo, table, + deletionKey, _, err := buildIndexKeyValue(index, rowToRemove, tc.Location(), tableInfo, indexInfo, table, handle) require.Nil(t, err) indexMutations := []mutation{ @@ -296,7 +295,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { {key: deletionKey, indexID: indexInfo.ID}, } err = checkIndexKeys( - sessVars, table, rowToInsert, rowToRemove, indexMutations, maps.IndexIDToInfo, + tc, table, rowToInsert, rowToRemove, indexMutations, maps.IndexIDToInfo, maps.IndexIDToRowColInfos, ) require.Nil(t, err) @@ -304,7 +303,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { // test checkHandleConsistency rowKey := tablecodec.EncodeRowKeyWithHandle(table.tableID, handle) corruptedRowKey := tablecodec.EncodeRowKeyWithHandle(table.tableID, corruptedHandle) - rowValue, err := tablecodec.EncodeRow(sessVars.StmtCtx.TimeZone(), rowToInsert, []int64{1, 2}, nil, nil, nil, &rd) + rowValue, err := tablecodec.EncodeRow(tc.Location(), rowToInsert, []int64{1, 2}, nil, nil, nil, &rd) require.Nil(t, err) rowMutation := mutation{key: rowKey, value: rowValue} corruptedRowMutation := mutation{key: corruptedRowKey, value: rowValue} @@ -318,21 +317,21 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { } } -func buildIndexKeyValue(index table.Index, rowToInsert []types.Datum, sessVars *variable.SessionVars, +func buildIndexKeyValue(index table.Index, rowToInsert []types.Datum, loc *time.Location, tableInfo model.TableInfo, indexInfo *model.IndexInfo, table *TableCommon, handle kv.Handle) ([]byte, []byte, error) { indexedValues, err := index.FetchValues(rowToInsert, nil) if err != nil { return nil, nil, err } key, distinct, err := tablecodec.GenIndexKey( - sessVars.StmtCtx.TimeZone(), &tableInfo, indexInfo, 1, indexedValues, handle, nil, + loc, &tableInfo, indexInfo, 1, indexedValues, handle, nil, ) if err != nil { return nil, nil, err } rsData := TryGetHandleRestoredDataWrapper(table.meta, rowToInsert, nil, indexInfo) value, err := tablecodec.GenIndexValuePortal( - sessVars.StmtCtx.TimeZone(), &tableInfo, indexInfo, NeedRestoredData(indexInfo.Columns, tableInfo.Columns), + loc, &tableInfo, indexInfo, NeedRestoredData(indexInfo.Columns, tableInfo.Columns), distinct, false, indexedValues, handle, 0, rsData, nil, ) if err != nil { diff --git a/pkg/table/tables/tables.go b/pkg/table/tables/tables.go index 31f8bf10f417e..2d2ffb6191bc0 100644 --- a/pkg/table/tables/tables.go +++ b/pkg/table/tables/tables.go @@ -543,8 +543,8 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx table.MutateContext } key := t.RecordKey(h) - sc := sessVars.StmtCtx - err = encodeRowBuffer.WriteMemBufferEncoded(sctx.GetRowEncodingConfig(), sc.TimeZone(), sc.ErrCtx(), memBuffer, key) + tc, ec := evalCtx.TypeCtx(), evalCtx.ErrCtx() + err = encodeRowBuffer.WriteMemBufferEncoded(sctx.GetRowEncodingConfig(), tc.Location(), ec, memBuffer, key) if err != nil { return err } @@ -574,7 +574,7 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx table.MutateContext return err } if sessVars.EnableMutationChecker { - if err = CheckDataConsistency(txn, sessVars, t, newData, oldData, memBuffer, sh); err != nil { + if err = CheckDataConsistency(txn, tc, t, newData, oldData, memBuffer, sh); err != nil { return errors.Trace(err) } } @@ -593,12 +593,12 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx table.MutateContext } colSizeBuffer := mutateBuffers.GetColSizeDeltaBufferWithCap(len(t.Cols())) for id, col := range t.Cols() { - size, err := codec.EstimateValueSize(sc.TypeCtx(), newData[id]) + size, err := codec.EstimateValueSize(tc, newData[id]) if err != nil { continue } newLen := size - 1 - size, err = codec.EstimateValueSize(sc.TypeCtx(), oldData[id]) + size, err = codec.EstimateValueSize(tc, oldData[id]) if err != nil { continue } @@ -793,6 +793,10 @@ func (t *TableCommon) AddRecord(sctx table.MutateContext, r []types.Datum, opts } else { ctx = context.Background() } + + evalCtx := sctx.GetExprCtx().GetEvalCtx() + tc, ec := evalCtx.TypeCtx(), evalCtx.ErrCtx() + var hasRecordID bool cols := t.Cols() // opt.IsUpdate is a flag for update. @@ -816,8 +820,8 @@ func (t *TableCommon) AddRecord(sctx table.MutateContext, r []types.Datum, opts } tablecodec.TruncateIndexValues(tblInfo, pkIdx, pkDts) var handleBytes []byte - handleBytes, err = codec.EncodeKey(sctx.GetSessionVars().StmtCtx.TimeZone(), nil, pkDts...) - err = sctx.GetSessionVars().StmtCtx.HandleError(err) + handleBytes, err = codec.EncodeKey(tc.Location(), nil, pkDts...) + err = ec.HandleError(err) if err != nil { return } @@ -949,8 +953,7 @@ func (t *TableCommon) AddRecord(sctx table.MutateContext, r []types.Datum, opts } } - sc := sessVars.StmtCtx - err = encodeRowBuffer.WriteMemBufferEncoded(sctx.GetRowEncodingConfig(), sc.TimeZone(), sc.ErrCtx(), memBuffer, key, flags...) + err = encodeRowBuffer.WriteMemBufferEncoded(sctx.GetRowEncodingConfig(), tc.Location(), ec, memBuffer, key, flags...) if err != nil { return nil, err } @@ -994,7 +997,7 @@ func (t *TableCommon) AddRecord(sctx table.MutateContext, r []types.Datum, opts return nil, err } if sessVars.EnableMutationChecker { - if err = CheckDataConsistency(txn, sessVars, t, r, nil, memBuffer, sh); err != nil { + if err = CheckDataConsistency(txn, tc, t, r, nil, memBuffer, sh); err != nil { return nil, errors.Trace(err) } } @@ -1015,7 +1018,7 @@ func (t *TableCommon) AddRecord(sctx table.MutateContext, r []types.Datum, opts colSizeBuffer := sctx.GetMutateBuffers().GetColSizeDeltaBufferWithCap(len(t.Cols())) for id, col := range t.Cols() { - size, err := codec.EstimateValueSize(sc.TypeCtx(), r[id]) + size, err := codec.EstimateValueSize(tc, r[id]) if err != nil { continue } @@ -1256,13 +1259,13 @@ func (t *TableCommon) RemoveRecord(ctx table.MutateContext, h kv.Handle, r []typ return err } - sessVars := ctx.GetSessionVars() - sc := sessVars.StmtCtx if err = injectMutationError(t, txn, sh); err != nil { return err } - if sessVars.EnableMutationChecker { - if err = CheckDataConsistency(txn, sessVars, t, nil, r, memBuffer, sh); err != nil { + + tc := ctx.GetExprCtx().GetEvalCtx().TypeCtx() + if ctx.GetSessionVars().EnableMutationChecker { + if err = CheckDataConsistency(txn, tc, t, nil, r, memBuffer, sh); err != nil { return errors.Trace(err) } } @@ -1298,7 +1301,7 @@ func (t *TableCommon) RemoveRecord(ctx table.MutateContext, h kv.Handle, r []typ // It can only act as a temporary buffer for the current function call. colSizeBuffer := ctx.GetMutateBuffers().GetColSizeDeltaBufferWithCap(len(t.Cols())) for id, col := range t.Cols() { - size, err := codec.EstimateValueSize(sc.TypeCtx(), r[id]) + size, err := codec.EstimateValueSize(tc, r[id]) if err != nil { continue }