diff --git a/docs/generated/http/BUILD.bazel b/docs/generated/http/BUILD.bazel index ad6183f1a2a0..8feeea12be71 100644 --- a/docs/generated/http/BUILD.bazel +++ b/docs/generated/http/BUILD.bazel @@ -25,6 +25,7 @@ genrule( "//pkg/sql/catalog/fetchpb:fetchpb_proto", "//pkg/sql/contentionpb:contentionpb_proto", "//pkg/sql/execinfrapb:execinfrapb_proto", + "//pkg/sql/sem/semenumpb:semenumpb_proto", "//pkg/sql/inverted:inverted_proto", "//pkg/sql/lex:lex_proto", "//pkg/sql/pgwire/pgerror:pgerror_proto", diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index b3f39ab8c972..4668bd99d87d 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1799,6 +1799,7 @@ GO_TARGETS = [ "//pkg/sql/sem/eval:eval_test", "//pkg/sql/sem/normalize:normalize", "//pkg/sql/sem/normalize:normalize_test", + "//pkg/sql/sem/semenumpb:semenumpb", "//pkg/sql/sem/transform:transform", "//pkg/sql/sem/tree/evalgen:evalgen", "//pkg/sql/sem/tree/evalgen:evalgen_lib", @@ -2931,6 +2932,7 @@ GET_X_DATA_TARGETS = [ "//pkg/sql/sem/eval/cast_test:get_x_data", "//pkg/sql/sem/eval/eval_test:get_x_data", "//pkg/sql/sem/normalize:get_x_data", + "//pkg/sql/sem/semenumpb:get_x_data", "//pkg/sql/sem/transform:get_x_data", "//pkg/sql/sem/tree:get_x_data", "//pkg/sql/sem/tree/evalgen:get_x_data", diff --git a/pkg/bench/rttanalysis/testdata/benchmark_expectations b/pkg/bench/rttanalysis/testdata/benchmark_expectations index 43805aa131a9..8ce14a971792 100644 --- a/pkg/bench/rttanalysis/testdata/benchmark_expectations +++ b/pkg/bench/rttanalysis/testdata/benchmark_expectations @@ -9,8 +9,8 @@ exp,benchmark 12,AlterTableAddColumn/alter_table_add_2_columns 12,AlterTableAddColumn/alter_table_add_3_columns 13,AlterTableAddForeignKey/alter_table_add_1_foreign_key -17,AlterTableAddForeignKey/alter_table_add_2_foreign_keys -21,AlterTableAddForeignKey/alter_table_add_3_foreign_keys +13,AlterTableAddForeignKey/alter_table_add_2_foreign_keys +13,AlterTableAddForeignKey/alter_table_add_3_foreign_keys 13,AlterTableAddForeignKey/alter_table_add_foreign_key_with_3_columns 8,AlterTableConfigureZone/alter_table_configure_zone_5_replicas 8,AlterTableConfigureZone/alter_table_configure_zone_7_replicas_ diff --git a/pkg/ccl/schemachangerccl/backup_base_generated_test.go b/pkg/ccl/schemachangerccl/backup_base_generated_test.go index 128b8172fd88..b66b7c6bf0a5 100644 --- a/pkg/ccl/schemachangerccl/backup_base_generated_test.go +++ b/pkg/ccl/schemachangerccl/backup_base_generated_test.go @@ -48,6 +48,11 @@ func TestBackupbase_alter_table_add_check_with_seq_and_udt(t *testing.T) { defer log.Scope(t).Close(t) sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt", newCluster) } +func TestBackupbase_alter_table_add_foreign_key(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key", newCluster) +} func TestBackupbase_alter_table_add_primary_key_drop_rowid(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index ac2471328c57..1407962d7e56 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -62,6 +62,7 @@ PROTOBUF_SRCS = [ "//pkg/sql/protoreflect/test:protoreflecttest_go_proto", "//pkg/sql/rowenc/rowencpb:rowencpb_go_proto", "//pkg/sql/schemachanger/scpb:scpb_go_proto", + "//pkg/sql/sem/semenumpb:semenumpb_go_proto", "//pkg/sql/sessiondatapb:sessiondatapb_go_proto", "//pkg/sql/sqlstats/insights:insights_go_proto", "//pkg/sql/sqlstats/persistedsqlstats:persistedsqlstats_go_proto", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 98ef2f815b63..5ee5800809bf 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -1095,7 +1095,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { execCfg.ProtectedTimestampManager, sql.ValidateForwardIndexes, sql.ValidateInvertedIndexes, - sql.ValidateCheckConstraint, + sql.ValidateConstraint, sql.NewFakeSessionData, ) execCfg.InternalExecutorFactory = ieFactory diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 229d7606dc69..50ff5d78e4ab 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -434,6 +434,7 @@ go_library( "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", + "//pkg/sql/sem/semenumpb", "//pkg/sql/sem/transform", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treebin", diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index d4d089b04773..dfa09c2d0583 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" @@ -923,8 +924,8 @@ func applyColumnMutation( for _, fk := range tableDesc.OutboundFKs { for _, colID := range fk.OriginColumnIDs { if colID == col.GetID() && - fk.OnUpdate != catpb.ForeignKeyAction_NO_ACTION && - fk.OnUpdate != catpb.ForeignKeyAction_RESTRICT { + fk.OnUpdate != semenumpb.ForeignKeyAction_NO_ACTION && + fk.OnUpdate != semenumpb.ForeignKeyAction_RESTRICT { return pgerror.Newf( pgcode.InvalidColumnDefinition, "column %s(%d) cannot have both an ON UPDATE expression and a foreign"+ diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 7ded154c3f63..02ae4c0f22f0 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -1502,18 +1503,19 @@ func (e InvalidIndexesError) Error() string { return fmt.Sprintf("found %d invalid indexes", len(e.Indexes)) } -// ValidateCheckConstraint validates the check constraint against all rows -// in index `indexIDForValidation` in table `tableDesc`. -func ValidateCheckConstraint( +// ValidateConstraint validates the constraint against all rows +// in `tbl`. +// +// TODO (xiang): Support validating UNIQUE_WITHOUT_INDEX constraint in this function. +func ValidateConstraint( ctx context.Context, tableDesc catalog.TableDescriptor, - checkConstraint catalog.CheckConstraint, + constraint catalog.Constraint, indexIDForValidation descpb.IndexID, sessionData *sessiondata.SessionData, runHistoricalTxn descs.HistoricalInternalExecTxnRunner, execOverride sessiondata.InternalExecutorOverride, ) (err error) { - tableDesc, err = tableDesc.MakeFirstMutationPublic(catalog.IgnoreConstraints) if err != nil { return err @@ -1530,10 +1532,35 @@ func ValidateCheckConstraint( semaCtx.TableNameResolver = resolver defer func() { descriptors.ReleaseAll(ctx) }() - return ie.WithSyntheticDescriptors([]catalog.Descriptor{tableDesc}, func() error { - return validateCheckExpr(ctx, &semaCtx, txn, sessionData, checkConstraint.GetExpr(), - tableDesc.(*tabledesc.Mutable), ie, indexIDForValidation) - }) + switch catalog.GetConstraintType(constraint) { + case catconstants.ConstraintTypeCheck: + ck := constraint.AsCheck() + return ie.WithSyntheticDescriptors( + []catalog.Descriptor{tableDesc}, + func() error { + return validateCheckExpr(ctx, &semaCtx, txn, sessionData, ck.GetExpr(), + tableDesc.(*tabledesc.Mutable), ie, indexIDForValidation) + }, + ) + case catconstants.ConstraintTypeFK: + fk := constraint.AsForeignKey() + targetTable, err := descriptors.ByID(txn).Get().Table(ctx, fk.GetReferencedTableID()) + if err != nil { + return err + } + if targetTable.GetID() == tableDesc.GetID() { + targetTable = tableDesc + } + return ie.WithSyntheticDescriptors( + []catalog.Descriptor{tableDesc}, + func() error { + return validateForeignKey(ctx, tableDesc.(*tabledesc.Mutable), targetTable, fk.ForeignKeyDesc(), + indexIDForValidation, txn, ie) + }, + ) + default: + return errors.AssertionFailedf("validation of unsupported constraint type") + } }) } @@ -2661,7 +2688,7 @@ func validateFkInTxn( return ie.WithSyntheticDescriptors( syntheticDescs, func() error { - return validateForeignKey(ctx, srcTable, targetTable, fk, ie, txn) + return validateForeignKey(ctx, srcTable, targetTable, fk, 0 /* indexIDForValidation */, txn, ie) }) } diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index e6127c0f6172..257de924276d 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -36,6 +36,8 @@ go_library( "//pkg/sql/pgwire/pgerror", "//pkg/sql/privilege", "//pkg/sql/schemachanger/scpb", + "//pkg/sql/sem/catconstants", + "//pkg/sql/sem/semenumpb", "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util", diff --git a/pkg/sql/catalog/catpb/BUILD.bazel b/pkg/sql/catalog/catpb/BUILD.bazel index e844bc37a1a0..dcd74f81da42 100644 --- a/pkg/sql/catalog/catpb/BUILD.bazel +++ b/pkg/sql/catalog/catpb/BUILD.bazel @@ -9,6 +9,7 @@ proto_library( name = "catpb_proto", srcs = [ "catalog.proto", + "enum.proto", "function.proto", "privilege.proto", ], @@ -33,7 +34,6 @@ go_library( name = "catpb", srcs = [ "catalog.go", - "constraint.go", "default_privilege.go", "doc.go", "expression.go", @@ -53,7 +53,6 @@ go_library( "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", "@com_github_cockroachdb_errors//:errors", - "@com_github_cockroachdb_redact//:redact", ], ) diff --git a/pkg/sql/catalog/catpb/catalog.proto b/pkg/sql/catalog/catpb/catalog.proto index fd022f919d06..c052a7085ce7 100644 --- a/pkg/sql/catalog/catpb/catalog.proto +++ b/pkg/sql/catalog/catpb/catalog.proto @@ -18,17 +18,6 @@ option go_package = "catpb"; import "gogoproto/gogo.proto"; -// ForeignKeyAction describes the action which should be taken when a foreign -// key constraint reference is acted upon. -enum ForeignKeyAction { - option (gogoproto.goproto_enum_stringer) = false; - NO_ACTION = 0; - RESTRICT = 1; - SET_NULL = 2; - SET_DEFAULT = 3; - CASCADE = 4; -} - // LocalityConfig is used to figure the locality of a table. message LocalityConfig { option (gogoproto.equal) = true; @@ -66,18 +55,6 @@ message LocalityConfig { } } -// SystemColumnKind is an enum representing the different kind of system -// columns that can be synthesized by the execution engine. -enum SystemColumnKind { - // Default value, unused. - NONE = 0; - // A system column containing the value of the MVCC timestamp associated - // with the kv's corresponding to the row. - MVCCTIMESTAMP = 1; - // A system column containing the OID of the table that the row came from. - TABLEOID = 2; -} - // GeneratedAsIdentityType is an enum representing how the creation of // a column is associated with the GENERATED {ALWAYS | BY DEFAULT} AS IDENTITY // syntax. @@ -234,23 +211,3 @@ message AutoStatsSettings { // FractionStaleRows is table setting sql_stats_automatic_collection_fraction_stale_rows. optional double fraction_stale_rows = 3; } - -// InvertedIndexColumnKind is the kind of the inverted index on a column. The -// reason this needs to be stored is that we need to be able to check that the -// "opclass" passed into an inverted index declaration (for example, -// gin_trgm_ops) is compatible with the datatype of a particular column -// (gin_tgrm_ops is only valid on text). A future reason is that it's possible -// to desire having more than one type of inverted index on a particular -// datatype - for example, you might want to create a "stemming" inverted index -// on text. And without this extra kind, it wouldn't be possible to distinguish -// a text inverted index that uses trigrams, vs a text inverted index that uses -// stemming. -enum InvertedIndexColumnKind { - // DEFAULT is the default kind of inverted index column. JSON, Array, and - // geo inverted indexes all are DEFAULT, though prior to 22.2 they had no - // kind at all. - DEFAULT = 0; - // TRIGRAM is the trigram kind of inverted index column. It's only valid on - // text columns. - TRIGRAM = 1; -} diff --git a/pkg/sql/catalog/catpb/enum.proto b/pkg/sql/catalog/catpb/enum.proto new file mode 100644 index 000000000000..5fbefd01d423 --- /dev/null +++ b/pkg/sql/catalog/catpb/enum.proto @@ -0,0 +1,51 @@ +// Copyright 2022 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. + +// This file should contain only EMUN definitions for concepts that +// are internal and not visible to the SQL layer. +// It uses proto3 so other packages can import those enum definitions +// when needed. +syntax = "proto3"; +package cockroach.sql.catalog.catpb; +option go_package = "catpb"; + +import "gogoproto/gogo.proto"; + +// SystemColumnKind is an enum representing the different kind of system +// columns that can be synthesized by the execution engine. +enum SystemColumnKind { + // Default value, unused. + NONE = 0; + // A system column containing the value of the MVCC timestamp associated + // with the kv's corresponding to the row. + MVCCTIMESTAMP = 1; + // A system column containing the OID of the table that the row came from. + TABLEOID = 2; +} + +// InvertedIndexColumnKind is the kind of the inverted index on a column. The +// reason this needs to be stored is that we need to be able to check that the +// "opclass" passed into an inverted index declaration (for example, +// gin_trgm_ops) is compatible with the datatype of a particular column +// (gin_tgrm_ops is only valid on text). A future reason is that it's possible +// to desire having more than one type of inverted index on a particular +// datatype - for example, you might want to create a "stemming" inverted index +// on text. And without this extra kind, it wouldn't be possible to distinguish +// a text inverted index that uses trigrams, vs a text inverted index that uses +// stemming. +enum InvertedIndexColumnKind { + // DEFAULT is the default kind of inverted index column. JSON, Array, and + // geo inverted indexes all are DEFAULT, though prior to 22.2 they had no + // kind at all. + DEFAULT = 0; + // TRIGRAM is the trigram kind of inverted index column. It's only valid on + // text columns. + TRIGRAM = 1; +} diff --git a/pkg/sql/catalog/descpb/BUILD.bazel b/pkg/sql/catalog/descpb/BUILD.bazel index 4514ae11e43a..46735b7fac31 100644 --- a/pkg/sql/catalog/descpb/BUILD.bazel +++ b/pkg/sql/catalog/descpb/BUILD.bazel @@ -9,7 +9,6 @@ go_library( name = "descpb", srcs = [ "column.go", - "constraint.go", "descriptor.go", "index.go", "join_type.go", @@ -23,7 +22,6 @@ go_library( deps = [ "//pkg/keys", "//pkg/sql/catalog/catenumpb", - "//pkg/sql/catalog/catpb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/protoreflect", @@ -64,6 +62,7 @@ proto_library( "//pkg/sql/catalog/catenumpb:catenumpb_proto", "//pkg/sql/catalog/catpb:catpb_proto", "//pkg/sql/schemachanger/scpb:scpb_proto", + "//pkg/sql/sem/semenumpb:semenumpb_proto", "//pkg/sql/types:types_proto", "//pkg/util/hlc:hlc_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", @@ -84,6 +83,7 @@ go_proto_library( "//pkg/sql/catalog/catenumpb", "//pkg/sql/catalog/catpb", "//pkg/sql/schemachanger/scpb", + "//pkg/sql/sem/semenumpb", "//pkg/sql/types", "//pkg/util/hlc", "@com_github_gogo_protobuf//gogoproto", diff --git a/pkg/sql/catalog/descpb/constraint.go b/pkg/sql/catalog/descpb/constraint.go deleted file mode 100644 index ff1b012e1a75..000000000000 --- a/pkg/sql/catalog/descpb/constraint.go +++ /dev/null @@ -1,83 +0,0 @@ -// Copyright 2020 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 descpb - -import ( - "strconv" - - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" -) - -// CompositeKeyMatchMethodValue allows the conversion from a -// tree.ReferenceCompositeKeyMatchMethod to a ForeignKeyReference_Match. -var CompositeKeyMatchMethodValue = [...]ForeignKeyReference_Match{ - tree.MatchSimple: ForeignKeyReference_SIMPLE, - tree.MatchFull: ForeignKeyReference_FULL, - tree.MatchPartial: ForeignKeyReference_PARTIAL, -} - -// ForeignKeyReferenceMatchValue allows the conversion from a -// ForeignKeyReference_Match to a tree.ReferenceCompositeKeyMatchMethod. -// This should match CompositeKeyMatchMethodValue. -var ForeignKeyReferenceMatchValue = [...]tree.CompositeKeyMatchMethod{ - ForeignKeyReference_SIMPLE: tree.MatchSimple, - ForeignKeyReference_FULL: tree.MatchFull, - ForeignKeyReference_PARTIAL: tree.MatchPartial, -} - -// String implements the fmt.Stringer interface. -func (x ForeignKeyReference_Match) String() string { - switch x { - case ForeignKeyReference_SIMPLE: - return "MATCH SIMPLE" - case ForeignKeyReference_FULL: - return "MATCH FULL" - case ForeignKeyReference_PARTIAL: - return "MATCH PARTIAL" - default: - return strconv.Itoa(int(x)) - } -} - -// ForeignKeyReferenceActionType allows the conversion between a -// tree.ReferenceAction and a ForeignKeyReference_Action. -var ForeignKeyReferenceActionType = [...]tree.ReferenceAction{ - catpb.ForeignKeyAction_NO_ACTION: tree.NoAction, - catpb.ForeignKeyAction_RESTRICT: tree.Restrict, - catpb.ForeignKeyAction_SET_DEFAULT: tree.SetDefault, - catpb.ForeignKeyAction_SET_NULL: tree.SetNull, - catpb.ForeignKeyAction_CASCADE: tree.Cascade, -} - -// ForeignKeyReferenceActionValue allows the conversion between a -// catpb.ForeignKeyAction_Action and a tree.ReferenceAction. -var ForeignKeyReferenceActionValue = [...]catpb.ForeignKeyAction{ - tree.NoAction: catpb.ForeignKeyAction_NO_ACTION, - tree.Restrict: catpb.ForeignKeyAction_RESTRICT, - tree.SetDefault: catpb.ForeignKeyAction_SET_DEFAULT, - tree.SetNull: catpb.ForeignKeyAction_SET_NULL, - tree.Cascade: catpb.ForeignKeyAction_CASCADE, -} - -// ConstraintType is used to identify the type of a constraint. -type ConstraintType string - -const ( - // ConstraintTypePK identifies a PRIMARY KEY constraint. - ConstraintTypePK ConstraintType = "PRIMARY KEY" - // ConstraintTypeFK identifies a FOREIGN KEY constraint. - ConstraintTypeFK ConstraintType = "FOREIGN KEY" - // ConstraintTypeUnique identifies a UNIQUE constraint. - ConstraintTypeUnique ConstraintType = "UNIQUE" - // ConstraintTypeCheck identifies a CHECK constraint. - ConstraintTypeCheck ConstraintType = "CHECK" -) diff --git a/pkg/sql/catalog/descpb/structured.go b/pkg/sql/catalog/descpb/structured.go index d3b7c1299f06..500bb7023fec 100644 --- a/pkg/sql/catalog/descpb/structured.go +++ b/pkg/sql/catalog/descpb/structured.go @@ -318,9 +318,6 @@ func (DescriptorMutation_State) SafeValue() {} // SafeValue implements the redact.SafeValue interface. func (DescriptorState) SafeValue() {} -// SafeValue implements the redact.SafeValue interface. -func (ConstraintType) SafeValue() {} - // IsPartial returns true if the constraint is a partial unique constraint. func (u *UniqueWithoutIndexConstraint) IsPartial() bool { return u.Predicate != "" diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index 2bf6ad345a6e..6f26469a1a60 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -17,6 +17,8 @@ import "config/zonepb/zone.proto"; import "util/hlc/timestamp.proto"; import "sql/catalog/catenumpb/index.proto"; import "sql/catalog/catpb/catalog.proto"; +import "sql/catalog/catpb/enum.proto"; +import "sql/sem/semenumpb/constraint.proto"; import "sql/catalog/catpb/privilege.proto"; import "sql/catalog/catpb/function.proto"; import "sql/schemachanger/scpb/scpb.proto"; @@ -47,15 +49,6 @@ enum ConstraintValidity { // traditional foreign key references. message ForeignKeyReference { option (gogoproto.equal) = true; - - // Match is the algorithm used to compare composite keys. - enum Match { - option (gogoproto.goproto_enum_stringer) = false; - SIMPLE = 0; - FULL = 1; - PARTIAL = 2; // Note: not actually supported, but we reserve the value for future use. - } - optional uint32 table = 1 [(gogoproto.nullable) = false, (gogoproto.casttype) = "ID"]; optional uint32 index = 2 [(gogoproto.nullable) = false, (gogoproto.casttype) = "IndexID"]; optional string name = 3 [(gogoproto.nullable) = false]; @@ -63,11 +56,11 @@ message ForeignKeyReference { // If this FK only uses a prefix of the columns in its index, we record how // many to avoid spuriously counting the additional cols as used by this FK. optional int32 shared_prefix_len = 5 [(gogoproto.nullable) = false]; - optional cockroach.sql.catalog.catpb.ForeignKeyAction on_delete = 6 [(gogoproto.nullable) = false]; - optional cockroach.sql.catalog.catpb.ForeignKeyAction on_update = 7 [(gogoproto.nullable) = false]; + optional cockroach.sql.sem.semenumpb.ForeignKeyAction on_delete = 6 [(gogoproto.nullable) = false]; + optional cockroach.sql.sem.semenumpb.ForeignKeyAction on_update = 7 [(gogoproto.nullable) = false]; // This is only important for composite keys. For all prior matches before // the addition of this value, MATCH SIMPLE will be used. - optional Match match = 8 [(gogoproto.nullable) = false]; + optional cockroach.sql.sem.semenumpb.Match match = 8 [(gogoproto.nullable) = false]; } // ForeignKeyConstraint is the new (as of 19.2 and VersionTopLevelForeignKeys) @@ -89,11 +82,11 @@ message ForeignKeyConstraint { (gogoproto.casttype) = "ID"]; optional string name = 5 [(gogoproto.nullable) = false]; optional ConstraintValidity validity = 6 [(gogoproto.nullable) = false]; - optional cockroach.sql.catalog.catpb.ForeignKeyAction on_delete = 7 [(gogoproto.nullable) = false]; - optional cockroach.sql.catalog.catpb.ForeignKeyAction on_update = 8 [(gogoproto.nullable) = false]; + optional cockroach.sql.sem.semenumpb.ForeignKeyAction on_delete = 7 [(gogoproto.nullable) = false]; + optional cockroach.sql.sem.semenumpb.ForeignKeyAction on_update = 8 [(gogoproto.nullable) = false]; // This is only important for composite keys. For all prior matches before // the addition of this value, MATCH SIMPLE will be used. - optional ForeignKeyReference.Match match = 9 [(gogoproto.nullable) = false]; + optional cockroach.sql.sem.semenumpb.Match match = 9 [(gogoproto.nullable) = false]; // These fields were used for foreign keys until 20.1. reserved 10, 11, 12, 13; diff --git a/pkg/sql/catalog/rewrite/rewrite.go b/pkg/sql/catalog/rewrite/rewrite.go index 1d3221f973a6..a495b193841c 100644 --- a/pkg/sql/catalog/rewrite/rewrite.go +++ b/pkg/sql/catalog/rewrite/rewrite.go @@ -99,7 +99,7 @@ func TableDescs( table.ViewQuery = viewQuery } - // TODO(lucy): deal with outbound foreign key mutations here as well. + // Rewrite outbound FKs in both `OutboundFKs` and `Mutations` slice. origFKs := table.OutboundFKs table.OutboundFKs = nil for i := range origFKs { @@ -120,6 +120,16 @@ func TableDescs( // we update the FK to point to it? table.OutboundFKs = append(table.OutboundFKs, *fk) } + for idx := range table.Mutations { + if c := table.Mutations[idx].GetConstraint(); c != nil && + c.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY { + fk := &c.ForeignKey + if rewriteOfReferencedTable, ok := descriptorRewrites[fk.ReferencedTableID]; ok { + fk.ReferencedTableID = rewriteOfReferencedTable.ID + fk.OriginTableID = tableRewrite.ID + } + } + } origInboundFks := table.InboundFKs table.InboundFKs = nil diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 2279847684b8..8657622abb98 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -18,11 +18,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/errors" ) // TableElementMaybeMutation is an interface used as a subtype for the various @@ -565,13 +568,13 @@ type ForeignKeyConstraint interface { CollectReferencedColumnIDs() TableColSet // OnDelete returns the action to take ON DELETE. - OnDelete() catpb.ForeignKeyAction + OnDelete() semenumpb.ForeignKeyAction // OnUpdate returns the action to take ON UPDATE. - OnUpdate() catpb.ForeignKeyAction + OnUpdate() semenumpb.ForeignKeyAction // Match returns the type of algorithm used to match composite keys. - Match() descpb.ForeignKeyReference_Match + Match() semenumpb.Match } // UniqueWithoutIndexConstraint is an interface around a unique constraint @@ -997,3 +1000,22 @@ func ColumnNeedsBackfill(col Column) bool { } return col.HasDefault() || !col.IsNullable() || col.IsComputed() } + +// GetConstraintType finds the type of constraint. +func GetConstraintType(c Constraint) catconstants.ConstraintType { + if c.AsCheck() != nil { + return catconstants.ConstraintTypeCheck + } else if c.AsForeignKey() != nil { + return catconstants.ConstraintTypeFK + } else if c.AsUniqueWithoutIndex() != nil { + return catconstants.ConstraintTypeUniqueWithoutIndex + } else if c.AsUniqueWithIndex() != nil { + if c.AsUniqueWithIndex().GetEncodingType() == catenumpb.PrimaryIndexEncoding { + return catconstants.ConstraintTypePK + } else { + return catconstants.ConstraintTypeUnique + } + } else { + panic(errors.AssertionFailedf("unknown constraint type %T", c)) + } +} diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 5687dd2e2445..024c3dec0079 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -49,6 +49,7 @@ go_library( "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", + "//pkg/sql/sem/semenumpb", "//pkg/sql/sem/tree", "//pkg/sql/sem/volatility", "//pkg/sql/types", @@ -109,6 +110,7 @@ go_test( "//pkg/sql/privilege", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", + "//pkg/sql/sem/semenumpb", "//pkg/sql/types", "//pkg/testutils", "//pkg/testutils/serverutils", diff --git a/pkg/sql/catalog/tabledesc/constraint.go b/pkg/sql/catalog/tabledesc/constraint.go index ff14d97dd321..b7e4f0551af2 100644 --- a/pkg/sql/catalog/tabledesc/constraint.go +++ b/pkg/sql/catalog/tabledesc/constraint.go @@ -14,8 +14,8 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/util" ) @@ -278,17 +278,17 @@ func (c foreignKeyConstraint) CollectReferencedColumnIDs() catalog.TableColSet { } // OnDelete implements the catalog.ForeignKeyConstraint interface. -func (c foreignKeyConstraint) OnDelete() catpb.ForeignKeyAction { +func (c foreignKeyConstraint) OnDelete() semenumpb.ForeignKeyAction { return c.desc.OnDelete } // OnUpdate implements the catalog.ForeignKeyConstraint interface. -func (c foreignKeyConstraint) OnUpdate() catpb.ForeignKeyAction { +func (c foreignKeyConstraint) OnUpdate() semenumpb.ForeignKeyAction { return c.desc.OnUpdate } // Match implements the catalog.ForeignKeyConstraint interface. -func (c foreignKeyConstraint) Match() descpb.ForeignKeyReference_Match { +func (c foreignKeyConstraint) Match() semenumpb.Match { return c.desc.Match } diff --git a/pkg/sql/catalog/tabledesc/safe_format_test.go b/pkg/sql/catalog/tabledesc/safe_format_test.go index 4d1249efef4d..ec861e3dc5e7 100644 --- a/pkg/sql/catalog/tabledesc/safe_format_test.go +++ b/pkg/sql/catalog/tabledesc/safe_format_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/validate" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -130,8 +131,8 @@ func TestSafeMessage(t *testing.T) { ReferencedColumnIDs: []descpb.ColumnID{2}, ReferencedTableID: 112, Validity: descpb.ConstraintValidity_Validated, - OnDelete: catpb.ForeignKeyAction_CASCADE, - Match: descpb.ForeignKeyReference_PARTIAL, + OnDelete: semenumpb.ForeignKeyAction_CASCADE, + Match: semenumpb.Match_PARTIAL, ConstraintID: 3, }) mutable.OutboundFKs = append(mutable.OutboundFKs, descpb.ForeignKeyConstraint{ @@ -141,8 +142,8 @@ func TestSafeMessage(t *testing.T) { ReferencedColumnIDs: []descpb.ColumnID{1}, ReferencedTableID: 3, Validity: descpb.ConstraintValidity_Validated, - OnDelete: catpb.ForeignKeyAction_SET_DEFAULT, - Match: descpb.ForeignKeyReference_SIMPLE, + OnDelete: semenumpb.ForeignKeyAction_SET_DEFAULT, + Match: semenumpb.Match_SIMPLE, ConstraintID: 4, }) @@ -158,8 +159,8 @@ func TestSafeMessage(t *testing.T) { OriginColumnIDs: []descpb.ColumnID{2}, ReferencedTableID: 2, ReferencedColumnIDs: []descpb.ColumnID{3}, - Validity: descpb.ConstraintValidity_Unvalidated, OnDelete: catpb.ForeignKeyAction_SET_NULL, - Match: descpb.ForeignKeyReference_FULL, + Validity: descpb.ConstraintValidity_Unvalidated, OnDelete: semenumpb.ForeignKeyAction_SET_NULL, + Match: semenumpb.Match_FULL, ConstraintID: 5, }, }, diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 993d68e5268c..1c47b7cf8ee6 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -873,8 +874,8 @@ func ValidateOnUpdate(desc catalog.TableDescriptor, errReportFn func(err error)) } for _, fk := range desc.OutboundForeignKeys() { - if fk.OnUpdate() == catpb.ForeignKeyAction_NO_ACTION || - fk.OnUpdate() == catpb.ForeignKeyAction_RESTRICT { + if fk.OnUpdate() == semenumpb.ForeignKeyAction_NO_ACTION || + fk.OnUpdate() == semenumpb.ForeignKeyAction_RESTRICT { continue } for i, n := 0, fk.NumOriginColumns(); i < n; i++ { diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 1ef3f5ea8835..9b285771eed2 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" @@ -170,12 +171,18 @@ func matchFullUnacceptableKeyQuery( // WHERE t.a IS NULL // LIMIT 1 -- if limitResults is set // +// It is possible to force FK validation query to perform against a particular +// index, as specified in `indexIDForValidation` when it's non zero. This is necessary +// if we are validating a FK constraint on a primary index that's being added (e.g. +// `ADD COLUMN ... REFERENCES other_table(...)`). +// // TODO(radu): change this to a query which executes as an anti-join when we // remove the heuristic planner. func nonMatchingRowQuery( srcTbl catalog.TableDescriptor, fk *descpb.ForeignKeyConstraint, targetTbl catalog.TableDescriptor, + indexIDForValidation descpb.IndexID, limitResults bool, ) (sql string, originColNames []string, _ error) { originColNames, err := srcTbl.NamesForColumnIDs(fk.OriginColumnIDs) @@ -228,7 +235,7 @@ func nonMatchingRowQuery( if limitResults { limit = " LIMIT 1" } - return fmt.Sprintf( + query := fmt.Sprintf( `SELECT %[1]s FROM (SELECT %[2]s FROM [%[3]d AS src]@{IGNORE_FOREIGN_KEYS} WHERE %[4]s) AS s LEFT OUTER JOIN @@ -244,7 +251,28 @@ func nonMatchingRowQuery( // Sufficient to check the first column to see whether there was no matching row targetCols[0], // 7 limit, // 8 - ), originColNames, nil + ) + if indexIDForValidation != 0 { + query = fmt.Sprintf( + `SELECT %[1]s FROM + (SELECT %[2]s FROM [%[3]d AS src]@{IGNORE_FOREIGN_KEYS, FORCE_INDEX=[%[4]d]} WHERE %[5]s) AS s + LEFT OUTER JOIN + [%[6]d AS target] AS t + ON %[7]s + WHERE %[8]s IS NULL %[9]s`, + strings.Join(qualifiedSrcCols, ", "), // 1 + strings.Join(srcCols, ", "), // 2 + srcTbl.GetID(), // 3 + indexIDForValidation, // 4 + strings.Join(srcWhere, " AND "), // 5 + targetTbl.GetID(), // 6 + strings.Join(on, " AND "), // 7 + // Sufficient to check the first column to see whether there was no matching row + targetCols[0], // 8 + limit, // 9 + ) + } + return query, originColNames, nil } // validateForeignKey verifies that all the rows in the srcTable @@ -257,8 +285,9 @@ func validateForeignKey( srcTable *tabledesc.Mutable, targetTable catalog.TableDescriptor, fk *descpb.ForeignKeyConstraint, - ie sqlutil.InternalExecutor, + indexIDForValidation descpb.IndexID, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { nCols := len(fk.OriginColumnIDs) @@ -270,7 +299,7 @@ func validateForeignKey( // For MATCH FULL FKs, first check whether any disallowed keys containing both // null and non-null values exist. // (The matching options only matter for FKs with more than one column.) - if nCols > 1 && fk.Match == descpb.ForeignKeyReference_FULL { + if nCols > 1 && fk.Match == semenumpb.Match_FULL { query, colNames, err := matchFullUnacceptableKeyQuery( srcTable, fk, true, /* limitResults */ ) @@ -298,10 +327,7 @@ func validateForeignKey( ), fk.Name) } } - query, colNames, err := nonMatchingRowQuery( - srcTable, fk, targetTable, - true, /* limitResults */ - ) + query, colNames, err := nonMatchingRowQuery(srcTable, fk, targetTable, indexIDForValidation, true /* limitResults */) if err != nil { return err } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 94121b9c5c91..0398fcb2fa85 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1028,9 +1028,9 @@ func ResolveFK( ReferencedTableID: target.ID, Name: constraintName, Validity: validity, - OnDelete: descpb.ForeignKeyReferenceActionValue[d.Actions.Delete], - OnUpdate: descpb.ForeignKeyReferenceActionValue[d.Actions.Update], - Match: descpb.CompositeKeyMatchMethodValue[d.Match], + OnDelete: tree.ForeignKeyReferenceActionValue[d.Actions.Delete], + OnUpdate: tree.ForeignKeyReferenceActionValue[d.Actions.Update], + Match: tree.CompositeKeyMatchMethodValue[d.Match], ConstraintID: tbl.NextConstraintID, } tbl.NextConstraintID++ diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 37ad14947143..a09537090150 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" @@ -35,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -869,10 +869,10 @@ var ( matchOptionPartial = tree.NewDString("PARTIAL") matchOptionNone = tree.NewDString("NONE") - matchOptionMap = map[descpb.ForeignKeyReference_Match]tree.Datum{ - descpb.ForeignKeyReference_SIMPLE: matchOptionNone, - descpb.ForeignKeyReference_FULL: matchOptionFull, - descpb.ForeignKeyReference_PARTIAL: matchOptionPartial, + matchOptionMap = map[semenumpb.Match]tree.Datum{ + semenumpb.Match_SIMPLE: matchOptionNone, + semenumpb.Match_FULL: matchOptionFull, + semenumpb.Match_PARTIAL: matchOptionPartial, } refConstraintRuleNoAction = tree.NewDString("NO ACTION") @@ -882,17 +882,17 @@ var ( refConstraintRuleCascade = tree.NewDString("CASCADE") ) -func dStringForFKAction(action catpb.ForeignKeyAction) tree.Datum { +func dStringForFKAction(action semenumpb.ForeignKeyAction) tree.Datum { switch action { - case catpb.ForeignKeyAction_NO_ACTION: + case semenumpb.ForeignKeyAction_NO_ACTION: return refConstraintRuleNoAction - case catpb.ForeignKeyAction_RESTRICT: + case semenumpb.ForeignKeyAction_RESTRICT: return refConstraintRuleRestrict - case catpb.ForeignKeyAction_SET_NULL: + case semenumpb.ForeignKeyAction_SET_NULL: return refConstraintRuleSetNull - case catpb.ForeignKeyAction_SET_DEFAULT: + case semenumpb.ForeignKeyAction_SET_DEFAULT: return refConstraintRuleSetDefault - case catpb.ForeignKeyAction_CASCADE: + case semenumpb.ForeignKeyAction_CASCADE: return refConstraintRuleCascade } panic(errors.Errorf("unexpected ForeignKeyReference_Action: %v", action)) @@ -1282,13 +1282,13 @@ https://www.postgresql.org/docs/9.5/infoschema-table-constraints.html`, tbNameStr := tree.NewDString(table.GetName()) for _, c := range table.AllConstraints() { - kind := descpb.ConstraintTypeUnique + kind := catconstants.ConstraintTypeUnique if c.AsCheck() != nil { - kind = descpb.ConstraintTypeCheck + kind = catconstants.ConstraintTypeCheck } else if c.AsForeignKey() != nil { - kind = descpb.ConstraintTypeFK + kind = catconstants.ConstraintTypeFK } else if u := c.AsUniqueWithIndex(); u != nil && u.Primary() { - kind = descpb.ConstraintTypePK + kind = catconstants.ConstraintTypePK } if err := addRow( dbNameStr, // constraint_catalog diff --git a/pkg/sql/logictest/testdata/logic_test/fk b/pkg/sql/logictest/testdata/logic_test/fk index 0fa7277b8575..93dfa5dd1780 100644 --- a/pkg/sql/logictest/testdata/logic_test/fk +++ b/pkg/sql/logictest/testdata/logic_test/fk @@ -2715,7 +2715,7 @@ statement ok CREATE TABLE child_duplicate_cols (id INT, parent_id INT, PRIMARY KEY (id)) # The fk constraint is invalid because it has duplicate columns, so automatically adding the index fails -statement error foreign key contains duplicate column \"parent_id\" +statement error pgcode 42830 foreign key contains duplicate column ".*parent_id" ALTER TABLE child_duplicate_cols ADD CONSTRAINT fk FOREIGN KEY (parent_id, parent_id) references parent statement ok diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 621b91a0da12..deaef5fb034a 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -974,9 +974,9 @@ func newOptTable( referencedTable: cat.StableID(fk.GetReferencedTableID()), referencedColumns: fk.ForeignKeyDesc().ReferencedColumnIDs, validity: fk.GetConstraintValidity(), - match: fk.Match(), - deleteAction: fk.OnDelete(), - updateAction: fk.OnUpdate(), + match: tree.CompositeKeyMatchMethodType[fk.Match()], + deleteAction: tree.ForeignKeyReferenceActionType[fk.OnDelete()], + updateAction: tree.ForeignKeyReferenceActionType[fk.OnUpdate()], }) } for _, fk := range ot.desc.InboundForeignKeys() { @@ -987,9 +987,9 @@ func newOptTable( referencedTable: ot.ID(), referencedColumns: fk.ForeignKeyDesc().ReferencedColumnIDs, validity: fk.GetConstraintValidity(), - match: fk.Match(), - deleteAction: fk.OnDelete(), - updateAction: fk.OnUpdate(), + match: tree.CompositeKeyMatchMethodType[fk.Match()], + deleteAction: tree.ForeignKeyReferenceActionType[fk.OnDelete()], + updateAction: tree.ForeignKeyReferenceActionType[fk.OnUpdate()], }) } @@ -1902,9 +1902,9 @@ type optForeignKeyConstraint struct { referencedColumns []descpb.ColumnID validity descpb.ConstraintValidity - match descpb.ForeignKeyReference_Match - deleteAction catpb.ForeignKeyAction - updateAction catpb.ForeignKeyAction + match tree.CompositeKeyMatchMethod + deleteAction tree.ReferenceAction + updateAction tree.ReferenceAction } var _ cat.ForeignKeyConstraint = &optForeignKeyConstraint{} @@ -1963,17 +1963,17 @@ func (fk *optForeignKeyConstraint) Validated() bool { // MatchMethod is part of the cat.ForeignKeyConstraint interface. func (fk *optForeignKeyConstraint) MatchMethod() tree.CompositeKeyMatchMethod { - return descpb.ForeignKeyReferenceMatchValue[fk.match] + return fk.match } // DeleteReferenceAction is part of the cat.ForeignKeyConstraint interface. func (fk *optForeignKeyConstraint) DeleteReferenceAction() tree.ReferenceAction { - return descpb.ForeignKeyReferenceActionType[fk.deleteAction] + return fk.deleteAction } // UpdateReferenceAction is part of the cat.ForeignKeyConstraint interface. func (fk *optForeignKeyConstraint) UpdateReferenceAction() tree.ReferenceAction { - return descpb.ForeignKeyReferenceActionType[fk.updateAction] + return fk.updateAction } // optVirtualTable is similar to optTable but is used with virtual tables. diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 24654ea3ced4..db05c9d2fefa 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" @@ -846,22 +847,22 @@ var ( fkActionSetNull = tree.NewDString("n") fkActionSetDefault = tree.NewDString("d") - fkActionMap = map[catpb.ForeignKeyAction]tree.Datum{ - catpb.ForeignKeyAction_NO_ACTION: fkActionNone, - catpb.ForeignKeyAction_RESTRICT: fkActionRestrict, - catpb.ForeignKeyAction_CASCADE: fkActionCascade, - catpb.ForeignKeyAction_SET_NULL: fkActionSetNull, - catpb.ForeignKeyAction_SET_DEFAULT: fkActionSetDefault, + fkActionMap = map[semenumpb.ForeignKeyAction]tree.Datum{ + semenumpb.ForeignKeyAction_NO_ACTION: fkActionNone, + semenumpb.ForeignKeyAction_RESTRICT: fkActionRestrict, + semenumpb.ForeignKeyAction_CASCADE: fkActionCascade, + semenumpb.ForeignKeyAction_SET_NULL: fkActionSetNull, + semenumpb.ForeignKeyAction_SET_DEFAULT: fkActionSetDefault, } fkMatchTypeFull = tree.NewDString("f") fkMatchTypePartial = tree.NewDString("p") fkMatchTypeSimple = tree.NewDString("s") - fkMatchMap = map[descpb.ForeignKeyReference_Match]tree.Datum{ - descpb.ForeignKeyReference_SIMPLE: fkMatchTypeSimple, - descpb.ForeignKeyReference_FULL: fkMatchTypeFull, - descpb.ForeignKeyReference_PARTIAL: fkMatchTypePartial, + fkMatchMap = map[semenumpb.Match]tree.Datum{ + semenumpb.Match_SIMPLE: fkMatchTypeSimple, + semenumpb.Match_FULL: fkMatchTypeFull, + semenumpb.Match_PARTIAL: fkMatchTypePartial, } ) diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 393c043a9d24..d537004d8214 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -5544,6 +5544,7 @@ func TestTableValidityWhileAddingFK(t *testing.T) { CREATE DATABASE t; CREATE TABLE t.child (a INT PRIMARY KEY, b INT, INDEX (b)); CREATE TABLE t.parent (a INT PRIMARY KEY); +SET use_declarative_schema_changer = off; `); err != nil { t.Fatal(err) } @@ -6763,6 +6764,7 @@ func TestRollbackForeignKeyAddition(t *testing.T) { tdb.Exec(t, `CREATE DATABASE db`) tdb.Exec(t, `CREATE TABLE db.t (a INT PRIMARY KEY)`) tdb.Exec(t, `CREATE TABLE db.t2 (a INT)`) + tdb.Exec(t, `SET use_declarative_schema_changer = off`) g := ctxgroup.WithContext(ctx) g.GoCtx(func(ctx context.Context) error { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index b7db499073b3..644499dfcac7 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -61,6 +61,11 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{ return true } + // Support ALTER TABLE ... ADD CONSTRAINT FOREIGN KEY + if _, ok := t.ConstraintDef.(*tree.ForeignKeyConstraintTableDef); ok && t.ValidationBehavior == tree.ValidationDefault { + return true + } + return false }}, } @@ -72,6 +77,7 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{ var alterTableAddConstraintMinSupportedClusterVersion = map[string]clusterversion.Key{ "ADD_PRIMARY_KEY_DEFAULT": clusterversion.V22_2Start, "ADD_CHECK_DEFAULT": clusterversion.V23_1Start, + "ADD_FOREIGN_KEY_DEFAULT": clusterversion.V23_1Start, } func init() { @@ -163,6 +169,8 @@ func alterTableAddConstraintSupportedInCurrentClusterVersion( } case *tree.CheckConstraintTableDef: cmdKey = "ADD_CHECK" + case *tree.ForeignKeyConstraintTableDef: + cmdKey = "ADD_FOREIGN_KEY" } // Figure out command validation behavior: DEFAULT or SKIP if constraint.ValidationBehavior == tree.ValidationDefault { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go index 05e72b453440..a97b6b581cc5 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go @@ -7,18 +7,22 @@ // 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 scbuildstmt import ( "bytes" "fmt" + "strings" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" @@ -26,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/errors" ) @@ -41,10 +46,15 @@ func alterTableAddConstraint( if t.ValidationBehavior == tree.ValidationDefault { alterTableAddCheck(b, tn, tbl, t) } + case *tree.ForeignKeyConstraintTableDef: + if t.ValidationBehavior == tree.ValidationDefault { + alterTableAddForeignKey(b, tn, tbl, t) + } } } -// alterTableAddPrimaryKey contains logics for building ALTER TABLE ... ADD PRIMARY KEY. +// alterTableAddPrimaryKey contains logics for building +// `ALTER TABLE ... ADD PRIMARY KEY`. // It assumes `t` is such a command. func alterTableAddPrimaryKey( b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableAddConstraint, @@ -67,23 +77,25 @@ func alterTableAddPrimaryKey( }) } -// alterTableAddCheck contains logic for building ALTER TABLE ... ADD CONSTRAINT ... CHECK. +// alterTableAddCheck contains logic for building +// `ALTER TABLE ... ADD CONSTRAINT ... CHECK`. // It assumes `t` is such a command. func alterTableAddCheck( b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableAddConstraint, ) { - // 1. Validate whether the to-be-created check constraint has a name that has already been used. + // 1. Validate whether the to-be-created check constraint has a name that + // has already been used. if skip, err := validateConstraintNameIsNotUsed(b, tn, tbl, t); err != nil { panic(err) } else if skip { return } - // 2. CheckDeepCopy whether this check constraint is syntactically valid. See the comments of - // DequalifyAndValidateExprImpl for criteria. + // 2. CheckDeepCopy whether this check constraint is syntactically valid. + // See the comments of DequalifyAndValidateExprImpl for criteria. ckDef := t.ConstraintDef.(*tree.CheckConstraintTableDef) - ckExpr, _, colIDs, err := schemaexpr.DequalifyAndValidateExprImpl(b, ckDef.Expr, types.Bool, "CHECK", - b.SemaCtx(), volatility.Volatile, tn, + ckExpr, _, colIDs, err := schemaexpr.DequalifyAndValidateExprImpl(b, ckDef.Expr, types.Bool, + "CHECK", b.SemaCtx(), volatility.Volatile, tn, func() colinfo.ResultColumns { return getNonDropResultColumns(b, tbl.TableID) }, @@ -121,26 +133,285 @@ func alterTableAddCheck( }) } -// getIndexIDForValidationForConstraint returns the index ID this check constraint is supposed to -// check against and will be used to hint the constraint validation query -// in `backfill.go`. -// Normally, it will return zero, which means we won't hint the validation query -// but instead let the optimizer pick the appropriate index to serve it. -// It will return non-zero if the constraint is added while a new column is being -// added (e.g. `ALTER TABLE t ADD COLUMN j INT DEFAULT 1 CHECK (j < 0);`), in which -// case we have to explicitly hint the validation query to check against the -// new primary index that is being added (as a result of adding a new column), instead -// of against the old primary index (which does not contain backfilled values for -// the new column and hence would mistakenly allow the validation query to succeed). +// getIndexIDForValidationForConstraint returns the index ID this check +// constraint is supposed to check against and will be used to hint the +// constraint validation query in `backfill.go`. +// Normally, it will return zero, which means we won't hint the validation +// query but instead let the optimizer pick the appropriate index to serve it. +// It will return non-zero if the constraint is added while a new column is +// being added (e.g. `ALTER TABLE t ADD COLUMN j INT DEFAULT 1 CHECK (j < 0)`), +// in which case we have to explicitly hint the validation query to check +// against the new primary index that is being added (as a result of adding a +// new column), instead of against the old primary index (which does not +// contain backfilled values for the new column and hence would mistakenly +// allow the validation query to succeed). func getIndexIDForValidationForConstraint(b BuildCtx, tableID catid.DescID) (ret catid.IndexID) { - b.QueryByID(tableID).ForEachElementStatus(func(current scpb.Status, target scpb.TargetStatus, e scpb.Element) { - if pie, ok := e.(*scpb.PrimaryIndex); ok && target == scpb.ToPublic && current != scpb.Status_PUBLIC { + b.QueryByID(tableID).ForEachElementStatus(func( + current scpb.Status, target scpb.TargetStatus, e scpb.Element, + ) { + if pie, ok := e.(*scpb.PrimaryIndex); ok && + target == scpb.ToPublic && current != scpb.Status_PUBLIC { ret = pie.IndexID } }) return ret } +// alterTableAddForeignKey contains logic for building +// `ALTER TABLE ... ADD CONSTRAINT ... FOREIGN KEY`. +// It assumes `t` is such a command. +func alterTableAddForeignKey( + b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableAddConstraint, +) { + fkDef := t.ConstraintDef.(*tree.ForeignKeyConstraintTableDef) + // fromColsFRNames is fully resolved column names from `fkDef.FromCols`, and + // is only used in constructing error messages to be consistent with legacy + // schema changer. + fromColsFRNames := getFullyResolvedColNames(b, tbl.TableID, fkDef.FromCols) + + // 1. If this FK's `ON UPDATE behavior` is not NO ACTION nor RESTRICT, and + // any of the originColumns has an `ON UPDATE expr`, panic with error. + if fkDef.Actions.Update != tree.NoAction && fkDef.Actions.Update != tree.Restrict { + for _, colName := range fkDef.FromCols { + colID := mustGetColumnIDFromColumnName(b, tbl.TableID, colName) + colHasOnUpdate := retrieveColumnOnUpdateExpressionElem(b, tbl.TableID, colID) != nil + if colHasOnUpdate { + panic(pgerror.Newf( + pgcode.InvalidTableDefinition, + "cannot specify a foreign key update action and an ON UPDATE"+ + " expression on the same column", + )) + } + } + } + + // 2. If this FK has SET NULL action (ON UPDATE or ON DELETE) && any one of + // the originColumns is NOT NULL, then panic with error. + if fkDef.Actions.Delete == tree.SetNull || fkDef.Actions.Update == tree.SetNull { + for i, colName := range fkDef.FromCols { + colID := mustGetColumnIDFromColumnName(b, tbl.TableID, colName) + colIsNullable := mustRetrieveColumnTypeElem(b, tbl.TableID, colID).IsNullable + if !colIsNullable { + panic(pgerror.Newf(pgcode.InvalidForeignKey, + "cannot add a SET NULL cascading action on column %q which has a NOT NULL constraint", fromColsFRNames[i], + )) + } + } + } + + // 3. If this FK has SET DEFAULT action (ON UPDATE or ON DELETE) && any one + // of the originColumns does not have a default expression (which implies + // NULL will be used as the default) && that column is NOT NULL, then panic + // with error. + if fkDef.Actions.Delete == tree.SetDefault || fkDef.Actions.Update == tree.SetDefault { + for i, colName := range fkDef.FromCols { + colID := mustGetColumnIDFromColumnName(b, tbl.TableID, colName) + colHasDefault := retrieveColumnDefaultExpressionElem(b, tbl.TableID, colID) != nil + colIsNullable := mustRetrieveColumnTypeElem(b, tbl.TableID, colID).IsNullable + if !colHasDefault && !colIsNullable { + panic(pgerror.Newf(pgcode.InvalidForeignKey, + "cannot add a SET DEFAULT cascading action on column %q which has a "+ + "NOT NULL constraint and a NULL default expression", fromColsFRNames[i], + )) + } + } + } + + // 4. Check whether each originColumns can be used for an outbound FK + // constraint, and no duplicates exist in originColumns. + var originColIDs []catid.ColumnID + var originColSet catalog.TableColSet + for i, colName := range fkDef.FromCols { + colID := mustGetColumnIDFromColumnName(b, tbl.TableID, colName) + ensureColCanBeUsedInOutboundFK(b, tbl.TableID, colID) + if originColSet.Contains(colID) { + panic(pgerror.Newf(pgcode.InvalidForeignKey, + "foreign key contains duplicate column %q", fromColsFRNames[i])) + } + originColSet.Add(colID) + originColIDs = append(originColIDs, colID) + } + + // 5. Resolve `t.(*ForeignKeyConstraintTableDef).Table` (i.e. referenced + // table name) and check whether it's in the same database as the originTable + // (i.e. tbl). Cross database FK references is a deprecated feature and is in + // practice no longer supported. We will return an error here directly. + referencedTableID := mustGetTableIDFromTableName(b, fkDef.Table) + originalTableNamespaceElem := mustRetrieveNamespaceElem(b, tbl.TableID) + referencedTableNamespaceElem := mustRetrieveNamespaceElem(b, referencedTableID) + if originalTableNamespaceElem.DatabaseID != referencedTableNamespaceElem.DatabaseID { + panic(scerrors.NotImplementedErrorf(t, "cross DB FK reference is a deprecated feature "+ + "and is no longer supported.")) + } + + // 6. Check that temporary tables can only reference temporary tables, or, + // permanent tables can only reference permanent tables. + // In other words, we don't allow it if originTable's temporariness does not + // match referencedTable's temporariness. + referencedTableElem := mustRetrieveTableElem(b, referencedTableID) + fkDef.Table.ObjectNamePrefix = b.NamePrefix(referencedTableElem) + if tbl.IsTemporary != referencedTableElem.IsTemporary { + persistenceType := "permanent" + if tbl.IsTemporary { + persistenceType = "temporary" + } + panic(pgerror.Newf( + pgcode.InvalidTableDefinition, + "constraints on %s tables may reference only %s tables", + persistenceType, + persistenceType, + )) + } + + // 7. Compute referencedColumnNames, which is usually provided in + // `t.(*ForeignKeyConstraintTableDef).ToCols`. But if it's empty, then we + // attempt to add this FK on the PK of the referenced table, excluding + // implicit columns. + if len(fkDef.ToCols) == 0 { + primaryIndexIDInReferencedTable := getCurrentPrimaryIndexID(b, referencedTableID) + numImplicitCols := 0 + primaryIndexPartitioningElemInReferencedTable := maybeRetrieveIndexPartitioningElem(b, referencedTableID, primaryIndexIDInReferencedTable) + if primaryIndexPartitioningElemInReferencedTable != nil { + numImplicitCols = int(primaryIndexPartitioningElemInReferencedTable.NumImplicitColumns) + } + keyColIDsOfPrimaryIndexInReferencedTable, _, _ := getSortedColumnIDsInIndex(b, referencedTableID, primaryIndexIDInReferencedTable) + for i := numImplicitCols; i < len(keyColIDsOfPrimaryIndexInReferencedTable); i++ { + fkDef.ToCols = append( + fkDef.ToCols, + tree.Name(mustRetrieveColumnNameElem(b, referencedTableID, keyColIDsOfPrimaryIndexInReferencedTable[i]).Name), + ) + } + } + + // 8. Similarly to 4, check whether each referencedColumn can be used for an + // inbound FK constraint, and the length of referencedColumns must be equal + // to the length of originColumns. + var referencedColIDs []catid.ColumnID + for _, colName := range fkDef.ToCols { + colID := mustGetColumnIDFromColumnName(b, referencedTableID, colName) + ensureColCanBeUsedInInboundFK(b, referencedTableID, colID) + referencedColIDs = append(referencedColIDs, colID) + } + if len(originColIDs) != len(referencedColIDs) { + panic(pgerror.Newf(pgcode.Syntax, + "%d columns must reference exactly %d columns in referenced table (found %d)", + len(originColIDs), len(originColIDs), len(referencedColIDs))) + } + + // 9. Check whether types of originColumns match types of referencedColumns. + // Namely, we will panic if their column type is not "compatible" (i.e. we do + // not insist that the column types are of exactly the same type; it's okay + // if they are compatible but not exactly the same. E.g. Two types under the + // same type family are usually compatible). If the column types are + // compatible but not exactly the same, send a notice to the client about this. + for i := range originColIDs { + originColName := mustRetrieveColumnNameElem(b, tbl.TableID, originColIDs[i]).Name + originColType := mustRetrieveColumnTypeElem(b, tbl.TableID, originColIDs[i]).Type + referencedColName := mustRetrieveColumnNameElem(b, referencedTableID, referencedColIDs[i]).Name + referencedColType := mustRetrieveColumnTypeElem(b, referencedTableID, referencedColIDs[i]).Type + if !originColType.Equivalent(referencedColType) { + panic(pgerror.Newf(pgcode.DatatypeMismatch, + "type of %q (%s) does not match foreign key %q.%q (%s)", originColName, originColType.String(), + referencedTableNamespaceElem.Name, referencedColName, referencedColType.String())) + } + if !originColType.Identical(referencedColType) { + b.EvalCtx().ClientNoticeSender.BufferClientNotice(b, + pgnotice.Newf( + "type of foreign key column %q (%s) is not identical to referenced column %q.%q (%s)", + originColName, originColType.String(), + referencedTableNamespaceElem.Name, referencedColName, referencedColType.String()), + ) + } + } + + // 10. Check that the name of this to-be-added FK constraint hasn't been used; + // Or, give it one if no name is specified. + if skip, err := validateConstraintNameIsNotUsed(b, tn, tbl, t); err != nil { + panic(err) + } else if skip { + return + } + if fkDef.Name == "" { + fkDef.Name = tree.Name(tabledesc.GenerateUniqueName( + tabledesc.ForeignKeyConstraintName(tn.Table(), fkDef.FromCols.ToStrings()), + func(name string) bool { + return constraintNameInUse(b, tbl.TableID, name) + }, + )) + } + + // 11. Verify that the referencedTable guarantees uniqueness on the + // referencedColumns. In code, this means we need to find either a + // PRIMARY INDEX, a UNIQUE INDEX, or a UNIQUE_WITHOUT_INDEX CONSTRAINT, + // that covers exactly referencedColumns. + if areColsUnique := areColsUniqueInTable(b, referencedTableID, referencedColIDs); !areColsUnique { + panic(pgerror.Newf( + pgcode.ForeignKeyViolation, + "there is no unique constraint matching given keys for referenced table %s", + referencedTableNamespaceElem.Name, + )) + } + + // 12. (Finally!) Add a ForeignKey_Constraint, ConstraintName element to + // builder state. + constraintID := b.NextTableConstraintID(tbl.TableID) + b.Add(&scpb.ForeignKeyConstraint{ + TableID: tbl.TableID, + ConstraintID: constraintID, + ColumnIDs: originColIDs, + ReferencedTableID: referencedTableID, + ReferencedColumnIDs: referencedColIDs, + OnUpdateAction: tree.ForeignKeyReferenceActionValue[fkDef.Actions.Update], + OnDeleteAction: tree.ForeignKeyReferenceActionValue[fkDef.Actions.Delete], + CompositeKeyMatchMethod: tree.CompositeKeyMatchMethodValue[fkDef.Match], + IndexIDForValidation: getIndexIDForValidationForConstraint(b, tbl.TableID), + }) + b.Add(&scpb.ConstraintWithoutIndexName{ + TableID: tbl.TableID, + ConstraintID: constraintID, + Name: string(fkDef.Name), + }) +} + +// getFullyResolvedColNames returns fully resolved column names. +// For each name in `colNames`, its fully resolved name will be "db.sc.tbl.col". +// The order of column names in the return is in syc with the input `colNames`. +func getFullyResolvedColNames( + b BuildCtx, tableID catid.DescID, colNames tree.NameList, +) (ret tree.NameList) { + ns := mustRetrieveNamespaceElem(b, tableID) + tableName := ns.Name + schemaName := mustRetrieveNamespaceElem(b, ns.SchemaID).Name + databaseName := mustRetrieveNamespaceElem(b, ns.DatabaseID).Name + + for _, colName := range colNames { + fullyResolvedColName := strings.Join([]string{databaseName, schemaName, tableName, string(colName)}, ".") + ret = append(ret, tree.Name(fullyResolvedColName)) + } + return ret +} + +// areColsUniqueInTable ensures uniqueness on columns is guaranteed on this table. +func areColsUniqueInTable(b BuildCtx, tableID catid.DescID, columnIDs []catid.ColumnID) (ret bool) { + b.QueryByID(tableID).ForEachElementStatus(func(current scpb.Status, target scpb.TargetStatus, e scpb.Element) { + if ret { + return + } + + switch te := e.(type) { + case *scpb.PrimaryIndex: + ret = isIndexUniqueAndCanServeFK(b, &te.Index, columnIDs) + case *scpb.SecondaryIndex: + ret = isIndexUniqueAndCanServeFK(b, &te.Index, columnIDs) + case *scpb.UniqueWithoutIndexConstraint: + if te.Predicate == nil && descpb.ColumnIDs(te.ColumnIDs).PermutationOf(columnIDs) { + ret = true + } + } + }) + return ret +} + // validateConstraintNameIsNotUsed checks that the name of the constraint we're // trying to add isn't already used, and, if it is, whether the constraint // addition should be skipped: @@ -159,7 +430,8 @@ func validateConstraintNameIsNotUsed( name = d.Name ifNotExists = d.IfNotExists case *tree.ForeignKeyConstraintTableDef: - panic(scerrors.NotImplementedErrorf(t, "FK constraint %v not yet implemented", d.Name)) + name = d.Name + ifNotExists = d.IfNotExists case *tree.UniqueConstraintTableDef: panic(scerrors.NotImplementedErrorf(t, "UNIQUE constraint %v not yet implemented", d.Name)) default: @@ -193,8 +465,8 @@ func validateConstraintNameIsNotUsed( "duplicate constraint name: %q", name) } -// getNonDropResultColumns returns all public and adding columns, sorted by column ID in ascending order, -// in the format of ResultColumns. +// getNonDropResultColumns returns all public and adding columns, sorted by +// column ID in ascending order, in the format of ResultColumns. func getNonDropResultColumns(b BuildCtx, tableID catid.DescID) (ret colinfo.ResultColumns) { for _, col := range getNonDropColumns(b, tableID) { ret = append(ret, colinfo.ResultColumn{ @@ -245,7 +517,8 @@ func generateUniqueCheckConstraintName(b BuildCtx, tableID catid.DescID, ckExpr return name } -// constraintNameInUse returns whether `name` has been used by any other non-dropping constraint. +// constraintNameInUse returns whether `name` has been used by any other +// non-dropping constraint. func constraintNameInUse(b BuildCtx, tableID catid.DescID, name string) (ret bool) { scpb.ForEachConstraintWithoutIndexName(b.QueryByID(tableID).Filter(publicTargetFilter), func( _ scpb.Status, _ scpb.TargetStatus, e *scpb.ConstraintWithoutIndexName, @@ -322,3 +595,111 @@ func iterateColNamesInExpr( panic(err) } } + +func retrieveColumnDefaultExpressionElem( + b BuildCtx, tableID catid.DescID, columnID catid.ColumnID, +) *scpb.ColumnDefaultExpression { + _, _, ret := scpb.FindColumnDefaultExpression(b.QueryByID(tableID).Filter(hasColumnIDAttrFilter(columnID))) + return ret +} + +func retrieveColumnOnUpdateExpressionElem( + b BuildCtx, tableID catid.DescID, columnID catid.ColumnID, +) (columnOnUpdateExpression *scpb.ColumnOnUpdateExpression) { + scpb.ForEachColumnOnUpdateExpression(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.ColumnOnUpdateExpression, + ) { + if e.ColumnID == columnID { + columnOnUpdateExpression = e + } + }) + return columnOnUpdateExpression +} + +// ensureColCanBeUsedInOutboundFK ensures the column can be used in an outbound +// FK reference. Panic if it cannot. +func ensureColCanBeUsedInOutboundFK(b BuildCtx, tableID catid.DescID, columnID catid.ColumnID) { + colNameElem := mustRetrieveColumnNameElem(b, tableID, columnID) + colTypeElem := mustRetrieveColumnTypeElem(b, tableID, columnID) + colElem := mustRetrieveColumnElem(b, tableID, columnID) + + if colElem.IsInaccessible { + panic(pgerror.Newf( + pgcode.UndefinedColumn, + "column %q is inaccessible and cannot reference a foreign key", + colNameElem.Name, + )) + } + + if colTypeElem.IsVirtual { + panic(unimplemented.NewWithIssuef( + 59671, "virtual column %q cannot reference a foreign key", + colNameElem.Name, + )) + } + + if colTypeElem.ComputeExpr != nil { + panic(unimplemented.NewWithIssuef( + 46672, "computed column %q cannot reference a foreign key", + colNameElem.Name, + )) + } +} + +// ensureColCanBeUsedInInboundFK ensures the column can be used in an inbound +// FK reference. Panic if it cannot. +func ensureColCanBeUsedInInboundFK(b BuildCtx, tableID catid.DescID, columnID catid.ColumnID) { + colNameElem := mustRetrieveColumnNameElem(b, tableID, columnID) + colTypeElem := mustRetrieveColumnTypeElem(b, tableID, columnID) + colElem := mustRetrieveColumnElem(b, tableID, columnID) + + if colElem.IsInaccessible { + panic(pgerror.Newf( + pgcode.UndefinedColumn, + "column %q is inaccessible and cannot be referenced by a foreign key", + colNameElem.Name, + )) + } + if colTypeElem.IsVirtual { + panic(unimplemented.NewWithIssuef( + 59671, "virtual column %q cannot be referenced by a foreign key", + colNameElem.Name, + )) + } +} + +func mustRetrieveTableElem(b BuildCtx, tableID catid.DescID) *scpb.Table { + _, _, tblElem := scpb.FindTable(b.QueryByID(tableID)) + if tblElem == nil { + panic(errors.AssertionFailedf("programming error: cannot find a Table element for table %v", tableID)) + } + return tblElem +} + +func mustRetrieveNamespaceElem(b BuildCtx, tableID catid.DescID) *scpb.Namespace { + _, _, ns := scpb.FindNamespace(b.QueryByID(tableID)) + if ns == nil { + panic(errors.AssertionFailedf("programming error: cannot find a Namespace element for table %v", tableID)) + } + return ns +} + +func maybeRetrieveIndexPartitioningElem( + b BuildCtx, tableID catid.DescID, indexID catid.IndexID, +) (ret *scpb.IndexPartitioning) { + scpb.ForEachIndexPartitioning(b.QueryByID(tableID), func(current scpb.Status, target scpb.TargetStatus, e *scpb.IndexPartitioning) { + if e.IndexID == indexID { + ret = e + } + }) + return ret +} + +func getCurrentPrimaryIndexID(b BuildCtx, tableID catid.DescID) (ret catid.IndexID) { + b.QueryByID(tableID).ForEachElementStatus(func(current scpb.Status, target scpb.TargetStatus, e scpb.Element) { + if pie, ok := e.(*scpb.PrimaryIndex); ok && current == scpb.Status_PUBLIC { + ret = pie.IndexID + } + }) + return ret +} diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go index b827562694ce..bdf7b235b9da 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go @@ -379,12 +379,6 @@ func explicitColumnStartIdx(b BuildCtx, ie *scpb.Index) int { start = int(ipe.NumImplicitColumns) } }) - // Currently, we only allow implicit partitioning on hash sharded index. When - // that happens, the shard column always comes after implicit partition - // columns. - if ie.Sharding != nil && ie.Sharding.IsSharded { - start++ - } return start } @@ -420,6 +414,10 @@ func explicitKeyColumnIDsWithoutShardColumn(b BuildCtx, ie *scpb.Index) descpb.C func isIndexUniqueAndCanServeFK( b BuildCtx, ie *scpb.Index, fkReferencedColIDs []tree.ColumnID, ) bool { + if !ie.IsUnique { + return false + } + isPartial := false scpb.ForEachSecondaryIndexPartial(b.QueryByID(ie.TableID), func( current scpb.Status, target scpb.TargetStatus, sipe *scpb.SecondaryIndexPartial, @@ -428,9 +426,16 @@ func isIndexUniqueAndCanServeFK( isPartial = true } }) + if isPartial { + return false + } - return ie.IsUnique && !isPartial && - explicitKeyColumnIDsWithoutShardColumn(b, ie).PermutationOf(fkReferencedColIDs) + keyColIDs, _, _ := getSortedColumnIDsInIndex(b, ie.TableID, ie.IndexID) + implicitKeyColIDs := keyColIDs[:explicitColumnStartIdx(b, ie)] + explicitKeyColIDsWithoutShardCol := explicitKeyColumnIDsWithoutShardColumn(b, ie) + allKeyColIDsWithoutShardCol := descpb.ColumnIDs(append(implicitKeyColIDs, explicitKeyColIDsWithoutShardCol...)) + return explicitKeyColIDsWithoutShardCol.PermutationOf(fkReferencedColIDs) || + allKeyColIDsWithoutShardCol.PermutationOf(fkReferencedColIDs) } // uniqueConstraintHasReplacementCandidate returns true if `elms` contains an index diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go index cfbc2708af89..a6761ebbee24 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go @@ -329,6 +329,30 @@ func getColumnIDFromColumnName( return colElem.ColumnID } +// mustGetColumnIDFromColumnName looks up a column's ID by its name. +// If no column with this name exists, panic. +func mustGetColumnIDFromColumnName( + b BuildCtx, tableID catid.DescID, columnName tree.Name, +) catid.ColumnID { + colID := getColumnIDFromColumnName(b, tableID, columnName) + if colID == 0 { + panic(errors.AssertionFailedf("cannot find column with name %v", columnName)) + } + return colID +} + +func mustGetTableIDFromTableName(b BuildCtx, tableName tree.TableName) catid.DescID { + tableElems := b.ResolveTable(tableName.ToUnresolvedObjectName(), ResolveParams{ + IsExistenceOptional: false, + RequiredPrivilege: privilege.CREATE, + }) + _, _, tableElem := scpb.FindTable(tableElems) + if tableElem == nil { + panic(errors.AssertionFailedf("programming error: cannot find a Table element for table %v", tableName)) + } + return tableElem.TableID +} + func toPublicNotCurrentlyPublicFilter( status scpb.Status, target scpb.TargetStatus, _ scpb.Element, ) bool { diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_table b/pkg/sql/schemachanger/scbuild/testdata/drop_table index 9fdbfc406b9c..8f2f39c57854 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_table +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_table @@ -121,13 +121,13 @@ DROP TABLE defaultdb.shipments CASCADE; {comment: pkey is good, indexId: 1, tableId: 109} - [[IndexData:{DescID: 109, IndexID: 1}, ABSENT], PUBLIC] {indexId: 1, tableId: 109} -- [[ForeignKeyConstraint:{DescID: 109, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], PUBLIC] +- [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], PUBLIC] {columnIds: [4], constraintId: 2, referencedColumnIds: [1], referencedTableId: 104, tableId: 109} - [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT], PUBLIC] {constraintId: 2, name: fk_customers, tableId: 109} - [[ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is important}, ABSENT], PUBLIC] {comment: customer is important, constraintId: 2, tableId: 109} -- [[ForeignKeyConstraint:{DescID: 109, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], PUBLIC] +- [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], PUBLIC] {columnIds: [4], constraintId: 3, referencedColumnIds: [2], referencedTableId: 105, tableId: 109} - [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT], PUBLIC] {constraintId: 3, name: fk_orders, tableId: 109} diff --git a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table index 9c0de7b4c51d..8d66b2c247cc 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table +++ b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table @@ -12,10 +12,6 @@ CREATE TABLE defaultdb.foo ( ); ---- -unimplemented -ALTER TABLE defaultdb.foo ADD COLUMN j INT REFERENCES defaultdb.foo(i) ----- - unimplemented ALTER TABLE defaultdb.foo ADD COLUMN j SERIAL ---- @@ -92,10 +88,6 @@ unimplemented ALTER TABLE defaultdb.foo ADD UNIQUE(i); ---- -unimplemented -ALTER TABLE defaultdb.foo ADD FOREIGN KEY (i) REFERENCES defaultdb.foo(i); ----- - unimplemented ALTER TABLE defaultdb.foo ADD PRIMARY KEY (l); ---- diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go index ef5fd7bb0c12..6e48de6a830f 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp.go +++ b/pkg/sql/schemachanger/scdecomp/decomp.go @@ -656,11 +656,14 @@ func (w *walkCtx) walkForeignKeyConstraint( ) { // TODO(postamar): proper handling of constraint status w.ev(scpb.Status_PUBLIC, &scpb.ForeignKeyConstraint{ - TableID: tbl.GetID(), - ConstraintID: c.GetConstraintID(), - ColumnIDs: c.ForeignKeyDesc().OriginColumnIDs, - ReferencedTableID: c.GetReferencedTableID(), - ReferencedColumnIDs: c.ForeignKeyDesc().ReferencedColumnIDs, + TableID: tbl.GetID(), + ConstraintID: c.GetConstraintID(), + ColumnIDs: c.ForeignKeyDesc().OriginColumnIDs, + ReferencedTableID: c.GetReferencedTableID(), + ReferencedColumnIDs: c.ForeignKeyDesc().ReferencedColumnIDs, + OnUpdateAction: c.OnUpdate(), + OnDeleteAction: c.OnDelete(), + CompositeKeyMatchMethod: c.Match(), }) w.ev(scpb.Status_PUBLIC, &scpb.ConstraintWithoutIndexName{ TableID: tbl.GetID(), diff --git a/pkg/sql/schemachanger/scdecomp/testdata/table b/pkg/sql/schemachanger/scdecomp/testdata/table index bcca59ff9244..31d87d1a86ae 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/table +++ b/pkg/sql/schemachanger/scdecomp/testdata/table @@ -307,7 +307,11 @@ ElementState: - ForeignKeyConstraint: columnIds: - 1 + compositeKeyMatchMethod: SIMPLE constraintId: 2 + indexIdForValidation: 0 + onDeleteAction: NO_ACTION + onUpdateAction: NO_ACTION referencedColumnIds: - 1 referencedTableId: 104 diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index e7596f8fff70..254f45eaa060 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -999,15 +999,28 @@ func (s *TestState) Validator() scexec.Validator { return s } -// ValidateCheckConstraint implements the validator interface. -func (s *TestState) ValidateCheckConstraint( +// ValidateConstraint implements the validator interface. +func (s *TestState) ValidateConstraint( ctx context.Context, tbl catalog.TableDescriptor, - constraint catalog.CheckConstraint, + constraint catalog.Constraint, indexIDForValidation descpb.IndexID, override sessiondata.InternalExecutorOverride, ) error { - s.LogSideEffectf("validate check constraint %v in table #%d", constraint.GetName(), tbl.GetID()) + s.LogSideEffectf("validate %v constraint %v in table #%d", + catalog.GetConstraintType(constraint), constraint.GetName(), tbl.GetID()) + return nil +} + +func (s *TestState) ValidateForeignKeyConstraint( + ctx context.Context, + out catalog.TableDescriptor, + in catalog.TableDescriptor, + constraint catalog.Constraint, + override sessiondata.InternalExecutorOverride, +) error { + s.LogSideEffectf("validate foreign key constraint %v from table #%d to table #%d", + constraint.GetName(), out.GetID(), in.GetID()) return nil } diff --git a/pkg/sql/schemachanger/scdeps/validator.go b/pkg/sql/schemachanger/scdeps/validator.go index f3c1bd981c60..d18b281486d8 100644 --- a/pkg/sql/schemachanger/scdeps/validator.go +++ b/pkg/sql/schemachanger/scdeps/validator.go @@ -53,11 +53,11 @@ type ValidateInvertedIndexesFn func( protectedTSProvider scexec.ProtectedTimestampManager, ) error -// ValidateCheckConstraintFn callback function for validting check constraints. -type ValidateCheckConstraintFn func( +// ValidateConstraintFn callback function for validating constraints. +type ValidateConstraintFn func( ctx context.Context, tbl catalog.TableDescriptor, - constraint catalog.CheckConstraint, + constraint catalog.Constraint, indexIDForValidation descpb.IndexID, sessionData *sessiondata.SessionData, runHistoricalTxn descs.HistoricalInternalExecTxnRunner, @@ -75,7 +75,7 @@ type validator struct { ieFactory sqlutil.InternalExecutorFactory validateForwardIndexes ValidateForwardIndexesFn validateInvertedIndexes ValidateInvertedIndexesFn - validateCheckConstraint ValidateCheckConstraintFn + validateConstraint ValidateConstraintFn newFakeSessionData NewFakeSessionDataFn protectedTimestampProvider scexec.ProtectedTimestampManager } @@ -113,14 +113,14 @@ func (vd validator) ValidateInvertedIndexes( ) } -func (vd validator) ValidateCheckConstraint( +func (vd validator) ValidateConstraint( ctx context.Context, tbl catalog.TableDescriptor, - constraint catalog.CheckConstraint, + constraint catalog.Constraint, indexIDForValidation descpb.IndexID, override sessiondata.InternalExecutorOverride, ) error { - return vd.validateCheckConstraint(ctx, tbl, constraint, indexIDForValidation, vd.newFakeSessionData(&vd.settings.SV), + return vd.validateConstraint(ctx, tbl, constraint, indexIDForValidation, vd.newFakeSessionData(&vd.settings.SV), vd.makeHistoricalInternalExecTxnRunner(), override) } @@ -151,7 +151,7 @@ func NewValidator( protectedTimestampProvider scexec.ProtectedTimestampManager, validateForwardIndexes ValidateForwardIndexesFn, validateInvertedIndexes ValidateInvertedIndexesFn, - validateCheckConstraint ValidateCheckConstraintFn, + validateCheckConstraint ValidateConstraintFn, newFakeSessionData NewFakeSessionDataFn, ) scexec.Validator { return validator{ @@ -161,7 +161,7 @@ func NewValidator( ieFactory: ieFactory, validateForwardIndexes: validateForwardIndexes, validateInvertedIndexes: validateInvertedIndexes, - validateCheckConstraint: validateCheckConstraint, + validateConstraint: validateCheckConstraint, newFakeSessionData: newFakeSessionData, protectedTimestampProvider: protectedTimestampProvider, } diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index 1c4ff8d4a430..603293422990 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -234,10 +234,10 @@ type Validator interface { override sessiondata.InternalExecutorOverride, ) error - ValidateCheckConstraint( + ValidateConstraint( ctx context.Context, tbl catalog.TableDescriptor, - constraint catalog.CheckConstraint, + constraint catalog.Constraint, indexIDForValidation descpb.IndexID, override sessiondata.InternalExecutorOverride, ) error diff --git a/pkg/sql/schemachanger/scexec/exec_validation.go b/pkg/sql/schemachanger/scexec/exec_validation.go index 908da1a82ef9..92b06a13544f 100644 --- a/pkg/sql/schemachanger/scexec/exec_validation.go +++ b/pkg/sql/schemachanger/scexec/exec_validation.go @@ -50,8 +50,8 @@ func executeValidateUniqueIndex( return nil } -func executeValidateCheckConstraint( - ctx context.Context, deps Dependencies, op *scop.ValidateCheckConstraint, +func executeValidateConstraint( + ctx context.Context, deps Dependencies, op *scop.ValidateConstraint, ) error { descs, err := deps.Catalog().MustReadImmutableDescriptors(ctx, op.TableID) if err != nil { @@ -66,15 +66,10 @@ func executeValidateCheckConstraint( if err != nil { return err } - check := constraint.AsCheck() - if check == nil { - return errors.Newf("constraint ID %v does not identify a check constraint in table %v.", - op.ConstraintID, op.TableID) - } // Execute the validation operation as a root user. execOverride := sessiondata.RootUserSessionDataOverride - err = deps.Validator().ValidateCheckConstraint(ctx, table, check, op.IndexIDForValidation, execOverride) + err = deps.Validator().ValidateConstraint(ctx, table, constraint, op.IndexIDForValidation, execOverride) if err != nil { return scerrors.SchemaChangerUserError(err) } @@ -99,8 +94,8 @@ func executeValidationOp(ctx context.Context, deps Dependencies, op scop.Op) (er } return err } - case *scop.ValidateCheckConstraint: - if err = executeValidateCheckConstraint(ctx, deps, op); err != nil { + case *scop.ValidateConstraint: + if err = executeValidateConstraint(ctx, deps, op); err != nil { if !scerrors.HasSchemaChangerUserError(err) { return errors.Wrapf(err, "%T: %v", op, op) } diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 39b0f0398178..ba106484ce93 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -469,10 +469,10 @@ func (noopValidator) ValidateInvertedIndexes( return nil } -func (noopValidator) ValidateCheckConstraint( +func (noopValidator) ValidateConstraint( ctx context.Context, tbl catalog.TableDescriptor, - constraint catalog.CheckConstraint, + constraint catalog.Constraint, indexIDForValidation descpb.IndexID, override sessiondata.InternalExecutorOverride, ) error { diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go index e79aceb732fa..09348d53ab0f 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go @@ -169,20 +169,131 @@ func (m *visitor) RemoveForeignKeyConstraint( if err != nil || out.Dropped() { return err } + var found bool for i, fk := range out.OutboundFKs { if fk.ConstraintID == op.ConstraintID { out.OutboundFKs = append(out.OutboundFKs[:i], out.OutboundFKs[i+1:]...) - return nil + if len(out.OutboundFKs) == 0 { + out.OutboundFKs = nil + } + found = true + break } } for i, m := range out.Mutations { if c := m.GetConstraint(); c != nil && - c.ConstraintType != descpb.ConstraintToUpdate_FOREIGN_KEY && + c.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY && c.ForeignKey.ConstraintID == op.ConstraintID { out.Mutations = append(out.Mutations[:i], out.Mutations[i+1:]...) - return nil + if len(out.Mutations) == 0 { + out.Mutations = nil + } + found = true + break + } + } + if !found { + return errors.AssertionFailedf("foreign key with ID %d not found in origin table %q (%d)", + op.ConstraintID, out.GetName(), out.GetID()) + } + return nil +} + +func (m *visitor) MakeAbsentForeignKeyConstraintWriteOnly( + ctx context.Context, op scop.MakeAbsentForeignKeyConstraintWriteOnly, +) error { + out, err := m.checkOutTable(ctx, op.TableID) + if err != nil || out.Dropped() { + return err + } + if op.ConstraintID >= out.NextConstraintID { + out.NextConstraintID = op.ConstraintID + 1 + } + + fk := &descpb.ForeignKeyConstraint{ + OriginTableID: op.TableID, + OriginColumnIDs: op.ColumnIDs, + ReferencedColumnIDs: op.ReferencedColumnIDs, + ReferencedTableID: op.ReferencedTableID, + Name: tabledesc.ConstraintNamePlaceholder(op.ConstraintID), + Validity: descpb.ConstraintValidity_Validating, + OnDelete: op.OnDeleteAction, + OnUpdate: op.OnUpdateAction, + Match: op.CompositeKeyMatchMethod, + ConstraintID: op.ConstraintID, + } + if err = enqueueAddForeignKeyConstraintMutation(out, fk); err != nil { + return err + } + out.Mutations[len(out.Mutations)-1].State = descpb.DescriptorMutation_WRITE_ONLY + return nil +} + +func (m *visitor) MakeValidatedForeignKeyConstraintPublic( + ctx context.Context, op scop.MakeValidatedForeignKeyConstraintPublic, +) error { + out, err := m.checkOutTable(ctx, op.TableID) + if err != nil || out.Dropped() { + return err + } + in, err := m.checkOutTable(ctx, op.ReferencedTableID) + if err != nil || in.Dropped() { + return err + } + + var found bool + for idx, mutation := range out.Mutations { + if c := mutation.GetConstraint(); c != nil && + c.ConstraintType == descpb.ConstraintToUpdate_FOREIGN_KEY && + c.ForeignKey.ConstraintID == op.ConstraintID { + out.OutboundFKs = append(out.OutboundFKs, c.ForeignKey) + in.InboundFKs = append(in.InboundFKs, c.ForeignKey) + + // Remove the mutation from the mutation slice. The `MakeMutationComplete` + // call will also mark the above added check as VALIDATED. + // If this is a rollback of a drop, we are trying to add the foreign key constraint + // back, so swap the direction before making it complete. + mutation.Direction = descpb.DescriptorMutation_ADD + err = out.MakeMutationComplete(mutation) + if err != nil { + return err + } + out.Mutations = append(out.Mutations[:idx], out.Mutations[idx+1:]...) + + found = true + break + } + } + + if !found { + return errors.AssertionFailedf("failed to find foreign key constraint %d in table %q (%d)", + op.ConstraintID, out.GetName(), out.GetID()) + } + + if len(out.Mutations) == 0 { + out.Mutations = nil + } + + return nil +} + +func (m *visitor) MakePublicForeignKeyConstraintValidated( + ctx context.Context, op scop.MakePublicForeignKeyConstraintValidated, +) error { + tbl, err := m.checkOutTable(ctx, op.TableID) + if err != nil { + return err + } + for i, fk := range tbl.OutboundFKs { + if fk.ConstraintID == op.ConstraintID { + tbl.OutboundFKs = append(tbl.OutboundFKs[:i], tbl.OutboundFKs[i+1:]...) + if len(tbl.OutboundFKs) == 0 { + tbl.OutboundFKs = nil + } + fk.Validity = descpb.ConstraintValidity_Dropping + return enqueueDropForeignKeyConstraintMutation(tbl, &fk) } } - return errors.AssertionFailedf("foreign key with ID %d not found in origin table %q (%d)", - op.ConstraintID, out.GetName(), out.GetID()) + + return errors.AssertionFailedf("failed to find FK constraint %d in descriptor %v", op.ConstraintID, tbl) } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go index fc68b4f2c4a3..b48cd255e350 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go @@ -227,6 +227,22 @@ func enqueueDropCheckConstraintMutation( return nil } +func enqueueAddForeignKeyConstraintMutation( + tbl *tabledesc.Mutable, fk *descpb.ForeignKeyConstraint, +) error { + tbl.AddForeignKeyMutation(fk, descpb.DescriptorMutation_ADD) + tbl.NextMutationID-- + return nil +} + +func enqueueDropForeignKeyConstraintMutation( + tbl *tabledesc.Mutable, fk *descpb.ForeignKeyConstraint, +) error { + tbl.AddForeignKeyMutation(fk, descpb.DescriptorMutation_DROP) + tbl.NextMutationID-- + return nil +} + func enqueueAddIndexMutation( tbl *tabledesc.Mutable, idx *descpb.IndexDescriptor, state descpb.DescriptorMutation_State, ) error { diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go index a4d994aa65ce..e2ab1ba671c2 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go @@ -91,29 +91,24 @@ func (m *visitor) RemoveForeignKeyBackReference( op.OriginConstraintID, out.GetName(), out.GetID()) } } - // Remove back reference. - var found bool + // Attempt to remove back reference. + // Note how we + // 1. only check to remove from `in.InboundFKs` but not from `in.Mutations`: + // this is because we only add the back-reference in `in` when we publish + // the adding FK in `out`, so it's impossible for a back-reference to exist + // on the mutation slice. + // 2. only attempt to remove (i.e. we do not panic when it's not found): + // this is because if we roll back before the adding FK is published in `out`, + // such a back-reference won't exist in `in` yet. for i, fk := range in.InboundFKs { if fk.OriginTableID == op.OriginTableID && fk.Name == name { in.InboundFKs = append(in.InboundFKs[:i], in.InboundFKs[i+1:]...) - found = true - break - } - } - for i, m := range in.Mutations { - if c := m.GetConstraint(); c != nil && - c.ConstraintType != descpb.ConstraintToUpdate_FOREIGN_KEY && - c.ForeignKey.OriginTableID == op.OriginTableID && - c.Name == name { - in.Mutations = append(in.Mutations[:i], in.Mutations[i+1:]...) - found = true + if len(in.InboundFKs) == 0 { + in.InboundFKs = nil + } break } } - if !found { - return errors.AssertionFailedf("foreign key %q not found in referenced table %q (%d)", - name, in.GetName(), in.GetID()) - } return nil } diff --git a/pkg/sql/schemachanger/scop/BUILD.bazel b/pkg/sql/schemachanger/scop/BUILD.bazel index 001dc2dc5b60..abf11840c257 100644 --- a/pkg/sql/schemachanger/scop/BUILD.bazel +++ b/pkg/sql/schemachanger/scop/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/scpb", + "//pkg/sql/sem/semenumpb", ], ) diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/mutation.go index a4dbe00327d3..dc22a4af5f53 100644 --- a/pkg/sql/schemachanger/scop/mutation.go +++ b/pkg/sql/schemachanger/scop/mutation.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" ) //go:generate go run ./generate_visitor.go scop Mutation mutation.go mutation_visitor_generated.go @@ -320,6 +321,37 @@ type MakeValidatedCheckConstraintPublic struct { ConstraintID descpb.ConstraintID } +// MakeAbsentForeignKeyConstraintWriteOnly adds a non-existent foreign key +// constraint to the table in the WRITE_ONLY state. +type MakeAbsentForeignKeyConstraintWriteOnly struct { + mutationOp + TableID descpb.ID + ConstraintID descpb.ConstraintID + ColumnIDs []descpb.ColumnID + ReferencedTableID descpb.ID + ReferencedColumnIDs []descpb.ColumnID + OnUpdateAction semenumpb.ForeignKeyAction + OnDeleteAction semenumpb.ForeignKeyAction + CompositeKeyMatchMethod semenumpb.Match +} + +// MakeValidatedForeignKeyConstraintPublic moves a new, validated foreign key +// constraint from mutation to public. +type MakeValidatedForeignKeyConstraintPublic struct { + mutationOp + TableID descpb.ID + ConstraintID descpb.ConstraintID + ReferencedTableID descpb.ID +} + +// MakePublicForeignKeyConstraintValidated moves a public +// check constraint to VALIDATED. +type MakePublicForeignKeyConstraintValidated struct { + mutationOp + TableID descpb.ID + ConstraintID descpb.ConstraintID +} + // RemoveForeignKeyConstraint removes a foreign key from the origin table. type RemoveForeignKeyConstraint struct { mutationOp diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go index 1c52669969bd..a947168976ed 100644 --- a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go @@ -58,6 +58,9 @@ type MutationVisitor interface { MakeAbsentCheckConstraintWriteOnly(context.Context, MakeAbsentCheckConstraintWriteOnly) error MakePublicCheckConstraintValidated(context.Context, MakePublicCheckConstraintValidated) error MakeValidatedCheckConstraintPublic(context.Context, MakeValidatedCheckConstraintPublic) error + MakeAbsentForeignKeyConstraintWriteOnly(context.Context, MakeAbsentForeignKeyConstraintWriteOnly) error + MakeValidatedForeignKeyConstraintPublic(context.Context, MakeValidatedForeignKeyConstraintPublic) error + MakePublicForeignKeyConstraintValidated(context.Context, MakePublicForeignKeyConstraintValidated) error RemoveForeignKeyConstraint(context.Context, RemoveForeignKeyConstraint) error RemoveForeignKeyBackReference(context.Context, RemoveForeignKeyBackReference) error RemoveSchemaParent(context.Context, RemoveSchemaParent) error @@ -282,6 +285,21 @@ func (op MakeValidatedCheckConstraintPublic) Visit(ctx context.Context, v Mutati return v.MakeValidatedCheckConstraintPublic(ctx, op) } +// Visit is part of the MutationOp interface. +func (op MakeAbsentForeignKeyConstraintWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeAbsentForeignKeyConstraintWriteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeValidatedForeignKeyConstraintPublic) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeValidatedForeignKeyConstraintPublic(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakePublicForeignKeyConstraintValidated) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakePublicForeignKeyConstraintValidated(ctx, op) +} + // Visit is part of the MutationOp interface. func (op RemoveForeignKeyConstraint) Visit(ctx context.Context, v MutationVisitor) error { return v.RemoveForeignKeyConstraint(ctx, op) diff --git a/pkg/sql/schemachanger/scop/validation.go b/pkg/sql/schemachanger/scop/validation.go index 7a4f2dcaf726..78b318478d0e 100644 --- a/pkg/sql/schemachanger/scop/validation.go +++ b/pkg/sql/schemachanger/scop/validation.go @@ -28,8 +28,8 @@ type ValidateIndex struct { IndexID descpb.IndexID } -// ValidateCheckConstraint validates a check constraint on a table's columns. -type ValidateCheckConstraint struct { +// ValidateConstraint validates a check constraint on a table's columns. +type ValidateConstraint struct { validationOp TableID descpb.ID ConstraintID descpb.ConstraintID diff --git a/pkg/sql/schemachanger/scop/validation_visitor_generated.go b/pkg/sql/schemachanger/scop/validation_visitor_generated.go index b40da5a9dfaf..d5ea13279b22 100644 --- a/pkg/sql/schemachanger/scop/validation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/validation_visitor_generated.go @@ -23,7 +23,7 @@ type ValidationOp interface { // ValidationVisitor is a visitor for ValidationOp operations. type ValidationVisitor interface { ValidateIndex(context.Context, ValidateIndex) error - ValidateCheckConstraint(context.Context, ValidateCheckConstraint) error + ValidateConstraint(context.Context, ValidateConstraint) error } // Visit is part of the ValidationOp interface. @@ -32,6 +32,6 @@ func (op ValidateIndex) Visit(ctx context.Context, v ValidationVisitor) error { } // Visit is part of the ValidationOp interface. -func (op ValidateCheckConstraint) Visit(ctx context.Context, v ValidationVisitor) error { - return v.ValidateCheckConstraint(ctx, op) +func (op ValidateConstraint) Visit(ctx context.Context, v ValidationVisitor) error { + return v.ValidateConstraint(ctx, op) } diff --git a/pkg/sql/schemachanger/scpb/BUILD.bazel b/pkg/sql/schemachanger/scpb/BUILD.bazel index 13c21257b40c..a7af7eb5096d 100644 --- a/pkg/sql/schemachanger/scpb/BUILD.bazel +++ b/pkg/sql/schemachanger/scpb/BUILD.bazel @@ -17,6 +17,7 @@ go_library( deps = [ "//pkg/sql/catalog/catpb", # keep "//pkg/sql/sem/catid", # keep + "//pkg/sql/sem/tree", # keep "//pkg/util/protoutil", "@com_github_cockroachdb_errors//:errors", ], @@ -33,6 +34,7 @@ go_proto_library( "//pkg/sql/catalog/catenumpb", "//pkg/sql/catalog/catpb", "//pkg/sql/sem/catid", # keep + "//pkg/sql/sem/semenumpb", "//pkg/sql/types", "@com_github_gogo_protobuf//gogoproto", ], @@ -50,6 +52,7 @@ proto_library( "//pkg/geo/geoindex:geoindex_proto", "//pkg/sql/catalog/catenumpb:catenumpb_proto", "//pkg/sql/catalog/catpb:catpb_proto", + "//pkg/sql/sem/semenumpb:semenumpb_proto", "//pkg/sql/types:types_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", ], diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index 7cc1efbe6b3c..e989b88a7db6 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -14,6 +14,7 @@ option go_package = "scpb"; import "sql/catalog/catenumpb/index.proto"; import "sql/catalog/catpb/catalog.proto"; +import "sql/sem/semenumpb/constraint.proto"; import "sql/types/types.proto"; import "gogoproto/gogo.proto"; import "geo/geoindex/config.proto"; @@ -325,6 +326,8 @@ message UniqueWithoutIndexConstraint { 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 { @@ -345,6 +348,12 @@ message ForeignKeyConstraint { repeated uint32 column_ids = 3 [(gogoproto.customname) = "ColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"]; uint32 referenced_table_id = 4 [(gogoproto.customname) = "ReferencedTableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; repeated uint32 referenced_column_ids = 5 [(gogoproto.customname) = "ReferencedColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"]; + cockroach.sql.sem.semenumpb.ForeignKeyAction on_update_action = 6 [(gogoproto.customname) = "OnUpdateAction"]; + cockroach.sql.sem.semenumpb.ForeignKeyAction on_delete_action = 7 [(gogoproto.customname) = "OnDeleteAction"]; + cockroach.sql.sem.semenumpb.Match composite_key_match_method = 8 [(gogoproto.customname) = "CompositeKeyMatchMethod"]; + // IndexIDForValidation is the index id to hint to the foreign key constraint validation SQL query about which index + // to validate against. It is used exclusively by sql.validateFKExpr. + uint32 index_id_for_validation = 9 [(gogoproto.customname) = "IndexIDForValidation", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.IndexID"]; } message EnumType { diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index eacc83583c74..d9e5205dfaa5 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -80,6 +80,7 @@ object UniqueWithoutIndexConstraint UniqueWithoutIndexConstraint : TableID UniqueWithoutIndexConstraint : ConstraintID UniqueWithoutIndexConstraint : []ColumnIDs +UniqueWithoutIndexConstraint : Predicate object CheckConstraint @@ -97,6 +98,10 @@ ForeignKeyConstraint : ConstraintID ForeignKeyConstraint : []ColumnIDs ForeignKeyConstraint : ReferencedTableID ForeignKeyConstraint : []ReferencedColumnIDs +ForeignKeyConstraint : OnUpdateAction +ForeignKeyConstraint : OnDeleteAction +ForeignKeyConstraint : CompositeKeyMatchMethod +ForeignKeyConstraint : IndexIDForValidation object TableComment diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go index addaca2d893d..3d3687195e9a 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go @@ -49,8 +49,8 @@ func init() { }), ), to(scpb.Status_VALIDATED, - emit(func(this *scpb.CheckConstraint) *scop.ValidateCheckConstraint { - return &scop.ValidateCheckConstraint{ + emit(func(this *scpb.CheckConstraint) *scop.ValidateConstraint { + return &scop.ValidateConstraint{ TableID: this.TableID, ConstraintID: this.ConstraintID, IndexIDForValidation: this.IndexIDForValidation, diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go index fa71fd44bfe1..52d514a6ee72 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go @@ -19,14 +19,50 @@ func init() { opRegistry.register((*scpb.ForeignKeyConstraint)(nil), toPublic( scpb.Status_ABSENT, + to(scpb.Status_WRITE_ONLY, + emit(func(this *scpb.ForeignKeyConstraint) *scop.MakeAbsentForeignKeyConstraintWriteOnly { + return &scop.MakeAbsentForeignKeyConstraintWriteOnly{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + ColumnIDs: this.ColumnIDs, + ReferencedTableID: this.ReferencedTableID, + ReferencedColumnIDs: this.ReferencedColumnIDs, + OnUpdateAction: this.OnUpdateAction, + OnDeleteAction: this.OnDeleteAction, + CompositeKeyMatchMethod: this.CompositeKeyMatchMethod, + } + }), + ), + to(scpb.Status_VALIDATED, + emit(func(this *scpb.ForeignKeyConstraint) *scop.ValidateConstraint { + return &scop.ValidateConstraint{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + IndexIDForValidation: this.IndexIDForValidation, + } + }), + ), to(scpb.Status_PUBLIC, - emit(func(this *scpb.ForeignKeyConstraint) *scop.NotImplemented { - return notImplemented(this) + emit(func(this *scpb.ForeignKeyConstraint) *scop.MakeValidatedForeignKeyConstraintPublic { + return &scop.MakeValidatedForeignKeyConstraintPublic{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + ReferencedTableID: this.ReferencedTableID, + } }), ), ), toAbsent( scpb.Status_PUBLIC, + to(scpb.Status_VALIDATED, + emit(func(this *scpb.ForeignKeyConstraint) *scop.MakePublicForeignKeyConstraintValidated { + return &scop.MakePublicForeignKeyConstraintValidated{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + } + }), + ), + equiv(scpb.Status_WRITE_ONLY), to(scpb.Status_ABSENT, // TODO(postamar): remove revertibility constraint when possible revertible(false), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/assertions_test.go b/pkg/sql/schemachanger/scplan/internal/rules/assertions_test.go index a45781026769..54ea5de42c6c 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/assertions_test.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/assertions_test.go @@ -58,10 +58,15 @@ func nonNilElement(element scpb.Element) scpb.Element { // Assert that only simple dependents (non-descriptor, non-index, non-column) // and data elements have screl.ReferencedDescID attributes. +// One exception is foreign key constraint, which is not simple dependent nor data +// element but it has a screl.ReferencedDescID attribute. func checkSimpleDependentsReferenceDescID(e scpb.Element) error { if isSimpleDependent(e) || isData(e) { return nil } + if _, ok := e.(*scpb.ForeignKeyConstraint); ok { + return nil + } if _, err := screl.Schema.GetAttribute(screl.ReferencedDescID, e); err == nil { return errors.New("unexpected screl.ReferencedDescID attr") } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index c865493927cc..82fa65e55bb4 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -374,8 +374,11 @@ func isIndexDependent(e scpb.Element) bool { // TODO (xiang): Expand this predicate to include other non-index-backed constraints // when we properly support adding/dropping them in the new schema changer. func isSupportedNonIndexBackedConstraint(e scpb.Element) bool { - _, ok := e.(*scpb.CheckConstraint) - return ok + switch e.(type) { + case *scpb.CheckConstraint, *scpb.ForeignKeyConstraint: + return true + } + return false } func isConstraint(e scpb.Element) bool { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go b/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go index 783bc4481815..c7c565b0db23 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/op_drop.go @@ -169,6 +169,7 @@ func init() { ), constraint.Type( (*scpb.CheckConstraint)(nil), + (*scpb.ForeignKeyConstraint)(nil), (*scpb.UniqueWithoutIndexConstraint)(nil), ), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules index fcfb53ad617d..6644c695a33d 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules @@ -317,6 +317,103 @@ deprules - $column-node[CurrentStatus] = WRITE_ONLY - joinTargetNode($expr, $expr-target, $expr-node) - joinTargetNode($column, $column-target, $column-node) +- name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.ForeignKeyConstraint' + - $next[Type] = '*scpb.ForeignKeyConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = ABSENT + - $prev-node[CurrentStatus] = PUBLIC + - $next-node[CurrentStatus] = VALIDATED + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.ForeignKeyConstraint' + - $next[Type] = '*scpb.ForeignKeyConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = ABSENT + - $prev-node[CurrentStatus] = VALIDATED + - $next-node[CurrentStatus] = ABSENT + - descriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-target) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.ForeignKeyConstraint' + - $next[Type] = '*scpb.ForeignKeyConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = ABSENT + - $prev-node[CurrentStatus] = WRITE_ONLY + - $next-node[CurrentStatus] = VALIDATED + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.ForeignKeyConstraint' + - $next[Type] = '*scpb.ForeignKeyConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = PUBLIC + - $prev-node[CurrentStatus] = ABSENT + - $next-node[CurrentStatus] = WRITE_ONLY + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.ForeignKeyConstraint' + - $next[Type] = '*scpb.ForeignKeyConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = PUBLIC + - $prev-node[CurrentStatus] = VALIDATED + - $next-node[CurrentStatus] = PUBLIC + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.ForeignKeyConstraint' + - $next[Type] = '*scpb.ForeignKeyConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = PUBLIC + - $prev-node[CurrentStatus] = WRITE_ONLY + - $next-node[CurrentStatus] = VALIDATED + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' from: prev-node kind: PreviousTransactionPrecedence @@ -1438,10 +1535,9 @@ deprules - $column[Type] = '*scpb.Column' - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - joinOnColumnID($column, $dependent, $table-id, $col-id) - - $column-target[TargetStatus] = ABSENT + - toAbsent($column-target, $dependent-target) - $column-node[CurrentStatus] = WRITE_ONLY - - $dependent-target[TargetStatus] = TRANSIENT_ABSENT - - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $dependent-node[CurrentStatus] = ABSENT - joinTargetNode($column, $column-target, $column-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) - name: column no longer public before dependents @@ -1452,9 +1548,9 @@ deprules - $column[Type] = '*scpb.Column' - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - joinOnColumnID($column, $dependent, $table-id, $col-id) - - toAbsent($column-target, $dependent-target) - - $column-node[CurrentStatus] = WRITE_ONLY - - $dependent-node[CurrentStatus] = ABSENT + - transient($column-target, $dependent-target) + - $column-node[CurrentStatus] = TRANSIENT_WRITE_ONLY + - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($column, $column-target, $column-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) - name: column no longer public before dependents @@ -1465,9 +1561,10 @@ deprules - $column[Type] = '*scpb.Column' - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - joinOnColumnID($column, $dependent, $table-id, $col-id) - - transient($column-target, $dependent-target) + - $column-target[TargetStatus] = TRANSIENT_ABSENT - $column-node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $dependent-target[TargetStatus] = ABSENT + - $dependent-node[CurrentStatus] = ABSENT - joinTargetNode($column, $column-target, $column-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) - name: column no longer public before dependents @@ -1478,10 +1575,10 @@ deprules - $column[Type] = '*scpb.Column' - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - joinOnColumnID($column, $dependent, $table-id, $col-id) - - $column-target[TargetStatus] = TRANSIENT_ABSENT - - $column-node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - $dependent-target[TargetStatus] = ABSENT - - $dependent-node[CurrentStatus] = ABSENT + - $column-target[TargetStatus] = ABSENT + - $column-node[CurrentStatus] = WRITE_ONLY + - $dependent-target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($column, $column-target, $column-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) - name: column type dependents removed right before column type @@ -1529,12 +1626,11 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - $constraint-target[TargetStatus] = ABSENT - - $constraint-node[CurrentStatus] = VALIDATED - - $dependent-target[TargetStatus] = TRANSIENT_ABSENT + - transient($constraint-target, $dependent-target) + - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) @@ -1543,11 +1639,12 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - toAbsent($constraint-target, $dependent-target) - - $constraint-node[CurrentStatus] = VALIDATED + - $constraint-target[TargetStatus] = TRANSIENT_ABSENT + - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED + - $dependent-target[TargetStatus] = ABSENT - $dependent-node[CurrentStatus] = ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) @@ -1556,12 +1653,11 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - $constraint-target[TargetStatus] = TRANSIENT_ABSENT - - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED - - $dependent-target[TargetStatus] = ABSENT + - toAbsent($constraint-target, $dependent-target) + - $constraint-node[CurrentStatus] = VALIDATED - $dependent-node[CurrentStatus] = ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) @@ -1570,11 +1666,12 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - transient($constraint-target, $dependent-target) - - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED + - $constraint-target[TargetStatus] = ABSENT + - $constraint-node[CurrentStatus] = VALIDATED + - $dependent-target[TargetStatus] = TRANSIENT_ABSENT - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) @@ -1586,8 +1683,9 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - transient($view-target, $index-target) - - $view-node[CurrentStatus] = TRANSIENT_ABSENT + - $view-target[TargetStatus] = ABSENT + - $view-node[CurrentStatus] = ABSENT + - $index-target[TargetStatus] = TRANSIENT_ABSENT - $index-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) @@ -1599,8 +1697,9 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - toAbsent($view-target, $index-target) - - $view-node[CurrentStatus] = ABSENT + - $view-target[TargetStatus] = TRANSIENT_ABSENT + - $view-node[CurrentStatus] = TRANSIENT_ABSENT + - $index-target[TargetStatus] = ABSENT - $index-node[CurrentStatus] = ABSENT - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) @@ -1612,10 +1711,9 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - $view-target[TargetStatus] = ABSENT + - toAbsent($view-target, $index-target) - $view-node[CurrentStatus] = ABSENT - - $index-target[TargetStatus] = TRANSIENT_ABSENT - - $index-node[CurrentStatus] = TRANSIENT_ABSENT + - $index-node[CurrentStatus] = ABSENT - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependent view absent before secondary index @@ -1626,10 +1724,9 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - $view-target[TargetStatus] = TRANSIENT_ABSENT + - transient($view-target, $index-target) - $view-node[CurrentStatus] = TRANSIENT_ABSENT - - $index-target[TargetStatus] = ABSENT - - $index-node[CurrentStatus] = ABSENT + - $index-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependent view no longer public before secondary index @@ -1667,10 +1764,9 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - $view-target[TargetStatus] = TRANSIENT_ABSENT + - transient($view-target, $index-target) - $view-node[CurrentStatus] = TRANSIENT_DROPPED - - $index-target[TargetStatus] = ABSENT - - $index-node[CurrentStatus] = VALIDATED + - $index-node[CurrentStatus] = TRANSIENT_VALIDATED - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependent view no longer public before secondary index @@ -1681,9 +1777,10 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - transient($view-target, $index-target) + - $view-target[TargetStatus] = TRANSIENT_ABSENT - $view-node[CurrentStatus] = TRANSIENT_DROPPED - - $index-node[CurrentStatus] = TRANSIENT_VALIDATED + - $index-target[TargetStatus] = ABSENT + - $index-node[CurrentStatus] = VALIDATED - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependents removed before column @@ -1694,9 +1791,9 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - toAbsent($dependent-target, $column-target) - - $dependent-node[CurrentStatus] = ABSENT - - $column-node[CurrentStatus] = ABSENT + - transient($dependent-target, $column-target) + - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $column-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) - name: dependents removed before column @@ -1707,9 +1804,9 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - transient($dependent-target, $column-target) - - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - - $column-node[CurrentStatus] = TRANSIENT_ABSENT + - toAbsent($dependent-target, $column-target) + - $dependent-node[CurrentStatus] = ABSENT + - $column-node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) - name: dependents removed before column @@ -1720,10 +1817,10 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - $dependent-target[TargetStatus] = ABSENT - - $dependent-node[CurrentStatus] = ABSENT - - $column-target[TargetStatus] = TRANSIENT_ABSENT - - $column-node[CurrentStatus] = TRANSIENT_ABSENT + - $dependent-target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $column-target[TargetStatus] = ABSENT + - $column-node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) - name: dependents removed before column @@ -1734,10 +1831,10 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - $dependent-target[TargetStatus] = TRANSIENT_ABSENT - - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - - $column-target[TargetStatus] = ABSENT - - $column-node[CurrentStatus] = ABSENT + - $dependent-target[TargetStatus] = ABSENT + - $dependent-node[CurrentStatus] = ABSENT + - $column-target[TargetStatus] = TRANSIENT_ABSENT + - $column-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) - name: dependents removed before constraint @@ -1746,11 +1843,10 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - - $dependents-target[TargetStatus] = TRANSIENT_ABSENT - - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT - - $constraint-target[TargetStatus] = ABSENT + - toAbsent($dependents-target, $constraint-target) + - $dependents-node[CurrentStatus] = ABSENT - $constraint-node[CurrentStatus] = ABSENT - joinTargetNode($dependents, $dependents-target, $dependents-node) - joinTargetNode($constraint, $constraint-target, $constraint-node) @@ -1760,7 +1856,7 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - transient($dependents-target, $constraint-target) - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT @@ -1773,10 +1869,11 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - - toAbsent($dependents-target, $constraint-target) - - $dependents-node[CurrentStatus] = ABSENT + - $dependents-target[TargetStatus] = TRANSIENT_ABSENT + - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT + - $constraint-target[TargetStatus] = ABSENT - $constraint-node[CurrentStatus] = ABSENT - joinTargetNode($dependents, $dependents-target, $dependents-node) - joinTargetNode($constraint, $constraint-target, $constraint-node) @@ -1786,7 +1883,7 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - $dependents-target[TargetStatus] = ABSENT - $dependents-node[CurrentStatus] = ABSENT @@ -1802,9 +1899,10 @@ deprules - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] - joinOnIndexID($dependent, $index, $table-id, $index-id) - - transient($dependent-target, $index-target) + - $dependent-target[TargetStatus] = TRANSIENT_ABSENT - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - - $index-node[CurrentStatus] = TRANSIENT_ABSENT + - $index-target[TargetStatus] = ABSENT + - $index-node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($index, $index-target, $index-node) - name: dependents removed before index @@ -1828,10 +1926,9 @@ deprules - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] - joinOnIndexID($dependent, $index, $table-id, $index-id) - - $dependent-target[TargetStatus] = TRANSIENT_ABSENT + - transient($dependent-target, $index-target) - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - - $index-target[TargetStatus] = ABSENT - - $index-node[CurrentStatus] = ABSENT + - $index-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($index, $index-target, $index-node) - name: dependents removed before index @@ -1880,7 +1977,7 @@ deprules to: dependent-node query: - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] + - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-target, $dependent-target) - $descriptor-node[CurrentStatus] = DROPPED @@ -1894,7 +1991,7 @@ deprules to: referencing-via-attr-node query: - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] + - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-target, $referencing-via-attr-target) - $referenced-descriptor-node[CurrentStatus] = DROPPED @@ -2002,7 +2099,7 @@ deprules to: constraint-node query: - $index[Type] = '*scpb.PrimaryIndex' - - $constraint[Type] = '*scpb.CheckConstraint' + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnDescID($index, $constraint, $table-id) - $index[IndexID] = $index-id-for-validation - $constraint[IndexID] = $index-id-for-validation diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules index cc1257990463..cdccc24693da 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules @@ -154,7 +154,7 @@ oprules from: constraint-node query: - $relation[Type] IN ['*scpb.Table', '*scpb.View'] - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.UniqueWithoutIndexConstraint'] + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.UniqueWithoutIndexConstraint'] - joinOnDescID($relation, $constraint, $relation-id) - joinTarget($relation, $relation-target) - $relation-target[TargetStatus] = ABSENT diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check index 4c43a1e60aba..b797c2a58843 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check @@ -32,7 +32,7 @@ PostCommitPhase stage 1 of 2 with 1 ValidationType op transitions: [[CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, PUBLIC], WRITE_ONLY] -> VALIDATED ops: - *scop.ValidateCheckConstraint + *scop.ValidateConstraint ConstraintID: 2 TableID: 104 PostCommitPhase stage 2 of 2 with 4 MutationType ops diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index b49356631f92..fb51ce462576 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -41,6 +41,8 @@ StatementPhase stage 1 of 1 with 3 MutationType ops [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], PUBLIC] -> VALIDATED + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], PUBLIC] -> VALIDATED [[Sequence:{DescID: 110}, ABSENT], PUBLIC] -> TXN_DROPPED [[View:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED [[Column:{DescID: 111, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY @@ -103,10 +105,10 @@ PreCommitPhase stage 1 of 1 with 42 MutationType ops [[SecondaryIndexPartial:{DescID: 109, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> ABSENT [[IndexName:{DescID: 109, Name: partialidx, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ForeignKeyConstraint:{DescID: 109, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], VALIDATED] -> ABSENT [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT [[ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT], PUBLIC] -> ABSENT - [[ForeignKeyConstraint:{DescID: 109, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], VALIDATED] -> ABSENT [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 110, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT @@ -159,20 +161,6 @@ PreCommitPhase stage 1 of 1 with 42 MutationType ops *scop.RemoveDroppedIndexPartialPredicate IndexID: 2 TableID: 109 - *scop.RemoveForeignKeyBackReference - OriginConstraintID: 2 - OriginTableID: 109 - ReferencedTableID: 104 - *scop.RemoveForeignKeyConstraint - ConstraintID: 2 - TableID: 109 - *scop.RemoveForeignKeyBackReference - OriginConstraintID: 3 - OriginTableID: 109 - ReferencedTableID: 105 - *scop.RemoveForeignKeyConstraint - ConstraintID: 3 - TableID: 109 *scop.MarkDescriptorAsDropped DescriptorID: 110 *scop.RemoveAllTableComments @@ -241,6 +229,20 @@ PreCommitPhase stage 1 of 1 with 42 MutationType ops SourceElementID: 1 SubWorkID: 1 TableID: 109 + *scop.RemoveForeignKeyBackReference + OriginConstraintID: 2 + OriginTableID: 109 + ReferencedTableID: 104 + *scop.RemoveForeignKeyConstraint + ConstraintID: 2 + TableID: 109 + *scop.RemoveForeignKeyBackReference + OriginConstraintID: 3 + OriginTableID: 109 + ReferencedTableID: 105 + *scop.RemoveForeignKeyConstraint + ConstraintID: 3 + TableID: 109 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -735,6 +737,30 @@ DROP TABLE defaultdb.shipments CASCADE; to: [Column:{DescID: 111, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: dependents removed before column +- from: [ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT] + to: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT] + kind: Precedence + rule: dependents removed before constraint +- from: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT] + to: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT] + kind: Precedence + rule: dependents removed before constraint +- from: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT] + to: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT] + kind: Precedence + rule: dependents removed before constraint +- from: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, VALIDATED] + to: [ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT] + kind: Precedence + rule: constraint no longer public before dependents +- from: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, VALIDATED] + to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT] + kind: Precedence + rule: constraint no longer public before dependents +- from: [ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, VALIDATED] + to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT] + kind: Precedence + rule: constraint no longer public before dependents - from: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT] to: [Column:{DescID: 109, ColumnID: 1}, ABSENT] kind: Precedence @@ -1011,14 +1037,6 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT] kind: Precedence rule: descriptor drop right before dependent element removal -- from: [Table:{DescID: 109}, DROPPED] - to: [ForeignKeyConstraint:{DescID: 109, ConstraintID: 2, ReferencedDescID: 104}, ABSENT] - kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Table:{DescID: 109}, DROPPED] - to: [ForeignKeyConstraint:{DescID: 109, ConstraintID: 3, ReferencedDescID: 105}, ABSENT] - kind: Precedence - rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT] kind: Precedence diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index d7afad361c0b..68a50141b17e 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -189,6 +189,7 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityAttr(DescID, "TableID"), rel.EntityAttr(ReferencedDescID, "ReferencedTableID"), rel.EntityAttr(ConstraintID, "ConstraintID"), + rel.EntityAttr(IndexID, "IndexIDForValidation"), ), rel.EntityMapping(t((*scpb.RowLevelTTL)(nil)), rel.EntityAttr(DescID, "TableID"), diff --git a/pkg/sql/schemachanger/screl/query_test.go b/pkg/sql/schemachanger/screl/query_test.go index 212f2c7a1b17..fea2133c30fe 100644 --- a/pkg/sql/schemachanger/screl/query_test.go +++ b/pkg/sql/schemachanger/screl/query_test.go @@ -131,13 +131,13 @@ func TestQueryBasic(t *testing.T) { exp: []string{` [[Table:{DescID: 2}, PUBLIC], ABSENT] [[Sequence:{DescID: 1}, PUBLIC], ABSENT] -[[ForeignKeyConstraint:{DescID: 2, ConstraintID: 0, ReferencedDescID: 1}, PUBLIC], ABSENT]`, ` +[[ForeignKeyConstraint:{DescID: 2, IndexID: 0, ConstraintID: 0, ReferencedDescID: 1}, PUBLIC], ABSENT]`, ` [[Table:{DescID: 2}, PUBLIC], PUBLIC] [[Sequence:{DescID: 1}, PUBLIC], PUBLIC] -[[ForeignKeyConstraint:{DescID: 2, ConstraintID: 0, ReferencedDescID: 1}, PUBLIC], PUBLIC]`, ` +[[ForeignKeyConstraint:{DescID: 2, IndexID: 0, ConstraintID: 0, ReferencedDescID: 1}, PUBLIC], PUBLIC]`, ` [[Table:{DescID: 4}, PUBLIC], ABSENT] [[Sequence:{DescID: 3}, PUBLIC], ABSENT] -[[ForeignKeyConstraint:{DescID: 4, ConstraintID: 0, ReferencedDescID: 3}, PUBLIC], ABSENT]`, +[[ForeignKeyConstraint:{DescID: 4, IndexID: 0, ConstraintID: 0, ReferencedDescID: 3}, PUBLIC], ABSENT]`, }, }, }, diff --git a/pkg/sql/schemachanger/sctest_generated_test.go b/pkg/sql/schemachanger/sctest_generated_test.go index f096688e360e..236392efb499 100644 --- a/pkg/sql/schemachanger/sctest_generated_test.go +++ b/pkg/sql/schemachanger/sctest_generated_test.go @@ -170,6 +170,31 @@ func TestRollback_alter_table_add_check_with_seq_and_udt(t *testing.T) { defer log.Scope(t).Close(t) sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt", sctest.SingleNodeCluster) } +func TestEndToEndSideEffects_alter_table_add_foreign_key(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.EndToEndSideEffects(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key", sctest.SingleNodeCluster) +} +func TestExecuteWithDMLInjection_alter_table_add_foreign_key(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.ExecuteWithDMLInjection(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key", sctest.SingleNodeCluster) +} +func TestGenerateSchemaChangeCorpus_alter_table_add_foreign_key(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.GenerateSchemaChangeCorpus(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key", sctest.SingleNodeCluster) +} +func TestPause_alter_table_add_foreign_key(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Pause(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key", sctest.SingleNodeCluster) +} +func TestRollback_alter_table_add_foreign_key(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key", sctest.SingleNodeCluster) +} func TestEndToEndSideEffects_alter_table_add_primary_key_drop_rowid(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla index 764994fce54a..65a8b55289ec 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla @@ -81,7 +81,7 @@ begin transaction #2 commit transaction #2 begin transaction #3 ## PostCommitPhase stage 1 of 2 with 1 ValidationType op -validate check constraint crdb_internal_constraint_2_name_placeholder in table #104 +validate CHECK constraint crdb_internal_constraint_2_name_placeholder in table #104 commit transaction #3 begin transaction #4 ## PostCommitPhase stage 2 of 2 with 4 MutationType ops diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt index f8acf42a1897..9d23c02e47b6 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt @@ -164,7 +164,7 @@ begin transaction #2 commit transaction #2 begin transaction #3 ## PostCommitPhase stage 1 of 2 with 1 ValidationType op -validate check constraint crdb_internal_constraint_2_name_placeholder in table #107 +validate CHECK constraint crdb_internal_constraint_2_name_placeholder in table #107 commit transaction #3 begin transaction #4 ## PostCommitPhase stage 2 of 2 with 7 MutationType ops diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key new file mode 100644 index 000000000000..7e9587d48380 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key @@ -0,0 +1,208 @@ +setup +CREATE TABLE t1 (i INT PRIMARY KEY); +CREATE TABLE t2 (i INT PRIMARY KEY) +---- +... ++object {100 101 t1} -> 104 ++object {100 101 t2} -> 105 + +test +ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i) +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.add_constraint +## StatementPhase stage 1 of 1 with 1 MutationType op +upsert descriptor #104 + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: FOREIGN_KEY + + foreignKey: + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + originColumnIds: + + - 1 + + originTableId: 104 + + referencedColumnIds: + + - 1 + + referencedTableId: 105 + + validity: Validating + + name: crdb_internal_constraint_2_name_placeholder + + uniqueWithoutIndexConstraint: {} + + direction: ADD + + mutationId: 1 + + state: WRITE_ONLY + name: t1 + nextColumnId: 2 + - nextConstraintId: 2 + + nextConstraintId: 3 + nextFamilyId: 1 + nextIndexId: 2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 3 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CONSTRAINT ‹t1_i_fkey› + + FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.‹public›.‹t2› (‹i›) + + statement: ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2 (i) + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + targets: + families: + - columnIds: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +upsert descriptor #105 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + jobId: "1" + + revertible: true + families: + - columnIds: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t1 ADD CONSTRAINT t1_i_fkey FOREIGN KEY (i) REFERENCES defaultdb.public.t2 (i)" + descriptor IDs: [104 105] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 2 with 1 ValidationType op +validate FOREIGN KEY constraint crdb_internal_constraint_2_name_placeholder in table #104 +commit transaction #3 +begin transaction #4 +## PostCommitPhase stage 2 of 2 with 5 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CONSTRAINT ‹t1_i_fkey› + - FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.‹public›.‹t2› (‹i›) + - statement: ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2 (i) + - statementTag: ALTER TABLE + - revertible: true + - targetRanks: + - targets: + families: + - columnIds: + ... + id: 104 + modificationTime: {} + - mutations: + - - constraint: + - check: {} + - constraintType: FOREIGN_KEY + - foreignKey: + - constraintId: 2 + - name: crdb_internal_constraint_2_name_placeholder + - originColumnIds: + - - 1 + - originTableId: 104 + - referencedColumnIds: + - - 1 + - referencedTableId: 105 + - validity: Validating + - name: crdb_internal_constraint_2_name_placeholder + - uniqueWithoutIndexConstraint: {} + - direction: ADD + - mutationId: 1 + - state: WRITE_ONLY + name: t1 + nextColumnId: 2 + ... + nextIndexId: 2 + nextMutationId: 1 + + outboundFks: + + - constraintId: 2 + + name: t1_i_fkey + + originColumnIds: + + - 1 + + originTableId: 104 + + referencedColumnIds: + + - 1 + + referencedTableId: 105 + parentId: 100 + primaryIndex: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +upsert descriptor #105 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - jobId: "1" + - revertible: true + families: + - columnIds: + ... + formatVersion: 3 + id: 105 + + inboundFks: + + - constraintId: 2 + + name: t1_i_fkey + + originColumnIds: + + - 1 + + originTableId: 104 + + referencedColumnIds: + + - 1 + + referencedTableId: 105 + + validity: Validating + modificationTime: {} + name: t2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + version: "3" +update progress of schema change job #1: "all stages completed" +set schema change job #1 to non-cancellable +updated schema change job #1 descriptor IDs to [] +commit transaction #4 +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla index 3662c3336071..c3de41458f8b 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla @@ -22,7 +22,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CHEC │ ├── 1 element transitioning toward PUBLIC │ │ └── WRITE_ONLY → VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ └── 1 Validation operation - │ └── ValidateCheckConstraint {"ConstraintID":2,"TableID":104} + │ └── ValidateConstraint {"ConstraintID":2,"TableID":104} └── Stage 2 of 2 in PostCommitPhase ├── 2 elements transitioning toward PUBLIC │ ├── VALIDATED → PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt index b38159abb564..cfbd152944c5 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt @@ -28,7 +28,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CHEC │ ├── 1 element transitioning toward PUBLIC │ │ └── WRITE_ONLY → VALIDATED CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ └── 1 Validation operation - │ └── ValidateCheckConstraint {"ConstraintID":2,"TableID":107} + │ └── ValidateConstraint {"ConstraintID":2,"TableID":107} └── Stage 2 of 2 in PostCommitPhase ├── 2 elements transitioning toward PUBLIC │ ├── VALIDATED → PUBLIC CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key new file mode 100644 index 000000000000..f1e50a614857 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key @@ -0,0 +1,36 @@ +/* setup */ +CREATE TABLE t1 (i INT PRIMARY KEY); +CREATE TABLE t2 (i INT PRIMARY KEY); + +/* test */ +EXPLAIN (ddl) ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CONSTRAINT ‹t1_i_fkey› FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.‹public›.‹t2› (‹i›); + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── ABSENT → WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ └── 1 Mutation operation + │ └── MakeAbsentForeignKeyConstraintWriteOnly {"ConstraintID":2,"ReferencedTableID":105,"TableID":104} + ├── PreCommitPhase + │ └── Stage 1 of 1 in PreCommitPhase + │ └── 3 Mutation operations + │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} + │ ├── SetJobStateOnDescriptor {"DescriptorID":105,"Initialize":true} + │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} + └── PostCommitPhase + ├── Stage 1 of 2 in PostCommitPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── WRITE_ONLY → VALIDATED ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ └── 1 Validation operation + │ └── ValidateConstraint {"ConstraintID":2,"TableID":104} + └── Stage 2 of 2 in PostCommitPhase + ├── 2 elements transitioning toward PUBLIC + │ ├── VALIDATED → PUBLIC ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ └── ABSENT → PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} + └── 5 Mutation operations + ├── SetConstraintName {"ConstraintID":2,"Name":"t1_i_fkey","TableID":104} + ├── MakeValidatedForeignKeyConstraintPublic {"ConstraintID":2,"ReferencedTableID":105,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":105} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key.rollback_1_of_2 new file mode 100644 index 000000000000..f135aa1e6247 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key.rollback_1_of_2 @@ -0,0 +1,19 @@ +/* setup */ +CREATE TABLE t1 (i INT PRIMARY KEY); +CREATE TABLE t2 (i INT PRIMARY KEY); + +/* test */ +ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); +EXPLAIN (ddl) rollback at post-commit stage 1 of 2; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t1› ADD CONSTRAINT ‹t1_i_fkey› FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.public.‹t2› (‹i›); + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── WRITE_ONLY → ABSENT ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + └── 5 Mutation operations + ├── RemoveForeignKeyBackReference {"OriginConstraintID":2,"OriginTableID":104,"ReferencedTableID":105} + ├── RemoveForeignKeyConstraint {"ConstraintID":2,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":105} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key.rollback_2_of_2 new file mode 100644 index 000000000000..66f0b3181bf8 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key.rollback_2_of_2 @@ -0,0 +1,19 @@ +/* setup */ +CREATE TABLE t1 (i INT PRIMARY KEY); +CREATE TABLE t2 (i INT PRIMARY KEY); + +/* test */ +ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); +EXPLAIN (ddl) rollback at post-commit stage 2 of 2; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t1› ADD CONSTRAINT ‹t1_i_fkey› FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.public.‹t2› (‹i›); + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── WRITE_ONLY → ABSENT ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + └── 5 Mutation operations + ├── RemoveForeignKeyBackReference {"OriginConstraintID":2,"OriginTableID":104,"ReferencedTableID":105} + ├── RemoveForeignKeyConstraint {"ConstraintID":2,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":105} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla index 907a8da13276..7ae32e037330 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla @@ -60,7 +60,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > 0) │ │ │ └── • 1 Validation operation │ │ - │ └── • ValidateCheckConstraint + │ └── • ValidateConstraint │ ConstraintID: 2 │ TableID: 104 │ diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt index f7f7ab6c835d..f33841786156 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt @@ -88,7 +88,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > nextval('s') OR j::typ = 'a' │ │ │ └── • 1 Validation operation │ │ - │ └── • ValidateCheckConstraint + │ └── • ValidateConstraint │ ConstraintID: 2 │ TableID: 107 │ diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key new file mode 100644 index 000000000000..0ff6b0de9326 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key @@ -0,0 +1,121 @@ +/* setup */ +CREATE TABLE t1 (i INT PRIMARY KEY); +CREATE TABLE t2 (i INT PRIMARY KEY); + +/* test */ +EXPLAIN (ddl, verbose) ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); +---- +• Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CONSTRAINT ‹t1_i_fkey› FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.‹public›.‹t2› (‹i›); +│ +├── • StatementPhase +│ │ +│ └── • Stage 1 of 1 in StatementPhase +│ │ +│ ├── • 1 element transitioning toward PUBLIC +│ │ │ +│ │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} +│ │ │ ABSENT → WRITE_ONLY +│ │ │ +│ │ └── • PreviousTransactionPrecedence dependency from ABSENT ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} +│ │ rule: "ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" +│ │ +│ └── • 1 Mutation operation +│ │ +│ └── • MakeAbsentForeignKeyConstraintWriteOnly +│ ColumnIDs: +│ - 1 +│ ConstraintID: 2 +│ ReferencedColumnIDs: +│ - 1 +│ ReferencedTableID: 105 +│ TableID: 104 +│ +├── • PreCommitPhase +│ │ +│ └── • Stage 1 of 1 in PreCommitPhase +│ │ +│ └── • 3 Mutation operations +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 104 +│ │ Initialize: true +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 105 +│ │ Initialize: true +│ │ +│ └── • CreateSchemaChangerJob +│ Authorization: +│ UserName: root +│ DescriptorIDs: +│ - 104 +│ - 105 +│ JobID: 1 +│ RunningStatus: PostCommitPhase stage 1 of 2 with 1 ValidationType op pending +│ Statements: +│ - statement: ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2 (i) +│ redactedstatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CONSTRAINT ‹t1_i_fkey› +│ FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.‹public›.‹t2› (‹i›) +│ statementtag: ALTER TABLE +│ +└── • PostCommitPhase + │ + ├── • Stage 1 of 2 in PostCommitPhase + │ │ + │ ├── • 1 element transitioning toward PUBLIC + │ │ │ + │ │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ │ WRITE_ONLY → VALIDATED + │ │ │ + │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ rule: "ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" + │ │ + │ └── • 1 Validation operation + │ │ + │ └── • ValidateConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + └── • Stage 2 of 2 in PostCommitPhase + │ + ├── • 2 elements transitioning toward PUBLIC + │ │ + │ ├── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ │ VALIDATED → PUBLIC + │ │ │ + │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ │ rule: "ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" + │ │ │ + │ │ └── • SameStagePrecedence dependency from PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} + │ │ rule: "constraint dependent public right before constraint" + │ │ + │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} + │ ABSENT → PUBLIC + │ + └── • 5 Mutation operations + │ + ├── • SetConstraintName + │ ConstraintID: 2 + │ Name: t1_i_fkey + │ TableID: 104 + │ + ├── • MakeValidatedForeignKeyConstraintPublic + │ ConstraintID: 2 + │ ReferencedTableID: 105 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 105 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + - 105 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_1_of_2 new file mode 100644 index 000000000000..81ea88a930a3 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_1_of_2 @@ -0,0 +1,51 @@ +/* setup */ +CREATE TABLE t1 (i INT PRIMARY KEY); +CREATE TABLE t2 (i INT PRIMARY KEY); + +/* test */ +ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); +EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 2; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t1› ADD CONSTRAINT ‹t1_i_fkey› FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.public.‹t2› (‹i›); +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ WRITE_ONLY → ABSENT + │ │ + │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ rule: "ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" + │ │ + │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} + │ rule: "dependents removed before constraint" + │ + └── • 5 Mutation operations + │ + ├── • RemoveForeignKeyBackReference + │ OriginConstraintID: 2 + │ OriginTableID: 104 + │ ReferencedTableID: 105 + │ + ├── • RemoveForeignKeyConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 105 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + - 105 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_2_of_2 new file mode 100644 index 000000000000..1c9c92f7a799 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_2_of_2 @@ -0,0 +1,51 @@ +/* setup */ +CREATE TABLE t1 (i INT PRIMARY KEY); +CREATE TABLE t2 (i INT PRIMARY KEY); + +/* test */ +ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); +EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 2; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t1› ADD CONSTRAINT ‹t1_i_fkey› FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.public.‹t2› (‹i›); +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ WRITE_ONLY → ABSENT + │ │ + │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ rule: "ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" + │ │ + │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} + │ rule: "dependents removed before constraint" + │ + └── • 5 Mutation operations + │ + ├── • RemoveForeignKeyBackReference + │ OriginConstraintID: 2 + │ OriginTableID: 104 + │ ReferencedTableID: 105 + │ + ├── • RemoveForeignKeyConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 105 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + - 105 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index c84b30d9c04c..583fbbaf88c7 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/scrub" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -65,12 +66,8 @@ func newSQLForeignKeyCheckOperation( func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { ctx := params.ctx - checkQuery, _, err := nonMatchingRowQuery( - o.tableDesc, - o.constraint.ForeignKeyDesc(), - o.referencedTableDesc, - false, /* limitResults */ - ) + checkQuery, _, err := nonMatchingRowQuery(o.tableDesc, o.constraint.ForeignKeyDesc(), o.referencedTableDesc, + 0 /* indexIDForValidation */, false) if err != nil { return err } @@ -83,7 +80,7 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { } o.run.rows = rows - if o.constraint.NumOriginColumns() > 1 && o.constraint.Match() == descpb.ForeignKeyReference_FULL { + if o.constraint.NumOriginColumns() > 1 && o.constraint.Match() == semenumpb.Match_FULL { // Check if there are any disallowed references where some columns are NULL // and some aren't. checkNullsQuery, _, err := matchFullUnacceptableKeyQuery( diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index b85d51c1ad45..4938d2a070b8 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -394,3 +394,22 @@ const ( PgExtensionSpatialRefSysTableID MinVirtualID = PgExtensionSpatialRefSysTableID ) + +// ConstraintType is used to identify the type of a constraint. +type ConstraintType string + +const ( + // ConstraintTypePK identifies a PRIMARY KEY constraint. + ConstraintTypePK ConstraintType = "PRIMARY KEY" + // ConstraintTypeFK identifies a FOREIGN KEY constraint. + ConstraintTypeFK ConstraintType = "FOREIGN KEY" + // ConstraintTypeUnique identifies a UNIQUE constraint. + ConstraintTypeUnique ConstraintType = "UNIQUE" + // ConstraintTypeCheck identifies a CHECK constraint. + ConstraintTypeCheck ConstraintType = "CHECK" + // ConstraintTypeUniqueWithoutIndex identifies a UNIQUE_WITHOUT_INDEX constraint. + ConstraintTypeUniqueWithoutIndex ConstraintType = "UNIQUE WITHOUT INDEX" +) + +// SafeValue implements the redact.SafeValue interface. +func (ConstraintType) SafeValue() {} diff --git a/pkg/sql/sem/semenumpb/BUILD.bazel b/pkg/sql/sem/semenumpb/BUILD.bazel new file mode 100644 index 000000000000..feab3c3304aa --- /dev/null +++ b/pkg/sql/sem/semenumpb/BUILD.bazel @@ -0,0 +1,32 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +proto_library( + name = "semenumpb_proto", + srcs = ["constraint.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto:gogo_proto"], +) + +go_proto_library( + name = "semenumpb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb", + proto = ":semenumpb_proto", + visibility = ["//visibility:public"], + deps = ["@com_github_gogo_protobuf//gogoproto"], +) + +go_library( + name = "semenumpb", + srcs = ["constraint.go"], + embed = [":semenumpb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb", + visibility = ["//visibility:public"], + deps = ["@com_github_cockroachdb_redact//:redact"], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/sql/catalog/catpb/constraint.go b/pkg/sql/sem/semenumpb/constraint.go similarity index 53% rename from pkg/sql/catalog/catpb/constraint.go rename to pkg/sql/sem/semenumpb/constraint.go index 3f951d5bf80a..799b481fa975 100644 --- a/pkg/sql/catalog/catpb/constraint.go +++ b/pkg/sql/sem/semenumpb/constraint.go @@ -8,29 +8,9 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package catpb +package semenumpb -import ( - "strconv" - - "github.com/cockroachdb/redact" -) - -// String implements the fmt.Stringer interface. -func (x ForeignKeyAction) String() string { - switch x { - case ForeignKeyAction_RESTRICT: - return "RESTRICT" - case ForeignKeyAction_SET_DEFAULT: - return "SET DEFAULT" - case ForeignKeyAction_SET_NULL: - return "SET NULL" - case ForeignKeyAction_CASCADE: - return "CASCADE" - default: - return strconv.Itoa(int(x)) - } -} +import "github.com/cockroachdb/redact" var _ redact.SafeValue = ForeignKeyAction(0) diff --git a/pkg/sql/sem/semenumpb/constraint.proto b/pkg/sql/sem/semenumpb/constraint.proto new file mode 100644 index 000000000000..4df89f1441bc --- /dev/null +++ b/pkg/sql/sem/semenumpb/constraint.proto @@ -0,0 +1,37 @@ +// Copyright 2022 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. + +// This file should contain only EMUN definitions for concepts that +// are visible in the SQL layer (i.e. concepts that can be configured +// in a SQL query). +// It uses proto3 so other packages can import those enum definitions +// when needed. +syntax = "proto3"; +package cockroach.sql.sem.semenumpb; +option go_package = "semenumpb"; + +import "gogoproto/gogo.proto"; + +// ForeignKeyAction describes the action which should be taken when a foreign +// key constraint reference is acted upon. +enum ForeignKeyAction { + NO_ACTION = 0; + RESTRICT = 1; + SET_NULL = 2; + SET_DEFAULT = 3; + CASCADE = 4; +} + +// Match is the algorithm used to compare composite keys. +enum Match { + SIMPLE = 0; + FULL = 1; + PARTIAL = 2; // Note: not actually supported, but we reserve the value for future use. +} diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 6ff46dc2fd4f..366a76425db4 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -35,6 +35,7 @@ go_library( "constant.go", "constant_eval.go", "constants.go", + "constraint.go", "copy.go", "create.go", "cursor.go", @@ -131,6 +132,7 @@ go_library( "//pkg/sql/sem/cast", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/catid", + "//pkg/sql/sem/semenumpb", "//pkg/sql/sem/tree/treebin", "//pkg/sql/sem/tree/treecmp", "//pkg/sql/sem/tree/treewindow", diff --git a/pkg/sql/sem/tree/constraint.go b/pkg/sql/sem/tree/constraint.go new file mode 100644 index 000000000000..d097cc64dbcf --- /dev/null +++ b/pkg/sql/sem/tree/constraint.go @@ -0,0 +1,129 @@ +// Copyright 2022 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 tree + +import ( + "strconv" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" +) + +// ReferenceAction is the method used to maintain referential integrity through +// foreign keys. +type ReferenceAction semenumpb.ForeignKeyAction + +// The values for ReferenceAction. It has a one-to-one mapping to semenumpb.ForeignKeyAction. +const ( + NoAction ReferenceAction = iota + Restrict + SetNull + SetDefault + Cascade +) + +// ReferenceActions contains the actions specified to maintain referential +// integrity through foreign keys for different operations. +type ReferenceActions struct { + Delete ReferenceAction + Update ReferenceAction +} + +// Format implements the NodeFormatter interface. +func (node *ReferenceActions) Format(ctx *FmtCtx) { + if node.Delete != NoAction { + ctx.WriteString(" ON DELETE ") + ctx.WriteString(node.Delete.String()) + } + if node.Update != NoAction { + ctx.WriteString(" ON UPDATE ") + ctx.WriteString(node.Update.String()) + } +} + +// ForeignKeyReferenceActionType allows the conversion between a +// tree.ReferenceAction and a ForeignKeyReference_Action. +var ForeignKeyReferenceActionType = [...]ReferenceAction{ + semenumpb.ForeignKeyAction_NO_ACTION: NoAction, + semenumpb.ForeignKeyAction_RESTRICT: Restrict, + semenumpb.ForeignKeyAction_SET_DEFAULT: SetDefault, + semenumpb.ForeignKeyAction_SET_NULL: SetNull, + semenumpb.ForeignKeyAction_CASCADE: Cascade, +} + +// ForeignKeyReferenceActionValue allows the conversion between a +// semenumpb.ForeignKeyAction_Action and a tree.ReferenceAction. +var ForeignKeyReferenceActionValue = [...]semenumpb.ForeignKeyAction{ + NoAction: semenumpb.ForeignKeyAction_NO_ACTION, + Restrict: semenumpb.ForeignKeyAction_RESTRICT, + SetDefault: semenumpb.ForeignKeyAction_SET_DEFAULT, + SetNull: semenumpb.ForeignKeyAction_SET_NULL, + Cascade: semenumpb.ForeignKeyAction_CASCADE, +} + +// String implements the fmt.Stringer interface. +func (x ReferenceAction) String() string { + switch x { + case Restrict: + return "RESTRICT" + case SetDefault: + return "SET DEFAULT" + case SetNull: + return "SET NULL" + case Cascade: + return "CASCADE" + default: + return strconv.Itoa(int(x)) + } +} + +// CompositeKeyMatchMethod is the algorithm use when matching composite keys. +// See https://github.com/cockroachdb/cockroach/issues/20305 or +// https://www.postgresql.org/docs/11/sql-createtable.html for details on the +// different composite foreign key matching methods. +type CompositeKeyMatchMethod semenumpb.Match + +// The values for CompositeKeyMatchMethod. +const ( + MatchSimple CompositeKeyMatchMethod = iota + MatchFull + MatchPartial // Note: PARTIAL not actually supported at this point. +) + +// CompositeKeyMatchMethodType allows the conversion from a +// ForeignKeyReference_Match to a tree.ReferenceCompositeKeyMatchMethod. +// This should match CompositeKeyMatchMethodValue. +var CompositeKeyMatchMethodType = [...]CompositeKeyMatchMethod{ + semenumpb.Match_SIMPLE: MatchSimple, + semenumpb.Match_FULL: MatchFull, + semenumpb.Match_PARTIAL: MatchPartial, +} + +// CompositeKeyMatchMethodValue allows the conversion from a +// tree.ReferenceCompositeKeyMatchMethod to a ForeignKeyReference_Match. +var CompositeKeyMatchMethodValue = [...]semenumpb.Match{ + MatchSimple: semenumpb.Match_SIMPLE, + MatchFull: semenumpb.Match_FULL, + MatchPartial: semenumpb.Match_PARTIAL, +} + +// String implements the fmt.Stringer interface. +func (x CompositeKeyMatchMethod) String() string { + switch x { + case MatchSimple: + return "MATCH SIMPLE" + case MatchFull: + return "MATCH FULL" + case MatchPartial: + return "MATCH PARTIAL" + default: + return strconv.Itoa(int(x)) + } +} diff --git a/pkg/sql/sem/tree/create.go b/pkg/sql/sem/tree/create.go index f63f1aea67e0..92b0b1489783 100644 --- a/pkg/sql/sem/tree/create.go +++ b/pkg/sql/sem/tree/create.go @@ -1133,73 +1133,6 @@ func (node *UniqueConstraintTableDef) Format(ctx *FmtCtx) { } } -// ReferenceAction is the method used to maintain referential integrity through -// foreign keys. -type ReferenceAction int - -// The values for ReferenceAction. -const ( - NoAction ReferenceAction = iota - Restrict - SetNull - SetDefault - Cascade -) - -var referenceActionName = [...]string{ - NoAction: "NO ACTION", - Restrict: "RESTRICT", - SetNull: "SET NULL", - SetDefault: "SET DEFAULT", - Cascade: "CASCADE", -} - -func (ra ReferenceAction) String() string { - return referenceActionName[ra] -} - -// ReferenceActions contains the actions specified to maintain referential -// integrity through foreign keys for different operations. -type ReferenceActions struct { - Delete ReferenceAction - Update ReferenceAction -} - -// Format implements the NodeFormatter interface. -func (node *ReferenceActions) Format(ctx *FmtCtx) { - if node.Delete != NoAction { - ctx.WriteString(" ON DELETE ") - ctx.WriteString(node.Delete.String()) - } - if node.Update != NoAction { - ctx.WriteString(" ON UPDATE ") - ctx.WriteString(node.Update.String()) - } -} - -// CompositeKeyMatchMethod is the algorithm use when matching composite keys. -// See https://github.com/cockroachdb/cockroach/issues/20305 or -// https://www.postgresql.org/docs/11/sql-createtable.html for details on the -// different composite foreign key matching methods. -type CompositeKeyMatchMethod int - -// The values for CompositeKeyMatchMethod. -const ( - MatchSimple CompositeKeyMatchMethod = iota - MatchFull - MatchPartial // Note: PARTIAL not actually supported at this point. -) - -var compositeKeyMatchMethodName = [...]string{ - MatchSimple: "MATCH SIMPLE", - MatchFull: "MATCH FULL", - MatchPartial: "MATCH PARTIAL", -} - -func (c CompositeKeyMatchMethod) String() string { - return compositeKeyMatchMethodName[c] -} - // ForeignKeyConstraintTableDef represents a FOREIGN KEY constraint in the AST. type ForeignKeyConstraintTableDef struct { Name Name diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index e24c4d443a03..d66ae0327e45 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -472,17 +473,17 @@ func showForeignKeyConstraint( formatQuoteNames(buf, refNames...) buf.WriteByte(')') // We omit MATCH SIMPLE because it is the default. - if fk.Match != descpb.ForeignKeyReference_SIMPLE { + if fk.Match != semenumpb.Match_SIMPLE { buf.WriteByte(' ') - buf.WriteString(fk.Match.String()) + buf.WriteString(tree.CompositeKeyMatchMethodType[fk.Match].String()) } - if fk.OnDelete != catpb.ForeignKeyAction_NO_ACTION { + if fk.OnDelete != semenumpb.ForeignKeyAction_NO_ACTION { buf.WriteString(" ON DELETE ") - buf.WriteString(fk.OnDelete.String()) + buf.WriteString(tree.ForeignKeyReferenceActionType[fk.OnDelete].String()) } - if fk.OnUpdate != catpb.ForeignKeyAction_NO_ACTION { + if fk.OnUpdate != semenumpb.ForeignKeyAction_NO_ACTION { buf.WriteString(" ON UPDATE ") - buf.WriteString(fk.OnUpdate.String()) + buf.WriteString(tree.ForeignKeyReferenceActionType[fk.OnUpdate].String()) } if fk.Validity != descpb.ConstraintValidity_Validated { buf.WriteString(" NOT VALID") diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index 1cab3fa33131..778a159c029a 100644 --- a/pkg/testutils/lint/passes/redactcheck/redactcheck.go +++ b/pkg/testutils/lint/passes/redactcheck/redactcheck.go @@ -105,11 +105,9 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "ConnectionClass": {}, }, "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb": { - "ForeignKeyAction": {}, - "JobID": {}, + "JobID": {}, }, "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb": { - "ConstraintType": {}, "ConstraintValidity": {}, "DescriptorMutation_Direction": {}, "DescriptorMutation_State": {}, @@ -118,6 +116,12 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "IndexDescriptorVersion": {}, "MutationID": {}, }, + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants": { + "ConstraintType": {}, + }, + "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb": { + "ForeignKeyAction": {}, + }, "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph": { "RuleName": {}, },