Skip to content

Commit a53b568

Browse files
committed
ddl: remove mock.Context from DDL reorg
1 parent edbec22 commit a53b568

File tree

10 files changed

+243
-39
lines changed

10 files changed

+243
-39
lines changed

pkg/ddl/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -133,6 +133,7 @@ go_library(
133133
"//pkg/store/driver/txn",
134134
"//pkg/store/helper",
135135
"//pkg/table",
136+
"//pkg/table/context",
136137
"//pkg/table/tables",
137138
"//pkg/tablecodec",
138139
"//pkg/tidb-binlog/pump_client",

pkg/ddl/backfilling.go

Lines changed: 23 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -33,12 +33,12 @@ import (
3333
"github.com/pingcap/tidb/pkg/disttask/operator"
3434
"github.com/pingcap/tidb/pkg/expression"
3535
exprctx "github.com/pingcap/tidb/pkg/expression/context"
36+
"github.com/pingcap/tidb/pkg/expression/contextstatic"
3637
"github.com/pingcap/tidb/pkg/kv"
3738
"github.com/pingcap/tidb/pkg/meta/model"
3839
"github.com/pingcap/tidb/pkg/metrics"
3940
pmodel "github.com/pingcap/tidb/pkg/parser/model"
4041
"github.com/pingcap/tidb/pkg/parser/terror"
41-
"github.com/pingcap/tidb/pkg/sessionctx"
4242
"github.com/pingcap/tidb/pkg/sessionctx/variable"
4343
"github.com/pingcap/tidb/pkg/table"
4444
"github.com/pingcap/tidb/pkg/tablecodec"
@@ -155,7 +155,6 @@ type backfillTaskContext struct {
155155
type backfillCtx struct {
156156
id int
157157
*ddlCtx
158-
sessCtx sessionctx.Context
159158
warnings contextutil.WarnHandlerExt
160159
loc *time.Location
161160
exprCtx exprctx.BuildContext
@@ -168,26 +167,40 @@ type backfillCtx struct {
168167
}
169168

170169
func newBackfillCtx(id int, rInfo *reorgInfo,
171-
schemaName string, tbl table.Table, jobCtx *ReorgContext, label string, isDistributed bool) (*backfillCtx, error) {
172-
// TODO: remove newReorgSessCtx
173-
sessCtx := newReorgSessCtx(rInfo.jobCtx.store)
174-
if err := initSessCtx(sessCtx, rInfo.ReorgMeta); err != nil {
170+
schemaName string, tbl table.Table, jobCtx *ReorgContext, label string, isDistributed bool, isUpdateColumn bool) (*backfillCtx, error) {
171+
warnHandler := contextutil.NewStaticWarnHandler(0)
172+
exprCtx, err := newReorgExprCtxWithReorgMeta(rInfo.ReorgMeta, warnHandler)
173+
if err != nil {
175174
return nil, errors.Trace(err)
176175
}
177176

177+
if isUpdateColumn {
178+
// The below case should be compatible with mysql behavior:
179+
// > create table t (a int);
180+
// > insert into t values (0);
181+
// > alter table t modify column a date;
182+
// The alter DDL should return an error in strict mode and success in non-strict mode.
183+
// See: https://github.com/pingcap/tidb/pull/25728 for more details.
184+
hasStrictMode := rInfo.ReorgMeta.SQLMode.HasStrictMode()
185+
tc := exprCtx.GetStaticEvalCtx().TypeCtx()
186+
evalCtx := exprCtx.GetStaticEvalCtx().Apply(contextstatic.WithTypeFlags(
187+
tc.Flags().WithIgnoreZeroDateErr(!hasStrictMode),
188+
))
189+
exprCtx = exprCtx.Apply(contextstatic.WithEvalCtx(evalCtx))
190+
}
191+
192+
tblCtx := newReorgTableMutateContext(exprCtx)
178193
if isDistributed {
179194
id = int(backfillContextID.Add(1))
180195
}
181196

182-
exprCtx := sessCtx.GetExprCtx()
183197
batchCnt := rInfo.ReorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize()))
184198
return &backfillCtx{
185199
id: id,
186200
ddlCtx: rInfo.jobCtx.oldDDLCtx,
187-
sessCtx: sessCtx,
188-
warnings: sessCtx.GetSessionVars().StmtCtx.WarnHandler,
201+
warnings: warnHandler,
189202
exprCtx: exprCtx,
190-
tblCtx: sessCtx.GetTableCtx(),
203+
tblCtx: tblCtx,
191204
loc: exprCtx.GetEvalCtx().Location(),
192205
schemaName: schemaName,
193206
table: tbl,

pkg/ddl/backfilling_scheduler.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -275,7 +275,7 @@ func (b *txnBackfillScheduler) adjustWorkerSize() error {
275275
)
276276
switch b.tp {
277277
case typeAddIndexWorker:
278-
backfillCtx, err := newBackfillCtx(i, reorgInfo, job.SchemaName, b.tbl, jc, "add_idx_rate", false)
278+
backfillCtx, err := newBackfillCtx(i, reorgInfo, job.SchemaName, b.tbl, jc, "add_idx_rate", false, false)
279279
if err != nil {
280280
return err
281281
}
@@ -288,7 +288,7 @@ func (b *txnBackfillScheduler) adjustWorkerSize() error {
288288
runner = newBackfillWorker(b.ctx, idxWorker)
289289
worker = idxWorker
290290
case typeAddIndexMergeTmpWorker:
291-
backfillCtx, err := newBackfillCtx(i, reorgInfo, job.SchemaName, b.tbl, jc, "merge_tmp_idx_rate", false)
291+
backfillCtx, err := newBackfillCtx(i, reorgInfo, job.SchemaName, b.tbl, jc, "merge_tmp_idx_rate", false, false)
292292
if err != nil {
293293
return err
294294
}

pkg/ddl/backfilling_test.go

Lines changed: 83 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ package ddl
1717
import (
1818
"bytes"
1919
"testing"
20+
"time"
2021

2122
"github.com/pingcap/tidb/pkg/ddl/ingest"
2223
distsqlctx "github.com/pingcap/tidb/pkg/distsql/context"
@@ -27,6 +28,8 @@ import (
2728
"github.com/pingcap/tidb/pkg/parser/mysql"
2829
"github.com/pingcap/tidb/pkg/privilege"
2930
"github.com/pingcap/tidb/pkg/sessionctx"
31+
"github.com/pingcap/tidb/pkg/sessionctx/variable"
32+
"github.com/pingcap/tidb/pkg/table"
3033
"github.com/pingcap/tidb/pkg/types"
3134
contextutil "github.com/pingcap/tidb/pkg/util/context"
3235
"github.com/pingcap/tidb/pkg/util/deeptest"
@@ -224,13 +227,25 @@ func assertStaticExprContextEqual(t *testing.T, sctx sessionctx.Context, exprCtx
224227
)
225228
}
226229

230+
// newMockReorgSessCtx creates a mock session context for reorg test.
231+
// In old implementations, DDL is using `mock.Context` to construct the contexts used in DDL reorg.
232+
// After refactoring, we just need it to do test the new implementation is compatible with the old one.
233+
func newMockReorgSessCtx(store kv.Storage) sessionctx.Context {
234+
c := mock.NewContext()
235+
c.Store = store
236+
c.GetSessionVars().SetStatusFlag(mysql.ServerStatusAutocommit, false)
237+
tz := *time.UTC
238+
c.ResetSessionAndStmtTimeZone(&tz)
239+
return c
240+
}
241+
227242
// TestReorgExprContext is used in refactor stage to make sure the newReorgExprCtx() is
228-
// compatible with newReorgSessCtx(nil).GetExprCtx() to make it safe to replace `mock.Context` usage.
243+
// compatible with newMockReorgSessCtx(nil).GetExprCtx() to make it safe to replace `mock.Context` usage.
229244
// After refactor, the TestReorgExprContext can be removed.
230245
func TestReorgExprContext(t *testing.T) {
231246
// test default expr context
232247
store := &mockStorage{client: &mock.Client{}}
233-
sctx := newReorgSessCtx(store)
248+
sctx := newMockReorgSessCtx(store)
234249
defaultCtx := newReorgExprCtx()
235250
// should use an empty static warn handler by default
236251
evalCtx := defaultCtx.GetStaticEvalCtx()
@@ -255,7 +270,7 @@ func TestReorgExprContext(t *testing.T) {
255270
ResourceGroupName: "rg2",
256271
},
257272
} {
258-
sctx = newReorgSessCtx(store)
273+
sctx = newMockReorgSessCtx(store)
259274
require.NoError(t, initSessCtx(sctx, &reorg))
260275
ctx, err := newReorgExprCtxWithReorgMeta(&reorg, sctx.GetSessionVars().StmtCtx.WarnHandler)
261276
require.NoError(t, err)
@@ -278,6 +293,68 @@ func TestReorgExprContext(t *testing.T) {
278293
}
279294
}
280295

296+
func TestReorgTableMutateContext(t *testing.T) {
297+
originalRowFmt := variable.GetDDLReorgRowFormat()
298+
defer variable.SetDDLReorgRowFormat(originalRowFmt)
299+
300+
exprCtx := contextstatic.NewStaticExprContext()
301+
302+
assertTblCtxMatchSessionCtx := func(ctx table.MutateContext, sctx sessionctx.Context) {
303+
sctxTblCtx := sctx.GetTableCtx()
304+
require.Equal(t, uint64(0), ctx.ConnectionID())
305+
require.Equal(t, sctxTblCtx.ConnectionID(), ctx.ConnectionID())
306+
307+
require.False(t, ctx.InRestrictedSQL())
308+
require.Equal(t, sctxTblCtx.InRestrictedSQL(), ctx.InRestrictedSQL())
309+
310+
require.Equal(t, variable.AssertionLevelOff, ctx.TxnAssertionLevel())
311+
require.Equal(t, sctxTblCtx.TxnAssertionLevel(), ctx.TxnAssertionLevel())
312+
313+
require.Equal(t, variable.GetDDLReorgRowFormat() != variable.DefTiDBRowFormatV1, ctx.GetRowEncodingConfig().IsRowLevelChecksumEnabled)
314+
require.Equal(t, variable.GetDDLReorgRowFormat() != variable.DefTiDBRowFormatV1, ctx.GetRowEncodingConfig().RowEncoder.Enable)
315+
require.Equal(t, sctxTblCtx.GetRowEncodingConfig(), ctx.GetRowEncodingConfig())
316+
317+
require.NotNil(t, ctx.GetMutateBuffers())
318+
require.Equal(t, sctxTblCtx.GetMutateBuffers(), ctx.GetMutateBuffers())
319+
320+
require.Equal(t, variable.DefTiDBShardAllocateStep, ctx.GetRowIDShardGenerator().GetShardStep())
321+
sctx.GetSessionVars().TxnCtx.StartTS = 123 // make sure GetRowIDShardGenerator() pass assert
322+
require.Equal(t, sctxTblCtx.GetRowIDShardGenerator().GetShardStep(), ctx.GetRowIDShardGenerator().GetShardStep())
323+
require.GreaterOrEqual(t, ctx.GetRowIDShardGenerator().GetCurrentShard(1), int64(0))
324+
325+
alloc1, ok := sctxTblCtx.GetReservedRowIDAlloc()
326+
require.True(t, ok)
327+
alloc2, ok := ctx.GetReservedRowIDAlloc()
328+
require.True(t, ok)
329+
require.Equal(t, alloc1, alloc2)
330+
require.True(t, alloc2.Exhausted())
331+
332+
binlog, ok := ctx.GetBinlogSupport()
333+
require.False(t, ok)
334+
require.Nil(t, binlog)
335+
statistics, ok := ctx.GetStatisticsSupport()
336+
require.False(t, ok)
337+
require.Nil(t, statistics)
338+
cached, ok := ctx.GetCachedTableSupport()
339+
require.False(t, ok)
340+
require.Nil(t, cached)
341+
temp, ok := ctx.GetTemporaryTableSupport()
342+
require.False(t, ok)
343+
require.Nil(t, temp)
344+
dml, ok := ctx.GetExchangePartitionDMLSupport()
345+
require.False(t, ok)
346+
require.Nil(t, dml)
347+
}
348+
349+
// test when the row format is v1
350+
variable.SetDDLReorgRowFormat(variable.DefTiDBRowFormatV1)
351+
sctx := newMockReorgSessCtx(&mockStorage{client: &mock.Client{}})
352+
require.NoError(t, initSessCtx(sctx, &model.DDLReorgMeta{}))
353+
ctx := newReorgTableMutateContext(exprCtx)
354+
require.Same(t, exprCtx, ctx.GetExprCtx())
355+
assertTblCtxMatchSessionCtx(ctx, sctx)
356+
}
357+
281358
type mockStorage struct {
282359
kv.Storage
283360
client kv.Client
@@ -313,13 +390,13 @@ func assertDistSQLCtxEqual(t *testing.T, expected *distsqlctx.DistSQLContext, ac
313390
}
314391

315392
// TestReorgExprContext is used in refactor stage to make sure the newDefaultReorgDistSQLCtx() is
316-
// compatible with newReorgSessCtx(nil).GetDistSQLCtx() to make it safe to replace `mock.Context` usage.
393+
// compatible with newMockReorgSessCtx(nil).GetDistSQLCtx() to make it safe to replace `mock.Context` usage.
317394
// After refactor, the TestReorgExprContext can be removed.
318395
func TestReorgDistSQLCtx(t *testing.T) {
319396
store := &mockStorage{client: &mock.Client{}}
320397

321398
// test default dist sql context
322-
expected := newReorgSessCtx(store).GetDistSQLCtx()
399+
expected := newMockReorgSessCtx(store).GetDistSQLCtx()
323400
defaultCtx := newDefaultReorgDistSQLCtx(store.client, expected.WarnHandler)
324401
assertDistSQLCtxEqual(t, expected, defaultCtx)
325402

@@ -339,7 +416,7 @@ func TestReorgDistSQLCtx(t *testing.T) {
339416
ResourceGroupName: "rg2",
340417
},
341418
} {
342-
sctx := newReorgSessCtx(store)
419+
sctx := newMockReorgSessCtx(store)
343420
require.NoError(t, initSessCtx(sctx, &reorg))
344421
expected = sctx.GetDistSQLCtx()
345422
ctx, err := newReorgDistSQLCtxWithReorgMeta(store.client, &reorg, expected.WarnHandler)

pkg/ddl/column.go

Lines changed: 1 addition & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -606,18 +606,11 @@ type updateColumnWorker struct {
606606
}
607607

608608
func newUpdateColumnWorker(id int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *ReorgContext) (*updateColumnWorker, error) {
609-
bCtx, err := newBackfillCtx(id, reorgInfo, reorgInfo.SchemaName, t, jc, "update_col_rate", false)
609+
bCtx, err := newBackfillCtx(id, reorgInfo, reorgInfo.SchemaName, t, jc, "update_col_rate", false, true)
610610
if err != nil {
611611
return nil, err
612612
}
613613

614-
sessCtx := bCtx.sessCtx
615-
sessCtx.GetSessionVars().StmtCtx.SetTypeFlags(
616-
sessCtx.GetSessionVars().StmtCtx.TypeFlags().
617-
WithIgnoreZeroDateErr(!reorgInfo.ReorgMeta.SQLMode.HasStrictMode()))
618-
bCtx.exprCtx = bCtx.sessCtx.GetExprCtx()
619-
bCtx.tblCtx = bCtx.sessCtx.GetTableCtx()
620-
621614
if !bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) {
622615
logutil.DDLLogger().Error("Element type for updateColumnWorker incorrect", zap.String("jobQuery", reorgInfo.Query),
623616
zap.Stringer("reorgInfo", reorgInfo))

pkg/ddl/index.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2397,7 +2397,7 @@ type cleanUpIndexWorker struct {
23972397
}
23982398

23992399
func newCleanUpIndexWorker(id int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *ReorgContext) (*cleanUpIndexWorker, error) {
2400-
bCtx, err := newBackfillCtx(id, reorgInfo, reorgInfo.SchemaName, t, jc, "cleanup_idx_rate", false)
2400+
bCtx, err := newBackfillCtx(id, reorgInfo, reorgInfo.SchemaName, t, jc, "cleanup_idx_rate", false, false)
24012401
if err != nil {
24022402
return nil, err
24032403
}

pkg/ddl/partition.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3600,7 +3600,7 @@ type reorgPartitionWorker struct {
36003600
}
36013601

36023602
func newReorgPartitionWorker(i int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *ReorgContext) (*reorgPartitionWorker, error) {
3603-
bCtx, err := newBackfillCtx(i, reorgInfo, reorgInfo.SchemaName, t, jc, "reorg_partition_rate", false)
3603+
bCtx, err := newBackfillCtx(i, reorgInfo, reorgInfo.SchemaName, t, jc, "reorg_partition_rate", false, false)
36043604
if err != nil {
36053605
return nil, err
36063606
}

0 commit comments

Comments
 (0)