From 6ed91486435892122ba1fb9f1bc64cc339f160c3 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 23 Nov 2022 19:19:37 -0500 Subject: [PATCH 1/3] liveness: move IsLiveMap type We'll depend on just this directly in a future commit, without wanting to depend on the much larger liveness package. Instead of introducing a standalone package for it, we'll just re-use the thin livenesspb package instead. Release note: None --- pkg/kv/kvserver/liveness/liveness.go | 16 ++---- .../kvserver/liveness/livenesspb/BUILD.bazel | 1 + .../kvserver/liveness/livenesspb/liveness.go | 11 ++++ pkg/kv/kvserver/node_liveness_test.go | 4 +- pkg/kv/kvserver/replica_circuit_breaker.go | 6 +-- .../kvserver/replica_circuit_breaker_test.go | 6 +-- pkg/kv/kvserver/replica_metrics.go | 16 +++--- pkg/kv/kvserver/replica_metrics_test.go | 54 +++++++++---------- pkg/kv/kvserver/replica_proposal_buf.go | 4 +- pkg/kv/kvserver/replica_raft.go | 10 ++-- pkg/kv/kvserver/replica_raft_quiesce.go | 11 ++-- pkg/kv/kvserver/replica_test.go | 43 ++++++++------- pkg/kv/kvserver/store.go | 3 +- pkg/kv/kvserver/store_raft.go | 3 +- pkg/server/problem_ranges.go | 6 +-- 15 files changed, 97 insertions(+), 97 deletions(-) diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index d2e92628c4db..b32728e453e1 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -1025,21 +1025,11 @@ func (nl *NodeLiveness) SelfEx() (_ Record, ok bool) { return nl.getLivenessLocked(nl.gossip.NodeID.Get()) } -// IsLiveMapEntry encapsulates data about current liveness for a -// node. -type IsLiveMapEntry struct { - livenesspb.Liveness - IsLive bool -} - -// IsLiveMap is a type alias for a map from NodeID to IsLiveMapEntry. -type IsLiveMap map[roachpb.NodeID]IsLiveMapEntry - // GetIsLiveMap returns a map of nodeID to boolean liveness status of // each node. This excludes nodes that were removed completely (dead + // decommissioning). -func (nl *NodeLiveness) GetIsLiveMap() IsLiveMap { - lMap := IsLiveMap{} +func (nl *NodeLiveness) GetIsLiveMap() livenesspb.IsLiveMap { + lMap := livenesspb.IsLiveMap{} nl.mu.RLock() defer nl.mu.RUnlock() now := nl.clock.Now().GoTime() @@ -1049,7 +1039,7 @@ func (nl *NodeLiveness) GetIsLiveMap() IsLiveMap { // This is a node that was completely removed. Skip over it. continue } - lMap[nID] = IsLiveMapEntry{ + lMap[nID] = livenesspb.IsLiveMapEntry{ Liveness: l.Liveness, IsLive: isLive, } diff --git a/pkg/kv/kvserver/liveness/livenesspb/BUILD.bazel b/pkg/kv/kvserver/liveness/livenesspb/BUILD.bazel index a056795590e9..edaaaa99f792 100644 --- a/pkg/kv/kvserver/liveness/livenesspb/BUILD.bazel +++ b/pkg/kv/kvserver/liveness/livenesspb/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb", visibility = ["//visibility:public"], deps = [ + "//pkg/roachpb", "//pkg/util/timeutil", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//status", diff --git a/pkg/kv/kvserver/liveness/livenesspb/liveness.go b/pkg/kv/kvserver/liveness/livenesspb/liveness.go index 9f8c61c1a0f9..d6184c1d56de 100644 --- a/pkg/kv/kvserver/liveness/livenesspb/liveness.go +++ b/pkg/kv/kvserver/liveness/livenesspb/liveness.go @@ -14,6 +14,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -132,3 +133,13 @@ func ValidateTransition(old, new Liveness) error { return nil } + +// IsLiveMapEntry encapsulates data about current liveness for a +// node. +type IsLiveMapEntry struct { + Liveness + IsLive bool +} + +// IsLiveMap is a type alias for a map from NodeID to IsLiveMapEntry. +type IsLiveMap map[roachpb.NodeID]IsLiveMapEntry diff --git a/pkg/kv/kvserver/node_liveness_test.go b/pkg/kv/kvserver/node_liveness_test.go index 33ff87970de5..f98f3f84a850 100644 --- a/pkg/kv/kvserver/node_liveness_test.go +++ b/pkg/kv/kvserver/node_liveness_test.go @@ -669,7 +669,7 @@ func TestNodeLivenessGetIsLiveMap(t *testing.T) { l1, _ := nl.GetLiveness(1) l2, _ := nl.GetLiveness(2) l3, _ := nl.GetLiveness(3) - expectedLMap := liveness.IsLiveMap{ + expectedLMap := livenesspb.IsLiveMap{ 1: {Liveness: l1.Liveness, IsLive: true}, 2: {Liveness: l2.Liveness, IsLive: true}, 3: {Liveness: l3.Liveness, IsLive: true}, @@ -705,7 +705,7 @@ func TestNodeLivenessGetIsLiveMap(t *testing.T) { l1, _ = nl.GetLiveness(1) l2, _ = nl.GetLiveness(2) l3, _ = nl.GetLiveness(3) - expectedLMap = liveness.IsLiveMap{ + expectedLMap = livenesspb.IsLiveMap{ 1: {Liveness: l1.Liveness, IsLive: true}, 2: {Liveness: l2.Liveness, IsLive: false}, 3: {Liveness: l3.Liveness, IsLive: false}, diff --git a/pkg/kv/kvserver/replica_circuit_breaker.go b/pkg/kv/kvserver/replica_circuit_breaker.go index 8a5ccd897b81..f71169e40eb3 100644 --- a/pkg/kv/kvserver/replica_circuit_breaker.go +++ b/pkg/kv/kvserver/replica_circuit_breaker.go @@ -15,7 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" @@ -239,7 +239,7 @@ func replicaUnavailableError( err error, desc *roachpb.RangeDescriptor, replDesc roachpb.ReplicaDescriptor, - lm liveness.IsLiveMap, + lm livenesspb.IsLiveMap, rs *raft.Status, closedTS hlc.Timestamp, ) error { @@ -280,7 +280,7 @@ func (r *Replica) replicaUnavailableError(err error) error { desc := r.Desc() replDesc, _ := desc.GetReplicaDescriptor(r.store.StoreID()) - isLiveMap, _ := r.store.livenessMap.Load().(liveness.IsLiveMap) + isLiveMap, _ := r.store.livenessMap.Load().(livenesspb.IsLiveMap) ct := r.GetCurrentClosedTimestamp(context.Background()) return replicaUnavailableError(err, desc, replDesc, isLiveMap, r.RaftStatus(), ct) } diff --git a/pkg/kv/kvserver/replica_circuit_breaker_test.go b/pkg/kv/kvserver/replica_circuit_breaker_test.go index 5125be4c95bc..bcbe5d723f5d 100644 --- a/pkg/kv/kvserver/replica_circuit_breaker_test.go +++ b/pkg/kv/kvserver/replica_circuit_breaker_test.go @@ -15,7 +15,7 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/echotest" @@ -36,8 +36,8 @@ func TestReplicaUnavailableError(t *testing.T) { repls.AddReplica(roachpb.ReplicaDescriptor{NodeID: 1, StoreID: 10, ReplicaID: 100}) repls.AddReplica(roachpb.ReplicaDescriptor{NodeID: 2, StoreID: 20, ReplicaID: 200}) desc := roachpb.NewRangeDescriptor(10, roachpb.RKey("a"), roachpb.RKey("z"), repls) - lm := liveness.IsLiveMap{ - 1: liveness.IsLiveMapEntry{IsLive: true}, + lm := livenesspb.IsLiveMap{ + 1: livenesspb.IsLiveMapEntry{IsLive: true}, } ts, err := time.Parse("2006-01-02 15:04:05", "2006-01-02 15:04:05") require.NoError(t, err) diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index 563620e1f173..1cae44da5804 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "go.etcd.io/etcd/raft/v3" @@ -60,7 +60,7 @@ type ReplicaMetrics struct { // Metrics returns the current metrics for the replica. func (r *Replica) Metrics( - ctx context.Context, now hlc.ClockTimestamp, livenessMap liveness.IsLiveMap, clusterNodes int, + ctx context.Context, now hlc.ClockTimestamp, livenessMap livenesspb.IsLiveMap, clusterNodes int, ) ReplicaMetrics { r.store.unquiescedReplicas.Lock() _, ticking := r.store.unquiescedReplicas.m[r.RangeID] @@ -107,7 +107,7 @@ func (r *Replica) Metrics( type calcReplicaMetricsInput struct { raftCfg *base.RaftConfig conf roachpb.SpanConfig - livenessMap liveness.IsLiveMap + livenessMap livenesspb.IsLiveMap clusterNodes int desc *roachpb.RangeDescriptor raftStatus *raft.Status @@ -194,7 +194,7 @@ func calcRangeCounter( storeID roachpb.StoreID, desc *roachpb.RangeDescriptor, leaseStatus kvserverpb.LeaseStatus, - livenessMap liveness.IsLiveMap, + livenessMap livenesspb.IsLiveMap, numVoters, numReplicas int32, clusterNodes int, ) (rangeCounter, unavailable, underreplicated, overreplicated bool) { @@ -240,17 +240,17 @@ func calcRangeCounter( // replica is determined by checking its node in the provided liveness map. This // method is used when indicating under-replication so only voter replicas are // considered. -func calcLiveVoterReplicas(desc *roachpb.RangeDescriptor, livenessMap liveness.IsLiveMap) int { +func calcLiveVoterReplicas(desc *roachpb.RangeDescriptor, livenessMap livenesspb.IsLiveMap) int { return calcLiveReplicas(desc.Replicas().VoterDescriptors(), livenessMap) } // calcLiveNonVoterReplicas returns a count of the live non-voter replicas; a live // replica is determined by checking its node in the provided liveness map. -func calcLiveNonVoterReplicas(desc *roachpb.RangeDescriptor, livenessMap liveness.IsLiveMap) int { +func calcLiveNonVoterReplicas(desc *roachpb.RangeDescriptor, livenessMap livenesspb.IsLiveMap) int { return calcLiveReplicas(desc.Replicas().NonVoterDescriptors(), livenessMap) } -func calcLiveReplicas(repls []roachpb.ReplicaDescriptor, livenessMap liveness.IsLiveMap) int { +func calcLiveReplicas(repls []roachpb.ReplicaDescriptor, livenessMap livenesspb.IsLiveMap) int { var live int for _, rd := range repls { if livenessMap[rd.NodeID].IsLive { @@ -263,7 +263,7 @@ func calcLiveReplicas(repls []roachpb.ReplicaDescriptor, livenessMap liveness.Is // calcBehindCount returns a total count of log entries that follower replicas // are behind. This can only be computed on the raft leader. func calcBehindCount( - raftStatus *raft.Status, desc *roachpb.RangeDescriptor, livenessMap liveness.IsLiveMap, + raftStatus *raft.Status, desc *roachpb.RangeDescriptor, livenessMap livenesspb.IsLiveMap, ) int64 { var behindCount int64 for _, rd := range desc.Replicas().Descriptors() { diff --git a/pkg/kv/kvserver/replica_metrics_test.go b/pkg/kv/kvserver/replica_metrics_test.go index 045020410bdc..8c270bc609bf 100644 --- a/pkg/kv/kvserver/replica_metrics_test.go +++ b/pkg/kv/kvserver/replica_metrics_test.go @@ -14,7 +14,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -55,8 +55,8 @@ func TestCalcRangeCounterIsLiveMap(t *testing.T) { })) { - ctr, down, under, over := calcRangeCounter(1100, threeVotersAndSingleNonVoter, leaseStatus, liveness.IsLiveMap{ - 1000: liveness.IsLiveMapEntry{IsLive: true}, // by NodeID + ctr, down, under, over := calcRangeCounter(1100, threeVotersAndSingleNonVoter, leaseStatus, livenesspb.IsLiveMap{ + 1000: livenesspb.IsLiveMapEntry{IsLive: true}, // by NodeID }, 3 /* numVoters */, 4 /* numReplicas */, 4 /* clusterNodes */) require.True(t, ctr) @@ -66,8 +66,8 @@ func TestCalcRangeCounterIsLiveMap(t *testing.T) { } { - ctr, down, under, over := calcRangeCounter(1000, threeVotersAndSingleNonVoter, leaseStatus, liveness.IsLiveMap{ - 1000: liveness.IsLiveMapEntry{IsLive: false}, + ctr, down, under, over := calcRangeCounter(1000, threeVotersAndSingleNonVoter, leaseStatus, livenesspb.IsLiveMap{ + 1000: livenesspb.IsLiveMapEntry{IsLive: false}, }, 3 /* numVoters */, 4 /* numReplicas */, 4 /* clusterNodes */) // Does not confuse a non-live entry for a live one. In other words, @@ -79,11 +79,11 @@ func TestCalcRangeCounterIsLiveMap(t *testing.T) { } { - ctr, down, under, over := calcRangeCounter(11, threeVotersAndSingleNonVoter, leaseStatus, liveness.IsLiveMap{ - 10: liveness.IsLiveMapEntry{IsLive: true}, - 100: liveness.IsLiveMapEntry{IsLive: true}, - 1000: liveness.IsLiveMapEntry{IsLive: true}, - 2000: liveness.IsLiveMapEntry{IsLive: true}, + ctr, down, under, over := calcRangeCounter(11, threeVotersAndSingleNonVoter, leaseStatus, livenesspb.IsLiveMap{ + 10: livenesspb.IsLiveMapEntry{IsLive: true}, + 100: livenesspb.IsLiveMapEntry{IsLive: true}, + 1000: livenesspb.IsLiveMapEntry{IsLive: true}, + 2000: livenesspb.IsLiveMapEntry{IsLive: true}, }, 3 /* numVoters */, 4 /* numReplicas */, 4 /* clusterNodes */) require.True(t, ctr) @@ -94,11 +94,11 @@ func TestCalcRangeCounterIsLiveMap(t *testing.T) { { // Single non-voter dead - ctr, down, under, over := calcRangeCounter(11, oneVoterAndThreeNonVoters, leaseStatus, liveness.IsLiveMap{ - 10: liveness.IsLiveMapEntry{IsLive: true}, - 100: liveness.IsLiveMapEntry{IsLive: true}, - 1000: liveness.IsLiveMapEntry{IsLive: false}, - 2000: liveness.IsLiveMapEntry{IsLive: true}, + ctr, down, under, over := calcRangeCounter(11, oneVoterAndThreeNonVoters, leaseStatus, livenesspb.IsLiveMap{ + 10: livenesspb.IsLiveMapEntry{IsLive: true}, + 100: livenesspb.IsLiveMapEntry{IsLive: true}, + 1000: livenesspb.IsLiveMapEntry{IsLive: false}, + 2000: livenesspb.IsLiveMapEntry{IsLive: true}, }, 1 /* numVoters */, 4 /* numReplicas */, 4 /* clusterNodes */) require.True(t, ctr) @@ -109,11 +109,11 @@ func TestCalcRangeCounterIsLiveMap(t *testing.T) { { // All non-voters are dead, but range is not unavailable - ctr, down, under, over := calcRangeCounter(11, oneVoterAndThreeNonVoters, leaseStatus, liveness.IsLiveMap{ - 10: liveness.IsLiveMapEntry{IsLive: true}, - 100: liveness.IsLiveMapEntry{IsLive: false}, - 1000: liveness.IsLiveMapEntry{IsLive: false}, - 2000: liveness.IsLiveMapEntry{IsLive: false}, + ctr, down, under, over := calcRangeCounter(11, oneVoterAndThreeNonVoters, leaseStatus, livenesspb.IsLiveMap{ + 10: livenesspb.IsLiveMapEntry{IsLive: true}, + 100: livenesspb.IsLiveMapEntry{IsLive: false}, + 1000: livenesspb.IsLiveMapEntry{IsLive: false}, + 2000: livenesspb.IsLiveMapEntry{IsLive: false}, }, 1 /* numVoters */, 4 /* numReplicas */, 4 /* clusterNodes */) require.True(t, ctr) @@ -124,11 +124,11 @@ func TestCalcRangeCounterIsLiveMap(t *testing.T) { { // More non-voters than needed - ctr, down, under, over := calcRangeCounter(11, oneVoterAndThreeNonVoters, leaseStatus, liveness.IsLiveMap{ - 10: liveness.IsLiveMapEntry{IsLive: true}, - 100: liveness.IsLiveMapEntry{IsLive: true}, - 1000: liveness.IsLiveMapEntry{IsLive: true}, - 2000: liveness.IsLiveMapEntry{IsLive: true}, + ctr, down, under, over := calcRangeCounter(11, oneVoterAndThreeNonVoters, leaseStatus, livenesspb.IsLiveMap{ + 10: livenesspb.IsLiveMapEntry{IsLive: true}, + 100: livenesspb.IsLiveMapEntry{IsLive: true}, + 1000: livenesspb.IsLiveMapEntry{IsLive: true}, + 2000: livenesspb.IsLiveMapEntry{IsLive: true}, }, 1 /* numVoters */, 3 /* numReplicas */, 4 /* clusterNodes */) require.True(t, ctr) @@ -238,9 +238,9 @@ func TestCalcRangeCounterLeaseHolder(t *testing.T) { for _, tc := range testcases { t.Run(tc.desc, func(t *testing.T) { - livenessMap := liveness.IsLiveMap{} + livenessMap := livenesspb.IsLiveMap{} for _, nodeID := range tc.liveNodes { - livenessMap[nodeID] = liveness.IsLiveMapEntry{IsLive: true} + livenessMap[nodeID] = livenesspb.IsLiveMapEntry{IsLive: true} } ctr, _, _, _ := calcRangeCounter(tc.storeID, rangeDesc, tc.leaseStatus, livenessMap, 3 /* numVoters */, 4 /* numReplicas */, 4 /* clusterNodes */) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index fbdb52cd319c..28fe399a6036 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -1239,7 +1239,7 @@ func (rp *replicaProposer) ownsValidLease(ctx context.Context, now hlc.ClockTime func (rp *replicaProposer) shouldCampaignOnRedirect(raftGroup proposerRaft) bool { r := (*Replica)(rp) - livenessMap, _ := r.store.livenessMap.Load().(liveness.IsLiveMap) + livenessMap, _ := r.store.livenessMap.Load().(livenesspb.IsLiveMap) return shouldCampaignOnLeaseRequestRedirect( raftGroup.BasicStatus(), livenessMap, diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 7d077dc231ed..1aefdcb21728 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" @@ -1093,7 +1093,7 @@ func maybeFatalOnRaftReadyErr(ctx context.Context, err error) (removed bool) { // tick the Raft group, returning true if the raft group exists and should // be queued for Ready processing; false otherwise. func (r *Replica) tick( - ctx context.Context, livenessMap liveness.IsLiveMap, ioThresholdMap *ioThresholdMap, + ctx context.Context, livenessMap livenesspb.IsLiveMap, ioThresholdMap *ioThresholdMap, ) (bool, error) { r.unreachablesMu.Lock() remotes := r.unreachablesMu.remotes @@ -1804,7 +1804,7 @@ func shouldCampaignOnWake( leaseStatus kvserverpb.LeaseStatus, storeID roachpb.StoreID, raftStatus raft.BasicStatus, - livenessMap liveness.IsLiveMap, + livenessMap livenesspb.IsLiveMap, desc *roachpb.RangeDescriptor, requiresExpiringLease bool, ) bool { @@ -1863,7 +1863,7 @@ func (r *Replica) maybeCampaignOnWakeLocked(ctx context.Context) { leaseStatus := r.leaseStatusAtRLocked(ctx, r.store.Clock().NowAsClockTimestamp()) raftStatus := r.mu.internalRaftGroup.BasicStatus() - livenessMap, _ := r.store.livenessMap.Load().(liveness.IsLiveMap) + livenessMap, _ := r.store.livenessMap.Load().(livenesspb.IsLiveMap) if shouldCampaignOnWake(leaseStatus, r.store.StoreID(), raftStatus, livenessMap, r.descRLocked(), r.requiresExpiringLeaseRLocked()) { r.campaignLocked(ctx) } @@ -1887,7 +1887,7 @@ func (r *Replica) maybeCampaignOnWakeLocked(ctx context.Context) { // become leader and can proceed with a future attempt to acquire the lease. func shouldCampaignOnLeaseRequestRedirect( raftStatus raft.BasicStatus, - livenessMap liveness.IsLiveMap, + livenessMap livenesspb.IsLiveMap, desc *roachpb.RangeDescriptor, requiresExpiringLease bool, now hlc.Timestamp, diff --git a/pkg/kv/kvserver/replica_raft_quiesce.go b/pkg/kv/kvserver/replica_raft_quiesce.go index 40157633ed45..06414ebc9b53 100644 --- a/pkg/kv/kvserver/replica_raft_quiesce.go +++ b/pkg/kv/kvserver/replica_raft_quiesce.go @@ -15,7 +15,6 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -180,7 +179,7 @@ func (r *Replica) canUnquiesceRLocked() bool { // elections which will cause throughput hiccups to the range, but not // correctness issues. func (r *Replica) maybeQuiesceRaftMuLockedReplicaMuLocked( - ctx context.Context, now hlc.ClockTimestamp, livenessMap liveness.IsLiveMap, + ctx context.Context, now hlc.ClockTimestamp, livenessMap livenesspb.IsLiveMap, ) bool { status, lagging, ok := shouldReplicaQuiesce(ctx, r, now, livenessMap, r.mu.pausedFollowers) if !ok { @@ -224,7 +223,7 @@ func (s laggingReplicaSet) MemberStale(l livenesspb.Liveness) bool { // AnyMemberStale returns whether any liveness information in the set is older // than liveness information contained in the IsLiveMap. -func (s laggingReplicaSet) AnyMemberStale(livenessMap liveness.IsLiveMap) bool { +func (s laggingReplicaSet) AnyMemberStale(livenessMap livenesspb.IsLiveMap) bool { for _, laggingL := range s { if l, ok := livenessMap[laggingL.NodeID]; ok { if laggingL.Compare(l.Liveness) < 0 { @@ -270,7 +269,7 @@ func shouldReplicaQuiesce( ctx context.Context, q quiescer, now hlc.ClockTimestamp, - livenessMap liveness.IsLiveMap, + livenessMap livenesspb.IsLiveMap, pausedFollowers map[roachpb.ReplicaID]struct{}, ) (*raft.Status, laggingReplicaSet, bool) { if testingDisableQuiescence { @@ -476,7 +475,7 @@ func shouldFollowerQuiesceOnNotify( q quiescer, msg raftpb.Message, lagging laggingReplicaSet, - livenessMap liveness.IsLiveMap, + livenessMap livenesspb.IsLiveMap, ) bool { // If another replica tells us to quiesce, we verify that according to // it, we are fully caught up, and that we believe it to be the leader. @@ -559,7 +558,7 @@ func (r *Replica) maybeQuiesceOnNotify( // NOTE: it is important that we grab the livenessMap under lock so // that we properly synchronize with Store.nodeIsLiveCallback, which // updates the map and then tries to unquiesce. - livenessMap, _ := r.store.livenessMap.Load().(liveness.IsLiveMap) + livenessMap, _ := r.store.livenessMap.Load().(livenesspb.IsLiveMap) if !shouldFollowerQuiesceOnNotify(ctx, r, msg, lagging, livenessMap) { return false } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 17e6df7121e5..f8097d3e3455 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -40,7 +40,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" @@ -9035,10 +9034,10 @@ func TestReplicaMetrics(t *testing.T) { } return d } - live := func(ids ...roachpb.NodeID) liveness.IsLiveMap { - m := liveness.IsLiveMap{} + live := func(ids ...roachpb.NodeID) livenesspb.IsLiveMap { + m := livenesspb.IsLiveMap{} for _, id := range ids { - m[id] = liveness.IsLiveMapEntry{IsLive: true} + m[id] = livenesspb.IsLiveMapEntry{IsLive: true} } return m } @@ -9055,7 +9054,7 @@ func TestReplicaMetrics(t *testing.T) { storeID roachpb.StoreID desc roachpb.RangeDescriptor raftStatus *raft.Status - liveness liveness.IsLiveMap + liveness livenesspb.IsLiveMap raftLogSize int64 expected ReplicaMetrics }{ @@ -9986,7 +9985,7 @@ type testQuiescer struct { isDestroyed bool // Not used to implement quiescer, but used by tests. - livenessMap liveness.IsLiveMap + livenessMap livenesspb.IsLiveMap paused map[roachpb.ReplicaID]struct{} } @@ -10073,7 +10072,7 @@ func TestShouldReplicaQuiesce(t *testing.T) { lastIndex: logIndex, raftReady: false, ownsValidLease: true, - livenessMap: liveness.IsLiveMap{ + livenessMap: livenesspb.IsLiveMap{ 1: {IsLive: true}, 2: {IsLive: true}, 3: {IsLive: true}, @@ -10179,7 +10178,7 @@ func TestShouldReplicaQuiesce(t *testing.T) { for _, i := range []uint64{1, 2, 3} { test(true, func(q *testQuiescer) *testQuiescer { nodeID := roachpb.NodeID(i) - q.livenessMap[nodeID] = liveness.IsLiveMapEntry{ + q.livenessMap[nodeID] = livenesspb.IsLiveMapEntry{ Liveness: livenesspb.Liveness{NodeID: nodeID}, IsLive: false, } @@ -10236,7 +10235,7 @@ func TestFollowerQuiesceOnNotify(t *testing.T) { }, }, }, - livenessMap: liveness.IsLiveMap{ + livenessMap: livenesspb.IsLiveMap{ 1: {IsLive: true}, 2: {IsLive: true}, 3: {IsLive: true}, @@ -10287,7 +10286,7 @@ func TestFollowerQuiesceOnNotify(t *testing.T) { Epoch: 7, Expiration: hlc.LegacyTimestamp{WallTime: 8}, } - q.livenessMap[l.NodeID] = liveness.IsLiveMapEntry{ + q.livenessMap[l.NodeID] = livenesspb.IsLiveMapEntry{ Liveness: l, IsLive: false, } @@ -10301,7 +10300,7 @@ func TestFollowerQuiesceOnNotify(t *testing.T) { Epoch: 7, Expiration: hlc.LegacyTimestamp{WallTime: 8}, } - q.livenessMap[l.NodeID] = liveness.IsLiveMapEntry{ + q.livenessMap[l.NodeID] = livenesspb.IsLiveMapEntry{ Liveness: l, IsLive: false, } @@ -10316,7 +10315,7 @@ func TestFollowerQuiesceOnNotify(t *testing.T) { Epoch: 7, Expiration: hlc.LegacyTimestamp{WallTime: 8}, } - q.livenessMap[l.NodeID] = liveness.IsLiveMapEntry{ + q.livenessMap[l.NodeID] = livenesspb.IsLiveMapEntry{ Liveness: l, IsLive: false, } @@ -10332,7 +10331,7 @@ func TestFollowerQuiesceOnNotify(t *testing.T) { Epoch: 7, Expiration: hlc.LegacyTimestamp{WallTime: 8}, } - q.livenessMap[l.NodeID] = liveness.IsLiveMapEntry{ + q.livenessMap[l.NodeID] = livenesspb.IsLiveMapEntry{ Liveness: l, IsLive: false, } @@ -10347,7 +10346,7 @@ func TestFollowerQuiesceOnNotify(t *testing.T) { Epoch: 7, Expiration: hlc.LegacyTimestamp{WallTime: 8}, } - q.livenessMap[l.NodeID] = liveness.IsLiveMapEntry{ + q.livenessMap[l.NodeID] = livenesspb.IsLiveMapEntry{ Liveness: l, IsLive: false, } @@ -11446,10 +11445,10 @@ func TestReplicaShouldCampaignOnWake(t *testing.T) { }, NextReplicaID: 4, } - livenessMap := liveness.IsLiveMap{ - 1: liveness.IsLiveMapEntry{IsLive: true}, - 2: liveness.IsLiveMapEntry{IsLive: false}, - 4: liveness.IsLiveMapEntry{IsLive: false}, + livenessMap := livenesspb.IsLiveMap{ + 1: livenesspb.IsLiveMapEntry{IsLive: true}, + 2: livenesspb.IsLiveMapEntry{IsLive: false}, + 4: livenesspb.IsLiveMapEntry{IsLive: false}, } myLease := roachpb.Lease{ @@ -11515,7 +11514,7 @@ func TestReplicaShouldCampaignOnWake(t *testing.T) { tests := []struct { leaseStatus kvserverpb.LeaseStatus raftStatus raft.BasicStatus - livenessMap liveness.IsLiveMap + livenessMap livenesspb.IsLiveMap desc *roachpb.RangeDescriptor requiresExpiringLease bool exp bool @@ -11584,12 +11583,12 @@ func TestReplicaShouldCampaignOnLeaseRequestRedirect(t *testing.T) { } now := hlc.Timestamp{WallTime: 100} - livenessMap := liveness.IsLiveMap{ - 1: liveness.IsLiveMapEntry{ + livenessMap := livenesspb.IsLiveMap{ + 1: livenesspb.IsLiveMapEntry{ IsLive: true, Liveness: livenesspb.Liveness{Expiration: now.Add(1, 0).ToLegacyTimestamp()}, }, - 2: liveness.IsLiveMapEntry{ + 2: livenesspb.IsLiveMapEntry{ // NOTE: we purposefully set IsLive to true in disagreement with the // Liveness expiration to ensure that we're only looking at node liveness // in shouldCampaignOnLeaseRequestRedirect and not at whether this node is diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 1afc9e40ee6b..4899a27128ed 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/multiqueue" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed" @@ -3228,7 +3229,7 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { ) now := s.cfg.Clock.NowAsClockTimestamp() - var livenessMap liveness.IsLiveMap + var livenessMap livenesspb.IsLiveMap if s.cfg.NodeLiveness != nil { livenessMap = s.cfg.NodeLiveness.GetIsLiveMap() } diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index 3eccf9c7a897..51acf460fc74 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -17,7 +17,6 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -651,7 +650,7 @@ func (s *Store) processTick(_ context.Context, rangeID roachpb.RangeID) bool { if !ok { return false } - livenessMap, _ := s.livenessMap.Load().(liveness.IsLiveMap) + livenessMap, _ := s.livenessMap.Load().(livenesspb.IsLiveMap) ioThresholds := s.ioThresholds.Current() start := timeutil.Now() diff --git a/pkg/server/problem_ranges.go b/pkg/server/problem_ranges.go index 12fb4649ed6f..c95e04a30ddc 100644 --- a/pkg/server/problem_ranges.go +++ b/pkg/server/problem_ranges.go @@ -14,7 +14,7 @@ import ( "context" "sort" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "google.golang.org/grpc/codes" @@ -43,8 +43,8 @@ func (s *statusServer) ProblemRanges( if err != nil { return nil, status.Errorf(codes.InvalidArgument, err.Error()) } - isLiveMap = liveness.IsLiveMap{ - requestedNodeID: liveness.IsLiveMapEntry{IsLive: true}, + isLiveMap = livenesspb.IsLiveMap{ + requestedNodeID: livenesspb.IsLiveMapEntry{IsLive: true}, } } From 0fbeb5749bd76b2d806fa136f1b659ec96a6a8a0 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 14 Oct 2022 07:40:07 -0400 Subject: [PATCH 2/3] spanconfig: introduce spanconfig.Reporter This is KV-side API for multi-tenant replication reports (#89987) Release note: None --- pkg/BUILD.bazel | 4 + pkg/ccl/kvccl/kvtenantccl/connector.go | 25 ++ pkg/ccl/kvccl/kvtenantccl/connector_test.go | 6 + .../spanconfigcomparedccl/BUILD.bazel | 0 pkg/kv/kvclient/kvcoord/send_test.go | 6 + pkg/kv/kvclient/kvcoord/transport_test.go | 6 + pkg/kv/kvclient/kvtenant/connector.go | 4 + .../allocator/allocatorimpl/allocator.go | 44 ++- .../allocatorimpl/allocator_scorer_test.go | 15 +- .../allocator/allocatorimpl/allocator_test.go | 15 +- pkg/kv/kvserver/constraint/analyzer.go | 28 +- pkg/kv/kvserver/replica_metrics.go | 8 +- .../reports/replication_stats_report.go | 5 +- pkg/kv/kvserver/store.go | 3 +- pkg/roachpb/api.proto | 4 + pkg/roachpb/metadata.go | 11 + pkg/roachpb/metadata_replicas.go | 33 +- pkg/roachpb/roachpbmock/mocks_generated.go | 20 ++ pkg/roachpb/span_config.go | 3 - pkg/roachpb/span_config.proto | 23 ++ pkg/rpc/context_test.go | 6 + pkg/rpc/nodedialer/nodedialer_test.go | 6 + pkg/server/BUILD.bazel | 1 + pkg/server/node.go | 19 ++ pkg/server/server.go | 17 + pkg/server/serverpb/migration.proto | 3 + pkg/server/serverpb/status.proto | 1 + pkg/server/status.go | 22 +- pkg/server/testserver.go | 10 + pkg/spanconfig/spanconfig.go | 8 + pkg/spanconfig/spanconfigreporter/BUILD.bazel | 55 +++ .../spanconfigreporter/datadriven_test.go | 318 ++++++++++++++++++ pkg/spanconfig/spanconfigreporter/disabled.go | 34 ++ .../spanconfigreporter/main_test.go | 33 ++ pkg/spanconfig/spanconfigreporter/reporter.go | 181 ++++++++++ .../spanconfigreporter/testdata/basic | 189 +++++++++++ .../testdata/constraint_conformance | 87 +++++ .../testdata/joint_consensus | 104 ++++++ .../testdata/over_under_replicated | 120 +++++++ .../spanconfigtestutils/BUILD.bazel | 2 + pkg/spanconfig/spanconfigtestutils/utils.go | 94 ++++++ pkg/testutils/serverutils/test_tenant_shim.go | 4 + 42 files changed, 1509 insertions(+), 68 deletions(-) delete mode 100644 pkg/ccl/spanconfigccl/spanconfigcomparedccl/BUILD.bazel create mode 100644 pkg/spanconfig/spanconfigreporter/BUILD.bazel create mode 100644 pkg/spanconfig/spanconfigreporter/datadriven_test.go create mode 100644 pkg/spanconfig/spanconfigreporter/disabled.go create mode 100644 pkg/spanconfig/spanconfigreporter/main_test.go create mode 100644 pkg/spanconfig/spanconfigreporter/reporter.go create mode 100644 pkg/spanconfig/spanconfigreporter/testdata/basic create mode 100644 pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance create mode 100644 pkg/spanconfig/spanconfigreporter/testdata/joint_consensus create mode 100644 pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 387686d27c57..5a538f97d615 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -269,6 +269,7 @@ ALL_TESTS = [ "//pkg/spanconfig/spanconfigmanager:spanconfigmanager_test", "//pkg/spanconfig/spanconfigptsreader:spanconfigptsreader_test", "//pkg/spanconfig/spanconfigreconciler:spanconfigreconciler_test", + "//pkg/spanconfig/spanconfigreporter:spanconfigreporter_test", "//pkg/spanconfig/spanconfigsqltranslator:spanconfigsqltranslator_test", "//pkg/spanconfig/spanconfigsqlwatcher:spanconfigsqlwatcher_test", "//pkg/spanconfig/spanconfigstore:spanconfigstore_test", @@ -1353,6 +1354,8 @@ GO_TARGETS = [ "//pkg/spanconfig/spanconfigptsreader:spanconfigptsreader_test", "//pkg/spanconfig/spanconfigreconciler:spanconfigreconciler", "//pkg/spanconfig/spanconfigreconciler:spanconfigreconciler_test", + "//pkg/spanconfig/spanconfigreporter:spanconfigreporter", + "//pkg/spanconfig/spanconfigreporter:spanconfigreporter_test", "//pkg/spanconfig/spanconfigsplitter:spanconfigsplitter", "//pkg/spanconfig/spanconfigsqltranslator:spanconfigsqltranslator", "//pkg/spanconfig/spanconfigsqltranslator:spanconfigsqltranslator_test", @@ -2612,6 +2615,7 @@ GET_X_DATA_TARGETS = [ "//pkg/spanconfig/spanconfigmanager:get_x_data", "//pkg/spanconfig/spanconfigptsreader:get_x_data", "//pkg/spanconfig/spanconfigreconciler:get_x_data", + "//pkg/spanconfig/spanconfigreporter:get_x_data", "//pkg/spanconfig/spanconfigsplitter:get_x_data", "//pkg/spanconfig/spanconfigsqltranslator:get_x_data", "//pkg/spanconfig/spanconfigsqlwatcher:get_x_data", diff --git a/pkg/ccl/kvccl/kvtenantccl/connector.go b/pkg/ccl/kvccl/kvtenantccl/connector.go index fa7bd6aa7532..4ee7f1a93618 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector.go @@ -129,6 +129,10 @@ var _ serverpb.TenantStatusServer = (*Connector)(nil) // Connector is capable of accessing span configurations for secondary tenants. var _ spanconfig.KVAccessor = (*Connector)(nil) +// Reporter is capable of generating span configuration conformance reports for +// secondary tenants. +var _ spanconfig.Reporter = (*Connector)(nil) + // NewConnector creates a new Connector. // NOTE: Calling Start will set cfg.RPCContext.ClusterID. func NewConnector(cfg kvtenant.ConnectorConfig, addrs []string) *Connector { @@ -526,6 +530,27 @@ func (c *Connector) UpdateSpanConfigRecords( }) } +// SpanConfigConformance implements the spanconfig.Reporter interface. +func (c *Connector) SpanConfigConformance( + ctx context.Context, spans []roachpb.Span, +) (roachpb.SpanConfigConformanceReport, error) { + var report roachpb.SpanConfigConformanceReport + if err := c.withClient(ctx, func(ctx context.Context, c *client) error { + resp, err := c.SpanConfigConformance(ctx, &roachpb.SpanConfigConformanceRequest{ + Spans: spans, + }) + if err != nil { + return err + } + + report = resp.Report + return nil + }); err != nil { + return roachpb.SpanConfigConformanceReport{}, err + } + return report, nil +} + // GetAllSystemSpanConfigsThatApply implements the spanconfig.KVAccessor // interface. func (c *Connector) GetAllSystemSpanConfigsThatApply( diff --git a/pkg/ccl/kvccl/kvtenantccl/connector_test.go b/pkg/ccl/kvccl/kvtenantccl/connector_test.go index 2b3499ad7289..e795c15c0444 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector_test.go @@ -125,6 +125,12 @@ func (m *mockServer) UpdateSpanConfigs( panic("unimplemented") } +func (m *mockServer) SpanConfigConformance( + context.Context, *roachpb.SpanConfigConformanceRequest, +) (*roachpb.SpanConfigConformanceResponse, error) { + panic("unimplemented") +} + func gossipEventForClusterID(clusterID uuid.UUID) *roachpb.GossipSubscriptionEvent { return &roachpb.GossipSubscriptionEvent{ Key: gossip.KeyClusterID, diff --git a/pkg/ccl/spanconfigccl/spanconfigcomparedccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfigcomparedccl/BUILD.bazel deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/pkg/kv/kvclient/kvcoord/send_test.go b/pkg/kv/kvclient/kvcoord/send_test.go index 3892ad88c799..3a96cb12e136 100644 --- a/pkg/kv/kvclient/kvcoord/send_test.go +++ b/pkg/kv/kvclient/kvcoord/send_test.go @@ -101,6 +101,12 @@ func (n Node) UpdateSpanConfigs( panic("unimplemented") } +func (n Node) SpanConfigConformance( + context.Context, *roachpb.SpanConfigConformanceRequest, +) (*roachpb.SpanConfigConformanceResponse, error) { + panic("implement me") +} + func (n Node) TenantSettings( *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, ) error { diff --git a/pkg/kv/kvclient/kvcoord/transport_test.go b/pkg/kv/kvclient/kvcoord/transport_test.go index 6a1f77292614..5c4764c773eb 100644 --- a/pkg/kv/kvclient/kvcoord/transport_test.go +++ b/pkg/kv/kvclient/kvcoord/transport_test.go @@ -218,6 +218,12 @@ func (m *mockInternalClient) GetSpanConfigs( return nil, fmt.Errorf("unsupported GetSpanConfigs call") } +func (m *mockInternalClient) SpanConfigConformance( + _ context.Context, _ *roachpb.SpanConfigConformanceRequest, _ ...grpc.CallOption, +) (*roachpb.SpanConfigConformanceResponse, error) { + return nil, fmt.Errorf("unsupported SpanConfigConformance call") +} + func (m *mockInternalClient) GetAllSystemSpanConfigsThatApply( context.Context, *roachpb.GetAllSystemSpanConfigsThatApplyRequest, ...grpc.CallOption, ) (*roachpb.GetAllSystemSpanConfigsThatApplyResponse, error) { diff --git a/pkg/kv/kvclient/kvtenant/connector.go b/pkg/kv/kvclient/kvtenant/connector.go index 5e2dfb29287e..891bff372e1b 100644 --- a/pkg/kv/kvclient/kvtenant/connector.go +++ b/pkg/kv/kvclient/kvtenant/connector.go @@ -76,6 +76,10 @@ type Connector interface { // applicable to secondary tenants. spanconfig.KVAccessor + // Reporter provides access to conformance reports, i.e. whether ranges + // backing queried keyspans conform the span configs that apply to them. + spanconfig.Reporter + // OverridesMonitor provides access to tenant cluster setting overrides. settingswatcher.OverridesMonitor diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index ddd6881f3f53..6711804360b1 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -1017,10 +1017,18 @@ func (a *Allocator) AllocateTargetFromList( targetType TargetReplicaType, ) (roachpb.ReplicationTarget, string) { existingReplicas := append(existingVoters, existingNonVoters...) - analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.StorePool.GetStoreDescriptor, - existingReplicas, conf.NumReplicas, conf.Constraints) - analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.StorePool.GetStoreDescriptor, - existingVoters, conf.GetNumVoters(), conf.VoterConstraints) + analyzedOverallConstraints := constraint.AnalyzeConstraints( + a.StorePool, + existingReplicas, + conf.NumReplicas, + conf.Constraints, + ) + analyzedVoterConstraints := constraint.AnalyzeConstraints( + a.StorePool, + existingVoters, + conf.GetNumVoters(), + conf.VoterConstraints, + ) var constraintsChecker constraintsCheckFn switch t := targetType; t { @@ -1159,10 +1167,18 @@ func (a Allocator) RemoveTarget( } existingReplicas := append(existingVoters, existingNonVoters...) - analyzedOverallConstraints := constraint.AnalyzeConstraints(ctx, a.StorePool.GetStoreDescriptor, - existingReplicas, conf.NumReplicas, conf.Constraints) - analyzedVoterConstraints := constraint.AnalyzeConstraints(ctx, a.StorePool.GetStoreDescriptor, - existingVoters, conf.GetNumVoters(), conf.VoterConstraints) + analyzedOverallConstraints := constraint.AnalyzeConstraints( + a.StorePool, + existingReplicas, + conf.NumReplicas, + conf.Constraints, + ) + analyzedVoterConstraints := constraint.AnalyzeConstraints( + a.StorePool, + existingVoters, + conf.GetNumVoters(), + conf.VoterConstraints, + ) var constraintsChecker constraintsCheckFn switch t := targetType; t { @@ -1294,9 +1310,17 @@ func (a Allocator) RebalanceTarget( zero := roachpb.ReplicationTarget{} analyzedOverallConstraints := constraint.AnalyzeConstraints( - ctx, a.StorePool.GetStoreDescriptor, existingReplicas, conf.NumReplicas, conf.Constraints) + a.StorePool, + existingReplicas, + conf.NumReplicas, + conf.Constraints, + ) analyzedVoterConstraints := constraint.AnalyzeConstraints( - ctx, a.StorePool.GetStoreDescriptor, existingVoters, conf.GetNumVoters(), conf.VoterConstraints) + a.StorePool, + existingVoters, + conf.GetNumVoters(), + conf.VoterConstraints, + ) var removalConstraintsChecker constraintsCheckFn var rebalanceConstraintsChecker rebalanceConstraintsCheckFn var replicaSetToRebalance, replicasWithExcludedStores []roachpb.ReplicaDescriptor diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer_test.go index f0d49c02eb34..cf480fe5c394 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer_test.go @@ -583,7 +583,13 @@ var ( } ) -func getTestStoreDesc(storeID roachpb.StoreID) (roachpb.StoreDescriptor, bool) { +type mockStoreResolver struct{} + +var _ constraint.StoreResolver = mockStoreResolver{} + +func (m mockStoreResolver) GetStoreDescriptor( + storeID roachpb.StoreID, +) (roachpb.StoreDescriptor, bool) { desc, ok := testStores[storeID] return desc, ok } @@ -936,9 +942,7 @@ func TestAllocateConstraintsCheck(t *testing.T) { Constraints: tc.constraints, NumReplicas: tc.numReplicas, } - analyzed := constraint.AnalyzeConstraints( - context.Background(), getTestStoreDesc, testStoreReplicas(tc.existing), - conf.NumReplicas, conf.Constraints) + analyzed := constraint.AnalyzeConstraints(mockStoreResolver{}, testStoreReplicas(tc.existing), conf.NumReplicas, conf.Constraints) for _, s := range testStores { valid, necessary := allocateConstraintsCheck(s, analyzed) if e, a := tc.expectedValid[s.StoreID], valid; e != a { @@ -1071,8 +1075,7 @@ func TestRemoveConstraintsCheck(t *testing.T) { Constraints: tc.constraints, NumReplicas: tc.numReplicas, } - analyzed := constraint.AnalyzeConstraints( - context.Background(), getTestStoreDesc, existing, conf.NumReplicas, conf.Constraints) + analyzed := constraint.AnalyzeConstraints(mockStoreResolver{}, 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/allocatorimpl/allocator_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go index df8207b30d2a..3850f5201084 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -3390,9 +3390,7 @@ func TestAllocateCandidatesExcludeNonReadyNodes(t *testing.T) { } // No constraints. conf := roachpb.SpanConfig{} - analyzed := constraint.AnalyzeConstraints( - ctx, a.StorePool.GetStoreDescriptor, existingRepls, conf.NumReplicas, - conf.Constraints) + analyzed := constraint.AnalyzeConstraints(a.StorePool, existingRepls, conf.NumReplicas, conf.Constraints) allocationConstraintsChecker := voterConstraintsCheckerForAllocation(analyzed, constraint.EmptyAnalyzedConstraints) removalConstraintsChecker := voterConstraintsCheckerForRemoval(analyzed, constraint.EmptyAnalyzedConstraints) rebalanceConstraintsChecker := voterConstraintsCheckerForRebalance(analyzed, constraint.EmptyAnalyzedConstraints) @@ -3747,9 +3745,7 @@ func TestAllocateCandidatesNumReplicasConstraints(t *testing.T) { } } conf := roachpb.SpanConfig{Constraints: tc.constraints} - analyzed := constraint.AnalyzeConstraints( - ctx, a.StorePool.GetStoreDescriptor, existingRepls, conf.NumReplicas, - conf.Constraints) + analyzed := constraint.AnalyzeConstraints(a.StorePool, existingRepls, conf.NumReplicas, conf.Constraints) checkFn := voterConstraintsCheckerForAllocation(analyzed, constraint.EmptyAnalyzedConstraints) candidates := rankedCandidateListForAllocation( @@ -3979,8 +3975,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { StoreID: storeID, } } - analyzed := constraint.AnalyzeConstraints(ctx, a.StorePool.GetStoreDescriptor, existingRepls, - 0 /* numReplicas */, tc.constraints) + analyzed := constraint.AnalyzeConstraints(a.StorePool, existingRepls, 0, tc.constraints) // Check behavior in a span config where `voter_constraints` are empty. checkFn := voterConstraintsCheckerForRemoval(analyzed, constraint.EmptyAnalyzedConstraints) @@ -5186,9 +5181,7 @@ func TestRebalanceCandidatesNumReplicasConstraints(t *testing.T) { Constraints: tc.constraints, NumReplicas: tc.numReplicas, } - analyzed := constraint.AnalyzeConstraints( - ctx, a.StorePool.GetStoreDescriptor, existingRepls, - conf.NumReplicas, conf.Constraints) + analyzed := constraint.AnalyzeConstraints(a.StorePool, existingRepls, conf.NumReplicas, conf.Constraints) removalConstraintsChecker := voterConstraintsCheckerForRemoval( analyzed, constraint.EmptyAnalyzedConstraints, diff --git a/pkg/kv/kvserver/constraint/analyzer.go b/pkg/kv/kvserver/constraint/analyzer.go index 90f0b5970fc6..5b3ead151f41 100644 --- a/pkg/kv/kvserver/constraint/analyzer.go +++ b/pkg/kv/kvserver/constraint/analyzer.go @@ -10,19 +10,15 @@ package constraint -import ( - "context" +import "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" -) - -// AnalyzedConstraints represents the result or AnalyzeConstraints(). It -// combines a zone's constraints with information about which stores satisfy -// what term of the constraints disjunction. +// AnalyzedConstraints represents the result of AnalyzeConstraints(). It +// combines a span config's constraints with information about which stores +// satisfy what term of the constraints disjunction. type AnalyzedConstraints struct { 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). + // replicas in the range (sum(constraints.NumReplicas) < config.NumReplicas). // In such cases, we allow replicas that don't match any of the per-replica // constraints, but never mark them as necessary. UnconstrainedReplicas bool @@ -39,13 +35,17 @@ type AnalyzedConstraints struct { // satisfied by any given configuration of replicas. var EmptyAnalyzedConstraints = AnalyzedConstraints{} -// AnalyzeConstraints processes the zone config constraints that apply to a +// StoreResolver resolves a store descriptor by a given ID. +type StoreResolver interface { + GetStoreDescriptor(storeID roachpb.StoreID) (roachpb.StoreDescriptor, bool) +} + +// AnalyzeConstraints processes the span config constraints that apply to a // range along with the current replicas for a range, spitting back out // information about which constraints are satisfied by which replicas and // which replicas satisfy which constraints, aiding in allocation decisions. func AnalyzeConstraints( - ctx context.Context, - getStoreDescFn func(roachpb.StoreID) (roachpb.StoreDescriptor, bool), + storeResolver StoreResolver, existing []roachpb.ReplicaDescriptor, numReplicas int32, constraints []roachpb.ConstraintsConjunction, @@ -67,7 +67,7 @@ func AnalyzeConstraints( // happen once a node is hooked into gossip), trust that it's valid. This // is a much more stable failure state than frantically moving everything // off such a node. - store, ok := getStoreDescFn(repl.StoreID) + store, ok := storeResolver.GetStoreDescriptor(repl.StoreID) if !ok || ConjunctionsCheck(store, subConstraints.Constraints) { result.SatisfiedBy[i] = append(result.SatisfiedBy[i], store.StoreID) result.Satisfies[store.StoreID] = append(result.Satisfies[store.StoreID], i) @@ -82,7 +82,7 @@ func AnalyzeConstraints( // ConjunctionsCheck checks a store against a single set of constraints (out of // 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 +// store matches the constraints. The constraints are AND'ed together; a store // matches the conjunction if it matches all of them. func ConjunctionsCheck(store roachpb.StoreDescriptor, constraints []roachpb.Constraint) bool { for _, constraint := range constraints { diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index 1cae44da5804..4f4fc81e1908 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -220,10 +220,10 @@ func calcRangeCounter( status := desc.Replicas().ReplicationStatus(func(rDesc roachpb.ReplicaDescriptor) bool { return livenessMap[rDesc.NodeID].IsLive }, - // neededVoters - we don't care about the under/over-replication - // determinations from the report because it's too magic. We'll do our own - // determination below. - 0) + // needed{Voters,NonVoters} - we don't care about the + // under/over-replication determinations from the report because + // it's too magic. We'll do our own determination below. + 0, 0) unavailable = !status.Available liveVoters := calcLiveVoterReplicas(desc, livenessMap) liveNonVoters := calcLiveNonVoterReplicas(desc, livenessMap) diff --git a/pkg/kv/kvserver/reports/replication_stats_report.go b/pkg/kv/kvserver/reports/replication_stats_report.go index 6644448f1065..863f91d097eb 100644 --- a/pkg/kv/kvserver/reports/replication_stats_report.go +++ b/pkg/kv/kvserver/reports/replication_stats_report.go @@ -401,7 +401,10 @@ func (v *replicationStatsVisitor) countRange( ) { status := r.Replicas().ReplicationStatus(func(rDesc roachpb.ReplicaDescriptor) bool { return v.nodeChecker(rDesc.NodeID) - }, replicationFactor) + // NB: this reporting code was written before ReplicationStatus reported + // on non-voting replicas. This code will also soon be removed in favor + // of something that works with multi-tenancy (#89987). + }, replicationFactor, 0) // Note that a range can be under-replicated and over-replicated at the same // time if it has many replicas, but sufficiently many of them are on dead // nodes. diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 4899a27128ed..3b1a9d7e3658 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -1098,7 +1098,8 @@ type StoreConfig struct { // SpanConfigsDisabled determines whether we're able to use the span configs // infrastructure or not. - // TODO(richardjcai): We can likely remove this. + // + // TODO(irfansharif): We can remove this. SpanConfigsDisabled bool // Used to subscribe to span configuration changes, keeping up-to-date a // data structure useful for retrieving span configs. Only available if diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index c9c0754504fd..65294a19b295 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -3029,6 +3029,10 @@ service Internal { // keyspans. rpc UpdateSpanConfigs (UpdateSpanConfigsRequest) returns (UpdateSpanConfigsResponse) { } + // SpanConfigConformance is used to determine whether ranges backing the given + // keyspans conform to span configs that apply over them. + rpc SpanConfigConformance (SpanConfigConformanceRequest) returns (SpanConfigConformanceResponse) { } + // TenantSettings is used by tenants to obtain and stay up to date with tenant // setting overrides. rpc TenantSettings (TenantSettingsRequest) returns (stream TenantSettingsEvent) { } diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index 882a7331d1d9..ff09e7730fc9 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -483,6 +483,17 @@ func (r ReplicaDescriptor) IsAnyVoter() bool { } } +// IsNonVoter returns true if the replica is a non-voter. Can be used as a +// filter for ReplicaDescriptors.Filter. +func (r ReplicaDescriptor) IsNonVoter() bool { + switch r.Type { + case NON_VOTER: + return true + default: + return false + } +} + // PercentilesFromData derives percentiles from a slice of data points. // Sorts the input data if it isn't already sorted. func PercentilesFromData(data []float64) Percentiles { diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index 280dcff656c8..4401237f15d4 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -378,7 +378,7 @@ func (d ReplicaSet) HasReplicaOnNode(nodeID NodeID) bool { // the replication layer. This is more complicated than just counting the number // of replicas due to the existence of joint quorums. func (d ReplicaSet) CanMakeProgress(liveFunc func(descriptor ReplicaDescriptor) bool) bool { - return d.ReplicationStatus(liveFunc, 0 /* neededVoters */).Available + return d.ReplicationStatus(liveFunc, 0 /* neededVoters */, 0 /* neededNonVoters*/).Available } // RangeStatusReport contains info about a range's replication status. Returned @@ -389,26 +389,32 @@ type RangeStatusReport struct { Available bool // UnderReplicated is set if the range is considered under-replicated // according to the desired replication factor and the replica liveness info - // passed to ReplicationStatus. Dead replicas are considered to be missing. + // passed to ReplicationStatus. Only voting replicas are counted here. Dead + // replicas are considered to be missing. UnderReplicated bool - // UnderReplicated is set if the range is considered under-replicated + // OverReplicated is set if the range is considered over-replicated // according to the desired replication factor passed to ReplicationStatus. - // Replica liveness is not considered. + // Only voting replicas are counted here. Replica liveness is not + // considered. // // Note that a range can be under-replicated and over-replicated at the same // time if it has many replicas, but sufficiently many of them are on dead // nodes. OverReplicated bool + // {Under,Over}ReplicatedNonVoters are like their {Under,Over}Replicated + // counterparts but applying only to non-voters. + UnderReplicatedNonVoters, OverReplicatedNonVoters bool } // ReplicationStatus returns availability and over/under-replication // determinations for the range. // -// replicationFactor is the replica's desired replication for purposes of -// determining over/under-replication. 0 can be passed if the caller is only -// interested in availability and not interested in the other report fields. +// neededVoters is the replica's desired replication for purposes of determining +// over/under-replication of voters. If the caller is only interested in +// availability of voting replicas, 0 can be passed in. neededNonVoters is the +// counterpart for non-voting replicas. func (d ReplicaSet) ReplicationStatus( - liveFunc func(descriptor ReplicaDescriptor) bool, neededVoters int, + liveFunc func(descriptor ReplicaDescriptor) bool, neededVoters int, neededNonVoters int, ) RangeStatusReport { var res RangeStatusReport // isBoth takes two replica predicates and returns their conjunction. @@ -440,13 +446,22 @@ func (d ReplicaSet) ReplicationStatus( res.Available = availableIncomingGroup && availableOutgoingGroup - // Determine over/under-replication. Note that learners don't matter. + // Determine over/under-replication of voting replicas. Note that learners + // don't matter. underReplicatedOldGroup := len(liveVotersOldGroup) < neededVoters underReplicatedNewGroup := len(liveVotersNewGroup) < neededVoters overReplicatedOldGroup := len(votersOldGroup) > neededVoters overReplicatedNewGroup := len(votersNewGroup) > neededVoters res.UnderReplicated = underReplicatedOldGroup || underReplicatedNewGroup res.OverReplicated = overReplicatedOldGroup || overReplicatedNewGroup + if neededNonVoters == 0 { + return res + } + + nonVoters := d.FilterToDescriptors(ReplicaDescriptor.IsNonVoter) + liveNonVoters := d.FilterToDescriptors(isBoth(ReplicaDescriptor.IsNonVoter, liveFunc)) + res.UnderReplicatedNonVoters = len(liveNonVoters) < neededNonVoters + res.OverReplicatedNonVoters = len(nonVoters) > neededNonVoters return res } diff --git a/pkg/roachpb/roachpbmock/mocks_generated.go b/pkg/roachpb/roachpbmock/mocks_generated.go index 377c7b7b0fcb..68143d9dbe83 100644 --- a/pkg/roachpb/roachpbmock/mocks_generated.go +++ b/pkg/roachpb/roachpbmock/mocks_generated.go @@ -217,6 +217,26 @@ func (mr *MockInternalClientMockRecorder) ResetQuorum(arg0, arg1 interface{}, ar return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetQuorum", reflect.TypeOf((*MockInternalClient)(nil).ResetQuorum), varargs...) } +// SpanConfigConformance mocks base method. +func (m *MockInternalClient) SpanConfigConformance(arg0 context.Context, arg1 *roachpb.SpanConfigConformanceRequest, arg2 ...grpc.CallOption) (*roachpb.SpanConfigConformanceResponse, error) { + m.ctrl.T.Helper() + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "SpanConfigConformance", varargs...) + ret0, _ := ret[0].(*roachpb.SpanConfigConformanceResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SpanConfigConformance indicates an expected call of SpanConfigConformance. +func (mr *MockInternalClientMockRecorder) SpanConfigConformance(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{arg0, arg1}, arg2...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SpanConfigConformance", reflect.TypeOf((*MockInternalClient)(nil).SpanConfigConformance), varargs...) +} + // TenantSettings mocks base method. func (m *MockInternalClient) TenantSettings(arg0 context.Context, arg1 *roachpb.TenantSettingsRequest, arg2 ...grpc.CallOption) (roachpb.Internal_TenantSettingsClient, error) { m.ctrl.T.Helper() diff --git a/pkg/roachpb/span_config.go b/pkg/roachpb/span_config.go index 02c621ea8ba1..ee7a933b06fd 100644 --- a/pkg/roachpb/span_config.go +++ b/pkg/roachpb/span_config.go @@ -97,9 +97,6 @@ func (s *SpanConfig) ValidateSystemTargetSpanConfig() error { // GetNumVoters returns the number of voting replicas as defined in the // span config. -// TODO(arul): We can get rid of this now that we're correctly populating -// -// numVoters when going from ZoneConfigs -> SpanConfigs. func (s *SpanConfig) GetNumVoters() int32 { if s.NumVoters != 0 { return s.NumVoters diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index 3a75e58efca3..dc298ac6bae2 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -14,6 +14,7 @@ option go_package = "roachpb"; import "errorspb/errors.proto"; import "roachpb/data.proto"; +import "roachpb/metadata.proto"; import "gogoproto/gogo.proto"; import "util/hlc/timestamp.proto"; @@ -277,6 +278,15 @@ message SpanConfigEntry { SpanConfig config = 2 [(gogoproto.nullable) = false]; }; +// SpanConfigConformanceReport lists out ranges that (i) don't conform to span +// configs that apply over them, and (ii) are unavailable. +message SpanConfigConformanceReport { + repeated RangeDescriptor under_replicated = 1 [(gogoproto.nullable) = false]; + repeated RangeDescriptor over_replicated = 2 [(gogoproto.nullable) = false]; + repeated RangeDescriptor violating_constraints = 3 [(gogoproto.nullable) = false]; + repeated RangeDescriptor unavailable = 4 [(gogoproto.nullable) = false]; +}; + // GetSpanConfigsRequest is used to fetch the span configurations and system // span configurations. message GetSpanConfigsRequest { @@ -344,6 +354,19 @@ message UpdateSpanConfigsResponse { errorspb.EncodedError error = 1 [(gogoproto.nullable) = false]; }; +// SpanConfigConformanceRequest is used to determine whether ranges backing the +// given keyspans conform to the span configs that apply over them. +message SpanConfigConformanceRequest { + // Spans to request the conformance data for. The spans listed here are not + // allowed to overlap with one another. + repeated Span spans = 1 [(gogoproto.nullable) = false]; +}; + +// SpanConfigConformanceResponse lists out ranges that (i) don't conform to span +// configs that apply over them, and (ii) are unavailable. +message SpanConfigConformanceResponse { + SpanConfigConformanceReport report = 1 [(gogoproto.nullable) = false]; +}; // GetAllSystemSpanConfigsThatApplyRequest is used to fetch all system span // configs that apply to a given tenant. For a specific tenant range, this diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index ab7fcfc41700..eed1d567e47d 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -425,6 +425,12 @@ func (*internalServer) UpdateSpanConfigs( panic("unimplemented") } +func (s *internalServer) SpanConfigConformance( + context.Context, *roachpb.SpanConfigConformanceRequest, +) (*roachpb.SpanConfigConformanceResponse, error) { + panic("unimplemented") +} + func (*internalServer) TenantSettings( *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, ) error { diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index 0d0baf7b1569..f27907f2fced 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -621,6 +621,12 @@ func (*internalServer) UpdateSpanConfigs( panic("unimplemented") } +func (s *internalServer) SpanConfigConformance( + context.Context, *roachpb.SpanConfigConformanceRequest, +) (*roachpb.SpanConfigConformanceResponse, error) { + panic("unimplemented") +} + func (*internalServer) TenantSettings( *roachpb.TenantSettingsRequest, roachpb.Internal_TenantSettingsServer, ) error { diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 5c0e4f4374bc..307bc74c0669 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -158,6 +158,7 @@ go_library( "//pkg/spanconfig/spanconfigmanager", "//pkg/spanconfig/spanconfigptsreader", "//pkg/spanconfig/spanconfigreconciler", + "//pkg/spanconfig/spanconfigreporter", "//pkg/spanconfig/spanconfigsplitter", "//pkg/spanconfig/spanconfigsqltranslator", "//pkg/spanconfig/spanconfigsqlwatcher", diff --git a/pkg/server/node.go b/pkg/server/node.go index dc687088da23..e9af8266d06c 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -243,6 +243,8 @@ type Node struct { spanConfigAccessor spanconfig.KVAccessor // powers the span configuration RPCs + spanConfigReporter spanconfig.Reporter // powers the span configuration RPCs + // Turns `Node.writeNodeStatus` into a no-op. This is a hack to enable the // COCKROACH_DEBUG_TS_IMPORT_FILE env var. suppressNodeStatus syncutil.AtomicBool @@ -372,6 +374,7 @@ func NewNode( tenantUsage multitenant.TenantUsageServer, tenantSettingsWatcher *tenantsettingswatcher.Watcher, spanConfigAccessor spanconfig.KVAccessor, + spanConfigReporter spanconfig.Reporter, ) *Node { n := &Node{ storeCfg: cfg, @@ -385,6 +388,7 @@ func NewNode( tenantUsage: tenantUsage, tenantSettingsWatcher: tenantSettingsWatcher, spanConfigAccessor: spanConfigAccessor, + spanConfigReporter: spanConfigReporter, testingErrorEvent: cfg.TestingKnobs.TestingResponseErrorEvent, } n.storeCfg.KVAdmissionController = kvadmission.MakeController( @@ -1904,3 +1908,18 @@ func (n *Node) UpdateSpanConfigs( } return &roachpb.UpdateSpanConfigsResponse{}, nil } + +// SpanConfigConformance implements the roachpb.InternalServer interface. +func (n *Node) SpanConfigConformance( + ctx context.Context, req *roachpb.SpanConfigConformanceRequest, +) (*roachpb.SpanConfigConformanceResponse, error) { + if n.storeCfg.SpanConfigSubscriber.LastUpdated().IsEmpty() { + return nil, errors.Newf("haven't (yet) subscribed to span configs") + } + + report, err := n.spanConfigReporter.SpanConfigConformance(ctx, req.Spans) + if err != nil { + return nil, err + } + return &roachpb.SpanConfigConformanceResponse{Report: report}, nil +} diff --git a/pkg/server/server.go b/pkg/server/server.go index 55ee5203e108..f3a7a127c806 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -65,6 +65,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvsubscriber" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigreporter" "github.com/cockroachdb/cockroach/pkg/sql" _ "github.com/cockroachdb/cockroach/pkg/sql/catalog/schematelemetry" // register schedules declared outside of pkg/sql "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" @@ -88,6 +89,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/netutil" + "github.com/cockroachdb/cockroach/pkg/util/rangedesciter" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/schedulerlatency" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -571,6 +573,8 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { // kvAccessor powers the span configuration RPCs and the host tenant's // reconciliation job. kvAccessor spanconfig.KVAccessor + // reporter is used to report over span config conformance. + reporter spanconfig.Reporter // subscriber is used by stores to subscribe to span configuration updates. subscriber spanconfig.KVSubscriber // kvAccessorForTenantRecords is when creating/destroying secondary @@ -621,6 +625,14 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { spanConfigKnobs, ) spanConfig.kvAccessor, spanConfig.kvAccessorForTenantRecords = scKVAccessor, scKVAccessor + spanConfig.reporter = spanconfigreporter.New( + nodeLiveness, + storePool, + spanConfig.subscriber, + rangedesciter.New(db), + cfg.Settings, + spanConfigKnobs, + ) } else { // If the spanconfigs infrastructure is disabled, there should be no // reconciliation jobs or RPCs issued against the infrastructure. Plug @@ -628,6 +640,9 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { // unexpected use. spanConfig.kvAccessor = spanconfigkvaccessor.DisabledKVAccessor + // Ditto for the spanconfig.Reporter. + spanConfig.reporter = spanconfigreporter.DisabledReporter + // Use a no-op accessor where tenant records are created/destroyed. spanConfig.kvAccessorForTenantRecords = spanconfigkvaccessor.NoopKVAccessor @@ -727,6 +742,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { tenantUsage, tenantSettingsWatcher, spanConfig.kvAccessor, + spanConfig.reporter, ) roachpb.RegisterInternalServer(grpcServer.Server, node) kvserver.RegisterPerReplicaServer(grpcServer.Server, node.perReplicaServer) @@ -804,6 +820,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { closedSessionCache, remoteFlowRunner, internalExecutor, + spanConfig.reporter, ) // Instantiate the KV prober. diff --git a/pkg/server/serverpb/migration.proto b/pkg/server/serverpb/migration.proto index 38ee1302fc17..3fd0af2a3407 100644 --- a/pkg/server/serverpb/migration.proto +++ b/pkg/server/serverpb/migration.proto @@ -87,5 +87,8 @@ service Migration { // WaitForSpanConfigSubscription waits until the target node is wholly // subscribed to the global span configurations state. + // + // TODO(irfansharif): This can be removed -- 22.2 nodes will never issue this + // RPC. rpc WaitForSpanConfigSubscription (WaitForSpanConfigSubscriptionRequest) returns (WaitForSpanConfigSubscriptionResponse) { } } diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index c1900f91d51a..17f49752bbee 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -19,6 +19,7 @@ import "jobs/jobspb/jobs.proto"; import "roachpb/app_stats.proto"; import "roachpb/data.proto"; import "roachpb/index_usage_stats.proto"; +import "roachpb/span_config.proto"; import "roachpb/metadata.proto"; import "server/diagnostics/diagnosticspb/diagnostics.proto"; import "server/serverpb/index_recommendations.proto"; diff --git a/pkg/server/status.go b/pkg/server/status.go index 18030e1b1fec..b45d67feaa49 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -52,6 +52,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -500,6 +501,7 @@ type statusServer struct { si systemInfoOnce stmtDiagnosticsRequester StmtDiagnosticsRequester internalExecutor *sql.InternalExecutor + spanConfigReporter spanconfig.Reporter } // StmtDiagnosticsRequester is the interface into *stmtdiagnostics.Registry @@ -558,6 +560,7 @@ func newStatusServer( closedSessionCache *sql.ClosedSessionCache, remoteFlowRunner *flowinfra.RemoteFlowRunner, internalExecutor *sql.InternalExecutor, + spanConfigReporter spanconfig.Reporter, ) *statusServer { ambient.AddLogTag("status", nil) server := &statusServer{ @@ -571,15 +574,16 @@ func newStatusServer( rpcCtx: rpcCtx, stopper: stopper, }, - cfg: cfg, - admin: adminServer, - db: db, - gossip: gossip, - metricSource: metricSource, - nodeLiveness: nodeLiveness, - storePool: storePool, - stores: stores, - internalExecutor: internalExecutor, + cfg: cfg, + admin: adminServer, + db: db, + gossip: gossip, + metricSource: metricSource, + nodeLiveness: nodeLiveness, + storePool: storePool, + stores: stores, + internalExecutor: internalExecutor, + spanConfigReporter: spanConfigReporter, } return server diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 22a281651c43..b6ede3c5b07d 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -725,6 +725,11 @@ func (t *TestTenant) SpanConfigKVAccessor() interface{} { return t.SQLServer.tenantConnect } +// SpanConfigReporter is part TestTenantInterface. +func (t *TestTenant) SpanConfigReporter() interface{} { + return t.SQLServer.tenantConnect +} + // SpanConfigReconciler is part TestTenantInterface. func (t *TestTenant) SpanConfigReconciler() interface{} { return t.SQLServer.spanconfigMgr.Reconciler @@ -1161,6 +1166,11 @@ func (ts *TestServer) SpanConfigKVAccessor() interface{} { return ts.Server.node.spanConfigAccessor } +// SpanConfigReporter is part of TestServerInterface. +func (ts *TestServer) SpanConfigReporter() interface{} { + return ts.Server.node.spanConfigReporter +} + // SpanConfigReconciler is part of TestServerInterface. func (ts *TestServer) SpanConfigReconciler() interface{} { if ts.sqlServer.spanconfigMgr == nil { diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 955154c59552..5bd6ebf4eb3a 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -350,6 +350,14 @@ func Delta( return delta, nil } +// Reporter generates a conformance report over the given spans, i.e. whether +// the backing ranges conform to the span configs that apply to them. +type Reporter interface { + SpanConfigConformance( + ctx context.Context, spans []roachpb.Span, + ) (roachpb.SpanConfigConformanceReport, error) +} + // SQLUpdate captures either a descriptor or a protected timestamp update. // It is the unit emitted by the SQLWatcher. type SQLUpdate struct { diff --git a/pkg/spanconfig/spanconfigreporter/BUILD.bazel b/pkg/spanconfig/spanconfigreporter/BUILD.bazel new file mode 100644 index 000000000000..220924de2eb2 --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/BUILD.bazel @@ -0,0 +1,55 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "spanconfigreporter", + srcs = [ + "disabled.go", + "reporter.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigreporter", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/constraint", + "//pkg/roachpb", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/spanconfig", + "//pkg/util/log", + "//pkg/util/rangedesciter", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "spanconfigreporter_test", + srcs = [ + "datadriven_test.go", + "main_test.go", + ], + args = ["-test.timeout=295s"], + data = glob(["testdata/**"]), + deps = [ + ":spanconfigreporter", + "//pkg/kv/kvserver/constraint", + "//pkg/roachpb", + "//pkg/security/securityassets", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/settings/cluster", + "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigstore", + "//pkg/spanconfig/spanconfigtestutils", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "//pkg/util/rangedesciter", + "@com_github_cockroachdb_datadriven//:datadriven", + "@com_github_stretchr_testify//require", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/spanconfig/spanconfigreporter/datadriven_test.go b/pkg/spanconfig/spanconfigreporter/datadriven_test.go new file mode 100644 index 000000000000..15fe3f4ac288 --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/datadriven_test.go @@ -0,0 +1,318 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigreporter_test + +import ( + "context" + "fmt" + "sort" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" + "github.com/cockroachdb/cockroach/pkg/roachpb" + clustersettings "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigreporter" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/rangedesciter" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" +) + +// TestDataDriven is a data-driven test for spanconfig.Reporter. It offers +// the following commands: +// +// init +// n1: attr-key=attr-value +// r1: [a,b) +// ---- +// +// liveness +// n1: live|dead +// ---- +// +// allocate +// r1: voters=[n1,n2] nonvoters=[n3] +// ---- +// +// configure +// [a,b): num_replicas=3 constraints='...' voter_constraints='...' +// ---- +// +// report +// [a,b) +// [c,d) +// ---- +func TestDataDriven(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + st := clustersettings.MakeTestingClusterSettings() + scKnobs := &spanconfig.TestingKnobs{} + + datadriven.Walk(t, testutils.TestDataPath(t), func(t *testing.T, path string) { + cluster := newMockCluster(t, st, scKnobs) + reporter := spanconfigreporter.New(cluster, cluster, cluster, cluster, st, scKnobs) + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + for _, line := range strings.Split(d.Input, "\n") { + line = strings.TrimSpace(line) + if line == "" { + continue + } + parts := strings.Split(line, ":") + require.Len(t, parts, 2) + id, data := strings.TrimSpace(parts[0]), strings.TrimSpace(parts[1]) + switch { + case strings.HasPrefix(id, "n"): // node + nodeID := spanconfigtestutils.ParseNodeID(t, id) + locality := &roachpb.Locality{} + if data != "" { + require.NoError(t, locality.Set(data)) + } + cluster.addNode(roachpb.NodeDescriptor{ + NodeID: nodeID, + Locality: *locality, + }) + case strings.HasPrefix(id, "r"): // range + rangeID := spanconfigtestutils.ParseRangeID(t, id) + span := spanconfigtestutils.ParseSpan(t, data) + cluster.addRange(roachpb.RangeDescriptor{ + RangeID: rangeID, + StartKey: roachpb.RKey(span.Key), + EndKey: roachpb.RKey(span.EndKey), + }) + default: + t.Fatalf("malformed line %q, expected to find 'n' or 'r' prefix", line) + } + } + + case "liveness": + for _, line := range strings.Split(d.Input, "\n") { + line = strings.TrimSpace(line) + if line == "" { + continue + } + parts := strings.Split(line, ":") + require.Len(t, parts, 2) + id, data := strings.TrimSpace(parts[0]), strings.TrimSpace(parts[1]) + if data != "live" && data != "dead" { + t.Fatalf("malformed line %q, expected to find 'live' or 'dead' annotation", line) + } + nodeID := spanconfigtestutils.ParseNodeID(t, id) + cluster.markLive(nodeID, data == "live") + } + + case "allocate": + for _, line := range strings.Split(d.Input, "\n") { + line = strings.TrimSpace(line) + if line == "" { + continue + } + parts := strings.Split(line, ":") + require.Len(t, parts, 2) + id, data := strings.TrimSpace(parts[0]), strings.TrimSpace(parts[1]) + rangeID := spanconfigtestutils.ParseRangeID(t, id) + desc := cluster.getRangeDescriptor(rangeID) + desc.SetReplicas(spanconfigtestutils.ParseReplicaSet(t, data)) + cluster.setRangeDescriptor(desc) + } + + case "configure": + for _, line := range strings.Split(d.Input, "\n") { + line = strings.TrimSpace(line) + if line == "" { + continue + } + tag, data, found := strings.Cut(line, ":") + require.True(t, found) + tag, data = strings.TrimSpace(tag), strings.TrimSpace(data) + span := spanconfigtestutils.ParseSpan(t, tag) + conf := spanconfigtestutils.ParseZoneConfig(t, data).AsSpanConfig() + cluster.applyConfig(ctx, span, conf) + } + + case "report": + var spans []roachpb.Span + for _, line := range strings.Split(d.Input, "\n") { + line = strings.TrimSpace(line) + if line == "" { + continue + } + spans = append(spans, spanconfigtestutils.ParseSpan(t, line)) + } + report, err := reporter.SpanConfigConformance(ctx, spans) + require.NoError(t, err) + printRangeDesc := func(r roachpb.RangeDescriptor) string { + var buf strings.Builder + buf.WriteString(fmt.Sprintf("r%d:", r.RangeID)) + buf.WriteString(r.RSpan().String()) + buf.WriteString(" [") + if allReplicas := r.Replicas().Descriptors(); len(allReplicas) > 0 { + for i, rep := range allReplicas { + if i > 0 { + buf.WriteString(", ") + } + buf.WriteString(rep.String()) + } + } else { + buf.WriteString("") + } + buf.WriteString("]") + return buf.String() + } + printList := func(tag string, descs []roachpb.RangeDescriptor) string { + var buf strings.Builder + for i, desc := range descs { + if i == 0 { + buf.WriteString(fmt.Sprintf("%s:\n", tag)) + } + buf.WriteString(fmt.Sprintf(" %s\n", printRangeDesc(desc))) + } + return buf.String() + } + var buf strings.Builder + buf.WriteString(printList("unavailable", report.Unavailable)) + buf.WriteString(printList("under replicated", report.UnderReplicated)) + buf.WriteString(printList("over replicated", report.OverReplicated)) + buf.WriteString(printList("violating constraints", report.ViolatingConstraints)) + if buf.Len() == 0 { + return "ok" + } + return buf.String() + + default: + t.Fatalf("unknown command: %s", d.Cmd) + } + + return "" + }) + }) +} + +type mockCluster struct { + t *testing.T + + nodes map[roachpb.NodeID]roachpb.NodeDescriptor + ranges map[roachpb.RangeID]roachpb.RangeDescriptor + liveness map[roachpb.NodeID]bool + store *spanconfigstore.Store +} + +var _ spanconfigreporter.Liveness = &mockCluster{} +var _ constraint.StoreResolver = &mockCluster{} +var _ rangedesciter.Iterator = &mockCluster{} +var _ spanconfig.StoreReader = &mockCluster{} + +func newMockCluster( + t *testing.T, st *clustersettings.Settings, scKnobs *spanconfig.TestingKnobs, +) *mockCluster { + return &mockCluster{ + t: t, + nodes: make(map[roachpb.NodeID]roachpb.NodeDescriptor), + ranges: make(map[roachpb.RangeID]roachpb.RangeDescriptor), + liveness: make(map[roachpb.NodeID]bool), + store: spanconfigstore.New(roachpb.TestingDefaultSpanConfig(), st, scKnobs), + } +} + +// IsLive implements spanconfigreporter.Liveness. +func (s *mockCluster) IsLive(id roachpb.NodeID) (bool, error) { + live, found := s.liveness[id] + require.True(s.t, found, "undeclared node n%d", id) + return live, nil +} + +// GetStoreDescriptor implements constraint.StoreResolver. +func (s *mockCluster) GetStoreDescriptor(storeID roachpb.StoreID) (roachpb.StoreDescriptor, bool) { + desc, found := s.nodes[roachpb.NodeID(storeID)] + require.True(s.t, found, "undeclared node n%d", storeID) + + return roachpb.StoreDescriptor{ + StoreID: storeID, // simulate storeIDs == nodeIDs + Node: desc, + }, true +} + +// Iterate implements rangedesciter.Iterator. +func (s *mockCluster) Iterate( + _ context.Context, _ int, _ func(), fn func(...roachpb.RangeDescriptor) error, +) error { + var descs []roachpb.RangeDescriptor + for _, d := range s.ranges { + descs = append(descs, d) + } + sort.Slice(descs, func(i, j int) bool { + return descs[i].StartKey.Less(descs[j].StartKey) + }) + return fn(descs...) +} + +// NeedsSplit implements spanconfig.StoreReader. +func (s *mockCluster) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { + return s.store.NeedsSplit(ctx, start, end) +} + +// ComputeSplitKey implements spanconfig.StoreReader. +func (s *mockCluster) ComputeSplitKey(ctx context.Context, start, end roachpb.RKey) roachpb.RKey { + return s.store.ComputeSplitKey(ctx, start, end) +} + +// GetSpanConfigForKey implements spanconfig.StoreReader. +func (s *mockCluster) GetSpanConfigForKey( + ctx context.Context, key roachpb.RKey, +) (roachpb.SpanConfig, error) { + return s.store.GetSpanConfigForKey(ctx, key) +} + +func (s *mockCluster) addNode(desc roachpb.NodeDescriptor) { + _, found := s.nodes[desc.NodeID] + require.Falsef(s.t, found, "attempting to re-add n%d", desc.NodeID) + s.nodes[desc.NodeID] = desc + s.markLive(desc.NodeID, true /* live */) +} + +func (s *mockCluster) markLive(id roachpb.NodeID, live bool) { + _, found := s.nodes[id] + require.Truef(s.t, found, "n%d not found", id) + s.liveness[id] = live +} + +func (s *mockCluster) addRange(desc roachpb.RangeDescriptor) { + _, found := s.ranges[desc.RangeID] + require.Falsef(s.t, found, "attempting to re-add r%d", desc.RangeID) + s.ranges[desc.RangeID] = desc +} + +func (s *mockCluster) setRangeDescriptor(desc roachpb.RangeDescriptor) { + _, found := s.ranges[desc.RangeID] + require.Truef(s.t, found, "r%d not found", desc.RangeID) + s.ranges[desc.RangeID] = desc +} + +func (s *mockCluster) getRangeDescriptor(id roachpb.RangeID) roachpb.RangeDescriptor { + desc, found := s.ranges[id] + require.Truef(s.t, found, "r%d not found", id) + return desc +} + +func (s *mockCluster) applyConfig(ctx context.Context, span roachpb.Span, conf roachpb.SpanConfig) { + update, err := spanconfig.Addition(spanconfig.MakeTargetFromSpan(span), conf) + require.NoError(s.t, err) + s.store.Apply(ctx, false, update) +} diff --git a/pkg/spanconfig/spanconfigreporter/disabled.go b/pkg/spanconfig/spanconfigreporter/disabled.go new file mode 100644 index 000000000000..545079c36de7 --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/disabled.go @@ -0,0 +1,34 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigreporter + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/errors" +) + +// DisabledReporter is a spanconfig.Reporter that only returns "disabled" +// errors. +var DisabledReporter = disabled{} + +type disabled struct{} + +var _ spanconfig.Reporter = disabled{} + +// SpanConfigConformance implements the spanconfig.Reporter interface. +func (d disabled) SpanConfigConformance( + ctx context.Context, spans []roachpb.Span, +) (roachpb.SpanConfigConformanceReport, error) { + return roachpb.SpanConfigConformanceReport{}, errors.New("span configs disabled") +} diff --git a/pkg/spanconfig/spanconfigreporter/main_test.go b/pkg/spanconfig/spanconfigreporter/main_test.go new file mode 100644 index 000000000000..f6c86b6937ab --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigreporter_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go + +func TestMain(m *testing.M) { + securityassets.SetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/spanconfig/spanconfigreporter/reporter.go b/pkg/spanconfig/spanconfigreporter/reporter.go new file mode 100644 index 000000000000..ad1d7f707372 --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/reporter.go @@ -0,0 +1,181 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Package spanconfigreporter reports on whether ranges over the queried spans +// conform to the span configs that apply to them. +package spanconfigreporter + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" + "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/util/log" + "github.com/cockroachdb/cockroach/pkg/util/rangedesciter" +) + +// rangeDescPageSize controls the page size when iterating through range +// descriptors. It's settable only by the system tenant. +var rangeDescPageSize = settings.RegisterIntSetting( + settings.SystemOnly, + "spanconfig.reporter.range_desc_page_size", + "pa", + 100, + func(i int64) error { + if i < 5 || i > 25000 { + return fmt.Errorf("expected range_desc_page_size to be in range [5, 25000], got %d", i) + } + return nil + }, +) + +// Liveness is the subset of the interface satisfied by CRDB's node liveness +// component that the reporter relies on. +type Liveness interface { + IsLive(roachpb.NodeID) (bool, error) +} + +// Reporter is used to figure out whether ranges backing specific spans conform +// to the span configs that apply over them. It's a concrete implementation of +// the spanconfig.Reporter interface. +type Reporter struct { + dep struct { + Liveness + constraint.StoreResolver + rangedesciter.Iterator + spanconfig.StoreReader + } + + settings *cluster.Settings + knobs *spanconfig.TestingKnobs +} + +var _ spanconfig.Reporter = &Reporter{} + +// New constructs and returns a Reporter. +func New( + liveness Liveness, + resolver constraint.StoreResolver, + reader spanconfig.StoreReader, + iterator rangedesciter.Iterator, + settings *cluster.Settings, + knobs *spanconfig.TestingKnobs, +) *Reporter { + r := &Reporter{ + settings: settings, + knobs: knobs, + } + r.dep.Liveness = liveness + r.dep.StoreResolver = resolver + r.dep.Iterator = iterator + r.dep.StoreReader = reader + return r +} + +// TODO(irfansharif): Support the equivalent of "critical localities", perhaps +// through a different API than the one below since it's not quite +// span-oriented. + +// SpanConfigConformance implements the spanconfig.Reporter interface. +func (r *Reporter) SpanConfigConformance( + ctx context.Context, spans []roachpb.Span, +) (roachpb.SpanConfigConformanceReport, error) { + // XXX: Actually use the spans parameter. Update the rangedesc.Iterator + // interfaces to take in a keyspan and bound meta{1,2} search just to + // segments that would possibly overlap with that keyspan. Until this + // keyspan scoping is done, we can't let this be used in tenants. + _ = spans + + // XXX: Write an end-to-end test using actual SQL and zone configs. Set configs + // on a table, disable replication, see conformance. Enable repl, change + // configs, etc. Use tenants as well for this mode. Do this for tenants as well. + // Do this after some form of this API is exposed through SQL/an endpoint. + + // XXX: Can we improve the SpanConfigConformanceReport proto type? Perhaps + // include some {meta,}data about the span config being violated as well? Or + // include the span config directly and provide helper libraries to compute + // human-readable "why is this in violation" text. + // - Only include range ID + replica descriptors + keys? + // - Type to represent exactly which constraint exactly is being violated? + // - Segment over/under replicated by what replica type (voter/non-voter) + // exactly is over/under replicated? + + report := roachpb.SpanConfigConformanceReport{} + if err := r.dep.Iterate(ctx, int(rangeDescPageSize.Get(&r.settings.SV)), func() { + report = roachpb.SpanConfigConformanceReport{} // init + }, func(descriptors ...roachpb.RangeDescriptor) error { + for _, desc := range descriptors { + conf, err := r.dep.StoreReader.GetSpanConfigForKey(ctx, desc.StartKey) + if err != nil { + return err + } + + status := desc.Replicas().ReplicationStatus( + func(rDesc roachpb.ReplicaDescriptor) bool { + isLive, err := r.dep.Liveness.IsLive(rDesc.NodeID) + if err != nil { + // As of 2022-10, this error only appears if we're + // asking for the liveness of a node ID that doesn't + // exist, which should never happen. Shout loudly + // and declare things as non-live. + log.Errorf(ctx, "programming error: unexpected err: %v", err) + return false + } + return isLive + }, int(conf.GetNumVoters()), int(conf.GetNumNonVoters())) + if !status.Available { + report.Unavailable = append(report.Unavailable, desc) + } + if status.UnderReplicated || status.UnderReplicatedNonVoters { + report.UnderReplicated = append(report.UnderReplicated, desc) + } + if status.OverReplicated || status.OverReplicatedNonVoters { + report.OverReplicated = append(report.OverReplicated, desc) + } + + // Compute constraint violations for the overall (affecting voters + // and non-voters alike) and voter constraints. + overall := constraint.AnalyzeConstraints( + r.dep.StoreResolver, + desc.Replicas().Descriptors(), + conf.NumReplicas, conf.Constraints) + for i, c := range overall.Constraints { + if c.NumReplicas == 0 { + c.NumReplicas = conf.NumReplicas + } + if len(overall.SatisfiedBy[i]) < int(c.NumReplicas) { + report.ViolatingConstraints = append(report.ViolatingConstraints, desc) + break + } + } + voters := constraint.AnalyzeConstraints( + r.dep.StoreResolver, + desc.Replicas().Voters().Descriptors(), + conf.GetNumVoters(), conf.VoterConstraints) + for i, c := range voters.Constraints { + if c.NumReplicas == 0 { + c.NumReplicas = conf.GetNumVoters() + } + if len(voters.SatisfiedBy[i]) < int(c.NumReplicas) { + report.ViolatingConstraints = append(report.ViolatingConstraints, desc) + break + } + } + } + return nil + }); err != nil { + return roachpb.SpanConfigConformanceReport{}, err + } + return report, nil +} diff --git a/pkg/spanconfig/spanconfigreporter/testdata/basic b/pkg/spanconfig/spanconfigreporter/testdata/basic new file mode 100644 index 000000000000..144b03dbe909 --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/testdata/basic @@ -0,0 +1,189 @@ +# Walk through the basics of the datadriven syntax. We initialize a six node +# cluster with two nodes in each region (us-{west,central,east}) and three +# ranges. + +init +n1: region=us-west +n2: region=us-west +n3: region=us-central +n4: region=us-central +n5: region=us-east +n6: region=us-east +r1: [a,b) +r2: [b,c) +r3: [c,d) +---- + +# Set-up a replication factor of 3 across the entire keyspan, and allocate +# replicas accordingly. Our conformance report should indicate no problems. +configure +[a,d): num_replicas=3 +---- + +allocate +r1: voters=[n1,n3,n5] +r2: voters=[n1,n3,n5] +r3: voters=[n1,n3,n5] +---- + +report +---- +ok + + +# Shift around the replicas for r1 to the second node in each region. We'll +# kill these nodes one by one and verify that we report the range to first be +# under-replicated, and then both under-replicated and unavailable. +allocate +r1: voters=[n2,n4,n6] +---- + +liveness +n6: dead +---- + +report +---- +under replicated: + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] + +liveness +n4: dead +---- + +report +---- +unavailable: + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] +under replicated: + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] + +liveness +n4: live +n6: live +---- + +report +---- +ok + + +# Add extra replicas for r2, and verify that it shows up as over-replicated. +allocate +r2: voters=[n1,n3,n5,n6] +---- + +report +---- +over replicated: + r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5, (n6,s6):6] + +# It should also work when we don't have enough replicas. +allocate +r2: voters=[n1] +---- + +report +---- +under replicated: + r2:{b-c} [(n1,s1):1] + +allocate +r2: voters=[n1,n3,n5] +---- + +report +---- +ok + + +# Configuring different parts of the keyspan with different replication factors +# will work as expected. All ranges currently have 3 replicas each, so if the +# span configs indicate that we want a different number of replicas, the +# reports should indicate as much. +configure +[c,d): num_replicas=5 +[a,b): num_replicas=1 +---- + +report +---- +under replicated: + r3:{c-d} [(n1,s1):1, (n3,s3):3, (n5,s5):5] +over replicated: + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] + +configure +[a,d): num_replicas=3 +---- + +report +---- +ok + + +# Verify that conformance reports also work for voter/non-voter +# constraints/replica counts. +configure +[b,c): num_replicas=6 num_voters=3 +---- + +allocate +r2: voters=[n1,n3,n5] +---- + +report +---- +under replicated: + r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5] + +allocate +r2: voters=[n1,n2,n3,n4,n5,n6] +---- + +# We're under replicated due to non-voters, over replicated due to voters. +report +---- +under replicated: + r2:{b-c} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4, (n5,s5):5, (n6,s6):6] +over replicated: + r2:{b-c} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4, (n5,s5):5, (n6,s6):6] + +allocate +r2: voters=[n1,n3,n5] non-voters=[n2,n4,n6] +---- + +report +---- +ok + +configure +[a,d): num_replicas=3 +---- + +allocate +r2: voters=[n1,n3,n5] +---- + +report +---- +ok + + +# Verify that constraints are also reported on. +configure +[b,c): num_replicas=3 constraints={'+region=us-central':2} +---- + +report +---- +violating constraints: + r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5] + +allocate +r2: voters=[n1,n3,n4] +---- + +report +---- +ok diff --git a/pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance b/pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance new file mode 100644 index 000000000000..942091d86e23 --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance @@ -0,0 +1,87 @@ +# Walk through basics of constraint conformance reporting. We'll use a six-node +# cluster with various attributes and see how constraints/allocation interact. + +init +n1: region=us-west,dc=dc-a +n2: region=us-west,dc=dc-b +n3: region=us-west,dc=dc-c +n4: region=us-east,dc=dc-d +n5: region=us-east,dc=dc-e +n6: region=us-east,dc=dc-f +r1: [a,b) +---- + +# Pin all three replicas to us-west. If any replica is found outside of it, it +# should be in violation. +configure +[a,b): num_replicas=3 constraints={'+region=us-west':3} +---- + +allocate +r1: voters=[n1,n2,n4] +---- + +report +---- +violating constraints: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n4,s4):4] + +# Pin replicas to two specific DCs. A conforming replica placement should show +# up as such. +configure +[a,b): num_replicas=2 constraints={'+region=us-west,+dc=dc-a':1,'+region=us-east,+dc=dc-d':1} +---- + +allocate +r1: voters=[n1,n4] +---- + +report +---- +ok + +# Pin a voter and a non-voter to two specific DCs (n1 and n4 respectively). +# It's in violation until we get exactly what we're looking for. +configure +[a,b): num_replicas=2 num_voters=1 constraints={'+dc=dc-a':1,'+dc=dc-d':1} voter_constraints={'+dc=dc-a':1} +---- + +allocate +r1: voters=[n1] non-voters=[n3] +---- + +report +---- +violating constraints: + r1:{a-b} [(n1,s1):1, (n3,s3):3NON_VOTER] + +allocate +r1: voters=[n1] non-voters=[n4] +---- + +report +---- +ok + +# Try negative constraints over all replicas. If any are found in n1, n2 or n3, +# we're in violation. +configure +[a,b): num_replicas=3 constraints=[-region=us-west] +---- + +allocate +r1: voters=[n1,n3,n5] +---- + +report +---- +violating constraints: + r1:{a-b} [(n1,s1):1, (n3,s3):3, (n5,s5):5] + +allocate +r1: voters=[n4,n5,n6] +---- + +report +---- +ok diff --git a/pkg/spanconfig/spanconfigreporter/testdata/joint_consensus b/pkg/spanconfig/spanconfigreporter/testdata/joint_consensus new file mode 100644 index 000000000000..bc0b4e09704c --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/testdata/joint_consensus @@ -0,0 +1,104 @@ +# Walk through a few scenarios where we generate reports in the presence of +# voter-{incoming,outgoing,demoting-learners,demoting-non-voters} and learners. + +init +n1: +n2: +n3: +n4: +n5: +n6: +r1: [a,b) +---- + +liveness +n4: dead +n5: dead +---- + +configure +[a,b): num_replicas=3 +---- + +allocate +r1: voters=[n1,n2,n3] +---- + +report +---- +ok + +# Under-replication in the "old group". +allocate +r1: voters=[n1,n2] voters-incoming=[n3] +---- + +report +---- +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_INCOMING] + +# Under-replication in the "new group". +allocate +r1: voters=[n1,n2] voters-outgoing=[n3] +---- + +report +---- +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING] + +# Under-replication in the old group because 4 is dead. +allocate +r1: voters=[n1,n2] voters-outgoing=[n4] voters-incoming=[n3] +---- + +report +---- +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n4,s4):4VOTER_OUTGOING, (n3,s3):3VOTER_INCOMING] + +# Unavailable in the new group (and also under-replicated), and also +# over-replicated in the new group. +allocate +r1: voters=[n1,n2] voters-outgoing=[n3] voters-incoming=[n4,n5] +---- + +report +---- +unavailable: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] +over replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] + +# Over-replicated in the new group. +allocate +r1: voters=[n1,n2] voters-outgoing=[n3] voters-incoming=[n5,n6] +---- + +report +---- +over replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n5,s5):5VOTER_INCOMING, (n6,s6):6VOTER_INCOMING] + + +# Many learners. No problems, since learners don't count. +allocate +r1: voters=[n1,n2,n3] learners=[n4,n5,n6] +---- + +report +---- +ok + +# Under replicated. Learners don't count. +allocate +r1: voters=[n1,n2] learners=[n3] +---- + +report +---- +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3LEARNER] diff --git a/pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated b/pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated new file mode 100644 index 000000000000..5d73280c2b2d --- /dev/null +++ b/pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated @@ -0,0 +1,120 @@ +# Walk through a few scenarios where a replica is {over,under}-replicated, +# and/or unavailable. + +init +n1: +n2: +n3: +n4: +r1: [a,b) +---- + +configure +[a,b): num_replicas=3 +---- + +allocate +r1: voters=[n1,n2,n3] +---- + +# We want 3 replicas and we have them, report should be ok. +report +---- +ok + +# ----------------------------------------------------------------------------- +# We have 4 replica when we want 3, we're over replicated. +allocate +r1: voters=[n1,n2,n3,n4] +---- + +report +---- +over replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4] + +# ----------------------------------------------------------------------------- +# We have 1 or 2 replicas when we want 3, we're under replicated. +allocate +r1: voters=[n1] +---- + +report +---- +under replicated: + r1:{a-b} [(n1,s1):1] + +allocate +r1: voters=[n1,n2] +---- + +report +---- +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2] + +# ----------------------------------------------------------------------------- +# We have the desired number of replicas, but one of them is on a dead node so +# we're under-replicated. +liveness +n3: dead +---- + +allocate +r1: voters=[n1,n2,n3] +---- + +report +---- +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + +# If we've lost quorum we're also unavailable. +liveness +n2: dead +n3: dead +---- + +report +---- +unavailable: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + +liveness +n2: live +n3: live +---- + +report +---- +ok + +# ----------------------------------------------------------------------------- +# We can be under-replicated and over-replicated at the same time if it +# has many replicas but sufficiently many of them are on dead nodes. It can +# also be unavailable. Set up a triply replicated range where we want two +# replicas (so over-replicated), except two of the range's replicas are on dead +# nodes (under-replicated + unavailable). +allocate +r1: voters=[n1,n2,n3] +---- + +configure +[a,b): num_replicas=2 +---- + +liveness +n1: dead +n2: dead +---- + +report +---- +unavailable: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] +under replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] +over replicated: + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] diff --git a/pkg/spanconfig/spanconfigtestutils/BUILD.bazel b/pkg/spanconfig/spanconfigtestutils/BUILD.bazel index 3a76a3bc1e5f..ccd602dbb69f 100644 --- a/pkg/spanconfig/spanconfigtestutils/BUILD.bazel +++ b/pkg/spanconfig/spanconfigtestutils/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils", visibility = ["//visibility:public"], deps = [ + "//pkg/config/zonepb", "//pkg/kv", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb", @@ -19,6 +20,7 @@ go_library( "//pkg/util/syncutil", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_stretchr_testify//require", + "@in_gopkg_yaml_v2//:yaml_v2", ], ) diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 665c09b20164..527592bac371 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -20,6 +20,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" @@ -27,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" + "gopkg.in/yaml.v2" ) // spanRe matches strings of the form "[start, end)", capturing both the "start" @@ -43,6 +45,98 @@ var systemTargetRe = regexp.MustCompile( // shorthand for declaring a unique tagged config. var configRe = regexp.MustCompile(`^(FALLBACK)|(^\w)$`) +// ParseRangeID is helper function that constructs a roachpb.RangeID from a +// string of the form "r". +func ParseRangeID(t testing.TB, s string) roachpb.RangeID { + rangeID, err := strconv.Atoi(strings.TrimPrefix(s, "r")) + require.NoError(t, err) + return roachpb.RangeID(rangeID) +} + +// ParseNodeID is helper function that constructs a roachpb.NodeID from a string +// of the form "n". +func ParseNodeID(t testing.TB, s string) roachpb.NodeID { + nodeID, err := strconv.Atoi(strings.TrimPrefix(s, "n")) + require.NoError(t, err) + return roachpb.NodeID(nodeID) +} + +// ParseReplicaSet is helper function that constructs a roachpb.ReplicaSet from +// a string of the form "voters=[n1,n2,...] non-voters=[n3,...]". The +// {store,replica} IDs for each replica is set to be equal to the corresponding +// node ID. +func ParseReplicaSet(t testing.TB, s string) roachpb.ReplicaSet { + replSet := roachpb.ReplicaSet{} + rtypes := map[string]roachpb.ReplicaType{ + "voters": roachpb.VOTER_FULL, + "voters-incoming": roachpb.VOTER_INCOMING, + "voters-outgoing": roachpb.VOTER_OUTGOING, + "voters-demoting-learners": roachpb.VOTER_DEMOTING_LEARNER, + "voters-demoting-non-voters": roachpb.VOTER_DEMOTING_NON_VOTER, + "learners": roachpb.LEARNER, + "non-voters": roachpb.NON_VOTER, + } + for _, part := range strings.Split(s, " ") { + inner := strings.Split(part, "=") + require.Len(t, inner, 2) + rtype, found := rtypes[inner[0]] + require.Truef(t, found, "unexpected replica type: %s", inner[0]) + nodes := strings.TrimSuffix(strings.TrimPrefix(inner[1], "["), "]") + + for _, n := range strings.Split(nodes, ",") { + n = strings.TrimSpace(n) + if n == "" { + continue + } + nodeID := ParseNodeID(t, n) + replSet.AddReplica(roachpb.ReplicaDescriptor{ + NodeID: nodeID, + StoreID: roachpb.StoreID(nodeID), + ReplicaID: roachpb.ReplicaID(nodeID), + Type: rtype, + }) + } + } + return replSet +} + +// ParseZoneConfig is helper function that constructs a zonepb.ZoneConfig from a +// string of the form "num_replicas= num_voters= constraints='..' +// voter_constraints='..'". +func ParseZoneConfig(t testing.TB, s string) zonepb.ZoneConfig { + config := zonepb.DefaultZoneConfig() + parts := strings.Split(s, " ") + for _, part := range parts { + switch { + case strings.HasPrefix(part, "num_replicas="): + part = strings.TrimPrefix(part, "num_replicas=") + n, err := strconv.Atoi(part) + require.NoError(t, err) + n32 := int32(n) + config.NumReplicas = &n32 + case strings.HasPrefix(part, "num_voters="): + part = strings.TrimPrefix(part, "num_voters=") + n, err := strconv.Atoi(part) + require.NoError(t, err) + n32 := int32(n) + config.NumVoters = &n32 + case strings.HasPrefix(part, "constraints="): + cl := zonepb.ConstraintsList{} + part = strings.TrimPrefix(part, "constraints=") + require.NoError(t, yaml.UnmarshalStrict([]byte(part), &cl)) + config.Constraints = cl.Constraints + case strings.HasPrefix(part, "voter_constraints="): + cl := zonepb.ConstraintsList{} + part = strings.TrimPrefix(part, "voter_constraints=") + require.NoError(t, yaml.UnmarshalStrict([]byte(part), &cl)) + config.VoterConstraints = cl.Constraints + default: + t.Fatalf("unrecognized suffix for %s, expected 'num_replicas=', 'num_voters=', 'constraints=', or 'voter_constraints='", part) + } + } + return config +} + // ParseSpan is helper function that constructs a roachpb.Span from a string of // the form "[start, end)". func ParseSpan(t testing.TB, sp string) roachpb.Span { diff --git a/pkg/testutils/serverutils/test_tenant_shim.go b/pkg/testutils/serverutils/test_tenant_shim.go index 636254295760..1d15d83611cb 100644 --- a/pkg/testutils/serverutils/test_tenant_shim.go +++ b/pkg/testutils/serverutils/test_tenant_shim.go @@ -99,6 +99,10 @@ type TestTenantInterface interface { // interface{}. SpanConfigKVAccessor() interface{} + // SpanConfigReporter returns the underlying spanconfig.Reporter as an + // interface{}. + SpanConfigReporter() interface{} + // SpanConfigReconciler returns the underlying spanconfig.Reconciler as an // interface{}. SpanConfigReconciler() interface{} From 6adcb7d8db28a4dbf5cc93ae6fff09e2c52703ec Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 24 Oct 2022 19:16:31 -0400 Subject: [PATCH 3/3] *: review fixup (will be squashed in) --- pkg/kv/kvclient/kvcoord/send_test.go | 2 +- .../allocator/allocatorimpl/allocator_test.go | 2 +- pkg/kv/kvserver/replica_metrics.go | 2 +- .../reports/replication_stats_report.go | 2 +- pkg/roachpb/metadata_replicas.go | 7 +- pkg/roachpb/span_config.proto | 20 +- pkg/rpc/auth_tenant.go | 42 ++-- pkg/rpc/auth_test.go | 35 +++ pkg/spanconfig/spanconfig.go | 5 + pkg/spanconfig/spanconfigreporter/BUILD.bazel | 4 +- .../spanconfigreporter/datadriven_test.go | 28 ++- pkg/spanconfig/spanconfigreporter/reporter.go | 200 +++++++++++------- .../spanconfigreporter/testdata/basic | 22 +- .../testdata/constraint_conformance | 6 +- .../testdata/joint_consensus | 16 +- .../testdata/over_under_replicated | 18 +- 16 files changed, 262 insertions(+), 149 deletions(-) diff --git a/pkg/kv/kvclient/kvcoord/send_test.go b/pkg/kv/kvclient/kvcoord/send_test.go index 3a96cb12e136..81b6a19e7c99 100644 --- a/pkg/kv/kvclient/kvcoord/send_test.go +++ b/pkg/kv/kvclient/kvcoord/send_test.go @@ -104,7 +104,7 @@ func (n Node) UpdateSpanConfigs( func (n Node) SpanConfigConformance( context.Context, *roachpb.SpanConfigConformanceRequest, ) (*roachpb.SpanConfigConformanceResponse, error) { - panic("implement me") + panic("unimplemented") } func (n Node) TenantSettings( diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go index 3850f5201084..293276eda7eb 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -3975,7 +3975,7 @@ func TestRemoveCandidatesNumReplicasConstraints(t *testing.T) { StoreID: storeID, } } - analyzed := constraint.AnalyzeConstraints(a.StorePool, existingRepls, 0, tc.constraints) + analyzed := constraint.AnalyzeConstraints(a.StorePool, existingRepls, 0 /* numReplicas */, tc.constraints) // Check behavior in a span config where `voter_constraints` are empty. checkFn := voterConstraintsCheckerForRemoval(analyzed, constraint.EmptyAnalyzedConstraints) diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index 4f4fc81e1908..8cc8156b6acf 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -223,7 +223,7 @@ func calcRangeCounter( // needed{Voters,NonVoters} - we don't care about the // under/over-replication determinations from the report because // it's too magic. We'll do our own determination below. - 0, 0) + 0, -1) unavailable = !status.Available liveVoters := calcLiveVoterReplicas(desc, livenessMap) liveNonVoters := calcLiveNonVoterReplicas(desc, livenessMap) diff --git a/pkg/kv/kvserver/reports/replication_stats_report.go b/pkg/kv/kvserver/reports/replication_stats_report.go index 863f91d097eb..ee5f08d4218e 100644 --- a/pkg/kv/kvserver/reports/replication_stats_report.go +++ b/pkg/kv/kvserver/reports/replication_stats_report.go @@ -404,7 +404,7 @@ func (v *replicationStatsVisitor) countRange( // NB: this reporting code was written before ReplicationStatus reported // on non-voting replicas. This code will also soon be removed in favor // of something that works with multi-tenancy (#89987). - }, replicationFactor, 0) + }, replicationFactor, -1 /* neededNonVoters */) // Note that a range can be under-replicated and over-replicated at the same // time if it has many replicas, but sufficiently many of them are on dead // nodes. diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index 4401237f15d4..2b874d082cdd 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -378,7 +378,7 @@ func (d ReplicaSet) HasReplicaOnNode(nodeID NodeID) bool { // the replication layer. This is more complicated than just counting the number // of replicas due to the existence of joint quorums. func (d ReplicaSet) CanMakeProgress(liveFunc func(descriptor ReplicaDescriptor) bool) bool { - return d.ReplicationStatus(liveFunc, 0 /* neededVoters */, 0 /* neededNonVoters*/).Available + return d.ReplicationStatus(liveFunc, 0 /* neededVoters */, -1 /* neededNonVoters*/).Available } // RangeStatusReport contains info about a range's replication status. Returned @@ -412,7 +412,8 @@ type RangeStatusReport struct { // neededVoters is the replica's desired replication for purposes of determining // over/under-replication of voters. If the caller is only interested in // availability of voting replicas, 0 can be passed in. neededNonVoters is the -// counterpart for non-voting replicas. +// counterpart for non-voting replicas but with -1 as the sentinel value (unlike +// voters, it's possible to expect 0 non-voters). func (d ReplicaSet) ReplicationStatus( liveFunc func(descriptor ReplicaDescriptor) bool, neededVoters int, neededNonVoters int, ) RangeStatusReport { @@ -454,7 +455,7 @@ func (d ReplicaSet) ReplicationStatus( overReplicatedNewGroup := len(votersNewGroup) > neededVoters res.UnderReplicated = underReplicatedOldGroup || underReplicatedNewGroup res.OverReplicated = overReplicatedOldGroup || overReplicatedNewGroup - if neededNonVoters == 0 { + if neededNonVoters == -1 { return res } diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index dc298ac6bae2..5c6a9bb5e604 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -278,15 +278,23 @@ message SpanConfigEntry { SpanConfig config = 2 [(gogoproto.nullable) = false]; }; -// SpanConfigConformanceReport lists out ranges that (i) don't conform to span -// configs that apply over them, and (ii) are unavailable. +// SpanConfigConformanceReport reports ranges that (i) don't conform to span +// configs that apply over them, and (ii) are unavailable. Also included in this +// report are the IDs of unavailable nodes (possibly contributing to +// under-replication or range-unavailability). message SpanConfigConformanceReport { - repeated RangeDescriptor under_replicated = 1 [(gogoproto.nullable) = false]; - repeated RangeDescriptor over_replicated = 2 [(gogoproto.nullable) = false]; - repeated RangeDescriptor violating_constraints = 3 [(gogoproto.nullable) = false]; - repeated RangeDescriptor unavailable = 4 [(gogoproto.nullable) = false]; + repeated ConformanceReportedRange under_replicated = 1 [(gogoproto.nullable) = false]; + repeated ConformanceReportedRange over_replicated = 2 [(gogoproto.nullable) = false]; + repeated ConformanceReportedRange violating_constraints = 3 [(gogoproto.nullable) = false]; + repeated ConformanceReportedRange unavailable = 4 [(gogoproto.nullable) = false]; + repeated int32 unavailable_node_ids = 5 [(gogoproto.customname) = "UnavailableNodeIDs"]; }; +message ConformanceReportedRange { + RangeDescriptor range_descriptor = 1 [(gogoproto.nullable) = false]; + SpanConfig config = 2 [(gogoproto.nullable) = false]; +} + // GetSpanConfigsRequest is used to fetch the span configurations and system // span configurations. message GetSpanConfigsRequest { diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index 2df3cbd4ba8d..fe7fd716d123 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -115,6 +115,9 @@ func (a tenantAuthorizer) authorize( case "/cockroach.roachpb.Internal/GetSpanConfigs": return a.authGetSpanConfigs(tenID, req.(*roachpb.GetSpanConfigsRequest)) + case "/cockroach.roachpb.Internal/SpanConfigConformance": + return a.authSpanConfigConformance(tenID, req.(*roachpb.SpanConfigConformanceRequest)) + case "/cockroach.roachpb.Internal/GetAllSystemSpanConfigsThatApply": return a.authGetAllSystemSpanConfigsThatApply(tenID, req.(*roachpb.GetAllSystemSpanConfigsThatApplyRequest)) @@ -339,6 +342,19 @@ func (a tenantAuthorizer) authUpdateSpanConfigs( return nil } +// authSpanConfigConformance authorizes the provided tenant to invoke the +// SpanConfigConformance RPC with the provided args. +func (a tenantAuthorizer) authSpanConfigConformance( + tenID roachpb.TenantID, args *roachpb.SpanConfigConformanceRequest, +) error { + for _, sp := range args.Spans { + if err := validateSpan(tenID, sp); err != nil { + return err + } + } + return nil +} + // validateSpanConfigTarget validates that the tenant is authorized to interact // with the supplied span config target. In particular, span targets must be // wholly contained within the tenant keyspace and system span config targets @@ -371,21 +387,9 @@ func validateSpanConfigTarget( return nil } - validateSpan := func(sp roachpb.Span) error { - tenSpan := tenantPrefix(tenID) - rSpan, err := keys.SpanAddr(sp) - if err != nil { - return authError(err.Error()) - } - if !tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) { - return authErrorf("requested key span %s not fully contained in tenant keyspace %s", rSpan, tenSpan) - } - return nil - } - switch spanConfigTarget.Union.(type) { case *roachpb.SpanConfigTarget_Span: - return validateSpan(*spanConfigTarget.GetSpan()) + return validateSpan(tenID, *spanConfigTarget.GetSpan()) case *roachpb.SpanConfigTarget_SystemSpanConfigTarget: return validateSystemTarget(*spanConfigTarget.GetSystemSpanConfigTarget()) default: @@ -393,6 +397,18 @@ func validateSpanConfigTarget( } } +func validateSpan(tenID roachpb.TenantID, sp roachpb.Span) error { + tenSpan := tenantPrefix(tenID) + rSpan, err := keys.SpanAddr(sp) + if err != nil { + return authError(err.Error()) + } + if !tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) { + return authErrorf("requested key span %s not fully contained in tenant keyspace %s", rSpan, tenSpan) + } + return nil +} + func contextWithTenant(ctx context.Context, tenID roachpb.TenantID) context.Context { ctx = roachpb.NewContextForTenant(ctx, tenID) ctx = logtags.AddTag(ctx, "tenant", tenID.String()) diff --git a/pkg/rpc/auth_test.go b/pkg/rpc/auth_test.go index 30d5f8bc4573..6ecef733260e 100644 --- a/pkg/rpc/auth_test.go +++ b/pkg/rpc/auth_test.go @@ -221,6 +221,11 @@ func TestTenantAuthRequest(t *testing.T) { }, }} } + makeSpanConfigConformanceReq := func( + span roachpb.Span, + ) *roachpb.SpanConfigConformanceRequest { + return &roachpb.SpanConfigConformanceRequest{Spans: []roachpb.Span{span}} + } const noError = "" for method, tests := range map[string][]struct { @@ -700,6 +705,36 @@ func TestTenantAuthRequest(t *testing.T) { expErr: `secondary tenants cannot target the entire keyspace`, }, }, + "/cockroach.roachpb.Internal/SpanConfigConformance": { + { + req: &roachpb.SpanConfigConformanceRequest{}, + expErr: noError, + }, + { + req: makeSpanConfigConformanceReq(makeSpan("a", "b")), + expErr: `requested key span {a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeSpanConfigConformanceReq(makeSpan(prefix(5, "a"), prefix(5, "b"))), + expErr: `requested key span /Tenant/5{a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeSpanConfigConformanceReq(makeSpan(prefix(10, "a"), prefix(10, "b"))), + expErr: noError, + }, + { + req: makeSpanConfigConformanceReq(makeSpan(prefix(50, "a"), prefix(50, "b"))), + expErr: `requested key span /Tenant/50{a-b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeSpanConfigConformanceReq(makeSpan("a", prefix(10, "b"))), + expErr: `requested key span {a-/Tenant/10b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + { + req: makeSpanConfigConformanceReq(makeSpan(prefix(10, "a"), prefix(20, "b"))), + expErr: `requested key span /Tenant/{10a-20b} not fully contained in tenant keyspace /Tenant/1{0-1}`, + }, + }, "/cockroach.roachpb.Internal/GetAllSystemSpanConfigsThatApply": { { req: &roachpb.GetAllSystemSpanConfigsThatApplyRequest{}, diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 5bd6ebf4eb3a..cfe4bf0eacba 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -352,6 +352,11 @@ func Delta( // Reporter generates a conformance report over the given spans, i.e. whether // the backing ranges conform to the span configs that apply to them. +// +// NB: The standard implementation does not use a point-in-time snapshot of span +// config state, but could be made to do so if needed. See commentary on the +// spanconfigreporter.Reporter type for more details ("... we might not have a +// point-in-time snapshot ..."). type Reporter interface { SpanConfigConformance( ctx context.Context, spans []roachpb.Span, diff --git a/pkg/spanconfig/spanconfigreporter/BUILD.bazel b/pkg/spanconfig/spanconfigreporter/BUILD.bazel index 220924de2eb2..83c5c423eaed 100644 --- a/pkg/spanconfig/spanconfigreporter/BUILD.bazel +++ b/pkg/spanconfig/spanconfigreporter/BUILD.bazel @@ -11,11 +11,11 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/constraint", + "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", - "//pkg/util/log", "//pkg/util/rangedesciter", "@com_github_cockroachdb_errors//:errors", ], @@ -31,7 +31,9 @@ go_test( data = glob(["testdata/**"]), deps = [ ":spanconfigreporter", + "//pkg/keys", "//pkg/kv/kvserver/constraint", + "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/spanconfig/spanconfigreporter/datadriven_test.go b/pkg/spanconfig/spanconfigreporter/datadriven_test.go index 15fe3f4ac288..00f36098bc32 100644 --- a/pkg/spanconfig/spanconfigreporter/datadriven_test.go +++ b/pkg/spanconfig/spanconfigreporter/datadriven_test.go @@ -17,7 +17,9 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" clustersettings "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" @@ -156,6 +158,9 @@ func TestDataDriven(t *testing.T) { } spans = append(spans, spanconfigtestutils.ParseSpan(t, line)) } + if len(spans) == 0 { + spans = append(spans, keys.EverythingSpan) + } report, err := reporter.SpanConfigConformance(ctx, spans) require.NoError(t, err) printRangeDesc := func(r roachpb.RangeDescriptor) string { @@ -176,13 +181,14 @@ func TestDataDriven(t *testing.T) { buf.WriteString("]") return buf.String() } - printList := func(tag string, descs []roachpb.RangeDescriptor) string { + printList := func(tag string, ranges []roachpb.ConformanceReportedRange) string { var buf strings.Builder - for i, desc := range descs { + for i, r := range ranges { if i == 0 { buf.WriteString(fmt.Sprintf("%s:\n", tag)) } - buf.WriteString(fmt.Sprintf(" %s\n", printRangeDesc(desc))) + buf.WriteString(fmt.Sprintf(" %s applying %s\n", printRangeDesc(r.RangeDescriptor), + spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(r.Config))) } return buf.String() } @@ -231,11 +237,15 @@ func newMockCluster( } } -// IsLive implements spanconfigreporter.Liveness. -func (s *mockCluster) IsLive(id roachpb.NodeID) (bool, error) { - live, found := s.liveness[id] - require.True(s.t, found, "undeclared node n%d", id) - return live, nil +// GetIsLiveMap implements spanconfigreporter.Liveness. +func (s *mockCluster) GetIsLiveMap() livenesspb.IsLiveMap { + isLiveMap := livenesspb.IsLiveMap{} + for nid, isLive := range s.liveness { + isLiveMap[nid] = livenesspb.IsLiveMapEntry{ + IsLive: isLive, + } + } + return isLiveMap } // GetStoreDescriptor implements constraint.StoreResolver. @@ -251,7 +261,7 @@ func (s *mockCluster) GetStoreDescriptor(storeID roachpb.StoreID) (roachpb.Store // Iterate implements rangedesciter.Iterator. func (s *mockCluster) Iterate( - _ context.Context, _ int, _ func(), fn func(...roachpb.RangeDescriptor) error, + _ context.Context, _ int, _ func(), _ roachpb.Span, fn func(...roachpb.RangeDescriptor) error, ) error { var descs []roachpb.RangeDescriptor for _, d := range s.ranges { diff --git a/pkg/spanconfig/spanconfigreporter/reporter.go b/pkg/spanconfig/spanconfigreporter/reporter.go index ad1d7f707372..c95c75e52889 100644 --- a/pkg/spanconfig/spanconfigreporter/reporter.go +++ b/pkg/spanconfig/spanconfigreporter/reporter.go @@ -17,11 +17,11 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" + "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/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/rangedesciter" ) @@ -43,7 +43,7 @@ var rangeDescPageSize = settings.RegisterIntSetting( // Liveness is the subset of the interface satisfied by CRDB's node liveness // component that the reporter relies on. type Liveness interface { - IsLive(roachpb.NodeID) (bool, error) + GetIsLiveMap() livenesspb.IsLiveMap } // Reporter is used to figure out whether ranges backing specific spans conform @@ -51,9 +51,30 @@ type Liveness interface { // the spanconfig.Reporter interface. type Reporter struct { dep struct { + // NB: The data dependencies in the implementation are: + // + // i. point-in-time view over gossip-backed node liveness; + // ii. point-in-time view of range descriptors (done transactionally); + // iii. the store resolver resolving store IDs to store descriptors; + // iv. view over what span configs apply to what keyspans; + // + // TODO(irfansharif): For (iii) and (iv) we might not have a + // point-in-time snapshot of the data. + // - For (iii) it's possible that as we iterate through the set of range + // descriptors, a few of which refer to some store S, we're racing + // against that newly-added store's info not yet being available + // through gossip. This is exceedingly unlikely, but if we see it + // happen, we can expose some snapshot of the StoreResolver state like + // we have for liveness. + // - For (iv) too we're not grabbing a read lock over the backing + // spanconfig.KVSubscriber while reading off each span config, so it's + // possible we generate the report for two range descriptors with span + // configs from different points in time. If this too becomes a + // problem, we can explicitly generate a snapshot like we do for + // liveness. Liveness - constraint.StoreResolver rangedesciter.Iterator + constraint.StoreResolver spanconfig.StoreReader } @@ -86,96 +107,111 @@ func New( // TODO(irfansharif): Support the equivalent of "critical localities", perhaps // through a different API than the one below since it's not quite // span-oriented. +// +// TODO(irfansharif): Once wired up the SQL code or exposed through an endpoint, +// write an end-to-end test using actual SQL and zone configs. Set configs on a +// table, disable replication, see conformance report. Enable repl, change +// configs, repeat. Do this for tenants as well. // SpanConfigConformance implements the spanconfig.Reporter interface. func (r *Reporter) SpanConfigConformance( ctx context.Context, spans []roachpb.Span, ) (roachpb.SpanConfigConformanceReport, error) { - // XXX: Actually use the spans parameter. Update the rangedesc.Iterator - // interfaces to take in a keyspan and bound meta{1,2} search just to - // segments that would possibly overlap with that keyspan. Until this - // keyspan scoping is done, we can't let this be used in tenants. - _ = spans - - // XXX: Write an end-to-end test using actual SQL and zone configs. Set configs - // on a table, disable replication, see conformance. Enable repl, change - // configs, etc. Use tenants as well for this mode. Do this for tenants as well. - // Do this after some form of this API is exposed through SQL/an endpoint. - - // XXX: Can we improve the SpanConfigConformanceReport proto type? Perhaps - // include some {meta,}data about the span config being violated as well? Or - // include the span config directly and provide helper libraries to compute - // human-readable "why is this in violation" text. - // - Only include range ID + replica descriptors + keys? - // - Type to represent exactly which constraint exactly is being violated? - // - Segment over/under replicated by what replica type (voter/non-voter) - // exactly is over/under replicated? - report := roachpb.SpanConfigConformanceReport{} - if err := r.dep.Iterate(ctx, int(rangeDescPageSize.Get(&r.settings.SV)), func() { - report = roachpb.SpanConfigConformanceReport{} // init - }, func(descriptors ...roachpb.RangeDescriptor) error { - for _, desc := range descriptors { - conf, err := r.dep.StoreReader.GetSpanConfigForKey(ctx, desc.StartKey) - if err != nil { - return err - } - - status := desc.Replicas().ReplicationStatus( - func(rDesc roachpb.ReplicaDescriptor) bool { - isLive, err := r.dep.Liveness.IsLive(rDesc.NodeID) + unavailableNodes := make(map[roachpb.NodeID]struct{}) + + isLiveMap := r.dep.Liveness.GetIsLiveMap() + for _, span := range spans { + if err := r.dep.Iterate(ctx, int(rangeDescPageSize.Get(&r.settings.SV)), + func() { report = roachpb.SpanConfigConformanceReport{} /* init */ }, + span, + func(descriptors ...roachpb.RangeDescriptor) error { + for _, desc := range descriptors { + conf, err := r.dep.StoreReader.GetSpanConfigForKey(ctx, desc.StartKey) if err != nil { - // As of 2022-10, this error only appears if we're - // asking for the liveness of a node ID that doesn't - // exist, which should never happen. Shout loudly - // and declare things as non-live. - log.Errorf(ctx, "programming error: unexpected err: %v", err) - return false + return err } - return isLive - }, int(conf.GetNumVoters()), int(conf.GetNumNonVoters())) - if !status.Available { - report.Unavailable = append(report.Unavailable, desc) - } - if status.UnderReplicated || status.UnderReplicatedNonVoters { - report.UnderReplicated = append(report.UnderReplicated, desc) - } - if status.OverReplicated || status.OverReplicatedNonVoters { - report.OverReplicated = append(report.OverReplicated, desc) - } - // Compute constraint violations for the overall (affecting voters - // and non-voters alike) and voter constraints. - overall := constraint.AnalyzeConstraints( - r.dep.StoreResolver, - desc.Replicas().Descriptors(), - conf.NumReplicas, conf.Constraints) - for i, c := range overall.Constraints { - if c.NumReplicas == 0 { - c.NumReplicas = conf.NumReplicas - } - if len(overall.SatisfiedBy[i]) < int(c.NumReplicas) { - report.ViolatingConstraints = append(report.ViolatingConstraints, desc) - break - } - } - voters := constraint.AnalyzeConstraints( - r.dep.StoreResolver, - desc.Replicas().Voters().Descriptors(), - conf.GetNumVoters(), conf.VoterConstraints) - for i, c := range voters.Constraints { - if c.NumReplicas == 0 { - c.NumReplicas = conf.GetNumVoters() - } - if len(voters.SatisfiedBy[i]) < int(c.NumReplicas) { - report.ViolatingConstraints = append(report.ViolatingConstraints, desc) - break + status := desc.Replicas().ReplicationStatus( + func(rDesc roachpb.ReplicaDescriptor) bool { + isLive := isLiveMap[rDesc.NodeID].IsLive + if !isLive { + unavailableNodes[rDesc.NodeID] = struct{}{} + } + return isLive + }, int(conf.GetNumVoters()), int(conf.GetNumNonVoters())) + if !status.Available { + report.Unavailable = append(report.Unavailable, + roachpb.ConformanceReportedRange{ + RangeDescriptor: desc, + Config: conf, + }) + } + if status.UnderReplicated || status.UnderReplicatedNonVoters { + report.UnderReplicated = append(report.UnderReplicated, + roachpb.ConformanceReportedRange{ + RangeDescriptor: desc, + Config: conf, + }) + } + if status.OverReplicated || status.OverReplicatedNonVoters { + report.OverReplicated = append(report.OverReplicated, + roachpb.ConformanceReportedRange{ + RangeDescriptor: desc, + Config: conf, + }) + } + + // Compute constraint violations for the overall (affecting voters + // and non-voters alike) and voter constraints. + overall := constraint.AnalyzeConstraints( + r.dep.StoreResolver, + desc.Replicas().Descriptors(), + conf.NumReplicas, conf.Constraints) + for i, c := range overall.Constraints { + if c.NumReplicas == 0 { + // NB: This is a weird artifact of + // constraint.NumReplicas, which if set to zero is + // used to imply that the constraint will applies to + // all replicas. Setting it explicitly makes the + // code below less fragile. + c.NumReplicas = conf.NumReplicas + } + if len(overall.SatisfiedBy[i]) < int(c.NumReplicas) { + report.ViolatingConstraints = append(report.ViolatingConstraints, + roachpb.ConformanceReportedRange{ + RangeDescriptor: desc, + Config: conf, + }) + break + } + } + voters := constraint.AnalyzeConstraints( + r.dep.StoreResolver, + desc.Replicas().Voters().Descriptors(), + conf.GetNumVoters(), conf.VoterConstraints) + for i, c := range voters.Constraints { + if c.NumReplicas == 0 { + c.NumReplicas = conf.GetNumVoters() + } + if len(voters.SatisfiedBy[i]) < int(c.NumReplicas) { + report.ViolatingConstraints = append(report.ViolatingConstraints, + roachpb.ConformanceReportedRange{ + RangeDescriptor: desc, + Config: conf, + }) + break + } + } } - } + return nil + }); err != nil { + return roachpb.SpanConfigConformanceReport{}, err } - return nil - }); err != nil { - return roachpb.SpanConfigConformanceReport{}, err + } + + for nid := range unavailableNodes { + report.UnavailableNodeIDs = append(report.UnavailableNodeIDs, int32(nid)) } return report, nil } diff --git a/pkg/spanconfig/spanconfigreporter/testdata/basic b/pkg/spanconfig/spanconfigreporter/testdata/basic index 144b03dbe909..9a6e73024a32 100644 --- a/pkg/spanconfig/spanconfigreporter/testdata/basic +++ b/pkg/spanconfig/spanconfigreporter/testdata/basic @@ -45,7 +45,7 @@ n6: dead report ---- under replicated: - r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] applying range default liveness n4: dead @@ -54,9 +54,9 @@ n4: dead report ---- unavailable: - r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] applying range default under replicated: - r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] applying range default liveness n4: live @@ -76,7 +76,7 @@ r2: voters=[n1,n3,n5,n6] report ---- over replicated: - r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5, (n6,s6):6] + r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5, (n6,s6):6] applying range default # It should also work when we don't have enough replicas. allocate @@ -86,7 +86,7 @@ r2: voters=[n1] report ---- under replicated: - r2:{b-c} [(n1,s1):1] + r2:{b-c} [(n1,s1):1] applying range default allocate r2: voters=[n1,n3,n5] @@ -109,9 +109,9 @@ configure report ---- under replicated: - r3:{c-d} [(n1,s1):1, (n3,s3):3, (n5,s5):5] + r3:{c-d} [(n1,s1):1, (n3,s3):3, (n5,s5):5] applying range system over replicated: - r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] + r1:{a-b} [(n2,s2):2, (n4,s4):4, (n6,s6):6] applying num_replicas=1 configure [a,d): num_replicas=3 @@ -135,7 +135,7 @@ r2: voters=[n1,n3,n5] report ---- under replicated: - r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5] + r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5] applying num_replicas=6 num_voters=3 allocate r2: voters=[n1,n2,n3,n4,n5,n6] @@ -145,9 +145,9 @@ r2: voters=[n1,n2,n3,n4,n5,n6] report ---- under replicated: - r2:{b-c} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4, (n5,s5):5, (n6,s6):6] + r2:{b-c} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4, (n5,s5):5, (n6,s6):6] applying num_replicas=6 num_voters=3 over replicated: - r2:{b-c} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4, (n5,s5):5, (n6,s6):6] + r2:{b-c} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4, (n5,s5):5, (n6,s6):6] applying num_replicas=6 num_voters=3 allocate r2: voters=[n1,n3,n5] non-voters=[n2,n4,n6] @@ -178,7 +178,7 @@ configure report ---- violating constraints: - r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5] + r2:{b-c} [(n1,s1):1, (n3,s3):3, (n5,s5):5] applying constraints=[+region=us-central:2] allocate r2: voters=[n1,n3,n4] diff --git a/pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance b/pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance index 942091d86e23..105512830db0 100644 --- a/pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance +++ b/pkg/spanconfig/spanconfigreporter/testdata/constraint_conformance @@ -24,7 +24,7 @@ r1: voters=[n1,n2,n4] report ---- violating constraints: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n4,s4):4] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n4,s4):4] applying constraints=[+region=us-west:3] # Pin replicas to two specific DCs. A conforming replica placement should show # up as such. @@ -53,7 +53,7 @@ r1: voters=[n1] non-voters=[n3] report ---- violating constraints: - r1:{a-b} [(n1,s1):1, (n3,s3):3NON_VOTER] + r1:{a-b} [(n1,s1):1, (n3,s3):3NON_VOTER] applying num_replicas=2 num_voters=1 constraints=[+dc=dc-a:1 +dc=dc-d:1] voter_constraints=[+dc=dc-a:1] allocate r1: voters=[n1] non-voters=[n4] @@ -76,7 +76,7 @@ r1: voters=[n1,n3,n5] report ---- violating constraints: - r1:{a-b} [(n1,s1):1, (n3,s3):3, (n5,s5):5] + r1:{a-b} [(n1,s1):1, (n3,s3):3, (n5,s5):5] applying constraints=[-region=us-west] allocate r1: voters=[n4,n5,n6] diff --git a/pkg/spanconfig/spanconfigreporter/testdata/joint_consensus b/pkg/spanconfig/spanconfigreporter/testdata/joint_consensus index bc0b4e09704c..a0e87cfff9a0 100644 --- a/pkg/spanconfig/spanconfigreporter/testdata/joint_consensus +++ b/pkg/spanconfig/spanconfigreporter/testdata/joint_consensus @@ -36,7 +36,7 @@ r1: voters=[n1,n2] voters-incoming=[n3] report ---- under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_INCOMING] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_INCOMING] applying range default # Under-replication in the "new group". allocate @@ -46,7 +46,7 @@ r1: voters=[n1,n2] voters-outgoing=[n3] report ---- under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING] applying range default # Under-replication in the old group because 4 is dead. allocate @@ -56,7 +56,7 @@ r1: voters=[n1,n2] voters-outgoing=[n4] voters-incoming=[n3] report ---- under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n4,s4):4VOTER_OUTGOING, (n3,s3):3VOTER_INCOMING] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n4,s4):4VOTER_OUTGOING, (n3,s3):3VOTER_INCOMING] applying range default # Unavailable in the new group (and also under-replicated), and also # over-replicated in the new group. @@ -67,11 +67,11 @@ r1: voters=[n1,n2] voters-outgoing=[n3] voters-incoming=[n4,n5] report ---- unavailable: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] applying range default under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] applying range default over replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n4,s4):4VOTER_INCOMING, (n5,s5):5VOTER_INCOMING] applying range default # Over-replicated in the new group. allocate @@ -81,7 +81,7 @@ r1: voters=[n1,n2] voters-outgoing=[n3] voters-incoming=[n5,n6] report ---- over replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n5,s5):5VOTER_INCOMING, (n6,s6):6VOTER_INCOMING] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3VOTER_OUTGOING, (n5,s5):5VOTER_INCOMING, (n6,s6):6VOTER_INCOMING] applying range default # Many learners. No problems, since learners don't count. @@ -101,4 +101,4 @@ r1: voters=[n1,n2] learners=[n3] report ---- under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3LEARNER] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3LEARNER] applying range default diff --git a/pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated b/pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated index 5d73280c2b2d..3663de56825b 100644 --- a/pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated +++ b/pkg/spanconfig/spanconfigreporter/testdata/over_under_replicated @@ -31,7 +31,7 @@ r1: voters=[n1,n2,n3,n4] report ---- over replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3, (n4,s4):4] applying range default # ----------------------------------------------------------------------------- # We have 1 or 2 replicas when we want 3, we're under replicated. @@ -42,7 +42,7 @@ r1: voters=[n1] report ---- under replicated: - r1:{a-b} [(n1,s1):1] + r1:{a-b} [(n1,s1):1] applying range default allocate r1: voters=[n1,n2] @@ -51,7 +51,7 @@ r1: voters=[n1,n2] report ---- under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2] + r1:{a-b} [(n1,s1):1, (n2,s2):2] applying range default # ----------------------------------------------------------------------------- # We have the desired number of replicas, but one of them is on a dead node so @@ -67,7 +67,7 @@ r1: voters=[n1,n2,n3] report ---- under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] applying range default # If we've lost quorum we're also unavailable. liveness @@ -78,9 +78,9 @@ n3: dead report ---- unavailable: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] applying range default under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] applying range default liveness n2: live @@ -113,8 +113,8 @@ n2: dead report ---- unavailable: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] applying num_replicas=2 under replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] applying num_replicas=2 over replicated: - r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] + r1:{a-b} [(n1,s1):1, (n2,s2):2, (n3,s3):3] applying num_replicas=2