Skip to content

Commit

Permalink
test: fix data race (#42306)
Browse files Browse the repository at this point in the history
close #42270
  • Loading branch information
xiongjiwei authored Mar 16, 2023
1 parent f832456 commit 4f44dab
Show file tree
Hide file tree
Showing 9 changed files with 12 additions and 14 deletions.
2 changes: 1 addition & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error {
}

e.ctx.GetSessionVars().PlanID = 0
e.ctx.GetSessionVars().PlanColumnID = 0
e.ctx.GetSessionVars().PlanColumnID.Store(0)
e.ctx.GetSessionVars().MapHashCode2UniqueID4ExtendedCol = nil
// In MySQL prepare protocol, the server need to tell the client how many column the prepared statement would return when executing it.
// For a query with on result, e.g. an insert statement, there will be no result, so 'e.Fields' is not set.
Expand Down
2 changes: 1 addition & 1 deletion expression/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,6 @@ func createContext(t *testing.T) *mock.Context {
sc := ctx.GetSessionVars().StmtCtx
sc.TruncateAsWarning = true
require.NoError(t, ctx.GetSessionVars().SetSystemVar("max_allowed_packet", "67108864"))
ctx.GetSessionVars().PlanColumnID = 0
ctx.GetSessionVars().PlanColumnID.Store(0)
return ctx
}
2 changes: 1 addition & 1 deletion planner/core/casetest/stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func TestGroupNDVs(t *testing.T) {
ret := &core.PreprocessorReturn{}
err = core.Preprocess(context.Background(), tk.Session(), stmt, core.WithPreprocessorReturn(ret))
require.NoError(t, err)
tk.Session().GetSessionVars().PlanColumnID = 0
tk.Session().GetSessionVars().PlanColumnID.Store(0)
builder, _ := core.NewPlanBuilder().Init(tk.Session(), ret.InfoSchema, &hint.BlockHintProcessor{})
p, err := builder.Build(ctx, stmt)
require.NoError(t, err, comment)
Expand Down
2 changes: 1 addition & 1 deletion planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ type logicalOptRule interface {
// BuildLogicalPlanForTest builds a logical plan for testing purpose from ast.Node.
func BuildLogicalPlanForTest(ctx context.Context, sctx sessionctx.Context, node ast.Node, infoSchema infoschema.InfoSchema) (Plan, types.NameSlice, error) {
sctx.GetSessionVars().PlanID = 0
sctx.GetSessionVars().PlanColumnID = 0
sctx.GetSessionVars().PlanColumnID.Store(0)
builder, _ := NewPlanBuilder().Init(sctx, infoSchema, &utilhint.BlockHintProcessor{})
p, err := builder.Build(ctx, node)
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion planner/core/point_get_plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -535,7 +535,7 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) {
}

ctx.GetSessionVars().PlanID = 0
ctx.GetSessionVars().PlanColumnID = 0
ctx.GetSessionVars().PlanColumnID.Store(0)
switch x := node.(type) {
case *ast.SelectStmt:
defer func() {
Expand Down
3 changes: 1 addition & 2 deletions planner/core/rule_join_reorder_dp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,9 +138,8 @@ func newDataSource(ctx sessionctx.Context, name string, count int) LogicalPlan {
tan := model.NewCIStr(name)
ds.TableAsName = &tan
ds.schema = expression.NewSchema()
ctx.GetSessionVars().PlanColumnID++
ds.schema.Append(&expression.Column{
UniqueID: ctx.GetSessionVars().PlanColumnID,
UniqueID: ctx.GetSessionVars().PlanColumnID.Add(1),
RetType: types.NewFieldType(mysql.TypeLonglong),
})
ds.stats = &property.StatsInfo{
Expand Down
6 changes: 3 additions & 3 deletions planner/funcdep/extract_fd_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ func TestFDSet_ExtractFD(t *testing.T) {
stmt, err := par.ParseOneStmt(tt.sql, "", "")
require.NoError(t, err, comment)
tk.Session().GetSessionVars().PlanID = 0
tk.Session().GetSessionVars().PlanColumnID = 0
tk.Session().GetSessionVars().PlanColumnID.Store(0)
err = plannercore.Preprocess(context.Background(), tk.Session(), stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is}))
require.NoError(t, err)
require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).AdviseWarmup())
Expand Down Expand Up @@ -317,7 +317,7 @@ func TestFDSet_ExtractFDForApply(t *testing.T) {
stmt, err := par.ParseOneStmt(tt.sql, "", "")
require.NoError(t, err, comment)
tk.Session().GetSessionVars().PlanID = 0
tk.Session().GetSessionVars().PlanColumnID = 0
tk.Session().GetSessionVars().PlanColumnID.Store(0)
err = plannercore.Preprocess(context.Background(), tk.Session(), stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is}))
require.NoError(t, err, comment)
require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).AdviseWarmup())
Expand Down Expand Up @@ -365,7 +365,7 @@ func TestFDSet_MakeOuterJoin(t *testing.T) {
stmt, err := par.ParseOneStmt(tt.sql, "", "")
require.NoError(t, err, comment)
tk.Session().GetSessionVars().PlanID = 0
tk.Session().GetSessionVars().PlanColumnID = 0
tk.Session().GetSessionVars().PlanColumnID.Store(0)
err = plannercore.Preprocess(context.Background(), tk.Session(), stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is}))
require.NoError(t, err, comment)
require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).AdviseWarmup())
Expand Down
2 changes: 1 addition & 1 deletion planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -493,7 +493,7 @@ func OptimizeExecStmt(ctx context.Context, sctx sessionctx.Context,

func buildLogicalPlan(ctx context.Context, sctx sessionctx.Context, node ast.Node, builder *core.PlanBuilder) (core.Plan, error) {
sctx.GetSessionVars().PlanID = 0
sctx.GetSessionVars().PlanColumnID = 0
sctx.GetSessionVars().PlanColumnID.Store(0)
sctx.GetSessionVars().MapHashCode2UniqueID4ExtendedCol = nil

failpoint.Inject("mockRandomPlanID", func() {
Expand Down
5 changes: 2 additions & 3 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -702,7 +702,7 @@ type SessionVars struct {
PlanID int

// PlanColumnID is the unique id for column when building plan.
PlanColumnID int64
PlanColumnID atomic.Int64

// MapHashCode2UniqueID4ExtendedCol map the expr's hash code to specified unique ID.
MapHashCode2UniqueID4ExtendedCol map[string]int
Expand Down Expand Up @@ -1973,8 +1973,7 @@ func (s *SessionVars) CleanBuffers() {

// AllocPlanColumnID allocates column id for plan.
func (s *SessionVars) AllocPlanColumnID() int64 {
s.PlanColumnID++
return s.PlanColumnID
return s.PlanColumnID.Add(1)
}

// GetCharsetInfo gets charset and collation for current context.
Expand Down

0 comments on commit 4f44dab

Please sign in to comment.