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 80d489217d3f..acd468b306dd 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -857,12 +857,14 @@ 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, 1<<20, /* 1 MB bufferMemLimit */ cfg.stopper, + // TODO(irfansharif): What should this no-op cadence be? + 30*time.Second, /* checkpointNoopsEvery */ spanConfigKnobs, ) spanConfigMgr = spanconfigmanager.New( @@ -872,7 +874,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 19d00305765c..7e5bd04d5e56 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -999,14 +999,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 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 9026269e89b4..472a41e735da 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go @@ -12,7 +12,7 @@ package spanconfigsqlwatcher import ( "context" - "sync/atomic" + "time" "unsafe" "github.com/cockroachdb/cockroach/pkg/keys" @@ -23,102 +23,83 @@ 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" ) -// 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 { - codec keys.SQLCodec - settings *cluster.Settings - stopper *stop.Stopper - knobs *spanconfig.TestingKnobs - rangeFeedFactory *rangefeed.Factory - bufferMemLimit int64 +// 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 + knobs *spanconfig.TestingKnobs + rangeFeedFactory *rangefeed.Factory + bufferMemLimit int64 + checkpointNoopsEvery time.Duration } -// 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, + checkpointNoopsEvery time.Duration, knobs *spanconfig.TestingKnobs, -) *Factory { +) *SQLWatcher { if knobs == nil { knobs = &spanconfig.TestingKnobs{} } - return &Factory{ - codec: codec, - settings: settings, - rangeFeedFactory: rangeFeedFactory, - stopper: stopper, - bufferMemLimit: bufferMemLimit, - knobs: knobs, + return &SQLWatcher{ + codec: codec, + settings: settings, + rangeFeedFactory: rangeFeedFactory, + stopper: stopper, + bufferMemLimit: bufferMemLimit, + checkpointNoopsEvery: checkpointNoopsEvery, + knobs: knobs, } } -// 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 +// 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, + startTS hlc.Timestamp, + handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, +) error { + return s.watch(ctx, startTS, handler) +} + +func (s *SQLWatcher) watch( + ctx context.Context, + startTS 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") - } // 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 +107,9 @@ func (s *SQLWatcher) WatchForSQLUpdates( // serial semantics. errCh := make(chan error) frontierAdvanced := make(chan struct{}) + buf := newBuffer(int(s.bufferMemLimit/sqlWatcherBufferEntrySize), startTS) 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 @@ -142,7 +124,7 @@ func (s *SQLWatcher) WatchForSQLUpdates( 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,31 +137,32 @@ func (s *SQLWatcher) WatchForSQLUpdates( } } - 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(): return ctx.Err() case <-s.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 } - if len(events) == 0 { + if len(events) == 0 && !checkpointNoops.ShouldProcess(timeutil.Now()) { continue } if err := handler(ctx, events, combinedFrontierTS); err != nil { @@ -194,7 +177,7 @@ func (s *SQLWatcher) WatchForSQLUpdates( // 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) { @@ -263,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) { @@ -274,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 } @@ -284,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) { @@ -320,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) @@ -330,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 39e8dd9cd7f7..a5bead9a0dab 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,7 @@ 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/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -40,7 +42,6 @@ func TestSQLWatcherReactsToUpdates(t *testing.T) { defer leaktest.AfterTest(t)() testCases := []struct { - setup string stmt string expectedIDs descpb.IDs }{ @@ -49,17 +50,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 +92,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,81 +117,85 @@ 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. - startTS := ts.Clock().Now() - - var mu struct { - syncutil.Mutex - receivedIDs map[descpb.ID]struct{} - } - mu.receivedIDs = make(map[descpb.ID]struct{}) + 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'`) - 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) + 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 */ + ) + + var mu struct { + syncutil.Mutex + receivedIDs map[descpb.ID]struct{} + lastCheckpoint hlc.Timestamp + } + 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() + + _ = 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.LessEq(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 -// 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)() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ @@ -202,60 +211,113 @@ 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( + 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 */ ) - startTS := ts.Clock().Now() - _, err := sqlDB.Exec("CREATE TABLE t()") - require.NoError(t, err) - - sqlWatcher := sqlWatcherFactory.New() + 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 + 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() - wg.Wait() + watcher2Cancel() + watcher3Cancel() + wg.Wait() + } } // TestSQLWatcherOnEventError ensures that if there is an error processing a @@ -277,14 +339,18 @@ 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( + 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") @@ -293,11 +359,9 @@ 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,19 +387,22 @@ 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( + 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 */ - ).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 +419,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") } }() @@ -380,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() + } +} diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index c5e3c91993cd..e2fdedb30990 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -1,10 +1,14 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//build:STRINGER.bzl", "stringer") go_library( name = "scgraph", srcs = [ + "dep_edge_tree.go", + "edge.go", "graph.go", "iteration.go", + ":gen-depedgekind-stringer", # keep ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph", visibility = ["//visibility:public"], @@ -21,12 +25,23 @@ 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", ], ) + +stringer( + name = "gen-depedgekind-stringer", + src = "edge.go", + typ = "DepEdgeKind", +) 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..bcb34bce4090 --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/dep_edge_tree_test.go @@ -0,0 +1,181 @@ +// 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) + }) + } + }) + } +} + +// 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") + }) + } +} 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 new file mode 100644 index 000000000000..5b53cf2c408c --- /dev/null +++ b/pkg/sql/schemachanger/scgraph/edge.go @@ -0,0 +1,93 @@ +// 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 +} + +// 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. + 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 } + +// 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 0e5f9843a4fa..f6a105b98ee5 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 @@ -42,19 +41,18 @@ 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 *btree.BTree + depEdgesFrom *depEdgeTree - // 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 + // 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. @@ -89,15 +87,15 @@ 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), + opEdgesFrom: map[*scpb.Node]*OpEdge{}, optimizedOutOpEdges: map[*OpEdge]bool{}, opToNode: map[scop.Op]*scpb.Node{}, entities: db, statements: initial.Statements, 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]) @@ -125,9 +123,9 @@ func (g *Graph) ShallowClone() *Graph { authorization: g.authorization, targetNodes: g.targetNodes, targetIdxMap: g.targetIdxMap, - nodeOpEdgesFrom: g.nodeOpEdgesFrom, - nodeDepEdgesFrom: g.nodeDepEdgesFrom, - nodeDepEdgesTo: g.nodeDepEdgesTo, + opEdgesFrom: g.opEdgesFrom, + depEdgesFrom: g.depEdgesFrom, + sameStageDepEdgesTo: g.sameStageDepEdgesTo, opToNode: g.opToNode, edges: g.edges, entities: g.entities, @@ -185,7 +183,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 } @@ -204,7 +202,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) } @@ -219,7 +217,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() @@ -238,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 } @@ -251,16 +250,10 @@ 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.depEdgesFrom.insert(de) + if de.Kind() == SameStage { + g.sameStageDepEdgesTo.insert(de) + } return nil } @@ -290,60 +283,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 } - // GetNodeRanks fetches ranks of nodes in topological order. func (g *Graph) GetNodeRanks() (nodeRanks map[*scpb.Node]int, err error) { defer func() { @@ -355,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) @@ -376,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 @@ -400,42 +325,8 @@ 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) { +func (g *Graph) compareNodes(a, b *scpb.Node) (less, eq bool) { switch { case a == b: return false, true diff --git a/pkg/sql/schemachanger/scgraph/graph_test.go b/pkg/sql/schemachanger/scgraph/graph_test.go index 424a3ce6dca8..2770fc27faa3 100644 --- a/pkg/sql/schemachanger/scgraph/graph_test.go +++ b/pkg/sql/schemachanger/scgraph/graph_test.go @@ -22,147 +22,149 @@ 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, 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{ {0, 1}, {1, 0}, {2, 0}, }, - []int{1, 0, 2}, // We expect the drop to be ordered first. - "", - ) - }) + expectedRankErr: "graph is not a dag", + }, + } + + 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), + scgraph.HappensAfter, + 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) }) + } } diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index 74d8749a362f..19c9378ddc22 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,11 @@ 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.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 dc5998dd6c78..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 @@ -42,11 +42,13 @@ type rule struct { 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 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{}