Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
49 changes: 47 additions & 2 deletions pkg/sql/catalog/schemaexpr/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,51 @@ func iterColDescriptors(
return err
}

// iterColsWithLookupFn iterates over the expression's variable columns and
// calls f on each, providing the column information from the lookup function.
//
// If the expression references a column that does not exist according to the
// lookup function, iterColsWithLookupFn errs with pgcode.UndefinedColumn.
//
// The column lookup function allows looking up columns both in the descriptor
// or in declarative schema changer elements.
func iterColsWithLookupFn(
rootExpr tree.Expr,
columnLookupFn ColumnLookupFn,
f func(columnName tree.Name, id catid.ColumnID, typ *types.T, isAccessible, isComputed bool) error,
) error {
_, err := tree.SimpleVisit(rootExpr, func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) {
vBase, ok := expr.(tree.VarName)
if !ok {
// Not a VarName, don't do anything to this node.
return true, expr, nil
}

v, err := vBase.NormalizeVarName()
if err != nil {
return false, nil, err
}

c, ok := v.(*tree.ColumnItem)
if !ok {
return true, expr, nil
}

colExists, colIsAccessible, isComputed, colID, colType := columnLookupFn(c.ColumnName)
if !colExists {
return false, nil, pgerror.Newf(pgcode.UndefinedColumn,
"column %q does not exist, referenced in %q", c.ColumnName, rootExpr.String())
}

if err := f(c.ColumnName, colID, colType, colIsAccessible, isComputed); err != nil {
return false, nil, err
}
return false, expr, err
})

return err
}

// dummyColumn represents a variable column that can type-checked. It is used
// in validating check constraint and partial index predicate expressions. This
// validation requires that the expression can be both both typed-checked and
Expand Down Expand Up @@ -261,7 +306,7 @@ func (d *dummyColumn) ResolvedType() *types.T {
return d.typ
}

type ColumnLookupFn func(columnName tree.Name) (exists bool, accessible bool, id catid.ColumnID, typ *types.T)
type ColumnLookupFn func(columnName tree.Name) (exists, accessible, computed bool, id catid.ColumnID, typ *types.T)

// ReplaceColumnVars replaces the occurrences of column names in an expression with
// dummyColumns containing their type, so that they may be type-checked. It
Expand Down Expand Up @@ -295,7 +340,7 @@ func ReplaceColumnVars(
return true, expr, nil
}

colExists, colIsAccessible, colID, colType := columnLookupFn(c.ColumnName)
colExists, colIsAccessible, _, colID, colType := columnLookupFn(c.ColumnName)
if !colExists {
return false, nil, pgerror.Newf(pgcode.UndefinedColumn,
"column %q does not exist, referenced in %q", c.ColumnName, rootExpr.String())
Expand Down
159 changes: 114 additions & 45 deletions pkg/sql/catalog/schemaexpr/computed_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,12 @@ import (

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/parserutils"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/transform"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand All @@ -36,6 +38,9 @@ import (
//
// - It does not have a default value.
// - It does not reference other computed columns.
// - It does not reference inaccessible columns.
// - It does not depend on the region column name if the table is REGIONAL BY
// ROW and uses a foreign key to populate the region column.
//
// TODO(mgartner): Add unit tests for Validate.
func ValidateComputedColumnExpression(
Expand All @@ -47,69 +52,53 @@ func ValidateComputedColumnExpression(
semaCtx *tree.SemaContext,
version clusterversion.ClusterVersion,
) (serializedExpr string, _ *types.T, _ error) {
if d.HasDefaultExpr() {
return "", nil, pgerror.Newf(
pgcode.InvalidTableDefinition,
"%s cannot have default values",
context,
)
}

var depColIDs catalog.TableColSet
// First, check that no column in the expression is an inaccessible or
// computed column.
err := iterColDescriptors(desc, d.Computed.Expr, func(c catalog.Column) error {
if c.IsInaccessible() {
return pgerror.Newf(
pgcode.UndefinedColumn,
"column %q is inaccessible and cannot be referenced in a computed column expression",
c.GetName(),
)
}
if c.IsComputed() {
return pgerror.Newf(
pgcode.InvalidTableDefinition,
"%s expression cannot reference computed columns",
context,
)
// Create helper functions from the descriptor and delegate to the
// lookup-based validation function.
getAllNonDropColumnsFn := func() colinfo.ResultColumns {
cols := desc.NonDropColumns()
ret := make(colinfo.ResultColumns, len(cols))
for i, col := range cols {
ret[i] = colinfo.ResultColumn{
Name: col.GetName(),
Typ: col.GetType(),
Hidden: col.IsHidden(),
TableID: desc.GetID(),
PGAttributeNum: uint32(col.GetPGAttributeNum()),
}
}
depColIDs.Add(c.GetID())

return nil
})
if err != nil {
return "", nil, err
return ret
}

// Resolve the type of the computed column expression.
defType, err := tree.ResolveType(ctx, d.Type, semaCtx.GetTypeResolver())
if err != nil {
return "", nil, err
}
columnLookupFn := makeColumnLookupFnForTableDesc(desc)

// Check that the type of the expression is of type defType and that there
// are no variable expressions (besides dummyColumnItems) and no impure
// functions. In order to safely serialize user defined types and their
// members, we need to serialize the typed expression here.
expr, typ, _, err := DequalifyAndValidateExpr(
serializedExpr, typ, err := ValidateComputedColumnExpressionWithLookup(
ctx,
desc,
d.Computed.Expr,
defType,
d,
tn,
context,
semaCtx,
volatility.Immutable,
tn,
version,
getAllNonDropColumnsFn,
columnLookupFn,
)
if err != nil {
return "", nil, err
}

var depColIDs catalog.TableColSet
if err := iterColDescriptors(desc, d.Computed.Expr, func(c catalog.Column) error {
depColIDs.Add(c.GetID())
return nil
}); err != nil {
return "", nil, err
}

// Virtual computed columns must not refer to mutation columns because it
// would not be safe in the case that the mutation column was being
// backfilled and the virtual computed column value needed to be computed
// for the purpose of writing to a secondary index.
// This check is specific to the legacy schema changer.
if d.IsVirtual() {
var mutationColumnNames []string
var err error
Expand Down Expand Up @@ -144,6 +133,86 @@ func ValidateComputedColumnExpression(
}
}

return serializedExpr, typ, nil
}

// ValidateComputedColumnExpressionWithLookup verifies that an expression is a
// valid computed column expression using a column lookup function. It returns
// the serialized expression and its type if valid, and an error otherwise.
//
// This is similar to ValidateComputedColumnExpression but uses a ColumnLookupFn
// instead of a catalog.TableDescriptor, allowing it to work with declarative
// schema changer elements.
func ValidateComputedColumnExpressionWithLookup(
ctx context.Context,
desc catalog.TableDescriptor,
d *tree.ColumnTableDef,
tn *tree.TableName,
context tree.SchemaExprContext,
semaCtx *tree.SemaContext,
version clusterversion.ClusterVersion,
getAllNonDropColumnsFn func() colinfo.ResultColumns,
columnLookupFn ColumnLookupFn,
) (serializedExpr string, _ *types.T, _ error) {
if d.HasDefaultExpr() {
return "", nil, pgerror.Newf(
pgcode.InvalidTableDefinition,
"%s cannot have default values",
context,
)
}

var depColIDs catalog.TableColSet
// First, check that no column in the expression is an inaccessible or
// computed column.
err := iterColsWithLookupFn(d.Computed.Expr, columnLookupFn,
func(columnName tree.Name, id catid.ColumnID, typ *types.T, isAccessible, isComputed bool) error {
if !isAccessible {
return pgerror.Newf(
pgcode.UndefinedColumn,
"column %q is inaccessible and cannot be referenced in a computed column expression",
columnName,
)
}
if isComputed {
return pgerror.Newf(
pgcode.InvalidTableDefinition,
"%s expression cannot reference computed columns",
context,
)
}
depColIDs.Add(id)
return nil
})
if err != nil {
return "", nil, err
}

// Resolve the type of the computed column expression.
defType, err := tree.ResolveType(ctx, d.Type, semaCtx.GetTypeResolver())
if err != nil {
return "", nil, err
}

// Check that the type of the expression is of type defType and that there
// are no variable expressions (besides dummyColumnItems) and no impure
// functions. We use DequalifyAndValidateExprImpl with the lookup function.
expr, typ, _, err := DequalifyAndValidateExprImpl(
ctx,
d.Computed.Expr,
defType,
context,
semaCtx,
volatility.Immutable,
tn,
version,
getAllNonDropColumnsFn,
columnLookupFn,
)
if err != nil {
return "", nil, err
}

// If this is a REGIONAL BY ROW table using a foreign key to populate the
// region column, we need to check that the expression does not reference
// the region column. This is because the values of every (possibly computed)
Expand Down
18 changes: 9 additions & 9 deletions pkg/sql/catalog/schemaexpr/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,12 +113,12 @@ func DequalifyAndValidateExpr(
getAllNonDropColumnsFn := func() colinfo.ResultColumns {
return colinfo.ResultColumnsFromColumns(desc.GetID(), desc.NonDropColumns())
}
columnLookupByNameFn := func(columnName tree.Name) (exists bool, accessible bool, id catid.ColumnID, typ *types.T) {
columnLookupByNameFn := func(columnName tree.Name) (exists, accessible, computed bool, id catid.ColumnID, typ *types.T) {
col, err := catalog.MustFindColumnByTreeName(desc, columnName)
if err != nil || col.Dropped() {
return false, false, 0, nil
return false, false, false, 0, nil
}
return true, !col.IsInaccessible(), col.GetID(), col.GetType()
return true, !col.IsInaccessible(), col.IsComputed(), col.GetID(), col.GetType()
}

return DequalifyAndValidateExprImpl(ctx, expr, typ, context, semaCtx, maxVolatility, tn, version,
Expand Down Expand Up @@ -249,12 +249,12 @@ func FormatExprForExpressionIndexDisplay(
}

func makeColumnLookupFnForTableDesc(desc catalog.TableDescriptor) ColumnLookupFn {
return func(columnName tree.Name) (exists bool, accessible bool, id catid.ColumnID, typ *types.T) {
return func(columnName tree.Name) (exists, accessible, computed bool, id catid.ColumnID, typ *types.T) {
col, err := catalog.MustFindColumnByTreeName(desc, columnName)
if err != nil || col.Dropped() {
return false, false, 0, nil
return false, false, false, 0, nil
}
return true, !col.IsInaccessible(), col.GetID(), col.GetType()
return true, !col.IsInaccessible(), col.IsComputed(), col.GetID(), col.GetType()
}
}

Expand All @@ -269,14 +269,14 @@ func ParseTriggerWhenExprForDisplay(
semaCtx *tree.SemaContext,
fmtFlags tree.FmtFlags,
) (tree.Expr, error) {
lookupFn := func(columnName tree.Name) (exists bool, accessible bool, id catid.ColumnID, typ *types.T) {
lookupFn := func(columnName tree.Name) (exists, accessible, computed bool, id catid.ColumnID, typ *types.T) {
// Trigger WHEN expressions can reference only the special OLD and NEW
// columns.
switch columnName {
case "old", "new":
return true, true, 0, tableTyp
return true, true, false, 0, tableTyp
}
return false, false, 0, nil
return false, false, false, 0, nil
}
return parseExprForDisplayImpl(
ctx,
Expand Down
18 changes: 8 additions & 10 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -838,29 +838,27 @@ func (b *builderState) WrapExpression(tableID catid.DescID, expr tree.Expr) *scp

// ComputedColumnExpression implements the scbuildstmt.TableHelpers interface.
func (b *builderState) ComputedColumnExpression(
tbl *scpb.Table, d *tree.ColumnTableDef, exprContext tree.SchemaExprContext,
tbl *scpb.Table,
d *tree.ColumnTableDef,
exprContext tree.SchemaExprContext,
getAllNonDropColumnsFn func() colinfo.ResultColumns,
columnLookupByNameFn schemaexpr.ColumnLookupFn,
) (tree.Expr, *types.T) {
_, _, ns := scpb.FindNamespace(b.QueryByID(tbl.TableID))
tn := tree.MakeTableNameFromPrefix(b.NamePrefix(tbl), tree.Name(ns.Name))
b.ensureDescriptor(tbl.TableID)
// TODO(postamar): this doesn't work when referencing newly added columns.
expr, typ, err := schemaexpr.ValidateComputedColumnExpression(
expr, typ, err := schemaexpr.ValidateComputedColumnExpressionWithLookup(
b.ctx,
b.descCache[tbl.TableID].desc.(catalog.TableDescriptor),
d,
&tn,
exprContext,
b.semaCtx,
b.clusterSettings.Version.ActiveVersion(b.ctx),
getAllNonDropColumnsFn,
columnLookupByNameFn,
)
if err != nil {
// This may be referencing newly added columns, so cheat and return
// a not implemented error.
if pgerror.GetPGCode(err) == pgcode.UndefinedColumn {

panic(errors.Wrapf(errors.WithSecondaryError(scerrors.NotImplementedError(d), err),
"computed column validation error"))
}
panic(err)
}
parsedExpr, err := parser.ParseExpr(expr)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,13 @@ func alterTableAddColumn(
RequiredPrivilege: privilege.CREATE,
})
_, colTargetStatus, col := scpb.FindColumn(elts)
columnAlreadyExists := col != nil && colTargetStatus != scpb.ToAbsent
_, colNameTargetStatus, colName := scpb.FindColumnName(elts)
// A column name already exists if both the Column and ColumnName elements exist
// and are not transitioning to ABSENT. When a column is being renamed, the
// Column remains but the old ColumnName transitions to ABSENT, freeing up the
// name for reuse.
columnAlreadyExists := col != nil && colTargetStatus != scpb.ToAbsent &&
colName != nil && colNameTargetStatus != scpb.ToAbsent
// If the column exists and IF NOT EXISTS is specified, continue parsing
// to ensure there are no other errors before treating the operation as a no-op.
if columnAlreadyExists && !t.IfNotExists {
Expand Down Expand Up @@ -168,7 +174,15 @@ func alterTableAddColumn(
))
}
if desc.IsComputed() {
validExpr, _ := b.ComputedColumnExpression(tbl, d, tree.ComputedColumnExprContext(d.IsVirtual()))
validExpr, _ := b.ComputedColumnExpression(
tbl, d, tree.ComputedColumnExprContext(d.IsVirtual()),
func() colinfo.ResultColumns {
return getNonDropResultColumns(b, tbl.TableID)
},
func(columnName tree.Name) (exists, accessible, computed bool, id catid.ColumnID, typ *types.T) {
return columnLookupFn(b, tbl.TableID, columnName)
},
)
expr := b.WrapExpression(tbl.TableID, validExpr)
if spec.colType.ElementCreationMetadata.In_24_3OrLater {
spec.compute = &scpb.ColumnComputeExpression{
Expand Down
Loading
Loading