Skip to content

Commit 88ee935

Browse files
committed
schemaexpr: add iterColsWithLookupFn
This function allows the computed column validation to be used from the declarative schema changer -- it uses a lookup function to access columns rather than referring to the table descriptor directly. Release note: None
1 parent f0227c8 commit 88ee935

File tree

2 files changed

+159
-45
lines changed

2 files changed

+159
-45
lines changed

pkg/sql/catalog/schemaexpr/column.go

Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -221,6 +221,51 @@ func iterColDescriptors(
221221
return err
222222
}
223223

224+
// iterColsWithLookupFn iterates over the expression's variable columns and
225+
// calls f on each, providing the column information from the lookup function.
226+
//
227+
// If the expression references a column that does not exist according to the
228+
// lookup function, iterColsWithLookupFn errs with pgcode.UndefinedColumn.
229+
//
230+
// The column lookup function allows looking up columns both in the descriptor
231+
// or in declarative schema changer elements.
232+
func iterColsWithLookupFn(
233+
rootExpr tree.Expr,
234+
columnLookupFn ColumnLookupFn,
235+
f func(columnName tree.Name, id catid.ColumnID, typ *types.T, isAccessible, isComputed bool) error,
236+
) error {
237+
_, err := tree.SimpleVisit(rootExpr, func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) {
238+
vBase, ok := expr.(tree.VarName)
239+
if !ok {
240+
// Not a VarName, don't do anything to this node.
241+
return true, expr, nil
242+
}
243+
244+
v, err := vBase.NormalizeVarName()
245+
if err != nil {
246+
return false, nil, err
247+
}
248+
249+
c, ok := v.(*tree.ColumnItem)
250+
if !ok {
251+
return true, expr, nil
252+
}
253+
254+
colExists, colIsAccessible, isComputed, colID, colType := columnLookupFn(c.ColumnName)
255+
if !colExists {
256+
return false, nil, pgerror.Newf(pgcode.UndefinedColumn,
257+
"column %q does not exist, referenced in %q", c.ColumnName, rootExpr.String())
258+
}
259+
260+
if err := f(c.ColumnName, colID, colType, colIsAccessible, isComputed); err != nil {
261+
return false, nil, err
262+
}
263+
return false, expr, err
264+
})
265+
266+
return err
267+
}
268+
224269
// dummyColumn represents a variable column that can type-checked. It is used
225270
// in validating check constraint and partial index predicate expressions. This
226271
// validation requires that the expression can be both both typed-checked and

pkg/sql/catalog/schemaexpr/computed_column.go

Lines changed: 114 additions & 45 deletions
Original file line numberDiff line numberDiff line change
@@ -12,10 +12,12 @@ import (
1212

1313
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1414
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
15+
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
1516
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1617
"github.com/cockroachdb/cockroach/pkg/sql/parserutils"
1718
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
1819
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
20+
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
1921
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
2022
"github.com/cockroachdb/cockroach/pkg/sql/sem/transform"
2123
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
@@ -36,6 +38,9 @@ import (
3638
//
3739
// - It does not have a default value.
3840
// - It does not reference other computed columns.
41+
// - It does not reference inaccessible columns.
42+
// - It does not depend on the region column name if the table is REGIONAL BY
43+
// ROW and uses a foreign key to populate the region column.
3944
//
4045
// TODO(mgartner): Add unit tests for Validate.
4146
func ValidateComputedColumnExpression(
@@ -47,69 +52,53 @@ func ValidateComputedColumnExpression(
4752
semaCtx *tree.SemaContext,
4853
version clusterversion.ClusterVersion,
4954
) (serializedExpr string, _ *types.T, _ error) {
50-
if d.HasDefaultExpr() {
51-
return "", nil, pgerror.Newf(
52-
pgcode.InvalidTableDefinition,
53-
"%s cannot have default values",
54-
context,
55-
)
56-
}
57-
58-
var depColIDs catalog.TableColSet
59-
// First, check that no column in the expression is an inaccessible or
60-
// computed column.
61-
err := iterColDescriptors(desc, d.Computed.Expr, func(c catalog.Column) error {
62-
if c.IsInaccessible() {
63-
return pgerror.Newf(
64-
pgcode.UndefinedColumn,
65-
"column %q is inaccessible and cannot be referenced in a computed column expression",
66-
c.GetName(),
67-
)
68-
}
69-
if c.IsComputed() {
70-
return pgerror.Newf(
71-
pgcode.InvalidTableDefinition,
72-
"%s expression cannot reference computed columns",
73-
context,
74-
)
55+
// Create helper functions from the descriptor and delegate to the
56+
// lookup-based validation function.
57+
getAllNonDropColumnsFn := func() colinfo.ResultColumns {
58+
cols := desc.NonDropColumns()
59+
ret := make(colinfo.ResultColumns, len(cols))
60+
for i, col := range cols {
61+
ret[i] = colinfo.ResultColumn{
62+
Name: col.GetName(),
63+
Typ: col.GetType(),
64+
Hidden: col.IsHidden(),
65+
TableID: desc.GetID(),
66+
PGAttributeNum: uint32(col.GetPGAttributeNum()),
67+
}
7568
}
76-
depColIDs.Add(c.GetID())
77-
78-
return nil
79-
})
80-
if err != nil {
81-
return "", nil, err
69+
return ret
8270
}
8371

84-
// Resolve the type of the computed column expression.
85-
defType, err := tree.ResolveType(ctx, d.Type, semaCtx.GetTypeResolver())
86-
if err != nil {
87-
return "", nil, err
88-
}
72+
columnLookupFn := makeColumnLookupFnForTableDesc(desc)
8973

90-
// Check that the type of the expression is of type defType and that there
91-
// are no variable expressions (besides dummyColumnItems) and no impure
92-
// functions. In order to safely serialize user defined types and their
93-
// members, we need to serialize the typed expression here.
94-
expr, typ, _, err := DequalifyAndValidateExpr(
74+
serializedExpr, typ, err := ValidateComputedColumnExpressionWithLookup(
9575
ctx,
9676
desc,
97-
d.Computed.Expr,
98-
defType,
77+
d,
78+
tn,
9979
context,
10080
semaCtx,
101-
volatility.Immutable,
102-
tn,
10381
version,
82+
getAllNonDropColumnsFn,
83+
columnLookupFn,
10484
)
10585
if err != nil {
10686
return "", nil, err
10787
}
10888

89+
var depColIDs catalog.TableColSet
90+
if err := iterColDescriptors(desc, d.Computed.Expr, func(c catalog.Column) error {
91+
depColIDs.Add(c.GetID())
92+
return nil
93+
}); err != nil {
94+
return "", nil, err
95+
}
96+
10997
// Virtual computed columns must not refer to mutation columns because it
11098
// would not be safe in the case that the mutation column was being
11199
// backfilled and the virtual computed column value needed to be computed
112100
// for the purpose of writing to a secondary index.
101+
// This check is specific to the legacy schema changer.
113102
if d.IsVirtual() {
114103
var mutationColumnNames []string
115104
var err error
@@ -144,6 +133,86 @@ func ValidateComputedColumnExpression(
144133
}
145134
}
146135

136+
return serializedExpr, typ, nil
137+
}
138+
139+
// ValidateComputedColumnExpressionWithLookup verifies that an expression is a
140+
// valid computed column expression using a column lookup function. It returns
141+
// the serialized expression and its type if valid, and an error otherwise.
142+
//
143+
// This is similar to ValidateComputedColumnExpression but uses a ColumnLookupFn
144+
// instead of a catalog.TableDescriptor, allowing it to work with declarative
145+
// schema changer elements.
146+
func ValidateComputedColumnExpressionWithLookup(
147+
ctx context.Context,
148+
desc catalog.TableDescriptor,
149+
d *tree.ColumnTableDef,
150+
tn *tree.TableName,
151+
context tree.SchemaExprContext,
152+
semaCtx *tree.SemaContext,
153+
version clusterversion.ClusterVersion,
154+
getAllNonDropColumnsFn func() colinfo.ResultColumns,
155+
columnLookupFn ColumnLookupFn,
156+
) (serializedExpr string, _ *types.T, _ error) {
157+
if d.HasDefaultExpr() {
158+
return "", nil, pgerror.Newf(
159+
pgcode.InvalidTableDefinition,
160+
"%s cannot have default values",
161+
context,
162+
)
163+
}
164+
165+
var depColIDs catalog.TableColSet
166+
// First, check that no column in the expression is an inaccessible or
167+
// computed column.
168+
err := iterColsWithLookupFn(d.Computed.Expr, columnLookupFn,
169+
func(columnName tree.Name, id catid.ColumnID, typ *types.T, isAccessible, isComputed bool) error {
170+
if !isAccessible {
171+
return pgerror.Newf(
172+
pgcode.UndefinedColumn,
173+
"column %q is inaccessible and cannot be referenced in a computed column expression",
174+
columnName,
175+
)
176+
}
177+
if isComputed {
178+
return pgerror.Newf(
179+
pgcode.InvalidTableDefinition,
180+
"%s expression cannot reference computed columns",
181+
context,
182+
)
183+
}
184+
depColIDs.Add(id)
185+
return nil
186+
})
187+
if err != nil {
188+
return "", nil, err
189+
}
190+
191+
// Resolve the type of the computed column expression.
192+
defType, err := tree.ResolveType(ctx, d.Type, semaCtx.GetTypeResolver())
193+
if err != nil {
194+
return "", nil, err
195+
}
196+
197+
// Check that the type of the expression is of type defType and that there
198+
// are no variable expressions (besides dummyColumnItems) and no impure
199+
// functions. We use DequalifyAndValidateExprImpl with the lookup function.
200+
expr, typ, _, err := DequalifyAndValidateExprImpl(
201+
ctx,
202+
d.Computed.Expr,
203+
defType,
204+
context,
205+
semaCtx,
206+
volatility.Immutable,
207+
tn,
208+
version,
209+
getAllNonDropColumnsFn,
210+
columnLookupFn,
211+
)
212+
if err != nil {
213+
return "", nil, err
214+
}
215+
147216
// If this is a REGIONAL BY ROW table using a foreign key to populate the
148217
// region column, we need to check that the expression does not reference
149218
// the region column. This is because the values of every (possibly computed)

0 commit comments

Comments
 (0)