Skip to content

Commit

Permalink
Merge #133452
Browse files Browse the repository at this point in the history
133452: schemachanger: copy current deprules to release_24_3 r=spilchen a=annrpom

With the server version changing soon, we need to copy the current dependency rules for the declarative schema changer to a separate package so they remain static. New rules added to the declarative schema changer will continue to modify the current rules, which now apply to version 25.1.

Epic: None

Release note: None

Co-authored-by: Annie Pompa <[email protected]>
  • Loading branch information
craig[bot] and annrpom committed Oct 25, 2024
2 parents e86550a + 768aa94 commit 98884f4
Show file tree
Hide file tree
Showing 28 changed files with 12,032 additions and 451 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -577,6 +577,7 @@ ALL_TESTS = [
"//pkg/sql/schemachanger/scplan/internal/rules/current:current_test",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_1:release_24_1_test",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_2:release_24_2_test",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_3:release_24_3_test",
"//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph_test",
"//pkg/sql/schemachanger/scplan:scplan_test",
"//pkg/sql/schemachanger/screl:screl_test",
Expand Down Expand Up @@ -2161,6 +2162,8 @@ GO_TARGETS = [
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_1:release_24_1_test",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_2:release_24_2",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_2:release_24_2_test",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_3:release_24_3",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_3:release_24_3_test",
"//pkg/sql/schemachanger/scplan/internal/rules:rules",
"//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph",
"//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph_test",
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/testdata/declarative-rules/invalid_version
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ invalid_version
----
debug declarative-print-rules 1.1 op
unsupported version number, the supported versions are:
latest
latest
1000024.2
1000024.1
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scplan/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
"//pkg/sql/schemachanger/scplan/internal/rules/current",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_1",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_2",
"//pkg/sql/schemachanger/scplan/internal/rules/release_24_3",
"//pkg/sql/schemachanger/scplan/internal/scgraph",
"//pkg/sql/schemachanger/scplan/internal/scgraphviz",
"//pkg/sql/schemachanger/scplan/internal/scstage",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,12 @@ import (

const (
// rulesVersion version of elements that can be appended to rel rule names.
rulesVersion = "-24.3"
rulesVersion = "-25.1"
)

// rulesVersionKey version of elements used by this rule set.
// TODO(annie): Need to update the rulesVersionKey here to point to
// clusterversion.V25_1 when that is available.
var rulesVersionKey = clusterversion.V24_3

// descriptorIsNotBeingDropped creates a clause which leads to the outer clause
Expand Down
900 changes: 450 additions & 450 deletions pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "release_24_3",
srcs = [
"dep_add_column.go",
"dep_add_constraint.go",
"dep_add_index.go",
"dep_add_index_and_column.go",
"dep_add_index_and_constraint.go",
"dep_alter_column_type.go",
"dep_create.go",
"dep_create_function.go",
"dep_drop_column.go",
"dep_drop_constraint.go",
"dep_drop_index.go",
"dep_drop_index_and_column.go",
"dep_drop_object.go",
"dep_garbage_collection.go",
"dep_swap_index.go",
"dep_two_version.go",
"helpers.go",
"registry.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules/release_24_3",
visibility = ["//pkg/sql/schemachanger/scplan:__subpackages__"],
deps = [
"//pkg/clusterversion",
"//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",
"//pkg/sql/sem/catid",
"@com_github_cockroachdb_errors//:errors",
],
)

go_test(
name = "release_24_3_test",
srcs = [
"assertions_test.go",
"rules_test.go",
],
data = glob(["testdata/**"]),
embed = [":release_24_3"],
deps = [
"//pkg/sql/catalog/catpb",
"//pkg/sql/schemachanger/rel",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/schemachanger/scplan/internal/opgen",
"//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",
],
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,207 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the CockroachDB Software License
// included in the /LICENSE file.

package release_24_3

import (
"reflect"
"runtime"
"strings"
"testing"

"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/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,
checkConstraintPartitions,
} {
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) {
_ = scpb.ForEachElementType(func(e scpb.Element) error {
e = nonNilElement(e)
if err := fn(e); err != nil {
t.Errorf("%T: %+v", e, err)
}
return nil
})
})
}
}

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)
// and data elements have screl.ReferencedDescID attributes.
// One exception is foreign key constraint, which is not simple dependent nor data
// element but it has a screl.ReferencedDescID attribute.
func checkSimpleDependentsReferenceDescID(e scpb.Element) error {
if isSimpleDependent(e) || isData(e) {
return nil
}
if _, ok := e.(*scpb.ForeignKeyConstraint); ok {
return nil
}
if _, err := screl.Schema.GetAttribute(screl.ReferencedDescID, e); err == nil {
return errors.New("unexpected screl.ReferencedDescID attr")
}
return nil
}

// Assert that elements can be grouped into three categories when transitioning
// from PUBLIC to ABSENT:
// - go via DROPPED iff they're descriptor or data 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_DROPPED:
if isDescriptor(e) || isData(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 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 isColumnDependent but doesn't have ColumnID attr")
}
} else if err == nil {
return errors.New("has ColumnID attr but doesn't verify isColumnDependent")
}
return nil
}

// Assert that isIndexDependent covers all dependent elements of an index
// element.
func checkIsIndexDependent(e scpb.Element) error {
// Exclude indexes themselves and their data.
if isIndex(e) || isData(e) || isNonIndexBackedConstraint(e) {
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 isIndexDependent but doesn't have IndexID attr")
}
} else if err == nil {
return errors.New("has IndexID attr but doesn't verify 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 isConstraintDependent but doesn't have ConstraintID attr")
}
} else if err == nil {
return errors.New("has ConstraintID attr but doesn't verify isConstraintDependent")
}
return nil
}

// Assert the following partitions about constraints:
// 1. An element `e` with ConstraintID attr is either a constraint
// or a constraint dependent.
// 2. A constraint is either index-backed or non-index-backed.
//
// TODO (xiang): Add test for cross-descriptor partition. We currently
// cannot have them until we added referenced.*ID attr for
// UniqueWithoutIndex[NotValid] element, which is required to support
// partial unique without index constraint with a predicate that references
// other descriptors.
func checkConstraintPartitions(e scpb.Element) error {
_, err := screl.Schema.GetAttribute(screl.ConstraintID, e)
if err != nil {
return nil //nolint:returnerrcheck
}
if !isConstraint(e) && !isConstraintDependent(e) {
return errors.New("has ConstraintID attr but is not a constraint nor a constraint dependent")
}
if isConstraintDependent(e) {
return nil
}
if !isNonIndexBackedConstraint(e) && !isIndex(e) {
return errors.New("verifies isConstraint but does not verify isNonIndexBackedConstraint nor isIndex")
}
return nil
}
Loading

0 comments on commit 98884f4

Please sign in to comment.