diff --git a/pkg/ccl/spanconfigccl/spanconfigcomparedccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigcomparedccl/datadriven_test.go index 07ee5f047641..305f1f3b40fe 100644 --- a/pkg/ccl/spanconfigccl/spanconfigcomparedccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigcomparedccl/datadriven_test.go @@ -199,8 +199,8 @@ func TestDataDriven(t *testing.T) { case "reconcile": tsBeforeReconcilerStart := tenant.Clock().Now() go func() { - err := tenant.Reconciler().Reconcile(ctx, hlc.Timestamp{}, func(checkpoint hlc.Timestamp) error { - tenant.Checkpoint(checkpoint) + err := tenant.Reconciler().Reconcile(ctx, hlc.Timestamp{} /* startTS */, func() error { + tenant.RecordCheckpoint() return nil }) require.NoError(t, err) diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go index 4c1c16535790..90bed1d76335 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go @@ -145,8 +145,8 @@ func TestDataDriven(t *testing.T) { case "reconcile": tsBeforeReconcilerStart := tenant.Clock().Now() go func() { - err := tenant.Reconciler().Reconcile(ctx, hlc.Timestamp{}, func(checkpoint hlc.Timestamp) error { - tenant.Checkpoint(checkpoint) + err := tenant.Reconciler().Reconcile(ctx, hlc.Timestamp{} /* startTS */, func() error { + tenant.RecordCheckpoint() return nil }) require.NoError(t, err) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index a8914f732563..f80353e7e6b8 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -223,7 +223,7 @@ const ( // This version comes with a migration to populate the same seed data // for existing tenants. SeedTenantSpanConfigs - // Public schema is backed by a descriptor. + // PublicSchemasWithDescriptors backs public schemas with descriptors. PublicSchemasWithDescriptors // AlterSystemProtectedTimestampAddColumn adds a target column to the // system.protected_ts_records table that describes what is protected by the diff --git a/pkg/col/coldata/BUILD.bazel b/pkg/col/coldata/BUILD.bazel index dedd1f533e44..f1c330f5f9a4 100644 --- a/pkg/col/coldata/BUILD.bazel +++ b/pkg/col/coldata/BUILD.bazel @@ -43,6 +43,7 @@ go_test( "vec_test.go", ], embed = [":coldata"], + tags = ["no-remote"], deps = [ "//pkg/col/coldatatestutils", "//pkg/sql/colconv", diff --git a/pkg/kv/kvclient/kvcoord/BUILD.bazel b/pkg/kv/kvclient/kvcoord/BUILD.bazel index 999764845e1a..080a7809ab36 100644 --- a/pkg/kv/kvclient/kvcoord/BUILD.bazel +++ b/pkg/kv/kvclient/kvcoord/BUILD.bazel @@ -138,6 +138,7 @@ go_test( ], data = glob(["testdata/**"]), embed = [":with-mocks"], # keep + tags = ["no-remote"], deps = [ "//build/bazelutil:noop", "//pkg/base", diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index 6fba02ecc8a0..cebee5decc89 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -1001,7 +1001,7 @@ func TestMVCCGCQueueIntentResolution(t *testing.T) { } // Process through GC queue. - confReader, err := tc.store.GetConfReader() + confReader, err := tc.store.GetConfReader(ctx) if err != nil { t.Fatal(err) } @@ -1062,7 +1062,7 @@ func TestMVCCGCQueueLastProcessedTimestamps(t *testing.T) { } } - confReader, err := tc.store.GetConfReader() + confReader, err := tc.store.GetConfReader(ctx) if err != nil { t.Fatal(err) } @@ -1167,7 +1167,7 @@ func TestMVCCGCQueueChunkRequests(t *testing.T) { } // Forward the clock past the default GC time. - confReader, err := tc.store.GetConfReader() + confReader, err := tc.store.GetConfReader(ctx) if err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 0a9f7c737b1d..40034f86a987 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -611,7 +611,7 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. var confReader spanconfig.StoreReader if bq.needsSystemConfig { var err error - confReader, err = bq.store.GetConfReader() + confReader, err = bq.store.GetConfReader(ctx) if err != nil { if errors.Is(err, errSysCfgUnavailable) && log.V(1) { log.Warningf(ctx, "unable to retrieve system config, skipping: %v", err) @@ -901,7 +901,7 @@ func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) er var confReader spanconfig.StoreReader if bq.needsSystemConfig { var err error - confReader, err = bq.store.GetConfReader() + confReader, err = bq.store.GetConfReader(ctx) if errors.Is(err, errSysCfgUnavailable) { if log.V(1) { log.Warningf(ctx, "unable to retrieve conf reader, skipping: %v", err) diff --git a/pkg/kv/kvserver/queue_helpers_testutil.go b/pkg/kv/kvserver/queue_helpers_testutil.go index ca232ddccfec..2f469cf95fc2 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -26,11 +26,11 @@ func (bq *baseQueue) testingAdd( return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority) } -func forceScanAndProcess(s *Store, q *baseQueue) error { +func forceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) error { // Check that the system config is available. It is needed by many queues. If // it's not available, some queues silently fail to process any replicas, // which is undesirable for this method. - if _, err := s.GetConfReader(); err != nil { + if _, err := s.GetConfReader(ctx); err != nil { return errors.Wrap(err, "unable to retrieve conf reader") } @@ -44,7 +44,7 @@ func forceScanAndProcess(s *Store, q *baseQueue) error { } func mustForceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) { - if err := forceScanAndProcess(s, q); err != nil { + if err := forceScanAndProcess(ctx, s, q); err != nil { log.Fatalf(ctx, "%v", err) } } @@ -52,7 +52,7 @@ func mustForceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) { // ForceReplicationScanAndProcess iterates over all ranges and // enqueues any that need to be replicated. func (s *Store) ForceReplicationScanAndProcess() error { - return forceScanAndProcess(s, s.replicateQueue.baseQueue) + return forceScanAndProcess(context.TODO(), s, s.replicateQueue.baseQueue) } // MustForceReplicaGCScanAndProcess iterates over all ranges and enqueues any that @@ -70,7 +70,7 @@ func (s *Store) MustForceMergeScanAndProcess() { // ForceSplitScanAndProcess iterates over all ranges and enqueues any that // may need to be split. func (s *Store) ForceSplitScanAndProcess() error { - return forceScanAndProcess(s, s.splitQueue.baseQueue) + return forceScanAndProcess(context.TODO(), s, s.splitQueue.baseQueue) } // MustForceRaftLogScanAndProcess iterates over all ranges and enqueues any that @@ -83,20 +83,20 @@ func (s *Store) MustForceRaftLogScanAndProcess() { // any that need time series maintenance, then processes the time series // maintenance queue. func (s *Store) ForceTimeSeriesMaintenanceQueueProcess() error { - return forceScanAndProcess(s, s.tsMaintenanceQueue.baseQueue) + return forceScanAndProcess(context.TODO(), s, s.tsMaintenanceQueue.baseQueue) } // ForceRaftSnapshotQueueProcess iterates over all ranges, enqueuing // any that need raft snapshots, then processes the raft snapshot // queue. func (s *Store) ForceRaftSnapshotQueueProcess() error { - return forceScanAndProcess(s, s.raftSnapshotQueue.baseQueue) + return forceScanAndProcess(context.TODO(), s, s.raftSnapshotQueue.baseQueue) } // ForceConsistencyQueueProcess runs all the ranges through the consistency // queue. func (s *Store) ForceConsistencyQueueProcess() error { - return forceScanAndProcess(s, s.consistencyQueue.baseQueue) + return forceScanAndProcess(context.TODO(), s, s.consistencyQueue.baseQueue) } // The methods below can be used to control a store's queues. Stopping a queue diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index c053453f1c31..87580ac7fcab 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -565,7 +565,7 @@ func TestNeedsSystemConfig(t *testing.T) { tc.StartWithStoreConfig(ctx, t, stopper, cfg) { - confReader, err := tc.store.GetConfReader() + confReader, err := tc.store.GetConfReader(ctx) require.Nil(t, confReader) require.True(t, errors.Is(err, errSysCfgUnavailable)) } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 648b81cc6a5b..3d9834956bb6 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2841,7 +2841,7 @@ func (s *Store) relocateOne( `range %s was either in a joint configuration or had learner replicas: %v`, desc, desc.Replicas()) } - confReader, err := s.GetConfReader() + confReader, err := s.GetConfReader(ctx) if err != nil { return nil, nil, errors.Wrap(err, "can't relocate range") } diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 86c5872ae148..ea5711c7bd14 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -684,7 +684,7 @@ func (r *Replica) updateRangeInfo(ctx context.Context, desc *roachpb.RangeDescri // the original range wont work as the original and new ranges might belong // to different zones. // Load the system config. - confReader, err := r.store.GetConfReader() + confReader, err := r.store.GetConfReader(ctx) if errors.Is(err, errSysCfgUnavailable) { // This could be before the system config was ever gossiped, or it // expired. Let the gossip callback set the info. diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 1e43b2ce8b5a..74483ef41d69 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2104,7 +2104,7 @@ func (s *Store) startGossip() { var errSysCfgUnavailable = errors.New("system config not available in gossip") // GetConfReader exposes access to a configuration reader. -func (s *Store) GetConfReader() (spanconfig.StoreReader, error) { +func (s *Store) GetConfReader(ctx context.Context) (spanconfig.StoreReader, error) { if s.cfg.TestingKnobs.MakeSystemConfigSpanUnavailableToQueues { return nil, errSysCfgUnavailable } @@ -3260,7 +3260,7 @@ func (s *Store) ManuallyEnqueue( return nil, nil, errors.Errorf("unknown queue type %q", queueName) } - confReader, err := s.GetConfReader() + confReader, err := s.GetConfReader(ctx) if err != nil { return nil, nil, errors.Wrap(err, "unable to retrieve conf reader, cannot run queue; make sure "+ diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index e9b733f67763..83b6d264012a 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -159,6 +159,7 @@ go_test( "version_test.go", ], embed = [":with-mocks"], # keep + tags = ["no-remote"], deps = [ "//pkg/cli/exit", "//pkg/kv/kvserver/concurrency/lock", diff --git a/pkg/server/diagnostics/BUILD.bazel b/pkg/server/diagnostics/BUILD.bazel index 5944ec06f7ca..0b66814c6f4b 100644 --- a/pkg/server/diagnostics/BUILD.bazel +++ b/pkg/server/diagnostics/BUILD.bazel @@ -56,6 +56,7 @@ go_test( "update_checker_test.go", ], embed = [":diagnostics"], + tags = ["no-remote"], deps = [ "//pkg/base", "//pkg/build", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index c0dac04a66ed..38ccd9ad15c6 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -907,7 +907,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { spanConfigKnobs, ) - execCfg.SpanConfigReconciliationJobDeps = spanConfig.manager + execCfg.SpanConfigReconciler = spanConfigReconciler } execCfg.SpanConfigKVAccessor = cfg.sqlServerOptionalKVArgs.spanConfigKVAccessor diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 111956cb98ba..9d6aa5091cfd 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -160,25 +160,24 @@ type Reconciler interface { // timestamp. If it does not find MVCC history going far back enough[1], it // falls back to a scan of all descriptors and zone configs before being // able to do more incremental work. The provided callback is invoked - // with timestamps that can be safely checkpointed. A future Reconciliation - // attempt can make use of this timestamp to reduce the amount of necessary - // work (provided the MVCC history is still available). + // whenever incremental progress has been made and a Checkpoint() timestamp + // is available. A future Reconcile() attempt can make use of this timestamp + // to reduce the amount of necessary work (provided the MVCC history is + // still available). // // [1]: It's possible for system.{zones,descriptor} to have been GC-ed away; // think suspended tenants. Reconcile( ctx context.Context, startTS hlc.Timestamp, - callback func(checkpoint hlc.Timestamp) error, + onCheckpoint func() error, ) error -} -// 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 { - Reconciler + // Checkpoint returns a timestamp suitable for checkpointing. A future + // Reconcile() attempt can make use of this timestamp to reduce the + // amount of necessary work (provided the MVCC history is + // still available). + Checkpoint() hlc.Timestamp } // Store is a data structure used to store spans and their corresponding diff --git a/pkg/spanconfig/spanconfigjob/job.go b/pkg/spanconfig/spanconfigjob/job.go index 5d73ee97e042..cfadc4d9ef86 100644 --- a/pkg/spanconfig/spanconfigjob/job.go +++ b/pkg/spanconfig/spanconfigjob/job.go @@ -30,7 +30,7 @@ var _ jobs.Resumer = (*resumer)(nil) // Resume implements the jobs.Resumer interface. func (r *resumer) Resume(ctx context.Context, execCtxI interface{}) error { execCtx := execCtxI.(sql.JobExecContext) - rc := execCtx.SpanConfigReconciliationJobDeps() + rc := execCtx.SpanConfigReconciler() // TODO(irfansharif): #73086 bubbles up retryable errors from the // reconciler/underlying watcher in the (very) unlikely event that it's @@ -41,10 +41,10 @@ func (r *resumer) Resume(ctx context.Context, execCtxI interface{}) error { // the job all over again after some time, it's just that the checks for // failed jobs happen infrequently. - if err := rc.Reconcile(ctx, hlc.Timestamp{}, func(checkpoint hlc.Timestamp) error { + if err := rc.Reconcile(ctx, hlc.Timestamp{}, func() error { // TODO(irfansharif): Stash this checkpoint somewhere and use it when // starting back up. - _ = checkpoint + _ = rc.Checkpoint() return nil }); err != nil { return err diff --git a/pkg/spanconfig/spanconfigmanager/manager.go b/pkg/spanconfig/spanconfigmanager/manager.go index 5a40edfab6bc..a82bcc466d76 100644 --- a/pkg/spanconfig/spanconfigmanager/manager.go +++ b/pkg/spanconfig/spanconfigmanager/manager.go @@ -54,8 +54,11 @@ var jobEnabledSetting = settings.RegisterBoolSetting( "enable the use of the kv accessor", false) // Manager is the coordinator of the span config subsystem. It ensures that -// there's only one span config reconciliation job for every tenant. It also +// there's only one span config reconciliation job[1] for every tenant. It also // captures all relevant dependencies for the job. +// +// [1]: The reconciliation job is responsible for reconciling a tenant's zone +// configurations with the clusters span configurations. type Manager struct { db *kv.DB jr *jobs.Registry @@ -67,8 +70,6 @@ type Manager struct { spanconfig.Reconciler } -var _ spanconfig.ReconciliationDependencies = &Manager{} - // New constructs a new Manager. func New( db *kv.DB, diff --git a/pkg/spanconfig/spanconfigreconciler/BUILD.bazel b/pkg/spanconfig/spanconfigreconciler/BUILD.bazel index 48eb36f5a59e..125692ab1058 100644 --- a/pkg/spanconfig/spanconfigreconciler/BUILD.bazel +++ b/pkg/spanconfig/spanconfigreconciler/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/sql/catalog/descs", "//pkg/sql/sem/tree", "//pkg/util/hlc", + "//pkg/util/syncutil", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/spanconfig/spanconfigreconciler/reconciler.go b/pkg/spanconfig/spanconfigreconciler/reconciler.go index 943dffdafde0..8e8d19eb6457 100644 --- a/pkg/spanconfig/spanconfigreconciler/reconciler.go +++ b/pkg/spanconfig/spanconfigreconciler/reconciler.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" ) @@ -38,6 +39,11 @@ type Reconciler struct { codec keys.SQLCodec tenID roachpb.TenantID knobs *spanconfig.TestingKnobs + + mu struct { + syncutil.RWMutex + lastCheckpoint hlc.Timestamp + } } var _ spanconfig.Reconciler = &Reconciler{} @@ -122,7 +128,7 @@ func New( // checkpoint. For changes to, say, RANGE DEFAULT, the RPC request proto is // proportional to the number of schema objects. func (r *Reconciler) Reconcile( - ctx context.Context, startTS hlc.Timestamp, callback func(checkpoint hlc.Timestamp) error, + ctx context.Context, startTS hlc.Timestamp, onCheckpoint func() error, ) error { // TODO(irfansharif): Check system.{zones,descriptors} for last GC timestamp // and avoid the full reconciliation pass if the startTS provided is @@ -136,15 +142,20 @@ func (r *Reconciler) Reconcile( codec: r.codec, tenID: r.tenID, } - latestStore, reconciledUpto, err := full.reconcile(ctx) + latestStore, reconciledUpUntil, err := full.reconcile(ctx) if err != nil { return err } - if err := callback(reconciledUpto); err != nil { + r.mu.Lock() + r.mu.lastCheckpoint = reconciledUpUntil + r.mu.Unlock() + + if err := onCheckpoint(); err != nil { return err } + incrementalStartTS := reconciledUpUntil incremental := incrementalReconciler{ sqlTranslator: r.sqlTranslator, sqlWatcher: r.sqlWatcher, @@ -154,7 +165,21 @@ func (r *Reconciler) Reconcile( codec: r.codec, knobs: r.knobs, } - return incremental.reconcile(ctx, reconciledUpto, callback) + return incremental.reconcile(ctx, incrementalStartTS, func(reconciledUpUntil hlc.Timestamp) error { + r.mu.Lock() + r.mu.lastCheckpoint = reconciledUpUntil + r.mu.Unlock() + + return onCheckpoint() + }) +} + +// Checkpoint is part of the spanconfig.Reconciler interface. +func (r *Reconciler) Checkpoint() hlc.Timestamp { + r.mu.RLock() + defer r.mu.RUnlock() + + return r.mu.lastCheckpoint } // fullReconciler is a single-use orchestrator for the full reconciliation diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go index 2628e630254b..dc7f7b933c16 100644 --- a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go @@ -66,9 +66,11 @@ func (s *Tenant) TimestampAfterLastExec() hlc.Timestamp { return s.mu.tsAfterLastExec } -// Checkpoint is used to record a checkpointed timestamp, retrievable via -// LastCheckpoint. -func (s *Tenant) Checkpoint(ts hlc.Timestamp) { +// RecordCheckpoint is used to record the reconciliation checkpoint, retrievable +// via LastCheckpoint. +func (s *Tenant) RecordCheckpoint() { + ts := s.Reconciler().Checkpoint() + s.mu.Lock() defer s.mu.Unlock() s.mu.lastCheckpoint = ts diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index b577bb5c5177..9612694eff9d 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -48,6 +48,7 @@ go_test( "table_col_set_test.go", ], embed = [":catalog"], + tags = ["no-remote"], deps = [ "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/dbdesc", diff --git a/pkg/sql/colexec/colexecagg/BUILD.bazel b/pkg/sql/colexec/colexecagg/BUILD.bazel index 494219062d94..2e2ce136e97d 100644 --- a/pkg/sql/colexec/colexecagg/BUILD.bazel +++ b/pkg/sql/colexec/colexecagg/BUILD.bazel @@ -35,6 +35,7 @@ go_test( name = "colexecagg_test", srcs = ["dep_test.go"], embed = [":colexecagg"], + tags = ["no-remote"], deps = ["//pkg/testutils/buildutil"], ) diff --git a/pkg/sql/colexec/colexecargs/BUILD.bazel b/pkg/sql/colexec/colexecargs/BUILD.bazel index 34047e3c0b59..fe799c2cb279 100644 --- a/pkg/sql/colexec/colexecargs/BUILD.bazel +++ b/pkg/sql/colexec/colexecargs/BUILD.bazel @@ -29,5 +29,6 @@ go_test( name = "colexecargs_test", srcs = ["dep_test.go"], embed = [":colexecargs"], + tags = ["no-remote"], deps = ["//pkg/testutils/buildutil"], ) diff --git a/pkg/sql/colexec/colexecbase/BUILD.bazel b/pkg/sql/colexec/colexecbase/BUILD.bazel index 0e6a6c3b2d2d..bc7be433178f 100644 --- a/pkg/sql/colexec/colexecbase/BUILD.bazel +++ b/pkg/sql/colexec/colexecbase/BUILD.bazel @@ -50,6 +50,7 @@ go_test( "ordinality_test.go", "simple_project_test.go", ], + tags = ["no-remote"], deps = [ ":colexecbase", "//pkg/col/coldata", diff --git a/pkg/sql/colexec/colexeccmp/BUILD.bazel b/pkg/sql/colexec/colexeccmp/BUILD.bazel index f137b6d2ebee..a4bebd7a0d2c 100644 --- a/pkg/sql/colexec/colexeccmp/BUILD.bazel +++ b/pkg/sql/colexec/colexeccmp/BUILD.bazel @@ -21,6 +21,7 @@ go_test( name = "colexeccmp_test", srcs = ["dep_test.go"], embed = [":colexeccmp"], + tags = ["no-remote"], deps = ["//pkg/testutils/buildutil"], ) diff --git a/pkg/sql/colexec/colexechash/BUILD.bazel b/pkg/sql/colexec/colexechash/BUILD.bazel index 36d46a01e3b7..7ce372d3b709 100644 --- a/pkg/sql/colexec/colexechash/BUILD.bazel +++ b/pkg/sql/colexec/colexechash/BUILD.bazel @@ -38,6 +38,7 @@ go_test( "main_test.go", ], embed = [":colexechash"], + tags = ["no-remote"], deps = [ "//pkg/col/coldata", "//pkg/col/coldataext", diff --git a/pkg/sql/colexec/colexecjoin/BUILD.bazel b/pkg/sql/colexec/colexecjoin/BUILD.bazel index e26849e799c2..8167e8c76958 100644 --- a/pkg/sql/colexec/colexecjoin/BUILD.bazel +++ b/pkg/sql/colexec/colexecjoin/BUILD.bazel @@ -47,6 +47,7 @@ go_test( "mergejoiner_test.go", ], embed = [":colexecjoin"], + tags = ["no-remote"], deps = [ "//pkg/col/coldata", "//pkg/col/coldataext", diff --git a/pkg/sql/colexec/colexecproj/BUILD.bazel b/pkg/sql/colexec/colexecproj/BUILD.bazel index cfbd81e259bc..89f64d7c8482 100644 --- a/pkg/sql/colexec/colexecproj/BUILD.bazel +++ b/pkg/sql/colexec/colexecproj/BUILD.bazel @@ -42,6 +42,7 @@ go_test( "projection_ops_test.go", ], embed = [":colexecproj"], + tags = ["no-remote"], deps = [ "//pkg/col/coldata", "//pkg/col/coldataext", diff --git a/pkg/sql/colexec/colexecsel/BUILD.bazel b/pkg/sql/colexec/colexecsel/BUILD.bazel index db1a5386fd65..121ab6ae5e54 100644 --- a/pkg/sql/colexec/colexecsel/BUILD.bazel +++ b/pkg/sql/colexec/colexecsel/BUILD.bazel @@ -37,6 +37,7 @@ go_test( "selection_ops_test.go", ], embed = [":colexecsel"], + tags = ["no-remote"], deps = [ "//pkg/col/coldata", "//pkg/col/coldataext", diff --git a/pkg/sql/colexec/colexecspan/BUILD.bazel b/pkg/sql/colexec/colexecspan/BUILD.bazel index 2083cea41cbc..75b4549f290d 100644 --- a/pkg/sql/colexec/colexecspan/BUILD.bazel +++ b/pkg/sql/colexec/colexecspan/BUILD.bazel @@ -39,6 +39,7 @@ go_test( "span_assembler_test.go", ], embed = [":colexecspan"], # keep + tags = ["no-remote"], deps = [ "//pkg/col/coldata", "//pkg/col/coldataext", diff --git a/pkg/sql/colexec/colexecutils/BUILD.bazel b/pkg/sql/colexec/colexecutils/BUILD.bazel index 2e6990b74e14..9fe4edc20e45 100644 --- a/pkg/sql/colexec/colexecutils/BUILD.bazel +++ b/pkg/sql/colexec/colexecutils/BUILD.bazel @@ -42,6 +42,7 @@ go_test( "spilling_queue_test.go", ], embed = [":colexecutils"], + tags = ["no-remote"], deps = [ "//pkg/col/coldata", "//pkg/col/coldataext", diff --git a/pkg/sql/colexec/colexecwindow/BUILD.bazel b/pkg/sql/colexec/colexecwindow/BUILD.bazel index 2c77157787e6..051aace60a27 100644 --- a/pkg/sql/colexec/colexecwindow/BUILD.bazel +++ b/pkg/sql/colexec/colexecwindow/BUILD.bazel @@ -54,6 +54,7 @@ go_test( "window_functions_test.go", ], embed = [":colexecwindow"], + tags = ["no-remote"], deps = [ "//pkg/col/coldata", "//pkg/col/coldataext", diff --git a/pkg/sql/colexecop/BUILD.bazel b/pkg/sql/colexecop/BUILD.bazel index 73f3a4b28cab..784641baa6db 100644 --- a/pkg/sql/colexecop/BUILD.bazel +++ b/pkg/sql/colexecop/BUILD.bazel @@ -26,6 +26,7 @@ go_test( size = "small", srcs = ["dep_test.go"], embed = [":colexecop"], + tags = ["no-remote"], deps = [ "//pkg/testutils/buildutil", "//pkg/util/leaktest", diff --git a/pkg/sql/colflow/BUILD.bazel b/pkg/sql/colflow/BUILD.bazel index 68d931945542..adb5f63d4959 100644 --- a/pkg/sql/colflow/BUILD.bazel +++ b/pkg/sql/colflow/BUILD.bazel @@ -72,6 +72,7 @@ go_test( "vectorized_panic_propagation_test.go", ], embed = [":colflow"], + tags = ["no-remote"], deps = [ "//pkg/base", "//pkg/col/coldata", diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index ce6343f6b5c3..7d2d69e82a69 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1232,9 +1232,9 @@ type ExecutorConfig struct { // CollectionFactory is used to construct a descs.Collection. CollectionFactory *descs.CollectionFactory - // SpanConfigReconciliationJobDeps are used to drive the span config - // reconciliation job. - SpanConfigReconciliationJobDeps spanconfig.ReconciliationDependencies + // SpanConfigReconciler is used to drive the span config reconciliation job + // and related migrations. + SpanConfigReconciler spanconfig.Reconciler // SpanConfigKVAccessor is used when creating and deleting tenant // records. diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 731ea24937ca..99c556e24e17 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -86,6 +86,7 @@ go_test( "main_test.go", ], embed = [":execinfra"], + tags = ["no-remote"], deps = [ "//pkg/security", "//pkg/security/securitytest", diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index fde72f30b317..48c3842798eb 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -62,6 +62,7 @@ go_test( "utils_test.go", ], embed = [":flowinfra"], + tags = ["no-remote"], deps = [ "//pkg/base", "//pkg/gossip", diff --git a/pkg/sql/job_exec_context.go b/pkg/sql/job_exec_context.go index db7a162b8fa9..c6df2280ddf3 100644 --- a/pkg/sql/job_exec_context.go +++ b/pkg/sql/job_exec_context.go @@ -61,8 +61,8 @@ func (e *plannerJobExecContext) User() security.SQLUsername { return e.p.Us func (e *plannerJobExecContext) MigrationJobDeps() migration.JobDeps { return e.p.MigrationJobDeps() } -func (e *plannerJobExecContext) SpanConfigReconciliationJobDeps() spanconfig.ReconciliationDependencies { - return e.p.SpanConfigReconciliationJobDeps() +func (e *plannerJobExecContext) SpanConfigReconciler() spanconfig.Reconciler { + return e.p.SpanConfigReconciler() } // JobExecContext provides the execution environment for a job. It is what is @@ -84,5 +84,5 @@ type JobExecContext interface { LeaseMgr() *lease.Manager User() security.SQLUsername MigrationJobDeps() migration.JobDeps - SpanConfigReconciliationJobDeps() spanconfig.ReconciliationDependencies + SpanConfigReconciler() spanconfig.Reconciler } diff --git a/pkg/sql/job_exec_context_test_util.go b/pkg/sql/job_exec_context_test_util.go index d591e988f3be..ad7fa5f7bce9 100644 --- a/pkg/sql/job_exec_context_test_util.go +++ b/pkg/sql/job_exec_context_test_util.go @@ -70,7 +70,7 @@ func (p *FakeJobExecContext) MigrationJobDeps() migration.JobDeps { panic("unimplemented") } -// SpanConfigReconciliationJobDeps implements the JobExecContext interface. -func (p *FakeJobExecContext) SpanConfigReconciliationJobDeps() spanconfig.ReconciliationDependencies { +// SpanConfigReconciler implements the JobExecContext interface. +func (p *FakeJobExecContext) SpanConfigReconciler() spanconfig.Reconciler { panic("unimplemented") } diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index 93dedf8e025d..262cade7561e 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -100,7 +100,7 @@ type PlanHookState interface { CreateSchemaNamespaceEntry(ctx context.Context, schemaNameKey roachpb.Key, schemaID descpb.ID) error MigrationJobDeps() migration.JobDeps - SpanConfigReconciliationJobDeps() spanconfig.ReconciliationDependencies + SpanConfigReconciler() spanconfig.Reconciler BufferClientNotice(ctx context.Context, notice pgnotice.Notice) } diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 1fb2a3af8216..cb228116dd86 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -561,9 +561,9 @@ func (p *planner) MigrationJobDeps() migration.JobDeps { return p.execCfg.MigrationJobDeps } -// SpanConfigReconciliationJobDeps returns the spanconfig.ReconciliationJobDeps. -func (p *planner) SpanConfigReconciliationJobDeps() spanconfig.ReconciliationDependencies { - return p.execCfg.SpanConfigReconciliationJobDeps +// SpanConfigReconciler returns the spanconfig.Reconciler. +func (p *planner) SpanConfigReconciler() spanconfig.Reconciler { + return p.execCfg.SpanConfigReconciler } // GetTypeFromValidSQLSyntax implements the tree.EvalPlanner interface. diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index d5fcb11016f1..53d149731093 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -132,6 +132,7 @@ go_test( "zigzagjoiner_test.go", ], embed = [":rowexec"], + tags = ["no-remote"], deps = [ "//pkg/base", "//pkg/gossip", diff --git a/pkg/sql/rowflow/BUILD.bazel b/pkg/sql/rowflow/BUILD.bazel index bda53523c7f8..2ffd09dc319b 100644 --- a/pkg/sql/rowflow/BUILD.bazel +++ b/pkg/sql/rowflow/BUILD.bazel @@ -38,6 +38,7 @@ go_test( "routers_test.go", ], embed = [":rowflow"], + tags = ["no-remote"], deps = [ "//pkg/base", "//pkg/keys", diff --git a/pkg/sql/stmtdiagnostics/BUILD.bazel b/pkg/sql/stmtdiagnostics/BUILD.bazel index 843c6e81aae3..4b90a14f8edb 100644 --- a/pkg/sql/stmtdiagnostics/BUILD.bazel +++ b/pkg/sql/stmtdiagnostics/BUILD.bazel @@ -35,6 +35,7 @@ go_test( "statement_diagnostics_test.go", ], embed = [":stmtdiagnostics"], + tags = ["no-remote"], deps = [ "//pkg/base", "//pkg/keys",