diff --git a/pkg/sql/catalog/schemaexpr/BUILD.bazel b/pkg/sql/catalog/schemaexpr/BUILD.bazel index 8501a48dfc6b..9009c5d6a930 100644 --- a/pkg/sql/catalog/schemaexpr/BUILD.bazel +++ b/pkg/sql/catalog/schemaexpr/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "hash_sharded_compute_expr.go", "partial_index.go", "select_name_resolution.go", + "sequence_options.go", "unique_contraint.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr", diff --git a/pkg/sql/catalog/schemaexpr/column.go b/pkg/sql/catalog/schemaexpr/column.go index fdbce0c8a763..7fa7061a93aa 100644 --- a/pkg/sql/catalog/schemaexpr/column.go +++ b/pkg/sql/catalog/schemaexpr/column.go @@ -107,7 +107,7 @@ func FormatColumnForDisplay( f.WriteString(" GENERATED BY DEFAULT AS IDENTITY") } if col.HasGeneratedAsIdentitySequenceOption() { - seqOpt := col.GetGeneratedAsIdentitySequenceOption() + seqOpt := col.GetGeneratedAsIdentitySequenceOptionStr() s := formatGeneratedAsIdentitySequenceOption(seqOpt) f.WriteString(s) } diff --git a/pkg/sql/catalog/schemaexpr/sequence_options.go b/pkg/sql/catalog/schemaexpr/sequence_options.go new file mode 100644 index 000000000000..a25a99f106c8 --- /dev/null +++ b/pkg/sql/catalog/schemaexpr/sequence_options.go @@ -0,0 +1,322 @@ +// 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 schemaexpr + +import ( + "math" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" + "github.com/cockroachdb/errors" +) + +// ParseSequenceOpts is to transform the sequence options saved the +// descriptor to a descpb.TableDescriptor_SequenceOpts. +// Note that this function is used to acquire the sequence option for the +// information schema table, so it doesn't parse for the sequence owner info. +func ParseSequenceOpts(s string) (*descpb.TableDescriptor_SequenceOpts, error) { + stmt, err := parser.ParseOne("CREATE SEQUENCE fake_seq " + s) + if err != nil { + return nil, errors.Wrap(err, "cannot parse sequence option") + } + + createSeqNode, ok := stmt.AST.(*tree.CreateSequence) + if !ok { + return nil, errors.New("cannot convert parsed result to tree.CreateSequence") + } + + opts := &descpb.TableDescriptor_SequenceOpts{ + Increment: 1, + } + if err := AssignSequenceOptions( + opts, + createSeqNode.Options, + true, /* setDefaults */ + nil, /* existingType */ + ); err != nil { + return nil, err + } + + return opts, nil +} + +func getSequenceIntegerBounds( + integerType *types.T, +) (lowerIntBound int64, upperIntBound int64, err error) { + switch integerType { + case types.Int2: + return math.MinInt16, math.MaxInt16, nil + case types.Int4: + return math.MinInt32, math.MaxInt32, nil + case types.Int: + return math.MinInt64, math.MaxInt64, nil + } + + return 0, 0, errors.AssertionFailedf( + "CREATE SEQUENCE option AS received type %s, must be integer", + integerType, + ) +} + +func setSequenceIntegerBounds( + opts *descpb.TableDescriptor_SequenceOpts, + integerType *types.T, + isAscending bool, + setMinValue bool, + setMaxValue bool, +) error { + var minValue int64 = math.MinInt64 + var maxValue int64 = math.MaxInt64 + + if isAscending { + minValue = 1 + + switch integerType { + case types.Int2: + maxValue = math.MaxInt16 + case types.Int4: + maxValue = math.MaxInt32 + case types.Int: + // Do nothing, it's the default. + default: + return errors.AssertionFailedf( + "CREATE SEQUENCE option AS received type %s, must be integer", + integerType, + ) + } + } else { + maxValue = -1 + switch integerType { + case types.Int2: + minValue = math.MinInt16 + case types.Int4: + minValue = math.MinInt32 + case types.Int: + // Do nothing, it's the default. + default: + return errors.AssertionFailedf( + "CREATE SEQUENCE option AS received type %s, must be integer", + integerType, + ) + } + } + if setMinValue { + opts.MinValue = minValue + } + if setMaxValue { + opts.MaxValue = maxValue + } + return nil +} + +// AssignSequenceOptions moves options from the AST node to the sequence options descriptor, +// starting with defaults and overriding them with user-provided options. +func AssignSequenceOptions( + opts *descpb.TableDescriptor_SequenceOpts, + optsNode tree.SequenceOptions, + setDefaults bool, + existingType *types.T, +) error { + wasAscending := opts.Increment > 0 + + // Set the default integer type of a sequence. + var integerType = types.Int + // All other defaults are dependent on the value of increment + // and the AS integerType. (i.e. whether the sequence is ascending + // or descending, bigint vs. smallint) + for _, option := range optsNode { + if option.Name == tree.SeqOptIncrement { + opts.Increment = *option.IntVal + } else if option.Name == tree.SeqOptAs { + integerType = option.AsIntegerType + opts.AsIntegerType = integerType.SQLString() + } + } + if opts.Increment == 0 { + return errors.New("INCREMENT must not be zero") + } + isAscending := opts.Increment > 0 + + // Set increment-dependent defaults. + if setDefaults { + if isAscending { + opts.MinValue = 1 + opts.MaxValue = math.MaxInt64 + opts.Start = opts.MinValue + } else { + opts.MinValue = math.MinInt64 + opts.MaxValue = -1 + opts.Start = opts.MaxValue + } + opts.CacheSize = 1 + } + + lowerIntBound, upperIntBound, err := getSequenceIntegerBounds(integerType) + if err != nil { + return err + } + + // Set default MINVALUE and MAXVALUE if AS option value for integer type is specified. + if opts.AsIntegerType != "" { + // We change MINVALUE and MAXVALUE if it is the originally set to the default during ALTER. + setMinValue := setDefaults + setMaxValue := setDefaults + if !setDefaults && existingType != nil { + existingLowerIntBound, existingUpperIntBound, err := getSequenceIntegerBounds(existingType) + if err != nil { + return err + } + if (wasAscending && opts.MinValue == 1) || (!wasAscending && opts.MinValue == existingLowerIntBound) { + setMinValue = true + } + if (wasAscending && opts.MaxValue == existingUpperIntBound) || (!wasAscending && opts.MaxValue == -1) { + setMaxValue = true + } + } + + if err := setSequenceIntegerBounds( + opts, + integerType, + isAscending, + setMinValue, + setMaxValue, + ); err != nil { + return err + } + } + + // Fill in all other options. + var restartVal *int64 + optionsSeen := map[string]bool{} + for _, option := range optsNode { + // Error on duplicate options. + _, seenBefore := optionsSeen[option.Name] + if seenBefore { + return errors.New("conflicting or redundant options") + } + optionsSeen[option.Name] = true + + switch option.Name { + case tree.SeqOptCycle: + return unimplemented.NewWithIssue(20961, + "CYCLE option is not supported") + case tree.SeqOptNoCycle: + // Do nothing; this is the default. + case tree.SeqOptCache: + if v := *option.IntVal; v >= 1 { + opts.CacheSize = v + } else { + return errors.Newf( + "CACHE (%d) must be greater than zero", v) + } + case tree.SeqOptIncrement: + // Do nothing; this has already been set. + case tree.SeqOptMinValue: + // A value of nil represents the user explicitly saying `NO MINVALUE`. + if option.IntVal != nil { + opts.MinValue = *option.IntVal + } + case tree.SeqOptMaxValue: + // A value of nil represents the user explicitly saying `NO MAXVALUE`. + if option.IntVal != nil { + opts.MaxValue = *option.IntVal + } + case tree.SeqOptStart: + opts.Start = *option.IntVal + case tree.SeqOptRestart: + // The RESTART option does not get saved, but still gets validated below. + restartVal = option.IntVal + case tree.SeqOptVirtual: + opts.Virtual = true + } + } + + if setDefaults || (wasAscending && opts.Start == 1) || (!wasAscending && opts.Start == -1) { + // If start option not specified, set it to MinValue (for ascending sequences) + // or MaxValue (for descending sequences). + // We only do this if we're setting it for the first time, or the sequence was + // ALTERed with the default original values. + if _, startSeen := optionsSeen[tree.SeqOptStart]; !startSeen { + if opts.Increment > 0 { + opts.Start = opts.MinValue + } else { + opts.Start = opts.MaxValue + } + } + } + + if opts.MinValue < lowerIntBound { + return errors.Newf( + "MINVALUE (%d) must be greater than (%d) for type %s", + opts.MinValue, + lowerIntBound, + integerType.SQLString(), + ) + } + if opts.MaxValue < lowerIntBound { + return errors.Newf( + "MAXVALUE (%d) must be greater than (%d) for type %s", + opts.MaxValue, + lowerIntBound, + integerType.SQLString(), + ) + } + if opts.MinValue > upperIntBound { + return errors.Newf( + "MINVALUE (%d) must be less than (%d) for type %s", + opts.MinValue, + upperIntBound, + integerType.SQLString(), + ) + } + if opts.MaxValue > upperIntBound { + return errors.Newf( + "MAXVALUE (%d) must be less than (%d) for type %s", + opts.MaxValue, + upperIntBound, + integerType.SQLString(), + ) + } + if opts.Start > opts.MaxValue { + return errors.Newf( + "START value (%d) cannot be greater than MAXVALUE (%d)", + opts.Start, + opts.MaxValue, + ) + } + if opts.Start < opts.MinValue { + return errors.Newf( + "START value (%d) cannot be less than MINVALUE (%d)", + opts.Start, + opts.MinValue, + ) + } + if restartVal != nil { + if *restartVal > opts.MaxValue { + return errors.Newf( + "RESTART value (%d) cannot be greater than MAXVALUE (%d)", + *restartVal, + opts.MaxValue, + ) + } + if *restartVal < opts.MinValue { + return errors.Newf( + "RESTART value (%d) cannot be less than MINVALUE (%d)", + *restartVal, + opts.MinValue, + ) + } + } + return nil +} diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 2856549c6662..7a3e072ea5ab 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -367,9 +367,17 @@ type Column interface { // `GENERATED AS IDENTITY` column. HasGeneratedAsIdentitySequenceOption() bool + // GetGeneratedAsIdentitySequenceOptionStr returns the string representation + // of the column's `GENERATED AS IDENTITY` sequence option if it exists, empty + // string otherwise. + GetGeneratedAsIdentitySequenceOptionStr() string + // GetGeneratedAsIdentitySequenceOption returns the column's `GENERATED AS - // IDENTITY` sequence option if it exists, empty string otherwise. - GetGeneratedAsIdentitySequenceOption() string + // IDENTITY` sequence option if it exists, and possible error. + // If the column is not an identity column, return nil for both sequence option + // and the error. + // Note it doesn't return the sequence owner info. + GetGeneratedAsIdentitySequenceOption() (*descpb.TableDescriptor_SequenceOpts, error) } // ConstraintToUpdate is an interface around a constraint mutation. diff --git a/pkg/sql/catalog/tabledesc/column.go b/pkg/sql/catalog/tabledesc/column.go index df8a8dad0464..2571f62e92aa 100644 --- a/pkg/sql/catalog/tabledesc/column.go +++ b/pkg/sql/catalog/tabledesc/column.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -233,15 +234,35 @@ func (w column) GetGeneratedAsIdentityType() catpb.GeneratedAsIdentityType { return w.desc.GeneratedAsIdentityType } -// GetGeneratedAsIdentitySequenceOption returns the column's `GENERATED AS -// IDENTITY` sequence option if it exists, empty string otherwise. -func (w column) GetGeneratedAsIdentitySequenceOption() string { +// GetGeneratedAsIdentitySequenceOptionStr returns the string representation +// of the column's `GENERATED AS IDENTITY` sequence option if it exists, empty +// string otherwise. +func (w column) GetGeneratedAsIdentitySequenceOptionStr() string { if !w.HasGeneratedAsIdentitySequenceOption() { return "" } return strings.TrimSpace(*w.desc.GeneratedAsIdentitySequenceOption) } +// GetGeneratedAsIdentitySequenceOption returns the column's `GENERATED AS +// IDENTITY` sequence option if it exists, and possible error. +// If the column is not an identity column, return nil for both sequence option +// and the error. +// Note it doesn't return the sequence owner info. +func (w column) GetGeneratedAsIdentitySequenceOption() ( + *descpb.TableDescriptor_SequenceOpts, + error, +) { + if !w.HasGeneratedAsIdentitySequenceOption() { + return nil, nil + } + seqOpts, err := schemaexpr.ParseSequenceOpts(*w.desc.GeneratedAsIdentitySequenceOption) + if err != nil { + return nil, err + } + return seqOpts, nil +} + // HasGeneratedAsIdentitySequenceOption returns true if there is a // customized sequence option when this column is created as a // `GENERATED AS IDENTITY` column. diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index e45f2915f4f9..e3b4d43eea93 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -1022,7 +1022,7 @@ func (s *Server) newConnExecutorWithTxn( // initialize the state. ex.machine = fsm.MakeMachine( BoundTxnStateTransitions, - stateOpen{ImplicitTxn: fsm.False}, + stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, &ex.state, ) ex.state.resetForNewSQLTxn( @@ -2214,8 +2214,7 @@ func (ex *connExecutor) updateTxnRewindPosMaybe( return nil } if advInfo.txnEvent.eventType == txnStart || - advInfo.txnEvent.eventType == txnRestart || - advInfo.txnEvent.eventType == txnUpgradeToExplicit { + advInfo.txnEvent.eventType == txnRestart { var nextPos CmdPos switch advInfo.code { case stayInPlace: diff --git a/pkg/sql/conn_executor_test.go b/pkg/sql/conn_executor_test.go index 360f797e4d93..c2a4290fac00 100644 --- a/pkg/sql/conn_executor_test.go +++ b/pkg/sql/conn_executor_test.go @@ -790,6 +790,7 @@ func TestRetriableErrorDuringUpgradedTransaction(t *testing.T) { var fooTableId uint32 testDB.Exec(t, "SET enable_implicit_transaction_for_batch_statements = true") + testDB.Exec(t, "CREATE TABLE bar (a INT PRIMARY KEY)") testDB.Exec(t, "CREATE TABLE foo (a INT PRIMARY KEY)") testDB.QueryRow(t, "SELECT 'foo'::regclass::oid").Scan(&fooTableId) @@ -813,8 +814,14 @@ func TestRetriableErrorDuringUpgradedTransaction(t *testing.T) { return nil }) - testDB.Exec(t, "SELECT 1; BEGIN; INSERT INTO foo VALUES(1); COMMIT;") + testDB.Exec(t, "INSERT INTO bar VALUES(2); BEGIN; INSERT INTO foo VALUES(1); COMMIT;") require.Equal(t, numToRetry+1, int(retryCount)) + + var x int + testDB.QueryRow(t, "select * from foo").Scan(&x) + require.Equal(t, 1, x) + testDB.QueryRow(t, "select * from bar").Scan(&x) + require.Equal(t, 2, x) } // This test ensures that when in an explicit transaction and statement diff --git a/pkg/sql/conn_fsm.go b/pkg/sql/conn_fsm.go index e2ca0bbdf92d..b26bc3ad6c56 100644 --- a/pkg/sql/conn_fsm.go +++ b/pkg/sql/conn_fsm.go @@ -49,7 +49,11 @@ func (stateNoTxn) String() string { } type stateOpen struct { + // ImplicitTxn is false if the txn included a BEGIN command. ImplicitTxn fsm.Bool + // WasUpgraded is true if the txn started as implicit, but a BEGIN made it + // become explicit. + WasUpgraded fsm.Bool } var _ fsm.State = &stateOpen{} @@ -60,7 +64,12 @@ func (stateOpen) String() string { // stateAborted is entered on errors (retriable and non-retriable). A ROLLBACK // TO SAVEPOINT can move the transaction back to stateOpen. -type stateAborted struct{} +type stateAborted struct { + // WasUpgraded is true if the txn started as implicit, but a BEGIN made it + // become explicit. This is needed so that when ROLLBACK TO SAVEPOINT moves + // to stateOpen, we keep tracking WasUpgraded correctly. + WasUpgraded fsm.Bool +} var _ fsm.State = &stateAborted{} @@ -231,7 +240,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ stateNoTxn{}: { eventTxnStart{fsm.Var("implicitTxn")}: { Description: "BEGIN, or before a statement running as an implicit txn", - Next: stateOpen{ImplicitTxn: fsm.Var("implicitTxn")}, + Next: stateOpen{ImplicitTxn: fsm.Var("implicitTxn"), WasUpgraded: fsm.False}, Action: noTxnToOpen, }, eventNonRetriableErr{IsCommit: fsm.Any}: { @@ -248,7 +257,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ }, /// Open - stateOpen{ImplicitTxn: fsm.Any}: { + stateOpen{ImplicitTxn: fsm.Any, WasUpgraded: fsm.Any}: { eventTxnFinishCommitted{}: { Description: "COMMIT, or after a statement running as an implicit txn", Next: stateNoTxn{}, @@ -279,17 +288,15 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ Action: cleanupAndFinishOnError, }, }, - stateOpen{ImplicitTxn: fsm.Var("implicitTxn")}: { + stateOpen{ImplicitTxn: fsm.True, WasUpgraded: fsm.False}: { // This is the case where we auto-retry. eventRetriableErr{CanAutoRetry: fsm.True, IsCommit: fsm.Any}: { - // Rewind and auto-retry - the transaction should stay in the Open state. + // Rewind and auto-retry - the transaction should stay in the Open state Description: "Retriable err; will auto-retry", - Next: stateOpen{ImplicitTxn: fsm.Var("implicitTxn")}, + Next: stateOpen{ImplicitTxn: fsm.True, WasUpgraded: fsm.False}, Action: prepareTxnForRetryWithRewind, }, - }, - // Handle the errors in implicit txns. They move us to NoTxn. - stateOpen{ImplicitTxn: fsm.True}: { + // Handle the errors in implicit txns. They move us to NoTxn. eventRetriableErr{CanAutoRetry: fsm.False, IsCommit: fsm.False}: { Next: stateNoTxn{}, Action: cleanupAndFinishOnError, @@ -298,8 +305,9 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ Next: stateNoTxn{}, Action: cleanupAndFinishOnError, }, + // Handle a txn getting upgraded to an explicit txn. eventTxnUpgradeToExplicit{}: { - Next: stateOpen{ImplicitTxn: fsm.False}, + Next: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.True}, Action: func(args fsm.Args) error { args.Extended.(*txnState).setAdvanceInfo( advanceOne, @@ -310,10 +318,10 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ }, }, }, - // Handle the errors in explicit txns. They move us to Aborted. - stateOpen{ImplicitTxn: fsm.False}: { + stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.Var("wasUpgraded")}: { + // Handle the errors in explicit txns. eventNonRetriableErr{IsCommit: fsm.False}: { - Next: stateAborted{}, + Next: stateAborted{WasUpgraded: fsm.Var("wasUpgraded")}, Action: func(args fsm.Args) error { ts := args.Extended.(*txnState) ts.setAdvanceInfo(skipBatch, noRewind, txnEvent{eventType: noEvent}) @@ -322,13 +330,15 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ }, // ROLLBACK TO SAVEPOINT cockroach. There's not much to do other than generating a // txnRestart output event. + // The next state must be an explicit txn, since the SAVEPOINT + // creation can only happen in an explicit txn. eventTxnRestart{}: { Description: "ROLLBACK TO SAVEPOINT cockroach_restart", - Next: stateOpen{ImplicitTxn: fsm.False}, + Next: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.Var("wasUpgraded")}, Action: prepareTxnForRetry, }, eventRetriableErr{CanAutoRetry: fsm.False, IsCommit: fsm.False}: { - Next: stateAborted{}, + Next: stateAborted{WasUpgraded: fsm.Var("wasUpgraded")}, Action: func(args fsm.Args) error { args.Extended.(*txnState).setAdvanceInfo( skipBatch, @@ -338,6 +348,16 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ 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. + // Retrying can cause the transaction to become implicit if we see that + // the transaction was previously upgraded. During the retry, BEGIN + // will be executed again and upgrade the transaction back to explicit. + Description: "Retriable err; will auto-retry", + Next: stateOpen{ImplicitTxn: fsm.Var("wasUpgraded"), WasUpgraded: fsm.False}, + Action: prepareTxnForRetryWithRewind, + }, eventTxnReleased{}: { Description: "RELEASE SAVEPOINT cockroach_restart", Next: stateCommitWait{}, @@ -362,7 +382,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ // // Note that we don't handle any error events here. Any statement but a // ROLLBACK (TO SAVEPOINT) is expected to not be passed to the state machine. - stateAborted{}: { + stateAborted{WasUpgraded: fsm.Var("wasUpgraded")}: { eventTxnFinishAborted{}: { Description: "ROLLBACK", Next: stateNoTxn{}, @@ -378,7 +398,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ eventNonRetriableErr{IsCommit: fsm.False}: { // This event doesn't change state, but it returns a skipBatch code. Description: "any other statement", - Next: stateAborted{}, + Next: stateAborted{WasUpgraded: fsm.Var("wasUpgraded")}, Action: func(args fsm.Args) error { args.Extended.(*txnState).setAdvanceInfo( skipBatch, @@ -392,13 +412,15 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ eventNonRetriableErr{IsCommit: fsm.True}: { // This event doesn't change state, but it returns a skipBatch code. Description: "ConnExecutor closing", - Next: stateAborted{}, + Next: stateAborted{WasUpgraded: fsm.Var("wasUpgraded")}, Action: cleanupAndFinishOnError, }, // ROLLBACK TO SAVEPOINT success. + // The next state must be an explicit txn, since the SAVEPOINT + // creation can only happen in an explicit txn. eventSavepointRollback{}: { Description: "ROLLBACK TO SAVEPOINT (not cockroach_restart) success", - Next: stateOpen{ImplicitTxn: fsm.False}, + Next: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.Var("wasUpgraded")}, Action: func(args fsm.Args) error { args.Extended.(*txnState).setAdvanceInfo( advanceOne, @@ -412,7 +434,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ eventRetriableErr{CanAutoRetry: fsm.Any, IsCommit: fsm.Any}: { // This event doesn't change state, but it returns a skipBatch code. Description: "ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart", - Next: stateAborted{}, + Next: stateAborted{WasUpgraded: fsm.Var("wasUpgraded")}, Action: func(args fsm.Args) error { args.Extended.(*txnState).setAdvanceInfo( skipBatch, @@ -423,9 +445,11 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ }, }, // ROLLBACK TO SAVEPOINT cockroach_restart. + // The next state must be an explicit txn, since the SAVEPOINT + // creation can only happen in an explicit txn. eventTxnRestart{}: { Description: "ROLLBACK TO SAVEPOINT cockroach_restart", - Next: stateOpen{ImplicitTxn: fsm.False}, + Next: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.Var("wasUpgraded")}, Action: prepareTxnForRetry, }, }, @@ -559,7 +583,7 @@ func prepareTxnForRetryWithRewind(args fsm.Args) error { // when running SQL inside a higher-level txn. It's a very limited state // machine: it doesn't allow starting or finishing txns, auto-retries, etc. var BoundTxnStateTransitions = fsm.Compile(fsm.Pattern{ - stateOpen{ImplicitTxn: fsm.False}: { + stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}: { // We accept eventNonRetriableErr with both IsCommit={True, fsm.False}, even // though this state machine does not support COMMIT statements because // connExecutor.close() sends an eventNonRetriableErr{IsCommit: fsm.True} event. diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index bcc532a71381..d7299955dfd3 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -478,7 +478,23 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`, udtSchema = tree.NewDString(typeMetaName.Schema) } - err := addRow( + // Get the sequence option if it's an identity column. + identityStart := tree.DNull + identityIncrement := tree.DNull + identityMax := tree.DNull + identityMin := tree.DNull + generatedAsIdentitySeqOpt, err := column.GetGeneratedAsIdentitySequenceOption() + if err != nil { + return err + } + if generatedAsIdentitySeqOpt != nil { + identityStart = tree.NewDString(strconv.FormatInt(generatedAsIdentitySeqOpt.Start, 10)) + identityIncrement = tree.NewDString(strconv.FormatInt(generatedAsIdentitySeqOpt.Increment, 10)) + identityMax = tree.NewDString(strconv.FormatInt(generatedAsIdentitySeqOpt.MaxValue, 10)) + identityMin = tree.NewDString(strconv.FormatInt(generatedAsIdentitySeqOpt.MinValue, 10)) + } + + err = addRow( dbNameStr, // table_catalog scNameStr, // table_schema tree.NewDString(table.GetName()), // table_name @@ -516,12 +532,12 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`, tree.DNull, // is_self_referencing yesOrNoDatum(column.IsGeneratedAsIdentity()), // is_identity colGeneratedAsIdentity, // identity_generation - // TODO(janexing): parse the GeneratedAsIdentitySequenceOption to - // fill out these "identity_x" columns. - tree.DNull, // identity_start - tree.DNull, // identity_increment - tree.DNull, // identity_maximum - tree.DNull, // identity_minimum + identityStart, // identity_start + identityIncrement, // identity_increment + identityMax, // identity_maximum + identityMin, // identity_minimum + // TODO(janexing): we don't support CYCLE syntax for sequences yet. + // https://github.com/cockroachdb/cockroach/issues/20961 tree.DNull, // identity_cycle yesOrNoDatum(column.IsComputed()), // is_generated colComputed, // generation_expression diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index a326ecb4f0eb..27938ecb7e01 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -4813,3 +4813,172 @@ SELECT column_name FROM information_schema.columns WHERE table_name = 't70505' k a b + + +subtest identity_columns + +statement ok +CREATE TABLE t ( + id1 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (START WITH 10), + id2 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (START 10), + id3 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MINVALUE 5), + id4 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (INCREMENT 5), + id5 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (START 2 INCREMENT 5), + id6 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (INCREMENT BY -1 START -5), + id7 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MINVALUE 5 MAXVALUE 10), + id8 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MAXVALUE 10 START WITH 9), + id9 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (INCREMENT BY -1 MINVALUE -10 START WITH -10), + id10 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (START WITH 9223372036854775807), + id11 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MINVALUE -9223372036854775808 START WITH -9223372036854775808 INCREMENT -1), + id12 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (VIRTUAL), + id13 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (CACHE 10 INCREMENT 1), + id14 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (INCREMENT 5), + id15 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MAXVALUE 4 START WITH 2 CACHE 5 INCREMENT BY 2), + id16 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MINVALUE -4 START WITH -2 CACHE 5 INCREMENT BY -2), + id17 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MINVALUE -2 MAXVALUE 2 START WITH 2 CACHE 5 INCREMENT BY -2), + id18 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (AS int2 START WITH -4 INCREMENT BY -3), + id19 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (AS integer), + id20 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (AS int8), + id21 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (AS smallint), + id22 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (AS integer START WITH 2 INCREMENT BY 1 MINVALUE 0 MAXVALUE 234567 CACHE 1), + id23 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (AS smallint START WITH -4 INCREMENT BY -3), + id24 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (AS integer START WITH 1 INCREMENT BY 1 MAXVALUE 9001 CACHE 1), + id25 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (INCREMENT BY 3 MINVALUE 6 MAXVALUE 10), + id26 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (MINVALUE -2 MAXVALUE 2 START WITH 1 CACHE 5 INCREMENT BY -2), + id27 INT8 NOT NULL GENERATED BY DEFAULT AS IDENTITY (INCREMENT BY 3 MINVALUE 6 MAXVALUE 12) +) + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id1'; +---- +10 1 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id2'; +---- +10 1 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id3'; +---- +5 1 9223372036854775807 5 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id4'; +---- +1 5 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id5'; +---- +2 5 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id6'; +---- +-5 -1 -1 -9223372036854775808 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id7'; +---- +5 1 10 5 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id8'; +---- +9 1 10 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id9'; +---- +-10 -1 -1 -10 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id10'; +---- +9223372036854775807 1 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id11'; +---- +-9223372036854775808 -1 -1 -9223372036854775808 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id12'; +---- +1 1 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id13'; +---- +1 1 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id14'; +---- +1 5 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id15'; +---- +2 2 4 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id16'; +---- +-2 -2 -1 -4 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id17'; +---- +2 -2 2 -2 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id18'; +---- +-4 -3 -1 -32768 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id19'; +---- +1 1 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id20'; +---- +1 1 9223372036854775807 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id21'; +---- +1 1 32767 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id22'; +---- +2 1 234567 0 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id23'; +---- +-4 -3 -1 -32768 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id24'; +---- +1 1 9001 1 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id25'; +---- +6 3 10 6 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id26'; +---- +1 -2 2 -2 NULL + +query TTTTT +select identity_start, identity_increment, identity_maximum, identity_minimum, identity_cycle from information_schema.columns where table_name = 't' and column_name='id27'; +---- +6 3 12 6 NULL diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go index c30b4e86f1ee..35b89b4f7fe8 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp.go +++ b/pkg/sql/schemachanger/scdecomp/decomp.go @@ -365,7 +365,7 @@ func (w *walkCtx) walkColumn(tbl catalog.TableDescriptor, col catalog.Column) { IsHidden: col.IsHidden(), IsInaccessible: col.IsInaccessible(), GeneratedAsIdentityType: col.GetGeneratedAsIdentityType(), - GeneratedAsIdentitySequenceOption: col.GetGeneratedAsIdentitySequenceOption(), + GeneratedAsIdentitySequenceOption: col.GetGeneratedAsIdentitySequenceOptionStr(), PgAttributeNum: col.GetPGAttributeNum(), IsSystemColumn: col.IsSystemColumn(), } diff --git a/pkg/sql/schemachanger/sctest/end_to_end.go b/pkg/sql/schemachanger/sctest/end_to_end.go index 6f460051d00e..5e24b88e8755 100644 --- a/pkg/sql/schemachanger/sctest/end_to_end.go +++ b/pkg/sql/schemachanger/sctest/end_to_end.go @@ -17,6 +17,7 @@ import ( "context" gosql "database/sql" "fmt" + "io" "os" "path/filepath" "regexp" @@ -139,7 +140,7 @@ func EndToEndSideEffects(t *testing.T, dir string, newCluster NewClusterFunc) { sctestdeps.WithComments(sctestdeps.ReadCommentsFromDB(t, tdb)), ) stmtStates := execStatementWithTestDeps(ctx, t, deps, stmts...) - writeExplainDiagrams(t, path, setupStmts, stmts, stmtStates) + checkExplainDiagrams(t, path, setupStmts, stmts, stmtStates, d.Rewrite) return replaceNonDeterministicOutput(deps.SideEffectLog()) default: @@ -156,10 +157,15 @@ const ( explainVerboseDirName = "explain_verbose" ) -// writeExplainDiagrams writes out the compact and verbose explain diagrams -// for the statements and plans from the test. -func writeExplainDiagrams( - t *testing.T, path string, setupStmts, stmts parser.Statements, states []scpb.CurrentState, +// checkExplainDiagrams checks the output of the compact and verbose explain +// diagrams for the statements and plans from the test. If rewrite is passed, +// the plans will be rewritten. +func checkExplainDiagrams( + t *testing.T, + path string, + setupStmts, stmts parser.Statements, + states []scpb.CurrentState, + rewrite bool, ) { makeSharedPrefix := func() []byte { var prefixBuf bytes.Buffer @@ -189,26 +195,42 @@ func writeExplainDiagrams( explainDir := mkdir(explainDirName) explainVerboseDir := mkdir(explainVerboseDirName) baseName := filepath.Base(path) - makeFile := func(dir string, i int) *os.File { + makeFile := func(dir string, i int, openFunc func(string) (*os.File, error)) *os.File { name := baseName if len(stmts) > 1 { name = fmt.Sprintf("%s.%d", name, i) } - explainFile, err := os.Create(filepath.Join(dir, name)) - require.NoError(t, err) - _, err = explainFile.Write(makeStatementPrefix(i)) + explainFile, err := openFunc(filepath.Join(dir, name)) require.NoError(t, err) return explainFile } - writePlan := func(dir, tag string, i int, fn func() (string, error)) { - file := makeFile(dir, i) - _, err := fmt.Fprintf(file, "EXPLAIN (%s) %s;\n----\n", tag, stmts[i].SQL) + writePlan := func(file io.Writer, tag string, i int, fn func() (string, error)) { + _, err := file.Write(makeStatementPrefix(i)) + require.NoError(t, err) + _, err = fmt.Fprintf(file, "EXPLAIN (%s) %s;\n----\n", tag, stmts[i].SQL) require.NoError(t, err) out, err := fn() require.NoError(t, err) - _, err = file.WriteString(out) + _, err = io.WriteString(file, out) + require.NoError(t, err) + } + writePlanToFile := func(dir, tag string, i int, fn func() (string, error)) { + file := makeFile(dir, i, os.Create) + defer func() { require.NoError(t, file.Close()) }() + writePlan(file, tag, i, fn) + } + checkPlan := func(dir, tag string, i int, fn func() (string, error)) { + file := makeFile(dir, i, os.Open) + defer func() { require.NoError(t, file.Close()) }() + var buf bytes.Buffer + writePlan(&buf, tag, i, fn) + got, err := io.ReadAll(file) require.NoError(t, err) - require.NoError(t, file.Close()) + require.Equal(t, string(got), buf.String(), filepath.Base(dir)) + } + action := checkPlan + if rewrite { + action = writePlanToFile } for i, stmt := range stmts { pl, err := scplan.MakePlan(states[i], scplan.Params{ @@ -217,8 +239,8 @@ func writeExplainDiagrams( SchemaChangerJobIDSupplier: func() jobspb.JobID { return 1 }, }) require.NoErrorf(t, err, "%d: %s", i, stmt.SQL) - writePlan(explainDir, "ddl", i, pl.ExplainCompact) - writePlan(explainVerboseDir, "ddl, verbose", i, pl.ExplainVerbose) + action(explainDir, "ddl", i, pl.ExplainCompact) + action(explainVerboseDir, "ddl, verbose", i, pl.ExplainVerbose) } } diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index 6f7fa940ec2c..871b95a86700 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -13,7 +13,6 @@ package sql import ( "context" "fmt" - "math" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -23,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -391,162 +390,14 @@ func readOnlyError(s string) error { "cannot execute %s in a read-only transaction", s) } -func getSequenceIntegerBounds( - integerType *types.T, -) (lowerIntBound int64, upperIntBound int64, err error) { - switch integerType { - case types.Int2: - return math.MinInt16, math.MaxInt16, nil - case types.Int4: - return math.MinInt32, math.MaxInt32, nil - case types.Int: - return math.MinInt64, math.MaxInt64, nil - } - - return 0, 0, pgerror.Newf( - pgcode.InvalidParameterValue, - "CREATE SEQUENCE option AS received type %s, must be integer", - integerType, - ) -} - -func setSequenceIntegerBounds( - opts *descpb.TableDescriptor_SequenceOpts, - integerType *types.T, - isAscending bool, - setMinValue bool, - setMaxValue bool, -) error { - var minValue int64 = math.MinInt64 - var maxValue int64 = math.MaxInt64 - - if isAscending { - minValue = 1 - - switch integerType { - case types.Int2: - maxValue = math.MaxInt16 - case types.Int4: - maxValue = math.MaxInt32 - case types.Int: - // Do nothing, it's the default. - default: - return pgerror.Newf( - pgcode.InvalidParameterValue, - "CREATE SEQUENCE option AS received type %s, must be integer", - integerType, - ) - } - } else { - maxValue = -1 - switch integerType { - case types.Int2: - minValue = math.MinInt16 - case types.Int4: - minValue = math.MinInt32 - case types.Int: - // Do nothing, it's the default. - default: - return pgerror.Newf( - pgcode.InvalidParameterValue, - "CREATE SEQUENCE option AS received type %s, must be integer", - integerType, - ) - } - } - if setMinValue { - opts.MinValue = minValue - } - if setMaxValue { - opts.MaxValue = maxValue - } - return nil -} - -// assignSequenceOptions moves options from the AST node to the sequence options descriptor, -// starting with defaults and overriding them with user-provided options. -func assignSequenceOptions( +func assignSequenceOwner( ctx context.Context, p *planner, opts *descpb.TableDescriptor_SequenceOpts, optsNode tree.SequenceOptions, - setDefaults bool, sequenceID descpb.ID, sequenceParentID descpb.ID, - existingType *types.T, ) error { - wasAscending := opts.Increment > 0 - - // Set the default integer type of a sequence. - var integerType = types.Int - // All other defaults are dependent on the value of increment - // and the AS integerType. (i.e. whether the sequence is ascending - // or descending, bigint vs. smallint) - for _, option := range optsNode { - if option.Name == tree.SeqOptIncrement { - opts.Increment = *option.IntVal - } else if option.Name == tree.SeqOptAs { - integerType = option.AsIntegerType - opts.AsIntegerType = integerType.SQLString() - } - } - if opts.Increment == 0 { - return pgerror.New( - pgcode.InvalidParameterValue, "INCREMENT must not be zero") - } - isAscending := opts.Increment > 0 - - // Set increment-dependent defaults. - if setDefaults { - if isAscending { - opts.MinValue = 1 - opts.MaxValue = math.MaxInt64 - opts.Start = opts.MinValue - } else { - opts.MinValue = math.MinInt64 - opts.MaxValue = -1 - opts.Start = opts.MaxValue - } - // No Caching - opts.CacheSize = 1 - } - - lowerIntBound, upperIntBound, err := getSequenceIntegerBounds(integerType) - if err != nil { - return err - } - - // Set default MINVALUE and MAXVALUE if AS option value for integer type is specified. - if opts.AsIntegerType != "" { - // We change MINVALUE and MAXVALUE if it is the originally set to the default during ALTER. - setMinValue := setDefaults - setMaxValue := setDefaults - if !setDefaults && existingType != nil { - existingLowerIntBound, existingUpperIntBound, err := getSequenceIntegerBounds(existingType) - if err != nil { - return err - } - if (wasAscending && opts.MinValue == 1) || (!wasAscending && opts.MinValue == existingLowerIntBound) { - setMinValue = true - } - if (wasAscending && opts.MaxValue == existingUpperIntBound) || (!wasAscending && opts.MaxValue == -1) { - setMaxValue = true - } - } - - if err := setSequenceIntegerBounds( - opts, - integerType, - isAscending, - setMinValue, - setMaxValue, - ); err != nil { - return err - } - } - - // Fill in all other options. - var restartVal *int64 optionsSeen := map[string]bool{} for _, option := range optsNode { // Error on duplicate options. @@ -555,39 +406,7 @@ func assignSequenceOptions( return pgerror.New(pgcode.Syntax, "conflicting or redundant options") } optionsSeen[option.Name] = true - switch option.Name { - case tree.SeqOptCycle: - return unimplemented.NewWithIssue(20961, - "CYCLE option is not supported") - case tree.SeqOptNoCycle: - // Do nothing; this is the default. - case tree.SeqOptCache: - if v := *option.IntVal; v >= 1 { - opts.CacheSize = v - } else { - return pgerror.Newf(pgcode.InvalidParameterValue, - "CACHE (%d) must be greater than zero", v) - } - case tree.SeqOptIncrement: - // Do nothing; this has already been set. - case tree.SeqOptMinValue: - // A value of nil represents the user explicitly saying `NO MINVALUE`. - if option.IntVal != nil { - opts.MinValue = *option.IntVal - } - case tree.SeqOptMaxValue: - // A value of nil represents the user explicitly saying `NO MAXVALUE`. - if option.IntVal != nil { - opts.MaxValue = *option.IntVal - } - case tree.SeqOptStart: - opts.Start = *option.IntVal - case tree.SeqOptRestart: - // The RESTART option does not get saved, but still gets validated below. - restartVal = option.IntVal - case tree.SeqOptVirtual: - opts.Virtual = true case tree.SeqOptOwnedBy: if p == nil { return pgerror.Newf(pgcode.Internal, @@ -630,90 +449,43 @@ func assignSequenceOptions( } } } + return nil +} - if setDefaults || (wasAscending && opts.Start == 1) || (!wasAscending && opts.Start == -1) { - // If start option not specified, set it to MinValue (for ascending sequences) - // or MaxValue (for descending sequences). - // We only do this if we're setting it for the first time, or the sequence was - // ALTERed with the default original values. - if _, startSeen := optionsSeen[tree.SeqOptStart]; !startSeen { - if opts.Increment > 0 { - opts.Start = opts.MinValue - } else { - opts.Start = opts.MaxValue - } +// checkDupSeqOption check if there is any duplicate sequence option. +func checkDupSeqOption(optsNode tree.SequenceOptions) error { + optionsSeen := map[string]bool{} + for _, option := range optsNode { + // Error on duplicate options. + _, seenBefore := optionsSeen[option.Name] + if seenBefore { + return pgerror.New(pgcode.Syntax, "conflicting or redundant options") } + optionsSeen[option.Name] = true } + return nil +} - if opts.MinValue < lowerIntBound { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "MINVALUE (%d) must be greater than (%d) for type %s", - opts.MinValue, - lowerIntBound, - integerType.SQLString(), - ) - } - if opts.MaxValue < lowerIntBound { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "MAXVALUE (%d) must be greater than (%d) for type %s", - opts.MaxValue, - lowerIntBound, - integerType.SQLString(), - ) - } - if opts.MinValue > upperIntBound { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "MINVALUE (%d) must be less than (%d) for type %s", - opts.MinValue, - upperIntBound, - integerType.SQLString(), - ) - } - if opts.MaxValue > upperIntBound { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "MAXVALUE (%d) must be less than (%d) for type %s", - opts.MaxValue, - upperIntBound, - integerType.SQLString(), - ) - } - if opts.Start > opts.MaxValue { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "START value (%d) cannot be greater than MAXVALUE (%d)", - opts.Start, - opts.MaxValue, - ) +// assignSequenceOptions moves options from the AST node to the sequence options descriptor, +// starting with defaults and overriding them with user-provided options. +func assignSequenceOptions( + ctx context.Context, + p *planner, + opts *descpb.TableDescriptor_SequenceOpts, + optsNode tree.SequenceOptions, + setDefaults bool, + sequenceID descpb.ID, + sequenceParentID descpb.ID, + existingType *types.T, +) error { + if err := checkDupSeqOption(optsNode); err != nil { + return err } - if opts.Start < opts.MinValue { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "START value (%d) cannot be less than MINVALUE (%d)", - opts.Start, - opts.MinValue, - ) + if err := schemaexpr.AssignSequenceOptions(opts, optsNode, setDefaults, existingType); err != nil { + return pgerror.WithCandidateCode(err, pgcode.InvalidParameterValue) } - if restartVal != nil { - if *restartVal > opts.MaxValue { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "RESTART value (%d) cannot be greater than MAXVALUE (%d)", - *restartVal, - opts.MaxValue, - ) - } - if *restartVal < opts.MinValue { - return pgerror.Newf( - pgcode.InvalidParameterValue, - "RESTART value (%d) cannot be less than MINVALUE (%d)", - *restartVal, - opts.MinValue, - ) - } + if err := assignSequenceOwner(ctx, p, opts, optsNode, sequenceID, sequenceParentID); err != nil { + return pgerror.WithCandidateCode(err, pgcode.InvalidParameterValue) } return nil } diff --git a/pkg/sql/txn_state.go b/pkg/sql/txn_state.go index edc048da31fe..8f39a6d4bfb5 100644 --- a/pkg/sql/txn_state.go +++ b/pkg/sql/txn_state.go @@ -143,6 +143,9 @@ const ( // explicitTxn means that the txn was explicitly started with a BEGIN // statement. explicitTxn + // upgradedExplicitTxn means that the txn started as implicit, but a BEGIN + // in the middle of it caused it to become explicit. + upgradedExplicitTxn ) // resetForNewSQLTxn (re)initializes the txnState for a new transaction. diff --git a/pkg/sql/txn_state_test.go b/pkg/sql/txn_state_test.go index d8171a7a5e85..2eb5f80e7704 100644 --- a/pkg/sql/txn_state_test.go +++ b/pkg/sql/txn_state_test.go @@ -107,14 +107,17 @@ func (tc *testContext) createOpenState(typ txnType) (fsm.State, *txnState) { state := stateOpen{ ImplicitTxn: fsm.FromBool(typ == implicitTxn), + WasUpgraded: fsm.FromBool(typ == upgradedExplicitTxn), } return state, &ts } // createAbortedState returns a txnState initialized with an aborted txn. -func (tc *testContext) createAbortedState() (fsm.State, *txnState) { - _, ts := tc.createOpenState(explicitTxn) - return stateAborted{}, ts +func (tc *testContext) createAbortedState(typ txnType) (fsm.State, *txnState) { + _, ts := tc.createOpenState(typ) + return stateAborted{ + WasUpgraded: fsm.FromBool(typ == upgradedExplicitTxn), + }, ts } func (tc *testContext) createCommitWaitState() (fsm.State, *txnState, error) { @@ -298,7 +301,7 @@ func TestTransitions(t *testing.T) { ev: eventTxnStart{ImplicitTxn: fsm.True}, evPayload: makeEventTxnStartPayload(pri, tree.ReadWrite, timeutil.Now(), nil /* historicalTimestamp */, tranCtx, sessiondatapb.Normal), - expState: stateOpen{ImplicitTxn: fsm.True}, + expState: stateOpen{ImplicitTxn: fsm.True, WasUpgraded: fsm.False}, expAdv: expAdvance{ // We expect to stayInPlace; upon starting a txn the statement is // executed again, this time in state Open. @@ -323,7 +326,7 @@ func TestTransitions(t *testing.T) { ev: eventTxnStart{ImplicitTxn: fsm.False}, evPayload: makeEventTxnStartPayload(pri, tree.ReadWrite, timeutil.Now(), nil /* historicalTimestamp */, tranCtx, sessiondatapb.Normal), - expState: stateOpen{ImplicitTxn: fsm.False}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, expAdv: expAdvance{ expCode: advanceOne, expEv: txnStart, @@ -381,6 +384,27 @@ func TestTransitions(t *testing.T) { }, expTxn: nil, }, + { + // Finish an upgraded explicit txn. + name: "Open (upgraded explicit) -> NoTxn", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createOpenState(upgradedExplicitTxn) + // We commit the KV transaction, as that's done by the layer below + // txnState. + if err := ts.mu.txn.Commit(ts.Ctx); err != nil { + return nil, nil, emptyTxnID, err + } + return s, ts, ts.mu.txn.ID(), nil + }, + ev: eventTxnFinishCommitted{}, + evPayload: nil, + expState: stateNoTxn{}, + expAdv: expAdvance{ + expCode: advanceOne, + expEv: txnCommit, + }, + expTxn: nil, + }, { // Get a retriable error while we can auto-retry. name: "Open + auto-retry", @@ -395,7 +419,31 @@ func TestTransitions(t *testing.T) { } return eventRetriableErr{CanAutoRetry: fsm.True, IsCommit: fsm.False}, b }, - expState: stateOpen{ImplicitTxn: fsm.False}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, + expAdv: expAdvance{ + expCode: rewind, + expEv: txnRestart, + }, + // Expect non-nil txn. + expTxn: &expKVTxn{}, + expAutoRetryCounter: 1, + expAutoRetryError: "test retriable err", + }, + { + // Get a retriable error while we can auto-retry. + name: "Open + auto-retry on upgraded txn", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createOpenState(upgradedExplicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + evFun: func(ts *txnState) (fsm.Event, fsm.EventPayload) { + b := eventRetriableErrPayload{ + err: ts.mu.txn.PrepareRetryableError(ctx, "test retriable err"), + rewCap: dummyRewCap, + } + return eventRetriableErr{CanAutoRetry: fsm.True, IsCommit: fsm.False}, b + }, + expState: stateOpen{ImplicitTxn: fsm.True, WasUpgraded: fsm.False}, expAdv: expAdvance{ expCode: rewind, expEv: txnRestart, @@ -421,7 +469,32 @@ func TestTransitions(t *testing.T) { } return eventRetriableErr{CanAutoRetry: fsm.True, IsCommit: fsm.True}, b }, - expState: stateOpen{ImplicitTxn: fsm.False}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, + expAdv: expAdvance{ + expCode: rewind, + expEv: txnRestart, + }, + // Expect non-nil txn. + expTxn: &expKVTxn{}, + expAutoRetryCounter: 1, + }, + { + // Like the above test - get a retriable error while we can auto-retry, + // except this time the txn was upgraded to explicit. This shouldn't make + // any difference; we should still auto-retry like the above. + name: "Open + auto-retry (COMMIT) on upgraded txn", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createOpenState(upgradedExplicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + evFun: func(ts *txnState) (fsm.Event, fsm.EventPayload) { + b := eventRetriableErrPayload{ + err: ts.mu.txn.PrepareRetryableError(ctx, "test retriable err"), + rewCap: dummyRewCap, + } + return eventRetriableErr{CanAutoRetry: fsm.True, IsCommit: fsm.True}, b + }, + expState: stateOpen{ImplicitTxn: fsm.True, WasUpgraded: fsm.False}, expAdv: expAdvance{ expCode: rewind, expEv: txnRestart, @@ -445,7 +518,34 @@ func TestTransitions(t *testing.T) { } return eventRetriableErr{CanAutoRetry: fsm.False, IsCommit: fsm.False}, b }, - expState: stateAborted{}, + expState: stateAborted{ + WasUpgraded: fsm.False, + }, + expAdv: expAdvance{ + expCode: skipBatch, + expEv: noEvent, + }, + // Expect non-nil txn. + expTxn: &expKVTxn{}, + }, + { + // Get a retriable error when we can no longer auto-retry, but the client + // is doing client-side retries. + name: "Open (upgraded) + client retry", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createOpenState(upgradedExplicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + evFun: func(ts *txnState) (fsm.Event, fsm.EventPayload) { + b := eventRetriableErrPayload{ + err: ts.mu.txn.PrepareRetryableError(ctx, "test retriable err"), + rewCap: rewindCapability{}, + } + return eventRetriableErr{CanAutoRetry: fsm.False, IsCommit: fsm.False}, b + }, + expState: stateAborted{ + WasUpgraded: fsm.True, + }, expAdv: expAdvance{ expCode: skipBatch, expEv: noEvent, @@ -529,7 +629,27 @@ func TestTransitions(t *testing.T) { }, ev: eventNonRetriableErr{IsCommit: fsm.False}, evPayload: eventNonRetriableErrPayload{err: fmt.Errorf("test non-retriable err")}, - expState: stateAborted{}, + expState: stateAborted{ + WasUpgraded: fsm.False, + }, + expAdv: expAdvance{ + expCode: skipBatch, + expEv: noEvent, + }, + expTxn: &expKVTxn{}, + }, + { + // We get a non-retriable error. + name: "Open (upgraded) + non-retriable error", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createOpenState(upgradedExplicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + ev: eventNonRetriableErr{IsCommit: fsm.False}, + evPayload: eventNonRetriableErrPayload{err: fmt.Errorf("test non-retriable err")}, + expState: stateAborted{ + WasUpgraded: fsm.True, + }, expAdv: expAdvance{ expCode: skipBatch, expEv: noEvent, @@ -561,7 +681,24 @@ func TestTransitions(t *testing.T) { return s, ts, ts.mu.txn.ID(), nil }, ev: eventTxnRestart{}, - expState: stateOpen{ImplicitTxn: fsm.False}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, + expAdv: expAdvance{ + expCode: advanceOne, + expEv: txnRestart, + }, + // We would like to test that the transaction's epoch bumped if the txn + // performed any operations, but it's not easy to do the test. + expTxn: &expKVTxn{}, + }, + { + // Restarting from Open via ROLLBACK TO SAVEPOINT on an upgraded txn. + name: "Open (upgraded) + restart", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createOpenState(upgradedExplicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + ev: eventTxnRestart{}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.True}, expAdv: expAdvance{ expCode: advanceOne, expEv: txnRestart, @@ -577,7 +714,22 @@ func TestTransitions(t *testing.T) { // The txn finished, such as after a ROLLBACK. name: "Aborted->NoTxn", init: func() (fsm.State, *txnState, uuid.UUID, error) { - s, ts := testCon.createAbortedState() + s, ts := testCon.createAbortedState(explicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + ev: eventTxnFinishAborted{}, + expState: stateNoTxn{}, + expAdv: expAdvance{ + expCode: advanceOne, + expEv: txnRollback, + }, + expTxn: nil, + }, + { + // The upgraded txn finished, such as after a ROLLBACK. + name: "Aborted(upgraded)->NoTxn", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createAbortedState(upgradedExplicitTxn) return s, ts, ts.mu.txn.ID(), nil }, ev: eventTxnFinishAborted{}, @@ -592,11 +744,26 @@ func TestTransitions(t *testing.T) { // The txn is starting again (ROLLBACK TO SAVEPOINT while in Aborted). name: "Aborted->Open", init: func() (fsm.State, *txnState, uuid.UUID, error) { - s, ts := testCon.createAbortedState() + s, ts := testCon.createAbortedState(explicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + ev: eventSavepointRollback{}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, + expAdv: expAdvance{ + expCode: advanceOne, + expEv: noEvent, + }, + expTxn: &expKVTxn{}, + }, + { + // The upgraded txn is starting again (ROLLBACK TO SAVEPOINT while in Aborted). + name: "Aborted(upgraded)->Open", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createAbortedState(upgradedExplicitTxn) return s, ts, ts.mu.txn.ID(), nil }, ev: eventSavepointRollback{}, - expState: stateOpen{ImplicitTxn: fsm.False}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.True}, expAdv: expAdvance{ expCode: advanceOne, expEv: noEvent, @@ -607,11 +774,31 @@ func TestTransitions(t *testing.T) { // The txn is starting again (ROLLBACK TO SAVEPOINT cockroach_restart while in Aborted). name: "Aborted->Restart", init: func() (fsm.State, *txnState, uuid.UUID, error) { - s, ts := testCon.createAbortedState() + s, ts := testCon.createAbortedState(explicitTxn) return s, ts, ts.mu.txn.ID(), nil }, ev: eventTxnRestart{}, - expState: stateOpen{ImplicitTxn: fsm.False}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, + expAdv: expAdvance{ + expCode: advanceOne, + expEv: txnRestart, + }, + expTxn: &expKVTxn{ + userPriority: &pri, + writeTSNanos: &now.WallTime, + readTSNanos: &now.WallTime, + uncertaintyLimitNanos: &uncertaintyLimit.WallTime, + }, + }, + { + // The upgraded txn is starting again (ROLLBACK TO SAVEPOINT cockroach_restart while in Aborted). + name: "Aborted(upgraded)->Restart", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createAbortedState(upgradedExplicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + ev: eventTxnRestart{}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.True}, expAdv: expAdvance{ expCode: advanceOne, expEv: txnRestart, @@ -629,11 +816,30 @@ func TestTransitions(t *testing.T) { // to the expTxn. name: "Aborted->Starting (historical)", init: func() (fsm.State, *txnState, uuid.UUID, error) { - s, ts := testCon.createAbortedState() + s, ts := testCon.createAbortedState(explicitTxn) + return s, ts, ts.mu.txn.ID(), nil + }, + ev: eventTxnRestart{}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.False}, + expAdv: expAdvance{ + expCode: advanceOne, + expEv: txnRestart, + }, + expTxn: &expKVTxn{ + writeTSNanos: proto.Int64(now.WallTime), + }, + }, + { + // The upgraded txn is starting again (e.g. ROLLBACK TO SAVEPOINT while in Aborted). + // Verify that the historical timestamp from the evPayload is propagated + // to the expTxn. + name: "Aborted(upgraded)->Starting (historical)", + init: func() (fsm.State, *txnState, uuid.UUID, error) { + s, ts := testCon.createAbortedState(upgradedExplicitTxn) return s, ts, ts.mu.txn.ID(), nil }, ev: eventTxnRestart{}, - expState: stateOpen{ImplicitTxn: fsm.False}, + expState: stateOpen{ImplicitTxn: fsm.False, WasUpgraded: fsm.True}, expAdv: expAdvance{ expCode: advanceOne, expEv: txnRestart, diff --git a/pkg/sql/txnstatetransitions_diagram.gv b/pkg/sql/txnstatetransitions_diagram.gv index 64e31bc7d849..bdb0cf593d06 100644 --- a/pkg/sql/txnstatetransitions_diagram.gv +++ b/pkg/sql/txnstatetransitions_diagram.gv @@ -13,39 +13,58 @@ digraph finite_state_machine { qi -> "NoTxn{}"; node [shape = circle]; - "Aborted{}" -> "Aborted{}" [label = any other statement>] - "Aborted{}" -> "Aborted{}" [label = ConnExecutor closing>] - "Aborted{}" -> "Aborted{}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] - "Aborted{}" -> "Aborted{}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] - "Aborted{}" -> "Aborted{}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] - "Aborted{}" -> "Aborted{}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] - "Aborted{}" -> "Open{ImplicitTxn:false}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) success>] - "Aborted{}" -> "NoTxn{}" [label = ROLLBACK>] - "Aborted{}" -> "Open{ImplicitTxn:false}" [label = ROLLBACK TO SAVEPOINT cockroach_restart>] + "Aborted{WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = any other statement>] + "Aborted{WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = ConnExecutor closing>] + "Aborted{WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:false}" -> "Open{ImplicitTxn:false, WasUpgraded:false}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) success>] + "Aborted{WasUpgraded:false}" -> "NoTxn{}" [label = ROLLBACK>] + "Aborted{WasUpgraded:false}" -> "Open{ImplicitTxn:false, WasUpgraded:false}" [label = ROLLBACK TO SAVEPOINT cockroach_restart>] + "Aborted{WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = any other statement>] + "Aborted{WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = ConnExecutor closing>] + "Aborted{WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) failed because txn needs restart>] + "Aborted{WasUpgraded:true}" -> "Open{ImplicitTxn:false, WasUpgraded:true}" [label = ROLLBACK TO SAVEPOINT (not cockroach_restart) success>] + "Aborted{WasUpgraded:true}" -> "NoTxn{}" [label = ROLLBACK>] + "Aborted{WasUpgraded:true}" -> "Open{ImplicitTxn:false, WasUpgraded:true}" [label = ROLLBACK TO SAVEPOINT cockroach_restart>] "CommitWait{}" -> "CommitWait{}" [label = any other statement>] "CommitWait{}" -> "CommitWait{}" [label = any other statement>] "CommitWait{}" -> "NoTxn{}" [label = COMMIT>] "NoTxn{}" -> "NoTxn{}" [label = anything but BEGIN or extended protocol command error>] "NoTxn{}" -> "NoTxn{}" [label = anything but BEGIN or extended protocol command error>] - "NoTxn{}" -> "Open{ImplicitTxn:false}" [label = BEGIN, or before a statement running as an implicit txn>] - "NoTxn{}" -> "Open{ImplicitTxn:true}" [label = BEGIN, or before a statement running as an implicit txn>] - "Open{ImplicitTxn:false}" -> "Aborted{}" [label = "NonRetriableErr{IsCommit:false}"] - "Open{ImplicitTxn:false}" -> "NoTxn{}" [label = "NonRetriableErr{IsCommit:true}"] - "Open{ImplicitTxn:false}" -> "Aborted{}" [label = "RetriableErr{CanAutoRetry:false, IsCommit:false}"] - "Open{ImplicitTxn:false}" -> "NoTxn{}" [label = Retriable err on COMMIT>] - "Open{ImplicitTxn:false}" -> "Open{ImplicitTxn:false}" [label = Retriable err; will auto-retry>] - "Open{ImplicitTxn:false}" -> "Open{ImplicitTxn:false}" [label = Retriable err; will auto-retry>] - "Open{ImplicitTxn:false}" -> "NoTxn{}" [label = ROLLBACK, or after a statement running as an implicit txn fails>] - "Open{ImplicitTxn:false}" -> "NoTxn{}" [label = COMMIT, or after a statement running as an implicit txn>] - "Open{ImplicitTxn:false}" -> "CommitWait{}" [label = RELEASE SAVEPOINT cockroach_restart>] - "Open{ImplicitTxn:false}" -> "Open{ImplicitTxn:false}" [label = ROLLBACK TO SAVEPOINT cockroach_restart>] - "Open{ImplicitTxn:true}" -> "NoTxn{}" [label = "NonRetriableErr{IsCommit:false}"] - "Open{ImplicitTxn:true}" -> "NoTxn{}" [label = "NonRetriableErr{IsCommit:true}"] - "Open{ImplicitTxn:true}" -> "NoTxn{}" [label = "RetriableErr{CanAutoRetry:false, IsCommit:false}"] - "Open{ImplicitTxn:true}" -> "NoTxn{}" [label = Retriable err on COMMIT>] - "Open{ImplicitTxn:true}" -> "Open{ImplicitTxn:true}" [label = Retriable err; will auto-retry>] - "Open{ImplicitTxn:true}" -> "Open{ImplicitTxn:true}" [label = Retriable err; will auto-retry>] - "Open{ImplicitTxn:true}" -> "NoTxn{}" [label = ROLLBACK, or after a statement running as an implicit txn fails>] - "Open{ImplicitTxn:true}" -> "NoTxn{}" [label = COMMIT, or after a statement running as an implicit txn>] - "Open{ImplicitTxn:true}" -> "Open{ImplicitTxn:false}" [label = "TxnUpgradeToExplicit{}"] + "NoTxn{}" -> "Open{ImplicitTxn:false, WasUpgraded:false}" [label = BEGIN, or before a statement running as an implicit txn>] + "NoTxn{}" -> "Open{ImplicitTxn:true, WasUpgraded:false}" [label = BEGIN, or before a statement running as an implicit txn>] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = "NonRetriableErr{IsCommit:false}"] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "NoTxn{}" [label = "NonRetriableErr{IsCommit:true}"] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "Aborted{WasUpgraded:false}" [label = "RetriableErr{CanAutoRetry:false, IsCommit:false}"] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "NoTxn{}" [label = Retriable err on COMMIT>] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "Open{ImplicitTxn:false, WasUpgraded:false}" [label = Retriable err; will auto-retry>] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "Open{ImplicitTxn:false, WasUpgraded:false}" [label = Retriable err; will auto-retry>] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "NoTxn{}" [label = ROLLBACK, or after a statement running as an implicit txn fails>] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "NoTxn{}" [label = COMMIT, or after a statement running as an implicit txn>] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "CommitWait{}" [label = RELEASE SAVEPOINT cockroach_restart>] + "Open{ImplicitTxn:false, WasUpgraded:false}" -> "Open{ImplicitTxn:false, WasUpgraded:false}" [label = ROLLBACK TO SAVEPOINT cockroach_restart>] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = "NonRetriableErr{IsCommit:false}"] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "NoTxn{}" [label = "NonRetriableErr{IsCommit:true}"] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "Aborted{WasUpgraded:true}" [label = "RetriableErr{CanAutoRetry:false, IsCommit:false}"] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "NoTxn{}" [label = Retriable err on COMMIT>] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "Open{ImplicitTxn:true, WasUpgraded:false}" [label = Retriable err; will auto-retry>] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "Open{ImplicitTxn:true, WasUpgraded:false}" [label = Retriable err; will auto-retry>] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "NoTxn{}" [label = ROLLBACK, or after a statement running as an implicit txn fails>] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "NoTxn{}" [label = COMMIT, or after a statement running as an implicit txn>] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "CommitWait{}" [label = RELEASE SAVEPOINT cockroach_restart>] + "Open{ImplicitTxn:false, WasUpgraded:true}" -> "Open{ImplicitTxn:false, WasUpgraded:true}" [label = ROLLBACK TO SAVEPOINT cockroach_restart>] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "NoTxn{}" [label = "NonRetriableErr{IsCommit:false}"] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "NoTxn{}" [label = "NonRetriableErr{IsCommit:true}"] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "NoTxn{}" [label = "RetriableErr{CanAutoRetry:false, IsCommit:false}"] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "NoTxn{}" [label = Retriable err on COMMIT>] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "Open{ImplicitTxn:true, WasUpgraded:false}" [label = Retriable err; will auto-retry>] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "Open{ImplicitTxn:true, WasUpgraded:false}" [label = Retriable err; will auto-retry>] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "NoTxn{}" [label = ROLLBACK, or after a statement running as an implicit txn fails>] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "NoTxn{}" [label = COMMIT, or after a statement running as an implicit txn>] + "Open{ImplicitTxn:true, WasUpgraded:false}" -> "Open{ImplicitTxn:false, WasUpgraded:true}" [label = "TxnUpgradeToExplicit{}"] } diff --git a/pkg/sql/txnstatetransitions_report.txt b/pkg/sql/txnstatetransitions_report.txt index e04d7d82582b..40bad130717f 100644 --- a/pkg/sql/txnstatetransitions_report.txt +++ b/pkg/sql/txnstatetransitions_report.txt @@ -1,6 +1,23 @@ // Code generated; DO NOT EDIT. -Aborted{} +Aborted{WasUpgraded:false} + handled events: + NonRetriableErr{IsCommit:false} + NonRetriableErr{IsCommit:true} + RetriableErr{CanAutoRetry:false, IsCommit:false} + RetriableErr{CanAutoRetry:false, IsCommit:true} + RetriableErr{CanAutoRetry:true, IsCommit:false} + RetriableErr{CanAutoRetry:true, IsCommit:true} + SavepointRollback{} + TxnFinishAborted{} + TxnRestart{} + missing events: + TxnFinishCommitted{} + TxnReleased{} + TxnStart{ImplicitTxn:false} + TxnStart{ImplicitTxn:true} + TxnUpgradeToExplicit{} +Aborted{WasUpgraded:true} handled events: NonRetriableErr{IsCommit:false} NonRetriableErr{IsCommit:true} @@ -51,7 +68,24 @@ NoTxn{} TxnReleased{} TxnRestart{} TxnUpgradeToExplicit{} -Open{ImplicitTxn:false} +Open{ImplicitTxn:false, WasUpgraded:false} + handled events: + NonRetriableErr{IsCommit:false} + NonRetriableErr{IsCommit:true} + RetriableErr{CanAutoRetry:false, IsCommit:false} + RetriableErr{CanAutoRetry:false, IsCommit:true} + RetriableErr{CanAutoRetry:true, IsCommit:false} + RetriableErr{CanAutoRetry:true, IsCommit:true} + TxnFinishAborted{} + TxnFinishCommitted{} + TxnReleased{} + TxnRestart{} + missing events: + SavepointRollback{} + TxnStart{ImplicitTxn:false} + TxnStart{ImplicitTxn:true} + TxnUpgradeToExplicit{} +Open{ImplicitTxn:false, WasUpgraded:true} handled events: NonRetriableErr{IsCommit:false} NonRetriableErr{IsCommit:true} @@ -68,7 +102,7 @@ Open{ImplicitTxn:false} TxnStart{ImplicitTxn:false} TxnStart{ImplicitTxn:true} TxnUpgradeToExplicit{} -Open{ImplicitTxn:true} +Open{ImplicitTxn:true, WasUpgraded:false} handled events: NonRetriableErr{IsCommit:false} NonRetriableErr{IsCommit:true} @@ -85,3 +119,21 @@ Open{ImplicitTxn:true} TxnRestart{} TxnStart{ImplicitTxn:false} TxnStart{ImplicitTxn:true} +Open{ImplicitTxn:true, WasUpgraded:true} + unreachable! + handled events: + NonRetriableErr{IsCommit:true} + RetriableErr{CanAutoRetry:false, IsCommit:true} + TxnFinishAborted{} + TxnFinishCommitted{} + missing events: + NonRetriableErr{IsCommit:false} + RetriableErr{CanAutoRetry:false, IsCommit:false} + RetriableErr{CanAutoRetry:true, IsCommit:false} + RetriableErr{CanAutoRetry:true, IsCommit:true} + SavepointRollback{} + TxnReleased{} + TxnRestart{} + TxnStart{ImplicitTxn:false} + TxnStart{ImplicitTxn:true} + TxnUpgradeToExplicit{} diff --git a/pkg/sql/txntype_string.go b/pkg/sql/txntype_string.go index c940af8d5d1c..2f50054ae2cb 100644 --- a/pkg/sql/txntype_string.go +++ b/pkg/sql/txntype_string.go @@ -10,11 +10,12 @@ func _() { var x [1]struct{} _ = x[implicitTxn-0] _ = x[explicitTxn-1] + _ = x[upgradedExplicitTxn-2] } -const _txnType_name = "implicitTxnexplicitTxn" +const _txnType_name = "implicitTxnexplicitTxnupgradedExplicitTxn" -var _txnType_index = [...]uint8{0, 11, 22} +var _txnType_index = [...]uint8{0, 11, 22, 41} func (i txnType) String() string { if i < 0 || i >= txnType(len(_txnType_index)-1) { diff --git a/pkg/util/fsm/debug.go b/pkg/util/fsm/debug.go index e1584bb82ca8..5a396c4fdd3a 100644 --- a/pkg/util/fsm/debug.go +++ b/pkg/util/fsm/debug.go @@ -44,7 +44,7 @@ func (di debugInfo) reachable(sName string) bool { func typeName(i interface{}) string { s := fmt.Sprintf("%#v", i) parts := strings.Split(s, ".") - return parts[len(parts)-1] + return strings.Join(parts[1:], ".") } func trimState(s string) string { return strings.TrimPrefix(s, "state") } func trimEvent(s string) string { return strings.TrimPrefix(s, "event") }