Skip to content

Commit

Permalink
sql: Allow restart savepoint to be renamed
Browse files Browse the repository at this point in the history
Currently, the only allowable savepoint names are those beginning with
`cockroach_restart` which triggers a Cockroach-specific transaction-retry flow.
This creates compatibility issues with certain integration scenarios (e.g. apps
using Spring ORM) that can take advantage of restartable transactions, but
which assume than an arbitrary savepoint identifier may be used.

This patch introduces a new session variable `force_savepoint_restart` which
allows the user to customize the savepoint name to any valid identifier, while
always triggering a restartable transaction.

Telemetry:
* Unimplemented-feature error now includes reference to #10735.
* `sql.force_savepoint_restart` event recorded when enabled.

Resolves #30588 (customize `SAVEPOINT` name)
Relates to #15012 (possible change for detecting "empty" transactions)

Release note (sql change): Users can customize the auto-retry savepoint name
used by the `SAVEPOINT` command by setting the `force_savepoint_restart`
session variable.  For example: `SET force_savepoint_restart=true; BEGIN;
SAVEPOINT foo` will now function as desired. This session variable may also be
supplied as part of a connection string to support existing code that assumes
that arbitrary savepoint names may be used.

Release note (sql change): The names supplied to a `SAVEPOINT` command are now
properly treated as SQL identifiers.  That is `SAVEPOINT foo` and `SAVEPOINT
FOO` are now equivalent statements.
  • Loading branch information
bobvawter committed Nov 12, 2018
1 parent b1c87fd commit a66d7f8
Show file tree
Hide file tree
Showing 16 changed files with 207 additions and 57 deletions.
70 changes: 61 additions & 9 deletions pkg/sql/conn_executor_exec.go
Expand Up @@ -41,9 +41,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/tracing"
)

// RestartSavepointName is the only savepoint ident that we accept.
const RestartSavepointName string = "cockroach_restart"

var errSavepointNotUsed = pgerror.NewErrorf(
pgerror.CodeSavepointExceptionError,
"savepoint %s has not been used", tree.RestartSavepointName)
"savepoint %s has not been used", RestartSavepointName)

// execStmt executes one statement by dispatching according to the current
// state. Returns an Event to be passed to the state machine, or nil if no
Expand Down Expand Up @@ -221,7 +224,7 @@ func (ex *connExecutor) execStmtInOpenState(
return ev, payload, nil

case *tree.ReleaseSavepoint:
if err := tree.ValidateRestartCheckpoint(s.Savepoint); err != nil {
if err := ex.validateSavepointName(s.Savepoint); err != nil {
return makeErrEvent(err)
}
if !ex.machine.CurState().(stateOpen).RetryIntent.Get() {
Expand All @@ -239,29 +242,40 @@ func (ex *connExecutor) execStmtInOpenState(
return ev, payload, nil

case *tree.Savepoint:
if err := tree.ValidateRestartCheckpoint(s.Name); err != nil {
// Ensure that the user isn't trying to run BEGIN; SAVEPOINT; SAVEPOINT;
if ex.state.activeSavepointName != "" {
err := fmt.Errorf("SAVEPOINT may not be nested")
return makeErrEvent(err)
}
if err := ex.validateSavepointName(s.Name); err != nil {
return makeErrEvent(err)
}
// We want to disallow SAVEPOINTs to be issued after a transaction has
// We want to disallow SAVEPOINTs to be issued after a KV transaction has
// started running. The client txn's statement count indicates how many
// statements have been executed as part of this transaction.
// statements have been executed as part of this transaction. It is
// desirable to allow metadata queries against vtables to proceed
// before starting a SAVEPOINT for better ORM compatibility.
// See also:
// https://github.com/cockroachdb/cockroach/issues/15012
meta := ex.state.mu.txn.GetTxnCoordMeta(ctx)
if meta.CommandCount > 0 {
err := fmt.Errorf("SAVEPOINT %s needs to be the first statement in a "+
"transaction", tree.RestartSavepointName)
"transaction", RestartSavepointName)
return makeErrEvent(err)
}
ex.state.activeSavepointName = s.Name
// Note that Savepoint doesn't have a corresponding plan node.
// This here is all the execution there is.
return eventRetryIntentSet{}, nil /* payload */, nil

case *tree.RollbackToSavepoint:
if err := tree.ValidateRestartCheckpoint(s.Savepoint); err != nil {
if err := ex.validateSavepointName(s.Savepoint); err != nil {
return makeErrEvent(err)
}
if !os.RetryIntent.Get() {
return makeErrEvent(errSavepointNotUsed)
}
ex.state.activeSavepointName = ""

res.ResetStmtType((*tree.Savepoint)(nil))
return eventTxnRestart{}, nil /* payload */, nil
Expand Down Expand Up @@ -573,6 +587,7 @@ func (ex *connExecutor) checkTableTwoVersionInvariant(ctx context.Context) error
func (ex *connExecutor) commitSQLTransaction(
ctx context.Context, stmt tree.Statement,
) (fsm.Event, fsm.EventPayload) {
ex.state.activeSavepointName = ""
isRelease := false
if _, ok := stmt.(*tree.ReleaseSavepoint); ok {
isRelease = true
Expand All @@ -595,6 +610,7 @@ func (ex *connExecutor) commitSQLTransaction(
// rollbackSQLTransaction executes a ROLLBACK statement: the KV transaction is
// rolled-back and an event is produced.
func (ex *connExecutor) rollbackSQLTransaction(ctx context.Context) (fsm.Event, fsm.EventPayload) {
ex.state.activeSavepointName = ""
if err := ex.state.mu.txn.Rollback(ctx); err != nil {
log.Warningf(ctx, "txn rollback failed: %s", err)
}
Expand Down Expand Up @@ -1073,6 +1089,7 @@ func (ex *connExecutor) execStmtInAbortedState(
ev, payload := ex.rollbackSQLTransaction(ctx)
return ev, payload
}
ex.state.activeSavepointName = ""

// Note: Postgres replies to COMMIT of failed txn with "ROLLBACK" too.
res.ResetStmtType((*tree.RollbackTransaction)(nil))
Expand All @@ -1082,22 +1099,36 @@ func (ex *connExecutor) execStmtInAbortedState(
// We accept both the "ROLLBACK TO SAVEPOINT cockroach_restart" and the
// "SAVEPOINT cockroach_restart" commands to indicate client intent to
// retry a transaction in a RestartWait state.
var spName string
var spName tree.Name
var isRollback bool
switch n := s.(type) {
case *tree.RollbackToSavepoint:
spName = n.Savepoint
isRollback = true
case *tree.Savepoint:
spName = n.Name
default:
panic("unreachable")
}
if err := tree.ValidateRestartCheckpoint(spName); err != nil {
// If the user issued a SAVEPOINT in the abort state, validate
// as though there were no active savepoint.
if !isRollback {
ex.state.activeSavepointName = ""
}
if err := ex.validateSavepointName(spName); err != nil {
ev := eventNonRetriableErr{IsCommit: fsm.False}
payload := eventNonRetriableErrPayload{
err: err,
}
return ev, payload
}
// Either clear or reset the current savepoint name so that
// ROLLBACK TO; SAVEPOINT; works.
if isRollback {
ex.state.activeSavepointName = ""
} else {
ex.state.activeSavepointName = spName
}

if !(inRestartWait || ex.machine.CurState().(stateAborted).RetryIntent.Get()) {
ev := eventNonRetriableErr{IsCommit: fsm.False}
Expand Down Expand Up @@ -1354,3 +1385,24 @@ func (ex *connExecutor) incrementStmtCounter(stmt Statement) {
ex.server.StatementCounters.incrementCount(stmt.AST)
}
}

// validateSavepointName validates that it is that the provided ident
// matches the active savepoint name, begins with RestartSavepointName,
// or that force_savepoint_restart==true. We accept everything with the
// desired prefix because at least the C++ libpqxx appends sequence
// numbers to the savepoint name specified by the user.
func (ex *connExecutor) validateSavepointName(savepoint tree.Name) error {
if ex.state.activeSavepointName != "" {
if savepoint == ex.state.activeSavepointName {
return nil
}
return errors.Errorf(`SAVEPOINT "%s" is in use`, ex.state.activeSavepointName)
}
if !ex.sessionData.ForceSavepointRestart && !strings.HasPrefix(savepoint.String(), RestartSavepointName) {
return pgerror.UnimplementedWithIssueHintError(10735,
"SAVEPOINT not supported except for "+RestartSavepointName,
"Retryable transactions with arbitrary SAVEPOINT names can be enabled "+
"with SET force_savepoint_restart=true;")
}
return nil
}
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Expand Up @@ -1618,6 +1618,10 @@ func (m *sessionDataMutator) SetDistSQLMode(val sessiondata.DistSQLExecMode) {
m.data.DistSQLMode = val
}

func (m *sessionDataMutator) SetForceSavepointRestart(val bool) {
m.data.ForceSavepointRestart = val
}

func (m *sessionDataMutator) SetLookupJoinEnabled(val bool) {
m.data.LookupJoinEnabled = val
}
Expand Down
69 changes: 68 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/manual_retry
Expand Up @@ -28,8 +28,13 @@ ROLLBACK TO SAVEPOINT cockroach_restart
# wait until the transaction is at least 1 second
sleep 1s

# Ensure that ident case rules are used.
statement error pq: unimplemented: SAVEPOINT not supported except for cockroach_restart
SAVEPOINT "COCKROACH_RESTART"

# Ensure that ident case rules are used.
statement ok
SAVEPOINT cockroach_restart
SAVEPOINT COCKROACH_RESTART

query I
SELECT crdb_internal.force_retry('500ms':::INTERVAL)
Expand Down Expand Up @@ -77,3 +82,65 @@ query I
SELECT id FROM t
----
1

subtest rename_savepoint

query T
show session force_savepoint_restart
----
off

statement ok
SET force_savepoint_restart = true

query T
show session force_savepoint_restart
----
on

# We can now use anything that we want.
statement ok
BEGIN TRANSACTION; SAVEPOINT something_else; COMMIT

# Ensure that we can't mix-and-match names.
statement ok
BEGIN TRANSACTION; SAVEPOINT foo

statement error pq: SAVEPOINT "foo" is in use
ROLLBACK TO SAVEPOINT bar

# Verify we're doing the right thing for non-quoted idents.
statement ok
ROLLBACK TO SAVEPOINT FOO

# Verify use of quoted idents.
statement ok
SAVEPOINT "Foo Bar"

statement error pq: SAVEPOINT "Foo Bar" is in use
ROLLBACK TO SAVEPOINT FooBar

# Verify case-sensitivity of quoted idents.
statement error pq: SAVEPOINT "Foo Bar" is in use
ROLLBACK TO SAVEPOINT "foo bar"

statement ok
ROLLBACK TO SAVEPOINT "Foo Bar"

# Verify case-sensitivity of quoted vs. unquoted idents.
statement ok
SAVEPOINT "UpperCase"

statement error pq: SAVEPOINT "UpperCase" is in use
ROLLBACK TO SAVEPOINT UpperCase

statement ok
ABORT

statement ok
RESET force_savepoint_restart

query T
show session force_savepoint_restart
----
off
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Expand Up @@ -1340,6 +1340,7 @@ experimental_force_zigzag_join off NULL NULL NULL
experimental_serial_normalization rowid NULL NULL NULL string
experimental_vectorize off NULL NULL NULL string
extra_float_digits 0 NULL NULL NULL string
force_savepoint_restart off NULL NULL NULL string
integer_datetimes on NULL NULL NULL string
intervalstyle postgres NULL NULL NULL string
max_index_keys 32 NULL NULL NULL string
Expand Down Expand Up @@ -1383,6 +1384,7 @@ experimental_force_zigzag_join off NULL user NULL off
experimental_serial_normalization rowid NULL user NULL rowid rowid
experimental_vectorize off NULL user NULL off off
extra_float_digits 0 NULL user NULL 0 2
force_savepoint_restart off NULL user NULL off off
integer_datetimes on NULL user NULL on on
intervalstyle postgres NULL user NULL postgres postgres
max_index_keys 32 NULL user NULL 32 32
Expand Down Expand Up @@ -1422,6 +1424,7 @@ experimental_force_zigzag_join NULL NULL NULL NULL NULL
experimental_serial_normalization NULL NULL NULL NULL NULL
experimental_vectorize NULL NULL NULL NULL NULL
extra_float_digits NULL NULL NULL NULL NULL
force_savepoint_restart NULL NULL NULL NULL NULL
integer_datetimes NULL NULL NULL NULL NULL
intervalstyle NULL NULL NULL NULL NULL
max_index_keys NULL NULL NULL NULL NULL
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/show_source
Expand Up @@ -39,6 +39,7 @@ experimental_force_zigzag_join off
experimental_serial_normalization rowid
experimental_vectorize off
extra_float_digits 0
force_savepoint_restart off
integer_datetimes on
intervalstyle postgres
max_index_keys 32
Expand Down
16 changes: 8 additions & 8 deletions pkg/sql/logictest/testdata/logic_test/txn
Expand Up @@ -661,7 +661,7 @@ SHOW TRANSACTION STATUS
----
RestartWait

statement error SAVEPOINT not supported except for COCKROACH_RESTART
statement error pq: SAVEPOINT "cockroach_restart" is in use
ROLLBACK TO SAVEPOINT bogus_name

query T
Expand All @@ -676,7 +676,7 @@ ROLLBACK
statement ok
BEGIN TRANSACTION

statement error SAVEPOINT not supported except for COCKROACH_RESTART
statement error SAVEPOINT not supported except for cockroach_restart
SAVEPOINT other

statement ok
Expand All @@ -685,7 +685,7 @@ ROLLBACK
statement ok
BEGIN TRANSACTION

statement error SAVEPOINT not supported except for COCKROACH_RESTART
statement error SAVEPOINT not supported except for cockroach_restart
RELEASE SAVEPOINT other

statement ok
Expand All @@ -694,7 +694,7 @@ ROLLBACK
statement ok
BEGIN TRANSACTION

statement error SAVEPOINT not supported except for COCKROACH_RESTART
statement error SAVEPOINT not supported except for cockroach_restart
ROLLBACK TO SAVEPOINT other

statement ok
Expand All @@ -704,7 +704,7 @@ ROLLBACK
statement ok
BEGIN TRANSACTION; UPSERT INTO kv VALUES('savepoint', 'true')

statement error SAVEPOINT COCKROACH_RESTART needs to be the first statement in a transaction
statement error SAVEPOINT cockroach_restart needs to be the first statement in a transaction
SAVEPOINT cockroach_restart

statement ok
Expand Down Expand Up @@ -766,7 +766,7 @@ ROLLBACK
statement ok
BEGIN

statement error pgcode 3B000 savepoint COCKROACH_RESTART has not been used
statement error pgcode 3B000 savepoint cockroach_restart has not been used
ROLLBACK TO SAVEPOINT cockroach_restart

statement ok
Expand All @@ -780,7 +780,7 @@ BEGIN
statement error pq: relation "bogus_name" does not exist
SELECT * from bogus_name

statement error pgcode 3B000 savepoint COCKROACH_RESTART has not been used
statement error pgcode 3B000 savepoint cockroach_restart has not been used
ROLLBACK TO SAVEPOINT cockroach_restart

statement ok
Expand Down Expand Up @@ -980,7 +980,7 @@ ROLLBACK

# Check that we don't crash when doing a release that wasn't preceded by a
# savepoint.
statement error pgcode 3B000 savepoint COCKROACH_RESTART has not been used
statement error pgcode 3B000 savepoint cockroach_restart has not been used
BEGIN; RELEASE SAVEPOINT cockroach_restart

statement ok
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/logictest/testdata/planner_test/explain
Expand Up @@ -171,7 +171,7 @@ render · ·
└── filter · ·
│ filter variable = 'database'
└── values · ·
· size 2 columns, 35 rows
· size 2 columns, 36 rows

query TTT
EXPLAIN SHOW TIME ZONE
Expand All @@ -180,7 +180,7 @@ render · ·
└── filter · ·
│ filter variable = 'timezone'
└── values · ·
· size 2 columns, 35 rows
· size 2 columns, 36 rows

query TTT
EXPLAIN SHOW DEFAULT_TRANSACTION_ISOLATION
Expand All @@ -189,7 +189,7 @@ render · ·
└── filter · ·
│ filter variable = 'default_transaction_isolation'
└── values · ·
· size 2 columns, 35 rows
· size 2 columns, 36 rows

query TTT
EXPLAIN SHOW TRANSACTION ISOLATION LEVEL
Expand All @@ -198,7 +198,7 @@ render · ·
└── filter · ·
│ filter variable = 'transaction_isolation'
└── values · ·
· size 2 columns, 35 rows
· size 2 columns, 36 rows

query TTT
EXPLAIN SHOW TRANSACTION PRIORITY
Expand All @@ -207,7 +207,7 @@ render · ·
└── filter · ·
│ filter variable = 'transaction_priority'
└── values · ·
· size 2 columns, 35 rows
· size 2 columns, 36 rows

query TTT
EXPLAIN SHOW COLUMNS FROM foo
Expand Down

0 comments on commit a66d7f8

Please sign in to comment.