Skip to content

Commit

Permalink
sql/schemachanger: Enable column type changes for virtual computed co…
Browse files Browse the repository at this point in the history
…lumns

This update introduces support for altering the column type of virtual
computed columns in the declarative schema changer (DSC). Previously,
such operations were not allowed.

A key characteristic of virtual columns is that type changes never
require a backfill or column rewrite, as their values are computed
dynamically during access. This change includes adjustments to handle
type changes that would typically involve a rewrite as simple metadata
updates instead.

Additionally, the USING expression in ALTER TABLE statements is now
explicitly disallowed for virtual columns, as no data rewriting occurs.
Attempts to use this option will result in an error.

Epic: CRDB-25314
Closes: #125840
Release note: none
  • Loading branch information
spilchen committed Nov 15, 2024
1 parent cd7b297 commit e2b5dd6
Show file tree
Hide file tree
Showing 8 changed files with 273 additions and 37 deletions.
4 changes: 2 additions & 2 deletions pkg/sql/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,12 +77,12 @@ func AlterColumnType(
}

typ, err = schemachange.ValidateAlterColumnTypeChecks(ctx, t,
params.EvalContext().Settings, typ, col.IsGeneratedAsIdentity())
params.EvalContext().Settings, typ, col.IsGeneratedAsIdentity(), col.IsVirtual())
if err != nil {
return err
}

kind, err := schemachange.ClassifyConversionFromTree(ctx, t, col.GetType(), typ)
kind, err := schemachange.ClassifyConversionFromTree(ctx, t, col.GetType(), typ, col.IsVirtual())
if err != nil {
return err
}
Expand Down
139 changes: 139 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_column_type
Original file line number Diff line number Diff line change
Expand Up @@ -1490,4 +1490,143 @@ ALTER TABLE stored1 ALTER COLUMN comp1 SET DATA TYPE INT4 USING -1, ALTER COLUMN
statement ok
DROP TABLE stored1;

subtest virtual_compute

statement ok
CREATE TABLE virt1 (c1 BIGINT NOT NULL PRIMARY KEY, v1 BIGINT NOT NULL AS (c1) virtual);

statement ok
INSERT INTO virt1 VALUES (100), (2147483647);

# Ensure the USING expression cannot be used
statement error pq: type change for virtual column "v1" cannot be altered with a USING expression
ALTER TABLE virt1 ALTER COLUMN v1 SET DATA TYPE INT4 USING 10;

skipif config local-legacy-schema-changer
skipif config local-mixed-24.1
skipif config local-mixed-24.2
statement ok
ALTER TABLE virt1 ALTER COLUMN v1 SET DATA TYPE INT4;

skipif config local-legacy-schema-changer
skipif config local-mixed-24.1
skipif config local-mixed-24.2
query II
SELECT * from virt1 ORDER BY c1;
----
100 100
2147483647 2147483647

skipif config local-legacy-schema-changer
skipif config local-mixed-24.1
skipif config local-mixed-24.2
query TT
SHOW CREATE TABLE virt1;
----
virt1 CREATE TABLE public.virt1 (
c1 INT8 NOT NULL,
v1 INT4 NOT NULL AS (c1) VIRTUAL,
CONSTRAINT virt1_pkey PRIMARY KEY (c1 ASC)
)

skipif config local-legacy-schema-changer
skipif config local-mixed-24.1
statement error pq: validate check constraint: integer out of range for type int2
ALTER TABLE virt1 ALTER COLUMN v1 SET DATA TYPE INT2;

statement ok
DELETE FROM virt1 WHERE c1 = 2147483647;

skipif config local-legacy-schema-changer
skipif config local-mixed-24.1
statement ok
ALTER TABLE virt1 ALTER COLUMN v1 SET DATA TYPE INT2;

statement ok
INSERT INTO virt1 VALUES (-9999);

skipif config local-legacy-schema-changer
skipif config local-mixed-24.1
query II
SELECT * from virt1 ORDER BY c1;
----
-9999 -9999
100 100

skipif config local-legacy-schema-changer
skipif config local-mixed-24.1
query TT
SHOW CREATE TABLE virt1;
----
virt1 CREATE TABLE public.virt1 (
c1 INT8 NOT NULL,
v1 INT2 NOT NULL AS (c1) VIRTUAL,
CONSTRAINT virt1_pkey PRIMARY KEY (c1 ASC)
)

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 'c1' has type int
ALTER TABLE virt1 ALTER COLUMN v1 SET DATA TYPE TEXT;

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 float, but 'c1' has type int
ALTER TABLE virt1 ALTER COLUMN v1 SET DATA TYPE FLOAT;

statement ok
DROP TABLE virt1;

subtest virtual_compute_tz

statement ok
CREATE TABLE virt2 (C1 TIMESTAMP(6) NOT NULL PRIMARY KEY, v1 TIMESTAMP(3) AS (c1) VIRTUAL);

statement ok
INSERT INTO virt2 VALUES ('2024-10-31 16:50:00.123456');

query TT
SELECT * FROM virt2 ORDER BY c1;
----
2024-10-31 16:50:00.123456 +0000 +0000 2024-10-31 16:50:00.123 +0000 +0000

statement ok
ALTER TABLE virt2 ALTER COLUMN v1 SET DATA TYPE TIMESTAMP(2);

query TT
SELECT * FROM virt2 ORDER BY c1;
----
2024-10-31 16:50:00.123456 +0000 +0000 2024-10-31 16:50:00.12 +0000 +0000

query TT
SHOW CREATE TABLE virt2;
----
virt2 CREATE TABLE public.virt2 (
c1 TIMESTAMP(6) NOT NULL,
v1 TIMESTAMP(2) NULL AS (c1) VIRTUAL,
CONSTRAINT virt2_pkey PRIMARY KEY (c1 ASC)
)

statement ok
ALTER TABLE virt2 ALTER COLUMN v1 SET DATA TYPE TIMESTAMP(5);

query TT
SELECT * FROM virt2 ORDER BY c1;
----
2024-10-31 16:50:00.123456 +0000 +0000 2024-10-31 16:50:00.12346 +0000 +0000

query TT
SHOW CREATE TABLE virt2;
----
virt2 CREATE TABLE public.virt2 (
c1 TIMESTAMP(6) NOT NULL,
v1 TIMESTAMP(5) NULL AS (c1) VIRTUAL,
CONSTRAINT virt2_pkey PRIMARY KEY (c1 ASC)
)

statement ok
DROP TABLE virt2;

subtest end
1 change: 1 addition & 0 deletions pkg/sql/schemachange/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ go_test(
"//pkg/util/log",
"//pkg/util/randutil",
"//pkg/util/uuid",
"@com_github_stretchr_testify//require",
],
)

Expand Down
59 changes: 56 additions & 3 deletions pkg/sql/schemachange/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,27 @@ var classifiers = map[types.Family]map[types.Family]classifier{
},
}

// virtualGeneralReclassifier is used to classify general conversions for virtual
// computed columns. General conversions don’t apply to these columns, as they
// aren’t physically stored on disk. If this map is used and the type family is
// missing, it’s assumed that the type conversion cannot be applied.
var virtualGeneralReclassifier = map[types.Family]map[types.Family]classifier{
types.TimestampFamily: {
types.TimestampTZFamily: ColumnConversionTrivial.classifier(),
types.TimestampFamily: ColumnConversionTrivial.classifier(),
},
types.TimestampTZFamily: {
types.TimestampFamily: ColumnConversionTrivial.classifier(),
types.TimestampTZFamily: ColumnConversionTrivial.classifier(),
},
types.TimeFamily: {
types.TimeFamily: ColumnConversionTrivial.classifier(),
},
types.TimeTZFamily: {
types.TimeTZFamily: ColumnConversionTrivial.classifier(),
},
}

// classifierHardestOf creates a composite classifier that returns the
// hardest kind of the enclosed classifiers. If any of the
// classifiers report impossible, impossible will be returned.
Expand Down Expand Up @@ -246,15 +267,39 @@ func ClassifyConversion(
// ClassifyConversionFromTree is a wrapper for ClassifyConversion when we want
// to take into account the parsed AST for ALTER TABLE .. ALTER COLUMN.
func ClassifyConversionFromTree(
ctx context.Context, t *tree.AlterTableAlterColumnType, oldType *types.T, newType *types.T,
) (ColumnConversionKind, error) {
ctx context.Context,
t *tree.AlterTableAlterColumnType,
oldType *types.T,
newType *types.T,
isVirtual bool,
) (kind ColumnConversionKind, err error) {
if t.Using != nil {
// If an expression is provided, we always need to try a general conversion.
// We have to follow the process to create a new column and backfill it
// using the expression.
return ColumnConversionGeneral, nil
}
return ClassifyConversion(ctx, oldType, newType)
kind, err = ClassifyConversion(ctx, oldType, newType)
if err != nil {
return
}
// A general rewrite isn't applicable for virtual columns since they don’t exist
// physically. We need to pick a new classifier. For conversions that would require
// general handling due to incompatible type families (e.g., INT -> TEXT), we
// assume these will already be rejected because the computed expression doesn’t
// match the new type. Such cases are handled by validateNewTypeForComputedColumn.
if isVirtual && kind == ColumnConversionGeneral {
if inner, oldTypeFamilyFound := virtualGeneralReclassifier[oldType.Family()]; oldTypeFamilyFound {
if fn, newTypeFamilyFound := inner[newType.Family()]; newTypeFamilyFound {
kind = fn(oldType, newType)
return
}
}
kind = ColumnConversionImpossible
err = pgerror.Newf(pgcode.CannotCoerce, "cannot convert %s to %s for a virtual column",
oldType.SQLString(), newType.SQLString())
}
return
}

// ValidateAlterColumnTypeChecks performs validation checks on the proposed type
Expand All @@ -267,6 +312,7 @@ func ValidateAlterColumnTypeChecks(
settions *cluster.Settings,
origTyp *types.T,
isGeneratedAsIdentity bool,
isVirtual bool,
) (*types.T, error) {
typ := origTyp
// Special handling for STRING COLLATE xy to verify that we recognize the language.
Expand All @@ -286,5 +332,12 @@ func ValidateAlterColumnTypeChecks(
}
}

// A USING expression is unnecessary when altering the type of a virtual column,
// as its value is always computed at runtime and is not stored on disk.
if isVirtual && t.Using != nil {
return typ, pgerror.Newf(pgcode.FeatureNotSupported,
"type change for virtual column %q cannot be altered with a USING expression", t.Column)
}

return typ, colinfo.ValidateColumnDefType(ctx, settions, typ)
}
68 changes: 60 additions & 8 deletions pkg/sql/schemachange/alter_column_type_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,21 +25,22 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
)

func columnType(typStr string) *types.T {
t, err := parser.GetTypeFromValidSQLSyntax(typStr)
if err != nil {
panic(err)
}
return tree.MustBeStaticallyKnownType(t)
}

// TestColumnConversions rolls-up a lot of test plumbing to prevent
// top-level namespace pollution.
func TestColumnConversions(t *testing.T) {
defer leaktest.AfterTest(t)()

columnType := func(typStr string) *types.T {
t, err := parser.GetTypeFromValidSQLSyntax(typStr)
if err != nil {
panic(err)
}
return tree.MustBeStaticallyKnownType(t)
}

// columnConversionInfo is where we document conversions that
// don't require a fully-generalized conversion path or where there are
// restrictions on conversions that seem non-obvious at first glance.
Expand Down Expand Up @@ -424,3 +425,54 @@ func TestColumnConversions(t *testing.T) {
}
})
}

// TestVirtualColumnConversions will validate column conversions for virtual
// compute columns.
func TestVirtualColumnConversions(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
for _, tc := range []struct {
from string
to string
expectedConversion ColumnConversionKind
}{
{"BYTES", "BYTES", ColumnConversionTrivial},
{"BYTES", "STRING(255)", ColumnConversionValidate},
{"BYTES", "UUID", ColumnConversionValidate},
{"BYTES", "INT8", ColumnConversionImpossible},
{"DECIMAL(9,3)", "DECIMAL(9,2)", ColumnConversionValidate},
{"DECIMAL(9,3)", "DECIMAL(9,4)", ColumnConversionTrivial},
{"DECIMAL(9,3)", "DECIMAL(10,3)", ColumnConversionTrivial},
{"DECIMAL(9,3)", "DECIMAL(8,3)", ColumnConversionValidate},
{"FLOAT(8)", "FLOAT(4)", ColumnConversionTrivial},
{"INT8", "INT4", ColumnConversionValidate},
{"INT4", "INT8", ColumnConversionTrivial},
{"INT2", "TEXT", ColumnConversionImpossible},
{"BIT(1)", "BIT(4)", ColumnConversionTrivial},
{"BIT(4)", "BIT(1)", ColumnConversionValidate},
{"BIT(4)", "BIT(0)", ColumnConversionTrivial},
{"VARCHAR(10)", "CHAR(5)", ColumnConversionValidate},
{"CHAR(15)", "TEXT", ColumnConversionTrivial},
{"TIMESTAMP(6)", "TIMESTAMP(4)", ColumnConversionTrivial},
{"TIMESTAMP(4)", "TIMESTAMP(6)", ColumnConversionTrivial},
{"TIMESTAMP(4)", "TIMESTAMPTZ(2)", ColumnConversionTrivial},
{"TIMESTAMPTZ(5)", "TIMESTAMPTZ(2)", ColumnConversionTrivial},
{"TIMESTAMPTZ(5)", "TIMESTAMP(3)", ColumnConversionTrivial},
{"TIMESTAMPTZ(5)", "TIME", ColumnConversionImpossible},
{"TIME", "TIME(4)", ColumnConversionTrivial},
{"TIME(6)", "TIME(4)", ColumnConversionTrivial},
{"TIME(6)", "TIMESTAMP(6)", ColumnConversionImpossible},
} {
t.Run(fmt.Sprintf("%s->%s", tc.from, tc.to), func(t *testing.T) {
actual, err := ClassifyConversionFromTree(ctx, &tree.AlterTableAlterColumnType{},
columnType(tc.from), columnType(tc.to), true)
if tc.expectedConversion != ColumnConversionImpossible {
require.NoError(t, err)
} else {
require.Error(t, err)
}
require.Equal(t, tc.expectedConversion, actual)
})
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,8 @@ func alterTableAlterColumnType(
var err error
newColType.Type, err = schemachange.ValidateAlterColumnTypeChecks(
b, t, b.ClusterSettings(), newColType.Type,
col.GeneratedAsIdentityType != catpb.GeneratedAsIdentityType_NOT_IDENTITY_COLUMN)
col.GeneratedAsIdentityType != catpb.GeneratedAsIdentityType_NOT_IDENTITY_COLUMN,
newColType.IsVirtual)
if err != nil {
panic(err)
}
Expand All @@ -92,7 +93,8 @@ func alterTableAlterColumnType(
validateAutomaticCastForNewType(b, tbl.TableID, colID, t.Column.String(),
oldColType.Type, newColType.Type, t.Using != nil)

kind, err := schemachange.ClassifyConversionFromTree(b, t, oldColType.Type, newColType.Type)
kind, err := schemachange.ClassifyConversionFromTree(b, t, oldColType.Type, newColType.Type,
newColType.IsVirtual)
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -285,10 +287,10 @@ func handleGeneralColumnConversion(
}
})

// This code path should never be reached for virtual columns, as their values
// are always computed dynamically on access and are never stored on disk.
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"))
panic(errors.AssertionFailedf("virtual columns cannot be backfilled"))
}

// We block any attempt to alter the type of a column that is a key column in
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@ go_library(
"//pkg/sql/catalog/typedesc",
"//pkg/sql/parser",
"//pkg/sql/privilege",
"//pkg/sql/schemachange",
"//pkg/sql/schemachanger/scop",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/sem/catconstants",
Expand Down
Loading

0 comments on commit e2b5dd6

Please sign in to comment.