From adba0a8a15f7f7f2fdd52ee4aafb612ff0523fd1 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Thu, 2 May 2024 03:11:35 -0600 Subject: [PATCH 1/2] sql/opt: add error handling during query plan cache invalidation This commit adds some missing error-handling to the metadata staleness check. This is necessary because object resolution error during staleness checking must be swallowed so that the query can be replanned. Fixes #123456 Release note (bug fix): Fixed a bug that could cause calling a routine to return an unexpected `function ... does not exist` error. The bug is triggered when the routine is called twice using the exact same SQL query, and either: (a) the routine has polymorphic arguments, or: (b) in between the two calls, the routine is replaced by a routine with the same name and different parameters. This bug has existed since alpha versions of 23.1. --- pkg/sql/logictest/testdata/logic_test/udf | 26 +++++++++++++++++++++++ pkg/sql/opt/metadata.go | 2 +- 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index c4609e393c94..0ce2976eda25 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -899,4 +899,30 @@ DROP FUNCTION f; statement error pgcode 42P13 pq: return type mismatch in function declared to return record CREATE FUNCTION f(OUT x INT, OUT y INT) LANGUAGE SQL AS $$ SELECT ROW(ROW(1, 2)); $$; +statement ok +CREATE FUNCTION f(x ANYELEMENT) RETURNS INT LANGUAGE SQL AS $$ SELECT 1; $$; + +statement ok +SELECT f(0); + +statement ok +SELECT f(0); + +statement ok +DROP FUNCTION f; +CREATE FUNCTION f(x INT) RETURNS INT LANGUAGE SQL AS $$ SELECT 1; $$; + +statement ok +SELECT f('0'); + +statement ok +DROP FUNCTION f; +CREATE FUNCTION f(x TEXT) RETURNS INT LANGUAGE SQL AS $$ SELECT 1; $$; + +statement ok +SELECT f('0'); + +statement ok +DROP FUNCTION f; + subtest end diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index f2ea969ac3e4..e555f579b950 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -477,7 +477,7 @@ func (md *Metadata) CheckDependencies( tryDefaultExprs, ) if err != nil || toCheck.Oid != overload.Oid || toCheck.Version != overload.Version { - return false, err + return false, maybeSwallowMetadataResolveErr(err) } } } else { From 3095ec7b86f7edcab46e468b6c5e0d0384fb1230 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 2 May 2024 09:36:43 -0700 Subject: [PATCH 2/2] rowexec: fix oid handling in merge join and zigzag join This commit is a follow up fix to a4b62344866457f99847ffc366f57b8fd5588a19 which fixed how we handle zero value Oid types. In particular, we now need to have the precise type information for Oid type family to display zero Oid correctly. Previously, we could have imprecise information in merge join and zigzag joins that was stored in the right-hand side EncDatum when it was decoded using the LHS type, and this is now fixed. I don't think other join types (hash join and lookup join) are susceptible to this since they do decoding at different points in time, so it's unlikely we'd get a similar mix up there. There is no release note since it seems like an edge case (comparing Oid types with different Oids in non-default row-by-row engine). Release note: None --- .../logictest/testdata/logic_test/pgoidtype | 23 +++++++++++++ pkg/sql/rowenc/encoded_datum.go | 33 ++++++++++++++++--- pkg/sql/rowexec/stream_merger.go | 6 ++-- pkg/sql/rowexec/zigzagjoiner.go | 16 ++++++--- 4 files changed, 66 insertions(+), 12 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/pgoidtype b/pkg/sql/logictest/testdata/logic_test/pgoidtype index 2cbb6015019a..91ae78296b63 100644 --- a/pkg/sql/logictest/testdata/logic_test/pgoidtype +++ b/pkg/sql/logictest/testdata/logic_test/pgoidtype @@ -710,3 +710,26 @@ query TT SELECT 0::REGROLE, 0::REGROLE::TEXT; ---- - - + +# Regression test for using the correct type when decoding EncDatum of Oid type +# family (#123474). +statement ok +CREATE TABLE t123474 ( + col_0 REGROLE, col_1 OID, col_2 INT, + INDEX (col_1 DESC) STORING (col_2), + INDEX (col_0) STORING (col_1) +); +INSERT INTO t123474 (col_0, col_1, col_2) VALUES (0, 0, 0); +SET testing_optimizer_random_seed = 6047211422050928467; +SET testing_optimizer_disable_rule_probability = 0.500000; + +query T +SELECT t2.col_1 + FROM t123474 AS t1 JOIN t123474 AS t2 ON (t1.col_0) = (t2.col_1) + ORDER BY t1.col_0; +---- +0 + +statement ok +RESET testing_optimizer_random_seed; +RESET testing_optimizer_disable_rule_probability; diff --git a/pkg/sql/rowenc/encoded_datum.go b/pkg/sql/rowenc/encoded_datum.go index 1c2f6d73f8dc..4ef878736cf8 100644 --- a/pkg/sql/rowenc/encoded_datum.go +++ b/pkg/sql/rowenc/encoded_datum.go @@ -396,6 +396,15 @@ func (ed *EncDatum) Fingerprint( // +1 if the receiver is greater than rhs. func (ed *EncDatum) Compare( typ *types.T, a *tree.DatumAlloc, evalCtx *eval.Context, rhs *EncDatum, +) (int, error) { + return ed.CompareEx(typ, a, evalCtx, rhs, typ) +} + +// CompareEx is the same as Compare but allows specifying the type of RHS +// EncDatum in case it's different from ed (e.g. we might be comparing Oid +// family types with different Oids). +func (ed *EncDatum) CompareEx( + typ *types.T, a *tree.DatumAlloc, evalCtx *eval.Context, rhs *EncDatum, rhsTyp *types.T, ) (int, error) { // TODO(radu): if we have both the Datum and a key encoding available, which // one would be faster to use? @@ -410,7 +419,7 @@ func (ed *EncDatum) Compare( if err := ed.EnsureDecoded(typ, a); err != nil { return 0, err } - if err := rhs.EnsureDecoded(typ, a); err != nil { + if err := rhs.EnsureDecoded(rhsTyp, a); err != nil { return 0, err } return ed.Datum.CompareError(evalCtx, rhs.Datum) @@ -553,11 +562,27 @@ func (r EncDatumRow) Compare( evalCtx *eval.Context, rhs EncDatumRow, ) (int, error) { - if len(r) != len(types) || len(rhs) != len(types) { - panic(errors.AssertionFailedf("length mismatch: %d types, %d lhs, %d rhs\n%+v\n%+v\n%+v", len(types), len(r), len(rhs), types, r, rhs)) + return r.CompareEx(types, a, ordering, evalCtx, rhs, types) +} + +// CompareEx is the same as Compare but allows specifying a different type +// schema for RHS row. +func (r EncDatumRow) CompareEx( + types []*types.T, + a *tree.DatumAlloc, + ordering colinfo.ColumnOrdering, + evalCtx *eval.Context, + rhs EncDatumRow, + rhsTypes []*types.T, +) (int, error) { + if len(r) != len(types) || len(rhs) != len(rhsTypes) || len(r) != len(rhs) { + panic(errors.AssertionFailedf( + "length mismatch: %d types, %d rhs types, %d lhs, %d rhs\n%+v\n%+v\n%+v", + len(types), len(rhsTypes), len(r), len(rhs), types, r, rhs, + )) } for _, c := range ordering { - cmp, err := r[c.ColIdx].Compare(types[c.ColIdx], a, evalCtx, &rhs[c.ColIdx]) + cmp, err := r[c.ColIdx].CompareEx(types[c.ColIdx], a, evalCtx, &rhs[c.ColIdx], rhsTypes[c.ColIdx]) if err != nil { return 0, err } diff --git a/pkg/sql/rowexec/stream_merger.go b/pkg/sql/rowexec/stream_merger.go index db45caabaf76..1ab21cce8089 100644 --- a/pkg/sql/rowexec/stream_merger.go +++ b/pkg/sql/rowexec/stream_merger.go @@ -79,7 +79,7 @@ func (sm *streamMerger) NextBatch( } cmp, err := CompareEncDatumRowForMerge( - sm.left.types, lrow, rrow, sm.left.ordering, sm.right.ordering, + sm.left.types, sm.right.types, lrow, rrow, sm.left.ordering, sm.right.ordering, sm.nullEquality, &sm.datumAlloc, evalCtx, ) if err != nil { @@ -108,7 +108,7 @@ func (sm *streamMerger) NextBatch( // a DatumAlloc which is used for decoding if any underlying EncDatum is not // yet decoded. func CompareEncDatumRowForMerge( - lhsTypes []*types.T, + lhsTypes, rhsTypes []*types.T, lhs, rhs rowenc.EncDatumRow, leftOrdering, rightOrdering colinfo.ColumnOrdering, nullEquality bool, @@ -144,7 +144,7 @@ func CompareEncDatumRowForMerge( } continue } - cmp, err := lhs[lIdx].Compare(lhsTypes[lIdx], da, evalCtx, &rhs[rIdx]) + cmp, err := lhs[lIdx].CompareEx(lhsTypes[lIdx], da, evalCtx, &rhs[rIdx], rhsTypes[rIdx]) if err != nil { return 0, err } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index d0c16b326cd5..c34ae6b0c643 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -573,12 +573,14 @@ func (z *zigzagJoiner) matchBase(curRow rowenc.EncDatumRow, side int) (bool, err prevEqDatums := z.extractEqDatums(z.baseRow, z.prevSide()) curEqDatums := z.extractEqDatums(curRow, side) - - eqColTypes := z.infos[side].eqColTypes + prevEqColTypes := z.infos[z.prevSide()].eqColTypes + curEqColTypes := z.infos[side].eqColTypes ordering := z.infos[side].eqColOrdering // Compare the equality columns of the baseRow to that of the curRow. - cmp, err := prevEqDatums.Compare(eqColTypes, &z.infos[side].alloc, ordering, z.FlowCtx.EvalCtx, curEqDatums) + cmp, err := prevEqDatums.CompareEx( + prevEqColTypes, &z.infos[side].alloc, ordering, z.FlowCtx.EvalCtx, curEqDatums, curEqColTypes, + ) if err != nil { return false, err } @@ -718,9 +720,13 @@ func (z *zigzagJoiner) nextRow(ctx context.Context) (rowenc.EncDatumRow, error) prevEqCols := z.extractEqDatums(prevNext, prevSide) currentEqCols := z.extractEqDatums(curNext, z.side) - eqColTypes := curInfo.eqColTypes + prevEqColTypes := z.infos[prevSide].eqColTypes + curEqColTypes := z.infos[z.side].eqColTypes ordering := curInfo.eqColOrdering - cmp, err := prevEqCols.Compare(eqColTypes, &z.infos[z.side].alloc, ordering, z.FlowCtx.EvalCtx, currentEqCols) + cmp, err := prevEqCols.CompareEx( + prevEqColTypes, &z.infos[z.side].alloc, ordering, + z.FlowCtx.EvalCtx, currentEqCols, curEqColTypes, + ) if err != nil { return nil, err }