Skip to content

Commit

Permalink
opt: fix bug with incorrect results produced by paired left lookup join
Browse files Browse the repository at this point in the history
Prior to this patch, it was possible for a paired join to produce incorrect
results for a left lookup join. In particular, some output rows had
non-NULL values for right-side columns when the right-side columns should
have been NULL.

This commit fixes the issue by updating the optimizer to ensure that
only columns from the second join in the paired join (the index join)
are projected, not columns from the first (the lookup join).

Fixes cockroachdb#81968

Release note (bug fix): Fixed an issue where a left lookup join could
have incorrect results. In particular, some output rows could have non-NULL
values for right-side columns when the right-side columns should
have been NULL. This issue only exists in 22.1.0 and prior development
releases of 22.1.
  • Loading branch information
rytaft committed May 30, 2022
1 parent 9549e03 commit 67893af
Show file tree
Hide file tree
Showing 6 changed files with 244 additions and 77 deletions.
39 changes: 39 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/lookup_join
Original file line number Diff line number Diff line change
Expand Up @@ -693,3 +693,42 @@ query I
SELECT k FROM t79384a INNER LOOKUP JOIN t79384b ON k = a AND b IN (1, 2, 3) AND c > 0
----
1

# Regression test for #81968. Paired left joins should set all right columns
# for unmatched rows to NULL.
statement ok
CREATE TABLE items (
id INT NOT NULL PRIMARY KEY,
chat_id INT NOT NULL,
author_id INT NOT NULL,
INDEX chat_id_idx (chat_id)
);
CREATE TABLE views (
chat_id INT NOT NULL,
user_id INT NOT NULL,
PRIMARY KEY (chat_id, user_id)
);
INSERT INTO views(chat_id, user_id) VALUES (1, 1);
INSERT INTO items(id, chat_id, author_id) VALUES
(1, 1, 1),
(2, 1, 1),
(3, 1, 1);

query I
SELECT (SELECT count(items.id)
FROM items
WHERE items.chat_id = views.chat_id
AND items.author_id != views.user_id)
FROM views
WHERE chat_id = 1
AND user_id = 1;
----
0

query IIIII
SELECT * FROM views LEFT LOOKUP JOIN items
ON items.chat_id = views.chat_id
AND items.author_id != views.user_id
WHERE views.chat_id = 1 and views.user_id = 1;
----
1 1 NULL NULL NULL
12 changes: 6 additions & 6 deletions pkg/sql/opt/exec/execbuilder/testdata/lookup_join
Original file line number Diff line number Diff line change
Expand Up @@ -721,14 +721,14 @@ vectorized: true
│ estimated row count: 336
└── • lookup join (left outer)
│ columns: (c, a, b, cont, d)
│ columns: (c, a, b, c, cont, b, d)
│ table: large@large_pkey
│ equality: (a, b) = (a,b)
│ equality cols are key
│ pred: d < 30
└── • lookup join (left outer)
│ columns: (c, a, b, cont)
│ columns: (c, a, b, c, cont)
│ estimated row count: 1,000
│ table: large@bc
│ equality: (c) = (b)
Expand All @@ -751,14 +751,14 @@ vectorized: true
│ estimated row count: 100
└── • lookup join (semi)
│ columns: (c, a, b, cont)
│ columns: (c, a, b, c, cont)
│ table: large@large_pkey
│ equality: (a, b) = (a,b)
│ equality cols are key
│ pred: d < 30
└── • lookup join (inner)
│ columns: (c, a, b, cont)
│ columns: (c, a, b, c, cont)
│ estimated row count: 990
│ table: large@bc
│ equality: (c) = (b)
Expand All @@ -781,14 +781,14 @@ vectorized: true
│ estimated row count: 0
└── • lookup join (anti)
│ columns: (c, a, b, cont)
│ columns: (c, a, b, c, cont)
│ table: large@large_pkey
│ equality: (a, b) = (a,b)
│ equality cols are key
│ pred: d < 30
└── • lookup join (left outer)
│ columns: (c, a, b, cont)
│ columns: (c, a, b, c, cont)
│ estimated row count: 1,000
│ table: large@bc
│ equality: (c) = (b)
Expand Down
65 changes: 64 additions & 1 deletion pkg/sql/opt/xform/join_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,6 +354,7 @@ func (c *CustomFuncs) generateLookupJoinsImpl(

var pkCols opt.ColList
var eqColMap opt.ColMap
var newScanPrivate *memo.ScanPrivate
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, on, rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, onFilters memo.FiltersExpr, indexCols opt.ColSet, _ bool, _ memo.ProjectionsExpr) {
Expand Down Expand Up @@ -730,11 +731,38 @@ func (c *CustomFuncs) generateLookupJoinsImpl(
indexJoin.On = c.ExtractUnboundConditions(conditions, onCols)
}
if pairedJoins {
// Create a new ScanPrivate, which will be used below for the first lookup
// join in the pair. Note: this must happen before the continuation column
// is created to ensure that the continuation column will have the highest
// column ID.
//
// See the comment where this newScanPrivate is used below in mapLookupJoin
// for details about why it's needed.
if newScanPrivate == nil {
newScanPrivate = c.DuplicateScanPrivate(scanPrivate)
}

lookupJoin.JoinType = lowerJoinType
continuationCol = c.constructContinuationColumnForPairedJoin()
lookupJoin.IsFirstJoinInPairedJoiner = true
lookupJoin.ContinuationCol = continuationCol
lookupJoin.Cols.Add(continuationCol)

// Map the lookup join to use the new table and column IDs from the
// newScanPrivate created above. We want to make sure that the column IDs
// returned by the lookup join are different from the IDs that will be
// returned by the top level index join.
//
// In addition to avoiding subtle bugs in the optimizer when the same
// column ID is reused, this mapping is also essential for correct behavior
// at execution time in the case of a left paired join. This is because a
// row that matches in the first left join (the lookup join) might be a
// false positive and fail to match in the second left join (the index
// join). If an original left row has no matches after the second left join,
// it must appear as a null-extended row with all right-hand columns null.
// If one of the right-hand columns comes from the lookup join, however,
// it might incorrectly show up as non-null (see #58892 and #81968).
c.mapLookupJoin(&lookupJoin, indexCols, newScanPrivate)
}

indexJoin.Input = c.e.f.ConstructLookupJoin(
Expand All @@ -745,7 +773,7 @@ func (c *CustomFuncs) generateLookupJoinsImpl(
indexJoin.JoinType = joinType
indexJoin.Table = scanPrivate.Table
indexJoin.Index = cat.PrimaryIndex
indexJoin.KeyCols = pkCols
indexJoin.KeyCols = c.getPkCols(lookupJoin.Table)
indexJoin.Cols = rightCols.Union(inputProps.OutputCols)
indexJoin.LookupColsAreTableKey = true
if pairedJoins {
Expand Down Expand Up @@ -951,6 +979,41 @@ func (c *CustomFuncs) constructContinuationColumnForPairedJoin() opt.ColumnID {
return c.e.f.Metadata().AddColumn("continuation", c.BoolType())
}

// mapLookupJoin maps the given lookup join to use the table and columns
// provided in newScanPrivate. The lookup join is modified in place. indexCols
// contains the pre-calculated index columns used by the given lookupJoin.
//
// Note that columns from the input are not mapped. For example, KeyCols
// does not need to be mapped below since it only contains input columns.
func (c *CustomFuncs) mapLookupJoin(
lookupJoin *memo.LookupJoinExpr, indexCols opt.ColSet, newScanPrivate *memo.ScanPrivate,
) {
tabID := lookupJoin.Table
newTabID := newScanPrivate.Table

// Get the new index columns.
newIndexCols := c.e.mem.Metadata().TableMeta(newTabID).IndexColumns(lookupJoin.Index)

// Create a map from the source columns to the destination columns.
var srcColsToDstCols opt.ColMap
for srcCol, ok := indexCols.Next(0); ok; srcCol, ok = indexCols.Next(srcCol + 1) {
ord := tabID.ColumnOrdinal(srcCol)
dstCol := newTabID.ColumnID(ord)
srcColsToDstCols.Set(int(srcCol), int(dstCol))
}

lookupJoin.Table = newTabID
lookupExpr := c.RemapCols(&lookupJoin.LookupExpr, srcColsToDstCols).(*memo.FiltersExpr)
lookupJoin.LookupExpr = *lookupExpr
remoteLookupExpr := c.RemapCols(&lookupJoin.RemoteLookupExpr, srcColsToDstCols).(*memo.FiltersExpr)
lookupJoin.RemoteLookupExpr = *remoteLookupExpr
lookupJoin.Cols = lookupJoin.Cols.Difference(indexCols).Union(newIndexCols)
constFilters := c.RemapCols(&lookupJoin.ConstFilters, srcColsToDstCols).(*memo.FiltersExpr)
lookupJoin.ConstFilters = *constFilters
on := c.RemapCols(&lookupJoin.On, srcColsToDstCols).(*memo.FiltersExpr)
lookupJoin.On = *on
}

// GenerateInvertedJoins is similar to GenerateLookupJoins, but instead
// of generating lookup joins with regular indexes, it generates lookup joins
// with inverted indexes. Similar to GenerateLookupJoins, there are two cases
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/opt/xform/testdata/external/liquibase
Original file line number Diff line number Diff line change
Expand Up @@ -206,17 +206,17 @@ project
│ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (84)-->(85), (1,84)-->(36,37,91,105,106), (105)-->(106), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3)
│ │ │ │ │ ├── left-join (lookup pg_index [as=ind])
│ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 indexrelid:84 indrelid:85 indisclustered:91 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140
│ │ │ │ │ │ ├── key columns: [84] = [84]
│ │ │ │ │ │ ├── key columns: [932] = [84]
│ │ │ │ │ │ ├── lookup columns are key
│ │ │ │ │ │ ├── second join in paired joiner
│ │ │ │ │ │ ├── key: (1,84)
│ │ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (84)-->(85), (1,84)-->(36,37,91), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3)
│ │ │ │ │ │ ├── left-join (lookup pg_index@pg_index_indrelid_index [as=ind])
│ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 indexrelid:84 indrelid:85 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140 continuation:239
│ │ │ │ │ │ │ ├── key columns: [1] = [85]
│ │ │ │ │ │ │ ├── first join in paired joiner; continuation column: continuation:239
│ │ │ │ │ │ │ ├── key: (1,84)
│ │ │ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,36,37,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3), (84)-->(85,239)
│ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140 indexrelid:932 indrelid:933 continuation:953
│ │ │ │ │ │ │ ├── key columns: [1] = [933]
│ │ │ │ │ │ │ ├── first join in paired joiner; continuation column: continuation:953
│ │ │ │ │ │ │ ├── key: (1,932)
│ │ │ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,36,37,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3), (932)-->(933,953)
│ │ │ │ │ │ │ ├── left-join (lookup pg_tablespace [as=t])
│ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140
│ │ │ │ │ │ │ │ ├── key columns: [8] = [36]
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/opt/xform/testdata/external/navicat
Original file line number Diff line number Diff line change
Expand Up @@ -210,17 +210,17 @@ sort
│ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (84)-->(85), (1,84)-->(36,37,91,105,106), (105)-->(106), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3)
│ │ │ │ │ ├── left-join (lookup pg_index [as=ind])
│ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 indexrelid:84 indrelid:85 indisclustered:91 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140
│ │ │ │ │ │ ├── key columns: [84] = [84]
│ │ │ │ │ │ ├── key columns: [932] = [84]
│ │ │ │ │ │ ├── lookup columns are key
│ │ │ │ │ │ ├── second join in paired joiner
│ │ │ │ │ │ ├── key: (1,84)
│ │ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (84)-->(85), (1,84)-->(36,37,91), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3)
│ │ │ │ │ │ ├── left-join (lookup pg_index@pg_index_indrelid_index [as=ind])
│ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 indexrelid:84 indrelid:85 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140 continuation:239
│ │ │ │ │ │ │ ├── key columns: [1] = [85]
│ │ │ │ │ │ │ ├── first join in paired joiner; continuation column: continuation:239
│ │ │ │ │ │ │ ├── key: (1,84)
│ │ │ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,36,37,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3), (84)-->(85,239)
│ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140 indexrelid:932 indrelid:933 continuation:953
│ │ │ │ │ │ │ ├── key columns: [1] = [933]
│ │ │ │ │ │ │ ├── first join in paired joiner; continuation column: continuation:953
│ │ │ │ │ │ │ ├── key: (1,932)
│ │ │ │ │ │ │ ├── fd: ()-->(3,30,31), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,36,37,134-136,139,140), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (134)-->(135,136,139,140), (139)~~>(140), (140)~~>(139), (36)-->(37), (37)-->(36), (3)==(30), (30)==(3), (932)-->(933,953)
│ │ │ │ │ │ │ ├── left-join (lookup pg_tablespace [as=t])
│ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:30!null n.nspname:31!null t.oid:36 spcname:37 ftrelid:134 ftserver:135 ftoptions:136 fs.oid:139 srvname:140
│ │ │ │ │ │ │ │ ├── key columns: [8] = [36]
Expand Down
Loading

0 comments on commit 67893af

Please sign in to comment.