Skip to content

Commit 1219537

Browse files
committed
sql: use correct context in recursive CTE iterations
Previously, the `execbuilder.Builder` that we use in recursive CTE iterations referenced the same context that the Builder captured on the main query path. This is problematic since that context might have a tracing span that's already been finished. This commit fixes this issue by explicitly passing the context argument into the iteration function. This was only exposed because we added some logging in the following commit but has been present for a while now. Release note: None
1 parent 4e225c5 commit 1219537

File tree

3 files changed

+13
-9
lines changed

3 files changed

+13
-9
lines changed

pkg/sql/opt/exec/execbuilder/relational.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3418,7 +3418,6 @@ func (b *Builder) buildRecursiveCTE(
34183418
// To implement exec.RecursiveCTEIterationFn, we create a special Builder.
34193419

34203420
innerBldTemplate := &Builder{
3421-
ctx: b.ctx,
34223421
mem: b.mem,
34233422
catalog: b.catalog,
34243423
semaCtx: b.semaCtx,
@@ -3430,9 +3429,10 @@ func (b *Builder) buildRecursiveCTE(
34303429
withExprs: b.withExprs[:len(b.withExprs):len(b.withExprs)],
34313430
}
34323431

3433-
fn := func(ef exec.Factory, bufferRef exec.Node) (exec.Plan, error) {
3432+
fn := func(ctx context.Context, ef exec.Factory, bufferRef exec.Node) (exec.Plan, error) {
34343433
// Use a separate builder each time.
34353434
innerBld := *innerBldTemplate
3435+
innerBld.ctx = ctx
34363436
innerBld.factory = ef
34373437
innerBld.addBuiltWithExpr(rec.WithID, initialCols, bufferRef)
34383438
// TODO(mgartner): I think colOrdsAlloc can be reused for each recursive

pkg/sql/opt/exec/factory.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -285,7 +285,7 @@ type KVOption struct {
285285
// RecursiveCTEIterationFn creates a plan for an iteration of WITH RECURSIVE,
286286
// given the result of the last iteration (as a node created by
287287
// ConstructBuffer).
288-
type RecursiveCTEIterationFn func(ef Factory, bufferRef Node) (Plan, error)
288+
type RecursiveCTEIterationFn func(ctx context.Context, ef Factory, bufferRef Node) (Plan, error)
289289

290290
// ApplyJoinPlanRightSideFn creates a plan for an iteration of ApplyJoin, given
291291
// a row produced from the left side. The plan is guaranteed to produce the

pkg/sql/recursive_cte.go

Lines changed: 10 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -135,17 +135,21 @@ func (n *recursiveCTENode) Next(params runParams) (bool, error) {
135135
rows: lastWorkingRows,
136136
label: n.label,
137137
}
138-
newPlan, err := n.genIterationFn(newExecFactory(params.ctx, params.p), buf)
139-
if err != nil {
140-
return false, err
141-
}
142138

139+
// Create a separate tracing span that will be used when planning and
140+
// running this iteration. Note that we'll still use the "outer" params.ctx
141+
// when accessing rows in the container.
143142
n.iterationCount++
144143
opName := "recursive-cte-iteration-" + strconv.Itoa(n.iterationCount)
145-
ctx, sp := tracing.ChildSpan(params.ctx, opName)
144+
planAndRunCtx, sp := tracing.ChildSpan(params.ctx, opName)
146145
defer sp.Finish()
146+
147+
newPlan, err := n.genIterationFn(planAndRunCtx, newExecFactory(planAndRunCtx, params.p), buf)
148+
if err != nil {
149+
return false, err
150+
}
147151
if err := runPlanInsidePlan(
148-
ctx, params, newPlan.(*planComponents), rowResultWriter(n),
152+
planAndRunCtx, params, newPlan.(*planComponents), rowResultWriter(n),
149153
nil /* deferredRoutineSender */, "", /* stmtForDistSQLDiagram */
150154
); err != nil {
151155
return false, err

0 commit comments

Comments
 (0)