diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 7b662c995e16..7abb7610b1ed 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -474,6 +474,7 @@ ALL_TESTS = [ "//pkg/sql/schemachanger/scexec:scexec_test", "//pkg/sql/schemachanger/scplan/internal/opgen:opgen_test", "//pkg/sql/schemachanger/scplan/internal/rules/current:current_test", + "//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:release_22_2_test", "//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph_test", "//pkg/sql/schemachanger/scplan:scplan_test", "//pkg/sql/schemachanger/screl:screl_test", @@ -1772,6 +1773,8 @@ GO_TARGETS = [ "//pkg/sql/schemachanger/scplan/internal/opgen:opgen_test", "//pkg/sql/schemachanger/scplan/internal/rules/current:current", "//pkg/sql/schemachanger/scplan/internal/rules/current:current_test", + "//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:release_22_2", + "//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:release_22_2_test", "//pkg/sql/schemachanger/scplan/internal/rules:rules", "//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph", "//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph_test", @@ -2921,6 +2924,7 @@ GET_X_DATA_TARGETS = [ "//pkg/sql/schemachanger/scplan/internal/opgen:get_x_data", "//pkg/sql/schemachanger/scplan/internal/rules:get_x_data", "//pkg/sql/schemachanger/scplan/internal/rules/current:get_x_data", + "//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:get_x_data", "//pkg/sql/schemachanger/scplan/internal/scgraph:get_x_data", "//pkg/sql/schemachanger/scplan/internal/scgraphviz:get_x_data", "//pkg/sql/schemachanger/scplan/internal/scstage:get_x_data", diff --git a/pkg/sql/schemachanger/scplan/BUILD.bazel b/pkg/sql/schemachanger/scplan/BUILD.bazel index b72f7c3730be..4094eea0b30d 100644 --- a/pkg/sql/schemachanger/scplan/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/BUILD.bazel @@ -16,7 +16,9 @@ go_library( "//pkg/sql/schemachanger/scop", "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scplan/internal/opgen", + "//pkg/sql/schemachanger/scplan/internal/rules", "//pkg/sql/schemachanger/scplan/internal/rules/current", + "//pkg/sql/schemachanger/scplan/internal/rules/release_22_2", "//pkg/sql/schemachanger/scplan/internal/scgraph", "//pkg/sql/schemachanger/scplan/internal/scgraphviz", "//pkg/sql/schemachanger/scplan/internal/scstage", diff --git a/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel index 6b2ebf0afb58..38ccfe1cdaf3 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules", visibility = ["//pkg/sql/schemachanger/scplan:__subpackages__"], deps = [ + "//pkg/clusterversion", "//pkg/sql/schemachanger/rel", "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scplan/internal/scgraph", diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/registry.go b/pkg/sql/schemachanger/scplan/internal/rules/current/registry.go index 037a2f6c532d..c3340be12123 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/registry.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/registry.go @@ -11,8 +11,6 @@ package current import ( - "context" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" @@ -54,10 +52,7 @@ func registerDepRuleForDrop( fn) } -func ApplyOpRules(ctx context.Context, g *scgraph.Graph) (*scgraph.Graph, error) { - return registry.ApplyOpRules(ctx, g) -} - -func ApplyDepRules(ctx context.Context, g *scgraph.Graph) error { - return registry.ApplyDepRules(ctx, g) +// GetRegistry returns the registry for this cockroach release. +func GetRegistry() *Registry { + return registry } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index 7d4de3eb9837..fd055294ef89 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -15,6 +15,7 @@ import ( "reflect" "strings" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" @@ -242,6 +243,22 @@ func ForEachElement(fn func(element scpb.Element) error) error { return nil } +func ForEachElementInActiveVersion( + fn func(element scpb.Element) error, version clusterversion.ClusterVersion, +) error { + var ep scpb.ElementProto + vep := reflect.ValueOf(ep) + for i := 0; i < vep.NumField(); i++ { + e := vep.Field(i).Interface().(scpb.Element) + if version.IsActive(screl.MinVersion(e)) { + if err := fn(e); err != nil { + return iterutil.Map(err) + } + } + } + return nil +} + // IsDescriptor returns true for a descriptor-element, i.e. an element which // owns its corresponding descriptor. func IsDescriptor(e scpb.Element) bool { @@ -428,6 +445,12 @@ func Or(predicates ...elementTypePredicate) elementTypePredicate { } } +func Not(predicate elementTypePredicate) elementTypePredicate { + return func(e scpb.Element) bool { + return !predicate(e) + } +} + // RegisterDepRuleForDrop is a convenience function which calls // RegisterDepRule with the cross-product of (ToAbsent,Transient)^2 Target // states, which can't easily be composed. diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/BUILD.bazel new file mode 100644 index 000000000000..b92fce58ef28 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/BUILD.bazel @@ -0,0 +1,60 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "release_22_2", + srcs = [ + "dep_add_column.go", + "dep_add_constraint.go", + "dep_add_index.go", + "dep_add_index_and_column.go", + "dep_drop_column.go", + "dep_drop_constraint.go", + "dep_drop_index.go", + "dep_drop_index_and_column.go", + "dep_drop_object.go", + "dep_swap_index.go", + "dep_two_version.go", + "op_drop.go", + "op_index_and_column.go", + "registry.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules/release_22_2", + visibility = ["//pkg/sql/schemachanger/scplan:__subpackages__"], + deps = [ + "//pkg/sql/schemachanger/rel", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scplan/internal/opgen", + "//pkg/sql/schemachanger/scplan/internal/rules", + "//pkg/sql/schemachanger/scplan/internal/scgraph", + "//pkg/sql/schemachanger/screl", + ], +) + +go_test( + name = "release_22_2_test", + srcs = [ + "assertions_test.go", + "rules_test.go", + ], + args = ["-test.timeout=295s"], + data = glob(["testdata/**"]), + embed = [":release_22_2"], + deps = [ + "//pkg/clusterversion", + "//pkg/sql/catalog/catpb", + "//pkg/sql/schemachanger/rel", + "//pkg/sql/schemachanger/scpb", + "//pkg/sql/schemachanger/scplan/internal/opgen", + "//pkg/sql/schemachanger/scplan/internal/rules", + "//pkg/sql/schemachanger/screl", + "//pkg/sql/types", + "//pkg/testutils/datapathutils", + "@com_github_cockroachdb_datadriven//:datadriven", + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + "@in_gopkg_yaml_v3//:yaml_v3", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/assertions_test.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/assertions_test.go new file mode 100644 index 000000000000..0227d2439ef8 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/assertions_test.go @@ -0,0 +1,190 @@ +// 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 release_22_2 + +import ( + "reflect" + "runtime" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/opgen" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" +) + +// TestRuleAssertions verifies that important helper functions verify certain +// properties that the rule definitions rely on. +func TestRuleAssertions(t *testing.T) { + for _, fn := range []func(e scpb.Element) error{ + checkSimpleDependentsReferenceDescID, + checkToAbsentCategories, + checkIsWithTypeT, + checkIsWithExpression, + checkIsColumnDependent, + checkIsIndexDependent, + checkIsConstraintDependent, + } { + version := clusterversion.ClusterVersion{Version: clusterversion.ByKey(clusterversion.V22_2)} + var fni interface{} = fn + fullName := runtime.FuncForPC(reflect.ValueOf(fni).Pointer()).Name() + nameParts := strings.Split(fullName, "rules.") + shortName := nameParts[len(nameParts)-1] + t.Run(shortName, func(t *testing.T) { + _ = ForEachElementInActiveVersion(func(e scpb.Element) error { + e = nonNilElement(e) + if err := fn(e); err != nil { + t.Errorf("%T: %+v", e, err) + } + return nil + }, version) + }) + } +} + +func nonNilElement(element scpb.Element) scpb.Element { + return reflect.New(reflect.ValueOf(element).Type().Elem()).Interface().(scpb.Element) +} + +// Assert that only simple dependents (non-descriptor, non-index, non-column) +// have screl.ReferencedDescID attributes. +func checkSimpleDependentsReferenceDescID(e scpb.Element) error { + if IsSimpleDependent(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") + } + return nil +} + +// Assert that elements can be grouped into three categories when transitioning +// from PUBLIC to ABSENT: +// - go via DROPPED iff they're descriptor elements +// - go via a non-read status iff they're indexes or columns, which are +// subject to the two-version invariant. +// - go direct to ABSENT in all other cases. +func checkToAbsentCategories(e scpb.Element) error { + s0 := opgen.InitialStatus(e, scpb.Status_ABSENT) + s1 := opgen.NextStatus(e, scpb.Status_ABSENT, s0) + switch s1 { + case scpb.Status_TXN_DROPPED, scpb.Status_DROPPED: + if IsDescriptor(e) { + return nil + } + case scpb.Status_VALIDATED, scpb.Status_WRITE_ONLY, scpb.Status_DELETE_ONLY: + if IsSubjectTo2VersionInvariant(e) { + return nil + } + case scpb.Status_ABSENT: + if IsSimpleDependent(e) { + return nil + } + } + return errors.Newf("unexpected transition %s -> %s in direction ABSENT", s0, s1) +} + +// Assert that isWithTypeT covers all elements with embedded TypeTs. +func checkIsWithTypeT(e scpb.Element) error { + return screl.WalkTypes(e, func(t *types.T) error { + if IsWithTypeT(e) { + return nil + } + return errors.New("should verify isWithTypeT but doesn't") + }) +} + +// Assert that isWithExpression covers all elements with embedded +// expressions. +func checkIsWithExpression(e scpb.Element) error { + return screl.WalkExpressions(e, func(t *catpb.Expression) error { + switch e.(type) { + // Ignore elements which have catpb.Expression fields but which don't + // have them within an scpb.Expression for valid reasons. + case *scpb.RowLevelTTL: + return nil + } + if IsWithExpression(e) { + return nil + } + return errors.New("should verify isWithExpression but doesn't") + }) +} + +// Assert that rules.IsColumnDependent covers all dependent elements of a column +// element. +func checkIsColumnDependent(e scpb.Element) error { + // Exclude columns themselves. + if IsColumn(e) { + return nil + } + // A column dependent should have a ColumnID attribute. + _, err := screl.Schema.GetAttribute(screl.ColumnID, e) + if IsColumnDependent(e) { + if err != nil { + return errors.New("verifies rules.IsColumnDependent but doesn't have ColumnID attr") + } + } else if err == nil { + return errors.New("has ColumnID attr but doesn't verify rules.IsColumnDependent") + } + return nil +} + +// Assert that rules.IsIndexDependent covers all dependent elements of an index +// element. +func checkIsIndexDependent(e scpb.Element) error { + // Exclude indexes themselves. + if IsIndex(e) || IsSupportedNonIndexBackedConstraint(e) { + return nil + } + // Skip check constraints, in 22.2 these didn't have + // index IDs. + if _, ok := e.(*scpb.CheckConstraint); ok { + return nil + } + // An index dependent should have an IndexID attribute. + _, err := screl.Schema.GetAttribute(screl.IndexID, e) + if IsIndexDependent(e) { + if err != nil { + return errors.New("verifies rules.IsIndexDependent but doesn't have IndexID attr") + } + } else if err == nil { + return errors.New("has IndexID attr but doesn't verify rules.IsIndexDependent") + } + return nil +} + +// Assert that checkIsConstraintDependent covers all elements of a constraint +// element. +func checkIsConstraintDependent(e scpb.Element) error { + // Exclude constraints themselves. + if IsConstraint(e) { + return nil + } + // A constraint dependent should have a ConstraintID attribute. + _, err := screl.Schema.GetAttribute(screl.ConstraintID, e) + if IsConstraintDependent(e) { + if err != nil { + return errors.New("verifies rules.IsConstraintDependent but doesn't have ConstraintID attr") + } + } else if err == nil { + return errors.New("has ConstraintID attr but doesn't verify rules.IsConstraintDependent") + } + return nil +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go new file mode 100644 index 000000000000..2ead38aa679e --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go @@ -0,0 +1,111 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" +) + +// These rules ensure that column-dependent elements, like a column's name, its +// DEFAULT expression, etc. appear once the column reaches a suitable state. +func init() { + + registerDepRule( + "column existence precedes column dependents", + scgraph.Precedence, + "column", "dependent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.TypeFilter(IsColumnDependent), + JoinOnColumnID(from, to, "table-id", "col-id"), + StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), + } + }, + ) + + registerDepRule( + "column dependents exist before column becomes public", + scgraph.Precedence, + "dependent", "column", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsColumnDependent), + to.Type((*scpb.Column)(nil)), + JoinOnColumnID(from, to, "table-id", "col-id"), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), + } + }, + ) +} + +// Special cases of the above. +func init() { + registerDepRule( + "column name and type set right after column existence", + scgraph.SameStagePrecedence, + "column", "column-name-or-type", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.Type( + (*scpb.ColumnName)(nil), + (*scpb.ColumnType)(nil), + ), + StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), + JoinOnColumnID(from, to, "table-id", "col-id"), + } + }, + ) + + registerDepRule( + "DEFAULT or ON UPDATE existence precedes writes to column", + scgraph.Precedence, + "expr", "column", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type( + (*scpb.ColumnDefaultExpression)(nil), + (*scpb.ColumnOnUpdateExpression)(nil), + ), + to.Type((*scpb.Column)(nil)), + JoinOnColumnID(from, to, "table-id", "col-id"), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_WRITE_ONLY), + } + }, + ) +} + +// This rule ensures that columns depend on each other in increasing order. +func init() { + registerDepRule( + "ensure columns are in increasing order", + scgraph.SameStagePrecedence, + "later-column", "earlier-column", + func(from, to NodeVars) rel.Clauses { + status := rel.Var("status") + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.Type((*scpb.Column)(nil)), + JoinOnDescID(from, to, "table-id"), + ToPublicOrTransient(from, to), + status.In(scpb.Status_WRITE_ONLY, scpb.Status_PUBLIC), + status.Entities(screl.CurrentStatus, from.Node, to.Node), + FilterElements("SmallerColumnIDFirst", from, to, func(from, to *scpb.Column) bool { + return from.ColumnID < to.ColumnID + }), + } + }) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_constraint.go new file mode 100644 index 000000000000..44e42209d8c8 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_constraint.go @@ -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. + +package release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +// These rules ensure that constraint-dependent elements, like a constraint's +// name, etc. appear once the constraint reaches a suitable state. +func init() { + + registerDepRule( + "constraint dependent public right before constraint", + scgraph.SameStagePrecedence, + "constraint", "dependent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsConstraint), + to.TypeFilter(IsConstraintDependent), + JoinOnConstraintID(from, to, "table-id", "constraint-id"), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index.go new file mode 100644 index 000000000000..011b82d5a2e6 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index.go @@ -0,0 +1,160 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" +) + +// These rules ensure that index-dependent elements, like an index's name, its +// partitioning, etc. appear once the index reaches a suitable state. +func init() { + + registerDepRule( + "index existence precedes index dependents", + scgraph.Precedence, + "index", "dependent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type( + (*scpb.PrimaryIndex)(nil), + (*scpb.SecondaryIndex)(nil), + ), + to.TypeFilter(IsIndexDependent), + JoinOnIndexID(from, to, "table-id", "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_BACKFILL_ONLY, to, scpb.Status_PUBLIC), + } + }, + ) + + registerDepRule( + "temp index existence precedes index dependents", + scgraph.Precedence, + "index", "dependent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.TemporaryIndex)(nil)), + to.TypeFilter(IsIndexDependent), + JoinOnIndexID(from, to, "table-id", "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), + } + }, + ) + + registerDepRule( + "index dependents exist before index becomes public", + scgraph.Precedence, + "dependent", "index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsIndexDependent), + to.TypeFilter(IsIndex), + JoinOnIndexID(from, to, "table-id", "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), + } + }, + ) +} + +// Special cases of the above. +func init() { + + registerDepRule( + "index named right before index becomes public", + scgraph.SameStagePrecedence, + "index-name", "index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.IndexName)(nil)), + to.Type( + (*scpb.PrimaryIndex)(nil), + (*scpb.SecondaryIndex)(nil), + ), + JoinOnIndexID(from, to, "table-id", "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), + } + }, + ) +} + +// This rule ensures that before an offline-backfilled index can begin +// backfilling, the corresponding temporary index exists in WRITE_ONLY. +func init() { + + registerDepRule( + "temp index is WRITE_ONLY before backfill", + scgraph.Precedence, + "temp", "index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.TemporaryIndex)(nil)), + to.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), + JoinOnDescID(from, to, "table-id"), + JoinOn( + from, screl.IndexID, + to, screl.TemporaryIndexID, + "temp-index-id", + ), + from.TargetStatus(scpb.Transient), + to.TargetStatus(scpb.ToPublic, scpb.Transient), + from.CurrentStatus(scpb.Status_WRITE_ONLY), + to.CurrentStatus(scpb.Status_BACKFILLED), + } + }, + ) +} + +// We want to say that all columns which are part of a secondary index need +// to be in a primary index which is validated +// To do that, we want to find a secondary index which has a source which +// is a primary index which is itself new. +func init() { + + registerDepRule( + "primary index with new columns should exist before secondary indexes", + scgraph.Precedence, + "primary-index", "secondary-index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.SecondaryIndex)(nil)), + JoinOnDescID(from, to, "table-id"), + JoinOn( + from, screl.IndexID, + to, screl.SourceIndexID, + "primary-index-id", + ), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_BACKFILL_ONLY), + } + }) + + registerDepRule( + "primary index with new columns should exist before temp indexes", + scgraph.Precedence, + "primary-index", "temp-index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.TemporaryIndex)(nil)), + JoinOnDescID(from, to, "table-id"), + JoinOn( + from, screl.IndexID, + to, screl.SourceIndexID, + "primary-index-id", + ), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_DELETE_ONLY), + } + }) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index_and_column.go new file mode 100644 index 000000000000..9f0a4906e721 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index_and_column.go @@ -0,0 +1,125 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +// These rules ensure that columns and indexes containing these columns +// appear into existence in the correct order. +func init() { + + // We need to make sure that no columns are added to the index after it + // receives any data due to a backfill. + registerDepRule("index-column added to index before index is backfilled", + scgraph.Precedence, + "index-column", "index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.IndexColumn)(nil)), + to.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), + JoinOnIndexID(from, to, "table-id", "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_BACKFILLED), + } + }) + + // We need to make sure that no columns are added to the temp index after it + // receives any writes. + registerDepRule("index-column added to index before temp index receives writes", + scgraph.Precedence, + "index-column", "index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.IndexColumn)(nil)), + to.Type((*scpb.TemporaryIndex)(nil)), + JoinOnIndexID(from, to, "table-id", "index-id"), + StatusesTransient(from, scpb.Status_PUBLIC, to, scpb.Status_WRITE_ONLY), + } + }) + + registerDepRule( + "column existence precedes index existence", + scgraph.Precedence, + "column", "index", + func(from, to NodeVars) rel.Clauses { + ic := MkNodeVars("index-column") + relationID, columnID := rel.Var("table-id"), rel.Var("column-id") + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), + JoinOnColumnID(from, ic, relationID, columnID), + ColumnInIndex(ic, to, relationID, columnID, "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_BACKFILL_ONLY), + } + }, + ) + + registerDepRule( + "column existence precedes temp index existence", + scgraph.Precedence, + "column", "index", + func(from, to NodeVars) rel.Clauses { + ic := MkNodeVars("index-column") + relationID, columnID := rel.Var("table-id"), rel.Var("column-id") + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.Type((*scpb.TemporaryIndex)(nil)), + JoinOnColumnID(ic, from, relationID, columnID), + ColumnInIndex(ic, to, relationID, columnID, "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_DELETE_ONLY), + } + }, + ) + + // We need to ensure that the temporary index has all the relevant writes + // to any columns it contains. We ensure elsewhere that any index which + // will later be merged with the temporary index is rules.Not backfilled until + // that temporary index is receiving writes. This rule ensures that those + // write operations contain data for all columns. + registerDepRule( + "column is WRITE_ONLY before temporary index is WRITE_ONLY", + scgraph.Precedence, + "column", "index", + func(from, to NodeVars) rel.Clauses { + ic := MkNodeVars("index-column") + relationID, columnID := rel.Var("table-id"), rel.Var("column-id") + return rel.Clauses{ + from.El.Type((*scpb.Column)(nil)), + to.El.Type((*scpb.TemporaryIndex)(nil)), + JoinOnColumnID(ic, from, relationID, columnID), + ColumnInIndex(ic, to, relationID, columnID, "index-id"), + StatusesToPublicOrTransient(from, scpb.Status_WRITE_ONLY, to, scpb.Status_WRITE_ONLY), + } + }, + ) + + registerDepRule( + "swapped primary index public before column", + scgraph.Precedence, + "index", "column", + func(from, to NodeVars) rel.Clauses { + ic := MkNodeVars("index-column") + relationID, columnID := rel.Var("table-id"), rel.Var("column-id") + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.Column)(nil)), + ColumnInSwappedInPrimaryIndex(ic, from, relationID, columnID, "index-id"), + JoinOnColumnID(ic, to, relationID, columnID), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), + } + }, + ) + +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go new file mode 100644 index 000000000000..dbb18eafbb14 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go @@ -0,0 +1,100 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +// These rules ensure that column-dependent elements, like a column's name, its +// DEFAULT expression, etc. disappear once the column reaches a suitable state. +func init() { + + registerDepRuleForDrop( + "column no longer public before dependents", + scgraph.Precedence, + "column", "dependent", + scpb.Status_WRITE_ONLY, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Column)(nil)), + to.TypeFilter(IsColumnDependent), + JoinOnColumnID(from, to, "table-id", "col-id"), + } + }, + ) + + registerDepRuleForDrop( + "dependents removed before column", + scgraph.Precedence, + "dependent", "column", + scpb.Status_ABSENT, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsColumnDependent), + to.Type((*scpb.Column)(nil)), + JoinOnColumnID(from, to, "table-id", "col-id"), + } + }, + ) +} + +// Special cases of the above. +func init() { + + registerDepRule( + "column type dependents removed right before column type", + scgraph.SameStagePrecedence, + "dependent", "column-type", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsColumnTypeDependent), + to.Type((*scpb.ColumnType)(nil)), + JoinOnColumnID(from, to, "table-id", "col-id"), + StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), + } + }, + ) + + // Special cases for removal of column types, which hold references to other + // descriptors. + // + // When the whole table is dropped, we can (and in fact, should) remove these + // right away in-txn. However, when only the column is dropped but the table + // remains, we need to wait until the column is DELETE_ONLY, which happens + // post-commit because of the need to uphold the 2-version invariant. + // + // We distinguish the two cases using a flag in ColumnType which is set iff + // the parent relation is dropped. This is a dirty hack, ideally we should be + // able to express the _absence_ of a target element as a query clause. + // + // Note that DEFAULT and ON UPDATE expressions are column-dependent elements + // which also hold references to other descriptors. The rule prior to this one + // ensures that they transition to ABSENT before scpb.ColumnType does. + registerDepRule( + "column type removed right before column when rules.Not dropping relation", + scgraph.SameStagePrecedence, + "column-type", "column", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.ColumnType)(nil)), + from.DescriptorIsNotBeingDropped(), + to.Type((*scpb.Column)(nil)), + JoinOnColumnID(from, to, "table-id", "col-id"), + StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), + } + }, + ) + +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go new file mode 100644 index 000000000000..cef8c6e7b089 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go @@ -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. + +package release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +// These rules ensure that constraint-dependent elements, like an constraint's +// name, etc. disappear once the constraint reaches a suitable state. +func init() { + + registerDepRuleForDrop( + "constraint dependent absent right before constraint", + scgraph.Precedence, + "dependent", "constraint", + scpb.Status_ABSENT, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsConstraintDependent), + to.TypeFilter(IsConstraint, Not(IsIndex)), + JoinOnConstraintID(from, to, "table-id", "constraint-id"), + } + }, + ) + + registerDepRuleForDrop( + "constraint dependent absent right before constraint", + scgraph.SameStagePrecedence, + "dependent", "constraint", + scpb.Status_VALIDATED, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsConstraintDependent), + to.TypeFilter(IsConstraint, IsIndex), + JoinOnConstraintID(from, to, "table-id", "constraint-id"), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go new file mode 100644 index 000000000000..9cc37b424752 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go @@ -0,0 +1,107 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +// These rules ensure that index-dependent elements, like an index's name, its +// partitioning, etc. disappear once the index reaches a suitable state. +func init() { + + registerDepRuleForDrop( + "index no longer public before dependents", + scgraph.Precedence, + "index", "dependent", + scpb.Status_VALIDATED, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsIndex), + to.TypeFilter(IsIndexDependent), + JoinOnIndexID(from, to, "table-id", "index-id"), + } + }, + ) + registerDepRuleForDrop( + "dependents removed before index", + scgraph.Precedence, + "dependent", "index", + scpb.Status_ABSENT, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsIndexDependent), + to.TypeFilter(IsIndex), + JoinOnIndexID(from, to, "table-id", "index-id"), + } + }, + ) +} + +// Special cases of the above. +func init() { + + // If we're going to be removing columns from an index, we know that + // it'll be because we're dropping the index. If we're dropping the + // index and rules.Not the descriptor, we need to make sure that we only + // do it once the index is definitely being dropped. The reason for + // this is roundabout: dropping a column from an index which is itself + // being dropped is treated as a no-op by the op rules. + // + // TODO(ajwerner): This rule really feels like it ought to be a + // same stage precedence sort of rule where we remove the columns from the + // index when we remove the index, but for some reason, that overconstrains + // the graph when dropping the table. Because of that, we allow the column + // to be removed from the index in DELETE_ONLY, and we no-op the removal. + registerDepRuleForDrop( + "remove columns from index right before removing index", + scgraph.Precedence, + "index", "index-column", + scpb.Status_DELETE_ONLY, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.IndexColumn)(nil)), + to.TypeFilter(IsIndex), + JoinOnIndexID(from, to, "table-id", "index-id"), + } + }, + ) + + // Special case for removal of partial predicates, which hold references to + // other descriptors. + // + // When the whole table is dropped, we can (and in fact, should) remove these + // right away in-txn. However, when only the index is dropped but the table + // remains, we need to wait until the index is DELETE_ONLY, which happens + // post-commit because of the need to uphold the 2-version invariant. + // + // We distinguish the two cases using a flag in SecondaryIndexPartial which is + // set iff the parent relation is dropped. This is a dirty hack, ideally we + // should be able to express the _absence_ of a target element as a query + // clause. + registerDepRuleForDrop( + "partial predicate removed right before secondary index when rules.Not dropping relation", + scgraph.SameStagePrecedence, + "partial-predicate", "index", + scpb.Status_ABSENT, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.SecondaryIndexPartial)(nil)), + from.DescriptorIsNotBeingDropped(), + to.Type((*scpb.SecondaryIndex)(nil)), + JoinOnIndexID(from, to, "table-id", "index-id"), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index_and_column.go new file mode 100644 index 000000000000..554a309d5179 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index_and_column.go @@ -0,0 +1,64 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +// These rules ensure that columns and indexes containing these columns +// disappear from existence in the correct order. +func init() { + + // Without this rule, we might have an index which exists and contains + // a column which does rules.Not exist. This would lead to panics inside the + // optimizer and an invalid table descriptor. + registerDepRuleForDrop("indexes containing column reach absent before column", + scgraph.Precedence, + "index", "column", + scpb.Status_ABSENT, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + ic := MkNodeVars("index-column") + relationID, columnID := rel.Var("table-id"), rel.Var("column-id") + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), + to.Type((*scpb.Column)(nil)), + ColumnInIndex(ic, from, relationID, columnID, "index-id"), + JoinOnColumnID(ic, to, relationID, columnID), + ic.DescriptorIsNotBeingDropped(), + } + }) + + registerDepRule("secondary indexes containing column as key reach write-only before column", + scgraph.Precedence, + "index", "column", + func(from, to NodeVars) rel.Clauses { + ic := MkNodeVars("index-column") + relationID, columnID := rel.Var("table-id"), rel.Var("column-id") + return rel.Clauses{ + from.Type((*scpb.SecondaryIndex)(nil)), + to.Type((*scpb.Column)(nil)), + ColumnInIndex(ic, from, relationID, columnID, "index-id"), + JoinOnColumnID(ic, to, relationID, columnID), + StatusesToAbsent(from, scpb.Status_VALIDATED, to, scpb.Status_WRITE_ONLY), + ic.DescriptorIsNotBeingDropped(), + rel.Filter("rules.IsIndexKeyColumnKey", ic.El)( + func(ic *scpb.IndexColumn) bool { + return ic.Kind == scpb.IndexColumn_KEY + }, + ), + } + }) + +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_object.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_object.go new file mode 100644 index 000000000000..44ebe1e2794d --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_object.go @@ -0,0 +1,148 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" +) + +// These rules ensure that: +// - a descriptor reaches the TXN_DROPPED state in the statement phase, and +// it does rules.Not reach DROPPED until the pre-commit phase. +// - a descriptor reaches ABSENT in a different transaction than it reaches +// DROPPED (i.e. it canrules.Not be removed until PostCommit). +// - a descriptor element reaches the DROPPED state in the txn before +// its dependent elements (namespace entry, comments, column names, etc) reach +// the ABSENT state; +// - for those dependent elements which have to wait post-commit to reach the +// ABSENT state, we tie them to the same stage as when the descriptor element +// reaches the ABSENT state, but afterwards in the stage, so as to rules.Not +// interfere with the event logging op which is tied to the descriptor element +// removal. +func init() { + + registerDepRule( + "descriptor TXN_DROPPED before DROPPED", + scgraph.PreviousStagePrecedence, + "txn_dropped", "dropped", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsDescriptor), + from.El.AttrEqVar(screl.DescID, "_"), + from.El.AttrEqVar(rel.Self, to.El), + StatusesToAbsent(from, scpb.Status_TXN_DROPPED, to, scpb.Status_DROPPED), + } + }) + registerDepRule( + "descriptor DROPPED in transaction before removal", + scgraph.PreviousTransactionPrecedence, + "dropped", "absent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsDescriptor), + from.El.AttrEqVar(screl.DescID, "_"), + from.El.AttrEqVar(rel.Self, to.El), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + } + }) + + registerDepRule( + "descriptor drop right before dependent element removal", + scgraph.SameStagePrecedence, + "descriptor", "dependent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsDescriptor), + to.TypeFilter(IsSimpleDependent), + JoinOnDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + FromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL(from.Target, from.El, to.El), + } + }) + + registerDepRule( + "descriptor removal right before dependent element removal", + scgraph.SameStagePrecedence, + "descriptor", "idx-or-col", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsDescriptor), + to.TypeFilter(IsSubjectTo2VersionInvariant), + JoinOnDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), + } + }, + ) +} + +// These rules ensure that cross-referencing simple dependent elements reach +// ABSENT in the same stage right after the referenced descriptor element +// reaches DROPPED. +// +// References from simple dependent elements to other descriptors exist as +// follows: +// - simple dependent elements with a ReferencedDescID attribute, +// - those which embed a TypeT, +// - those which embed an Expression. +func init() { + + registerDepRule( + "descriptor drop right before removing dependent with attr ref", + scgraph.SameStagePrecedence, + "referenced-descriptor", "referencing-via-attr", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(IsDescriptor), + to.TypeFilter(IsSimpleDependent), + JoinReferencedDescID(to, from, "desc-id"), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + } + }, + ) + + registerDepRule( + "descriptor drop right before removing dependent with type ref", + scgraph.SameStagePrecedence, + "referenced-descriptor", "referencing-via-type", + func(from, to NodeVars) rel.Clauses { + fromDescID := rel.Var("fromDescID") + return rel.Clauses{ + from.TypeFilter(IsTypeDescriptor), + from.JoinTargetNode(), + from.DescIDEq(fromDescID), + to.ReferencedTypeDescIDsContain(fromDescID), + to.TypeFilter(IsSimpleDependent, Or(IsWithTypeT, IsWithExpression)), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + } + }, + ) + + registerDepRule( + "descriptor drop right before removing dependent with expr ref to sequence", + scgraph.SameStagePrecedence, + "referenced-descriptor", "referencing-via-expr", + func(from, to NodeVars) rel.Clauses { + seqID := rel.Var("seqID") + return rel.Clauses{ + from.Type((*scpb.Sequence)(nil)), + from.JoinTargetNode(), + from.DescIDEq(seqID), + to.ReferencedSequenceIDsContains(seqID), + to.TypeFilter(IsSimpleDependent, IsWithExpression), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_swap_index.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_swap_index.go new file mode 100644 index 000000000000..84ad01187f60 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_swap_index.go @@ -0,0 +1,127 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" +) + +// This rule ensures that a new primary index becomes public right after the +// old primary index starts getting removed, effectively swapping one for the +// other. This rule also applies when the schema change gets reverted. +func init() { + + registerDepRule( + "primary index swap", + scgraph.SameStagePrecedence, + "old-index", "new-index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.PrimaryIndex)(nil)), + JoinOnDescID(from, to, "table-id"), + JoinOn( + from, screl.IndexID, + to, screl.SourceIndexID, + "old-index-id", + ), + from.TargetStatus(scpb.ToAbsent), + from.CurrentStatus(scpb.Status_VALIDATED), + to.TargetStatus(scpb.ToPublic, scpb.Transient), + to.CurrentStatus(scpb.Status_PUBLIC), + } + }, + ) + + registerDepRule( + "primary index swap", + scgraph.SameStagePrecedence, + "old-index", "new-index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.PrimaryIndex)(nil)), + JoinOnDescID(from, to, "table-id"), + JoinOn( + from, screl.IndexID, + to, screl.SourceIndexID, + "old-index-id", + ), + from.TargetStatus(scpb.Transient), + from.CurrentStatus(scpb.Status_TRANSIENT_VALIDATED), + to.TargetStatus(scpb.ToPublic, scpb.Transient), + to.CurrentStatus(scpb.Status_PUBLIC), + } + }, + ) + + registerDepRule( + "primary index swap", + scgraph.SameStagePrecedence, + "new-index", "old-index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.PrimaryIndex)(nil)), + JoinOnDescID(from, to, "table-id"), + JoinOn( + from, screl.SourceIndexID, + to, screl.IndexID, + "old-index-id", + ), + from.TargetStatus(scpb.ToAbsent), + from.CurrentStatus(scpb.Status_VALIDATED), + to.TargetStatus(scpb.ToPublic), + to.CurrentStatus(scpb.Status_PUBLIC), + } + }, + ) +} + +// This rule ensures that when a transient primary index is involved in the +// swap, the old index is gone before the new index is instated. +func init() { + + registerDepRule( + "old index absent before new index public when swapping with transient", + scgraph.Precedence, + "old-primary-index", "new-primary-index", + func(from, to NodeVars) rel.Clauses { + union := MkNodeVars("transient-primary-index") + relationID := rel.Var("table-id") + return rel.Clauses{ + from.Type((*scpb.PrimaryIndex)(nil)), + union.Type((*scpb.PrimaryIndex)(nil)), + to.Type((*scpb.PrimaryIndex)(nil)), + JoinOnDescID(from, union, relationID), + JoinOn( + from, screl.IndexID, + union, screl.SourceIndexID, + "old-index-id", + ), + JoinOnDescID(union, to, relationID), + JoinOn( + union, screl.IndexID, + to, screl.SourceIndexID, + "transient-index-id", + ), + from.TargetStatus(scpb.ToAbsent), + from.CurrentStatus(scpb.Status_ABSENT), + to.TargetStatus(scpb.ToPublic), + to.CurrentStatus(scpb.Status_PUBLIC), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_two_version.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_two_version.go new file mode 100644 index 000000000000..a6e4c9805adb --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_two_version.go @@ -0,0 +1,119 @@ +// 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 release_22_2 + +import ( + "fmt" + "reflect" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/opgen" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" +) + +// These rules ensure that changes to properties of descriptors which need to +// be sequenced in order to safely enact online schema changes are sequenced +// in separate transactions. +func init() { + + findNoopSourceStatuses := func( + el scpb.Element, targetStatus scpb.TargetStatus, + ) map[scpb.Status][]scpb.Status { + // We want to skip the dependency edges if the status which got us into + // the current status was a no-op. We track the no-op status parent nodes, + // and we'll add a rules.Not-join to make sure there does rules.Not exist a node + // with such a status when installing the rule. + // + // This is necessary to deal with cases like the transition from + // BACKFILL_ONLY to its equivalent DELETE_ONLY in the rollback of an + // ADD COLUMN. We don't want or need a dependency edge from DELETE_ONLY + // to ABSENT in that case, but if we didn't check whether we got to + // DELETE_ONLY from BACKFILL_ONLY, then we'd have one implicitly. + statusMap := map[scpb.Status][]scpb.Status{} + if err := opgen.IterateTransitions(el, targetStatus, func( + t opgen.Transition, + ) error { + if !t.OpType().IsValid() { + statusMap[t.To()] = append(statusMap[t.To()], t.From()) + } + return nil + }); err != nil { + panic(err) + } + return statusMap + } + clausesForTwoVersionEdge := func( + from, to NodeVars, + el scpb.Element, + targetStatus scpb.TargetStatus, + t opgen.Transition, + prePrevStatuses []scpb.Status, + ) rel.Clauses { + clauses := rel.Clauses{ + from.Type(el), + to.Type(el), + from.El.AttrEqVar(screl.DescID, "_"), + from.El.AttrEqVar(rel.Self, to.El), + from.Target.AttrEqVar(rel.Self, to.Target), + from.Target.AttrEq(screl.TargetStatus, targetStatus.Status()), + from.Node.AttrEq(screl.CurrentStatus, t.From()), + to.Node.AttrEq(screl.CurrentStatus, t.To()), + from.DescriptorIsNotBeingDropped(), + } + if len(prePrevStatuses) > 0 { + clauses = append(clauses, + GetNotJoinOnNodeWithStatusIn(prePrevStatuses)(from.Target), + ) + } + return clauses + } + addRules := func(el scpb.Element, targetStatus scpb.TargetStatus) { + statusMap := findNoopSourceStatuses(el, targetStatus) + if err := opgen.IterateTransitions(el, targetStatus, func( + t opgen.Transition, + ) error { + elemName := reflect.TypeOf(el).Elem().Name() + ruleName := scgraph.RuleName(fmt.Sprintf( + "%s transitions to %s uphold 2-version invariant: %s->%s", + elemName, targetStatus.Status(), t.From(), t.To(), + )) + registerDepRule( + ruleName, + scgraph.PreviousTransactionPrecedence, + "prev", "next", + func(from, to NodeVars) rel.Clauses { + return clausesForTwoVersionEdge( + from, to, el, targetStatus, t, statusMap[t.From()], + ) + }, + ) + return nil + }); err != nil { + panic(err) + } + } + _ = ForEachElement(func(el scpb.Element) error { + if !IsSubjectTo2VersionInvariant(el) { + return nil + } + if opgen.HasPublic(el) { + addRules(el, scpb.ToPublic) + } + if opgen.HasTransient(el) { + addRules(el, scpb.Transient) + } + addRules(el, scpb.ToAbsent) // every element has ToAbsent + return nil + }) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go new file mode 100644 index 000000000000..3f2a0b70e682 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go @@ -0,0 +1,319 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" +) + +// When dropping a table or a view, skip all removal ops for column elements +// and column-dependent elements (like column names). +// Columns get implicitly removed once the descriptor is marked as dropped. +// +// The column transition to ABSENT needs to remain to clean up back-references +// in types referred to by the column in its type or its computed expression. +// +// We can't skip ops for column-dependent elements which reference other +// descriptors, like for example default expressions, again because of the need +// to clean up back-references. +// +// We also can't skip ops for column-dependent elements which don't like in the +// descriptor, like column comments (which live in a dedicated system table). +func init() { + + relation := MkNodeVars("relation") + column := MkNodeVars("column") + dep := MkNodeVars("column-dep") + relationID, columnID := rel.Var("relation-id"), rel.Var("column-id") + registerOpRule( + "skip column removal ops on relation drop", + column.Node, + screl.MustQuery( + relation.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + ), + column.Type( + (*scpb.Column)(nil), + ), + + JoinOnDescID(relation, column, relationID), + + relation.JoinTarget(), + relation.TargetStatus(scpb.ToAbsent), + column.JoinTargetNode(), + column.TargetStatus(scpb.ToAbsent), + column.CurrentStatus( + // All but DELETE_ONLY which is the status leading to ABSENT. + scpb.Status_PUBLIC, + scpb.Status_WRITE_ONLY, + ), + ), + ) + + registerOpRule( + "skip column dependents removal ops on relation drop", + dep.Node, + screl.MustQuery( + relation.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + ), + column.Type( + (*scpb.Column)(nil), + ), + dep.Type( + (*scpb.ColumnName)(nil), + ), + + JoinOnDescID(relation, column, relationID), + JoinOnColumnID(column, dep, relationID, columnID), + + relation.JoinTarget(), + relation.TargetStatus(scpb.ToAbsent), + column.JoinTarget(), + column.TargetStatus(scpb.ToAbsent), + dep.JoinTargetNode(), + dep.TargetStatus(scpb.ToAbsent), + ), + ) +} + +// When dropping a table or a view, skip all removal ops for index elements +// as well as elements which depend on them, provided there are no +// back-references that need to be cleaned up. This is similar as for columns. +func init() { + relation := MkNodeVars("relation") + index := MkNodeVars("index") + dep := MkNodeVars("index-dep") + relationID, indexID := rel.Var("relation-id"), rel.Var("index-id") + + registerOpRule( + "skip index removal ops on relation drop", + index.Node, + screl.MustQuery( + relation.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + ), + index.Type( + (*scpb.PrimaryIndex)(nil), + (*scpb.SecondaryIndex)(nil), + (*scpb.TemporaryIndex)(nil), + ), + + JoinOnDescID(relation, index, relationID), + + relation.JoinTarget(), + relation.TargetStatus(scpb.ToAbsent), + index.JoinTargetNode(), + index.TargetStatus(scpb.ToAbsent), + ), + ) + + registerOpRule( + "skip index dependents removal ops on relation drop", + dep.Node, + screl.MustQuery( + relation.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + ), + index.TypeFilter(IsIndex), + dep.Type( + (*scpb.IndexName)(nil), + (*scpb.IndexPartitioning)(nil), + (*scpb.IndexColumn)(nil), + ), + + JoinOnDescID(relation, index, relationID), + JoinOnIndexID(index, dep, relationID, indexID), + + relation.JoinTarget(), + relation.TargetStatus(scpb.ToAbsent), + index.JoinTarget(), + index.TargetStatus(scpb.ToAbsent), + dep.JoinTargetNode(), + dep.TargetStatus(scpb.ToAbsent), + ), + ) +} + +// When dropping a table or a view, skip all removal ops for constraint elements +// as well as elements which depend on them, provided there are no +// back-references that need to be cleaned up. This is similar as for columns +// and indexes. +func init() { + relation := MkNodeVars("relation") + constraint := MkNodeVars("constraint") + dep, constraintID := MkNodeVars("constraint-dep"), rel.Var("constraint-id") + relationID := rel.Var("relation-id") + + registerOpRule( + "skip constraint removal ops on relation drop", + constraint.Node, + screl.MustQuery( + relation.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + ), + constraint.Type( + (*scpb.CheckConstraint)(nil), + (*scpb.ForeignKeyConstraint)(nil), + (*scpb.UniqueWithoutIndexConstraint)(nil), + ), + + JoinOnDescID(relation, constraint, relationID), + + relation.JoinTarget(), + relation.TargetStatus(scpb.ToAbsent), + constraint.JoinTargetNode(), + constraint.TargetStatus(scpb.ToAbsent), + ), + ) + + registerOpRule( + "skip constraint dependents removal ops on relation drop", + dep.Node, + screl.MustQuery( + relation.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + ), + constraint.Type( + (*scpb.UniqueWithoutIndexConstraint)(nil), + (*scpb.CheckConstraint)(nil), + (*scpb.ForeignKeyConstraint)(nil), + ), + dep.Type( + (*scpb.ConstraintWithoutIndexName)(nil), + ), + + JoinOnDescID(relation, constraint, relationID), + JoinOnConstraintID(constraint, dep, relationID, constraintID), + + relation.JoinTarget(), + relation.TargetStatus(scpb.ToAbsent), + constraint.JoinTarget(), + constraint.TargetStatus(scpb.ToAbsent), + dep.JoinTargetNode(), + dep.TargetStatus(scpb.ToAbsent), + ), + ) +} + +// Skip all removal ops on elements which depend on a descriptor element that +// is being removed as well. These elements get implicitly removed once the +// descriptor is marked as dropped. +// This rule excludes: +// - index and column elements, which were handled above, +// - elements which don't live inside the descriptor, like comments, +// - elements which have forward references to other descriptors: back- +// references need to be cleaned up. +func init() { + desc := MkNodeVars("desc") + dep := MkNodeVars("dep") + descID := rel.Var("desc-id") + + registerOpRule( + "skip element removal ops on descriptor drop", + dep.Node, + screl.MustQuery( + desc.TypeFilter(IsDescriptor), + dep.Type( + (*scpb.ColumnFamily)(nil), + (*scpb.Owner)(nil), + (*scpb.UserPrivileges)(nil), + (*scpb.EnumTypeValue)(nil), + ), + + JoinOnDescID(desc, dep, descID), + + desc.JoinTarget(), + desc.TargetStatus(scpb.ToAbsent), + dep.JoinTargetNode(), + dep.TargetStatus(scpb.ToAbsent), + ), + ) +} + +// Skip all removal ops for dropping table comments corresponding to elements +// when dropping the table itself. +func init() { + desc := MkNodeVars("desc") + dep := MkNodeVars("dep") + descID := rel.Var("desc-id") + + registerOpRule( + "skip table comment removal ops on descriptor drop", + dep.Node, + screl.MustQuery( + desc.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + (*scpb.Sequence)(nil), + ), + dep.Type( + (*scpb.ColumnComment)(nil), + (*scpb.IndexComment)(nil), + (*scpb.ConstraintComment)(nil), + (*scpb.TableComment)(nil), + ), + + JoinOnDescID(desc, dep, descID), + + desc.JoinTarget(), + desc.TargetStatus(scpb.ToAbsent), + dep.JoinTargetNode(), + dep.TargetStatus(scpb.ToAbsent), + ), + ) +} + +// Skip all removal ops for table zone configs. +func init() { + desc := MkNodeVars("desc") + dep := MkNodeVars("dep") + descID := rel.Var("desc-id") + + registerOpRule( + "skip table zone config removal ops on descriptor drop", + dep.Node, + screl.MustQuery( + desc.Type( + (*scpb.Table)(nil), + (*scpb.View)(nil), + (*scpb.Sequence)(nil), + ), + dep.Type( + (*scpb.TableZoneConfig)(nil), + ), + + JoinOnDescID(desc, dep, descID), + + desc.JoinTarget(), + desc.TargetStatus(scpb.ToAbsent), + dep.JoinTargetNode(), + dep.TargetStatus(scpb.ToAbsent), + ), + ) +} + +// TODO(fqazi): For create operations we will need to have the ability +// to have transformations that will combine transitions into a single +// stage for execution. For example, a newly CREATE TABLE will be represented +// by a TABLE, COLUMN, and INDEX elements (among others), all of the operations +// for these elements are executable in a single stage. Having them execute +// across multiple stages would be problematic both from a validation and +// correctness viewpoint. diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_index_and_column.go new file mode 100644 index 000000000000..5a9253c4e49a --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_index_and_column.go @@ -0,0 +1,41 @@ +// 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" +) + +// Skill all IndexColumn removal ops for indexes which are also being removed. +func init() { + + ic := MkNodeVars("index-column") + index := MkNodeVars("index") + relationID, indexID := rel.Var("relation-id"), rel.Var("index-id") + + registerOpRule( + "skip index-column removal ops on index removal", + ic.Node, + screl.MustQuery( + ic.Type((*scpb.IndexColumn)(nil)), + index.TypeFilter(IsIndex), + JoinOnIndexID(ic, index, relationID, indexID), + ic.JoinTargetNode(), + ic.TargetStatus(scpb.ToAbsent, scpb.Transient), + ic.CurrentStatus(scpb.Status_PUBLIC, scpb.Status_TRANSIENT_PUBLIC), + index.JoinTarget(), + index.TargetStatus(scpb.ToAbsent, scpb.Transient), + ), + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/registry.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/registry.go new file mode 100644 index 000000000000..ae75d7257d0f --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/registry.go @@ -0,0 +1,58 @@ +// Copyright 2023 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 release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +var registry = NewRegistry() + +func registerDepRule( + ruleName scgraph.RuleName, + kind scgraph.DepEdgeKind, + fromEl, toEl string, + def func(from, to NodeVars) rel.Clauses, +) { + registry.RegisterDepRule(ruleName, + kind, + fromEl, toEl, + def) +} + +func registerOpRule(rn scgraph.RuleName, from rel.Var, q *rel.Query) { + registry.RegisterOpRule(rn, + from, + q) +} + +func registerDepRuleForDrop( + ruleName scgraph.RuleName, + kind scgraph.DepEdgeKind, + from, to string, + fromStatus, toStatus scpb.Status, + fn func(from, to NodeVars) rel.Clauses, +) { + RegisterDepRuleForDrop(registry, + ruleName, + kind, + from, to, + fromStatus, toStatus, + fn) +} + +// GetRegistry returns the registry for this cockroach release. +func GetRegistry() *Registry { + return registry +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/rules_test.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/rules_test.go new file mode 100644 index 000000000000..99382dcb5e58 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/rules_test.go @@ -0,0 +1,66 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package release_22_2 + +import ( + "sort" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v3" +) + +// TestRulesYAML outputs the rules to yaml as a way to visualize changes. +// Rules are sorted by name to ensure stable output. +func TestRulesYAML(t *testing.T) { + datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "rules": + var m yaml.Node + m.Kind = yaml.MappingNode + var s []rel.RuleDef + screl.Schema.ForEachRule(func(def rel.RuleDef) { + s = append(s, def) + }) + sort.Slice(s, func(i, j int) bool { + return s[i].Name < s[j].Name + }) + for _, def := range s { + var rule yaml.Node + if err := rule.Encode(def); err != nil { + panic(err) + } + m.Content = append(m.Content, rule.Content...) + } + out, err := yaml.Marshal(m) + if err != nil { + d.Fatalf(t, "failed to marshal rules: %v", err) + } + return string(out) + case "deprules": + out, err := registry.MarshalDepRules() + require.NoError(t, err) + return out + case "oprules": + out, err := registry.MarshalOpRules() + require.NoError(t, err) + return out + } + d.Fatalf(t, "deprules, oprules, and rules are the only commands, got %s", d.Cmd) + return "" + }) + }) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules new file mode 100644 index 000000000000..4fbc5c2b5509 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules @@ -0,0 +1,2497 @@ +rules +---- +ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id): + - $index-column[Type] = '*scpb.IndexColumn' + - $index-column[DescID] = $_ + - $index-column[ColumnID] = $column-id + - $index[IndexID] = $index-id + - joinOnIndexID($index, $index-column, $table-id, $index-id) +ColumnInSwappedInPrimaryIndex($index-column, $index, $table-id, $column-id, $index-id): + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - sourceIndexIsSet($index) +DescriptorIsNotBeingDropped($element): + not-join: + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - joinTarget($descriptor, $descriptor-Target) + - joinOnDescID($descriptor, $element, $id) + - $descriptor-Target[TargetStatus] = ABSENT +ToPublicOrTransient($target1, $target2): + - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] +fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($fromTarget, $fromEl, $toEl): + not-join: + - $fromEl[Type] = '*scpb.Table' + - $toEl[Type] = '*scpb.RowLevelTTL' + - $n[Type] = '*screl.Node' + - $n[Target] = $fromTarget + - nodeHasNoPublicStatus($n) +joinOnColumnID($a, $b, $desc-id, $col-id): + - joinOnDescID($a, $b, $desc-id) + - $a[ColumnID] = $col-id + - $b[ColumnID] = $col-id +joinOnConstraintID($a, $b, $desc-id, $constraint-id): + - joinOnDescID($a, $b, $desc-id) + - $a[ConstraintID] = $constraint-id + - $b[ConstraintID] = $constraint-id +joinOnDescID($a, $b, $id): + - $a[DescID] = $id + - $b[DescID] = $id +joinOnIndexID($a, $b, $desc-id, $index-id): + - joinOnDescID($a, $b, $desc-id) + - $a[IndexID] = $index-id + - $b[IndexID] = $index-id +joinReferencedDescID($referrer, $referenced, $id): + - $referrer[ReferencedDescID] = $id + - $referenced[DescID] = $id +joinTarget($element, $target): + - $target[Type] = '*scpb.Target' + - $target[Element] = $element + - $element[DescID] = $_ +joinTargetNode($element, $target, $node): + - joinTarget($element, $target) + - $node[Type] = '*screl.Node' + - $node[Target] = $target +nodeHasNoPublicStatus($n): + not-join: + - $public = PUBLIC + - $n[CurrentStatus] = $public +nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [BACKFILLED, BACKFILL_ONLY] +nodeNotExistsWithStatusIn_TRANSIENT_BACKFILLED_TRANSIENT_BACKFILL_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_BACKFILLED, TRANSIENT_BACKFILL_ONLY] +nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_DELETE_ONLY] +? nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY_BACKFILLED_TRANSIENT_BACKFILLED_BACKFILL_ONLY_TRANSIENT_BACKFILL_ONLY($sharedTarget) +: not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_DELETE_ONLY, BACKFILLED, TRANSIENT_BACKFILLED, BACKFILL_ONLY, TRANSIENT_BACKFILL_ONLY] +nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_VALIDATED] +nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED_TRANSIENT_MERGE_ONLY_TRANSIENT_MERGED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_VALIDATED, TRANSIENT_MERGE_ONLY, TRANSIENT_MERGED] +nodeNotExistsWithStatusIn_VALIDATED_MERGE_ONLY_MERGED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [VALIDATED, MERGE_ONLY, MERGED] +nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_MERGE_ONLY_MERGED_TRANSIENT_MERGED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [VALIDATED, TRANSIENT_WRITE_ONLY, MERGE_ONLY, TRANSIENT_MERGE_ONLY, MERGED, TRANSIENT_MERGED] +nodeNotExistsWithStatusIn_WRITE_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [WRITE_ONLY] +sourceIndexIsSet($index): + - $index[SourceIndexID] != 0 +toAbsent($target1, $target2): + - $target1[TargetStatus] = ABSENT + - $target2[TargetStatus] = ABSENT +transient($target1, $target2): + - $target1[TargetStatus] = TRANSIENT_ABSENT + - $target2[TargetStatus] = TRANSIENT_ABSENT + +deprules +---- +- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.CheckConstraint' + - $next[Type] = '*scpb.CheckConstraint' + - $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: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.CheckConstraint' + - $next[Type] = '*scpb.CheckConstraint' + - $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: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.CheckConstraint' + - $next[Type] = '*scpb.CheckConstraint' + - $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: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.CheckConstraint' + - $next[Type] = '*scpb.CheckConstraint' + - $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: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.CheckConstraint' + - $next[Type] = '*scpb.CheckConstraint' + - $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: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.CheckConstraint' + - $next[Type] = '*scpb.CheckConstraint' + - $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: 'Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.Column' + - $next[Type] = '*scpb.Column' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = ABSENT + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.Column' + - $next[Type] = '*scpb.Column' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = PUBLIC + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.Column' + - $next[Type] = '*scpb.Column' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = WRITE_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.Column' + - $next[Type] = '*scpb.Column' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = ABSENT + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.Column' + - $next[Type] = '*scpb.Column' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.Column' + - $next[Type] = '*scpb.Column' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = WRITE_ONLY + - $next-Node[CurrentStatus] = PUBLIC + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: DEFAULT or ON UPDATE existence precedes writes to column + from: expr-Node + kind: Precedence + to: column-Node + query: + - $expr[Type] IN ['*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression'] + - $column[Type] = '*scpb.Column' + - joinOnColumnID($expr, $column, $table-id, $col-id) + - ToPublicOrTransient($expr-Target, $column-Target) + - $expr-Node[CurrentStatus] = PUBLIC + - $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 + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = BACKFILLED + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = BACKFILL_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = ABSENT + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY_BACKFILLED_TRANSIENT_BACKFILLED_BACKFILL_ONLY_TRANSIENT_BACKFILL_ONLY($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = MERGED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = MERGE_ONLY + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $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: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT + - $next-Node[CurrentStatus] = ABSENT + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILL_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_MERGED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_MERGE_ONLY + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED + - $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: TRANSIENT_WRITE_ONLY->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = VALIDATED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = WRITE_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_MERGE_ONLY_MERGED_TRANSIENT_MERGED($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = ABSENT + - $next-Node[CurrentStatus] = BACKFILL_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = BACKFILLED + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = BACKFILL_ONLY + - $next-Node[CurrentStatus] = BACKFILLED + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = MERGE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = MERGED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = MERGE_ONLY + - $next-Node[CurrentStatus] = MERGED + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $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: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $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 TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = ABSENT + - $next-Node[CurrentStatus] = BACKFILL_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = BACKFILLED + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = BACKFILL_ONLY + - $next-Node[CurrentStatus] = BACKFILLED + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = MERGE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = MERGED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGE_ONLY->MERGED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = MERGE_ONLY + - $next-Node[CurrentStatus] = MERGED + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: PUBLIC->TRANSIENT_VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = PUBLIC + - $next-Node[CurrentStatus] = TRANSIENT_VALIDATED + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->TRANSIENT_DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED + - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->TRANSIENT_DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILL_ONLY + - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - $next-Node[CurrentStatus] = TRANSIENT_ABSENT + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_TRANSIENT_BACKFILLED_TRANSIENT_BACKFILL_ONLY($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_MERGED->TRANSIENT_WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_MERGED + - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->TRANSIENT_WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_MERGE_ONLY + - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->TRANSIENT_WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED + - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_WRITE_ONLY->TRANSIENT_DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY + - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED_TRANSIENT_MERGE_ONLY_TRANSIENT_MERGED($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: VALIDATED->PUBLIC' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = VALIDATED + - $next-Node[CurrentStatus] = PUBLIC + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.PrimaryIndex' + - $next[Type] = '*scpb.PrimaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_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: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = BACKFILLED + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = BACKFILL_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = ABSENT + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = MERGED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = MERGE_ONLY + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $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: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = VALIDATED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = WRITE_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_VALIDATED_MERGE_ONLY_MERGED($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = ABSENT + - $next-Node[CurrentStatus] = BACKFILL_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = BACKFILLED + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = BACKFILL_ONLY + - $next-Node[CurrentStatus] = BACKFILLED + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = MERGE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = MERGED + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = PUBLIC + - $prev-Node[CurrentStatus] = MERGE_ONLY + - $next-Node[CurrentStatus] = MERGED + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $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: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.SecondaryIndex' + - $next[Type] = '*scpb.SecondaryIndex' + - $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: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = ABSENT + - DescriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY($prev-Target) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT + - $next-Node[CurrentStatus] = ABSENT + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = ABSENT + - $prev-Node[CurrentStatus] = WRITE_ONLY + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = ABSENT + - $next-Node[CurrentStatus] = DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = DELETE_ONLY + - $next-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - $next-Node[CurrentStatus] = TRANSIENT_ABSENT + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.TemporaryIndex' + - $next[Type] = '*scpb.TemporaryIndex' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-Target[Self] = $next-Target + - $prev-Target[TargetStatus] = TRANSIENT_ABSENT + - $prev-Node[CurrentStatus] = WRITE_ONLY + - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - DescriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-Target, $prev-Node) + - joinTargetNode($next, $next-Target, $next-Node) +- name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $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: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $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: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $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: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $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: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $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: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-Node + kind: PreviousTransactionPrecedence + to: next-Node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $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: column dependents exist before column becomes public + from: dependent-Node + kind: Precedence + to: column-Node + query: + - $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) + - ToPublicOrTransient($dependent-Target, $column-Target) + - $dependent-Node[CurrentStatus] = PUBLIC + - $column-Node[CurrentStatus] = PUBLIC + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: column existence precedes column dependents + from: column-Node + kind: Precedence + to: dependent-Node + query: + - $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) + - ToPublicOrTransient($column-Target, $dependent-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $dependent-Node[CurrentStatus] = PUBLIC + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: column existence precedes index existence + from: column-Node + kind: Precedence + to: index-Node + query: + - $column[Type] = '*scpb.Column' + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - joinOnColumnID($column, $index-column, $table-id, $column-id) + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - ToPublicOrTransient($column-Target, $index-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $index-Node[CurrentStatus] = BACKFILL_ONLY + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: column existence precedes temp index existence + from: column-Node + kind: Precedence + to: index-Node + query: + - $column[Type] = '*scpb.Column' + - $index[Type] = '*scpb.TemporaryIndex' + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - ToPublicOrTransient($column-Target, $index-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $index-Node[CurrentStatus] = DELETE_ONLY + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: column is WRITE_ONLY before temporary index is WRITE_ONLY + from: column-Node + kind: Precedence + to: index-Node + query: + - $column[Type] = '*scpb.Column' + - $index[Type] = '*scpb.TemporaryIndex' + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - ToPublicOrTransient($column-Target, $index-Target) + - $column-Node[CurrentStatus] = WRITE_ONLY + - $index-Node[CurrentStatus] = WRITE_ONLY + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: column name and type set right after column existence + from: column-Node + kind: SameStagePrecedence + to: column-name-or-type-Node + query: + - $column[Type] = '*scpb.Column' + - $column-name-or-type[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType'] + - ToPublicOrTransient($column-Target, $column-name-or-type-Target) + - $column-Node[CurrentStatus] = DELETE_ONLY + - $column-name-or-type-Node[CurrentStatus] = PUBLIC + - joinOnColumnID($column, $column-name-or-type, $table-id, $col-id) + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($column-name-or-type, $column-name-or-type-Target, $column-name-or-type-Node) +- name: column no longer public before dependents + from: column-Node + kind: Precedence + to: dependent-Node + query: + - $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 + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: column no longer public before dependents + from: column-Node + kind: Precedence + to: dependent-Node + query: + - $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-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 + from: column-Node + kind: Precedence + to: dependent-Node + query: + - $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 + - joinTargetNode($column, $column-Target, $column-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: column no longer public before dependents + from: column-Node + kind: Precedence + to: dependent-Node + query: + - $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 + - $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 + from: dependent-Node + kind: SameStagePrecedence + to: column-type-Node + query: + - $dependent[Type] IN ['*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner'] + - $column-type[Type] = '*scpb.ColumnType' + - joinOnColumnID($dependent, $column-type, $table-id, $col-id) + - toAbsent($dependent-Target, $column-type-Target) + - $dependent-Node[CurrentStatus] = ABSENT + - $column-type-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($column-type, $column-type-Target, $column-type-Node) +- name: column type removed right before column when rules.Not dropping relation + from: column-type-Node + kind: SameStagePrecedence + to: column-Node + query: + - $column-type[Type] = '*scpb.ColumnType' + - DescriptorIsNotBeingDropped($column-type) + - $column[Type] = '*scpb.Column' + - joinOnColumnID($column-type, $column, $table-id, $col-id) + - toAbsent($column-type-Target, $column-Target) + - $column-type-Node[CurrentStatus] = ABSENT + - $column-Node[CurrentStatus] = ABSENT + - joinTargetNode($column-type, $column-type-Target, $column-type-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: Precedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - toAbsent($dependent-Target, $constraint-Target) + - $dependent-Node[CurrentStatus] = ABSENT + - $constraint-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: Precedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - transient($dependent-Target, $constraint-Target) + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - $constraint-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: Precedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - $dependent-Target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - $constraint-Target[TargetStatus] = ABSENT + - $constraint-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: Precedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - $dependent-Target[TargetStatus] = ABSENT + - $dependent-Node[CurrentStatus] = ABSENT + - $constraint-Target[TargetStatus] = TRANSIENT_ABSENT + - $constraint-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: SameStagePrecedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - toAbsent($dependent-Target, $constraint-Target) + - $dependent-Node[CurrentStatus] = VALIDATED + - $constraint-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: SameStagePrecedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - transient($dependent-Target, $constraint-Target) + - $dependent-Node[CurrentStatus] = TRANSIENT_VALIDATED + - $constraint-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: SameStagePrecedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - $dependent-Target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-Node[CurrentStatus] = TRANSIENT_VALIDATED + - $constraint-Target[TargetStatus] = ABSENT + - $constraint-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent absent right before constraint + from: dependent-Node + kind: SameStagePrecedence + to: constraint-Node + query: + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnConstraintID($dependent, $constraint, $table-id, $constraint-id) + - $dependent-Target[TargetStatus] = ABSENT + - $dependent-Node[CurrentStatus] = VALIDATED + - $constraint-Target[TargetStatus] = TRANSIENT_ABSENT + - $constraint-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: constraint dependent public right before constraint + from: constraint-Node + kind: SameStagePrecedence + to: dependent-Node + query: + - $constraint[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] + - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) + - ToPublicOrTransient($constraint-Target, $dependent-Target) + - $constraint-Node[CurrentStatus] = PUBLIC + - $dependent-Node[CurrentStatus] = PUBLIC + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: dependents removed before column + from: dependent-Node + kind: Precedence + to: column-Node + query: + - $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 + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: dependents removed before column + from: dependent-Node + kind: Precedence + to: column-Node + query: + - $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 + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: dependents removed before column + from: dependent-Node + kind: Precedence + to: column-Node + query: + - $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 + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: dependents removed before column + from: dependent-Node + kind: Precedence + to: column-Node + query: + - $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 + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: dependents removed before index + from: dependent-Node + kind: Precedence + to: index-Node + query: + - $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) + - toAbsent($dependent-Target, $index-Target) + - $dependent-Node[CurrentStatus] = ABSENT + - $index-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: dependents removed before index + from: dependent-Node + kind: Precedence + to: index-Node + query: + - $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-Node[CurrentStatus] = TRANSIENT_ABSENT + - $index-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: dependents removed before index + from: dependent-Node + kind: Precedence + to: index-Node + query: + - $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 + - $dependent-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 + from: dependent-Node + kind: Precedence + to: index-Node + query: + - $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] = ABSENT + - $dependent-Node[CurrentStatus] = ABSENT + - $index-Target[TargetStatus] = TRANSIENT_ABSENT + - $index-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: descriptor DROPPED in transaction before removal + from: dropped-Node + kind: PreviousTransactionPrecedence + to: absent-Node + query: + - $dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $dropped[DescID] = $_ + - $dropped[Self] = $absent + - toAbsent($dropped-Target, $absent-Target) + - $dropped-Node[CurrentStatus] = DROPPED + - $absent-Node[CurrentStatus] = ABSENT + - joinTargetNode($dropped, $dropped-Target, $dropped-Node) + - joinTargetNode($absent, $absent-Target, $absent-Node) +- name: descriptor TXN_DROPPED before DROPPED + from: txn_dropped-Node + kind: PreviousStagePrecedence + to: dropped-Node + query: + - $txn_dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $txn_dropped[DescID] = $_ + - $txn_dropped[Self] = $dropped + - toAbsent($txn_dropped-Target, $dropped-Target) + - $txn_dropped-Node[CurrentStatus] = TXN_DROPPED + - $dropped-Node[CurrentStatus] = DROPPED + - joinTargetNode($txn_dropped, $txn_dropped-Target, $txn_dropped-Node) + - joinTargetNode($dropped, $dropped-Target, $dropped-Node) +- name: descriptor drop right before dependent element removal + from: descriptor-Node + kind: SameStagePrecedence + 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.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 + - $dependent-Node[CurrentStatus] = ABSENT + - fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($descriptor-Target, $descriptor, $dependent) + - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: descriptor drop right before removing dependent with attr ref + from: referenced-descriptor-Node + kind: SameStagePrecedence + 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.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 + - $referencing-via-attr-Node[CurrentStatus] = ABSENT + - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) + - joinTargetNode($referencing-via-attr, $referencing-via-attr-Target, $referencing-via-attr-Node) +- name: descriptor drop right before removing dependent with expr ref to sequence + from: referenced-descriptor-Node + kind: SameStagePrecedence + to: referencing-via-expr-Node + query: + - $referenced-descriptor[Type] = '*scpb.Sequence' + - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) + - $referenced-descriptor[DescID] = $seqID + - $referencing-via-expr[ReferencedSequenceIDs] CONTAINS $seqID + - $referencing-via-expr[Type] IN ['*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] + - toAbsent($referenced-descriptor-Target, $referencing-via-expr-Target) + - $referenced-descriptor-Node[CurrentStatus] = DROPPED + - $referencing-via-expr-Node[CurrentStatus] = ABSENT + - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) + - joinTargetNode($referencing-via-expr, $referencing-via-expr-Target, $referencing-via-expr-Node) +- name: descriptor drop right before removing dependent with type ref + from: referenced-descriptor-Node + kind: SameStagePrecedence + to: referencing-via-type-Node + query: + - $referenced-descriptor[Type] IN ['*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) + - $referenced-descriptor[DescID] = $fromDescID + - $referencing-via-type[ReferencedTypeIDs] CONTAINS $fromDescID + - $referencing-via-type[Type] IN ['*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] + - toAbsent($referenced-descriptor-Target, $referencing-via-type-Target) + - $referenced-descriptor-Node[CurrentStatus] = DROPPED + - $referencing-via-type-Node[CurrentStatus] = ABSENT + - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) + - joinTargetNode($referencing-via-type, $referencing-via-type-Target, $referencing-via-type-Node) +- name: descriptor removal right before dependent element removal + from: descriptor-Node + kind: SameStagePrecedence + to: idx-or-col-Node + query: + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $idx-or-col[Type] IN ['*scpb.Column', '*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - joinOnDescID($descriptor, $idx-or-col, $desc-id) + - toAbsent($descriptor-Target, $idx-or-col-Target) + - $descriptor-Node[CurrentStatus] = ABSENT + - $idx-or-col-Node[CurrentStatus] = ABSENT + - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) + - joinTargetNode($idx-or-col, $idx-or-col-Target, $idx-or-col-Node) +- name: ensure columns are in increasing order + from: later-column-Node + kind: SameStagePrecedence + to: earlier-column-Node + query: + - $later-column[Type] = '*scpb.Column' + - $earlier-column[Type] = '*scpb.Column' + - joinOnDescID($later-column, $earlier-column, $table-id) + - ToPublicOrTransient($later-column-Target, $earlier-column-Target) + - $status IN [WRITE_ONLY, PUBLIC] + - $later-column-Node[CurrentStatus] = $status + - $earlier-column-Node[CurrentStatus] = $status + - SmallerColumnIDFirst(*scpb.Column, *scpb.Column)($later-column, $earlier-column) + - joinTargetNode($later-column, $later-column-Target, $later-column-Node) + - joinTargetNode($earlier-column, $earlier-column-Target, $earlier-column-Node) +- name: index dependents exist before index becomes public + from: dependent-Node + kind: Precedence + to: index-Node + query: + - $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) + - ToPublicOrTransient($dependent-Target, $index-Target) + - $dependent-Node[CurrentStatus] = PUBLIC + - $index-Node[CurrentStatus] = PUBLIC + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: index existence precedes index dependents + from: index-Node + kind: Precedence + to: dependent-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] + - joinOnIndexID($index, $dependent, $table-id, $index-id) + - ToPublicOrTransient($index-Target, $dependent-Target) + - $index-Node[CurrentStatus] = BACKFILL_ONLY + - $dependent-Node[CurrentStatus] = PUBLIC + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: index named right before index becomes public + from: index-name-Node + kind: SameStagePrecedence + to: index-Node + query: + - $index-name[Type] = '*scpb.IndexName' + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - joinOnIndexID($index-name, $index, $table-id, $index-id) + - ToPublicOrTransient($index-name-Target, $index-Target) + - $index-name-Node[CurrentStatus] = PUBLIC + - $index-Node[CurrentStatus] = PUBLIC + - joinTargetNode($index-name, $index-name-Target, $index-name-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: index no longer public before dependents + from: index-Node + kind: Precedence + to: dependent-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] + - joinOnIndexID($index, $dependent, $table-id, $index-id) + - toAbsent($index-Target, $dependent-Target) + - $index-Node[CurrentStatus] = VALIDATED + - $dependent-Node[CurrentStatus] = ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: index no longer public before dependents + from: index-Node + kind: Precedence + to: dependent-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] + - joinOnIndexID($index, $dependent, $table-id, $index-id) + - transient($index-Target, $dependent-Target) + - $index-Node[CurrentStatus] = TRANSIENT_VALIDATED + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: index no longer public before dependents + from: index-Node + kind: Precedence + to: dependent-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] + - joinOnIndexID($index, $dependent, $table-id, $index-id) + - $index-Target[TargetStatus] = TRANSIENT_ABSENT + - $index-Node[CurrentStatus] = TRANSIENT_VALIDATED + - $dependent-Target[TargetStatus] = ABSENT + - $dependent-Node[CurrentStatus] = ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: index no longer public before dependents + from: index-Node + kind: Precedence + to: dependent-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] + - joinOnIndexID($index, $dependent, $table-id, $index-id) + - $index-Target[TargetStatus] = ABSENT + - $index-Node[CurrentStatus] = VALIDATED + - $dependent-Target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: index-column added to index before index is backfilled + from: index-column-Node + kind: Precedence + to: index-Node + query: + - $index-column[Type] = '*scpb.IndexColumn' + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - joinOnIndexID($index-column, $index, $table-id, $index-id) + - ToPublicOrTransient($index-column-Target, $index-Target) + - $index-column-Node[CurrentStatus] = PUBLIC + - $index-Node[CurrentStatus] = BACKFILLED + - joinTargetNode($index-column, $index-column-Target, $index-column-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: index-column added to index before temp index receives writes + from: index-column-Node + kind: Precedence + to: index-Node + query: + - $index-column[Type] = '*scpb.IndexColumn' + - $index[Type] = '*scpb.TemporaryIndex' + - joinOnIndexID($index-column, $index, $table-id, $index-id) + - transient($index-column-Target, $index-Target) + - $index-column-Node[CurrentStatus] = PUBLIC + - $index-Node[CurrentStatus] = WRITE_ONLY + - joinTargetNode($index-column, $index-column-Target, $index-column-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: indexes containing column reach absent before column + from: index-Node + kind: Precedence + to: column-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - $column[Type] = '*scpb.Column' + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - DescriptorIsNotBeingDropped($index-column) + - toAbsent($index-Target, $column-Target) + - $index-Node[CurrentStatus] = ABSENT + - $column-Node[CurrentStatus] = ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: indexes containing column reach absent before column + from: index-Node + kind: Precedence + to: column-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - $column[Type] = '*scpb.Column' + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - DescriptorIsNotBeingDropped($index-column) + - transient($index-Target, $column-Target) + - $index-Node[CurrentStatus] = TRANSIENT_ABSENT + - $column-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: indexes containing column reach absent before column + from: index-Node + kind: Precedence + to: column-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - $column[Type] = '*scpb.Column' + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - DescriptorIsNotBeingDropped($index-column) + - $index-Target[TargetStatus] = TRANSIENT_ABSENT + - $index-Node[CurrentStatus] = TRANSIENT_ABSENT + - $column-Target[TargetStatus] = ABSENT + - $column-Node[CurrentStatus] = ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: indexes containing column reach absent before column + from: index-Node + kind: Precedence + to: column-Node + query: + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - $column[Type] = '*scpb.Column' + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - DescriptorIsNotBeingDropped($index-column) + - $index-Target[TargetStatus] = ABSENT + - $index-Node[CurrentStatus] = ABSENT + - $column-Target[TargetStatus] = TRANSIENT_ABSENT + - $column-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: old index absent before new index public when swapping with transient + from: old-primary-index-Node + kind: Precedence + to: new-primary-index-Node + query: + - $old-primary-index[Type] = '*scpb.PrimaryIndex' + - $transient-primary-index[Type] = '*scpb.PrimaryIndex' + - $new-primary-index[Type] = '*scpb.PrimaryIndex' + - joinOnDescID($old-primary-index, $transient-primary-index, $table-id) + - $old-primary-index[IndexID] = $old-index-id + - $transient-primary-index[SourceIndexID] = $old-index-id + - joinOnDescID($transient-primary-index, $new-primary-index, $table-id) + - $transient-primary-index[IndexID] = $transient-index-id + - $new-primary-index[SourceIndexID] = $transient-index-id + - $old-primary-index-Target[TargetStatus] = ABSENT + - $old-primary-index-Node[CurrentStatus] = ABSENT + - $new-primary-index-Target[TargetStatus] = PUBLIC + - $new-primary-index-Node[CurrentStatus] = PUBLIC + - joinTargetNode($old-primary-index, $old-primary-index-Target, $old-primary-index-Node) + - joinTargetNode($new-primary-index, $new-primary-index-Target, $new-primary-index-Node) +- name: partial predicate removed right before secondary index when rules.Not dropping relation + from: partial-predicate-Node + kind: SameStagePrecedence + to: index-Node + query: + - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - DescriptorIsNotBeingDropped($partial-predicate) + - $index[Type] = '*scpb.SecondaryIndex' + - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) + - toAbsent($partial-predicate-Target, $index-Target) + - $partial-predicate-Node[CurrentStatus] = ABSENT + - $index-Node[CurrentStatus] = ABSENT + - joinTargetNode($partial-predicate, $partial-predicate-Target, $partial-predicate-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: partial predicate removed right before secondary index when rules.Not dropping relation + from: partial-predicate-Node + kind: SameStagePrecedence + to: index-Node + query: + - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - DescriptorIsNotBeingDropped($partial-predicate) + - $index[Type] = '*scpb.SecondaryIndex' + - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) + - transient($partial-predicate-Target, $index-Target) + - $partial-predicate-Node[CurrentStatus] = TRANSIENT_ABSENT + - $index-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($partial-predicate, $partial-predicate-Target, $partial-predicate-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: partial predicate removed right before secondary index when rules.Not dropping relation + from: partial-predicate-Node + kind: SameStagePrecedence + to: index-Node + query: + - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - DescriptorIsNotBeingDropped($partial-predicate) + - $index[Type] = '*scpb.SecondaryIndex' + - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) + - $partial-predicate-Target[TargetStatus] = TRANSIENT_ABSENT + - $partial-predicate-Node[CurrentStatus] = TRANSIENT_ABSENT + - $index-Target[TargetStatus] = ABSENT + - $index-Node[CurrentStatus] = ABSENT + - joinTargetNode($partial-predicate, $partial-predicate-Target, $partial-predicate-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: partial predicate removed right before secondary index when rules.Not dropping relation + from: partial-predicate-Node + kind: SameStagePrecedence + to: index-Node + query: + - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - DescriptorIsNotBeingDropped($partial-predicate) + - $index[Type] = '*scpb.SecondaryIndex' + - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) + - $partial-predicate-Target[TargetStatus] = ABSENT + - $partial-predicate-Node[CurrentStatus] = ABSENT + - $index-Target[TargetStatus] = TRANSIENT_ABSENT + - $index-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($partial-predicate, $partial-predicate-Target, $partial-predicate-Node) + - joinTargetNode($index, $index-Target, $index-Node) +- name: primary index swap + from: old-index-Node + kind: SameStagePrecedence + to: new-index-Node + query: + - $old-index[Type] = '*scpb.PrimaryIndex' + - $new-index[Type] = '*scpb.PrimaryIndex' + - joinOnDescID($old-index, $new-index, $table-id) + - $old-index[IndexID] = $old-index-id + - $new-index[SourceIndexID] = $old-index-id + - $old-index-Target[TargetStatus] = ABSENT + - $old-index-Node[CurrentStatus] = VALIDATED + - $new-index-Target[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $new-index-Node[CurrentStatus] = PUBLIC + - joinTargetNode($old-index, $old-index-Target, $old-index-Node) + - joinTargetNode($new-index, $new-index-Target, $new-index-Node) +- name: primary index swap + from: old-index-Node + kind: SameStagePrecedence + to: new-index-Node + query: + - $old-index[Type] = '*scpb.PrimaryIndex' + - $new-index[Type] = '*scpb.PrimaryIndex' + - joinOnDescID($old-index, $new-index, $table-id) + - $old-index[IndexID] = $old-index-id + - $new-index[SourceIndexID] = $old-index-id + - $old-index-Target[TargetStatus] = TRANSIENT_ABSENT + - $old-index-Node[CurrentStatus] = TRANSIENT_VALIDATED + - $new-index-Target[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $new-index-Node[CurrentStatus] = PUBLIC + - joinTargetNode($old-index, $old-index-Target, $old-index-Node) + - joinTargetNode($new-index, $new-index-Target, $new-index-Node) +- name: primary index swap + from: new-index-Node + kind: SameStagePrecedence + to: old-index-Node + query: + - $new-index[Type] = '*scpb.PrimaryIndex' + - $old-index[Type] = '*scpb.PrimaryIndex' + - joinOnDescID($new-index, $old-index, $table-id) + - $new-index[SourceIndexID] = $old-index-id + - $old-index[IndexID] = $old-index-id + - $new-index-Target[TargetStatus] = ABSENT + - $new-index-Node[CurrentStatus] = VALIDATED + - $old-index-Target[TargetStatus] = PUBLIC + - $old-index-Node[CurrentStatus] = PUBLIC + - joinTargetNode($new-index, $new-index-Target, $new-index-Node) + - joinTargetNode($old-index, $old-index-Target, $old-index-Node) +- name: primary index with new columns should exist before secondary indexes + from: primary-index-Node + kind: Precedence + to: secondary-index-Node + query: + - $primary-index[Type] = '*scpb.PrimaryIndex' + - $secondary-index[Type] = '*scpb.SecondaryIndex' + - joinOnDescID($primary-index, $secondary-index, $table-id) + - $primary-index[IndexID] = $primary-index-id + - $secondary-index[SourceIndexID] = $primary-index-id + - ToPublicOrTransient($primary-index-Target, $secondary-index-Target) + - $primary-index-Node[CurrentStatus] = PUBLIC + - $secondary-index-Node[CurrentStatus] = BACKFILL_ONLY + - joinTargetNode($primary-index, $primary-index-Target, $primary-index-Node) + - joinTargetNode($secondary-index, $secondary-index-Target, $secondary-index-Node) +- name: primary index with new columns should exist before temp indexes + from: primary-index-Node + kind: Precedence + to: temp-index-Node + query: + - $primary-index[Type] = '*scpb.PrimaryIndex' + - $temp-index[Type] = '*scpb.TemporaryIndex' + - joinOnDescID($primary-index, $temp-index, $table-id) + - $primary-index[IndexID] = $primary-index-id + - $temp-index[SourceIndexID] = $primary-index-id + - ToPublicOrTransient($primary-index-Target, $temp-index-Target) + - $primary-index-Node[CurrentStatus] = PUBLIC + - $temp-index-Node[CurrentStatus] = DELETE_ONLY + - joinTargetNode($primary-index, $primary-index-Target, $primary-index-Node) + - joinTargetNode($temp-index, $temp-index-Target, $temp-index-Node) +- name: remove columns from index right before removing index + from: index-Node + kind: Precedence + to: index-column-Node + query: + - $index[Type] = '*scpb.IndexColumn' + - $index-column[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnIndexID($index, $index-column, $table-id, $index-id) + - toAbsent($index-Target, $index-column-Target) + - $index-Node[CurrentStatus] = DELETE_ONLY + - $index-column-Node[CurrentStatus] = ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($index-column, $index-column-Target, $index-column-Node) +- name: remove columns from index right before removing index + from: index-Node + kind: Precedence + to: index-column-Node + query: + - $index[Type] = '*scpb.IndexColumn' + - $index-column[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnIndexID($index, $index-column, $table-id, $index-id) + - transient($index-Target, $index-column-Target) + - $index-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - $index-column-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($index-column, $index-column-Target, $index-column-Node) +- name: remove columns from index right before removing index + from: index-Node + kind: Precedence + to: index-column-Node + query: + - $index[Type] = '*scpb.IndexColumn' + - $index-column[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnIndexID($index, $index-column, $table-id, $index-id) + - $index-Target[TargetStatus] = TRANSIENT_ABSENT + - $index-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY + - $index-column-Target[TargetStatus] = ABSENT + - $index-column-Node[CurrentStatus] = ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($index-column, $index-column-Target, $index-column-Node) +- name: remove columns from index right before removing index + from: index-Node + kind: Precedence + to: index-column-Node + query: + - $index[Type] = '*scpb.IndexColumn' + - $index-column[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnIndexID($index, $index-column, $table-id, $index-id) + - $index-Target[TargetStatus] = ABSENT + - $index-Node[CurrentStatus] = DELETE_ONLY + - $index-column-Target[TargetStatus] = TRANSIENT_ABSENT + - $index-column-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($index-column, $index-column-Target, $index-column-Node) +- name: secondary indexes containing column as key reach write-only before column + from: index-Node + kind: Precedence + to: column-Node + query: + - $index[Type] = '*scpb.SecondaryIndex' + - $column[Type] = '*scpb.Column' + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - toAbsent($index-Target, $column-Target) + - $index-Node[CurrentStatus] = VALIDATED + - $column-Node[CurrentStatus] = WRITE_ONLY + - DescriptorIsNotBeingDropped($index-column) + - rules.IsIndexKeyColumnKey(*scpb.IndexColumn)($index-column) + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: swapped primary index public before column + from: index-Node + kind: Precedence + to: column-Node + query: + - $index[Type] = '*scpb.PrimaryIndex' + - $column[Type] = '*scpb.Column' + - ColumnInSwappedInPrimaryIndex($index-column, $index, $table-id, $column-id, $index-id) + - joinOnColumnID($index-column, $column, $table-id, $column-id) + - ToPublicOrTransient($index-Target, $column-Target) + - $index-Node[CurrentStatus] = PUBLIC + - $column-Node[CurrentStatus] = PUBLIC + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: temp index existence precedes index dependents + from: index-Node + kind: Precedence + to: dependent-Node + query: + - $index[Type] = '*scpb.TemporaryIndex' + - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] + - joinOnIndexID($index, $dependent, $table-id, $index-id) + - ToPublicOrTransient($index-Target, $dependent-Target) + - $index-Node[CurrentStatus] = DELETE_ONLY + - $dependent-Node[CurrentStatus] = PUBLIC + - joinTargetNode($index, $index-Target, $index-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: temp index is WRITE_ONLY before backfill + from: temp-Node + kind: Precedence + to: index-Node + query: + - $temp[Type] = '*scpb.TemporaryIndex' + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] + - joinOnDescID($temp, $index, $table-id) + - $temp[IndexID] = $temp-index-id + - $index[TemporaryIndexID] = $temp-index-id + - $temp-Target[TargetStatus] = TRANSIENT_ABSENT + - $index-Target[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $temp-Node[CurrentStatus] = WRITE_ONLY + - $index-Node[CurrentStatus] = BACKFILLED + - joinTargetNode($temp, $temp-Target, $temp-Node) + - joinTargetNode($index, $index-Target, $index-Node) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules new file mode 100644 index 000000000000..37682f812530 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules @@ -0,0 +1,227 @@ +rules +---- +ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id): + - $index-column[Type] = '*scpb.IndexColumn' + - $index-column[DescID] = $_ + - $index-column[ColumnID] = $column-id + - $index[IndexID] = $index-id + - joinOnIndexID($index, $index-column, $table-id, $index-id) +ColumnInSwappedInPrimaryIndex($index-column, $index, $table-id, $column-id, $index-id): + - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) + - sourceIndexIsSet($index) +DescriptorIsNotBeingDropped($element): + not-join: + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - joinTarget($descriptor, $descriptor-Target) + - joinOnDescID($descriptor, $element, $id) + - $descriptor-Target[TargetStatus] = ABSENT +ToPublicOrTransient($target1, $target2): + - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] +fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($fromTarget, $fromEl, $toEl): + not-join: + - $fromEl[Type] = '*scpb.Table' + - $toEl[Type] = '*scpb.RowLevelTTL' + - $n[Type] = '*screl.Node' + - $n[Target] = $fromTarget + - nodeHasNoPublicStatus($n) +joinOnColumnID($a, $b, $desc-id, $col-id): + - joinOnDescID($a, $b, $desc-id) + - $a[ColumnID] = $col-id + - $b[ColumnID] = $col-id +joinOnConstraintID($a, $b, $desc-id, $constraint-id): + - joinOnDescID($a, $b, $desc-id) + - $a[ConstraintID] = $constraint-id + - $b[ConstraintID] = $constraint-id +joinOnDescID($a, $b, $id): + - $a[DescID] = $id + - $b[DescID] = $id +joinOnIndexID($a, $b, $desc-id, $index-id): + - joinOnDescID($a, $b, $desc-id) + - $a[IndexID] = $index-id + - $b[IndexID] = $index-id +joinReferencedDescID($referrer, $referenced, $id): + - $referrer[ReferencedDescID] = $id + - $referenced[DescID] = $id +joinTarget($element, $target): + - $target[Type] = '*scpb.Target' + - $target[Element] = $element + - $element[DescID] = $_ +joinTargetNode($element, $target, $node): + - joinTarget($element, $target) + - $node[Type] = '*screl.Node' + - $node[Target] = $target +nodeHasNoPublicStatus($n): + not-join: + - $public = PUBLIC + - $n[CurrentStatus] = $public +nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [BACKFILLED, BACKFILL_ONLY] +nodeNotExistsWithStatusIn_TRANSIENT_BACKFILLED_TRANSIENT_BACKFILL_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_BACKFILLED, TRANSIENT_BACKFILL_ONLY] +nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_DELETE_ONLY] +? nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY_BACKFILLED_TRANSIENT_BACKFILLED_BACKFILL_ONLY_TRANSIENT_BACKFILL_ONLY($sharedTarget) +: not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_DELETE_ONLY, BACKFILLED, TRANSIENT_BACKFILLED, BACKFILL_ONLY, TRANSIENT_BACKFILL_ONLY] +nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_VALIDATED] +nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED_TRANSIENT_MERGE_ONLY_TRANSIENT_MERGED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [TRANSIENT_VALIDATED, TRANSIENT_MERGE_ONLY, TRANSIENT_MERGED] +nodeNotExistsWithStatusIn_VALIDATED_MERGE_ONLY_MERGED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [VALIDATED, MERGE_ONLY, MERGED] +nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_MERGE_ONLY_MERGED_TRANSIENT_MERGED($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [VALIDATED, TRANSIENT_WRITE_ONLY, MERGE_ONLY, TRANSIENT_MERGE_ONLY, MERGED, TRANSIENT_MERGED] +nodeNotExistsWithStatusIn_WRITE_ONLY($sharedTarget): + not-join: + - $n[Type] = '*screl.Node' + - $n[Target] = $sharedTarget + - $n[CurrentStatus] IN [WRITE_ONLY] +sourceIndexIsSet($index): + - $index[SourceIndexID] != 0 +toAbsent($target1, $target2): + - $target1[TargetStatus] = ABSENT + - $target2[TargetStatus] = ABSENT +transient($target1, $target2): + - $target1[TargetStatus] = TRANSIENT_ABSENT + - $target2[TargetStatus] = TRANSIENT_ABSENT + +oprules +---- +- name: skip column dependents removal ops on relation drop + from: column-dep-Node + query: + - $relation[Type] IN ['*scpb.Table', '*scpb.View'] + - $column[Type] = '*scpb.Column' + - $column-dep[Type] = '*scpb.ColumnName' + - joinOnDescID($relation, $column, $relation-id) + - joinOnColumnID($column, $column-dep, $relation-id, $column-id) + - joinTarget($relation, $relation-Target) + - $relation-Target[TargetStatus] = ABSENT + - joinTarget($column, $column-Target) + - $column-Target[TargetStatus] = ABSENT + - joinTargetNode($column-dep, $column-dep-Target, $column-dep-Node) + - $column-dep-Target[TargetStatus] = ABSENT +- name: skip column removal ops on relation drop + from: column-Node + query: + - $relation[Type] IN ['*scpb.Table', '*scpb.View'] + - $column[Type] = '*scpb.Column' + - joinOnDescID($relation, $column, $relation-id) + - joinTarget($relation, $relation-Target) + - $relation-Target[TargetStatus] = ABSENT + - joinTargetNode($column, $column-Target, $column-Node) + - $column-Target[TargetStatus] = ABSENT + - $column-Node[CurrentStatus] IN [PUBLIC, WRITE_ONLY] +- name: skip constraint dependents removal ops on relation drop + from: constraint-dep-Node + query: + - $relation[Type] IN ['*scpb.Table', '*scpb.View'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint-dep[Type] = '*scpb.ConstraintWithoutIndexName' + - joinOnDescID($relation, $constraint, $relation-id) + - joinOnConstraintID($constraint, $constraint-dep, $relation-id, $constraint-id) + - joinTarget($relation, $relation-Target) + - $relation-Target[TargetStatus] = ABSENT + - joinTarget($constraint, $constraint-Target) + - $constraint-Target[TargetStatus] = ABSENT + - joinTargetNode($constraint-dep, $constraint-dep-Target, $constraint-dep-Node) + - $constraint-dep-Target[TargetStatus] = ABSENT +- name: skip constraint removal ops on relation drop + from: constraint-Node + query: + - $relation[Type] IN ['*scpb.Table', '*scpb.View'] + - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.UniqueWithoutIndexConstraint'] + - joinOnDescID($relation, $constraint, $relation-id) + - joinTarget($relation, $relation-Target) + - $relation-Target[TargetStatus] = ABSENT + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) + - $constraint-Target[TargetStatus] = ABSENT +- name: skip element removal ops on descriptor drop + from: dep-Node + query: + - $desc[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $dep[Type] IN ['*scpb.ColumnFamily', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.EnumTypeValue'] + - joinOnDescID($desc, $dep, $desc-id) + - joinTarget($desc, $desc-Target) + - $desc-Target[TargetStatus] = ABSENT + - joinTargetNode($dep, $dep-Target, $dep-Node) + - $dep-Target[TargetStatus] = ABSENT +- name: skip index dependents removal ops on relation drop + from: index-dep-Node + query: + - $relation[Type] IN ['*scpb.Table', '*scpb.View'] + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - $index-dep[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexColumn'] + - joinOnDescID($relation, $index, $relation-id) + - joinOnIndexID($index, $index-dep, $relation-id, $index-id) + - joinTarget($relation, $relation-Target) + - $relation-Target[TargetStatus] = ABSENT + - joinTarget($index, $index-Target) + - $index-Target[TargetStatus] = ABSENT + - joinTargetNode($index-dep, $index-dep-Target, $index-dep-Node) + - $index-dep-Target[TargetStatus] = ABSENT +- name: skip index removal ops on relation drop + from: index-Node + query: + - $relation[Type] IN ['*scpb.Table', '*scpb.View'] + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnDescID($relation, $index, $relation-id) + - joinTarget($relation, $relation-Target) + - $relation-Target[TargetStatus] = ABSENT + - joinTargetNode($index, $index-Target, $index-Node) + - $index-Target[TargetStatus] = ABSENT +- name: skip index-column removal ops on index removal + from: index-column-Node + query: + - $index-column[Type] = '*scpb.IndexColumn' + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnIndexID($index-column, $index, $relation-id, $index-id) + - joinTargetNode($index-column, $index-column-Target, $index-column-Node) + - $index-column-Target[TargetStatus] IN [ABSENT, TRANSIENT_ABSENT] + - $index-column-Node[CurrentStatus] IN [PUBLIC, TRANSIENT_PUBLIC] + - joinTarget($index, $index-Target) + - $index-Target[TargetStatus] IN [ABSENT, TRANSIENT_ABSENT] +- name: skip table comment removal ops on descriptor drop + from: dep-Node + query: + - $desc[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence'] + - $dep[Type] IN ['*scpb.ColumnComment', '*scpb.IndexComment', '*scpb.ConstraintComment', '*scpb.TableComment'] + - joinOnDescID($desc, $dep, $desc-id) + - joinTarget($desc, $desc-Target) + - $desc-Target[TargetStatus] = ABSENT + - joinTargetNode($dep, $dep-Target, $dep-Node) + - $dep-Target[TargetStatus] = ABSENT +- name: skip table zone config removal ops on descriptor drop + from: dep-Node + query: + - $desc[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence'] + - $dep[Type] = '*scpb.TableZoneConfig' + - joinOnDescID($desc, $dep, $desc-id) + - joinTarget($desc, $desc-Target) + - $desc-Target[TargetStatus] = ABSENT + - joinTargetNode($dep, $dep-Target, $dep-Node) + - $dep-Target[TargetStatus] = ABSENT diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index ab0cfe921f57..ca1f62d30e02 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -19,7 +19,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/opgen" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules/current" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules/release_22_2" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scstage" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -128,6 +130,44 @@ func makePlan(ctx context.Context, p *Plan) (err error) { return nil } +// getRulesRegistryForRelease returns the rules registry based on the current +// active version of cockroach. In a mixed version state, it's possible for the state +// generated by a newer version of cockroach to be incompatible with old versions. +// For example dependent objects or combinations of them in a partially executed, +// plan may reach states where older versions of cockroach may not be able +// to plan further (and vice versa). To eliminate the possibility of these issues, we +// will plan with the set of rules belonging to the currently active version. One +// example of this is the dependency between index name and secondary indexes +// is more relaxed on 23.1 vs 22.2, which can lead to scenarios where the index +// name may become public before the index is public (which was disallowed on older +// versions). +func getRulesRegistryForRelease( + ctx context.Context, activeVersion clusterversion.ClusterVersion, +) *rules.Registry { + if activeVersion.IsActive(clusterversion.V23_1) { + return current.GetRegistry() + } else if activeVersion.IsActive(clusterversion.V22_2) { + return release_22_2.GetRegistry() + } else { + log.Warningf(ctx, "Falling back to the oldest supported version 22.2") + return release_22_2.GetRegistry() + } +} + +func applyOpRules( + ctx context.Context, activeVersion clusterversion.ClusterVersion, g *scgraph.Graph, +) (*scgraph.Graph, error) { + registry := getRulesRegistryForRelease(ctx, activeVersion) + return registry.ApplyOpRules(ctx, g) +} + +func applyDepRules( + ctx context.Context, activeVersion clusterversion.ClusterVersion, g *scgraph.Graph, +) error { + registry := getRulesRegistryForRelease(ctx, activeVersion) + return registry.ApplyDepRules(ctx, g) +} + func buildGraph( ctx context.Context, activeVersion clusterversion.ClusterVersion, cs scpb.CurrentState, ) *scgraph.Graph { @@ -135,7 +175,7 @@ func buildGraph( if err != nil { panic(errors.Wrapf(err, "build graph op edges")) } - err = current.ApplyDepRules(ctx, g) + err = applyDepRules(ctx, activeVersion, g) if err != nil { panic(errors.Wrapf(err, "build graph dep edges")) } @@ -143,7 +183,7 @@ func buildGraph( if err != nil { panic(errors.Wrapf(err, "validate graph")) } - g, err = current.ApplyOpRules(ctx, g) + g, err = applyOpRules(ctx, activeVersion, g) if err != nil { panic(errors.Wrapf(err, "mark op edges as no-op")) }