Skip to content

Commit

Permalink
Merge #55702
Browse files Browse the repository at this point in the history
55702: sql: fix partial index and foreign key bugs r=mgartner a=mgartner

#### opt: build partial index predicates only when all table columns are in scope

This commit fixes a panic induced by trying to build partial index
predicate expressions without all of a table's columns in-scope. Some
scans, like scans built for foreign key checks, do not include all of a
table's column in their scope. For such scans, optbuilder no longer
attempts to build a partial index predicate because the predicate may
reference a column that is not in scope.

As a result of this change, `opt.TableMeta` may not have a predicate
expression for all partial indexes. The `memo.PartialIndexPredicate`
function which retrieves the predicate expressions has been updated to
account for this case.

Fixes #55672

Release justification: This is a critical bug fix for a new feature,
partial indexes.

Release note (bug fix): An INSERT into a table with a foreign key
reference to a table with a partial index no longer causes an error.

#### sql: disqualify partial unique indexes as foreign key reference indexes

Release justification: This is a critical bug fix for a new feature,
partial indexes.

Release note (bug fix): Foreign keys can no longer reference columns
that are only indexed by a partial unique index. A partial unique index
does not guarantee uniqueness in the entire table, therefore the column
indexed is not guaranteed to be a unique key.


Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
  • Loading branch information
craig[bot] and mgartner committed Oct 19, 2020
2 parents 9083be9 + 21fd3a6 commit e4d1eb1
Show file tree
Hide file tree
Showing 10 changed files with 148 additions and 22 deletions.
4 changes: 2 additions & 2 deletions pkg/sql/catalog/descpb/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,13 +160,13 @@ func (desc *IndexDescriptor) ColNamesString() string {
// IsValidOriginIndex returns whether the index can serve as an origin index for a foreign
// key constraint with the provided set of originColIDs.
func (desc *IndexDescriptor) IsValidOriginIndex(originColIDs ColumnIDs) bool {
return ColumnIDs(desc.ColumnIDs).HasPrefix(originColIDs)
return !desc.IsPartial() && ColumnIDs(desc.ColumnIDs).HasPrefix(originColIDs)
}

// IsValidReferencedIndex returns whether the index can serve as a referenced index for a foreign
// key constraint with the provided set of referencedColumnIDs.
func (desc *IndexDescriptor) IsValidReferencedIndex(referencedColIDs ColumnIDs) bool {
return desc.Unique && ColumnIDs(desc.ColumnIDs).Equals(referencedColIDs)
return desc.Unique && !desc.IsPartial() && ColumnIDs(desc.ColumnIDs).Equals(referencedColIDs)
}

// HasOldStoredColumns returns whether the index has stored columns in the old
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/fk
Original file line number Diff line number Diff line change
Expand Up @@ -3626,3 +3626,13 @@ USE db2

statement ok
CREATE TABLE child2 (c INT PRIMARY KEY, p INT REFERENCES db1.public.parent(p))

# Test that foreign keys cannot reference columns that are indexed by a partial
# unique index. Partial unique indexes do not guarantee uniqueness in the entire
# table.

statement ok
CREATE TABLE partial_parent (p INT, UNIQUE INDEX (p) WHERE p > 100)

statement error there is no unique constraint matching given keys for referenced table partial_parent
CREATE TABLE partial_child (p INT REFERENCES partial_parent (p))
28 changes: 28 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/partial_index
Original file line number Diff line number Diff line change
Expand Up @@ -1423,3 +1423,31 @@ INSERT INTO t55387 VALUES (1, 1, 5);
query I rowsort
SELECT k FROM t55387 WHERE a > 1 AND b > 3
----

# Regression test for #55672. Do not build partial index predicates when the
# scope does not include all table columns, like FK check scans.

statement ok
CREATE TABLE t55672_a (
a INT PRIMARY KEY,
t TIMESTAMPTZ DEFAULT NULL,
UNIQUE INDEX (a) WHERE t is NULL
)

statement ok
CREATE TABLE t55672_b (
b INT PRIMARY KEY,
a INT NOT NULL REFERENCES t55672_a (a)
)

statement ok
INSERT INTO t55672_a (a) VALUES (1)

statement ok
INSERT INTO t55672_b (b,a) VALUES (1,1)

statement ok
INSERT INTO t55672_a (a, t) VALUES (2, now())

statement ok
INSERT INTO t55672_b (b,a) VALUES (2,2)
25 changes: 10 additions & 15 deletions pkg/sql/opt/memo/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -645,11 +645,18 @@ func (s *ScanPrivate) UsesPartialIndex(md *opt.Metadata) bool {

// PartialIndexPredicate returns the FiltersExpr representing the predicate of
// the partial index that the scan uses. If the scan does not use a partial
// index, this function panics. UsesPartialIndex should be called first to
// determine if the scan operates over a partial index.
// index or if a partial index predicate was not built for this index, this
// function panics. UsesPartialIndex should be called first to determine if the
// scan operates over a partial index.
func (s *ScanPrivate) PartialIndexPredicate(md *opt.Metadata) FiltersExpr {
tabMeta := md.TableMeta(s.Table)
return PartialIndexPredicate(tabMeta, s.Index)
p, ok := tabMeta.PartialIndexPredicates[s.Index]
if !ok {
// A partial index predicate expression was not built for the
// partial index.
panic(errors.AssertionFailedf("partial index predicate not found for %s", tabMeta.Table.Index(s.Index).Name()))
}
return *p.(*FiltersExpr)
}

// UsesPartialIndex returns true if the the LookupJoinPrivate looks-up via a
Expand Down Expand Up @@ -894,18 +901,6 @@ func OutputColumnIsAlwaysNull(e RelExpr, col opt.ColumnID) bool {
return false
}

// PartialIndexPredicate returns the FiltersExpr representing the partial index
// predicate at the given index ordinal. If the index at the ordinal is not a
// partial index, this function panics. cat.Index.Predicate should be used first
// to determine if the index is a partial index.
//
// Note that TableMeta.PartialIndexPredicates is not initialized for mutation
// queries. This function will panic in the context of a mutation.
func PartialIndexPredicate(tabMeta *opt.TableMeta, ord cat.IndexOrdinal) FiltersExpr {
p := tabMeta.PartialIndexPredicates[ord]
return *p.(*FiltersExpr)
}

// FKCascades stores metadata necessary for building cascading queries.
type FKCascades []FKCascade

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/mutation_builder_fk.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ func (mb *mutationBuilder) buildFKChecksAndCascadesForDelete() {
// - with Cascade/SetNull/SetDefault, we create a cascading mutation to
// modify or delete "orphaned" rows in the child table.
// - with Restrict/NoAction, we create a check that causes an error if
// there are any "orhpaned" rows in the child table.
// there are any "orphaned" rows in the child table.
if a := h.fk.DeleteReferenceAction(); a != tree.Restrict && a != tree.NoAction {
telemetry.Inc(sqltelemetry.ForeignKeyCascadesUseCounter)
var builder memo.CascadeBuilder
Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/opt/optbuilder/partial_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)

// buildPartialIndexPredicate builds a memo.FiltersExpr from the given
Expand All @@ -33,7 +34,7 @@ import (
func (b *Builder) buildPartialIndexPredicate(
tableScope *scope, expr tree.Expr, context string,
) (memo.FiltersExpr, error) {
texpr := tableScope.resolveAndRequireType(expr, types.Bool)
texpr := resolvePartialIndexPredicate(tableScope, expr)

var scalar opt.ScalarExpr
b.factory.FoldingControl().TemporarilyDisallowStableFolds(func() {
Expand Down Expand Up @@ -67,3 +68,16 @@ func (b *Builder) buildPartialIndexPredicate(

return memo.TrueFilter, nil
}

// resolvePartialIndexPredicate attempts to resolve the type of expr as a
// boolean and return a tree.TypedExpr if successful. It asserts that no errors
// occur during resolution because the predicate should always be valid within
// this context. If an error occurs, it is likely due to a bug in the optimizer.
func resolvePartialIndexPredicate(tableScope *scope, expr tree.Expr) tree.TypedExpr {
defer func() {
if r := recover(); r != nil {
panic(errors.AssertionFailedf("unexpected error during partial index predicate type resolution: %v", r))
}
}()
return tableScope.resolveAndRequireType(expr, types.Bool)
}
22 changes: 20 additions & 2 deletions pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -523,8 +523,26 @@ func (b *Builder) buildScan(
outScope.expr = b.factory.ConstructScan(&private)

// Add the partial indexes after constructing the scan so we can use the
// logical properties of the scan to fully normalize the index predicates.
b.addPartialIndexPredicatesForTable(tabMeta, outScope)
// logical properties of the scan to fully normalize the index
// predicates. Partial index predicates are only added if the outScope
// contains all the table's ordinary columns. If it does not, partial
// index predicates cannot be built because they may reference columns
// not in outScope. In the most common case, the outScope has the same
// number of columns as the table and we can skip checking that each
// ordinary column exists in outScope.
containsAllOrdinaryTableColumns := true
if len(outScope.cols) != tab.ColumnCount() {
for i := 0; i < tab.ColumnCount(); i++ {
col := tab.Column(i)
if col.Kind() == cat.Ordinary && !outScope.colSet().Contains(tabID.ColumnID(col.Ordinal())) {
containsAllOrdinaryTableColumns = false
break
}
}
}
if containsAllOrdinaryTableColumns {
b.addPartialIndexPredicatesForTable(tabMeta, outScope)
}

if b.trackViewDeps {
dep := opt.ViewDep{DataSource: tab}
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/opt/testutils/testcat/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,6 +365,9 @@ func (tc *Catalog) resolveFK(tab *Table, d *tree.ForeignKeyConstraintTableDef) {
return false
}
}
if _, isPartialIndex := idx.Predicate(); isPartialIndex {
return false
}
return true
}

Expand Down
9 changes: 8 additions & 1 deletion pkg/sql/opt/xform/scan_index_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,14 @@ func (it *scanIndexIter) ForEachStartingAfter(ord int, f enumerateIndexFunc) {
// If the index is a partial index, check whether or not the
// originalFilters imply the predicate.
if isPartialIndex {
pred := memo.PartialIndexPredicate(it.tabMeta, ord)
p, ok := it.tabMeta.PartialIndexPredicates[ord]
if !ok {
// A partial index predicate expression was not built for the
// partial index. Implication cannot be proven so it must be
// skipped.
continue
}
pred := *p.(*memo.FiltersExpr)

// If there are no originalFilters, then skip over any partial
// indexes that are not pseudo-partial indexes.
Expand Down
51 changes: 51 additions & 0 deletions pkg/sql/opt/xform/testdata/rules/scan
Original file line number Diff line number Diff line change
Expand Up @@ -748,3 +748,54 @@ scan with_time_index@secondary
├── constraint: /2/1: [/'2017-05-10 00:00:00' - ]
├── key: (1)
└── fd: (1)-->(2)

exec-ddl
CREATE TABLE fk_a (
a INT PRIMARY KEY,
t TIMESTAMPTZ DEFAULT NULL,
UNIQUE INDEX (a) WHERE t is NULL
)
----

exec-ddl
CREATE TABLE fk_b (
b INT PRIMARY KEY,
a INT NOT NULL REFERENCES fk_a (a)
)
----

# Do not use a non-implied partial index for FK check scans.
opt expect-not=(GenerateIndexScans, GeneratePartialIndexScans)
INSERT INTO fk_b (b,a) VALUES (1,1)
----
insert fk_b
├── columns: <none>
├── insert-mapping:
│ ├── column1:4 => b:1
│ └── column2:5 => fk_b.a:2
├── input binding: &1
├── cardinality: [0 - 0]
├── volatile, mutations
├── values
│ ├── columns: column1:4!null column2:5!null
│ ├── cardinality: [1 - 1]
│ ├── key: ()
│ ├── fd: ()-->(4,5)
│ └── (1, 1)
└── f-k-checks
└── f-k-checks-item: fk_b(a) -> fk_a(a)
└── anti-join (lookup fk_a)
├── columns: column2:6!null
├── key columns: [6] = [7]
├── lookup columns are key
├── cardinality: [0 - 1]
├── key: ()
├── fd: ()-->(6)
├── with-scan &1
│ ├── columns: column2:6!null
│ ├── mapping:
│ │ └── column2:5 => column2:6
│ ├── cardinality: [1 - 1]
│ ├── key: ()
│ └── fd: ()-->(6)
└── filters (true)

0 comments on commit e4d1eb1

Please sign in to comment.