Skip to content

Commit

Permalink
sql: support adding virtual columns
Browse files Browse the repository at this point in the history
Support adding and removing virtual columns (when no indexes on those
columns are in play).

Note that because these columns don't need backfill in existing
indexes, they don't need special WriteOnly / DeleteOnly logic (other
than having the right visibility).

Release note: None
  • Loading branch information
RaduBerinde committed Jan 26, 2021
1 parent 706006e commit 402dc8d
Show file tree
Hide file tree
Showing 7 changed files with 166 additions and 38 deletions.
4 changes: 3 additions & 1 deletion pkg/sql/add_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ 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/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
Expand All @@ -29,6 +30,7 @@ func (p *planner) addColumnImpl(
tn *tree.TableName,
desc *tabledesc.Mutable,
t *tree.AlterTableAddColumn,
sessionData *sessiondata.SessionData,
) error {
d := t.ColumnDef
version := params.ExecCfg().Settings.Version.ActiveVersionOrEmpty(params.ctx)
Expand Down Expand Up @@ -143,7 +145,7 @@ func (p *planner) addColumnImpl(
}

if d.IsComputed() {
if d.IsVirtual() {
if d.IsVirtual() && !sessionData.VirtualColumnsEnabled {
return unimplemented.NewWithIssue(57608, "virtual computed columns")
}
computedColValidator := schemaexpr.MakeComputedColumnValidator(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,7 +169,7 @@ func (n *alterTableNode) startExec(params runParams) error {
}
var err error
params.p.runWithOptions(resolveFlags{contextDatabaseID: n.tableDesc.ParentID}, func() {
err = params.p.addColumnImpl(params, n, tn, n.tableDesc, t)
err = params.p.addColumnImpl(params, n, tn, n.tableDesc, t, params.SessionData())
})
if err != nil {
return err
Expand Down
10 changes: 6 additions & 4 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error {
case descpb.DescriptorMutation_ADD:
switch t := m.Descriptor_.(type) {
case *descpb.DescriptorMutation_Column:
if tabledesc.ColumnNeedsBackfill(m.GetColumn()) {
if tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) {
needColumnBackfill = true
}
case *descpb.DescriptorMutation_Index:
Expand Down Expand Up @@ -249,7 +249,9 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error {
case descpb.DescriptorMutation_DROP:
switch t := m.Descriptor_.(type) {
case *descpb.DescriptorMutation_Column:
needColumnBackfill = true
if tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) {
needColumnBackfill = true
}
case *descpb.DescriptorMutation_Index:
if !canClearRangeForDrop(t.Index) {
droppedIndexDescs = append(droppedIndexDescs, *t.Index)
Expand Down Expand Up @@ -1803,7 +1805,7 @@ func runSchemaChangesInTxn(
case *descpb.DescriptorMutation_ComputedColumnSwap:
return AlterColTypeInTxnNotSupportedErr
case *descpb.DescriptorMutation_Column:
if doneColumnBackfill || !tabledesc.ColumnNeedsBackfill(m.GetColumn()) {
if doneColumnBackfill || !tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) {
break
}
if err := columnBackfillInTxn(ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV); err != nil {
Expand All @@ -1830,7 +1832,7 @@ func runSchemaChangesInTxn(
// Drop the name and drop the associated data later.
switch t := m.Descriptor_.(type) {
case *descpb.DescriptorMutation_Column:
if doneColumnBackfill {
if doneColumnBackfill || !tabledesc.ColumnNeedsBackfill(m.Direction, m.GetColumn()) {
break
}
if err := columnBackfillInTxn(
Expand Down
28 changes: 21 additions & 7 deletions pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -3777,9 +3777,26 @@ func (desc *wrapper) MakeFirstMutationPublic(includeConstraints bool) (*Mutable,
return table, nil
}

// ColumnNeedsBackfill returns true if adding the given column requires a
// backfill (dropping a column always requires a backfill).
func ColumnNeedsBackfill(desc *descpb.ColumnDescriptor) bool {
// ColumnNeedsBackfill returns true if adding or dropping (according to
// the direction) the given column requires backfill.
func ColumnNeedsBackfill(
direction descpb.DescriptorMutation_Direction, desc *descpb.ColumnDescriptor,
) bool {
if desc.Virtual {
// Virtual columns can only be used as secondary index keys; as such they do
// not need backfill in other indexes.
// TODO(radu): revisit this if/when we allow STORING virtual columns.
return false
}
if direction == descpb.DescriptorMutation_DROP {
// In all other cases, DROP requires backfill.
return true
}
// ADD requires backfill for:
// - columns with non-NULL default value
// - computed columns
// - non-nullable columns (note: if a non-nullable column doesn't have a
// default value, the backfill will fail unless the table is empty).
if desc.HasNullDefault() {
return false
}
Expand All @@ -3800,10 +3817,7 @@ func (desc *wrapper) HasColumnBackfillMutation() bool {
// Index backfills don't affect changefeeds.
continue
}
// It's unfortunate that there's no one method we can call to check if a
// mutation will be a backfill or not, but this logic was extracted from
// backfill.go.
if m.Direction == descpb.DescriptorMutation_DROP || ColumnNeedsBackfill(col) {
if ColumnNeedsBackfill(m.Direction, col) {
return true
}
}
Expand Down
78 changes: 56 additions & 22 deletions pkg/sql/catalog/tabledesc/structured_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1823,32 +1823,66 @@ func TestColumnNeedsBackfill(t *testing.T) {
// Define variable strings here such that we can pass their address below.
null := "NULL"
four := "4:::INT8"

// Create Column Descriptors that reflect the definition of a column with a
// default value of NULL that was set implicitly, one that was set explicitly,
// and one that has an INT default value, respectively.
implicitNull := &descpb.ColumnDescriptor{
Name: "im", ID: 2, Type: types.Int, DefaultExpr: nil, Nullable: true, ComputeExpr: nil,
}
explicitNull := &descpb.ColumnDescriptor{
Name: "ex", ID: 3, Type: types.Int, DefaultExpr: &null, Nullable: true, ComputeExpr: nil,
}
defaultNotNull := &descpb.ColumnDescriptor{
Name: "four", ID: 4, Type: types.Int, DefaultExpr: &four, Nullable: true, ComputeExpr: nil,
}
// Verify that a backfill doesn't occur according to the ColumnNeedsBackfill
// function for the default NULL values, and that it does occur for an INT
// default value.
if ColumnNeedsBackfill(implicitNull) != false {
t.Fatal("Expected implicit SET DEFAULT NULL to not require a backfill," +
" ColumnNeedsBackfill states that it does.")
}
if ColumnNeedsBackfill(explicitNull) != false {
t.Fatal("Expected explicit SET DEFAULT NULL to not require a backfill," +
" ColumnNeedsBackfill states that it does.")
testCases := []struct {
info string
desc descpb.ColumnDescriptor
// add is true of we expect backfill when adding this column.
add bool
// drop is true of we expect backfill when adding this column.
drop bool
}{
{
info: "implicit SET DEFAULT NULL",
desc: descpb.ColumnDescriptor{
Name: "am", ID: 2, Type: types.Int, DefaultExpr: nil, Nullable: true, ComputeExpr: nil,
},
add: false,
drop: true,
}, {
info: "explicit SET DEFAULT NULL",
desc: descpb.ColumnDescriptor{
Name: "ex", ID: 3, Type: types.Int, DefaultExpr: &null, Nullable: true, ComputeExpr: nil,
},
add: false,
drop: true,
},
{
info: "explicit SET DEFAULT non-NULL",
desc: descpb.ColumnDescriptor{
Name: "four", ID: 4, Type: types.Int, DefaultExpr: &four, Nullable: true, ComputeExpr: nil,
},
add: true,
drop: true,
},
{
info: "computed stored",
desc: descpb.ColumnDescriptor{
Name: "stored", ID: 5, Type: types.Int, DefaultExpr: nil, ComputeExpr: &four,
},
add: true,
drop: true,
},
{
info: "computed virtual",
desc: descpb.ColumnDescriptor{
Name: "virtual", ID: 6, Type: types.Int, DefaultExpr: nil, ComputeExpr: &four, Virtual: true,
},
add: false,
drop: false,
},
}
if ColumnNeedsBackfill(defaultNotNull) != true {
t.Fatal("Expected explicit SET DEFAULT NULL to require a backfill," +
" ColumnNeedsBackfill states that it does not.")

for _, tc := range testCases {
if ColumnNeedsBackfill(descpb.DescriptorMutation_ADD, &tc.desc) != tc.add {
t.Errorf("expected ColumnNeedsBackfill to be %v for adding %s", tc.add, tc.info)
}
if ColumnNeedsBackfill(descpb.DescriptorMutation_DROP, &tc.desc) != tc.drop {
t.Errorf("expected ColumnNeedsBackfill to be %v for dropping %s", tc.drop, tc.info)
}
}
}

Expand Down
76 changes: 76 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/virtual_columns
Original file line number Diff line number Diff line change
Expand Up @@ -247,3 +247,79 @@ UPDATE t_idx SET b=b+1 RETURNING w
----
3
7

# Test schema changes with virtual columns.

statement ok
CREATE TABLE sc (a INT PRIMARY KEY, b INT)

statement ok
INSERT INTO sc VALUES (1, 10), (2, 20), (3, 30);

statement ok
ALTER TABLE sc ADD COLUMN v INT AS (a+b) VIRTUAL

query III rowsort,colnames
SELECT * FROM sc
----
a b v
1 10 11
2 20 22
3 30 33

statement ok
ALTER TABLE sc ADD COLUMN x INT AS (a+1) VIRTUAL, ADD COLUMN y INT AS (b+1) VIRTUAL, ADD COLUMN z INT AS (a+b) VIRTUAL

query IIIIII rowsort,colnames
SELECT * FROM sc
----
a b v x y z
1 10 11 2 11 11
2 20 22 3 21 22
3 30 33 4 31 33

statement error computed columns cannot reference other computed columns
ALTER TABLE sc ADD COLUMN u INT AS (a+v) VIRTUAL

statement ok
ALTER TABLE sc DROP COLUMN z

query IIIII rowsort,colnames
SELECT * FROM sc
----
a b v x y
1 10 11 2 11
2 20 22 3 21
3 30 33 4 31

statement ok
ALTER TABLE sc DROP COLUMN x, DROP COLUMN y

query III rowsort,colnames
SELECT * FROM sc
----
a b v
1 10 11
2 20 22
3 30 33

# Add virtual columns inside an explicit transactions.
statement ok
BEGIN

statement ok
ALTER TABLE sc ADD COLUMN w1 INT AS (a*b) VIRTUAL

statement ok
ALTER TABLE sc ADD COLUMN w2 INT AS (b*2) VIRTUAL

statement ok
COMMIT

query IIIII rowsort,colnames
SELECT * FROM sc
----
a b v w1 w2
1 10 11 10 20
2 20 22 40 40
3 30 33 90 60
6 changes: 3 additions & 3 deletions pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -670,9 +670,9 @@ func newOptTable(
desc.ComputeExpr,
)
} else {
if kind != cat.Ordinary {
return nil, errors.AssertionFailedf("virtual mutation column")
}
// Note: a WriteOnly or DeleteOnly mutation column doesn't require any
// special treatment inside the optimizer, other than having the correct
// visibility.
ot.columns[ordinal].InitVirtualComputed(
ordinal,
cat.StableID(desc.ID),
Expand Down

0 comments on commit 402dc8d

Please sign in to comment.