diff --git a/pkg/ccl/schemachangerccl/backup_base_generated_test.go b/pkg/ccl/schemachangerccl/backup_base_generated_test.go index 84e34c28f049..a6db029fad89 100644 --- a/pkg/ccl/schemachangerccl/backup_base_generated_test.go +++ b/pkg/ccl/schemachangerccl/backup_base_generated_test.go @@ -158,6 +158,11 @@ func TestBackup_base_create_schema_drop_schema_separate_statements(t *testing.T) defer log.Scope(t).Close(t) sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/create_schema_drop_schema_separate_statements", sctest.SingleNodeCluster) } +func TestBackup_base_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeCluster) +} func TestBackup_base_drop_column_basic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/ccl/schemachangerccl/backup_base_mixed_generated_test.go b/pkg/ccl/schemachangerccl/backup_base_mixed_generated_test.go index 91e37d59314d..995f87ae5b56 100644 --- a/pkg/ccl/schemachangerccl/backup_base_mixed_generated_test.go +++ b/pkg/ccl/schemachangerccl/backup_base_mixed_generated_test.go @@ -158,6 +158,11 @@ func TestBackupMixedVersionElements_base_create_schema_drop_schema_separate_stat defer log.Scope(t).Close(t) sctest.BackupMixedVersionElements(t, "pkg/sql/schemachanger/testdata/end_to_end/create_schema_drop_schema_separate_statements", sctest.SingleNodeMixedCluster) } +func TestBackupMixedVersionElements_base_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.BackupMixedVersionElements(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeMixedCluster) +} func TestBackupMixedVersionElements_base_drop_column_basic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 30e7a02e7db4..a29bcd2eddbe 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1201,7 +1201,7 @@ func attachToExistingCluster( } if !opt.skipWipe { if clusterWipe { - if err := c.WipeE(ctx, l, c.All()); err != nil { + if err := c.WipeE(ctx, l, false /* preserveCerts */, c.All()); err != nil { return nil, err } } else { @@ -1499,10 +1499,13 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger, dest // Ignore the files in the the log directory; we pull the logs separately anyway // so this would only cause duplication. excludeFiles := "*.log,*.txt,*.pprof" - cmd := fmt.Sprintf( - "%s debug zip --include-range-info --exclude-files='%s' --url {pgurl:%d} %s", - defaultCockroachPath, excludeFiles, i, zipName, - ) + cmd := roachtestutil.NewCommand("%s debug zip", defaultCockroachPath). + Option("include-range-info"). + Flag("exclude-files", fmt.Sprintf("'%s'", excludeFiles)). + Flag("url", fmt.Sprintf("{pgurl:%d}", i)). + MaybeFlag(c.IsSecure(), "certs-dir", "certs"). + Arg(zipName). + String() if err := c.RunE(ctx, c.Node(i), cmd); err != nil { l.Printf("%s debug zip failed on node %d: %v", defaultCockroachPath, i, err) if i < c.spec.NodeCount { @@ -2096,10 +2099,6 @@ func (c *clusterImpl) StartE( if ctx.Err() != nil { return errors.Wrap(ctx.Err(), "cluster.StartE") } - // If the test failed (indicated by a canceled ctx), short-circuit. - if ctx.Err() != nil { - return ctx.Err() - } c.setStatusForClusterOpt("starting", startOpts.RoachtestOpts.Worker, opts...) defer c.clearStatusForClusterOpt(startOpts.RoachtestOpts.Worker) @@ -2134,7 +2133,9 @@ func (c *clusterImpl) StartE( return err } - if settings.Secure { + // Do not refetch certs if that step already happened once (i.e., we + // are restarting a node). + if settings.Secure && c.localCertsDir == "" { if err := c.RefetchCertsFromNode(ctx, 1); err != nil { return err } @@ -2248,7 +2249,9 @@ func (c *clusterImpl) Signal( // WipeE wipes a subset of the nodes in a cluster. See cluster.Start() for a // description of the nodes parameter. -func (c *clusterImpl) WipeE(ctx context.Context, l *logger.Logger, nodes ...option.Option) error { +func (c *clusterImpl) WipeE( + ctx context.Context, l *logger.Logger, preserveCerts bool, nodes ...option.Option, +) error { if ctx.Err() != nil { return errors.Wrap(ctx.Err(), "cluster.WipeE") } @@ -2258,16 +2261,16 @@ func (c *clusterImpl) WipeE(ctx context.Context, l *logger.Logger, nodes ...opti } c.setStatusForClusterOpt("wiping", false, nodes...) defer c.clearStatusForClusterOpt(false) - return roachprod.Wipe(ctx, l, c.MakeNodes(nodes...), false /* preserveCerts */) + return roachprod.Wipe(ctx, l, c.MakeNodes(nodes...), preserveCerts) } // Wipe is like WipeE, except instead of returning an error, it does // c.t.Fatal(). c.t needs to be set. -func (c *clusterImpl) Wipe(ctx context.Context, nodes ...option.Option) { +func (c *clusterImpl) Wipe(ctx context.Context, preserveCerts bool, nodes ...option.Option) { if ctx.Err() != nil { return } - if err := c.WipeE(ctx, c.l, nodes...); err != nil { + if err := c.WipeE(ctx, c.l, preserveCerts, nodes...); err != nil { c.t.Fatal(err) } } diff --git a/pkg/cmd/roachtest/cluster/cluster_interface.go b/pkg/cmd/roachtest/cluster/cluster_interface.go index 62b80862d1cb..fecb5421cbaf 100644 --- a/pkg/cmd/roachtest/cluster/cluster_interface.go +++ b/pkg/cmd/roachtest/cluster/cluster_interface.go @@ -115,8 +115,8 @@ type Cluster interface { // Deleting CockroachDB data and logs on nodes. - WipeE(ctx context.Context, l *logger.Logger, opts ...option.Option) error - Wipe(ctx context.Context, opts ...option.Option) + WipeE(ctx context.Context, l *logger.Logger, preserveCerts bool, opts ...option.Option) error + Wipe(ctx context.Context, preserveCerts bool, opts ...option.Option) // Internal niche tools. diff --git a/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go b/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go index 9ec4041271b5..f3c0ecff1aff 100644 --- a/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go +++ b/pkg/cmd/roachtest/roachtestutil/clusterupgrade/clusterupgrade.go @@ -137,17 +137,17 @@ func InstallFixtures( return nil } -// StartWithBinary starts a cockroach binary, assumed to already be -// present in the nodes in the path given. -func StartWithBinary( +// StartWithSettings starts cockroach and constructs settings according +// to the setting options passed. +func StartWithSettings( ctx context.Context, l *logger.Logger, c cluster.Cluster, nodes option.NodeListOption, - binaryPath string, startOpts option.StartOpts, + opts ...install.ClusterSettingOption, ) error { - settings := install.MakeClusterSettings(install.BinaryOption(binaryPath)) + settings := install.MakeClusterSettings(opts...) return c.StartE(ctx, l, startOpts, settings, nodes) } @@ -171,6 +171,7 @@ func RestartNodesWithNewBinary( nodes option.NodeListOption, startOpts option.StartOpts, newVersion string, + settings ...install.ClusterSettingOption, ) error { // NB: We could technically stage the binary on all nodes before // restarting each one, but on Unix it's invalid to write to an @@ -200,7 +201,9 @@ func RestartNodesWithNewBinary( if err != nil { return err } - if err := StartWithBinary(ctx, l, c, c.Node(node), binary, startOpts); err != nil { + if err := StartWithSettings( + ctx, l, c, c.Node(node), startOpts, append(settings, install.BinaryOption(binary))..., + ); err != nil { return err } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel index a91372831bea..7fd3c00f33a1 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/cmd/roachtest/roachtestutil/clusterupgrade", "//pkg/cmd/roachtest/test", "//pkg/roachpb", + "//pkg/roachprod/install", "//pkg/roachprod/logger", "//pkg/util/ctxgroup", "//pkg/util/randutil", diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index 880d9fb43ce7..8f0754f82d8f 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -80,6 +80,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/version" @@ -114,6 +115,13 @@ var ( possibleDelaysMs = []int{ 0, 50, 100, 200, 500, } + + // defaultClusterSettings is the set of cluster settings always + // passed to `clusterupgrade.StartWithSettings` when (re)starting + // nodes in a cluster. + defaultClusterSettings = []install.ClusterSettingOption{ + install.SecureOption(true), + } ) type ( @@ -483,7 +491,11 @@ func (s startFromCheckpointStep) Run( startOpts := option.DefaultStartOptsNoBackups() startOpts.RoachprodOpts.Sequential = false - return clusterupgrade.StartWithBinary(ctx, l, c, s.crdbNodes, binaryPath, startOpts) + clusterSettings := append( + append([]install.ClusterSettingOption{}, defaultClusterSettings...), + install.BinaryOption(binaryPath), + ) + return clusterupgrade.StartWithSettings(ctx, l, c, s.crdbNodes, startOpts, clusterSettings...) } // uploadCurrentVersionStep uploads the current cockroach binary to @@ -606,6 +618,7 @@ func (s restartWithNewBinaryStep) Run( // scheduled backup if necessary. option.DefaultStartOptsNoBackups(), s.version, + defaultClusterSettings..., ) } diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 21e041573bff..c7e87abcbf7f 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -587,7 +587,7 @@ func (r *testRunner) runWorker( if c != nil && testToRun.canReuseCluster { err = func() error { l.PrintfCtx(ctx, "Using existing cluster: %s (arch=%q). Wiping", c.name, c.arch) - if err := c.WipeE(ctx, l); err != nil { + if err := c.WipeE(ctx, l, false /* preserveCerts */); err != nil { return err } if err := c.RunE(ctx, c.All(), "rm -rf "+perfArtifactsDir); err != nil { diff --git a/pkg/cmd/roachtest/tests/allocation_bench.go b/pkg/cmd/roachtest/tests/allocation_bench.go index 77e866672b40..ab657bdfa3ee 100644 --- a/pkg/cmd/roachtest/tests/allocation_bench.go +++ b/pkg/cmd/roachtest/tests/allocation_bench.go @@ -323,7 +323,7 @@ func setupStatCollector( if err := c.StopGrafana(ctx, t.L(), t.ArtifactsDir()); err != nil { t.L().ErrorfCtx(ctx, "Error(s) shutting down prom/grafana %s", err) } - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) } promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), cfg) diff --git a/pkg/cmd/roachtest/tests/autoupgrade.go b/pkg/cmd/roachtest/tests/autoupgrade.go index d1f61325533c..2f9eeaf65424 100644 --- a/pkg/cmd/roachtest/tests/autoupgrade.go +++ b/pkg/cmd/roachtest/tests/autoupgrade.go @@ -248,7 +248,7 @@ func registerAutoUpgrade(r registry.Registry) { // Wipe n3 to exclude it from the dead node check the roachtest harness // will perform after the test. - c.Wipe(ctx, c.Node(nodeDecommissioned)) + c.Wipe(ctx, false /* preserveCerts */, c.Node(nodeDecommissioned)) } r.Add(registry.TestSpec{ diff --git a/pkg/cmd/roachtest/tests/backup_fixtures.go b/pkg/cmd/roachtest/tests/backup_fixtures.go index abd859e02ac8..735e4517f7ea 100644 --- a/pkg/cmd/roachtest/tests/backup_fixtures.go +++ b/pkg/cmd/roachtest/tests/backup_fixtures.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -135,10 +136,10 @@ func (bd *backupDriver) prepareCluster(ctx context.Context) { bd.sp.hardware.getCRDBNodes(), version) require.NoError(bd.t, err) - require.NoError(bd.t, clusterupgrade.StartWithBinary(ctx, bd.t.L(), bd.c, + require.NoError(bd.t, clusterupgrade.StartWithSettings(ctx, bd.t.L(), bd.c, bd.sp.hardware.getCRDBNodes(), - binaryPath, - option.DefaultStartOptsNoBackups())) + option.DefaultStartOptsNoBackups(), + install.BinaryOption(binaryPath))) bd.assertCorrectCockroachBinary(ctx) if !bd.sp.backup.ignoreExistingBackups { diff --git a/pkg/cmd/roachtest/tests/clock_jump_crash.go b/pkg/cmd/roachtest/tests/clock_jump_crash.go index 21d1a4747bbb..341edd3a1d09 100644 --- a/pkg/cmd/roachtest/tests/clock_jump_crash.go +++ b/pkg/cmd/roachtest/tests/clock_jump_crash.go @@ -39,7 +39,7 @@ func runClockJump(ctx context.Context, t test.Test, c cluster.Cluster, tc clockJ if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) } - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) db := c.Conn(ctx, t.L(), c.Spec().NodeCount) diff --git a/pkg/cmd/roachtest/tests/clock_monotonic.go b/pkg/cmd/roachtest/tests/clock_monotonic.go index 379ec398c548..469dda8032d8 100644 --- a/pkg/cmd/roachtest/tests/clock_monotonic.go +++ b/pkg/cmd/roachtest/tests/clock_monotonic.go @@ -41,7 +41,7 @@ func runClockMonotonicity( if err := c.RunE(ctx, c.Node(1), "test -x ./cockroach"); err != nil { c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) } - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings()) db := c.Conn(ctx, t.L(), c.Spec().NodeCount) diff --git a/pkg/cmd/roachtest/tests/cluster_init.go b/pkg/cmd/roachtest/tests/cluster_init.go index 5f5165eff18b..45e5b0b6ebd1 100644 --- a/pkg/cmd/roachtest/tests/cluster_init.go +++ b/pkg/cmd/roachtest/tests/cluster_init.go @@ -52,7 +52,7 @@ func runClusterInit(ctx context.Context, t test.Test, c cluster.Cluster) { // default to just the first node) and we want to make sure that we're not // relying on it. for _, initNode := range []int{2, 1} { - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) t.L().Printf("starting test with init node %d", initNode) startOpts := option.DefaultStartOpts() diff --git a/pkg/cmd/roachtest/tests/decommission.go b/pkg/cmd/roachtest/tests/decommission.go index 58fc0eeee38b..5129f0fbcfa3 100644 --- a/pkg/cmd/roachtest/tests/decommission.go +++ b/pkg/cmd/roachtest/tests/decommission.go @@ -746,7 +746,7 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust } // Now stop+wipe them for good to keep the logs simple and the dead node detector happy. c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Nodes(targetNodeA, targetNodeB)) - c.Wipe(ctx, c.Nodes(targetNodeA, targetNodeB)) + c.Wipe(ctx, false /* preserveCerts */, c.Nodes(targetNodeA, targetNodeB)) } } @@ -871,7 +871,7 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust t.L().Printf("wiping n%d and adding it back to the cluster as a new node\n", targetNode) c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Node(targetNode)) - c.Wipe(ctx, c.Node(targetNode)) + c.Wipe(ctx, false /*preserveCerts */, c.Node(targetNode)) joinNode := h.getRandNode() internalAddrs, err := c.InternalAddr(ctx, t.L(), c.Node(joinNode)) diff --git a/pkg/cmd/roachtest/tests/decommission_self.go b/pkg/cmd/roachtest/tests/decommission_self.go index db39d031a3b6..fe753af542d9 100644 --- a/pkg/cmd/roachtest/tests/decommission_self.go +++ b/pkg/cmd/roachtest/tests/decommission_self.go @@ -31,7 +31,7 @@ func runDecommissionSelf(ctx context.Context, t test.Test, c cluster.Cluster) { // Stop n2 and exclude it from post-test consistency checks, // as this node can't contact cluster any more and operations // on it will hang. - u.c.Wipe(ctx, c.Node(2)) + u.c.Wipe(ctx, false /* preserveCerts */, c.Node(2)) }, checkOneMembership(1, "decommissioned"), ) diff --git a/pkg/cmd/roachtest/tests/inconsistency.go b/pkg/cmd/roachtest/tests/inconsistency.go index 9ac12a566e0d..2266889f1375 100644 --- a/pkg/cmd/roachtest/tests/inconsistency.go +++ b/pkg/cmd/roachtest/tests/inconsistency.go @@ -160,5 +160,5 @@ func runInconsistency(ctx context.Context, t test.Test, c cluster.Cluster) { // roachtest checks that no nodes are down when the test finishes, but in this // case we have a down node that we can't restart. Remove the data dir, which // tells roachtest to ignore this node. - c.Wipe(ctx, c.Node(1)) + c.Wipe(ctx, false /* preserveCerts */, c.Node(1)) } diff --git a/pkg/cmd/roachtest/tests/kv.go b/pkg/cmd/roachtest/tests/kv.go index 864ef2a600f2..38d79d17382f 100644 --- a/pkg/cmd/roachtest/tests/kv.go +++ b/pkg/cmd/roachtest/tests/kv.go @@ -767,7 +767,7 @@ func registerKVScalability(r registry.Registry) { const maxPerNodeConcurrency = 64 for i := nodes; i <= nodes*maxPerNodeConcurrency; i += nodes { i := i // capture loop variable - c.Wipe(ctx, c.Range(1, nodes)) + c.Wipe(ctx, false /* preserveCerts */, c.Range(1, nodes)) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), c.Range(1, nodes)) t.Status("running workload") diff --git a/pkg/cmd/roachtest/tests/kvbench.go b/pkg/cmd/roachtest/tests/kvbench.go index 108d85d45901..c90c52a51bcc 100644 --- a/pkg/cmd/roachtest/tests/kvbench.go +++ b/pkg/cmd/roachtest/tests/kvbench.go @@ -229,7 +229,7 @@ func runKVBench(ctx context.Context, t test.Test, c cluster.Cluster, b kvBenchSp // Wipe cluster before starting a new run because factors like load-based // splitting can significantly change the underlying layout of the table and // affect benchmark results. - c.Wipe(ctx, roachNodes) + c.Wipe(ctx, false /* preserveCerts */, roachNodes) c.Start(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), roachNodes) time.Sleep(restartWait) diff --git a/pkg/cmd/roachtest/tests/mixed_version_backup.go b/pkg/cmd/roachtest/tests/mixed_version_backup.go index 7ca5f4a9ca84..2c6a54514418 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_backup.go +++ b/pkg/cmd/roachtest/tests/mixed_version_backup.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" @@ -811,7 +812,7 @@ func (sc *systemTableContents) loadShowResults( query := fmt.Sprintf("SELECT * FROM [%s]%s", showStmt, aostFor(timestamp)) showCmd := roachtestutil.NewCommand("%s sql", mixedversion.CurrentCockroachPath). - Option("insecure"). + Flag("certs-dir", "certs"). Flag("e", fmt.Sprintf("%q", query)). String() @@ -1963,14 +1964,15 @@ func (mvb *mixedVersionBackup) verifyBackupCollection( func (mvb *mixedVersionBackup) resetCluster( ctx context.Context, l *logger.Logger, version string, ) error { - l.Printf("resetting cluster using version %s", version) - if err := mvb.cluster.WipeE(ctx, l, mvb.roachNodes); err != nil { + l.Printf("resetting cluster using version %q", clusterupgrade.VersionMsg(version)) + if err := mvb.cluster.WipeE(ctx, l, true /* preserveCerts */, mvb.roachNodes); err != nil { return fmt.Errorf("failed to wipe cluster: %w", err) } cockroachPath := clusterupgrade.BinaryPathFromVersion(version) - return clusterupgrade.StartWithBinary( - ctx, l, mvb.cluster, mvb.roachNodes, cockroachPath, option.DefaultStartOptsNoBackups(), + return clusterupgrade.StartWithSettings( + ctx, l, mvb.cluster, mvb.roachNodes, option.DefaultStartOptsNoBackups(), + install.BinaryOption(cockroachPath), install.SecureOption(true), ) } diff --git a/pkg/cmd/roachtest/tests/multitenant_tpch.go b/pkg/cmd/roachtest/tests/multitenant_tpch.go index ad0e44c78d67..e444e458752b 100644 --- a/pkg/cmd/roachtest/tests/multitenant_tpch.go +++ b/pkg/cmd/roachtest/tests/multitenant_tpch.go @@ -82,7 +82,7 @@ func runMultiTenantTPCH( // Restart and wipe the cluster to remove advantage of the second TPCH run. c.Stop(ctx, t.L(), option.DefaultStopOpts()) - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) start() singleTenantConn = c.Conn(ctx, t.L(), 1) // Disable merge queue in the system tenant. diff --git a/pkg/cmd/roachtest/tests/query_comparison_util.go b/pkg/cmd/roachtest/tests/query_comparison_util.go index 44be42337840..85cdad3ff35d 100644 --- a/pkg/cmd/roachtest/tests/query_comparison_util.go +++ b/pkg/cmd/roachtest/tests/query_comparison_util.go @@ -82,7 +82,7 @@ func runQueryComparison( return } c.Stop(clusterCtx, t.L(), option.DefaultStopOpts()) - c.Wipe(clusterCtx) + c.Wipe(clusterCtx, false /* preserveCerts */) } } diff --git a/pkg/cmd/roachtest/tests/rapid_restart.go b/pkg/cmd/roachtest/tests/rapid_restart.go index 1533fea72459..52d513c2bade 100644 --- a/pkg/cmd/roachtest/tests/rapid_restart.go +++ b/pkg/cmd/roachtest/tests/rapid_restart.go @@ -39,7 +39,7 @@ func runRapidRestart(ctx context.Context, t test.Test, c cluster.Cluster) { return timeutil.Now().After(deadline) } for j := 1; !done(); j++ { - c.Wipe(ctx, node) + c.Wipe(ctx, false /* preserveCerts */, node) // The first 2 iterations we start the cockroach node and kill it right // away. The 3rd iteration we let cockroach run so that we can check after @@ -97,5 +97,5 @@ func runRapidRestart(ctx context.Context, t test.Test, c cluster.Cluster) { // that consistency checks can be run, but in this case there's not much // there in the first place anyway. c.Stop(ctx, t.L(), option.DefaultStopOpts(), node) - c.Wipe(ctx, node) + c.Wipe(ctx, false /* preserveCerts */, node) } diff --git a/pkg/cmd/roachtest/tests/sstable_corruption.go b/pkg/cmd/roachtest/tests/sstable_corruption.go index 79d47b719982..42924c5cd1a3 100644 --- a/pkg/cmd/roachtest/tests/sstable_corruption.go +++ b/pkg/cmd/roachtest/tests/sstable_corruption.go @@ -127,7 +127,7 @@ func runSSTableCorruption(ctx context.Context, t test.Test, c cluster.Cluster) { if err := c.StartE(ctx, t.L(), option.DefaultStartOptsNoBackups(), install.MakeClusterSettings(), crdbNodes); err != nil { // Node detected corruption on start and crashed. This is good. No need // to run workload; the test is complete. - _ = c.WipeE(ctx, t.L(), corruptNodes) + _ = c.WipeE(ctx, t.L(), false /* preserveCerts */, corruptNodes) return } @@ -174,7 +174,7 @@ func runSSTableCorruption(ctx context.Context, t test.Test, c cluster.Cluster) { } // Exempt corrupted nodes from roachtest harness' post-test liveness checks. - _ = c.WipeE(ctx, t.L(), corruptNodes) + _ = c.WipeE(ctx, t.L(), false /* preserveCerts */, corruptNodes) } func registerSSTableCorruption(r registry.Registry) { diff --git a/pkg/cmd/roachtest/tests/tlp.go b/pkg/cmd/roachtest/tests/tlp.go index 713829f91360..2ee359cd1d23 100644 --- a/pkg/cmd/roachtest/tests/tlp.go +++ b/pkg/cmd/roachtest/tests/tlp.go @@ -82,7 +82,7 @@ func runTLP(ctx context.Context, t test.Test, c cluster.Cluster) { return } c.Stop(ctx, t.L(), option.DefaultStopOpts()) - c.Wipe(ctx) + c.Wipe(ctx, false /* preserveCerts */) } } diff --git a/pkg/cmd/roachtest/tests/tpc_utils.go b/pkg/cmd/roachtest/tests/tpc_utils.go index 640bd2885a4e..9d12197ba5dc 100644 --- a/pkg/cmd/roachtest/tests/tpc_utils.go +++ b/pkg/cmd/roachtest/tests/tpc_utils.go @@ -88,7 +88,7 @@ func loadTPCHDataset( // If the scale factor was smaller than the required scale factor, wipe the // cluster and restore. m.ExpectDeaths(int32(c.Spec().NodeCount)) - c.Wipe(ctx, roachNodes) + c.Wipe(ctx, false /* preserveCerts */, roachNodes) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), roachNodes) m.ResetDeaths() } else if pqErr := (*pq.Error)(nil); !(errors.As(err, &pqErr) && diff --git a/pkg/cmd/roachtest/tests/tpcc.go b/pkg/cmd/roachtest/tests/tpcc.go index ac795b049fb2..510aca8d7589 100644 --- a/pkg/cmd/roachtest/tests/tpcc.go +++ b/pkg/cmd/roachtest/tests/tpcc.go @@ -1147,7 +1147,7 @@ func loadTPCCBench( // If the dataset exists but is not large enough, wipe the cluster // before restoring. - c.Wipe(ctx, roachNodes) + c.Wipe(ctx, false /* preserveCerts */, roachNodes) startOpts, settings := b.startOpts() c.Start(ctx, t.L(), startOpts, settings, roachNodes) } else if pqErr := (*pq.Error)(nil); !(errors.As(err, &pqErr) && diff --git a/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go b/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go index 90ed4d0941db..e8b613657c39 100644 --- a/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go +++ b/pkg/cmd/roachtest/tests/validate_system_schema_after_version_upgrade.go @@ -75,7 +75,7 @@ func registerValidateSystemSchemaAfterVersionUpgrade(r registry.Registry) { // Wipe nodes in this test's cluster. wipeClusterStep := func(nodes option.NodeListOption) versionStep { return func(ctx context.Context, t test.Test, u *versionUpgradeTest) { - u.c.Wipe(ctx, nodes) + u.c.Wipe(ctx, false /* preserveCerts */, nodes) } } diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index 2e2f458442d5..3d435e94b21f 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/mixedversion" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -100,17 +101,22 @@ DROP TABLE splitmerge.t; func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.All()) mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All()) - mvt.OnStartup("setup schema changer workload", func(ctx context.Context, l *logger.Logger, r *rand.Rand, helper *mixedversion.Helper) error { - // Execute the workload init. - return c.RunE(ctx, c.All(), "./workload init schemachange") - }) - mvt.InMixedVersion("run backup", func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { - // Verify that backups can be created in various configurations. This is - // important to test because changes in system tables might cause backups to - // fail in mixed-version clusters. - dest := fmt.Sprintf("nodelocal://1/%d", timeutil.Now().UnixNano()) - return h.Exec(rng, `BACKUP TO $1`, dest) - }) + mvt.OnStartup( + "setup schema changer workload", + func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { + node := h.RandomNode(rng, c.All()) + l.Printf("executing workload init on node %d", node) + return c.RunE(ctx, c.Node(node), fmt.Sprintf("./workload init schemachange {pgurl%s}", c.All())) + }) + mvt.InMixedVersion( + "run backup", + func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { + // Verify that backups can be created in various configurations. This is + // important to test because changes in system tables might cause backups to + // fail in mixed-version clusters. + dest := fmt.Sprintf("nodelocal://1/%d", timeutil.Now().UnixNano()) + return h.Exec(rng, `BACKUP TO $1`, dest) + }) mvt.InMixedVersion( "test features", func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { @@ -247,7 +253,9 @@ func uploadAndStartFromCheckpointFixture(nodes option.NodeListOption, v string) startOpts := option.DefaultStartOpts() // NB: can't start sequentially since cluster already bootstrapped. startOpts.RoachprodOpts.Sequential = false - if err := clusterupgrade.StartWithBinary(ctx, t.L(), u.c, nodes, binary, startOpts); err != nil { + if err := clusterupgrade.StartWithSettings( + ctx, t.L(), u.c, nodes, startOpts, install.BinaryOption(binary), + ); err != nil { t.Fatal(err) } } @@ -259,7 +267,9 @@ func uploadAndStart(nodes option.NodeListOption, v string) versionStep { startOpts := option.DefaultStartOpts() // NB: can't start sequentially since cluster already bootstrapped. startOpts.RoachprodOpts.Sequential = false - if err := clusterupgrade.StartWithBinary(ctx, t.L(), u.c, nodes, binary, startOpts); err != nil { + if err := clusterupgrade.StartWithSettings( + ctx, t.L(), u.c, nodes, startOpts, install.BinaryOption(binary), + ); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index aea8ad626276..648e75391396 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -103,10 +103,6 @@ const ( // rangeIDAllocCount is the number of Range IDs to allocate per allocation. rangeIDAllocCount = 10 - // defaultRaftEntryCacheSize is the default size in bytes for a - // store's Raft log entry cache. - defaultRaftEntryCacheSize = 1 << 24 // 16M - // replicaQueueExtraSize is the number of requests that a replica's incoming // message queue can keep over RaftConfig.RaftMaxInflightMsgs. When the leader // maxes out RaftMaxInflightMsgs, we want the receiving replica to still have @@ -138,6 +134,14 @@ var defaultRaftSchedulerConcurrency = envutil.EnvOrDefaultInt( // counts, while also avoiding starvation by excessive sharding. var defaultRaftSchedulerShardSize = envutil.EnvOrDefaultInt("COCKROACH_SCHEDULER_SHARD_SIZE", 16) +// defaultRaftEntryCacheSize is the default size in bytes for a store's Raft +// entry cache. The Raft entry cache is shared by all Raft groups managed by the +// store. It is used to cache uncommitted raft log entries such that once those +// entries are committed, their application can avoid disk reads to retrieve +// them from the persistent log. +var defaultRaftEntryCacheSize = envutil.EnvOrDefaultBytes( + "COCKROACH_RAFT_ENTRY_CACHE_SIZE", 16<<20 /* 16 MiB */) + // defaultRaftSchedulerPriorityShardSize specifies the default size of the Raft // scheduler priority shard, used for certain system ranges. This shard is // always fully populated with workers that don't count towards the concurrency @@ -1240,7 +1244,7 @@ func (sc *StoreConfig) SetDefaults(numStores int) { sc.RaftSchedulerShardSize = defaultRaftSchedulerShardSize } if sc.RaftEntryCacheSize == 0 { - sc.RaftEntryCacheSize = defaultRaftEntryCacheSize + sc.RaftEntryCacheSize = uint64(defaultRaftEntryCacheSize) } if raftDisableLeaderFollowsLeaseholder { sc.TestingKnobs.DisableLeaderFollowsLeaseholder = true diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index 84db907c15bf..1a6e4fa5260c 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -514,14 +514,16 @@ func (c *SyncedCluster) Wipe(ctx context.Context, l *logger.Logger, preserveCert cmd += fmt.Sprintf(`rm -fr %s/%s ;`, c.localVMDir(c.Nodes[i]), dir) } } else { - cmd = `sudo find /mnt/data* -maxdepth 1 -type f -exec rm -f {} \; && -sudo rm -fr /mnt/data*/{auxiliary,local,tmp,cassandra,cockroach,cockroach-temp*,mongo-data} && -sudo rm -fr logs && -` + rmCmds := []string{ + `sudo find /mnt/data* -maxdepth 1 -type f -exec rm -f {} \;`, + `sudo rm -fr /mnt/data*/{auxiliary,local,tmp,cassandra,cockroach,cockroach-temp*,mongo-data}`, + `sudo rm -fr logs`, + } if !preserveCerts { - cmd += "sudo rm -fr certs* ;\n" - cmd += "sudo rm -fr tenant-certs* ;\n" + rmCmds = append(rmCmds, "sudo rm -fr certs*", "sudo rm -fr tenant-certs*") } + + cmd = strings.Join(rmCmds, " && ") } sess := c.newSession(l, node, cmd, withDebugName("node-wipe")) defer sess.Close() @@ -1531,19 +1533,11 @@ func (c *SyncedCluster) fileExistsOnFirstNode( ctx context.Context, l *logger.Logger, path string, ) (bool, error) { l.Printf("%s: checking %s", c.Name, path) - testCmd := `$(test -e ` + path + `);` - // Do not log output to stdout/stderr because in some cases this call will be expected to exit 1. - result, err := c.runCmdOnSingleNode(ctx, l, c.Nodes[0], testCmd, true, nil, nil) - if (result.RemoteExitStatus != 0 && result.RemoteExitStatus != 1) || err != nil { - // Unexpected exit status (neither 0 nor 1) or non-nil error. Return combined output along with err returned - // from the call if it's not nil. - if err != nil { - return false, errors.Wrapf(err, "running '%s' failed with exit code=%d: got %s", testCmd, result.RemoteExitStatus, string(result.CombinedOut)) - } else { - return false, errors.Newf("running '%s' failed with exit code=%d: got %s", testCmd, result.RemoteExitStatus, string(result.CombinedOut)) - } - } - return result.RemoteExitStatus == 0, nil + // We use `echo -n` below stop echo from including a newline + // character in the output, allowing us to compare it directly with + // "0". + result, err := c.runCmdOnSingleNode(ctx, l, 1, `$(test -e `+path+`); echo -n $?`, false, l.Stdout, l.Stderr) + return result.Stdout == "0", err } // createNodeCertArguments returns a list of strings appropriate for use as diff --git a/pkg/roachprod/install/cockroach.go b/pkg/roachprod/install/cockroach.go index a7cef6277787..e237eb44b3cd 100644 --- a/pkg/roachprod/install/cockroach.go +++ b/pkg/roachprod/install/cockroach.go @@ -777,8 +777,11 @@ func (c *SyncedCluster) useStartSingleNode() bool { // distributeCerts distributes certs if it's a secure cluster and we're // starting n1. func (c *SyncedCluster) distributeCerts(ctx context.Context, l *logger.Logger) error { + if !c.Secure { + return nil + } for _, node := range c.TargetNodes() { - if node == 1 && c.Secure { + if node == 1 { return c.DistributeCerts(ctx, l) } } diff --git a/pkg/sql/catalog/dbdesc/database_desc_builder.go b/pkg/sql/catalog/dbdesc/database_desc_builder.go index b7d4390ddf88..e2af8d6db089 100644 --- a/pkg/sql/catalog/dbdesc/database_desc_builder.go +++ b/pkg/sql/catalog/dbdesc/database_desc_builder.go @@ -162,7 +162,7 @@ func (ddb *databaseDescriptorBuilder) RunRestoreChanges( version clusterversion.ClusterVersion, descLookupFn func(id descpb.ID) catalog.Descriptor, ) error { // Upgrade the declarative schema changer state. - if scpb.MigrateDescriptorState(version, ddb.maybeModified.DeclarativeSchemaChangerState) { + if scpb.MigrateDescriptorState(version, descpb.InvalidID, ddb.maybeModified.DeclarativeSchemaChangerState) { ddb.changes.Add(catalog.UpgradedDeclarativeSchemaChangerState) } return nil diff --git a/pkg/sql/catalog/funcdesc/func_desc_builder.go b/pkg/sql/catalog/funcdesc/func_desc_builder.go index 7c0a8ebf6aba..3a6309b02b73 100644 --- a/pkg/sql/catalog/funcdesc/func_desc_builder.go +++ b/pkg/sql/catalog/funcdesc/func_desc_builder.go @@ -109,7 +109,7 @@ func (fdb *functionDescriptorBuilder) RunRestoreChanges( version clusterversion.ClusterVersion, descLookupFn func(id descpb.ID) catalog.Descriptor, ) error { // Upgrade the declarative schema changer state. - if scpb.MigrateDescriptorState(version, fdb.maybeModified.DeclarativeSchemaChangerState) { + if scpb.MigrateDescriptorState(version, fdb.maybeModified.ParentID, fdb.maybeModified.DeclarativeSchemaChangerState) { fdb.changes.Add(catalog.UpgradedDeclarativeSchemaChangerState) } return nil diff --git a/pkg/sql/catalog/rewrite/rewrite.go b/pkg/sql/catalog/rewrite/rewrite.go index e3e26a5998c7..96c0b3ad92a0 100644 --- a/pkg/sql/catalog/rewrite/rewrite.go +++ b/pkg/sql/catalog/rewrite/rewrite.go @@ -651,6 +651,18 @@ func rewriteSchemaChangerState( var droppedConstraints catalog.ConstraintIDSet for i := 0; i < len(state.Targets); i++ { t := &state.Targets[i] + // Since the parent database ID is never written in the descriptorRewrites + // map we need to special case certain elements that need their ParentID + // re-written. + if data := t.GetTableData(); data != nil { + rewrite, ok := descriptorRewrites[data.TableID] + if !ok { + return errors.Errorf("missing rewrite for id %d in %s", data.TableID, screl.ElementString(t.Element())) + } + data.TableID = rewrite.ID + data.DatabaseID = rewrite.ParentID + continue + } if err := screl.WalkDescIDs(t.Element(), func(id *descpb.ID) error { if *id == descpb.InvalidID { // Some descriptor ID fields in elements may be deliberately unset. diff --git a/pkg/sql/catalog/schemadesc/schema_desc_builder.go b/pkg/sql/catalog/schemadesc/schema_desc_builder.go index ddcc4ff8086a..1f287492b680 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc_builder.go +++ b/pkg/sql/catalog/schemadesc/schema_desc_builder.go @@ -122,7 +122,7 @@ func (sdb *schemaDescriptorBuilder) RunRestoreChanges( version clusterversion.ClusterVersion, descLookupFn func(id descpb.ID) catalog.Descriptor, ) error { // Upgrade the declarative schema changer state. - if scpb.MigrateDescriptorState(version, sdb.maybeModified.DeclarativeSchemaChangerState) { + if scpb.MigrateDescriptorState(version, sdb.maybeModified.ParentID, sdb.maybeModified.DeclarativeSchemaChangerState) { sdb.changes.Add(catalog.UpgradedDeclarativeSchemaChangerState) } return nil diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index aa7e458802d8..313340bf5e99 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -177,7 +177,7 @@ func (tdb *tableDescriptorBuilder) RunRestoreChanges( } // Upgrade the declarative schema changer state - if scpb.MigrateDescriptorState(version, tdb.maybeModified.DeclarativeSchemaChangerState) { + if scpb.MigrateDescriptorState(version, tdb.maybeModified.ParentID, tdb.maybeModified.DeclarativeSchemaChangerState) { tdb.changes.Add(catalog.UpgradedDeclarativeSchemaChangerState) } diff --git a/pkg/sql/catalog/typedesc/type_desc_builder.go b/pkg/sql/catalog/typedesc/type_desc_builder.go index 940e8fccaff6..5ae688b8ef5e 100644 --- a/pkg/sql/catalog/typedesc/type_desc_builder.go +++ b/pkg/sql/catalog/typedesc/type_desc_builder.go @@ -123,7 +123,7 @@ func (tdb *typeDescriptorBuilder) RunRestoreChanges( version clusterversion.ClusterVersion, descLookupFn func(id descpb.ID) catalog.Descriptor, ) error { // Upgrade the declarative schema changer state - if scpb.MigrateDescriptorState(version, tdb.maybeModified.DeclarativeSchemaChangerState) { + if scpb.MigrateDescriptorState(version, tdb.maybeModified.ParentID, tdb.maybeModified.DeclarativeSchemaChangerState) { tdb.changes.Add(catalog.UpgradedDeclarativeSchemaChangerState) } return nil diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index 50c231761216..5671074f9a2e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -194,7 +194,7 @@ query TT $trace_query ---- sql query rows affected: 0 -commit sql txn CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 not_visible:false invisibility:0 > state:WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:VALIDATED current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 relevant_statements: > authorization: name_mapping: columns: columns: columns: families: indexes: > > drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false > +commit sql txn CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:2 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 not_visible:false invisibility:0 > state:WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata:<> target_status:PUBLIC > targets: > metadata:<> target_status:PUBLIC > current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:VALIDATED current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 relevant_statements: > authorization: name_mapping: columns: columns: columns: families: indexes: > > drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false > statement ok SET tracing = on,kv,results; DROP TABLE t.kv diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index a1c1e997d3cf..d4fa2e03bce6 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -3178,3 +3178,17 @@ func (p *planner) CanPerformDropOwnedBy( } return tree.MustBeDInt(row[0]) == 0, err } + +// CanCreateCrossDBSequenceOwnerRef returns if cross database sequence +// owner references are allowed. +func (p *planner) CanCreateCrossDBSequenceOwnerRef() error { + if !allowCrossDatabaseSeqOwner.Get(&p.execCfg.Settings.SV) { + return errors.WithHintf( + pgerror.Newf(pgcode.FeatureNotSupported, + "OWNED BY cannot refer to other databases; (see the '%s' cluster setting)", + allowCrossDatabaseSeqOwnerSetting), + crossDBReferenceDeprecationHint(), + ) + } + return nil +} diff --git a/pkg/sql/schemachanger/scbuild/build.go b/pkg/sql/schemachanger/scbuild/build.go index c96d3f20064a..925166eb06da 100644 --- a/pkg/sql/schemachanger/scbuild/build.go +++ b/pkg/sql/schemachanger/scbuild/build.go @@ -104,10 +104,18 @@ func Build( current := make([]scpb.Status, 0, len(bs.output)) version := dependencies.ClusterSettings().Version.ActiveVersion(ctx) withLogEvent := make([]scpb.Target, 0, len(bs.output)) + var extraTargets []struct { + e elementState + t scpb.Target + } for _, e := range bs.output { - if e.metadata.Size() == 0 { + if !e.metadata.TargetIsLinkedToSchemaChange() && !shouldElementBeRetainedWithoutMetadata(e.element, e.current) { // Exclude targets which weren't explicitly set. // Explicitly-set targets have non-zero values in the target metadata. + // Exceptions are TableData/IndexData elements which allow our planning + // execution to skip certain transitions and fences like the two version + // invariant. We will only keep these for descriptors going through + // a transition. continue } if !version.IsActive(screl.MinElementVersion(e.element)) { @@ -121,15 +129,37 @@ func Build( // max version. continue } + t := scpb.MakeTarget(e.target, e.element, &e.metadata) - ts.Targets = append(ts.Targets, t) - initial = append(initial, e.initial) - current = append(current, e.current) - if e.withLogEvent { - withLogEvent = append(withLogEvent, t) + if t.TargetIsLinkedToSchemaChange() { + ts.Targets = append(ts.Targets, t) + initial = append(initial, e.initial) + current = append(current, e.current) + if e.withLogEvent { + withLogEvent = append(withLogEvent, t) + } + } else if b.ClusterSettings().Version.IsActive(ctx, clusterversion.V23_2) { + extraTargets = append(extraTargets, struct { + e elementState + t scpb.Target + }{e: e, t: t}) } - } + } + seenDescriptors := screl.AllTargetDescIDs(ts) + // We are going to retain certain elements for metadata purposes, like + // TableData/IndexData elements, which will allow the declarative schema + // changer to know if a given table is empty. Once these elements exist + // the two-version invariant is applied when making mutations to elements. + // Only emit data elements for descriptors if they are references with + // some transition. + for _, ex := range extraTargets { + if seenDescriptors.Contains(screl.GetDescID(ex.t.Element())) { + ts.Targets = append(ts.Targets, ex.t) + initial = append(initial, ex.e.initial) + current = append(current, ex.e.current) + } + } // Ensure none of the involving descriptors have an ongoing schema change, // unless it's newly created. ensureNoConcurrentSchemaChange(&ts, bs) @@ -442,3 +472,16 @@ func (b buildCtx) WithNewSourceElementID() scbuildstmt.BuildCtx { EventLogState: b.EventLogStateWithNewSourceElementID(), } } + +// shouldElementBeRetainedWithoutMetadata tracks which elements should +// be retained even if no metadata exists. These elements may contain +// other hints that are used for planning such as TableData/IndexData elements +// that allow us to skip the two version invariant or backfills/validation +// at runtime. +func shouldElementBeRetainedWithoutMetadata(element scpb.Element, status scpb.Status) bool { + switch element.(type) { + case *scpb.TableData, *scpb.IndexData: + return status == scpb.Status_PUBLIC + } + return false +} diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index bd2f02c621ab..451d438ad3ed 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -101,7 +101,8 @@ func (b *builderState) Ensure(e scpb.Element, target scpb.TargetStatus, meta scp // of it and investigate it further if needed. if dst.current == scpb.Status_ABSENT && dst.target == scpb.ToAbsent && - (target == scpb.ToPublic || target == scpb.Transient) { + (target == scpb.ToPublic || target == scpb.Transient) && + dst.metadata.TargetIsLinkedToSchemaChange() { panic(scerrors.NotImplementedErrorf(nil, "attempt to revive a ghost element:"+ " [elem=%v],[current=ABSENT],[target=ToAbsent],[newTarget=%v]", dst.element.String(), target.Status())) } diff --git a/pkg/sql/schemachanger/scbuild/event_log.go b/pkg/sql/schemachanger/scbuild/event_log.go index 7973c17c1236..67cd327690a5 100644 --- a/pkg/sql/schemachanger/scbuild/event_log.go +++ b/pkg/sql/schemachanger/scbuild/event_log.go @@ -322,7 +322,9 @@ func (pb payloadBuilder) build(b buildCtx) logpb.EventPayload { } case *scpb.Sequence: if pb.TargetStatus == scpb.Status_PUBLIC { - return nil + return &eventpb.CreateSequence{ + SequenceName: fullyQualifiedName(b, e), + } } else { return &eventpb.DropSequence{ SequenceName: fullyQualifiedName(b, e), diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel index 0500e35d149c..f5d298e6d8d5 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "create_function.go", "create_index.go", "create_schema.go", + "create_sequence.go", "dependencies.go", "drop_database.go", "drop_function.go", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_sequence.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_sequence.go new file mode 100644 index 000000000000..abd11492b310 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_sequence.go @@ -0,0 +1,191 @@ +// 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 scbuildstmt + +import ( + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdecomp" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +func CreateSequence(b BuildCtx, n *tree.CreateSequence) { + dbElts, scElts := b.ResolvePrefix(n.Name.ObjectNamePrefix, privilege.CREATE) + _, _, schemaElem := scpb.FindSchema(scElts) + _, _, dbElem := scpb.FindDatabase(dbElts) + _, _, scName := scpb.FindNamespace(scElts) + _, _, dbname := scpb.FindNamespace(dbElts) + n.Name.SchemaName = tree.Name(scName.Name) + n.Name.CatalogName = tree.Name(dbname.Name) + n.Name.ExplicitCatalog = true + n.Name.ExplicitSchema = true + owner := b.CurrentUser() + + // Detect duplicate sequence names. + ers := b.ResolveSequence(n.Name.ToUnresolvedObjectName(), + ResolveParams{ + IsExistenceOptional: true, + RequiredPrivilege: privilege.USAGE, + WithOffline: true, // We search schema with name `schema`, including offline ones. + }) + if ers != nil && !ers.IsEmpty() { + if n.IfNotExists { + return + } + panic(sqlerrors.NewRelationAlreadyExistsError(n.Name.FQString())) + } + // Sanity check for duplication options on the sequence. + optionsSeen := map[string]bool{} + var sequenceOwnedBy *tree.ColumnItem + var restartWith *int64 + for _, opt := range n.Options { + _, seenBefore := optionsSeen[opt.Name] + if seenBefore { + panic(pgerror.New(pgcode.Syntax, "conflicting or redundant options")) + } + optionsSeen[opt.Name] = true + if opt.Name == tree.SeqOptOwnedBy { + sequenceOwnedBy = opt.ColumnItemVal + } + if opt.Name == tree.SeqOptRestart { + restartWith = opt.IntVal + } + } + // If the database is multi-region then CREATE SEQUENCE will fallback. + if _, _, dbRegionConfig := scpb.FindDatabaseRegionConfig(dbElts); dbRegionConfig != nil { + panic(scerrors.NotImplementedErrorf(n, "create sequence unsupported"+ + "on multi-region clusters, since locality will not be set")) + } + // Parse the sequence options before and validate they + // are supported. + defaultIntSize := b.SessionData().DefaultIntSize + tempSequenceOpts := descpb.TableDescriptor_SequenceOpts{ + Increment: 1, + } + if err := schemaexpr.AssignSequenceOptions( + &tempSequenceOpts, + n.Options, + defaultIntSize, + true, /*setDefaults*/ + nil, /*existingType*/ + ); err != nil { + panic(pgerror.WithCandidateCode(err, pgcode.InvalidParameterValue)) + } + // Generate the sequence elements. + sequenceID := b.GenerateUniqueDescID() + sequenceElem := &scpb.Sequence{ + SequenceID: sequenceID, + IsTemporary: false, + } + if restartWith != nil { + sequenceElem.RestartWith = *restartWith + sequenceElem.UseRestartWith = true + } + b.Add(sequenceElem) + // Setup the namespace entry. + sequenceNamespace := &scpb.Namespace{ + DatabaseID: dbElem.DatabaseID, + SchemaID: schemaElem.SchemaID, + DescriptorID: sequenceID, + Name: string(n.Name.ObjectName), + } + b.Add(sequenceNamespace) + // Add any sequence options. + options := scdecomp.GetSequenceOptions(sequenceElem.SequenceID, &tempSequenceOpts) + for _, opt := range options { + b.Add(opt) + } + // Add any sequence owned by element. + if sequenceOwnedBy != nil { + maybeAssignSequenceOwner(b, sequenceNamespace, sequenceOwnedBy) + } + // Add the single column for a sequence. + b.Add(&scpb.Column{ + TableID: sequenceID, + ColumnID: tabledesc.SequenceColumnID, + }) + b.Add(&scpb.ColumnType{ + TableID: sequenceID, + ColumnID: tabledesc.SequenceColumnID, + TypeT: scpb.TypeT{Type: types.Int}, + ElementCreationMetadata: &scpb.ElementCreationMetadata{In_23_1OrLater: true}, + }) + b.Add(&scpb.ColumnNotNull{ + TableID: sequenceID, + ColumnID: tabledesc.SequenceColumnID, + }) + b.Add(&scpb.ColumnName{ + TableID: sequenceID, + ColumnID: tabledesc.SequenceColumnID, + Name: tabledesc.SequenceColumnName, + }) + // Setup the primary index on the value column. + b.Add(&scpb.PrimaryIndex{ + Index: scpb.Index{ + TableID: sequenceID, + IndexID: keys.SequenceIndexID, + IsUnique: true, + }, + }) + b.Add(&scpb.IndexName{ + TableID: sequenceID, + IndexID: keys.SequenceIndexID, + Name: tabledesc.LegacyPrimaryKeyIndexName, + }) + b.Add(&scpb.IndexColumn{ + TableID: sequenceID, + IndexID: keys.SequenceIndexID, + ColumnID: tabledesc.SequenceColumnID, + OrdinalInKind: 0, + Kind: scpb.IndexColumn_KEY, + Direction: catenumpb.IndexColumn_ASC, + }) + // Setup ownership elements. + ownerElem, userPrivsElems := + b.BuildUserPrivilegesFromDefaultPrivileges(dbElem, schemaElem, sequenceID, privilege.Sequences, owner) + b.Add(ownerElem) + for _, userPrivsElem := range userPrivsElems { + b.Add(userPrivsElem) + } + // Log the creation of this sequence. + b.LogEventForExistingTarget(sequenceElem) +} + +func maybeAssignSequenceOwner(b BuildCtx, sequence *scpb.Namespace, owner *tree.ColumnItem) { + // Resolve the column first to validate its sane. + tableElts := b.ResolveTable(owner.TableName, ResolveParams{}) + _, _, tbl := scpb.FindTable(tableElts) + _, _, tblNamespace := scpb.FindNamespace(tableElts) + if tblNamespace.DatabaseID != sequence.DatabaseID { + if err := b.CanCreateCrossDBSequenceOwnerRef(); err != nil { + panic(err) + } + } + // Next resolve the column + colElts := b.ResolveColumn(tbl.TableID, owner.ColumnName, ResolveParams{}) + _, _, col := scpb.FindColumn(colElts) + // Create a sequence owner element + b.Add(&scpb.SequenceOwner{ + SequenceID: sequence.DescriptorID, + TableID: tbl.TableID, + ColumnID: col.ColumnID, + }) +} diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go index b5e1ec3d6088..536e7f617591 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go @@ -196,6 +196,10 @@ type SchemaFeatureChecker interface { CanPerformDropOwnedBy( ctx context.Context, role username.SQLUsername, ) (bool, error) + + // CanCreateCrossDBSequenceOwnerRef returns if cross database sequence + // owner references are allowed. + CanCreateCrossDBSequenceOwnerRef() error } // PrivilegeChecker checks an element's privileges. diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go index 1b2985f46b28..bcc95a909656 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go @@ -63,6 +63,7 @@ var supportedStatements = map[reflect.Type]supportedStatement{ reflect.TypeOf((*tree.DropFunction)(nil)): {fn: DropFunction, on: true, checks: isV231Active}, reflect.TypeOf((*tree.CreateFunction)(nil)): {fn: CreateFunction, on: true, checks: isV231Active}, reflect.TypeOf((*tree.CreateSchema)(nil)): {fn: CreateSchema, on: true, checks: isV232Active}, + reflect.TypeOf((*tree.CreateSequence)(nil)): {fn: CreateSequence, on: true, checks: isV232Active}, } func init() { diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check index 3cd0f25f3e65..ae07e308fafc 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check @@ -9,3 +9,7 @@ ALTER TABLE t ADD CHECK (i > 0) {columnIds: [1], constraintId: 2, expr: 'i > 0:::INT8', referencedColumnIds: [1], tableId: 104} - [[ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2}, PUBLIC], ABSENT] {constraintId: 2, name: check_i, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check_unvalidated b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check_unvalidated index ac291db4324f..4a1da86a76fe 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check_unvalidated +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_check_unvalidated @@ -9,3 +9,7 @@ ALTER TABLE t ADD CHECK (i > 0) NOT VALID {columnIds: [1], constraintId: 2, expr: 'i > 0:::INT8', referencedColumnIds: [1], tableId: 104} - [[ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2}, PUBLIC], ABSENT] {constraintId: 2, name: check_i, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column index 40cd7b40f9e6..23202e7be40e 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_column @@ -13,6 +13,10 @@ ALTER TABLE defaultdb.foo ADD COLUMN j INT {columnId: 2, elementCreationMetadata: {in231OrLater: true}, isNullable: true, tableId: 104, type: {family: IntFamily, oid: 20, width: 64}} - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1}, PUBLIC], ABSENT] {columnId: 2, indexId: 1, kind: STORED, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build ALTER TABLE defaultdb.foo ADD COLUMN j INT NOT NULL DEFAULT 123 @@ -53,6 +57,8 @@ ALTER TABLE defaultdb.foo ADD COLUMN j INT NOT NULL DEFAULT 123 {columnId: 2, indexId: 3, kind: STORED, tableId: 104} - [[ColumnNotNull:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] {columnId: 2, indexIdForValidation: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123; @@ -104,6 +110,8 @@ ALTER TABLE defaultdb.foo ADD COLUMN k INT DEFAULT 456; {columnId: 3, indexId: 2, kind: STORED, ordinalInKind: 1, tableId: 104} - [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] {columnId: 3, indexId: 3, kind: STORED, ordinalInKind: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED @@ -140,6 +148,8 @@ ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED {columnId: 2, indexId: 2, kind: STORED, tableId: 104} - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] {columnId: 2, indexId: 3, kind: STORED, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} setup CREATE TABLE defaultdb.bar (j INT); @@ -165,6 +175,14 @@ ALTER TABLE defaultdb.bar ADD COLUMN b INT; {columnId: 3, elementCreationMetadata: {in231OrLater: true}, isNullable: true, tableId: 105, type: {family: IntFamily, oid: 20, width: 64}} - [[IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 1}, PUBLIC], ABSENT] {columnId: 3, indexId: 1, kind: STORED, ordinalInKind: 1, tableId: 105} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} +- [[IndexData:{DescID: 105, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 105} +- [[TableData:{DescID: 105, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 105} build ALTER TABLE defaultdb.foo ADD COLUMN a INT, ADD COLUMN b INT; @@ -185,6 +203,10 @@ ALTER TABLE defaultdb.foo ADD COLUMN a INT, ADD COLUMN b INT; {columnId: 3, elementCreationMetadata: {in231OrLater: true}, isNullable: true, tableId: 104, type: {family: IntFamily, oid: 20, width: 64}} - [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1}, PUBLIC], ABSENT] {columnId: 3, indexId: 1, kind: STORED, ordinalInKind: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} setup CREATE TABLE t (i INT PRIMARY KEY, j INT NOT NULL) @@ -239,3 +261,5 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j) {columnId: 2, indexId: 3, kind: KEY_SUFFIX, tableId: 106} - [[IndexName:{DescID: 106, Name: t_i_key, IndexID: 2}, PUBLIC], ABSENT] {indexId: 2, name: t_i_key, tableId: 106} +- [[TableData:{DescID: 106, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 106} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key index aa77fd74a243..c71495cc0c00 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key @@ -10,3 +10,11 @@ ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); {columnIds: [1], constraintId: 2, referencedColumnIds: [1], referencedTableId: 104, tableId: 105} - [[ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2}, PUBLIC], ABSENT] {constraintId: 2, name: t1_i_fkey, tableId: 105} +- [[IndexData:{DescID: 105, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 105} +- [[TableData:{DescID: 105, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 105} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key_unvalidated b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key_unvalidated index 27171d7ef64a..b29530954fae 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key_unvalidated +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_foreign_key_unvalidated @@ -10,3 +10,11 @@ ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i) NOT VALID; {columnIds: [1], constraintId: 2, referencedColumnIds: [1], referencedTableId: 104, tableId: 105} - [[ConstraintWithoutIndexName:{DescID: 105, Name: t1_i_fkey, ConstraintID: 2}, PUBLIC], ABSENT] {constraintId: 2, name: t1_i_fkey, tableId: 105} +- [[IndexData:{DescID: 105, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 105} +- [[TableData:{DescID: 105, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 105} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_primary_key b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_primary_key index 0a05f8f5b446..ed3e7f4f7398 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_primary_key +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_primary_key @@ -58,3 +58,5 @@ ALTER TABLE defaultdb.bar ADD PRIMARY KEY (i) {columnId: 1, indexId: 5, tableId: 105} - [[IndexData:{DescID: 105, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] {indexId: 5, tableId: 105} +- [[TableData:{DescID: 105, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 105} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index index 0701f9eee60f..0437126c7fd2 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index @@ -9,3 +9,7 @@ ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); {columnIds: [2], constraintId: 2, tableId: 104} - [[ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2}, PUBLIC], ABSENT] {constraintId: 2, name: unique_j, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index_unvalidated b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index_unvalidated index b9352bd40e4a..1880f24c9938 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index_unvalidated +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_add_unique_without_index_unvalidated @@ -9,3 +9,7 @@ ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j) NOT VALID; {columnIds: [2], constraintId: 2, tableId: 104} - [[ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2}, PUBLIC], ABSENT] {constraintId: 2, name: unique_j, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_primary_key b/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_primary_key index d6b4df32c054..439d5a0958f0 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_primary_key +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_alter_primary_key @@ -62,6 +62,8 @@ ALTER TABLE defaultdb.foo ALTER PRIMARY KEY USING COLUMNS (j) {columnId: 2, indexId: 3, kind: KEY_SUFFIX, tableId: 104} - [[IndexName:{DescID: 104, Name: foo_i_key, IndexID: 2}, PUBLIC], ABSENT] {indexId: 2, name: foo_i_key, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build ALTER TABLE defaultdb.bar ALTER PRIMARY KEY USING COLUMNS (i) @@ -118,3 +120,5 @@ ALTER TABLE defaultdb.bar ALTER PRIMARY KEY USING COLUMNS (i) {columnId: 1, indexId: 5, tableId: 105} - [[IndexData:{DescID: 105, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] {indexId: 5, tableId: 105} +- [[TableData:{DescID: 105, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 105} diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column b/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column index d385424aebfa..234e6e53bb96 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column @@ -84,6 +84,8 @@ ALTER TABLE defaultdb.t DROP COLUMN j {columnId: 4, indexId: 5, kind: STORED, ordinalInKind: 1, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] {indexId: 5, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build ALTER TABLE defaultdb.t DROP COLUMN k, DROP COLUMN l @@ -166,6 +168,10 @@ ALTER TABLE defaultdb.t DROP COLUMN k, DROP COLUMN l {childObjectId: 105, schemaId: 101} - [[TableData:{DescID: 105, ReferencedDescID: 100}, ABSENT], PUBLIC] {databaseId: 100, tableId: 105} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build ALTER TABLE defaultdb.t DROP COLUMN l @@ -232,3 +238,9 @@ ALTER TABLE defaultdb.t DROP COLUMN l {columnId: 3, indexId: 5, kind: STORED, ordinalInKind: 1, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] {indexId: 5, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 3}, PUBLIC], PUBLIC] + {indexId: 3, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/comment_on b/pkg/sql/schemachanger/scbuild/testdata/comment_on index 62b85a67e6bd..c67cf634fc13 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/comment_on +++ b/pkg/sql/schemachanger/scbuild/testdata/comment_on @@ -65,6 +65,12 @@ COMMENT ON TABLE customer IS 'customer is god'; ---- - [[TableComment:{DescID: 104, Comment: customer is god}, PUBLIC], ABSENT] {comment: customer is god, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} setup COMMENT ON TABLE customer IS 'customer real comment'; @@ -75,18 +81,36 @@ COMMENT ON TABLE customer IS 'customer gets better'; ---- - [[TableComment:{DescID: 104, Comment: customer gets better}, PUBLIC], ABSENT] {comment: customer gets better, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON TABLE customer IS NULL; ---- - [[TableComment:{DescID: 104, Comment: customer real comment}, ABSENT], PUBLIC] {comment: customer real comment, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON COLUMN customer.name IS 'some comment for name'; ---- - [[ColumnComment:{DescID: 104, ColumnID: 2, Comment: some comment for name}, PUBLIC], ABSENT] {columnId: 2, comment: some comment for name, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} setup COMMENT ON COLUMN customer.name IS 'real comment for name'; @@ -97,30 +121,60 @@ COMMENT ON COLUMN customer.name IS 'some better comment for name'; ---- - [[ColumnComment:{DescID: 104, ColumnID: 2, Comment: some better comment for name}, PUBLIC], ABSENT] {columnId: 2, comment: some better comment for name, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON COLUMN customer.name IS NULL; ---- - [[ColumnComment:{DescID: 104, ColumnID: 2, Comment: real comment for name}, ABSENT], PUBLIC] {columnId: 2, comment: real comment for name, pgAttributeNum: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON INDEX customer@idx_customer_name IS 'comment on name index'; ---- - [[IndexComment:{DescID: 104, IndexID: 2, Comment: comment on name index}, PUBLIC], ABSENT] {comment: comment on name index, indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON INDEX public.idx_customer_name IS 'comment on name index'; ---- - [[IndexComment:{DescID: 104, IndexID: 2, Comment: comment on name index}, PUBLIC], ABSENT] {comment: comment on name index, indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON INDEX defaultdb.idx_customer_name IS 'comment on name index'; ---- - [[IndexComment:{DescID: 104, IndexID: 2, Comment: comment on name index}, PUBLIC], ABSENT] {comment: comment on name index, indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} setup COMMENT ON INDEX customer@idx_customer_name IS 'real comment on name index'; @@ -131,30 +185,60 @@ COMMENT ON INDEX customer@idx_customer_name IS 'better comment for name index'; ---- - [[IndexComment:{DescID: 104, IndexID: 2, Comment: better comment for name index}, PUBLIC], ABSENT] {comment: better comment for name index, indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON INDEX customer@idx_customer_name IS NULL; ---- - [[IndexComment:{DescID: 104, IndexID: 2, Comment: real comment on name index}, ABSENT], PUBLIC] {comment: real comment on name index, indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON INDEX customer@idx_customer_name IS 'better comment for name index' ---- - [[IndexComment:{DescID: 104, IndexID: 2, Comment: better comment for name index}, PUBLIC], ABSENT] {comment: better comment for name index, indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON INDEX customer@idx_customer_name IS NULL; ---- - [[IndexComment:{DescID: 104, IndexID: 2, Comment: real comment on name index}, ABSENT], PUBLIC] {comment: real comment on name index, indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON CONSTRAINT customer_age_gt_21 ON customer IS 'better be old enough' ---- - [[ConstraintComment:{DescID: 104, ConstraintID: 2, Comment: better be old enough}, PUBLIC], ABSENT] {comment: better be old enough, constraintId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} setup COMMENT ON CONSTRAINT customer_age_gt_21 ON customer IS 'real constraint comment' @@ -165,9 +249,21 @@ COMMENT ON CONSTRAINT customer_age_gt_21 ON customer IS 'better be old enough' ---- - [[ConstraintComment:{DescID: 104, ConstraintID: 2, Comment: better be old enough}, PUBLIC], ABSENT] {comment: better be old enough, constraintId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build COMMENT ON CONSTRAINT customer_age_gt_21 ON customer IS NULL ---- - [[ConstraintComment:{DescID: 104, ConstraintID: 2, Comment: real constraint comment}, ABSENT], PUBLIC] {comment: real constraint comment, constraintId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/create_function b/pkg/sql/schemachanger/scbuild/testdata/create_function index c2b277073dda..762d1cbd06b7 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/create_function +++ b/pkg/sql/schemachanger/scbuild/testdata/create_function @@ -37,3 +37,13 @@ $$; {descriptorId: 110, privileges: "2", userName: root, withGrantOption: "2"} - [[FunctionBody:{DescID: 110}, PUBLIC], ABSENT] {body: "SELECT a FROM t;\nSELECT b FROM t@t_idx_b;\nSELECT c FROM t@t_idx_c;\nSELECT a FROM v;\nSELECT nextval(105:::REGCLASS);", functionId: 110, lang: {lang: SQL}, usesSequenceIds: [105], usesTables: [{columnIds: [1], tableId: 104}, {columnIds: [2], indexId: 2, tableId: 104}, {columnIds: [3], indexId: 3, tableId: 104}], usesTypeIds: [108, 109], usesViews: [{columnIds: [1], viewId: 107}]} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 3}, PUBLIC], PUBLIC] + {indexId: 3, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} +- [[TableData:{DescID: 105, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 105} diff --git a/pkg/sql/schemachanger/scbuild/testdata/create_index b/pkg/sql/schemachanger/scbuild/testdata/create_index index ed1f77fe6cdf..a03b5245d55f 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/create_index +++ b/pkg/sql/schemachanger/scbuild/testdata/create_index @@ -27,6 +27,10 @@ CREATE INDEX id1 ON defaultdb.t1(id, name) STORING (money) {columnId: 3, indexId: 3, kind: STORED, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] {indexId: 3, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build CREATE INVERTED INDEX CONCURRENTLY id2 @@ -50,6 +54,10 @@ CREATE INVERTED INDEX CONCURRENTLY id2 {columnId: 2, indexId: 3, invertedKind: 1, ordinalInKind: 1, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] {indexId: 3, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build skip=sql_dependencies CREATE INDEX id3 @@ -83,6 +91,10 @@ CREATE INDEX id3 {columnId: 3, indexId: 3, kind: STORED, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] {indexId: 3, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build CREATE INDEX id4 @@ -126,3 +138,7 @@ CREATE INDEX id4 {columnId: 3, indexId: 3, kind: STORED, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] {indexId: 3, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/create_sequence b/pkg/sql/schemachanger/scbuild/testdata/create_sequence new file mode 100644 index 000000000000..2283a91b64ab --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/testdata/create_sequence @@ -0,0 +1,40 @@ +setup +CREATE DATABASE db; +CREATE TABLE db.public.tbl (i INT PRIMARY KEY, k INT); +---- + +build +CREATE SEQUENCE db.public.sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32 OWNED BY db.public.tbl.k; +---- +- [[Sequence:{DescID: 107}, PUBLIC], ABSENT] + {sequenceId: 107} +- [[Namespace:{DescID: 107, Name: sq1, ReferencedDescID: 104}, PUBLIC], ABSENT] + {databaseId: 104, descriptorId: 107, name: sq1, schemaId: 105} +- [[SequenceOption:{DescID: 107, Name: START}, PUBLIC], ABSENT] + {key: START, sequenceId: 107, value: "32"} +- [[SequenceOwner:{DescID: 106, ColumnID: 2, ReferencedDescID: 107}, PUBLIC], ABSENT] + {columnId: 2, sequenceId: 107, tableId: 106} +- [[Column:{DescID: 107, ColumnID: 1}, PUBLIC], ABSENT] + {columnId: 1, tableId: 107} +- [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC], ABSENT] + {columnId: 1, elementCreationMetadata: {in231OrLater: true}, tableId: 107, type: {family: IntFamily, oid: 20, width: 64}} +- [[ColumnNotNull:{DescID: 107, ColumnID: 1, IndexID: 0}, PUBLIC], ABSENT] + {columnId: 1, tableId: 107} +- [[ColumnName:{DescID: 107, Name: value, ColumnID: 1}, PUBLIC], ABSENT] + {columnId: 1, name: value, tableId: 107} +- [[PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 0}, PUBLIC], ABSENT] + {indexId: 1, isUnique: true, tableId: 107} +- [[IndexName:{DescID: 107, Name: primary, IndexID: 1}, PUBLIC], ABSENT] + {indexId: 1, name: primary, tableId: 107} +- [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, PUBLIC], ABSENT] + {columnId: 1, indexId: 1, tableId: 107} +- [[Owner:{DescID: 107}, PUBLIC], ABSENT] + {descriptorId: 107, owner: root} +- [[UserPrivileges:{DescID: 107, Name: admin}, PUBLIC], ABSENT] + {descriptorId: 107, privileges: "2", userName: admin, withGrantOption: "2"} +- [[UserPrivileges:{DescID: 107, Name: root}, PUBLIC], ABSENT] + {descriptorId: 107, privileges: "2", userName: root, withGrantOption: "2"} +- [[IndexData:{DescID: 106, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 106} +- [[TableData:{DescID: 106, ReferencedDescID: 104}, PUBLIC], PUBLIC] + {databaseId: 104, tableId: 106} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_index b/pkg/sql/schemachanger/scbuild/testdata/drop_index index f8fa79e30776..3b96a1777098 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_index +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_index @@ -27,6 +27,14 @@ DROP INDEX idx1 CASCADE {indexId: 2, name: idx1, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 2}, ABSENT], PUBLIC] {indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], PUBLIC] + {indexId: 4, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 6}, PUBLIC], PUBLIC] + {indexId: 6, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build DROP INDEX idx2 CASCADE @@ -47,6 +55,14 @@ DROP INDEX idx2 CASCADE {indexId: 4, name: idx2, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 4}, ABSENT], PUBLIC] {indexId: 4, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 6}, PUBLIC], PUBLIC] + {indexId: 6, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build DROP INDEX idx3 CASCADE @@ -105,6 +121,14 @@ DROP INDEX idx3 CASCADE {columnId: 4.294967294e+09, name: tableoid, tableId: 105} - [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] {columnId: 4.294967294e+09, elementCreationMetadata: {in231OrLater: true}, isNullable: true, tableId: 105, type: {family: OidFamily, oid: 26}} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] + {indexId: 2, tableId: 104} +- [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], PUBLIC] + {indexId: 4, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} build DROP INDEX v2@idx CASCADE @@ -173,3 +197,7 @@ DROP INDEX v2@idx CASCADE {indexId: 1, tableId: 107} - [[TableData:{DescID: 107, ReferencedDescID: 100}, ABSENT], PUBLIC] {databaseId: 100, tableId: 107} +- [[IndexData:{DescID: 106, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 106} +- [[TableData:{DescID: 106, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 106} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by b/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by index 5d3e88a8d843..d4e7d6a7cf68 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by @@ -282,3 +282,7 @@ DROP OWNED BY r {columnId: 4.294967294e+09, name: tableoid, tableId: 113} - [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] {columnId: 4.294967294e+09, elementCreationMetadata: {in231OrLater: true}, isNullable: true, tableId: 113, type: {family: OidFamily, oid: 26}} +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_sequence b/pkg/sql/schemachanger/scbuild/testdata/drop_sequence index 42ab5540d4f2..896f468ec001 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_sequence @@ -50,6 +50,18 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE {columnId: 2, expr: 'nextval(104:::REGCLASS)', tableId: 106, usesSequenceIds: [104]} - [[ColumnDefaultExpression:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] {columnId: 2, expr: 'CAST(chr(nextval(104:::REGCLASS)) AS @100107)', tableId: 109, usesSequenceIds: [104], usesTypeIds: [107, 108]} +- [[IndexData:{DescID: 105, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 105} +- [[TableData:{DescID: 105, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 105} +- [[IndexData:{DescID: 106, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 106} +- [[TableData:{DescID: 106, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 106} +- [[IndexData:{DescID: 109, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 109} +- [[TableData:{DescID: 109, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 109} setup CREATE TABLE defaultdb.ownertbl (id INT NOT NULL); @@ -75,3 +87,7 @@ DROP SEQUENCE defaultdb.ownedseq CASCADE {databaseId: 100, tableId: 111} - [[SequenceOwner:{DescID: 110, ColumnID: 1, ReferencedDescID: 111}, ABSENT], PUBLIC] {columnId: 1, sequenceId: 111, tableId: 110} +- [[IndexData:{DescID: 110, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 110} +- [[TableData:{DescID: 110, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 110} diff --git a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_create b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_create index a4455d7c9ba3..4d48725adee2 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_create +++ b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_create @@ -14,10 +14,6 @@ unimplemented CREATE SCHEMA sc AUTHORIZATION roacher; ---- -unimplemented -CREATE SEQUENCE seq; ----- - unimplemented CREATE TYPE typ AS ENUM('a','b'); ---- diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go index 3a84dc625e09..f7449b955519 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp.go +++ b/pkg/sql/schemachanger/scdecomp/decomp.go @@ -12,6 +12,8 @@ package scdecomp import ( "context" + "fmt" + "reflect" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" @@ -19,10 +21,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -225,6 +229,52 @@ func (w *walkCtx) walkType(typ catalog.TypeDescriptor) { } } +func GetSequenceOptions( + sequenceID descpb.ID, opts *descpb.TableDescriptor_SequenceOpts, +) []*scpb.SequenceOption { + // Compute the default sequence options. + defaultOpts := descpb.TableDescriptor_SequenceOpts{ + Increment: 1, + } + err := schemaexpr.AssignSequenceOptions(&defaultOpts, + nil, + 64, + true, + nil, + ) + if err != nil { + panic(err) + } + var sequenceOptions []*scpb.SequenceOption + addSequenceOption := func(key string, defaultValue, value interface{}) { + // Nil or empty values can be skipped. Or values which + // are the defaults. + if value == nil || reflect.DeepEqual(defaultValue, value) { + return + } + valueStr := fmt.Sprintf("%v", value) + if len(valueStr) == 0 { + return + } + sequenceOptions = append( + sequenceOptions, + &scpb.SequenceOption{ + SequenceID: sequenceID, + Key: key, + Value: valueStr, + }) + } + + addSequenceOption(tree.SeqOptIncrement, defaultOpts.Increment, opts.Increment) + addSequenceOption(tree.SeqOptMinValue, defaultOpts.MinValue, opts.MinValue) + addSequenceOption(tree.SeqOptMaxValue, defaultOpts.MaxValue, opts.MaxValue) + addSequenceOption(tree.SeqOptStart, defaultOpts.Start, opts.Start) + addSequenceOption(tree.SeqOptVirtual, defaultOpts.Virtual, opts.Virtual) + addSequenceOption(tree.SeqOptCache, defaultOpts.CacheSize, opts.CacheSize) + addSequenceOption(tree.SeqOptAs, defaultOpts.AsIntegerType, opts.AsIntegerType) + return sequenceOptions +} + func (w *walkCtx) walkRelation(tbl catalog.TableDescriptor) { switch { case tbl.IsSequence(): @@ -233,7 +283,13 @@ func (w *walkCtx) walkRelation(tbl catalog.TableDescriptor) { IsTemporary: tbl.IsTemporary(), }) if opts := tbl.GetSequenceOpts(); opts != nil { + w.backRefs.Add(opts.SequenceOwner.OwnerTableID) + options := GetSequenceOptions(tbl.GetID(), opts) + for _, opt := range options { + w.ev(descriptorStatus(tbl), + opt) + } } case tbl.IsView(): w.ev(descriptorStatus(tbl), &scpb.View{ diff --git a/pkg/sql/schemachanger/scdecomp/testdata/sequence b/pkg/sql/schemachanger/scdecomp/testdata/sequence index be0f6fc7d70b..98ce8263b5e0 100644 --- a/pkg/sql/schemachanger/scdecomp/testdata/sequence +++ b/pkg/sql/schemachanger/scdecomp/testdata/sequence @@ -31,7 +31,9 @@ ElementState: Status: PUBLIC - Sequence: isTemporary: false + restartWith: "0" sequenceId: 104 + useRestartWith: false Status: PUBLIC - TableData: databaseId: 100 @@ -76,7 +78,9 @@ ElementState: Status: PUBLIC - Sequence: isTemporary: false + restartWith: "0" sequenceId: 106 + useRestartWith: false Status: PUBLIC - TableData: databaseId: 100 @@ -700,7 +704,9 @@ ElementState: Status: PUBLIC - Sequence: isTemporary: false + restartWith: "0" sequenceId: 104 + useRestartWith: false Status: PUBLIC - TableData: databaseId: 100 diff --git a/pkg/sql/schemachanger/scdeps/BUILD.bazel b/pkg/sql/schemachanger/scdeps/BUILD.bazel index 33847b54dbb5..0782c30dd70e 100644 --- a/pkg/sql/schemachanger/scdeps/BUILD.bazel +++ b/pkg/sql/schemachanger/scdeps/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "//pkg/sql/schemachanger/scexec/backfiller", "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scrun", + "//pkg/sql/sem/catconstants", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/sql/schemachanger/scdeps/build_deps.go b/pkg/sql/schemachanger/scdeps/build_deps.go index 4a30c829e643..9d787284f9d1 100644 --- a/pkg/sql/schemachanger/scdeps/build_deps.go +++ b/pkg/sql/schemachanger/scdeps/build_deps.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -129,17 +130,41 @@ func (d *buildDeps) MayResolveSchema( func (d *buildDeps) MustResolvePrefix( ctx context.Context, name tree.ObjectNamePrefix, ) (catalog.DatabaseDescriptor, catalog.SchemaDescriptor) { - if !name.ExplicitCatalog { - name.CatalogName = tree.Name(d.schemaResolver.CurrentDatabase()) - name.ExplicitCatalog = true - } - + const withOffline = false if name.ExplicitSchema { - db, sc := d.MayResolveSchema(ctx, name, false /* withOffline */) - if sc == nil { - panic(errors.AssertionFailedf("prefix %s does not exist", name.String())) + if name.ExplicitCatalog { + db, sc := d.MayResolveSchema(ctx, name, withOffline) + if sc == nil || db == nil { + panic(errors.AssertionFailedf("prefix %s does not exist", name.String())) + } + return db, sc } - return db, sc + + // Two parts: D.T. + // Try to use the current database, and be satisfied if it's sufficient to find the object. + db, sc := d.MayResolveSchema(ctx, tree.ObjectNamePrefix{ + CatalogName: tree.Name(d.CurrentDatabase()), + SchemaName: name.SchemaName, + ExplicitCatalog: true, + ExplicitSchema: true, + }, + withOffline) + if db != nil && sc != nil { + return db, sc + } + + // No luck so far. Compatibility with CockroachDB v1.1: use D.public.T instead. + db, sc = d.MayResolveSchema(ctx, tree.ObjectNamePrefix{ + CatalogName: name.SchemaName, + SchemaName: catconstants.PublicSchemaName, + ExplicitCatalog: true, + ExplicitSchema: true, + }, + withOffline) + if db != nil && sc != nil { + return db, sc + } + panic(errors.AssertionFailedf("prefix %s does not exist", name.String())) } path := d.sessionData.SearchPath @@ -152,7 +177,6 @@ func (d *buildDeps) MustResolvePrefix( return db, sc } } - panic(errors.AssertionFailedf("prefix %s does not exist", name.String())) } diff --git a/pkg/sql/schemachanger/scdeps/exec_deps.go b/pkg/sql/schemachanger/scdeps/exec_deps.go index 1926669ffa6f..bd8acb0a4635 100644 --- a/pkg/sql/schemachanger/scdeps/exec_deps.go +++ b/pkg/sql/schemachanger/scdeps/exec_deps.go @@ -238,6 +238,13 @@ func (d *txnDeps) Run(ctx context.Context) error { return nil } +// InitializeSequence implements the scexec.Caatalog interface. +func (d *txnDeps) InitializeSequence(id descpb.ID, startVal int64) { + batch := d.getOrCreateBatch() + sequenceKey := d.codec.SequenceKey(uint32(id)) + batch.Inc(sequenceKey, startVal) +} + // Reset implements the scexec.Catalog interface. func (d *txnDeps) Reset(ctx context.Context) error { d.descsCollection.ResetUncommitted(ctx) diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 59c156658ca5..be6721caad4d 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -1374,3 +1374,8 @@ func getNameEntryDescriptorType(parentID, parentSchemaID descpb.ID) string { } return ret } + +// InitializeSequence is part of the scexec.Catalog interface. +func (s *TestState) InitializeSequence(id descpb.ID, startVal int64) { + s.LogSideEffectf("initializing sequence %d with starting value of %d", id, startVal) +} diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go index 1c9a288c436c..aad9b64778eb 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go @@ -214,6 +214,11 @@ func (s *TestState) CanPerformDropOwnedBy( return true, nil } +// CanCreateCrossDBSequenceOwnerRef implements scbuild.SchemaFeatureCheck. +func (s *TestState) CanCreateCrossDBSequenceOwnerRef() error { + return nil +} + // FeatureChecker implements scbuild.Dependencies func (s *TestState) FeatureChecker() scbuild.FeatureChecker { return s diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index b1a84f7b9d2c..b84d281c2821 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -94,6 +94,9 @@ type Catalog interface { // transaction so far, assuming that they haven't been persisted yet // by calling Run. Reset(ctx context.Context) error + + // InitializeSequence initializes the initial value for a sequence. + InitializeSequence(id descpb.ID, startVal int64) } // Telemetry encapsulates metrics gather for the declarative schema changer. diff --git a/pkg/sql/schemachanger/scexec/exec_immediate_mutation.go b/pkg/sql/schemachanger/scexec/exec_immediate_mutation.go index 9b7b683a2d15..c77a0d605ae2 100644 --- a/pkg/sql/schemachanger/scexec/exec_immediate_mutation.go +++ b/pkg/sql/schemachanger/scexec/exec_immediate_mutation.go @@ -29,6 +29,7 @@ type immediateState struct { newDescriptors map[descpb.ID]catalog.MutableDescriptor addedNames map[descpb.ID]descpb.NameInfo withReset bool + sequencesToInit []sequenceToInit } type commentToUpdate struct { @@ -38,6 +39,11 @@ type commentToUpdate struct { comment string } +type sequenceToInit struct { + id descpb.ID + startVal int64 +} + var _ scmutationexec.ImmediateMutationStateUpdater = (*immediateState)(nil) func (s *immediateState) AddToCheckedOutDescriptors(mut catalog.MutableDescriptor) { @@ -108,6 +114,14 @@ func (s *immediateState) CreateDescriptor(desc catalog.MutableDescriptor) { s.newDescriptors[desc.GetID()] = desc } +func (s *immediateState) InitSequence(id descpb.ID, startVal int64) { + s.sequencesToInit = append(s.sequencesToInit, + sequenceToInit{ + id: id, + startVal: startVal, + }) +} + func (s *immediateState) Reset() { s.withReset = true } @@ -161,5 +175,8 @@ func (s *immediateState) exec(ctx context.Context, c Catalog) error { } } } + for _, s := range s.sequencesToInit { + c.InitializeSequence(s.id, s.startVal) + } return c.Validate(ctx) } diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 1624bbe84d83..231f2e82e6d9 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -297,12 +297,21 @@ func TestSchemaChanger(t *testing.T) { }, metadata, ), + scpb.MakeTarget( + scpb.ToPublic, + &scpb.TableData{ + TableID: fooTable.GetID(), + DatabaseID: fooTable.GetParentID(), + }, + metadata, + ), } initial := []scpb.Status{ scpb.Status_ABSENT, scpb.Status_ABSENT, scpb.Status_ABSENT, scpb.Status_ABSENT, + scpb.Status_PUBLIC, } cs = scpb.CurrentState{ TargetState: scpb.TargetState{Statements: stmts, Targets: targets}, @@ -333,6 +342,7 @@ func TestSchemaChanger(t *testing.T) { scpb.Status_PUBLIC, scpb.Status_PUBLIC, scpb.Status_PUBLIC, + scpb.Status_PUBLIC, }, after.Current) ti.tsql.Exec(t, "INSERT INTO db.foo VALUES (1, 1)") }) diff --git a/pkg/sql/schemachanger/scexec/mocks_generated_test.go b/pkg/sql/schemachanger/scexec/mocks_generated_test.go index 11c30b7094c5..a699d6a10940 100644 --- a/pkg/sql/schemachanger/scexec/mocks_generated_test.go +++ b/pkg/sql/schemachanger/scexec/mocks_generated_test.go @@ -141,6 +141,18 @@ func (mr *MockCatalogMockRecorder) GetFullyQualifiedName(arg0, arg1 interface{}) return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetFullyQualifiedName", reflect.TypeOf((*MockCatalog)(nil).GetFullyQualifiedName), arg0, arg1) } +// InitializeSequence mocks base method. +func (m *MockCatalog) InitializeSequence(arg0 catid.DescID, arg1 int64) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "InitializeSequence", arg0, arg1) +} + +// InitializeSequence indicates an expected call of InitializeSequence. +func (mr *MockCatalogMockRecorder) InitializeSequence(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InitializeSequence", reflect.TypeOf((*MockCatalog)(nil).InitializeSequence), arg0, arg1) +} + // MustReadImmutableDescriptors mocks base method. func (m *MockCatalog) MustReadImmutableDescriptors(arg0 context.Context, arg1 ...catid.DescID) ([]catalog.Descriptor, error) { m.ctrl.T.Helper() diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel index 621b4fe46d69..84c390f414f4 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "schema.go", "schema_change_job.go", "scmutationexec.go", + "sequence.go", "stats.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec", diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/create.go b/pkg/sql/schemachanger/scexec/scmutationexec/create.go index 22d4e560fd5b..1792e08f6ff7 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/create.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/create.go @@ -34,13 +34,23 @@ func (i *immediateVisitor) MarkDescriptorAsPublic( return nil } -func (i *immediateVisitor) AddDescriptorName(_ context.Context, op scop.AddDescriptorName) error { +func (i *immediateVisitor) AddDescriptorName(ctx context.Context, op scop.AddDescriptorName) error { nameDetails := descpb.NameInfo{ ParentID: op.Namespace.DatabaseID, ParentSchemaID: op.Namespace.SchemaID, Name: op.Namespace.Name, } i.AddName(op.Namespace.DescriptorID, nameDetails) + desc, err := i.checkOutDescriptor(ctx, op.Namespace.DescriptorID) + if err != nil { + return err + } + + switch t := desc.(type) { + case *tabledesc.Mutable: + t.ParentID = op.Namespace.DatabaseID + t.UnexposedParentSchemaID = op.Namespace.SchemaID + } return nil } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go b/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go index a0aaf3fe8047..09a0dcba7d04 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go @@ -79,6 +79,9 @@ type ImmediateMutationStateUpdater interface { // AddComment adds comments for a descriptor. AddComment(id descpb.ID, subID int, commentType catalogkeys.CommentType, comment string) + // InitSequence initializes a sequence. + InitSequence(id descpb.ID, startVal int64) + // Reset schedules a reset of the in-txn catalog state // to undo the modifications from earlier stages. Reset() diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go index 2ab4e3324891..863cfa9668bb 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go @@ -71,6 +71,34 @@ func (i *immediateVisitor) RemoveOwnerBackReferenceInSequence( return nil } +func (i *immediateVisitor) AddOwnerBackReferenceInSequence( + ctx context.Context, op scop.AddOwnerBackReferenceInSequence, +) error { + seq, err := i.checkOutTable(ctx, op.SequenceID) + if err != nil || seq.Dropped() { + return err + } + opts := seq.GetSequenceOpts() + opts.SequenceOwner.OwnerColumnID = op.ColumnID + opts.SequenceOwner.OwnerTableID = op.TableID + return nil +} + +func (i *immediateVisitor) AddSequenceOwner(ctx context.Context, op scop.AddSequenceOwner) error { + tbl, err := i.checkOutTable(ctx, op.TableID) + if err != nil || tbl.Dropped() { + return err + } + col, err := catalog.MustFindColumnByID(tbl, op.ColumnID) + if err != nil { + return err + } + ids := catalog.MakeDescriptorIDSet(col.ColumnDesc().OwnsSequenceIds...) + ids.Add(op.OwnedSequenceID) + col.ColumnDesc().OwnsSequenceIds = ids.Ordered() + return nil +} + func (i *immediateVisitor) RemoveSequenceOwner( ctx context.Context, op scop.RemoveSequenceOwner, ) error { @@ -79,7 +107,7 @@ func (i *immediateVisitor) RemoveSequenceOwner( return err } col, err := catalog.MustFindColumnByID(tbl, op.ColumnID) - if err != nil || col == nil { + if err != nil { return err } ids := catalog.MakeDescriptorIDSet(col.ColumnDesc().OwnsSequenceIds...) diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/sequence.go b/pkg/sql/schemachanger/scexec/scmutationexec/sequence.go new file mode 100644 index 000000000000..518c79e81431 --- /dev/null +++ b/pkg/sql/schemachanger/scexec/scmutationexec/sequence.go @@ -0,0 +1,106 @@ +// 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 scmutationexec + +import ( + "context" + "strconv" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +func (i *immediateVisitor) CreateSequenceDescriptor( + _ context.Context, op scop.CreateSequenceDescriptor, +) error { + mut := tabledesc.NewBuilder(&descpb.TableDescriptor{ + ParentID: catid.InvalidDescID, // Set by `SchemaParent` element + Name: "", // Set by `Namespace` element + ID: op.SequenceID, + Privileges: &catpb.PrivilegeDescriptor{Version: catpb.Version21_2}, // Populated by `UserPrivileges` elements and `Owner` element + Version: 1, + FormatVersion: descpb.InterleavedFormatVersion, + }).BuildCreatedMutable() + tabledDesc := mut.(*tabledesc.Mutable) + tabledDesc.State = descpb.DescriptorState_ADD + // Set the default sequence options. + tabledDesc.SequenceOpts = &descpb.TableDescriptor_SequenceOpts{ + Increment: 1, + } + if err := schemaexpr.AssignSequenceOptions(tabledDesc.SequenceOpts, + nil, + 64, + true, + nil, + ); err != nil { + return err + } + i.CreateDescriptor(mut) + return nil +} + +func (i *immediateVisitor) SetSequenceOptions( + ctx context.Context, op scop.SetSequenceOptions, +) error { + sc, err := i.checkOutTable(ctx, op.SequenceID) + if err != nil { + return err + } + + setIntValue := func(target *int64) func(Value string) error { + return func(Value string) error { + var err error + *target, err = strconv.ParseInt(Value, 10, 64) + return err + } + } + setBoolValue := func(target *bool) func(Value string) error { + return func(Value string) error { + var err error + *target, err = strconv.ParseBool(Value) + return err + } + } + sequenceOptionMeta := map[string]struct { + SetFunc func(Value string) error + }{ + tree.SeqOptIncrement: {SetFunc: setIntValue(&sc.SequenceOpts.Increment)}, + tree.SeqOptMinValue: {SetFunc: setIntValue(&sc.SequenceOpts.MinValue)}, + tree.SeqOptMaxValue: {SetFunc: setIntValue(&sc.SequenceOpts.MaxValue)}, + tree.SeqOptStart: {SetFunc: setIntValue(&sc.SequenceOpts.Start)}, + tree.SeqOptCache: {SetFunc: setIntValue(&sc.SequenceOpts.CacheSize)}, + tree.SeqOptVirtual: {SetFunc: setBoolValue(&sc.SequenceOpts.Virtual)}, + tree.SeqOptAs: {SetFunc: func(Value string) error { + sc.SequenceOpts.AsIntegerType = Value + return nil + }}, + } + return sequenceOptionMeta[op.Key].SetFunc(op.Value) +} + +func (i *immediateVisitor) InitSequence(ctx context.Context, op scop.InitSequence) error { + sc, err := i.checkOutTable(ctx, op.SequenceID) + if err != nil { + return err + } + startVal := sc.SequenceOpts.Start + if op.UseRestartWith { + startVal = op.RestartWith + } + startVal = startVal - sc.SequenceOpts.Increment + i.ImmediateMutationStateUpdater.InitSequence(sc.ID, startVal) + return nil +} diff --git a/pkg/sql/schemachanger/scop/immediate_mutation.go b/pkg/sql/schemachanger/scop/immediate_mutation.go index b61a87c5ae91..513604c20517 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation.go @@ -259,6 +259,24 @@ type MakeDeleteOnlyColumnAbsent struct { ColumnID descpb.ColumnID } +// AddOwnerBackReferenceInSequence adds a sequence ownership +// back-reference from a sequence. +type AddOwnerBackReferenceInSequence struct { + immediateMutationOp + SequenceID descpb.ID + TableID descpb.ID + ColumnID descpb.ColumnID +} + +// AddSequenceOwner adds a sequence ownership reference from the owning +// table column. +type AddSequenceOwner struct { + immediateMutationOp + TableID descpb.ID + ColumnID descpb.ColumnID + OwnedSequenceID descpb.ID +} + // RemoveOwnerBackReferenceInSequence removes a sequence ownership // back-reference from a sequence. type RemoveOwnerBackReferenceInSequence struct { @@ -805,3 +823,22 @@ type CreateSchemaDescriptor struct { immediateMutationOp SchemaID descpb.ID } + +type CreateSequenceDescriptor struct { + immediateMutationOp + SequenceID descpb.ID +} + +type SetSequenceOptions struct { + immediateMutationOp + SequenceID descpb.ID + Key string + Value string +} + +type InitSequence struct { + immediateMutationOp + SequenceID descpb.ID + RestartWith int64 + UseRestartWith bool +} diff --git a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go index 7bf67e24e3f4..cbd4819cbd2d 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go @@ -52,6 +52,8 @@ type ImmediateMutationVisitor interface { MakeWriteOnlyColumnDeleteOnly(context.Context, MakeWriteOnlyColumnDeleteOnly) error RemoveDroppedColumnType(context.Context, RemoveDroppedColumnType) error MakeDeleteOnlyColumnAbsent(context.Context, MakeDeleteOnlyColumnAbsent) error + AddOwnerBackReferenceInSequence(context.Context, AddOwnerBackReferenceInSequence) error + AddSequenceOwner(context.Context, AddSequenceOwner) error RemoveOwnerBackReferenceInSequence(context.Context, RemoveOwnerBackReferenceInSequence) error RemoveSequenceOwner(context.Context, RemoveSequenceOwner) error RemoveCheckConstraint(context.Context, RemoveCheckConstraint) error @@ -123,6 +125,9 @@ type ImmediateMutationVisitor interface { UpdateUserPrivileges(context.Context, UpdateUserPrivileges) error UpdateOwner(context.Context, UpdateOwner) error CreateSchemaDescriptor(context.Context, CreateSchemaDescriptor) error + CreateSequenceDescriptor(context.Context, CreateSequenceDescriptor) error + SetSequenceOptions(context.Context, SetSequenceOptions) error + InitSequence(context.Context, InitSequence) error } // Visit is part of the ImmediateMutationOp interface. @@ -275,6 +280,16 @@ func (op MakeDeleteOnlyColumnAbsent) Visit(ctx context.Context, v ImmediateMutat return v.MakeDeleteOnlyColumnAbsent(ctx, op) } +// Visit is part of the ImmediateMutationOp interface. +func (op AddOwnerBackReferenceInSequence) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.AddOwnerBackReferenceInSequence(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op AddSequenceOwner) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.AddSequenceOwner(ctx, op) +} + // Visit is part of the ImmediateMutationOp interface. func (op RemoveOwnerBackReferenceInSequence) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveOwnerBackReferenceInSequence(ctx, op) @@ -629,3 +644,18 @@ func (op UpdateOwner) Visit(ctx context.Context, v ImmediateMutationVisitor) err func (op CreateSchemaDescriptor) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.CreateSchemaDescriptor(ctx, op) } + +// Visit is part of the ImmediateMutationOp interface. +func (op CreateSequenceDescriptor) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.CreateSequenceDescriptor(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op SetSequenceOptions) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.SetSequenceOptions(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op InitSequence) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.InitSequence(ctx, op) +} diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index aff98b2dfa06..1842fa0d9b56 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -81,7 +81,7 @@ message ElementProto { IndexZoneConfig index_zone_config = 122 [(gogoproto.moretags) = "parent:\"Index\""]; TableData table_data = 131 [(gogoproto.customname) = "TableData", (gogoproto.moretags) = "parent:\"Table, View, Sequence\""]; TablePartitioning table_partitioning = 132 [(gogoproto.customname) = "TablePartitioning", (gogoproto.moretags) = "parent:\"Table\""]; - TableSchemaLocked table_schema_locked = 133 [(gogoproto.customname) = "TableSchemaLocked", (gogoproto.moretags) = "parent:\"Table\""]; + TableSchemaLocked table_schema_locked = 133 [(gogoproto.customname) = "TableSchemaLocked", (gogoproto.moretags) = "parent:\"Table\""]; // Multi-region elements. TableLocalityGlobal table_locality_global = 110 [(gogoproto.moretags) = "parent:\"Table\""]; @@ -98,6 +98,10 @@ message ElementProto { ColumnComment column_comment = 35 [(gogoproto.moretags) = "parent:\"Column\""]; ColumnNotNull column_not_null = 36 [(gogoproto.moretags) = "parent:\"Column\""]; + // Sequence elements. + SequenceOption sequence_option = 37 [(gogoproto.moretags) = "parent:\"Sequence\""]; + + // Index elements. IndexName index_name = 40 [(gogoproto.moretags) = "parent:\"PrimaryIndex, SecondaryIndex\""]; IndexPartitioning index_partitioning = 41 [(gogoproto.moretags) = "parent:\"PrimaryIndex, SecondaryIndex\""]; @@ -121,9 +125,10 @@ message ElementProto { DatabaseComment database_comment = 82 [(gogoproto.moretags) = "parent:\"Database\""]; DatabaseData database_data = 83 [(gogoproto.customname) = "DatabaseData", (gogoproto.moretags) = "parent:\"Database\""]; - // Schema elements. - SchemaParent schema_parent = 90 [(gogoproto.moretags) = "parent:\"Schema\""]; - SchemaComment schema_comment = 91 [(gogoproto.moretags) = "parent:\"Schema\""]; + // Schema elements. + SchemaParent schema_parent = 90 [(gogoproto.moretags) = "parent:\"Schema\""]; + SchemaComment schema_comment = 91 [(gogoproto.moretags) = "parent:\"Schema\""]; + // SchemaChild elements. SchemaChild schema_child = 100 [(gogoproto.moretags) = "parent:\"AliasType, EnumType, Table, View, Sequence\""]; @@ -303,10 +308,18 @@ message SchemaChild { message Sequence { uint32 sequence_id = 1 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; - + int64 restart_with = 2; + bool use_restart_with = 3; bool is_temporary = 10; } +message SequenceOption { + uint32 sequence_id = 1 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; + string key = 2; + string value = 3; +} + + message SequenceOwner { uint32 sequence_id = 1 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; uint32 table_id = 2 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; diff --git a/pkg/sql/schemachanger/scpb/elements_generated.go b/pkg/sql/schemachanger/scpb/elements_generated.go index 55a90fa3f2a7..f77c99bc3a4b 100644 --- a/pkg/sql/schemachanger/scpb/elements_generated.go +++ b/pkg/sql/schemachanger/scpb/elements_generated.go @@ -1828,6 +1828,43 @@ func (c *ElementCollection[E]) FilterSequence() *ElementCollection[*Sequence] { return (*ElementCollection[*Sequence])(ret) } +func (e SequenceOption) element() {} + +// Element implements ElementGetter. +func (e * ElementProto_SequenceOption) Element() Element { + return e.SequenceOption +} + +// ForEachSequenceOption iterates over elements of type SequenceOption. +// Deprecated +func ForEachSequenceOption( + c *ElementCollection[Element], fn func(current Status, target TargetStatus, e *SequenceOption), +) { + c.FilterSequenceOption().ForEach(fn) +} + +// FindSequenceOption finds the first element of type SequenceOption. +// Deprecated +func FindSequenceOption( + c *ElementCollection[Element], +) (current Status, target TargetStatus, element *SequenceOption) { + if tc := c.FilterSequenceOption(); !tc.IsEmpty() { + var e Element + current, target, e = tc.Get(0) + element = e.(*SequenceOption) + } + return current, target, element +} + +// SequenceOptionElements filters elements of type SequenceOption. +func (c *ElementCollection[E]) FilterSequenceOption() *ElementCollection[*SequenceOption] { + ret := c.genericFilter(func(_ Status, _ TargetStatus, e Element) bool { + _, ok := e.(*SequenceOption) + return ok + }) + return (*ElementCollection[*SequenceOption])(ret) +} + func (e SequenceOwner) element() {} // Element implements ElementGetter. @@ -2523,6 +2560,8 @@ func (e* ElementProto) SetElement(element Element) { e.ElementOneOf = &ElementProto_SecondaryIndexPartial{ SecondaryIndexPartial: t} case *Sequence: e.ElementOneOf = &ElementProto_Sequence{ Sequence: t} + case *SequenceOption: + e.ElementOneOf = &ElementProto_SequenceOption{ SequenceOption: t} case *SequenceOwner: e.ElementOneOf = &ElementProto_SequenceOwner{ SequenceOwner: t} case *Table: @@ -2610,6 +2649,7 @@ func GetElementOneOfProtos() []interface{} { ((*ElementProto_SecondaryIndex)(nil)), ((*ElementProto_SecondaryIndexPartial)(nil)), ((*ElementProto_Sequence)(nil)), + ((*ElementProto_SequenceOption)(nil)), ((*ElementProto_SequenceOwner)(nil)), ((*ElementProto_Table)(nil)), ((*ElementProto_TableComment)(nil)), @@ -2682,6 +2722,7 @@ func GetElementTypes() []interface{} { ((*SecondaryIndex)(nil)), ((*SecondaryIndexPartial)(nil)), ((*Sequence)(nil)), + ((*SequenceOption)(nil)), ((*SequenceOwner)(nil)), ((*Table)(nil)), ((*TableComment)(nil)), diff --git a/pkg/sql/schemachanger/scpb/migration.go b/pkg/sql/schemachanger/scpb/migration.go index c05da0aa1089..0419d3828e76 100644 --- a/pkg/sql/schemachanger/scpb/migration.go +++ b/pkg/sql/schemachanger/scpb/migration.go @@ -10,7 +10,12 @@ package scpb -import "github.com/cockroachdb/cockroach/pkg/clusterversion" +import ( + "math" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" +) // HasDeprecatedElements returns if the target contains any element marked // for deprecation. @@ -99,14 +104,33 @@ func MigrateCurrentState(version clusterversion.ClusterVersion, state *CurrentSt return updated } +func checkForTableDataElement(target Target) (createID catid.DescID, existingID catid.DescID) { + if target.TargetStatus != Status_PUBLIC { + return catid.InvalidDescID, catid.InvalidDescID + } + switch e := target.Element().(type) { + case *PrimaryIndex: + return e.TableID, catid.InvalidDescID + case *SecondaryIndex: + return e.TableID, catid.InvalidDescID + case *TableData: + return catid.InvalidDescID, e.TableID + } + return catid.InvalidDescID, catid.InvalidDescID +} + // MigrateDescriptorState migrates descriptor state and applies any changes // relevant for the current cluster version. -func MigrateDescriptorState(version clusterversion.ClusterVersion, state *DescriptorState) bool { +func MigrateDescriptorState( + version clusterversion.ClusterVersion, parentID catid.DescID, state *DescriptorState, +) bool { // Nothing to do for empty states. if state == nil { return false } targetsToRemove := make(map[int]struct{}) + newIndexes := make(map[catid.DescID]bool) + newTargets := 0 updated := false for idx, target := range state.Targets { if HasDeprecatedElements(version, target) { @@ -114,6 +138,14 @@ func MigrateDescriptorState(version clusterversion.ClusterVersion, state *Descri migrateTargetElement(state.Targets, idx) targetsToRemove[idx] = struct{}{} } + if newIndexID, descID := checkForTableDataElement(target); descID != catid.InvalidDescID || newIndexID != catid.InvalidDescID { + if _, ok := newIndexes[newIndexID]; newIndexID != catid.InvalidDescID && !ok { + newIndexes[newIndexID] = false + } + if descID != catid.InvalidDescID { + newIndexes[descID] = true + } + } current, targetStatus, update := migrateStatuses(state.CurrentStatuses[idx], target.TargetStatus) if update { state.CurrentStatuses[idx] = current @@ -121,6 +153,19 @@ func MigrateDescriptorState(version clusterversion.ClusterVersion, state *Descri updated = true } } + for id, skip := range newIndexes { + if skip { + continue + } + // Generate a TableData element + state.Targets = append(state.Targets, MakeTarget(ToPublic, &TableData{ + TableID: id, + DatabaseID: parentID, + }, &TargetMetadata{})) + state.CurrentStatuses = append(state.CurrentStatuses, Status_PUBLIC) + state.TargetRanks = append(state.TargetRanks, math.MaxUint32-uint32(newTargets)) + newTargets += 1 + } if !updated { return updated } @@ -138,5 +183,6 @@ func MigrateDescriptorState(version clusterversion.ClusterVersion, state *Descri state.TargetRanks = append(state.TargetRanks, existingTargetRanks[idx]) state.CurrentStatuses = append(state.CurrentStatuses, existingStatuses[idx]) } + return updated } diff --git a/pkg/sql/schemachanger/scpb/state.go b/pkg/sql/schemachanger/scpb/state.go index 57dbbad6019c..dd94c0e261c2 100644 --- a/pkg/sql/schemachanger/scpb/state.go +++ b/pkg/sql/schemachanger/scpb/state.go @@ -94,6 +94,11 @@ func (s *CurrentState) Rollback() { } for i := range s.Targets { t := &s.Targets[i] + // If the metadata is not populated this element + // only usd for tracking. + if !t.TargetIsLinkedToSchemaChange() { + continue + } switch t.TargetStatus { case Status_ABSENT: t.TargetStatus = Status_PUBLIC @@ -141,6 +146,16 @@ func (e *ElementProto) Element() Element { return e.GetElementOneOf().(ElementGetter).Element() } +// TargetIsLinkedToSchemaChange return if a Target is linked to a schema change. +func (t *Target) TargetIsLinkedToSchemaChange() bool { + return t.Metadata.TargetIsLinkedToSchemaChange() +} + +// TargetIsLinkedToSchemaChange return if a TargetMetadata is linked to a schema change. +func (t *TargetMetadata) TargetIsLinkedToSchemaChange() bool { + return t.Size() > 0 +} + // MakeTarget constructs a new Target. The passed elem must be one of the oneOf // members of Element. If not, this call will panic. func MakeTarget(status TargetStatus, elem Element, metadata *TargetMetadata) Target { diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index 0a489b00e8c0..1089f7094f20 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -304,8 +304,16 @@ SecondaryIndexPartial : Expression object Sequence Sequence : SequenceID +Sequence : RestartWith +Sequence : UseRestartWith Sequence : IsTemporary +object SequenceOption + +SequenceOption : SequenceID +SequenceOption : Key +SequenceOption : Value + object SequenceOwner SequenceOwner : SequenceID @@ -469,6 +477,7 @@ Schema <|-- SchemaParent Table <|-- SecondaryIndex View <|-- SecondaryIndex SecondaryIndex <|-- SecondaryIndexPartial +Sequence <|-- SequenceOption Column <|-- SequenceOwner Table <|-- TableComment View <|-- TableComment diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel index 62bf2ff4c946..5a9f5dca2f69 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel @@ -54,6 +54,7 @@ go_library( "opgen_secondary_index.go", "opgen_secondary_index_partial.go", "opgen_sequence.go", + "opgen_sequence_option.go", "opgen_sequence_owner.go", "opgen_table.go", "opgen_table_comment.go", diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go b/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go index 8eb8c215d7bb..fc7dd5188bc0 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go @@ -14,6 +14,7 @@ import ( "reflect" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" @@ -49,6 +50,7 @@ func statementForDropJob(e scpb.Element, md *opGenContext) scop.StatementForDrop // lookup. type opGenContext struct { scpb.TargetState + Current []scpb.Status ActiveVersion clusterversion.ClusterVersion elementToTarget map[scpb.Element]int InRollback bool @@ -61,6 +63,7 @@ func makeOpgenContext( ActiveVersion: activeVersion, InRollback: cs.InRollback, TargetState: cs.TargetState, + Current: cs.Current, elementToTarget: make(map[scpb.Element]int), } for i := range cs.Targets { @@ -164,3 +167,31 @@ func checkOpFunc(el scpb.Element, fn interface{}) (opType scop.Type, _ error) { } return opType, nil } + +// checkIfDescriptorIsWithoutData given the context this will determine if +// a descriptor is an added state, and has no data. This can allow us to +// skip certain operations like backfills / validation. +func checkIfDescriptorIsWithoutData(id descpb.ID, md *opGenContext) bool { + // Older versions did not emit the data element. + if !md.ActiveVersion.IsActive(clusterversion.V23_2) { + return false + } + doesDescriptorHaveData := false + for idx, t := range md.Targets { + // Validate this is the descriptor ID we are + // looking for. + if screl.GetDescID(t.Element()) != id { + continue + } + switch t.Element().(type) { + case *scpb.IndexData, *scpb.TableData: + // Check if this descriptor has any data within + // a public state. + if md.Current[idx] == scpb.Status_PUBLIC || + t.TargetStatus == scpb.Status_PUBLIC { + doesDescriptorHaveData = true + } + } + } + return !doesDescriptorHaveData +} diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go index 1aa6d0ad1e5b..96a702fb4da3 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_check_constraint.go @@ -61,7 +61,10 @@ func init() { }), ), to(scpb.Status_VALIDATED, - emit(func(this *scpb.CheckConstraint) *scop.ValidateConstraint { + emit(func(this *scpb.CheckConstraint, md *opGenContext) *scop.ValidateConstraint { + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.ValidateConstraint{ TableID: this.TableID, ConstraintID: this.ConstraintID, diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_not_null.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_not_null.go index 569ca07e5f55..5e5536efe636 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_not_null.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_column_not_null.go @@ -28,7 +28,10 @@ func init() { }), ), to(scpb.Status_VALIDATED, - emit(func(this *scpb.ColumnNotNull) *scop.ValidateColumnNotNull { + emit(func(this *scpb.ColumnNotNull, md *opGenContext) *scop.ValidateColumnNotNull { + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.ValidateColumnNotNull{ TableID: this.TableID, ColumnID: this.ColumnID, diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go index bd9ea483f661..3dff1bc6b957 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_foreign_key_constraint.go @@ -36,7 +36,10 @@ func init() { }), ), to(scpb.Status_VALIDATED, - emit(func(this *scpb.ForeignKeyConstraint) *scop.ValidateConstraint { + emit(func(this *scpb.ForeignKeyConstraint, md *opGenContext) *scop.ValidateConstraint { + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.ValidateConstraint{ TableID: this.TableID, ConstraintID: this.ConstraintID, diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_primary_index.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_primary_index.go index c6ac9c352c8f..3f32f7dc73de 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_primary_index.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_primary_index.go @@ -27,7 +27,11 @@ func init() { Index: *protoutil.Clone(&this.Index).(*scpb.Index), } }), - emit(func(this *scpb.PrimaryIndex) *scop.MaybeAddSplitForIndex { + emit(func(this *scpb.PrimaryIndex, md *opGenContext) *scop.MaybeAddSplitForIndex { + // Avoid adding splits for tables without any data (i.e. newly created ones). + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.MaybeAddSplitForIndex{ TableID: this.TableID, IndexID: this.IndexID, @@ -35,7 +39,12 @@ func init() { }), ), to(scpb.Status_BACKFILLED, - emit(func(this *scpb.PrimaryIndex) *scop.BackfillIndex { + emit(func(this *scpb.PrimaryIndex, md *opGenContext) *scop.BackfillIndex { + // No need to backfill indexes for added descriptors, these will + // be empty. + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.BackfillIndex{ TableID: this.TableID, SourceIndexID: this.SourceIndexID, @@ -60,7 +69,12 @@ func init() { }), ), to(scpb.Status_MERGED, - emit(func(this *scpb.PrimaryIndex) *scop.MergeIndex { + emit(func(this *scpb.PrimaryIndex, md *opGenContext) *scop.MergeIndex { + // No need to merge indexes for added descriptors, these will + // be empty. + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.MergeIndex{ TableID: this.TableID, TemporaryIndexID: this.TemporaryIndexID, @@ -83,7 +97,12 @@ func init() { }), ), to(scpb.Status_VALIDATED, - emit(func(this *scpb.PrimaryIndex) *scop.ValidateIndex { + emit(func(this *scpb.PrimaryIndex, md *opGenContext) *scop.ValidateIndex { + // No need to backfill validate for added descriptors, these will + // be empty. + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.ValidateIndex{ TableID: this.TableID, IndexID: this.IndexID, diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_secondary_index.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_secondary_index.go index 35750122bf4e..f31f97760d8c 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_secondary_index.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_secondary_index.go @@ -28,7 +28,11 @@ func init() { IsSecondaryIndex: true, } }), - emit(func(this *scpb.SecondaryIndex) *scop.MaybeAddSplitForIndex { + emit(func(this *scpb.SecondaryIndex, md *opGenContext) *scop.MaybeAddSplitForIndex { + // Avoid adding splits for tables without any data (i.e. newly created ones). + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.MaybeAddSplitForIndex{ TableID: this.TableID, IndexID: this.IndexID, @@ -56,7 +60,12 @@ func init() { }), ), to(scpb.Status_BACKFILLED, - emit(func(this *scpb.SecondaryIndex) *scop.BackfillIndex { + emit(func(this *scpb.SecondaryIndex, md *opGenContext) *scop.BackfillIndex { + // No need to backfill indexes for added descriptors, these will + // be empty. + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.BackfillIndex{ TableID: this.TableID, SourceIndexID: this.SourceIndexID, @@ -81,7 +90,12 @@ func init() { }), ), to(scpb.Status_MERGED, - emit(func(this *scpb.SecondaryIndex) *scop.MergeIndex { + emit(func(this *scpb.SecondaryIndex, md *opGenContext) *scop.MergeIndex { + // No need to merge indexes for added descriptors, these will + // be empty. + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.MergeIndex{ TableID: this.TableID, TemporaryIndexID: this.TemporaryIndexID, @@ -104,7 +118,12 @@ func init() { }), ), to(scpb.Status_VALIDATED, - emit(func(this *scpb.SecondaryIndex) *scop.ValidateIndex { + emit(func(this *scpb.SecondaryIndex, md *opGenContext) *scop.ValidateIndex { + // No need to backfill validate for added descriptors, these will + // be empty. + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.ValidateIndex{ TableID: this.TableID, IndexID: this.IndexID, diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go index b214ed033922..05d08d8eb3db 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go @@ -21,12 +21,23 @@ func init() { opRegistry.register((*scpb.Sequence)(nil), toPublic( scpb.Status_ABSENT, - to(scpb.Status_DROPPED, - emit(func(this *scpb.Sequence) *scop.NotImplemented { - return notImplemented(this) + equiv(scpb.Status_DROPPED), + to(scpb.Status_DESCRIPTOR_ADDED, + emit(func(this *scpb.Sequence) *scop.CreateSequenceDescriptor { + return &scop.CreateSequenceDescriptor{ + SequenceID: this.SequenceID, + } }), ), to(scpb.Status_PUBLIC, + revertible(false), + emit(func(this *scpb.Sequence) *scop.InitSequence { + return &scop.InitSequence{ + SequenceID: this.SequenceID, + RestartWith: this.RestartWith, + UseRestartWith: this.UseRestartWith, + } + }), emit(func(this *scpb.Sequence) *scop.MarkDescriptorAsPublic { return &scop.MarkDescriptorAsPublic{ DescriptorID: this.SequenceID, @@ -35,6 +46,7 @@ func init() { ), ), toAbsent(scpb.Status_PUBLIC, + equiv(scpb.Status_DESCRIPTOR_ADDED), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.Sequence) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence_option.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence_option.go new file mode 100644 index 000000000000..f3f5a34075bd --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence_option.go @@ -0,0 +1,41 @@ +// 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 opgen + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" +) + +func init() { + opRegistry.register((*scpb.SequenceOption)(nil), + toPublic( + scpb.Status_ABSENT, + to(scpb.Status_PUBLIC, + emit(func(this *scpb.SequenceOption) *scop.SetSequenceOptions { + return &scop.SetSequenceOptions{ + SequenceID: this.SequenceID, + Key: this.Key, + Value: this.Value, + } + }), + ), + ), + toAbsent( + scpb.Status_PUBLIC, + to(scpb.Status_ABSENT, + emit(func(this *scpb.SequenceOption) *scop.NotImplementedForPublicObjects { + return notImplementedForPublicObjects(this) + }), + ), + ), + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence_owner.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence_owner.go index 96236b3feeaf..ade618593d56 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence_owner.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence_owner.go @@ -20,8 +20,19 @@ func init() { toPublic( scpb.Status_ABSENT, to(scpb.Status_PUBLIC, - emit(func(this *scpb.SequenceOwner) *scop.NotImplemented { - return notImplemented(this) + emit(func(this *scpb.SequenceOwner) *scop.AddSequenceOwner { + return &scop.AddSequenceOwner{ + OwnedSequenceID: this.SequenceID, + TableID: this.TableID, + ColumnID: this.ColumnID, + } + }), + emit(func(this *scpb.SequenceOwner) *scop.AddOwnerBackReferenceInSequence { + return &scop.AddOwnerBackReferenceInSequence{ + SequenceID: this.SequenceID, + TableID: this.TableID, + ColumnID: this.ColumnID, + } }), ), ), diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_temporary_index.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_temporary_index.go index b02bd1f44cb8..c33f4baa705f 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_temporary_index.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_temporary_index.go @@ -28,7 +28,11 @@ func init() { IsSecondaryIndex: this.IsUsingSecondaryEncoding, } }), - emit(func(this *scpb.TemporaryIndex) *scop.MaybeAddSplitForIndex { + emit(func(this *scpb.TemporaryIndex, md *opGenContext) *scop.MaybeAddSplitForIndex { + // Avoid adding splits for tables without any data (i.e. newly created ones). + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.MaybeAddSplitForIndex{ TableID: this.TableID, IndexID: this.IndexID, diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go index 76c868d55f6a..320373fa7e93 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go @@ -46,7 +46,10 @@ func init() { }), ), to(scpb.Status_VALIDATED, - emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.ValidateConstraint { + emit(func(this *scpb.UniqueWithoutIndexConstraint, md *opGenContext) *scop.ValidateConstraint { + if checkIfDescriptorIsWithoutData(this.TableID, md) { + return nil + } return &scop.ValidateConstraint{ TableID: this.TableID, ConstraintID: this.ConstraintID, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go index 693da917385d..cf51b38ee42e 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go @@ -60,16 +60,22 @@ func init() { t opgen.Transition, prePrevStatuses []scpb.Status, ) rel.Clauses { + descriptorData := MkNodeVars("descriptor-data") + var descID rel.Var = "descID" clauses := rel.Clauses{ from.Type(el), to.Type(el), - from.El.AttrEqVar(screl.DescID, "_"), + from.El.AttrEqVar(screl.DescID, 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()), descriptorIsNotBeingDropped(from.El), + // Make sure to join a data element o confirm that data exists. + descriptorData.Type((*scpb.TableData)(nil)), + descriptorData.JoinTarget(), + descriptorData.DescIDEq(descID), } if len(prePrevStatuses) > 0 { clauses = append(clauses, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index dc559c913fef..7c92c5787eea 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -7,13 +7,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -23,13 +26,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -40,13 +46,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -56,13 +65,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -72,13 +84,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -88,13 +103,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -104,13 +122,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY' @@ -120,13 +141,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -136,13 +160,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY' @@ -152,13 +179,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -168,13 +198,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC' @@ -184,13 +217,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -200,13 +236,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -216,13 +255,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -233,13 +275,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -249,13 +294,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -265,13 +313,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -281,13 +332,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: Constraint should be hidden before name @@ -418,13 +472,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -434,13 +491,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -451,13 +511,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -467,13 +530,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -483,13 +549,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -499,13 +568,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -515,13 +587,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -531,13 +606,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -547,13 +625,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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) @@ -564,13 +645,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -580,13 +664,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -596,13 +683,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' @@ -612,13 +702,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -628,13 +721,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -644,13 +740,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -660,13 +759,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -676,13 +778,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -692,13 +797,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->VALIDATED' @@ -708,13 +816,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -724,13 +835,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -740,13 +854,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -757,13 +874,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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) @@ -774,13 +894,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -790,13 +913,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -806,13 +932,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -822,13 +951,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -838,13 +970,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -854,13 +989,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -870,13 +1008,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -886,13 +1027,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -902,13 +1046,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -918,13 +1065,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -934,13 +1084,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -950,13 +1103,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -966,13 +1122,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -982,13 +1141,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -998,13 +1160,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = TRANSIENT_VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1014,13 +1179,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1030,13 +1198,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1046,13 +1217,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_BACKFILLED_TRANSIENT_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1063,13 +1237,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1079,13 +1256,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1095,13 +1275,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1111,13 +1294,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED_TRANSIENT_MERGE_ONLY_TRANSIENT_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1128,13 +1314,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1144,13 +1333,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -1160,13 +1352,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1176,13 +1371,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -1192,13 +1390,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1209,13 +1410,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1225,13 +1429,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -1241,13 +1448,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -1257,13 +1467,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1273,13 +1486,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_VALIDATED_MERGE_ONLY_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1290,13 +1506,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -1306,13 +1525,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -1322,13 +1544,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1338,13 +1563,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -1354,13 +1582,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -1370,13 +1601,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -1386,13 +1620,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -1402,13 +1639,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -1418,13 +1658,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1435,13 +1678,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1451,13 +1697,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1467,13 +1716,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1483,13 +1735,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1499,13 +1754,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1515,13 +1773,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -1531,13 +1792,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -1547,13 +1811,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -1563,13 +1830,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1580,13 +1850,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -1596,13 +1869,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -1612,13 +1888,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -1628,13 +1907,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: all adding indexes reached BACKFILL_ONLY before any of their columns disappear @@ -2113,7 +2395,7 @@ deprules kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -2357,7 +2639,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -2413,7 +2695,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -2452,7 +2734,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -2880,7 +3162,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) @@ -3536,13 +3818,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -3552,13 +3837,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -3569,13 +3857,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -3585,13 +3876,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -3601,13 +3895,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -3617,13 +3914,16 @@ deprules query: - $prev[Type] = '*scpb.CheckConstraint' - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -3633,13 +3933,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY' @@ -3649,13 +3952,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -3665,13 +3971,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY' @@ -3681,13 +3990,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -3697,13 +4009,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC' @@ -3713,13 +4028,16 @@ deprules query: - $prev[Type] = '*scpb.Column' - $next[Type] = '*scpb.Column' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -3729,13 +4047,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -3745,13 +4066,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -3762,13 +4086,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -3778,13 +4105,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -3794,13 +4124,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ColumnNotNull transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -3810,13 +4143,16 @@ deprules query: - $prev[Type] = '*scpb.ColumnNotNull' - $next[Type] = '*scpb.ColumnNotNull' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: Constraint should be hidden before name @@ -3947,13 +4283,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -3963,13 +4302,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -3980,13 +4322,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -3996,13 +4341,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -4012,13 +4360,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -4028,13 +4379,16 @@ deprules query: - $prev[Type] = '*scpb.ForeignKeyConstraint' - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -4044,13 +4398,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4060,13 +4417,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -4076,13 +4436,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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) @@ -4093,13 +4456,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4109,13 +4475,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -4125,13 +4494,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' @@ -4141,13 +4513,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4157,13 +4532,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4173,13 +4551,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4189,13 +4570,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4205,13 +4589,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4221,13 +4608,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->VALIDATED' @@ -4237,13 +4627,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4253,13 +4646,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -4269,13 +4665,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -4286,13 +4685,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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) @@ -4303,13 +4705,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -4319,13 +4724,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -4335,13 +4743,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4351,13 +4762,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -4367,13 +4781,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -4383,13 +4800,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -4399,13 +4819,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -4415,13 +4838,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4431,13 +4857,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4447,13 +4876,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4463,13 +4895,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4479,13 +4914,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4495,13 +4933,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4511,13 +4952,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4527,13 +4971,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = TRANSIENT_VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4543,13 +4990,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4559,13 +5009,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4575,13 +5028,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_BACKFILLED_TRANSIENT_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -4592,13 +5048,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4608,13 +5067,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4624,13 +5086,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4640,13 +5105,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED_TRANSIENT_MERGE_ONLY_TRANSIENT_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -4657,13 +5125,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4673,13 +5144,16 @@ deprules query: - $prev[Type] = '*scpb.PrimaryIndex' - $next[Type] = '*scpb.PrimaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -4689,13 +5163,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4705,13 +5182,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -4721,13 +5201,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -4738,13 +5221,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4754,13 +5240,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -4770,13 +5259,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -4786,13 +5278,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4802,13 +5297,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_VALIDATED_MERGE_ONLY_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -4819,13 +5317,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -4835,13 +5336,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -4851,13 +5355,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4867,13 +5374,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -4883,13 +5393,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -4899,13 +5412,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -4915,13 +5431,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -4931,13 +5450,16 @@ deprules query: - $prev[Type] = '*scpb.SecondaryIndex' - $next[Type] = '*scpb.SecondaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -4947,13 +5469,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -4964,13 +5489,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4980,13 +5508,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -4996,13 +5527,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -5012,13 +5546,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -5028,13 +5565,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -5044,13 +5584,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - 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' @@ -5060,13 +5603,16 @@ deprules query: - $prev[Type] = '*scpb.TemporaryIndex' - $next[Type] = '*scpb.TemporaryIndex' - - $prev[DescID] = $_ + - $prev[DescID] = $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-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -5076,13 +5622,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -5092,13 +5641,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -5109,13 +5661,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -5125,13 +5680,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -5141,13 +5699,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -5157,13 +5718,16 @@ deprules query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ + - $prev[DescID] = $descID - $prev[Self] = $next - $prev-Target[Self] = $next-Target - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - descriptorIsNotBeingDropped-23.2($prev) + - $descriptor-data[Type] = '*scpb.TableData' + - joinTarget($descriptor-data, $descriptor-data-Target) + - $descriptor-data[DescID] = $descID - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: all adding indexes reached BACKFILL_ONLY before any of their columns disappear @@ -5642,7 +6206,7 @@ deprules kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -5886,7 +6450,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -5942,7 +6506,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -5981,7 +6545,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -6409,7 +6973,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionParamDefaultExpression', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) diff --git a/pkg/sql/schemachanger/scplan/internal/scstage/build.go b/pkg/sql/schemachanger/scplan/internal/scstage/build.go index 60fe8b2be5f5..2c88d191fb81 100644 --- a/pkg/sql/schemachanger/scplan/internal/scstage/build.go +++ b/pkg/sql/schemachanger/scplan/internal/scstage/build.go @@ -63,9 +63,10 @@ func BuildStages( } return m }(), - startingPhase: phase, - descIDs: screl.AllTargetDescIDs(init.TargetState), - withSanityChecks: withSanityChecks, + startingPhase: phase, + descIDs: screl.AllTargetDescIDs(init.TargetState), + withSanityChecks: withSanityChecks, + anyRemainingOpsCanFail: checkIfAnyRemainingOpsCanFail(init.TargetState, g), } // Build stages for all remaining phases. stages := buildStages(bc) @@ -98,17 +99,57 @@ func BuildStages( // buildContext contains the global constants for building the stages. // It's read-only everywhere after being initialized in BuildStages. type buildContext struct { - ctx context.Context - rollback bool - g *scgraph.Graph - scJobID func() jobspb.JobID - targetState scpb.TargetState - initial []scpb.Status - current []scpb.Status - targetToIdx map[*scpb.Target]int - startingPhase scop.Phase - descIDs catalog.DescriptorIDSet - withSanityChecks bool + ctx context.Context + rollback bool + g *scgraph.Graph + scJobID func() jobspb.JobID + targetState scpb.TargetState + initial []scpb.Status + current []scpb.Status + targetToIdx map[*scpb.Target]int + startingPhase scop.Phase + descIDs catalog.DescriptorIDSet + anyRemainingOpsCanFail map[*screl.Node]bool + withSanityChecks bool +} + +// checkIfAnyRemainingOpsCanFail returns a map which indicates if +// a given screl.Node can encounter a failure in the future due to +// either validation or backfill. +func checkIfAnyRemainingOpsCanFail( + targetState scpb.TargetState, g *scgraph.Graph, +) map[*screl.Node]bool { + // Determine which op edges can potentially fail later on due to backfill + // failures. + anyRemainingOpsCanFail := make(map[*screl.Node]bool) + for i := range targetState.Targets { + t := &targetState.Targets[i] + currentStatus := t.TargetStatus + anyRemainingCanFail := false + for { + if n, ok := g.GetNode(t, currentStatus); ok { + if oe, ok := g.GetOpEdgeTo(n); ok { + anyRemainingOpsCanFail[n] = anyRemainingCanFail + // If this can potentially lead to failures, validate + // we have ops which are non-mutation types. + if oe.CanFail() { + for _, op := range oe.Op() { + anyRemainingCanFail = anyRemainingCanFail || + op.Type() != scop.MutationType + } + } + currentStatus = oe.From().CurrentStatus + } else { + // Terminal status + anyRemainingOpsCanFail[n] = anyRemainingCanFail + break + } + } else { + break + } + } + } + return anyRemainingOpsCanFail } // buildStages builds all stages according to the starting parameters @@ -320,12 +361,10 @@ func (bc buildContext) makeStageBuilderForType(bs buildState, opType scop.Type) // Determine whether there are any backfill or validation operations // remaining which should prevent the scheduling of any non-revertible // operations. This information will be used when building the current - // set of targets below. + // set of targets below. We will do this over the set of generated operations, + // since some of these may be no-oped for newly created tables. for _, n := range nodes { - if oe, ok := bc.g.GetOpEdgeFrom(n); ok && oe.CanFail() { - sb.anyRemainingOpsCanFail = true - break - } + sb.anyRemainingOpsCanFail = sb.anyRemainingOpsCanFail || bc.anyRemainingOpsCanFail[n] } for i, n := range nodes { diff --git a/pkg/sql/schemachanger/scplan/testdata/create_sequence b/pkg/sql/schemachanger/scplan/testdata/create_sequence new file mode 100644 index 000000000000..ffcf0972c6a8 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/testdata/create_sequence @@ -0,0 +1,398 @@ +ops +CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; +---- +StatementPhase stage 1 of 1 with 23 MutationType ops + transitions: + [[Sequence:{DescID: 104}, PUBLIC], ABSENT] -> PUBLIC + [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, PUBLIC], ABSENT] -> PUBLIC + [[SequenceOption:{DescID: 104, Name: START}, PUBLIC], ABSENT] -> PUBLIC + [[Column:{DescID: 104, ColumnID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnNotNull:{DescID: 104, ColumnID: 1, IndexID: 0}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnName:{DescID: 104, Name: value, ColumnID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 0}, PUBLIC], ABSENT] -> PUBLIC + [[IndexName:{DescID: 104, Name: primary, IndexID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[Owner:{DescID: 104}, PUBLIC], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: admin}, PUBLIC], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: root}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.CreateSequenceDescriptor + SequenceID: 104 + *scop.SetNameInDescriptor + DescriptorID: 104 + Name: sq1 + *scop.AddDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: sq1 + SchemaID: 101 + *scop.SetSequenceOptions + Key: START + SequenceID: 104 + Value: "32" + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 1 + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 1 + ElementCreationMetadata: + in231OrLater: true + TableID: 104 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.SetColumnName + ColumnID: 1 + Name: value + TableID: 104 + *scop.MakeAbsentIndexBackfilling + Index: + IndexID: 1 + IsUnique: true + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 1 + TableID: 104 + *scop.UpdateOwner + Owner: + DescriptorID: 104 + Owner: root + *scop.UpdateUserPrivileges + Privileges: + DescriptorID: 104 + Privileges: 2 + UserName: admin + WithGrantOption: 2 + *scop.UpdateUserPrivileges + Privileges: + DescriptorID: 104 + Privileges: 2 + UserName: root + WithGrantOption: 2 + *scop.MakeDeleteOnlyColumnWriteOnly + ColumnID: 1 + TableID: 104 + *scop.MakeAbsentColumnNotNullWriteOnly + ColumnID: 1 + TableID: 104 + *scop.MakeBackfillingIndexDeleteOnly + IndexID: 1 + TableID: 104 + *scop.MakeValidatedColumnNotNullPublic + ColumnID: 1 + TableID: 104 + *scop.MakeBackfilledIndexMerging + IndexID: 1 + TableID: 104 + *scop.MakeWriteOnlyColumnPublic + ColumnID: 1 + TableID: 104 + *scop.MakeMergedIndexWriteOnly + IndexID: 1 + TableID: 104 + *scop.SetIndexName + IndexID: 1 + Name: primary + TableID: 104 + *scop.MakeValidatedPrimaryIndexPublic + IndexID: 1 + TableID: 104 + *scop.InitSequence + SequenceID: 104 + *scop.MarkDescriptorAsPublic + DescriptorID: 104 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Sequence:{DescID: 104}, PUBLIC], PUBLIC] -> ABSENT + [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, PUBLIC], PUBLIC] -> ABSENT + [[SequenceOption:{DescID: 104, Name: START}, PUBLIC], PUBLIC] -> ABSENT + [[Column:{DescID: 104, ColumnID: 1}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnNotNull:{DescID: 104, ColumnID: 1, IndexID: 0}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnName:{DescID: 104, Name: value, ColumnID: 1}, PUBLIC], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 0}, PUBLIC], PUBLIC] -> ABSENT + [[IndexName:{DescID: 104, Name: primary, IndexID: 1}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1}, PUBLIC], PUBLIC] -> ABSENT + [[Owner:{DescID: 104}, PUBLIC], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: admin}, PUBLIC], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: root}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 24 MutationType ops + transitions: + [[Sequence:{DescID: 104}, PUBLIC], ABSENT] -> PUBLIC + [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, PUBLIC], ABSENT] -> PUBLIC + [[SequenceOption:{DescID: 104, Name: START}, PUBLIC], ABSENT] -> PUBLIC + [[Column:{DescID: 104, ColumnID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnNotNull:{DescID: 104, ColumnID: 1, IndexID: 0}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnName:{DescID: 104, Name: value, ColumnID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 0}, PUBLIC], ABSENT] -> PUBLIC + [[IndexName:{DescID: 104, Name: primary, IndexID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[Owner:{DescID: 104}, PUBLIC], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: admin}, PUBLIC], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: root}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.CreateSequenceDescriptor + SequenceID: 104 + *scop.SetNameInDescriptor + DescriptorID: 104 + Name: sq1 + *scop.AddDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: sq1 + SchemaID: 101 + *scop.SetSequenceOptions + Key: START + SequenceID: 104 + Value: "32" + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 1 + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 1 + ElementCreationMetadata: + in231OrLater: true + TableID: 104 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.SetColumnName + ColumnID: 1 + Name: value + TableID: 104 + *scop.MakeAbsentIndexBackfilling + Index: + IndexID: 1 + IsUnique: true + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 1 + TableID: 104 + *scop.UpdateOwner + Owner: + DescriptorID: 104 + Owner: root + *scop.UpdateUserPrivileges + Privileges: + DescriptorID: 104 + Privileges: 2 + UserName: admin + WithGrantOption: 2 + *scop.UpdateUserPrivileges + Privileges: + DescriptorID: 104 + Privileges: 2 + UserName: root + WithGrantOption: 2 + *scop.MakeDeleteOnlyColumnWriteOnly + ColumnID: 1 + TableID: 104 + *scop.MakeAbsentColumnNotNullWriteOnly + ColumnID: 1 + TableID: 104 + *scop.MakeBackfillingIndexDeleteOnly + IndexID: 1 + TableID: 104 + *scop.MakeValidatedColumnNotNullPublic + ColumnID: 1 + TableID: 104 + *scop.MakeBackfilledIndexMerging + IndexID: 1 + TableID: 104 + *scop.MakeWriteOnlyColumnPublic + ColumnID: 1 + TableID: 104 + *scop.RefreshStats + TableID: 104 + *scop.MakeMergedIndexWriteOnly + IndexID: 1 + TableID: 104 + *scop.SetIndexName + IndexID: 1 + Name: primary + TableID: 104 + *scop.MakeValidatedPrimaryIndexPublic + IndexID: 1 + TableID: 104 + *scop.InitSequence + SequenceID: 104 + *scop.MarkDescriptorAsPublic + DescriptorID: 104 + + +deps +CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; +---- +- from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] + to: [ColumnName:{DescID: 105, Name: value, ColumnID: 1}, PUBLIC] + kind: SameStagePrecedence + rules: [column existence precedes column dependents; column name and type set right after column existence] +- from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] + to: [ColumnNotNull:{DescID: 105, ColumnID: 1, IndexID: 0}, PUBLIC] + kind: Precedence + rule: column existence precedes column dependents +- from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] + to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC] + kind: SameStagePrecedence + rules: [column existence precedes column dependents; column name and type set right after column existence] +- from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] + to: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] + kind: Precedence + rule: column existence precedes column dependents +- from: [Column:{DescID: 105, ColumnID: 1}, DELETE_ONLY] + to: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, BACKFILL_ONLY] + kind: Precedence + rule: column existence precedes index existence +- from: [Column:{DescID: 105, ColumnID: 1}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [Column:{DescID: 105, ColumnID: 1}, WRITE_ONLY] + to: [ColumnNotNull:{DescID: 105, ColumnID: 1, IndexID: 0}, WRITE_ONLY] + kind: SameStagePrecedence + rule: column writable right before column constraint is enforced. +- from: [ColumnName:{DescID: 105, Name: value, ColumnID: 1}, PUBLIC] + to: [Column:{DescID: 105, ColumnID: 1}, PUBLIC] + kind: Precedence + rule: column dependents exist before column becomes public +- from: [ColumnName:{DescID: 105, Name: value, ColumnID: 1}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [ColumnNotNull:{DescID: 105, ColumnID: 1, IndexID: 0}, PUBLIC] + to: [Column:{DescID: 105, ColumnID: 1}, PUBLIC] + kind: Precedence + rule: column dependents exist before column becomes public +- from: [ColumnNotNull:{DescID: 105, ColumnID: 1, IndexID: 0}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC] + to: [Column:{DescID: 105, ColumnID: 1}, PUBLIC] + kind: Precedence + rule: column dependents exist before column becomes public +- from: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] + to: [Column:{DescID: 105, ColumnID: 1}, PUBLIC] + kind: Precedence + rule: column dependents exist before column becomes public +- from: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] + to: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, BACKFILLED] + kind: Precedence + rule: index-column added to index before index is backfilled +- from: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] + to: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, PUBLIC] + kind: Precedence + rule: index dependents exist before index becomes public +- from: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [IndexName:{DescID: 105, Name: primary, IndexID: 1}, PUBLIC] + to: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, PUBLIC] + kind: SameStagePrecedence + rules: [index dependents exist before index becomes public; primary index named right before index becomes public] +- from: [IndexName:{DescID: 105, Name: primary, IndexID: 1}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [Namespace:{DescID: 105, Name: sq1, ReferencedDescID: 100}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [Owner:{DescID: 105}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, BACKFILL_ONLY] + to: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] + kind: Precedence + rule: index existence precedes index dependents +- from: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, BACKFILL_ONLY] + to: [IndexName:{DescID: 105, Name: primary, IndexID: 1}, PUBLIC] + kind: Precedence + rule: index existence precedes index dependents +- from: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [Column:{DescID: 105, ColumnID: 1}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [ColumnName:{DescID: 105, Name: value, ColumnID: 1}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [ColumnNotNull:{DescID: 105, ColumnID: 1, IndexID: 0}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [IndexName:{DescID: 105, Name: primary, IndexID: 1}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [Namespace:{DescID: 105, Name: sq1, ReferencedDescID: 100}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [Owner:{DescID: 105}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [PrimaryIndex:{DescID: 105, IndexID: 1, ConstraintID: 0}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [SequenceOption:{DescID: 105, Name: START}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [UserPrivileges:{DescID: 105, Name: admin}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [Sequence:{DescID: 105}, DESCRIPTOR_ADDED] + to: [UserPrivileges:{DescID: 105, Name: root}, PUBLIC] + kind: Precedence + rule: descriptor existence precedes dependents +- from: [SequenceOption:{DescID: 105, Name: START}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [UserPrivileges:{DescID: 105, Name: admin}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public +- from: [UserPrivileges:{DescID: 105, Name: root}, PUBLIC] + to: [Sequence:{DescID: 105}, PUBLIC] + kind: Precedence + rule: dependents exist before descriptor becomes public diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index fb78475e2bfa..47a667d25967 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -239,6 +239,10 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityAttr(ColumnID, "ColumnID"), rel.EntityAttr(ReferencedTypeIDs, "ClosedTypeIDs"), ), + rel.EntityMapping(t((*scpb.SequenceOption)(nil)), + rel.EntityAttr(DescID, "SequenceID"), + rel.EntityAttr(Name, "Key"), + ), rel.EntityMapping(t((*scpb.SequenceOwner)(nil)), rel.EntityAttr(DescID, "TableID"), rel.EntityAttr(ColumnID, "ColumnID"), diff --git a/pkg/sql/schemachanger/screl/scalars.go b/pkg/sql/schemachanger/screl/scalars.go index e2452e30fd35..05016dd50598 100644 --- a/pkg/sql/schemachanger/screl/scalars.go +++ b/pkg/sql/schemachanger/screl/scalars.go @@ -128,6 +128,8 @@ func MinElementVersion(el scpb.Element) clusterversion.Key { *scpb.UniqueWithoutIndexConstraintUnvalidated, *scpb.ForeignKeyConstraintUnvalidated, *scpb.IndexZoneConfig, *scpb.TableSchemaLocked: return clusterversion.V23_1 + case *scpb.SequenceOption: + return clusterversion.V23_2 default: panic(errors.AssertionFailedf("unknown element %T", el)) } diff --git a/pkg/sql/schemachanger/scrun/scrun.go b/pkg/sql/schemachanger/scrun/scrun.go index 9d678b8d4cff..2811726eb2db 100644 --- a/pkg/sql/schemachanger/scrun/scrun.go +++ b/pkg/sql/schemachanger/scrun/scrun.go @@ -274,7 +274,8 @@ func logSchemaChangeEvents( ) error { var ids catalog.DescriptorIDSet for _, t := range state.TargetState.Targets { - if t.Metadata.SourceElementID > 1 { + if t.Metadata.SourceElementID > 1 || + !t.TargetIsLinkedToSchemaChange() { // Ignore empty metadata // Ignore targets which are the product of CASCADEs. continue } @@ -335,7 +336,7 @@ func makeState( // but TXN_DROPPED is special and should be cleaned up in memory before // executing on a newer node. cs = protoutil.Clone(cs).(*scpb.DescriptorState) - scpb.MigrateDescriptorState(version, cs) + scpb.MigrateDescriptorState(version, desc.GetParentID(), cs) if cs == nil { return errors.New("missing schema changer state") } diff --git a/pkg/sql/schemachanger/sctest_generated_test.go b/pkg/sql/schemachanger/sctest_generated_test.go index 94726fa56add..9a8b54f128c7 100644 --- a/pkg/sql/schemachanger/sctest_generated_test.go +++ b/pkg/sql/schemachanger/sctest_generated_test.go @@ -720,6 +720,31 @@ func TestRollback_create_schema_drop_schema_separate_statements(t *testing.T) { defer log.Scope(t).Close(t) sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/create_schema_drop_schema_separate_statements", sctest.SingleNodeCluster) } +func TestEndToEndSideEffects_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.EndToEndSideEffects(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeCluster) +} +func TestExecuteWithDMLInjection_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.ExecuteWithDMLInjection(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeCluster) +} +func TestGenerateSchemaChangeCorpus_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.GenerateSchemaChangeCorpus(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeCluster) +} +func TestPause_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Pause(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeCluster) +} +func TestRollback_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeCluster) +} func TestEndToEndSideEffects_drop_column_basic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/schemachanger/sctest_mixed_generated_test.go b/pkg/sql/schemachanger/sctest_mixed_generated_test.go index 3382bfec60db..2730ba2dda11 100644 --- a/pkg/sql/schemachanger/sctest_mixed_generated_test.go +++ b/pkg/sql/schemachanger/sctest_mixed_generated_test.go @@ -160,6 +160,11 @@ func TestValidateMixedVersionElements_create_schema_drop_schema_separate_stateme defer log.Scope(t).Close(t) sctest.ValidateMixedVersionElements(t, "pkg/sql/schemachanger/testdata/end_to_end/create_schema_drop_schema_separate_statements", sctest.SingleNodeMixedCluster) } +func TestValidateMixedVersionElements_create_sequence(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.ValidateMixedVersionElements(t, "pkg/sql/schemachanger/testdata/end_to_end/create_sequence", sctest.SingleNodeMixedCluster) +} func TestValidateMixedVersionElements_drop_column_basic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq/add_column_default_seq.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq/add_column_default_seq.side_effects index f224510b7ae2..868db93a1f65 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq/add_column_default_seq.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq/add_column_default_seq.side_effects @@ -134,8 +134,8 @@ upsert descriptor #107 + columnIds: + - 2 + id: 106 - families: - - columnIds: + formatVersion: 3 + id: 107 ... start: "1" unexposedParentSchemaId: 105 @@ -279,18 +279,27 @@ upsert descriptor #107 + declarativeSchemaChangerState: + authorization: + userName: root + + currentStatuses: + jobId: "1" + nameMapping: + id: 107 + name: sq1 + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT + + NULL DEFAULT nextval(‹'db.public.sq1'›) + + statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + + statementTag: ALTER TABLE + revertible: true + + targetRanks: + + targets: + dependedOnBy: + - byId: true + columnIds: + - 2 + id: 106 - families: - - columnIds: + formatVersion: 3 + id: 107 ... start: "1" unexposedParentSchemaId: 105 @@ -609,11 +618,11 @@ upsert descriptor #106 + version: "7" upsert descriptor #107 ... - id: 107 - name: sq1 + statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + statementTag: ALTER TABLE - revertible: true - dependedOnBy: - - byId: true + targetRanks: + targets: ... start: "1" unexposedParentSchemaId: 105 @@ -722,10 +731,19 @@ upsert descriptor #107 - declarativeSchemaChangerState: - authorization: - userName: root + - currentStatuses: - jobId: "1" - nameMapping: - id: 107 - name: sq1 + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ‹l› INT8 NOT + - NULL DEFAULT nextval(‹'db.public.sq1'›) + - statement: ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1') + - statementTag: ALTER TABLE + - targetRanks: + - targets: dependedOnBy: - byId: true ... diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt/alter_table_add_check_with_seq_and_udt.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt/alter_table_add_check_with_seq_and_udt.side_effects index e6e174010a19..fba53eaf8bd3 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt/alter_table_add_check_with_seq_and_udt.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt/alter_table_add_check_with_seq_and_udt.side_effects @@ -34,8 +34,8 @@ upsert descriptor #104 + dependedOnBy: + - byId: true + id: 107 - families: - - columnIds: + formatVersion: 3 + id: 104 ... start: "1" unexposedParentSchemaId: 101 @@ -112,16 +112,25 @@ upsert descriptor #104 + declarativeSchemaChangerState: + authorization: + userName: root + + currentStatuses: + jobId: "1" + nameMapping: + id: 104 + name: s + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CHECK ((‹i› > + + nextval(‹'s'›)) OR (‹j›::‹typ› = ‹'a'›)) + + statement: ALTER TABLE t ADD CHECK ((i > nextval('s')) OR (j::typ = 'a')) + + statementTag: ALTER TABLE + revertible: true + + targetRanks: + + targets: + dependedOnBy: + - byId: true + id: 107 - families: - - columnIds: + formatVersion: 3 + id: 104 ... start: "1" unexposedParentSchemaId: 101 @@ -263,11 +272,20 @@ upsert descriptor #104 - declarativeSchemaChangerState: - authorization: - userName: root + - currentStatuses: - jobId: "1" - nameMapping: - id: 104 - name: s + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CHECK ((‹i› > + - nextval(‹'s'›)) OR (‹j›::‹typ› = ‹'a'›)) + - statement: ALTER TABLE t ADD CHECK ((i > nextval('s')) OR (j::typ = 'a')) + - statementTag: ALTER TABLE - revertible: true + - targetRanks: + - targets: dependedOnBy: - byId: true ... diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key/alter_table_add_foreign_key.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key/alter_table_add_foreign_key.side_effects index f8beff76ac1a..3466ab2389ae 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key/alter_table_add_foreign_key.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key/alter_table_add_foreign_key.side_effects @@ -160,6 +160,7 @@ upsert descriptor #105 + declarativeSchemaChangerState: + authorization: + userName: root + + currentStatuses: + jobId: "1" + nameMapping: + columns: @@ -172,7 +173,15 @@ upsert descriptor #105 + indexes: + "1": t2_pkey + name: t2 + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CONSTRAINT ‹t1_i_fkey› + + FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.‹public›.‹t2› (‹i›) + + statement: ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2 (i) + + statementTag: ALTER TABLE + revertible: true + + targetRanks: + + targets: families: - columnIds: ... @@ -293,6 +302,7 @@ upsert descriptor #105 - declarativeSchemaChangerState: - authorization: - userName: root + - currentStatuses: - jobId: "1" - nameMapping: - columns: @@ -305,7 +315,15 @@ upsert descriptor #105 - indexes: - "1": t2_pkey - name: t2 + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CONSTRAINT ‹t1_i_fkey› + - FOREIGN KEY (‹i›) REFERENCES ‹defaultdb›.‹public›.‹t2› (‹i›) + - statement: ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2 (i) + - statementTag: ALTER TABLE - revertible: true + - targetRanks: + - targets: families: - columnIds: ... diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_function/create_function.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/create_function/create_function.side_effects index ef8a75e8ade2..39bc16e41b62 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_function/create_function.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_function/create_function.side_effects @@ -142,8 +142,8 @@ upsert descriptor #105 + dependedOnBy: + - byId: true + id: 110 - families: - - columnIds: + formatVersion: 3 + id: 105 ... start: "1" unexposedParentSchemaId: 101 @@ -284,8 +284,8 @@ upsert descriptor #105 + dependedOnBy: + - byId: true + id: 110 - families: - - columnIds: + formatVersion: 3 + id: 105 ... start: "1" unexposedParentSchemaId: 101 diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_function_in_txn/create_function_in_txn.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/create_function_in_txn/create_function_in_txn.side_effects index 26c119bc2ab5..224439d01336 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_function_in_txn/create_function_in_txn.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_function_in_txn/create_function_in_txn.side_effects @@ -245,6 +245,12 @@ upsert descriptor #104 + name: t + relevantStatements: + - statement: + + redactedStatement: "CREATE FUNCTION ‹defaultdb›.‹public›.‹t›()\n\tRETURNS + + INT8\n\tLANGUAGE SQL\n\tAS $$SELECT ‹1›;$$" + + statement: "CREATE FUNCTION t()\n\tRETURNS INT8\n\tLANGUAGE SQL\n\tAS $$ SELECT + + 1 $$" + + statementTag: CREATE FUNCTION + + - statement: + redactedStatement: CREATE UNIQUE INDEX ‹idx› ON ‹defaultdb›.‹public›.‹t› (‹b›) + statement: CREATE UNIQUE INDEX idx ON t (b) + statementTag: CREATE INDEX @@ -478,6 +484,12 @@ upsert descriptor #104 - name: t - relevantStatements: - - statement: + - redactedStatement: "CREATE FUNCTION ‹defaultdb›.‹public›.‹t›()\n\tRETURNS + - INT8\n\tLANGUAGE SQL\n\tAS $$SELECT ‹1›;$$" + - statement: "CREATE FUNCTION t()\n\tRETURNS INT8\n\tLANGUAGE SQL\n\tAS $$ SELECT + - 1 $$" + - statementTag: CREATE FUNCTION + - - statement: - redactedStatement: CREATE UNIQUE INDEX ‹idx› ON ‹defaultdb›.‹public›.‹t› (‹b›) - statement: CREATE UNIQUE INDEX idx ON t (b) - statementTag: CREATE INDEX diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.definition b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.definition new file mode 100644 index 000000000000..41ac30ab52c9 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.definition @@ -0,0 +1,3 @@ +test +CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; +---- diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.explain b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.explain new file mode 100644 index 000000000000..821bde301f43 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.explain @@ -0,0 +1,104 @@ +/* setup */ + +/* test */ +EXPLAIN (DDL) CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; +---- +Schema change plan for CREATE SEQUENCE ‹defaultdb›.‹public›.‹sq1› MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 13 elements transitioning toward PUBLIC + │ │ ├── ABSENT → PUBLIC Sequence:{DescID: 104 (sq1+)} + │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 104 (sq1+), Name: "sq1", ReferencedDescID: 100 (defaultdb)} + │ │ ├── ABSENT → PUBLIC SequenceOption:{DescID: 104 (sq1+), Name: "START"} + │ │ ├── ABSENT → PUBLIC Column:{DescID: 104 (sq1+), ColumnID: 1 (value+)} + │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 104 (sq1+), ColumnFamilyID: 0, ColumnID: 1 (value+)} + │ │ ├── ABSENT → PUBLIC ColumnNotNull:{DescID: 104 (sq1+), ColumnID: 1 (value+), IndexID: 0} + │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104 (sq1+), Name: "value", ColumnID: 1 (value+)} + │ │ ├── ABSENT → PUBLIC PrimaryIndex:{DescID: 104 (sq1+), IndexID: 1 (primary+)} + │ │ ├── ABSENT → PUBLIC IndexName:{DescID: 104 (sq1+), Name: "primary", IndexID: 1 (primary+)} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (sq1+), ColumnID: 1 (value+), IndexID: 1 (primary+)} + │ │ ├── ABSENT → PUBLIC Owner:{DescID: 104 (sq1+)} + │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 104 (sq1+), Name: "admin"} + │ │ └── ABSENT → PUBLIC UserPrivileges:{DescID: 104 (sq1+), Name: "root"} + │ └── 23 Mutation operations + │ ├── CreateSequenceDescriptor {"SequenceID":104} + │ ├── SetNameInDescriptor {"DescriptorID":104,"Name":"sq1"} + │ ├── AddDescriptorName {"Namespace":{"DatabaseID":100,"DescriptorID":104,"Name":"sq1","SchemaID":101}} + │ ├── SetSequenceOptions {"Key":"START","SequenceID":104,"Value":"32"} + │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":1,"TableID":104}} + │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":1,"TableID":104}} + │ ├── SetColumnName {"ColumnID":1,"Name":"value","TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"IndexID":1,"IsUnique":true,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":1,"TableID":104} + │ ├── UpdateOwner {"Owner":{"DescriptorID":104,"Owner":"root"}} + │ ├── UpdateUserPrivileges {"Privileges":{"DescriptorID":104,"Privileges":2,"UserName":"admin","WithGrantOption":2}} + │ ├── UpdateUserPrivileges {"Privileges":{"DescriptorID":104,"Privileges":2,"UserName":"root","WithGrantOption":2}} + │ ├── MakeDeleteOnlyColumnWriteOnly {"ColumnID":1,"TableID":104} + │ ├── MakeAbsentColumnNotNullWriteOnly {"ColumnID":1,"TableID":104} + │ ├── MakeBackfillingIndexDeleteOnly {"IndexID":1,"TableID":104} + │ ├── MakeValidatedColumnNotNullPublic {"ColumnID":1,"TableID":104} + │ ├── MakeBackfilledIndexMerging {"IndexID":1,"TableID":104} + │ ├── MakeWriteOnlyColumnPublic {"ColumnID":1,"TableID":104} + │ ├── MakeMergedIndexWriteOnly {"IndexID":1,"TableID":104} + │ ├── SetIndexName {"IndexID":1,"Name":"primary","TableID":104} + │ ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} + │ ├── InitSequence {"SequenceID":104} + │ └── MarkDescriptorAsPublic {"DescriptorID":104} + └── PreCommitPhase + ├── Stage 1 of 2 in PreCommitPhase + │ ├── 13 elements transitioning toward PUBLIC + │ │ ├── PUBLIC → ABSENT Sequence:{DescID: 104 (sq1+)} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 104 (sq1+), Name: "sq1", ReferencedDescID: 100 (defaultdb)} + │ │ ├── PUBLIC → ABSENT SequenceOption:{DescID: 104 (sq1+), Name: "START"} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 104 (sq1+), ColumnID: 1 (value+)} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 104 (sq1+), ColumnFamilyID: 0, ColumnID: 1 (value+)} + │ │ ├── PUBLIC → ABSENT ColumnNotNull:{DescID: 104 (sq1+), ColumnID: 1 (value+), IndexID: 0} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104 (sq1+), Name: "value", ColumnID: 1 (value+)} + │ │ ├── PUBLIC → ABSENT PrimaryIndex:{DescID: 104 (sq1+), IndexID: 1 (primary+)} + │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104 (sq1+), Name: "primary", IndexID: 1 (primary+)} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104 (sq1+), ColumnID: 1 (value+), IndexID: 1 (primary+)} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 104 (sq1+)} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104 (sq1+), Name: "admin"} + │ │ └── PUBLIC → ABSENT UserPrivileges:{DescID: 104 (sq1+), Name: "root"} + │ └── 1 Mutation operation + │ └── UndoAllInTxnImmediateMutationOpSideEffects + └── Stage 2 of 2 in PreCommitPhase + ├── 13 elements transitioning toward PUBLIC + │ ├── ABSENT → PUBLIC Sequence:{DescID: 104 (sq1+)} + │ ├── ABSENT → PUBLIC Namespace:{DescID: 104 (sq1+), Name: "sq1", ReferencedDescID: 100 (defaultdb)} + │ ├── ABSENT → PUBLIC SequenceOption:{DescID: 104 (sq1+), Name: "START"} + │ ├── ABSENT → PUBLIC Column:{DescID: 104 (sq1+), ColumnID: 1 (value+)} + │ ├── ABSENT → PUBLIC ColumnType:{DescID: 104 (sq1+), ColumnFamilyID: 0, ColumnID: 1 (value+)} + │ ├── ABSENT → PUBLIC ColumnNotNull:{DescID: 104 (sq1+), ColumnID: 1 (value+), IndexID: 0} + │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104 (sq1+), Name: "value", ColumnID: 1 (value+)} + │ ├── ABSENT → PUBLIC PrimaryIndex:{DescID: 104 (sq1+), IndexID: 1 (primary+)} + │ ├── ABSENT → PUBLIC IndexName:{DescID: 104 (sq1+), Name: "primary", IndexID: 1 (primary+)} + │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104 (sq1+), ColumnID: 1 (value+), IndexID: 1 (primary+)} + │ ├── ABSENT → PUBLIC Owner:{DescID: 104 (sq1+)} + │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 104 (sq1+), Name: "admin"} + │ └── ABSENT → PUBLIC UserPrivileges:{DescID: 104 (sq1+), Name: "root"} + └── 24 Mutation operations + ├── CreateSequenceDescriptor {"SequenceID":104} + ├── SetNameInDescriptor {"DescriptorID":104,"Name":"sq1"} + ├── AddDescriptorName {"Namespace":{"DatabaseID":100,"DescriptorID":104,"Name":"sq1","SchemaID":101}} + ├── SetSequenceOptions {"Key":"START","SequenceID":104,"Value":"32"} + ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":1,"TableID":104}} + ├── SetAddedColumnType {"ColumnType":{"ColumnID":1,"TableID":104}} + ├── SetColumnName {"ColumnID":1,"Name":"value","TableID":104} + ├── MakeAbsentIndexBackfilling {"Index":{"IndexID":1,"IsUnique":true,"TableID":104}} + ├── AddColumnToIndex {"ColumnID":1,"IndexID":1,"TableID":104} + ├── UpdateOwner {"Owner":{"DescriptorID":104,"Owner":"root"}} + ├── UpdateUserPrivileges {"Privileges":{"DescriptorID":104,"Privileges":2,"UserName":"admin","WithGrantOption":2}} + ├── UpdateUserPrivileges {"Privileges":{"DescriptorID":104,"Privileges":2,"UserName":"root","WithGrantOption":2}} + ├── MakeDeleteOnlyColumnWriteOnly {"ColumnID":1,"TableID":104} + ├── MakeAbsentColumnNotNullWriteOnly {"ColumnID":1,"TableID":104} + ├── MakeBackfillingIndexDeleteOnly {"IndexID":1,"TableID":104} + ├── MakeValidatedColumnNotNullPublic {"ColumnID":1,"TableID":104} + ├── MakeBackfilledIndexMerging {"IndexID":1,"TableID":104} + ├── MakeWriteOnlyColumnPublic {"ColumnID":1,"TableID":104} + ├── RefreshStats {"TableID":104} + ├── MakeMergedIndexWriteOnly {"IndexID":1,"TableID":104} + ├── SetIndexName {"IndexID":1,"Name":"primary","TableID":104} + ├── MakeValidatedPrimaryIndexPublic {"IndexID":1,"TableID":104} + ├── InitSequence {"SequenceID":104} + └── MarkDescriptorAsPublic {"DescriptorID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.explain_shape new file mode 100644 index 000000000000..776aab1a4df2 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.explain_shape @@ -0,0 +1,7 @@ +/* setup */ + +/* test */ +EXPLAIN (DDL, SHAPE) CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; +---- +Schema change plan for CREATE SEQUENCE ‹defaultdb›.‹public›.‹sq1› MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; + └── execute 1 system table mutations transaction diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.side_effects new file mode 100644 index 000000000000..9386ecc1f339 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_sequence/create_sequence.side_effects @@ -0,0 +1,171 @@ +/* setup */ +---- + + +/* test */ +CREATE SEQUENCE sq1 MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 32; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: CREATE SEQUENCE +write *eventpb.CreateSequence to event log: + sequenceName: defaultdb.public.sq1 + sql: + descriptorId: 104 + statement: CREATE SEQUENCE ‹defaultdb›.‹public›.‹sq1› MINVALUE 1 MAXVALUE 9223372036854775807 + INCREMENT 1 START 32 + tag: CREATE SEQUENCE + user: root +## StatementPhase stage 1 of 1 with 23 MutationType ops +initializing sequence 104 with starting value of 31 +add object namespace entry {100 101 sq1} -> 104 +upsert descriptor #104 + - + +table: + + checks: [] + + columns: + + - id: 1 + + name: value + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + - defaultExpr: unique_rowid() + + hidden: true + + id: 2 + + name: rowid + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + createAsOfTime: {} + + formatVersion: 3 + + id: 104 + + modificationTime: {} + + mutations: [] + + name: sq1 + + nextColumnId: 3 + + nextConstraintId: 1 + + nextIndexId: 2 + + parentId: 100 + + primaryIndex: + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 1 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - value + + name: primary + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + privileges: + + ownerProto: root + + users: + + - privileges: "2" + + userProto: admin + + withGrantOption: "2" + + - privileges: "2" + + userProto: root + + withGrantOption: "2" + + version: 2 + + replacementOf: + + time: {} + + sequenceOpts: + + cacheSize: "1" + + increment: "1" + + maxValue: "9223372036854775807" + + minValue: "1" + + sequenceOwner: {} + + start: "32" + + unexposedParentSchemaId: 101 + + version: "1" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 2 with 1 MutationType op +undo all catalog changes within txn #1 +persist all catalog changes to storage +## PreCommitPhase stage 2 of 2 with 24 MutationType ops +initializing sequence 104 with starting value of 31 +add object namespace entry {100 101 sq1} -> 104 +upsert descriptor #104 + - + +table: + + checks: [] + + columns: + + - id: 1 + + name: value + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + - defaultExpr: unique_rowid() + + hidden: true + + id: 2 + + name: rowid + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + createAsOfTime: {} + + formatVersion: 3 + + id: 104 + + modificationTime: {} + + mutations: [] + + name: sq1 + + nextColumnId: 3 + + nextConstraintId: 1 + + nextIndexId: 2 + + parentId: 100 + + primaryIndex: + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 1 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - value + + name: primary + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + privileges: + + ownerProto: root + + users: + + - privileges: "2" + + userProto: admin + + withGrantOption: "2" + + - privileges: "2" + + userProto: root + + withGrantOption: "2" + + version: 2 + + replacementOf: + + time: {} + + sequenceOpts: + + cacheSize: "1" + + increment: "1" + + maxValue: "9223372036854775807" + + minValue: "1" + + sequenceOwner: {} + + start: "32" + + unexposedParentSchemaId: 101 + + version: "1" +persist all catalog changes to storage +adding table for stats refresh: 104 +# end PreCommitPhase +commit transaction #1 diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_function/drop_function.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/drop_function/drop_function.side_effects index ca111ae57613..1f10044eeeff 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_function/drop_function.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_function/drop_function.side_effects @@ -92,8 +92,8 @@ upsert descriptor #105 - dependedOnBy: - - byId: true - id: 109 - families: - - columnIds: + formatVersion: 3 + id: 105 ... start: "1" unexposedParentSchemaId: 101 @@ -210,8 +210,8 @@ upsert descriptor #105 + userName: root + jobId: "1" + nameMapping: {} - families: - - columnIds: + formatVersion: 3 + id: 105 ... start: "1" unexposedParentSchemaId: 101 @@ -335,8 +335,8 @@ upsert descriptor #105 - userName: root - jobId: "1" - nameMapping: {} - families: - - columnIds: + formatVersion: 3 + id: 105 ... start: "1" unexposedParentSchemaId: 101 diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index 85b3ddb1f260..7bb1e744bc1a 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -428,14 +428,10 @@ func assignSequenceOwner( if err != nil { return err } - if tableDesc.ParentID != sequenceParentID && - !allowCrossDatabaseSeqOwner.Get(&p.execCfg.Settings.SV) { - return errors.WithHintf( - pgerror.Newf(pgcode.FeatureNotSupported, - "OWNED BY cannot refer to other databases; (see the '%s' cluster setting)", - allowCrossDatabaseSeqOwnerSetting), - crossDBReferenceDeprecationHint(), - ) + if tableDesc.ParentID != sequenceParentID { + if err := p.CanCreateCrossDBSequenceOwnerRef(); err != nil { + return err + } } // We only want to trigger schema changes if the owner is not what we // want it to be. diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush_test.go b/pkg/sql/sqlstats/persistedsqlstats/flush_test.go index ac699ba97029..ae2ffe6cdb26 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush_test.go @@ -459,7 +459,7 @@ func TestSQLStatsGatewayNodeSetting(t *testing.T) { } func TestSQLStatsPersistedLimitReached(t *testing.T) { - skip.WithIssue(t, 97488) + skip.UnderStressWithIssue(t, 97488) defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx index 5603601be2b9..c6dc850a60ee 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx @@ -13,9 +13,7 @@ import { Link, RouteComponentProps } from "react-router-dom"; import { Tooltip } from "antd"; import "antd/lib/tooltip/style"; import classNames from "classnames/bind"; -import { Breadcrumbs } from "src/breadcrumbs"; import { Dropdown, DropdownOption } from "src/dropdown"; -import { CaretRight } from "src/icon/caretRight"; import { DatabaseIcon } from "src/icon/databaseIcon"; import { StackIcon } from "src/icon/stackIcon"; import { PageConfig, PageConfigItem } from "src/pageConfig"; @@ -27,25 +25,13 @@ import { SortSetting, } from "src/sortedtable"; import * as format from "src/util/format"; -import { - DATE_FORMAT, - EncodeDatabaseTableUri, - EncodeDatabaseUri, - EncodeUriName, -} from "src/util/format"; -import { - getMatchParamByName, - mvccGarbage, - schemaNameAttr, - syncHistory, - unique, -} from "../util"; +import { DATE_FORMAT, EncodeDatabaseTableUri } from "src/util/format"; +import { mvccGarbage, syncHistory, unique } from "../util"; import styles from "./databaseDetailsPage.module.scss"; import sortableTableStyles from "src/sortedtable/sortedtable.module.scss"; import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses"; import { Moment } from "moment-timezone"; -import { Caution } from "@cockroachlabs/icons"; import { Anchor } from "../anchor"; import LoadingError from "../sqlActivity/errorComponent"; import { Loading } from "../loading"; @@ -198,12 +184,10 @@ function filterBySearchQuery( return matchString.includes(search); } - const res = search + return search .toLowerCase() .split(" ") .every(val => matchString.includes(val)); - - return res; } export class DatabaseDetailsPage extends React.Component< @@ -540,159 +524,161 @@ export class DatabaseDetailsPage extends React.Component< }; private columnsForTablesViewMode(): ColumnDescriptor[] { - return [ - { - title: ( - - Tables - - ), - cell: table => , - sort: table => table.name, - className: cx("database-table__col-name"), - name: "name", - }, - { - title: ( - - Replication Size - - ), - cell: table => - this.checkInfoAvailable( - table.lastError, - format.Bytes(table.details.replicationSizeInBytes), + return ( + [ + { + title: ( + + Tables + ), - sort: table => table.details.replicationSizeInBytes, - className: cx("database-table__col-size"), - name: "replicationSize", - }, - { - title: ( - - Ranges - - ), - cell: table => - this.checkInfoAvailable(table.lastError, table.details.rangeCount), - sort: table => table.details.rangeCount, - className: cx("database-table__col-range-count"), - name: "rangeCount", - }, - { - title: ( - - Columns - - ), - cell: table => - this.checkInfoAvailable(table.lastError, table.details.columnCount), - sort: table => table.details.columnCount, - className: cx("database-table__col-column-count"), - name: "columnCount", - }, - { - title: ( - - Indexes - - ), - cell: table => { - return table.details.hasIndexRecommendations && - this.props.showIndexRecommendations - ? this.checkInfoAvailable( - table.lastError, - , - ) - : this.checkInfoAvailable( - table.lastError, - table.details.indexCount, - ); + cell: table => , + sort: table => table.name, + className: cx("database-table__col-name"), + name: "name", }, - sort: table => table.details.indexCount, - className: cx("database-table__col-index-count"), - name: "indexCount", - }, - { - title: ( - - Regions - - ), - cell: table => - this.checkInfoAvailable( - table.lastError, - table.details.nodesByRegionString || "None", + { + title: ( + + Replication Size + ), - sort: table => table.details.nodesByRegionString, - className: cx("database-table__col--regions"), - name: "regions", - showByDefault: this.props.showNodeRegionsColumn, - hideIfTenant: true, - }, - { - title: ( - - {"% of total uncompressed logical data that has not been modified (updated or deleted). " + - "A low percentage can cause statements to scan more data ("} - - MVCC values - - {") than required, which can reduce performance."} - - } - > - % of Live Data - - ), - cell: table => - this.checkInfoAvailable( - table.lastError, - , + cell: table => + this.checkInfoAvailable( + table.lastError, + format.Bytes(table.details.replicationSizeInBytes), + ), + sort: table => table.details.replicationSizeInBytes, + className: cx("database-table__col-size"), + name: "replicationSize", + }, + { + title: ( + + Ranges + ), - sort: table => table.details.livePercentage, - className: cx("database-table__col-column-count"), - name: "livePercentage", - }, - { - title: ( - - Table Stats Last Updated - - ), - cell: table => ( - - ), - sort: table => table.details.statsLastUpdated, - className: cx("database-table__col--table-stats"), - name: "tableStatsUpdated", - }, - ]; + cell: table => + this.checkInfoAvailable(table.lastError, table.details.rangeCount), + sort: table => table.details.rangeCount, + className: cx("database-table__col-range-count"), + name: "rangeCount", + }, + { + title: ( + + Columns + + ), + cell: table => + this.checkInfoAvailable(table.lastError, table.details.columnCount), + sort: table => table.details.columnCount, + className: cx("database-table__col-column-count"), + name: "columnCount", + }, + { + title: ( + + Indexes + + ), + cell: table => { + return table.details.hasIndexRecommendations && + this.props.showIndexRecommendations + ? this.checkInfoAvailable( + table.lastError, + , + ) + : this.checkInfoAvailable( + table.lastError, + table.details.indexCount, + ); + }, + sort: table => table.details.indexCount, + className: cx("database-table__col-index-count"), + name: "indexCount", + }, + { + title: ( + + Regions + + ), + cell: table => + this.checkInfoAvailable( + table.lastError, + table.details.nodesByRegionString || "None", + ), + sort: table => table.details.nodesByRegionString, + className: cx("database-table__col--regions"), + name: "regions", + showByDefault: this.props.showNodeRegionsColumn, + hideIfTenant: true, + }, + { + title: ( + + {"% of total uncompressed logical data that has not been modified (updated or deleted). " + + "A low percentage can cause statements to scan more data ("} + + MVCC values + + {") than required, which can reduce performance."} + + } + > + % of Live Data + + ), + cell: table => + this.checkInfoAvailable( + table.lastError, + , + ), + sort: table => table.details.livePercentage, + className: cx("database-table__col-column-count"), + name: "livePercentage", + }, + { + title: ( + + Table Stats Last Updated + + ), + cell: table => ( + + ), + sort: table => table.details.statsLastUpdated, + className: cx("database-table__col--table-stats"), + name: "tableStatsUpdated", + }, + ] as ColumnDescriptor[] + ).filter(c => c.showByDefault !== false); } private columnsForGrantsViewMode(): ColumnDescriptor[] { diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx index 5ac5a16f7edd..7213d579133e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx @@ -276,16 +276,17 @@ export class DatabasesPage extends React.Component< } componentDidUpdate( - prevProp: Readonly, + prevProps: Readonly, prevState: Readonly, ): void { - if (this.shouldRefreshDatabaseInformation(prevState, prevProp)) { + if (this.shouldRefreshDatabaseInformation(prevState, prevProps)) { this.updateQueryParams(); this.refresh(); } if ( - prevProp.indexRecommendationsEnabled !== - this.props.indexRecommendationsEnabled + prevProps.indexRecommendationsEnabled !== + this.props.indexRecommendationsEnabled || + prevProps.showNodeRegionsColumn !== this.props.showNodeRegionsColumn ) { this.setState({ columns: this.columns() }); } diff --git a/pkg/upgrade/upgrades/schemachanger_elements.go b/pkg/upgrade/upgrades/schemachanger_elements.go index bbc0a3924c2b..d2f2d6c052e5 100644 --- a/pkg/upgrade/upgrades/schemachanger_elements.go +++ b/pkg/upgrade/upgrades/schemachanger_elements.go @@ -102,7 +102,7 @@ WHERE if err != nil { return err } - scpb.MigrateDescriptorState(clusterVersion, mutDesc.GetDeclarativeSchemaChangerState()) + scpb.MigrateDescriptorState(clusterVersion, mutDesc.GetParentID(), mutDesc.GetDeclarativeSchemaChangerState()) err = descs.WriteDesc(ctx, false, mutDesc, kvTxn) if err != nil { return err