Skip to content

Commit

Permalink
schemachanger: Implement ADD UNIQUE WITHOUT INDEX NOT VALID and add…
Browse files Browse the repository at this point in the history
… tests
  • Loading branch information
Xiang-Gu committed Feb 3, 2023
1 parent 99a7b47 commit e7d394b
Show file tree
Hide file tree
Showing 23 changed files with 285 additions and 123 deletions.
6 changes: 5 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/drop_table
Original file line number Diff line number Diff line change
Expand Up @@ -118,10 +118,14 @@ DROP TABLE to_drop;
subtest drop_table_with_not_valid_constraints

statement ok
CREATE TABLE t_with_not_valid_constraints_1 (i INT PRIMARY KEY);
CREATE TABLE t_with_not_valid_constraints_1 (i INT PRIMARY KEY, j INT);

statement ok
ALTER TABLE t_with_not_valid_constraints_1 ADD CHECK (i > 0) NOT VALID;

statement ok
SET experimental_enable_unique_without_index_constraints = true;
ALTER TABLE t_with_not_valid_constraints_1 ADD UNIQUE WITHOUT INDEX (j) NOT VALID;

statement ok
DROP TABLE t_with_not_valid_constraints_1;
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scbuild/builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ func TestBuildDataDriven(t *testing.T) {
// changer will allow non-fully implemented operations.
sd.NewSchemaChangerMode = sessiondatapb.UseNewSchemaChangerUnsafe
sd.ApplicationName = ""
sd.EnableUniqueWithoutIndexConstraints = true
},
),
),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,10 +52,11 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{
reflect.TypeOf((*tree.AlterTableAddConstraint)(nil)): {fn: alterTableAddConstraint, on: true, extraChecks: func(
t *tree.AlterTableAddConstraint,
) bool {
// Support ALTER TABLE ... ADD PRIMARY KEY
if d, ok := t.ConstraintDef.(*tree.UniqueConstraintTableDef); ok && d.PrimaryKey && t.ValidationBehavior == tree.ValidationDefault {
// Support ALTER TABLE ... ADD PRIMARY KEY
return true
} else if ok && d.WithoutIndex && t.ValidationBehavior == tree.ValidationDefault {
} else if ok && d.WithoutIndex {
// Support ALTER TABLE ... ADD UNIQUE WITHOUT INDEX [NOT VALID]
return true
}

Expand Down Expand Up @@ -84,6 +85,7 @@ var alterTableAddConstraintMinSupportedClusterVersion = map[string]clusterversio
"ADD_FOREIGN_KEY_DEFAULT": clusterversion.V23_1Start,
"ADD_UNIQUE_WITHOUT_INDEX_DEFAULT": clusterversion.V23_1Start,
"ADD_CHECK_SKIP": clusterversion.V23_1,
"ADD_UNIQUE_WITHOUT_INDEX_SKIP": clusterversion.V23_1,
}

func init() {
Expand Down Expand Up @@ -187,7 +189,10 @@ func alterTableAddConstraintSupportedInCurrentClusterVersion(
cmdKey += "_SKIP"
}

minSupportedClusterVersion := alterTableAddConstraintMinSupportedClusterVersion[cmdKey]
minSupportedClusterVersion, ok := alterTableAddConstraintMinSupportedClusterVersion[cmdKey]
if !ok {
return false
}
return b.EvalCtx().Settings.Version.IsActive(b, minSupportedClusterVersion)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func alterTableAddConstraint(
case *tree.UniqueConstraintTableDef:
if d.PrimaryKey && t.ValidationBehavior == tree.ValidationDefault {
alterTableAddPrimaryKey(b, tn, tbl, t)
} else if d.WithoutIndex && t.ValidationBehavior == tree.ValidationDefault {
} else if d.WithoutIndex {
alterTableAddUniqueWithoutIndex(b, tn, tbl, t)
}
case *tree.CheckConstraintTableDef:
Expand Down Expand Up @@ -389,7 +389,8 @@ func alterTableAddForeignKey(
})
}

// alterTableAddUniqueWithoutIndex contains logic for building ALTER TABLE ... ADD CONSTRAINT ... UNIQUE WITHOUT INDEX.
// alterTableAddUniqueWithoutIndex contains logic for building
// `ALTER TABLE ... ADD UNIQUE WITHOUT INDEX ... [NOT VALID]`.
// It assumes `t` is such a command.
func alterTableAddUniqueWithoutIndex(
b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableAddConstraint,
Expand Down Expand Up @@ -473,17 +474,30 @@ func alterTableAddUniqueWithoutIndex(

// 5. (Finally!) Add a UniqueWithoutIndex, ConstraintName element to builder state.
constraintID := b.NextTableConstraintID(tbl.TableID)
uwi := &scpb.UniqueWithoutIndexConstraint{
TableID: tbl.TableID,
ConstraintID: constraintID,
ColumnIDs: colIDs,
IndexIDForValidation: getIndexIDForValidationForConstraint(b, tbl.TableID),
}
if d.Predicate != nil {
uwi.Predicate = b.WrapExpression(tbl.TableID, d.Predicate)
if t.ValidationBehavior == tree.ValidationDefault {
uwi := &scpb.UniqueWithoutIndexConstraint{
TableID: tbl.TableID,
ConstraintID: constraintID,
ColumnIDs: colIDs,
IndexIDForValidation: getIndexIDForValidationForConstraint(b, tbl.TableID),
}
if d.Predicate != nil {
uwi.Predicate = b.WrapExpression(tbl.TableID, d.Predicate)
}
b.Add(uwi)
b.LogEventForExistingTarget(uwi)
} else {
uwi := &scpb.UniqueWithoutIndexConstraintUnvalidated{
TableID: tbl.TableID,
ConstraintID: constraintID,
ColumnIDs: colIDs,
}
if d.Predicate != nil {
uwi.Predicate = b.WrapExpression(tbl.TableID, d.Predicate)
}
b.Add(uwi)
b.LogEventForExistingTarget(uwi)
}
b.Add(uwi)
b.LogEventForExistingTarget(uwi)
b.Add(&scpb.ConstraintWithoutIndexName{
TableID: tbl.TableID,
ConstraintID: constraintID,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,11 @@
setup
CREATE TABLE t (i INT PRIMARY KEY, j INT);
----

build
ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j);
----
- [[UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2}, PUBLIC], ABSENT]
{columnIds: [2], constraintId: 2, tableId: 104}
- [[ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2}, PUBLIC], ABSENT]
{constraintId: 2, name: unique_j, tableId: 104}
Original file line number Diff line number Diff line change
@@ -0,0 +1,11 @@
setup
CREATE TABLE t (i INT PRIMARY KEY, j INT);
----

build
ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j) NOT VALID;
----
- [[UniqueWithoutIndexConstraintUnvalidated:{DescID: 104, ConstraintID: 2}, PUBLIC], ABSENT]
{columnIds: [2], constraintId: 2, tableId: 104}
- [[ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2}, PUBLIC], ABSENT]
{constraintId: 2, name: unique_j, tableId: 104}
28 changes: 20 additions & 8 deletions pkg/sql/schemachanger/scdecomp/decomp.go
Original file line number Diff line number Diff line change
Expand Up @@ -601,20 +601,32 @@ func (w *walkCtx) walkIndex(tbl catalog.TableDescriptor, idx catalog.Index) {
func (w *walkCtx) walkUniqueWithoutIndexConstraint(
tbl catalog.TableDescriptor, c catalog.UniqueWithoutIndexConstraint,
) {
uwi := &scpb.UniqueWithoutIndexConstraint{
TableID: tbl.GetID(),
ConstraintID: c.GetConstraintID(),
ColumnIDs: c.CollectKeyColumnIDs().Ordered(),
}
var expr *scpb.Expression
var err error
if c.IsPartial() {
expr, err := w.newExpression(c.GetPredicate())
expr, err = w.newExpression(c.GetPredicate())
if err != nil {
panic(errors.NewAssertionErrorWithWrappedErrf(err, "unique without index constraint %q in table %q (%d)",
c.GetName(), tbl.GetName(), tbl.GetID()))
}
uwi.Predicate = expr
}
w.ev(scpb.Status_PUBLIC, uwi)
if c.IsConstraintUnvalidated() && w.clusterVersion.IsActive(clusterversion.V23_1) {
uwi := &scpb.UniqueWithoutIndexConstraintUnvalidated{
TableID: tbl.GetID(),
ConstraintID: c.GetConstraintID(),
ColumnIDs: c.CollectKeyColumnIDs().Ordered(),
Predicate: expr,
}
w.ev(scpb.Status_PUBLIC, uwi)
} else {
uwi := &scpb.UniqueWithoutIndexConstraint{
TableID: tbl.GetID(),
ConstraintID: c.GetConstraintID(),
ColumnIDs: c.CollectKeyColumnIDs().Ordered(),
Predicate: expr,
}
w.ev(scpb.Status_PUBLIC, uwi)
}
w.ev(scpb.Status_PUBLIC, &scpb.ConstraintWithoutIndexName{
TableID: tbl.GetID(),
ConstraintID: c.GetConstraintID(),
Expand Down
28 changes: 28 additions & 0 deletions pkg/sql/schemachanger/scdecomp/testdata/table
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
setup
SET experimental_enable_unique_without_index_constraints = true;
CREATE TABLE parent (id INT PRIMARY KEY);
CREATE TABLE tbl (
id INT PRIMARY KEY,
Expand All @@ -15,6 +16,8 @@ COMMENT ON CONSTRAINT myfk ON tbl IS 'must have a parent';
ALTER TABLE tbl CONFIGURE ZONE USING gc.ttlseconds=10;
COMMENT ON CONSTRAINT tbl_pkey ON tbl IS 'primary key';
ALTER TABLE tbl ADD CONSTRAINT mycheck2 CHECK (id < 10) NOT VALID;
ALTER TABLE tbl ADD CONSTRAINT myuwi1 UNIQUE WITHOUT INDEX (price);
ALTER TABLE tbl ADD CONSTRAINT myuwi2 UNIQUE WITHOUT INDEX (price) NOT VALID;
----

decompose
Expand Down Expand Up @@ -319,6 +322,21 @@ ElementState:
tableId: 105
temporaryIndexId: 0
Status: PUBLIC
- UniqueWithoutIndexConstraint:
columnIds:
- 3
constraintId: 5
indexIdForValidation: 0
predicate: null
tableId: 105
Status: PUBLIC
- UniqueWithoutIndexConstraintUnvalidated:
columnIds:
- 3
constraintId: 6
predicate: null
tableId: 105
Status: PUBLIC
- CheckConstraint:
columnIds:
- 1
Expand Down Expand Up @@ -650,6 +668,16 @@ ElementState:
name: mycheck2
tableId: 105
Status: PUBLIC
- ConstraintWithoutIndexName:
constraintId: 5
name: myuwi1
tableId: 105
Status: PUBLIC
- ConstraintWithoutIndexName:
constraintId: 6
name: myuwi2
tableId: 105
Status: PUBLIC
- ConstraintComment:
comment: must have a parent
constraintId: 2
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scdeps/sctestutils/sctestutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@ func WithBuilderDependenciesFromTestServer(
// For setting up a builder inside tests we will ensure that the new schema
// changer will allow non-fully implemented operations.
planner.SessionData().NewSchemaChangerMode = sessiondatapb.UseNewSchemaChangerUnsafe
planner.SessionData().EnableUniqueWithoutIndexConstraints = true
fn(scdeps.NewBuilderDependencies(
execCfg.NodeInfo.LogicalClusterID(),
execCfg.Codec,
Expand Down
14 changes: 9 additions & 5 deletions pkg/sql/schemachanger/scexec/scmutationexec/constraint.go
Original file line number Diff line number Diff line change
Expand Up @@ -493,8 +493,8 @@ func (i *immediateVisitor) MakePublicForeignKeyConstraintValidated(
return errors.AssertionFailedf("failed to find FK constraint %d in descriptor %v", op.ConstraintID, out)
}

func (i *immediateVisitor) MakeAbsentUniqueWithoutIndexConstraintWriteOnly(
ctx context.Context, op scop.MakeAbsentUniqueWithoutIndexConstraintWriteOnly,
func (i *immediateVisitor) AddUniqueWithoutIndexConstraint(
ctx context.Context, op scop.AddUniqueWithoutIndexConstraint,
) error {
tbl, err := i.checkOutTable(ctx, op.TableID)
if err != nil || tbl.Dropped() {
Expand All @@ -508,13 +508,17 @@ func (i *immediateVisitor) MakeAbsentUniqueWithoutIndexConstraintWriteOnly(
TableID: op.TableID,
ColumnIDs: op.ColumnIDs,
Name: tabledesc.ConstraintNamePlaceholder(op.ConstraintID),
Validity: descpb.ConstraintValidity_Validating,
Validity: op.Validity,
ConstraintID: op.ConstraintID,
Predicate: string(op.PartialExpr),
}
if op.Validity == descpb.ConstraintValidity_Unvalidated {
// Unvalidated constraint doesn't need to transition through an intermediate
// state, so we don't enqueue a mutation for it.
tbl.UniqueWithoutIndexConstraints = append(tbl.UniqueWithoutIndexConstraints, *uwi)
return nil
}
enqueueNonIndexMutation(tbl, tbl.AddUniqueWithoutIndexMutation, uwi, descpb.DescriptorMutation_ADD)
// Fast-forward the mutation state to WRITE_ONLY because this constraint
// is now considered as enforced.
tbl.Mutations[len(tbl.Mutations)-1].State = descpb.DescriptorMutation_WRITE_ONLY
return nil
}
Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/schemachanger/scop/immediate_mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,14 +365,15 @@ type RemoveForeignKeyBackReference struct {
OriginConstraintID descpb.ConstraintID
}

// MakeAbsentUniqueWithoutIndexConstraintWriteOnly adds a non-existent
// unique_without_index constraint to the table in the WRITE_ONLY state.
type MakeAbsentUniqueWithoutIndexConstraintWriteOnly struct {
// AddUniqueWithoutIndexConstraint adds a non-existent
// unique_without_index constraint to the table.
type AddUniqueWithoutIndexConstraint struct {
immediateMutationOp
TableID descpb.ID
ConstraintID descpb.ConstraintID
ColumnIDs []descpb.ColumnID
PartialExpr catpb.Expression
Validity descpb.ConstraintValidity
}

// MakeValidatedUniqueWithoutIndexConstraintPublic moves a new, validated unique_without_index
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

9 changes: 9 additions & 0 deletions pkg/sql/schemachanger/scpb/elements.proto
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ message ElementProto {
SecondaryIndex secondary_index = 23 [(gogoproto.moretags) = "parent:\"Table, View\""];
TemporaryIndex temporary_index = 24 [(gogoproto.moretags) = "parent:\"Table, View\""];
UniqueWithoutIndexConstraint unique_without_index_constraint = 25 [(gogoproto.moretags) = "parent:\"Table\""];
UniqueWithoutIndexConstraintUnvalidated unique_without_index_constraint_unvalidated = 171 [(gogoproto.moretags) = "parent:\"Table\""];
CheckConstraint check_constraint = 26 [(gogoproto.moretags) = "parent:\"Table\""];
CheckConstraintUnvalidated check_constraint_unvalidated = 170 [(gogoproto.moretags) = "parent:\"Table\""];
ForeignKeyConstraint foreign_key_constraint = 27 [(gogoproto.moretags) = "parent:\"Table\""];
Expand Down Expand Up @@ -356,6 +357,14 @@ message UniqueWithoutIndexConstraint {
uint32 index_id_for_validation = 5 [(gogoproto.customname) = "IndexIDForValidation", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.IndexID"];
}

message UniqueWithoutIndexConstraintUnvalidated {
uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
uint32 constraint_id = 2 [(gogoproto.customname) = "ConstraintID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ConstraintID"];
repeated uint32 column_ids = 3 [(gogoproto.customname) = "ColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"];
// Predicate, if non-nil, means a partial uniqueness constraint.
Expression predicate = 4 [(gogoproto.customname) = "Predicate"];
}

message CheckConstraint {
uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
uint32 constraint_id = 2 [(gogoproto.customname) = "ConstraintID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ConstraintID"];
Expand Down
31 changes: 31 additions & 0 deletions pkg/sql/schemachanger/scpb/elements_generated.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

8 changes: 8 additions & 0 deletions pkg/sql/schemachanger/scpb/uml/table.puml
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,13 @@ UniqueWithoutIndexConstraint : []ColumnIDs
UniqueWithoutIndexConstraint : Predicate
UniqueWithoutIndexConstraint : IndexIDForValidation

object UniqueWithoutIndexConstraintUnvalidated

UniqueWithoutIndexConstraintUnvalidated : TableID
UniqueWithoutIndexConstraintUnvalidated : ConstraintID
UniqueWithoutIndexConstraintUnvalidated : []ColumnIDs
UniqueWithoutIndexConstraintUnvalidated : Predicate

object CheckConstraint

CheckConstraint : TableID
Expand Down Expand Up @@ -375,6 +382,7 @@ View <|-- SecondaryIndex
Table <|-- TemporaryIndex
View <|-- TemporaryIndex
Table <|-- UniqueWithoutIndexConstraint
Table <|-- UniqueWithoutIndexConstraintUnvalidated
Table <|-- CheckConstraint
Table <|-- CheckConstraintUnvalidated
Table <|-- ForeignKeyConstraint
Expand Down
Loading

0 comments on commit e7d394b

Please sign in to comment.