Skip to content

Commit

Permalink
cat: add "virtual computed" columns
Browse files Browse the repository at this point in the history
This change splits the Virtual column kind into the existing
VirtualInverted and a new VirtualComputed kind. The latter is a
virtual column that corresponds to an expression, to be used for
expression-based indexes.

Release note: None
  • Loading branch information
RaduBerinde authored and jayshrivastava committed Oct 8, 2020
1 parent 8d61e7f commit 3ee7ca7
Show file tree
Hide file tree
Showing 13 changed files with 107 additions and 63 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/exec_factory_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,7 @@ func makeScanColumnsConfig(table cat.Table, cols exec.TableColumnOrdinalSet) sca
}
for ord, ok := cols.Next(0); ok; ord, ok = cols.Next(ord + 1) {
col := table.Column(ord)
if col.Kind() == cat.Virtual {
if col.Kind() == cat.VirtualInverted {
col = table.Column(col.InvertedSourceColumnOrdinal())
}
colCfg.wantedColumns = append(colCfg.wantedColumns, tree.ColumnID(col.ColID()))
Expand Down
49 changes: 39 additions & 10 deletions pkg/sql/opt/cat/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func (c *Column) Ordinal() int {
// Virtual columns don't have stable IDs; for these columns ColID() must not be
// called.
func (c *Column) ColID() StableID {
if c.kind == Virtual {
if c.kind.IsVirtual() {
panic(errors.AssertionFailedf("virtual columns have no StableID"))
}
return c.stableID
Expand Down Expand Up @@ -134,7 +134,7 @@ func (c *Column) ComputedExprStr() string {
//
// Must not be called if this is not a virtual column.
func (c *Column) InvertedSourceColumnOrdinal() int {
if c.kind != Virtual {
if c.kind != VirtualInverted {
panic(errors.AssertionFailedf("non-virtual columns have no inverted source column ordinal"))
}
return c.invertedSourceColumnOrdinal
Expand All @@ -158,11 +158,20 @@ const (
// as part of mutations. They also cannot be part of the lax or key columns
// for indexes. System columns are not members of any column family.
System
// Virtual columns are implicit columns that are used by inverted indexes (and
// later, expression-based indexes).
Virtual
// 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
)

// IsVirtual returns true if the column kind is VirtualInverted or
// VirtualComputed.
func (k ColumnKind) IsVirtual() bool {
return k == VirtualInverted || k == VirtualComputed
}

// InitNonVirtual is used by catalog implementations to populate a non-virtual
// Column. It should not be used anywhere else.
func (c *Column) InitNonVirtual(
Expand All @@ -176,7 +185,7 @@ func (c *Column) InitNonVirtual(
defaultExpr *string,
computedExpr *string,
) {
if kind == Virtual {
if kind.IsVirtual() {
panic(errors.AssertionFailedf("incorrect init method"))
}
c.ordinal = ordinal
Expand All @@ -199,19 +208,39 @@ func (c *Column) InitNonVirtual(
c.invertedSourceColumnOrdinal = -1
}

// InitVirtual is used by catalog implementations to populate a virtual Column.
// It should not be used anywhere else.
func (c *Column) InitVirtual(
// InitVirtualInverted is used by catalog implementations to populate a
// VirtualInverted Column. It should not be used anywhere else.
func (c *Column) InitVirtualInverted(
ordinal int, name tree.Name, datumType *types.T, nullable bool, invertedSourceColumnOrdinal int,
) {
c.ordinal = ordinal
c.stableID = 0
c.name = name
c.kind = Virtual
c.kind = VirtualInverted
c.datumType = datumType
c.nullable = nullable
c.hidden = true
c.defaultExpr = ""
c.computedExpr = ""
c.invertedSourceColumnOrdinal = invertedSourceColumnOrdinal
}

// 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,
) {
c.ordinal = ordinal
c.stableID = 0
c.name = name
c.kind = VirtualComputed
c.datumType = datumType
c.nullable = nullable
c.hidden = true
c.defaultExpr = ""
c.computedExpr = computedExpr
c.invertedSourceColumnOrdinal = -1
}

// Quiet the linter until this is used.
var _ = (*Column).InitVirtualComputed
6 changes: 4 additions & 2 deletions pkg/sql/opt/cat/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -307,8 +307,10 @@ func formatColumn(col *Column, buf *bytes.Buffer) {
fmt.Fprintf(buf, " [mutation]")
case System:
fmt.Fprintf(buf, " [system]")
case Virtual:
fmt.Fprintf(buf, " [virtual]")
case VirtualInverted:
fmt.Fprintf(buf, " [virtual-inverted]")
case VirtualComputed:
fmt.Fprintf(buf, " [virtual-computed]")
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/memo/check_expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ func (m *Memo) CheckExpr(e opt.Expr) {
if (kind == cat.Ordinary || kind == cat.WriteOnly) && t.InsertCols[i] == 0 {
panic(errors.AssertionFailedf("insert values not provided for all table columns"))
}
if (kind == cat.System || kind == cat.Virtual) && t.InsertCols[i] != 0 {
if (kind == cat.System || kind.IsVirtual()) && t.InsertCols[i] != 0 {
panic(errors.AssertionFailedf("system or virtual column found in insertion columns"))
}
}
Expand Down
14 changes: 8 additions & 6 deletions pkg/sql/opt/optbuilder/fk_cascade.go
Original file line number Diff line number Diff line change
Expand Up @@ -285,9 +285,10 @@ func (b *Builder) buildDeleteCascadeMutationInput(
outScope = b.buildScan(
b.addTable(childTable, childTableAlias),
tableOrdinals(childTable, columnKinds{
includeMutations: false,
includeSystem: false,
includeVirtual: false,
includeMutations: false,
includeSystem: false,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
nil, /* indexFlags */
noRowLocking,
Expand Down Expand Up @@ -531,9 +532,10 @@ func (b *Builder) buildUpdateCascadeMutationInput(
outScope = b.buildScan(
b.addTable(childTable, childTableAlias),
tableOrdinals(childTable, columnKinds{
includeMutations: false,
includeSystem: false,
includeVirtual: false,
includeMutations: false,
includeSystem: false,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
nil, /* indexFlags */
noRowLocking,
Expand Down
23 changes: 13 additions & 10 deletions pkg/sql/opt/optbuilder/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -373,7 +373,7 @@ func (mb *mutationBuilder) needExistingRows() bool {
// #1: Don't consider key columns.
continue
}
if kind := mb.tab.Column(i).Kind(); kind == cat.System || kind == cat.Virtual {
if kind := mb.tab.Column(i).Kind(); kind == cat.System || kind.IsVirtual() {
// #2: Don't consider system or virtual columns.
continue
}
Expand Down Expand Up @@ -701,9 +701,10 @@ func (mb *mutationBuilder) buildInputForDoNothing(
fetchScope := mb.b.buildScan(
mb.b.addTable(mb.tab, &mb.alias),
tableOrdinals(mb.tab, columnKinds{
includeMutations: false,
includeSystem: false,
includeVirtual: false,
includeMutations: false,
includeSystem: false,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
nil, /* indexFlags */
noRowLocking,
Expand Down Expand Up @@ -903,9 +904,10 @@ func (mb *mutationBuilder) buildInputForUpsert(
fetchScope := mb.b.buildScan(
mb.b.addTable(mb.tab, &mb.alias),
tableOrdinals(mb.tab, columnKinds{
includeMutations: true,
includeSystem: true,
includeVirtual: false,
includeMutations: true,
includeSystem: true,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
nil, /* indexFlags */
noRowLocking,
Expand Down Expand Up @@ -1248,9 +1250,10 @@ func (mb *mutationBuilder) arbiterIndexes(
if tableScope == nil {
tableScope = mb.b.buildScan(
tabMeta, tableOrdinals(tabMeta.Table, columnKinds{
includeMutations: false,
includeSystem: false,
includeVirtual: false,
includeMutations: false,
includeSystem: false,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
nil, /* indexFlags */
noRowLocking,
Expand Down
16 changes: 9 additions & 7 deletions pkg/sql/opt/optbuilder/mutation_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,9 +254,10 @@ func (mb *mutationBuilder) buildInputForUpdate(
scanScope := mb.b.buildScan(
mb.b.addTable(mb.tab, &mb.alias),
tableOrdinals(mb.tab, columnKinds{
includeMutations: true,
includeSystem: true,
includeVirtual: false,
includeMutations: true,
includeSystem: true,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
indexFlags,
noRowLocking,
Expand Down Expand Up @@ -364,9 +365,10 @@ func (mb *mutationBuilder) buildInputForDelete(
scanScope := mb.b.buildScan(
mb.b.addTable(mb.tab, &mb.alias),
tableOrdinals(mb.tab, columnKinds{
includeMutations: true,
includeSystem: true,
includeVirtual: false,
includeMutations: true,
includeSystem: true,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
indexFlags,
noRowLocking,
Expand Down Expand Up @@ -563,7 +565,7 @@ func (mb *mutationBuilder) addSynthesizedCols(colIDs opt.ColList, addCol func(co
continue
}
// Skip system and virtual columns.
if kind == cat.System || kind == cat.Virtual {
if kind == cat.System || kind.IsVirtual() {
continue
}
// Skip columns that are already specified.
Expand Down
14 changes: 8 additions & 6 deletions pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,9 +114,10 @@ func (b *Builder) buildDataSource(
return b.buildScan(
tabMeta,
tableOrdinals(t, columnKinds{
includeMutations: false,
includeSystem: true,
includeVirtual: false,
includeMutations: false,
includeSystem: true,
includeVirtualInverted: false,
includeVirtualComputed: false,
}),
indexFlags, locking, inScope,
)
Expand Down Expand Up @@ -397,9 +398,10 @@ func (b *Builder) buildScanFromTableRef(
ordinals = resolveNumericColumnRefs(tab, ref.Columns)
} else {
ordinals = tableOrdinals(tab, columnKinds{
includeMutations: false,
includeSystem: true,
includeVirtual: false,
includeMutations: false,
includeSystem: true,
includeVirtualInverted: false,
includeVirtualComputed: false,
})
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/optbuilder/testdata/inverted-indexes
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,11 @@ TABLE kj
├── k int not null
├── j jsonb
├── crdb_internal_mvcc_timestamp decimal [hidden] [system]
├── j_inverted_key jsonb not null [hidden] [virtual]
├── j_inverted_key jsonb not null [hidden] [virtual-inverted]
├── INDEX primary
│ └── k int not null
└── INVERTED INDEX secondary
├── j_inverted_key jsonb not null [hidden] [virtual]
├── j_inverted_key jsonb not null [hidden] [virtual-inverted]
└── k int not null

build
Expand Down
18 changes: 11 additions & 7 deletions pkg/sql/opt/optbuilder/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -682,20 +682,24 @@ type columnKinds struct {
// If true, include system columns.
includeSystem bool

// If true, include virtual columns.
includeVirtual bool
// If true, include virtual inverted index columns.
includeVirtualInverted bool

// If true, include virtual computed columns.
includeVirtualComputed bool
}

// tableOrdinals returns a slice of ordinals that correspond to table columns of
// the desired kinds.
func tableOrdinals(tab cat.Table, k columnKinds) []int {
n := tab.ColumnCount()
shouldInclude := [...]bool{
cat.Ordinary: true,
cat.WriteOnly: k.includeMutations,
cat.DeleteOnly: k.includeMutations,
cat.System: k.includeSystem,
cat.Virtual: k.includeVirtual,
cat.Ordinary: true,
cat.WriteOnly: k.includeMutations,
cat.DeleteOnly: k.includeMutations,
cat.System: k.includeSystem,
cat.VirtualInverted: k.includeVirtualInverted,
cat.VirtualComputed: k.includeVirtualComputed,
}
ordinals := make([]int, 0, n)
for i := 0; i < n; i++ {
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/testutils/testcat/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -567,7 +567,7 @@ func (tt *Table) addIndex(def *tree.IndexTableDef, typ indexType) *Index {
}
// Add the rest of the columns in the table.
for i, col := range tt.Columns {
if !pkOrdinals.Contains(i) && col.Kind() != cat.Virtual {
if !pkOrdinals.Contains(i) && !col.Kind().IsVirtual() {
idx.addColumnByOrdinal(tt, i, tree.Ascending, nonKeyCol)
}
}
Expand Down Expand Up @@ -686,7 +686,7 @@ func (ti *Index) addColumn(
// TODO(radu,mjibson): update this when the corresponding type in the real
// catalog is fixed (see sql.newOptTable).
typ := tt.Columns[ordinal].DatumType()
col.InitVirtual(
col.InitVirtualInverted(
len(tt.Columns),
tree.Name(name+"_inverted_key"),
typ,
Expand Down
16 changes: 8 additions & 8 deletions pkg/sql/opt/testutils/testcat/testdata/table
Original file line number Diff line number Diff line change
Expand Up @@ -177,15 +177,15 @@ TABLE inv
├── j jsonb
├── g geometry
├── crdb_internal_mvcc_timestamp decimal [hidden] [system]
├── j_inverted_key jsonb not null [hidden] [virtual]
├── g_inverted_key geometry not null [hidden] [virtual]
├── j_inverted_key jsonb not null [hidden] [virtual-inverted]
├── g_inverted_key geometry not null [hidden] [virtual-inverted]
├── INDEX primary
│ └── k int not null
├── INVERTED INDEX secondary
│ ├── j_inverted_key jsonb not null [hidden] [virtual]
│ ├── j_inverted_key jsonb not null [hidden] [virtual-inverted]
│ └── k int not null
└── INVERTED INDEX secondary
├── g_inverted_key geometry not null [hidden] [virtual]
├── g_inverted_key geometry not null [hidden] [virtual-inverted]
└── k int not null

# Table with inverted indexes and implicit primary index.
Expand All @@ -208,13 +208,13 @@ TABLE inv2
├── g geometry
├── rowid int not null default (unique_rowid()) [hidden]
├── crdb_internal_mvcc_timestamp decimal [hidden] [system]
├── j_inverted_key jsonb not null [hidden] [virtual]
├── g_inverted_key geometry not null [hidden] [virtual]
├── j_inverted_key jsonb not null [hidden] [virtual-inverted]
├── g_inverted_key geometry not null [hidden] [virtual-inverted]
├── INDEX primary
│ └── rowid int not null default (unique_rowid()) [hidden]
├── INVERTED INDEX secondary
│ ├── j_inverted_key jsonb not null [hidden] [virtual]
│ ├── j_inverted_key jsonb not null [hidden] [virtual-inverted]
│ └── rowid int not null default (unique_rowid()) [hidden]
└── INVERTED INDEX secondary
├── g_inverted_key geometry not null [hidden] [virtual]
├── g_inverted_key geometry not null [hidden] [virtual-inverted]
└── rowid int not null default (unique_rowid()) [hidden]
2 changes: 1 addition & 1 deletion pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -737,7 +737,7 @@ func newOptTable(
// read data directly into a DBytes (i.e., don't call
// encoding.DecodeBytesAscending).
typ := ot.Column(invertedSourceColOrdinal).DatumType()
virtualCol.InitVirtual(
virtualCol.InitVirtualInverted(
virtualColOrd,
tree.Name(string(ot.Column(invertedSourceColOrdinal).ColName())+"_inverted_key"),
typ,
Expand Down

0 comments on commit 3ee7ca7

Please sign in to comment.