Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
80848: sql: add SHOW COMMIT TIMESTAMP to retrieve a causality token r=ajwerner a=ajwerner

Fixes #79591

Relates to #7945

Release note (sql change): A new sql statement `SHOW COMMIT TIMESTAMP` has been
added. This statement can be used to retrieve the commit timestamp of the
current explicit transaction, current multi-statement implicit transaction, or
previous transaction. The statement may be used in a variety of settings to
maximize its utility in the face of connection pooling.

When used as a part of an explicit transaction, the statement implicitly
commits the transaction internally before being able to return a causality
token. This is similar to the `RELEASE cockroach_restart` behavior; after
issuing this statement, commands to the transaction will be rejected until
`COMMIT` is issued.

When used as part of a multi-statement implicit transaction, the statement
must be the final statement. If it occurs in the middle of a multi-statement
implicit transaction, it will be rejected with an error.

When sent as a stand-alone single-statement implicit transaction, it will
return the commit timestamp of the previously committed transaction. If there
was no transaction on the connection, or the previous transaction did not
commit (either rolled back or encountered an error), the command will fail with
an error code 25000 (`InvalidTransactionState`).

The `SHOW COMMIT TIMESTAMP` statement is idempotent; it can be issued
multiple times both inside of and after transactions and will return the same result.
One rough edge is that the `cockroach sql` cli command will, by default, send
statements on behalf of the user which will lead to repeated issuances of 
`SHOW COMMIT TIMESTAMP` from the CLI returning different values. If one
disables syntax checking with `\set check_syntax=false` and one changes their
prompt1 to not require a query, perhaps with `\set prompt1=%n@%M>;`, the
command will become idempotent from the CLI.

91832: CODEOWNERS: updater importer to sql-exp r=rafiss a=dt

Release note: none.
Epic: none.

92338: util/stop: finish task span before Stop() returns r=andreimatei a=andreimatei

Before this patch, a task's span was finished after the stopper considered the task to be finished. This was a problem for a test who wanted to assume that, once stopper.Stop() returns, all task spans are finished - which is a reasonable contract to expect. This patch reorders the span finish accordingly.

Fixes #83886

Release note: None
Epic: None

Co-authored-by: Andrew Werner <awerner32@gmail.com>
Co-authored-by: David Taylor <tinystatemachine@gmail.com>
Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
  • Loading branch information
4 people committed Nov 22, 2022
4 parents 1ff09bf + 2b86f13 + 10ea9e6 + efb93cb commit b12438a
Show file tree
Hide file tree
Showing 39 changed files with 1,068 additions and 79 deletions.
8 changes: 4 additions & 4 deletions .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,8 @@
/pkg/sql/pgwire/auth.go @cockroachdb/sql-experience @cockroachdb/server-prs @cockroachdb/prodsec
/pkg/sql/sem/builtins/ @cockroachdb/sql-experience
/pkg/sql/vtable/ @cockroachdb/sql-experience
/pkg/sql/importer/ @cockroachdb/sql-experience
/pkg/ccl/importerccl/ @cockroachdb/sql-experience

/pkg/sql/sessiondata/ @cockroachdb/sql-experience
/pkg/sql/tests/rsg_test.go @cockroachdb/sql-experience
Expand Down Expand Up @@ -100,7 +102,7 @@
/pkg/cli/debug*.go @cockroachdb/kv-prs @cockroachdb/cli-prs
/pkg/cli/debug_job_trace*.go @cockroachdb/jobs-prs
/pkg/cli/doctor*.go @cockroachdb/sql-schema @cockroachdb/cli-prs
/pkg/cli/import_test.go @cockroachdb/disaster-recovery @cockroachdb/cli-prs
/pkg/cli/import_test.go @cockroachdb/sql-experience @cockroachdb/cli-prs
/pkg/cli/sql*.go @cockroachdb/sql-experience @cockroachdb/cli-prs
/pkg/cli/clisqlshell/ @cockroachdb/sql-experience @cockroachdb/cli-prs
/pkg/cli/clisqlclient/ @cockroachdb/sql-experience @cockroachdb/cli-prs
Expand Down Expand Up @@ -163,9 +165,6 @@

/pkg/ccl/backupccl/ @cockroachdb/disaster-recovery
/pkg/ccl/backupccl/*_job.go @cockroachdb/disaster-recovery @cockroachdb/jobs-prs
/pkg/sql/importer/ @cockroachdb/disaster-recovery
/pkg/ccl/importerccl/ @cockroachdb/disaster-recovery
/pkg/ccl/spanconfigccl/ @cockroachdb/kv-prs
/pkg/ccl/storageccl/ @cockroachdb/disaster-recovery
/pkg/ccl/cloudccl/ @cockroachdb/disaster-recovery
/pkg/cloud/ @cockroachdb/disaster-recovery
Expand All @@ -174,6 +173,7 @@
/pkg/geo/ @cockroachdb/geospatial

/pkg/kv/ @cockroachdb/kv-prs
/pkg/ccl/spanconfigccl/ @cockroachdb/kv-prs
/pkg/kv/kvclient/kvstreamer @cockroachdb/sql-queries

/pkg/ccl/storageccl/engineccl @cockroachdb/storage
Expand Down
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -210,6 +210,7 @@ FILES = [
"show_backup",
"show_cluster_setting",
"show_columns_stmt",
"show_commit_timestamp_stmt",
"show_constraints_stmt",
"show_create_stmt",
"show_create_schedules_stmt",
Expand Down
2 changes: 2 additions & 0 deletions docs/generated/sql/bnf/show_commit_timestamp_stmt.bnf
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
show_commit_timestamp_stmt ::=
'SHOW' 'COMMIT' 'TIMESTAMP'
4 changes: 4 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ stmt_without_legacy_transaction ::=
| fetch_cursor_stmt
| move_cursor_stmt
| unlisten_stmt
| show_commit_timestamp_stmt

legacy_transaction_stmt ::=
legacy_begin_stmt
Expand Down Expand Up @@ -160,6 +161,9 @@ unlisten_stmt ::=
'UNLISTEN' type_name
| 'UNLISTEN' '*'

show_commit_timestamp_stmt ::=
'SHOW' 'COMMIT' 'TIMESTAMP'

legacy_begin_stmt ::=
'BEGIN' opt_transaction begin_transaction

Expand Down
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/stmt_without_legacy_transaction.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -21,3 +21,4 @@ stmt_without_legacy_transaction ::=
| fetch_cursor_stmt
| move_cursor_stmt
| unlisten_stmt
| show_commit_timestamp_stmt
7 changes: 7 additions & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions pkg/gen/bnf.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -210,6 +210,7 @@ BNF_SRCS = [
"//docs/generated/sql/bnf:show_backup.bnf",
"//docs/generated/sql/bnf:show_cluster_setting.bnf",
"//docs/generated/sql/bnf:show_columns_stmt.bnf",
"//docs/generated/sql/bnf:show_commit_timestamp_stmt.bnf",
"//docs/generated/sql/bnf:show_constraints_stmt.bnf",
"//docs/generated/sql/bnf:show_create_external_connections_stmt.bnf",
"//docs/generated/sql/bnf:show_create_schedules_stmt.bnf",
Expand Down
1 change: 1 addition & 0 deletions pkg/gen/diagrams.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,7 @@ DIAGRAMS_SRCS = [
"//docs/generated/sql/bnf:show_backup.html",
"//docs/generated/sql/bnf:show_cluster_setting.html",
"//docs/generated/sql/bnf:show_columns.html",
"//docs/generated/sql/bnf:show_commit_timestamp.html",
"//docs/generated/sql/bnf:show_constraints.html",
"//docs/generated/sql/bnf:show_create.html",
"//docs/generated/sql/bnf:show_create_external_connections.html",
Expand Down
1 change: 1 addition & 0 deletions pkg/gen/docs.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,7 @@ DOCS_SRCS = [
"//docs/generated/sql/bnf:show_backup.bnf",
"//docs/generated/sql/bnf:show_cluster_setting.bnf",
"//docs/generated/sql/bnf:show_columns_stmt.bnf",
"//docs/generated/sql/bnf:show_commit_timestamp_stmt.bnf",
"//docs/generated/sql/bnf:show_constraints_stmt.bnf",
"//docs/generated/sql/bnf:show_create_external_connections_stmt.bnf",
"//docs/generated/sql/bnf:show_create_schedules_stmt.bnf",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ go_library(
"conn_executor_exec.go",
"conn_executor_prepare.go",
"conn_executor_savepoints.go",
"conn_executor_show_commit_timestamp.go",
"conn_fsm.go",
"conn_io.go",
"constraint.go",
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/colinfo/result_columns.go
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,11 @@ var ExplainPlanColumns = ResultColumns{
{Name: "info", Typ: types.String},
}

// ShowCommitTimestampColumns are the result columns of SHOW COMMIT TIMESTAMP.
var ShowCommitTimestampColumns = ResultColumns{
{Name: "commit_timestamp", Typ: types.Decimal},
}

// ShowTraceColumns are the result columns of a SHOW [KV] TRACE statement.
var ShowTraceColumns = ResultColumns{
{Name: "timestamp", Typ: types.TimestampTZ},
Expand Down
18 changes: 16 additions & 2 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1358,7 +1358,7 @@ type connExecutor struct {
txnFinishClosure struct {
// txnStartTime is the time that the transaction started.
txnStartTime time.Time
// implicit is whether or not the transaction was implicit.
// implicit is whether the transaction was implicit.
implicit bool
}

Expand Down Expand Up @@ -1414,6 +1414,13 @@ type connExecutor struct {
rowsWrittenLogged bool
rowsReadLogged bool

// shouldAcceptReleaseCockroachRestartInCommitWait is set to true if the
// transaction had SAVEPOINT cockroach_restart installed at the time that
// SHOW COMMIT TIMESTAMP was executed to commit the transaction. If so, the
// connExecutor will permit one invocation of RELEASE SAVEPOINT
// cockroach_restart while in the CommitWait state.
shouldAcceptReleaseCockroachRestartInCommitWait bool

// hasAdminRole is used to cache if the user running the transaction
// has admin privilege. hasAdminRoleCache is set for the first statement
// in a transaction.
Expand Down Expand Up @@ -1542,6 +1549,11 @@ type connExecutor struct {
// totalActiveTimeStopWatch tracks the total active time of the session.
// This is defined as the time spent executing transactions and statements.
totalActiveTimeStopWatch *timeutil.StopWatch

// previousTransactionCommitTimestamp is the timestamp of the previous
// transaction which committed. It is zero-valued when there is a transaction
// open or the previous transaction did not successfully commit.
previousTransactionCommitTimestamp hlc.Timestamp
}

// ctxHolder contains a connection's context and, while session tracing is
Expand Down Expand Up @@ -1928,7 +1940,9 @@ func (ex *connExecutor) execCmd() error {
// The behavior is configurable, in case users want to preserve the
// behavior from v21.2 and earlier.
implicitTxnForBatch := ex.sessionData().EnableImplicitTransactionForBatchStatements
canAutoCommit := ex.implicitTxn() && (tcmd.LastInBatch || !implicitTxnForBatch)
canAutoCommit := ex.implicitTxn() &&
(tcmd.LastInBatchBeforeShowCommitTimestamp ||
tcmd.LastInBatch || !implicitTxnForBatch)
ev, payload, err = ex.execStmt(
ctx, tcmd.Statement, nil /* prepared */, nil /* pinfo */, stmtRes, canAutoCommit,
)
Expand Down
35 changes: 27 additions & 8 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ func (ex *connExecutor) execStmt(
// Note: when not using explicit transactions, we go through this transition
// for every statement. It is important to minimize the amount of work and
// allocations performed up to this point.
ev, payload = ex.execStmtInNoTxnState(ctx, ast)
ev, payload = ex.execStmtInNoTxnState(ctx, ast, res)

case stateOpen:
err = ex.execWithProfiling(ctx, ast, prepared, func(ctx context.Context) error {
Expand Down Expand Up @@ -590,6 +590,10 @@ func (ex *connExecutor) execStmtInOpenState(
ev, payload := ex.execRollbackToSavepointInOpenState(ctx, s, res)
return ev, payload, nil

case *tree.ShowCommitTimestamp:
ev, payload := ex.execShowCommitTimestampInOpenState(ctx, s, res, canAutoCommit)
return ev, payload, nil

case *tree.Prepare:
// This is handling the SQL statement "PREPARE". See execPrepare for
// handling of the protocol-level command for preparing statements.
Expand Down Expand Up @@ -777,6 +781,9 @@ func (ex *connExecutor) execStmtInOpenState(
// the timestamps of the transaction accordingly.
func (ex *connExecutor) handleAOST(ctx context.Context, stmt tree.Statement) error {
if _, isNoTxn := ex.machine.CurState().(stateNoTxn); isNoTxn {
if _, ok := stmt.(*tree.ShowCommitTimestamp); ok {
return nil
}
return errors.AssertionFailedf(
"cannot handle AOST clause without a transaction",
)
Expand Down Expand Up @@ -1670,7 +1677,7 @@ var eventStartExplicitTxn fsm.Event = eventTxnStart{ImplicitTxn: fsm.False}
// the cursor is not advanced. This means that the statement will run again in
// stateOpen, at each point its results will also be flushed.
func (ex *connExecutor) execStmtInNoTxnState(
ctx context.Context, ast tree.Statement,
ctx context.Context, ast tree.Statement, res RestrictedCommandResult,
) (_ fsm.Event, payload fsm.EventPayload) {
switch s := ast.(type) {
case *tree.BeginTransaction:
Expand All @@ -1693,6 +1700,8 @@ func (ex *connExecutor) execStmtInNoTxnState(
historicalTs,
ex.transitionCtx,
ex.QualityOfService())
case *tree.ShowCommitTimestamp:
return ex.execShowCommitTimestampInNoTxnState(ctx, s, res)
case *tree.CommitTransaction, *tree.ReleaseSavepoint,
*tree.RollbackTransaction, *tree.SetTransaction, *tree.Savepoint:
return ex.makeErrEvent(errNoTransactionInProgress, ast)
Expand Down Expand Up @@ -1815,7 +1824,15 @@ func (ex *connExecutor) execStmtInCommitWaitState(
ex.incrementExecutedStmtCounter(ast)
}
}()
switch ast.(type) {
switch s := ast.(type) {
case *tree.ReleaseSavepoint:
if ex.extraTxnState.shouldAcceptReleaseCockroachRestartInCommitWait &&
s.Savepoint == commitOnReleaseSavepointName {
ex.extraTxnState.shouldAcceptReleaseCockroachRestartInCommitWait = false
return nil, nil
}
case *tree.ShowCommitTimestamp:
return ex.execShowCommitTimestampInCommitWaitState(ctx, s, res)
case *tree.CommitTransaction, *tree.RollbackTransaction:
// Reply to a rollback with the COMMIT tag, by analogy to what we do when we
// get a COMMIT in state Aborted.
Expand All @@ -1828,13 +1845,11 @@ func (ex *connExecutor) execStmtInCommitWaitState(
return nil
},
)
default:
ev = eventNonRetriableErr{IsCommit: fsm.False}
payload = eventNonRetriableErrPayload{
}
return eventNonRetriableErr{IsCommit: fsm.False},
eventNonRetriableErrPayload{
err: sqlerrors.NewTransactionCommittedError(),
}
return ev, payload
}
}

// runObserverStatement executes the given observer statement.
Expand Down Expand Up @@ -2257,6 +2272,8 @@ func (ex *connExecutor) onTxnFinish(ctx context.Context, ev txnEvent) {
}
ex.server.ServerMetrics.StatsMetrics.DiscardedStatsCount.Inc(1)
}
// If we have a commitTimestamp, we should use it.
ex.previousTransactionCommitTimestamp.Forward(ev.commitTimestamp)
}
}

Expand Down Expand Up @@ -2320,6 +2337,8 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) {
ex.extraTxnState.shouldExecuteOnTxnRestart = true

ex.statsCollector.StartTransaction()

ex.previousTransactionCommitTimestamp = hlc.Timestamp{}
}

func (ex *connExecutor) recordTransactionFinish(
Expand Down
15 changes: 13 additions & 2 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,13 @@ func (ex *connExecutor) execPrepare(
// descriptors for type checking. This implicit txn will be open until
// the Sync message is handled.
if _, isNoTxn := ex.machine.CurState().(stateNoTxn); isNoTxn {
return ex.beginImplicitTxn(ctx, parseCmd.AST)
// The one exception is that we must not open a new transaction when
// preparing SHOW COMMIT TIMESTAMP. If we did, it would destroy the
// information about the previous transaction. We expect to execute
// this command in NoTxn.
if _, ok := parseCmd.AST.(*tree.ShowCommitTimestamp); !ok {
return ex.beginImplicitTxn(ctx, parseCmd.AST)
}
} else if _, isAbortedTxn := ex.machine.CurState().(stateAborted); isAbortedTxn {
if !ex.isAllowedInAbortedTxn(parseCmd.AST) {
return retErr(sqlerrors.NewTransactionAbortedError("" /* customMsg */))
Expand Down Expand Up @@ -315,7 +321,12 @@ func (ex *connExecutor) execBind(
// SQL EXECUTE command (which also needs to bind and resolve types) is
// handled separately in conn_executor_exec.
if _, isNoTxn := ex.machine.CurState().(stateNoTxn); isNoTxn {
return ex.beginImplicitTxn(ctx, ps.AST)
// The one critical exception is that we must not open a transaction when
// executing SHOW COMMIT TIMESTAMP as it would destroy the information
// about the previously committed transaction.
if _, ok := ps.AST.(*tree.ShowCommitTimestamp); !ok {
return ex.beginImplicitTxn(ctx, ps.AST)
}
} else if _, isAbortedTxn := ex.machine.CurState().(stateAborted); isAbortedTxn {
if !ex.isAllowedInAbortedTxn(ps.AST) {
return retErr(sqlerrors.NewTransactionAbortedError("" /* customMsg */))
Expand Down
Loading

0 comments on commit b12438a

Please sign in to comment.