Skip to content

Commit

Permalink
sql: support INSERT with partial UNIQUE WITHOUT INDEX constraints
Browse files Browse the repository at this point in the history
Uniqueness checks on INSERTs are now performed for partial UNIQUE
WITHOUT INDEX constraints. The semi-join filters in the uniqueness
checks filter out rows that do not satisfy the predicate. Rows on both
sides of the join are filtered. This is required to prevent duplicate
key violations from occurring for rows that do not satisfy the predicate
of the partial unique constraint.

The WithScan and Scan of the uniqueness check now produce all ordinary
table columns, rather than just the PK and unique columns, because
predicates may refer any columns in the table.

Informs cockroachdb#59195

There is no release note because these constraints are gated behind the
experimental_enable_unique_without_index_constraints session variable.

Release note: None
  • Loading branch information
mgartner committed Feb 12, 2021
1 parent 80b33c8 commit a2ad21a
Show file tree
Hide file tree
Showing 6 changed files with 1,241 additions and 717 deletions.
43 changes: 43 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/unique
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,49 @@ us-west foo 1 1
eu-west bar 2 2


# Insert some non-null data into a table with a partial unique without index
# constraint.
statement ok
INSERT INTO uniq_partial VALUES (1, 1), (1, -1), (2, 2)

# Partial unique constraint violation.
statement error pgcode 23505 pq: duplicate key value violates unique constraint "unique_a"\nDETAIL: Key \(a\)=\(1\) already exists\.
INSERT INTO uniq_partial VALUES (1, 3)

# No partial unique constraint violation because b <= 0.
statement ok
INSERT INTO uniq_partial VALUES (1, -3)

# Attempt to insert conflicting keys twice in the same statement.
statement error pgcode 23505 pq: duplicate key value violates unique constraint "unique_a"\nDETAIL: Key \(a\)=\(3\) already exists\.
INSERT INTO uniq_partial VALUES (3, 3), (3, 4)

# Attempt to insert one conflicting key and one non-conflicting key in the same
# statement.
statement error pgcode 23505 pq: duplicate key value violates unique constraint "unique_a"\nDETAIL: Key \(a\)=\(1\) already exists\.
INSERT INTO uniq_partial VALUES (1, 3), (3, 3)

# Insert some rows with NULL keys.
statement ok
INSERT INTO uniq_partial VALUES (NULL, 5), (5, 5), (NULL, 5)

# Insert with non-constant input.
statement error pgcode 23505 pq: duplicate key value violates unique constraint "unique_a"\nDETAIL: Key \(a\)=\(1\) already exists\.
INSERT INTO uniq_partial SELECT w, x FROM other

query II colnames,rowsort
SELECT * FROM uniq_partial
----
a b
1 1
1 -1
1 -3
2 2
5 5
NULL 5
NULL 5


# -- Tests with UPDATE --
subtest Update

Expand Down
34 changes: 34 additions & 0 deletions pkg/sql/opt/optbuilder/mutation_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,11 @@ type mutationBuilder struct {
// reuse.
parsedIndexExprs []tree.Expr

// parsedUniqueConstraintExprs is a cached set of parsed partial unique
// constraint predicate expressions from the table schema. These are parsed
// once and cached for reuse.
parsedUniqueConstraintExprs []tree.Expr

// uniqueChecks contains unique check queries; see buildUnique* methods.
uniqueChecks memo.UniqueChecksExpr

Expand Down Expand Up @@ -1193,6 +1198,35 @@ func (mb *mutationBuilder) parsePartialIndexPredicateExpr(idx cat.IndexOrdinal)
return expr
}

// parseUniqueConstraintPredicateExpr parses the predicate of the given partial
// unique constraint and caches it for reuse. This function panics if the unique
// constraint at the given ordinal is not partial.
func (mb *mutationBuilder) parseUniqueConstraintPredicateExpr(idx cat.UniqueOrdinal) tree.Expr {
uniqueConstraint := mb.tab.Unique(idx)

predStr, isPartial := uniqueConstraint.Predicate()
if !isPartial {
panic(errors.AssertionFailedf("unique constraints at ordinal %d is not a partial unique constraint", idx))
}

if mb.parsedUniqueConstraintExprs == nil {
mb.parsedUniqueConstraintExprs = make([]tree.Expr, mb.tab.UniqueCount())
}

// Return expression from the cache, if it was already parsed previously.
if mb.parsedUniqueConstraintExprs[idx] != nil {
return mb.parsedUniqueConstraintExprs[idx]
}

expr, err := parser.ParseExpr(predStr)
if err != nil {
panic(err)
}

mb.parsedUniqueConstraintExprs[idx] = expr
return expr
}

// getIndexLaxKeyOrdinals returns the ordinals of all lax key columns in the
// given index. A column's ordinal is the ordered position of that column in the
// owning table.
Expand Down
103 changes: 70 additions & 33 deletions pkg/sql/opt/optbuilder/mutation_builder_unique.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
)

Expand Down Expand Up @@ -129,12 +130,11 @@ type uniqueCheckHelper struct {
// uniqueOrdinals are the table ordinals of the unique columns in the table
// that is being mutated. They correspond 1-to-1 to the columns in the
// UniqueConstraint.
uniqueOrdinals []int
uniqueOrdinals util.FastIntSet

// uniqueAndPrimaryKeyOrdinals includes all the ordinals from uniqueOrdinals,
// plus the ordinals from any primary key columns that are not already
// included in uniqueOrdinals.
uniqueAndPrimaryKeyOrdinals []int
// primaryKeyOrdinals includes the ordinals from any primary key columns
// that are not included in uniqueOrdinals.
primaryKeyOrdinals util.FastIntSet
}

// init initializes the helper with a unique constraint.
Expand All @@ -150,15 +150,18 @@ func (h *uniqueCheckHelper) init(mb *mutationBuilder, uniqueOrdinal int) bool {
uniqueOrdinal: uniqueOrdinal,
}

uniqueCount := h.unique.ColumnCount()

var uniqueOrds util.FastIntSet
for i := 0; i < uniqueCount; i++ {
for i, n := 0, h.unique.ColumnCount(); i < n; i++ {
uniqueOrds.Add(h.unique.ColumnOrdinal(mb.tab, i))
}

// Find the primary key columns that are not part of the unique constraint.
// If there aren't any, we don't need a check.
// TODO(mgartner): We also don't need a check if there exists a unique index
// with columns that are a subset of the unique constraint columns.
// Similarly, we don't need a check for a partial unique constraint if there
// exists a non-partial unique constraint with columns that are a subset of
// the partial unique constrain columns.
primaryOrds := getIndexLaxKeyOrdinals(mb.tab.Index(cat.PrimaryIndex))
primaryOrds.DifferenceWith(uniqueOrds)
if primaryOrds.Empty() {
Expand All @@ -167,13 +170,13 @@ func (h *uniqueCheckHelper) init(mb *mutationBuilder, uniqueOrdinal int) bool {
return false
}

h.uniqueAndPrimaryKeyOrdinals = append(uniqueOrds.Ordered(), primaryOrds.Ordered()...)
h.uniqueOrdinals = h.uniqueAndPrimaryKeyOrdinals[:uniqueCount]
h.uniqueOrdinals = uniqueOrds
h.primaryKeyOrdinals = primaryOrds

// Check if we are setting NULL values for the unique columns, like when this
// mutation is the result of a SET NULL cascade action.
numNullCols := 0
for _, tabOrd := range h.uniqueOrdinals {
for tabOrd, ok := h.uniqueOrdinals.Next(0); ok; tabOrd, ok = h.uniqueOrdinals.Next(tabOrd + 1) {
colID := mb.mapToReturnColID(tabOrd)
if memo.OutputColumnIsAlwaysNull(mb.outScope.expr, colID) {
numNullCols++
Expand All @@ -189,26 +192,31 @@ func (h *uniqueCheckHelper) init(mb *mutationBuilder, uniqueOrdinal int) bool {
// table. The input to the insertion check will be produced from the input to
// the mutation operator.
func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem {
withScanScope, _ := h.mb.buildCheckInputScan(
checkInputScanNewVals, h.uniqueAndPrimaryKeyOrdinals,
)

numCols := len(withScanScope.cols)
f := h.mb.b.factory

// Build a self semi-join, with the new values on the left and the
// existing values on the right.
scanScope, ordinals := h.buildTableScan()

scanScope, _ := h.buildTableScan()
withScanScope, _ := h.mb.buildCheckInputScan(
checkInputScanNewVals, ordinals,
)

// Build the join filters:
// (new_a = existing_a) AND (new_b = existing_b) AND ...
//
// Set the capacity to len(h.uniqueOrdinals)+1 since we'll have an equality
// Set the capacity to h.uniqueOrdinals.Len()+1 since we'll have an equality
// condition for each column in the unique constraint, plus one additional
// condition to prevent rows from matching themselves (see below).
semiJoinFilters := make(memo.FiltersExpr, 0, len(h.uniqueOrdinals)+1)
for i := 0; i < len(h.uniqueOrdinals); i++ {
// condition to prevent rows from matching themselves (see below). If the
// constraint is partial, add 2 to account for filtering both the WithScan
// and the Scan by the partial unique constraint predicate.
numFilters := h.uniqueOrdinals.Len() + 1
_, isPartial := h.unique.Predicate()
if isPartial {
numFilters += 2
}
semiJoinFilters := make(memo.FiltersExpr, 0, numFilters)
for i, ok := h.uniqueOrdinals.Next(0); ok; i, ok = h.uniqueOrdinals.Next(i + 1) {
semiJoinFilters = append(semiJoinFilters, f.ConstructFiltersItem(
f.ConstructEq(
f.ConstructVariable(withScanScope.cols[i].id),
Expand All @@ -217,12 +225,29 @@ func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem {
))
}

// If the unique constraint is partial, we need to filter out inserted rows
// that don't satisfy the predicate. We also need to make sure that rows do
// not match existing rows in the the table that do not satisfy the
// predicate. So we add the predicate as a filter on both the WithScan
// columns and the Scan columns.
if isPartial {
pred := h.mb.parseUniqueConstraintPredicateExpr(h.uniqueOrdinal)

typedPred := withScanScope.resolveAndRequireType(pred, types.Bool)
withScanPred := h.mb.b.buildScalar(typedPred, withScanScope, nil, nil, nil)
semiJoinFilters = append(semiJoinFilters, f.ConstructFiltersItem(withScanPred))

typedPred = scanScope.resolveAndRequireType(pred, types.Bool)
scanPred := h.mb.b.buildScalar(typedPred, scanScope, nil, nil, nil)
semiJoinFilters = append(semiJoinFilters, f.ConstructFiltersItem(scanPred))
}

// We need to prevent rows from matching themselves in the semi join. We can
// do this by adding another filter that uses the primary keys to check if
// two rows are identical:
// (new_pk1 != existing_pk1) OR (new_pk2 != existing_pk2) OR ...
var pkFilter opt.ScalarExpr
for i := len(h.uniqueOrdinals); i < numCols; i++ {
for i, ok := h.primaryKeyOrdinals.Next(0); ok; i, ok = h.primaryKeyOrdinals.Next(i + 1) {
pkFilterLocal := f.ConstructNe(
f.ConstructVariable(withScanScope.cols[i].id),
f.ConstructVariable(scanScope.cols[i].id),
Expand All @@ -237,26 +262,38 @@ func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem {

semiJoin := f.ConstructSemiJoin(withScanScope.expr, scanScope.expr, semiJoinFilters, memo.EmptyJoinPrivate)

// Collect the key columns that will be shown in the error message if there
// is a duplicate key violation resulting from this uniqueness check.
keyCols := make(opt.ColList, 0, h.uniqueOrdinals.Len())
for i, ok := h.uniqueOrdinals.Next(0); ok; i, ok = h.uniqueOrdinals.Next(i + 1) {
keyCols = append(keyCols, withScanScope.cols[i].id)
}

return f.ConstructUniqueChecksItem(semiJoin, &memo.UniqueChecksItemPrivate{
Table: h.mb.tabID,
CheckOrdinal: h.uniqueOrdinal,
// uniqueOrdinals is always a prefix of uniqueAndPrimaryKeyOrdinals,
// which maps 1-to-1 to the columns in withScanScope.cols. The remaining
// columns are primary key columns and should not be included in the
// KeyCols.
KeyCols: withScanScope.colList()[:len(h.uniqueOrdinals)],
OpName: h.mb.opName,
KeyCols: keyCols,
OpName: h.mb.opName,
})
}

// buildTableScan builds a Scan of the table.
func (h *uniqueCheckHelper) buildTableScan() (outScope *scope, tabMeta *opt.TableMeta) {
tabMeta = h.mb.b.addTable(h.mb.tab, tree.NewUnqualifiedTableName(h.mb.tab.Name()))
// buildTableScan builds a Scan of the table. The ordinals of the columns
// scanned are also returned.
func (h *uniqueCheckHelper) buildTableScan() (outScope *scope, ordinals []int) {
tabMeta := h.mb.b.addTable(h.mb.tab, tree.NewUnqualifiedTableName(h.mb.tab.Name()))
// TODO(mgartner): Should mutation columns be included here? Does uniqueness
// need to hold true for write-only columns?
ordinals = tableOrdinals(tabMeta.Table, columnKinds{
includeMutations: false,
includeSystem: false,
includeVirtualInverted: false,
includeVirtualComputed: true,
})
return h.mb.b.buildScan(
tabMeta,
h.uniqueAndPrimaryKeyOrdinals,
ordinals,
nil, /* indexFlags */
noRowLocking,
h.mb.b.allocScope(),
), tabMeta
), ordinals
}
Loading

0 comments on commit a2ad21a

Please sign in to comment.