From ee68771412fb4a6a2e3fd34c77071d9a32cbe423 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 25 Nov 2021 12:25:31 -0500 Subject: [PATCH 01/10] spanconfig/sqlwatcher: speed up tests Using a more aggressive closed timestamp target duration brings the runtime for TestSQLWatcherReactsToUpdate down from 45s+ to single digits. To speed it up, we also just re-use the same sqlwatcher instead of creating a new one for every test case. Other tests in the package benefit from a similar treatment. Using the default target duration in fact masked a buggy test; in a future commit we end up rewrite that test so it's skipped for now. Release note: None --- .../spanconfigsqlwatcher/BUILD.bazel | 1 + .../spanconfigsqlwatcher/sqlwatcher_test.go | 160 ++++++++++-------- 2 files changed, 88 insertions(+), 73 deletions(-) diff --git a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel index bb47eba6a7c7..1b3b8bf0a9eb 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel @@ -54,6 +54,7 @@ go_test( "//pkg/sql/catalog/descpb", "//pkg/testutils", "//pkg/testutils/serverutils", + "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/hlc", diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go index 39e8dd9cd7f7..0dd028122cb0 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go @@ -15,6 +15,7 @@ import ( "sync" "sync/atomic" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -24,6 +25,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsqlwatcher" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -40,7 +43,6 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { defer leaktest.AfterTest(t)() testCases := []struct { - setup string stmt string expectedIDs descpb.IDs }{ @@ -49,17 +51,22 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { expectedIDs: descpb.IDs{52}, }, { - setup: "CREATE TABLE t2()", - stmt: "ALTER TABLE t2 CONFIGURE ZONE USING num_replicas = 3", + stmt: "CREATE TABLE t2(); ALTER TABLE t2 CONFIGURE ZONE USING num_replicas = 3", expectedIDs: descpb.IDs{53}, }, { - setup: "CREATE DATABASE d; CREATE TABLE d.t1(); CREATE TABLE d.t2()", + stmt: "CREATE DATABASE d; CREATE TABLE d.t1(); CREATE TABLE d.t2()", + expectedIDs: descpb.IDs{54, 55, 56}, + }, + { stmt: "ALTER DATABASE d CONFIGURE ZONE USING num_replicas=5", expectedIDs: descpb.IDs{54}, }, { - setup: "CREATE TABLE t3(); CREATE TABLE t4()", + stmt: "CREATE TABLE t3(); CREATE TABLE t4()", + expectedIDs: descpb.IDs{57, 58}, + }, + { stmt: "ALTER TABLE t3 CONFIGURE ZONE USING num_replicas=5; CREATE TABLE t5(); DROP TABLE t4;", expectedIDs: descpb.IDs{57, 58, 59}, }, @@ -86,8 +93,7 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { }, // Test that events on types/schemas are also captured. { - setup: "CREATE DATABASE db", - stmt: "CREATE SCHEMA db.sc", + stmt: "CREATE DATABASE db; CREATE SCHEMA db.sc", // One ID each for the parent database and the schema. expectedIDs: descpb.IDs{60, 61}, }, @@ -112,75 +118,81 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { ts := tc.Server(0 /* idx */) - sqlDB := tc.ServerConn(0 /* idx */) - for _, tc := range testCases { - sqlWatcher := spanconfigsqlwatcher.NewFactory( - keys.SystemSQLCodec, - ts.ClusterSettings(), - ts.RangeFeedFactory().(*rangefeed.Factory), - 1<<20, /* 1 MB, bufferMemLimit */ - ts.Stopper(), - nil, /* knobs */ - ).New() - - _, err := sqlDB.Exec(tc.setup) - require.NoError(t, err) - - // Save the startTS here before the test case is executed to ensure the - // watcher can pick up the change when we start it in a separate goroutine. + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0 /* idx */)) + tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) + + sqlWatcher := spanconfigsqlwatcher.NewFactory( + keys.SystemSQLCodec, + ts.ClusterSettings(), + ts.RangeFeedFactory().(*rangefeed.Factory), + 1<<20, /* 1 MB, bufferMemLimit */ + ts.Stopper(), + nil, /* knobs */ + ).New() + + var mu struct { + syncutil.Mutex + receivedIDs map[descpb.ID]struct{} + lastCheckpoint hlc.Timestamp + } + mu.receivedIDs = make(map[descpb.ID]struct{}) + + var wg sync.WaitGroup + watcherCtx, watcherCancel := context.WithCancel(context.Background()) + wg.Add(1) + go func() { + defer wg.Done() + startTS := ts.Clock().Now() + mu.Lock() + mu.lastCheckpoint = startTS + mu.Unlock() - var mu struct { - syncutil.Mutex - receivedIDs map[descpb.ID]struct{} - } - mu.receivedIDs = make(map[descpb.ID]struct{}) - - watcherCtx, watcherCancel := context.WithCancel(context.Background()) - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - prevCheckpointTS := startTS - _ = sqlWatcher.WatchForSQLUpdates(watcherCtx, - startTS, - func(ctx context.Context, updates []spanconfig.DescriptorUpdate, checkpointTS hlc.Timestamp) error { - require.True(t, prevCheckpointTS.Less(checkpointTS)) - mu.Lock() - defer mu.Unlock() - for _, update := range updates { - mu.receivedIDs[update.ID] = struct{}{} - } - prevCheckpointTS = checkpointTS - return nil - }) - }() - - _, err = sqlDB.Exec(tc.stmt) - require.NoError(t, err) + _ = sqlWatcher.WatchForSQLUpdates(watcherCtx, startTS, + func(ctx context.Context, updates []spanconfig.DescriptorUpdate, checkpointTS hlc.Timestamp) error { + mu.Lock() + defer mu.Unlock() + + require.True(t, mu.lastCheckpoint.Less(checkpointTS)) + mu.lastCheckpoint = checkpointTS + + for _, update := range updates { + mu.receivedIDs[update.ID] = struct{}{} + } + return nil + }) + }() + + for _, tc := range testCases { + tdb.Exec(t, tc.stmt) + afterStmtTS := ts.Clock().Now() testutils.SucceedsSoon(t, func() error { mu.Lock() defer mu.Unlock() - if len(mu.receivedIDs) == len(tc.expectedIDs) { - return nil + + if mu.lastCheckpoint.Less(afterStmtTS) { + return errors.New("checkpoint precedes statement timestamp") } - return errors.Newf("expected to receive %d IDs, but found %d", len(tc.expectedIDs), len(mu.receivedIDs)) + return nil }) // Rangefeed events aren't guaranteed to be in any particular order for // different keys. mu.Lock() + require.Equal(t, len(tc.expectedIDs), len(mu.receivedIDs)) for _, id := range tc.expectedIDs { _, seen := mu.receivedIDs[id] require.True(t, seen) + delete(mu.receivedIDs, id) } mu.Unlock() - - // Stop the watcher and wait for the goroutine to complete. - watcherCancel() - wg.Wait() } + + // Stop the watcher and wait for the goroutine to complete. + watcherCancel() + wg.Wait() } // TestSQLWatcherFactory tests that the SQLWatcherFactory can create multiple @@ -188,6 +200,7 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { // WatchForSQLUpdates. func TestSQLWatcherFactory(t *testing.T) { defer leaktest.AfterTest(t)() + skip.IgnoreLint(t, "buggy test; handler's invoked but test doesn't expect it to be") tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ @@ -202,7 +215,9 @@ func TestSQLWatcherFactory(t *testing.T) { ctx := context.Background() defer tc.Stopper().Stop(ctx) ts := tc.Server(0 /* idx */) - sqlDB := tc.ServerConn(0 /* idx */) + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0 /* idx */)) + tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) sqlWatcherFactory := spanconfigsqlwatcher.NewFactory( keys.SystemSQLCodec, @@ -214,8 +229,7 @@ func TestSQLWatcherFactory(t *testing.T) { ) startTS := ts.Clock().Now() - _, err := sqlDB.Exec("CREATE TABLE t()") - require.NoError(t, err) + tdb.Exec(t, "CREATE TABLE t()") sqlWatcher := sqlWatcherFactory.New() @@ -234,7 +248,7 @@ func TestSQLWatcherFactory(t *testing.T) { watcherCancel() wg.Wait() - err = sqlWatcher.WatchForSQLUpdates(watcherCtx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { + err := sqlWatcher.WatchForSQLUpdates(watcherCtx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { t.Fatal("handler should never run") return nil }) @@ -277,7 +291,9 @@ func TestSQLWatcherOnEventError(t *testing.T) { ctx := context.Background() defer tc.Stopper().Stop(ctx) ts := tc.Server(0 /* idx */) - sqlDB := tc.ServerConn(0 /* idx */) + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0 /* idx */)) + tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) sqlWatcherFactory := spanconfigsqlwatcher.NewFactory( keys.SystemSQLCodec, @@ -293,11 +309,10 @@ func TestSQLWatcherOnEventError(t *testing.T) { ) startTS := ts.Clock().Now() - _, err := sqlDB.Exec("CREATE TABLE t()") - require.NoError(t, err) + tdb.Exec(t, "CREATE TABLE t()") sqlWatcher := sqlWatcherFactory.New() - err = sqlWatcher.WatchForSQLUpdates(ctx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { + err := sqlWatcher.WatchForSQLUpdates(ctx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { t.Fatal("handler should never run") return nil }) @@ -323,7 +338,9 @@ func TestSQLWatcherHandlerError(t *testing.T) { ctx := context.Background() defer tc.Stopper().Stop(ctx) ts := tc.Server(0 /* idx */) - sqlDB := tc.ServerConn(0 /* idx */) + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0 /* idx */)) + tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) sqlWatcher := spanconfigsqlwatcher.NewFactory( keys.SystemSQLCodec, @@ -334,8 +351,7 @@ func TestSQLWatcherHandlerError(t *testing.T) { nil, /* knobs */ ).New() - _, err := sqlDB.Exec("CREATE TABLE t()") - require.NoError(t, err) + tdb.Exec(t, "CREATE TABLE t()") stopTraffic := make(chan struct{}) startTS := ts.Clock().Now() @@ -352,11 +368,9 @@ func TestSQLWatcherHandlerError(t *testing.T) { case <-stopTraffic: return default: + time.Sleep(100 * time.Millisecond) } - _, err = sqlDB.Exec("ALTER TABLE t CONFIGURE ZONE USING num_replicas=5") - if err != nil { - t.Errorf("unexpected error: %v", err) - } + tdb.Exec(t, "ALTER TABLE t CONFIGURE ZONE USING num_replicas=5") } }() From 46c7597875cc01414aa0384c2e5f1e299961211a Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 25 Nov 2021 10:54:26 -0500 Subject: [PATCH 02/10] spanconfig/sqlwatcher: hide the factory pattern The only mutable state across concurrent WatchForSQLUpdate calls was the internal buffer, which does not need to hang off the surrounding struct. This obviates the need for the factory pattern we were using -- callers can set up multiple SQLWatchers concurrently as is (see TestSQLWatcherMultiple). This commit simply hides the factory under the package boundary; in a future commit we'll shed it altogether. This has the benefit of reducing the number of symbols in pkg/spanconfig and making it symmetric with the other spanconfig dependencies typically found together (KVAccessor, SQLTranslator). It's also every-so-slightly less verbose to use in the upcoming spanconfig.Reconciler (#71994). Release note: None --- .../changefeedccl/helpers_tenant_shim_test.go | 54 ++++---- pkg/server/server_sql.go | 4 +- pkg/server/testserver.go | 6 +- pkg/spanconfig/spanconfig.go | 45 ++----- pkg/spanconfig/spanconfigmanager/manager.go | 22 +-- .../spanconfigmanager/manager_test.go | 6 +- .../spanconfigsqlwatcher/BUILD.bazel | 1 - .../spanconfigsqlwatcher/sqlwatcher.go | 118 +++++++--------- .../spanconfigsqlwatcher/sqlwatcher_test.go | 127 ++++++++++++------ pkg/testutils/serverutils/test_server_shim.go | 6 +- 10 files changed, 200 insertions(+), 189 deletions(-) diff --git a/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go b/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go index 3ce3711ef227..328375feaa5e 100644 --- a/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go +++ b/pkg/ccl/changefeedccl/helpers_tenant_shim_test.go @@ -51,33 +51,33 @@ func (t *testServerShim) ServingSQLAddr() string { return t.SQLAddr() } -func (t *testServerShim) Stopper() *stop.Stopper { panic(unsupportedShimMethod) } -func (t *testServerShim) Start(context.Context) error { panic(unsupportedShimMethod) } -func (t *testServerShim) Node() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) NodeID() roachpb.NodeID { panic(unsupportedShimMethod) } -func (t *testServerShim) ClusterID() uuid.UUID { panic(unsupportedShimMethod) } -func (t *testServerShim) ServingRPCAddr() string { panic(unsupportedShimMethod) } -func (t *testServerShim) RPCAddr() string { panic(unsupportedShimMethod) } -func (t *testServerShim) DB() *kv.DB { panic(unsupportedShimMethod) } -func (t *testServerShim) RPCContext() *rpc.Context { panic(unsupportedShimMethod) } -func (t *testServerShim) LeaseManager() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) InternalExecutor() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) ExecutorConfig() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) TracerI() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) GossipI() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) RangeFeedFactory() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) Clock() *hlc.Clock { panic(unsupportedShimMethod) } -func (t *testServerShim) DistSenderI() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) MigrationServer() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) SpanConfigAccessor() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) SpanConfigSQLTranslator() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) SpanConfigSQLWatcherFactory() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) SQLServer() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) SQLLivenessProvider() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) StartupMigrationsManager() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) NodeLiveness() interface{} { panic(unsupportedShimMethod) } -func (t *testServerShim) HeartbeatNodeLiveness() error { panic(unsupportedShimMethod) } -func (t *testServerShim) NodeDialer() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) Stopper() *stop.Stopper { panic(unsupportedShimMethod) } +func (t *testServerShim) Start(context.Context) error { panic(unsupportedShimMethod) } +func (t *testServerShim) Node() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) NodeID() roachpb.NodeID { panic(unsupportedShimMethod) } +func (t *testServerShim) ClusterID() uuid.UUID { panic(unsupportedShimMethod) } +func (t *testServerShim) ServingRPCAddr() string { panic(unsupportedShimMethod) } +func (t *testServerShim) RPCAddr() string { panic(unsupportedShimMethod) } +func (t *testServerShim) DB() *kv.DB { panic(unsupportedShimMethod) } +func (t *testServerShim) RPCContext() *rpc.Context { panic(unsupportedShimMethod) } +func (t *testServerShim) LeaseManager() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) InternalExecutor() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) ExecutorConfig() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) TracerI() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) GossipI() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) RangeFeedFactory() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) Clock() *hlc.Clock { panic(unsupportedShimMethod) } +func (t *testServerShim) DistSenderI() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) MigrationServer() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) SpanConfigAccessor() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) SpanConfigSQLTranslator() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) SpanConfigSQLWatcher() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) SQLServer() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) SQLLivenessProvider() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) StartupMigrationsManager() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) NodeLiveness() interface{} { panic(unsupportedShimMethod) } +func (t *testServerShim) HeartbeatNodeLiveness() error { panic(unsupportedShimMethod) } +func (t *testServerShim) NodeDialer() interface{} { panic(unsupportedShimMethod) } func (t *testServerShim) SetDistSQLSpanResolver(spanResolver interface{}) { panic(unsupportedShimMethod) } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 8fdf4f36d151..cc9bcfac9536 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -855,7 +855,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { // only do it if COCKROACH_EXPERIMENTAL_SPAN_CONFIGS is set. spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) sqlTranslator := spanconfigsqltranslator.New(execCfg, codec) - sqlWatcherFactory := spanconfigsqlwatcher.NewFactory( + sqlWatcher := spanconfigsqlwatcher.New( codec, cfg.Settings, cfg.rangeFeedFactory, @@ -870,7 +870,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { cfg.stopper, cfg.Settings, cfg.spanConfigAccessor, - sqlWatcherFactory, + sqlWatcher, sqlTranslator, spanConfigKnobs, ) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 278b6952a43e..c02f2469f8b1 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -978,14 +978,14 @@ func (ts *TestServer) SpanConfigSQLTranslator() interface{} { return ts.sqlServer.spanconfigMgr.SQLTranslator } -// SpanConfigSQLWatcherFactory is part of TestServerInterface. -func (ts *TestServer) SpanConfigSQLWatcherFactory() interface{} { +// SpanConfigSQLWatcher is part of TestServerInterface. +func (ts *TestServer) SpanConfigSQLWatcher() interface{} { if ts.sqlServer.spanconfigMgr == nil { panic( "span config manager uninitialized; see EnableSpanConfigs testing knob to use span configs", ) } - return ts.sqlServer.spanconfigMgr.SQLWatcherFactory + return ts.sqlServer.spanconfigMgr.SQLWatcher } // SQLServer is part of TestServerInterface. diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 469ca83a56aa..ebaf855c611b 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -120,32 +120,21 @@ func FullTranslate( // SQLWatcher watches for events on system.zones and system.descriptors. type SQLWatcher interface { - // WatchForSQLUpdates watches for updates to zones and descriptors starting at - // the given timestamp (exclusive), informing callers using the handler - // callback. + // WatchForSQLUpdates watches for updates to zones and descriptors starting + // at the given timestamp (exclusive), informing callers periodically using + // the given handler[1] and a checkpoint timestamp. The handler is invoked: + // - serially, in the same thread where WatchForSQLUpdates was called; + // - with a monotonically increasing timestamp; + // - with updates from the last provided timestamp (exclusive) to the + // current one (inclusive). // - // The handler callback[1] is invoked from time to time with a list of updates - // and a checkpointTS. Invocations of the handler callback provide the - // following semantics: - // 1. Calls to the handler are serial. - // 2. The timestamp supplied to the handler is monotonically increasing. - // 3. The list of DescriptorUpdates supplied to handler includes all events - // in the window (prevInvocationCheckpointTS, checkpointTS]. - // 4. No further calls to the handler are made if a call to the handler - // returns an error. + // If the handler errors out, it's not invoked subsequently (and internal + // processes are wound down accordingly). Callers are free to persist the + // checkpoint timestamps and use it to re-establish the watcher without + // missing any updates. // - // These guarantees mean that users of this interface are free to persist the - // checkpointTS and later use it to re-establish a SQLWatcher without missing - // any updates. + // [1]: Users should avoid doing expensive work in the handler. // - // WatchForSQLUpdates can only ever be called once, effectively making the - // SQLWatcher a single use interface. - // - // WatchForSQLUpdates may run out of memory and return an error if it is - // tracking too many events between two checkpoints. - // - // [1] Users of this interface should not intend to do expensive work in the - // handler callback. // TODO(arul): Possibly get rid of this limitation. WatchForSQLUpdates( ctx context.Context, @@ -165,22 +154,14 @@ type DescriptorUpdate struct { DescriptorType catalog.DescriptorType } -// SQLWatcherFactory is used to construct new SQLWatchers. -type SQLWatcherFactory interface { - // New returns a new SQLWatcher. - New() SQLWatcher -} - // ReconciliationDependencies captures what's needed by the span config // reconciliation job to perform its task. The job is responsible for // reconciling a tenant's zone configurations with the clusters span // configurations. type ReconciliationDependencies interface { KVAccessor - SQLTranslator - - SQLWatcherFactory + SQLWatcher } // Store is a data structure used to store spans and their corresponding diff --git a/pkg/spanconfig/spanconfigmanager/manager.go b/pkg/spanconfig/spanconfigmanager/manager.go index 010fe2e8d9de..6ca2f58bf9e9 100644 --- a/pkg/spanconfig/spanconfigmanager/manager.go +++ b/pkg/spanconfig/spanconfigmanager/manager.go @@ -60,7 +60,7 @@ type Manager struct { knobs *spanconfig.TestingKnobs spanconfig.KVAccessor - spanconfig.SQLWatcherFactory + spanconfig.SQLWatcher spanconfig.SQLTranslator } @@ -74,7 +74,7 @@ func New( stopper *stop.Stopper, settings *cluster.Settings, kvAccessor spanconfig.KVAccessor, - sqlWatcherFactory spanconfig.SQLWatcherFactory, + sqlWatcher spanconfig.SQLWatcher, sqlTranslator spanconfig.SQLTranslator, knobs *spanconfig.TestingKnobs, ) *Manager { @@ -82,15 +82,15 @@ func New( knobs = &spanconfig.TestingKnobs{} } return &Manager{ - db: db, - jr: jr, - ie: ie, - stopper: stopper, - settings: settings, - KVAccessor: kvAccessor, - SQLWatcherFactory: sqlWatcherFactory, - SQLTranslator: sqlTranslator, - knobs: knobs, + db: db, + jr: jr, + ie: ie, + stopper: stopper, + settings: settings, + KVAccessor: kvAccessor, + SQLWatcher: sqlWatcher, + SQLTranslator: sqlTranslator, + knobs: knobs, } } diff --git a/pkg/spanconfig/spanconfigmanager/manager_test.go b/pkg/spanconfig/spanconfigmanager/manager_test.go index 067befba1464..af51454b86f0 100644 --- a/pkg/spanconfig/spanconfigmanager/manager_test.go +++ b/pkg/spanconfig/spanconfigmanager/manager_test.go @@ -74,7 +74,7 @@ func TestManagerConcurrentJobCreation(t *testing.T) { ts.Stopper(), ts.ClusterSettings(), ts.SpanConfigAccessor().(spanconfig.KVAccessor), - ts.SpanConfigSQLWatcherFactory().(spanconfig.SQLWatcherFactory), + ts.SpanConfigSQLWatcher().(spanconfig.SQLWatcher), ts.SpanConfigSQLTranslator().(spanconfig.SQLTranslator), &spanconfig.TestingKnobs{ ManagerCreatedJobInterceptor: func(jobI interface{}) { @@ -163,7 +163,7 @@ func TestManagerStartsJobIfFailed(t *testing.T) { ts.Stopper(), ts.ClusterSettings(), ts.SpanConfigAccessor().(spanconfig.KVAccessor), - ts.SpanConfigSQLWatcherFactory().(spanconfig.SQLWatcherFactory), + ts.SpanConfigSQLWatcher().(spanconfig.SQLWatcher), ts.SpanConfigSQLTranslator().(spanconfig.SQLTranslator), &spanconfig.TestingKnobs{ ManagerAfterCheckedReconciliationJobExistsInterceptor: func(exists bool) { @@ -239,7 +239,7 @@ func TestManagerCheckJobConditions(t *testing.T) { ts.Stopper(), ts.ClusterSettings(), ts.SpanConfigAccessor().(spanconfig.KVAccessor), - ts.SpanConfigSQLWatcherFactory().(spanconfig.SQLWatcherFactory), + ts.SpanConfigSQLWatcher().(spanconfig.SQLWatcher), ts.SpanConfigSQLTranslator().(spanconfig.SQLTranslator), &spanconfig.TestingKnobs{ ManagerDisableJobCreation: true, diff --git a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel index 1b3b8bf0a9eb..bb47eba6a7c7 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel @@ -54,7 +54,6 @@ go_test( "//pkg/sql/catalog/descpb", "//pkg/testutils", "//pkg/testutils/serverutils", - "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/hlc", diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go index 9026269e89b4..15502e2f86a1 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go @@ -12,7 +12,6 @@ package spanconfigsqlwatcher import ( "context" - "sync/atomic" "unsafe" "github.com/cockroachdb/cockroach/pkg/keys" @@ -30,14 +29,13 @@ import ( "github.com/cockroachdb/errors" ) -// Factory implements the spanconfig.SQLWatcherFactory interface. -var _ spanconfig.SQLWatcherFactory = &Factory{} - // SQLWatcher implements the spanconfig.SQLWatcher interface. var _ spanconfig.SQLWatcher = &SQLWatcher{} -// Factory is used to construct spanconfig.SQLWatcher interfaces. -type Factory struct { +// SQLWatcher is the concrete implementation of spanconfig.SQLWatcher. It +// establishes rangefeeds over system.zones and system.descriptors to +// incrementally watch for SQL updates. +type SQLWatcher struct { codec keys.SQLCodec settings *cluster.Settings stopper *stop.Stopper @@ -46,19 +44,19 @@ type Factory struct { bufferMemLimit int64 } -// NewFactory constructs a new Factory. -func NewFactory( +// New constructs a new SQLWatcher. +func New( codec keys.SQLCodec, settings *cluster.Settings, rangeFeedFactory *rangefeed.Factory, bufferMemLimit int64, stopper *stop.Stopper, knobs *spanconfig.TestingKnobs, -) *Factory { +) *SQLWatcher { if knobs == nil { knobs = &spanconfig.TestingKnobs{} } - return &Factory{ + return &SQLWatcher{ codec: codec, settings: settings, rangeFeedFactory: rangeFeedFactory, @@ -68,57 +66,42 @@ func NewFactory( } } -// SQLWatcher is the concrete implementation of spanconfig.SQLWatcher. It -// establishes rangefeeds over system.zones and system.descriptors to -// incrementally watch for SQL updates. -type SQLWatcher struct { - codec keys.SQLCodec - settings *cluster.Settings - rangeFeedFactory *rangefeed.Factory - stopper *stop.Stopper - - buffer *buffer - - knobs *spanconfig.TestingKnobs - - started int32 // accessed atomically. -} - // sqlWatcherBufferEntrySize is the size of an entry stored in the sqlWatcher's // buffer. We use this value to calculate the buffer capacity. const sqlWatcherBufferEntrySize = int64(unsafe.Sizeof(event{}) + unsafe.Sizeof(rangefeedbuffer.Event(nil))) -// New constructs a spanconfig.SQLWatcher. -func (f *Factory) New() spanconfig.SQLWatcher { - return &SQLWatcher{ - codec: f.codec, - settings: f.settings, - rangeFeedFactory: f.rangeFeedFactory, - stopper: f.stopper, - buffer: newBuffer(int(f.bufferMemLimit / sqlWatcherBufferEntrySize)), - knobs: f.knobs, - } -} - // WatchForSQLUpdates is part of the spanconfig.SQLWatcher interface. func (s *SQLWatcher) WatchForSQLUpdates( ctx context.Context, timestamp hlc.Timestamp, handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, ) error { - if !atomic.CompareAndSwapInt32(&s.started, 0, 1) { - return errors.AssertionFailedf("watcher already started watching") - } + w := &inner{sqlWatcher: s} + return w.watch(ctx, timestamp, handler) +} + +type inner struct { + sqlWatcher *SQLWatcher + + // TODO(irfansharif): We could forego this factory pattern altogether; + // there's no mutable state shared across multiple watchers. +} + +func (i *inner) watch( + ctx context.Context, + timestamp hlc.Timestamp, + handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, +) error { // The callbacks below are invoked by both the rangefeeds we establish, both - // of which run on separate goroutines. To ensure calls to the handler - // function are serial we only ever run it on the main thread of - // WatchForSQLUpdates (instead of pushing it into the rangefeed callbacks). - // The rangefeed callbacks use channels to report errors and notifications to - // to flush events from the buffer. As WatchForSQLUpdate's main thread is the - // sole listener on these channels, doing expensive work in the handler - // function can lead to blocking the rangefeed, which isn't great. This is an - // unfortunate asterisk for users of this interface to be aware of. + // of which run on separate goroutines. We serialize calls to the handler + // function by invoking in this single watch thread (instead of pushing it + // into the rangefeed callbacks). The rangefeed callbacks use channels to + // report errors and notifications to flush events from the buffer. As + // WatchForSQLUpdate's main thread is the sole listener on these channels, + // doing expensive work in the handler function can lead to blocking the + // rangefeed, which isn't great. This is an unfortunate asterisk for users + // of this interface to be aware of. // // TODO(arul): Possibly get rid of this limitation by introducing another // buffer interface here to store updates produced by the Watcher so that @@ -126,8 +109,9 @@ func (s *SQLWatcher) WatchForSQLUpdates( // serial semantics. errCh := make(chan error) frontierAdvanced := make(chan struct{}) + buf := newBuffer(int(i.sqlWatcher.bufferMemLimit / sqlWatcherBufferEntrySize)) onFrontierAdvance := func(ctx context.Context, rangefeed rangefeedKind, timestamp hlc.Timestamp) { - s.buffer.advance(rangefeed, timestamp) + buf.advance(rangefeed, timestamp) select { case <-ctx.Done(): // The context is canceled when the rangefeed is being closed, which @@ -137,12 +121,12 @@ func (s *SQLWatcher) WatchForSQLUpdates( } onEvent := func(ctx context.Context, event event) { err := func() error { - if fn := s.knobs.SQLWatcherOnEventInterceptor; fn != nil { + if fn := i.sqlWatcher.knobs.SQLWatcherOnEventInterceptor; fn != nil { if err := fn(); err != nil { return err } } - return s.buffer.add(event) + return buf.add(event) }() if err != nil { log.Warningf(ctx, "error adding event %v: %v", event, err) @@ -155,12 +139,12 @@ func (s *SQLWatcher) WatchForSQLUpdates( } } - descriptorsRF, err := s.watchForDescriptorUpdates(ctx, timestamp, onEvent, onFrontierAdvance) + descriptorsRF, err := i.watchForDescriptorUpdates(ctx, timestamp, onEvent, onFrontierAdvance) if err != nil { return errors.Wrapf(err, "error establishing rangefeed over system.descriptors") } defer descriptorsRF.Close() - zonesRF, err := s.watchForZoneConfigUpdates(ctx, timestamp, onEvent, onFrontierAdvance) + zonesRF, err := i.watchForZoneConfigUpdates(ctx, timestamp, onEvent, onFrontierAdvance) if err != nil { return errors.Wrapf(err, "error establishing rangefeed over system.zones") } @@ -170,12 +154,12 @@ func (s *SQLWatcher) WatchForSQLUpdates( select { case <-ctx.Done(): return ctx.Err() - case <-s.stopper.ShouldQuiesce(): + case <-i.sqlWatcher.stopper.ShouldQuiesce(): return nil - case err = <-errCh: + case err := <-errCh: return err case <-frontierAdvanced: - events, combinedFrontierTS, err := s.buffer.flush(ctx) + events, combinedFrontierTS, err := buf.flush(ctx) if err != nil { return err } @@ -192,13 +176,13 @@ func (s *SQLWatcher) WatchForSQLUpdates( // watchForDescriptorUpdates establishes a rangefeed over system.descriptors and // invokes the onEvent callback for observed events. The onFrontierAdvance // callback is invoked whenever the rangefeed frontier is advanced as well. -func (s *SQLWatcher) watchForDescriptorUpdates( +func (i *inner) watchForDescriptorUpdates( ctx context.Context, timestamp hlc.Timestamp, onEvent func(context.Context, event), onFrontierAdvance func(context.Context, rangefeedKind, hlc.Timestamp), ) (*rangefeed.RangeFeed, error) { - descriptorTableStart := s.codec.TablePrefix(keys.DescriptorTableID) + descriptorTableStart := i.sqlWatcher.codec.TablePrefix(keys.DescriptorTableID) descriptorTableSpan := roachpb.Span{ Key: descriptorTableStart, EndKey: descriptorTableStart.PrefixEnd(), @@ -218,7 +202,7 @@ func (s *SQLWatcher) watchForDescriptorUpdates( if err := value.GetProto(&descriptor); err != nil { logcrash.ReportOrPanic( ctx, - &s.settings.SV, + &i.sqlWatcher.settings.SV, "%s: failed to unmarshal descriptor %v", ev.Key, value, @@ -247,7 +231,7 @@ func (s *SQLWatcher) watchForDescriptorUpdates( id = schema.GetID() descType = catalog.Schema default: - logcrash.ReportOrPanic(ctx, &s.settings.SV, "unknown descriptor unmarshalled %v", descriptor) + logcrash.ReportOrPanic(ctx, &i.sqlWatcher.settings.SV, "unknown descriptor unmarshalled %v", descriptor) } rangefeedEvent := event{ @@ -259,7 +243,7 @@ func (s *SQLWatcher) watchForDescriptorUpdates( } onEvent(ctx, rangefeedEvent) } - rf, err := s.rangeFeedFactory.RangeFeed( + rf, err := i.sqlWatcher.rangeFeedFactory.RangeFeed( ctx, "sql-watcher-descriptor-rangefeed", descriptorTableSpan, @@ -282,25 +266,25 @@ func (s *SQLWatcher) watchForDescriptorUpdates( // invokes the onEvent callback whenever an event is observed. The // onFrontierAdvance callback is also invoked whenever the rangefeed frontier is // advanced. -func (s *SQLWatcher) watchForZoneConfigUpdates( +func (i *inner) watchForZoneConfigUpdates( ctx context.Context, timestamp hlc.Timestamp, onEvent func(context.Context, event), onFrontierAdvance func(context.Context, rangefeedKind, hlc.Timestamp), ) (*rangefeed.RangeFeed, error) { - zoneTableStart := s.codec.TablePrefix(keys.ZonesTableID) + zoneTableStart := i.sqlWatcher.codec.TablePrefix(keys.ZonesTableID) zoneTableSpan := roachpb.Span{ Key: zoneTableStart, EndKey: zoneTableStart.PrefixEnd(), } - decoder := newZonesDecoder(s.codec) + decoder := newZonesDecoder(i.sqlWatcher.codec) handleEvent := func(ctx context.Context, ev *roachpb.RangeFeedValue) { descID, err := decoder.DecodePrimaryKey(ev.Key) if err != nil { logcrash.ReportOrPanic( ctx, - &s.settings.SV, + &i.sqlWatcher.settings.SV, "sql watcher zones range feed error: %v", err, ) @@ -316,7 +300,7 @@ func (s *SQLWatcher) watchForZoneConfigUpdates( } onEvent(ctx, rangefeedEvent) } - rf, err := s.rangeFeedFactory.RangeFeed( + rf, err := i.sqlWatcher.rangeFeedFactory.RangeFeed( ctx, "sql-watcher-zones-rangefeed", zoneTableSpan, diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go index 0dd028122cb0..75d35a7a7c57 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsqlwatcher" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -122,14 +121,14 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) - sqlWatcher := spanconfigsqlwatcher.NewFactory( + sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), 1<<20, /* 1 MB, bufferMemLimit */ ts.Stopper(), nil, /* knobs */ - ).New() + ) var mu struct { syncutil.Mutex @@ -195,12 +194,10 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { wg.Wait() } -// TestSQLWatcherFactory tests that the SQLWatcherFactory can create multiple -// SQLWatchers and that every SQLWatcher is only good for a single -// WatchForSQLUpdates. -func TestSQLWatcherFactory(t *testing.T) { +// TestSQLWatcherMultiple tests that we're able to fire off multiple sql watcher +// processes, both sequentially and concurrently. +func TestSQLWatcherMultiple(t *testing.T) { defer leaktest.AfterTest(t)() - skip.IgnoreLint(t, "buggy test; handler's invoked but test doesn't expect it to be") tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ @@ -219,7 +216,7 @@ func TestSQLWatcherFactory(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) - sqlWatcherFactory := spanconfigsqlwatcher.NewFactory( + sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), @@ -228,47 +225,98 @@ func TestSQLWatcherFactory(t *testing.T) { nil, /* knobs */ ) - startTS := ts.Clock().Now() + beforeStmtTS := ts.Clock().Now() tdb.Exec(t, "CREATE TABLE t()") - - sqlWatcher := sqlWatcherFactory.New() + afterStmtTS := ts.Clock().Now() + const expDescID descpb.ID = 52 var wg sync.WaitGroup + mu := struct { + syncutil.Mutex + w1LastCheckpoint, w2LastCheckpoint, w3LastCheckpoint hlc.Timestamp + }{} - watcherCtx, watcherCancel := context.WithCancel(ctx) - wg.Add(1) - go func() { + f := func(ctx context.Context, onExit func(), onCheckpoint func(hlc.Timestamp)) { defer wg.Done() - _ = sqlWatcher.WatchForSQLUpdates(watcherCtx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { - t.Error("handler should never run") + + receivedIDs := make(map[descpb.ID]struct{}) + err := sqlWatcher.WatchForSQLUpdates(ctx, beforeStmtTS, + func(_ context.Context, updates []spanconfig.DescriptorUpdate, checkpointTS hlc.Timestamp) error { + onCheckpoint(checkpointTS) + + for _, update := range updates { + receivedIDs[update.ID] = struct{}{} + } + return nil + }) + require.True(t, testutils.IsError(err, "context canceled")) + require.Equal(t, 1, len(receivedIDs)) + _, seen := receivedIDs[expDescID] + require.True(t, seen) + } + + { + // Run the first watcher; wait for it to observe the update before + // tearing it down. + + watcher1Ctx, watcher1Cancel := context.WithCancel(ctx) + wg.Add(1) + go f(watcher1Ctx, func() { wg.Done() }, func(ts hlc.Timestamp) { + mu.Lock() + mu.w1LastCheckpoint = ts + mu.Unlock() + }) + + testutils.SucceedsSoon(t, func() error { + mu.Lock() + defer mu.Unlock() + + if mu.w1LastCheckpoint.Less(afterStmtTS) { + return errors.New("w1 checkpoint precedes statement timestamp") + } return nil }) - }() + watcher1Cancel() + wg.Wait() + } - watcherCancel() - wg.Wait() + { + // Run two more watchers; wait for each to independently observe the + // update before tearing them down. - err := sqlWatcher.WatchForSQLUpdates(watcherCtx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { - t.Fatal("handler should never run") - return nil - }) - require.Error(t, err) - require.True(t, testutils.IsError(err, "watcher already started watching")) + watcher2Ctx, watcher2Cancel := context.WithCancel(ctx) + wg.Add(1) + go f(watcher2Ctx, func() { wg.Done() }, func(ts hlc.Timestamp) { + mu.Lock() + mu.w2LastCheckpoint = ts + mu.Unlock() + }) - newSQLWatcher := sqlWatcherFactory.New() - newWatcherCtx, newWatcherCancel := context.WithCancel(ctx) + watcher3Ctx, watcher3Cancel := context.WithCancel(ctx) + wg.Add(1) + go f(watcher3Ctx, func() { wg.Done() }, func(ts hlc.Timestamp) { + mu.Lock() + mu.w3LastCheckpoint = ts + mu.Unlock() + }) - wg.Add(1) - go func() { - defer wg.Done() - err = newSQLWatcher.WatchForSQLUpdates(newWatcherCtx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { - t.Error("handler should never run") + testutils.SucceedsSoon(t, func() error { + mu.Lock() + defer mu.Unlock() + + if mu.w2LastCheckpoint.Less(afterStmtTS) { + return errors.New("w2 checkpoint precedes statement timestamp") + } + if mu.w3LastCheckpoint.Less(afterStmtTS) { + return errors.New("w3 checkpoint precedes statement timestamp") + } return nil }) - require.True(t, testutils.IsError(err, "context canceled")) - }() - newWatcherCancel() + watcher2Cancel() + watcher3Cancel() + } + wg.Wait() } @@ -295,7 +343,7 @@ func TestSQLWatcherOnEventError(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) - sqlWatcherFactory := spanconfigsqlwatcher.NewFactory( + sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), @@ -311,7 +359,6 @@ func TestSQLWatcherOnEventError(t *testing.T) { startTS := ts.Clock().Now() tdb.Exec(t, "CREATE TABLE t()") - sqlWatcher := sqlWatcherFactory.New() err := sqlWatcher.WatchForSQLUpdates(ctx, startTS, func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error { t.Fatal("handler should never run") return nil @@ -342,14 +389,14 @@ func TestSQLWatcherHandlerError(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) - sqlWatcher := spanconfigsqlwatcher.NewFactory( + sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), 1<<20, /* 1 MB, bufferMemLimit */ ts.Stopper(), nil, /* knobs */ - ).New() + ) tdb.Exec(t, "CREATE TABLE t()") diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 5bc6701c3a16..9de4a23a259b 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -122,9 +122,9 @@ type TestServerInterface interface { // an interface{}. SpanConfigSQLTranslator() interface{} - // SpanConfigSQLWatcherFactory returns the underlying - // spanconfig.SQLWatcherFactory as an interface{}. - SpanConfigSQLWatcherFactory() interface{} + // SpanConfigSQLWatcher returns the underlying spanconfig.SQLWatcher as an + // interface{}. + SpanConfigSQLWatcher() interface{} // SQLServer returns the *sql.Server as an interface{}. SQLServer() interface{} From 6fdc4752ff5c0dd5d1d4c042c2ee999051338c22 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 25 Nov 2021 13:13:26 -0500 Subject: [PATCH 03/10] spanconfig/sqlwatcher: remove internal factory pattern A previous commit obviated the need for a factory altogether, but went only as far hiding it within the package boundary. This commit gets rid of it entirely. Release note: None --- .../spanconfigsqlwatcher/sqlwatcher.go | 44 ++++++++----------- 1 file changed, 18 insertions(+), 26 deletions(-) diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go index 15502e2f86a1..5d3a1d6ac8dd 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go @@ -66,7 +66,7 @@ func New( } } -// sqlWatcherBufferEntrySize is the size of an entry stored in the sqlWatcher's +// sqlWatcherBufferEntrySize is the size of an entry stored in the SQLWatcher's // buffer. We use this value to calculate the buffer capacity. const sqlWatcherBufferEntrySize = int64(unsafe.Sizeof(event{}) + unsafe.Sizeof(rangefeedbuffer.Event(nil))) @@ -76,18 +76,10 @@ func (s *SQLWatcher) WatchForSQLUpdates( timestamp hlc.Timestamp, handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, ) error { - w := &inner{sqlWatcher: s} - return w.watch(ctx, timestamp, handler) + return s.watch(ctx, timestamp, handler) } -type inner struct { - sqlWatcher *SQLWatcher - - // TODO(irfansharif): We could forego this factory pattern altogether; - // there's no mutable state shared across multiple watchers. -} - -func (i *inner) watch( +func (s *SQLWatcher) watch( ctx context.Context, timestamp hlc.Timestamp, handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, @@ -109,7 +101,7 @@ func (i *inner) watch( // serial semantics. errCh := make(chan error) frontierAdvanced := make(chan struct{}) - buf := newBuffer(int(i.sqlWatcher.bufferMemLimit / sqlWatcherBufferEntrySize)) + buf := newBuffer(int(s.bufferMemLimit / sqlWatcherBufferEntrySize)) onFrontierAdvance := func(ctx context.Context, rangefeed rangefeedKind, timestamp hlc.Timestamp) { buf.advance(rangefeed, timestamp) select { @@ -121,7 +113,7 @@ func (i *inner) watch( } onEvent := func(ctx context.Context, event event) { err := func() error { - if fn := i.sqlWatcher.knobs.SQLWatcherOnEventInterceptor; fn != nil { + if fn := s.knobs.SQLWatcherOnEventInterceptor; fn != nil { if err := fn(); err != nil { return err } @@ -139,12 +131,12 @@ func (i *inner) watch( } } - descriptorsRF, err := i.watchForDescriptorUpdates(ctx, timestamp, onEvent, onFrontierAdvance) + descriptorsRF, err := s.watchForDescriptorUpdates(ctx, timestamp, onEvent, onFrontierAdvance) if err != nil { return errors.Wrapf(err, "error establishing rangefeed over system.descriptors") } defer descriptorsRF.Close() - zonesRF, err := i.watchForZoneConfigUpdates(ctx, timestamp, onEvent, onFrontierAdvance) + zonesRF, err := s.watchForZoneConfigUpdates(ctx, timestamp, onEvent, onFrontierAdvance) if err != nil { return errors.Wrapf(err, "error establishing rangefeed over system.zones") } @@ -154,7 +146,7 @@ func (i *inner) watch( select { case <-ctx.Done(): return ctx.Err() - case <-i.sqlWatcher.stopper.ShouldQuiesce(): + case <-s.stopper.ShouldQuiesce(): return nil case err := <-errCh: return err @@ -176,13 +168,13 @@ func (i *inner) watch( // watchForDescriptorUpdates establishes a rangefeed over system.descriptors and // invokes the onEvent callback for observed events. The onFrontierAdvance // callback is invoked whenever the rangefeed frontier is advanced as well. -func (i *inner) watchForDescriptorUpdates( +func (s *SQLWatcher) watchForDescriptorUpdates( ctx context.Context, timestamp hlc.Timestamp, onEvent func(context.Context, event), onFrontierAdvance func(context.Context, rangefeedKind, hlc.Timestamp), ) (*rangefeed.RangeFeed, error) { - descriptorTableStart := i.sqlWatcher.codec.TablePrefix(keys.DescriptorTableID) + descriptorTableStart := s.codec.TablePrefix(keys.DescriptorTableID) descriptorTableSpan := roachpb.Span{ Key: descriptorTableStart, EndKey: descriptorTableStart.PrefixEnd(), @@ -202,7 +194,7 @@ func (i *inner) watchForDescriptorUpdates( if err := value.GetProto(&descriptor); err != nil { logcrash.ReportOrPanic( ctx, - &i.sqlWatcher.settings.SV, + &s.settings.SV, "%s: failed to unmarshal descriptor %v", ev.Key, value, @@ -231,7 +223,7 @@ func (i *inner) watchForDescriptorUpdates( id = schema.GetID() descType = catalog.Schema default: - logcrash.ReportOrPanic(ctx, &i.sqlWatcher.settings.SV, "unknown descriptor unmarshalled %v", descriptor) + logcrash.ReportOrPanic(ctx, &s.settings.SV, "unknown descriptor unmarshalled %v", descriptor) } rangefeedEvent := event{ @@ -243,7 +235,7 @@ func (i *inner) watchForDescriptorUpdates( } onEvent(ctx, rangefeedEvent) } - rf, err := i.sqlWatcher.rangeFeedFactory.RangeFeed( + rf, err := s.rangeFeedFactory.RangeFeed( ctx, "sql-watcher-descriptor-rangefeed", descriptorTableSpan, @@ -266,25 +258,25 @@ func (i *inner) watchForDescriptorUpdates( // invokes the onEvent callback whenever an event is observed. The // onFrontierAdvance callback is also invoked whenever the rangefeed frontier is // advanced. -func (i *inner) watchForZoneConfigUpdates( +func (s *SQLWatcher) watchForZoneConfigUpdates( ctx context.Context, timestamp hlc.Timestamp, onEvent func(context.Context, event), onFrontierAdvance func(context.Context, rangefeedKind, hlc.Timestamp), ) (*rangefeed.RangeFeed, error) { - zoneTableStart := i.sqlWatcher.codec.TablePrefix(keys.ZonesTableID) + zoneTableStart := s.codec.TablePrefix(keys.ZonesTableID) zoneTableSpan := roachpb.Span{ Key: zoneTableStart, EndKey: zoneTableStart.PrefixEnd(), } - decoder := newZonesDecoder(i.sqlWatcher.codec) + decoder := newZonesDecoder(s.codec) handleEvent := func(ctx context.Context, ev *roachpb.RangeFeedValue) { descID, err := decoder.DecodePrimaryKey(ev.Key) if err != nil { logcrash.ReportOrPanic( ctx, - &i.sqlWatcher.settings.SV, + &s.settings.SV, "sql watcher zones range feed error: %v", err, ) @@ -300,7 +292,7 @@ func (i *inner) watchForZoneConfigUpdates( } onEvent(ctx, rangefeedEvent) } - rf, err := i.sqlWatcher.rangeFeedFactory.RangeFeed( + rf, err := s.rangeFeedFactory.RangeFeed( ctx, "sql-watcher-zones-rangefeed", zoneTableSpan, From 35e66c17a700471153bf358bb9ad4ac11a0a104b Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 17:54:10 -0500 Subject: [PATCH 04/10] sql/schemachanger/scgraph: refactor dependency edge iteration Release note: None --- pkg/sql/schemachanger/scgraph/BUILD.bazel | 10 +- .../schemachanger/scgraph/dep_edge_tree.go | 102 ++++++++++++++ .../scgraph/dep_edge_tree_test.go | 129 ++++++++++++++++++ pkg/sql/schemachanger/scgraph/graph.go | 91 +++--------- pkg/sql/schemachanger/scgraph/iteration.go | 27 +--- .../schemachanger/scplan/deprules/registry.go | 7 +- 6 files changed, 264 insertions(+), 102 deletions(-) create mode 100644 pkg/sql/schemachanger/scgraph/dep_edge_tree.go create mode 100644 pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index c5e3c91993cd..e229c3cee857 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "scgraph", srcs = [ + "dep_edge_tree.go", "graph.go", "iteration.go", ], @@ -21,12 +22,17 @@ go_library( go_test( name = "scgraph_test", - srcs = ["graph_test.go"], + srcs = [ + "dep_edge_tree_test.go", + "graph_test.go", + ], + embed = [":scgraph"], deps = [ - ":scgraph", "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/scop", "//pkg/sql/schemachanger/scpb", + "//pkg/util/iterutil", + "//pkg/util/leaktest", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/sql/schemachanger/scgraph/dep_edge_tree.go b/pkg/sql/schemachanger/scgraph/dep_edge_tree.go new file mode 100644 index 000000000000..dcc3354574d5 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/dep_edge_tree.go @@ -0,0 +1,102 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scgraph + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/google/btree" +) + +type depEdgeTree struct { + t *btree.BTree + order edgeTreeOrder + cmp nodeCmpFn +} + +type nodeCmpFn func(a, b *scpb.Node) (less, eq bool) + +func newDepEdgeTree(order edgeTreeOrder, cmp nodeCmpFn) *depEdgeTree { + const degree = 8 // arbitrary + return &depEdgeTree{ + t: btree.New(degree), + order: order, + cmp: cmp, + } +} + +// edgeTreeOrder order in which the edge tree is sorted, +// either based on from/to node indexes. +type edgeTreeOrder bool + +func (o edgeTreeOrder) first(e Edge) *scpb.Node { + if o == fromTo { + return e.From() + } + return e.To() +} + +func (o edgeTreeOrder) second(e Edge) *scpb.Node { + if o == toFrom { + return e.From() + } + return e.To() +} + +const ( + fromTo edgeTreeOrder = true + toFrom edgeTreeOrder = false +) + +// edgeTreeEntry BTree items for tracking edges +// in an ordered manner. +type edgeTreeEntry struct { + t *depEdgeTree + edge *DepEdge +} + +func (et *depEdgeTree) insert(e *DepEdge) { + et.t.ReplaceOrInsert(&edgeTreeEntry{ + t: et, + edge: e, + }) +} + +func (et *depEdgeTree) iterateSourceNode(n *scpb.Node, it DepEdgeIterator) (err error) { + e := &edgeTreeEntry{t: et, edge: &DepEdge{}} + if et.order == fromTo { + e.edge.from = n + } else { + e.edge.to = n + } + et.t.AscendGreaterOrEqual(e, func(i btree.Item) (wantMore bool) { + e := i.(*edgeTreeEntry) + if et.order.first(e.edge) != n { + return false + } + err = it(e.edge) + return err == nil + }) + if iterutil.Done(err) { + err = nil + } + return err +} + +// Less implements btree.Item. +func (e *edgeTreeEntry) Less(otherItem btree.Item) bool { + o := otherItem.(*edgeTreeEntry) + if less, eq := e.t.cmp(e.t.order.first(e.edge), e.t.order.first(o.edge)); !eq { + return less + } + less, _ := e.t.cmp(e.t.order.second(e.edge), e.t.order.second(o.edge)) + return less +} diff --git a/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go new file mode 100644 index 000000000000..0b25a7aeef25 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go @@ -0,0 +1,129 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scgraph + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +// TestDepEdgeTree exercises the depEdgeTree data structure to ensure it works +// as expected. +func TestDepEdgeTree(t *testing.T) { + defer leaktest.AfterTest(t)() + + type nodeID int + type edge [2]nodeID + // queryCase runs a query to iterate all edges sources at the node with id q. + type queryCase struct { + q nodeID + take int // if > 1, indicates a desire to stop early + res []edge // expected results + } + // testCase describes the edges to be added and the queries to run. + type testCase struct { + order edgeTreeOrder + edges []edge + queries []queryCase + } + testCases := []testCase{ + { + order: fromTo, + edges: []edge{ + {2, 4}, {2, 3}, {4, 5}, {1, 2}, + }, + queries: []queryCase{ + {q: 1, res: []edge{{1, 2}}}, + {q: 2, res: []edge{{2, 3}, {2, 4}}}, + {q: 2, take: 1, res: []edge{{2, 3}}}, + }, + }, + { + order: toFrom, + edges: []edge{ + {2, 4}, {2, 3}, {4, 5}, {1, 2}, {2, 5}, {1, 5}, + }, + queries: []queryCase{ + {q: 1, res: nil}, + {q: 2, res: []edge{{1, 2}}}, + {q: 5, res: []edge{{1, 5}, {2, 5}, {4, 5}}}, + {q: 5, take: 1, res: []edge{{1, 5}}}, + }, + }, + } + + // testCaseState is used for each queryCase in a testCase. + type testCaseState struct { + tree *depEdgeTree + nodes []*scpb.Node // nodes with lower indexes sort lower + nodesToID map[*scpb.Node]nodeID + } + makeTestCaseState := func(tc testCase) testCaseState { + tcs := testCaseState{ + nodesToID: make(map[*scpb.Node]nodeID), + } + target := scpb.Target{} + getNode := func(i nodeID) *scpb.Node { + if i > nodeID(len(tcs.nodes)-1) { + for j := nodeID(len(tcs.nodes)); j <= i; j++ { + tcs.nodes = append(tcs.nodes, &scpb.Node{ + Target: &target, + Status: scpb.Status(j), + }) + tcs.nodesToID[tcs.nodes[j]] = j + } + } + return tcs.nodes[i] + } + tcs.tree = newDepEdgeTree(tc.order, func(a, b *scpb.Node) (less, eq bool) { + ai, bi := tcs.nodesToID[a], tcs.nodesToID[b] + return ai < bi, ai == bi + }) + for _, e := range tc.edges { + tcs.tree.insert(&DepEdge{ + from: getNode(e[0]), + to: getNode(e[1]), + }) + } + return tcs + } + runQueryCase := func(t *testing.T, tcs testCaseState, qc queryCase) { + i := 0 + var res []edge + require.NoError(t, tcs.tree.iterateSourceNode(tcs.nodes[qc.q], func(de *DepEdge) error { + if i++; qc.take > 0 && i > qc.take { + return iterutil.StopIteration() + } + res = append(res, edge{ + tcs.nodesToID[de.From()], + tcs.nodesToID[de.To()], + }) + return nil + })) + require.Equal(t, qc.res, res) + } + + for _, tc := range testCases { + t.Run(fmt.Sprintf("%v,%v", tc.order, tc.edges), func(t *testing.T) { + tcs := makeTestCaseState(tc) + for _, qc := range tc.queries { + t.Run(fmt.Sprintf("%d,%d", qc.q, qc.take), func(t *testing.T) { + runQueryCase(t, tcs, qc) + }) + } + }) + } +} diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 0e5f9843a4fa..20f88fb8f28a 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/errors" - "github.com/google/btree" ) // Graph is a graph whose nodes are *scpb.Nodes. Graphs are constructed during @@ -49,12 +48,7 @@ type Graph struct { // nodeDepEdgesFrom maps a Node from its dependencies. // A Node dependency is another target node which must be // reached before or concurrently with this node. - nodeDepEdgesFrom *btree.BTree - - // nodeDepEdgesTo maps a Node to its dependencies. - // A Node dependency is another target node which must be - // reached before or concurrently with this node. - nodeDepEdgesTo *btree.BTree + nodeDepEdgesFrom *depEdgeTree // opToNode maps from an operation back to the // opEdge that generated it as an index. @@ -90,14 +84,13 @@ func New(initial scpb.State) (*Graph, error) { g := Graph{ targetIdxMap: map[*scpb.Target]int{}, nodeOpEdgesFrom: map[*scpb.Node]*OpEdge{}, - nodeDepEdgesFrom: btree.New(2), - nodeDepEdgesTo: btree.New(2), optimizedOutOpEdges: map[*OpEdge]bool{}, opToNode: map[scop.Op]*scpb.Node{}, entities: db, statements: initial.Statements, authorization: initial.Authorization, } + g.nodeDepEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) for _, n := range initial.Nodes { if existing, ok := g.targetIdxMap[n.Target]; ok { return nil, errors.Errorf("invalid initial state contains duplicate target: %v and %v", n, initial.Nodes[existing]) @@ -127,7 +120,6 @@ func (g *Graph) ShallowClone() *Graph { targetIdxMap: g.targetIdxMap, nodeOpEdgesFrom: g.nodeOpEdgesFrom, nodeDepEdgesFrom: g.nodeDepEdgesFrom, - nodeDepEdgesTo: g.nodeDepEdgesTo, opToNode: g.opToNode, edges: g.edges, entities: g.entities, @@ -251,16 +243,7 @@ func (g *Graph) AddDepEdge( return err } g.edges = append(g.edges, de) - g.nodeDepEdgesFrom.ReplaceOrInsert(&edgeTreeEntry{ - g: g, - edge: de, - order: fromTo, - }) - g.nodeDepEdgesTo.ReplaceOrInsert(&edgeTreeEntry{ - g: g, - edge: de, - order: toFrom, - }) + g.nodeDepEdgesFrom.insert(de) return nil } @@ -344,6 +327,23 @@ func (de *DepEdge) To() *scpb.Node { return de.to } // Name returns the name of the rule which generated this edge. func (de *DepEdge) Name() string { return de.rule } +// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. +func (g *Graph) compareNodes(a, b *scpb.Node) (less, eq bool) { + switch { + case a == b: + return false, true + case a == nil: + return true, false + case b == nil: + return false, false + case a.Target == b.Target: + return a.Status < b.Status, a.Status == b.Status + default: + aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] + return aIdx < bIdx, aIdx == bIdx + } +} + // GetNodeRanks fetches ranks of nodes in topological order. func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { defer func() { @@ -399,54 +399,3 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { } return rank, nil } - -// edgeTreeOrder order in which the edge tree is sorted, -// either based on from/to node indexes. -type edgeTreeOrder bool - -const ( - fromTo edgeTreeOrder = true - toFrom edgeTreeOrder = false -) - -// edgeTreeEntry BTree items for tracking edges -// in an ordered manner. -type edgeTreeEntry struct { - g *Graph - edge Edge - order edgeTreeOrder -} - -// Less implements btree.Item -func (e *edgeTreeEntry) Less(other btree.Item) bool { - o := other.(*edgeTreeEntry) - var a1, b1, a2, b2 *scpb.Node - switch e.order { - case fromTo: - a1, b1, a2, b2 = e.edge.From(), o.edge.From(), e.edge.To(), o.edge.To() - case toFrom: - a1, b1, a2, b2 = e.edge.To(), o.edge.To(), e.edge.From(), o.edge.From() - } - less, eq := compareNodes(e.g, a1, b1) - if eq { - less, _ = compareNodes(e.g, a2, b2) - } - return less -} - -// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. -func compareNodes(g *Graph, a, b *scpb.Node) (less, eq bool) { - switch { - case a == b: - return false, true - case a == nil: - return true, false - case b == nil: - return false, false - case a.Target == b.Target: - return a.Status < b.Status, a.Status == b.Status - default: - aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] - return aIdx < bIdx, aIdx == bIdx - } -} diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index 74d8749a362f..70346216ea97 100644 --- a/pkg/sql/schemachanger/scgraph/iteration.go +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -13,7 +13,6 @@ package scgraph import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/util/iterutil" - "github.com/google/btree" ) // NodeIterator is used to iterate nodes. Return iterutil.StopIteration to @@ -60,29 +59,5 @@ type DepEdgeIterator func(de *DepEdge) error // ForEachDepEdgeFrom iterates the dep edges in the graph. func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) (err error) { - g.nodeDepEdgesFrom.AscendGreaterOrEqual(&edgeTreeEntry{ - g: g, - edge: &DepEdge{ - from: n, - to: nil, - rule: "", - }, - order: fromTo, - }, - func(i btree.Item) bool { - e := i.(*edgeTreeEntry) - // End the iteration once the from nodes - // stop matching. - if e.edge.From() != n { - return false - } - if err = it(e.edge.(*DepEdge)); err != nil { - if iterutil.Done(err) { - err = nil - } - return false - } - return true - }) - return err + return g.nodeDepEdgesFrom.iterateSourceNode(n, it) } diff --git a/pkg/sql/schemachanger/scplan/deprules/registry.go b/pkg/sql/schemachanger/scplan/deprules/registry.go index dc5998dd6c78..ed5fb7d90c59 100644 --- a/pkg/sql/schemachanger/scplan/deprules/registry.go +++ b/pkg/sql/schemachanger/scplan/deprules/registry.go @@ -39,9 +39,10 @@ func Apply(g *scgraph.Graph) error { var depRules []rule type rule struct { - name string - from, to rel.Var - q *rel.Query + name string + from, to rel.Var + q *rel.Query + sameStage bool } func register(ruleName string, from, to rel.Var, query *rel.Query) { From b5e7b2767caf5bb3ee343db1a914ea0ca4941065 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:03:18 -0500 Subject: [PATCH 05/10] sql/schemachanger/scgraph: refactor rank ordering test Just pulling symbols into the test and making the table a bit more explicit. Release note: None --- pkg/sql/schemachanger/scgraph/graph_test.go | 252 ++++++++++---------- 1 file changed, 128 insertions(+), 124 deletions(-) diff --git a/pkg/sql/schemachanger/scgraph/graph_test.go b/pkg/sql/schemachanger/scgraph/graph_test.go index 424a3ce6dca8..48df7e930c2b 100644 --- a/pkg/sql/schemachanger/scgraph/graph_test.go +++ b/pkg/sql/schemachanger/scgraph/graph_test.go @@ -22,147 +22,151 @@ import ( "github.com/stretchr/testify/require" ) -type depEdge struct { - from, to int -} +// TestPlanGraphSort sanity checks sorting of the graph. +func TestGraphRanks(t *testing.T) { -func runRankTestForGraph( - t *testing.T, addNode []bool, depEdges []depEdge, expectedOrder []int, expectedRankErr string, -) { - // Setup a state based on if it is a add or drop. - state := scpb.State{ - Nodes: make([]*scpb.Node, 0, len(addNode)), - } - for idx := range addNode { - if addNode[idx] { - state.Nodes = append(state.Nodes, &scpb.Node{ - Target: scpb.NewTarget(scpb.Target_ADD, - &scpb.Table{ - TableID: descpb.ID(idx), - }, - nil /* metadata */), - Status: scpb.Status_ABSENT, - }) - } else { - state.Nodes = append(state.Nodes, &scpb.Node{ - Target: scpb.NewTarget(scpb.Target_DROP, - &scpb.Table{ - TableID: descpb.ID(idx), - }, - nil /* metadata */), - Status: scpb.Status_PUBLIC, - }) - } - } - // Setup the nodes first. - graph, err := scgraph.New(state) - require.NoError(t, err) - // Setup op edges for all the nodes. - for idx := range addNode { - if addNode[idx] { - require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, - scpb.Status_ABSENT, - scpb.Status_PUBLIC, - true, - &scop.MakeColumnAbsent{})) - } else { - require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, - scpb.Status_PUBLIC, - scpb.Status_ABSENT, - true, - &scop.MakeColumnAbsent{})) - } - } - // Add the dep edges next. - for _, edge := range depEdges { - require.NoError(t, graph.AddDepEdge( - fmt.Sprintf("%d to %d", edge.from, edge.to), - state.Nodes[edge.from].Target, - scpb.Status_PUBLIC, - state.Nodes[edge.to].Target, - scpb.Status_PUBLIC, - )) + type depEdge struct { + from, to int } - // Validates the rank order for nodes. - validateNodeRanks := func(graph *scgraph.Graph, expectedOrder []int) { - rank, err := graph.GetNodeRanks() - if expectedRankErr != "" { - require.Errorf(t, err, expectedRankErr) - return // Nothing else to validate - } else { - require.NoError(t, err) - } - unsortedNodes := make([]*scpb.Node, 0, len(state.Nodes)) - for _, node := range state.Nodes { - publicNode, ok := graph.GetNode(node.Target, scpb.Status_PUBLIC) - require.Truef(t, ok, "public node doesn't exist") - unsortedNodes = append(unsortedNodes, publicNode) - } - sort.SliceStable(unsortedNodes, func(i, j int) bool { - return rank[unsortedNodes[i]] > rank[unsortedNodes[j]] - }) - sortedOrder := make([]int, 0, len(unsortedNodes)) - for _, node := range unsortedNodes { - sortedOrder = append(sortedOrder, int(node.Table.TableID)) - } - require.EqualValues(t, expectedOrder, sortedOrder, "ranks are not in expected order") + type testCase struct { + name string + addNode []bool + depEdges []depEdge + expectedOrder []int + expectedRankErr string } - validateNodeRanks(graph, expectedOrder) -} -// TestPlanGraphSort sanity checks sorting of the graph. -func TestGraphRanks(t *testing.T) { - // We will set up the dependency graph for basic ordering, so that: - // 1) 0 depends on 1 - // 2) 3 depends on 0 - // 3) 2 depends on nothing - t.Run("simple dependency graph", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, true, true, true}, - []depEdge{ + testCases := []testCase{ + + // We will set up the dependency graph for basic ordering, so that + // 2 depends on nothing. + { + name: "simple dependency graph", + addNode: []bool{true, true, true, true}, + depEdges: []depEdge{ {0, 1}, {3, 0}, }, - []int{1, 0, 2, 3}, - "", - ) - }) + expectedOrder: []int{1, 0, 2, 3}, + }, - // We will set up the dependency graph, so that its - // intentionally cyclic, which should panic: - // 1) 0 depends on 1 - // 2) 3 depends on 0 - // 3) 1 depends on 3 - // 4) 3 depends on 1 - t.Run("cyclic graph", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, true, true, true}, - []depEdge{ + // We will set up the dependency graph, so that its intentionally cyclic, + // which should result in an error. + { + name: "cyclic graph", + addNode: []bool{true, true, true, true}, + depEdges: []depEdge{ {0, 1}, {3, 0}, {1, 3}, {3, 1}, }, - nil, // Not expecting this to run. - "graph is not a dag", - ) - }) + expectedRankErr: "graph is not a dag", + }, - // We will set up the dependency graph to have a swap - // 1) 0 (adding) depends on 1 (dropping) - // 2) 1 (dropping) depends on 0 (adding) - // 3) 2 (adding) depends on 0 (adding) - t.Run("dependency graph with a swap", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, false, true}, - []depEdge{ + // We will set up the dependency graph to have a swap: + // 1) 0 (adding) depends on 1 (dropping) + // 2) 1 (dropping) depends on 0 (adding) + // 3) 2 (adding) depends on 0 (adding) + { + + name: "dependency graph with a swap", + addNode: []bool{true, false, true}, + depEdges: []depEdge{ {0, 1}, {1, 0}, {2, 0}, }, - []int{1, 0, 2}, // We expect the drop to be ordered first. - "", - ) - }) + expectedOrder: []int{1, 0, 2}, // We expect the drop to be ordered first. + }, + } + + run := func( + t *testing.T, tc testCase, + ) { + // Setup a state based on if it is a add or drop. + state := scpb.State{ + Nodes: make([]*scpb.Node, 0, len(tc.addNode)), + } + for idx := range tc.addNode { + if tc.addNode[idx] { + state.Nodes = append(state.Nodes, &scpb.Node{ + Target: scpb.NewTarget(scpb.Target_ADD, + &scpb.Table{ + TableID: descpb.ID(idx), + }, + nil /* metadata */), + Status: scpb.Status_ABSENT, + }) + } else { + state.Nodes = append(state.Nodes, &scpb.Node{ + Target: scpb.NewTarget(scpb.Target_DROP, + &scpb.Table{ + TableID: descpb.ID(idx), + }, + nil /* metadata */), + Status: scpb.Status_PUBLIC, + }) + } + } + // Setup the nodes first. + graph, err := scgraph.New(state) + require.NoError(t, err) + // Setup op edges for all the nodes. + for idx := range tc.addNode { + if tc.addNode[idx] { + require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, + scpb.Status_ABSENT, + scpb.Status_PUBLIC, + true, + &scop.MakeColumnAbsent{})) + } else { + require.NoError(t, graph.AddOpEdges(state.Nodes[idx].Target, + scpb.Status_PUBLIC, + scpb.Status_ABSENT, + true, + &scop.MakeColumnAbsent{})) + } + } + // Add the dep edges next. + for _, edge := range tc.depEdges { + require.NoError(t, graph.AddDepEdge( + fmt.Sprintf("%d to %d", edge.from, edge.to), + state.Nodes[edge.from].Target, + scpb.Status_PUBLIC, + state.Nodes[edge.to].Target, + scpb.Status_PUBLIC, + )) + } + + // Validates the rank order for nodes. + validateNodeRanks := func(graph *scgraph.Graph, expectedOrder []int) { + rank, err := graph.GetNodeRanks() + if tc.expectedRankErr != "" { + require.Regexp(t, tc.expectedRankErr, err) + return // Nothing else to validate + } else { + require.NoError(t, err) + } + unsortedNodes := make([]*scpb.Node, 0, len(state.Nodes)) + for _, node := range state.Nodes { + publicNode, ok := graph.GetNode(node.Target, scpb.Status_PUBLIC) + require.Truef(t, ok, "public node doesn't exist") + unsortedNodes = append(unsortedNodes, publicNode) + } + sort.SliceStable(unsortedNodes, func(i, j int) bool { + return rank[unsortedNodes[i]] > rank[unsortedNodes[j]] + }) + sortedOrder := make([]int, 0, len(unsortedNodes)) + for _, node := range unsortedNodes { + sortedOrder = append(sortedOrder, int(node.Table.TableID)) + } + require.EqualValues(t, expectedOrder, sortedOrder, "ranks are not in expected order") + } + validateNodeRanks(graph, tc.expectedOrder) + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { run(t, tc) }) + } } From 7d8f9fc4cbc962ed1f684be219f317750439abf2 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:19:08 -0500 Subject: [PATCH 06/10] sql/schemachanger/scgraph: add a simple test for (*Graph).compareNodes() Release note: None --- .../scgraph/dep_edge_tree_test.go | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go index 0b25a7aeef25..bcb34bce4090 100644 --- a/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go +++ b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go @@ -127,3 +127,55 @@ func TestDepEdgeTree(t *testing.T) { }) } } + +// TestGraphCompareNodes ensures the semantics of (*Graph).compareNodes is sane. +func TestGraphCompareNodes(t *testing.T) { + defer leaktest.AfterTest(t)() + t1 := scpb.NewTarget(scpb.Target_ADD, &scpb.Table{TableID: 1}, nil) + t2 := scpb.NewTarget(scpb.Target_DROP, &scpb.Table{TableID: 2}, nil) + mkNode := func(t *scpb.Target, s scpb.Status) *scpb.Node { + return &scpb.Node{Target: t, Status: s} + } + t1ABSENT := mkNode(t1, scpb.Status_ABSENT) + t2PUBLIC := mkNode(t2, scpb.Status_PUBLIC) + g, err := New(scpb.State{ + Nodes: []*scpb.Node{t1ABSENT, t2PUBLIC}, + }) + targetStr := func(target *scpb.Target) string { + switch target { + case t1: + return "t1" + case t2: + return "t2" + default: + panic("unexpected target") + } + } + nodeStr := func(n *scpb.Node) string { + if n == nil { + return "nil" + } + return fmt.Sprintf("%s:%s", targetStr(n.Target), n.Status.String()) + } + + require.NoError(t, err) + for _, tc := range []struct { + a, b *scpb.Node + less, eq bool + }{ + {a: nil, b: nil, less: false, eq: true}, + {a: t1ABSENT, b: nil, less: false, eq: false}, + {a: nil, b: t1ABSENT, less: true, eq: false}, + {a: t1ABSENT, b: t1ABSENT, less: false, eq: true}, + {a: t2PUBLIC, b: t1ABSENT, less: false, eq: false}, + {a: t1ABSENT, b: t2PUBLIC, less: true, eq: false}, + {a: t1ABSENT, b: mkNode(t1, scpb.Status_PUBLIC), less: true, eq: false}, + {a: mkNode(t1, scpb.Status_PUBLIC), b: t1ABSENT, less: false, eq: false}, + } { + t.Run(fmt.Sprintf("cmp(%s,%s)", nodeStr(tc.a), nodeStr(tc.b)), func(t *testing.T) { + less, eq := g.compareNodes(tc.a, tc.b) + require.Equal(t, tc.less, less, "less") + require.Equal(t, tc.eq, eq, "eq") + }) + } +} From 8e1a3135b5f8a90e3c0daf388e708f19d982f832 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:25:25 -0500 Subject: [PATCH 07/10] sql/schemachanger/scgraph: move edge defitions to a file Release note: None --- pkg/sql/schemachanger/scgraph/BUILD.bazel | 1 + pkg/sql/schemachanger/scgraph/edge.go | 70 ++++++++++++++++++ pkg/sql/schemachanger/scgraph/graph.go | 88 +++++------------------ 3 files changed, 88 insertions(+), 71 deletions(-) create mode 100644 pkg/sql/schemachanger/scgraph/edge.go diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index e229c3cee857..ccb05e325211 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -4,6 +4,7 @@ go_library( name = "scgraph", srcs = [ "dep_edge_tree.go", + "edge.go", "graph.go", "iteration.go", ], diff --git a/pkg/sql/schemachanger/scgraph/edge.go b/pkg/sql/schemachanger/scgraph/edge.go new file mode 100644 index 000000000000..5c11b1b2b018 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/edge.go @@ -0,0 +1,70 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scgraph + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" +) + +// Edge represents a relationship between two Nodes. +// +// TODO(ajwerner): Consider hiding Node pointers behind an interface to clarify +// mutability. +type Edge interface { + From() *scpb.Node + To() *scpb.Node +} + +// OpEdge represents an edge changing the state of a target with an op. +type OpEdge struct { + from, to *scpb.Node + op []scop.Op + typ scop.Type + revertible bool +} + +// From implements the Edge interface. +func (oe *OpEdge) From() *scpb.Node { return oe.from } + +// To implements the Edge interface. +func (oe *OpEdge) To() *scpb.Node { return oe.to } + +// Op returns the scop.Op for execution that is associated with the op edge. +func (oe *OpEdge) Op() []scop.Op { return oe.op } + +// Revertible returns if the dependency edge is revertible +func (oe *OpEdge) Revertible() bool { return oe.revertible } + +// Type returns the types of operations associated with this edge. +func (oe *OpEdge) Type() scop.Type { + return oe.typ +} + +// DepEdge represents a dependency between two nodes. A dependency +// implies that the To() node cannot be reached before the From() node. It +// can be reached concurrently. +type DepEdge struct { + from, to *scpb.Node + + // TODO(ajwerner): Deal with the possibility that multiple rules could + // generate the same edge. + rule string +} + +// From implements the Edge interface. +func (de *DepEdge) From() *scpb.Node { return de.from } + +// To implements the Edge interface. +func (de *DepEdge) To() *scpb.Node { return de.to } + +// Name returns the name of the rule which generated this edge. +func (de *DepEdge) Name() string { return de.rule } diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 20f88fb8f28a..0708291ec908 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -273,77 +273,6 @@ func (g *Graph) GetMetadataFromTarget(target *scpb.Target) scpb.ElementMetadata } } -// Edge represents a relationship between two Nodes. -// -// TODO(ajwerner): Consider hiding Node pointers behind an interface to clarify -// mutability. -type Edge interface { - From() *scpb.Node - To() *scpb.Node -} - -// OpEdge represents an edge changing the state of a target with an op. -type OpEdge struct { - from, to *scpb.Node - op []scop.Op - typ scop.Type - revertible bool -} - -// From implements the Edge interface. -func (oe *OpEdge) From() *scpb.Node { return oe.from } - -// To implements the Edge interface. -func (oe *OpEdge) To() *scpb.Node { return oe.to } - -// Op returns the scop.Op for execution that is associated with the op edge. -func (oe *OpEdge) Op() []scop.Op { return oe.op } - -// Revertible returns if the dependency edge is revertible -func (oe *OpEdge) Revertible() bool { return oe.revertible } - -// Type returns the types of operations associated with this edge. -func (oe *OpEdge) Type() scop.Type { - return oe.typ -} - -// DepEdge represents a dependency between two nodes. A dependency -// implies that the To() node cannot be reached before the From() node. It -// can be reached concurrently. -type DepEdge struct { - from, to *scpb.Node - - // TODO(ajwerner): Deal with the possibility that multiple rules could - // generate the same edge. - rule string -} - -// From implements the Edge interface. -func (de *DepEdge) From() *scpb.Node { return de.from } - -// To implements the Edge interface. -func (de *DepEdge) To() *scpb.Node { return de.to } - -// Name returns the name of the rule which generated this edge. -func (de *DepEdge) Name() string { return de.rule } - -// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. -func (g *Graph) compareNodes(a, b *scpb.Node) (less, eq bool) { - switch { - case a == b: - return false, true - case a == nil: - return true, false - case b == nil: - return false, false - case a.Target == b.Target: - return a.Status < b.Status, a.Status == b.Status - default: - aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] - return aIdx < bIdx, aIdx == bIdx - } -} - // GetNodeRanks fetches ranks of nodes in topological order. func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { defer func() { @@ -399,3 +328,20 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { } return rank, nil } + +// compareNodes compares two nodes in a graph. A nil nodes is the minimum value. +func (g *Graph) compareNodes(a, b *scpb.Node) (less, eq bool) { + switch { + case a == b: + return false, true + case a == nil: + return true, false + case b == nil: + return false, false + case a.Target == b.Target: + return a.Status < b.Status, a.Status == b.Status + default: + aIdx, bIdx := g.targetIdxMap[a.Target], g.targetIdxMap[b.Target] + return aIdx < bIdx, aIdx == bIdx + } +} From 2c7d4ab479d81f3e5cf4a50ba3d152fa468ca650 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 18:51:24 -0500 Subject: [PATCH 08/10] sql/schemachanger/scgraph: rename some fields Release note: None --- pkg/sql/schemachanger/scgraph/graph.go | 24 +++++++++++----------- pkg/sql/schemachanger/scgraph/iteration.go | 2 +- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 0708291ec908..86767c604eed 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -41,14 +41,14 @@ type Graph struct { // Maps a target to its index in targetNodes. targetIdxMap map[*scpb.Target]int - // nodeOpEdgesFrom maps a Node to an opEdge that proceeds + // opEdgesFrom maps a Node to an opEdge that proceeds // from it. A Node may have at most one opEdge from it. - nodeOpEdgesFrom map[*scpb.Node]*OpEdge + opEdgesFrom map[*scpb.Node]*OpEdge - // nodeDepEdgesFrom maps a Node from its dependencies. + // depEdgesFrom maps a Node from its dependencies. // A Node dependency is another target node which must be // reached before or concurrently with this node. - nodeDepEdgesFrom *depEdgeTree + depEdgesFrom *depEdgeTree // opToNode maps from an operation back to the // opEdge that generated it as an index. @@ -83,14 +83,14 @@ func New(initial scpb.State) (*Graph, error) { } g := Graph{ targetIdxMap: map[*scpb.Target]int{}, - nodeOpEdgesFrom: map[*scpb.Node]*OpEdge{}, + opEdgesFrom: map[*scpb.Node]*OpEdge{}, optimizedOutOpEdges: map[*OpEdge]bool{}, opToNode: map[scop.Op]*scpb.Node{}, entities: db, statements: initial.Statements, authorization: initial.Authorization, } - g.nodeDepEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) + g.depEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) for _, n := range initial.Nodes { if existing, ok := g.targetIdxMap[n.Target]; ok { return nil, errors.Errorf("invalid initial state contains duplicate target: %v and %v", n, initial.Nodes[existing]) @@ -118,8 +118,8 @@ func (g *Graph) ShallowClone() *Graph { authorization: g.authorization, targetNodes: g.targetNodes, targetIdxMap: g.targetIdxMap, - nodeOpEdgesFrom: g.nodeOpEdgesFrom, - nodeDepEdgesFrom: g.nodeDepEdgesFrom, + opEdgesFrom: g.opEdgesFrom, + depEdgesFrom: g.depEdgesFrom, opToNode: g.opToNode, edges: g.edges, entities: g.entities, @@ -177,7 +177,7 @@ var _ = (*Graph)(nil).containsTarget // GetOpEdgeFrom returns the unique outgoing op edge from the specified node, // if one exists. func (g *Graph) GetOpEdgeFrom(n *scpb.Node) (*OpEdge, bool) { - oe, ok := g.nodeOpEdgesFrom[n] + oe, ok := g.opEdgesFrom[n] return oe, ok } @@ -196,7 +196,7 @@ func (g *Graph) AddOpEdges( if oe.to, err = g.getOrCreateNode(t, to); err != nil { return err } - if existing, exists := g.nodeOpEdgesFrom[oe.from]; exists { + if existing, exists := g.opEdgesFrom[oe.from]; exists { return errors.Errorf("duplicate outbound op edge %v and %v", oe, existing) } @@ -211,7 +211,7 @@ func (g *Graph) AddOpEdges( } } oe.typ = typ - g.nodeOpEdgesFrom[oe.from] = oe + g.opEdgesFrom[oe.from] = oe // Store mapping from op to Edge for _, op := range ops { g.opToNode[op] = oe.To() @@ -243,7 +243,7 @@ func (g *Graph) AddDepEdge( return err } g.edges = append(g.edges, de) - g.nodeDepEdgesFrom.insert(de) + g.depEdgesFrom.insert(de) return nil } diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index 70346216ea97..cd15c4676599 100644 --- a/pkg/sql/schemachanger/scgraph/iteration.go +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -59,5 +59,5 @@ type DepEdgeIterator func(de *DepEdge) error // ForEachDepEdgeFrom iterates the dep edges in the graph. func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) (err error) { - return g.nodeDepEdgesFrom.iterateSourceNode(n, it) + return g.depEdgesFrom.iterateSourceNode(n, it) } From 05080aa35aca1ba4e105abd1c373997a5a9be773 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 29 Nov 2021 20:00:18 -0500 Subject: [PATCH 09/10] sql/schemachanger: remove cycle DepEdge rules, add SameStage kind This commit seeks to rectify an early mistake in the architecture of the declarative schema changer. In the original design, we knew we wanted certain transitions to happen in the same stage. In order to deal with that, we created rules that allowed for special types of cycles in dependencies to exist. This was a mistake. Instead, we replace this by a `Kind` property of `DepEdge`s which indicates whether the target pointed to merely needs to `HappenBefore` or whether it also needs to happen in the `SameStage`. This allows us to express exactly what we meant. This change also uncovered some broken cycles which never were intended to exist. The resultant plans generally look better. Release note: None --- pkg/sql/schemachanger/scgraph/BUILD.bazel | 8 + .../scgraph/depedgekind_string.go | 25 ++ pkg/sql/schemachanger/scgraph/edge.go | 23 ++ pkg/sql/schemachanger/scgraph/graph.go | 28 +-- pkg/sql/schemachanger/scgraph/graph_test.go | 10 +- pkg/sql/schemachanger/scgraph/iteration.go | 6 + pkg/sql/schemachanger/scplan/BUILD.bazel | 1 + .../schemachanger/scplan/deprules/registry.go | 13 +- .../schemachanger/scplan/deprules/rules.go | 76 ++---- .../scplan/deprules/testdata/rules | 81 ------- pkg/sql/schemachanger/scplan/plan.go | 44 +++- pkg/sql/schemachanger/scplan/plan_test.go | 2 + .../scplan/testdata/create_index | 14 ++ .../scplan/testdata/drop_database | 202 ++++++++++------ .../schemachanger/scplan/testdata/drop_schema | 170 +++++++++----- .../scplan/testdata/drop_sequence | 32 +-- .../schemachanger/scplan/testdata/drop_table | 46 ++-- .../schemachanger/scplan/testdata/drop_view | 112 +++++---- pkg/sql/schemachanger/testdata/drop | 217 +++++++----------- 19 files changed, 588 insertions(+), 522 deletions(-) create mode 100644 pkg/sql/schemachanger/scgraph/depedgekind_string.go diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index ccb05e325211..e2fdedb30990 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//build:STRINGER.bzl", "stringer") go_library( name = "scgraph", @@ -7,6 +8,7 @@ go_library( "edge.go", "graph.go", "iteration.go", + ":gen-depedgekind-stringer", # keep ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph", visibility = ["//visibility:public"], @@ -37,3 +39,9 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +stringer( + name = "gen-depedgekind-stringer", + src = "edge.go", + typ = "DepEdgeKind", +) diff --git a/pkg/sql/schemachanger/scgraph/depedgekind_string.go b/pkg/sql/schemachanger/scgraph/depedgekind_string.go new file mode 100644 index 000000000000..ecabe22091d5 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/depedgekind_string.go @@ -0,0 +1,25 @@ +// Code generated by "stringer"; DO NOT EDIT. + +package scgraph + +import "strconv" + +func _() { + // An "invalid array index" compiler error signifies that the constant values have changed. + // Re-run the stringer command to generate them again. + var x [1]struct{} + _ = x[HappensAfter-1] + _ = x[SameStage-2] +} + +const _DepEdgeKind_name = "HappensAfterSameStage" + +var _DepEdgeKind_index = [...]uint8{0, 12, 21} + +func (i DepEdgeKind) String() string { + i -= 1 + if i < 0 || i >= DepEdgeKind(len(_DepEdgeKind_index)-1) { + return "DepEdgeKind(" + strconv.FormatInt(int64(i+1), 10) + ")" + } + return _DepEdgeKind_name[_DepEdgeKind_index[i]:_DepEdgeKind_index[i+1]] +} diff --git a/pkg/sql/schemachanger/scgraph/edge.go b/pkg/sql/schemachanger/scgraph/edge.go index 5c11b1b2b018..5b53cf2c408c 100644 --- a/pkg/sql/schemachanger/scgraph/edge.go +++ b/pkg/sql/schemachanger/scgraph/edge.go @@ -49,11 +49,31 @@ func (oe *OpEdge) Type() scop.Type { return oe.typ } +// DepEdgeKind indicates the kind of constraint enforced by the edge. +type DepEdgeKind int + +//go:generate stringer -type DepEdgeKind + +const ( + _ DepEdgeKind = iota + + // HappensAfter indicates that the source (from) of the edge must not be + // entered until after the destination (to) has entered the state. It could + // be in the same stage, or it could be in a subsequent stage. + HappensAfter + + // SameStage indicates that the source (from) of the edge must + // not be entered until after the destination (to) has entered the state and + // that both nodes must enter the state in the same stage. + SameStage +) + // DepEdge represents a dependency between two nodes. A dependency // implies that the To() node cannot be reached before the From() node. It // can be reached concurrently. type DepEdge struct { from, to *scpb.Node + kind DepEdgeKind // TODO(ajwerner): Deal with the possibility that multiple rules could // generate the same edge. @@ -68,3 +88,6 @@ func (de *DepEdge) To() *scpb.Node { return de.to } // Name returns the name of the rule which generated this edge. func (de *DepEdge) Name() string { return de.rule } + +// Kind returns the kind of the DepEdge. +func (de *DepEdge) Kind() DepEdgeKind { return de.kind } diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 86767c604eed..f6a105b98ee5 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -50,6 +50,10 @@ type Graph struct { // reached before or concurrently with this node. depEdgesFrom *depEdgeTree + // sameStageDepEdgesTo maps a Node to the DepEdges with the + // SameStage kind incident upon the indexed node. + sameStageDepEdgesTo *depEdgeTree + // opToNode maps from an operation back to the // opEdge that generated it as an index. opToNode map[scop.Op]*scpb.Node @@ -91,6 +95,7 @@ func New(initial scpb.State) (*Graph, error) { authorization: initial.Authorization, } g.depEdgesFrom = newDepEdgeTree(fromTo, g.compareNodes) + g.sameStageDepEdgesTo = newDepEdgeTree(toFrom, g.compareNodes) for _, n := range initial.Nodes { if existing, ok := g.targetIdxMap[n.Target]; ok { return nil, errors.Errorf("invalid initial state contains duplicate target: %v and %v", n, initial.Nodes[existing]) @@ -120,6 +125,7 @@ func (g *Graph) ShallowClone() *Graph { targetIdxMap: g.targetIdxMap, opEdgesFrom: g.opEdgesFrom, depEdgesFrom: g.depEdgesFrom, + sameStageDepEdgesTo: g.sameStageDepEdgesTo, opToNode: g.opToNode, edges: g.edges, entities: g.entities, @@ -230,12 +236,13 @@ var _ = (*Graph)(nil).GetNodeFromOp // and statuses). func (g *Graph) AddDepEdge( rule string, + kind DepEdgeKind, fromTarget *scpb.Target, fromStatus scpb.Status, toTarget *scpb.Target, toStatus scpb.Status, ) (err error) { - de := &DepEdge{rule: rule} + de := &DepEdge{rule: rule, kind: kind} if de.from, err = g.getOrCreateNode(fromTarget, fromStatus); err != nil { return err } @@ -244,6 +251,9 @@ func (g *Graph) AddDepEdge( } g.edges = append(g.edges, de) g.depEdgesFrom.insert(de) + if de.Kind() == SameStage { + g.sameStageDepEdgesTo.insert(de) + } return nil } @@ -284,14 +294,6 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { err = rAsErr } }() - backCycleExists := func(n *scpb.Node, de *DepEdge) bool { - var foundBack bool - _ = g.ForEachDepEdgeFrom(de.To(), func(maybeBack *DepEdge) error { - foundBack = foundBack || maybeBack.To() == n - return nil - }) - return foundBack - } l := list.New() marks := make(map[*scpb.Node]bool) var visit func(n *scpb.Node) @@ -305,18 +307,12 @@ func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { } marks[n] = false _ = g.ForEachDepEdgeFrom(n, func(de *DepEdge) error { - // We want to eliminate cycles caused by swaps. In that - // case, we want to pretend that there is no edge from the - // add to the drop, and, in that way, the drop is ordered first. - if n.Direction == scpb.Target_ADD || !backCycleExists(n, de) { - visit(de.To()) - } + visit(de.To()) return nil }) marks[n] = true l.PushFront(n) } - _ = g.ForEachNode(func(n *scpb.Node) error { visit(n) return nil diff --git a/pkg/sql/schemachanger/scgraph/graph_test.go b/pkg/sql/schemachanger/scgraph/graph_test.go index 48df7e930c2b..2770fc27faa3 100644 --- a/pkg/sql/schemachanger/scgraph/graph_test.go +++ b/pkg/sql/schemachanger/scgraph/graph_test.go @@ -65,12 +65,9 @@ func TestGraphRanks(t *testing.T) { expectedRankErr: "graph is not a dag", }, - // We will set up the dependency graph to have a swap: - // 1) 0 (adding) depends on 1 (dropping) - // 2) 1 (dropping) depends on 0 (adding) - // 3) 2 (adding) depends on 0 (adding) + // We will set up the dependency graph to have a swap, which won't affect + // the fact that there's still a cycle. { - name: "dependency graph with a swap", addNode: []bool{true, false, true}, depEdges: []depEdge{ @@ -78,7 +75,7 @@ func TestGraphRanks(t *testing.T) { {1, 0}, {2, 0}, }, - expectedOrder: []int{1, 0, 2}, // We expect the drop to be ordered first. + expectedRankErr: "graph is not a dag", }, } @@ -133,6 +130,7 @@ func TestGraphRanks(t *testing.T) { for _, edge := range tc.depEdges { require.NoError(t, graph.AddDepEdge( fmt.Sprintf("%d to %d", edge.from, edge.to), + scgraph.HappensAfter, state.Nodes[edge.from].Target, scpb.Status_PUBLIC, state.Nodes[edge.to].Target, diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index cd15c4676599..19c9378ddc22 100644 --- a/pkg/sql/schemachanger/scgraph/iteration.go +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -61,3 +61,9 @@ type DepEdgeIterator func(de *DepEdge) error func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) (err error) { return g.depEdgesFrom.iterateSourceNode(n, it) } + +// ForEachSameStageDepEdgeTo iterates the dep edges in the graph of kind +// SameStage which point to the provided node. +func (g *Graph) ForEachSameStageDepEdgeTo(n *scpb.Node, it DepEdgeIterator) (err error) { + return g.sameStageDepEdgesTo.iterateSourceNode(n, it) +} diff --git a/pkg/sql/schemachanger/scplan/BUILD.bazel b/pkg/sql/schemachanger/scplan/BUILD.bazel index e626a7c3cc31..e70404c23568 100644 --- a/pkg/sql/schemachanger/scplan/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/sql/schemachanger/scplan/deprules", "//pkg/sql/schemachanger/scplan/opgen", "//pkg/sql/schemachanger/scplan/scopt", + "//pkg/sql/schemachanger/screl", "//pkg/util/iterutil", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/sql/schemachanger/scplan/deprules/registry.go b/pkg/sql/schemachanger/scplan/deprules/registry.go index ed5fb7d90c59..99366c66eac1 100644 --- a/pkg/sql/schemachanger/scplan/deprules/registry.go +++ b/pkg/sql/schemachanger/scplan/deprules/registry.go @@ -26,7 +26,7 @@ func Apply(g *scgraph.Graph) error { from := r.Var(dr.from).(*scpb.Node) to := r.Var(dr.to).(*scpb.Node) return g.AddDepEdge( - dr.name, from.Target, from.Status, to.Target, to.Status, + dr.name, dr.kind, from.Target, from.Status, to.Target, to.Status, ) }); err != nil { return err @@ -39,15 +39,16 @@ func Apply(g *scgraph.Graph) error { var depRules []rule type rule struct { - name string - from, to rel.Var - q *rel.Query - sameStage bool + name string + from, to rel.Var + q *rel.Query + kind scgraph.DepEdgeKind } -func register(ruleName string, from, to rel.Var, query *rel.Query) { +func register(ruleName string, edgeKind scgraph.DepEdgeKind, from, to rel.Var, query *rel.Query) { depRules = append(depRules, rule{ name: ruleName, + kind: edgeKind, from: from, to: to, q: query, diff --git a/pkg/sql/schemachanger/scplan/deprules/rules.go b/pkg/sql/schemachanger/scplan/deprules/rules.go index fe63c69511fe..40f2f928d33f 100644 --- a/pkg/sql/schemachanger/scplan/deprules/rules.go +++ b/pkg/sql/schemachanger/scplan/deprules/rules.go @@ -13,6 +13,7 @@ package deprules import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/errors" @@ -52,9 +53,9 @@ func init() { // are dropped before any children are dealt with. register( "parent dependencies", + scgraph.HappensAfter, parentNode, otherNode, screl.MustQuery( - parent.Type((*scpb.Database)(nil), (*scpb.Schema)(nil)), other.Type( (*scpb.Type)(nil), (*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), @@ -102,9 +103,9 @@ func init() { var id, status, direction rel.Var = "id", "index-status", "direction" register( "column depends on indexes", + scgraph.HappensAfter, columnNode, indexNode, screl.MustQuery( - status.In(deleteAndWriteOnly, public), direction.Eq(add), @@ -145,9 +146,9 @@ func init() { register( "index depends on column", + scgraph.HappensAfter, indexNode, columnNode, screl.MustQuery( - column.Type((*scpb.Column)(nil)), index.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -204,14 +205,10 @@ func init() { register( "primary index add depends on drop", + scgraph.SameStage, addNode, dropNode, primaryIndexReferenceEachOther, ) - register( - "primary index drop depends on add", - dropNode, addNode, - primaryIndexReferenceEachOther, - ) } func init() { @@ -222,6 +219,7 @@ func init() { register( "partitioning information needs the basic index as created", + scgraph.HappensAfter, partitioningNode, addNode, screl.MustQuery( addIdx.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -244,6 +242,7 @@ func init() { register( "index needs partitioning information to be filled", + scgraph.HappensAfter, partitioningNode, addNode, screl.MustQuery( addIdx.Type((*scpb.PrimaryIndex)(nil)), @@ -268,6 +267,7 @@ func init() { var id rel.Var = "id" register( "dependency needs relation/type as non-synthetically dropped", + scgraph.SameStage, depNode, relationNode, screl.MustQuery( @@ -293,52 +293,6 @@ func init() { screl.ReferencedDescID) } -func init() { - relationNeedsDepToBeRemoved := func(ruleName string, depTypes []interface{}, depDescIDMatch rel.Attr, swapped bool) { - // Before any parts of a relation can be dropped, the relation - // should exit the synthetic drop state. - relation, relationTarget, relationNode := targetNodeVars("relation") - dep, depTarget, depNode := targetNodeVars("dep") - var id rel.Var = "id" - firstNode, secondNode := relationNode, depNode - if swapped { - firstNode, secondNode = depNode, relationNode - } - register( - "relation/type needs dependency as dropped", - firstNode, secondNode, - screl.MustQuery( - - relation.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), (*scpb.Type)(nil)), - dep.Type(depTypes[0], depTypes[1:]...), - - id.Entities(screl.DescID, relation, dep), - - joinTargetNode(relation, relationTarget, relationNode, drop, absent), - joinTargetNode(dep, depTarget, depNode, drop, absent), - ), - ) - } - relationNeedsDepToBeRemoved("relation/type needs dependency as dropped", - []interface{}{(*scpb.DefaultExpression)(nil), (*scpb.RelationDependedOnBy)(nil), - (*scpb.SequenceOwnedBy)(nil), (*scpb.ForeignKey)(nil)}, - screl.DescID, - false /*swapped*/) - - relationNeedsDepToBeRemoved("relation/type (ref desc) needs dependency as dropped", - []interface{}{(*scpb.ForeignKeyBackReference)(nil), - (*scpb.ViewDependsOnType)(nil), (*scpb.DefaultExprTypeReference)(nil), - (*scpb.OnUpdateExprTypeReference)(nil), (*scpb.ComputedExprTypeReference)(nil), - (*scpb.ColumnTypeReference)(nil)}, - screl.ReferencedDescID, - false /*swapped*/) - - relationNeedsDepToBeRemoved("relation dependency clean up needs dependent relation to be dropped first", - []interface{}{(*scpb.RelationDependedOnBy)(nil)}, - screl.ReferencedDescID, - true /*swapped*/) -} - func init() { // Ensures that the name is drained first, only when // the descriptor is cleaned up. @@ -347,9 +301,9 @@ func init() { tabID := rel.Var("desc-id") register( "namespace needs descriptor to be dropped", + scgraph.HappensAfter, nsNode, depNode, screl.MustQuery( - ns.Type((*scpb.Namespace)(nil)), dep.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), (*scpb.Database)(nil), (*scpb.Schema)(nil)), @@ -365,9 +319,9 @@ func init() { // dropped. register( "descriptor can only be cleaned up once the name is drained", + scgraph.HappensAfter, depNode, nsNode, screl.MustQuery( - ns.Type((*scpb.Namespace)(nil)), dep.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil), (*scpb.Database)(nil), (*scpb.Schema)(nil)), @@ -388,6 +342,7 @@ func init() { register( "column name is assigned once the column is created", + scgraph.HappensAfter, columnNameNode, columnNode, screl.MustQuery( @@ -404,6 +359,7 @@ func init() { register( "column needs a name to be assigned", + scgraph.HappensAfter, columnNode, columnNameNode, screl.MustQuery( @@ -427,9 +383,9 @@ func init() { register( "index name is assigned once the index is created", + scgraph.HappensAfter, indexNameNode, indexNode, screl.MustQuery( - indexName.Type((*scpb.IndexName)(nil)), index.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -443,9 +399,9 @@ func init() { register( "index needs a name to be assigned", + scgraph.HappensAfter, indexNode, indexNameNode, screl.MustQuery( - indexName.Type((*scpb.IndexName)(nil)), index.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), @@ -466,7 +422,9 @@ func init() { typeID := rel.Var("type-id") tableID := rel.Var("table-id") - register("type ref drop is no-op if ref is being added", + register( + "type ref drop is no-op if ref is being added", + scgraph.HappensAfter, typeRefDropNode, typeRefDropNode, screl.MustQuery( typeRefDrop.Type((*scpb.DefaultExprTypeReference)(nil), (*scpb.ColumnTypeReference)(nil), diff --git a/pkg/sql/schemachanger/scplan/deprules/testdata/rules b/pkg/sql/schemachanger/scplan/deprules/testdata/rules index cc09123a72ed..fd61857d68c0 100644 --- a/pkg/sql/schemachanger/scplan/deprules/testdata/rules +++ b/pkg/sql/schemachanger/scplan/deprules/testdata/rules @@ -84,27 +84,6 @@ rules - $drop-idx-node[Target] = $drop-idx-target - $drop-idx-target[Direction] = DROP - $drop-idx-node[Status] = DELETE_AND_WRITE_ONLY -- name: primary index drop depends on add - from: drop-idx-node - to: add-idx-node - query: - - $add-idx[Type] = '*scpb.PrimaryIndex' - - $drop-idx[Type] = '*scpb.PrimaryIndex' - - $add-idx[DescID] = $id - - $drop-idx[DescID] = $id - - referenceEachOther(*scpb.PrimaryIndex, *scpb.PrimaryIndex)($add-idx, $drop-idx) - - $add-idx-target[Type] = '*scpb.Target' - - $add-idx-target[Element] = $add-idx - - $add-idx-node[Type] = '*scpb.Node' - - $add-idx-node[Target] = $add-idx-target - - $add-idx-target[Direction] = ADD - - $add-idx-node[Status] = PUBLIC - - $drop-idx-target[Type] = '*scpb.Target' - - $drop-idx-target[Element] = $drop-idx - - $drop-idx-node[Type] = '*scpb.Node' - - $drop-idx-node[Target] = $drop-idx-target - - $drop-idx-target[Direction] = DROP - - $drop-idx-node[Status] = DELETE_AND_WRITE_ONLY - name: partitioning information needs the basic index as created from: partitioning-node to: add-idx-node @@ -189,66 +168,6 @@ rules - $dep-node[Target] = $dep-target - $dep-target[Direction] = DROP - $dep-node[Status] = ABSENT -- name: relation/type needs dependency as dropped - from: relation-node - to: dep-node - query: - - $relation[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence', '*scpb.Type'] - - $dep[Type] IN ['*scpb.DefaultExpression', '*scpb.RelationDependedOnBy', '*scpb.SequenceOwnedBy', '*scpb.ForeignKey'] - - $relation[DescID] = $id - - $dep[DescID] = $id - - $relation-target[Type] = '*scpb.Target' - - $relation-target[Element] = $relation - - $relation-node[Type] = '*scpb.Node' - - $relation-node[Target] = $relation-target - - $relation-target[Direction] = DROP - - $relation-node[Status] = ABSENT - - $dep-target[Type] = '*scpb.Target' - - $dep-target[Element] = $dep - - $dep-node[Type] = '*scpb.Node' - - $dep-node[Target] = $dep-target - - $dep-target[Direction] = DROP - - $dep-node[Status] = ABSENT -- name: relation/type needs dependency as dropped - from: relation-node - to: dep-node - query: - - $relation[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence', '*scpb.Type'] - - $dep[Type] IN ['*scpb.ForeignKeyBackReference', '*scpb.ViewDependsOnType', '*scpb.DefaultExprTypeReference', '*scpb.OnUpdateExprTypeReference', '*scpb.ComputedExprTypeReference', '*scpb.ColumnTypeReference'] - - $relation[DescID] = $id - - $dep[DescID] = $id - - $relation-target[Type] = '*scpb.Target' - - $relation-target[Element] = $relation - - $relation-node[Type] = '*scpb.Node' - - $relation-node[Target] = $relation-target - - $relation-target[Direction] = DROP - - $relation-node[Status] = ABSENT - - $dep-target[Type] = '*scpb.Target' - - $dep-target[Element] = $dep - - $dep-node[Type] = '*scpb.Node' - - $dep-node[Target] = $dep-target - - $dep-target[Direction] = DROP - - $dep-node[Status] = ABSENT -- name: relation/type needs dependency as dropped - from: dep-node - to: relation-node - query: - - $relation[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence', '*scpb.Type'] - - $dep[Type] = '*scpb.RelationDependedOnBy' - - $relation[DescID] = $id - - $dep[DescID] = $id - - $relation-target[Type] = '*scpb.Target' - - $relation-target[Element] = $relation - - $relation-node[Type] = '*scpb.Node' - - $relation-node[Target] = $relation-target - - $relation-target[Direction] = DROP - - $relation-node[Status] = ABSENT - - $dep-target[Type] = '*scpb.Target' - - $dep-target[Element] = $dep - - $dep-node[Type] = '*scpb.Node' - - $dep-node[Target] = $dep-target - - $dep-target[Direction] = DROP - - $dep-node[Status] = ABSENT - name: namespace needs descriptor to be dropped from: namespace-node to: dep-node diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index 6996972708a2..d1251d7ee804 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/deprules" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/opgen" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/scopt" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/errors" ) @@ -111,24 +112,59 @@ func buildStages(init scpb.State, g *scgraph.Graph, params Params) []Stage { cur := init fulfilled := map[*scpb.Node]struct{}{} filterUnsatisfiedEdgesStep := func(edges []*scgraph.OpEdge) ([]*scgraph.OpEdge, bool) { - candidates := make(map[*scpb.Node]struct{}) + candidates := make(map[*scpb.Node]*scgraph.OpEdge) for _, e := range edges { - candidates[e.To()] = struct{}{} + candidates[e.To()] = e } // Check to see if the current set of edges will have their dependencies met // if they are all run. Any which will not must be pruned. This greedy // algorithm works, but a justification is in order. failed := map[*scgraph.OpEdge]struct{}{} for _, e := range edges { - _ = g.ForEachDepEdgeFrom(e.To(), func(de *scgraph.DepEdge) error { + if err := g.ForEachDepEdgeFrom(e.To(), func(de *scgraph.DepEdge) error { _, isFulfilled := fulfilled[de.To()] _, isCandidate := candidates[de.To()] + if de.Kind() == scgraph.SameStage && isFulfilled { + // This is bad, we have a happens-after relationship, and it has + // already happened. + return errors.AssertionFailedf("failed to satisfy %v->%v (%s) dependency", + screl.NodeString(de.From()), screl.NodeString(de.To()), de.Name()) + } if isFulfilled || isCandidate { return nil } failed[e] = struct{}{} return iterutil.StopIteration() - }) + }); err != nil { + panic(err) + } + } + // Ensure that all SameStage DepEdges are met appropriately. + for _, e := range edges { + if err := g.ForEachSameStageDepEdgeTo(e.To(), func(de *scgraph.DepEdge) error { + if _, isFulfilled := fulfilled[de.From()]; isFulfilled { + // This is bad, we have a happens-after relationship, and it has + // already happened. + return errors.AssertionFailedf("failed to satisfy %v->%v (%s) dependency", + screl.NodeString(de.From()), screl.NodeString(de.To()), de.Name()) + } + fromCandidate, fromIsCandidate := candidates[de.From()] + if !fromIsCandidate { + failed[e] = struct{}{} + return iterutil.StopIteration() + } + _, fromIsFailed := failed[fromCandidate] + if fromIsFailed { + failed[e] = struct{}{} + return iterutil.StopIteration() + } + if _, eIsFailed := failed[e]; eIsFailed { + failed[fromCandidate] = struct{}{} + } + return nil + }); err != nil { + panic(err) + } } if len(failed) == 0 { return edges, true diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go index 5faadcec6406..2a173c5f677a 100644 --- a/pkg/sql/schemachanger/scplan/plan_test.go +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -156,6 +156,8 @@ func marshalDeps(t *testing.T, plan *scplan.Plan) string { screl.ElementString(de.From().Element()), de.From().Status) fmt.Fprintf(&deps, " to: [%s, %s]\n", screl.ElementString(de.To().Element()), de.To().Status) + fmt.Fprintf(&deps, " kind: %s\n", de.Kind()) + fmt.Fprintf(&deps, " rule: %s\n", de.Name()) sortedDeps = append(sortedDeps, deps.String()) return nil }) diff --git a/pkg/sql/schemachanger/scplan/testdata/create_index b/pkg/sql/schemachanger/scplan/testdata/create_index index bb3330f00415..8dfcdd3dffb3 100644 --- a/pkg/sql/schemachanger/scplan/testdata/create_index +++ b/pkg/sql/schemachanger/scplan/testdata/create_index @@ -60,8 +60,12 @@ CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ---- - from: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: index name is assigned once the index is created - from: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_AND_WRITE_ONLY] to: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] + kind: HappensAfter + rule: index needs a name to be assigned ops CREATE INVERTED INDEX concurrently id1 on defaultdb.t1(id, name) storing (money) @@ -123,8 +127,12 @@ CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ---- - from: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: index name is assigned once the index is created - from: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_AND_WRITE_ONLY] to: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] + kind: HappensAfter + rule: index needs a name to be assigned ops CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (id) ( @@ -199,7 +207,13 @@ CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (i ---- - from: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: index name is assigned once the index is created - from: [Partitioning:{DescID: 52, IndexID: 2}, PUBLIC] to: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_ONLY] + kind: HappensAfter + rule: partitioning information needs the basic index as created - from: [SecondaryIndex:{DescID: 52, IndexID: 2}, DELETE_AND_WRITE_ONLY] to: [IndexName:{DescID: 52, IndexID: 2, Name: id1}, PUBLIC] + kind: HappensAfter + rule: index needs a name to be assigned diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_database b/pkg/sql/schemachanger/scplan/testdata/drop_database index 83d121ba897d..8f0b809a6e20 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_database +++ b/pkg/sql/schemachanger/scplan/testdata/drop_database @@ -133,6 +133,7 @@ Stage 1 Stage 2 (non-revertible) transitions: [Sequence:{DescID: 54}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 57}, TXN_DROPPED, DROP] -> DROPPED [Column:{DescID: 57, ColumnID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 57, ColumnID: 2}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY @@ -140,16 +141,23 @@ Stage 2 (non-revertible) [DefaultExpression:{DescID: 57, ColumnID: 3}, PUBLIC, DROP] -> ABSENT [PrimaryIndex:{DescID: 57, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Sequence:{DescID: 55}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 56}, TXN_DROPPED, DROP] -> DROPPED [Column:{DescID: 56, ColumnID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 56, ColumnID: 2}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 56, ColumnID: 3}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [DefaultExpression:{DescID: 56, ColumnID: 3}, PUBLIC, DROP] -> ABSENT [PrimaryIndex:{DescID: 56, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 58}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 60}, TXN_DROPPED, DROP] -> DROPPED [View:{DescID: 61}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, PUBLIC, DROP] -> ABSENT [View:{DescID: 64}, TXN_DROPPED, DROP] -> DROPPED [ViewDependsOnType:{DescID: 64, ReferencedDescID: 62}, PUBLIC, DROP] -> ABSENT [ViewDependsOnType:{DescID: 64, ReferencedDescID: 63}, PUBLIC, DROP] -> ABSENT @@ -162,6 +170,9 @@ Stage 2 (non-revertible) DescID: 54 *scop.MarkDescriptorAsDropped DescID: 57 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 54 *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 57 @@ -171,6 +182,9 @@ Stage 2 (non-revertible) DescID: 55 *scop.MarkDescriptorAsDropped DescID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 56 + TableID: 55 *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 56 @@ -178,14 +192,32 @@ Stage 2 (non-revertible) TableID: 56 *scop.MarkDescriptorAsDropped DescID: 58 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 58 + TableID: 56 *scop.MarkDescriptorAsDropped DescID: 59 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 59 + TableID: 58 *scop.MarkDescriptorAsDropped DescID: 60 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 60 + TableID: 58 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 60 + TableID: 59 *scop.MarkDescriptorAsDropped DescID: 61 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 61 + TableID: 59 *scop.MarkDescriptorAsDropped DescID: 64 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 64 + TableID: 61 *scop.MarkDescriptorAsDropped DescID: 62 *scop.RemoveTypeBackRef @@ -203,29 +235,21 @@ Stage 2 (non-revertible) Stage 3 (non-revertible) transitions: [Sequence:{DescID: 54}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 57}, DROPPED, DROP] -> ABSENT [Column:{DescID: 57, ColumnID: 1}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 57, ColumnID: 2}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 57, ColumnID: 3}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 57, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT [Sequence:{DescID: 55}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 56}, DROPPED, DROP] -> ABSENT [Column:{DescID: 56, ColumnID: 1}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 56, ColumnID: 2}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 56, ColumnID: 3}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 56, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 58}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 60}, DROPPED, DROP] -> ABSENT [View:{DescID: 61}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, PUBLIC, DROP] -> ABSENT [View:{DescID: 64}, DROPPED, DROP] -> ABSENT [Type:{DescID: 62}, DROPPED, DROP] -> ABSENT [Type:{DescID: 63}, DROPPED, DROP] -> ABSENT @@ -246,9 +270,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 54 *scop.LogEvent DescID: 57 Direction: 2 @@ -277,9 +298,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 55 *scop.LogEvent DescID: 56 Direction: 2 @@ -294,9 +312,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 58 - TableID: 56 *scop.LogEvent DescID: 58 Direction: 2 @@ -311,12 +326,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 58 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 59 - TableID: 58 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 60 - TableID: 58 *scop.LogEvent DescID: 59 Direction: 2 @@ -331,12 +340,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 59 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 60 - TableID: 59 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 61 - TableID: 59 *scop.LogEvent DescID: 60 Direction: 2 @@ -365,9 +368,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 61 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 64 - TableID: 61 *scop.LogEvent DescID: 64 Direction: 2 @@ -434,139 +434,197 @@ DROP DATABASE db1 CASCADE ---- - from: [Database:{DescID: 52}, ABSENT] to: [Sequence:{DescID: 54}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Database:{DescID: 52}, ABSENT] to: [Table:{DescID: 57}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [DefaultExpression:{DescID: 56, ColumnID: 3}, ABSENT] to: [Table:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [DefaultExpression:{DescID: 57, ColumnID: 3}, ABSENT] to: [Table:{DescID: 57}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Namespace:{DescID: 54, Name: sq1}, ABSENT] to: [Sequence:{DescID: 54}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 55, Name: sq1}, ABSENT] to: [Sequence:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 56, Name: t1}, ABSENT] to: [Table:{DescID: 56}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 57, Name: t1}, ABSENT] to: [Table:{DescID: 57}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 58, Name: v1}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 59, Name: v2}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 60, Name: v3}, ABSENT] to: [View:{DescID: 60}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 61, Name: v4}, ABSENT] to: [View:{DescID: 61}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 64, Name: v5}, ABSENT] to: [View:{DescID: 64}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] - to: [Sequence:{DescID: 54}, ABSENT] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] to: [Sequence:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] to: [Table:{DescID: 57}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] - to: [Sequence:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [Sequence:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [Table:{DescID: 56}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] - to: [Table:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [Table:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [View:{DescID: 58}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] - to: [View:{DescID: 58}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 59}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] - to: [View:{DescID: 58}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 60}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] - to: [View:{DescID: 59}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] to: [View:{DescID: 60}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] - to: [View:{DescID: 59}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 61}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] - to: [View:{DescID: 61}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] to: [View:{DescID: 61}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] to: [View:{DescID: 64}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Schema:{DescID: 53}, ABSENT] to: [Sequence:{DescID: 55}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [Table:{DescID: 56}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [Type:{DescID: 62}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [Type:{DescID: 63}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 58}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 59}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 60}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 61}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 53}, ABSENT] to: [View:{DescID: 64}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Sequence:{DescID: 54}, ABSENT] to: [Namespace:{DescID: 54, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 57}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Sequence:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] -- from: [Table:{DescID: 56}, ABSENT] - to: [DefaultExpression:{DescID: 56, ColumnID: 3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Table:{DescID: 56}, ABSENT] to: [Namespace:{DescID: 56, Name: t1}, ABSENT] -- from: [Table:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] -- from: [Table:{DescID: 57}, ABSENT] - to: [DefaultExpression:{DescID: 57, ColumnID: 3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Table:{DescID: 57}, ABSENT] to: [Namespace:{DescID: 57, Name: t1}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 58}, ABSENT] to: [Namespace:{DescID: 58, Name: v1}, ABSENT] -- from: [View:{DescID: 58}, ABSENT] - to: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 59}, ABSENT] -- from: [View:{DescID: 58}, ABSENT] - to: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 60}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 59}, ABSENT] to: [Namespace:{DescID: 59, Name: v2}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 60}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [RelationDependedOnBy:{DescID: 59, ReferencedDescID: 61}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 60}, ABSENT] to: [Namespace:{DescID: 60, Name: v3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 61}, ABSENT] to: [Namespace:{DescID: 61, Name: v4}, ABSENT] -- from: [View:{DescID: 61}, ABSENT] - to: [RelationDependedOnBy:{DescID: 61, ReferencedDescID: 64}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 64}, ABSENT] to: [Namespace:{DescID: 64, Name: v5}, ABSENT] -- from: [View:{DescID: 64}, ABSENT] - to: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 62}, ABSENT] -- from: [View:{DescID: 64}, ABSENT] - to: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 63}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 62}, ABSENT] to: [Type:{DescID: 62}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ViewDependsOnType:{DescID: 64, ReferencedDescID: 63}, ABSENT] to: [Type:{DescID: 63}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_schema b/pkg/sql/schemachanger/scplan/testdata/drop_schema index a46cfd5bf1d8..714bda07cb90 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_schema +++ b/pkg/sql/schemachanger/scplan/testdata/drop_schema @@ -39,118 +39,164 @@ DROP SCHEMA defaultdb.SC1 CASCADE ---- - from: [DefaultExpression:{DescID: 54, ColumnID: 3}, ABSENT] to: [Table:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Namespace:{DescID: 53, Name: sq1}, ABSENT] to: [Sequence:{DescID: 53}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 54, Name: t1}, ABSENT] to: [Table:{DescID: 54}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 55, Name: v1}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 56, Name: v2}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 57, Name: v3}, ABSENT] to: [View:{DescID: 57}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 58, Name: v4}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 61, Name: v5}, ABSENT] to: [View:{DescID: 61}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] - to: [Sequence:{DescID: 53}, ABSENT] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [Sequence:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [Table:{DescID: 54}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] - to: [Table:{DescID: 54}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [Table:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] - to: [View:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 56}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] - to: [View:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 57}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] - to: [View:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 57}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] - to: [View:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] to: [View:{DescID: 58}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] - to: [View:{DescID: 58}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 58}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] to: [View:{DescID: 61}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Schema:{DescID: 52}, ABSENT] to: [Sequence:{DescID: 53}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [Table:{DescID: 54}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [Type:{DescID: 59}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [Type:{DescID: 60}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 55}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 56}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 57}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 58}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Schema:{DescID: 52}, ABSENT] to: [View:{DescID: 61}, ABSENT] + kind: HappensAfter + rule: parent dependencies - from: [Sequence:{DescID: 53}, ABSENT] to: [Namespace:{DescID: 53, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 53}, ABSENT] - to: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] -- from: [Table:{DescID: 54}, ABSENT] - to: [DefaultExpression:{DescID: 54, ColumnID: 3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [Table:{DescID: 54}, ABSENT] to: [Namespace:{DescID: 54, Name: t1}, ABSENT] -- from: [Table:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: v1}, ABSENT] -- from: [View:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, ABSENT] -- from: [View:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 56}, ABSENT] to: [Namespace:{DescID: 56, Name: v2}, ABSENT] -- from: [View:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, ABSENT] -- from: [View:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 57}, ABSENT] to: [Namespace:{DescID: 57, Name: v3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 58}, ABSENT] to: [Namespace:{DescID: 58, Name: v4}, ABSENT] -- from: [View:{DescID: 58}, ABSENT] - to: [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 61}, ABSENT] to: [Namespace:{DescID: 61, Name: v5}, ABSENT] -- from: [View:{DescID: 61}, ABSENT] - to: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 59}, ABSENT] -- from: [View:{DescID: 61}, ABSENT] - to: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 60}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 59}, ABSENT] to: [Type:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ViewDependsOnType:{DescID: 61, ReferencedDescID: 60}, ABSENT] to: [Type:{DescID: 60}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped ops DROP SCHEMA defaultdb.SC1 CASCADE @@ -219,16 +265,23 @@ Stage 1 Stage 2 (non-revertible) transitions: [Sequence:{DescID: 53}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 54}, TXN_DROPPED, DROP] -> DROPPED [Column:{DescID: 54, ColumnID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 54, ColumnID: 2}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [Column:{DescID: 54, ColumnID: 3}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [DefaultExpression:{DescID: 54, ColumnID: 3}, PUBLIC, DROP] -> ABSENT [PrimaryIndex:{DescID: 54, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 56}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, TXN_DROPPED, DROP] -> DROPPED [View:{DescID: 58}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 61}, TXN_DROPPED, DROP] -> DROPPED [ViewDependsOnType:{DescID: 61, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT [ViewDependsOnType:{DescID: 61, ReferencedDescID: 60}, PUBLIC, DROP] -> ABSENT @@ -240,6 +293,9 @@ Stage 2 (non-revertible) DescID: 53 *scop.MarkDescriptorAsDropped DescID: 54 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 54 + TableID: 53 *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 54 @@ -247,14 +303,32 @@ Stage 2 (non-revertible) TableID: 54 *scop.MarkDescriptorAsDropped DescID: 55 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 55 + TableID: 54 *scop.MarkDescriptorAsDropped DescID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 56 + TableID: 55 *scop.MarkDescriptorAsDropped DescID: 57 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 55 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 56 *scop.MarkDescriptorAsDropped DescID: 58 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 58 + TableID: 56 *scop.MarkDescriptorAsDropped DescID: 61 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 61 + TableID: 58 *scop.MarkDescriptorAsDropped DescID: 59 *scop.RemoveTypeBackRef @@ -270,22 +344,15 @@ Stage 2 (non-revertible) Stage 3 (non-revertible) transitions: [Sequence:{DescID: 53}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT [Table:{DescID: 54}, DROPPED, DROP] -> ABSENT [Column:{DescID: 54, ColumnID: 1}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 54, ColumnID: 2}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 54, ColumnID: 3}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 54, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 56}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, DROPPED, DROP] -> ABSENT [View:{DescID: 58}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 58, ReferencedDescID: 61}, PUBLIC, DROP] -> ABSENT [View:{DescID: 61}, DROPPED, DROP] -> ABSENT [Type:{DescID: 59}, DROPPED, DROP] -> ABSENT [Type:{DescID: 60}, DROPPED, DROP] -> ABSENT @@ -305,9 +372,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 53 *scop.LogEvent DescID: 54 Direction: 2 @@ -322,9 +386,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 55 - TableID: 54 *scop.LogEvent DescID: 55 Direction: 2 @@ -339,12 +400,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 55 *scop.LogEvent DescID: 56 Direction: 2 @@ -359,12 +414,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 58 - TableID: 56 *scop.LogEvent DescID: 57 Direction: 2 @@ -393,9 +442,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 58 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 61 - TableID: 58 *scop.LogEvent DescID: 61 Direction: 2 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence index ef795c98acce..5eefacf67c85 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -68,14 +68,20 @@ Stage 1 Stage 2 (non-revertible) transitions: [Sequence:{DescID: 52}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT ops: *scop.MarkDescriptorAsDropped DescID: 52 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 53 + TableID: 52 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 54 + TableID: 52 Stage 3 (non-revertible) transitions: [Sequence:{DescID: 52}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT ops: *scop.LogEvent DescID: 52 @@ -91,12 +97,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 52 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 53 - TableID: 52 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 52 deps @@ -104,17 +104,17 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE ---- - from: [Namespace:{DescID: 52, Name: sq1}, ABSENT] to: [Sequence:{DescID: 52}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] - to: [Sequence:{DescID: 52}, ABSENT] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] to: [Sequence:{DescID: 52}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, ABSENT] - to: [Sequence:{DescID: 52}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, ABSENT] to: [Sequence:{DescID: 52}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Sequence:{DescID: 52}, ABSENT] to: [Namespace:{DescID: 52, Name: sq1}, ABSENT] -- from: [Sequence:{DescID: 52}, ABSENT] - to: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] -- from: [Sequence:{DescID: 52}, ABSENT] - to: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 54}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index d7ed733b27fd..406f56980162 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -77,6 +77,7 @@ Stage 2 (non-revertible) [PrimaryIndex:{DescID: 55, IndexID: 1}, DELETE_AND_WRITE_ONLY, DROP] -> DELETE_ONLY [ForeignKey:{DescID: 55, ReferencedDescID: 52, Name: fk_customers}, PUBLIC, DROP] -> ABSENT [ForeignKey:{DescID: 55, ReferencedDescID: 53, Name: fk_orders}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, TXN_DROPPED, DROP] -> DROPPED ops: *scop.MarkDescriptorAsDropped @@ -86,18 +87,21 @@ Stage 2 (non-revertible) TableID: 55 *scop.UpdateRelationDeps TableID: 55 + *scop.RemoveSequenceOwnedBy + TableID: 56 *scop.RemoveColumnDefaultExpression ColumnID: 5 TableID: 55 *scop.UpdateRelationDeps TableID: 55 - *scop.RemoveSequenceOwnedBy - TableID: 56 *scop.RemoveRelationDependedOnBy DependedOnBy: 55 TableID: 54 *scop.MarkDescriptorAsDropped DescID: 57 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 55 Stage 3 (non-revertible) transitions: [Table:{DescID: 55}, DROPPED, DROP] -> ABSENT @@ -107,7 +111,6 @@ Stage 3 (non-revertible) [Column:{DescID: 55, ColumnID: 4}, DELETE_ONLY, DROP] -> ABSENT [Column:{DescID: 55, ColumnID: 5}, DELETE_ONLY, DROP] -> ABSENT [PrimaryIndex:{DescID: 55, IndexID: 1}, DELETE_ONLY, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [View:{DescID: 57}, DROPPED, DROP] -> ABSENT ops: *scop.LogEvent @@ -124,9 +127,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 55 *scop.LogEvent DescID: 57 Direction: 2 @@ -147,35 +147,45 @@ DROP TABLE defaultdb.shipments CASCADE; ---- - from: [DefaultExpression:{DescID: 55, ColumnID: 1}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [DefaultExpression:{DescID: 55, ColumnID: 5}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ForeignKey:{DescID: 55, ReferencedDescID: 52, Name: fk_customers}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [ForeignKey:{DescID: 55, ReferencedDescID: 53, Name: fk_orders}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Namespace:{DescID: 55, Name: shipments}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 57, Name: v1}, ABSENT] to: [View:{DescID: 57}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [Table:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] - to: [Table:{DescID: 55}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [Table:{DescID: 55}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] to: [View:{DescID: 57}, DROPPED] -- from: [Table:{DescID: 55}, ABSENT] - to: [DefaultExpression:{DescID: 55, ColumnID: 1}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [DefaultExpression:{DescID: 55, ColumnID: 5}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [ForeignKey:{DescID: 55, ReferencedDescID: 52, Name: fk_customers}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [ForeignKey:{DescID: 55, ReferencedDescID: 53, Name: fk_orders}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [Table:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: shipments}, ABSENT] -- from: [Table:{DescID: 55}, ABSENT] - to: [RelationDependedOnBy:{DescID: 55, ReferencedDescID: 57}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 57}, ABSENT] to: [Namespace:{DescID: 57, Name: v1}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_view b/pkg/sql/schemachanger/scplan/testdata/drop_view index 7763706efd3c..8ff043a25b52 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_view +++ b/pkg/sql/schemachanger/scplan/testdata/drop_view @@ -55,10 +55,16 @@ DROP VIEW defaultdb.v1 ---- - from: [Namespace:{DescID: 53, Name: v1}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [View:{DescID: 53}, ABSENT] to: [Namespace:{DescID: 53, Name: v1}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained create-view CREATE VIEW defaultdb.v2 AS (SELECT name AS n1, name AS n2 FROM v1) @@ -123,9 +129,14 @@ Stage 2 (non-revertible) transitions: [View:{DescID: 53}, TXN_DROPPED, DROP] -> DROPPED [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 54}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT + [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, TXN_DROPPED, DROP] -> DROPPED [View:{DescID: 56}, TXN_DROPPED, DROP] -> DROPPED + [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, TXN_DROPPED, DROP] -> DROPPED [ViewDependsOnType:{DescID: 59, ReferencedDescID: 57}, PUBLIC, DROP] -> ABSENT [ViewDependsOnType:{DescID: 59, ReferencedDescID: 58}, PUBLIC, DROP] -> ABSENT @@ -137,12 +148,27 @@ Stage 2 (non-revertible) TableID: 52 *scop.MarkDescriptorAsDropped DescID: 54 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 54 + TableID: 53 *scop.MarkDescriptorAsDropped DescID: 55 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 55 + TableID: 53 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 55 + TableID: 54 *scop.MarkDescriptorAsDropped DescID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 56 + TableID: 54 *scop.MarkDescriptorAsDropped DescID: 59 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 59 + TableID: 56 *scop.RemoveTypeBackRef DescID: 59 TypeID: 57 @@ -152,14 +178,9 @@ Stage 2 (non-revertible) Stage 3 (non-revertible) transitions: [View:{DescID: 53}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT [View:{DescID: 54}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, PUBLIC, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, PUBLIC, DROP] -> ABSENT [View:{DescID: 55}, DROPPED, DROP] -> ABSENT [View:{DescID: 56}, DROPPED, DROP] -> ABSENT - [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, PUBLIC, DROP] -> ABSENT [View:{DescID: 59}, DROPPED, DROP] -> ABSENT ops: *scop.LogEvent @@ -176,12 +197,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 55 - TableID: 53 *scop.LogEvent DescID: 54 Direction: 2 @@ -196,12 +211,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 55 - TableID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 54 *scop.LogEvent DescID: 55 Direction: 2 @@ -230,9 +239,6 @@ Stage 3 (non-revertible) Username: root *scop.CreateGcJobForDescriptor DescID: 56 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 59 - TableID: 56 *scop.LogEvent DescID: 59 Direction: 2 @@ -253,67 +259,85 @@ DROP VIEW defaultdb.v1 CASCADE ---- - from: [Namespace:{DescID: 53, Name: v1}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 54, Name: v2}, ABSENT] to: [View:{DescID: 54}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 55, Name: v3}, ABSENT] to: [View:{DescID: 55}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 56, Name: v4}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [Namespace:{DescID: 59, Name: v5}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: HappensAfter + rule: namespace needs descriptor to be dropped - from: [RelationDependedOnBy:{DescID: 52, ReferencedDescID: 53}, ABSENT] to: [View:{DescID: 53}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] - to: [View:{DescID: 53}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] to: [View:{DescID: 54}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] - to: [View:{DescID: 53}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 53}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] - to: [View:{DescID: 54}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] to: [View:{DescID: 55}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] - to: [View:{DescID: 54}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 54}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] to: [View:{DescID: 56}, DROPPED] -- from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] - to: [View:{DescID: 56}, ABSENT] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 56}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] to: [View:{DescID: 59}, DROPPED] + kind: SameStage + rule: dependency needs relation/type as non-synthetically dropped - from: [View:{DescID: 53}, ABSENT] to: [Namespace:{DescID: 53, Name: v1}, ABSENT] -- from: [View:{DescID: 53}, ABSENT] - to: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 54}, ABSENT] -- from: [View:{DescID: 53}, ABSENT] - to: [RelationDependedOnBy:{DescID: 53, ReferencedDescID: 55}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 54}, ABSENT] to: [Namespace:{DescID: 54, Name: v2}, ABSENT] -- from: [View:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 55}, ABSENT] -- from: [View:{DescID: 54}, ABSENT] - to: [RelationDependedOnBy:{DescID: 54, ReferencedDescID: 56}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 55}, ABSENT] to: [Namespace:{DescID: 55, Name: v3}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 56}, ABSENT] to: [Namespace:{DescID: 56, Name: v4}, ABSENT] -- from: [View:{DescID: 56}, ABSENT] - to: [RelationDependedOnBy:{DescID: 56, ReferencedDescID: 59}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained - from: [View:{DescID: 59}, ABSENT] to: [Namespace:{DescID: 59, Name: v5}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [ViewDependsOnType:{DescID: 59, ReferencedDescID: 57}, ABSENT] -- from: [View:{DescID: 59}, ABSENT] - to: [ViewDependsOnType:{DescID: 59, ReferencedDescID: 58}, ABSENT] + kind: HappensAfter + rule: descriptor can only be cleaned up once the name is drained diff --git a/pkg/sql/schemachanger/testdata/drop b/pkg/sql/schemachanger/testdata/drop index 8c9d19a0519a..ce99df34fdfa 100644 --- a/pkg/sql/schemachanger/testdata/drop +++ b/pkg/sql/schemachanger/testdata/drop @@ -213,7 +213,7 @@ delete object namespace entry {58 59 v2} -> 65 delete object namespace entry {58 59 v3} -> 66 delete object namespace entry {58 59 v4} -> 67 delete object namespace entry {58 59 v5} -> 70 -## stage 2 in PreCommitPhase: 19 MutationType ops +## stage 2 in PreCommitPhase: 27 MutationType ops upsert descriptor #58 ... sc1: @@ -229,6 +229,16 @@ upsert descriptor #59 + state: DROP + version: "2" upsert descriptor #60 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - byId: true + - columnIds: + - - 3 + - id: 63 + families: + - columnIds: ... sequenceOwner: {} start: "1" @@ -237,6 +247,16 @@ upsert descriptor #60 - version: "2" + version: "3" upsert descriptor #61 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - byId: true + - columnIds: + - - 3 + - id: 62 + families: + - columnIds: ... sequenceOwner: {} start: "1" @@ -301,6 +321,20 @@ upsert descriptor #63 - version: "1" + version: "2" upsert descriptor #64 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - id: 65 + - - columnIds: + - - 1 + - id: 66 + - dependsOn: + - - 62 + formatVersion: 3 + id: 64 ... replacementOf: time: {} @@ -310,6 +344,22 @@ upsert descriptor #64 + version: "4" viewQuery: (SELECT name FROM db1.sc1.t1) upsert descriptor #65 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - - 2 + - id: 66 + - - columnIds: + - - 1 + - - 2 + - id: 67 + - dependsOn: + - - 64 + formatVersion: 3 + id: 65 ... replacementOf: time: {} @@ -319,6 +369,14 @@ upsert descriptor #65 + version: "4" viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) upsert descriptor #66 + ... + createAsOfTime: + wallTime: "1" + - dependsOn: + - - 64 + - - 65 + formatVersion: 3 + id: 66 ... replacementOf: time: {} @@ -328,6 +386,18 @@ upsert descriptor #66 + version: "2" viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) upsert descriptor #67 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - - 2 + - id: 70 + - dependsOn: + - - 65 + formatVersion: 3 + id: 67 ... replacementOf: time: {} @@ -357,6 +427,13 @@ upsert descriptor #69 + state: DROP + version: "3" upsert descriptor #70 + ... + createAsOfTime: + wallTime: "1" + - dependsOn: + - - 67 + dependsOnTypes: + - 68 ... replacementOf: time: {} @@ -446,147 +523,11 @@ upsert descriptor #70 commit transaction #1 # begin PostCommitPhase begin transaction #2 -## stage 1 in PostCommitPhase: 30 MutationType ops +## stage 1 in PostCommitPhase: 22 MutationType ops create job #2: "GC for Dropping descriptors 60 63 61 62 64 65 66 67 70 " descriptor IDs: [60 63 61 62 64 65 66 67 70] delete database namespace entry {0 0 db1} -> 58 delete schema namespace entry {58 0 sc1} -> 59 -upsert descriptor #60 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - byId: true - - columnIds: - - - 3 - - id: 63 - families: - - columnIds: - ... - state: DROP - unexposedParentSchemaId: 29 - - version: "3" - + version: "4" -upsert descriptor #61 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - byId: true - - columnIds: - - - 3 - - id: 62 - families: - - columnIds: - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "3" - + version: "4" -upsert descriptor #62 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "3" - + version: "4" -upsert descriptor #63 - ... - state: DROP - unexposedParentSchemaId: 29 - - version: "2" - + version: "3" -upsert descriptor #64 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - id: 65 - - - columnIds: - - - 1 - - id: 66 - - dependsOn: - - - 62 - formatVersion: 3 - id: 64 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "4" - + version: "5" - viewQuery: (SELECT name FROM db1.sc1.t1) -upsert descriptor #65 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - - 2 - - id: 66 - - - columnIds: - - - 1 - - - 2 - - id: 67 - - dependsOn: - - - 64 - formatVersion: 3 - id: 65 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "4" - + version: "5" - viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) -upsert descriptor #66 - ... - createAsOfTime: - wallTime: "1" - - dependsOn: - - - 64 - - - 65 - formatVersion: 3 - id: 66 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "2" - + version: "3" - viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) -upsert descriptor #67 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - - 2 - - id: 70 - - dependsOn: - - - 65 - formatVersion: 3 - id: 67 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "3" - + version: "4" - viewQuery: (SELECT n2, n1 FROM db1.sc1.v2) -upsert descriptor #70 - ... - createAsOfTime: - wallTime: "1" - - dependsOn: - - - 67 - dependsOnTypes: - - 68 - ... - state: DROP - unexposedParentSchemaId: 59 - - version: "2" - + version: "3" - viewQuery: (SELECT 'a':::sc1.typ::STRING AS k, n2, n1 FROM db1.sc1.v4) update progress of schema change job #1 commit transaction #2 begin transaction #3 From d00baeb41dff177f53280740cb0de43ac13708fb Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 25 Nov 2021 14:46:49 -0500 Subject: [PATCH 10/10] spanconfig/sqlwatcher: checkpoint, even without updates If `system.{descriptor,zones}` is static, we would previously discard all checkpoint events. This is not what we want -- we'd still like to know how caught up we are, with ongoing updates or otherwise. MVCC GC, after all, can still happen; if our last checkpoint was when the last update occurred, we may end up doing a lot of unnecessary work when finding out our last checkpoint was GC-ed from underneath us. Lack of periodic checkpoints also makes tests difficult to write -- you'd have to induce a benign update to flush out all earlier ones. This latent flakiness was uncovered after speeding up some existing tests in an earlier commit. NB: For incremental (possibly noop) checkpoints, we need to ensure that the sqlwatcher's buffer is initialized with a low watermark. When flushing events, we take the most conservative timestamp. If not initialized to a high water, this might be 0 -- violating the sqlwatcher's monotonically increasing checkpoint ts invariant. Release note: None --- pkg/server/server_sql.go | 2 + .../spanconfigsqlwatcher/BUILD.bazel | 2 + pkg/spanconfig/spanconfigsqlwatcher/buffer.go | 8 +- .../spanconfigsqlwatcher/buffer_test.go | 11 +- .../spanconfigsqlwatcher/sqlwatcher.go | 57 +++++---- .../spanconfigsqlwatcher/sqlwatcher_test.go | 114 ++++++++++++++++-- 6 files changed, 153 insertions(+), 41 deletions(-) diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index cc9bcfac9536..6a913b046e52 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -861,6 +861,8 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { cfg.rangeFeedFactory, 1<<20, /* 1 MB bufferMemLimit */ cfg.stopper, + // TODO(irfansharif): What should this no-op cadence be? + 30*time.Second, /* checkpointNoopsEvery */ spanConfigKnobs, ) spanConfigMgr = spanconfigmanager.New( diff --git a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel index bb47eba6a7c7..976946678e7d 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel @@ -22,11 +22,13 @@ go_library( "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/util", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/log/logcrash", "//pkg/util/stop", "//pkg/util/syncutil", + "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/spanconfig/spanconfigsqlwatcher/buffer.go b/pkg/spanconfig/spanconfigsqlwatcher/buffer.go index 8a50ecd2cba1..c6d64f2356dc 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/buffer.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/buffer.go @@ -71,11 +71,15 @@ const ( numRangefeeds int = iota ) -// newBuffer constructs and returns a new buffer. -func newBuffer(limit int) *buffer { +// newBuffer constructs a new buffer initialized with a starting frontier +// timestamp. +func newBuffer(limit int, initialFrontierTS hlc.Timestamp) *buffer { rangefeedBuffer := rangefeedbuffer.New(limit) eventBuffer := &buffer{} eventBuffer.mu.buffer = rangefeedBuffer + for i := range eventBuffer.mu.rangefeedFrontiers { + eventBuffer.mu.rangefeedFrontiers[i].Forward(initialFrontierTS) + } return eventBuffer } diff --git a/pkg/spanconfig/spanconfigsqlwatcher/buffer_test.go b/pkg/spanconfig/spanconfigsqlwatcher/buffer_test.go index 1005940394b7..986340bcbccf 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/buffer_test.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/buffer_test.go @@ -52,12 +52,12 @@ func TestBuffer(t *testing.T) { } ctx := context.Background() - buffer := newBuffer(10 /* limit */) + buffer := newBuffer(10 /* limit */, ts(1)) // Sanity check the newly initialized event buffer. updates, combinedFrontierTS, err := buffer.flush(ctx) require.NoError(t, err) - require.Equal(t, ts(0), combinedFrontierTS) + require.Equal(t, ts(1), combinedFrontierTS) require.True(t, len(updates) == 0) // Add a few events without advancing any of the frontiers. We don't expect @@ -69,7 +69,7 @@ func TestBuffer(t *testing.T) { require.NoError(t, err) updates, combinedFrontierTS, err = buffer.flush(ctx) require.NoError(t, err) - require.Equal(t, ts(0), combinedFrontierTS) + require.Equal(t, ts(1), combinedFrontierTS) require.True(t, len(updates) == 0) // Advance the zones frontier. We expect flush to still not return any results @@ -77,7 +77,7 @@ func TestBuffer(t *testing.T) { buffer.advance(zonesRangefeed, ts(11)) updates, combinedFrontierTS, err = buffer.flush(ctx) require.NoError(t, err) - require.Equal(t, ts(0), combinedFrontierTS) + require.Equal(t, ts(1), combinedFrontierTS) require.True(t, len(updates) == 0) // Advance the descriptors frontier to a lower timestamp than the zones @@ -150,7 +150,6 @@ func TestBuffer(t *testing.T) { func TestBufferCombinesDescriptorTypes(t *testing.T) { defer leaktest.AfterTest(t)() - buffer := newBuffer(10 /* limit */) ctx := context.Background() ts := func(nanos int) hlc.Timestamp { @@ -158,6 +157,8 @@ func TestBufferCombinesDescriptorTypes(t *testing.T) { WallTime: int64(nanos), } } + buffer := newBuffer(10 /* limit */, ts(0)) + makeEvent := func(descID int, descType catalog.DescriptorType, timestamp hlc.Timestamp) event { return event{ update: spanconfig.DescriptorUpdate{ diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go index 5d3a1d6ac8dd..472a41e735da 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go @@ -12,6 +12,7 @@ package spanconfigsqlwatcher import ( "context" + "time" "unsafe" "github.com/cockroachdb/cockroach/pkg/keys" @@ -22,10 +23,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -36,12 +39,13 @@ var _ spanconfig.SQLWatcher = &SQLWatcher{} // establishes rangefeeds over system.zones and system.descriptors to // incrementally watch for SQL updates. type SQLWatcher struct { - codec keys.SQLCodec - settings *cluster.Settings - stopper *stop.Stopper - knobs *spanconfig.TestingKnobs - rangeFeedFactory *rangefeed.Factory - bufferMemLimit int64 + codec keys.SQLCodec + settings *cluster.Settings + stopper *stop.Stopper + knobs *spanconfig.TestingKnobs + rangeFeedFactory *rangefeed.Factory + bufferMemLimit int64 + checkpointNoopsEvery time.Duration } // New constructs a new SQLWatcher. @@ -51,18 +55,20 @@ func New( rangeFeedFactory *rangefeed.Factory, bufferMemLimit int64, stopper *stop.Stopper, + checkpointNoopsEvery time.Duration, knobs *spanconfig.TestingKnobs, ) *SQLWatcher { if knobs == nil { knobs = &spanconfig.TestingKnobs{} } return &SQLWatcher{ - codec: codec, - settings: settings, - rangeFeedFactory: rangeFeedFactory, - stopper: stopper, - bufferMemLimit: bufferMemLimit, - knobs: knobs, + codec: codec, + settings: settings, + rangeFeedFactory: rangeFeedFactory, + stopper: stopper, + bufferMemLimit: bufferMemLimit, + checkpointNoopsEvery: checkpointNoopsEvery, + knobs: knobs, } } @@ -73,15 +79,15 @@ const sqlWatcherBufferEntrySize = int64(unsafe.Sizeof(event{}) + unsafe.Sizeof(r // WatchForSQLUpdates is part of the spanconfig.SQLWatcher interface. func (s *SQLWatcher) WatchForSQLUpdates( ctx context.Context, - timestamp hlc.Timestamp, + startTS hlc.Timestamp, handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, ) error { - return s.watch(ctx, timestamp, handler) + return s.watch(ctx, startTS, handler) } func (s *SQLWatcher) watch( ctx context.Context, - timestamp hlc.Timestamp, + startTS hlc.Timestamp, handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, ) error { @@ -101,7 +107,7 @@ func (s *SQLWatcher) watch( // serial semantics. errCh := make(chan error) frontierAdvanced := make(chan struct{}) - buf := newBuffer(int(s.bufferMemLimit / sqlWatcherBufferEntrySize)) + buf := newBuffer(int(s.bufferMemLimit/sqlWatcherBufferEntrySize), startTS) onFrontierAdvance := func(ctx context.Context, rangefeed rangefeedKind, timestamp hlc.Timestamp) { buf.advance(rangefeed, timestamp) select { @@ -131,17 +137,18 @@ func (s *SQLWatcher) watch( } } - descriptorsRF, err := s.watchForDescriptorUpdates(ctx, timestamp, onEvent, onFrontierAdvance) + descriptorsRF, err := s.watchForDescriptorUpdates(ctx, startTS, onEvent, onFrontierAdvance) if err != nil { return errors.Wrapf(err, "error establishing rangefeed over system.descriptors") } defer descriptorsRF.Close() - zonesRF, err := s.watchForZoneConfigUpdates(ctx, timestamp, onEvent, onFrontierAdvance) + zonesRF, err := s.watchForZoneConfigUpdates(ctx, startTS, onEvent, onFrontierAdvance) if err != nil { return errors.Wrapf(err, "error establishing rangefeed over system.zones") } defer zonesRF.Close() + checkpointNoops := util.Every(s.checkpointNoopsEvery) for { select { case <-ctx.Done(): @@ -155,7 +162,7 @@ func (s *SQLWatcher) watch( if err != nil { return err } - if len(events) == 0 { + if len(events) == 0 && !checkpointNoops.ShouldProcess(timeutil.Now()) { continue } if err := handler(ctx, events, combinedFrontierTS); err != nil { @@ -170,7 +177,7 @@ func (s *SQLWatcher) watch( // callback is invoked whenever the rangefeed frontier is advanced as well. func (s *SQLWatcher) watchForDescriptorUpdates( ctx context.Context, - timestamp hlc.Timestamp, + startTS hlc.Timestamp, onEvent func(context.Context, event), onFrontierAdvance func(context.Context, rangefeedKind, hlc.Timestamp), ) (*rangefeed.RangeFeed, error) { @@ -239,7 +246,7 @@ func (s *SQLWatcher) watchForDescriptorUpdates( ctx, "sql-watcher-descriptor-rangefeed", descriptorTableSpan, - timestamp, + startTS, handleEvent, rangefeed.WithDiff(), rangefeed.WithOnFrontierAdvance(func(ctx context.Context, resolvedTS hlc.Timestamp) { @@ -250,7 +257,7 @@ func (s *SQLWatcher) watchForDescriptorUpdates( return nil, err } - log.Infof(ctx, "established range feed over system.descriptors table starting at time %s", timestamp) + log.Infof(ctx, "established range feed over system.descriptors table starting at time %s", startTS) return rf, nil } @@ -260,7 +267,7 @@ func (s *SQLWatcher) watchForDescriptorUpdates( // advanced. func (s *SQLWatcher) watchForZoneConfigUpdates( ctx context.Context, - timestamp hlc.Timestamp, + startTS hlc.Timestamp, onEvent func(context.Context, event), onFrontierAdvance func(context.Context, rangefeedKind, hlc.Timestamp), ) (*rangefeed.RangeFeed, error) { @@ -296,7 +303,7 @@ func (s *SQLWatcher) watchForZoneConfigUpdates( ctx, "sql-watcher-zones-rangefeed", zoneTableSpan, - timestamp, + startTS, handleEvent, rangefeed.WithOnFrontierAdvance(func(ctx context.Context, resolvedTS hlc.Timestamp) { onFrontierAdvance(ctx, zonesRangefeed, resolvedTS) @@ -306,6 +313,6 @@ func (s *SQLWatcher) watchForZoneConfigUpdates( return nil, err } - log.Infof(ctx, "established range feed over system.zones table starting at time %s", timestamp) + log.Infof(ctx, "established range feed over system.zones table starting at time %s", startTS) return rf, nil } diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go index 75d35a7a7c57..a5bead9a0dab 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher_test.go @@ -121,12 +121,14 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) + noopCheckpointDuration := 100 * time.Millisecond sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), 1<<20, /* 1 MB, bufferMemLimit */ ts.Stopper(), + noopCheckpointDuration, nil, /* knobs */ ) @@ -138,22 +140,19 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { mu.receivedIDs = make(map[descpb.ID]struct{}) var wg sync.WaitGroup + watcherStartTS := ts.Clock().Now() + watcherCtx, watcherCancel := context.WithCancel(context.Background()) wg.Add(1) go func() { defer wg.Done() - startTS := ts.Clock().Now() - mu.Lock() - mu.lastCheckpoint = startTS - mu.Unlock() - - _ = sqlWatcher.WatchForSQLUpdates(watcherCtx, startTS, + _ = sqlWatcher.WatchForSQLUpdates(watcherCtx, watcherStartTS, func(ctx context.Context, updates []spanconfig.DescriptorUpdate, checkpointTS hlc.Timestamp) error { mu.Lock() defer mu.Unlock() - require.True(t, mu.lastCheckpoint.Less(checkpointTS)) + require.True(t, mu.lastCheckpoint.LessEq(checkpointTS)) mu.lastCheckpoint = checkpointTS for _, update := range updates { @@ -216,12 +215,14 @@ func TestSQLWatcherMultiple(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) + noopCheckpointDuration := 100 * time.Millisecond sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), 1<<20, /* 1 MB, bufferMemLimit */ ts.Stopper(), + noopCheckpointDuration, nil, /* knobs */ ) @@ -315,9 +316,8 @@ func TestSQLWatcherMultiple(t *testing.T) { watcher2Cancel() watcher3Cancel() + wg.Wait() } - - wg.Wait() } // TestSQLWatcherOnEventError ensures that if there is an error processing a @@ -343,12 +343,14 @@ func TestSQLWatcherOnEventError(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) + noopCheckpointDuration := 100 * time.Millisecond sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), 1<<20, /* 1 MB, bufferMemLimit */ ts.Stopper(), + noopCheckpointDuration, &spanconfig.TestingKnobs{ SQLWatcherOnEventInterceptor: func() error { return errors.New("boom") @@ -389,12 +391,14 @@ func TestSQLWatcherHandlerError(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) + noopCheckpointDuration := 100 * time.Millisecond sqlWatcher := spanconfigsqlwatcher.New( keys.SystemSQLCodec, ts.ClusterSettings(), ts.RangeFeedFactory().(*rangefeed.Factory), 1<<20, /* 1 MB, bufferMemLimit */ ts.Stopper(), + noopCheckpointDuration, nil, /* knobs */ ) @@ -441,3 +445,95 @@ func TestSQLWatcherHandlerError(t *testing.T) { // Ensure that the handler was called only once. require.Equal(t, int32(1), atomic.LoadInt32(&numCalled)) } + +func TestWatcherReceivesNoopCheckpoints(t *testing.T) { + defer leaktest.AfterTest(t)() + + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + SpanConfig: &spanconfig.TestingKnobs{ + ManagerDisableJobCreation: true, // disable the automatic job creation. + }, + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speed up schema changes. + }, + }, + }) + ctx := context.Background() + defer tc.Stopper().Stop(ctx) + ts := tc.Server(0 /* idx */) + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0 /* idx */)) + tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) + + noopCheckpointDuration := 25 * time.Millisecond + sqlWatcher := spanconfigsqlwatcher.New( + keys.SystemSQLCodec, + ts.ClusterSettings(), + ts.RangeFeedFactory().(*rangefeed.Factory), + 1<<20, /* 1 MB, bufferMemLimit */ + ts.Stopper(), + noopCheckpointDuration, + nil, /* knobs */ + ) + + beforeStmtTS := ts.Clock().Now() + tdb.Exec(t, "CREATE TABLE t()") + afterStmtTS := ts.Clock().Now() + const expDescID descpb.ID = 52 + + var wg sync.WaitGroup + mu := struct { + syncutil.Mutex + numCheckpoints int + lastCheckpoint hlc.Timestamp + }{} + + watch := func(ctx context.Context, onExit func(), onCheckpoint func(hlc.Timestamp)) { + defer wg.Done() + + receivedIDs := make(map[descpb.ID]struct{}) + err := sqlWatcher.WatchForSQLUpdates(ctx, beforeStmtTS, + func(_ context.Context, updates []spanconfig.DescriptorUpdate, checkpointTS hlc.Timestamp) error { + onCheckpoint(checkpointTS) + + for _, update := range updates { + receivedIDs[update.ID] = struct{}{} + } + return nil + }) + require.True(t, testutils.IsError(err, "context canceled")) + require.Equal(t, 1, len(receivedIDs)) + _, seen := receivedIDs[expDescID] + require.True(t, seen) + } + + { + // Run the first watcher; wait for it to observe the update before + // tearing it down. + + watcherCtx, watcherCancel := context.WithCancel(ctx) + wg.Add(1) + go watch(watcherCtx, func() { wg.Done() }, func(ts hlc.Timestamp) { + mu.Lock() + mu.lastCheckpoint = ts + mu.numCheckpoints++ + mu.Unlock() + }) + + testutils.SucceedsSoon(t, func() error { + mu.Lock() + defer mu.Unlock() + + if mu.lastCheckpoint.Less(afterStmtTS) { + return errors.New("last checkpoint precedes statement timestamp") + } + if mu.numCheckpoints < 3 { + return errors.New("didn't receive no-op checkpoints") + } + return nil + }) + watcherCancel() + wg.Wait() + } +}