diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index de0951454412..a92602ce1886 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -26,23 +26,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" + "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/errors" ) -var colInIndexNotSupportedErr = unimplemented.NewWithIssuef( - 47636, "ALTER COLUMN TYPE requiring rewrite of on-disk "+ - "data is currently not supported for columns that are part of an index") - -var colOwnsSequenceNotSupportedErr = unimplemented.NewWithIssuef( - 48244, "ALTER COLUMN TYPE for a column that owns a sequence "+ - "is currently not supported") - -var colWithConstraintNotSupportedErr = unimplemented.NewWithIssuef( - 48288, "ALTER COLUMN TYPE for a column that has a constraint "+ - "is currently not supported") - // AlterColTypeInTxnNotSupportedErr is returned when an ALTER COLUMN TYPE // is tried in an explicit transaction. var AlterColTypeInTxnNotSupportedErr = unimplemented.NewWithIssuef( @@ -177,14 +166,14 @@ func alterColumnTypeGeneral( // Disallow ALTER COLUMN TYPE general for columns that own sequences. if col.NumOwnsSequences() != 0 { - return colOwnsSequenceNotSupportedErr + return sqlerrors.NewAlterColumnTypeColOwnsSequenceNotSupportedErr() } // Disallow ALTER COLUMN TYPE general for columns that have a check // constraint. for _, ck := range tableDesc.EnforcedCheckConstraints() { if ck.CollectReferencedColumnIDs().Contains(col.GetID()) { - return colWithConstraintNotSupportedErr + return sqlerrors.NewAlterColumnTypeColWithConstraintNotSupportedErr() } } @@ -192,7 +181,7 @@ func alterColumnTypeGeneral( // UNIQUE WITHOUT INDEX constraint. for _, uc := range tableDesc.UniqueConstraintsWithoutIndex() { if uc.CollectKeyColumnIDs().Contains(col.GetID()) { - return colWithConstraintNotSupportedErr + return sqlerrors.NewAlterColumnTypeColWithConstraintNotSupportedErr() } } @@ -200,11 +189,13 @@ func alterColumnTypeGeneral( // constraint. for _, fk := range tableDesc.OutboundForeignKeys() { if fk.CollectOriginColumnIDs().Contains(col.GetID()) { - return colWithConstraintNotSupportedErr + return sqlerrors.NewAlterColumnTypeColWithConstraintNotSupportedErr() } + } + for _, fk := range tableDesc.InboundForeignKeys() { if fk.GetReferencedTableID() == tableDesc.GetID() && fk.CollectReferencedColumnIDs().Contains(col.GetID()) { - return colWithConstraintNotSupportedErr + return sqlerrors.NewAlterColumnTypeColWithConstraintNotSupportedErr() } } @@ -214,7 +205,7 @@ func alterColumnTypeGeneral( if idx.CollectKeyColumnIDs().Contains(col.GetID()) || idx.CollectKeySuffixColumnIDs().Contains(col.GetID()) || idx.CollectSecondaryStoredColumnIDs().Contains(col.GetID()) { - return colInIndexNotSupportedErr + return sqlerrors.NewAlterColumnTypeColInIndexNotSupportedErr() } } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_column_type b/pkg/sql/logictest/testdata/logic_test/alter_column_type index 9f99ca01a027..44657597c5ca 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_column_type +++ b/pkg/sql/logictest/testdata/logic_test/alter_column_type @@ -170,8 +170,8 @@ CREATE TABLE t1 (date string) statement ok INSERT INTO t1 VALUES ('hello') -statement error pq: ALTER COLUMN TYPE from string to timestamp is only supported experimentally -ALTER TABLE t1 ALTER COLUMN date TYPE timestamp +statement error pq: ALTER COLUMN TYPE from string to char is only supported experimentally +ALTER TABLE t1 ALTER COLUMN date TYPE CHAR(10) # After setting enable_experimental_alter_column_type_general, ALTER COLUMN TYPE should work. statement ok @@ -419,7 +419,7 @@ CREATE TABLE t18 (x INT NOT NULL PRIMARY KEY); statement ok CREATE TABLE t19 (y INT NOT NULL REFERENCES t18 (x), INDEX(y)); -statement error pq: unimplemented: ALTER COLUMN TYPE requiring rewrite of on-disk data is currently not supported for columns that are part of an index +statement error pq: unimplemented: ALTER COLUMN TYPE for a column that has a constraint is currently not supported ALTER TABLE t18 ALTER COLUMN x TYPE STRING statement error pq: unimplemented: ALTER COLUMN TYPE for a column that has a constraint is currently not supported @@ -872,4 +872,328 @@ statement ok DROP TABLE T_TTL_W_EXPIRE_AT; DROP TABLE T_TTL_W_DEFAULT; +subtest col_owns_sequence + +statement ok +CREATE TABLE T1_FOR_SEQ (C1 INT); + +statement ok +CREATE SEQUENCE SEQ1 OWNED BY T1_FOR_SEQ.C1; + +statement error pq: unimplemented: ALTER COLUMN TYPE for a column that owns a sequence is currently not supported +ALTER TABLE T1_FOR_SEQ ALTER COLUMN C1 SET DATA TYPE TEXT; + +statement ok +DROP TABLE T1_FOR_SEQ CASCADE; + +# Should fail because T1 owned the sequence and was dropped in the prior statement +statement error pq: relation "seq1" does not exist +DROP SEQUENCE SEQ1; + +subtest validation_only_bytes + +statement ok +CREATE TABLE t_bytes (c1 BYTES, c2 BYTES, c3 BYTES, FAMILY F1(c1,c2,c3, rowid)); + +statement ok +INSERT INTO t_bytes VALUES ('hello', 'world', 'worldhello'),(NULL,NULL,NULL); + +query TTT +SELECT * FROM t_bytes ORDER BY c1; +---- +NULL NULL NULL +hello world worldhello + +statement ok +ALTER TABLE t_bytes ALTER COLUMN c1 SET DATA TYPE STRING; + +# TODO(spilchen): This works in the legacy schema changer, but it ends up with +# wrong results. Once we query the table again we will see the data for c2 +# truncated. Leaving this here for now but will be addressed when we add +# validation only logic in the dsc. This will be done in issue #127516. +statement ok +ALTER TABLE t_bytes ALTER COLUMN c2 SET DATA TYPE CHAR(3); + +query TTT +SELECT * FROM t_bytes ORDER BY c1; +---- +NULL NULL NULL +\x68656c6c6f \x7 worldhello + +statement ok +UPDATE t_bytes SET c2 = 'w'; + +statement ok +ALTER TABLE t_bytes ALTER COLUMN c2 SET DATA TYPE CHAR(3); + +statement error pq: column "c3" cannot be cast automatically to type UUID\nHINT: You might need to specify "USING c3::UUID". +ALTER TABLE t_bytes ALTER COLUMN c3 SET DATA TYPE UUID; + +statement error pq: could not parse "worldhello" as type uuid: uuid: UUID must be exactly 16 bytes long, got 10 bytes +ALTER TABLE t_bytes ALTER COLUMN c3 SET DATA TYPE UUID USING c3::UUID; + +statement ok +UPDATE t_bytes SET c3='\x3b5692c80f7349ec91868f1478f3064a' WHERE c1 IS NOT NULL; + +statement ok +ALTER TABLE t_bytes ALTER COLUMN c3 SET DATA TYPE UUID USING c3::UUID; + +query TTT +SELECT * FROM t_bytes ORDER BY c1; +---- +NULL w NULL +\x68656c6c6f w 3b5692c8-0f73-49ec-9186-8f1478f3064a + +query TT +SHOW CREATE TABLE t_bytes; +---- +t_bytes CREATE TABLE public.t_bytes ( + c1 STRING NULL, + c2 CHAR(3) NULL, + c3 UUID NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t_bytes_pkey PRIMARY KEY (rowid ASC), + FAMILY f1 (c1, c2, c3, rowid) + ) + +statement ok +DROP TABLE t_bytes; + +subtest validation_only_decimal + +statement ok +CREATE TABLE t_decimal (c1 DECIMAL(20,5), c2 DECIMAL(10,5), FAMILY F1(c1,c2,rowid)); + +statement ok +INSERT INTO t_decimal VALUES (12345.6, 1.23456),(NULL,NULL),(100012.34,4563.21); + +query FF +SELECT * FROM t_decimal ORDER BY c1; +---- +NULL NULL +12345.6 1.23456 +100012.34 4563.21 + +# Reduce the overall precision of the decimal +statement error pq: type DECIMAL\(7,2\): value with precision 7, scale 2 must round to an absolute value less than 10\^5 +ALTER TABLE t_decimal ALTER COLUMN c1 SET DATA TYPE DECIMAL(7,2); + +# Reduce the overall precision of the decimal so that the alter will work +statement ok +UPDATE t_decimal SET c1 = 10012.34 WHERE c1 = 100012.34; + +statement ok +ALTER TABLE t_decimal ALTER COLUMN c1 SET DATA TYPE DECIMAL(7,2); + +# Reduce just the scale of the decimal +# +# TODO(spilchen): The legacy schema changer allows this, even though existing +# rows will not fit in the (10,2) decimal since the scale is too small for +# existing rows. It actually truncates the data, which doesn't seem right. This +# will get addressed when we add the validation only logic in #127516. +statement ok +ALTER TABLE t_decimal ALTER COLUMN c2 SET DATA TYPE DECIMAL(10,2); + +query FF +SELECT * FROM t_decimal ORDER BY c1; +---- +NULL NULL +10012.34 4563.21 +12345.6 1.23 + +statement ok +UPDATE t_decimal SET c2 = 1.23 WHERE c1 = 12345.6; + +statement ok +ALTER TABLE t_decimal ALTER COLUMN c2 SET DATA TYPE DECIMAL(10,2); + +query FF +SELECT * FROM t_decimal ORDER BY c1; +---- +NULL NULL +10012.34 4563.21 +12345.60 1.23 + +query TT +SHOW CREATE TABLE t_decimal; +---- +t_decimal CREATE TABLE public.t_decimal ( + c1 DECIMAL(7,2) NULL, + c2 DECIMAL(10,2) NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t_decimal_pkey PRIMARY KEY (rowid ASC), + FAMILY f1 (c1, c2, rowid) + ) + +statement ok +DROP TABLE t_decimal; + +subtest validation_only_bit_and_string + +statement ok +CREATE TABLE t_bit_string (pk INT PRIMARY KEY, c1 BIT(8), c2 VARBIT(8), c3 STRING, c4 CHAR(20), c5 VARCHAR(30), FAMILY F1(pk,c1,c2,c3,c4,c5)); + +statement ok +INSERT INTO t_bit_string VALUES (1,B'10101010', B'10101010', 'hello', 'world', 'worldhello'),(2,NULL,NULL,NULL,NULL,NULL); + +query TTTTT +SELECT c1,c2,c3,c4,c5 FROM t_bit_string ORDER BY pk; +---- +10101010 10101010 hello world worldhello +NULL NULL NULL NULL NULL + +statement error pq: unimplemented: ALTER COLUMN TYPE cannot be used in combination with other ALTER TABLE commands +ALTER TABLE t_bit_string ALTER COLUMN c1 SET DATA TYPE BIT(4), ALTER COLUMN c2 SET DATA TYPE VARBIT(4); + +# TODO(spilchen): The legacy schema changer allows this, even though existing +# rows will not fit in the new type. The data of existing rows gets truncated, +# which isn't correct. This will get addressed when we add the validation +# only logic in #127516. +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c1 SET DATA TYPE BIT(4); + +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c2 SET DATA TYPE VARBIT(4); + +query TTTTT +SELECT c1,c2,c3,c4,c5 FROM t_bit_string ORDER BY pk; +---- +1010 1010 hello world worldhello +NULL NULL NULL NULL NULL + +statement ok +UPDATE t_bit_string SET c1 = B'1010', c2 = B'1010' WHERE pk = 1; + +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c1 SET DATA TYPE BIT(4); + +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c2 SET DATA TYPE VARBIT(4); + +query TTTTT +SELECT c1,c2,c3,c4,c5 FROM t_bit_string ORDER BY pk; +---- +1010 1010 hello world worldhello +NULL NULL NULL NULL NULL + +statement error pq: column "c3" cannot be cast automatically to type BYTES\nHINT: You might need to specify "USING c3::BYTES". +ALTER TABLE t_bit_string ALTER COLUMN c3 SET DATA TYPE BYTES; + +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c3 SET DATA TYPE BYTES USING C3::BYTES; + +# TODO(spilchen): The legacy schema changer allows this, even though existing +# rows will not fit in the new type. We will address this in #127516. +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c4 SET DATA TYPE CHAR(4); + +query TTTTT +SELECT c1,c2,c3,c4,c5 FROM t_bit_string ORDER BY pk; +---- +1010 1010 hello worl worldhello +NULL NULL NULL NULL NULL + +statement ok +UPDATE t_bit_string SET c4 = 'worl' WHERE pk = 1; + +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c4 SET DATA TYPE VARCHAR(4); + +query TTTTT +SELECT c1,c2,c3,c4,c5 FROM t_bit_string ORDER BY pk; +---- +1010 1010 hello worl worldhello +NULL NULL NULL NULL NULL + +# Change c5 from VARCHAR(30) to CHAR(6) +# TODO(spilchen): this should be blocked when we add validation-only logic in #127516. +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c5 SET DATA TYPE CHAR(6); + +query TTTTT +SELECT c1,c2,c3,c4,c5 FROM t_bit_string ORDER BY pk; +---- +1010 1010 hello worl worldh +NULL NULL NULL NULL NULL + +statement ok +UPDATE t_bit_string SET c5 = 'worldh' WHERE pk = 1; + +statement ok +ALTER TABLE t_bit_string ALTER COLUMN c5 SET DATA TYPE CHAR(6); + +query TTTTT +SELECT c1,c2,c3,c4,c5 FROM t_bit_string ORDER BY pk; +---- +1010 1010 hello worl worldh +NULL NULL NULL NULL NULL + +query TT +SHOW CREATE TABLE t_bit_string; +---- +t_bit_string CREATE TABLE public.t_bit_string ( + pk INT8 NOT NULL, + c1 BIT(4) NULL, + c2 VARBIT(4) NULL, + c3 BYTES NULL, + c4 VARCHAR(4) NULL, + c5 CHAR(6) NULL, + CONSTRAINT t_bit_string_pkey PRIMARY KEY (pk ASC), + FAMILY f1 (pk, c1, c2, c3, c4, c5) + ) + +statement ok +DROP TABLE t_bit_string; + +subtest validation_only_int + +statement ok +CREATE TABLE t_int (pk INT PRIMARY KEY, c1 INT8, FAMILY F1(pk,c1)); + +statement ok +INSERT INTO t_int VALUES (1, 2147483648),(2,NULL); + +statement error pq: integer out of range for type int4 +ALTER TABLE t_int ALTER COLUMN c1 SET DATA TYPE INT4; + +statement ok +UPDATE t_int SET c1 = c1 - 1 WHERE pk = 1; + +query I +SELECT c1 FROM t_int ORDER BY pk; +---- +2147483647 +NULL + +statement ok +ALTER TABLE t_int ALTER COLUMN c1 SET DATA TYPE INT4; + +statement error pq: integer out of range for type int2 +ALTER TABLE t_int ALTER COLUMN c1 SET DATA TYPE INT2; + +statement ok +UPDATE t_int SET c1 = 32767 WHERE pk = 1; + +statement ok +ALTER TABLE t_int ALTER COLUMN c1 SET DATA TYPE INT2; + +query I +SELECT c1 FROM t_int ORDER BY pk; +---- +32767 +NULL + +query TT +SHOW CREATE TABLE t_int; +---- +t_int CREATE TABLE public.t_int ( + pk INT8 NOT NULL, + c1 INT2 NULL, + CONSTRAINT t_int_pkey PRIMARY KEY (pk ASC), + FAMILY f1 (pk, c1) + ) + +statement ok +DROP TABLE t_int; + subtest end diff --git a/pkg/sql/logictest/testdata/logic_test/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer index 0d20e3090828..609b4b18664c 100644 --- a/pkg/sql/logictest/testdata/logic_test/new_schema_changer +++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer @@ -183,9 +183,6 @@ j b statement ok SET use_declarative_schema_changer = 'unsafe' -statement error pgcode 0A000 cannot explain a statement which is not supported by the declarative schema changer -EXPLAIN (DDL) ALTER TABLE bar ALTER COLUMN j TYPE BOOL - statement ok DROP TABLE foo, bar diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go index f730504ea85c..aa4525ce31ed 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_alter_column_type.go @@ -11,6 +11,7 @@ package scbuildstmt import ( + "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -81,23 +82,22 @@ func alterTableAlterColumnType( validateAutomaticCastForNewType(b, tbl.TableID, colID, t.Column.String(), oldColType.Type, newColType.Type, t.Using != nil) - // We currently only support trivial conversions. Fallback to legacy schema - // changer if not trivial. kind, err := schemachange.ClassifyConversionFromTree(b, t, oldColType.Type, newColType.Type) if err != nil { panic(err) } - if kind != schemachange.ColumnConversionTrivial { - // TODO(spilchen): implement support for non-trivial type changes in issue #127014 + + switch kind { + case schemachange.ColumnConversionTrivial: + handleTrivialColumnConversion(b, oldColType, &newColType) + case schemachange.ColumnConversionValidate: + handleValidationOnlyColumnConversion(b, t, oldColType, &newColType) + case schemachange.ColumnConversionGeneral: + handleGeneralColumnConversion(b, t, col, oldColType, &newColType) + default: panic(scerrors.NotImplementedErrorf(t, - "alter type conversion not supported in the declarative schema changer")) + "alter type conversion %v not handled", kind)) } - - // Okay to proceed with DSC. Add the new type and remove the old type. The - // removal of the old type is a no-op in opgen. But we need the drop here so - // that we only have 1 public type for the column. - b.Drop(oldColType) - b.Add(&newColType) } // ValidateColExprForNewType will ensure that the existing expressions for @@ -110,6 +110,32 @@ func validateAutomaticCastForNewType( fromType, toType *types.T, hasUsingExpr bool, ) { + if validCast := cast.ValidCast(fromType, toType, cast.ContextAssignment); validCast { + return + } + + // If the USING expression is missing, we will report an error with a + // suggested hint to use one. + if !hasUsingExpr { + // Compute a suggested default computed expression for inclusion in the error hint. + hintExpr := tree.CastExpr{ + Expr: &tree.ColumnItem{ColumnName: tree.Name(colName)}, + Type: toType, + SyntaxMode: tree.CastShort, + } + panic(errors.WithHintf( + pgerror.Newf( + pgcode.DatatypeMismatch, + "column %q cannot be cast automatically to type %s", + colName, + toType.SQLString(), + ), "You might need to specify \"USING %s\".", tree.Serialize(&hintExpr), + )) + } + + // We have a USING clause, but if we have DEFAULT or ON UPDATE expressions, + // then we raise an error because those expressions cannot be automatically + // cast to the new type. columnElements(b, tableID, colID).ForEach(func( _ scpb.Status, _ scpb.TargetStatus, e scpb.Element, ) { @@ -119,36 +145,84 @@ func validateAutomaticCastForNewType( exprType = "default" case *scpb.ColumnOnUpdateExpression: exprType = "on update" + default: + return } - if exprType != "" { - if validCast := cast.ValidCast(fromType, toType, cast.ContextAssignment); !validCast { - // If the USING expression is missing, we will report an error with a - // suggested hint to use one. This is mainly done for compatability - // with the legacy schema changer. - if !hasUsingExpr { - // Compute a suggested default computed expression for inclusion in the error hint. - hintExpr := tree.CastExpr{ - Expr: &tree.ColumnItem{ColumnName: tree.Name(colName)}, - Type: toType, - SyntaxMode: tree.CastShort, - } - panic(errors.WithHintf( - pgerror.Newf( - pgcode.DatatypeMismatch, - "column %q cannot be cast automatically to type %s", - colName, - toType.SQLString(), - ), "You might need to specify \"USING %s\".", tree.Serialize(&hintExpr), - )) - } - panic(pgerror.Newf( - pgcode.DatatypeMismatch, - "%s for column %q cannot be cast automatically to type %s", - exprType, - colName, - toType.SQLString(), - )) - } + panic(pgerror.Newf( + pgcode.DatatypeMismatch, + "%s for column %q cannot be cast automatically to type %s", + exprType, + colName, + toType.SQLString(), + )) + }) +} + +// handleTrivialColumnConversion is called to just change the type in-place without +// no rewrite or validation required. +func handleTrivialColumnConversion(b BuildCtx, oldColType, newColType *scpb.ColumnType) { + // Add the new type and remove the old type. The removal of the old type is a + // no-op in opgen. But we need the drop here so that we only have 1 public + // type for the column. + b.Drop(oldColType) + b.Add(newColType) +} + +// handleValidationOnlyColumnConversion is called when we don't need to rewrite +// data, only validate the existing data is compatible with the type. +func handleValidationOnlyColumnConversion( + b BuildCtx, t *tree.AlterTableAlterColumnType, oldColType, newColType *scpb.ColumnType, +) { + failIfExperimentalSettingNotSet(b, oldColType, newColType) + + // TODO(spilchen): Implement the validation-only logic in #127516 + panic(scerrors.NotImplementedErrorf(t, + "alter type conversion that requires validation only is not supported in the declarative schema changer")) +} + +// handleGeneralColumnConversion is called when we need to rewrite the data in order +// to complete the data type conversion. +func handleGeneralColumnConversion( + b BuildCtx, + t *tree.AlterTableAlterColumnType, + col *scpb.Column, + oldColType, newColType *scpb.ColumnType, +) { + failIfExperimentalSettingNotSet(b, oldColType, newColType) + + // Because we need to rewrite data to change the data type, there are + // additional validation checks required that are incompatible with this + // process. + walkColumnDependencies(b, col, "alter type of", "column", func(e scpb.Element, op, objType string) { + switch e.(type) { + case *scpb.SequenceOwner: + panic(sqlerrors.NewAlterColumnTypeColOwnsSequenceNotSupportedErr()) + case *scpb.CheckConstraint, *scpb.CheckConstraintUnvalidated, + *scpb.UniqueWithoutIndexConstraint, *scpb.UniqueWithoutIndexConstraintUnvalidated, + *scpb.ForeignKeyConstraint, *scpb.ForeignKeyConstraintUnvalidated: + panic(sqlerrors.NewAlterColumnTypeColWithConstraintNotSupportedErr()) + case *scpb.SecondaryIndex: + panic(sqlerrors.NewAlterColumnTypeColInIndexNotSupportedErr()) } }) + + // TODO(spilchen): Implement the general conversion logic in #127014 + panic(scerrors.NotImplementedErrorf(t, "general alter type conversion not supported in the declarative schema changer")) +} + +// failIfExperimentalSettingNotSet checks if the setting that allows altering +// types is enabled. If the setting is not enabled, this function will panic. +func failIfExperimentalSettingNotSet(b BuildCtx, oldColType, newColType *scpb.ColumnType) { + if !b.SessionData().AlterColumnTypeGeneralEnabled { + panic(pgerror.WithCandidateCode( + errors.WithHint( + errors.WithIssueLink( + errors.Newf("ALTER COLUMN TYPE from %v to %v is only "+ + "supported experimentally", + oldColType.Type, newColType.Type), + errors.IssueLink{IssueURL: build.MakeIssueURL(49329)}), + "you can enable alter column type general support by running "+ + "`SET enable_experimental_alter_column_type_general = true`"), + pgcode.ExperimentalFeature)) + } } diff --git a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table index 96879fa562f0..ecc469652668 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table +++ b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table @@ -16,10 +16,6 @@ unimplemented ALTER TABLE defaultdb.foo ADD COLUMN j SERIAL ---- -unimplemented -ALTER TABLE defaultdb.foo ALTER COLUMN i SET DATA TYPE STRING ----- - unimplemented ALTER TABLE defaultdb.foo ALTER COLUMN i DROP NOT NULL ---- diff --git a/pkg/sql/sqlerrors/BUILD.bazel b/pkg/sql/sqlerrors/BUILD.bazel index 33711eeb67e6..4dc2e1edd729 100644 --- a/pkg/sql/sqlerrors/BUILD.bazel +++ b/pkg/sql/sqlerrors/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/sql/sem/catconstants", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/util/errorutil/unimplemented", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", ], diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index 33ae806ed70b..3a89af0fd731 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "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" "github.com/cockroachdb/redact" ) @@ -73,6 +74,24 @@ func NewNonNullViolationError(columnName string) error { return pgerror.Newf(pgcode.NotNullViolation, "null value in column %q violates not-null constraint", columnName) } +func NewAlterColumnTypeColOwnsSequenceNotSupportedErr() error { + return unimplemented.NewWithIssuef( + 48244, "ALTER COLUMN TYPE for a column that owns a sequence "+ + "is currently not supported") +} + +func NewAlterColumnTypeColWithConstraintNotSupportedErr() error { + return unimplemented.NewWithIssuef( + 48288, "ALTER COLUMN TYPE for a column that has a constraint "+ + "is currently not supported") +} + +func NewAlterColumnTypeColInIndexNotSupportedErr() error { + return unimplemented.NewWithIssuef( + 47636, "ALTER COLUMN TYPE requiring rewrite of on-disk "+ + "data is currently not supported for columns that are part of an index") +} + // NewInvalidAssignmentCastError creates an error that is used when a mutation // cannot be performed because there is not a valid assignment cast from a // value's type to the type of the target column.