From 2e00964f6ba48f190929665c39933148cfaf8592 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 18 Aug 2021 14:24:36 -0400 Subject: [PATCH 1/2] kvserver: abstract away system config span usage in kv Part of #67679. We'll hide `config.SystemConfig` behind the `spanconfig.StoreReader` interface, and use that instead in the various queues that need access to the system config span. In future PRs we'll introduce a data structure that maintains a mapping between spans and configs that implements this same interface, except it will be powered by a view over `system.span_configurations` that was introduced in \#69047. When we do make that switch, i.e. have KV consult this new thing for splits, merges, GC and replication, instead of the gossip backed system config span, ideally it'd be as easy as swapping the source. This PR helps pave the way for just that. In #66348 we described how `zonepb.ZoneConfigs` going forward were going to be an exclusively SQL-level construct. Consequently we purge[*] all usages of it in KV, storing on each replica a `roachpb.SpanConfig` instead. [*]: The only remaining use is what powers our replication reports, which does not extend well to multi-tenancy in its current form and needs replacing. Release note: None Release justification: low risk, high benefit changes to existing functionality --- pkg/cli/debug.go | 16 +- pkg/config/system.go | 12 + pkg/config/system_test.go | 7 +- pkg/config/zonepb/zone.go | 53 +- pkg/config/zonepb/zone_test.go | 54 +- pkg/kv/kvclient/kvcoord/BUILD.bazel | 1 - pkg/kv/kvclient/kvcoord/split_test.go | 10 +- pkg/kv/kvserver/BUILD.bazel | 2 + pkg/kv/kvserver/allocator.go | 95 +- pkg/kv/kvserver/allocator_scorer.go | 3 +- pkg/kv/kvserver/allocator_scorer_test.go | 222 ++--- pkg/kv/kvserver/allocator_test.go | 830 ++++++++---------- pkg/kv/kvserver/client_merge_test.go | 35 +- .../client_replica_backpressure_test.go | 16 +- pkg/kv/kvserver/client_replica_test.go | 4 +- pkg/kv/kvserver/consistency_queue.go | 11 +- pkg/kv/kvserver/constraint/BUILD.bazel | 5 +- pkg/kv/kvserver/constraint/analyzer.go | 13 +- pkg/kv/kvserver/gc/BUILD.bazel | 2 - pkg/kv/kvserver/gc/gc.go | 19 +- pkg/kv/kvserver/gc/gc_old_test.go | 26 +- pkg/kv/kvserver/gc/gc_random_test.go | 15 +- pkg/kv/kvserver/gc/gc_test.go | 26 +- pkg/kv/kvserver/gc_queue.go | 41 +- pkg/kv/kvserver/gc_queue_test.go | 68 +- pkg/kv/kvserver/helpers_test.go | 5 +- pkg/kv/kvserver/merge_queue.go | 19 +- pkg/kv/kvserver/merge_queue_test.go | 4 +- pkg/kv/kvserver/queue.go | 68 +- pkg/kv/kvserver/queue_concurrency_test.go | 17 +- pkg/kv/kvserver/queue_helpers_testutil.go | 4 +- pkg/kv/kvserver/queue_test.go | 79 +- pkg/kv/kvserver/raft_log_queue.go | 17 +- pkg/kv/kvserver/raft_snapshot_queue.go | 13 +- pkg/kv/kvserver/replica.go | 90 +- .../replica_application_state_machine.go | 2 +- pkg/kv/kvserver/replica_command.go | 12 +- pkg/kv/kvserver/replica_gc_queue.go | 13 +- pkg/kv/kvserver/replica_init.go | 2 +- pkg/kv/kvserver/replica_metrics.go | 15 +- .../kvserver/replica_protected_timestamp.go | 8 +- .../replica_protected_timestamp_test.go | 15 +- pkg/kv/kvserver/replica_raftstorage.go | 26 +- pkg/kv/kvserver/replica_range_lease.go | 8 +- pkg/kv/kvserver/replica_test.go | 9 +- pkg/kv/kvserver/replicate_queue.go | 71 +- pkg/kv/kvserver/replicate_queue_test.go | 12 +- pkg/kv/kvserver/reports/reporter.go | 6 + pkg/kv/kvserver/split_queue.go | 27 +- pkg/kv/kvserver/split_queue_test.go | 6 +- pkg/kv/kvserver/store.go | 73 +- pkg/kv/kvserver/store_pool.go | 3 +- pkg/kv/kvserver/store_pool_test.go | 21 +- pkg/kv/kvserver/store_rebalancer.go | 21 +- pkg/kv/kvserver/store_rebalancer_test.go | 16 +- pkg/kv/kvserver/store_split.go | 2 +- pkg/kv/kvserver/store_test.go | 23 +- pkg/kv/kvserver/testing_knobs.go | 4 + pkg/kv/kvserver/ts_maintenance_queue.go | 11 +- pkg/roachpb/BUILD.bazel | 1 + pkg/roachpb/span_config.go | 84 ++ pkg/roachpb/span_config.pb.go | 436 +++++++-- pkg/roachpb/span_config.proto | 28 +- pkg/server/config.go | 3 +- pkg/server/server.go | 2 +- pkg/server/testing_knobs.go | 6 +- pkg/spanconfig/spanconfig.go | 22 + pkg/sql/zone_config_test.go | 45 +- .../localtestcluster/local_test_cluster.go | 2 +- 69 files changed, 1607 insertions(+), 1330 deletions(-) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 0356f87f3b6e..67cb39b738ad 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -32,7 +32,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cli/clierrorplus" "github.com/cockroachdb/cockroach/pkg/cli/syncbench" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -672,7 +671,7 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { defer stopper.Stop(context.Background()) var rangeID roachpb.RangeID - gcTTLInSeconds := int64((24 * time.Hour).Seconds()) + gcTTL := 24 * time.Hour intentAgeThreshold := gc.IntentAgeThreshold.Default() intentBatchSize := gc.MaxIntentsPerCleanupBatch.Default() @@ -683,10 +682,11 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { } } if len(args) > 2 { - var err error - if gcTTLInSeconds, err = parsePositiveInt(args[2]); err != nil { + gcTTLInSeconds, err := parsePositiveInt(args[2]) + if err != nil { return errors.Wrapf(err, "unable to parse %v as TTL", args[2]) } + gcTTL = time.Duration(gcTTLInSeconds) * time.Second } if len(args) > 1 { var err error @@ -736,14 +736,14 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { for _, desc := range descs { snap := db.NewSnapshot() defer snap.Close() - policy := zonepb.GCPolicy{TTLSeconds: int32(gcTTLInSeconds)} now := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - thresh := gc.CalculateThreshold(now, policy) + thresh := gc.CalculateThreshold(now, gcTTL) info, err := gc.Run( context.Background(), &desc, snap, - now, thresh, gc.RunOptions{IntentAgeThreshold: intentAgeThreshold, MaxIntentsPerIntentCleanupBatch: intentBatchSize}, policy, - gc.NoopGCer{}, + now, thresh, + gc.RunOptions{IntentAgeThreshold: intentAgeThreshold, MaxIntentsPerIntentCleanupBatch: intentBatchSize}, + gcTTL, gc.NoopGCer{}, func(_ context.Context, _ []roachpb.Intent) error { return nil }, func(_ context.Context, _ *roachpb.Transaction) error { return nil }, ) diff --git a/pkg/config/system.go b/pkg/config/system.go index 5c855c86f351..523aa48ebd93 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -294,6 +294,18 @@ func (s *SystemConfig) GetZoneConfigForKey(key roachpb.RKey) (*zonepb.ZoneConfig return s.getZoneConfigForKey(DecodeKeyIntoZoneIDAndSuffix(key)) } +// GetSpanConfigForKey looks of the span config for the given key. It's part of +// spanconfig.StoreReader interface. +func (s *SystemConfig) GetSpanConfigForKey( + ctx context.Context, key roachpb.RKey, +) (roachpb.SpanConfig, error) { + zone, err := s.GetZoneConfigForKey(key) + if err != nil { + return roachpb.SpanConfig{}, err + } + return zone.AsSpanConfig(), nil +} + // DecodeKeyIntoZoneIDAndSuffix figures out the zone that the key belongs to. func DecodeKeyIntoZoneIDAndSuffix(key roachpb.RKey) (id SystemTenantObjectID, keySuffix []byte) { objectID, keySuffix, ok := DecodeSystemTenantObjectID(key) diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index 77fff4e43774..97e2fa61be96 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -544,6 +544,7 @@ func TestComputeSplitKeyTenantBoundaries(t *testing.T) { func TestGetZoneConfigForKey(t *testing.T) { defer leaktest.AfterTest(t)() + ctx := context.Background() testCases := []struct { key roachpb.RKey expectedID config.SystemTenantObjectID @@ -614,11 +615,11 @@ func TestGetZoneConfigForKey(t *testing.T) { _ *config.SystemConfig, id config.SystemTenantObjectID, ) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) { objectID = id - return &zonepb.ZoneConfig{}, nil, false, nil + return cfg.DefaultZoneConfig, nil, false, nil } - _, err := cfg.GetZoneConfigForKey(tc.key) + _, err := cfg.GetSpanConfigForKey(ctx, tc.key) if err != nil { - t.Errorf("#%d: GetZoneConfigForKey(%v) got error: %v", tcNum, tc.key, err) + t.Errorf("#%d: GetSpanConfigForKey(%v) got error: %v", tcNum, tc.key, err) } if objectID != tc.expectedID { t.Errorf("#%d: GetZoneConfigForKey(%v) got %d; want %d", tcNum, tc.key, objectID, tc.expectedID) diff --git a/pkg/config/zonepb/zone.go b/pkg/config/zonepb/zone.go index 69d876e4904d..05cfe30fc9c4 100644 --- a/pkg/config/zonepb/zone.go +++ b/pkg/config/zonepb/zone.go @@ -15,7 +15,6 @@ import ( "context" "fmt" "strings" - "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" @@ -206,20 +205,6 @@ func NewZoneConfig() *ZoneConfig { } } -// EmptyCompleteZoneConfig is the zone configuration where -// all fields are set but set to their respective zero values. -func EmptyCompleteZoneConfig() *ZoneConfig { - return &ZoneConfig{ - NumReplicas: proto.Int32(0), - NumVoters: proto.Int32(0), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(0), - GC: &GCPolicy{TTLSeconds: 0}, - InheritedConstraints: true, - InheritedLeasePreferences: true, - } -} - // DefaultZoneConfig is the default zone configuration used when no custom // config has been specified. func DefaultZoneConfig() ZoneConfig { @@ -977,37 +962,6 @@ func (z ZoneConfig) GetSubzoneForKeySuffix(keySuffix []byte) (*Subzone, int32) { return nil, -1 } -// GetNumVoters returns the number of voting replicas for the given zone config. -// -// This method will panic if called on a ZoneConfig with an uninitialized -// NumReplicas attribute. -func (z *ZoneConfig) GetNumVoters() int32 { - if z.NumReplicas == nil { - panic("NumReplicas must not be nil") - } - if z.NumVoters != nil && *z.NumVoters != 0 { - return *z.NumVoters - } - return *z.NumReplicas -} - -// GetNumNonVoters returns the number of non-voting replicas as defined in the -// zone config. -// -// This method will panic if called on a ZoneConfig with an uninitialized -// NumReplicas attribute. -func (z *ZoneConfig) GetNumNonVoters() int32 { - if z.NumReplicas == nil { - panic("NumReplicas must not be nil") - } - if z.NumVoters != nil && *z.NumVoters != 0 { - return *z.NumReplicas - *z.NumVoters - } - // `num_voters` hasn't been explicitly configured. Every replica should be a - // voting replica. - return 0 -} - // SetSubzone installs subzone into the ZoneConfig, overwriting any existing // subzone with the same IndexID and PartitionName. func (z *ZoneConfig) SetSubzone(subzone Subzone) { @@ -1150,11 +1104,6 @@ func (c *Constraint) GetValue() string { return c.Value } -// TTL returns the implies TTL as a time.Duration. -func (m *GCPolicy) TTL() time.Duration { - return time.Duration(m.TTLSeconds) * time.Second -} - // EnsureFullyHydrated returns an assertion error if the zone config is not // fully hydrated. A fully hydrated zone configuration must have all required // fields set, which are RangeMaxBytes, RangeMinBytes, GC, and NumReplicas. @@ -1201,7 +1150,7 @@ func (z *ZoneConfig) toSpanConfig() (roachpb.SpanConfig, error) { // Copy over the values. sc.RangeMinBytes = *z.RangeMinBytes sc.RangeMaxBytes = *z.RangeMaxBytes - sc.GCTTL = z.GC.TTLSeconds + sc.GCPolicy.TTLSeconds = z.GC.TTLSeconds // GlobalReads is false by default. if z.GlobalReads != nil { diff --git a/pkg/config/zonepb/zone_test.go b/pkg/config/zonepb/zone_test.go index 3359b9470b6c..df9189d33709 100644 --- a/pkg/config/zonepb/zone_test.go +++ b/pkg/config/zonepb/zone_test.go @@ -1183,9 +1183,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) { }, }, expectSpanConfig: roachpb.SpanConfig{ - RangeMinBytes: 100000, - RangeMaxBytes: 200000, - GCTTL: 2400, + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCPolicy: roachpb.GCPolicy{ + TTLSeconds: 2400, + }, GlobalReads: false, NumVoters: 0, NumReplicas: 3, @@ -1206,9 +1208,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) { }, }, expectSpanConfig: roachpb.SpanConfig{ - RangeMinBytes: 100000, - RangeMaxBytes: 200000, - GCTTL: 2400, + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCPolicy: roachpb.GCPolicy{ + TTLSeconds: 2400, + }, GlobalReads: true, NumVoters: 0, NumReplicas: 3, @@ -1229,9 +1233,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) { }, }, expectSpanConfig: roachpb.SpanConfig{ - RangeMinBytes: 100000, - RangeMaxBytes: 200000, - GCTTL: 2400, + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCPolicy: roachpb.GCPolicy{ + TTLSeconds: 2400, + }, GlobalReads: false, NumVoters: 0, NumReplicas: 3, @@ -1287,10 +1293,12 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) { expectSpanConfig: roachpb.SpanConfig{ RangeMinBytes: 100000, RangeMaxBytes: 200000, - GCTTL: 2400, - GlobalReads: false, - NumVoters: 0, - NumReplicas: 3, + GCPolicy: roachpb.GCPolicy{ + TTLSeconds: 2400, + }, + GlobalReads: false, + NumVoters: 0, + NumReplicas: 3, Constraints: []roachpb.ConstraintsConjunction{ { NumReplicas: 1, @@ -1333,11 +1341,13 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) { expectSpanConfig: roachpb.SpanConfig{ RangeMinBytes: 100000, RangeMaxBytes: 200000, - GCTTL: 2400, - GlobalReads: false, - NumVoters: 0, - NumReplicas: 3, - Constraints: []roachpb.ConstraintsConjunction{}, + GCPolicy: roachpb.GCPolicy{ + TTLSeconds: 2400, + }, + GlobalReads: false, + NumVoters: 0, + NumReplicas: 3, + Constraints: []roachpb.ConstraintsConjunction{}, VoterConstraints: []roachpb.ConstraintsConjunction{ { Constraints: []roachpb.Constraint{ @@ -1381,9 +1391,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) { }, }, expectSpanConfig: roachpb.SpanConfig{ - RangeMinBytes: 100000, - RangeMaxBytes: 200000, - GCTTL: 2400, + RangeMinBytes: 100000, + RangeMaxBytes: 200000, + GCPolicy: roachpb.GCPolicy{ + TTLSeconds: 2400, + }, GlobalReads: false, NumVoters: 0, NumReplicas: 3, diff --git a/pkg/kv/kvclient/kvcoord/BUILD.bazel b/pkg/kv/kvclient/kvcoord/BUILD.bazel index c2e7e3a46017..caa7d661e6f7 100644 --- a/pkg/kv/kvclient/kvcoord/BUILD.bazel +++ b/pkg/kv/kvclient/kvcoord/BUILD.bazel @@ -184,7 +184,6 @@ go_test( "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_errors//errutil", "@com_github_cockroachdb_redact//:redact", - "@com_github_gogo_protobuf//proto", "@com_github_golang_mock//gomock", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", diff --git a/pkg/kv/kvclient/kvcoord/split_test.go b/pkg/kv/kvclient/kvcoord/split_test.go index 84357d01e73c..9552061812f5 100644 --- a/pkg/kv/kvclient/kvcoord/split_test.go +++ b/pkg/kv/kvclient/kvcoord/split_test.go @@ -19,7 +19,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -32,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/errors" - "github.com/gogo/protobuf/proto" ) // startTestWriter creates a writer which initiates a sequence of @@ -176,15 +174,15 @@ func TestRangeSplitsWithConcurrentTxns(t *testing.T) { func TestRangeSplitsWithWritePressure(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - // Override default zone config. - cfg := zonepb.DefaultZoneConfigRef() - cfg.RangeMaxBytes = proto.Int64(1 << 18) + // Override default span config. + cfg := kvserver.TestingDefaultSpanConfig() + cfg.RangeMaxBytes = 1 << 18 // Manually create the local test cluster so that the split queue // is not disabled (LocalTestCluster disables it by default). s := &localtestcluster.LocalTestCluster{ Cfg: kvserver.StoreConfig{ - DefaultZoneConfig: cfg, + DefaultSpanConfig: cfg, }, StoreTestingKnobs: &kvserver.StoreTestingKnobs{ DisableScanner: true, diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index b4041fa543ca..57f7512c44a3 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -152,6 +152,7 @@ go_library( "//pkg/server/telemetry", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/spanconfig", "//pkg/sql/sessiondata", "//pkg/sql/sqlutil", "//pkg/storage", @@ -340,6 +341,7 @@ go_test( "//pkg/server/serverpb", "//pkg/server/telemetry", "//pkg/settings/cluster", + "//pkg/spanconfig", "//pkg/sql", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index 7b94b584ae9f..53b33bf2afdc 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -19,7 +19,6 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -238,8 +237,8 @@ const ( // can be retried quickly as soon as new stores come online, or additional // space frees up. type allocatorError struct { - constraints []zonepb.ConstraintsConjunction - voterConstraints []zonepb.ConstraintsConjunction + constraints []roachpb.ConstraintsConjunction + voterConstraints []roachpb.ConstraintsConjunction existingVoterCount int existingNonVoterCount int aliveStores int @@ -438,7 +437,7 @@ func GetNeededNonVoters(numVoters, zoneConfigNonVoterCount, clusterNodes int) in // supplied range, as governed by the supplied zone configuration. It // returns the required action that should be taken and a priority. func (a *Allocator) ComputeAction( - ctx context.Context, zone *zonepb.ZoneConfig, desc *roachpb.RangeDescriptor, + ctx context.Context, conf roachpb.SpanConfig, desc *roachpb.RangeDescriptor, ) (action AllocatorAction, priority float64) { if a.storePool == nil { // Do nothing if storePool is nil for some unittests. @@ -498,14 +497,14 @@ func (a *Allocator) ComputeAction( return action, action.Priority() } - return a.computeAction(ctx, zone, desc.Replicas().VoterDescriptors(), + return a.computeAction(ctx, conf, desc.Replicas().VoterDescriptors(), desc.Replicas().NonVoterDescriptors()) } func (a *Allocator) computeAction( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, voterReplicas []roachpb.ReplicaDescriptor, nonVoterReplicas []roachpb.ReplicaDescriptor, ) (action AllocatorAction, adjustedPriority float64) { @@ -526,7 +525,7 @@ func (a *Allocator) computeAction( // Node count including dead nodes but excluding // decommissioning/decommissioned nodes. clusterNodes := a.storePool.ClusterNodeCount() - neededVoters := GetNeededVoters(zone.GetNumVoters(), clusterNodes) + neededVoters := GetNeededVoters(conf.GetNumVoters(), clusterNodes) desiredQuorum := computeQuorum(neededVoters) quorum := computeQuorum(haveVoters) @@ -622,7 +621,7 @@ func (a *Allocator) computeAction( // // Non-voting replica addition / replacement. haveNonVoters := len(nonVoterReplicas) - neededNonVoters := GetNeededNonVoters(haveVoters, int(zone.GetNumNonVoters()), clusterNodes) + neededNonVoters := GetNeededNonVoters(haveVoters, int(conf.GetNumNonVoters()), clusterNodes) if haveNonVoters < neededNonVoters { action = AllocatorAddNonVoter log.VEventf(ctx, 3, "%s - missing non-voter need=%d, have=%d, priority=%.2f", @@ -726,7 +725,7 @@ type decisionDetails struct { func (a *Allocator) allocateTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, targetType targetReplicaType, ) (*roachpb.StoreDescriptor, string, error) { @@ -735,7 +734,7 @@ func (a *Allocator) allocateTarget( target, details := a.allocateTargetFromList( ctx, candidateStoreList, - zone, + conf, existingVoters, existingNonVoters, a.scorerOptions(), @@ -759,8 +758,8 @@ func (a *Allocator) allocateTarget( ) } return nil, "", &allocatorError{ - voterConstraints: zone.VoterConstraints, - constraints: zone.Constraints, + voterConstraints: conf.VoterConstraints, + constraints: conf.Constraints, existingVoterCount: len(existingVoters), existingNonVoterCount: len(existingNonVoters), aliveStores: aliveStoreCount, @@ -773,10 +772,10 @@ func (a *Allocator) allocateTarget( // voting replicas are ruled out as targets. func (a *Allocator) AllocateVoter( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, ) (*roachpb.StoreDescriptor, string, error) { - return a.allocateTarget(ctx, zone, existingVoters, existingNonVoters, voterTarget) + return a.allocateTarget(ctx, conf, existingVoters, existingNonVoters, voterTarget) } // AllocateNonVoter returns a suitable store for a new allocation of a @@ -784,16 +783,16 @@ func (a *Allocator) AllocateVoter( // _any_ existing replicas are ruled out as targets. func (a *Allocator) AllocateNonVoter( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, ) (*roachpb.StoreDescriptor, string, error) { - return a.allocateTarget(ctx, zone, existingVoters, existingNonVoters, nonVoterTarget) + return a.allocateTarget(ctx, conf, existingVoters, existingNonVoters, nonVoterTarget) } func (a *Allocator) allocateTargetFromList( ctx context.Context, candidateStores StoreList, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, options scorerOptions, allowMultipleReplsPerNode bool, @@ -801,9 +800,9 @@ func (a *Allocator) allocateTargetFromList( ) (*roachpb.StoreDescriptor, string) { existingReplicas := append(existingVoters, existingNonVoters...) analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, - existingReplicas, *zone.NumReplicas, zone.Constraints) + existingReplicas, conf.NumReplicas, conf.Constraints) analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, - existingVoters, zone.GetNumVoters(), zone.VoterConstraints) + existingVoters, conf.GetNumVoters(), conf.VoterConstraints) var constraintsChecker constraintsCheckFn switch t := targetType; t { @@ -853,7 +852,7 @@ func (a *Allocator) allocateTargetFromList( func (a Allocator) simulateRemoveTarget( ctx context.Context, targetStore roachpb.StoreID, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, candidates []roachpb.ReplicaDescriptor, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, @@ -875,7 +874,7 @@ func (a Allocator) simulateRemoveTarget( ) log.VEventf(ctx, 3, "simulating which voter would be removed after adding s%d", targetStore) - return a.RemoveVoter(ctx, zone, candidates, existingVoters, existingNonVoters) + return a.RemoveVoter(ctx, conf, candidates, existingVoters, existingNonVoters) case nonVoterTarget: a.storePool.updateLocalStoreAfterRebalance(targetStore, rangeUsageInfo, roachpb.ADD_NON_VOTER) defer a.storePool.updateLocalStoreAfterRebalance( @@ -885,7 +884,7 @@ func (a Allocator) simulateRemoveTarget( ) log.VEventf(ctx, 3, "simulating which non-voter would be removed after adding s%d", targetStore) - return a.RemoveNonVoter(ctx, zone, candidates, existingVoters, existingNonVoters) + return a.RemoveNonVoter(ctx, conf, candidates, existingVoters, existingNonVoters) default: panic(fmt.Sprintf("unknown targetReplicaType: %s", t)) } @@ -893,7 +892,7 @@ func (a Allocator) simulateRemoveTarget( func (a Allocator) removeTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, candidates []roachpb.ReplicationTarget, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, @@ -912,9 +911,9 @@ func (a Allocator) removeTarget( } candidateStoreList, _, _ := a.storePool.getStoreListFromIDs(candidateStoreIDs, storeFilterNone) analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, - existingReplicas, *zone.NumReplicas, zone.Constraints) + existingReplicas, conf.NumReplicas, conf.Constraints) analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, - existingVoters, zone.GetNumVoters(), zone.VoterConstraints) + existingVoters, conf.GetNumVoters(), conf.VoterConstraints) options := a.scorerOptions() var constraintsChecker constraintsCheckFn @@ -965,14 +964,14 @@ func (a Allocator) removeTarget( // back to selecting a random target from any of the existing voting replicas. func (a Allocator) RemoveVoter( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, voterCandidates []roachpb.ReplicaDescriptor, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, ) (roachpb.ReplicaDescriptor, string, error) { return a.removeTarget( ctx, - zone, + conf, roachpb.MakeReplicaSet(voterCandidates).ReplicationTargets(), existingVoters, existingNonVoters, @@ -987,14 +986,14 @@ func (a Allocator) RemoveVoter( // non-voting replicas. func (a Allocator) RemoveNonVoter( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, nonVoterCandidates []roachpb.ReplicaDescriptor, existingVoters []roachpb.ReplicaDescriptor, existingNonVoters []roachpb.ReplicaDescriptor, ) (roachpb.ReplicaDescriptor, string, error) { return a.removeTarget( ctx, - zone, + conf, roachpb.MakeReplicaSet(nonVoterCandidates).ReplicationTargets(), existingVoters, existingNonVoters, @@ -1004,7 +1003,7 @@ func (a Allocator) RemoveNonVoter( func (a Allocator) rebalanceTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, raftStatus *raft.Status, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, rangeUsageInfo RangeUsageInfo, @@ -1016,9 +1015,9 @@ func (a Allocator) rebalanceTarget( zero := roachpb.ReplicationTarget{} analyzedOverallConstraints := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, existingReplicas, *zone.NumReplicas, zone.Constraints) + ctx, a.storePool.getStoreDescriptor, existingReplicas, conf.NumReplicas, conf.Constraints) analyzedVoterConstraints := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, existingVoters, zone.GetNumVoters(), zone.VoterConstraints) + ctx, a.storePool.getStoreDescriptor, existingVoters, conf.GetNumVoters(), conf.VoterConstraints) var removalConstraintsChecker constraintsCheckFn var rebalanceConstraintsChecker rebalanceConstraintsCheckFn var replicaSetToRebalance, replicasWithExcludedStores []roachpb.ReplicaDescriptor @@ -1111,7 +1110,7 @@ func (a Allocator) rebalanceTarget( removeReplica, removeDetails, err = a.simulateRemoveTarget( ctx, target.store.StoreID, - zone, + conf, replicaCandidates, existingPlusOneNew, otherReplicaSet, @@ -1181,7 +1180,7 @@ func (a Allocator) rebalanceTarget( // opportunity was found). func (a Allocator) RebalanceVoter( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, raftStatus *raft.Status, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, rangeUsageInfo RangeUsageInfo, @@ -1189,7 +1188,7 @@ func (a Allocator) RebalanceVoter( ) (add roachpb.ReplicationTarget, remove roachpb.ReplicationTarget, details string, ok bool) { return a.rebalanceTarget( ctx, - zone, + conf, raftStatus, existingVoters, existingNonVoters, @@ -1213,7 +1212,7 @@ func (a Allocator) RebalanceVoter( // replicas. func (a Allocator) RebalanceNonVoter( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, raftStatus *raft.Status, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, rangeUsageInfo RangeUsageInfo, @@ -1221,7 +1220,7 @@ func (a Allocator) RebalanceNonVoter( ) (add roachpb.ReplicationTarget, remove roachpb.ReplicationTarget, details string, ok bool) { return a.rebalanceTarget( ctx, - zone, + conf, raftStatus, existingVoters, existingNonVoters, @@ -1254,7 +1253,7 @@ func (a *Allocator) scorerOptions() scorerOptions { // to a learner. func (a *Allocator) TransferLeaseTarget( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, existing []roachpb.ReplicaDescriptor, leaseStoreID roachpb.StoreID, stats *replicaStats, @@ -1263,8 +1262,8 @@ func (a *Allocator) TransferLeaseTarget( alwaysAllowDecisionWithoutStats bool, ) roachpb.ReplicaDescriptor { sl, _, _ := a.storePool.getStoreList(storeFilterSuspect) - sl = sl.filter(zone.Constraints) - sl = sl.filter(zone.VoterConstraints) + sl = sl.filter(conf.Constraints) + sl = sl.filter(conf.VoterConstraints) // The only thing we use the storeList for is for the lease mean across the // eligible stores, make that explicit here. candidateLeasesMean := sl.candidateLeases.mean @@ -1280,7 +1279,7 @@ func (a *Allocator) TransferLeaseTarget( // is the current one and checkTransferLeaseSource is false). var preferred []roachpb.ReplicaDescriptor if checkTransferLeaseSource { - preferred = a.preferredLeaseholders(zone, existing) + preferred = a.preferredLeaseholders(conf, existing) } else { // TODO(a-robinson): Should we just always remove the source store from // existing when checkTransferLeaseSource is false? I'd do it now, but @@ -1291,7 +1290,7 @@ func (a *Allocator) TransferLeaseTarget( candidates = append(candidates, repl) } } - preferred = a.preferredLeaseholders(zone, candidates) + preferred = a.preferredLeaseholders(conf, candidates) } if len(preferred) == 1 { if preferred[0].StoreID == leaseStoreID { @@ -1387,7 +1386,7 @@ func (a *Allocator) TransferLeaseTarget( // attributes. func (a *Allocator) ShouldTransferLease( ctx context.Context, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, existing []roachpb.ReplicaDescriptor, leaseStoreID roachpb.StoreID, stats *replicaStats, @@ -1400,7 +1399,7 @@ func (a *Allocator) ShouldTransferLease( // Determine which store(s) is preferred based on user-specified preferences. // If any stores match, only consider those stores as options. If only one // store matches, it's where the lease should be. - preferred := a.preferredLeaseholders(zone, existing) + preferred := a.preferredLeaseholders(conf, existing) if len(preferred) == 1 { return preferred[0].StoreID != leaseStoreID } else if len(preferred) > 1 { @@ -1413,8 +1412,8 @@ func (a *Allocator) ShouldTransferLease( } sl, _, _ := a.storePool.getStoreList(storeFilterSuspect) - sl = sl.filter(zone.Constraints) - sl = sl.filter(zone.VoterConstraints) + sl = sl.filter(conf.Constraints) + sl = sl.filter(conf.VoterConstraints) log.VEventf(ctx, 3, "ShouldTransferLease (lease-holder=%d):\n%s", leaseStoreID, sl) // Only consider live, non-draining, non-suspect replicas. @@ -1692,12 +1691,12 @@ func (a Allocator) shouldTransferLeaseWithoutStats( } func (a Allocator) preferredLeaseholders( - zone *zonepb.ZoneConfig, existing []roachpb.ReplicaDescriptor, + conf roachpb.SpanConfig, existing []roachpb.ReplicaDescriptor, ) []roachpb.ReplicaDescriptor { // Go one preference at a time. As soon as we've found replicas that match a // preference, we don't need to look at the later preferences, because // they're meant to be ordered by priority. - for _, preference := range zone.LeasePreferences { + for _, preference := range conf.LeasePreferences { var preferred []roachpb.ReplicaDescriptor for _, repl := range existing { // TODO(a-robinson): Do all these lookups at once, up front? We could diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 2b9e1272b324..195e50dc6722 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -18,7 +18,6 @@ import ( "sort" "strconv" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -1183,7 +1182,7 @@ func containsStore(stores []roachpb.StoreID, target roachpb.StoreID) bool { // constraintsCheck returns true iff the provided store would be a valid in a // range with the provided constraints. func constraintsCheck( - store roachpb.StoreDescriptor, constraints []zonepb.ConstraintsConjunction, + store roachpb.StoreDescriptor, constraints []roachpb.ConstraintsConjunction, ) bool { if len(constraints) == 0 { return true diff --git a/pkg/kv/kvserver/allocator_scorer_test.go b/pkg/kv/kvserver/allocator_scorer_test.go index fce8aae67c55..cb9d0fb5ec74 100644 --- a/pkg/kv/kvserver/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator_scorer_test.go @@ -20,12 +20,10 @@ import ( "sort" "testing" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/gogo/protobuf/proto" "github.com/kr/pretty" ) @@ -551,15 +549,15 @@ func TestConstraintsCheck(t *testing.T) { testCases := []struct { name string - constraints []zonepb.ConstraintsConjunction + constraints []roachpb.ConstraintsConjunction expected map[roachpb.StoreID]bool }{ { name: "required constraint", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, }, }, @@ -570,10 +568,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "required locality constraints", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "us", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "us", Type: roachpb.Constraint_REQUIRED}, }, }, }, @@ -586,10 +584,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "prohibited constraints", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_PROHIBITED}, }, }, }, @@ -601,10 +599,10 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "prohibited locality constraints", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "us", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "us", Type: roachpb.Constraint_PROHIBITED}, }, }, }, @@ -612,48 +610,12 @@ func TestConstraintsCheck(t *testing.T) { testStoreEurope: true, }, }, - { - name: "positive constraints are ignored", - constraints: []zonepb.ConstraintsConjunction{ - { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, - {Value: "b", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, - {Value: "c", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, - }, - }, - }, - expected: map[roachpb.StoreID]bool{ - testStoreUSa15: true, - testStoreUSa15Dupe: true, - testStoreUSa1: true, - testStoreUSb: true, - testStoreEurope: true, - }, - }, - { - name: "positive locality constraints are ignored", - constraints: []zonepb.ConstraintsConjunction{ - { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "eur", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, - }, - }, - }, - expected: map[roachpb.StoreID]bool{ - testStoreUSa15: true, - testStoreUSa15Dupe: true, - testStoreUSa1: true, - testStoreUSb: true, - testStoreEurope: true, - }, - }, { name: "NumReplicas doesn't affect constraint checking", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "eur", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "eur", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -664,16 +626,16 @@ func TestConstraintsCheck(t *testing.T) { }, { name: "multiple per-replica constraints are respected", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "eur", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "eur", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -706,18 +668,18 @@ func TestAllocateConstraintsCheck(t *testing.T) { testCases := []struct { name string - constraints []zonepb.ConstraintsConjunction - zoneNumReplicas int32 + constraints []roachpb.ConstraintsConjunction + numReplicas int32 existing []roachpb.StoreID expectedValid map[roachpb.StoreID]bool expectedNecessary map[roachpb.StoreID]bool }{ { name: "prohibited constraint", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_PROHIBITED}, }, }, }, @@ -731,10 +693,10 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "required constraint", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, }, }, @@ -747,10 +709,10 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "required constraint with NumReplicas", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 3, }, @@ -767,16 +729,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -797,16 +759,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas and existing replicas", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -822,16 +784,16 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas and not enough existing replicas", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 2, }, @@ -849,23 +811,23 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, }, { - name: "multiple required constraints with NumReplicas and sum(NumReplicas) < zone.NumReplicas", - constraints: []zonepb.ConstraintsConjunction{ + name: "multiple required constraints with NumReplicas and sum(NumReplicas) < conf.NumReplicas", + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, }, - zoneNumReplicas: 3, - existing: nil, + numReplicas: 3, + existing: nil, expectedValid: map[roachpb.StoreID]bool{ testStoreUSa15: true, testStoreUSa15Dupe: true, @@ -881,23 +843,23 @@ func TestAllocateConstraintsCheck(t *testing.T) { }, }, { - name: "multiple required constraints with sum(NumReplicas) < zone.NumReplicas and not enough existing replicas", - constraints: []zonepb.ConstraintsConjunction{ + name: "multiple required constraints with sum(NumReplicas) < conf.NumReplicas and not enough existing replicas", + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 2, }, }, - zoneNumReplicas: 5, - existing: []roachpb.StoreID{testStoreUSa1}, + numReplicas: 5, + existing: []roachpb.StoreID{testStoreUSa1}, expectedValid: map[roachpb.StoreID]bool{ testStoreUSa15: true, testStoreUSa15Dupe: true, @@ -914,13 +876,13 @@ func TestAllocateConstraintsCheck(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - zone := &zonepb.ZoneConfig{ + conf := roachpb.SpanConfig{ Constraints: tc.constraints, - NumReplicas: proto.Int32(tc.zoneNumReplicas), + NumReplicas: tc.numReplicas, } analyzed := constraint.AnalyzeConstraints( context.Background(), getTestStoreDesc, testStoreReplicas(tc.existing), - *zone.NumReplicas, zone.Constraints) + conf.NumReplicas, conf.Constraints) for _, s := range testStores { valid, necessary := allocateConstraintsCheck(s, analyzed) if e, a := tc.expectedValid[s.StoreID], valid; e != a { @@ -944,17 +906,17 @@ func TestRemoveConstraintsCheck(t *testing.T) { valid, necessary bool } testCases := []struct { - name string - constraints []zonepb.ConstraintsConjunction - zoneNumReplicas int32 - expected map[roachpb.StoreID]expected + name string + constraints []roachpb.ConstraintsConjunction + numReplicas int32 + expected map[roachpb.StoreID]expected }{ { name: "prohibited constraint", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_PROHIBITED}, }, }, }, @@ -967,10 +929,10 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, { name: "required constraint", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, }, }, @@ -983,10 +945,10 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, { name: "required constraint with NumReplicas", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 2, }, @@ -1000,16 +962,16 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, { name: "multiple required constraints with NumReplicas", - constraints: []zonepb.ConstraintsConjunction{ + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -1021,16 +983,16 @@ func TestRemoveConstraintsCheck(t *testing.T) { }, }, { - name: "required constraint with NumReplicas and sum(NumReplicas) < zone.NumReplicas", - constraints: []zonepb.ConstraintsConjunction{ + name: "required constraint with NumReplicas and sum(NumReplicas) < conf.NumReplicas", + constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 2, }, }, - zoneNumReplicas: 3, + numReplicas: 3, expected: map[roachpb.StoreID]expected{ testStoreUSa15: {true, false}, testStoreEurope: {true, false}, @@ -1049,12 +1011,12 @@ func TestRemoveConstraintsCheck(t *testing.T) { StoreID: storeID, }) } - zone := &zonepb.ZoneConfig{ + conf := roachpb.SpanConfig{ Constraints: tc.constraints, - NumReplicas: proto.Int32(tc.zoneNumReplicas), + NumReplicas: tc.numReplicas, } analyzed := constraint.AnalyzeConstraints( - context.Background(), getTestStoreDesc, existing, *zone.NumReplicas, zone.Constraints) + context.Background(), getTestStoreDesc, existing, conf.NumReplicas, conf.Constraints) for storeID, expected := range tc.expected { valid, necessary := removeConstraintsCheck(testStores[storeID], analyzed) if e, a := expected.valid, valid; e != a { diff --git a/pkg/kv/kvserver/allocator_test.go b/pkg/kv/kvserver/allocator_test.go index 192506c4806a..b24f41d2645d 100644 --- a/pkg/kv/kvserver/allocator_test.go +++ b/pkg/kv/kvserver/allocator_test.go @@ -42,7 +42,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" - "github.com/gogo/protobuf/proto" "github.com/olekukonko/tablewriter" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -52,53 +51,58 @@ import ( const firstRangeID = roachpb.RangeID(1) -var simpleZoneConfig = zonepb.ZoneConfig{ - NumReplicas: proto.Int32(1), - Constraints: []zonepb.ConstraintsConjunction{ +var simpleSpanConfig = roachpb.SpanConfig{ + NumReplicas: 1, + Constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, - {Value: "ssd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, + {Value: "ssd", Type: roachpb.Constraint_REQUIRED}, }, }, }, } -var multiDCConfigSSD = zonepb.ZoneConfig{ - NumReplicas: proto.Int32(2), - Constraints: []zonepb.ConstraintsConjunction{ - {Constraints: []zonepb.Constraint{{Value: "ssd", Type: zonepb.Constraint_REQUIRED}}}, +var multiDCConfigSSD = roachpb.SpanConfig{ + NumReplicas: 2, + Constraints: []roachpb.ConstraintsConjunction{ + {Constraints: []roachpb.Constraint{{Value: "ssd", Type: roachpb.Constraint_REQUIRED}}}, }, } -var multiDCConfigConstrainToA = zonepb.ZoneConfig{ - NumReplicas: proto.Int32(2), - Constraints: []zonepb.ConstraintsConjunction{ - {Constraints: []zonepb.Constraint{{Value: "a", Type: zonepb.Constraint_REQUIRED}}}, +var multiDCConfigConstrainToA = roachpb.SpanConfig{ + NumReplicas: 2, + Constraints: []roachpb.ConstraintsConjunction{ + {Constraints: []roachpb.Constraint{{Value: "a", Type: roachpb.Constraint_REQUIRED}}}, }, } -var multiDCConfigUnsatisfiableVoterConstraints = zonepb.ZoneConfig{ - NumReplicas: proto.Int32(2), - VoterConstraints: []zonepb.ConstraintsConjunction{ - {Constraints: []zonepb.Constraint{{Value: "doesNotExist", Type: zonepb.Constraint_REQUIRED}}}, +var multiDCConfigUnsatisfiableVoterConstraints = roachpb.SpanConfig{ + NumReplicas: 2, + VoterConstraints: []roachpb.ConstraintsConjunction{ + {Constraints: []roachpb.Constraint{{Value: "doesNotExist", Type: roachpb.Constraint_REQUIRED}}}, }, } // multiDCConfigVoterAndNonVoter prescribes that one voting replica be placed in // DC "b" and one non-voting replica be placed in DC "a". -var multiDCConfigVoterAndNonVoter = zonepb.ZoneConfig{ - NumReplicas: proto.Int32(2), - Constraints: []zonepb.ConstraintsConjunction{ +var multiDCConfigVoterAndNonVoter = roachpb.SpanConfig{ + NumReplicas: 2, + Constraints: []roachpb.ConstraintsConjunction{ // Constrain the non-voter to "a". - {Constraints: []zonepb.Constraint{{Value: "a", Type: zonepb.Constraint_REQUIRED}}, NumReplicas: 1}, + {Constraints: []roachpb.Constraint{{Value: "a", Type: roachpb.Constraint_REQUIRED}}, NumReplicas: 1}, }, - VoterConstraints: []zonepb.ConstraintsConjunction{ + VoterConstraints: []roachpb.ConstraintsConjunction{ // Constrain the voter to "b". - {Constraints: []zonepb.Constraint{{Value: "b", Type: zonepb.Constraint_REQUIRED}}}, + {Constraints: []roachpb.Constraint{{Value: "b", Type: roachpb.Constraint_REQUIRED}}}, }, } +// emptySpanConfig returns the empty span configuration. +func emptySpanConfig() roachpb.SpanConfig { + return roachpb.SpanConfig{} +} + var singleStore = []*roachpb.StoreDescriptor{ { StoreID: 1, @@ -479,7 +483,7 @@ func TestAllocatorSimpleRetrieval(t *testing.T) { gossiputil.NewStoreGossiper(g).GossipStores(singleStore, t) result, _, err := a.AllocateVoter( context.Background(), - &simpleZoneConfig, + simpleSpanConfig, nil /* existingVoters */, nil, /* existingNonVoters */ ) if err != nil { @@ -498,7 +502,7 @@ func TestAllocatorNoAvailableDisks(t *testing.T) { defer stopper.Stop(context.Background()) result, _, err := a.AllocateVoter( context.Background(), - &simpleZoneConfig, + simpleSpanConfig, nil /* existingVoters */, nil, /* existingNonVoters */ ) if result != nil { @@ -519,7 +523,7 @@ func TestAllocatorTwoDatacenters(t *testing.T) { ctx := context.Background() result1, _, err := a.AllocateVoter( ctx, - &multiDCConfigSSD, + multiDCConfigSSD, nil /* existingVoters */, nil, /* existingNonVoters */ ) if err != nil { @@ -527,7 +531,7 @@ func TestAllocatorTwoDatacenters(t *testing.T) { } result2, _, err := a.AllocateVoter( ctx, - &multiDCConfigSSD, + multiDCConfigSSD, []roachpb.ReplicaDescriptor{{ NodeID: result1.Node.NodeID, StoreID: result1.StoreID, @@ -544,7 +548,7 @@ func TestAllocatorTwoDatacenters(t *testing.T) { // Verify that no result is forthcoming if we already have a replica. result3, _, err := a.AllocateVoter( ctx, - &multiDCConfigSSD, + multiDCConfigSSD, []roachpb.ReplicaDescriptor{ { NodeID: result1.Node.NodeID, @@ -570,13 +574,13 @@ func TestAllocatorExistingReplica(t *testing.T) { gossiputil.NewStoreGossiper(g).GossipStores(sameDCStores, t) result, _, err := a.AllocateVoter( context.Background(), - &zonepb.ZoneConfig{ - NumReplicas: proto.Int32(0), - Constraints: []zonepb.ConstraintsConjunction{ + roachpb.SpanConfig{ + NumReplicas: 0, + Constraints: []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "a", Type: zonepb.Constraint_REQUIRED}, - {Value: "hdd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "a", Type: roachpb.Constraint_REQUIRED}, + {Value: "hdd", Type: roachpb.Constraint_REQUIRED}, }, }, }, @@ -688,7 +692,7 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { for _, tc := range testCases { { result, _, err := a.AllocateVoter( - context.Background(), zonepb.EmptyCompleteZoneConfig(), tc.existing, nil, + context.Background(), emptySpanConfig(), tc.existing, nil, ) if e, a := tc.expectTargetAllocate, result != nil; e != a { t.Errorf( @@ -702,7 +706,7 @@ func TestAllocatorMultipleStoresPerNode(t *testing.T) { var rangeUsageInfo RangeUsageInfo target, _, details, ok := a.RebalanceVoter( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), nil, tc.existing, nil, @@ -772,7 +776,7 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { // After that we should not be seeing replicas move. var rangeUsageInfo RangeUsageInfo for i := 1; i < 40; i++ { - add, remove, _, ok := a.RebalanceVoter(context.Background(), zonepb.EmptyCompleteZoneConfig(), nil, ranges[i].InternalReplicas, nil, rangeUsageInfo, storeFilterThrottled) + add, remove, _, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), nil, ranges[i].InternalReplicas, nil, rangeUsageInfo, storeFilterThrottled) if ok { // Update the descriptor. newReplicas := make([]roachpb.ReplicaDescriptor, 0, len(ranges[i].InternalReplicas)) @@ -804,7 +808,7 @@ func TestAllocatorMultipleStoresPerNodeLopsided(t *testing.T) { // We dont expect any range wanting to move since the system should have // reached a stable state at this point. for i := 1; i < 40; i++ { - _, _, _, ok := a.RebalanceVoter(context.Background(), zonepb.EmptyCompleteZoneConfig(), nil, ranges[i].InternalReplicas, nil, rangeUsageInfo, storeFilterThrottled) + _, _, _, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), nil, ranges[i].InternalReplicas, nil, rangeUsageInfo, storeFilterThrottled) require.False(t, ok) } } @@ -867,7 +871,7 @@ func TestAllocatorRebalance(t *testing.T) { // Every rebalance target must be either store 1 or 2. for i := 0; i < 10; i++ { var rangeUsageInfo RangeUsageInfo - target, _, _, ok := a.RebalanceVoter(ctx, zonepb.EmptyCompleteZoneConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: 3, StoreID: 3}}, nil, rangeUsageInfo, storeFilterThrottled) + target, _, _, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: 3, StoreID: 3}}, nil, rangeUsageInfo, storeFilterThrottled) if !ok { i-- // loop until we find 10 candidates continue @@ -1011,7 +1015,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { } } for i := 0; i < 10; i++ { - result, _, details, ok := a.RebalanceVoter(context.Background(), zonepb.EmptyCompleteZoneConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) + result, _, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) if ok { t.Fatalf("expected no rebalance, but got target s%d; details: %s", result.StoreID, details) } @@ -1024,7 +1028,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { stores[2].Capacity.RangeCount = 46 sg.GossipStores(stores, t) for i := 0; i < 10; i++ { - target, _, details, ok := a.RebalanceVoter(context.Background(), zonepb.EmptyCompleteZoneConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) if ok { t.Fatalf("expected no rebalance, but got target s%d; details: %s", target.StoreID, details) } @@ -1034,7 +1038,7 @@ func TestAllocatorRebalanceTarget(t *testing.T) { stores[1].Capacity.RangeCount = 44 sg.GossipStores(stores, t) for i := 0; i < 10; i++ { - target, origin, details, ok := a.RebalanceVoter(context.Background(), zonepb.EmptyCompleteZoneConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) + target, origin, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), status, replicas, nil, rangeUsageInfo, storeFilterThrottled) expTo := stores[1].StoreID expFrom := stores[0].StoreID if !ok || target.StoreID != expTo || origin.StoreID != expFrom { @@ -1104,7 +1108,7 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { var rangeUsageInfo RangeUsageInfo - target, _, _, ok := a.RebalanceVoter(ctx, zonepb.EmptyCompleteZoneConfig(), nil, c.existing, nil, rangeUsageInfo, storeFilterThrottled) + target, _, _, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, c.existing, nil, rangeUsageInfo, storeFilterThrottled) if c.expected > 0 { if !ok { t.Fatalf("expected %d, but found nil", c.expected) @@ -1294,7 +1298,7 @@ func TestAllocatorRebalanceByCount(t *testing.T) { // Every rebalance target must be store 4 (or nil for case of missing the only option). for i := 0; i < 10; i++ { var rangeUsageInfo RangeUsageInfo - result, _, _, ok := a.RebalanceVoter(ctx, zonepb.EmptyCompleteZoneConfig(), nil, []roachpb.ReplicaDescriptor{{StoreID: stores[0].StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) + result, _, _, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{StoreID: stores[0].StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) if ok && result.StoreID != 4 { t.Errorf("expected store 4; got %d", result.StoreID) } @@ -1362,7 +1366,7 @@ func TestAllocatorTransferLeaseTarget(t *testing.T) { t.Run("", func(t *testing.T) { target := a.TransferLeaseTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), c.existing, c.leaseholder, nil, /* replicaStats */ @@ -1410,26 +1414,26 @@ func TestAllocatorTransferLeaseTargetConstraints(t *testing.T) { existing := replicas(1, 3, 5) - constraint := func(value string) []zonepb.ConstraintsConjunction { - return []zonepb.ConstraintsConjunction{ + constraint := func(value string) []roachpb.ConstraintsConjunction { + return []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "dc", Value: value, Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "dc", Value: value, Type: roachpb.Constraint_REQUIRED}, }, }, } } - constraints := func(value string) *zonepb.ZoneConfig { - return &zonepb.ZoneConfig{ - NumReplicas: proto.Int32(1), + constraints := func(value string) roachpb.SpanConfig { + return roachpb.SpanConfig{ + NumReplicas: 1, Constraints: constraint(value), } } - voterConstraints := func(value string) *zonepb.ZoneConfig { - return &zonepb.ZoneConfig{ - NumReplicas: proto.Int32(1), + voterConstraints := func(value string) roachpb.SpanConfig { + return roachpb.SpanConfig{ + NumReplicas: 1, VoterConstraints: constraint(value), } } @@ -1438,19 +1442,19 @@ func TestAllocatorTransferLeaseTargetConstraints(t *testing.T) { existing []roachpb.ReplicaDescriptor leaseholder roachpb.StoreID expected roachpb.StoreID - zone *zonepb.ZoneConfig + conf roachpb.SpanConfig }{ - {existing: existing, leaseholder: 5, expected: 1, zone: constraints("1")}, - {existing: existing, leaseholder: 5, expected: 1, zone: voterConstraints("1")}, - {existing: existing, leaseholder: 5, expected: 0, zone: constraints("0")}, - {existing: existing, leaseholder: 5, expected: 0, zone: voterConstraints("0")}, - {existing: existing, leaseholder: 5, expected: 1, zone: zonepb.EmptyCompleteZoneConfig()}, + {existing: existing, leaseholder: 5, expected: 1, conf: constraints("1")}, + {existing: existing, leaseholder: 5, expected: 1, conf: voterConstraints("1")}, + {existing: existing, leaseholder: 5, expected: 0, conf: constraints("0")}, + {existing: existing, leaseholder: 5, expected: 0, conf: voterConstraints("0")}, + {existing: existing, leaseholder: 5, expected: 1, conf: emptySpanConfig()}, } for _, c := range testCases { t.Run("", func(t *testing.T) { target := a.TransferLeaseTarget( context.Background(), - c.zone, + c.conf, c.existing, c.leaseholder, nil, /* replicaStats */ @@ -1503,12 +1507,13 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { sg.GossipStores(stores, t) nl.setNodeStatus(1, livenesspb.NodeLivenessStatus_DRAINING) - preferDC1 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "1", Type: zonepb.Constraint_REQUIRED}}}, + preferDC1 := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Key: "dc", Value: "1", Type: roachpb.Constraint_REQUIRED}}}, } - //This means odd nodes. - preferRegion1 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "region", Value: "1", Type: zonepb.Constraint_REQUIRED}}}, + + // This means odd nodes. + preferRegion1 := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Key: "region", Value: "1", Type: roachpb.Constraint_REQUIRED}}}, } existing := []roachpb.ReplicaDescriptor{ @@ -1522,33 +1527,33 @@ func TestAllocatorTransferLeaseTargetDraining(t *testing.T) { leaseholder roachpb.StoreID check bool expected roachpb.StoreID - zone *zonepb.ZoneConfig + conf roachpb.SpanConfig }{ // No existing lease holder, nothing to do. - {existing: existing, leaseholder: 0, check: true, expected: 0, zone: zonepb.EmptyCompleteZoneConfig()}, + {existing: existing, leaseholder: 0, check: true, expected: 0, conf: emptySpanConfig()}, // Store 1 is draining, so it will try to transfer its lease if // checkTransferLeaseSource is false. This behavior isn't relied upon, // though; leases are manually transferred when draining. - {existing: existing, leaseholder: 1, check: true, expected: 0, zone: zonepb.EmptyCompleteZoneConfig()}, - {existing: existing, leaseholder: 1, check: false, expected: 2, zone: zonepb.EmptyCompleteZoneConfig()}, + {existing: existing, leaseholder: 1, check: true, expected: 0, conf: emptySpanConfig()}, + {existing: existing, leaseholder: 1, check: false, expected: 2, conf: emptySpanConfig()}, // Store 2 is not a lease transfer source. - {existing: existing, leaseholder: 2, check: true, expected: 0, zone: zonepb.EmptyCompleteZoneConfig()}, - {existing: existing, leaseholder: 2, check: false, expected: 3, zone: zonepb.EmptyCompleteZoneConfig()}, + {existing: existing, leaseholder: 2, check: true, expected: 0, conf: emptySpanConfig()}, + {existing: existing, leaseholder: 2, check: false, expected: 3, conf: emptySpanConfig()}, // Store 3 is a lease transfer source, but won't transfer to // node 1 because it's draining. - {existing: existing, leaseholder: 3, check: true, expected: 2, zone: zonepb.EmptyCompleteZoneConfig()}, - {existing: existing, leaseholder: 3, check: false, expected: 2, zone: zonepb.EmptyCompleteZoneConfig()}, + {existing: existing, leaseholder: 3, check: true, expected: 2, conf: emptySpanConfig()}, + {existing: existing, leaseholder: 3, check: false, expected: 2, conf: emptySpanConfig()}, // Verify that lease preferences dont impact draining - {existing: existing, leaseholder: 2, check: true, expected: 0, zone: &zonepb.ZoneConfig{LeasePreferences: preferDC1}}, - {existing: existing, leaseholder: 2, check: false, expected: 0, zone: &zonepb.ZoneConfig{LeasePreferences: preferDC1}}, - {existing: existing, leaseholder: 2, check: true, expected: 3, zone: &zonepb.ZoneConfig{LeasePreferences: preferRegion1}}, - {existing: existing, leaseholder: 2, check: false, expected: 3, zone: &zonepb.ZoneConfig{LeasePreferences: preferRegion1}}, + {existing: existing, leaseholder: 2, check: true, expected: 0, conf: roachpb.SpanConfig{LeasePreferences: preferDC1}}, + {existing: existing, leaseholder: 2, check: false, expected: 0, conf: roachpb.SpanConfig{LeasePreferences: preferDC1}}, + {existing: existing, leaseholder: 2, check: true, expected: 3, conf: roachpb.SpanConfig{LeasePreferences: preferRegion1}}, + {existing: existing, leaseholder: 2, check: false, expected: 3, conf: roachpb.SpanConfig{LeasePreferences: preferRegion1}}, } for _, c := range testCases { t.Run("", func(t *testing.T) { target := a.TransferLeaseTarget( context.Background(), - c.zone, + c.conf, c.existing, c.leaseholder, nil, /* replicaStats */ @@ -1681,7 +1686,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { for i, tc := range testCases { var rangeUsageInfo RangeUsageInfo - result, _, details, ok := a.RebalanceVoter(ctx, zonepb.EmptyCompleteZoneConfig(), nil, tc.existing, nil, rangeUsageInfo, storeFilterThrottled) + result, _, details, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, tc.existing, nil, rangeUsageInfo, storeFilterThrottled) var resultID roachpb.StoreID if ok { resultID = result.StoreID @@ -1744,7 +1749,7 @@ func TestAllocatorRebalanceDifferentLocalitySizes(t *testing.T) { for i, tc := range testCases2 { log.Infof(ctx, "case #%d", i) var rangeUsageInfo RangeUsageInfo - result, _, details, ok := a.RebalanceVoter(ctx, zonepb.EmptyCompleteZoneConfig(), nil, tc.existing, nil, rangeUsageInfo, storeFilterThrottled) + result, _, details, ok := a.RebalanceVoter(ctx, emptySpanConfig(), nil, tc.existing, nil, rangeUsageInfo, storeFilterThrottled) var gotExpected bool if !ok { gotExpected = (tc.expected == nil) @@ -1805,7 +1810,7 @@ func TestAllocatorShouldTransferLease(t *testing.T) { t.Run("", func(t *testing.T) { result := a.ShouldTransferLease( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), c.existing, c.leaseholder, nil, /* replicaStats */ @@ -1867,7 +1872,7 @@ func TestAllocatorShouldTransferLeaseDraining(t *testing.T) { t.Run("", func(t *testing.T) { result := a.ShouldTransferLease( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), c.existing, c.leaseholder, nil, /* replicaStats */ @@ -1908,7 +1913,7 @@ func TestAllocatorShouldTransferSuspected(t *testing.T) { t.Helper() result := a.ShouldTransferLease( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), replicas(1, 2, 3), 2, nil, /* replicaStats */ @@ -1957,39 +1962,39 @@ func TestAllocatorLeasePreferences(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(stores, t) - preferDC1 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "1", Type: zonepb.Constraint_REQUIRED}}}, + preferDC1 := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Key: "dc", Value: "1", Type: roachpb.Constraint_REQUIRED}}}, } - preferDC4Then3Then2 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "4", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "3", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "2", Type: zonepb.Constraint_REQUIRED}}}, + preferDC4Then3Then2 := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Key: "dc", Value: "4", Type: roachpb.Constraint_REQUIRED}}}, + {Constraints: []roachpb.Constraint{{Key: "dc", Value: "3", Type: roachpb.Constraint_REQUIRED}}}, + {Constraints: []roachpb.Constraint{{Key: "dc", Value: "2", Type: roachpb.Constraint_REQUIRED}}}, } - preferN2ThenS3 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Value: "n2", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Value: "s3", Type: zonepb.Constraint_REQUIRED}}}, + preferN2ThenS3 := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Value: "n2", Type: roachpb.Constraint_REQUIRED}}}, + {Constraints: []roachpb.Constraint{{Value: "s3", Type: roachpb.Constraint_REQUIRED}}}, } - preferNotS1ThenNotN2 := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Value: "s1", Type: zonepb.Constraint_PROHIBITED}}}, - {Constraints: []zonepb.Constraint{{Value: "n2", Type: zonepb.Constraint_PROHIBITED}}}, + preferNotS1ThenNotN2 := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Value: "s1", Type: roachpb.Constraint_PROHIBITED}}}, + {Constraints: []roachpb.Constraint{{Value: "n2", Type: roachpb.Constraint_PROHIBITED}}}, } - preferNotS1AndNotN2 := []zonepb.LeasePreference{ + preferNotS1AndNotN2 := []roachpb.LeasePreference{ { - Constraints: []zonepb.Constraint{ - {Value: "s1", Type: zonepb.Constraint_PROHIBITED}, - {Value: "n2", Type: zonepb.Constraint_PROHIBITED}, + Constraints: []roachpb.Constraint{ + {Value: "s1", Type: roachpb.Constraint_PROHIBITED}, + {Value: "n2", Type: roachpb.Constraint_PROHIBITED}, }, }, } - preferMatchesNothing := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "dc", Value: "5", Type: zonepb.Constraint_REQUIRED}}}, - {Constraints: []zonepb.Constraint{{Value: "n6", Type: zonepb.Constraint_REQUIRED}}}, + preferMatchesNothing := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Key: "dc", Value: "5", Type: roachpb.Constraint_REQUIRED}}}, + {Constraints: []roachpb.Constraint{{Value: "n6", Type: roachpb.Constraint_REQUIRED}}}, } testCases := []struct { leaseholder roachpb.StoreID existing []roachpb.ReplicaDescriptor - preferences []zonepb.LeasePreference + preferences []roachpb.LeasePreference expectedCheckTrue roachpb.StoreID /* checkTransferLeaseSource = true */ expectedCheckFalse roachpb.StoreID /* checkTransferLeaseSource = false */ }{ @@ -2044,10 +2049,10 @@ func TestAllocatorLeasePreferences(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), LeasePreferences: c.preferences} + conf := roachpb.SpanConfig{LeasePreferences: c.preferences} result := a.ShouldTransferLease( context.Background(), - zone, + conf, c.existing, c.leaseholder, nil, /* replicaStats */ @@ -2058,7 +2063,7 @@ func TestAllocatorLeasePreferences(t *testing.T) { } target := a.TransferLeaseTarget( context.Background(), - zone, + conf, c.existing, c.leaseholder, nil, /* replicaStats */ @@ -2071,7 +2076,7 @@ func TestAllocatorLeasePreferences(t *testing.T) { } target = a.TransferLeaseTarget( context.Background(), - zone, + conf, c.existing, c.leaseholder, nil, /* replicaStats */ @@ -2123,17 +2128,17 @@ func TestAllocatorLeasePreferencesMultipleStoresPerLocality(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(stores, t) - preferEast := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "region", Value: "us-east1", Type: zonepb.Constraint_REQUIRED}}}, + preferEast := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Key: "region", Value: "us-east1", Type: roachpb.Constraint_REQUIRED}}}, } - preferNotEast := []zonepb.LeasePreference{ - {Constraints: []zonepb.Constraint{{Key: "region", Value: "us-east1", Type: zonepb.Constraint_PROHIBITED}}}, + preferNotEast := []roachpb.LeasePreference{ + {Constraints: []roachpb.Constraint{{Key: "region", Value: "us-east1", Type: roachpb.Constraint_PROHIBITED}}}, } testCases := []struct { leaseholder roachpb.StoreID existing []roachpb.ReplicaDescriptor - preferences []zonepb.LeasePreference + preferences []roachpb.LeasePreference expectedCheckTrue roachpb.StoreID /* checkTransferLeaseSource = true */ expectedCheckFalse roachpb.StoreID /* checkTransferLeaseSource = false */ }{ @@ -2151,10 +2156,10 @@ func TestAllocatorLeasePreferencesMultipleStoresPerLocality(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), LeasePreferences: c.preferences} + conf := roachpb.SpanConfig{LeasePreferences: c.preferences} target := a.TransferLeaseTarget( context.Background(), - zone, + conf, c.existing, c.leaseholder, nil, /* replicaStats */ @@ -2167,7 +2172,7 @@ func TestAllocatorLeasePreferencesMultipleStoresPerLocality(t *testing.T) { } target = a.TransferLeaseTarget( context.Background(), - zone, + conf, c.existing, c.leaseholder, nil, /* replicaStats */ @@ -2241,7 +2246,7 @@ func TestAllocatorRemoveBasedOnDiversity(t *testing.T) { for _, c := range testCases { targetVoter, details, err := a.RemoveVoter( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), c.existingVoters, /* voterCandidates */ c.existingVoters, c.existingNonVoters, @@ -2259,7 +2264,7 @@ func TestAllocatorRemoveBasedOnDiversity(t *testing.T) { // diversity score calculations, we would fail here. targetVoter, _, err = a.RemoveVoter( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), c.existingVoters, /* voterCandidates */ c.existingVoters, nil, /* existingNonVoters */ @@ -2271,7 +2276,7 @@ func TestAllocatorRemoveBasedOnDiversity(t *testing.T) { targetNonVoter, _, err := a.RemoveNonVoter( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), c.existingNonVoters, /* nonVoterCandidates */ c.existingVoters, c.existingNonVoters, @@ -2292,7 +2297,7 @@ func TestAllocatorConstraintsAndVoterConstraints(t *testing.T) { name string existingVoters, existingNonVoters []roachpb.ReplicaDescriptor stores []*roachpb.StoreDescriptor - zone *zonepb.ZoneConfig + conf roachpb.SpanConfig expectedVoters, expectedNonVoters []roachpb.StoreID shouldVoterAllocFail, shouldNonVoterAllocFail bool expError string @@ -2300,21 +2305,21 @@ func TestAllocatorConstraintsAndVoterConstraints(t *testing.T) { { name: "one store satisfies constraints for each type of replica", stores: multiDCStores, - zone: &multiDCConfigVoterAndNonVoter, + conf: multiDCConfigVoterAndNonVoter, expectedVoters: []roachpb.StoreID{2}, expectedNonVoters: []roachpb.StoreID{1}, }, { name: "only voter can satisfy constraints", stores: multiDCStores, - zone: &multiDCConfigConstrainToA, + conf: multiDCConfigConstrainToA, expectedVoters: []roachpb.StoreID{1}, shouldNonVoterAllocFail: true, }, { name: "only non_voter can satisfy constraints", stores: multiDCStores, - zone: &multiDCConfigUnsatisfiableVoterConstraints, + conf: multiDCConfigUnsatisfiableVoterConstraints, shouldVoterAllocFail: true, expectedNonVoters: []roachpb.StoreID{1, 2}, }, @@ -2340,7 +2345,7 @@ func TestAllocatorConstraintsAndVoterConstraints(t *testing.T) { // Allocate the voting replica first, before the non-voter. This is the // order in which we'd expect the allocator to repair a given range. See // TestAllocatorComputeAction. - voterTarget, _, err := a.AllocateVoter(ctx, test.zone, test.existingVoters, test.existingNonVoters) + voterTarget, _, err := a.AllocateVoter(ctx, test.conf, test.existingVoters, test.existingNonVoters) if test.shouldVoterAllocFail { require.Errorf(t, err, "expected voter allocation to fail; got %v as a valid target instead", voterTarget) } else { @@ -2349,7 +2354,7 @@ func TestAllocatorConstraintsAndVoterConstraints(t *testing.T) { test.existingVoters = append(test.existingVoters, replicas(voterTarget.StoreID)...) } - nonVoterTarget, _, err := a.AllocateNonVoter(ctx, test.zone, test.existingVoters, test.existingNonVoters) + nonVoterTarget, _, err := a.AllocateNonVoter(ctx, test.conf, test.existingVoters, test.existingNonVoters) if test.shouldNonVoterAllocFail { require.Errorf(t, err, "expected non-voter allocation to fail; got %v as a valid target instead", nonVoterTarget) } else { @@ -2422,7 +2427,7 @@ func TestAllocatorAllocateTargetLocality(t *testing.T) { StoreID: storeID, } } - targetStore, details, err := a.AllocateVoter(context.Background(), zonepb.EmptyCompleteZoneConfig(), existingRepls, nil) + targetStore, details, err := a.AllocateVoter(context.Background(), emptySpanConfig(), existingRepls, nil) if err != nil { t.Fatal(err) } @@ -2540,7 +2545,7 @@ func TestAllocatorRebalanceTargetLocality(t *testing.T) { } } var rangeUsageInfo RangeUsageInfo - target, _, details, ok := a.RebalanceVoter(context.Background(), zonepb.EmptyCompleteZoneConfig(), nil, existingRepls, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := a.RebalanceVoter(context.Background(), emptySpanConfig(), nil, existingRepls, nil, rangeUsageInfo, storeFilterThrottled) if !ok { t.Fatalf("%d: RebalanceVoter(%v) returned no target store; details: %s", i, c.existing, details) } @@ -2559,114 +2564,114 @@ func TestAllocatorRebalanceTargetLocality(t *testing.T) { } var ( - threeSpecificLocalities = []zonepb.ConstraintsConjunction{ + threeSpecificLocalities = []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "c", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "c", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - twoAndOneLocalities = []zonepb.ConstraintsConjunction{ + twoAndOneLocalities = []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 2, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - threeInOneLocality = []zonepb.ConstraintsConjunction{ + threeInOneLocality = []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 3, }, } - twoAndOneNodeAttrs = []zonepb.ConstraintsConjunction{ + twoAndOneNodeAttrs = []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "ssd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "ssd", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 2, }, { - Constraints: []zonepb.Constraint{ - {Value: "hdd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "hdd", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - twoAndOneStoreAttrs = []zonepb.ConstraintsConjunction{ + twoAndOneStoreAttrs = []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "odd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "odd", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 2, }, { - Constraints: []zonepb.Constraint{ - {Value: "even", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "even", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - mixLocalityAndAttrs = []zonepb.ConstraintsConjunction{ + mixLocalityAndAttrs = []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, - {Value: "ssd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "a", Type: roachpb.Constraint_REQUIRED}, + {Value: "ssd", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, - {Value: "odd", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "b", Type: roachpb.Constraint_REQUIRED}, + {Value: "odd", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Value: "even", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "even", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, } - twoSpecificLocalities = []zonepb.ConstraintsConjunction{ + twoSpecificLocalities = []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "a", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "a", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, { - Constraints: []zonepb.Constraint{ - {Key: "datacenter", Value: "b", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Key: "datacenter", Value: "b", Type: roachpb.Constraint_REQUIRED}, }, NumReplicas: 1, }, @@ -2748,10 +2753,10 @@ func TestAllocateCandidatesExcludeNonReadyNodes(t *testing.T) { } } // No constraints. - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: nil} + conf := roachpb.SpanConfig{} analyzed := constraint.AnalyzeConstraints( - context.Background(), a.storePool.getStoreDescriptor, existingRepls, *zone.NumReplicas, - zone.Constraints) + context.Background(), a.storePool.getStoreDescriptor, existingRepls, conf.NumReplicas, + conf.Constraints) allocationConstraintsChecker := voterConstraintsCheckerForAllocation(analyzed, constraint.EmptyAnalyzedConstraints) removalConstraintsChecker := voterConstraintsCheckerForRemoval(analyzed, constraint.EmptyAnalyzedConstraints) rebalanceConstraintsChecker := voterConstraintsCheckerForRebalance(analyzed, constraint.EmptyAnalyzedConstraints) @@ -2825,7 +2830,7 @@ func TestAllocatorNonVoterAllocationExcludesVoterNodes(t *testing.T) { name string existingVoters, existingNonVoters []roachpb.ReplicaDescriptor stores []*roachpb.StoreDescriptor - zone *zonepb.ZoneConfig + conf roachpb.SpanConfig expected roachpb.StoreID shouldFail bool expError string @@ -2834,7 +2839,7 @@ func TestAllocatorNonVoterAllocationExcludesVoterNodes(t *testing.T) { name: "voters only", existingNonVoters: replicas(1, 2, 3, 4), stores: sameDCStores, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), // Expect that that the store that doesn't have any replicas would be // the one to receive a new non-voter. expected: roachpb.StoreID(5), @@ -2843,7 +2848,7 @@ func TestAllocatorNonVoterAllocationExcludesVoterNodes(t *testing.T) { name: "non-voters only", existingNonVoters: replicas(1, 2, 3, 4), stores: sameDCStores, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), expected: roachpb.StoreID(5), }, { @@ -2851,7 +2856,7 @@ func TestAllocatorNonVoterAllocationExcludesVoterNodes(t *testing.T) { existingVoters: replicas(1, 2), existingNonVoters: replicas(3, 4), stores: sameDCStores, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), expected: roachpb.StoreID(5), }, { @@ -2860,7 +2865,7 @@ func TestAllocatorNonVoterAllocationExcludesVoterNodes(t *testing.T) { // `multiDCConfigConstrainToA`. existingVoters: replicas(1), stores: multiDCStores, - zone: &multiDCConfigConstrainToA, + conf: multiDCConfigConstrainToA, shouldFail: true, expError: "0 of 2 live stores are able to take a new replica for the range", }, @@ -2870,7 +2875,7 @@ func TestAllocatorNonVoterAllocationExcludesVoterNodes(t *testing.T) { // `multiDCConfigConstrainToA`. existingNonVoters: replicas(1), stores: multiDCStores, - zone: &multiDCConfigConstrainToA, + conf: multiDCConfigConstrainToA, shouldFail: true, expError: "0 of 2 live stores are able to take a new replica for the range", }, @@ -2884,7 +2889,7 @@ func TestAllocatorNonVoterAllocationExcludesVoterNodes(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(test.stores, t) - result, _, err := a.AllocateNonVoter(ctx, test.zone, test.existingVoters, test.existingNonVoters) + result, _, err := a.AllocateNonVoter(ctx, test.conf, test.existingVoters, test.existingNonVoters) if test.shouldFail { require.Error(t, err) require.Regexp(t, test.expError, err) @@ -2910,7 +2915,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { // stores from multiDiversityDCStores would be the best addition to the range // purely on the basis of constraint satisfaction and locality diversity. testCases := []struct { - constraints []zonepb.ConstraintsConjunction + constraints []roachpb.ConstraintsConjunction existing []roachpb.StoreID expected []roachpb.StoreID }{ @@ -3104,10 +3109,10 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { StoreID: storeID, } } - zone := &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: tc.constraints} + conf := roachpb.SpanConfig{Constraints: tc.constraints} analyzed := constraint.AnalyzeConstraints( - context.Background(), a.storePool.getStoreDescriptor, existingRepls, *zone.NumReplicas, - zone.Constraints) + context.Background(), a.storePool.getStoreDescriptor, existingRepls, conf.NumReplicas, + conf.Constraints) checkFn := voterConstraintsCheckerForAllocation(analyzed, constraint.EmptyAnalyzedConstraints) candidates := rankedCandidateListForAllocation( @@ -3155,7 +3160,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { // stores would be best to remove if we had to remove one purely on the basis // of constraint-matching and locality diversity. testCases := []struct { - constraints []zonepb.ConstraintsConjunction + constraints []roachpb.ConstraintsConjunction existing []roachpb.StoreID expected []roachpb.StoreID }{ @@ -3338,7 +3343,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { analyzed := constraint.AnalyzeConstraints(ctx, a.storePool.getStoreDescriptor, existingRepls, 0 /* numReplicas */, tc.constraints) - // Check behavior in a zone config where `voter_constraints` are empty. + // Check behavior in a span config where `voter_constraints` are empty. checkFn := voterConstraintsCheckerForRemoval(analyzed, constraint.EmptyAnalyzedConstraints) candidates := rankedCandidateListForRemoval(sl, checkFn, @@ -3392,7 +3397,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { type testCase struct { name string stores []*roachpb.StoreDescriptor - zone *zonepb.ZoneConfig + conf roachpb.SpanConfig existingVoters, existingNonVoters []roachpb.ReplicaDescriptor expectNoAction bool expectedRemoveTargets, expectedAddTargets []roachpb.StoreID @@ -3401,7 +3406,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { { name: "no-op", stores: multiDiversityDCStores, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), existingVoters: replicas(1), existingNonVoters: replicas(3), expectNoAction: true, @@ -3413,7 +3418,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { { name: "diversity among non-voters", stores: multiDiversityDCStores, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), existingVoters: replicas(1, 2), existingNonVoters: replicas(3, 4, 6), expectedRemoveTargets: []roachpb.StoreID{3, 4}, @@ -3422,7 +3427,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { { name: "diversity among all existing replicas", stores: multiDiversityDCStores, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), existingVoters: replicas(1), existingNonVoters: replicas(2, 4, 6), expectedRemoveTargets: []roachpb.StoreID{2}, @@ -3434,7 +3439,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { name: "move off of nodes with full disk", // NB: Store 1 has a 97.5% full disk. stores: oneStoreWithFullDisk, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), existingVoters: replicas(3), existingNonVoters: replicas(1), expectedRemoveTargets: []roachpb.StoreID{1}, @@ -3444,7 +3449,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { name: "move off of nodes with too many ranges", // NB: Store 1 has 3x the number of ranges as the other stores. stores: oneStoreWithTooManyRanges, - zone: zonepb.EmptyCompleteZoneConfig(), + conf: emptySpanConfig(), existingVoters: replicas(3), existingNonVoters: replicas(1), expectedRemoveTargets: []roachpb.StoreID{1}, @@ -3456,7 +3461,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { name: "already on a store that satisfies constraints for non_voters", stores: multiDCStores, // Constrain a voter to store 2 and a non_voter to store 1. - zone: &multiDCConfigVoterAndNonVoter, + conf: multiDCConfigVoterAndNonVoter, existingVoters: replicas(2), existingNonVoters: replicas(1), expectNoAction: true, @@ -3465,7 +3470,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { name: "need to rebalance to conform to constraints", stores: multiDCStores, // Constrain a non_voter to store 1. - zone: &multiDCConfigVoterAndNonVoter, + conf: multiDCConfigVoterAndNonVoter, existingVoters: nil, existingNonVoters: replicas(2), expectedRemoveTargets: []roachpb.StoreID{2}, @@ -3477,7 +3482,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { // constraints. name: "need to rebalance, but cannot because a voter already exists", stores: multiDCStores, - zone: &multiDCConfigVoterAndNonVoter, + conf: multiDCConfigVoterAndNonVoter, existingVoters: replicas(1), existingNonVoters: replicas(2), expectNoAction: true, @@ -3501,7 +3506,7 @@ func TestAllocatorRebalanceNonVoters(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(test.stores, t) add, remove, _, ok := a.RebalanceNonVoter(ctx, - test.zone, + test.conf, nil, test.existingVoters, test.existingNonVoters, @@ -3535,17 +3540,17 @@ func TestVotersCanRebalanceToNonVoterStores(t *testing.T) { sg := gossiputil.NewStoreGossiper(g) sg.GossipStores(multiDiversityDCStores, t) - zone := zonepb.ZoneConfig{ - NumReplicas: proto.Int32(4), - NumVoters: proto.Int32(2), + conf := roachpb.SpanConfig{ + NumReplicas: 4, + NumVoters: 2, // We constrain 2 voting replicas to datacenter "a" (stores 1 and 2) but // place non voting replicas there. In order to achieve constraints // conformance, each of the voters must want to move to one of these stores. - VoterConstraints: []zonepb.ConstraintsConjunction{ + VoterConstraints: []roachpb.ConstraintsConjunction{ { NumReplicas: 2, - Constraints: []zonepb.Constraint{ - {Type: zonepb.Constraint_REQUIRED, Key: "datacenter", Value: "a"}, + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_REQUIRED, Key: "datacenter", Value: "a"}, }, }, }, @@ -3556,7 +3561,7 @@ func TestVotersCanRebalanceToNonVoterStores(t *testing.T) { existingVoters := replicas(3, 4) add, remove, _, ok := a.RebalanceVoter( ctx, - &zone, + conf, nil, existingVoters, existingNonVoters, @@ -3591,11 +3596,11 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { candidates []roachpb.StoreID } testCases := []struct { - constraints []zonepb.ConstraintsConjunction - zoneNumReplicas int32 - existing []roachpb.StoreID - expected []rebalanceStoreIDs - validTargets []roachpb.StoreID + constraints []roachpb.ConstraintsConjunction + numReplicas int32 + existing []roachpb.StoreID + expected []rebalanceStoreIDs + validTargets []roachpb.StoreID }{ { constraints: threeSpecificLocalities, @@ -4179,30 +4184,30 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { validTargets: []roachpb.StoreID{3, 4, 6, 8}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{1, 3, 5}, - expected: []rebalanceStoreIDs{}, - validTargets: []roachpb.StoreID{}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{1, 3, 5}, + expected: []rebalanceStoreIDs{}, + validTargets: []roachpb.StoreID{}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{1, 3, 7}, - expected: []rebalanceStoreIDs{}, - validTargets: []roachpb.StoreID{}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{1, 3, 7}, + expected: []rebalanceStoreIDs{}, + validTargets: []roachpb.StoreID{}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{2, 4, 8}, - expected: []rebalanceStoreIDs{}, - validTargets: []roachpb.StoreID{}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{2, 4, 8}, + expected: []rebalanceStoreIDs{}, + validTargets: []roachpb.StoreID{}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{1, 2, 3}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{1, 2, 3}, expected: []rebalanceStoreIDs{ { existing: []roachpb.StoreID{1}, @@ -4216,9 +4221,9 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { validTargets: []roachpb.StoreID{5, 6, 7, 8}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{2, 3, 4}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{2, 3, 4}, expected: []rebalanceStoreIDs{ { existing: []roachpb.StoreID{3}, @@ -4232,9 +4237,9 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { validTargets: []roachpb.StoreID{5, 6, 7, 8}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{1, 2, 5}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{1, 2, 5}, expected: []rebalanceStoreIDs{ { existing: []roachpb.StoreID{1}, @@ -4252,9 +4257,9 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { validTargets: []roachpb.StoreID{3, 4}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{3, 4, 5}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{3, 4, 5}, expected: []rebalanceStoreIDs{ { existing: []roachpb.StoreID{3}, @@ -4272,9 +4277,9 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { validTargets: []roachpb.StoreID{1, 2}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{1, 5, 7}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{1, 5, 7}, expected: []rebalanceStoreIDs{ { existing: []roachpb.StoreID{5}, @@ -4288,9 +4293,9 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { validTargets: []roachpb.StoreID{3, 4}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{1, 5, 6}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{1, 5, 6}, expected: []rebalanceStoreIDs{ { existing: []roachpb.StoreID{5}, @@ -4304,9 +4309,9 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { validTargets: []roachpb.StoreID{3, 4}, }, { - constraints: twoSpecificLocalities, - zoneNumReplicas: 3, - existing: []roachpb.StoreID{5, 6, 7}, + constraints: twoSpecificLocalities, + numReplicas: 3, + existing: []roachpb.StoreID{5, 6, 7}, expected: []rebalanceStoreIDs{ { existing: []roachpb.StoreID{5}, @@ -4334,13 +4339,13 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { } } var rangeUsageInfo RangeUsageInfo - zone := &zonepb.ZoneConfig{ + conf := roachpb.SpanConfig{ Constraints: tc.constraints, - NumReplicas: proto.Int32(tc.zoneNumReplicas), + NumReplicas: tc.numReplicas, } analyzed := constraint.AnalyzeConstraints( context.Background(), a.storePool.getStoreDescriptor, existingRepls, - *zone.NumReplicas, zone.Constraints) + conf.NumReplicas, conf.Constraints) removalConstraintsChecker := voterConstraintsCheckerForRemoval( analyzed, constraint.EmptyAnalyzedConstraints, @@ -4382,7 +4387,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { } else { // Also verify that RebalanceVoter picks out one of the best options as // the final rebalance choice. - target, _, details, ok := a.RebalanceVoter(context.Background(), zone, nil, existingRepls, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := a.RebalanceVoter(context.Background(), conf, nil, existingRepls, nil, rangeUsageInfo, storeFilterThrottled) var found bool if !ok && len(tc.validTargets) == 0 { found = true @@ -4557,7 +4562,7 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { }) target := a.TransferLeaseTarget( context.Background(), - zonepb.EmptyCompleteZoneConfig(), + emptySpanConfig(), existing, c.leaseholder, c.stats, @@ -4755,7 +4760,7 @@ func TestAllocatorRemoveTargetBasedOnCapacity(t *testing.T) { // Repeat this test 10 times, it should always be either store 2 or 3. for i := 0; i < 10; i++ { - targetRepl, _, err := a.RemoveVoter(ctx, zonepb.EmptyCompleteZoneConfig(), replicas, replicas, + targetRepl, _, err := a.RemoveVoter(ctx, emptySpanConfig(), replicas, replicas, nil) if err != nil { t.Fatal(err) @@ -4774,17 +4779,15 @@ func TestAllocatorComputeAction(t *testing.T) { // Each test case should describe a repair situation which has a lower // priority than the previous test case. testCases := []struct { - zone zonepb.ZoneConfig + conf roachpb.SpanConfig desc roachpb.RangeDescriptor expectedAction AllocatorAction }{ // Need three replicas, have three, one is on a dead store. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -4809,11 +4812,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, one is on a dead store. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -4848,11 +4849,10 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need 1 non-voter but a voter is on a dead store. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - NumVoters: proto.Int32(3), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + NumVoters: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -4883,11 +4883,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need 3 replicas, have 2, but one of them is dead so we don't have quorum. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -4910,11 +4908,9 @@ func TestAllocatorComputeAction(t *testing.T) { // Need three replicas, have two. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -4934,11 +4930,10 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need a voter and a non-voter. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - NumVoters: proto.Int32(3), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + NumVoters: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -4964,11 +4959,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, have four, one is on a dead store. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -4998,11 +4991,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, have four. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5032,11 +5023,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have four, one is on a dead store. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5066,11 +5055,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need five replicas, have six, one is on a dead store. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5110,11 +5097,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have five, one is on a dead store. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5149,11 +5134,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have four. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5183,11 +5166,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have five. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5222,11 +5203,10 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need 2 non-voting replicas, have 2 but one of them is on a dead node. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - NumVoters: proto.Int32(3), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + NumVoters: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5263,11 +5243,10 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need 2 non-voting replicas, have none. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - NumVoters: proto.Int32(3), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 5, + NumVoters: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5292,11 +5271,10 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need 2 non-voting replicas, have 1 but its on a dead node. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - NumVoters: proto.Int32(1), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + NumVoters: 1, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5316,11 +5294,10 @@ func TestAllocatorComputeAction(t *testing.T) { expectedAction: AllocatorAddNonVoter, }, { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(2), - NumVoters: proto.Int32(1), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 2, + NumVoters: 1, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5347,11 +5324,10 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need 1 non-voting replicas, have 2. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(2), - NumVoters: proto.Int32(1), - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 2, + NumVoters: 1, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5380,11 +5356,9 @@ func TestAllocatorComputeAction(t *testing.T) { // be a noop because there aren't enough live replicas for // a quorum. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5409,11 +5383,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have three, none of the replicas in the store pool. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5438,11 +5410,9 @@ func TestAllocatorComputeAction(t *testing.T) { }, // Need three replicas, have three. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - Constraints: []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "us-east", Type: zonepb.Constraint_DEPRECATED_POSITIVE}}}}, - RangeMinBytes: proto.Int64(0), - RangeMaxBytes: proto.Int64(64000), + conf: roachpb.SpanConfig{ + NumReplicas: 3, + RangeMaxBytes: 64000, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5484,7 +5454,7 @@ func TestAllocatorComputeAction(t *testing.T) { lastPriority := float64(999999999) for i, tcase := range testCases { - action, priority := a.ComputeAction(ctx, &tcase.zone, &tcase.desc) + action, priority := a.ComputeAction(ctx, tcase.conf, &tcase.desc) if tcase.expectedAction != action { t.Errorf("Test case %d expected action %q, got action %q", i, allocatorActionNames[tcase.expectedAction], allocatorActionNames[action]) @@ -5501,9 +5471,7 @@ func TestAllocatorComputeActionRemoveDead(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - zone := zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - } + conf := roachpb.SpanConfig{NumReplicas: 3} threeReplDesc := roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5583,7 +5551,7 @@ func TestAllocatorComputeActionRemoveDead(t *testing.T) { for i, tcase := range testCases { mockStorePool(sp, tcase.live, nil, tcase.dead, nil, nil, nil) - action, _ := a.ComputeAction(ctx, &zone, &tcase.desc) + action, _ := a.ComputeAction(ctx, conf, &tcase.desc) if tcase.expectedAction != action { t.Errorf("Test case %d expected action %d, got action %d", i, tcase.expectedAction, action) } @@ -5594,9 +5562,7 @@ func TestAllocatorComputeActionSuspect(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - zone := zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - } + conf := roachpb.SpanConfig{NumReplicas: 3} threeReplDesc := roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5658,7 +5624,7 @@ func TestAllocatorComputeActionSuspect(t *testing.T) { for i, tcase := range testCases { mockStorePool(sp, tcase.live, nil, nil, nil, nil, tcase.suspect) - action, _ := a.ComputeAction(ctx, &zone, &tcase.desc) + action, _ := a.ComputeAction(ctx, conf, &tcase.desc) if tcase.expectedAction != action { t.Errorf("Test case %d expected action %d, got action %d", i, tcase.expectedAction, action) } @@ -5670,7 +5636,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { defer log.Scope(t).Close(t) testCases := []struct { - zone zonepb.ZoneConfig + conf roachpb.SpanConfig desc roachpb.RangeDescriptor expectedAction AllocatorAction live []roachpb.StoreID @@ -5682,9 +5648,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // replace it (nor add a new replica) since there isn't a live target, // but that's still the action being emitted. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - }, + conf: roachpb.SpanConfig{NumReplicas: 3}, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5712,9 +5676,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // Has three replicas, one is in decommissioning status, and one is on a // dead node. Replacing the dead replica is more important. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - }, + conf: roachpb.SpanConfig{NumReplicas: 3}, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5742,9 +5704,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // Needs three replicas, has four, where one is decommissioning and one is // dead. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - }, + conf: roachpb.SpanConfig{NumReplicas: 3}, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5777,9 +5737,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { // Needs three replicas, has four, where one is decommissioning and one is // decommissioned. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - }, + conf: roachpb.SpanConfig{NumReplicas: 3}, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5812,9 +5770,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { }, // Needs three replicas, has three, all decommissioning { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - }, + conf: roachpb.SpanConfig{NumReplicas: 3}, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5841,9 +5797,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { }, // Needs 3. Has 1 live, 3 decommissioning. { - zone: zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - }, + conf: roachpb.SpanConfig{NumReplicas: 3}, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ { @@ -5874,9 +5828,9 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { decommissioning: []roachpb.StoreID{1, 2, 3}, }, { - zone: zonepb.ZoneConfig{ - NumVoters: proto.Int32(1), - NumReplicas: proto.Int32(3), + conf: roachpb.SpanConfig{ + NumVoters: 1, + NumReplicas: 3, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5911,9 +5865,9 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { decommissioning: []roachpb.StoreID{7}, }, { - zone: zonepb.ZoneConfig{ - NumVoters: proto.Int32(1), - NumReplicas: proto.Int32(3), + conf: roachpb.SpanConfig{ + NumVoters: 1, + NumReplicas: 3, }, desc: roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5949,7 +5903,7 @@ func TestAllocatorComputeActionDecommission(t *testing.T) { for i, tcase := range testCases { mockStorePool(sp, tcase.live, nil, tcase.dead, tcase.decommissioning, tcase.decommissioned, nil) - action, _ := a.ComputeAction(ctx, &tcase.zone, &tcase.desc) + action, _ := a.ComputeAction(ctx, tcase.conf, &tcase.desc) if tcase.expectedAction != action { t.Errorf("Test case %d expected action %s, got action %s", i, tcase.expectedAction, action) continue @@ -5961,9 +5915,7 @@ func TestAllocatorRemoveLearner(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - zone := zonepb.ZoneConfig{ - NumReplicas: proto.Int32(3), - } + conf := roachpb.SpanConfig{NumReplicas: 3} learnerType := roachpb.LEARNER rangeWithLearnerDesc := roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ @@ -5988,7 +5940,7 @@ func TestAllocatorRemoveLearner(t *testing.T) { defer stopper.Stop(ctx) live, dead := []roachpb.StoreID{1, 2}, []roachpb.StoreID{3} mockStorePool(sp, live, nil, dead, nil, nil, nil) - action, _ := a.ComputeAction(ctx, &zone, &rangeWithLearnerDesc) + action, _ := a.ComputeAction(ctx, conf, &rangeWithLearnerDesc) require.Equal(t, AllocatorRemoveLearner, action) } @@ -5996,7 +5948,7 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - // In this test, the configured zone config has a replication factor of five + // In this test, the configured span config has a replication factor of five // set. We are checking that the effective replication factor is rounded down // to the number of stores which are not decommissioned or decommissioning. testCases := []struct { @@ -6068,7 +6020,7 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { }, { // Effective replication factor can't dip below three (unless the - // zone config explicitly asks for that, which it does not), so three + // span config explicitly asks for that, which it does not), so three // it is and we are under-replicaed. storeList: []roachpb.StoreID{1, 2}, expectedNumReplicas: 3, @@ -6100,7 +6052,7 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { }, { // The usual case in which there are enough nodes to accommodate the - // zone config. + // span config. storeList: []roachpb.StoreID{1, 2, 3, 4, 5}, expectedNumReplicas: 5, expectedAction: AllocatorConsiderRebalance, @@ -6111,7 +6063,7 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { }, { // No dead or decommissioning node and enough nodes around, so - // sticking with the zone config. + // sticking with the span config. storeList: []roachpb.StoreID{1, 2, 3, 4, 5}, expectedNumReplicas: 5, expectedAction: AllocatorConsiderRebalance, @@ -6176,9 +6128,7 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { ctx := context.Background() defer stopper.Stop(ctx) - zone := &zonepb.ZoneConfig{ - NumReplicas: proto.Int32(5), - } + conf := roachpb.SpanConfig{NumReplicas: 5} for _, prefixKey := range []roachpb.RKey{ roachpb.RKey(keys.NodeLivenessPrefix), @@ -6193,10 +6143,10 @@ func TestAllocatorComputeActionDynamicNumReplicas(t *testing.T) { desc.EndKey = prefixKey clusterNodes := a.storePool.ClusterNodeCount() - effectiveNumReplicas := GetNeededVoters(*zone.NumReplicas, clusterNodes) + effectiveNumReplicas := GetNeededVoters(conf.NumReplicas, clusterNodes) require.Equal(t, c.expectedNumReplicas, effectiveNumReplicas, "clusterNodes=%d", clusterNodes) - action, _ := a.ComputeAction(ctx, zone, &desc) + action, _ := a.ComputeAction(ctx, conf, &desc) require.Equal(t, c.expectedAction.String(), action.String()) }) } @@ -6208,11 +6158,11 @@ func TestAllocatorGetNeededReplicas(t *testing.T) { defer log.Scope(t).Close(t) testCases := []struct { - zoneRepls int32 - availNodes int - expected int + numReplicas int32 + availNodes int + expected int }{ - // If zone.NumReplicas <= 3, GetNeededVoters should always return zone.NumReplicas. + // If conf.NumReplicas <= 3, GetNeededVoters should always return conf.NumReplicas. {1, 0, 1}, {1, 1, 1}, {2, 0, 2}, @@ -6221,7 +6171,7 @@ func TestAllocatorGetNeededReplicas(t *testing.T) { {3, 0, 3}, {3, 1, 3}, {3, 3, 3}, - // Things get more involved when zone.NumReplicas > 3. + // Things get more involved when conf.NumReplicas > 3. {4, 1, 3}, {4, 2, 3}, {4, 3, 3}, @@ -6247,10 +6197,10 @@ func TestAllocatorGetNeededReplicas(t *testing.T) { } for _, tc := range testCases { - if e, a := tc.expected, GetNeededVoters(tc.zoneRepls, tc.availNodes); e != a { + if e, a := tc.expected, GetNeededVoters(tc.numReplicas, tc.availNodes); e != a { t.Errorf( - "GetNeededVoters(zone.NumReplicas=%d, availNodes=%d) got %d; want %d", - tc.zoneRepls, tc.availNodes, a, e) + "GetNeededVoters(conf.NumReplicas=%d, availNodes=%d) got %d; want %d", + tc.numReplicas, tc.availNodes, a, e) } } } @@ -6280,7 +6230,7 @@ func TestAllocatorComputeActionNoStorePool(t *testing.T) { defer log.Scope(t).Close(t) a := MakeAllocator(nil /* storePool */, nil /* rpcContext */) - action, priority := a.ComputeAction(context.Background(), &zonepb.ZoneConfig{NumReplicas: proto.Int32(0)}, nil) + action, priority := a.ComputeAction(context.Background(), roachpb.SpanConfig{}, nil) if action != AllocatorNoop { t.Errorf("expected AllocatorNoop, but got %v", action) } @@ -6295,14 +6245,14 @@ func TestAllocatorError(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - constraint := []zonepb.ConstraintsConjunction{ - {Constraints: []zonepb.Constraint{{Value: "one", Type: zonepb.Constraint_REQUIRED}}}, + constraint := []roachpb.ConstraintsConjunction{ + {Constraints: []roachpb.Constraint{{Value: "one", Type: roachpb.Constraint_REQUIRED}}}, } - constraints := []zonepb.ConstraintsConjunction{ + constraints := []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Value: "one", Type: zonepb.Constraint_REQUIRED}, - {Value: "two", Type: zonepb.Constraint_REQUIRED}, + Constraints: []roachpb.Constraint{ + {Value: "one", Type: roachpb.Constraint_REQUIRED}, + {Value: "two", Type: roachpb.Constraint_REQUIRED}, }, }, } @@ -6363,14 +6313,14 @@ func TestAllocatorThrottled(t *testing.T) { defer stopper.Stop(ctx) // First test to make sure we would send the replica to purgatory. - _, _, err := a.AllocateVoter(ctx, &simpleZoneConfig, []roachpb.ReplicaDescriptor{}, nil) + _, _, err := a.AllocateVoter(ctx, simpleSpanConfig, []roachpb.ReplicaDescriptor{}, nil) if !errors.HasInterface(err, (*purgatoryError)(nil)) { t.Fatalf("expected a purgatory error, got: %+v", err) } // Second, test the normal case in which we can allocate to the store. gossiputil.NewStoreGossiper(g).GossipStores(singleStore, t) - result, _, err := a.AllocateVoter(ctx, &simpleZoneConfig, []roachpb.ReplicaDescriptor{}, nil) + result, _, err := a.AllocateVoter(ctx, simpleSpanConfig, []roachpb.ReplicaDescriptor{}, nil) if err != nil { t.Fatalf("unable to perform allocation: %+v", err) } @@ -6387,7 +6337,7 @@ func TestAllocatorThrottled(t *testing.T) { } storeDetail.throttledUntil = timeutil.Now().Add(24 * time.Hour) a.storePool.detailsMu.Unlock() - _, _, err = a.AllocateVoter(ctx, &simpleZoneConfig, []roachpb.ReplicaDescriptor{}, nil) + _, _, err = a.AllocateVoter(ctx, simpleSpanConfig, []roachpb.ReplicaDescriptor{}, nil) if errors.HasInterface(err, (*purgatoryError)(nil)) { t.Fatalf("expected a non purgatory error, got: %+v", err) } @@ -6594,7 +6544,7 @@ func TestSimulateFilterUnremovableReplicas(t *testing.T) { } // TestAllocatorRebalanceAway verifies that when a replica is on a node with a -// bad zone config, the replica will be rebalanced off of it. +// bad span config, the replica will be rebalanced off of it. func TestAllocatorRebalanceAway(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -6651,43 +6601,31 @@ func TestAllocatorRebalanceAway(t *testing.T) { {StoreID: stores[2].StoreID, NodeID: stores[2].Node.NodeID}, } testCases := []struct { - constraint zonepb.Constraint + constraint roachpb.Constraint expected *roachpb.StoreID }{ { - constraint: zonepb.Constraint{Key: "datacenter", Value: "us", Type: zonepb.Constraint_REQUIRED}, + constraint: roachpb.Constraint{Key: "datacenter", Value: "us", Type: roachpb.Constraint_REQUIRED}, expected: &stores[3].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "eur", Type: zonepb.Constraint_PROHIBITED}, + constraint: roachpb.Constraint{Key: "datacenter", Value: "eur", Type: roachpb.Constraint_PROHIBITED}, expected: &stores[3].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "eur", Type: zonepb.Constraint_REQUIRED}, + constraint: roachpb.Constraint{Key: "datacenter", Value: "eur", Type: roachpb.Constraint_REQUIRED}, expected: &stores[4].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "us", Type: zonepb.Constraint_PROHIBITED}, + constraint: roachpb.Constraint{Key: "datacenter", Value: "us", Type: roachpb.Constraint_PROHIBITED}, expected: &stores[4].StoreID, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "other", Type: zonepb.Constraint_REQUIRED}, - expected: nil, - }, - { - constraint: zonepb.Constraint{Key: "datacenter", Value: "other", Type: zonepb.Constraint_PROHIBITED}, - expected: nil, - }, - { - constraint: zonepb.Constraint{Key: "datacenter", Value: "other", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, - expected: nil, - }, - { - constraint: zonepb.Constraint{Key: "datacenter", Value: "us", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, + constraint: roachpb.Constraint{Key: "datacenter", Value: "other", Type: roachpb.Constraint_REQUIRED}, expected: nil, }, { - constraint: zonepb.Constraint{Key: "datacenter", Value: "eur", Type: zonepb.Constraint_DEPRECATED_POSITIVE}, + constraint: roachpb.Constraint{Key: "datacenter", Value: "other", Type: roachpb.Constraint_PROHIBITED}, expected: nil, }, } @@ -6699,14 +6637,14 @@ func TestAllocatorRebalanceAway(t *testing.T) { for _, tc := range testCases { t.Run(tc.constraint.String(), func(t *testing.T) { - constraints := zonepb.ConstraintsConjunction{ - Constraints: []zonepb.Constraint{ + constraints := roachpb.ConstraintsConjunction{ + Constraints: []roachpb.Constraint{ tc.constraint, }, } var rangeUsageInfo RangeUsageInfo - actual, _, _, ok := a.RebalanceVoter(ctx, &zonepb.ZoneConfig{NumReplicas: proto.Int32(0), Constraints: []zonepb.ConstraintsConjunction{constraints}}, nil, existingReplicas, nil, rangeUsageInfo, storeFilterThrottled) + actual, _, _, ok := a.RebalanceVoter(ctx, roachpb.SpanConfig{Constraints: []roachpb.ConstraintsConjunction{constraints}}, nil, existingReplicas, nil, rangeUsageInfo, storeFilterThrottled) if tc.expected == nil && ok { t.Errorf("rebalancing to the incorrect store, expected nil, got %d", actual.StoreID) @@ -6866,7 +6804,7 @@ func TestAllocatorFullDisks(t *testing.T) { // Rebalance until there's no more rebalancing to do. if ts.Capacity.RangeCount > 0 { var rangeUsageInfo RangeUsageInfo - target, _, details, ok := alloc.RebalanceVoter(ctx, zonepb.EmptyCompleteZoneConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := alloc.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) if ok { if log.V(1) { log.Infof(ctx, "rebalancing to %v; details: %s", target, details) @@ -6989,7 +6927,7 @@ func Example_rebalancing() { for j := 0; j < len(testStores); j++ { ts := &testStores[j] var rangeUsageInfo RangeUsageInfo - target, _, details, ok := alloc.RebalanceVoter(ctx, zonepb.EmptyCompleteZoneConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) + target, _, details, ok := alloc.RebalanceVoter(ctx, emptySpanConfig(), nil, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, nil, rangeUsageInfo, storeFilterThrottled) if ok { log.Infof(ctx, "rebalancing to %v; details: %s", target, details) testStores[j].rebalance(&testStores[int(target.StoreID)], alloc.randGen.Int63n(1<<20)) diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 8bca610e6b80..be652a63ac25 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -4159,13 +4159,12 @@ func TestMergeQueue(t *testing.T) { ctx := context.Background() manualClock := hlc.NewHybridManualClock() - zoneConfig := zonepb.DefaultZoneConfig() - rangeMinBytes := int64(1 << 10) // 1KB - zoneConfig.RangeMinBytes = &rangeMinBytes settings := cluster.MakeTestingClusterSettings() sv := &settings.SV kvserver.MergeQueueInterval.Override(ctx, sv, 0) // process greedily + zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig.RangeMinBytes = proto.Int64(1 << 10) // 1KB tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, @@ -4183,6 +4182,8 @@ func TestMergeQueue(t *testing.T) { }, }) defer tc.Stopper().Stop(ctx) + + conf := zoneConfig.AsSpanConfig() store := tc.GetFirstStoreFromServer(t, 0) // The cluster with manual replication disables the merge queue, // so we need to re-enable. @@ -4207,7 +4208,7 @@ func TestMergeQueue(t *testing.T) { } } rng, _ := randutil.NewPseudoRand() - randBytes := randutil.RandBytes(rng, int(*zoneConfig.RangeMinBytes)) + randBytes := randutil.RandBytes(rng, int(conf.RangeMinBytes)) lhsStartKey := roachpb.RKey(tc.ScratchRange(t)) rhsStartKey := lhsStartKey.Next().Next() @@ -4221,17 +4222,17 @@ func TestMergeQueue(t *testing.T) { // setThresholds simulates a zone config update that updates the ranges' // minimum and maximum sizes. - setZones := func(t *testing.T, zone zonepb.ZoneConfig) { + setSpanConfigs := func(t *testing.T, conf roachpb.SpanConfig) { t.Helper() if l := lhs(); l == nil { t.Fatal("left-hand side range not found") } else { - l.SetZoneConfig(&zone) + l.SetSpanConfig(conf) } if r := rhs(); r == nil { t.Fatal("right-hand side range not found") } else { - r.SetZoneConfig(&zone) + r.SetSpanConfig(conf) } } @@ -4243,7 +4244,7 @@ func TestMergeQueue(t *testing.T) { t.Fatal(err) } } - setZones(t, zoneConfig) + setSpanConfigs(t, conf) // Disable load-based splitting, so that the absence of sufficient QPS // measurements do not prevent ranges from merging. Certain subtests // re-enable the functionality. @@ -4271,9 +4272,9 @@ func TestMergeQueue(t *testing.T) { t.Run("lhs-undersize", func(t *testing.T) { reset(t) - zone := protoutil.Clone(&zoneConfig).(*zonepb.ZoneConfig) - *zone.RangeMinBytes *= 2 - lhs().SetZoneConfig(zone) + conf := conf + conf.RangeMinBytes *= 2 + lhs().SetSpanConfig(conf) store.MustForceMergeScanAndProcess() verifyMerged(t, store, lhsStartKey, rhsStartKey) }) @@ -4283,16 +4284,16 @@ func TestMergeQueue(t *testing.T) { // The ranges are individually beneath the minimum size threshold, but // together they'll exceed the maximum size threshold. - zone := protoutil.Clone(&zoneConfig).(*zonepb.ZoneConfig) - zone.RangeMinBytes = proto.Int64(rhs().GetMVCCStats().Total() + 1) - zone.RangeMaxBytes = proto.Int64(lhs().GetMVCCStats().Total() + rhs().GetMVCCStats().Total() - 1) - setZones(t, *zone) + conf := conf + conf.RangeMinBytes = rhs().GetMVCCStats().Total() + 1 + conf.RangeMaxBytes = lhs().GetMVCCStats().Total() + rhs().GetMVCCStats().Total() - 1 + setSpanConfigs(t, conf) store.MustForceMergeScanAndProcess() verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Once the maximum size threshold is increased, the merge can occur. - zone.RangeMaxBytes = proto.Int64(*zone.RangeMaxBytes + 1) - setZones(t, *zone) + conf.RangeMaxBytes += 1 + setSpanConfigs(t, conf) l := lhs().RangeID r := rhs().RangeID log.Infof(ctx, "Left=%s, Right=%s", l, r) diff --git a/pkg/kv/kvserver/client_replica_backpressure_test.go b/pkg/kv/kvserver/client_replica_backpressure_test.go index 60561a3fb649..c6ac59e3c838 100644 --- a/pkg/kv/kvserver/client_replica_backpressure_test.go +++ b/pkg/kv/kvserver/client_replica_backpressure_test.go @@ -127,14 +127,14 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { return tc, args, tdb, tablePrefix, unblockSplit, waitForBlockedRange } - waitForZoneConfig := func(t *testing.T, tc *testcluster.TestCluster, tablePrefix roachpb.Key, exp int64) { + waitForSpanConfig := func(t *testing.T, tc *testcluster.TestCluster, tablePrefix roachpb.Key, exp int64) { testutils.SucceedsSoon(t, func() error { for i := 0; i < tc.NumServers(); i++ { s := tc.Server(i) _, r := getFirstStoreReplica(t, s, tablePrefix) - _, zone := r.DescAndZone() - if *zone.RangeMaxBytes != exp { - return fmt.Errorf("expected %d, got %d", exp, *zone.RangeMaxBytes) + conf := r.SpanConfig() + if conf.RangeMaxBytes != exp { + return fmt.Errorf("expected %d, got %d", exp, conf.RangeMaxBytes) } } return nil @@ -177,7 +177,7 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { tdb.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING "+ "range_max_bytes = $1, range_min_bytes = $2", dataSize/5, dataSize/10) - waitForZoneConfig(t, tc, tablePrefix, dataSize/5) + waitForSpanConfig(t, tc, tablePrefix, dataSize/5) // Don't observe backpressure. tdb.Exec(t, "UPSERT INTO foo VALUES ($1, $2)", @@ -197,7 +197,7 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { tdb.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING "+ "range_max_bytes = $1, range_min_bytes = $2", dataSize/5, dataSize/10) - waitForZoneConfig(t, tc, tablePrefix, dataSize/5) + waitForSpanConfig(t, tc, tablePrefix, dataSize/5) // Then we'll add a new server and move the table there. moveTableToNewStore(t, tc, args, tablePrefix) @@ -227,7 +227,7 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { newMin := newMax / 4 tdb.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING "+ "range_max_bytes = $1, range_min_bytes = $2", newMax, newMin) - waitForZoneConfig(t, tc, tablePrefix, newMax) + waitForSpanConfig(t, tc, tablePrefix, newMax) // Don't observe backpressure because we remember the previous max size on // this node. @@ -263,7 +263,7 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { newMin := newMax / 4 tdb.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING "+ "range_max_bytes = $1, range_min_bytes = $2", newMax, newMin) - waitForZoneConfig(t, tc, tablePrefix, newMax) + waitForSpanConfig(t, tc, tablePrefix, newMax) // Then we'll add a new server and move the table there. moveTableToNewStore(t, tc, args, tablePrefix) diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index bd05507cebc1..a77220255383 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -3223,12 +3223,12 @@ func TestStrictGCEnforcement(t *testing.T) { for i := 0; i < tc.NumServers(); i++ { s := tc.Server(i) _, r := getFirstStoreReplica(t, s, tableKey) - if _, z := r.DescAndZone(); z.GC.TTLSeconds != int32(exp) { + if c := r.SpanConfig(); c.TTL().Seconds() != (time.Duration(exp) * time.Second).Seconds() { _, sysCfg := getFirstStoreReplica(t, tc.Server(i), keys.SystemConfigSpan.Key) sysCfg.RaftLock() require.NoError(t, sysCfg.MaybeGossipSystemConfigRaftMuLocked(ctx)) sysCfg.RaftUnlock() - return errors.Errorf("expected %d, got %d", exp, z.GC.TTLSeconds) + return errors.Errorf("expected %d, got %d", exp, c.TTL().Seconds()) } } return nil diff --git a/pkg/kv/kvserver/consistency_queue.go b/pkg/kv/kvserver/consistency_queue.go index 86b33e730e34..0e3af4753660 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -14,10 +14,9 @@ import ( "context" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -72,7 +71,7 @@ type consistencyShouldQueueData struct { } // newConsistencyQueue returns a new instance of consistencyQueue. -func newConsistencyQueue(store *Store, gossip *gossip.Gossip) *consistencyQueue { +func newConsistencyQueue(store *Store) *consistencyQueue { q := &consistencyQueue{ interval: func() time.Duration { return consistencyCheckInterval.Get(&store.ClusterSettings().SV) @@ -80,7 +79,7 @@ func newConsistencyQueue(store *Store, gossip *gossip.Gossip) *consistencyQueue replicaCountFn: store.ReplicaCount, } q.baseQueue = newBaseQueue( - "consistencyChecker", q, store, gossip, + "consistencyChecker", q, store, queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, @@ -97,7 +96,7 @@ func newConsistencyQueue(store *Store, gossip *gossip.Gossip) *consistencyQueue } func (q *consistencyQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ *config.SystemConfig, + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, ) (bool, float64) { return consistencyQueueShouldQueueImpl(ctx, now, consistencyShouldQueueData{ @@ -147,7 +146,7 @@ func consistencyQueueShouldQueueImpl( // process() is called on every range for which this node is a lease holder. func (q *consistencyQueue) process( - ctx context.Context, repl *Replica, _ *config.SystemConfig, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, ) (bool, error) { if q.interval() <= 0 { return false, nil diff --git a/pkg/kv/kvserver/constraint/BUILD.bazel b/pkg/kv/kvserver/constraint/BUILD.bazel index d2641fff4999..23086c799370 100644 --- a/pkg/kv/kvserver/constraint/BUILD.bazel +++ b/pkg/kv/kvserver/constraint/BUILD.bazel @@ -5,8 +5,5 @@ go_library( srcs = ["analyzer.go"], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint", visibility = ["//visibility:public"], - deps = [ - "//pkg/config/zonepb", - "//pkg/roachpb:with-mocks", - ], + deps = ["//pkg/roachpb:with-mocks"], ) diff --git a/pkg/kv/kvserver/constraint/analyzer.go b/pkg/kv/kvserver/constraint/analyzer.go index b037be2c678b..90f0b5970fc6 100644 --- a/pkg/kv/kvserver/constraint/analyzer.go +++ b/pkg/kv/kvserver/constraint/analyzer.go @@ -13,7 +13,6 @@ package constraint import ( "context" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -21,7 +20,7 @@ import ( // combines a zone's constraints with information about which stores satisfy // what term of the constraints disjunction. type AnalyzedConstraints struct { - Constraints []zonepb.ConstraintsConjunction + Constraints []roachpb.ConstraintsConjunction // True if the per-replica constraints don't fully cover all the desired // replicas in the range (sum(constraints.NumReplicas) < zone.NumReplicas). // In such cases, we allow replicas that don't match any of the per-replica @@ -49,7 +48,7 @@ func AnalyzeConstraints( getStoreDescFn func(roachpb.StoreID) (roachpb.StoreDescriptor, bool), existing []roachpb.ReplicaDescriptor, numReplicas int32, - constraints []zonepb.ConstraintsConjunction, + constraints []roachpb.ConstraintsConjunction, ) AnalyzedConstraints { result := AnalyzedConstraints{ Constraints: constraints, @@ -85,12 +84,12 @@ func AnalyzeConstraints( // the possibly numerous sets that apply to a range), returning true iff the // store matches the constraints. The contraints are AND'ed together; a store // matches the conjunction if it matches all of them. -func ConjunctionsCheck(store roachpb.StoreDescriptor, constraints []zonepb.Constraint) bool { +func ConjunctionsCheck(store roachpb.StoreDescriptor, constraints []roachpb.Constraint) bool { for _, constraint := range constraints { // StoreMatchesConstraint returns whether a store matches the given constraint. - hasConstraint := zonepb.StoreMatchesConstraint(store, constraint) - if (constraint.Type == zonepb.Constraint_REQUIRED && !hasConstraint) || - (constraint.Type == zonepb.Constraint_PROHIBITED && hasConstraint) { + hasConstraint := roachpb.StoreMatchesConstraint(store, constraint) + if (constraint.Type == roachpb.Constraint_REQUIRED && !hasConstraint) || + (constraint.Type == roachpb.Constraint_PROHIBITED && hasConstraint) { return false } } diff --git a/pkg/kv/kvserver/gc/BUILD.bazel b/pkg/kv/kvserver/gc/BUILD.bazel index 94f9348274cd..755e8f956ad0 100644 --- a/pkg/kv/kvserver/gc/BUILD.bazel +++ b/pkg/kv/kvserver/gc/BUILD.bazel @@ -10,7 +10,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/config/zonepb", "//pkg/keys", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/kvserverbase", @@ -41,7 +40,6 @@ go_test( ], embed = [":gc"], deps = [ - "//pkg/config/zonepb", "//pkg/keys", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/rditer", diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index d454aadba6d1..b8e137541510 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -22,7 +22,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -99,15 +98,15 @@ var MaxIntentKeyBytesPerCleanupBatch = settings.RegisterIntSetting( // CalculateThreshold calculates the GC threshold given the policy and the // current view of time. -func CalculateThreshold(now hlc.Timestamp, policy zonepb.GCPolicy) (threshold hlc.Timestamp) { - ttlNanos := int64(policy.TTLSeconds) * time.Second.Nanoseconds() +func CalculateThreshold(now hlc.Timestamp, gcttl time.Duration) (threshold hlc.Timestamp) { + ttlNanos := gcttl.Nanoseconds() return now.Add(-ttlNanos, 0) } // TimestampForThreshold inverts CalculateThreshold. It returns the timestamp // which should be used for now to arrive at the passed threshold. -func TimestampForThreshold(threshold hlc.Timestamp, policy zonepb.GCPolicy) (ts hlc.Timestamp) { - ttlNanos := int64(policy.TTLSeconds) * time.Second.Nanoseconds() +func TimestampForThreshold(threshold hlc.Timestamp, gcttl time.Duration) (ts hlc.Timestamp) { + ttlNanos := gcttl.Nanoseconds() return threshold.Add(ttlNanos, 0) } @@ -148,8 +147,8 @@ type Threshold struct { type Info struct { // Now is the timestamp used for age computations. Now hlc.Timestamp - // Policy is the policy used for this garbage collection cycle. - Policy zonepb.GCPolicy + // GCTTL is the TTL this garbage collection cycle. + GCTTL time.Duration // Stats about the userspace key-values considered, namely the number of // keys with GC'able data, the number of "old" intents and the number of // associated distinct transactions. @@ -177,7 +176,7 @@ type Info struct { // ResolveTotal is the total number of attempted intent resolutions in // this cycle. ResolveTotal int - // Threshold is the computed expiration timestamp. Equal to `Now - Policy`. + // Threshold is the computed expiration timestamp. Equal to `Now - GCTTL`. Threshold hlc.Timestamp // AffectedVersionsKeyBytes is the number of (fully encoded) bytes deleted from keys in the storage engine. // Note that this does not account for compression that the storage engine uses to store data on disk. Real @@ -231,7 +230,7 @@ func Run( snap storage.Reader, now, newThreshold hlc.Timestamp, options RunOptions, - policy zonepb.GCPolicy, + gcTTL time.Duration, gcer GCer, cleanupIntentsFn CleanupIntentsFunc, cleanupTxnIntentsAsyncFn CleanupTxnIntentsAsyncFunc, @@ -246,7 +245,7 @@ func Run( } info := Info{ - Policy: policy, + GCTTL: gcTTL, Now: now, Threshold: newThreshold, } diff --git a/pkg/kv/kvserver/gc/gc_old_test.go b/pkg/kv/kvserver/gc/gc_old_test.go index da88ec8642bb..b60e5f530485 100644 --- a/pkg/kv/kvserver/gc/gc_old_test.go +++ b/pkg/kv/kvserver/gc/gc_old_test.go @@ -14,8 +14,8 @@ import ( "context" "sort" "testing" + "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -45,7 +45,7 @@ func runGCOld( now hlc.Timestamp, _ hlc.Timestamp, // exists to make signature match RunGC options RunOptions, - policy zonepb.GCPolicy, + gcTTL time.Duration, gcer GCer, cleanupIntentsFn CleanupIntentsFunc, cleanupTxnIntentsAsyncFn CleanupTxnIntentsAsyncFunc, @@ -58,7 +58,7 @@ func runGCOld( intentExp := now.Add(-options.IntentAgeThreshold.Nanoseconds(), 0) txnExp := now.Add(-kvserverbase.TxnCleanupThreshold.Nanoseconds(), 0) - gc := MakeGarbageCollector(now, policy) + gc := MakeGarbageCollector(now, gcTTL) if err := gcer.SetGCThreshold(ctx, Threshold{ Key: gc.Threshold, @@ -75,7 +75,7 @@ func runGCOld( var keyBytes int64 var valBytes int64 info := Info{ - Policy: policy, + GCTTL: gcTTL, Now: now, Threshold: gc.Threshold, } @@ -245,15 +245,15 @@ func runGCOld( // versions and maximum age. type GarbageCollector struct { Threshold hlc.Timestamp - policy zonepb.GCPolicy + ttl time.Duration } // MakeGarbageCollector allocates and returns a new GC, with expiration -// computed based on current time and policy.TTLSeconds. -func MakeGarbageCollector(now hlc.Timestamp, policy zonepb.GCPolicy) GarbageCollector { +// computed based on current time and the gc TTL. +func MakeGarbageCollector(now hlc.Timestamp, gcTTL time.Duration) GarbageCollector { return GarbageCollector{ - Threshold: CalculateThreshold(now, policy), - policy: policy, + Threshold: CalculateThreshold(now, gcTTL), + ttl: gcTTL, } } @@ -275,7 +275,7 @@ func MakeGarbageCollector(now hlc.Timestamp, policy zonepb.GCPolicy) GarbageColl // would still allow for the tombstone bugs in #6227, so in the future we will // add checks that disallow writes before the last GC expiration time. func (gc GarbageCollector) Filter(keys []storage.MVCCKey, values [][]byte) (int, hlc.Timestamp) { - if gc.policy.TTLSeconds <= 0 { + if gc.ttl.Seconds() <= 0 { return -1, hlc.Timestamp{} } if len(keys) == 0 { @@ -328,8 +328,8 @@ var ( // different sorts of MVCC keys. func TestGarbageCollectorFilter(t *testing.T) { defer leaktest.AfterTest(t)() - gcA := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, zonepb.GCPolicy{TTLSeconds: 1}) - gcB := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, zonepb.GCPolicy{TTLSeconds: 2}) + gcA := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, time.Second) + gcB := MakeGarbageCollector(hlc.Timestamp{WallTime: 0, Logical: 0}, 2*time.Second) n := []byte("data") d := []byte(nil) testData := []struct { @@ -361,7 +361,7 @@ func TestGarbageCollectorFilter(t *testing.T) { } for i, test := range testData { test.gc.Threshold = test.time - test.gc.Threshold.WallTime -= int64(test.gc.policy.TTLSeconds) * 1e9 + test.gc.Threshold.WallTime -= test.gc.ttl.Nanoseconds() idx, delTS := test.gc.Filter(test.keys, test.values) if idx != test.expIdx { t.Errorf("%d: expected index %d; got %d", i, test.expIdx, idx) diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index f64017e06658..0bb630d395b8 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -18,7 +18,6 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -98,10 +97,10 @@ func TestRunNewVsOld(t *testing.T) { snap := eng.NewSnapshot() oldGCer := makeFakeGCer() - policy := zonepb.GCPolicy{TTLSeconds: tc.ttl} - newThreshold := CalculateThreshold(tc.now, policy) + ttl := time.Duration(tc.ttl) * time.Second + newThreshold := CalculateThreshold(tc.now, ttl) gcInfoOld, err := runGCOld(ctx, tc.ds.desc(), snap, tc.now, - newThreshold, RunOptions{IntentAgeThreshold: intentAgeThreshold}, policy, + newThreshold, RunOptions{IntentAgeThreshold: intentAgeThreshold}, ttl, &oldGCer, oldGCer.resolveIntents, oldGCer.resolveIntentsAsync) @@ -109,7 +108,7 @@ func TestRunNewVsOld(t *testing.T) { newGCer := makeFakeGCer() gcInfoNew, err := Run(ctx, tc.ds.desc(), snap, tc.now, - newThreshold, RunOptions{IntentAgeThreshold: intentAgeThreshold}, policy, + newThreshold, RunOptions{IntentAgeThreshold: intentAgeThreshold}, ttl, &newGCer, newGCer.resolveIntents, newGCer.resolveIntentsAsync) @@ -134,10 +133,10 @@ func BenchmarkRun(b *testing.B) { runGCFunc = runGCOld } snap := eng.NewSnapshot() - policy := zonepb.GCPolicy{TTLSeconds: spec.ttl} + ttl := time.Duration(spec.ttl) * time.Second return runGCFunc(ctx, spec.ds.desc(), snap, spec.now, - CalculateThreshold(spec.now, policy), RunOptions{IntentAgeThreshold: intentAgeThreshold}, - policy, + CalculateThreshold(spec.now, ttl), RunOptions{IntentAgeThreshold: intentAgeThreshold}, + ttl, NoopGCer{}, func(ctx context.Context, intents []roachpb.Intent) error { return nil diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 27f362d22ee9..3b21baa2deb3 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -13,13 +13,11 @@ package gc import ( "bytes" "context" - "errors" "fmt" "math/rand" "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -27,22 +25,22 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) func TestCalculateThreshold(t *testing.T) { for _, c := range []struct { - ttlSeconds int32 - ts hlc.Timestamp + gcTTL time.Duration + ts hlc.Timestamp }{ { - ts: hlc.Timestamp{WallTime: time.Hour.Nanoseconds(), Logical: 0}, - ttlSeconds: 1, + ts: hlc.Timestamp{WallTime: time.Hour.Nanoseconds(), Logical: 0}, + gcTTL: time.Second, }, } { - policy := zonepb.GCPolicy{TTLSeconds: c.ttlSeconds} - require.Equal(t, c.ts, TimestampForThreshold(CalculateThreshold(c.ts, policy), policy)) + require.Equal(t, c.ts, TimestampForThreshold(CalculateThreshold(c.ts, c.gcTTL), c.gcTTL)) } } @@ -130,7 +128,7 @@ func TestIntentAgeThresholdSetting(t *testing.T) { StartKey: roachpb.RKey(key), EndKey: roachpb.RKey("b"), } - policy := zonepb.GCPolicy{TTLSeconds: 1} + gcTTL := time.Second snap := eng.NewSnapshot() nowTs := hlc.Timestamp{ WallTime: now.Nanoseconds(), @@ -138,13 +136,13 @@ func TestIntentAgeThresholdSetting(t *testing.T) { fakeGCer := makeFakeGCer() // Test GC desired behavior. - info, err := Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{IntentAgeThreshold: intentAgeThreshold}, policy, &fakeGCer, fakeGCer.resolveIntents, + info, err := Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{IntentAgeThreshold: intentLongThreshold}, gcTTL, &fakeGCer, fakeGCer.resolveIntents, fakeGCer.resolveIntentsAsync) require.NoError(t, err, "GC Run shouldn't fail") assert.Zero(t, info.IntentsConsidered, "Expected no intents considered by GC with default threshold") - info, err = Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{IntentAgeThreshold: intentShortThreshold}, policy, &fakeGCer, fakeGCer.resolveIntents, + info, err = Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{IntentAgeThreshold: intentShortThreshold}, gcTTL, &fakeGCer, fakeGCer.resolveIntents, fakeGCer.resolveIntentsAsync) require.NoError(t, err, "GC Run shouldn't fail") assert.Equal(t, 1, info.IntentsConsidered, @@ -182,7 +180,7 @@ func TestIntentCleanupBatching(t *testing.T) { StartKey: roachpb.RKey([]byte{txnPrefixes[0], objectKeys[0]}), EndKey: roachpb.RKey("z"), } - policy := zonepb.GCPolicy{TTLSeconds: 1} + gcTTL := time.Second snap := eng.NewSnapshot() nowTs := hlc.Timestamp{ WallTime: now.Nanoseconds(), @@ -191,7 +189,7 @@ func TestIntentCleanupBatching(t *testing.T) { // Base GCer will cleanup all intents in one go and its result is used as a baseline // to compare batched runs for checking completeness. baseGCer := makeFakeGCer() - _, err := Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{IntentAgeThreshold: intentAgeThreshold}, policy, &baseGCer, baseGCer.resolveIntents, + _, err := Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{IntentAgeThreshold: intentAgeThreshold}, gcTTL, &baseGCer, baseGCer.resolveIntents, baseGCer.resolveIntentsAsync) if err != nil { t.Fatal("Can't prepare test fixture. Non batched GC run fails.") @@ -201,7 +199,7 @@ func TestIntentCleanupBatching(t *testing.T) { var batchSize int64 = 7 fakeGCer := makeFakeGCer() info, err := Run(ctx, &desc, snap, nowTs, nowTs, - RunOptions{IntentAgeThreshold: intentAgeThreshold, MaxIntentsPerIntentCleanupBatch: batchSize}, policy, + RunOptions{IntentAgeThreshold: intentAgeThreshold, MaxIntentsPerIntentCleanupBatch: batchSize}, gcTTL, &fakeGCer, fakeGCer.resolveIntents, fakeGCer.resolveIntentsAsync) require.NoError(t, err, "GC Run shouldn't fail") maxIntents := 0 diff --git a/pkg/kv/kvserver/gc_queue.go b/pkg/kv/kvserver/gc_queue.go index 9e91816bab23..63281b7f4ef4 100644 --- a/pkg/kv/kvserver/gc_queue.go +++ b/pkg/kv/kvserver/gc_queue.go @@ -18,14 +18,12 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -108,10 +106,10 @@ type gcQueue struct { } // newGCQueue returns a new instance of gcQueue. -func newGCQueue(store *Store, gossip *gossip.Gossip) *gcQueue { +func newGCQueue(store *Store) *gcQueue { gcq := &gcQueue{} gcq.baseQueue = newBaseQueue( - "gc", gcq, store, gossip, + "gc", gcq, store, queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, @@ -170,13 +168,12 @@ func (r gcQueueScore) String() string { // in the event that the cumulative ages of GC'able bytes or extant // intents exceed thresholds. func (gcq *gcQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ *config.SystemConfig, + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, ) (bool, float64) { - // Consult the protected timestamp state to determine whether we can GC and // the timestamp which can be used to calculate the score. - _, zone := repl.DescAndZone() - canGC, _, gcTimestamp, oldThreshold, newThreshold := repl.checkProtectedTimestampsForGC(ctx, *zone.GC) + _, conf := repl.DescAndSpanConfig() + canGC, _, gcTimestamp, oldThreshold, newThreshold := repl.checkProtectedTimestampsForGC(ctx, conf.TTL()) if !canGC { return false, 0 } @@ -186,7 +183,7 @@ func (gcq *gcQueue) shouldQueue( log.VErrEventf(ctx, 2, "failed to fetch last processed time: %v", err) return false, 0 } - r := makeGCQueueScore(ctx, repl, gcTimestamp, lastGC, *zone.GC, canAdvanceGCThreshold) + r := makeGCQueueScore(ctx, repl, gcTimestamp, lastGC, conf.TTL(), canAdvanceGCThreshold) return r.ShouldQueue, r.FinalScore } @@ -195,7 +192,7 @@ func makeGCQueueScore( repl *Replica, now hlc.Timestamp, lastGC hlc.Timestamp, - policy zonepb.GCPolicy, + gcTTL time.Duration, canAdvanceGCThreshold bool, ) gcQueueScore { repl.mu.Lock() @@ -210,7 +207,8 @@ func makeGCQueueScore( // have slightly different priorities and even symmetrical workloads don't // trigger GC at the same time. r := makeGCQueueScoreImpl( - ctx, int64(repl.RangeID), now, ms, policy, lastGC, canAdvanceGCThreshold) + ctx, int64(repl.RangeID), now, ms, gcTTL, lastGC, canAdvanceGCThreshold, + ) return r } @@ -308,7 +306,7 @@ func makeGCQueueScoreImpl( fuzzSeed int64, now hlc.Timestamp, ms enginepb.MVCCStats, - policy zonepb.GCPolicy, + gcTTL time.Duration, lastGC hlc.Timestamp, canAdvanceGCThreshold bool, ) gcQueueScore { @@ -318,7 +316,8 @@ func makeGCQueueScoreImpl( if !lastGC.IsEmpty() { r.LastGC = time.Duration(now.WallTime - lastGC.WallTime) } - r.TTL = policy.TTL() + + r.TTL = gcTTL // Treat a zero TTL as a one-second TTL, which avoids a priority of infinity // and otherwise behaves indistinguishable given that we can't possibly hope @@ -479,15 +478,15 @@ func (r *replicaGCer) GC(ctx context.Context, keys []roachpb.GCRequest_GCKey) er // 7) push these transactions (again, recreating txn entries). // 8) send a GCRequest. func (gcq *gcQueue) process( - ctx context.Context, repl *Replica, sysCfg *config.SystemConfig, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { // Lookup the descriptor and GC policy for the zone containing this key range. - desc, zone := repl.DescAndZone() + desc, conf := repl.DescAndSpanConfig() + // Consult the protected timestamp state to determine whether we can GC and // the timestamp which can be used to calculate the score and updated GC // threshold. - canGC, cacheTimestamp, gcTimestamp, oldThreshold, newThreshold := - repl.checkProtectedTimestampsForGC(ctx, *zone.GC) + canGC, cacheTimestamp, gcTimestamp, oldThreshold, newThreshold := repl.checkProtectedTimestampsForGC(ctx, conf.TTL()) if !canGC { return false, nil } @@ -499,7 +498,7 @@ func (gcq *gcQueue) process( lastGC = hlc.Timestamp{} log.VErrEventf(ctx, 2, "failed to fetch last processed time: %v", err) } - r := makeGCQueueScore(ctx, repl, gcTimestamp, lastGC, *zone.GC, canAdvanceGCThreshold) + r := makeGCQueueScore(ctx, repl, gcTimestamp, lastGC, conf.TTL(), canAdvanceGCThreshold) log.VEventf(ctx, 2, "processing replica %s with score %s", repl.String(), r) // Synchronize the new GC threshold decision with concurrent // AdminVerifyProtectedTimestamp requests. @@ -527,7 +526,7 @@ func (gcq *gcQueue) process( MaxTxnsPerIntentCleanupBatch: intentresolver.MaxTxnsPerIntentCleanupBatch, IntentCleanupBatchTimeout: gcQueueIntentBatchTimeout, }, - *zone.GC, + conf.TTL(), &replicaGCer{repl: repl}, func(ctx context.Context, intents []roachpb.Intent) error { intentCount, err := repl.store.intentResolver. @@ -564,7 +563,7 @@ func (gcq *gcQueue) process( log.Eventf(ctx, "MVCC stats after GC: %+v", repl.GetMVCCStats()) log.Eventf(ctx, "GC score after GC: %s", makeGCQueueScore( - ctx, repl, repl.store.Clock().Now(), lastGC, *zone.GC, canAdvanceGCThreshold)) + ctx, repl, repl.store.Clock().Now(), lastGC, conf.TTL(), canAdvanceGCThreshold)) updateStoreMetricsWithGCInfo(gcq.store.metrics, info) return true, nil } diff --git a/pkg/kv/kvserver/gc_queue_test.go b/pkg/kv/kvserver/gc_queue_test.go index e7051abb6964..0693205663eb 100644 --- a/pkg/kv/kvserver/gc_queue_test.go +++ b/pkg/kv/kvserver/gc_queue_test.go @@ -20,7 +20,6 @@ import ( "testing/quick" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -110,7 +109,7 @@ func TestGCQueueMakeGCScoreInvariantQuick(t *testing.T) { } now := initialNow.Add(timePassed.Nanoseconds(), 0) r := makeGCQueueScoreImpl( - ctx, int64(seed), now, ms, zonepb.GCPolicy{TTLSeconds: ttlSec}, hlc.Timestamp{}, + ctx, int64(seed), now, ms, time.Duration(ttlSec)*time.Second, hlc.Timestamp{}, true /* canAdvanceGCThreshold */) wouldHaveToDeleteSomething := gcBytes*int64(ttlSec) < ms.GCByteAge(now.WallTime) result := !r.ShouldQueue || wouldHaveToDeleteSomething @@ -133,7 +132,7 @@ func TestGCQueueMakeGCScoreAnomalousStats(t *testing.T) { LiveBytes: int64(liveBytes), ValBytes: int64(valBytes), KeyBytes: int64(keyBytes), - }, zonepb.GCPolicy{TTLSeconds: 60}, hlc.Timestamp{}, true /* canAdvanceGCThreshold */) + }, 60*time.Second, hlc.Timestamp{}, true /* canAdvanceGCThreshold */) return r.DeadFraction >= 0 && r.DeadFraction <= 1 }, &quick.Config{MaxCount: 1000}); err != nil { t.Fatal(err) @@ -156,7 +155,7 @@ func TestGCQueueMakeGCScoreLargeAbortSpan(t *testing.T) { r := makeGCQueueScoreImpl( context.Background(), seed, hlc.Timestamp{WallTime: expiration + 1}, - ms, zonepb.GCPolicy{TTLSeconds: 10000}, + ms, 10000*time.Second, hlc.Timestamp{}, true, /* canAdvanceGCThreshold */ ) require.True(t, r.ShouldQueue) @@ -171,7 +170,7 @@ func TestGCQueueMakeGCScoreLargeAbortSpan(t *testing.T) { r := makeGCQueueScoreImpl( context.Background(), seed, hlc.Timestamp{WallTime: expiration + 1}, - ms, zonepb.GCPolicy{TTLSeconds: 10000}, + ms, 10000*time.Second, hlc.Timestamp{}, true, /* canAdvanceGCThreshold */ ) require.True(t, r.ShouldQueue) @@ -182,7 +181,7 @@ func TestGCQueueMakeGCScoreLargeAbortSpan(t *testing.T) { { r := makeGCQueueScoreImpl(context.Background(), seed, hlc.Timestamp{WallTime: expiration}, - ms, zonepb.GCPolicy{TTLSeconds: 10000}, + ms, 10000*time.Second, hlc.Timestamp{WallTime: expiration - 100}, true, /* canAdvanceGCThreshold */ ) require.False(t, r.ShouldQueue) @@ -197,7 +196,7 @@ func TestGCQueueMakeGCScoreIntentCooldown(t *testing.T) { const seed = 1 ctx := context.Background() now := hlc.Timestamp{WallTime: 1e6 * 1e9} - policy := zonepb.GCPolicy{TTLSeconds: 1} + gcTTL := time.Second testcases := map[string]struct { lastGC hlc.Timestamp @@ -223,7 +222,7 @@ func TestGCQueueMakeGCScoreIntentCooldown(t *testing.T) { } r := makeGCQueueScoreImpl( - ctx, seed, now, ms, policy, tc.lastGC, true /* canAdvanceGCThreshold */) + ctx, seed, now, ms, gcTTL, tc.lastGC, true /* canAdvanceGCThreshold */) require.Equal(t, tc.expectGC, r.ShouldQueue) }) } @@ -342,9 +341,8 @@ func (cws *cachedWriteSimulator) shouldQueue( ) { cws.t.Helper() ts := hlc.Timestamp{}.Add(ms.LastUpdateNanos+after.Nanoseconds(), 0) - r := makeGCQueueScoreImpl(context.Background(), 0 /* seed */, ts, ms, zonepb.GCPolicy{ - TTLSeconds: int32(ttl.Seconds()), - }, hlc.Timestamp{}, true /* canAdvanceGCThreshold */) + r := makeGCQueueScoreImpl(context.Background(), 0 /* seed */, ts, ms, ttl, + hlc.Timestamp{}, true /* canAdvanceGCThreshold */) if fmt.Sprintf("%.2f", r.FinalScore) != fmt.Sprintf("%.2f", prio) || b != r.ShouldQueue { cws.t.Errorf("expected queued=%t (is %t), prio=%.2f, got %.2f: after=%s, ttl=%s:\nms: %+v\nscore: %s", b, r.ShouldQueue, prio, r.FinalScore, after, ttl, ms, r) @@ -602,15 +600,15 @@ func TestGCQueueProcess(t *testing.T) { desc := tc.repl.Desc() defer snap.Close() - zone, err := cfg.GetZoneConfigForKey(desc.StartKey) + conf, err := cfg.GetSpanConfigForKey(ctx, desc.StartKey) if err != nil { t.Fatalf("could not find zone config for range %s: %+v", tc.repl, err) } now := tc.Clock().Now() - newThreshold := gc.CalculateThreshold(now, *zone.GC) - return gc.Run(ctx, desc, snap, now, newThreshold, gc.RunOptions{IntentAgeThreshold: intentAgeThreshold}, *zone.GC, - gc.NoopGCer{}, + newThreshold := gc.CalculateThreshold(now, conf.TTL()) + return gc.Run(ctx, desc, snap, now, newThreshold, gc.RunOptions{IntentAgeThreshold: intentAgeThreshold}, + conf.TTL(), gc.NoopGCer{}, func(ctx context.Context, intents []roachpb.Intent) error { return nil }, @@ -629,7 +627,7 @@ func TestGCQueueProcess(t *testing.T) { } // Process through a scan queue. - gcQ := newGCQueue(tc.store, tc.gossip) + gcQ := newGCQueue(tc.store) processed, err := gcQ.process(ctx, tc.repl, cfg) if err != nil { t.Fatal(err) @@ -862,7 +860,7 @@ func TestGCQueueTransactionTable(t *testing.T) { } // Run GC. - gcQ := newGCQueue(tc.store, tc.gossip) + gcQ := newGCQueue(tc.store) cfg := tc.gossip.GetSystemConfig() if cfg == nil { t.Fatal("config not set") @@ -997,12 +995,12 @@ func TestGCQueueIntentResolution(t *testing.T) { } // Process through GC queue. - cfg := tc.gossip.GetSystemConfig() - if cfg == nil { - t.Fatal("config not set") + confReader, err := tc.store.GetConfReader() + if err != nil { + t.Fatal(err) } - gcQ := newGCQueue(tc.store, tc.gossip) - processed, err := gcQ.process(ctx, tc.repl, cfg) + gcQ := newGCQueue(tc.store) + processed, err := gcQ.process(ctx, tc.repl, confReader) if err != nil { t.Fatal(err) } @@ -1058,14 +1056,14 @@ func TestGCQueueLastProcessedTimestamps(t *testing.T) { } } - cfg := tc.gossip.GetSystemConfig() - if cfg == nil { - t.Fatal("config not set") + confReader, err := tc.store.GetConfReader() + if err != nil { + t.Fatal(err) } // Process through a scan queue. - gcQ := newGCQueue(tc.store, tc.gossip) - processed, err := gcQ.process(ctx, tc.repl, cfg) + gcQ := newGCQueue(tc.store) + processed, err := gcQ.process(ctx, tc.repl, confReader) if err != nil { t.Fatal(err) } @@ -1163,17 +1161,17 @@ func TestGCQueueChunkRequests(t *testing.T) { } // Forward the clock past the default GC time. - cfg := tc.gossip.GetSystemConfig() - if cfg == nil { - t.Fatal("config not set") + confReader, err := tc.store.GetConfReader() + if err != nil { + t.Fatal(err) } - zone, err := cfg.GetZoneConfigForKey(roachpb.RKey("key")) + conf, err := confReader.GetSpanConfigForKey(ctx, roachpb.RKey("key")) if err != nil { - t.Fatalf("could not find zone config for range %s", err) + t.Fatalf("could not find span config for range %s", err) } - tc.manualClock.Increment(int64(zone.GC.TTLSeconds)*1e9 + 1) - gcQ := newGCQueue(tc.store, tc.gossip) - processed, err := gcQ.process(ctx, tc.repl, cfg) + tc.manualClock.Increment(int64(conf.TTL().Nanoseconds()) + 1) + gcQ := newGCQueue(tc.store) + processed, err := gcQ.process(ctx, tc.repl, confReader) if err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 598a507bbfd3..bcbd7824b6d0 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -24,7 +24,6 @@ import ( "unsafe" circuit "github.com/cockroachdb/circuitbreaker" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" @@ -51,10 +50,10 @@ func (s *Store) Transport() *RaftTransport { } func (s *Store) FindTargetAndTransferLease( - ctx context.Context, repl *Replica, desc *roachpb.RangeDescriptor, zone *zonepb.ZoneConfig, + ctx context.Context, repl *Replica, desc *roachpb.RangeDescriptor, conf roachpb.SpanConfig, ) (bool, error) { transferStatus, err := s.replicateQueue.shedLease( - ctx, repl, desc, zone, transferLeaseOptions{}, + ctx, repl, desc, conf, transferLeaseOptions{}, ) return transferStatus == transferOK, err } diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index f7196838453b..76d7d29eb041 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -16,12 +16,11 @@ import ( "math" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -89,13 +88,13 @@ type mergeQueue struct { purgChan <-chan time.Time } -func newMergeQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *mergeQueue { +func newMergeQueue(store *Store, db *kv.DB) *mergeQueue { mq := &mergeQueue{ db: db, purgChan: time.NewTicker(mergeQueuePurgatoryCheckInterval).C, } mq.baseQueue = newBaseQueue( - "merge", mq, store, gossip, + "merge", mq, store, queueConfig{ maxSize: defaultQueueMaxSize, maxConcurrency: mergeQueueConcurrency, @@ -130,8 +129,8 @@ func (mq *mergeQueue) enabled() bool { } func (mq *mergeQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, sysCfg *config.SystemConfig, -) (shouldQ bool, priority float64) { + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, confReader spanconfig.StoreReader, +) (shouldQueue bool, priority float64) { if !mq.enabled() { return false, 0 } @@ -143,7 +142,7 @@ func (mq *mergeQueue) shouldQueue( return false, 0 } - if sysCfg.NeedsSplit(ctx, desc.StartKey, desc.EndKey.Next()) { + if confReader.NeedsSplit(ctx, desc.StartKey, desc.EndKey.Next()) { // This range would need to be split if it extended just one key further. // There is thus no possible right-hand neighbor that it could be merged // with. @@ -199,7 +198,7 @@ func (mq *mergeQueue) requestRangeStats( } func (mq *mergeQueue) process( - ctx context.Context, lhsRepl *Replica, sysCfg *config.SystemConfig, + ctx context.Context, lhsRepl *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { if !mq.enabled() { log.VEventf(ctx, 2, "skipping merge: queue has been disabled") @@ -271,7 +270,7 @@ func (mq *mergeQueue) process( // by a small increase in load. conservativeLoadBasedSplitThreshold := 0.5 * lhsRepl.SplitByLoadQPSThreshold() shouldSplit, _ := shouldSplitRange(ctx, mergedDesc, mergedStats, - lhsRepl.GetMaxBytes(), lhsRepl.shouldBackpressureWrites(), sysCfg) + lhsRepl.GetMaxBytes(), lhsRepl.shouldBackpressureWrites(), confReader) if shouldSplit || mergedQPS >= conservativeLoadBasedSplitThreshold { log.VEventf(ctx, 2, "skipping merge to avoid thrashing: merged range %s may split "+ @@ -387,7 +386,7 @@ func (mq *mergeQueue) process( return false, rangeMergePurgatoryError{err} } if testingAggressiveConsistencyChecks { - if _, err := mq.store.consistencyQueue.process(ctx, lhsRepl, sysCfg); err != nil { + if _, err := mq.store.consistencyQueue.process(ctx, lhsRepl, confReader); err != nil { log.Warningf(ctx, "%v", err) } } diff --git a/pkg/kv/kvserver/merge_queue_test.go b/pkg/kv/kvserver/merge_queue_test.go index 621f64928af1..547dcb0c6b9e 100644 --- a/pkg/kv/kvserver/merge_queue_test.go +++ b/pkg/kv/kvserver/merge_queue_test.go @@ -37,7 +37,7 @@ func TestMergeQueueShouldQueue(t *testing.T) { defer stopper.Stop(ctx) testCtx.Start(t, stopper) - mq := newMergeQueue(testCtx.store, testCtx.store.DB(), testCtx.gossip) + mq := newMergeQueue(testCtx.store, testCtx.store.DB()) kvserverbase.MergeQueueEnabled.Override(ctx, &testCtx.store.ClusterSettings().SV, true) tableKey := func(i uint32) []byte { @@ -154,7 +154,7 @@ func TestMergeQueueShouldQueue(t *testing.T) { repl.mu.state.Stats = &enginepb.MVCCStats{KeyBytes: tc.bytes} zoneConfig := zonepb.DefaultZoneConfigRef() zoneConfig.RangeMinBytes = proto.Int64(tc.minBytes) - repl.SetZoneConfig(zoneConfig) + repl.SetSpanConfig(zoneConfig.AsSpanConfig()) shouldQ, priority := mq.shouldQueue(ctx, hlc.ClockTimestamp{}, repl, config.NewSystemConfig(zoneConfig)) if tc.expShouldQ != shouldQ { t.Errorf("incorrect shouldQ: expected %v but got %v", tc.expShouldQ, shouldQ) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 12958f87bc09..b9561bd94bf2 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -17,12 +17,11 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -249,15 +248,13 @@ type queueImpl interface { // shouldQueue accepts current time, a replica, and the system config // and returns whether it should be queued and if so, at what priority. // The Replica is guaranteed to be initialized. - shouldQueue( - context.Context, hlc.ClockTimestamp, *Replica, *config.SystemConfig, - ) (shouldQueue bool, priority float64) + shouldQueue(context.Context, hlc.ClockTimestamp, *Replica, spanconfig.StoreReader) (shouldQueue bool, priority float64) // process accepts a replica, and the system config and executes // queue-specific work on it. The Replica is guaranteed to be initialized. // We return a boolean to indicate if the Replica was processed successfully - // (vs. it being being a no-op or an error). - process(context.Context, *Replica, *config.SystemConfig) (processed bool, err error) + // (vs. it being a no-op or an error). + process(context.Context, *Replica, spanconfig.StoreReader) (processed bool, err error) // timer returns a duration to wait between processing the next item // from the queue. The duration of the last processing of a replica @@ -402,9 +399,8 @@ type baseQueue struct { // from the constructor function will return a queueImpl containing // a pointer to a structure which is a copy of the one within which // it is contained. DANGER. - impl queueImpl - store *Store - gossip *gossip.Gossip + impl queueImpl + store *Store queueConfig incoming chan struct{} // Channel signaled when a new replica is added to the queue. processSem chan struct{} @@ -428,9 +424,7 @@ type baseQueue struct { // replicas from being added, it just limits the total size. Higher priority // replicas can still be added; their addition simply removes the lowest // priority replica. -func newBaseQueue( - name string, impl queueImpl, store *Store, gossip *gossip.Gossip, cfg queueConfig, -) *baseQueue { +func newBaseQueue(name string, impl queueImpl, store *Store, cfg queueConfig) *baseQueue { // Use the default process timeout if none specified. if cfg.processTimeoutFunc == nil { cfg.processTimeoutFunc = defaultProcessTimeoutFunc @@ -457,7 +451,6 @@ func newBaseQueue( name: name, impl: impl, store: store, - gossip: gossip, queueConfig: cfg, incoming: make(chan struct{}, 1), processSem: make(chan struct{}, cfg.maxConcurrency), @@ -612,12 +605,13 @@ func (bq *baseQueue) AddAsync(ctx context.Context, repl replicaInQueue, prio flo func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc.ClockTimestamp) { ctx = repl.AnnotateCtx(ctx) // Load the system config if it's needed. - var cfg *config.SystemConfig + var confReader spanconfig.StoreReader if bq.needsSystemConfig { - cfg = bq.gossip.GetSystemConfig() - if cfg == nil { - if log.V(1) { - log.Infof(ctx, "no system config available. skipping") + var err error + confReader, err = bq.store.GetConfReader() + if err != nil { + if errors.Is(err, errSysCfgUnavailable) && log.V(1) { + log.Warningf(ctx, "unable to retrieve system config, skipping: %v", err) } return } @@ -639,9 +633,9 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. repl.maybeInitializeRaftGroup(ctx) } - if cfg != nil && bq.requiresSplit(ctx, cfg, repl) { - // Range needs to be split due to zone configs, but queue does - // not accept unsplit ranges. + if !bq.acceptsUnsplitRanges && confReader.NeedsSplit(ctx, repl.Desc().StartKey, repl.Desc().EndKey) { + // Range needs to be split due to span configs, but queue does not + // accept unsplit ranges. if log.V(1) { log.Infof(ctx, "split needed; not adding") } @@ -663,7 +657,7 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. // it may not be and shouldQueue will be passed a nil realRepl. These tests // know what they're getting into so that's fine. realRepl, _ := repl.(*Replica) - should, priority := bq.impl.shouldQueue(ctx, now, realRepl, cfg) + should, priority := bq.impl.shouldQueue(ctx, now, realRepl, confReader) if !should { return } @@ -672,16 +666,6 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. } } -func (bq *baseQueue) requiresSplit( - ctx context.Context, cfg *config.SystemConfig, repl replicaInQueue, -) bool { - if bq.acceptsUnsplitRanges { - return false - } - desc := repl.Desc() - return cfg.NeedsSplit(ctx, desc.StartKey, desc.EndKey) -} - // addInternal adds the replica the queue with specified priority. If // the replica is already queued at a lower priority, updates the existing // priority. Expects the queue lock to be held by caller. @@ -907,16 +891,22 @@ func (bq *baseQueue) recordProcessDuration(ctx context.Context, dur time.Duratio // ctx should already be annotated by repl.AnnotateCtx(). func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) error { // Load the system config if it's needed. - var cfg *config.SystemConfig + var confReader spanconfig.StoreReader if bq.needsSystemConfig { - cfg = bq.gossip.GetSystemConfig() - if cfg == nil { - log.VEventf(ctx, 1, "no system config available. skipping") + var err error + confReader, err = bq.store.GetConfReader() + if errors.Is(err, errSysCfgUnavailable) { + if log.V(1) { + log.Warningf(ctx, "unable to retrieve conf reader, skipping: %v", err) + } return nil } + if err != nil { + return err + } } - if cfg != nil && bq.requiresSplit(ctx, cfg, repl) { + if !bq.acceptsUnsplitRanges && confReader.NeedsSplit(ctx, repl.Desc().StartKey, repl.Desc().EndKey) { // Range needs to be split due to zone configs, but queue does // not accept unsplit ranges. log.VEventf(ctx, 3, "split needed; skipping") @@ -966,7 +956,7 @@ func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) er // it may not be and shouldQueue will be passed a nil realRepl. These tests // know what they're getting into so that's fine. realRepl, _ := repl.(*Replica) - processed, err := bq.impl.process(ctx, realRepl, cfg) + processed, err := bq.impl.process(ctx, realRepl, confReader) if err != nil { return err } diff --git a/pkg/kv/kvserver/queue_concurrency_test.go b/pkg/kv/kvserver/queue_concurrency_test.go index 8864936a81e9..ac1ac6ce0cf6 100644 --- a/pkg/kv/kvserver/queue_concurrency_test.go +++ b/pkg/kv/kvserver/queue_concurrency_test.go @@ -17,11 +17,10 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -72,13 +71,13 @@ func TestBaseQueueConcurrent(t *testing.T) { cfg: StoreConfig{ Clock: hlc.NewClock(hlc.UnixNano, time.Second), AmbientCtx: log.AmbientContext{Tracer: tracing.NewTracer()}, - DefaultZoneConfig: zonepb.DefaultZoneConfigRef(), + DefaultSpanConfig: TestingDefaultSpanConfig(), }, } // Set up a queue impl that will return random results from processing. impl := fakeQueueImpl{ - pr: func(context.Context, *Replica, *config.SystemConfig) (bool, error) { + pr: func(context.Context, *Replica, spanconfig.StoreReader) (bool, error) { n := rand.Intn(4) if n == 0 { return true, nil @@ -90,7 +89,7 @@ func TestBaseQueueConcurrent(t *testing.T) { return false, &testPurgatoryError{} }, } - bq := newBaseQueue("test", impl, store, nil /* Gossip */, cfg) + bq := newBaseQueue("test", impl, store, cfg) bq.getReplica = func(id roachpb.RangeID) (replicaInQueue, error) { return &fakeReplica{rangeID: id}, nil } @@ -128,19 +127,19 @@ func TestBaseQueueConcurrent(t *testing.T) { } type fakeQueueImpl struct { - pr func(context.Context, *Replica, *config.SystemConfig) (processed bool, err error) + pr func(context.Context, *Replica, spanconfig.StoreReader) (processed bool, err error) } func (fakeQueueImpl) shouldQueue( - context.Context, hlc.ClockTimestamp, *Replica, *config.SystemConfig, + context.Context, hlc.ClockTimestamp, *Replica, spanconfig.StoreReader, ) (shouldQueue bool, priority float64) { return rand.Intn(5) != 0, 1.0 } func (fq fakeQueueImpl) process( - ctx context.Context, repl *Replica, cfg *config.SystemConfig, + ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, ) (bool, error) { - return fq.pr(ctx, repl, cfg) + return fq.pr(ctx, repl, confReader) } func (fakeQueueImpl) timer(time.Duration) time.Duration { diff --git a/pkg/kv/kvserver/queue_helpers_testutil.go b/pkg/kv/kvserver/queue_helpers_testutil.go index d40ba4131195..6ec847f01af1 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -30,8 +30,8 @@ func forceScanAndProcess(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 cfg := s.Gossip().GetSystemConfig(); cfg == nil { - return errors.Errorf("system config not available in gossip") + if _, err := s.GetConfReader(); err != nil { + return errors.Wrap(err, "unable to retrieve conf reader") } newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index 9c1685453a16..e1ce00ea8d49 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -19,14 +19,12 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -52,13 +50,13 @@ type testQueueImpl struct { } func (tq *testQueueImpl) shouldQueue( - _ context.Context, now hlc.ClockTimestamp, r *Replica, _ *config.SystemConfig, + _ context.Context, now hlc.ClockTimestamp, r *Replica, _ spanconfig.StoreReader, ) (bool, float64) { return tq.shouldQueueFn(now, r) } func (tq *testQueueImpl) process( - _ context.Context, _ *Replica, _ *config.SystemConfig, + _ context.Context, _ *Replica, _ spanconfig.StoreReader, ) (bool, error) { atomic.AddInt32(&tq.processed, 1) if tq.err != nil { @@ -85,9 +83,7 @@ func (tq *testQueueImpl) purgatoryChan() <-chan time.Time { return tq.pChan } -func makeTestBaseQueue( - name string, impl queueImpl, store *Store, gossip *gossip.Gossip, cfg queueConfig, -) *baseQueue { +func makeTestBaseQueue(name string, impl queueImpl, store *Store, cfg queueConfig) *baseQueue { if !cfg.acceptsUnsplitRanges { // Needed in order to pass the validation in newBaseQueue. cfg.needsSystemConfig = true @@ -97,7 +93,7 @@ func makeTestBaseQueue( cfg.pending = metric.NewGauge(metric.Metadata{Name: "pending"}) cfg.processingNanos = metric.NewCounter(metric.Metadata{Name: "processingnanos"}) cfg.purgatory = metric.NewGauge(metric.Metadata{Name: "purgatory"}) - return newBaseQueue(name, impl, store, gossip, cfg) + return newBaseQueue(name, impl, store, cfg) } func createReplicas(t *testing.T, tc *testContext, num int) []*Replica { @@ -195,7 +191,7 @@ func TestBaseQueueAddUpdateAndRemove(t *testing.T) { return shouldAddMap[r], priorityMap[r] }, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: 2}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) bq.maybeAdd(ctx, r1, hlc.ClockTimestamp{}) bq.maybeAdd(ctx, r2, hlc.ClockTimestamp{}) @@ -327,7 +323,7 @@ func TestBaseQueueSamePriorityFIFO(t *testing.T) { }, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: 100}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 100}) for _, repl := range repls { added, err := bq.testingAdd(ctx, repl, 0.0) @@ -367,7 +363,7 @@ func TestBaseQueueAdd(t *testing.T) { return false, 0.0 }, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: 1}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 1}) bq.maybeAdd(context.Background(), r, hlc.ClockTimestamp{}) if bq.Length() != 0 { t.Fatalf("expected length 0; got %d", bq.Length()) @@ -407,7 +403,7 @@ func TestBaseQueueNoop(t *testing.T) { }, noop: false, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: 2}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) bq.Start(stopper) ctx := context.Background() bq.maybeAdd(ctx, r1, hlc.ClockTimestamp{}) @@ -461,7 +457,7 @@ func TestBaseQueueProcess(t *testing.T) { return }, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: 2}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) bq.Start(stopper) ctx := context.Background() @@ -534,7 +530,7 @@ func TestBaseQueueAddRemove(t *testing.T) { return }, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: 2}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) bq.Start(stopper) bq.maybeAdd(ctx, r, hlc.ClockTimestamp{}) @@ -558,11 +554,21 @@ func TestBaseQueueAddRemove(t *testing.T) { func TestNeedsSystemConfig(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - tc := testContext{} stopper := stop.NewStopper() ctx := context.Background() defer stopper.Stop(ctx) - tc.Start(t, stopper) + + tc := testContext{} + cfg := TestStoreConfig(nil) + // Configure a gossip instance that won't have the system config available in it. + cfg.TestingKnobs.MakeSystemConfigSpanUnavailableToQueues = true + tc.StartWithStoreConfig(t, stopper, cfg) + + { + confReader, err := tc.store.GetConfReader() + require.Nil(t, confReader) + require.True(t, errors.Is(err, errSysCfgUnavailable)) + } r, err := tc.store.GetReplica(1) if err != nil { @@ -577,19 +583,8 @@ func TestNeedsSystemConfig(t *testing.T) { }, } - // Use a gossip instance that won't have the system config available in it. // bqNeedsSysCfg will not add the replica or process it without a system config. - rpcContext := rpc.NewContext(rpc.ContextOptions{ - TenantID: roachpb.SystemTenantID, - AmbientCtx: tc.store.cfg.AmbientCtx, - Config: &base.Config{Insecure: true}, - Clock: tc.store.cfg.Clock, - Stopper: stopper, - Settings: cluster.MakeTestingClusterSettings(), - }) - emptyGossip := gossip.NewTest( - tc.gossip.NodeID.Get(), rpcContext, rpc.NewServer(rpcContext), stopper, tc.store.Registry(), zonepb.DefaultZoneConfigRef()) - bqNeedsSysCfg := makeTestBaseQueue("test", testQueue, tc.store, emptyGossip, queueConfig{ + bqNeedsSysCfg := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{ needsSystemConfig: true, acceptsUnsplitRanges: true, maxSize: 1, @@ -615,7 +610,7 @@ func TestNeedsSystemConfig(t *testing.T) { // Now check that a queue which doesn't require the system config can // successfully add and process a replica. - bqNoSysCfg := makeTestBaseQueue("test", testQueue, tc.store, emptyGossip, queueConfig{ + bqNoSysCfg := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{ needsSystemConfig: false, acceptsUnsplitRanges: true, maxSize: 1, @@ -691,7 +686,7 @@ func TestAcceptsUnsplitRanges(t *testing.T) { }, } - bq := makeTestBaseQueue("test", testQueue, s, s.cfg.Gossip, queueConfig{maxSize: 2}) + bq := makeTestBaseQueue("test", testQueue, s, queueConfig{maxSize: 2}) bq.Start(stopper) // Check our config. @@ -801,7 +796,7 @@ func TestBaseQueuePurgatory(t *testing.T) { const replicaCount = 10 repls := createReplicas(t, &tc, replicaCount) - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: replicaCount}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: replicaCount}) bq.Start(stopper) for _, r := range repls { @@ -912,7 +907,7 @@ type processTimeoutQueueImpl struct { } func (pq *processTimeoutQueueImpl) process( - ctx context.Context, r *Replica, _ *config.SystemConfig, + ctx context.Context, r *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { <-ctx.Done() atomic.AddInt32(&pq.processed, 1) @@ -941,7 +936,7 @@ func TestBaseQueueProcessTimeout(t *testing.T) { }, }, } - bq := makeTestBaseQueue("test", ptQueue, tc.store, tc.gossip, + bq := makeTestBaseQueue("test", ptQueue, tc.store, queueConfig{ maxSize: 1, processTimeoutFunc: constantTimeoutFunc(time.Millisecond), @@ -1033,7 +1028,7 @@ type processTimeQueueImpl struct { } func (pq *processTimeQueueImpl) process( - _ context.Context, _ *Replica, _ *config.SystemConfig, + _ context.Context, _ *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { time.Sleep(5 * time.Millisecond) return true, nil @@ -1059,7 +1054,7 @@ func TestBaseQueueTimeMetric(t *testing.T) { }, }, } - bq := makeTestBaseQueue("test", ptQueue, tc.store, tc.gossip, + bq := makeTestBaseQueue("test", ptQueue, tc.store, queueConfig{ maxSize: 1, processTimeoutFunc: constantTimeoutFunc(time.Millisecond), @@ -1134,7 +1129,7 @@ func TestBaseQueueDisable(t *testing.T) { return true, 1.0 }, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{maxSize: 2}) + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{maxSize: 2}) bq.Start(stopper) bq.SetDisabled(true) @@ -1168,13 +1163,13 @@ type parallelQueueImpl struct { } func (pq *parallelQueueImpl) process( - ctx context.Context, repl *Replica, cfg *config.SystemConfig, + ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { atomic.AddInt32(&pq.processing, 1) if pq.processBlocker != nil { <-pq.processBlocker } - processed, err = pq.testQueueImpl.process(ctx, repl, cfg) + processed, err = pq.testQueueImpl.process(ctx, repl, confReader) atomic.AddInt32(&pq.processing, -1) return processed, err } @@ -1203,7 +1198,7 @@ func TestBaseQueueProcessConcurrently(t *testing.T) { }, processBlocker: make(chan struct{}, 1), } - bq := makeTestBaseQueue("test", pQueue, tc.store, tc.gossip, + bq := makeTestBaseQueue("test", pQueue, tc.store, queueConfig{ maxSize: 3, maxConcurrency: 2, @@ -1263,7 +1258,7 @@ func TestBaseQueueChangeReplicaID(t *testing.T) { return true, 1.0 }, } - bq := makeTestBaseQueue("test", testQueue, tc.store, tc.gossip, queueConfig{ + bq := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{ maxSize: defaultQueueMaxSize, acceptsUnsplitRanges: true, }) @@ -1316,7 +1311,7 @@ func TestBaseQueueRequeue(t *testing.T) { }, processBlocker: make(chan struct{}, 1), } - bq := makeTestBaseQueue("test", pQueue, tc.store, tc.gossip, + bq := makeTestBaseQueue("test", pQueue, tc.store, queueConfig{ maxSize: 3, maxConcurrency: 2, diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index f7f725ff4f0b..c7b3c4094235 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -16,10 +16,9 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -71,13 +70,13 @@ type raftLogQueue struct { // log short overall and allowing slower followers to catch up before they get // cut off by a truncation and need a snapshot. See newTruncateDecision for // details on this decision making process. -func newRaftLogQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *raftLogQueue { +func newRaftLogQueue(store *Store, db *kv.DB) *raftLogQueue { rlq := &raftLogQueue{ db: db, logSnapshots: util.Every(10 * time.Second), } rlq.baseQueue = newBaseQueue( - "raftlog", rlq, store, gossip, + "raftlog", rlq, store, queueConfig{ maxSize: defaultQueueMaxSize, maxConcurrency: raftLogQueueConcurrency, @@ -169,8 +168,8 @@ func newTruncateDecision(ctx context.Context, r *Replica) (truncateDecision, err // efficient to catch up via a snapshot than via applying a long tail of log // entries. targetSize := r.store.cfg.RaftLogTruncationThreshold - if targetSize > *r.mu.zone.RangeMaxBytes { - targetSize = *r.mu.zone.RangeMaxBytes + if targetSize > r.mu.conf.RangeMaxBytes { + targetSize = r.mu.conf.RangeMaxBytes } raftStatus := r.raftStatusRLocked() @@ -528,8 +527,8 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision { // is true only if the replica is the raft leader and if the total number of // the range's raft log's stale entries exceeds RaftLogQueueStaleThreshold. func (rlq *raftLogQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, r *Replica, _ *config.SystemConfig, -) (shouldQ bool, priority float64) { + ctx context.Context, now hlc.ClockTimestamp, r *Replica, _ spanconfig.StoreReader, +) (shouldQueue bool, priority float64) { decision, err := newTruncateDecision(ctx, r) if err != nil { log.Warningf(ctx, "%v", err) @@ -570,7 +569,7 @@ func (rlq *raftLogQueue) shouldQueueImpl( // leader and if the total number of the range's raft log's stale entries // exceeds RaftLogQueueStaleThreshold. func (rlq *raftLogQueue) process( - ctx context.Context, r *Replica, _ *config.SystemConfig, + ctx context.Context, r *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { decision, err := newTruncateDecision(ctx, r) if err != nil { diff --git a/pkg/kv/kvserver/raft_snapshot_queue.go b/pkg/kv/kvserver/raft_snapshot_queue.go index 3dd6bfe6e918..87444f5c784e 100644 --- a/pkg/kv/kvserver/raft_snapshot_queue.go +++ b/pkg/kv/kvserver/raft_snapshot_queue.go @@ -14,9 +14,8 @@ import ( "context" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -39,10 +38,10 @@ type raftSnapshotQueue struct { } // newRaftSnapshotQueue returns a new instance of raftSnapshotQueue. -func newRaftSnapshotQueue(store *Store, g *gossip.Gossip) *raftSnapshotQueue { +func newRaftSnapshotQueue(store *Store) *raftSnapshotQueue { rq := &raftSnapshotQueue{} rq.baseQueue = newBaseQueue( - "raftsnapshot", rq, store, g, + "raftsnapshot", rq, store, queueConfig{ maxSize: defaultQueueMaxSize, // The Raft leader (which sends Raft snapshots) may not be the @@ -62,8 +61,8 @@ func newRaftSnapshotQueue(store *Store, g *gossip.Gossip) *raftSnapshotQueue { } func (rq *raftSnapshotQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ *config.SystemConfig, -) (shouldQ bool, priority float64) { + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, +) (shouldQueue bool, priority float64) { // If a follower needs a snapshot, enqueue at the highest priority. if status := repl.RaftStatus(); status != nil { // raft.Status.Progress is only populated on the Raft group leader. @@ -80,7 +79,7 @@ func (rq *raftSnapshotQueue) shouldQueue( } func (rq *raftSnapshotQueue) process( - ctx context.Context, repl *Replica, _ *config.SystemConfig, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { // If a follower requires a Raft snapshot, perform it. if status := repl.RaftStatus(); status != nil { diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 688c5cc3655e..f6b118a78615 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan" @@ -377,8 +376,8 @@ type Replica struct { // lease extension that were in flight at the time of the transfer cannot be // used, if they eventually apply. minLeaseProposedTS hlc.ClockTimestamp - // A pointer to the zone config for this replica. - zone *zonepb.ZoneConfig + // The span config for this replica. + conf roachpb.SpanConfig // proposalBuf buffers Raft commands as they are passed to the Raft // replication subsystem. The buffer is populated by requests after // evaluation and is consumed by the Raft processing thread. Once @@ -550,14 +549,19 @@ type Replica struct { // the request. See the comment on the struct for more details. cachedProtectedTS cachedProtectedTimestampState - // largestPreviousMaxRangeSizeBytes tracks a previous zone.RangeMaxBytes - // which exceeded the current zone.RangeMaxBytes to help defeat the range + // largestPreviousMaxRangeSizeBytes tracks a previous conf.RangeMaxBytes + // which exceeded the current conf.RangeMaxBytes to help defeat the range // backpressure mechanism in cases where a user reduces the configured range - // size. It is set when the zone config changes to a smaller value and the + // size. It is set when the span config changes to a smaller value and the // current range size exceeds the new value. It is cleared after the range's - // size drops below its current zone.MaxRangeBytes or if the - // zone.MaxRangeBytes increases to surpass the current value. + // size drops below its current conf.MaxRangeBytes or if the + // conf.MaxRangeBytes increases to surpass the current value. largestPreviousMaxRangeSizeBytes int64 + // spanConfigExplicitlySet tracks whether a span config was explicitly set + // on this replica (as opposed to it having initialized with the default + // span config). It's used to reason about + // largestPreviousMaxRangeSizeBytes. + spanConfigExplicitlySet bool // failureToGossipSystemConfig is set to true when the leaseholder of the // range containing the system config span fails to gossip due to an @@ -681,48 +685,43 @@ func (r *Replica) cleanupFailedProposalLocked(p *ProposalData) { func (r *Replica) GetMinBytes() int64 { r.mu.RLock() defer r.mu.RUnlock() - return *r.mu.zone.RangeMinBytes + return r.mu.conf.RangeMinBytes } // GetMaxBytes gets the replica's maximum byte threshold. func (r *Replica) GetMaxBytes() int64 { r.mu.RLock() defer r.mu.RUnlock() - return *r.mu.zone.RangeMaxBytes + return r.mu.conf.RangeMaxBytes } -// SetZoneConfig sets the replica's zone config. -func (r *Replica) SetZoneConfig(zone *zonepb.ZoneConfig) { +// SetSpanConfig sets the replica's span config. +func (r *Replica) SetSpanConfig(conf roachpb.SpanConfig) { r.mu.Lock() defer r.mu.Unlock() - if r.isInitializedRLocked() && - r.mu.zone != nil && - zone != nil { + if r.isInitializedRLocked() && !r.mu.conf.IsEmpty() && !conf.IsEmpty() { total := r.mu.state.Stats.Total() - // Set largestPreviousMaxRangeSizeBytes if the current range size is above - // the new limit and we don't already have a larger value. Reset it if - // the new limit is larger than the current largest we're aware of. - if total > *zone.RangeMaxBytes && - *zone.RangeMaxBytes < *r.mu.zone.RangeMaxBytes && - r.mu.largestPreviousMaxRangeSizeBytes < *r.mu.zone.RangeMaxBytes && - // Check to make sure that we're replacing a real zone config. Otherwise - // the default value would prevent backpressure until the range was - // larger than the default value. When the store starts up it sets the - // zone for the replica to this default value; later on it overwrites it - // with a new instance even if the value is the same as the default. - r.mu.zone != r.store.cfg.DefaultZoneConfig && - r.mu.zone != r.store.cfg.DefaultSystemZoneConfig { - - r.mu.largestPreviousMaxRangeSizeBytes = *r.mu.zone.RangeMaxBytes + // Set largestPreviousMaxRangeSizeBytes if the current range size is + // greater than the new limit, if the limit has decreased from what we + // last remember, and we don't already have a larger value. + if total > conf.RangeMaxBytes && conf.RangeMaxBytes < r.mu.conf.RangeMaxBytes && + r.mu.largestPreviousMaxRangeSizeBytes < r.mu.conf.RangeMaxBytes && + // We also want to make sure that we're replacing a real span config. + // If we didn't have this check, the default value would prevent + // backpressure until the range got larger than it. + r.mu.spanConfigExplicitlySet { + r.mu.largestPreviousMaxRangeSizeBytes = r.mu.conf.RangeMaxBytes } else if r.mu.largestPreviousMaxRangeSizeBytes > 0 && - r.mu.largestPreviousMaxRangeSizeBytes < *zone.RangeMaxBytes { - + r.mu.largestPreviousMaxRangeSizeBytes < conf.RangeMaxBytes { + // Reset it if the new limit is larger than the largest we were + // aware of. r.mu.largestPreviousMaxRangeSizeBytes = 0 } } - r.mu.zone = zone + + r.mu.conf, r.mu.spanConfigExplicitlySet = conf, true } // IsFirstRange returns true if this is the first range. @@ -749,12 +748,19 @@ func (r *Replica) IsQuiescent() bool { return r.mu.quiescent } -// DescAndZone returns the authoritative range descriptor as well -// as the zone config for the replica. -func (r *Replica) DescAndZone() (*roachpb.RangeDescriptor, *zonepb.ZoneConfig) { +// DescAndSpanConfig returns the authoritative range descriptor as well +// as the span config for the replica. +func (r *Replica) DescAndSpanConfig() (*roachpb.RangeDescriptor, roachpb.SpanConfig) { r.mu.RLock() defer r.mu.RUnlock() - return r.mu.state.Desc, r.mu.zone + return r.mu.state.Desc, r.mu.conf +} + +// SpanConfig returns the authoritative span config for the replica. +func (r *Replica) SpanConfig() roachpb.SpanConfig { + r.mu.RLock() + defer r.mu.RUnlock() + return r.mu.conf } // Desc returns the authoritative range descriptor, acquiring a replica lock in @@ -771,11 +777,13 @@ func (r *Replica) descRLocked() *roachpb.RangeDescriptor { } // closedTimestampPolicyRLocked returns the closed timestamp policy of the -// range, which is updated asynchronously through gossip of zone configurations. +// range, which is updated asynchronously by listening in on span configuration +// changes. +// // NOTE: an exported version of this method which does not require the replica // lock exists in helpers_test.go. Move here if needed. func (r *Replica) closedTimestampPolicyRLocked() roachpb.RangeClosedTimestampPolicy { - if r.mu.zone.GlobalReads != nil && *r.mu.zone.GlobalReads { + if r.mu.conf.GlobalReads { if !r.mu.state.Desc.ContainsKey(roachpb.RKey(keys.NodeLivenessPrefix)) { return roachpb.LEAD_FOR_GLOBAL_READS } @@ -941,7 +949,7 @@ func (r *Replica) getImpliedGCThresholdRLocked( return threshold } - impliedThreshold := gc.CalculateThreshold(st.Now.ToTimestamp(), *r.mu.zone.GC) + impliedThreshold := gc.CalculateThreshold(st.Now.ToTimestamp(), r.mu.conf.TTL()) threshold.Forward(impliedThreshold) // If we have a protected timestamp record which precedes the implied @@ -1175,7 +1183,7 @@ func (r *Replica) State(ctx context.Context) kvserverpb.RangeInfo { } } } - ri.RangeMaxBytes = *r.mu.zone.RangeMaxBytes + ri.RangeMaxBytes = r.mu.conf.RangeMaxBytes if desc := ri.ReplicaState.Desc; desc != nil { // Learner replicas don't serve follower reads, but they still receive // closed timestamp updates, so include them here. diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 80de1ca9c959..c5ef9a7fa1cf 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -918,7 +918,7 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error { // If the range is now less than its RangeMaxBytes, clear the history of its // largest previous max bytes. - if r.mu.largestPreviousMaxRangeSizeBytes > 0 && b.state.Stats.Total() < *r.mu.zone.RangeMaxBytes { + if r.mu.largestPreviousMaxRangeSizeBytes > 0 && b.state.Stats.Total() < r.mu.conf.RangeMaxBytes { r.mu.largestPreviousMaxRangeSizeBytes = 0 } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index d8732a03bedf..10d48087b433 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2854,11 +2854,11 @@ func (s *Store) relocateOne( `range %s was either in a joint configuration or had learner replicas: %v`, desc, desc.Replicas()) } - sysCfg := s.cfg.Gossip.GetSystemConfig() - if sysCfg == nil { - return nil, nil, fmt.Errorf("no system config available, unable to perform RelocateRange") + confReader, err := s.GetConfReader() + if err != nil { + return nil, nil, errors.Wrap(err, "can't relocate range") } - zone, err := sysCfg.GetZoneConfigForKey(desc.StartKey) + conf, err := confReader.GetSpanConfigForKey(ctx, desc.StartKey) if err != nil { return nil, nil, err } @@ -2905,7 +2905,7 @@ func (s *Store) relocateOne( targetStore, _ := s.allocator.allocateTargetFromList( ctx, candidateStoreList, - zone, + conf, existingVoters, existingNonVoters, s.allocator.scorerOptions(), @@ -2976,7 +2976,7 @@ func (s *Store) relocateOne( // overreplicated. If we asked it instead to remove s3 from (s1,s2,s3) it // may not want to do that due to constraints. targetStore, _, err := s.allocator.removeTarget( - ctx, zone, args.targetsToRemove(), existingVoters, + ctx, conf, args.targetsToRemove(), existingVoters, existingNonVoters, args.targetType, ) if err != nil { diff --git a/pkg/kv/kvserver/replica_gc_queue.go b/pkg/kv/kvserver/replica_gc_queue.go index 249e88588ca4..33a82568c5a3 100644 --- a/pkg/kv/kvserver/replica_gc_queue.go +++ b/pkg/kv/kvserver/replica_gc_queue.go @@ -14,10 +14,9 @@ import ( "context" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" @@ -85,14 +84,14 @@ type replicaGCQueue struct { } // newReplicaGCQueue returns a new instance of replicaGCQueue. -func newReplicaGCQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *replicaGCQueue { +func newReplicaGCQueue(store *Store, db *kv.DB) *replicaGCQueue { rgcq := &replicaGCQueue{ metrics: makeReplicaGCQueueMetrics(), db: db, } store.metrics.registry.AddMetricStruct(&rgcq.metrics) rgcq.baseQueue = newBaseQueue( - "replicaGC", rgcq, store, gossip, + "replicaGC", rgcq, store, queueConfig{ maxSize: defaultQueueMaxSize, needsLease: false, @@ -116,8 +115,8 @@ func newReplicaGCQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *replicaG // check must have occurred more than ReplicaGCQueueInactivityThreshold // in the past. func (rgcq *replicaGCQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ *config.SystemConfig, -) (shouldQ bool, prio float64) { + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, +) (shouldQueue bool, priority float64) { if _, currentMember := repl.Desc().GetReplicaDescriptor(repl.store.StoreID()); !currentMember { return true, replicaGCPriorityRemoved } @@ -216,7 +215,7 @@ func replicaGCShouldQueueImpl(now, lastCheck hlc.Timestamp, isSuspect bool) (boo // process performs a consistent lookup on the range descriptor to see if we are // still a member of the range. func (rgcq *replicaGCQueue) process( - ctx context.Context, repl *Replica, _ *config.SystemConfig, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { // Note that the Replicas field of desc is probably out of date, so // we should only use `desc` for its static fields like RangeID and diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index 1dee742ce07d..e0d3a46b0a46 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -89,7 +89,7 @@ func newUnloadedReplica( r.mu.pendingLeaseRequest = makePendingLeaseRequest(r) r.mu.stateLoader = stateloader.Make(desc.RangeID) r.mu.quiescent = true - r.mu.zone = store.cfg.DefaultZoneConfig + r.mu.conf = store.cfg.DefaultSpanConfig r.mu.replicaID = replicaID split.Init(&r.loadBasedSplitter, rand.Intn, func() float64 { return float64(SplitByLoadQPSThreshold.Get(&store.cfg.Settings.SV)) diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index 7504ff1b8742..70ee3143dbde 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -14,7 +14,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -61,7 +60,7 @@ func (r *Replica) Metrics( leaseStatus := r.leaseStatusAtRLocked(ctx, now) quiescent := r.mu.quiescent || r.mu.internalRaftGroup == nil desc := r.mu.state.Desc - zone := r.mu.zone + conf := r.mu.conf raftLogSize := r.mu.raftLogSize raftLogSizeTrusted := r.mu.raftLogSizeTrusted r.mu.RUnlock() @@ -77,7 +76,7 @@ func (r *Replica) Metrics( ctx, now.ToTimestamp(), &r.store.cfg.RaftConfig, - zone, + conf, livenessMap, clusterNodes, desc, @@ -97,7 +96,7 @@ func calcReplicaMetrics( _ context.Context, _ hlc.Timestamp, raftCfg *base.RaftConfig, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, livenessMap liveness.IsLiveMap, clusterNodes int, desc *roachpb.RangeDescriptor, @@ -126,7 +125,7 @@ func calcReplicaMetrics( m.Ticking = ticking m.RangeCounter, m.Unavailable, m.Underreplicated, m.Overreplicated = calcRangeCounter( - storeID, desc, leaseStatus, livenessMap, zone.GetNumVoters(), *zone.NumReplicas, clusterNodes) + storeID, desc, leaseStatus, livenessMap, conf.GetNumVoters(), conf.NumReplicas, clusterNodes) const raftLogTooLargeMultiple = 4 m.RaftLogTooLarge = raftLogSize > (raftLogTooLargeMultiple*raftCfg.RaftLogTruncationThreshold) && @@ -272,7 +271,7 @@ func (r *Replica) needsSplitBySizeRLocked() bool { } func (r *Replica) needsMergeBySizeRLocked() bool { - return r.mu.state.Stats.Total() < *r.mu.zone.RangeMinBytes + return r.mu.state.Stats.Total() < r.mu.conf.RangeMinBytes } func (r *Replica) needsRaftLogTruncationLocked() bool { @@ -291,11 +290,11 @@ func (r *Replica) needsRaftLogTruncationLocked() bool { // exceedsMultipleOfSplitSizeRLocked returns whether the current size of the // range exceeds the max size times mult. If so, the bytes overage is also // returned. Note that the max size is determined by either the current maximum -// size as dictated by the zone config or a previous max size indicating that +// size as dictated by the span config or a previous max size indicating that // the max size has changed relatively recently and thus we should not // backpressure for being over. func (r *Replica) exceedsMultipleOfSplitSizeRLocked(mult float64) (exceeded bool, bytesOver int64) { - maxBytes := *r.mu.zone.RangeMaxBytes + maxBytes := r.mu.conf.RangeMaxBytes if r.mu.largestPreviousMaxRangeSizeBytes > maxBytes { maxBytes = r.mu.largestPreviousMaxRangeSizeBytes } diff --git a/pkg/kv/kvserver/replica_protected_timestamp.go b/pkg/kv/kvserver/replica_protected_timestamp.go index c1dd377a4ea7..b7bb8934e973 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp.go +++ b/pkg/kv/kvserver/replica_protected_timestamp.go @@ -13,8 +13,8 @@ package kvserver import ( "context" "fmt" + "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" @@ -260,7 +260,7 @@ func (r *Replica) protectedTimestampRecordCurrentlyApplies( // basis to calculate the new gc threshold (used for scoring and reporting), the // old gc threshold, and the new gc threshold. func (r *Replica) checkProtectedTimestampsForGC( - ctx context.Context, policy zonepb.GCPolicy, + ctx context.Context, gcTTL time.Duration, ) (canGC bool, cacheTimestamp, gcTimestamp, oldThreshold, newThreshold hlc.Timestamp) { // We may be reading the protected timestamp cache while we're holding @@ -285,7 +285,7 @@ func (r *Replica) checkProtectedTimestampsForGC( if read.earliestRecord != nil { // NB: we want to allow GC up to the timestamp preceding the earliest valid // record. - impliedGCTimestamp := gc.TimestampForThreshold(read.earliestRecord.Timestamp.Prev(), policy) + impliedGCTimestamp := gc.TimestampForThreshold(read.earliestRecord.Timestamp.Prev(), gcTTL) if impliedGCTimestamp.Less(gcTimestamp) { gcTimestamp = impliedGCTimestamp } @@ -297,7 +297,7 @@ func (r *Replica) checkProtectedTimestampsForGC( return false, hlc.Timestamp{}, hlc.Timestamp{}, hlc.Timestamp{}, hlc.Timestamp{} } - newThreshold = gc.CalculateThreshold(gcTimestamp, policy) + newThreshold = gc.CalculateThreshold(gcTimestamp, gcTTL) return true, read.readAt, gcTimestamp, oldThreshold, newThreshold } diff --git a/pkg/kv/kvserver/replica_protected_timestamp_test.go b/pkg/kv/kvserver/replica_protected_timestamp_test.go index 393945748206..874b5385d995 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp_test.go +++ b/pkg/kv/kvserver/replica_protected_timestamp_test.go @@ -16,7 +16,6 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" @@ -409,8 +408,8 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - makePolicy := func(ttlSec int32) zonepb.GCPolicy { - return zonepb.GCPolicy{TTLSeconds: ttlSec} + makeTTLDuration := func(ttlSec int32) time.Duration { + return time.Duration(ttlSec) * time.Second } for _, testCase := range []struct { name string @@ -422,7 +421,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { name: "lease is too new", test: func(t *testing.T, r *Replica, mt *manualCache) { r.mu.state.Lease.Start = r.store.Clock().NowAsClockTimestamp() - canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makePolicy(10)) + canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.False(t, canGC) require.Zero(t, gcTimestamp) }, @@ -444,7 +443,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { }) // We should allow gc to proceed with the normal new threshold if that // threshold is earlier than all of the records. - canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makePolicy(10)) + canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.True(t, canGC) require.Equal(t, mt.asOf, gcTimestamp) }, @@ -469,7 +468,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { // We should allow gc to proceed up to the timestamp which precedes the // protected timestamp. This means we expect a GC timestamp 10 seconds // after ts.Prev() given the policy. - canGC, _, gcTimestamp, oldThreshold, newThreshold := r.checkProtectedTimestampsForGC(ctx, makePolicy(10)) + canGC, _, gcTimestamp, oldThreshold, newThreshold := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.True(t, canGC) require.False(t, newThreshold.Equal(oldThreshold)) require.Equal(t, ts.Prev().Add(10*time.Second.Nanoseconds(), 0), gcTimestamp) @@ -498,7 +497,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { // predecessor of the earliest valid record. However, the GC // queue does not enqueue ranges in such cases, so this is only // applicable to manually enqueued ranges. - canGC, _, gcTimestamp, oldThreshold, newThreshold := r.checkProtectedTimestampsForGC(ctx, makePolicy(10)) + canGC, _, gcTimestamp, oldThreshold, newThreshold := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.True(t, canGC) require.True(t, newThreshold.Equal(oldThreshold)) require.Equal(t, th.Add(10*time.Second.Nanoseconds(), 0), gcTimestamp) @@ -522,7 +521,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { }, }, }) - canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makePolicy(10)) + canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.True(t, canGC) require.Equal(t, mt.asOf, gcTimestamp) }, diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index cf03ab608877..0f3b353fef03 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -689,7 +689,7 @@ func (r *Replica) append( // updateRangeInfo is called whenever a range is updated by ApplySnapshot // or is created by range splitting to setup the fields which are // uninitialized or need updating. -func (r *Replica) updateRangeInfo(desc *roachpb.RangeDescriptor) error { +func (r *Replica) updateRangeInfo(ctx context.Context, desc *roachpb.RangeDescriptor) error { // RangeMaxBytes should be updated by looking up Zone Config in two cases: // 1. After applying a snapshot, if the zone config was not updated for // this key range, then maxBytes of this range will not be updated either. @@ -697,22 +697,24 @@ func (r *Replica) updateRangeInfo(desc *roachpb.RangeDescriptor) error { // the original range wont work as the original and new ranges might belong // to different zones. // Load the system config. - cfg := r.store.Gossip().GetSystemConfig() - if cfg == nil { - // This could be before the system config was ever gossiped, - // or it expired. Let the gossip callback set the info. - ctx := r.AnnotateCtx(context.TODO()) - log.Warningf(ctx, "no system config available, cannot determine range MaxBytes") + confReader, err := r.store.GetConfReader() + 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. + log.Warningf(ctx, "unable to retrieve conf reader, cannot determine range MaxBytes") return nil } + if err != nil { + return err + } - // Find zone config for this range. - zone, err := cfg.GetZoneConfigForKey(desc.StartKey) + // Find span config for this range. + conf, err := confReader.GetSpanConfigForKey(ctx, desc.StartKey) if err != nil { - return errors.Errorf("%s: failed to lookup zone config: %s", r, err) + return errors.Errorf("%s: failed to lookup span config: %s", r, err) } - r.SetZoneConfig(zone) + r.SetSpanConfig(conf) return nil } @@ -1058,7 +1060,7 @@ func (r *Replica) applySnapshot( // Update the replica's cached byte thresholds. This is a no-op if the system // config is not available, in which case we rely on the next gossip update // to perform the update. - if err := r.updateRangeInfo(s.Desc); err != nil { + if err := r.updateRangeInfo(ctx, s.Desc); err != nil { log.Fatalf(ctx, "unable to update range info while applying snapshot: %+v", err) } diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index b20454dfb0e0..5b949aa25a73 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -1303,21 +1303,21 @@ func (r *Replica) maybeExtendLeaseAsync(ctx context.Context, st kvserverpb.Lease } // checkLeaseRespectsPreferences checks if current replica owns the lease and -// if it respects the lease preferences defined in the zone config. If there are no +// if it respects the lease preferences defined in the span config. If there are no // preferences defined then it will return true and consider that to be in-conformance. func (r *Replica) checkLeaseRespectsPreferences(ctx context.Context) (bool, error) { if !r.OwnsValidLease(ctx, r.store.cfg.Clock.NowAsClockTimestamp()) { return false, errors.Errorf("replica %s is not the leaseholder, cannot check lease preferences", r) } - _, zone := r.DescAndZone() - if len(zone.LeasePreferences) == 0 { + conf := r.SpanConfig() + if len(conf.LeasePreferences) == 0 { return true, nil } storeDesc, err := r.store.Descriptor(ctx, false /* useCached */) if err != nil { return false, err } - for _, preference := range zone.LeasePreferences { + for _, preference := range conf.LeasePreferences { if constraint.ConjunctionsCheck(*storeDesc, preference.Constraints) { return true, nil } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 05c639974999..0f10ee25d1ec 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -69,7 +69,6 @@ import ( "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" "github.com/cockroachdb/redact" - "github.com/gogo/protobuf/proto" "github.com/kr/pretty" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -230,7 +229,7 @@ func (tc *testContext) StartWithStoreConfigAndVersion( Settings: cfg.Settings, }) server := rpc.NewServer(rpcContext) // never started - tc.gossip = gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), cfg.DefaultZoneConfig) + tc.gossip = gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) } if tc.engine == nil { var err error @@ -9056,15 +9055,15 @@ func TestReplicaMetrics(t *testing.T) { for i, c := range testCases { t.Run("", func(t *testing.T) { - zoneConfig := protoutil.Clone(cfg.DefaultZoneConfig).(*zonepb.ZoneConfig) - zoneConfig.NumReplicas = proto.Int32(c.replicas) + spanConfig := cfg.DefaultSpanConfig + spanConfig.NumReplicas = c.replicas // Alternate between quiescent and non-quiescent replicas to test the // quiescent metric. c.expected.Quiescent = i%2 == 0 c.expected.Ticking = !c.expected.Quiescent metrics := calcReplicaMetrics( - context.Background(), hlc.Timestamp{}, &cfg.RaftConfig, zoneConfig, + context.Background(), hlc.Timestamp{}, &cfg.RaftConfig, spanConfig, c.liveness, 0, &c.desc, c.raftStatus, kvserverpb.LeaseStatus{}, c.storeID, c.expected.Quiescent, c.expected.Ticking, concurrency.LatchMetrics{}, concurrency.LockTableMetrics{}, c.raftLogSize, true) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 88eb11a6c8bb..976bc0603e8c 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -19,13 +19,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" @@ -169,7 +168,7 @@ type replicateQueue struct { } // newReplicateQueue returns a new instance of replicateQueue. -func newReplicateQueue(store *Store, g *gossip.Gossip, allocator Allocator) *replicateQueue { +func newReplicateQueue(store *Store, allocator Allocator) *replicateQueue { rq := &replicateQueue{ metrics: makeReplicateQueueMetrics(), allocator: allocator, @@ -177,7 +176,7 @@ func newReplicateQueue(store *Store, g *gossip.Gossip, allocator Allocator) *rep } store.metrics.registry.AddMetricStruct(&rq.metrics) rq.baseQueue = newBaseQueue( - "replicate", rq, store, g, + "replicate", rq, store, queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, @@ -205,7 +204,7 @@ func newReplicateQueue(store *Store, g *gossip.Gossip, allocator Allocator) *rep // Register gossip and node liveness callbacks to signal that // replicas in purgatory might be retried. - if g != nil { // gossip is nil for some unittests + if g := store.cfg.Gossip; g != nil { // gossip is nil for some unittests g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStorePrefix), func(key string, _ roachpb.Value) { if !rq.store.IsStarted() { return @@ -229,10 +228,10 @@ func newReplicateQueue(store *Store, g *gossip.Gossip, allocator Allocator) *rep } func (rq *replicateQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, sysCfg *config.SystemConfig, -) (shouldQ bool, priority float64) { - desc, zone := repl.DescAndZone() - action, priority := rq.allocator.ComputeAction(ctx, zone, desc) + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, +) (shouldQueue bool, priority float64) { + desc, conf := repl.DescAndSpanConfig() + action, priority := rq.allocator.ComputeAction(ctx, conf, desc) if action == AllocatorNoop { log.VEventf(ctx, 2, "no action to take") @@ -248,7 +247,7 @@ func (rq *replicateQueue) shouldQueue( rangeUsageInfo := rangeUsageInfoForRepl(repl) _, _, _, ok := rq.allocator.RebalanceVoter( ctx, - zone, + conf, repl.RaftStatus(), voterReplicas, nonVoterReplicas, @@ -261,7 +260,7 @@ func (rq *replicateQueue) shouldQueue( } _, _, _, ok = rq.allocator.RebalanceNonVoter( ctx, - zone, + conf, repl.RaftStatus(), voterReplicas, nonVoterReplicas, @@ -279,7 +278,7 @@ func (rq *replicateQueue) shouldQueue( status := repl.LeaseStatusAt(ctx, now) if status.IsValid() && rq.canTransferLeaseFrom(ctx, repl) && - rq.allocator.ShouldTransferLease(ctx, zone, voterReplicas, status.Lease.Replica.StoreID, repl.leaseholderStats) { + rq.allocator.ShouldTransferLease(ctx, conf, voterReplicas, status.Lease.Replica.StoreID, repl.leaseholderStats) { log.VEventf(ctx, 2, "lease transfer needed, enqueuing") return true, 0 @@ -289,7 +288,7 @@ func (rq *replicateQueue) shouldQueue( } func (rq *replicateQueue) process( - ctx context.Context, repl *Replica, sysCfg *config.SystemConfig, + ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { retryOpts := retry.Options{ InitialBackoff: 50 * time.Millisecond, @@ -320,7 +319,7 @@ func (rq *replicateQueue) process( } if testingAggressiveConsistencyChecks { - if _, err := rq.store.consistencyQueue.process(ctx, repl, sysCfg); err != nil { + if _, err := rq.store.consistencyQueue.process(ctx, repl, confReader); err != nil { log.Warningf(ctx, "%v", err) } } @@ -359,7 +358,7 @@ func (rq *replicateQueue) processOneChange( // upon that decision is a bit unfortunate. It means that we could // successfully execute a decision that was based on the state of a stale // range descriptor. - desc, zone := repl.DescAndZone() + desc, conf := repl.DescAndSpanConfig() // Avoid taking action if the range has too many dead replicas to make quorum. // Consider stores marked suspect as live in order to make this determination. @@ -376,7 +375,7 @@ func (rq *replicateQueue) processOneChange( // unavailability; see: _ = execChangeReplicasTxn - action, _ := rq.allocator.ComputeAction(ctx, zone, desc) + action, _ := rq.allocator.ComputeAction(ctx, conf, desc) log.VEventf(ctx, 1, "next replica action: %s", action) // For simplicity, the first thing the allocator does is remove learners, so @@ -519,7 +518,7 @@ func (rq *replicateQueue) addOrReplaceVoters( removeIdx int, dryRun bool, ) (requeue bool, _ error) { - desc, zone := repl.DescAndZone() + desc, conf := repl.DescAndSpanConfig() existingVoters := desc.Replicas().VoterDescriptors() if len(existingVoters) == 1 { // If only one replica remains, that replica is the leaseholder and @@ -555,7 +554,7 @@ func (rq *replicateQueue) addOrReplaceVoters( // we're removing it (i.e. dead or decommissioning). If we left the replica in // the slice, the allocator would not be guaranteed to pick a replica that // fills the gap removeRepl leaves once it's gone. - newStore, details, err := rq.allocator.AllocateVoter(ctx, zone, remainingLiveVoters, remainingLiveNonVoters) + newStore, details, err := rq.allocator.AllocateVoter(ctx, conf, remainingLiveVoters, remainingLiveNonVoters) if err != nil { return false, err } @@ -568,7 +567,7 @@ func (rq *replicateQueue) addOrReplaceVoters( } clusterNodes := rq.allocator.storePool.ClusterNodeCount() - neededVoters := GetNeededVoters(zone.GetNumVoters(), clusterNodes) + neededVoters := GetNeededVoters(conf.GetNumVoters(), clusterNodes) // Only up-replicate if there are suitable allocation targets such that, // either the replication goal is met, or it is possible to get to the next @@ -591,7 +590,7 @@ func (rq *replicateQueue) addOrReplaceVoters( NodeID: newStore.Node.NodeID, StoreID: newStore.StoreID, }) - _, _, err := rq.allocator.AllocateVoter(ctx, zone, oldPlusNewReplicas, remainingLiveNonVoters) + _, _, err := rq.allocator.AllocateVoter(ctx, conf, oldPlusNewReplicas, remainingLiveNonVoters) if err != nil { // It does not seem possible to go to the next odd replica state. Note // that AllocateVoter returns an allocatorError (a purgatoryError) @@ -667,10 +666,10 @@ func (rq *replicateQueue) addOrReplaceNonVoters( return false, errors.AssertionFailedf("non-voting replicas cannot be created pre-21.1") } - desc, zone := repl.DescAndZone() + desc, conf := repl.DescAndSpanConfig() existingNonVoters := desc.Replicas().NonVoterDescriptors() - newStore, details, err := rq.allocator.AllocateNonVoter(ctx, zone, liveVoterReplicas, liveNonVoterReplicas) + newStore, details, err := rq.allocator.AllocateNonVoter(ctx, conf, liveVoterReplicas, liveNonVoterReplicas) if err != nil { return false, err } @@ -723,13 +722,13 @@ func (rq *replicateQueue) addOrReplaceNonVoters( func (rq *replicateQueue) findRemoveVoter( ctx context.Context, repl interface { - DescAndZone() (*roachpb.RangeDescriptor, *zonepb.ZoneConfig) + DescAndSpanConfig() (*roachpb.RangeDescriptor, roachpb.SpanConfig) LastReplicaAdded() (roachpb.ReplicaID, time.Time) RaftStatus() *raft.Status }, existingVoters, existingNonVoters []roachpb.ReplicaDescriptor, ) (roachpb.ReplicaDescriptor, string, error) { - _, zone := repl.DescAndZone() + _, zone := repl.DescAndSpanConfig() // This retry loop involves quick operations on local state, so a // small MaxBackoff is good (but those local variables change on // network time scales as raft receives responses). @@ -813,7 +812,7 @@ func (rq *replicateQueue) maybeTransferLeaseAway( if canTransferLeaseFrom != nil && !canTransferLeaseFrom(ctx, repl) { return false, errors.Errorf("cannot transfer lease") } - desc, zone := repl.DescAndZone() + desc, conf := repl.DescAndSpanConfig() // The local replica was selected as the removal target, but that replica // is the leaseholder, so transfer the lease instead. We don't check that // the current store has too many leases in this case under the @@ -828,7 +827,7 @@ func (rq *replicateQueue) maybeTransferLeaseAway( ctx, repl, desc, - zone, + conf, transferLeaseOptions{ dryRun: dryRun, }, @@ -864,7 +863,7 @@ func (rq *replicateQueue) removeVoter( NodeID: removeVoter.NodeID, StoreID: removeVoter.StoreID, } - desc, _ := repl.DescAndZone() + desc := repl.Desc() // TODO(aayush): Directly removing the voter here is a bit of a missed // opportunity since we could potentially be 1 non-voter short and the // `target` could be a valid store for a non-voter. In such a scenario, we @@ -893,10 +892,10 @@ func (rq *replicateQueue) removeNonVoter( ) (requeue bool, _ error) { rq.metrics.RemoveReplicaCount.Inc(1) - desc, zone := repl.DescAndZone() + desc, conf := repl.DescAndSpanConfig() removeNonVoter, details, err := rq.allocator.RemoveNonVoter( ctx, - zone, + conf, existingNonVoters, existingVoters, existingNonVoters, ) @@ -929,7 +928,7 @@ func (rq *replicateQueue) removeNonVoter( func (rq *replicateQueue) removeDecommissioning( ctx context.Context, repl *Replica, targetType targetReplicaType, dryRun bool, ) (requeue bool, _ error) { - desc, _ := repl.DescAndZone() + desc := repl.Desc() var decommissioningReplicas []roachpb.ReplicaDescriptor switch targetType { case voterTarget: @@ -1069,13 +1068,13 @@ func (rq *replicateQueue) considerRebalance( canTransferLeaseFrom func(ctx context.Context, repl *Replica) bool, dryRun bool, ) (requeue bool, _ error) { - desc, zone := repl.DescAndZone() + desc, conf := repl.DescAndSpanConfig() rebalanceTargetType := voterTarget if !rq.store.TestingKnobs().DisableReplicaRebalancing { rangeUsageInfo := rangeUsageInfoForRepl(repl) addTarget, removeTarget, details, ok := rq.allocator.RebalanceVoter( ctx, - zone, + conf, repl.RaftStatus(), existingVoters, existingNonVoters, @@ -1088,7 +1087,7 @@ func (rq *replicateQueue) considerRebalance( log.VEventf(ctx, 1, "no suitable rebalance target for voters") addTarget, removeTarget, details, ok = rq.allocator.RebalanceNonVoter( ctx, - zone, + conf, repl.RaftStatus(), existingVoters, existingNonVoters, @@ -1156,7 +1155,7 @@ func (rq *replicateQueue) considerRebalance( ctx, repl, desc, - zone, + conf, transferLeaseOptions{ checkTransferLeaseSource: true, checkCandidateFullness: true, @@ -1303,14 +1302,14 @@ func (rq *replicateQueue) shedLease( ctx context.Context, repl *Replica, desc *roachpb.RangeDescriptor, - zone *zonepb.ZoneConfig, + conf roachpb.SpanConfig, opts transferLeaseOptions, ) (leaseTransferOutcome, error) { // Learner replicas aren't allowed to become the leaseholder or raft leader, // so only consider the `VoterDescriptors` replicas. target := rq.allocator.TransferLeaseTarget( ctx, - zone, + conf, desc.Replicas().VoterDescriptors(), repl.store.StoreID(), repl.leaseholderStats, diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index c3fcd4d8aa22..b44fb3463487 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -1234,13 +1234,13 @@ func TestTransferLeaseToLaggingNode(t *testing.T) { // Set the zone preference for the replica to show that it has to be moved // to the remote node. - desc, zone := leaseHolderRepl.DescAndZone() - newZone := *zone - newZone.LeasePreferences = []zonepb.LeasePreference{ + desc, conf := leaseHolderRepl.DescAndSpanConfig() + newConf := conf + newConf.LeasePreferences = []roachpb.LeasePreference{ { - Constraints: []zonepb.Constraint{ + Constraints: []roachpb.Constraint{ { - Type: zonepb.Constraint_REQUIRED, + Type: roachpb.Constraint_REQUIRED, Value: fmt.Sprintf("n%d", remoteNodeID), }, }, @@ -1270,7 +1270,7 @@ func TestTransferLeaseToLaggingNode(t *testing.T) { return err } transferred, err := leaseStore.FindTargetAndTransferLease( - ctx, leaseRepl, desc, &newZone) + ctx, leaseRepl, desc, newConf) if err != nil { return err } diff --git a/pkg/kv/kvserver/reports/reporter.go b/pkg/kv/kvserver/reports/reporter.go index 868462f14504..42b0ea7c9f0f 100644 --- a/pkg/kv/kvserver/reports/reporter.go +++ b/pkg/kv/kvserver/reports/reporter.go @@ -53,6 +53,12 @@ var ReporterInterval = settings.RegisterDurationSetting( // Reporter periodically produces a couple of reports on the cluster's data // distribution: the system tables: replication_constraint_stats, // replication_stats_report and replication_critical_localities. +// +// TODO(irfansharif): After #67679 these replication reports will be the last +// remaining use of the system config span in KV. Strawman: we could hoist all +// this code above KV and run it for each tenant. We'd have to expose a view +// into node liveness and store descriptors, and instead of using the system +// config span we could consult the tenant-scoped system.zones directly. type Reporter struct { // Contains the list of the stores of the current node localStores *kvserver.Stores diff --git a/pkg/kv/kvserver/split_queue.go b/pkg/kv/kvserver/split_queue.go index 5b27ed3988da..63af8a63bef3 100644 --- a/pkg/kv/kvserver/split_queue.go +++ b/pkg/kv/kvserver/split_queue.go @@ -15,12 +15,11 @@ import ( "fmt" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -58,7 +57,7 @@ type splitQueue struct { } // newSplitQueue returns a new instance of splitQueue. -func newSplitQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *splitQueue { +func newSplitQueue(store *Store, db *kv.DB) *splitQueue { var purgChan <-chan time.Time if c := store.TestingKnobs().SplitQueuePurgatoryChan; c != nil { purgChan = c @@ -73,7 +72,7 @@ func newSplitQueue(store *Store, db *kv.DB, gossip *gossip.Gossip) *splitQueue { loadBasedCount: telemetry.GetCounter("kv.split.load"), } sq.baseQueue = newBaseQueue( - "split", sq, store, gossip, + "split", sq, store, queueConfig{ maxSize: defaultQueueMaxSize, maxConcurrency: splitQueueConcurrency, @@ -96,9 +95,9 @@ func shouldSplitRange( ms enginepb.MVCCStats, maxBytes int64, shouldBackpressureWrites bool, - sysCfg *config.SystemConfig, + confReader spanconfig.StoreReader, ) (shouldQ bool, priority float64) { - if sysCfg.NeedsSplit(ctx, desc.StartKey, desc.EndKey) { + if confReader.NeedsSplit(ctx, desc.StartKey, desc.EndKey) { // Set priority to 1 in the event the range is split by zone configs. priority = 1 shouldQ = true @@ -135,10 +134,10 @@ func shouldSplitRange( // prefix or if the range's size in bytes exceeds the limit for the zone, // or if the range has too much load on it. func (sq *splitQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, sysCfg *config.SystemConfig, + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, confReader spanconfig.StoreReader, ) (shouldQ bool, priority float64) { shouldQ, priority = shouldSplitRange(ctx, repl.Desc(), repl.GetMVCCStats(), - repl.GetMaxBytes(), repl.shouldBackpressureWrites(), sysCfg) + repl.GetMaxBytes(), repl.shouldBackpressureWrites(), confReader) if !shouldQ && repl.SplitByLoadEnabled() { if splitKey := repl.loadBasedSplitter.MaybeSplitKey(timeutil.Now()); splitKey != nil { @@ -160,9 +159,9 @@ var _ purgatoryError = unsplittableRangeError{} // process synchronously invokes admin split for each proposed split key. func (sq *splitQueue) process( - ctx context.Context, r *Replica, sysCfg *config.SystemConfig, + ctx context.Context, r *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { - processed, err = sq.processAttempt(ctx, r, sysCfg) + processed, err = sq.processAttempt(ctx, r, confReader) if errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { // ConditionFailedErrors are an expected outcome for range split // attempts because splits can race with other descriptor modifications. @@ -177,11 +176,11 @@ func (sq *splitQueue) process( } func (sq *splitQueue) processAttempt( - ctx context.Context, r *Replica, sysCfg *config.SystemConfig, + ctx context.Context, r *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { desc := r.Desc() - // First handle the case of splitting due to zone config maps. - if splitKey := sysCfg.ComputeSplitKey(ctx, desc.StartKey, desc.EndKey); splitKey != nil { + // First handle the case of splitting due to span config maps. + if splitKey := confReader.ComputeSplitKey(ctx, desc.StartKey, desc.EndKey); splitKey != nil { if _, err := r.adminSplitWithDescriptor( ctx, roachpb.AdminSplitRequest{ @@ -193,7 +192,7 @@ func (sq *splitQueue) processAttempt( }, desc, false, /* delayable */ - "zone config", + "span config", ); err != nil { return false, errors.Wrapf(err, "unable to split %s at key %q", r, splitKey) } diff --git a/pkg/kv/kvserver/split_queue_test.go b/pkg/kv/kvserver/split_queue_test.go index 02386650723e..889bb76f084f 100644 --- a/pkg/kv/kvserver/split_queue_test.go +++ b/pkg/kv/kvserver/split_queue_test.go @@ -89,9 +89,9 @@ func TestSplitQueueShouldQueue(t *testing.T) { repl.mu.Lock() repl.mu.state.Stats = &enginepb.MVCCStats{KeyBytes: test.bytes} repl.mu.Unlock() - zoneConfig := zonepb.DefaultZoneConfig() - zoneConfig.RangeMaxBytes = proto.Int64(test.maxBytes) - repl.SetZoneConfig(&zoneConfig) + conf := TestingDefaultSpanConfig() + conf.RangeMaxBytes = test.maxBytes + repl.SetSpanConfig(conf) // Testing using shouldSplitRange instead of shouldQueue to avoid using the splitFinder // This tests the merge queue behavior too as a result. For splitFinder tests, diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 009defe0f4b8..2e146af1b171 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -53,6 +53,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -200,8 +201,8 @@ func TestStoreConfig(clock *hlc.Clock) StoreConfig { } st := cluster.MakeTestingClusterSettings() sc := StoreConfig{ - DefaultZoneConfig: zonepb.DefaultZoneConfigRef(), - DefaultSystemZoneConfig: zonepb.DefaultSystemZoneConfigRef(), + DefaultSpanConfig: zonepb.DefaultZoneConfigRef().AsSpanConfig(), + DefaultSystemSpanConfig: zonepb.DefaultSystemZoneConfigRef().AsSpanConfig(), Settings: st, AmbientCtx: log.AmbientContext{Tracer: st.Tracer}, Clock: clock, @@ -635,8 +636,8 @@ type StoreConfig struct { AmbientCtx log.AmbientContext base.RaftConfig - DefaultZoneConfig *zonepb.ZoneConfig - DefaultSystemZoneConfig *zonepb.ZoneConfig + DefaultSpanConfig roachpb.SpanConfig + DefaultSystemSpanConfig roachpb.SpanConfig Settings *cluster.Settings Clock *hlc.Clock DB *kv.DB @@ -924,14 +925,14 @@ func NewStore( s.cfg.AmbientCtx, s.cfg.Clock, cfg.ScanInterval, cfg.ScanMinIdleTime, cfg.ScanMaxIdleTime, newStoreReplicaVisitor(s), ) - s.gcQueue = newGCQueue(s, s.cfg.Gossip) - s.mergeQueue = newMergeQueue(s, s.db, s.cfg.Gossip) - s.splitQueue = newSplitQueue(s, s.db, s.cfg.Gossip) - s.replicateQueue = newReplicateQueue(s, s.cfg.Gossip, s.allocator) - s.replicaGCQueue = newReplicaGCQueue(s, s.db, s.cfg.Gossip) - s.raftLogQueue = newRaftLogQueue(s, s.db, s.cfg.Gossip) - s.raftSnapshotQueue = newRaftSnapshotQueue(s, s.cfg.Gossip) - s.consistencyQueue = newConsistencyQueue(s, s.cfg.Gossip) + s.gcQueue = newGCQueue(s) + s.mergeQueue = newMergeQueue(s, s.db) + s.splitQueue = newSplitQueue(s, s.db) + s.replicateQueue = newReplicateQueue(s, s.allocator) + s.replicaGCQueue = newReplicaGCQueue(s, s.db) + s.raftLogQueue = newRaftLogQueue(s, s.db) + s.raftSnapshotQueue = newRaftSnapshotQueue(s) + s.consistencyQueue = newConsistencyQueue(s) // NOTE: If more queue types are added, please also add them to the list of // queues on the EnqueueRange debug page as defined in // pkg/ui/src/views/reports/containers/enqueueRange/index.tsx @@ -944,7 +945,7 @@ func NewStore( } if tsDS != nil { s.tsMaintenanceQueue = newTimeSeriesMaintenanceQueue( - s, s.db, s.cfg.Gossip, tsDS, + s, s.db, tsDS, ) s.scanner.AddQueues(s.tsMaintenanceQueue) } @@ -1136,12 +1137,12 @@ func (s *Store) SetDraining(drain bool, reporter func(int, redact.SafeString)) { } if needsLeaseTransfer { - desc, zone := r.DescAndZone() + desc, conf := r.DescAndSpanConfig() transferStatus, err := s.replicateQueue.shedLease( ctx, r, desc, - zone, + conf, transferLeaseOptions{}, ) if transferStatus != transferOK { @@ -1734,6 +1735,22 @@ 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) { + if s.cfg.TestingKnobs.MakeSystemConfigSpanUnavailableToQueues { + return nil, errSysCfgUnavailable + } + + sysCfg := s.cfg.Gossip.GetSystemConfig() + if sysCfg == nil { + return nil, errSysCfgUnavailable + } + + return sysCfg, nil +} + // startLeaseRenewer runs an infinite loop in a goroutine which regularly // checks whether the store has any expiration-based leases that should be // proactively renewed and attempts to continue renewing them. @@ -1956,14 +1973,14 @@ func (s *Store) systemGossipUpdate(sysCfg *config.SystemConfig) { shouldQueue := s.systemConfigUpdateQueueRateLimiter.AdmitN(1) newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { key := repl.Desc().StartKey - zone, err := sysCfg.GetZoneConfigForKey(key) + conf, err := sysCfg.GetSpanConfigForKey(ctx, key) if err != nil { if log.V(1) { - log.Infof(context.TODO(), "failed to get zone config for key %s", key) + log.Infof(context.TODO(), "failed to get span config for key %s", key) } - zone = s.cfg.DefaultZoneConfig + conf = s.cfg.DefaultSpanConfig } - repl.SetZoneConfig(zone) + repl.SetSpanConfig(conf) if shouldQueue { s.splitQueue.Async(ctx, "gossip update", true /* wait */, func(ctx context.Context, h queueHelper) { h.MaybeAdd(ctx, repl, now) @@ -2807,10 +2824,11 @@ func (s *Store) ManuallyEnqueue( return nil, nil, errors.Errorf("unknown queue type %q", queueName) } - sysCfg := s.cfg.Gossip.GetSystemConfig() - if sysCfg == nil { - return nil, nil, errors.New("cannot run queue without a valid system config; make sure the cluster " + - "has been initialized and all nodes connected to it") + confReader, err := s.GetConfReader() + if err != nil { + return nil, nil, errors.Wrap(err, + "unable to retrieve conf reader, cannot run queue; make sure "+ + "the cluster has been initialized and all nodes connected to it") } // Many queues are only meant to be run on leaseholder replicas, so attempt to @@ -2831,7 +2849,7 @@ func (s *Store) ManuallyEnqueue( if !skipShouldQueue { log.Eventf(ctx, "running %s.shouldQueue", queueName) - shouldQueue, priority := queue.shouldQueue(ctx, s.cfg.Clock.NowAsClockTimestamp(), repl, sysCfg) + shouldQueue, priority := queue.shouldQueue(ctx, s.cfg.Clock.NowAsClockTimestamp(), repl, confReader) log.Eventf(ctx, "shouldQueue=%v, priority=%f", shouldQueue, priority) if !shouldQueue { return collect(), nil, nil @@ -2839,7 +2857,7 @@ func (s *Store) ManuallyEnqueue( } log.Eventf(ctx, "running %s.process", queueName) - processed, processErr := queue.process(ctx, repl, sysCfg) + processed, processErr := queue.process(ctx, repl, confReader) log.Eventf(ctx, "processed: %t (err: %v)", processed, processErr) return collect(), processErr, nil } @@ -2960,3 +2978,8 @@ func min(a, b int) int { } return b } + +// TestingDefaultSpanConfig exposes the default span config for testing purposes. +func TestingDefaultSpanConfig() roachpb.SpanConfig { + return zonepb.DefaultZoneConfigRef().AsSpanConfig() +} diff --git a/pkg/kv/kvserver/store_pool.go b/pkg/kv/kvserver/store_pool.go index 55bcfdf3a1cf..11fa84324420 100644 --- a/pkg/kv/kvserver/store_pool.go +++ b/pkg/kv/kvserver/store_pool.go @@ -17,7 +17,6 @@ import ( "sort" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -786,7 +785,7 @@ func (sl StoreList) String() string { // filter takes a store list and filters it using the passed in constraints. It // maintains the original order of the passed in store list. -func (sl StoreList) filter(constraints []zonepb.ConstraintsConjunction) StoreList { +func (sl StoreList) filter(constraints []roachpb.ConstraintsConjunction) StoreList { if len(constraints) == 0 { return sl } diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/store_pool_test.go index 0f1a55662a9c..4e8355142df1 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/store_pool_test.go @@ -156,7 +156,7 @@ func TestStorePoolGossipUpdate(t *testing.T) { // verifyStoreList ensures that the returned list of stores is correct. func verifyStoreList( sp *StorePool, - constraints []zonepb.ConstraintsConjunction, + constraints []roachpb.ConstraintsConjunction, storeIDs roachpb.StoreIDSlice, // optional filter storeFilter, expected []int, @@ -205,11 +205,11 @@ func TestStorePoolGetStoreList(t *testing.T) { livenesspb.NodeLivenessStatus_DEAD) defer stopper.Stop(context.Background()) sg := gossiputil.NewStoreGossiper(g) - constraints := []zonepb.ConstraintsConjunction{ + constraints := []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Type: zonepb.Constraint_REQUIRED, Value: "ssd"}, - {Type: zonepb.Constraint_REQUIRED, Value: "dc"}, + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_REQUIRED, Value: "ssd"}, + {Type: roachpb.Constraint_REQUIRED, Value: "dc"}, }, }, } @@ -403,13 +403,12 @@ func TestStoreListFilter(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - constraints := []zonepb.ConstraintsConjunction{ + constraints := []roachpb.ConstraintsConjunction{ { - Constraints: []zonepb.Constraint{ - {Type: zonepb.Constraint_REQUIRED, Key: "region", Value: "us-west"}, - {Type: zonepb.Constraint_REQUIRED, Value: "MustMatch"}, - {Type: zonepb.Constraint_DEPRECATED_POSITIVE, Value: "MatchingOptional"}, - {Type: zonepb.Constraint_PROHIBITED, Value: "MustNotMatch"}, + Constraints: []roachpb.Constraint{ + {Type: roachpb.Constraint_REQUIRED, Key: "region", Value: "us-west"}, + {Type: roachpb.Constraint_REQUIRED, Value: "MustMatch"}, + {Type: roachpb.Constraint_PROHIBITED, Value: "MustNotMatch"}, }, }, } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index cb4964f2ca0d..350c34351676 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -18,7 +18,6 @@ import ( "sort" "time" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -403,7 +402,7 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( continue } - desc, zone := replWithStats.repl.DescAndZone() + desc, conf := replWithStats.repl.DescAndSpanConfig() log.VEventf(ctx, 3, "considering lease transfer for r%d with %.2f qps", desc.RangeID, replWithStats.qps) @@ -424,7 +423,7 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( var raftStatus *raft.Status - preferred := sr.rq.allocator.preferredLeaseholders(zone, candidates) + preferred := sr.rq.allocator.preferredLeaseholders(conf, candidates) for _, candidate := range candidates { if candidate.StoreID == localDesc.StoreID { continue @@ -450,8 +449,8 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( continue } - filteredStoreList := storeList.filter(zone.Constraints) - filteredStoreList = storeList.filter(zone.VoterConstraints) + filteredStoreList := storeList.filter(conf.Constraints) + filteredStoreList = storeList.filter(conf.VoterConstraints) if sr.rq.allocator.followTheWorkloadPrefersLocal( ctx, filteredStoreList, @@ -479,7 +478,7 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( type rangeRebalanceContext struct { replWithStats replicaWithStats rangeDesc *roachpb.RangeDescriptor - zone *zonepb.ZoneConfig + conf roachpb.SpanConfig clusterNodes int numDesiredVoters, numDesiredNonVoters int } @@ -540,10 +539,10 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( log.VEventf(ctx, 3, "considering replica rebalance for r%d with %.2f qps", replWithStats.repl.GetRangeID(), replWithStats.qps) - rangeDesc, zone := replWithStats.repl.DescAndZone() + rangeDesc, conf := replWithStats.repl.DescAndSpanConfig() clusterNodes := sr.rq.allocator.storePool.ClusterNodeCount() - numDesiredVoters := GetNeededVoters(zone.GetNumVoters(), clusterNodes) - numDesiredNonVoters := GetNeededNonVoters(numDesiredVoters, int(zone.GetNumNonVoters()), clusterNodes) + numDesiredVoters := GetNeededVoters(conf.GetNumVoters(), clusterNodes) + numDesiredNonVoters := GetNeededNonVoters(numDesiredVoters, int(conf.GetNumNonVoters()), clusterNodes) if rs := rangeDesc.Replicas(); numDesiredVoters != len(rs.VoterDescriptors()) || numDesiredNonVoters != len(rs.NonVoterDescriptors()) { // If the StoreRebalancer is allowed past this point, it may accidentally @@ -557,7 +556,7 @@ func (sr *StoreRebalancer) chooseRangeToRebalance( rebalanceCtx := rangeRebalanceContext{ replWithStats: replWithStats, rangeDesc: rangeDesc, - zone: zone, + conf: conf, clusterNodes: clusterNodes, numDesiredVoters: numDesiredVoters, numDesiredNonVoters: numDesiredNonVoters, @@ -782,7 +781,7 @@ func (sr *StoreRebalancer) pickRemainingRepls( target, _ := sr.rq.allocator.allocateTargetFromList( ctx, storeList, - rebalanceCtx.zone, + rebalanceCtx.conf, partialVoterTargets, partialNonVoterTargets, options, diff --git a/pkg/kv/kvserver/store_rebalancer_test.go b/pkg/kv/kvserver/store_rebalancer_test.go index d9c3b82f32eb..98fbe8f5d08f 100644 --- a/pkg/kv/kvserver/store_rebalancer_test.go +++ b/pkg/kv/kvserver/store_rebalancer_test.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/gogo/protobuf/proto" "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft/v3" "go.etcd.io/etcd/raft/v3/tracker" @@ -82,7 +81,7 @@ func loadRanges(rr *replicaRankings, s *Store, ranges []testRange) { for _, r := range ranges { repl := &Replica{store: s} repl.mu.state.Desc = &roachpb.RangeDescriptor{} - repl.mu.zone = s.cfg.DefaultZoneConfig + repl.mu.conf = s.cfg.DefaultSpanConfig for _, storeID := range r.voters { repl.mu.state.Desc.InternalReplicas = append(repl.mu.state.Desc.InternalReplicas, roachpb.ReplicaDescriptor{ NodeID: roachpb.NodeID(storeID), @@ -135,9 +134,10 @@ func TestChooseLeaseToTransfer(t *testing.T) { localDesc := *noLocalityStores[0] cfg := TestStoreConfig(nil) + cfg.Gossip = g s := createTestStoreWithoutStart(t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) s.Ident = &roachpb.StoreIdent{StoreID: localDesc.StoreID} - rq := newReplicateQueue(s, g, a) + rq := newReplicateQueue(s, a) rr := newReplicaRankings() sr := NewStoreRebalancer(cfg.AmbientCtx, cfg.Settings, rq, rr) @@ -219,9 +219,10 @@ func TestChooseRangeToRebalance(t *testing.T) { localDesc := *noLocalityStores[0] cfg := TestStoreConfig(nil) + cfg.Gossip = g s := createTestStoreWithoutStart(t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) s.Ident = &roachpb.StoreIdent{StoreID: localDesc.StoreID} - rq := newReplicateQueue(s, g, a) + rq := newReplicateQueue(s, a) rr := newReplicaRankings() sr := NewStoreRebalancer(cfg.AmbientCtx, cfg.Settings, rq, rr) @@ -525,8 +526,8 @@ func TestChooseRangeToRebalance(t *testing.T) { return true } - s.cfg.DefaultZoneConfig.NumVoters = proto.Int32(int32(len(tc.voters))) - s.cfg.DefaultZoneConfig.NumReplicas = proto.Int32(int32(len(tc.voters) + len(tc.nonVoters))) + s.cfg.DefaultSpanConfig.NumVoters = int32(len(tc.voters)) + s.cfg.DefaultSpanConfig.NumReplicas = int32(len(tc.voters) + len(tc.nonVoters)) loadRanges( rr, s, []testRange{ {voters: tc.voters, nonVoters: tc.nonVoters, qps: tc.qps}, @@ -580,9 +581,10 @@ func TestNoLeaseTransferToBehindReplicas(t *testing.T) { localDesc := *noLocalityStores[0] cfg := TestStoreConfig(nil) + cfg.Gossip = g s := createTestStoreWithoutStart(t, stopper, testStoreOpts{createSystemRanges: true}, &cfg) s.Ident = &roachpb.StoreIdent{StoreID: localDesc.StoreID} - rq := newReplicateQueue(s, g, a) + rq := newReplicateQueue(s, a) rr := newReplicaRankings() sr := NewStoreRebalancer(cfg.AmbientCtx, cfg.Settings, rq, rr) diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index ed03315fc134..22fc471b2c21 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -341,7 +341,7 @@ func (s *Store) SplitRange( // Update the replica's cached byte thresholds. This is a no-op if the system // config is not available, in which case we rely on the next gossip update // to perform the update. - if err := rightRepl.updateRangeInfo(rightRepl.Desc()); err != nil { + if err := rightRepl.updateRangeInfo(ctx, rightRepl.Desc()); err != nil { return err } // Add the range to metrics and maybe gossip on capacity change. diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index d54369e574f1..12ca1da9e5ed 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -216,7 +216,7 @@ func createTestStoreWithoutStart( Settings: cfg.Settings, }) server := rpc.NewServer(rpcContext) // never started - cfg.Gossip = gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), cfg.DefaultZoneConfig) + cfg.Gossip = gossip.NewTest(1, rpcContext, server, stopper, metric.NewRegistry(), zonepb.DefaultZoneConfigRef()) cfg.StorePool = NewTestStorePool(*cfg) // Many tests using this test harness (as opposed to higher-level // ones like multiTestContext or TestServer) want to micro-manage @@ -245,7 +245,7 @@ func createTestStoreWithoutStart( } var splits []roachpb.RKey kvs, tableSplits := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig, + keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), ).GetInitialValues() if opts.createSystemRanges { splits = config.StaticSplits() @@ -463,7 +463,7 @@ func TestStoreInitAndBootstrap(t *testing.T) { // Bootstrap the system ranges. var splits []roachpb.RKey kvs, tableSplits := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig, + keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), ).GetInitialValues() splits = config.StaticSplits() splits = append(splits, tableSplits...) @@ -1356,22 +1356,23 @@ func TestStoreSetRangesMaxBytes(t *testing.T) { expMaxBytes int64 }{ {store.LookupReplica(roachpb.RKeyMin), - *store.cfg.DefaultZoneConfig.RangeMaxBytes}, + store.cfg.DefaultSpanConfig.RangeMaxBytes}, {splitTestRange(store, roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID)), t), 1 << 20}, {splitTestRange(store, roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID+1)), t), - *store.cfg.DefaultZoneConfig.RangeMaxBytes}, + store.cfg.DefaultSpanConfig.RangeMaxBytes}, {splitTestRange(store, roachpb.RKey(keys.SystemSQLCodec.TablePrefix(baseID+2)), t), 2 << 20}, } // Set zone configs. - config.TestingSetZoneConfig( - config.SystemTenantObjectID(baseID), zonepb.ZoneConfig{RangeMaxBytes: proto.Int64(1 << 20)}, - ) - config.TestingSetZoneConfig( - config.SystemTenantObjectID(baseID+2), zonepb.ZoneConfig{RangeMaxBytes: proto.Int64(2 << 20)}, - ) + zoneA := zonepb.DefaultZoneConfig() + zoneA.RangeMaxBytes = proto.Int64(1 << 20) + config.TestingSetZoneConfig(config.SystemTenantObjectID(baseID), zoneA) + + zoneB := zonepb.DefaultZoneConfig() + zoneB.RangeMaxBytes = proto.Int64(2 << 20) + config.TestingSetZoneConfig(config.SystemTenantObjectID(baseID+2), zoneB) // Despite faking the zone configs, we still need to have a system config // entry so that the store picks up the new zone configs. This new system diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 4466034deb86..73443190424e 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -354,6 +354,10 @@ type StoreTestingKnobs struct { // LeaseRenewalDurationOverride replaces the timer duration for proactively // renewing expiration based leases. LeaseRenewalDurationOverride time.Duration + + // MakeSystemConfigSpanUnavailableToQueues makes the system config span + // unavailable to queues that ask for it. + MakeSystemConfigSpanUnavailableToQueues bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/kv/kvserver/ts_maintenance_queue.go b/pkg/kv/kvserver/ts_maintenance_queue.go index 8fa102091903..f0476e1895f4 100644 --- a/pkg/kv/kvserver/ts_maintenance_queue.go +++ b/pkg/kv/kvserver/ts_maintenance_queue.go @@ -14,10 +14,9 @@ import ( "context" "time" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -88,7 +87,7 @@ type timeSeriesMaintenanceQueue struct { // newTimeSeriesMaintenanceQueue returns a new instance of // timeSeriesMaintenanceQueue. func newTimeSeriesMaintenanceQueue( - store *Store, db *kv.DB, g *gossip.Gossip, tsData TimeSeriesDataStore, + store *Store, db *kv.DB, tsData TimeSeriesDataStore, ) *timeSeriesMaintenanceQueue { q := &timeSeriesMaintenanceQueue{ tsData: tsData, @@ -107,7 +106,7 @@ func newTimeSeriesMaintenanceQueue( ), } q.baseQueue = newBaseQueue( - "timeSeriesMaintenance", q, store, g, + "timeSeriesMaintenance", q, store, queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, @@ -124,7 +123,7 @@ func newTimeSeriesMaintenanceQueue( } func (q *timeSeriesMaintenanceQueue) shouldQueue( - ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ *config.SystemConfig, + ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, ) (shouldQ bool, priority float64) { if !repl.store.cfg.TestingKnobs.DisableLastProcessedCheck { lpTS, err := repl.getQueueLastProcessed(ctx, q.name) @@ -144,7 +143,7 @@ func (q *timeSeriesMaintenanceQueue) shouldQueue( } func (q *timeSeriesMaintenanceQueue) process( - ctx context.Context, repl *Replica, _ *config.SystemConfig, + ctx context.Context, repl *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { desc := repl.Desc() snap := repl.store.Engine().NewSnapshot() diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 74c6227d8a11..0ef0fee92502 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -75,6 +75,7 @@ go_library( "metadata.go", "metadata_replicas.go", "method.go", + "span_config.go", "version.go", ], embed = [":roachpb_go_proto"], diff --git a/pkg/roachpb/span_config.go b/pkg/roachpb/span_config.go index c506586314c4..4ab884bc7fd0 100644 --- a/pkg/roachpb/span_config.go +++ b/pkg/roachpb/span_config.go @@ -10,7 +10,91 @@ package roachpb +import ( + "fmt" + "strings" + "time" +) + // Equal compares two span config entries. func (e SpanConfigEntry) Equal(other SpanConfigEntry) bool { return e.Span.Equal(other.Span) && e.Config.Equal(other.Config) } + +// StoreMatchesConstraint returns whether a store's attributes or node's +// locality match the constraint's spec. It notably ignores whether the +// constraint is required, prohibited, positive, or otherwise. +func StoreMatchesConstraint(store StoreDescriptor, c Constraint) bool { + if c.Key == "" { + for _, attrs := range []Attributes{store.Attrs, store.Node.Attrs} { + for _, attr := range attrs.Attrs { + if attr == c.Value { + return true + } + } + } + return false + } + for _, tier := range store.Node.Locality.Tiers { + if c.Key == tier.Key && c.Value == tier.Value { + return true + } + } + return false +} + +var emptySpanConfig = &SpanConfig{} + +// IsEmpty returns true if s is an empty SpanConfig. +func (s *SpanConfig) IsEmpty() bool { + return s.Equal(emptySpanConfig) +} + +// TTL returns the implies TTL as a time.Duration. +func (s *SpanConfig) TTL() time.Duration { + return time.Duration(s.GCPolicy.TTLSeconds) * time.Second +} + +// GetNumVoters returns the number of voting replicas as defined in the +// span config. +func (s *SpanConfig) GetNumVoters() int32 { + if s.NumVoters != 0 { + return s.NumVoters + } + return s.NumReplicas +} + +// GetNumNonVoters returns the number of non-voting replicas as defined in the +// span config. +func (s *SpanConfig) GetNumNonVoters() int32 { + return s.NumReplicas - s.GetNumVoters() +} + +func (c Constraint) String() string { + var str string + switch c.Type { + case Constraint_REQUIRED: + str += "+" + case Constraint_PROHIBITED: + str += "-" + } + if len(c.Key) > 0 { + str += c.Key + "=" + } + str += c.Value + return str +} + +func (c ConstraintsConjunction) String() string { + var sb strings.Builder + for i, cons := range c.Constraints { + if i > 0 { + sb.WriteRune(',') + } + sb.WriteString(cons.String()) + } + if c.NumReplicas != 0 { + fmt.Fprintf(&sb, ":%d", c.NumReplicas) + } + return sb.String() +} diff --git a/pkg/roachpb/span_config.pb.go b/pkg/roachpb/span_config.pb.go index 159b3ec02002..e8006bccd3a6 100644 --- a/pkg/roachpb/span_config.pb.go +++ b/pkg/roachpb/span_config.pb.go @@ -50,9 +50,47 @@ func (x Constraint_Type) String() string { } func (Constraint_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{0, 0} + return fileDescriptor_91c9f1dcea14470a, []int{1, 0} } +// GCPolicy dictates the garbage collection policy to apply over a given span. +// It parallels the definition found in zonepb/zone.proto. +type GCPolicy struct { + // TTLSeconds is the number of seconds overwritten values will be retained + // before garbage collection. A value <= 0 means older versions are never + // GC-ed. + TTLSeconds int32 `protobuf:"varint,1,opt,name=ttl_seconds,json=ttlSeconds,proto3" json:"ttl_seconds,omitempty"` +} + +func (m *GCPolicy) Reset() { *m = GCPolicy{} } +func (m *GCPolicy) String() string { return proto.CompactTextString(m) } +func (*GCPolicy) ProtoMessage() {} +func (*GCPolicy) Descriptor() ([]byte, []int) { + return fileDescriptor_91c9f1dcea14470a, []int{0} +} +func (m *GCPolicy) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCPolicy) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *GCPolicy) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCPolicy.Merge(m, src) +} +func (m *GCPolicy) XXX_Size() int { + return m.Size() +} +func (m *GCPolicy) XXX_DiscardUnknown() { + xxx_messageInfo_GCPolicy.DiscardUnknown(m) +} + +var xxx_messageInfo_GCPolicy proto.InternalMessageInfo + // Constraint constrains the stores that a replica can be stored on. It // parallels the definition found in zonepb/zone.proto. type Constraint struct { @@ -64,11 +102,10 @@ type Constraint struct { Value string `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` } -func (m *Constraint) Reset() { *m = Constraint{} } -func (m *Constraint) String() string { return proto.CompactTextString(m) } -func (*Constraint) ProtoMessage() {} +func (m *Constraint) Reset() { *m = Constraint{} } +func (*Constraint) ProtoMessage() {} func (*Constraint) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{0} + return fileDescriptor_91c9f1dcea14470a, []int{1} } func (m *Constraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -108,11 +145,10 @@ type ConstraintsConjunction struct { Constraints []Constraint `protobuf:"bytes,2,rep,name=constraints,proto3" json:"constraints"` } -func (m *ConstraintsConjunction) Reset() { *m = ConstraintsConjunction{} } -func (m *ConstraintsConjunction) String() string { return proto.CompactTextString(m) } -func (*ConstraintsConjunction) ProtoMessage() {} +func (m *ConstraintsConjunction) Reset() { *m = ConstraintsConjunction{} } +func (*ConstraintsConjunction) ProtoMessage() {} func (*ConstraintsConjunction) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{1} + return fileDescriptor_91c9f1dcea14470a, []int{2} } func (m *ConstraintsConjunction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -149,7 +185,7 @@ func (m *LeasePreference) Reset() { *m = LeasePreference{} } func (m *LeasePreference) String() string { return proto.CompactTextString(m) } func (*LeasePreference) ProtoMessage() {} func (*LeasePreference) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{2} + return fileDescriptor_91c9f1dcea14470a, []int{3} } func (m *LeasePreference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -177,16 +213,14 @@ var xxx_messageInfo_LeasePreference proto.InternalMessageInfo // SpanConfig holds the configuration that applies to a given keyspan. It // parallels the definition found in zonepb/zone.proto. type SpanConfig struct { - // RangeMinBytes is the minimum size, in bytes, for a range in the given - // keyspan. When a range is less than this size, it'll be merged with an - // adjacent range. + // RangeMinBytes is the minimum size, in bytes, a range can have. When a range + // is less than this size, it'll be merged with an adjacent range. RangeMinBytes int64 `protobuf:"varint,1,opt,name=range_min_bytes,json=rangeMinBytes,proto3" json:"range_min_bytes,omitempty"` - // RangeMaxBytes is the maximum size, in bytes, for a range in the given - // keyspan. When a range is more than this size, it'll split into two ranges. + // RangeMaxBytes is the maximum size, in bytes, a range can have. When a range + // is more than this size, it'll split into two ranges. RangeMaxBytes int64 `protobuf:"varint,2,opt,name=range_max_bytes,json=rangeMaxBytes,proto3" json:"range_max_bytes,omitempty"` - // GCTTL is the number of seconds overwritten values will be retained before - // garbage collection. A value <= 0 means older versions are never GC-ed. - GCTTL int32 `protobuf:"varint,3,opt,name=gc_ttl,json=gcTtl,proto3" json:"gc_ttl,omitempty"` + // GCPolicy dictates the the GC policy to apply over a range. + GCPolicy GCPolicy `protobuf:"bytes,3,opt,name=gc_policy,json=gcPolicy,proto3" json:"gc_policy"` // GlobalReads specifies whether transactions operating over the range(s) // should be configured to provide non-blocking behavior, meaning that reads // can be served consistently from all replicas and do not block on writes. In @@ -227,7 +261,7 @@ func (m *SpanConfig) Reset() { *m = SpanConfig{} } func (m *SpanConfig) String() string { return proto.CompactTextString(m) } func (*SpanConfig) ProtoMessage() {} func (*SpanConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{3} + return fileDescriptor_91c9f1dcea14470a, []int{4} } func (m *SpanConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -264,7 +298,7 @@ func (m *SpanConfigEntry) Reset() { *m = SpanConfigEntry{} } func (m *SpanConfigEntry) String() string { return proto.CompactTextString(m) } func (*SpanConfigEntry) ProtoMessage() {} func (*SpanConfigEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{4} + return fileDescriptor_91c9f1dcea14470a, []int{5} } func (m *SpanConfigEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -301,7 +335,7 @@ func (m *GetSpanConfigsRequest) Reset() { *m = GetSpanConfigsRequest{} } func (m *GetSpanConfigsRequest) String() string { return proto.CompactTextString(m) } func (*GetSpanConfigsRequest) ProtoMessage() {} func (*GetSpanConfigsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{5} + return fileDescriptor_91c9f1dcea14470a, []int{6} } func (m *GetSpanConfigsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -341,7 +375,7 @@ func (m *GetSpanConfigsResponse) Reset() { *m = GetSpanConfigsResponse{} func (m *GetSpanConfigsResponse) String() string { return proto.CompactTextString(m) } func (*GetSpanConfigsResponse) ProtoMessage() {} func (*GetSpanConfigsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{6} + return fileDescriptor_91c9f1dcea14470a, []int{7} } func (m *GetSpanConfigsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -393,7 +427,7 @@ func (m *UpdateSpanConfigsRequest) Reset() { *m = UpdateSpanConfigsReque func (m *UpdateSpanConfigsRequest) String() string { return proto.CompactTextString(m) } func (*UpdateSpanConfigsRequest) ProtoMessage() {} func (*UpdateSpanConfigsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{7} + return fileDescriptor_91c9f1dcea14470a, []int{8} } func (m *UpdateSpanConfigsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -425,7 +459,7 @@ func (m *UpdateSpanConfigsResponse) Reset() { *m = UpdateSpanConfigsResp func (m *UpdateSpanConfigsResponse) String() string { return proto.CompactTextString(m) } func (*UpdateSpanConfigsResponse) ProtoMessage() {} func (*UpdateSpanConfigsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_91c9f1dcea14470a, []int{8} + return fileDescriptor_91c9f1dcea14470a, []int{9} } func (m *UpdateSpanConfigsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -452,6 +486,7 @@ var xxx_messageInfo_UpdateSpanConfigsResponse proto.InternalMessageInfo func init() { proto.RegisterEnum("cockroach.roachpb.Constraint_Type", Constraint_Type_name, Constraint_Type_value) + proto.RegisterType((*GCPolicy)(nil), "cockroach.roachpb.GCPolicy") proto.RegisterType((*Constraint)(nil), "cockroach.roachpb.Constraint") proto.RegisterType((*ConstraintsConjunction)(nil), "cockroach.roachpb.ConstraintsConjunction") proto.RegisterType((*LeasePreference)(nil), "cockroach.roachpb.LeasePreference") @@ -466,53 +501,80 @@ func init() { func init() { proto.RegisterFile("roachpb/span_config.proto", fileDescriptor_91c9f1dcea14470a) } var fileDescriptor_91c9f1dcea14470a = []byte{ - // 691 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0xcf, 0x4f, 0x13, 0x4d, - 0x18, 0xee, 0xd0, 0x1f, 0xb4, 0x6f, 0xf9, 0xa0, 0x9d, 0x8f, 0x8f, 0x6f, 0xc1, 0xb0, 0xd4, 0x8d, - 0x31, 0xe5, 0x52, 0x22, 0x24, 0x1e, 0xf0, 0xd6, 0xd2, 0x20, 0x09, 0x46, 0x58, 0x5b, 0x63, 0x8c, - 0x71, 0x33, 0xdd, 0x0e, 0x75, 0x65, 0x3b, 0xb3, 0xee, 0xcc, 0x12, 0x7a, 0xf4, 0xe6, 0xd1, 0xa3, - 0x17, 0x13, 0xff, 0x1c, 0x8e, 0x1c, 0x39, 0x11, 0x2d, 0x17, 0xaf, 0xfe, 0x07, 0x66, 0x67, 0xb7, - 0x76, 0x81, 0x06, 0x89, 0xde, 0x66, 0x9f, 0x7d, 0xde, 0xe7, 0x7d, 0xe7, 0x79, 0x66, 0x06, 0x16, - 0x7d, 0x4e, 0xec, 0x37, 0x5e, 0x67, 0x4d, 0x78, 0x84, 0x59, 0x36, 0x67, 0x07, 0x4e, 0xaf, 0xe6, - 0xf9, 0x5c, 0x72, 0x5c, 0xb6, 0xb9, 0x7d, 0xa8, 0x7e, 0xd7, 0x62, 0xd2, 0x12, 0x1e, 0xb1, 0xbb, - 0x44, 0x92, 0x88, 0xb6, 0x34, 0xdf, 0xe3, 0x3d, 0xae, 0x96, 0x6b, 0xe1, 0x2a, 0x42, 0x8d, 0x4f, - 0x08, 0xa0, 0xc1, 0x99, 0x90, 0x3e, 0x71, 0x98, 0xc4, 0x0f, 0x21, 0x23, 0x07, 0x1e, 0xd5, 0x50, - 0x05, 0x55, 0x67, 0xd7, 0x8d, 0xda, 0x35, 0xe9, 0xda, 0x98, 0x5c, 0x6b, 0x0d, 0x3c, 0x6a, 0x2a, - 0x3e, 0x2e, 0x41, 0xfa, 0x90, 0x0e, 0xb4, 0xa9, 0x0a, 0xaa, 0x16, 0xcc, 0x70, 0x89, 0xe7, 0x21, - 0x7b, 0x44, 0xdc, 0x80, 0x6a, 0x69, 0x85, 0x45, 0x1f, 0xc6, 0x3d, 0xc8, 0x84, 0x55, 0x78, 0x06, - 0xf2, 0x66, 0x73, 0xbf, 0xbd, 0x63, 0x36, 0xb7, 0x4a, 0x29, 0x3c, 0x0b, 0xb0, 0x67, 0x3e, 0x7d, - 0xbc, 0x53, 0xdf, 0x69, 0x35, 0xb7, 0x4a, 0x68, 0x33, 0xf3, 0xfd, 0xcb, 0x0a, 0x32, 0x3e, 0x20, - 0x58, 0x18, 0x77, 0x13, 0x0d, 0xce, 0xde, 0x06, 0xcc, 0x96, 0x0e, 0x67, 0xf8, 0x2e, 0xcc, 0xb0, - 0xa0, 0x6f, 0xf9, 0xd4, 0x73, 0x1d, 0x9b, 0x08, 0x35, 0x6e, 0xd6, 0x2c, 0xb2, 0xa0, 0x6f, 0xc6, - 0x10, 0x6e, 0x42, 0xd1, 0x1e, 0x17, 0x6b, 0x53, 0x95, 0x74, 0xb5, 0xb8, 0xbe, 0x7c, 0xe3, 0x86, - 0xea, 0x99, 0x93, 0xf3, 0x95, 0x94, 0x99, 0xac, 0x8b, 0x47, 0x79, 0x0d, 0x73, 0xbb, 0x94, 0x08, - 0xba, 0xe7, 0xd3, 0x03, 0xea, 0x53, 0x66, 0xd3, 0xab, 0xfa, 0xe8, 0xaf, 0xf4, 0x7f, 0xa4, 0x01, - 0x9e, 0x79, 0x84, 0x35, 0x54, 0xae, 0xf8, 0x3e, 0xcc, 0xf9, 0x84, 0xf5, 0xa8, 0xd5, 0x77, 0x98, - 0xd5, 0x19, 0x48, 0x1a, 0xed, 0x30, 0x6d, 0xfe, 0xa3, 0xe0, 0x27, 0x0e, 0xab, 0x87, 0x60, 0x82, - 0x47, 0x8e, 0x63, 0xde, 0x54, 0x92, 0x47, 0x8e, 0x23, 0x5e, 0x05, 0x72, 0x3d, 0xdb, 0x92, 0xd2, - 0x55, 0x61, 0x64, 0xeb, 0x85, 0xe1, 0xf9, 0x4a, 0x76, 0xbb, 0xd1, 0x6a, 0xed, 0x9a, 0xd9, 0x9e, - 0xdd, 0x92, 0x6e, 0x68, 0x68, 0xcf, 0xe5, 0x1d, 0xe2, 0x5a, 0x3e, 0x25, 0x5d, 0xa1, 0x65, 0x2a, - 0xa8, 0x9a, 0x37, 0x8b, 0x11, 0x66, 0x86, 0xd0, 0x35, 0xcf, 0xb3, 0xd7, 0x3d, 0x5f, 0x06, 0x08, - 0x29, 0x47, 0x5c, 0x52, 0x5f, 0x68, 0x39, 0x45, 0x28, 0xb0, 0xa0, 0xff, 0x5c, 0x01, 0x78, 0xff, - 0xb2, 0x65, 0xd3, 0xca, 0xb2, 0xd5, 0x1b, 0x2d, 0x4b, 0xa6, 0x3e, 0xc1, 0x3e, 0xfc, 0x0a, 0xca, - 0xaa, 0x9b, 0x95, 0x14, 0xce, 0xff, 0x99, 0x70, 0x49, 0x29, 0x25, 0x28, 0xb8, 0x0d, 0x65, 0x37, - 0x8c, 0xdd, 0xf2, 0x7e, 0xe5, 0x2e, 0xb4, 0x82, 0x52, 0x9f, 0x74, 0x35, 0xae, 0x1c, 0x91, 0x91, - 0xac, 0x7b, 0x19, 0x1e, 0x65, 0xfe, 0x1e, 0xc1, 0xdc, 0x38, 0xf3, 0x26, 0x93, 0xfe, 0x00, 0x3f, - 0x80, 0x4c, 0x78, 0xbf, 0x55, 0xda, 0xc5, 0xf5, 0xff, 0x27, 0xf4, 0x08, 0x2b, 0x62, 0x61, 0x45, - 0xc5, 0x8f, 0x20, 0x17, 0xbd, 0x06, 0x2a, 0xfa, 0xc9, 0x47, 0x70, 0xdc, 0x26, 0x2e, 0x8d, 0x4b, - 0x8c, 0x5d, 0xf8, 0x6f, 0x9b, 0xca, 0xf1, 0x6f, 0x61, 0xd2, 0x77, 0x01, 0x15, 0x12, 0x6f, 0x40, - 0x36, 0x54, 0x1f, 0x9d, 0xeb, 0xdf, 0x4c, 0x12, 0x71, 0x0d, 0x1f, 0x16, 0xae, 0xaa, 0x09, 0x8f, - 0x33, 0x41, 0xf1, 0x0b, 0xf8, 0x37, 0xf1, 0x6e, 0x59, 0x94, 0x49, 0xdf, 0xa1, 0x23, 0x71, 0xe3, - 0xc6, 0x89, 0x95, 0x31, 0x71, 0x9f, 0xb2, 0xb8, 0x04, 0x3b, 0x54, 0x18, 0x9f, 0x11, 0x68, 0x6d, - 0xaf, 0x4b, 0x24, 0x9d, 0xb0, 0x8b, 0x4d, 0x28, 0x48, 0x6e, 0x75, 0xa9, 0x4b, 0x25, 0xbd, 0xdd, - 0x4e, 0xf2, 0x92, 0x6f, 0x29, 0x3a, 0x6e, 0xaa, 0xda, 0xc0, 0x13, 0xd4, 0x97, 0xf1, 0xeb, 0x71, - 0xfb, 0x41, 0xf3, 0x92, 0xb7, 0x55, 0xa5, 0x71, 0x07, 0x16, 0x27, 0x8c, 0x17, 0xd9, 0x52, 0x5f, - 0x3d, 0xf9, 0xa6, 0xa7, 0x4e, 0x86, 0x3a, 0x3a, 0x1d, 0xea, 0xe8, 0x6c, 0xa8, 0xa3, 0xaf, 0x43, - 0x1d, 0x7d, 0xbc, 0xd0, 0x53, 0xa7, 0x17, 0x7a, 0xea, 0xec, 0x42, 0x4f, 0xbd, 0x9c, 0x8e, 0xfb, - 0x74, 0x72, 0xea, 0xb9, 0xde, 0xf8, 0x19, 0x00, 0x00, 0xff, 0xff, 0x1a, 0x72, 0xb9, 0x59, 0x08, - 0x06, 0x00, 0x00, -} + // 752 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0x41, 0x4f, 0xdb, 0x48, + 0x14, 0xce, 0x90, 0x04, 0x9c, 0x17, 0x16, 0x92, 0x59, 0x96, 0x35, 0x20, 0x9c, 0x60, 0xad, 0x56, + 0xe1, 0x12, 0xb4, 0x41, 0xda, 0x03, 0xbd, 0x25, 0x44, 0x94, 0x8a, 0xaa, 0x60, 0x92, 0xaa, 0xaa, + 0xaa, 0x5a, 0x8e, 0x33, 0xa4, 0x2e, 0xce, 0x8c, 0xeb, 0x99, 0x20, 0x72, 0xec, 0xbd, 0x87, 0x1e, + 0x2b, 0xb5, 0x95, 0xf8, 0x19, 0xfd, 0x09, 0x1c, 0x39, 0x72, 0x42, 0x6d, 0xb8, 0xf4, 0x67, 0x54, + 0x1e, 0x3b, 0xc4, 0x40, 0x44, 0x11, 0xb7, 0xc9, 0xe7, 0xef, 0x7d, 0xef, 0x7b, 0xcf, 0x9f, 0x27, + 0xb0, 0xe0, 0x33, 0xcb, 0x7e, 0xe3, 0xb5, 0xd6, 0xb8, 0x67, 0x51, 0xd3, 0x66, 0xf4, 0xc0, 0xe9, + 0x94, 0x3d, 0x9f, 0x09, 0x86, 0xf3, 0x36, 0xb3, 0x0f, 0xe5, 0xe3, 0x72, 0x44, 0x5a, 0xc4, 0x43, + 0x76, 0xdb, 0x12, 0x56, 0x48, 0x5b, 0x9c, 0xeb, 0xb0, 0x0e, 0x93, 0xc7, 0xb5, 0xe0, 0x14, 0xa2, + 0x7a, 0x1d, 0x94, 0xad, 0xda, 0x2e, 0x73, 0x1d, 0xbb, 0x8f, 0xd7, 0x20, 0x2b, 0x84, 0x6b, 0x72, + 0x62, 0x33, 0xda, 0xe6, 0x2a, 0x2a, 0xa2, 0x52, 0xba, 0x3a, 0x33, 0xb8, 0x28, 0x40, 0xa3, 0xb1, + 0xb3, 0x1f, 0xa2, 0x06, 0x08, 0xe1, 0x46, 0xe7, 0x0d, 0xe5, 0xdb, 0x49, 0x01, 0xfd, 0x3c, 0x29, + 0x20, 0xfd, 0x0b, 0x02, 0xa8, 0x31, 0xca, 0x85, 0x6f, 0x39, 0x54, 0xe0, 0xff, 0x21, 0x25, 0xfa, + 0x1e, 0x91, 0x12, 0x33, 0x15, 0xbd, 0x7c, 0xcb, 0x61, 0x79, 0x44, 0x2e, 0x37, 0xfa, 0x1e, 0x31, + 0x24, 0x1f, 0xe7, 0x20, 0x79, 0x48, 0xfa, 0xea, 0x44, 0x11, 0x95, 0x32, 0x46, 0x70, 0xc4, 0x73, + 0x90, 0x3e, 0xb2, 0xdc, 0x1e, 0x51, 0x93, 0x12, 0x0b, 0x7f, 0xe8, 0xff, 0x40, 0x2a, 0xa8, 0xc2, + 0xd3, 0xa0, 0x18, 0xf5, 0xbd, 0xe6, 0xb6, 0x51, 0xdf, 0xcc, 0x25, 0xf0, 0x0c, 0xc0, 0xae, 0xf1, + 0xec, 0xf1, 0x76, 0x75, 0xbb, 0x51, 0xdf, 0xcc, 0xa1, 0x0d, 0xe5, 0xd3, 0x49, 0x21, 0x21, 0xed, + 0x7d, 0x40, 0x30, 0x3f, 0xea, 0xc8, 0x6b, 0x8c, 0xbe, 0xed, 0x51, 0x5b, 0x38, 0x8c, 0xe2, 0x15, + 0x98, 0xa6, 0xbd, 0xae, 0xe9, 0x13, 0xcf, 0x75, 0x6c, 0x2b, 0x9a, 0xda, 0xc8, 0xd2, 0x5e, 0xd7, + 0x88, 0x20, 0x5c, 0x87, 0xac, 0x3d, 0x2a, 0x56, 0x27, 0x8a, 0xc9, 0x52, 0xb6, 0xb2, 0x7c, 0xe7, + 0x50, 0xd5, 0xd4, 0xe9, 0x45, 0x21, 0x61, 0xc4, 0xeb, 0x62, 0x76, 0x5e, 0xc3, 0xec, 0x0e, 0xb1, + 0x38, 0xd9, 0xf5, 0xc9, 0x01, 0xf1, 0x09, 0xb5, 0xc9, 0xcd, 0x1e, 0xe8, 0x81, 0x3d, 0x52, 0x52, + 0xff, 0x73, 0x0a, 0x60, 0xdf, 0xb3, 0x68, 0x4d, 0xc6, 0x04, 0xff, 0x0b, 0xb3, 0xbe, 0x45, 0x3b, + 0xc4, 0xec, 0x3a, 0xd4, 0x6c, 0xf5, 0x05, 0x09, 0xa7, 0x4c, 0x1a, 0x7f, 0x48, 0xf8, 0xa9, 0x43, + 0xab, 0x01, 0x18, 0xe3, 0x59, 0xc7, 0x11, 0x6f, 0x22, 0xce, 0xb3, 0x8e, 0x43, 0xde, 0x13, 0xc8, + 0x74, 0x6c, 0xd3, 0x93, 0xa1, 0x91, 0xef, 0x25, 0x5b, 0x59, 0x1a, 0xe3, 0x74, 0x98, 0xab, 0x6a, + 0x2e, 0xf0, 0x39, 0xb8, 0x28, 0x5c, 0x25, 0xcd, 0x50, 0x3a, 0x76, 0x94, 0xb9, 0x15, 0x98, 0xee, + 0xb8, 0xac, 0x65, 0xb9, 0xa6, 0x4f, 0xac, 0x36, 0x57, 0x53, 0x45, 0x54, 0x52, 0x8c, 0x6c, 0x88, + 0x19, 0x01, 0x74, 0xeb, 0x0d, 0xa5, 0x6f, 0xbf, 0xa1, 0x65, 0x80, 0x80, 0x72, 0xc4, 0x04, 0xf1, + 0xb9, 0x3a, 0x29, 0x09, 0x19, 0xda, 0xeb, 0x3e, 0x97, 0x00, 0xde, 0xbb, 0xbe, 0xdc, 0x29, 0xb9, + 0xdc, 0xd5, 0x3b, 0x97, 0x1b, 0xcf, 0xc8, 0x98, 0x45, 0xe3, 0x57, 0x90, 0x97, 0xdd, 0xcc, 0xb8, + 0xb0, 0xf2, 0x30, 0xe1, 0x9c, 0x54, 0x8a, 0x51, 0x70, 0x13, 0xf2, 0x6e, 0x10, 0x10, 0xd3, 0xbb, + 0x4a, 0x08, 0x57, 0x33, 0x52, 0x7d, 0xdc, 0xc7, 0x74, 0x23, 0x4c, 0x43, 0x59, 0xf7, 0x3a, 0x3c, + 0x4c, 0xc7, 0x7b, 0x04, 0xb3, 0xa3, 0x74, 0xd4, 0xa9, 0xf0, 0xfb, 0xf8, 0x3f, 0x48, 0x05, 0x17, + 0x8b, 0xcc, 0x45, 0xb6, 0xf2, 0xf7, 0x98, 0x1e, 0x41, 0x45, 0x24, 0x2c, 0xa9, 0xf8, 0x11, 0x4c, + 0x86, 0xd7, 0x90, 0x0c, 0xc9, 0xf8, 0xb0, 0x8e, 0xda, 0x44, 0xa5, 0x51, 0x89, 0xbe, 0x03, 0x7f, + 0x6d, 0x11, 0x31, 0x7a, 0xcc, 0x0d, 0xf2, 0xae, 0x47, 0xb8, 0xc0, 0xeb, 0x90, 0x0e, 0xd4, 0x87, + 0x5f, 0xc0, 0x6f, 0x9c, 0x84, 0x5c, 0xdd, 0x87, 0xf9, 0x9b, 0x6a, 0xdc, 0x63, 0x94, 0x13, 0xfc, + 0x02, 0xfe, 0x8c, 0x5d, 0x98, 0x26, 0xa1, 0xc2, 0x77, 0xc8, 0x50, 0x5c, 0xbf, 0xd3, 0xb1, 0x5c, + 0x4c, 0xd4, 0x27, 0xcf, 0xaf, 0xc1, 0x0e, 0xe1, 0xfa, 0x57, 0x04, 0x6a, 0xd3, 0x6b, 0x5b, 0x82, + 0x8c, 0x99, 0x62, 0x03, 0x32, 0x82, 0x99, 0x6d, 0xe2, 0x12, 0x41, 0xee, 0x37, 0x89, 0x22, 0xd8, + 0xa6, 0xa4, 0xe3, 0xba, 0xac, 0xed, 0x79, 0x9c, 0xf8, 0x22, 0xba, 0x6b, 0xee, 0x6f, 0x54, 0x11, + 0xac, 0x29, 0x2b, 0xf5, 0x25, 0x58, 0x18, 0x63, 0x2f, 0x5c, 0x4b, 0x75, 0xf5, 0xf4, 0x87, 0x96, + 0x38, 0x1d, 0x68, 0xe8, 0x6c, 0xa0, 0xa1, 0xf3, 0x81, 0x86, 0xbe, 0x0f, 0x34, 0xf4, 0xf1, 0x52, + 0x4b, 0x9c, 0x5d, 0x6a, 0x89, 0xf3, 0x4b, 0x2d, 0xf1, 0x72, 0x2a, 0xea, 0xd3, 0x9a, 0x94, 0xff, + 0x13, 0xeb, 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xc2, 0x30, 0xc2, 0x33, 0x81, 0x06, 0x00, 0x00, +} + +func (this *GCPolicy) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + that1, ok := that.(*GCPolicy) + if !ok { + that2, ok := that.(GCPolicy) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TTLSeconds != that1.TTLSeconds { + return false + } + return true +} func (this *Constraint) Equal(that interface{}) bool { if that == nil { return this == nil @@ -629,7 +691,7 @@ func (this *SpanConfig) Equal(that interface{}) bool { if this.RangeMaxBytes != that1.RangeMaxBytes { return false } - if this.GCTTL != that1.GCTTL { + if !this.GCPolicy.Equal(&that1.GCPolicy) { return false } if this.GlobalReads != that1.GlobalReads { @@ -667,6 +729,34 @@ func (this *SpanConfig) Equal(that interface{}) bool { } return true } +func (m *GCPolicy) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GCPolicy) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *GCPolicy) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.TTLSeconds != 0 { + i = encodeVarintSpanConfig(dAtA, i, uint64(m.TTLSeconds)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *Constraint) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -870,11 +960,16 @@ func (m *SpanConfig) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x20 } - if m.GCTTL != 0 { - i = encodeVarintSpanConfig(dAtA, i, uint64(m.GCTTL)) - i-- - dAtA[i] = 0x18 + { + size, err := m.GCPolicy.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSpanConfig(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x1a if m.RangeMaxBytes != 0 { i = encodeVarintSpanConfig(dAtA, i, uint64(m.RangeMaxBytes)) i-- @@ -1090,6 +1185,101 @@ func encodeVarintSpanConfig(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } +func NewPopulatedGCPolicy(r randySpanConfig, easy bool) *GCPolicy { + this := &GCPolicy{} + this.TTLSeconds = int32(r.Int31()) + if r.Intn(2) == 0 { + this.TTLSeconds *= -1 + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +type randySpanConfig interface { + Float32() float32 + Float64() float64 + Int63() int64 + Int31() int32 + Uint32() uint32 + Intn(n int) int +} + +func randUTF8RuneSpanConfig(r randySpanConfig) rune { + ru := r.Intn(62) + if ru < 10 { + return rune(ru + 48) + } else if ru < 36 { + return rune(ru + 55) + } + return rune(ru + 61) +} +func randStringSpanConfig(r randySpanConfig) string { + v1 := r.Intn(100) + tmps := make([]rune, v1) + for i := 0; i < v1; i++ { + tmps[i] = randUTF8RuneSpanConfig(r) + } + return string(tmps) +} +func randUnrecognizedSpanConfig(r randySpanConfig, maxFieldNumber int) (dAtA []byte) { + l := r.Intn(5) + for i := 0; i < l; i++ { + wire := r.Intn(4) + if wire == 3 { + wire = 5 + } + fieldNumber := maxFieldNumber + r.Intn(100) + dAtA = randFieldSpanConfig(dAtA, r, fieldNumber, wire) + } + return dAtA +} +func randFieldSpanConfig(dAtA []byte, r randySpanConfig, fieldNumber int, wire int) []byte { + key := uint32(fieldNumber)<<3 | uint32(wire) + switch wire { + case 0: + dAtA = encodeVarintPopulateSpanConfig(dAtA, uint64(key)) + v2 := r.Int63() + if r.Intn(2) == 0 { + v2 *= -1 + } + dAtA = encodeVarintPopulateSpanConfig(dAtA, uint64(v2)) + case 1: + dAtA = encodeVarintPopulateSpanConfig(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + case 2: + dAtA = encodeVarintPopulateSpanConfig(dAtA, uint64(key)) + ll := r.Intn(100) + dAtA = encodeVarintPopulateSpanConfig(dAtA, uint64(ll)) + for j := 0; j < ll; j++ { + dAtA = append(dAtA, byte(r.Intn(256))) + } + default: + dAtA = encodeVarintPopulateSpanConfig(dAtA, uint64(key)) + dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) + } + return dAtA +} +func encodeVarintPopulateSpanConfig(dAtA []byte, v uint64) []byte { + for v >= 1<<7 { + dAtA = append(dAtA, uint8(uint64(v)&0x7f|0x80)) + v >>= 7 + } + dAtA = append(dAtA, uint8(v)) + return dAtA +} +func (m *GCPolicy) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TTLSeconds != 0 { + n += 1 + sovSpanConfig(uint64(m.TTLSeconds)) + } + return n +} + func (m *Constraint) Size() (n int) { if m == nil { return 0 @@ -1155,9 +1345,8 @@ func (m *SpanConfig) Size() (n int) { if m.RangeMaxBytes != 0 { n += 1 + sovSpanConfig(uint64(m.RangeMaxBytes)) } - if m.GCTTL != 0 { - n += 1 + sovSpanConfig(uint64(m.GCTTL)) - } + l = m.GCPolicy.Size() + n += 1 + l + sovSpanConfig(uint64(l)) if m.GlobalReads { n += 2 } @@ -1267,6 +1456,75 @@ func sovSpanConfig(x uint64) (n int) { func sozSpanConfig(x uint64) (n int) { return sovSpanConfig(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } +func (m *GCPolicy) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GCPolicy: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GCPolicy: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TTLSeconds", wireType) + } + m.TTLSeconds = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSpanConfig + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TTLSeconds |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipSpanConfig(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSpanConfig + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *Constraint) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -1655,10 +1913,10 @@ func (m *SpanConfig) Unmarshal(dAtA []byte) error { } } case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field GCTTL", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field GCPolicy", wireType) } - m.GCTTL = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSpanConfig @@ -1668,11 +1926,25 @@ func (m *SpanConfig) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.GCTTL |= int32(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthSpanConfig + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSpanConfig + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.GCPolicy.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 4: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field GlobalReads", wireType) diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index 3963442d6dde..762a01887fae 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -18,10 +18,23 @@ import "gogoproto/gogo.proto"; // TODO(irfansharif): We could have the proto definitions in pkg/config/zonepb // use these messages instead of duplicating everything. +// GCPolicy dictates the garbage collection policy to apply over a given span. +// It parallels the definition found in zonepb/zone.proto. +message GCPolicy { + option (gogoproto.equal) = true; + option (gogoproto.populate) = true; + + // TTLSeconds is the number of seconds overwritten values will be retained + // before garbage collection. A value <= 0 means older versions are never + // GC-ed. + int32 ttl_seconds = 1 [(gogoproto.customname) = "TTLSeconds"]; +} + // Constraint constrains the stores that a replica can be stored on. It // parallels the definition found in zonepb/zone.proto. message Constraint { option (gogoproto.equal) = true; + option (gogoproto.goproto_stringer) = false; enum Type { // REQUIRED ensures all replicas are placed on stores with locality tags @@ -47,6 +60,7 @@ message Constraint { // together by replicas. It parallels the definition found in zonepb/zone.proto. message ConstraintsConjunction { option (gogoproto.equal) = true; + option (gogoproto.goproto_stringer) = false; // NumReplicas is the number of replicas that should abide by the constraints // below. If set to zero, the constraints will apply to all replicas of the @@ -76,18 +90,16 @@ message LeasePreference { message SpanConfig { option (gogoproto.equal) = true; - // RangeMinBytes is the minimum size, in bytes, for a range in the given - // keyspan. When a range is less than this size, it'll be merged with an - // adjacent range. + // RangeMinBytes is the minimum size, in bytes, a range can have. When a range + // is less than this size, it'll be merged with an adjacent range. int64 range_min_bytes = 1; - // RangeMaxBytes is the maximum size, in bytes, for a range in the given - // keyspan. When a range is more than this size, it'll split into two ranges. + // RangeMaxBytes is the maximum size, in bytes, a range can have. When a range + // is more than this size, it'll split into two ranges. int64 range_max_bytes = 2; - // GCTTL is the number of seconds overwritten values will be retained before - // garbage collection. A value <= 0 means older versions are never GC-ed. - int32 gc_ttl = 3 [(gogoproto.customname) = "GCTTL"]; + // GCPolicy dictates the the GC policy to apply over a range. + GCPolicy gc_policy = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "GCPolicy"]; // GlobalReads specifies whether transactions operating over the range(s) // should be configured to provide non-blocking behavior, meaning that reads diff --git a/pkg/server/config.go b/pkg/server/config.go index 96dda3238f2a..3d99f8c1eea1 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -138,7 +138,8 @@ type BaseConfig struct { // DefaultZoneConfig is used to set the default zone config inside the server. // It can be overridden during tests by setting the DefaultZoneConfigOverride - // server testing knob. + // server testing knob. Whatever is installed here is in turn used to + // initialize stores, which need a default span config. DefaultZoneConfig zonepb.ZoneConfig // Locality is a description of the topography of the server. diff --git a/pkg/server/server.go b/pkg/server/server.go index e13ab1bc482d..e8f6f31c2f08 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -578,7 +578,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { })) storeCfg := kvserver.StoreConfig{ - DefaultZoneConfig: &cfg.DefaultZoneConfig, + DefaultSpanConfig: cfg.DefaultZoneConfig.AsSpanConfig(), Settings: st, AmbientCtx: cfg.AmbientCtx, RaftConfig: cfg.RaftConfig, diff --git a/pkg/server/testing_knobs.go b/pkg/server/testing_knobs.go index a569887e294d..66910ac59e65 100644 --- a/pkg/server/testing_knobs.go +++ b/pkg/server/testing_knobs.go @@ -27,9 +27,11 @@ type TestingKnobs struct { // DisableAutomaticVersionUpgrade, if set, temporarily disables the server's // automatic version upgrade mechanism. DisableAutomaticVersionUpgrade int32 // accessed atomically - // DefaultZoneConfigOverride, if set, overrides the default zone config defined in `pkg/config/zone.go` + // DefaultZoneConfigOverride, if set, overrides the default zone config + // defined in `pkg/config/zone.go`. DefaultZoneConfigOverride *zonepb.ZoneConfig - // DefaultSystemZoneConfigOverride, if set, overrides the default system zone config defined in `pkg/config/zone.go` + // DefaultSystemZoneConfigOverride, if set, overrides the default system + // zone config defined in `pkg/config/zone.go` DefaultSystemZoneConfigOverride *zonepb.ZoneConfig // SignalAfterGettingRPCAddress, if non-nil, is closed after the server gets // an RPC server address, and prior to waiting on PauseAfterGettingRPCAddress below. diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index c75739da996c..58997d41d354 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -46,3 +46,25 @@ type ReconciliationDependencies interface { // reconciliation job will react to these updates by installing them into KV // through the KVAccessor. } + +// Store is a data structure used to store span configs. +type Store interface { + StoreReader + + // TODO(irfansharif): We'll want to add a StoreWriter interface here once we + // implement a data structure to store span configs. We expect this data + // structure to be used in KV to eventually replace the use of the + // gossip-backed system config span. +} + +// Silence the unused linter. +var _ Store = nil + +// StoreReader is the read-only portion of the Store interface. It's an adaptor +// interface implemented by config.SystemConfig to let us later swap out the +// source with one backed by a view of `system.span_configurations`. +type StoreReader interface { + NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool + ComputeSplitKey(ctx context.Context, start, end roachpb.RKey) roachpb.RKey + GetSpanConfigForKey(ctx context.Context, key roachpb.RKey) (roachpb.SpanConfig, error) +} diff --git a/pkg/sql/zone_config_test.go b/pkg/sql/zone_config_test.go index 578edf8f6db2..84d71b0c074b 100644 --- a/pkg/sql/zone_config_test.go +++ b/pkg/sql/zone_config_test.go @@ -243,23 +243,23 @@ func TestGetZoneConfig(t *testing.T) { db1Cfg := defaultZoneConfig db1Cfg.NumReplicas = proto.Int32(1) - db1Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db1"}}}} + db1Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Type: zonepb.Constraint_REQUIRED, Value: "db1"}}}} tb11Cfg := defaultZoneConfig tb11Cfg.NumReplicas = proto.Int32(1) - tb11Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db1.tb1"}}}} + tb11Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Type: zonepb.Constraint_REQUIRED, Value: "db1.tb1"}}}} p211Cfg := defaultZoneConfig p211Cfg.NumReplicas = proto.Int32(1) - p211Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p1"}}}} + p211Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Type: zonepb.Constraint_REQUIRED, Value: "db2.tb1.p1"}}}} p212Cfg := defaultZoneConfig p212Cfg.NumReplicas = proto.Int32(1) - p212Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p2"}}}} + p212Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Type: zonepb.Constraint_REQUIRED, Value: "db2.tb1.p2"}}}} tb21Cfg := defaultZoneConfig tb21Cfg.NumReplicas = proto.Int32(1) - tb21Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} + tb21Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Type: zonepb.Constraint_REQUIRED, Value: "db2.tb1"}}}} tb21Cfg.Subzones = []zonepb.Subzone{ {IndexID: 1, PartitionName: "p0", Config: p211Cfg}, {IndexID: 1, PartitionName: "p1", Config: p212Cfg}, @@ -272,7 +272,7 @@ func TestGetZoneConfig(t *testing.T) { p221Cfg := defaultZoneConfig p221Cfg.NumReplicas = proto.Int32(1) - p221Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb2.p1"}}}} + p221Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Type: zonepb.Constraint_REQUIRED, Value: "db2.tb2.p1"}}}} // Subzone Placeholder tb22Cfg := *zonepb.NewZoneConfig() @@ -479,44 +479,53 @@ func TestCascadingZoneConfig(t *testing.T) { // tb2: no zone config // p1: true [1, 255) - inherits replciation factor from default + makeConstraints := func(value string) []zonepb.ConstraintsConjunction { + return []zonepb.ConstraintsConjunction{ + { + Constraints: []zonepb.Constraint{ + {Type: zonepb.Constraint_REQUIRED, Value: value}, + }, + }, + } + } db1Cfg := *zonepb.NewZoneConfig() db1Cfg.NumReplicas = proto.Int32(5) - db1Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db1"}}}} + db1Cfg.Constraints = makeConstraints("db1") db1Cfg.InheritedConstraints = false // Expected complete config expectedDb1Cfg := defaultZoneConfig expectedDb1Cfg.NumReplicas = proto.Int32(5) - expectedDb1Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db1"}}}} + expectedDb1Cfg.Constraints = makeConstraints("db1") tb11Cfg := *zonepb.NewZoneConfig() - tb11Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db1.tb1"}}}} + tb11Cfg.Constraints = makeConstraints("db1.tb1") tb11Cfg.InheritedConstraints = false // Expected complete config expectedTb11Cfg := expectedDb1Cfg - expectedTb11Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db1.tb1"}}}} + expectedTb11Cfg.Constraints = makeConstraints("db1.tb1") p211Cfg := *zonepb.NewZoneConfig() p211Cfg.NumReplicas = proto.Int32(1) - p211Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p1"}}}} + p211Cfg.Constraints = makeConstraints("db2.tb1.p1") p211Cfg.InheritedConstraints = false // Expected complete config expectedP211Cfg := defaultZoneConfig expectedP211Cfg.NumReplicas = proto.Int32(1) - expectedP211Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p1"}}}} + expectedP211Cfg.Constraints = makeConstraints("db2.tb1.p1") p212Cfg := *zonepb.NewZoneConfig() - p212Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p2"}}}} + p212Cfg.Constraints = makeConstraints("db2.tb1.p2") p212Cfg.InheritedConstraints = false // Expected complete config expectedP212Cfg := defaultZoneConfig - expectedP212Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1.p2"}}}} + expectedP212Cfg.Constraints = makeConstraints("db2.tb1.p2") tb21Cfg := *zonepb.NewZoneConfig() - tb21Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} + tb21Cfg.Constraints = makeConstraints("db2.tb1") tb21Cfg.InheritedConstraints = false tb21Cfg.Subzones = []zonepb.Subzone{ {IndexID: 1, PartitionName: "p0", Config: p211Cfg}, @@ -530,7 +539,7 @@ func TestCascadingZoneConfig(t *testing.T) { // Expected complete config expectedTb21Cfg := defaultZoneConfig - expectedTb21Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb1"}}}} + expectedTb21Cfg.Constraints = makeConstraints("db2.tb1") expectedTb21Cfg.Subzones = []zonepb.Subzone{ {IndexID: 1, PartitionName: "p0", Config: p211Cfg}, {IndexID: 1, PartitionName: "p1", Config: p212Cfg}, @@ -542,12 +551,12 @@ func TestCascadingZoneConfig(t *testing.T) { } p221Cfg := *zonepb.NewZoneConfig() - p221Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb2.p1"}}}} + p221Cfg.Constraints = makeConstraints("db2.tb2.p1") p221Cfg.InheritedConstraints = false // Expected complete config expectedP221Cfg := defaultZoneConfig - expectedP221Cfg.Constraints = []zonepb.ConstraintsConjunction{{Constraints: []zonepb.Constraint{{Value: "db2.tb2.p1"}}}} + expectedP221Cfg.Constraints = makeConstraints("db2.tb2.p1") // Subzone Placeholder tb22Cfg := *zonepb.NewZoneConfig() diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index 46175c467373..d2cf676aecd0 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -210,7 +210,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto var splits []roachpb.RKey if !ltc.DontCreateSystemRanges { schema := bootstrap.MakeMetadataSchema( - keys.SystemSQLCodec, cfg.DefaultZoneConfig, cfg.DefaultSystemZoneConfig, + keys.SystemSQLCodec, zonepb.DefaultZoneConfigRef(), zonepb.DefaultSystemZoneConfigRef(), ) var tableSplits []roachpb.RKey initialValues, tableSplits = schema.GetInitialValues() From d59529ff2ecae2ee4f307921a803afaed8c87f58 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Fri, 27 Aug 2021 20:42:17 -0400 Subject: [PATCH 2/2] kvserver: minor test improvement Remove an unncecessary clock. Release note: None Release justification: test only --- pkg/kv/kvserver/node_liveness_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index 4c3bea015459..606c1d39ded1 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -1020,12 +1020,8 @@ func TestNodeLivenessRetryAmbiguousResultError(t *testing.T) { return nil } ctx := context.Background() - manualClock := hlc.NewHybridManualClock() serv, _, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - ClockSource: manualClock.UnixNano, - }, Store: &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ TestingEvalFilter: testingEvalFilter,