diff --git a/docs/generated/sql/bnf/col_qualification.bnf b/docs/generated/sql/bnf/col_qualification.bnf index 63c584b9d018..8a8a5e3226f5 100644 --- a/docs/generated/sql/bnf/col_qualification.bnf +++ b/docs/generated/sql/bnf/col_qualification.bnf @@ -9,6 +9,8 @@ col_qualification ::= | 'CONSTRAINT' constraint_name 'REFERENCES' table_name opt_name_parens key_match reference_actions | 'CONSTRAINT' constraint_name 'AS' '(' a_expr ')' 'STORED' | 'CONSTRAINT' constraint_name 'GENERATED_ALWAYS' 'ALWAYS' 'AS' '(' a_expr ')' 'STORED' + | 'CONSTRAINT' constraint_name 'AS' '(' a_expr ')' 'VIRTUAL' + | 'CONSTRAINT' constraint_name 'GENERATED_ALWAYS' 'ALWAYS' 'AS' '(' a_expr ')' 'VIRTUAL' | 'NOT' 'NULL' | 'NULL' | 'UNIQUE' opt_without_index @@ -19,6 +21,8 @@ col_qualification ::= | 'REFERENCES' table_name opt_name_parens key_match reference_actions | 'AS' '(' a_expr ')' 'STORED' | 'GENERATED_ALWAYS' 'ALWAYS' 'AS' '(' a_expr ')' 'STORED' + | 'AS' '(' a_expr ')' 'VIRTUAL' + | 'GENERATED_ALWAYS' 'ALWAYS' 'AS' '(' a_expr ')' 'VIRTUAL' | 'COLLATE' collation_name | 'FAMILY' family_name | 'CREATE' 'FAMILY' family_name diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index fe927813c106..5690cdd30ad0 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -2872,6 +2872,7 @@ col_qualification_elem ::= | 'DEFAULT' b_expr | 'REFERENCES' table_name opt_name_parens key_match reference_actions | generated_as '(' a_expr ')' 'STORED' + | generated_as '(' a_expr ')' 'VIRTUAL' family_name ::= name diff --git a/pkg/sql/add_column.go b/pkg/sql/add_column.go index eee5bd6ad6d0..4f72e3399e7f 100644 --- a/pkg/sql/add_column.go +++ b/pkg/sql/add_column.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/errors" ) @@ -142,6 +143,9 @@ func (p *planner) addColumnImpl( } if d.IsComputed() { + if d.IsVirtual() { + return unimplemented.NewWithIssue(57608, "virtual computed columns") + } computedColValidator := schemaexpr.MakeComputedColumnValidator( params.ctx, n.tableDesc, diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 2719448d053c..cc3ea108a13c 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1297,6 +1297,10 @@ func NewTableDesc( n.Defs = append(n.Defs, checkConstraint) columnDefaultExprs = append(columnDefaultExprs, nil) } + if d.IsVirtual() { + return nil, unimplemented.NewWithIssue(57608, "virtual computed columns") + } + col, idx, expr, err := tabledesc.MakeColumnDefDescs(ctx, d, semaCtx, evalCtx) if err != nil { return nil, err diff --git a/pkg/sql/logictest/testdata/logic_test/computed b/pkg/sql/logictest/testdata/logic_test/computed index d3e445f6e27a..2694e7f976d7 100644 --- a/pkg/sql/logictest/testdata/logic_test/computed +++ b/pkg/sql/logictest/testdata/logic_test/computed @@ -271,11 +271,20 @@ CREATE TABLE y ( a INT AS (3) ) -statement error at or near "virtual": syntax error: unimplemented +statement error unimplemented: virtual computed columns CREATE TABLE y ( a INT AS (3) VIRTUAL ) +statement ok +CREATE TABLE tmp (x INT) + +statement error unimplemented: virtual computed columns +ALTER TABLE tmp ADD COLUMN y INT AS (x+1) VIRTUAL + +statement ok +DROP TABLE tmp + statement error expected computed column expression to have type int, but .* has type string CREATE TABLE y ( a INT AS ('not an integer!'::STRING) STORED diff --git a/pkg/sql/opt/cat/column.go b/pkg/sql/opt/cat/column.go index c9e77c40576c..a28d68b04765 100644 --- a/pkg/sql/opt/cat/column.go +++ b/pkg/sql/opt/cat/column.go @@ -161,8 +161,7 @@ const ( // VirtualInverted columns are implicit columns that are used by inverted // indexes. VirtualInverted - // VirtualComputed columns are non-stored computed columns that are used by - // expression-based indexes. + // VirtualComputed columns are non-stored computed columns. VirtualComputed ) @@ -228,15 +227,21 @@ func (c *Column) InitVirtualInverted( // InitVirtualComputed is used by catalog implementations to populate a // VirtualComputed Column. It should not be used anywhere else. func (c *Column) InitVirtualComputed( - ordinal int, name tree.Name, datumType *types.T, nullable bool, computedExpr string, + ordinal int, + stableID StableID, + name tree.Name, + datumType *types.T, + nullable bool, + hidden bool, + computedExpr string, ) { c.ordinal = ordinal - c.stableID = 0 + c.stableID = stableID c.name = name c.kind = VirtualComputed c.datumType = datumType c.nullable = nullable - c.hidden = true + c.hidden = hidden c.defaultExpr = "" c.computedExpr = computedExpr c.invertedSourceColumnOrdinal = -1 diff --git a/pkg/sql/opt/cat/utils.go b/pkg/sql/opt/cat/utils.go index b63d4e93cf82..62bf04684c15 100644 --- a/pkg/sql/opt/cat/utils.go +++ b/pkg/sql/opt/cat/utils.go @@ -306,7 +306,11 @@ func formatColumn(col *Column, buf *bytes.Buffer) { fmt.Fprintf(buf, " not null") } if col.IsComputed() { - fmt.Fprintf(buf, " as (%s) stored", col.ComputedExprStr()) + if col.Kind() == VirtualComputed { + fmt.Fprintf(buf, " as (%s) virtual", col.ComputedExprStr()) + } else { + fmt.Fprintf(buf, " as (%s) stored", col.ComputedExprStr()) + } } if col.HasDefault() { fmt.Fprintf(buf, " default (%s)", col.DefaultExprStr()) @@ -322,7 +326,7 @@ func formatColumn(col *Column, buf *bytes.Buffer) { case VirtualInverted: fmt.Fprintf(buf, " [virtual-inverted]") case VirtualComputed: - fmt.Fprintf(buf, " [virtual-computed]") + // No need to show anything more (it already shows up as virtual). } } diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 5722b18f5f42..316d236ecc75 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -117,7 +117,7 @@ func (b *Builder) buildDataSource( includeMutations: false, includeSystem: true, includeVirtualInverted: false, - includeVirtualComputed: false, + includeVirtualComputed: true, }), indexFlags, locking, inScope, ) @@ -401,7 +401,7 @@ func (b *Builder) buildScanFromTableRef( includeMutations: false, includeSystem: true, includeVirtualInverted: false, - includeVirtualComputed: false, + includeVirtualComputed: true, }) } @@ -419,9 +419,14 @@ func (b *Builder) addTable(tab cat.Table, alias *tree.TableName) *opt.TableMeta return md.TableMeta(tabID) } -// buildScan builds a memo group for a ScanOp expression on the given table. +// buildScan builds a memo group for a ScanOp expression on the given table. If +// the ordinals list contains any VirtualComputed columns, a ProjectOp is built +// on top. // -// The scan projects the given table ordinals. +// The resulting scope and expression output the given table ordinals. If an +// ordinal is for a VirtualComputed column, the ordinals it depends on must also +// be in the list (in practice, this coincides with all "ordinary" table columns +// being in the list). // // If scanMutationCols is true, then include columns being added or dropped from // the table. These are currently required by the execution engine as "fetch @@ -453,20 +458,26 @@ func (b *Builder) buildScan( outScope = inScope.push() - var tabColIDs opt.ColSet + // We collect VirtualComputed columns separately; these cannot be scanned, + // they can only be projected afterward. + var tabColIDs, virtualColIDs opt.ColSet outScope.cols = make([]scopeColumn, len(ordinals)) for i, ord := range ordinals { col := tab.Column(ord) colID := tabID.ColumnID(ord) - tabColIDs.Add(colID) name := col.ColName() kind := col.Kind() + if kind != cat.VirtualComputed { + tabColIDs.Add(colID) + } else { + virtualColIDs.Add(colID) + } outScope.cols[i] = scopeColumn{ id: colID, name: name, table: tabMeta.Alias, typ: col.DatumType(), - hidden: col.IsHidden() || kind != cat.Ordinary, + hidden: col.IsHidden() || (kind != cat.Ordinary && kind != cat.VirtualComputed), kind: kind, mutation: kind == cat.WriteOnly || kind == cat.DeleteOnly, tableOrdinal: ord, @@ -485,79 +496,96 @@ func (b *Builder) buildScan( private := memo.ScanPrivate{Table: tabID, Cols: tabColIDs} outScope.expr = b.factory.ConstructScan(&private) - // Virtual tables should not be collected as view dependencies. - } else { - private := memo.ScanPrivate{Table: tabID, Cols: tabColIDs} - if indexFlags != nil { - private.Flags.NoIndexJoin = indexFlags.NoIndexJoin - if indexFlags.Index != "" || indexFlags.IndexID != 0 { - idx := -1 - for i := 0; i < tab.IndexCount(); i++ { - if tab.Index(i).Name() == tree.Name(indexFlags.Index) || - tab.Index(i).ID() == cat.StableID(indexFlags.IndexID) { - idx = i - break - } + // Note: virtual tables should not be collected as view dependencies. + return outScope + } + + private := memo.ScanPrivate{Table: tabID, Cols: tabColIDs} + if indexFlags != nil { + private.Flags.NoIndexJoin = indexFlags.NoIndexJoin + if indexFlags.Index != "" || indexFlags.IndexID != 0 { + idx := -1 + for i := 0; i < tab.IndexCount(); i++ { + if tab.Index(i).Name() == tree.Name(indexFlags.Index) || + tab.Index(i).ID() == cat.StableID(indexFlags.IndexID) { + idx = i + break } - if idx == -1 { - var err error - if indexFlags.Index != "" { - err = errors.Errorf("index %q not found", tree.ErrString(&indexFlags.Index)) - } else { - err = errors.Errorf("index [%d] not found", indexFlags.IndexID) - } - panic(err) + } + if idx == -1 { + var err error + if indexFlags.Index != "" { + err = errors.Errorf("index %q not found", tree.ErrString(&indexFlags.Index)) + } else { + err = errors.Errorf("index [%d] not found", indexFlags.IndexID) } - private.Flags.ForceIndex = true - private.Flags.Index = idx - private.Flags.Direction = indexFlags.Direction + panic(err) } + private.Flags.ForceIndex = true + private.Flags.Index = idx + private.Flags.Direction = indexFlags.Direction } - if locking.isSet() { - private.Locking = locking.get() - } - - b.addCheckConstraintsForTable(tabMeta) - b.addComputedColsForTable(tabMeta) + } + if locking.isSet() { + private.Locking = locking.get() + } - outScope.expr = b.factory.ConstructScan(&private) + b.addCheckConstraintsForTable(tabMeta) + b.addComputedColsForTable(tabMeta) + + outScope.expr = b.factory.ConstructScan(&private) + + if !virtualColIDs.Empty() { + // Project the expressions for the virtual columns (and pass through all + // scanned columns). + // TODO(radu): we don't currently support virtual columns depending on other + // virtual columns. + proj := make(memo.ProjectionsExpr, 0, virtualColIDs.Len()) + virtualColIDs.ForEach(func(col opt.ColumnID) { + item := b.factory.ConstructProjectionsItem(tabMeta.ComputedCols[col], col) + if !item.ScalarProps().OuterCols.SubsetOf(tabColIDs) { + panic(errors.AssertionFailedf("scanned virtual column depends on non-scanned column")) + } + proj = append(proj, item) + }) + outScope.expr = b.factory.ConstructProject(outScope.expr, proj, tabColIDs) + } - // Add the partial indexes after constructing the scan so we can use the - // logical properties of the scan to fully normalize the index - // predicates. Partial index predicates are only added if the outScope - // contains all the table's ordinary columns. If it does not, partial - // index predicates cannot be built because they may reference columns - // not in outScope. In the most common case, the outScope has the same - // number of columns as the table and we can skip checking that each - // ordinary column exists in outScope. - containsAllOrdinaryTableColumns := true - if len(outScope.cols) != tab.ColumnCount() { - for i := 0; i < tab.ColumnCount(); i++ { - col := tab.Column(i) - if col.Kind() == cat.Ordinary && !outScope.colSet().Contains(tabID.ColumnID(col.Ordinal())) { - containsAllOrdinaryTableColumns = false - break - } + // Add the partial indexes after constructing the scan so we can use the + // logical properties of the scan to fully normalize the index + // predicates. Partial index predicates are only added if the outScope + // contains all the table's ordinary columns. If it does not, partial + // index predicates cannot be built because they may reference columns + // not in outScope. In the most common case, the outScope has the same + // number of columns as the table and we can skip checking that each + // ordinary column exists in outScope. + containsAllOrdinaryTableColumns := true + if len(outScope.cols) != tab.ColumnCount() { + for i := 0; i < tab.ColumnCount(); i++ { + col := tab.Column(i) + if col.Kind() == cat.Ordinary && !outScope.colSet().Contains(tabID.ColumnID(col.Ordinal())) { + containsAllOrdinaryTableColumns = false + break } } - if containsAllOrdinaryTableColumns { - b.addPartialIndexPredicatesForTable(tabMeta, outScope) - } + } + if containsAllOrdinaryTableColumns { + b.addPartialIndexPredicatesForTable(tabMeta, outScope) + } - if b.trackViewDeps { - dep := opt.ViewDep{DataSource: tab} - dep.ColumnIDToOrd = make(map[opt.ColumnID]int) - // We will track the ColumnID to Ord mapping so Ords can be added - // when a column is referenced. - for i, col := range outScope.cols { - dep.ColumnIDToOrd[col.id] = ordinals[i] - } - if private.Flags.ForceIndex { - dep.SpecificIndex = true - dep.Index = private.Flags.Index - } - b.viewDeps = append(b.viewDeps, dep) + if b.trackViewDeps { + dep := opt.ViewDep{DataSource: tab} + dep.ColumnIDToOrd = make(map[opt.ColumnID]int) + // We will track the ColumnID to Ord mapping so Ords can be added + // when a column is referenced. + for i, col := range outScope.cols { + dep.ColumnIDToOrd[col.id] = ordinals[i] + } + if private.Flags.ForceIndex { + dep.SpecificIndex = true + dep.Index = private.Flags.Index } + b.viewDeps = append(b.viewDeps, dep) } return outScope } diff --git a/pkg/sql/opt/optbuilder/testdata/virtual-columns b/pkg/sql/opt/optbuilder/testdata/virtual-columns new file mode 100644 index 000000000000..44f796fa9992 --- /dev/null +++ b/pkg/sql/opt/optbuilder/testdata/virtual-columns @@ -0,0 +1,37 @@ +exec-ddl +CREATE TABLE t ( + a INT PRIMARY KEY, + b INT, + c INT AS (a+b) VIRTUAL +) +---- + +build +SELECT * FROM t +---- +project + ├── columns: a:1!null b:2 c:3 + └── project + ├── columns: c:3 a:1!null b:2 crdb_internal_mvcc_timestamp:4 + ├── scan t + │ ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:4 + │ └── computed column expressions + │ └── c:3 + │ └── a:1 + b:2 + └── projections + └── a:1 + b:2 [as=c:3] + +build +SELECT c FROM t +---- +project + ├── columns: c:3 + └── project + ├── columns: c:3 a:1!null b:2 crdb_internal_mvcc_timestamp:4 + ├── scan t + │ ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:4 + │ └── computed column expressions + │ └── c:3 + │ └── a:1 + b:2 + └── projections + └── a:1 + b:2 [as=c:3] diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index 50fad1b9c725..7844e00531bb 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -76,32 +76,39 @@ func (tc *Catalog) CreateTable(stmt *tree.CreateTable) *Table { tab.interleaved = true } - // Add non-mutation columns. + // Find the PK columns; we have to force these to be non-nullable. + pkCols := make(map[tree.Name]struct{}) for _, def := range stmt.Defs { switch def := def.(type) { case *tree.ColumnTableDef: - if !isMutationColumn(def) { - tab.addColumn(def) + if def.PrimaryKey.IsPrimaryKey { + pkCols[def.Name] = struct{}{} + } + + case *tree.UniqueConstraintTableDef: + if def.PrimaryKey { + for i := range def.Columns { + pkCols[def.Columns[i].Column] = struct{}{} + } } } } - // If there is no primary index, add the hidden rowid column. - hasPrimaryIndex := false + // Add non-mutation columns. for _, def := range stmt.Defs { switch def := def.(type) { case *tree.ColumnTableDef: - if def.PrimaryKey.IsPrimaryKey { - hasPrimaryIndex = true - } - - case *tree.UniqueConstraintTableDef: - if def.PrimaryKey { - hasPrimaryIndex = true + if !isMutationColumn(def) { + if _, isPKCol := pkCols[def.Name]; isPKCol { + def.Nullable.Nullability = tree.NotNull + } + tab.addColumn(def) } } } + // If there is no primary index, add the hidden rowid column. + hasPrimaryIndex := len(pkCols) > 0 if !hasPrimaryIndex { var rowid cat.Column ordinal := len(tab.Columns) @@ -507,17 +514,29 @@ func (tt *Table) addColumn(def *tree.ColumnTableDef) { } var col cat.Column - col.InitNonVirtual( - ordinal, - cat.StableID(1+ordinal), - name, - kind, - typ, - nullable, - false, /* hidden */ - defaultExpr, - computedExpr, - ) + if def.Computed.Virtual { + col.InitVirtualComputed( + ordinal, + cat.StableID(1+ordinal), + name, + typ, + nullable, + false, /* hidden */ + *computedExpr, + ) + } else { + col.InitNonVirtual( + ordinal, + cat.StableID(1+ordinal), + name, + kind, + typ, + nullable, + false, /* hidden */ + defaultExpr, + computedExpr, + ) + } tt.Columns = append(tt.Columns, col) } @@ -562,31 +581,6 @@ func (tt *Table) addIndexWithVersion( } col := idx.addColumn(tt, colDef, keyCol, isLastIndexCol) - if typ == primaryIndex && col.IsNullable() { - // Reinitialize the column to make it non-nullable. - // TODO(radu): this is very hacky - var defaultExpr, computedExpr *string - if col.HasDefault() { - e := col.DefaultExprStr() - defaultExpr = &e - } - if col.IsComputed() { - e := col.ComputedExprStr() - computedExpr = &e - } - col.InitNonVirtual( - col.Ordinal(), - col.ColID(), - col.ColName(), - col.Kind(), - col.DatumType(), - false, /* nullable */ - col.IsHidden(), - defaultExpr, - computedExpr, - ) - } - if col.IsNullable() { notNullIndex = false } @@ -838,9 +832,11 @@ func columnForIndexElemExpr(tt *Table, expr tree.Expr) cat.Column { var col cat.Column col.InitVirtualComputed( len(tt.Columns), + cat.StableID(1+len(tt.Columns)), name, typ, true, /* nullable */ + true, /* hidden */ exprStr, ) tt.Columns = append(tt.Columns, col) diff --git a/pkg/sql/opt/testutils/testcat/testdata/index b/pkg/sql/opt/testutils/testcat/testdata/index index adf0cc3dcc54..389eabc67ba5 100644 --- a/pkg/sql/opt/testutils/testcat/testdata/index +++ b/pkg/sql/opt/testutils/testcat/testdata/index @@ -122,24 +122,24 @@ TABLE xyz ├── y int ├── z string ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] - ├── idx_expr_1 string as (lower(z)) stored [hidden] [virtual-computed] - ├── idx_expr_2 int as (y + 1) stored [hidden] [virtual-computed] - ├── idx_expr_3 int as (x + y) stored [hidden] [virtual-computed] + ├── idx_expr_1 string as (lower(z)) virtual [hidden] + ├── idx_expr_2 int as (y + 1) virtual [hidden] + ├── idx_expr_3 int as (x + y) virtual [hidden] ├── INDEX primary │ └── x int not null ├── INDEX idx1 - │ ├── idx_expr_1 string as (lower(z)) stored [hidden] [virtual-computed] + │ ├── idx_expr_1 string as (lower(z)) virtual [hidden] │ └── x int not null ├── INDEX idx2 - │ ├── idx_expr_1 string as (lower(z)) stored [hidden] [virtual-computed] + │ ├── idx_expr_1 string as (lower(z)) virtual [hidden] │ ├── y int │ └── x int not null ├── INDEX idx3 - │ ├── idx_expr_2 int as (y + 1) stored [hidden] [virtual-computed] - │ ├── idx_expr_1 string as (lower(z)) stored [hidden] [virtual-computed] + │ ├── idx_expr_2 int as (y + 1) virtual [hidden] + │ ├── idx_expr_1 string as (lower(z)) virtual [hidden] │ └── x int not null ├── INDEX idx4 - │ ├── idx_expr_3 int as (x + y) stored [hidden] [virtual-computed] + │ ├── idx_expr_3 int as (x + y) virtual [hidden] │ ├── y int │ ├── x int not null │ ├── z string (storing) diff --git a/pkg/sql/opt/testutils/testcat/testdata/table b/pkg/sql/opt/testutils/testcat/testdata/table index 18e3f66eaa61..45fcbd7c5a03 100644 --- a/pkg/sql/opt/testutils/testcat/testdata/table +++ b/pkg/sql/opt/testutils/testcat/testdata/table @@ -23,7 +23,8 @@ CREATE TABLE abcdef ( c INT DEFAULT (10), d INT AS (b + c + 1) STORED, e INT AS (a) STORED, - f INT CHECK (f > 2) + f INT CHECK (f > 2), + g INT AS (a+b) VIRTUAL ) ---- @@ -37,6 +38,7 @@ TABLE abcdef ├── d int as ((b + c) + 1) stored ├── e int as (a) stored ├── f int + ├── g int as (a + b) virtual ├── rowid int not null default (unique_rowid()) [hidden] ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── CHECK (f > 2) diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index 2f1f16c40e68..f3797c30505f 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -280,6 +280,7 @@ func TestParse(t *testing.T) { {`CREATE TABLE a.b (b INT8)`}, {`CREATE TABLE IF NOT EXISTS a (b INT8)`}, {`CREATE TABLE a (b INT8 AS (a + b) STORED)`}, + {`CREATE TABLE a (b INT8 AS (a + b) VIRTUAL)`}, {`CREATE TABLE view (view INT8)`}, {`CREATE TABLE a (b INT8 CONSTRAINT c PRIMARY KEY)`}, @@ -2720,6 +2721,7 @@ SKIP_MISSING_FOREIGN_KEYS, SKIP_MISSING_SEQUENCES, SKIP_MISSING_SEQUENCE_OWNERS, `CREATE TABLE a (b INT8, c STRING, FOREIGN KEY (b, c) REFERENCES other (x, y) ON DELETE CASCADE ON UPDATE SET NULL)`, }, {`CREATE TABLE a (b INT8 GENERATED ALWAYS AS (a + b) STORED)`, `CREATE TABLE a (b INT8 AS (a + b) STORED)`}, + {`CREATE TABLE a (b INT8 GENERATED ALWAYS AS (a + b) VIRTUAL)`, `CREATE TABLE a (b INT8 AS (a + b) VIRTUAL)`}, {`ALTER TABLE a ALTER b DROP STORED`, `ALTER TABLE a ALTER COLUMN b DROP STORED`}, {`ALTER TABLE a ADD b INT8`, `ALTER TABLE a ADD COLUMN b INT8`}, @@ -3186,7 +3188,6 @@ func TestUnimplementedSyntax(t *testing.T) { {`CREATE TABLE a AS SELECT b WITH NO DATA`, 0, `create table as with no data`, ``}, - {`CREATE TABLE a(b INT8 AS (123) VIRTUAL)`, 0, `virtual computed columns`, ``}, {`CREATE TABLE a(b INT8 REFERENCES c(x) MATCH PARTIAL`, 20305, `match partial`, ``}, {`CREATE TABLE a(b INT8, FOREIGN KEY (b) REFERENCES c(x) MATCH PARTIAL)`, 20305, `match partial`, ``}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 32d97929bfc6..65f1b1d7d521 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -6104,11 +6104,11 @@ col_qualification_elem: } | generated_as '(' a_expr ')' STORED { - $$.val = &tree.ColumnComputedDef{Expr: $3.expr()} + $$.val = &tree.ColumnComputedDef{Expr: $3.expr(), Virtual: false} } | generated_as '(' a_expr ')' VIRTUAL { - return unimplemented(sqllex, "virtual computed columns") + $$.val = &tree.ColumnComputedDef{Expr: $3.expr(), Virtual: true} } | generated_as error { diff --git a/pkg/sql/sem/tree/create.go b/pkg/sql/sem/tree/create.go index 7b3ce72a6ab8..f98bcc900f9e 100644 --- a/pkg/sql/sem/tree/create.go +++ b/pkg/sql/sem/tree/create.go @@ -405,6 +405,7 @@ type ColumnTableDef struct { Computed struct { Computed bool Expr Expr + Virtual bool } Family struct { Name Name @@ -532,6 +533,7 @@ func NewColumnTableDef( case *ColumnComputedDef: d.Computed.Computed = true d.Computed.Expr = t.Expr + d.Computed.Virtual = t.Virtual case *ColumnFamilyConstraint: if d.HasColumnFamily() { return nil, pgerror.Newf(pgcode.InvalidTableDefinition, @@ -562,6 +564,11 @@ func (node *ColumnTableDef) IsComputed() bool { return node.Computed.Computed } +// IsVirtual returns if the ColumnTableDef is a virtual column. +func (node *ColumnTableDef) IsVirtual() bool { + return node.Computed.Virtual +} + // HasColumnFamily returns if the ColumnTableDef has a column family. func (node *ColumnTableDef) HasColumnFamily() bool { return node.Family.Name != "" || node.Family.Create @@ -644,7 +651,11 @@ func (node *ColumnTableDef) Format(ctx *FmtCtx) { if node.IsComputed() { ctx.WriteString(" AS (") ctx.FormatNode(node.Computed.Expr) - ctx.WriteString(") STORED") + if node.Computed.Virtual { + ctx.WriteString(") VIRTUAL") + } else { + ctx.WriteString(") STORED") + } } if node.HasColumnFamily() { if node.Family.Create { @@ -748,7 +759,8 @@ type ColumnFKConstraint struct { // ColumnComputedDef represents the description of a computed column. type ColumnComputedDef struct { - Expr Expr + Expr Expr + Virtual bool } // ColumnFamilyConstraint represents FAMILY on a column. diff --git a/pkg/sql/sem/tree/pretty.go b/pkg/sql/sem/tree/pretty.go index 4c766e6f50c0..9e5d7c1862f7 100644 --- a/pkg/sql/sem/tree/pretty.go +++ b/pkg/sql/sem/tree/pretty.go @@ -1819,8 +1819,19 @@ func (node *ColumnTableDef) docRow(p *PrettyCfg) pretty.TableRow { // Compute expression (for computed columns). if node.IsComputed() { - clauses = append(clauses, pretty.ConcatSpace(pretty.Keyword("AS"), - p.bracket("(", p.Doc(node.Computed.Expr), ") STORED"), + var typ string + if node.Computed.Virtual { + typ = "VIRTUAL" + } else { + typ = "STORED" + } + + clauses = append(clauses, pretty.ConcatSpace( + pretty.Keyword("AS"), + pretty.ConcatSpace( + p.bracket("(", p.Doc(node.Computed.Expr), ")"), + pretty.Keyword(typ), + ), )) } diff --git a/pkg/sql/sem/tree/testdata/pretty/create_table.align-deindent.golden.short b/pkg/sql/sem/tree/testdata/pretty/create_table.align-deindent.golden.short index 83ff6a889728..c8ee1977e3c3 100644 --- a/pkg/sql/sem/tree/testdata/pretty/create_table.align-deindent.golden.short +++ b/pkg/sql/sem/tree/testdata/pretty/create_table.align-deindent.golden.short @@ -42,6 +42,11 @@ CREATE TABLE product_information ( last_name ) ) STORED, + virt_col INT8 + AS ( + product_id + * 10 + ) VIRTUAL, INDEX date_added_idx (date_added), INDEX supp_id_prod_status_idx ( supplier_id, diff --git a/pkg/sql/sem/tree/testdata/pretty/create_table.align-only.golden.short b/pkg/sql/sem/tree/testdata/pretty/create_table.align-only.golden.short index 83ff6a889728..c8ee1977e3c3 100644 --- a/pkg/sql/sem/tree/testdata/pretty/create_table.align-only.golden.short +++ b/pkg/sql/sem/tree/testdata/pretty/create_table.align-only.golden.short @@ -42,6 +42,11 @@ CREATE TABLE product_information ( last_name ) ) STORED, + virt_col INT8 + AS ( + product_id + * 10 + ) VIRTUAL, INDEX date_added_idx (date_added), INDEX supp_id_prod_status_idx ( supplier_id, diff --git a/pkg/sql/sem/tree/testdata/pretty/create_table.ref.golden.short b/pkg/sql/sem/tree/testdata/pretty/create_table.ref.golden.short index 640f9518c869..f240f0d906ee 100644 --- a/pkg/sql/sem/tree/testdata/pretty/create_table.ref.golden.short +++ b/pkg/sql/sem/tree/testdata/pretty/create_table.ref.golden.short @@ -46,6 +46,9 @@ CREATE TABLE product_information ( last_name ) ) STORED, + virt_col + INT8 + AS (product_id * 10) VIRTUAL, INDEX date_added_idx (date_added), INDEX supp_id_prod_status_idx ( supplier_id, diff --git a/pkg/sql/sem/tree/testdata/pretty/create_table.sql b/pkg/sql/sem/tree/testdata/pretty/create_table.sql index b36957c0b226..29278ce60c8b 100644 --- a/pkg/sql/sem/tree/testdata/pretty/create_table.sql +++ b/pkg/sql/sem/tree/testdata/pretty/create_table.sql @@ -13,6 +13,7 @@ CREATE TABLE product_information ( date_added DATE DEFAULT current_date(), misc JSONB, full_name STRING AS (concat(first_name, ' ', last_name)) STORED, + virt_col INT AS (product_id * 10) VIRTUAL, INDEX date_added_idx (date_added), INDEX supp_id_prod_status_idx (supplier_id, product_status), customer_id INT REFERENCES customers_2(id) MATCH FULL ON UPDATE CASCADE ON DELETE CASCADE,