Skip to content

Commit

Permalink
Merge #135067
Browse files Browse the repository at this point in the history
135067: sql/schemachanger: Support type changes for stored computed columns r=spilchen a=spilchen

This update adds support for type changes on columns with stored computed expressions in the declarative schema change (DSC). Previously, these type changes were blocked and would fall back to the legacy schema changer, which didn't fully support computed columns as it would lose the expression. A block has now been added to the legacy schema changer to prevent unsupported changes since it is deprecated.

In the DSC, a new dependency rule ensures that the compute expression is added after the temporary compute expression (used during the backfill to copy data from the old column to the new one) completes.

This change is only for stored computed columns. We have a follow-on issue to address this for virtual computed columns.

Note, changing the type of a computed column has limited applicability, as the expression itself remains unchanged. Therefore, the new type must be compatible with the original type, often meaning it belongs to the same type family (e.g., INT2 -> INT4).

Epic: CRDB-25314
Closes #125844
Release note: none

Co-authored-by: Matt Spilchen <[email protected]>
  • Loading branch information
craig[bot] and spilchen committed Nov 15, 2024
2 parents d06e762 + 181fa5d commit cd7b297
Show file tree
Hide file tree
Showing 7 changed files with 290 additions and 17 deletions.
10 changes: 10 additions & 0 deletions pkg/sql/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
143 changes: 143 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_column_type
Original file line number Diff line number Diff line change
Expand Up @@ -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
10 changes: 10 additions & 0 deletions pkg/sql/schemachanger/dml_injection_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 $$"},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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)
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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
Expand All @@ -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)
}
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
Loading

0 comments on commit cd7b297

Please sign in to comment.