Skip to content

Commit 5ec3db8

Browse files
committed
sql: support index->UDF dependency tracking in legacy schema changer
This matches the dependency tracking behavior that the declarative schema changer has. Release note: None
1 parent 6c2bef5 commit 5ec3db8

File tree

9 files changed

+295
-39
lines changed

9 files changed

+295
-39
lines changed

pkg/sql/add_column.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -156,12 +156,16 @@ func (p *planner) addColumnImpl(
156156
}
157157

158158
// We need to allocate new ID for the created column in order to correctly
159-
// assign sequence ownership.
159+
// assign sequence ownership and function dependencies.
160160
version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx)
161161
if err := n.tableDesc.AllocateIDs(params.ctx, version); err != nil {
162162
return err
163163
}
164164

165+
if err := params.p.maybeUpdateFunctionReferencesForColumn(params.ctx, n.tableDesc, col); err != nil {
166+
return err
167+
}
168+
165169
// If the new column has a DEFAULT or an ON UPDATE expression that uses a
166170
// sequence, add references between its descriptor and this column descriptor.
167171
if err := cdd.ForEachTypedExpr(func(expr tree.TypedExpr, colExprKind tabledesc.ColExprKind) error {

pkg/sql/backfill/backfill.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import (
2323
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
2424
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
2525
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
26-
"github.com/cockroachdb/cockroach/pkg/sql/isql"
2726
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
2827
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
2928
"github.com/cockroachdb/cockroach/pkg/sql/row"
@@ -231,7 +230,7 @@ func (cb *ColumnBackfiller) InitForDistributedUse(
231230
var defaultExprs, computedExprs []tree.TypedExpr
232231
// Install type metadata in the target descriptors, as well as resolve any
233232
// user defined types in the column expressions.
234-
if err := flowCtx.Cfg.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error {
233+
if err := flowCtx.Cfg.DB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
235234
resolver := flowCtx.NewTypeResolver(txn.KV())
236235
// Hydrate all the types present in the table.
237236
if err := typedesc.HydrateTypesInDescriptor(ctx, desc, &resolver); err != nil {
@@ -240,6 +239,7 @@ func (cb *ColumnBackfiller) InitForDistributedUse(
240239
// Set up a SemaContext to type check the default and computed expressions.
241240
semaCtx := tree.MakeSemaContext(&resolver)
242241
semaCtx.UnsupportedTypeChecker = eval.NewUnsupportedTypeChecker(evalCtx.Settings.Version)
242+
semaCtx.FunctionResolver = descs.NewDistSQLFunctionResolver(txn.Descriptors(), txn.KV())
243243
var err error
244244
defaultExprs, err = schemaexpr.MakeDefaultExprs(
245245
ctx, cb.added, &transform.ExprTransformContext{}, evalCtx, &semaCtx,

pkg/sql/catalog/tabledesc/column.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -186,7 +186,7 @@ func (w column) NumUsesFunctions() int {
186186
// GetUsesFunctionID returns the ID of a function used by this column at the
187187
// given ordinal.
188188
func (w column) GetUsesFunctionID(ordinal int) descpb.ID {
189-
return w.desc.UsesSequenceIds[ordinal]
189+
return w.desc.UsesFunctionIds[ordinal]
190190
}
191191

192192
// NumOwnsSequences returns the number of sequences owned by this column.

pkg/sql/create_index.go

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -607,6 +607,11 @@ func replaceExpressionElemsWithVirtualCols(
607607
Virtual: true,
608608
Nullable: true,
609609
}
610+
if fnIDs, err := schemaexpr.GetUDFIDsFromExprStr(expr); err != nil {
611+
return err
612+
} else {
613+
col.UsesFunctionIds = fnIDs.Ordered()
614+
}
610615

611616
// Add the column to the table descriptor. If the table already
612617
// exists, add it as a mutation column.
@@ -835,6 +840,25 @@ func (n *createIndexNode) startExec(params runParams) error {
835840
return err
836841
}
837842

843+
// Update function back-references for any column that was added as an index
844+
// expression and for any index with a partial predicate.
845+
for _, m := range n.tableDesc.GetMutations() {
846+
if col := m.GetColumn(); col != nil && m.Direction == descpb.DescriptorMutation_ADD {
847+
if err := params.p.maybeUpdateFunctionReferencesForColumn(
848+
params.ctx, n.tableDesc, col,
849+
); err != nil {
850+
return err
851+
}
852+
}
853+
if idx := m.GetIndex(); idx != nil && m.Direction == descpb.DescriptorMutation_ADD {
854+
if err := params.p.maybeAddFunctionReferencesForIndex(
855+
params.ctx, n.tableDesc, idx,
856+
); err != nil {
857+
return err
858+
}
859+
}
860+
}
861+
838862
// Record index creation in the event log. This is an auditable log
839863
// event and is recorded in the same transaction as the table descriptor
840864
// update.

pkg/sql/drop_index.go

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -174,6 +174,11 @@ func (n *dropIndexNode) startExec(params runParams) error {
174174
}
175175
if col.IsExpressionIndexColumn() && !keyColumnOfOtherIndex(col.GetID()) {
176176
n.queueDropColumn(tableDesc, col)
177+
if col.NumUsesFunctions() > 0 {
178+
if err := params.p.removeColumnBackReferenceInFunctions(params.ctx, tableDesc, col.ColumnDesc()); err != nil {
179+
return err
180+
}
181+
}
177182
columnsDropped = true
178183
}
179184
}
@@ -496,6 +501,10 @@ func (p *planner) dropIndexByName(
496501
return err
497502
}
498503

504+
if err := p.maybeRemoveFunctionReferencesForIndex(ctx, tableDesc, idxDesc); err != nil {
505+
return err
506+
}
507+
499508
mutationID := tableDesc.ClusterVersion().NextMutationID
500509
if err := p.writeSchemaChange(ctx, tableDesc, mutationID, jobDesc); err != nil {
501510
return err

pkg/sql/function_references.go

Lines changed: 61 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ import (
1010

1111
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
1212
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
13+
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
1314
)
1415

1516
func (p *planner) updateFunctionReferencesForCheck(
@@ -70,3 +71,63 @@ func (p *planner) maybeUpdateFunctionReferencesForColumn(
7071
}
7172
return nil
7273
}
74+
75+
func (p *planner) maybeAddFunctionReferencesForIndex(
76+
ctx context.Context, tblDesc catalog.TableDescriptor, idx *descpb.IndexDescriptor,
77+
) error {
78+
if !idx.IsPartial() {
79+
return nil
80+
}
81+
if idx.UseDeletePreservingEncoding {
82+
// These indexes are only used during the backfill and are discarded after.
83+
return nil
84+
}
85+
86+
// Extract function IDs from the partial index predicate.
87+
fnIDs, err := schemaexpr.GetUDFIDsFromExprStr(idx.Predicate)
88+
if err != nil {
89+
return err
90+
}
91+
92+
// Add back-references in function descriptors.
93+
for _, id := range fnIDs.Ordered() {
94+
fnDesc, err := p.descCollection.MutableByID(p.txn).Function(ctx, id)
95+
if err != nil {
96+
return err
97+
}
98+
if err := fnDesc.AddIndexReference(tblDesc.GetID(), idx.ID); err != nil {
99+
return err
100+
}
101+
if err := p.writeFuncSchemaChange(ctx, fnDesc); err != nil {
102+
return err
103+
}
104+
}
105+
return nil
106+
}
107+
108+
func (p *planner) maybeRemoveFunctionReferencesForIndex(
109+
ctx context.Context, tblDesc catalog.TableDescriptor, idx *descpb.IndexDescriptor,
110+
) error {
111+
if !idx.IsPartial() {
112+
return nil
113+
}
114+
115+
// Extract function IDs from the partial index predicate.
116+
fnIDs, err := schemaexpr.GetUDFIDsFromExprStr(idx.Predicate)
117+
if err != nil {
118+
return err
119+
}
120+
121+
// Remove back-references in function descriptors.
122+
for _, id := range fnIDs.Ordered() {
123+
fnDesc, err := p.descCollection.MutableByID(p.txn).Function(ctx, id)
124+
if err != nil {
125+
return err
126+
}
127+
fnDesc.RemoveIndexReference(tblDesc.GetID(), idx.ID)
128+
if err := p.writeFuncSchemaChange(ctx, fnDesc); err != nil {
129+
return err
130+
}
131+
}
132+
return nil
133+
}

pkg/sql/logictest/testdata/logic_test/udf_in_index

Lines changed: 32 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -91,7 +91,7 @@ CREATE FUNCTION test_tbl_f() RETURNS INT IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $
9191

9292
# Test that UDF can be used in index creation while creating table.
9393
statement ok
94-
CREATE TABLE test_tbl_t (a INT PRIMARY KEY, b INT, INDEX idx_b(test_tbl_f()));
94+
CREATE TABLE test_tbl_t (a INT PRIMARY KEY, b INT, INDEX idx_b((1 + test_tbl_f())));
9595

9696
let $tbl_id
9797
SELECT id FROM system.namespace WHERE name = 'test_tbl_t';
@@ -102,10 +102,10 @@ SELECT oid::int - 100000 FROM pg_catalog.pg_proc WHERE proname = 'test_tbl_f';
102102
query IITT colnames
103103
SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
104104
----
105-
id col_id compute_expr uses_fn_ids
106-
113 1 NULL NULL
107-
113 2 NULL NULL
108-
113 3 "[FUNCTION 100112]()" [112]
105+
id col_id compute_expr uses_fn_ids
106+
113 1 NULL NULL
107+
113 2 NULL NULL
108+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
109109

110110
query T
111111
SELECT get_fn_depended_on_by($fn_id)
@@ -121,16 +121,16 @@ SELECT get_fn_depended_on_by($fn_id)
121121

122122
# Test UDF index creation after table exists.
123123
statement ok
124-
CREATE INDEX t_idx ON test_tbl_t(test_tbl_f());
124+
CREATE INDEX t_idx ON test_tbl_t((2 + test_tbl_f()));
125125

126126
query IITT colnames
127127
SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
128128
----
129-
id col_id compute_expr uses_fn_ids
130-
113 1 NULL NULL
131-
113 2 NULL NULL
132-
113 3 "[FUNCTION 100112]()" [112]
133-
113 4 "[FUNCTION 100112]()" [112]
129+
id col_id compute_expr uses_fn_ids
130+
113 1 NULL NULL
131+
113 2 NULL NULL
132+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
133+
113 4 "2:::INT8 + [FUNCTION 100112]()" [112]
134134

135135
query T
136136
SELECT get_fn_depended_on_by($fn_id)
@@ -158,11 +158,11 @@ CREATE INDEX t_idx2 ON test_tbl_t(b) WHERE test_tbl_partial_f(b) > 0;
158158
query IITT colnames
159159
SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
160160
----
161-
id col_id compute_expr uses_fn_ids
162-
113 1 NULL NULL
163-
113 2 NULL NULL
164-
113 3 "[FUNCTION 100112]()" [112]
165-
113 4 "[FUNCTION 100112]()" [112]
161+
id col_id compute_expr uses_fn_ids
162+
113 1 NULL NULL
163+
113 2 NULL NULL
164+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
165+
113 4 "2:::INT8 + [FUNCTION 100112]()" [112]
166166

167167
query IIT colnames
168168
SELECT * FROM get_idx_fn_ids($tbl_id) ORDER BY 1, 2;
@@ -189,7 +189,7 @@ INSERT INTO test_tbl_t VALUES (1, 1), (2, -2), (3, 3);
189189

190190
# Verify that we can select from the partial index.
191191
query II
192-
SELECT * FROM test_tbl_t@t_idx2 WHERE test_tbl_partial_f(b) > 0;
192+
SELECT * FROM test_tbl_t@t_idx2 WHERE test_tbl_partial_f(b) > 0 ORDER BY 1, 2;
193193
----
194194
1 1
195195
3 3
@@ -210,12 +210,12 @@ CREATE INDEX t_idx3 ON test_tbl_t((b + test_tbl_f()));
210210
query IITT colnames
211211
SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
212212
----
213-
id col_id compute_expr uses_fn_ids
214-
113 1 NULL NULL
215-
113 2 NULL NULL
216-
113 3 "[FUNCTION 100112]()" [112]
217-
113 4 "[FUNCTION 100112]()" [112]
218-
113 5 "b + [FUNCTION 100112]()" [112]
213+
id col_id compute_expr uses_fn_ids
214+
113 1 NULL NULL
215+
113 2 NULL NULL
216+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
217+
113 4 "2:::INT8 + [FUNCTION 100112]()" [112]
218+
113 5 "b + [FUNCTION 100112]()" [112]
219219

220220
query T
221221
SELECT get_fn_depended_on_by($fn_id)
@@ -241,8 +241,8 @@ SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
241241
id col_id compute_expr uses_fn_ids
242242
113 1 NULL NULL
243243
113 2 NULL NULL
244-
113 3 "[FUNCTION 100112]()" [112]
245-
113 4 "[FUNCTION 100112]()" [112]
244+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
245+
113 4 "2:::INT8 + [FUNCTION 100112]()" [112]
246246
113 5 "b + [FUNCTION 100112]()" [112]
247247
113 6 "[FUNCTION 100112]()" [112]
248248
113 7 "[FUNCTION 100112]() + 1:::INT8" [112]
@@ -273,7 +273,7 @@ SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
273273
id col_id compute_expr uses_fn_ids
274274
113 1 NULL NULL
275275
113 2 NULL NULL
276-
113 3 "[FUNCTION 100112]()" [112]
276+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
277277
113 5 "b + [FUNCTION 100112]()" [112]
278278
113 6 "[FUNCTION 100112]()" [112]
279279
113 7 "[FUNCTION 100112]() + 1:::INT8" [112]
@@ -307,7 +307,7 @@ SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
307307
id col_id compute_expr uses_fn_ids
308308
113 1 NULL NULL
309309
113 2 NULL NULL
310-
113 3 "[FUNCTION 100112]()" [112]
310+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
311311
113 5 "b + [FUNCTION 100112]()" [112]
312312
113 6 "[FUNCTION 100112]()" [112]
313313
113 7 "[FUNCTION 100112]() + 1:::INT8" [112]
@@ -336,7 +336,7 @@ SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
336336
id col_id compute_expr uses_fn_ids
337337
113 1 NULL NULL
338338
113 2 NULL NULL
339-
113 3 "[FUNCTION 100112]()" [112]
339+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
340340
113 6 "[FUNCTION 100112]()" [112]
341341
113 7 "[FUNCTION 100112]() + 1:::INT8" [112]
342342

@@ -360,10 +360,10 @@ DROP INDEX t_idx4;
360360
query IITT colnames
361361
SELECT * FROM get_col_fn_ids($tbl_id) ORDER BY 1, 2;
362362
----
363-
id col_id compute_expr uses_fn_ids
364-
113 1 NULL NULL
365-
113 2 NULL NULL
366-
113 3 "[FUNCTION 100112]()" [112]
363+
id col_id compute_expr uses_fn_ids
364+
113 1 NULL NULL
365+
113 2 NULL NULL
366+
113 3 "1:::INT8 + [FUNCTION 100112]()" [112]
367367

368368
query T
369369
SELECT get_fn_depended_on_by($fn_id)

pkg/sql/logictest/testdata/logic_test/udf_unsupported

Lines changed: 22 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -10,13 +10,32 @@ CREATE TABLE test_tbl_t (a INT PRIMARY KEY, b INT);
1010
statement ok
1111
INSERT INTO test_tbl_t VALUES (1, 1);
1212

13-
statement error pgcode 0A000 failed to construct index entries during backfill: unimplemented: cannot evaluate function in this context
13+
skipif config local-mixed-25.2
14+
statement error pgcode 0A000 unimplemented: cannot evaluate function in this context
1415
ALTER TABLE test_tbl_t ADD COLUMN c int AS (test_tbl_f()) stored;
1516

16-
statement error pgcode 0A000 failed to construct index entries during backfill: unimplemented: cannot evaluate function in this context
17+
skipif config local-mixed-25.2
18+
statement error pgcode 0A000 unimplemented: cannot evaluate function in this context
1719
ALTER TABLE test_tbl_t ADD COLUMN c int DEFAULT (test_tbl_f());
1820

19-
statement error pgcode 0A000 failed to construct index entries during backfill: unimplemented: cannot evaluate function in this context
21+
skipif config local-mixed-25.2
22+
statement error pgcode 0A000 unimplemented: cannot evaluate function in this context
23+
CREATE INDEX t_idx_partial ON test_tbl_t(b) WHERE test_tbl_f() > 0;
24+
25+
skipif config local-mixed-25.2
26+
statement error pgcode 0A000 unimplemented: cannot evaluate function in this context
27+
CREATE INDEX idx_b ON test_tbl_t (test_tbl_f());
28+
29+
onlyif config local-mixed-25.2
30+
statement error pgcode 0A000 unimplemented: usage of user-defined function from relations not supported
31+
ALTER TABLE test_tbl_t ADD COLUMN c int AS (test_tbl_f()) stored;
32+
33+
onlyif config local-mixed-25.2
34+
statement error pgcode 0A000 unimplemented: usage of user-defined function from relations not supported
35+
ALTER TABLE test_tbl_t ADD COLUMN c int DEFAULT (test_tbl_f());
36+
37+
onlyif config local-mixed-25.2
38+
statement error pgcode 0A000 unimplemented: usage of user-defined function from relations not supported
2039
CREATE INDEX idx_b ON test_tbl_t (test_tbl_f());
2140

2241
subtest end

0 commit comments

Comments
 (0)