Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
123457: sql/opt: add error handling during query plan cache invalidation r=DrewKimball a=DrewKimball

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.

123486: rowexec: fix oid handling in merge join and zigzag join r=yuzefovich a=yuzefovich

This commit is a follow up fix to a4b6234 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).

Fixes: #123474.

Release note: None

Co-authored-by: Drew Kimball <drewk@cockroachlabs.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
  • Loading branch information
3 people committed May 2, 2024
3 parents 00dd860 + adba0a8 + 3095ec7 commit 6951ba0
Show file tree
Hide file tree
Showing 6 changed files with 93 additions and 13 deletions.
23 changes: 23 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pgoidtype
Original file line number Diff line number Diff line change
Expand Up @@ -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;
26 changes: 26 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -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
2 changes: 1 addition & 1 deletion pkg/sql/opt/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
33 changes: 29 additions & 4 deletions pkg/sql/rowenc/encoded_datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -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?
Expand All @@ -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)
Expand Down Expand Up @@ -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
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/rowexec/stream_merger.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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
}
Expand Down
16 changes: 11 additions & 5 deletions pkg/sql/rowexec/zigzagjoiner.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down

0 comments on commit 6951ba0

Please sign in to comment.