Skip to content

Commit

Permalink
Merge #60289
Browse files Browse the repository at this point in the history
60289: sql: allow creation of partial UNIQUE WITHOUT INDEX constraints r=mgartner a=mgartner

This commit allows creation of partial `UNIQUE WITHOUT INDEX`
constraints. These partial unique constraints have a predicate to
enforce uniqueness on the subset of rows in which the predicate
evaluates to true, similar to a unique partial index.

An example of a partial `UNIQUE WITHOUT INDEX` constraint:

    CREATE TABLE t (
      a INT,
      UNIQUE WITHOUT INDEX (a) WHERE a > 0
    )

**NOTE**: these constraints are not yet correctly enforcing uniqueness
on a subset of rows in the table. This requires optimizer changes that
will come in a follow-up commit. Specifically, this commit adds support
for:

  - Creating these constraints in `CREATE TABLE` and `ALTER TABLE`.
  - Storing the predicates in table descriptor constraints.
  - Validating the predicates both during creation and during descriptor
    validation.
  - Displaying the predicates in `SHOW CREATE` and `SHOW CONSTRAINTS`
    output, and in the pg_catalog.pg_constraint table.
  - Accessing the predicates via the opt catalog.

The primary motivation for this change is to simplify testing for
implicitly partitioned partial unique indexes. An experimental session
setting is required to use `UNIQUE WITHOUT INDEX` there are no current
plans to make these constraints available to users by default.

Informs #59195

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

Release note: None

Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
craig[bot] and mgartner committed Feb 12, 2021
2 parents ef917b6 + ed7d944 commit ea1ef0b
Show file tree
Hide file tree
Showing 23 changed files with 761 additions and 512 deletions.
2 changes: 2 additions & 0 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,8 +191,10 @@ func (n *alterTableNode) startExec(params runParams) error {
params.SessionData(),
d,
n.tableDesc,
*tn,
NonEmptyTable,
t.ValidationBehavior,
params.p.SemaCtx(),
); err != nil {
return err
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/descpb/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -362,3 +362,8 @@ func (u *UniqueWithoutIndexConstraint) IsValidReferencedUniqueConstraint(
func (u *UniqueWithoutIndexConstraint) GetName() string {
return u.Name
}

// IsPartial returns true if the constraint is a partial unique constraint.
func (u *UniqueWithoutIndexConstraint) IsPartial() bool {
return u.Predicate != ""
}
816 changes: 429 additions & 387 deletions pkg/sql/catalog/descpb/structured.pb.go

Large diffs are not rendered by default.

5 changes: 5 additions & 0 deletions pkg/sql/catalog/descpb/structured.proto
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,11 @@ message UniqueWithoutIndexConstraint {
(gogoproto.casttype) = "ColumnID"];
optional string name = 3 [(gogoproto.nullable) = false];
optional ConstraintValidity validity = 4 [(gogoproto.nullable) = false];

// Predicate, if it's not empty, indicates that the constraint is a partial
// unique constraint with Predicate as the expression. Columns are referred to
// in the expression by their name.
optional string predicate = 5 [(gogoproto.nullable) = false];
}

message ColumnDescriptor {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/schemaexpr/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_library(
"expr_filter.go",
"partial_index.go",
"select_name_resolution.go",
"unique_contraint.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr",
visibility = ["//visibility:public"],
Expand Down
54 changes: 54 additions & 0 deletions pkg/sql/catalog/schemaexpr/unique_contraint.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package schemaexpr

import (
"context"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

// ValidateUniqueWithoutIndexPredicate verifies that an expression is a valid
// unique without index predicate. If the expression is valid, it returns the
// serialized expression with the columns dequalified.
//
// A predicate expression is valid if all of the following are true:
//
// - It results in a boolean.
// - It refers only to columns in the table.
// - It does not include subqueries.
// - It does not include non-immutable, aggregate, window, or set returning
// functions.
//
func ValidateUniqueWithoutIndexPredicate(
ctx context.Context,
tn tree.TableName,
desc catalog.TableDescriptor,
pred tree.Expr,
semaCtx *tree.SemaContext,
) (string, error) {
expr, _, err := DequalifyAndValidateExpr(
ctx,
desc,
pred,
types.Bool,
"unique without index predicate",
semaCtx,
tree.VolatilityImmutable,
&tn,
)
if err != nil {
return "", err
}
return expr, nil
}
20 changes: 19 additions & 1 deletion pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -2220,6 +2220,24 @@ func (desc *wrapper) validateUniqueWithoutIndexConstraints(
}
seen.Add(int(colID))
}

if c.IsPartial() {
expr, err := parser.ParseExpr(c.Predicate)
if err != nil {
return err
}
valid, err := schemaexpr.HasValidColumnReferences(desc, expr)
if err != nil {
return err
}
if !valid {
return fmt.Errorf(
"partial unique without index constraint %q refers to unknown columns in predicate: %s",
c.Name,
c.Predicate,
)
}
}
}

return nil
Expand Down Expand Up @@ -3458,7 +3476,7 @@ func (desc *Mutable) AddForeignKeyMutation(
desc.addMutation(m)
}

// AddUniqueWithoutIndexMutation adds a unqiue without index constraint mutation
// AddUniqueWithoutIndexMutation adds a unique without index constraint mutation
// to desc.Mutations.
func (desc *Mutable) AddUniqueWithoutIndexMutation(
uc *descpb.UniqueWithoutIndexConstraint, direction descpb.DescriptorMutation_Direction,
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/catalog/tabledesc/validate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,16 @@ var validationMap = []struct {
"Match": {status: thisFieldReferencesNoObjects},
},
},
{
obj: descpb.UniqueWithoutIndexConstraint{},
fieldMap: map[string]validationStatusInfo{
"TableID": {status: iSolemnlySwearThisFieldIsValidated},
"ColumnIDs": {status: iSolemnlySwearThisFieldIsValidated},
"Name": {status: thisFieldReferencesNoObjects},
"Validity": {status: thisFieldReferencesNoObjects},
"Predicate": {status: iSolemnlySwearThisFieldIsValidated},
},
},
{
obj: descpb.TypeDescriptor{},
fieldMap: map[string]validationStatusInfo{
Expand Down
30 changes: 23 additions & 7 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -640,7 +640,13 @@ func addUniqueWithoutIndexColumnTableDef(
}
// Add a unique constraint.
if err := ResolveUniqueWithoutIndexConstraint(
ctx, desc, string(d.Unique.ConstraintName), []string{string(d.Name)}, ts, validationBehavior,
ctx,
desc,
string(d.Unique.ConstraintName),
[]string{string(d.Name)},
"", /* predicate */
ts,
validationBehavior,
); err != nil {
return err
}
Expand All @@ -656,8 +662,10 @@ func addUniqueWithoutIndexTableDef(
sessionData *sessiondata.SessionData,
d *tree.UniqueConstraintTableDef,
desc *tabledesc.Mutable,
tn tree.TableName,
ts TableState,
validationBehavior tree.ValidationBehavior,
semaCtx *tree.SemaContext,
) error {
if !evalCtx.Settings.Version.IsActive(ctx, clusterversion.UniqueWithoutIndexConstraints) {
return pgerror.Newf(pgcode.FeatureNotSupported,
Expand All @@ -684,20 +692,26 @@ func addUniqueWithoutIndexTableDef(
"partitioned unique constraints without an index are not supported",
)
}

// If there is a predicate, validate it.
var predicate string
if d.Predicate != nil {
// TODO(rytaft): It may be necessary to support predicates so that partial
// unique indexes will work correctly in multi-region deployments.
return pgerror.New(pgcode.FeatureNotSupported,
"unique constraints with a predicate but without an index are not supported",
var err error
predicate, err = schemaexpr.ValidateUniqueWithoutIndexPredicate(
ctx, tn, desc, d.Predicate, semaCtx,
)
if err != nil {
return err
}
}

// Add a unique constraint.
colNames := make([]string, len(d.Columns))
for i := range colNames {
colNames[i] = string(d.Columns[i].Column)
}
if err := ResolveUniqueWithoutIndexConstraint(
ctx, desc, string(d.Name), colNames, ts, validationBehavior,
ctx, desc, string(d.Name), colNames, predicate, ts, validationBehavior,
); err != nil {
return err
}
Expand All @@ -716,6 +730,7 @@ func ResolveUniqueWithoutIndexConstraint(
tbl *tabledesc.Mutable,
constraintName string,
colNames []string,
predicate string,
ts TableState,
validationBehavior tree.ValidationBehavior,
) error {
Expand Down Expand Up @@ -772,6 +787,7 @@ func ResolveUniqueWithoutIndexConstraint(
Name: constraintName,
TableID: tbl.ID,
ColumnIDs: columnIDs,
Predicate: predicate,
Validity: validity,
}

Expand Down Expand Up @@ -2156,7 +2172,7 @@ func NewTableDesc(
case *tree.UniqueConstraintTableDef:
if d.WithoutIndex {
if err := addUniqueWithoutIndexTableDef(
ctx, evalCtx, sessionData, d, &desc, NewTable, tree.ValidationDefault,
ctx, evalCtx, sessionData, d, &desc, n.Table, NewTable, tree.ValidationDefault, semaCtx,
); err != nil {
return nil, err
}
Expand Down
95 changes: 55 additions & 40 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -1471,7 +1471,17 @@ ALTER TABLE unique_without_index ADD COLUMN f INT UNIQUE WITHOUT INDEX
# constraint.
statement ok
ALTER TABLE unique_without_index ADD COLUMN f INT;
ALTER TABLE unique_without_index ADD CONSTRAINT my_unique_f UNIQUE WITHOUT INDEX (f)
ALTER TABLE unique_without_index ADD CONSTRAINT my_unique_f UNIQUE WITHOUT INDEX (f);
ALTER TABLE unique_without_index ADD CONSTRAINT my_partial_unique_f UNIQUE WITHOUT INDEX (f) WHERE f > 0

# The unique constraint predicate must be valid. It cannot reference
# non-existent columns.
# TODO(mgartner): If we ever decide to make UNIQUE WITHOUT INDEX available
# without the experimental setting, we should test that other types of invalid
# predicates produce errors. There are similar tests in the partial_index
# logictests that would be a good starting point.
statement error column "g" does not exist
ALTER TABLE unique_without_index ADD CONSTRAINT bad_partial_unique UNIQUE WITHOUT INDEX (f) WHERE g > 0

# The unique constraint prevents new duplicate values.
statement error pgcode 23505 pq: duplicate key value violates unique constraint "my_unique_f"\nDETAIL: Key \(f\)=\(1\) already exists\.
Expand Down Expand Up @@ -1508,17 +1518,18 @@ ALTER TABLE unique_without_index VALIDATE CONSTRAINT unique_without_index_c_key
query TTTTB colnames
SHOW CONSTRAINTS FROM unique_without_index
----
table_name constraint_name constraint_type details validated
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_e2 UNIQUE UNIQUE WITHOUT INDEX (e) NOT VALID false
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_a_b UNIQUE UNIQUE WITHOUT INDEX (a, b) true
unique_without_index unique_b UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_b_1 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true
table_name constraint_name constraint_type details validated
unique_without_index my_partial_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) WHERE (f > 0) true
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_e2 UNIQUE UNIQUE WITHOUT INDEX (e) NOT VALID false
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_a_b UNIQUE UNIQUE WITHOUT INDEX (a, b) true
unique_without_index unique_b UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_b_1 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true

statement ok
ALTER TABLE unique_without_index RENAME COLUMN a TO aa
Expand All @@ -1535,16 +1546,17 @@ ALTER TABLE unique_without_index RENAME CONSTRAINT unique_b TO unique_b_2
query TTTTB
SHOW CONSTRAINTS FROM unique_without_index
----
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_e2 UNIQUE UNIQUE WITHOUT INDEX (e) NOT VALID false
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_a_b UNIQUE UNIQUE WITHOUT INDEX (aa, b) true
unique_without_index unique_b_1 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_b_2 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true
unique_without_index my_partial_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) WHERE (f > 0) true
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_e2 UNIQUE UNIQUE WITHOUT INDEX (e) NOT VALID false
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_a_b UNIQUE UNIQUE WITHOUT INDEX (aa, b) true
unique_without_index unique_b_1 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_b_2 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true

statement error pgcode 0A000 cannot drop UNIQUE constraint \"unique_without_index_c_key\"
ALTER TABLE unique_without_index DROP CONSTRAINT unique_without_index_c_key
Expand All @@ -1563,14 +1575,15 @@ ALTER TABLE unique_without_index DROP CONSTRAINT my_unique_e2
query TTTTB
SHOW CONSTRAINTS FROM unique_without_index
----
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_a_b UNIQUE UNIQUE WITHOUT INDEX (aa, b) true
unique_without_index unique_b_1 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true
unique_without_index my_partial_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) WHERE (f > 0) true
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_a_b UNIQUE UNIQUE WITHOUT INDEX (aa, b) true
unique_without_index unique_b_1 UNIQUE UNIQUE WITHOUT INDEX (b) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true

# Dropping a column in a unique constraint drops the constraint.
statement ok
Expand All @@ -1579,12 +1592,13 @@ ALTER TABLE unique_without_index DROP COLUMN b
query TTTTB
SHOW CONSTRAINTS FROM unique_without_index
----
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true
unique_without_index my_partial_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) WHERE (f > 0) true
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_d UNIQUE UNIQUE WITHOUT INDEX (d) true
unique_without_index unique_d_e UNIQUE UNIQUE WITHOUT INDEX (d, e) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true

query TTTTB
SHOW CONSTRAINTS FROM uwi_child
Expand All @@ -1603,10 +1617,11 @@ ALTER TABLE unique_without_index DROP COLUMN d CASCADE
query TTTTB
SHOW CONSTRAINTS FROM unique_without_index
----
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true
unique_without_index my_partial_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) WHERE (f > 0) true
unique_without_index my_unique_e UNIQUE UNIQUE WITHOUT INDEX (e) true
unique_without_index my_unique_f UNIQUE UNIQUE WITHOUT INDEX (f) true
unique_without_index unique_c UNIQUE UNIQUE WITHOUT INDEX (c) true
unique_without_index unique_without_index_c_key UNIQUE UNIQUE (c ASC) true

query TTTTB
SHOW CONSTRAINTS FROM uwi_child
Expand Down
13 changes: 11 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/create_table
Original file line number Diff line number Diff line change
Expand Up @@ -403,8 +403,17 @@ CREATE TABLE error (a INT, b INT, UNIQUE WITHOUT INDEX (a) PARTITION BY LIST (b)
PARTITION p1 VALUES IN (1)
))

statement error pgcode 0A000 unique constraints with a predicate but without an index are not supported
CREATE TABLE error (a INT, b INT, UNIQUE WITHOUT INDEX (a) WHERE b > 5)
# The unique constraint predicate must be valid. It cannot reference
# non-existent columns.
# TODO(mgartner): If we ever decide to make UNIQUE WITHOUT INDEX available
# without the experimental setting, we should test that other types of invalid
# predicates produce errors. There are similar tests in the partial_index
# logictests that would be a good starting point.
statement error column "c" does not exist
CREATE TABLE unique_without_index_partial (a INT, b INT, UNIQUE WITHOUT INDEX (a) WHERE c > 0)

statement ok
CREATE TABLE unique_without_index_partial (a INT, b INT, UNIQUE WITHOUT INDEX (a) WHERE b > 5)

subtest regression_57630

Expand Down
Loading

0 comments on commit ea1ef0b

Please sign in to comment.