Skip to content

Commit

Permalink
eval: mark stored context.Context as deprecated
Browse files Browse the repository at this point in the history
This commit unexports the stored `context.Context` from `eval.Context`,
marks it as "deprecated" and provides getter/setter methods for now.
There are just two places that still use 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. In order to not lose
progress on the removal achieved in the previous commits we do this
deprecation.

The next step for the complete removal is to do analysis of the web of
callsites of these two methods to see whether we have incorrect
layering, and this will be done separately.

Release note: None
  • Loading branch information
yuzefovich committed Oct 3, 2022
1 parent 25f16ef commit 6f3b715
Show file tree
Hide file tree
Showing 10 changed files with 52 additions and 32 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2813,7 +2813,7 @@ func (ex *connExecutor) resetEvalCtx(evalCtx *extendedEvalContext, txn *kv.Txn,
evalCtx.Placeholders = nil
evalCtx.Annotations = nil
evalCtx.IVarContainer = nil
evalCtx.Context.Context = ex.Ctx()
evalCtx.SetDeprecatedContext(ex.Ctx())
evalCtx.Txn = txn
evalCtx.PrepareOnly = false
evalCtx.SkipNormalize = false
Expand Down
24 changes: 12 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 All @@ -386,6 +383,9 @@ func (ds *ServerImpl) setupFlow(
IndexUsageStatsController: ds.ServerConfig.IndexUsageStatsController,
RangeStatsFetcher: ds.ServerConfig.RangeStatsFetcher,
}
// Most processors will override this Context with their own context in
// ProcessorBase. StartInternal().
evalCtx.SetDeprecatedContext(ctx)
evalCtx.SetStmtTimestamp(timeutil.Unix(0 /* sec */, req.EvalContext.StmtTimestampNanos))
evalCtx.SetTxnTimestamp(timeutil.Unix(0 /* sec */, req.EvalContext.TxnTimestampNanos))
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/execinfra/processorsbase.go
Original file line number Diff line number Diff line change
Expand Up @@ -868,8 +868,8 @@ 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
pb.evalOrigCtx = pb.EvalCtx.GetDeprecatedContext()
pb.EvalCtx.SetDeprecatedContext(pb.Ctx)
return pb.Ctx
}

Expand Down Expand Up @@ -900,7 +900,7 @@ 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
pb.EvalCtx.SetDeprecatedContext(pb.evalOrigCtx)
return true
}

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

evalCtx := eval.Context{
Context: ctx,
Sequence: &importSequenceOperators{},
Regions: makeImportRegionOperator(""),
SessionDataStack: sessiondata.NewStack(&sessiondata.SessionData{}),
ClientNoticeSender: &faketreeeval.DummyClientNoticeSender{},
TxnTimestamp: timeutil.Unix(0, walltime),
Settings: st,
}
evalCtx.SetDeprecatedContext(ctx)
affected := make(map[descpb.ID]*tabledesc.Mutable)

tableDesc, err := sql.NewTableDesc(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -265,7 +265,7 @@ func (ih *instrumentationHelper) Setup(
ih.traceMetadata = make(execNodeTraceMetadata)
}
// Make sure that the builtins use the correct context.
ih.evalCtx.Context = newCtx
ih.evalCtx.SetDeprecatedContext(newCtx)
}()

if sp := tracing.SpanFromContext(ctx); sp != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -475,7 +475,6 @@ func internalExtendedEvalCtx(
TxnReadOnly: false,
TxnImplicit: true,
TxnIsSingleStmt: true,
Context: ctx,
TestingKnobs: evalContextTestingKnobs,
StmtTimestamp: stmtTimestamp,
TxnTimestamp: txnTimestamp,
Expand All @@ -490,6 +489,7 @@ func internalExtendedEvalCtx(
Descs: tables,
indexUsageStats: indexUsageStats,
}
ret.SetDeprecatedContext(ctx)
ret.copyFromExecCfg(execCfg)
return ret
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/row/expr_walker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,9 +83,9 @@ 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,
}
evalCtx.SetDeprecatedContext(ctx)

registry := s.JobRegistry().(*jobs.Registry)
testCases := []struct {
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -2500,10 +2500,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 All @@ -2521,6 +2518,9 @@ func createSchemaChangeEvalCtx(
Tracer: execCfg.AmbientCtx.Tracer,
},
}
// TODO(andrei): This is wrong (just like on the main code path on
// setupFlow). Each processor should override Ctx with its own context.
evalCtx.SetDeprecatedContext(ctx)
// The backfill is going to use the current timestamp for the various
// functions, like now(), that need it. It's possible that the backfill has
// been partially performed already by another SchemaChangeManager with
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/schemachanger/scbuild/tree_context_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,10 +45,9 @@ func (b buildCtx) EvalCtx() *eval.Context {
}

func newEvalCtx(ctx context.Context, d Dependencies) *eval.Context {
return &eval.Context{
evalCtx := &eval.Context{
ClusterID: d.ClusterID(),
SessionDataStack: sessiondata.NewStack(d.SessionData()),
Context: ctx,
Planner: &faketreeeval.DummyEvalPlanner{},
PrivilegedAccessor: &faketreeeval.DummyPrivilegedAccessor{},
SessionAccessor: &faketreeeval.DummySessionAccessor{},
Expand All @@ -59,4 +58,6 @@ func newEvalCtx(ctx context.Context, d Dependencies) *eval.Context {
Settings: d.ClusterSettings(),
Codec: d.Codec(),
}
evalCtx.SetDeprecatedContext(ctx)
return evalCtx
}
29 changes: 24 additions & 5 deletions pkg/sql/sem/eval/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,8 +130,11 @@ 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
// deprecatedContext holds the context in which the expression is evaluated.
//
// Deprecated: this field should not be used because an effort to remove it
// from Context is under way.
deprecatedContext context.Context

Planner Planner

Expand Down Expand Up @@ -269,6 +272,22 @@ type RangeProber interface {
) error
}

// GetDeprecatedContext returns the context.Context of this Context.
//
// Deprecated: this method should not be used because an effort to remove the
// context.Context from Context is under way.
func (ec *Context) GetDeprecatedContext() context.Context {
return ec.deprecatedContext
}

// SetDeprecatedContext updates the context.Context of this Context.
//
// Deprecated: this method should not be used because an effort to remove the
// context.Context from Context is under way.
func (ec *Context) SetDeprecatedContext(ctx context.Context) {
ec.deprecatedContext = ctx
}

// UnwrapDatum encapsulates UnwrapDatum for use in the tree.CompareContext.
func (ec *Context) UnwrapDatum(d tree.Datum) tree.Datum {
return UnwrapDatum(ec, d)
Expand All @@ -280,7 +299,7 @@ func (ec *Context) MustGetPlaceholderValue(p *tree.Placeholder) tree.Datum {
if !ok {
panic(errors.AssertionFailedf("fail"))
}
out, err := Expr(ec.Context, ec, e)
out, err := Expr(ec.deprecatedContext, ec, e)
if err != nil {
panic(errors.NewAssertionErrorWithWrappedErrf(err, "fail"))
}
Expand Down Expand Up @@ -315,7 +334,7 @@ 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()
ctx.deprecatedContext = context.TODO()
now := timeutil.Now()
ctx.SetTxnTimestamp(now)
ctx.SetStmtTimestamp(now)
Expand Down Expand Up @@ -641,7 +660,7 @@ 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)
ret, err := Expr(evalCtx.deprecatedContext, 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 6f3b715

Please sign in to comment.