Skip to content

Commit

Permalink
sql: remove the artifact of canceling the txn-scoped context
Browse files Browse the repository at this point in the history
This commit removes an old artifact of having a txn-scoped context
cancellation that is performed when finishing the txn. As Andrei points
out, this txn-scoped cancellation is likely a leftover from ancient
times and is no longer needed. In particular, this also fixes the bug of
using the span after it was finished (which would occur with high
vmodule on `context.go` file).

Release note: None
  • Loading branch information
yuzefovich committed Jul 19, 2022
1 parent 7191d81 commit cf69a32
Show file tree
Hide file tree
Showing 4 changed files with 16 additions and 29 deletions.
10 changes: 7 additions & 3 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/cancelchecker"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/fsm"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -278,6 +279,9 @@ func (ex *connExecutor) execStmtInOpenState(
ast := parserStmt.AST
ctx = withStatement(ctx, ast)

var cancelQuery context.CancelFunc
ctx, cancelQuery = contextutil.WithCancel(ctx)

makeErrEvent := func(err error) (fsm.Event, fsm.EventPayload, error) {
ev, payload := ex.makeErrEvent(err, ast)
return ev, payload, nil
Expand Down Expand Up @@ -323,7 +327,7 @@ func (ex *connExecutor) execStmtInOpenState(
ex.planner.EvalContext().Placeholders = pinfo
}

ex.addActiveQuery(ast, formatWithPlaceholders(ast, ex.planner.EvalContext()), queryID, ex.state.cancel)
ex.addActiveQuery(ast, formatWithPlaceholders(ast, ex.planner.EvalContext()), queryID, cancelQuery)
if ex.executorType != executorTypeInternal {
ex.metrics.EngineMetrics.SQLActiveStatements.Inc(1)
}
Expand Down Expand Up @@ -2025,7 +2029,7 @@ func (ex *connExecutor) enableTracing(modes []string) error {

// addActiveQuery adds a running query to the list of running queries.
func (ex *connExecutor) addActiveQuery(
ast tree.Statement, rawStmt string, queryID clusterunique.ID, cancelFun context.CancelFunc,
ast tree.Statement, rawStmt string, queryID clusterunique.ID, cancelQuery context.CancelFunc,
) {
_, hidden := ast.(tree.HiddenFromShowQueries)
qm := &queryMeta{
Expand All @@ -2035,7 +2039,7 @@ func (ex *connExecutor) addActiveQuery(
phase: preparing,
isDistributed: false,
isFullScan: false,
ctxCancel: cancelFun,
cancelQuery: cancelQuery,
hidden: hidden,
}
ex.mu.Lock()
Expand Down
11 changes: 6 additions & 5 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1893,8 +1893,9 @@ type queryMeta struct {
// Current phase of execution of query.
phase queryPhase

// Cancellation function for the context associated with this query's transaction.
ctxCancel context.CancelFunc
// Cancellation function for the context associated with this query's
// statement.
cancelQuery context.CancelFunc

// If set, this query will not be reported as part of SHOW QUERIES. This is
// set based on the statement implementing tree.HiddenFromShowQueries.
Expand All @@ -1903,10 +1904,10 @@ type queryMeta struct {
progressAtomic uint64
}

// cancel cancels the query associated with this queryMeta, by closing the associated
// txn context.
// cancel cancels the query associated with this queryMeta, by closing the
// associated stmt context.
func (q *queryMeta) cancel() {
q.ctxCancel()
q.cancelQuery()
}

// getStatement returns a cleaned version of the query associated
Expand Down
22 changes: 3 additions & 19 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
Expand Down Expand Up @@ -90,11 +89,6 @@ type txnState struct {
recordingThreshold time.Duration
recordingStart time.Time

// cancel is Ctx's cancellation function. Called upon COMMIT/ROLLBACK of the
// transaction to release resources associated with the context. nil when no
// txn is in progress.
cancel context.CancelFunc

// The timestamp to report for current_timestamp(), now() etc.
// This must be constant for the lifetime of a SQL transaction.
sqlTimestamp time.Time
Expand Down Expand Up @@ -190,16 +184,15 @@ func (ts *txnState) resetForNewSQLTxn(
opName := sqlTxnName
alreadyRecording := tranCtx.sessionTracing.Enabled()

var txnCtx context.Context
var sp *tracing.Span
duration := traceTxnThreshold.Get(&tranCtx.settings.SV)
if alreadyRecording || duration > 0 {
txnCtx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName,
ts.Ctx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName,
tracing.WithRecording(tracingpb.RecordingVerbose))
} else if ts.testingForceRealTracingSpans {
txnCtx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName, tracing.WithForceRealSpan())
ts.Ctx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName, tracing.WithForceRealSpan())
} else {
txnCtx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName)
ts.Ctx, sp = tracing.EnsureChildSpan(connCtx, tranCtx.tracer, opName)
}
if txnType == implicitTxn {
sp.SetTag("implicit", attribute.StringValue("true"))
Expand All @@ -210,7 +203,6 @@ func (ts *txnState) resetForNewSQLTxn(
ts.recordingStart = timeutil.Now()
}

ts.Ctx, ts.cancel = contextutil.WithCancel(txnCtx)
ts.mon.StartNoReserved(ts.Ctx, tranCtx.connMon)
txnID = func() (txnID uuid.UUID) {
ts.mu.Lock()
Expand Down Expand Up @@ -255,10 +247,6 @@ func (ts *txnState) resetForNewSQLTxn(
// returned.
func (ts *txnState) finishSQLTxn() (txnID uuid.UUID) {
ts.mon.Stop(ts.Ctx)
if ts.cancel != nil {
ts.cancel()
ts.cancel = nil
}
sp := tracing.SpanFromContext(ts.Ctx)
if sp == nil {
panic(errors.AssertionFailedf("No span in context? Was resetForNewSQLTxn() called previously?"))
Expand Down Expand Up @@ -292,10 +280,6 @@ func (ts *txnState) finishExternalTxn() {
} else {
ts.mon.Stop(ts.Ctx)
}
if ts.cancel != nil {
ts.cancel()
ts.cancel = nil
}

if ts.Ctx != nil {
if sp := tracing.SpanFromContext(ts.Ctx); sp != nil {
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/txn_state_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,6 @@ func makeTestContext(stopper *stop.Stopper) testContext {
func (tc *testContext) createOpenState(typ txnType) (fsm.State, *txnState) {
sp := tc.tracer.StartSpan("createOpenState")
ctx := tracing.ContextWithSpan(tc.ctx, sp)
ctx, cancel := context.WithCancel(ctx)

txnStateMon := mon.NewMonitor("test mon",
mon.MemoryResource,
Expand All @@ -97,7 +96,6 @@ func (tc *testContext) createOpenState(typ txnType) (fsm.State, *txnState) {
ts := txnState{
Ctx: ctx,
connCtx: tc.ctx,
cancel: cancel,
sqlTimestamp: timeutil.Now(),
priority: roachpb.NormalUserPriority,
mon: txnStateMon,
Expand Down

0 comments on commit cf69a32

Please sign in to comment.