diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 3a054d83a418..1571ad899ccf 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -3001,6 +3001,7 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)
crdb_internal.cluster_setting_encoded_default(setting: string) → string
Returns the encoded default value of the given cluster setting.
crdb_internal.commit_with_causality_token() → decimal
crdb_internal.completed_migrations() → string[]
This function is used only by CockroachDB’s developers for testing purposes.
crdb_internal.create_join_token() → string
Creates a join token for use when adding a new node to a secure cluster.
diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 0ca73310f421..c0c244b4a149 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "comment_on_table.go", "compact_sql_stats.go", "conn_executor.go", + "conn_executor_commit_with_causality_token.go", "conn_executor_exec.go", "conn_executor_prepare.go", "conn_executor_savepoints.go", @@ -494,6 +495,7 @@ go_test( "comment_on_database_test.go", "comment_on_index_test.go", "comment_on_table_test.go", + "commit_with_causality_token_test.go", "conn_executor_internal_test.go", "conn_executor_savepoints_test.go", "conn_executor_test.go", diff --git a/pkg/sql/commit_with_causality_token_test.go b/pkg/sql/commit_with_causality_token_test.go new file mode 100644 index 000000000000..b7999626bb53 --- /dev/null +++ b/pkg/sql/commit_with_causality_token_test.go @@ -0,0 +1,67 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +// TestIsSelectWithCausalityToken tests whether given sql statements match the +// criteria for isSelectWithCausalityToken. +func TestIsSelectWithCausalityToken(t *testing.T) { + defer leaktest.AfterTest(t)() + + for _, tc := range []struct { + in string + exp bool + }{ + {"Select crdb_internal.commit_with_causality_token()", true}, + {"select crdb_inteRnal.comMit_wiTh_cauSality_toKen()", true}, + {`select "crdb_internal".comMit_wiTh_cauSality_toKen()`, true}, + {"select crdb_inteRnal.comMit_wiTh_cauSality_toKen(), 1", false}, + {"select crdb_internal.commit_with_causality_token() from crdb_internal.ranges_no_leases", false}, + {"select crdb_internal.commit_with_causality_token() from generate_series(1, 100)", false}, + {`select distinct "crdb_internal".comMit_wiTh_cauSality_toKen()`, false}, + {`select "crdb_inteRnal".comMit_wiTh_cauSality_toKen()`, false}, + {"(select crdb_internal.commit_with_causality_token())", false}, + {"(select crdb_inteRnal.comMit_wiTh_cauSality_toKen())", false}, + {`(select "crdb_internal".comMit_wiTh_cauSality_toKen())`, false}, + {`(select "crdb_inteRnal".comMit_wiTh_cauSality_toKen())`, false}, + {`((select "crdb_internal".comMit_wiTh_cauSality_toKen()))`, false}, + {`SELECT ((select "crdb_internal".comMit_wiTh_cauSality_toKen()))`, false}, + {"SELECT crdb_internal.commit_with_causality_token() FOR UPDATE ", false}, + {"Select crdb_internal.commit_with_causality_token", false}, + {"with a as (select 1) select crdb_internal.commit_with_causality_token()", false}, + {"(select crdb_internal.commit_with_causality_token() limit 0)", false}, + {"(select crdb_internal.commit_with_causality_token()) limit 0", false}, + {"select crdb_internal.commit_with_causality_token() limit 0", false}, + {"(select crdb_internal.commit_with_causality_token() where true)", false}, + {"select crdb_internal.commit_with_causality_token() where true", false}, + {"select crdb_internal.commit_with_causality_token() having true", false}, + {"select crdb_internal.commit_with_causality_token() order by 1", false}, + {"(select crdb_internal.commit_with_causality_token()) order by 1", false}, + {"(select crdb_internal.commit_with_causality_token() order by 1)", false}, + } { + t.Run(tc.in, func(t *testing.T) { + stmts, err := parser.Parse(tc.in) + require.NoError(t, err) + require.Len(t, stmts, 1) + require.Equalf( + t, tc.exp, isSelectCommitWithCausalityToken(stmts[0].AST), + "%s", stmts[0].SQL, + ) + }) + } +} diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index ec35a584cd3b..a2de5e266d10 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -1383,6 +1383,12 @@ type connExecutor struct { rowsWrittenLogged bool rowsReadLogged bool + // shouldAcceptReleaseSavepointCockroachRestart is set to true + // when entering the commitOrReleaseWaitState. If set to true, the + // commitOrReleaseWaitState will allow one instance of RELEASE SAVEPOINT + // cockroach_restart. + shouldAcceptReleaseSavepointCockroachRestart 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. diff --git a/pkg/sql/conn_executor_commit_with_causality_token.go b/pkg/sql/conn_executor_commit_with_causality_token.go new file mode 100644 index 000000000000..6ab2c812afe6 --- /dev/null +++ b/pkg/sql/conn_executor_commit_with_causality_token.go @@ -0,0 +1,114 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/fsm" +) + +// handleCommitWithCausalityToken deals with the special statement +// SELECT commit_with_causality_token while in the OpenState of an +// explicit transaction. +func (ex *connExecutor) handleCommitWithCausalityToken( + ctx context.Context, res RestrictedCommandResult, +) (fsm.Event, fsm.EventPayload, error) { + res.ResetStmtType((*tree.CommitTransaction)(nil)) + err := ex.commitSQLTransactionInternal(ctx) + if err == nil { + + res.SetColumns(ctx, colinfo.ResultColumns{ + { + Name: "causality_token", + Typ: types.Decimal, + Hidden: false, + }, + }) + if err := res.AddRow(ctx, tree.Datums{ + eval.TimestampToDecimalDatum(ex.planner.Txn().CommitTimestamp()), + }); err != nil { + return nil, nil, err + } + + // If we have a SAVEPOINT cockroach_restart, then we need to note that + // fact now, as the SAVEPOINT stack will be destroyed as the state + // machine moves into COMMIT. This state in extraTxnState will be cleaned + // up as we process any statement in CommitWait. + if entry, _ := ex.extraTxnState.savepoints.find( + commitOnReleaseSavepointName, + ); entry != nil && entry.commitOnRelease { + ex.extraTxnState.shouldAcceptReleaseSavepointCockroachRestart = true + } + + return eventTxnCommittedWithCausalityToken{}, nil, nil + } + + // Committing the transaction failed. We'll go to state RestartWait if + // it's a retriable error, or to state RollbackWait otherwise. + if errIsRetriable(err) { + rc, canAutoRetry := ex.getRewindTxnCapability() + ev := eventRetriableErr{ + IsCommit: fsm.FromBool(false /* isCommit */), + CanAutoRetry: fsm.FromBool(canAutoRetry), + } + payload := eventRetriableErrPayload{err: err, rewCap: rc} + return ev, payload, nil + } + + ev := eventNonRetriableErr{IsCommit: fsm.FromBool(false)} + payload := eventNonRetriableErrPayload{err: err} + return ev, payload, nil +} + +// isSelectWithCausalityToken returns true if the statement is exactly the +// following, modulo capitalization: +// +// SELECT crdb_internal.commit_with_causality_token +// +func isSelectCommitWithCausalityToken(ast tree.Statement) bool { + sel, ok := ast.(*tree.Select) + if !ok { + return false + } + selStmt := sel.Select + if sel.With != nil || sel.Locking != nil || sel.Limit != nil || sel.OrderBy != nil { + return false + } + // We intentionally don't open up ParenSelect clauses. + sc, ok := selStmt.(*tree.SelectClause) + if !ok { + return false + } + // TODO(ajwerner): Find a more exhaustive way to do this. + if len(sc.From.Tables) != 0 || len(sc.Exprs) != 1 || sc.Distinct || + sc.Where != nil || sc.GroupBy != nil || sc.Having != nil || sc.Window != nil || + sc.From.AsOf.Expr != nil { + return false + } + funcExpr, isFuncExpr := sc.Exprs[0].Expr.(*tree.FuncExpr) + if !isFuncExpr || len(funcExpr.Exprs) != 0 { + return false + } + name, isName := funcExpr.Func.FunctionReference.(*tree.UnresolvedName) + if !isName || name.NumParts != 2 || + name.Parts[1] != catconstants.CRDBInternalSchemaName || + name.Parts[0] != builtins.CommitWithCausalityTokenName { + return false + } + return true +} diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 1f3790e110cc..c6bb7c29b6bc 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -603,6 +603,11 @@ func (ex *connExecutor) execStmtInOpenState( p.semaCtx.Annotations = tree.MakeAnnotations(stmt.NumAnnotations) + // Handle the special SELECT crdb_internal.commit_with_causality_token. + if !p.extendedEvalCtx.TxnImplicit && isSelectCommitWithCausalityToken(ast) { + return ex.handleCommitWithCausalityToken(ctx, res) + } + // For regular statements (the ones that get to this point), we // don't return any event unless an error happens. @@ -1691,6 +1696,14 @@ func (ex *connExecutor) execStmtInCommitWaitState( ex.incrementExecutedStmtCounter(ast) } }() + defer func() { + ex.extraTxnState.shouldAcceptReleaseSavepointCockroachRestart = false + }() + if s, ok := ast.(*tree.ReleaseSavepoint); ok && + s.Savepoint == commitOnReleaseSavepointName && + ex.extraTxnState.shouldAcceptReleaseSavepointCockroachRestart { + return nil, nil + } switch ast.(type) { case *tree.CommitTransaction, *tree.RollbackTransaction: // Reply to a rollback with the COMMIT tag, by analogy to what we do when we diff --git a/pkg/sql/conn_fsm.go b/pkg/sql/conn_fsm.go index cbfbeb21e6a2..86e1c1c9a3dc 100644 --- a/pkg/sql/conn_fsm.go +++ b/pkg/sql/conn_fsm.go @@ -193,20 +193,26 @@ type eventTxnRestart struct{} // generated by releasing regular savepoints. type eventTxnReleased struct{} +// eventTxnCommittedWithCausalityToken is generated after a successful +// SELECT crdb_internal.commit_with_causality_token(). It moves the state +// to CommitWait. +type eventTxnCommittedWithCausalityToken struct{} + // payloadWithError is a common interface for the payloads that wrap an error. type payloadWithError interface { errorCause() error } -func (eventTxnStart) Event() {} -func (eventTxnFinishCommitted) Event() {} -func (eventTxnFinishAborted) Event() {} -func (eventSavepointRollback) Event() {} -func (eventNonRetriableErr) Event() {} -func (eventRetriableErr) Event() {} -func (eventTxnRestart) Event() {} -func (eventTxnReleased) Event() {} -func (eventTxnUpgradeToExplicit) Event() {} +func (eventTxnStart) Event() {} +func (eventTxnFinishCommitted) Event() {} +func (eventTxnFinishAborted) Event() {} +func (eventSavepointRollback) Event() {} +func (eventNonRetriableErr) Event() {} +func (eventRetriableErr) Event() {} +func (eventTxnRestart) Event() {} +func (eventTxnReleased) Event() {} +func (eventTxnCommittedWithCausalityToken) Event() {} +func (eventTxnUpgradeToExplicit) Event() {} // TxnStateTransitions describe the transitions used by a connExecutor's // fsm.Machine. Args.Extended is a txnState, which is muted by the Actions. @@ -337,6 +343,22 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ return nil }, }, + eventTxnCommittedWithCausalityToken{}: { + Description: "SELECT crdb_internal.commit_with_causality_token()", + Next: stateCommitWait{}, + Action: func(args fsm.Args) error { + ts := args.Extended.(*txnState) + ts.mu.Lock() + txnID := ts.mu.txn.ID() + ts.mu.Unlock() + ts.setAdvanceInfo( + advanceOne, + noRewind, + txnEvent{eventType: txnCommit, txnID: txnID}, + ) + return nil + }, + }, eventTxnReleased{}: { Description: "RELEASE SAVEPOINT cockroach_restart", Next: stateCommitWait{}, diff --git a/pkg/sql/logictest/testdata/logic_test/commit_with_causality_token b/pkg/sql/logictest/testdata/logic_test/commit_with_causality_token new file mode 100644 index 000000000000..47182cb092a4 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/commit_with_causality_token @@ -0,0 +1,214 @@ +user testuser + +statement error pgcode 25000 cannot execute commit_with_causality_token outside of an explicit transaction in the open state +select crdb_internal.commit_with_causality_token() + +# Test that calling the statement during execution leads to an error. +subtest calling_during_execution + +statement ok +create table bar(i int primary key) + +statement ok +begin + +# If you don't call the function, you don't get an error. Maybe this is a +# bad design decision and we should fail to resolve the function. +statement ok +select crdb_internal.commit_with_causality_token() from bar + +statement ok +commit + +statement ok +begin; +insert into bar values (1) + +statement error pgcode 25000 cannot execute commit_with_causality_token outside of an explicit transaction in the open state +select crdb_internal.commit_with_causality_token() from bar + +statement ok +rollback + +statement ok +drop table bar + + +statement ok +begin; + +let $tok +select crdb_internal.commit_with_causality_token() + +statement ok +commit + +query B +select $tok < cluster_logical_timestamp() +---- +true + +subtest basic + +statement ok +begin; +savepoint cockroach_restart; + +let $tok +select crdb_internal.commit_with_causality_token() + +statement ok +release savepoint cockroach_restart + +statement ok +commit + + +subtest perform_some_dmls + +statement ok +create table foo (i int primary key); + +statement ok +begin; +savepoint cockroach_restart; +insert into foo values(1) + +query I +select * from foo +---- +1 + +statement ok +select crdb_internal.commit_with_causality_token() + +statement ok +commit + +query I +select * from foo +---- +1 + +# Do the same, but this time, release the savepoint. + +statement ok +begin; +savepoint cockroach_restart; +insert into foo values(2) + +query I +select * from foo +---- +1 +2 + +statement ok +select crdb_internal.commit_with_causality_token() + +statement ok +release savepoint cockroach_restart; +commit + +query I +select * from foo +---- +1 +2 + +subtest perform_some_ddls + +statement ok +begin; +savepoint cockroach_restart; +create table t (i int primary key); +drop table foo; + +let $tok +select crdb_internal.commit_with_causality_token() + +statement ok +commit + + +# Above we tested that we can issue RELEASE SAVEPOINT cockroach_restart +# after crdb_internal.commit_with_causality_token() if we had set it up. +# Now we test that it is not allowed if it had not been added. +subtest disallow_cockroach_restart_without_savepoint + +statement ok +begin; +select crdb_internal.commit_with_causality_token(); + +statement error pgcode 25000 current transaction is committed, commands ignored until end of transaction block +release savepoint cockroach_restart; + +statement ok +rollback + +# Test that the function can be used with wacky capitalization and +# with quotes. +subtest capitalization + +statement ok +begin; +savepoint cockroach_restart; +select cRdB_Internal.Commit_With_causality_token(); +commit; + +statement ok +begin; +savepoint cockroach_restart; +select cRdB_Internal."commit_with_causality_token"(); +commit; + +# Test that the causality token is the same timestamp as ends up on rows, +# even if the transaction gets pushed. +subtest causality_token_equals_mvcc_timestamp + +statement ok +create table foo (i int primary key); + +statement ok +begin; + insert into foo values (1), (3); + +let $ts1 +select crdb_internal.commit_with_causality_token() + +statement ok +commit + +statement ok +begin; + insert into foo values (2), (4); + +user root + +statement ok +begin priority high; select * from foo; commit; + +user testuser + +let $ts2 +select crdb_internal.commit_with_causality_token() + +statement ok +commit + +query IT + SELECT i, + CASE + WHEN ts = $ts1 THEN 'ts1' + WHEN ts = $ts2 THEN 'ts2' + END + FROM (SELECT i, crdb_internal_mvcc_timestamp AS ts FROM foo) +ORDER BY i ASC; +---- +1 ts1 +2 ts2 +3 ts1 +4 ts2 + +statement ok +drop table foo diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index dc85e1006650..22d15f3942d6 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -63,6 +63,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" "github.com/cockroachdb/cockroach/pkg/sql/sem/asof" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" @@ -240,6 +241,12 @@ func mustBeDIntInTenantRange(e tree.Expr) (tree.DInt, error) { return tenID, nil } +// CommitWithCausalityTokenName is the name of the special builtin function +// to commit a transaction and return the logical timestamp at which it +// committed. The implementation of this function here is a shim; the function +// is processed explicitly by the connExecutor. +const CommitWithCausalityTokenName = "commit_with_causality_token" + // builtins contains the built-in functions indexed by name. // // For use in other packages, see AllBuiltinNames and GetBuiltinProperties(). @@ -2761,6 +2768,28 @@ nearest replica.`, defaultFollowerReadDuration), }, ), + // CommitWithCausalityToken is a special sentinel builtin which returns an + // error when invoked directly. The connExecutor state machine detects + // invocations of this function as the sole expression in a simple select. + // When this happens, during an explicit transaction, the transaction is + // committed, the side-effects of that commit, if there are any, are run, + // and the timestamp at which the commit ocurred is returned. + catconstants.CRDBInternalSchemaName + "." + CommitWithCausalityTokenName: makeBuiltin( + tree.FunctionProperties{ + Category: categorySystemInfo, + }, + tree.Overload{ + Types: tree.ArgTypes{}, + Volatility: volatility.Volatile, + ReturnType: tree.FixedReturnType(types.Decimal), + Fn: func(*eval.Context, tree.Datums) (tree.Datum, error) { + return nil, pgerror.Newf(pgcode.InvalidTransactionState, + "cannot execute "+CommitWithCausalityTokenName+" outside of an "+ + "explicit transaction in the open state") + }, + }, + ), + "cluster_logical_timestamp": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, diff --git a/pkg/sql/txnstatetransitions_diagram.gv b/pkg/sql/txnstatetransitions_diagram.gv index 64e31bc7d849..d65241287232 100644 --- a/pkg/sql/txnstatetransitions_diagram.gv +++ b/pkg/sql/txnstatetransitions_diagram.gv @@ -35,6 +35,7 @@ digraph finite_state_machine { "Open{ImplicitTxn:false}" -> "NoTxn{}" [label =