diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index 17b942777af0..efb0fd59fee8 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -226,6 +226,16 @@ func alterColumnTypeGeneral( } } + // The algorithm relies heavily on the computed column expression, so we don’t + // support altering a column if it’s also computed. There’s currently no way to + // track the original expression in this case. However, this is supported in the + // declarative schema changer. + if col.IsComputed() { + return unimplemented.Newf("ALTER COLUMN ... TYPE", + "ALTER COLUMN TYPE requiring an on-disk data rewrite with the legacy schema changer "+ + "is not supported for computed columns") + } + nameExists := func(name string) bool { return catalog.FindColumnByName(tableDesc, name) != nil } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_column_type b/pkg/sql/logictest/testdata/logic_test/alter_column_type index 840020c5de6b..372e68286637 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_column_type +++ b/pkg/sql/logictest/testdata/logic_test/alter_column_type @@ -1347,4 +1347,147 @@ rowid INT8 false unique_rowid() · {t_many_pkey} true statement ok DROP TABLE t_many; +subtest stored_compute + +statement ok +CREATE TABLE stored1 (A INT NOT NULL PRIMARY KEY, COMP1 SMALLINT NOT NULL AS (A) STORED, FAMILY F1(A, COMP1)); + +statement ok +INSERT INTO stored1 VALUES (10),(150),(190),(2000); + +query II +SELECT * FROM stored1 ORDER BY A; +---- +10 10 +150 150 +190 190 +2000 2000 + +# Do trivial column change (SMALLINT -> BIGINT) +statement ok +ALTER TABLE stored1 ALTER COLUMN COMP1 SET DATA TYPE BIGINT; + +query TT +SHOW CREATE TABLE stored1; +---- +stored1 CREATE TABLE public.stored1 ( + a INT8 NOT NULL, + comp1 INT8 NOT NULL AS (a) STORED, + CONSTRAINT stored1_pkey PRIMARY KEY (a ASC), + FAMILY f1 (a, comp1) + ) + +query II +SELECT * FROM stored1 ORDER BY A; +---- +10 10 +150 150 +190 190 +2000 2000 + +# Do validation-only change (BIGINT -> INT4). But first insert a row that will violate INT4. +statement ok +INSERT INTO stored1 VALUES (2147483648),(2147483647); + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +statement error pq: validate check constraint: integer out of range for type int4 +ALTER TABLE stored1 ALTER COLUMN COMP1 SET DATA TYPE INT4; + +# Legacy blocks the type conversion entirely for computed columns +onlyif config local-legacy-schema-changer +statement error pq: unimplemented: ALTER COLUMN TYPE requiring an on-disk data rewrite with the legacy schema changer is not supported for computed columns +ALTER TABLE stored1 ALTER COLUMN COMP1 SET DATA TYPE INT4; + +statement ok +DELETE FROM stored1 WHERE a = 2147483648; + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +statement ok +ALTER TABLE stored1 ALTER COLUMN COMP1 SET DATA TYPE INT4; + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +query TT +SHOW CREATE TABLE stored1; +---- +stored1 CREATE TABLE public.stored1 ( + a INT8 NOT NULL, + comp1 INT4 NOT NULL AS (a) STORED, + CONSTRAINT stored1_pkey PRIMARY KEY (a ASC), + FAMILY f1 (a, comp1) + ) + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +query II +SELECT * FROM stored1 ORDER BY A; +---- +10 10 +150 150 +190 190 +2000 2000 +2147483647 2147483647 + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +skipif config local-mixed-24.2 +# Attempt to convert to a type that is incompatible with the computed expression +statement error pq: expected STORED COMPUTED COLUMN expression to have type bool, but 'a' has type int +ALTER TABLE stored1 ALTER COLUMN comp1 SET DATA TYPE BOOL; + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +skipif config local-mixed-24.2 +statement error pq: expected STORED COMPUTED COLUMN expression to have type string, but 'a' has type int +ALTER TABLE stored1 ALTER COLUMN comp1 SET DATA TYPE TEXT; + +# Convert the type to something compatible, but specify a custom value for the +# column with the USING expression. This will force a type conversion with a backfill. +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +skipif config local-mixed-24.2 +statement ok +ALTER TABLE stored1 ALTER COLUMN comp1 SET DATA TYPE INT2 USING -1; + +statement ok +INSERT INTO stored1 VALUES (-1000); + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +skipif config local-mixed-24.2 +query TT +SHOW CREATE TABLE stored1; +---- +stored1 CREATE TABLE public.stored1 ( + a INT8 NOT NULL, + comp1 INT2 NOT NULL AS (a) STORED, + CONSTRAINT stored1_pkey PRIMARY KEY (a ASC), + FAMILY f1 (a, comp1) + ) + +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +skipif config local-mixed-24.2 +query II +SELECT * FROM stored1 ORDER BY A; +---- +-1000 -1000 +10 -1 +150 -1 +190 -1 +2000 -1 +2147483647 -1 + +# Attempt to drop stored along with changing the column type +skipif config local-legacy-schema-changer +skipif config local-mixed-24.1 +skipif config local-mixed-24.2 +statement error pq: unimplemented: ALTER COLUMN TYPE cannot be used in combination with other ALTER TABLE commands +ALTER TABLE stored1 ALTER COLUMN comp1 SET DATA TYPE INT4 USING -1, ALTER COLUMN comp1 drop stored; + +statement ok +DROP TABLE stored1; + subtest end diff --git a/pkg/sql/schemachanger/dml_injection_test.go b/pkg/sql/schemachanger/dml_injection_test.go index 50f30650ea71..7e1122cf3503 100644 --- a/pkg/sql/schemachanger/dml_injection_test.go +++ b/pkg/sql/schemachanger/dml_injection_test.go @@ -188,6 +188,16 @@ func TestAlterTableDMLInjection(t *testing.T) { schemaChange: "ALTER TABLE tbl ALTER COLUMN new_col SET DATA TYPE TEXT", query: "SELECT new_col FROM tbl LIMIT 1", }, + { + desc: "alter column type general compute", + setup: []string{ + "SET enable_experimental_alter_column_type_general=TRUE", + "ALTER TABLE tbl ADD COLUMN new_col DATE NOT NULL DEFAULT '2013-05-06', " + + "ADD COLUMN new_comp DATE AS (new_col) STORED", + }, + schemaChange: "ALTER TABLE tbl ALTER COLUMN new_comp SET DATA TYPE DATE USING '2021-05-06'", + query: "SELECT new_comp FROM tbl LIMIT 1", + }, { desc: "add column default udf", setup: []string{"CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$"}, diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go index c3b7f449c19f..748e8083d52c 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_column.go @@ -319,10 +319,10 @@ type addColumnSpec struct { def *scpb.ColumnDefaultExpression onUpdate *scpb.ColumnOnUpdateExpression compute *scpb.ColumnComputeExpression + transientCompute *scpb.ColumnComputeExpression comment *scpb.ColumnComment unique bool notNull bool - transientCompute bool } // addColumn adds a column as specified in the `spec`. It delegates most of the work @@ -349,11 +349,10 @@ func addColumn(b BuildCtx, spec addColumnSpec, n tree.NodeFormatter) (backing *s b.Add(spec.onUpdate) } if spec.compute != nil { - if spec.transientCompute { - b.AddTransient(spec.compute) - } else { - b.Add(spec.compute) - } + b.Add(spec.compute) + } + if spec.transientCompute != nil { + b.AddTransient(spec.transientCompute) } if spec.comment != nil { b.Add(spec.comment) @@ -364,7 +363,7 @@ func addColumn(b BuildCtx, spec addColumnSpec, n tree.NodeFormatter) (backing *s } inflatedChain := getInflatedPrimaryIndexChain(b, spec.tbl.TableID) - if spec.def == nil && spec.colType.ComputeExpr == nil && spec.compute == nil { + if spec.def == nil && spec.colType.ComputeExpr == nil && spec.compute == nil && spec.transientCompute == nil { // Optimization opportunity: if we were to add a new column without default // value nor computed expression, then we can just add the column to existing // non-nil primary indexes without actually backfilling any data. This is 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 a34de65a447c..8e169c64875d 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 @@ -88,6 +88,7 @@ func alterTableAlterColumnType( panic(err) } + validateNewTypeForComputedColumn(b, tbl.TableID, colID, tn, newColType.Type) validateAutomaticCastForNewType(b, tbl.TableID, colID, t.Column.String(), oldColType.Type, newColType.Type, t.Using != nil) @@ -166,6 +167,47 @@ func validateAutomaticCastForNewType( }) } +// validateNewTypeForComputedColumn will check if the new type is valid for a +// computed column. +func validateNewTypeForComputedColumn( + b BuildCtx, tableID catid.DescID, colID catid.ColumnID, tn *tree.TableName, toType *types.T, +) { + colComputeExpression := b.QueryByID(tableID).FilterColumnComputeExpression().Filter( + func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.ColumnComputeExpression) bool { + return e.ColumnID == colID + }).MustGetZeroOrOneElement() + // Early out if the column isn't computed. + if colComputeExpression == nil { + return + } + + // The use case for type changes on computed columns is quite limited. The new type + // generally needs to belong to the same type family (e.g., INT2 -> INT4). This is because + // the computed expression itself isn’t changing, so it continues to return the same type. + // As a result, the old and new types must be compatible with each other. We use + // DequalifyAndValidateExprImpl to enforce this compatibility, and we only check for any + // errors returned by that call. + // + // Now, create a tree.Expr for the computed expression. + expr, err := parser.ParseExpr(string(colComputeExpression.Expression.Expr)) + if err != nil { + panic(err) + } + + _, _, _, err = schemaexpr.DequalifyAndValidateExprImpl(b, expr, toType, + tree.StoredComputedColumnExpr, b.SemaCtx(), volatility.Volatile, tn, b.ClusterSettings().Version.ActiveVersion(b), + func() colinfo.ResultColumns { + return getNonDropResultColumns(b, tableID) + }, + func(columnName tree.Name) (exists bool, accessible bool, id catid.ColumnID, typ *types.T) { + return columnLookupFn(b, tableID, columnName) + }, + ) + if err != nil { + panic(err) + } +} + // handleTrivialColumnConversion is called to just change the type in-place without // no rewrite or validation required. func handleTrivialColumnConversion( @@ -240,14 +282,15 @@ func handleGeneralColumnConversion( panic(sqlerrors.NewAlterColumnTypeColWithConstraintNotSupportedErr()) case *scpb.SecondaryIndex: panic(sqlerrors.NewAlterColumnTypeColInIndexNotSupportedErr()) - case *scpb.ColumnComputeExpression: - // TODO(#125844): we currently lose the original computed expression. - panic(scerrors.NotImplementedErrorf(t, - "backfilling during ALTER COLUMN TYPE for a column "+ - "with a computed expression is not supported")) } }) + if oldColType.IsVirtual { + // TODO(#125840): we currently don't support altering the type of a virtual column + panic(scerrors.NotImplementedErrorf(t, + "backfilling during ALTER COLUMN TYPE for a virtual column is not supported")) + } + // We block any attempt to alter the type of a column that is a key column in // the primary key. We can't use walkColumnDependencies here, as it doesn't // differentiate between key columns and stored columns. @@ -299,7 +342,8 @@ func handleGeneralColumnConversion( oldDefExpr, newDefExpr := getColumnDefaultExpressionsForColumnReplacement(b, tbl.TableID, col.ColumnID, newColID) oldOnUpdateExpr, newOnUpdateExpr := getColumnOnUpdateExpressionsForColumnReplacement(b, tbl.TableID, col.ColumnID, newColID) - oldColComment, newColComment := getColumnCommentForColumnReplacement(b, tbl.TableID, col.ColumnID, newColID) + oldComputeExpr, newComputeExpr := getColumnComputeExpressionsForColumnReplacement(b, tbl.TableID, col.ColumnID, newColID) + oldColComment, newColComment := getColumnCommentForColumnReplacement(b, tbl.TableID, col.ColumnID) // First, set the target status of the old column to drop. This column will be // replaced by a new one but remains visible until the new column is ready to be @@ -308,6 +352,9 @@ func handleGeneralColumnConversion( b.Drop(col) b.Drop(colName) b.Drop(oldColType) + if oldComputeExpr != nil { + b.Drop(oldComputeExpr) + } if oldDefExpr != nil { b.Drop(oldDefExpr) } @@ -358,14 +405,14 @@ func handleGeneralColumnConversion( onUpdate: newOnUpdateExpr, comment: newColComment, colType: newColType, - compute: &scpb.ColumnComputeExpression{ + compute: newComputeExpr, + transientCompute: &scpb.ColumnComputeExpression{ TableID: tbl.TableID, ColumnID: newColID, Expression: *b.WrapExpression(tbl.TableID, expr), Usage: scpb.ColumnComputeExpression_ALTER_TYPE_USING, }, - transientCompute: true, - notNull: retrieveColumnNotNull(b, tbl.TableID, col.ColumnID) != nil, + notNull: retrieveColumnNotNull(b, tbl.TableID, col.ColumnID) != nil, // The new column will be placed in the same column family as the one // it's replacing, so there's no need to specify a family. fam: nil, @@ -512,11 +559,27 @@ func getColumnOnUpdateExpressionsForColumnReplacement( return } +// getColumnComputeExpressionsForColumnReplacement returns both old and new ColumnComputeExpressions +// for a column type conversion needing a backfill. +func getColumnComputeExpressionsForColumnReplacement( + b BuildCtx, tableID catid.DescID, oldColID, newColID catid.ColumnID, +) (oldComputeExpr, newComputeExpr *scpb.ColumnComputeExpression) { + oldComputeExpr = b.QueryByID(tableID).FilterColumnComputeExpression().Filter( + func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.ColumnComputeExpression) bool { + return e.ColumnID == oldColID + }).MustGetZeroOrOneElement() + if oldComputeExpr != nil { + newComputeExpr = protoutil.Clone(oldComputeExpr).(*scpb.ColumnComputeExpression) + newComputeExpr.ColumnID = newColID + } + return +} + // getColumnCommentForColumnReplacement returns two versions of ColumnComment when // replacing a column: one for the old column and one for the new column. If no // column comment exists, both output parameters will be nil. func getColumnCommentForColumnReplacement( - b BuildCtx, tableID catid.DescID, oldColID, newColID catid.ColumnID, + b BuildCtx, tableID catid.DescID, oldColID catid.ColumnID, ) (oldColumnComment, newColumnComment *scpb.ColumnComment) { oldColumnComment = retrieveColumnComment(b, tableID, oldColID) if oldColumnComment != nil { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go index 2a6127a97892..b6936d9978a1 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go @@ -186,6 +186,24 @@ func init() { } }, ) + + registerDepRule( + "Final compute expression is always added after transient compute expression", + scgraph.SameStagePrecedence, + "transient-compute-expression", "final-compute-expression", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.ColumnComputeExpression)(nil)), + to.Type((*scpb.ColumnComputeExpression)(nil)), + JoinOnColumnID(from, to, "table-id", "col-id"), + from.El.AttrEq(screl.Usage, scpb.ColumnComputeExpression_ALTER_TYPE_USING), + to.El.AttrEq(screl.Usage, scpb.ColumnComputeExpression_REGULAR), + ToPublicOrTransient(from, to), + from.CurrentStatus(scpb.Status_TRANSIENT_ABSENT), + to.CurrentStatus(scpb.Status_PUBLIC), + } + }, + ) } // This rule ensures that columns depend on each other in increasing order. diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index f81856c46fe4..ad843541592c 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -763,6 +763,21 @@ deprules - $column-expr-Node[CurrentStatus] = PUBLIC - joinTargetNode($transient-compute-expression, $transient-compute-expression-Target, $transient-compute-expression-Node) - joinTargetNode($column-expr, $column-expr-Target, $column-expr-Node) +- name: Final compute expression is always added after transient compute expression + from: transient-compute-expression-Node + kind: SameStagePrecedence + to: final-compute-expression-Node + query: + - $transient-compute-expression[Type] = '*scpb.ColumnComputeExpression' + - $final-compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($transient-compute-expression, $final-compute-expression, $table-id, $col-id) + - $transient-compute-expression[Usage] = ALTER_TYPE_USING + - $final-compute-expression[Usage] = REGULAR + - ToPublicOrTransient($transient-compute-expression-Target, $final-compute-expression-Target) + - $transient-compute-expression-Node[CurrentStatus] = TRANSIENT_ABSENT + - $final-compute-expression-Node[CurrentStatus] = PUBLIC + - joinTargetNode($transient-compute-expression, $transient-compute-expression-Target, $transient-compute-expression-Node) + - joinTargetNode($final-compute-expression, $final-compute-expression-Target, $final-compute-expression-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' from: prev-Node kind: PreviousTransactionPrecedence @@ -5337,6 +5352,21 @@ deprules - $column-expr-Node[CurrentStatus] = PUBLIC - joinTargetNode($transient-compute-expression, $transient-compute-expression-Target, $transient-compute-expression-Node) - joinTargetNode($column-expr, $column-expr-Target, $column-expr-Node) +- name: Final compute expression is always added after transient compute expression + from: transient-compute-expression-Node + kind: SameStagePrecedence + to: final-compute-expression-Node + query: + - $transient-compute-expression[Type] = '*scpb.ColumnComputeExpression' + - $final-compute-expression[Type] = '*scpb.ColumnComputeExpression' + - joinOnColumnID($transient-compute-expression, $final-compute-expression, $table-id, $col-id) + - $transient-compute-expression[Usage] = ALTER_TYPE_USING + - $final-compute-expression[Usage] = REGULAR + - ToPublicOrTransient($transient-compute-expression-Target, $final-compute-expression-Target) + - $transient-compute-expression-Node[CurrentStatus] = TRANSIENT_ABSENT + - $final-compute-expression-Node[CurrentStatus] = PUBLIC + - joinTargetNode($transient-compute-expression, $transient-compute-expression-Target, $transient-compute-expression-Node) + - joinTargetNode($final-compute-expression, $final-compute-expression-Target, $final-compute-expression-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' from: prev-Node kind: PreviousTransactionPrecedence