Skip to content

Commit

Permalink
eval: WIP on removing context.Context from Context
Browse files Browse the repository at this point in the history
This commit removes `context.Context` from `eval.Context` object. There
were only two places that still used it - namely
`eval.Context.MustGetPlaceholderValue` and `eval.UnwrapDatum` (when
unwrapping a placeholder), and refactoring those is extremely difficult
since the amount of plumbing required is probably thousands of line
changes without an easy way to automate that. So this commit puts
`context.TODO` in those two spots. It seems like a reasonable tradeoff
to eliminate the anti-pattern of what we had in `eval.Context`.

As a sanity check, I put a `nil` and ran the CI, and there were no
failures.

TODO: consider whether we should rename `eval.Context` to something
else.

Release note: None
  • Loading branch information
yuzefovich committed Oct 2, 2022
1 parent 046f912 commit 0195be7
Show file tree
Hide file tree
Showing 14 changed files with 27 additions and 49 deletions.
6 changes: 3 additions & 3 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ func (sc *SchemaChanger) makeFixedTimestampRunner(readAsOf hlc.Timestamp) histor
ie sqlutil.InternalExecutor,
) error {
// We need to re-create the evalCtx since the txn may retry.
evalCtx := createSchemaChangeEvalCtx(ctx, sc.execCfg, sd, readAsOf, descriptors)
evalCtx := createSchemaChangeEvalCtx(sc.execCfg, sd, readAsOf, descriptors)
return retryable(ctx, txn, &evalCtx, ie)
})
}
Expand Down Expand Up @@ -1024,7 +1024,7 @@ func (sc *SchemaChanger) distIndexBackfill(
return err
}
sd := NewFakeSessionData(sc.execCfg.SV())
evalCtx = createSchemaChangeEvalCtx(ctx, sc.execCfg, sd, txn.ReadTimestamp(), descriptors)
evalCtx = createSchemaChangeEvalCtx(sc.execCfg, sd, txn.ReadTimestamp(), descriptors)
planCtx = sc.distSQLPlanner.NewPlanningCtx(ctx, &evalCtx, nil, /* planner */
txn, DistributionTypeSystemTenantOnly)
indexBatchSize := indexBackfillBatchSize.Get(&sc.execCfg.Settings.SV)
Expand Down Expand Up @@ -1321,7 +1321,7 @@ func (sc *SchemaChanger) distColumnBackfill(
}
cbw := MetadataCallbackWriter{rowResultWriter: &errOnlyResultWriter{}, fn: metaFn}
sd := NewFakeSessionData(sc.execCfg.SV())
evalCtx := createSchemaChangeEvalCtx(ctx, sc.execCfg, sd, txn.ReadTimestamp(), descriptors)
evalCtx := createSchemaChangeEvalCtx(sc.execCfg, sd, txn.ReadTimestamp(), descriptors)
recv := MakeDistSQLReceiver(
ctx,
&cbw,
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2813,7 +2813,6 @@ func (ex *connExecutor) resetEvalCtx(evalCtx *extendedEvalContext, txn *kv.Txn,
evalCtx.Placeholders = nil
evalCtx.Annotations = nil
evalCtx.IVarContainer = nil
evalCtx.Context.Context = ex.Ctx()
evalCtx.Txn = txn
evalCtx.PrepareOnly = false
evalCtx.SkipNormalize = false
Expand Down
21 changes: 9 additions & 12 deletions pkg/sql/distsql/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -360,18 +360,15 @@ func (ds *ServerImpl) setupFlow(
return nil, nil, nil, err
}
evalCtx = &eval.Context{
Settings: ds.ServerConfig.Settings,
SessionDataStack: sessiondata.NewStack(sd),
ClusterID: ds.ServerConfig.LogicalClusterID.Get(),
ClusterName: ds.ServerConfig.ClusterName,
NodeID: ds.ServerConfig.NodeID,
Codec: ds.ServerConfig.Codec,
ReCache: ds.regexpCache,
Locality: ds.ServerConfig.Locality,
Tracer: ds.ServerConfig.Tracer,
// Most processors will override this Context with their own context in
// ProcessorBase. StartInternal().
Context: ctx,
Settings: ds.ServerConfig.Settings,
SessionDataStack: sessiondata.NewStack(sd),
ClusterID: ds.ServerConfig.LogicalClusterID.Get(),
ClusterName: ds.ServerConfig.ClusterName,
NodeID: ds.ServerConfig.NodeID,
Codec: ds.ServerConfig.Codec,
ReCache: ds.regexpCache,
Locality: ds.ServerConfig.Locality,
Tracer: ds.ServerConfig.Tracer,
Planner: &faketreeeval.DummyEvalPlanner{Monitor: monitor},
PrivilegedAccessor: &faketreeeval.DummyPrivilegedAccessor{},
SessionAccessor: &faketreeeval.DummySessionAccessor{},
Expand Down
6 changes: 0 additions & 6 deletions pkg/sql/execinfra/processorsbase.go
Original file line number Diff line number Diff line change
Expand Up @@ -359,9 +359,6 @@ type ProcessorBaseNoHelper struct {
// origCtx is the context from which ctx was derived. InternalClose() resets
// ctx to this.
origCtx context.Context
// evalOrigCtx is the original context that was stored in the eval.Context.
// InternalClose() uses it to correctly reset the eval.Context.
evalOrigCtx context.Context

State procState

Expand Down Expand Up @@ -868,8 +865,6 @@ func (pb *ProcessorBaseNoHelper) StartInternal(ctx context.Context, name string)
pb.span.SetTag(execinfrapb.ProcessorIDTagKey, attribute.IntValue(int(pb.ProcessorID)))
}
}
pb.evalOrigCtx = pb.EvalCtx.Context
pb.EvalCtx.Context = pb.Ctx
return pb.Ctx
}

Expand Down Expand Up @@ -900,7 +895,6 @@ func (pb *ProcessorBaseNoHelper) InternalClose() bool {
// Reset the context so that any incidental uses after this point do not
// access the finished span.
pb.Ctx = pb.origCtx
pb.EvalCtx.Context = pb.evalOrigCtx
return true
}

Expand Down
1 change: 0 additions & 1 deletion pkg/sql/importer/import_table_creation.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,6 @@ func MakeSimpleTableDescriptor(
create.Defs = filteredDefs

evalCtx := eval.Context{
Context: ctx,
Sequence: &importSequenceOperators{},
Regions: makeImportRegionOperator(""),
SessionDataStack: sessiondata.NewStack(&sessiondata.SessionData{}),
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/index_backfiller.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ func (ib *IndexBackfillPlanner) plan(
ctx context.Context, txn *kv.Txn, descriptors *descs.Collection,
) error {
sd := NewFakeSessionData(ib.execCfg.SV())
evalCtx = createSchemaChangeEvalCtx(ctx, ib.execCfg, sd, nowTimestamp, descriptors)
evalCtx = createSchemaChangeEvalCtx(ib.execCfg, sd, nowTimestamp, descriptors)
planCtx = ib.execCfg.DistSQLPlanner.NewPlanningCtx(ctx, &evalCtx,
nil /* planner */, txn, DistributionTypeSystemTenantOnly)
// TODO(ajwerner): Adopt util.ConstantWithMetamorphicTestRange for the
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,8 +264,6 @@ func (ih *instrumentationHelper) Setup(
// the helper setup.
ih.traceMetadata = make(execNodeTraceMetadata)
}
// Make sure that the builtins use the correct context.
ih.evalCtx.Context = newCtx
}()

if sp := tracing.SpanFromContext(ctx); sp != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/mvcc_backfiller.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func (im *IndexBackfillerMergePlanner) plan(
ctx context.Context, txn *kv.Txn, descriptors *descs.Collection,
) error {
sd := NewFakeSessionData(im.execCfg.SV())
evalCtx = createSchemaChangeEvalCtx(ctx, im.execCfg, sd, txn.ReadTimestamp(), descriptors)
evalCtx = createSchemaChangeEvalCtx(im.execCfg, sd, txn.ReadTimestamp(), descriptors)
planCtx = im.execCfg.DistSQLPlanner.NewPlanningCtx(ctx, &evalCtx, nil /* planner */, txn,
DistributionTypeSystemTenantOnly)

Expand Down
4 changes: 1 addition & 3 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -378,7 +378,7 @@ func newInternalPlanner(
sessionDataMutatorCallbacks: sessionDataMutatorCallbacks{},
}

p.extendedEvalCtx = internalExtendedEvalCtx(ctx, sds, params.collection, txn, ts, ts, execCfg)
p.extendedEvalCtx = internalExtendedEvalCtx(sds, params.collection, txn, ts, ts, execCfg)
p.extendedEvalCtx.Planner = p
p.extendedEvalCtx.PrivilegedAccessor = p
p.extendedEvalCtx.SessionAccessor = p
Expand Down Expand Up @@ -436,7 +436,6 @@ func newInternalPlanner(
// there's no session to speak of here, different fields are filled in here to
// keep the tests using the internal planner passing.
func internalExtendedEvalCtx(
ctx context.Context,
sds *sessiondata.Stack,
tables *descs.Collection,
txn *kv.Txn,
Expand Down Expand Up @@ -475,7 +474,6 @@ func internalExtendedEvalCtx(
TxnReadOnly: false,
TxnImplicit: true,
TxnIsSingleStmt: true,
Context: ctx,
TestingKnobs: evalContextTestingKnobs,
StmtTimestamp: stmtTimestamp,
TxnTimestamp: txnTimestamp,
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/row/expr_walker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,7 @@ func TestJobBackedSeqChunkProvider(t *testing.T) {
defer s.Stopper().Stop(ctx)

evalCtx := &eval.Context{
Context: ctx,
Codec: s.ExecutorConfig().(sql.ExecutorConfig).Codec,
Codec: s.ExecutorConfig().(sql.ExecutorConfig).Codec,
}

registry := s.JobRegistry().(*jobs.Registry)
Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -2478,7 +2478,6 @@ func (sc *SchemaChanger) txnWithExecutor(
// used in the surrounding SQL session, so session tracing is unable
// to capture schema change activity.
func createSchemaChangeEvalCtx(
ctx context.Context,
execCfg *ExecutorConfig,
sd *sessiondata.SessionData,
ts hlc.Timestamp,
Expand All @@ -2492,10 +2491,7 @@ func createSchemaChangeEvalCtx(
ExecCfg: execCfg,
Descs: descriptors,
Context: eval.Context{
SessionDataStack: sessiondata.NewStack(sd),
// TODO(andrei): This is wrong (just like on the main code path on
// setupFlow). Each processor should override Ctx with its own context.
Context: ctx,
SessionDataStack: sessiondata.NewStack(sd),
Planner: &faketreeeval.DummyEvalPlanner{},
PrivilegedAccessor: &faketreeeval.DummyPrivilegedAccessor{},
SessionAccessor: &faketreeeval.DummySessionAccessor{},
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachanger/scbuild/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ func newBuilderState(ctx context.Context, d Dependencies, initial scpb.CurrentSt
bs := builderState{
ctx: ctx,
clusterSettings: d.ClusterSettings(),
evalCtx: newEvalCtx(ctx, d),
evalCtx: newEvalCtx(d),
semaCtx: newSemaCtx(d),
cr: d.CatalogReader(),
tr: d.TableReader(),
Expand Down
7 changes: 2 additions & 5 deletions pkg/sql/schemachanger/scbuild/tree_context_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,6 @@
package scbuild

import (
"context"

"github.com/cockroachdb/cockroach/pkg/sql/faketreeeval"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild/internal/scbuildstmt"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
Expand Down Expand Up @@ -41,14 +39,13 @@ func newSemaCtx(d Dependencies) *tree.SemaContext {

// EvalCtx implements the scbuildstmt.TreeContextBuilder interface.
func (b buildCtx) EvalCtx() *eval.Context {
return newEvalCtx(b.Context, b.Dependencies)
return newEvalCtx(b.Dependencies)
}

func newEvalCtx(ctx context.Context, d Dependencies) *eval.Context {
func newEvalCtx(d Dependencies) *eval.Context {
return &eval.Context{
ClusterID: d.ClusterID(),
SessionDataStack: sessiondata.NewStack(d.SessionData()),
Context: ctx,
Planner: &faketreeeval.DummyEvalPlanner{},
PrivilegedAccessor: &faketreeeval.DummyPrivilegedAccessor{},
SessionAccessor: &faketreeeval.DummySessionAccessor{},
Expand Down
13 changes: 7 additions & 6 deletions pkg/sql/sem/eval/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,9 +130,6 @@ type Context struct {
// need to restore once we finish evaluating it.
iVarContainerStack []tree.IndexedVarContainer

// Context holds the context in which the expression is evaluated.
Context context.Context

Planner Planner

// Not using sql.JobExecContext type to avoid cycle dependency with sql package
Expand Down Expand Up @@ -280,7 +277,9 @@ func (ec *Context) MustGetPlaceholderValue(p *tree.Placeholder) tree.Datum {
if !ok {
panic(errors.AssertionFailedf("fail"))
}
out, err := Expr(ec.Context, ec, e)
// It is very painful to propagate context.Context into this method since it
// requires adding it as an argument to hundreds (maybe more) of places.
out, err := Expr(context.TODO(), ec, e)
if err != nil {
panic(errors.NewAssertionErrorWithWrappedErrf(err, "fail"))
}
Expand Down Expand Up @@ -315,7 +314,6 @@ func MakeTestingEvalContextWithMon(st *cluster.Settings, monitor *mon.BytesMonit
monitor.Start(context.Background(), nil /* pool */, mon.NewStandaloneBudget(math.MaxInt64))
ctx.TestingMon = monitor
ctx.Planner = &fakePlannerWithMonitor{monitor: monitor}
ctx.Context = context.TODO()
now := timeutil.Now()
ctx.SetTxnTimestamp(now)
ctx.SetStmtTimestamp(now)
Expand Down Expand Up @@ -641,7 +639,10 @@ func arrayOfType(typ *types.T) (*tree.DArray, error) {
func UnwrapDatum(evalCtx *Context, d tree.Datum) tree.Datum {
d = tree.UnwrapDOidWrapper(d)
if p, ok := d.(*tree.Placeholder); ok && evalCtx != nil && evalCtx.HasPlaceholders() {
ret, err := Expr(evalCtx.Context, evalCtx, p)
// It is very painful to propagate context.Context into this method
// since it requires adding it as an argument to hundreds (maybe more)
// of places.
ret, err := Expr(context.TODO(), evalCtx, p)
if err != nil {
// If we fail to evaluate the placeholder, it's because we don't have
// a placeholder available. Just return the placeholder and someone else
Expand Down

0 comments on commit 0195be7

Please sign in to comment.