Skip to content

Commit

Permalink
Merge #118440
Browse files Browse the repository at this point in the history
118440: sql: add autocommit_before_ddl setting r=rafiss a=rafiss

fixes #87236
informs #87233

This setting can be used to improve compatibility with some tools that
do not work well due to our limitations with schema changes in explicit
transactions. It also can be used to use schema changes under READ
COMMITTED more easily, without needing to teach the schema changer about
READ COMMITTED.

Release note (sql change): Added the autocommit_before_ddl session
variable. When set to true, any schema change statement that is sent
during an explicit transaction will cause the transaction to commit
before executing the schema change.

---

### sql: handle COMMIT outside of txn when autocommit_before_ddl is on

With this setting on, COMMIT, ROLLBACK, and other statements that
normally cause errors when used outside of an explicit transaction are
instead treated as warnings. This is because the setting can cause a
transaction to end earlier than an application may expect.

### sql: add hint to use autocommit_before_ddl setting

Update the error message returned when using schema changes under READ
COMMITTED to hint towards using the autocommit_before_ddl setting.

Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
  • Loading branch information
craig[bot] and rafiss committed Jan 31, 2024
2 parents 07adffb + 4b9b150 commit 04dd7ea
Show file tree
Hide file tree
Showing 14 changed files with 141 additions and 29 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2703,7 +2703,7 @@ func (ex *connExecutor) updateTxnRewindPosMaybe(
// All statements with lower position in stmtBuf (if any) are removed, as we
// won't ever need them again.
func (ex *connExecutor) setTxnRewindPos(ctx context.Context, pos CmdPos) error {
if pos <= ex.extraTxnState.txnRewindPos {
if pos < ex.extraTxnState.txnRewindPos {
panic(errors.AssertionFailedf("can only move the txnRewindPos forward. "+
"Was: %d; new value: %d", ex.extraTxnState.txnRewindPos, pos))
}
Expand Down
41 changes: 35 additions & 6 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -937,6 +937,23 @@ func (ex *connExecutor) execStmtInOpenState(
// dispatchToExecutionEngine.
shouldLogToExecAndAudit = false

if tree.CanModifySchema(ast) && !ex.implicitTxn() &&
ex.sessionData().AutoCommitBeforeDDL && ex.executorType != executorTypeInternal {
if err := ex.planner.SendClientNotice(
ctx,
pgnotice.Newf("auto-committing transaction before processing DDL due to autocommit_before_ddl setting"),
); err != nil {
return nil, nil, err
}
retEv, retPayload = ex.handleAutoCommit(ctx, ast)
if _, committed := retEv.(eventTxnFinishCommitted); committed && retPayload == nil {
// Use eventTxnCommittedDueToDDL so that the current statement gets
// executed again when the state machine advances.
retEv = eventTxnCommittedDueToDDL{}
}
return
}

// For regular statements (the ones that get to this point), we
// don't return any event unless an error happens.

Expand Down Expand Up @@ -2237,16 +2254,19 @@ func (ex *connExecutor) handleTxnRowsWrittenReadLimits(ctx context.Context) erro
return errors.CombineErrors(writtenErr, readErr)
}

var txnSchemaChangeErr = pgerror.Newf(
pgcode.FeatureNotSupported,
"to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting",
)

// maybeUpgradeToSerializable checks if the statement is a schema change, and
// upgrades the transaction to serializable isolation if it is. If the
// transaction contains multiple statements, and an upgrade was attempted, an
// error is returned.
func (ex *connExecutor) maybeUpgradeToSerializable(ctx context.Context, stmt Statement) error {
p := &ex.planner
if ex.extraTxnState.upgradedToSerializable && ex.extraTxnState.firstStmtExecuted {
return pgerror.Newf(
pgcode.FeatureNotSupported, "multi-statement transaction involving a schema change needs to be SERIALIZABLE",
)
return txnSchemaChangeErr
}
if tree.CanModifySchema(stmt.AST) {
if ex.state.mu.txn.IsoLevel().ToleratesWriteSkew() {
Expand All @@ -2257,9 +2277,7 @@ func (ex *connExecutor) maybeUpgradeToSerializable(ctx context.Context, stmt Sta
ex.extraTxnState.upgradedToSerializable = true
p.BufferClientNotice(ctx, pgnotice.Newf("setting transaction isolation level to SERIALIZABLE due to schema change"))
} else {
return pgerror.Newf(
pgcode.FeatureNotSupported, "multi-statement transaction involving a schema change needs to be SERIALIZABLE",
)
return txnSchemaChangeErr
}
}
}
Expand Down Expand Up @@ -2547,6 +2565,17 @@ func (ex *connExecutor) execStmtInNoTxnState(
return ex.execShowCommitTimestampInNoTxnState(ctx, s, res)
case *tree.CommitTransaction, *tree.ReleaseSavepoint,
*tree.RollbackTransaction, *tree.SetTransaction, *tree.Savepoint:
if ex.sessionData().AutoCommitBeforeDDL {
// If autocommit_before_ddl is set, we allow these statements to be
// executed, and send a warning rather than an error.
if err := ex.planner.SendClientNotice(
ctx,
pgerror.WithSeverity(errNoTransactionInProgress, "WARNING"),
); err != nil {
return ex.makeErrEvent(err, ast)
}
return nil, nil
}
return ex.makeErrEvent(errNoTransactionInProgress, ast)
default:
// NB: Implicit transactions are created with the session's default
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/conn_fsm.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,6 +215,11 @@ type eventTxnReleased struct{}
// CommitWait.
type eventTxnCommittedWithShowCommitTimestamp struct{}

// eventTxnCommittedDueToDDL is generated when a DDL statement is received
// in an explicit transaction, and the autocommit_before_ddl session variable
// is enabled. It moves the state to NoTxn.
type eventTxnCommittedDueToDDL struct{}

// payloadWithError is a common interface for the payloads that wrap an error.
type payloadWithError interface {
errorCause() error
Expand All @@ -230,6 +235,7 @@ func (eventTxnRestart) Event() {}
func (eventTxnReleased) Event() {}
func (eventTxnCommittedWithShowCommitTimestamp) Event() {}
func (eventTxnUpgradeToExplicit) Event() {}
func (eventTxnCommittedDueToDDL) Event() {}

// TxnStateTransitions describe the transitions used by a connExecutor's
// fsm.Machine. Args.Extended is a txnState, which is muted by the Actions.
Expand Down Expand Up @@ -366,6 +372,18 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{
Next: stateCommitWait{},
Action: moveToCommitWaitAfterInternalCommit,
},
eventTxnCommittedDueToDDL{}: {
Description: "auto-commit before DDL",
Next: stateNoTxn{},
Action: func(args fsm.Args) error {
ts := args.Extended.(*txnState)
finishedTxnID, commitTimestamp := ts.finishSQLTxn()
ts.setAdvanceInfo(stayInPlace, noRewind, txnEvent{
eventType: txnCommit, txnID: finishedTxnID, commitTimestamp: commitTimestamp,
})
return nil
},
},
// This is the case where we auto-retry explicit transactions.
eventRetriableErr{CanAutoRetry: fsm.True, IsCommit: fsm.Any}: {
// Rewind and auto-retry - the transaction should stay in the Open state.
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3268,6 +3268,10 @@ func (m *sessionDataMutator) SetStrictDDLAtomicity(val bool) {
m.data.StrictDDLAtomicity = val
}

func (m *sessionDataMutator) SetAutoCommitBeforeDDL(val bool) {
m.data.AutoCommitBeforeDDL = val
}

func (m *sessionDataMutator) SetLocation(loc *time.Location) {
oldLocation := sessionDataTimeZoneFormat(m.data.Location)
m.data.Location = loc
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -5480,6 +5480,7 @@ allow_ordinal_column_references off
allow_role_memberships_to_change_during_transaction off
alter_primary_region_super_region_override off
application_name ·
autocommit_before_ddl off
avoid_buffering off
backslash_quote safe_encoding
bytea_output hex
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -2793,6 +2793,7 @@ allow_ordinal_column_references off N
allow_role_memberships_to_change_during_transaction off NULL NULL NULL string
alter_primary_region_super_region_override off NULL NULL NULL string
application_name · NULL NULL NULL string
autocommit_before_ddl off NULL NULL NULL string
avoid_buffering off NULL NULL NULL string
backslash_quote safe_encoding NULL NULL NULL string
bytea_output hex NULL NULL NULL string
Expand Down Expand Up @@ -2967,6 +2968,7 @@ allow_ordinal_column_references off N
allow_role_memberships_to_change_during_transaction off NULL user NULL off off
alter_primary_region_super_region_override off NULL user NULL off off
application_name · NULL user NULL · ·
autocommit_before_ddl off NULL user NULL off off
avoid_buffering off NULL user NULL false false
backslash_quote safe_encoding NULL user NULL safe_encoding safe_encoding
bytea_output hex NULL user NULL hex hex
Expand Down Expand Up @@ -3134,6 +3136,7 @@ allow_ordinal_column_references NULL NULL NULL
allow_role_memberships_to_change_during_transaction NULL NULL NULL NULL NULL
alter_primary_region_super_region_override NULL NULL NULL NULL NULL
application_name NULL NULL NULL NULL NULL
autocommit_before_ddl NULL NULL NULL NULL NULL
avoid_buffering NULL NULL NULL NULL NULL
backslash_quote NULL NULL NULL NULL NULL
bytea_output NULL NULL NULL NULL NULL
Expand Down
40 changes: 20 additions & 20 deletions pkg/sql/logictest/testdata/logic_test/read_committed
Original file line number Diff line number Diff line change
Expand Up @@ -286,87 +286,87 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
ALTER TABLE single_stmt_explicit ADD COLUMN b TEXT;
COMMIT

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
ALTER TABLE supermarket ADD COLUMN age INT

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
CREATE TABLE foo(a INT)

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
DROP TABLE supermarket

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
CREATE USER foo

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
DROP USER testuser

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
GRANT admin TO testuser

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
GRANT SELECT ON supermarket TO testuser

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
GRANT USAGE ON SCHEMA public TO testuser

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
GRANT CONNECT ON DATABASE postgres TO testuser

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
CREATE INDEX foo ON supermarket(ends_with, starts_with)

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
CREATE FUNCTION f (x INT) RETURNS INT LANGUAGE SQL AS $$
Expand All @@ -376,71 +376,71 @@ $$
statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
ALTER FUNCTION f (x INT) RENAME TO g

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
GRANT EXECUTE ON FUNCTION f (x INT) TO testuser

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
CREATE TYPE typ AS ENUM('a', 'b')

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
ALTER TYPE typ ADD VALUE 'c'

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
GRANT USAGE ON TYPE typ TO testuser

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
CREATE DATABASE foo;
SELECT 1;

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
ALTER DATABASE postgres RENAME TO foo;
SELECT 1;

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
CREATE SCHEMA s;
SELECT 1;

statement ok
ROLLBACK

statement error transaction involving a schema change needs to be SERIALIZABLE
statement error to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT 1;
ALTER SCHEMA s RENAME TO foo;
Expand Down
23 changes: 23 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/schema_change_in_txn
Original file line number Diff line number Diff line change
Expand Up @@ -1999,3 +1999,26 @@ COMMIT;

statement ok
DROP TABLE t1, t2 CASCADE

statement ok
SET autocommit_before_ddl = true

statement ok
CREATE TABLE t1 (id STRING PRIMARY KEY, other_id STRING NOT NULL);

statement ok
BEGIN

query T noticetrace
ALTER TABLE t1 ADD COLUMN c INT DEFAULT 1
----
NOTICE: auto-committing transaction before processing DDL due to autocommit_before_ddl setting

query T noticetrace
COMMIT
----
WARNING: there is no transaction in progress
SQLSTATE: 25P01

statement ok
DROP TABLE t1
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ allow_ordinal_column_references off
allow_role_memberships_to_change_during_transaction off
alter_primary_region_super_region_override off
application_name ·
autocommit_before_ddl off
avoid_buffering off
backslash_quote safe_encoding
bytea_output hex
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/pgwire/testdata/pgtest/read_committed
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ ReadyForQuery
----
{"Severity":"NOTICE","SeverityUnlocalized":"NOTICE","Code":"00000","Message":"setting transaction isolation level to SERIALIZABLE due to schema change","Detail":"","Hint":"","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"conn_executor_exec.go","Line":0,"Routine":"maybeUpgradeToSerializable","UnknownFields":null}
{"Type":"CommandComplete","CommandTag":"CREATE TABLE"}
{"Type":"ErrorResponse","Code":"0A000","Message":"multi-statement transaction involving a schema change needs to be SERIALIZABLE"}
{"Type":"ErrorResponse","Code":"0A000","Message":"to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting"}
{"Type":"ReadyForQuery","TxStatus":"I"}

send
Expand All @@ -69,5 +69,5 @@ ReadyForQuery
{"Type":"ParseComplete"}
{"Type":"BindComplete"}
{"Type":"CommandComplete","CommandTag":"CREATE TABLE"}
{"Type":"ErrorResponse","Code":"0A000","Message":"multi-statement transaction involving a schema change needs to be SERIALIZABLE"}
{"Type":"ErrorResponse","Code":"0A000","Message":"to use multi-statement transactions involving a schema change under weak isolation levels, enable the autocommit_before_ddl setting"}
{"Type":"ReadyForQuery","TxStatus":"I"}

0 comments on commit 04dd7ea

Please sign in to comment.