Skip to content

Commit

Permalink
sql/schemachanger: support ADD COLUMN SERIAL for DSC unique_rowid only
Browse files Browse the repository at this point in the history
Previously we could only add SERIAL columns with the legacy schema changer
with these code changes the ALTER TABLE ADD COLUMN statement
can now add SERIAL type columns via the declarative schema changer for
the default rowid serial_normalization mode.

Informs: #126900
Release note: none
  • Loading branch information
Dedej-Bergin committed Sep 5, 2024
1 parent df4c2cd commit e023883
Show file tree
Hide file tree
Showing 21 changed files with 1,820 additions and 38 deletions.
28 changes: 28 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"metadata.go",
"post_deserialization_changes.go",
"schema.go",
"serial_helper.go",
"system_table.go",
"table_col_map.go",
"table_col_set.go",
Expand Down
54 changes: 54 additions & 0 deletions pkg/sql/catalog/serial_helper.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
// Copyright 2024 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 catalog

import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

func UseRowID(d tree.ColumnTableDef) *tree.ColumnTableDef {
d.DefaultExpr.Expr = &tree.FuncExpr{Func: tree.WrapFunction("unique_rowid")}
d.Type = types.Int
// Column is non-nullable in all cases. PostgreSQL requires this.
d.Nullable.Nullability = tree.NotNull

return &d
}

func AssertValidSerialColumnDef(d *tree.ColumnTableDef, tableName *tree.TableName) error {
if d.HasDefaultExpr() {
// SERIAL implies a new default expression, we can't have one to
// start with. This is the error produced by pg in such case.
return pgerror.Newf(pgcode.Syntax,
"multiple default values specified for column %q of table %q",
tree.ErrString(&d.Name), tree.ErrString(tableName))
}

if d.Nullable.Nullability == tree.Null {
// SERIAL implies a non-NULL column, we can't accept a nullability
// spec. This is the error produced by pg in such case.
return pgerror.Newf(pgcode.Syntax,
"conflicting NULL/NOT NULL declarations for column %q of table %q",
tree.ErrString(&d.Name), tree.ErrString(tableName))
}

if d.Computed.Expr != nil {
// SERIAL cannot be a computed column.
return pgerror.Newf(pgcode.Syntax,
"SERIAL column %q of table %q cannot be computed",
tree.ErrString(&d.Name), tree.ErrString(tableName))
}

return nil
}
34 changes: 34 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -3967,3 +3967,37 @@ statement error pgcode 42601 variable sub-expressions are not allowed in EXPRESS
ALTER TABLE t_124546 ADD CONSTRAINT ident UNIQUE ( ( EXISTS ( TABLE error FOR READ ONLY ) ) DESC ) STORING ( ident , ident );

subtest end

subtest alter_table_add_column_serial

statement ok
create table roach (id int);
insert into roach DEFAULT VALUES;
insert into roach DEFAULT VALUES;
SET serial_normalization = rowid

statement ok
alter table roach add column serial_id SERIAL;

query TTBTTTB colnames,rowsort
show columns from roach;
----
column_name data_type is_nullable column_default generation_expression indices is_hidden
id INT8 true NULL · {roach_pkey} false
rowid INT8 false unique_rowid() · {roach_pkey} true
serial_id INT8 false unique_rowid() · {roach_pkey} false

subtest end

subtest unimplemented_for_non_rowid_in_DSC

statement ok
SET serial_normalization = sql_sequence

statement ok
SET use_declarative_schema_changer = unsafe_always

statement error pq: \*tree.ColumnTableDef not implemented in the new schema changer: contains serial data type in unsupported mode
alter table roach add column serial_id2 SERIAL

subtest end
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@ import (
"sort"
"strings"

"github.com/cockroachdb/cockroach/pkg/docs"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
Expand All @@ -25,12 +27,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdecomp"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -71,7 +75,11 @@ func alterTableAddColumn(
}
}
if d.IsSerial {
panic(scerrors.NotImplementedErrorf(d, "contains serial data type"))
if b.SessionData().SerialNormalizationMode != sessiondatapb.SerialUsesRowID {
panic(scerrors.NotImplementedErrorf(d, "contains serial data type in unsupported mode"))
}
d = alterTableAddColumnSerial(b, d, tn)

}
if d.GeneratedIdentity.IsGeneratedAsIdentity {
panic(scerrors.NotImplementedErrorf(d, "contains generated identity type"))
Expand Down Expand Up @@ -259,6 +267,43 @@ func alterTableAddColumn(
}
}

func alterTableAddColumnSerial(
b BuildCtx, d *tree.ColumnTableDef, tn *tree.TableName,
) *tree.ColumnTableDef {
if err := catalog.AssertValidSerialColumnDef(d, tn); err != nil {
panic(err)
}

defType, err := tree.ResolveType(b, d.Type, b.SemaCtx().GetTypeResolver())
if err != nil {
panic(err)
}

telemetry.Inc(sqltelemetry.SerialColumnNormalizationCounter(
defType.Name(), b.SessionData().SerialNormalizationMode.String()))

if defType.Width() < types.Int.Width() {
b.EvalCtx().ClientNoticeSender.BufferClientNotice(
b,
errors.WithHintf(
pgnotice.Newf(
"upgrading the column %s to %s to utilize the session serial_normalization setting",
d.Name.String(),
types.Int.SQLString(),
),
"change the serial_normalization to sql_sequence or sql_sequence_cached if you wish "+
"to use a smaller sized serial column at the cost of performance. See %s",
docs.URL("serial.html"),
),
)
}

// Serial is an alias for a real column definition. False indicates a remapped alias.
d.IsSerial = false

return catalog.UseRowID(*d)
}

func columnNamesToIDs(b BuildCtx, tbl *scpb.Table) map[string]descpb.ColumnID {
tableElts := b.QueryByID(tbl.TableID)
namesToIDs := make(map[string]descpb.ColumnID)
Expand Down
42 changes: 42 additions & 0 deletions pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,48 @@ ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED
- [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT]
{columnId: 2, indexId: 3, kind: STORED, tableId: 104}

build
ALTER TABLE defaultdb.foo ADD COLUMN serial_id SERIAL
----
- [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC]
{columnId: 1, indexId: 1, tableId: 104}
- [[PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC]
{constraintId: 1, indexId: 1, isUnique: true, tableId: 104}
- [[IndexName:{DescID: 104, Name: foo_pkey, IndexID: 1}, ABSENT], PUBLIC]
{indexId: 1, name: foo_pkey, tableId: 104}
- [[IndexData:{DescID: 104, IndexID: 1}, ABSENT], PUBLIC]
{indexId: 1, tableId: 104}
- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC]
{databaseId: 100, tableId: 104}
- [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT]
{columnId: 2, tableId: 104}
- [[ColumnName:{DescID: 104, Name: serial_id, ColumnID: 2}, PUBLIC], ABSENT]
{columnId: 2, name: serial_id, tableId: 104}
- [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2, TypeName: INT8}, PUBLIC], ABSENT]
{columnId: 2, elementCreationMetadata: {in231OrLater: true, in243OrLater: true}, tableId: 104, type: {family: IntFamily, oid: 20, width: 64}, typeName: INT8}
- [[ColumnDefaultExpression:{DescID: 104, ColumnID: 2, Expr: unique_rowid()}, PUBLIC], ABSENT]
{columnId: 2, expr: unique_rowid(), tableId: 104}
- [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT]
{constraintId: 2, indexId: 2, isUnique: true, sourceIndexId: 1, tableId: 104, temporaryIndexId: 3}
- [[IndexName:{DescID: 104, Name: foo_pkey, IndexID: 2}, PUBLIC], ABSENT]
{indexId: 2, name: foo_pkey, tableId: 104}
- [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT]
{columnId: 1, indexId: 2, tableId: 104}
- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT]
{indexId: 2, tableId: 104}
- [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT]
{constraintId: 3, indexId: 3, isUnique: true, sourceIndexId: 1, tableId: 104}
- [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT]
{columnId: 1, indexId: 3, tableId: 104}
- [[IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_ABSENT], ABSENT]
{indexId: 3, tableId: 104}
- [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT]
{columnId: 2, indexId: 2, kind: STORED, tableId: 104}
- [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT]
{columnId: 2, indexId: 3, kind: STORED, tableId: 104}
- [[ColumnNotNull:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT]
{columnId: 2, indexIdForValidation: 2, tableId: 104}

setup
CREATE TABLE defaultdb.bar (j INT);
----
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,6 @@ CREATE TABLE defaultdb.foo (
);
----

unimplemented
ALTER TABLE defaultdb.foo ADD COLUMN j SERIAL
----

unimplemented
ALTER TABLE defaultdb.foo ALTER COLUMN i DROP NOT NULL
----
Expand Down
Loading

0 comments on commit e023883

Please sign in to comment.