diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index 86f6a0d8c8bd..bd0b212fd68b 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -275,7 +275,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { require.Equal(t, []roachpb.ReplicaDescriptor{ {NodeID: 1, StoreID: 1, ReplicaID: 1}, {NodeID: 2, StoreID: 2, ReplicaID: 2}, - }, entry.Desc().Replicas().All()) + }, entry.Desc().Replicas().Descriptors()) // Relocate the follower. n2 will no longer have a replica. n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,3], 1)`) @@ -296,7 +296,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { require.Equal(t, []roachpb.ReplicaDescriptor{ {NodeID: 1, StoreID: 1, ReplicaID: 1}, {NodeID: 3, StoreID: 3, ReplicaID: 3}, - }, entry.Desc().Replicas().All()) + }, entry.Desc().Replicas().Descriptors()) // Make a note of the follower reads metric on n3. We'll check that it was // incremented. diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 6647b9c4c6bb..17b75d2ddda6 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -955,7 +955,7 @@ func removeDeadReplicas( err = kvserver.IterateRangeDescriptors(ctx, db, func(desc roachpb.RangeDescriptor) error { hasSelf := false numDeadPeers := 0 - allReplicas := desc.Replicas().All() + allReplicas := desc.Replicas().Descriptors() maxLivePeer := roachpb.StoreID(-1) for _, rep := range allReplicas { if rep.StoreID == storeIdent.StoreID { @@ -998,7 +998,7 @@ func removeDeadReplicas( StoreID: storeIdent.StoreID, ReplicaID: desc.NextReplicaID, }} - newDesc.SetReplicas(roachpb.MakeReplicaDescriptors(replicas)) + newDesc.SetReplicas(roachpb.MakeReplicaSet(replicas)) newDesc.NextReplicaID++ fmt.Printf("Replica %s -> %s\n", &desc, &newDesc) newDescs = append(newDescs, newDesc) diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index f89954b79eac..407e14f5327c 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -715,7 +715,9 @@ func (b *Batch) adminChangeReplicas( // adminRelocateRange is only exported on DB. It is here for symmetry with the // other operations. -func (b *Batch) adminRelocateRange(key interface{}, targets []roachpb.ReplicationTarget) { +func (b *Batch) adminRelocateRange( + key interface{}, voterTargets, nonVoterTargets []roachpb.ReplicationTarget, +) { k, err := marshalKey(key) if err != nil { b.initResult(0, 0, notRaw, err) @@ -725,7 +727,8 @@ func (b *Batch) adminRelocateRange(key interface{}, targets []roachpb.Replicatio RequestHeader: roachpb.RequestHeader{ Key: k, }, - Targets: targets, + VoterTargets: voterTargets, + NonVoterTargets: nonVoterTargets, } b.appendReqs(req) b.initResult(1, 0, notRaw, nil) diff --git a/pkg/kv/db.go b/pkg/kv/db.go index cb3376eae96a..3794fe5e2752 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -649,10 +649,10 @@ func (db *DB) AdminChangeReplicas( // AdminRelocateRange relocates the replicas for a range onto the specified // list of stores. func (db *DB) AdminRelocateRange( - ctx context.Context, key interface{}, targets []roachpb.ReplicationTarget, + ctx context.Context, key interface{}, voterTargets, nonVoterTargets []roachpb.ReplicationTarget, ) error { b := &Batch{} - b.adminRelocateRange(key, targets) + b.adminRelocateRange(key, voterTargets, nonVoterTargets) return getOneErr(db.Run(ctx, b), b) } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index 6a0cf1f2e955..4869167f33b6 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -586,7 +586,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - recognizedLeaseHolder := testUserRangeDescriptor3Replicas.Replicas().Voters()[1] + recognizedLeaseHolder := testUserRangeDescriptor3Replicas.Replicas().VoterDescriptors()[1] unrecognizedLeaseHolder := roachpb.ReplicaDescriptor{ NodeID: 99, StoreID: 999, @@ -642,7 +642,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) rpcContext := rpc.NewInsecureTestingContext(clock, stopper) g := makeGossip(t, stopper, rpcContext) - for _, n := range testUserRangeDescriptor3Replicas.Replicas().Voters() { + for _, n := range testUserRangeDescriptor3Replicas.Replicas().VoterDescriptors() { require.NoError(t, g.AddInfoProto( gossip.MakeNodeIDKey(n.NodeID), newNodeDesc(n.NodeID), @@ -823,7 +823,7 @@ func TestDistSenderMovesOnFromReplicaWithStaleLease(t *testing.T) { clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) rpcContext := rpc.NewInsecureTestingContext(clock, stopper) g := makeGossip(t, stopper, rpcContext) - for _, n := range testUserRangeDescriptor3Replicas.Replicas().Voters() { + for _, n := range testUserRangeDescriptor3Replicas.Replicas().VoterDescriptors() { require.NoError(t, g.AddInfoProto( gossip.MakeNodeIDKey(n.NodeID), newNodeDesc(n.NodeID), @@ -4329,17 +4329,17 @@ func TestDistSenderDescEvictionAfterLeaseUpdate(t *testing.T) { br := &roachpb.BatchResponse{} switch call { case 0: - expRepl := desc1.Replicas().All()[0] + expRepl := desc1.Replicas().Descriptors()[0] require.Equal(t, expRepl, ba.Replica) br.Error = roachpb.NewError(&roachpb.NotLeaseHolderError{ - Lease: &roachpb.Lease{Replica: desc1.Replicas().All()[1]}, + Lease: &roachpb.Lease{Replica: desc1.Replicas().Descriptors()[1]}, }) case 1: - expRep := desc1.Replicas().All()[1] + expRep := desc1.Replicas().Descriptors()[1] require.Equal(t, ba.Replica, expRep) br.Error = roachpb.NewError(roachpb.NewRangeNotFoundError(ba.RangeID, ba.Replica.StoreID)) case 2: - expRep := desc2.Replicas().All()[0] + expRep := desc2.Replicas().Descriptors()[0] require.Equal(t, ba.Replica, expRep) br = ba.CreateReply() default: @@ -4420,7 +4420,7 @@ func TestDistSenderRPCMetrics(t *testing.T) { br := &roachpb.BatchResponse{} if call == 0 { br.Error = roachpb.NewError(&roachpb.NotLeaseHolderError{ - Lease: &roachpb.Lease{Replica: desc.Replicas().All()[1]}, + Lease: &roachpb.Lease{Replica: desc.Replicas().Descriptors()[1]}, }) } else { br.Error = roachpb.NewError(&roachpb.ConditionFailedError{}) @@ -4449,7 +4449,7 @@ func TestDistSenderRPCMetrics(t *testing.T) { ds.rangeCache.Insert(ctx, roachpb.RangeInfo{ Desc: desc, Lease: roachpb.Lease{ - Replica: desc.Replicas().All()[0], + Replica: desc.Replicas().Descriptors()[0], }, }) var ba roachpb.BatchRequest diff --git a/pkg/kv/kvclient/kvcoord/replica_slice.go b/pkg/kv/kvclient/kvcoord/replica_slice.go index 7bfef63fb934..9617ecb63296 100644 --- a/pkg/kv/kvclient/kvcoord/replica_slice.go +++ b/pkg/kv/kvclient/kvcoord/replica_slice.go @@ -67,11 +67,11 @@ func NewReplicaSlice( } // Learner replicas won't serve reads/writes, so we'll send only to the - // `Voters` replicas. This is just an optimization to save a network hop, + // `VoterDescriptors` replicas. This is just an optimization to save a network hop, // everything would still work if we had `All` here. - voters := desc.Replicas().Voters() + voters := desc.Replicas().VoterDescriptors() // If we know a leaseholder, though, let's make sure we include it. - if leaseholder != nil && len(voters) < len(desc.Replicas().All()) { + if leaseholder != nil && len(voters) < len(desc.Replicas().Descriptors()) { found := false for _, v := range voters { if v == *leaseholder { diff --git a/pkg/kv/kvnemesis/applier.go b/pkg/kv/kvnemesis/applier.go index aa9844ad183c..305563e2ef2e 100644 --- a/pkg/kv/kvnemesis/applier.go +++ b/pkg/kv/kvnemesis/applier.go @@ -259,7 +259,7 @@ func newGetReplicasFn(dbs ...*kv.DB) GetReplicasFn { ctx := context.Background() return func(key roachpb.Key) []roachpb.ReplicationTarget { desc := getRangeDesc(ctx, key, dbs...) - replicas := desc.Replicas().All() + replicas := desc.Replicas().Descriptors() targets := make([]roachpb.ReplicationTarget, len(replicas)) for i, replica := range replicas { targets[i] = roachpb.ReplicationTarget{ diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 465960a68402..b84808fd20d0 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -98,6 +98,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/build", "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", diff --git a/pkg/kv/kvserver/allocator.go b/pkg/kv/kvserver/allocator.go index dfeff404672b..ca3b3665d346 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -346,7 +346,7 @@ func (a *Allocator) ComputeAction( // On the other hand if we get the race where a leaseholder starts adding a // replica in the replicate queue and during this loses its lease, it should // probably not retry. - if learners := desc.Replicas().Learners(); len(learners) > 0 { + if learners := desc.Replicas().LearnerDescriptors(); len(learners) > 0 { // TODO(dan): Since this goes before anything else, the priority here should // be influenced by whatever operations would happen right after the learner // is removed. In the meantime, we don't want to block something important @@ -356,7 +356,7 @@ func (a *Allocator) ComputeAction( return AllocatorRemoveLearner, removeLearnerReplicaPriority } // computeAction expects to operate only on voters. - return a.computeAction(ctx, zone, desc.Replicas().Voters()) + return a.computeAction(ctx, zone, desc.Replicas().VoterDescriptors()) } func (a *Allocator) computeAction( @@ -495,17 +495,17 @@ func (a *Allocator) AllocateTarget( func (a *Allocator) allocateTargetFromList( ctx context.Context, - sl StoreList, + candidateStores StoreList, zone *zonepb.ZoneConfig, - candidateReplicas []roachpb.ReplicaDescriptor, + existingReplicas []roachpb.ReplicaDescriptor, options scorerOptions, ) (*roachpb.StoreDescriptor, string) { analyzedConstraints := constraint.AnalyzeConstraints( - ctx, a.storePool.getStoreDescriptor, candidateReplicas, zone) + ctx, a.storePool.getStoreDescriptor, existingReplicas, zone) candidates := allocateCandidates( ctx, - sl, analyzedConstraints, candidateReplicas, - a.storePool.getLocalitiesByStore(candidateReplicas), + candidateStores, analyzedConstraints, existingReplicas, + a.storePool.getLocalitiesByStore(existingReplicas), a.storePool.isNodeReadyForRoutineReplicaTransfer, options, ) @@ -560,17 +560,17 @@ func (a Allocator) RemoveTarget( } // Retrieve store descriptors for the provided candidates from the StorePool. - existingStoreIDs := make(roachpb.StoreIDSlice, len(candidates)) + candidateStoreIDs := make(roachpb.StoreIDSlice, len(candidates)) for i, exist := range candidates { - existingStoreIDs[i] = exist.StoreID + candidateStoreIDs[i] = exist.StoreID } - sl, _, _ := a.storePool.getStoreListFromIDs(existingStoreIDs, storeFilterNone) + candidateStoreList, _, _ := a.storePool.getStoreListFromIDs(candidateStoreIDs, storeFilterNone) analyzedConstraints := constraint.AnalyzeConstraints( ctx, a.storePool.getStoreDescriptor, existingReplicas, zone) options := a.scorerOptions() rankedCandidates := removeCandidates( - sl, + candidateStoreList, analyzedConstraints, a.storePool.getLocalitiesByStore(existingReplicas), options, diff --git a/pkg/kv/kvserver/allocator_scorer.go b/pkg/kv/kvserver/allocator_scorer.go index 1da2292388d5..3e0789b570da 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -409,16 +409,16 @@ func (cl candidateList) removeCandidate(c candidate) candidateList { // stores that meet the criteria are included in the list. func allocateCandidates( ctx context.Context, - sl StoreList, + candidateStores StoreList, constraints constraint.AnalyzedConstraints, - existing []roachpb.ReplicaDescriptor, + existingReplicas []roachpb.ReplicaDescriptor, existingStoreLocalities map[roachpb.StoreID]roachpb.Locality, isNodeValidForRoutineReplicaTransfer func(context.Context, roachpb.NodeID) bool, options scorerOptions, ) candidateList { var candidates candidateList - for _, s := range sl.stores { - if nodeHasReplica(s.Node.NodeID, existing) { + for _, s := range candidateStores.stores { + if nodeHasReplica(s.Node.NodeID, existingReplicas) { continue } if !isNodeValidForRoutineReplicaTransfer(ctx, s.Node.NodeID) { @@ -433,14 +433,14 @@ func allocateCandidates( continue } diversityScore := diversityAllocateScore(s, existingStoreLocalities) - balanceScore := balanceScore(sl, s.Capacity, options) + balanceScore := balanceScore(candidateStores, s.Capacity, options) var convergesScore int if options.qpsRebalanceThreshold > 0 { - if s.Capacity.QueriesPerSecond < underfullThreshold(sl.candidateQueriesPerSecond.mean, options.qpsRebalanceThreshold) { + if s.Capacity.QueriesPerSecond < underfullThreshold(candidateStores.candidateQueriesPerSecond.mean, options.qpsRebalanceThreshold) { convergesScore = 1 - } else if s.Capacity.QueriesPerSecond < sl.candidateQueriesPerSecond.mean { + } else if s.Capacity.QueriesPerSecond < candidateStores.candidateQueriesPerSecond.mean { convergesScore = 0 - } else if s.Capacity.QueriesPerSecond < overfullThreshold(sl.candidateQueriesPerSecond.mean, options.qpsRebalanceThreshold) { + } else if s.Capacity.QueriesPerSecond < overfullThreshold(candidateStores.candidateQueriesPerSecond.mean, options.qpsRebalanceThreshold) { convergesScore = -1 } else { convergesScore = -2 diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_test.go b/pkg/kv/kvserver/batcheval/cmd_lease_test.go index e3c205b86158..ec707f9d4f8e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_test.go @@ -119,7 +119,7 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { {NodeID: 2, StoreID: learnerStoreID, Type: roachpb.ReplicaTypeLearner(), ReplicaID: 2}, } desc := roachpb.RangeDescriptor{} - desc.SetReplicas(roachpb.MakeReplicaDescriptors(replicas)) + desc.SetReplicas(roachpb.MakeReplicaSet(replicas)) cArgs := CommandArgs{ EvalCtx: (&MockEvalCtx{ StoreID: voterStoreID, diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 21e3dc1df5b3..787e579cf00f 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -18,6 +18,7 @@ import ( "math/rand" "reflect" "regexp" + "sort" "strconv" "strings" "sync" @@ -1989,14 +1990,14 @@ func TestStoreRangeMergeAddReplicaRace(t *testing.T) { afterDesc := tc.LookupRangeOrFatal(t, scratchStartKey) const acceptableMergeErr = `unexpected value: raw_bytes|ranges not collocated` + - `|cannot merge range with non-voter replicas` + `|is in a joint state or has learners` if mergeErr == nil && kvserver.IsRetriableReplicationChangeError(addErr) { // Merge won the race, no add happened. - require.Len(t, afterDesc.Replicas().Voters(), 1) + require.Len(t, afterDesc.Replicas().VoterDescriptors(), 1) require.Equal(t, origDesc.EndKey, afterDesc.EndKey) } else if addErr == nil && testutils.IsError(mergeErr, acceptableMergeErr) { // Add won the race, no merge happened. - require.Len(t, afterDesc.Replicas().Voters(), 2) + require.Len(t, afterDesc.Replicas().VoterDescriptors(), 2) require.Equal(t, beforeDesc.EndKey, afterDesc.EndKey) } else { t.Fatalf(`expected exactly one of merge or add to succeed got: [merge] %v [add] %v`, @@ -2032,7 +2033,7 @@ func TestStoreRangeMergeResplitAddReplicaRace(t *testing.T) { assert.Equal(t, origDesc.RangeID, resplitDesc.RangeID) assert.Equal(t, origDesc.StartKey, resplitDesc.StartKey) assert.Equal(t, origDesc.EndKey, resplitDesc.EndKey) - assert.Equal(t, origDesc.Replicas().All(), resplitDesc.Replicas().All()) + assert.Equal(t, origDesc.Replicas().Descriptors(), resplitDesc.Replicas().Descriptors()) assert.NotEqual(t, origDesc.Generation, resplitDesc.Generation) _, err := tc.Server(0).DB().AdminChangeReplicas( @@ -3475,6 +3476,22 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) { } } +func verifyMerged(t *testing.T, store *kvserver.Store, lhsStartKey, rhsStartKey roachpb.RKey) { + t.Helper() + repl := store.LookupReplica(rhsStartKey) + if !repl.Desc().StartKey.Equal(lhsStartKey) { + t.Fatalf("ranges unexpectedly unmerged") + } +} + +func verifyUnmerged(t *testing.T, store *kvserver.Store, lhsStartKey, rhsStartKey roachpb.RKey) { + t.Helper() + repl := store.LookupReplica(rhsStartKey) + if repl.Desc().StartKey.Equal(lhsStartKey) { + t.Fatalf("ranges unexpectedly merged") + } +} + func TestMergeQueue(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -3555,37 +3572,21 @@ func TestMergeQueue(t *testing.T) { split(t, roachpb.Key("b"), hlc.Timestamp{} /* expirationTime */) } - verifyMerged := func(t *testing.T) { - t.Helper() - repl := store.LookupReplica(rhsStartKey) - if !repl.Desc().StartKey.Equal(lhsStartKey) { - t.Fatalf("ranges unexpectedly unmerged") - } - } - - verifyUnmerged := func(t *testing.T) { - t.Helper() - repl := store.LookupReplica(rhsStartKey) - if repl.Desc().StartKey.Equal(lhsStartKey) { - t.Fatalf("ranges unexpectedly merged") - } - } - t.Run("sanity", func(t *testing.T) { // Check that ranges are not trivially merged after reset. reset(t) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) reset(t) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) }) t.Run("both-empty", func(t *testing.T) { reset(t) clearRange(t, lhsStartKey, rhsEndKey) store.MustForceMergeScanAndProcess() - verifyMerged(t) + verifyMerged(t, store, lhsStartKey, rhsStartKey) }) t.Run("lhs-undersize", func(t *testing.T) { @@ -3594,7 +3595,7 @@ func TestMergeQueue(t *testing.T) { *zone.RangeMinBytes *= 2 lhs().SetZoneConfig(zone) store.MustForceMergeScanAndProcess() - verifyMerged(t) + verifyMerged(t, store, lhsStartKey, rhsStartKey) }) t.Run("combined-threshold", func(t *testing.T) { @@ -3607,18 +3608,18 @@ func TestMergeQueue(t *testing.T) { zone.RangeMaxBytes = proto.Int64(lhs().GetMVCCStats().Total()*2 - 1) setZones(*zone) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Once the maximum size threshold is increased, the merge can occur. zone.RangeMaxBytes = proto.Int64(*zone.RangeMaxBytes + 1) setZones(*zone) store.MustForceMergeScanAndProcess() - verifyMerged(t) + verifyMerged(t, store, lhsStartKey, rhsStartKey) }) t.Run("non-collocated", func(t *testing.T) { reset(t) - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) rhsRangeID := rhs().RangeID mtc.replicateRange(rhsRangeID, 1) mtc.transferLease(ctx, rhsRangeID, 0, 1) @@ -3627,7 +3628,7 @@ func TestMergeQueue(t *testing.T) { clearRange(t, lhsStartKey, rhsEndKey) store.MustForceMergeScanAndProcess() - verifyMerged(t) + verifyMerged(t, store, lhsStartKey, rhsStartKey) }) // TODO(jeffreyxiao): Add subtest to consider load when making merging @@ -3636,13 +3637,13 @@ func TestMergeQueue(t *testing.T) { t.Run("sticky-bit", func(t *testing.T) { reset(t) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Perform manual merge and verify that no merge occurred. split(t, rhsStartKey.AsRawKey(), hlc.MaxTimestamp /* expirationTime */) clearRange(t, lhsStartKey, rhsEndKey) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Delete sticky bit and verify that merge occurs. unsplitArgs := &roachpb.AdminUnsplitRequest{ @@ -3654,33 +3655,263 @@ func TestMergeQueue(t *testing.T) { t.Fatal(err) } store.MustForceMergeScanAndProcess() - verifyMerged(t) + verifyMerged(t, store, lhsStartKey, rhsStartKey) }) t.Run("sticky-bit-expiration", func(t *testing.T) { manualSplitTTL := time.Millisecond * 200 reset(t) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Perform manual merge and verify that no merge occurred. split(t, rhsStartKey.AsRawKey(), clock.Now().Add(manualSplitTTL.Nanoseconds(), 0) /* expirationTime */) clearRange(t, lhsStartKey, rhsEndKey) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Sticky bit is not expired yet. manualClock.Set(manualSplitTTL.Nanoseconds()) store.MustForceMergeScanAndProcess() - verifyUnmerged(t) + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Sticky bit is expired. manualClock.Set(manualSplitTTL.Nanoseconds() * 2) store.MustForceMergeScanAndProcess() - verifyMerged(t) + verifyMerged(t, store, lhsStartKey, rhsStartKey) }) } +func TestMergeQueueSeesNonVoters(t *testing.T) { + defer leaktest.AfterTest(t)() + + type test struct { + name string + leftVoters, rightVoters, leftNonVoters, rightNonVoters []int + expectedRightVoters, expectedRightNonVoters []int + } + + // NB: The test setup code places a single voter replica on (n1,s1) for both + // left and right range, which we remove after setting the test up. + tests := []test{ + { + name: "collocated-per-type", + leftVoters: []int{2, 3, 4}, + rightVoters: []int{2, 3, 4}, + leftNonVoters: []int{1}, + rightNonVoters: []int{1}, + expectedRightVoters: []int{2, 3, 4}, + expectedRightNonVoters: []int{1}, + }, + { + name: "collocated-overall", + leftVoters: []int{3, 4}, + rightVoters: []int{1, 2}, + leftNonVoters: []int{1, 2}, + rightNonVoters: []int{3, 4}, + expectedRightVoters: []int{1, 2}, + expectedRightNonVoters: []int{3, 4}, + }, + { + name: "collocated-voters-only", + leftVoters: []int{3, 4}, + rightVoters: []int{3, 4}, + leftNonVoters: []int{2}, + rightNonVoters: []int{1}, + expectedRightVoters: []int{3, 4}, + expectedRightNonVoters: []int{2}, + }, + { + name: "collocated-non-voters-only", + leftVoters: []int{3}, + rightVoters: []int{4}, + leftNonVoters: []int{1, 2}, + rightNonVoters: []int{1, 2}, + expectedRightVoters: []int{3}, + expectedRightNonVoters: []int{1, 2}, + }, + { + name: "not-collocated", + leftVoters: []int{3}, + rightVoters: []int{4}, + leftNonVoters: []int{2}, + rightNonVoters: []int{1}, + expectedRightVoters: []int{3}, + expectedRightNonVoters: []int{2}, + }, + { + name: "partially-collocated-voters-only", + leftVoters: []int{2, 3}, + rightVoters: []int{1, 4}, + leftNonVoters: []int{1}, + rightNonVoters: []int{2}, + expectedRightVoters: []int{1, 3}, + expectedRightNonVoters: []int{2}, + }, + { + name: "partially-collocated-non-voters-only", + leftVoters: []int{4}, + rightVoters: []int{4}, + leftNonVoters: []int{1, 3}, + rightNonVoters: []int{1, 2}, + expectedRightVoters: []int{4}, + expectedRightNonVoters: []int{1, 3}, + }, + { + name: "partially-collocated", + leftVoters: []int{2}, + rightVoters: []int{4}, + leftNonVoters: []int{1, 3}, + rightNonVoters: []int{1, 2}, + expectedRightVoters: []int{3}, + expectedRightNonVoters: []int{1, 2}, + }, + { + name: "collocated-rhs-being-reconfigured-1", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1, 2, 3, 4, 5, 6}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{}, + expectedRightVoters: []int{1, 2, 3, 4, 5, 6}, + expectedRightNonVoters: []int{}, + }, + { + name: "collocated-rhs-being-reconfigured-2", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1, 2, 3, 4}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{}, + expectedRightVoters: []int{1, 2, 3, 4}, + expectedRightNonVoters: []int{5, 6}, + }, + { + name: "collocated-rhs-being-reconfigured-3", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{2, 3, 4, 5, 6}, + expectedRightVoters: []int{1}, + expectedRightNonVoters: []int{2, 3, 4, 5, 6}, + }, + { + name: "non-collocated-rhs-being-reconfigured", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{5}, + leftNonVoters: []int{4, 6}, + rightNonVoters: []int{}, + expectedRightVoters: []int{1, 2, 3}, + expectedRightNonVoters: []int{4, 6}, + }, + { + name: "partially-collocated-rhs-being-downreplicated", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1, 2, 3, 4, 5, 6}, + leftNonVoters: []int{4, 5}, + rightNonVoters: []int{}, + expectedRightVoters: []int{1, 2, 3, 4, 5}, + expectedRightNonVoters: []int{}, + }, + { + name: "partially-collocated-rhs-being-upreplicated", + leftVoters: []int{1, 2, 3}, + rightVoters: []int{1}, + leftNonVoters: []int{4, 5, 6}, + rightNonVoters: []int{}, + expectedRightVoters: []int{1, 2, 3}, + expectedRightNonVoters: []int{4, 5, 6}, + }, + } + + var clusterArgs = base.TestClusterArgs{ + // We dont want the replicate queue mucking with our test, so disable it. + ReplicationMode: base.ReplicationManual, + } + ctx := context.Background() + + dbName := "testdb" + numNodes := 7 + for _, subtest := range tests { + t.Run(subtest.name, func(t *testing.T) { + tc, _ := setupTestClusterWithDummyRange(t, clusterArgs, dbName, "kv", numNodes) + defer tc.Stopper().Stop(ctx) + // We're controlling merge queue operation via + // `store.SetMergeQueueActive`, so enable the cluster setting here. + _, err := tc.ServerConn(0).Exec(`SET CLUSTER SETTING kv.range_merge.queue_enabled=true`) + require.NoError(t, err) + + store, err := tc.Server(0).GetStores().(*kvserver.Stores).GetStore(1) + require.Nil(t, err) + // We're going to split the dummy range created above with an empty + // expiration time. Disable the merge queue before splitting so that the + // split ranges aren't immediately merged. + store.SetMergeQueueActive(false) + leftDesc, rightDesc := splitDummyRangeInTestCluster( + t, tc, dbName, "kv" /* tableName */, hlc.Timestamp{} /* splitExpirationTime */) + + leftRepls := leftDesc.Replicas().Descriptors() + rightRepls := rightDesc.Replicas().Descriptors() + + require.Equal(t, 1, len(leftRepls)) + require.Equal(t, 1, len(rightRepls)) + + for i, id := range subtest.leftVoters { + tc.AddVotersOrFatal(t, leftDesc.StartKey.AsRawKey(), tc.Target(id)) + if i == 0 { + // Transfer range lease away from n1,s1 to the first voting replica we + // add. Otherwise we will fail when trying to remove the voting + // replica from n1,s1 below. + require.NoError(t, tc.TransferRangeLease(leftDesc, tc.Target(id))) + store, err = tc.Server(id).GetStores().(*kvserver.Stores).GetStore(roachpb.StoreID(id + 1)) + require.NoError(t, err) + } + } + for i, id := range subtest.rightVoters { + tc.AddVotersOrFatal(t, rightDesc.StartKey.AsRawKey(), tc.Target(id)) + if i == 0 { + // Transfer range lease away from n1,s1 to the first voting replica we + // add. Otherwise we will fail when trying to remove the voting + // replica from n1,s1 below. + require.NoError(t, tc.TransferRangeLease(rightDesc, tc.Target(id))) + } + } + for _, id := range subtest.leftNonVoters { + tc.AddNonVotersOrFatal(t, leftDesc.StartKey.AsRawKey(), tc.Target(id)) + } + for _, id := range subtest.rightNonVoters { + tc.AddNonVotersOrFatal(t, rightDesc.StartKey.AsRawKey(), tc.Target(id)) + } + + // Remove the voting replicas on n1,s1 that we started off with. + tc.RemoveVotersOrFatal(t, leftDesc.StartKey.AsRawKey(), tc.Target(0)) + leftDesc = tc.LookupRangeOrFatal(t, leftDesc.StartKey.AsRawKey()) + tc.RemoveVotersOrFatal(t, rightDesc.StartKey.AsRawKey(), tc.Target(0)) + rightDesc = tc.LookupRangeOrFatal(t, rightDesc.StartKey.AsRawKey()) + + // Check that we're avoiding superfluous data movement. + voterTargets, nonVoterTargets, err := kvserver.GetTargetsToCollocateRHSForMerge(ctx, leftDesc.Replicas(), rightDesc.Replicas()) + require.NoError(t, err) + require.Equal(t, len(subtest.expectedRightVoters), len(voterTargets)) + require.Equal(t, len(subtest.expectedRightNonVoters), len(nonVoterTargets)) + sort.Slice(voterTargets, func(i, j int) bool { + return voterTargets[i].NodeID < voterTargets[j].NodeID + }) + sort.Slice(nonVoterTargets, func(i, j int) bool { + return nonVoterTargets[i].NodeID < nonVoterTargets[j].NodeID + }) + for i := range subtest.expectedRightVoters { + require.Equal(t, tc.Target(subtest.expectedRightVoters[i]), voterTargets[i]) + } + for i := range subtest.expectedRightNonVoters { + require.Equal(t, tc.Target(subtest.expectedRightNonVoters[i]), nonVoterTargets[i]) + } + + store.SetMergeQueueActive(true) + store.MustForceMergeScanAndProcess() + verifyMerged(t, store, leftDesc.StartKey, rightDesc.StartKey) + }) + } +} + func TestInvalidSubsumeRequest(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvserver/client_migration_test.go b/pkg/kv/kvserver/client_migration_test.go index 2796e18c4b40..a491eb5aa619 100644 --- a/pkg/kv/kvserver/client_migration_test.go +++ b/pkg/kv/kvserver/client_migration_test.go @@ -151,8 +151,8 @@ func TestMigrateWithInflightSnapshot(t *testing.T) { // added. <-blockUntilSnapshotCh desc := tc.LookupRangeOrFatal(t, k) - require.Len(t, desc.Replicas().Voters(), 1) - require.Len(t, desc.Replicas().Learners(), 1) + require.Len(t, desc.Replicas().VoterDescriptors(), 1) + require.Len(t, desc.Replicas().LearnerDescriptors(), 1) // Enqueue the replica in the raftsnapshot queue. We use SucceedsSoon // because it may take a bit for raft to figure out that we need to be diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 75df6631cafa..d49a3d4a6927 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -3173,19 +3173,19 @@ func TestDecommission(t *testing.T) { requireNoReplicas := func(storeID roachpb.StoreID, repFactor int) { testutils.SucceedsSoon(t, func() error { desc := tc.LookupRangeOrFatal(t, k) - for _, rDesc := range desc.Replicas().Voters() { + for _, rDesc := range desc.Replicas().VoterDescriptors() { store, err := tc.Servers[int(rDesc.NodeID-1)].Stores().GetStore(rDesc.StoreID) require.NoError(t, err) if err := store.ForceReplicationScanAndProcess(); err != nil { return err } } - if sl := desc.Replicas().Filter(func(rDesc roachpb.ReplicaDescriptor) bool { + if sl := desc.Replicas().FilterToDescriptors(func(rDesc roachpb.ReplicaDescriptor) bool { return rDesc.StoreID == storeID }); len(sl) > 0 { return errors.Errorf("still a replica on s%d: %s", storeID, &desc) } - if len(desc.Replicas().Voters()) != repFactor { + if len(desc.Replicas().VoterDescriptors()) != repFactor { return errors.Errorf("expected %d replicas: %s", repFactor, &desc) } return nil diff --git a/pkg/kv/kvserver/client_relocate_range_test.go b/pkg/kv/kvserver/client_relocate_range_test.go index d7a2d5aacf4d..5fe7e4c8a99d 100644 --- a/pkg/kv/kvserver/client_relocate_range_test.go +++ b/pkg/kv/kvserver/client_relocate_range_test.go @@ -31,11 +31,14 @@ func relocateAndCheck( t *testing.T, tc *testcluster.TestCluster, startKey roachpb.RKey, - targets []roachpb.ReplicationTarget, + voterTargets []roachpb.ReplicationTarget, + nonVoterTargets []roachpb.ReplicationTarget, ) (retries int) { testutils.SucceedsSoon(t, func() error { err := tc.Servers[0].DB(). - AdminRelocateRange(context.Background(), startKey.AsRawKey(), targets) + AdminRelocateRange( + context.Background(), startKey.AsRawKey(), voterTargets, nonVoterTargets, + ) if err != nil { retries++ } @@ -43,8 +46,10 @@ func relocateAndCheck( }) desc, err := tc.Servers[0].LookupRange(startKey.AsRawKey()) require.NoError(t, err) - requireDescMembers(t, desc, targets) - requireLeaseAt(t, tc, desc, targets[0]) + requireDescMembers(t, desc, append(voterTargets, nonVoterTargets...)) + if len(voterTargets) > 0 { + requireLeaseAt(t, tc, desc, voterTargets[0]) + } return retries } @@ -56,7 +61,7 @@ func requireDescMembers( sort.Slice(targets, func(i, j int) bool { return targets[i].StoreID < targets[j].StoreID }) have := make([]roachpb.ReplicationTarget, 0, len(targets)) - for _, rDesc := range desc.Replicas().All() { + for _, rDesc := range desc.Replicas().Descriptors() { have = append(have, roachpb.ReplicationTarget{ NodeID: rDesc.NodeID, StoreID: rDesc.StoreID, @@ -116,7 +121,7 @@ func TestAdminRelocateRange(t *testing.T) { if len(ic.ops) == 2 && ic.ops[0].ChangeType == roachpb.ADD_VOTER && ic.ops[1].ChangeType == roachpb.REMOVE_VOTER { actAtomic++ } else { - actSingle++ + actSingle += len(ic.ops) } } actAtomic -= retries @@ -149,7 +154,7 @@ func TestAdminRelocateRange(t *testing.T) { targets := tc.Targets(1, 0, 2) // Expect two single additions, and that's it. requireNumAtomic(0, 2, func() int { - return relocateAndCheck(t, tc, k, targets) + return relocateAndCheck(t, tc, k, targets, nil /* nonVoterTargets */) }) } @@ -163,7 +168,7 @@ func TestAdminRelocateRange(t *testing.T) { // in the process (i.e. internally the lease must've been moved around // to achieve that). requireNumAtomic(3, 0, func() int { - return relocateAndCheck(t, tc, k, targets) + return relocateAndCheck(t, tc, k, targets, nil /* nonVoterTargets */) }) } @@ -171,7 +176,7 @@ func TestAdminRelocateRange(t *testing.T) { // Pure downreplication. { requireNumAtomic(0, 2, func() int { - return relocateAndCheck(t, tc, k, tc.Targets(4)) + return relocateAndCheck(t, tc, k, tc.Targets(4), nil /* nonVoterTargets */) }) } @@ -180,7 +185,7 @@ func TestAdminRelocateRange(t *testing.T) { // replication changes cannot be used; we add-then-remove instead. { requireNumAtomic(0, 2, func() int { - return relocateAndCheck(t, tc, k, tc.Targets(2)) + return relocateAndCheck(t, tc, k, tc.Targets(2), nil /* nonVoterTargets */) }) } @@ -189,15 +194,33 @@ func TestAdminRelocateRange(t *testing.T) { { // s3 -(add)-> s3 s2 -(swap)-> s4 s2 -(add)-> s4 s2 s1 (=s2 s4 s1) requireNumAtomic(1, 2, func() int { - return relocateAndCheck(t, tc, k, tc.Targets(1, 3, 0)) + return relocateAndCheck(t, tc, k, tc.Targets(1, 3, 0), nil /* nonVoterTargets */) }) // s2 s4 s1 -(add)-> s2 s4 s1 s6 (=s4 s2 s6 s1) requireNumAtomic(0, 1, func() int { - return relocateAndCheck(t, tc, k, tc.Targets(3, 1, 5, 0)) + return relocateAndCheck(t, tc, k, tc.Targets(3, 1, 5, 0), nil /* nonVoterTargets */) }) // s4 s2 s6 s1 -(swap)-> s3 s2 s6 s1 -(swap)-> s3 s5 s6 s1 -(del)-> s3 s5 s6 -(del)-> s3 s5 requireNumAtomic(2, 2, func() int { - return relocateAndCheck(t, tc, k, tc.Targets(2, 4)) + return relocateAndCheck(t, tc, k, tc.Targets(2, 4), nil /* nonVoterTargets */) + }) + } + + // Relocation of non-voting replicas is not done atomically under any + // scenario. + // TODO(aayush): Update this comment and test once we support atomic swaps of + // more than 1 non-voter at a time. + { + requireNumAtomic(0, 2, func() int { + return relocateAndCheck(t, tc, k, tc.Targets(2, 4), tc.Targets(1, 3)) + }) + // Add & remove. + requireNumAtomic(0, 2, func() int { + return relocateAndCheck(t, tc, k, tc.Targets(2, 4), tc.Targets(1, 5)) + }) + // 2 add and 2 remove operations. + requireNumAtomic(0, 4, func() int { + return relocateAndCheck(t, tc, k, tc.Targets(2, 4), tc.Targets(0, 3)) }) } } diff --git a/pkg/kv/kvserver/client_replica_backpressure_test.go b/pkg/kv/kvserver/client_replica_backpressure_test.go index a5f6fb0ce3d8..a8132583e162 100644 --- a/pkg/kv/kvserver/client_replica_backpressure_test.go +++ b/pkg/kv/kvserver/client_replica_backpressure_test.go @@ -152,7 +152,7 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { // replicas to move according to them. tc.ToggleReplicateQueues(false) defer tc.ToggleReplicateQueues(true) - voters := desc.Replicas().Voters() + voters := desc.Replicas().VoterDescriptors() if len(voters) == 1 && voters[0].NodeID == tc.Server(1).NodeID() { return nil } @@ -270,7 +270,7 @@ func TestBackpressureNotAppliedWhenReducingRangeSize(t *testing.T) { s, repl := getFirstStoreReplica(t, tc.Server(1), tablePrefix) s.SetReplicateQueueActive(false) - require.Len(t, repl.Desc().Replicas().All(), 1) + require.Len(t, repl.Desc().Replicas().Descriptors(), 1) // We really need to make sure that the split queue has hit this range, // otherwise we'll fail to backpressure. go func() { _ = s.ForceSplitScanAndProcess() }() diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index e814b28eb2df..6cbc0ded37bd 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -1950,7 +1950,7 @@ func TestSystemZoneConfigs(t *testing.T) { replicas := make(map[roachpb.RangeID]roachpb.RangeDescriptor) for _, s := range tc.Servers { if err := kvserver.IterateRangeDescriptors(ctx, s.Engines()[0], func(desc roachpb.RangeDescriptor) error { - if len(desc.Replicas().Learners()) > 0 { + if len(desc.Replicas().LearnerDescriptors()) > 0 { return fmt.Errorf("descriptor contains learners: %v", desc) } if existing, ok := replicas[desc.RangeID]; ok && !existing.Equal(&desc) { @@ -1964,7 +1964,7 @@ func TestSystemZoneConfigs(t *testing.T) { } var totalReplicas int for _, desc := range replicas { - totalReplicas += len(desc.Replicas().Voters()) + totalReplicas += len(desc.Replicas().VoterDescriptors()) } if totalReplicas != expectedReplicas { return fmt.Errorf("got %d voters, want %d; details: %+v", totalReplicas, expectedReplicas, replicas) @@ -2314,7 +2314,9 @@ func TestRandomConcurrentAdminChangeReplicasRequests(t *testing.T) { var wg sync.WaitGroup key := roachpb.Key("a") db := tc.Servers[0].DB() - require.Nil(t, db.AdminRelocateRange(ctx, key, makeReplicationTargets(1, 2, 3))) + require.Nil(t, db.AdminRelocateRange( + ctx, key, makeReplicationTargets(1, 2, 3), nil, + )) // Random targets consisting of a random number of nodes from the set of nodes // in the cluster which currently do not have a replica. pickTargets := func() []roachpb.ReplicationTarget { @@ -2334,9 +2336,12 @@ func TestRandomConcurrentAdminChangeReplicasRequests(t *testing.T) { rangeInfo, err := getRangeInfo(ctx, db, key) require.Nil(t, err) addReplicas := func() error { + op := roachpb.ADD_VOTER + if rand.Intn(2) == 0 { + op = roachpb.ADD_NON_VOTER + } _, err := db.AdminChangeReplicas( - ctx, key, rangeInfo.Desc, roachpb.MakeReplicationChanges( - roachpb.ADD_VOTER, pickTargets()...)) + ctx, key, rangeInfo.Desc, roachpb.MakeReplicationChanges(op, pickTargets()...)) return err } wg.Add(actors) @@ -2820,7 +2825,9 @@ func TestAdminRelocateRangeSafety(t *testing.T) { // to set up the replication and then verify the assumed state. key := roachpb.Key("a") - assert.Nil(t, db.AdminRelocateRange(ctx, key, makeReplicationTargets(1, 2, 3))) + assert.Nil(t, db.AdminRelocateRange( + ctx, key, makeReplicationTargets(1, 2, 3), makeReplicationTargets(), + )) rangeInfo, err := getRangeInfo(ctx, db, key) assert.Nil(t, err) assert.Len(t, rangeInfo.Desc.InternalReplicas, 3) @@ -2855,7 +2862,9 @@ func TestAdminRelocateRangeSafety(t *testing.T) { changedDesc, changeErr = r1.ChangeReplicas(ctx, &expDescAfterAdd, kvserver.SnapshotRequest_REBALANCE, "replicate", "testing", chgs) } relocate := func() { - relocateErr = db.AdminRelocateRange(ctx, key, makeReplicationTargets(1, 2, 4)) + relocateErr = db.AdminRelocateRange( + ctx, key, makeReplicationTargets(1, 2, 4), makeReplicationTargets(), + ) } useSeenAdd.Store(true) var wg sync.WaitGroup diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index d71f70b91673..ab2882022372 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -525,8 +525,7 @@ SELECT count(*), sum(value) FROM crdb_internal.node_metrics WHERE numSnapsBefore := numRaftSnaps("before") doSplit := func(ctx context.Context, _ int) error { - _, _, err := tc.SplitRange( - []byte(fmt.Sprintf("key-%d", perm[atomic.AddInt32(&idx, 1)]))) + _, _, err := tc.SplitRange([]byte(fmt.Sprintf("key-%d", perm[atomic.AddInt32(&idx, 1)]))) return err } diff --git a/pkg/kv/kvserver/client_test.go b/pkg/kv/kvserver/client_test.go index 06b14a6d79c4..1149c3e73c21 100644 --- a/pkg/kv/kvserver/client_test.go +++ b/pkg/kv/kvserver/client_test.go @@ -239,7 +239,7 @@ func createTestStoreWithOpts( var ba roachpb.BatchRequest get := roachpb.GetRequest{} get.Key = keys.LocalMax - ba.Header.Replica = repl.Desc().Replicas().Voters()[0] + ba.Header.Replica = repl.Desc().Replicas().VoterDescriptors()[0] ba.Header.RangeID = repl.RangeID ba.Add(&get) _, pErr := store.Send(ctx, ba) diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index ad2d306fb613..db0a4699d038 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -66,9 +66,9 @@ func TestClosedTimestampCanServe(t *testing.T) { skip.UnderRace(t) ctx := context.Background() - tc, db0, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db0, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) if _, err := db0.Exec(`INSERT INTO cttest.kv VALUES(1, $1)`, "foo"); err != nil { t.Fatal(err) @@ -127,9 +127,10 @@ func TestClosedTimestampCanServeThroughoutLeaseTransfer(t *testing.T) { skip.UnderRace(t) ctx := context.Background() - tc, db0, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db0, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, + testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) if _, err := db0.Exec(`INSERT INTO cttest.kv VALUES(1, $1)`, "foo"); err != nil { t.Fatal(err) @@ -200,9 +201,10 @@ func TestClosedTimestampCanServeWithConflictingIntent(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc, _, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, _, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, + testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) ds := tc.Server(0).DistSenderI().(*kvcoord.DistSender) // Write N different intents for the same transaction, where N is the number @@ -282,8 +284,9 @@ func TestClosedTimestampCanServeAfterSplitAndMerges(t *testing.T) { skip.UnderRace(t) ctx := context.Background() - tc, db0, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db0, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, + testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") + repls := replsForRange(ctx, t, tc, desc, numNodes) // Disable the automatic merging. if _, err := db0.Exec("SET CLUSTER SETTING kv.range_merge.queue_enabled = false"); err != nil { t.Fatal(err) @@ -362,9 +365,10 @@ func TestClosedTimestampCantServeBasedOnMaxTimestamp(t *testing.T) { ctx := context.Background() // Set up the target duration to be very long and rely on lease transfers to // drive MaxClosed. - tc, db0, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, time.Hour, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db0, desc := setupClusterForClosedTSTesting(ctx, t, time.Hour, testingCloseFraction, + aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) if _, err := db0.Exec(`INSERT INTO cttest.kv VALUES(1, $1)`, "foo"); err != nil { t.Fatal(err) @@ -400,9 +404,10 @@ func TestClosedTimestampCantServeForWritingTransaction(t *testing.T) { skip.UnderRace(t) ctx := context.Background() - tc, db0, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db0, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, + testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) if _, err := db0.Exec(`INSERT INTO cttest.kv VALUES(1, $1)`, "foo"); err != nil { t.Fatal(err) @@ -434,9 +439,10 @@ func TestClosedTimestampCantServeForNonTransactionalReadRequest(t *testing.T) { skip.UnderRace(t) ctx := context.Background() - tc, db0, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, - testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db0, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, + testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) if _, err := db0.Exec(`INSERT INTO cttest.kv VALUES(1, $1)`, "foo"); err != nil { t.Fatal(err) @@ -556,7 +562,14 @@ func TestClosedTimestampInactiveAfterSubsumption(t *testing.T) { }, }, } - tc, _, leftDesc, rightDesc := initClusterWithSplitRanges(ctx, t, testingTargetDuration, + // If the initial phase of the merge txn takes longer than the closed + // timestamp target duration, its initial CPuts can have their write + // timestamps bumped due to an intervening closed timestamp update. This + // causes the entire merge txn to retry. So we use a long closed timestamp + // duration at the beginning of the test until we have the merge txn + // suspended at its commit trigger, and then change it back down to + // `testingTargetDuration`. + tc, leftDesc, rightDesc := setupClusterForClosedTSTestingWithSplitRanges(ctx, t, 5*time.Second, testingCloseFraction, clusterArgs) defer tc.Stopper().Stop(ctx) @@ -591,6 +604,13 @@ func TestClosedTimestampInactiveAfterSubsumption(t *testing.T) { case <-time.After(45 * time.Second): t.Fatal("did not receive merge commit trigger as expected") } + // Reduce the closed timestamp target duration in order to make the rest of + // the test faster. + db := tc.ServerConn(0) + if _, err := db.Exec(fmt.Sprintf(`SET CLUSTER SETTING kv.closed_timestamp.target_duration = '%s';`, + testingTargetDuration)); err != nil { + t.Fatal(err) + } // inactiveClosedTSBoundary indicates the low water mark for closed // timestamp updates beyond which we expect none of the followers to be able // to serve follower reads until the merge is complete. @@ -856,50 +876,70 @@ func mergeTxn(ctx context.Context, store *kvserver.Store, leftDesc roachpb.Range return err.GoError() } -func initClusterWithSplitRanges( +func setupClusterForClosedTSTestingWithSplitRanges( ctx context.Context, t *testing.T, targetDuration time.Duration, closeFraction float64, clusterArgs base.TestClusterArgs, -) ( - serverutils.TestClusterInterface, - *gosql.DB, - roachpb.RangeDescriptor, - roachpb.RangeDescriptor, -) { - tc, db0, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, targetDuration, - closeFraction, clusterArgs) +) (serverutils.TestClusterInterface, roachpb.RangeDescriptor, roachpb.RangeDescriptor) { + dbName, tableName := "cttest", "kv" + tc, _, _ := setupClusterForClosedTSTesting(ctx, t, targetDuration, closeFraction, + clusterArgs, dbName, tableName) + leftDesc, rightDesc := splitDummyRangeInTestCluster(t, tc, dbName, tableName, + hlc.Timestamp{} /* splitExpirationTime */) + return tc, leftDesc, rightDesc +} - if _, err := db0.Exec(`INSERT INTO cttest.kv VALUES(1, $1)`, "foo"); err != nil { +// splitDummyRangeInTestCluster is supposed to be used in conjunction with the +// dummy table created in setupTestClusterWithDummyRange. It adds two rows to +// the given table and performs splits on the table's range such that the 2 +// resulting ranges contain exactly one of the rows each. +func splitDummyRangeInTestCluster( + t *testing.T, + tc serverutils.TestClusterInterface, + dbName, tableName string, + splitExpirationTime hlc.Timestamp, +) (roachpb.RangeDescriptor, roachpb.RangeDescriptor) { + db0 := tc.ServerConn(0) + if _, err := db0.Exec(fmt.Sprintf(`INSERT INTO %s.%s VALUES(1, '%s')`, + dbName, tableName, "foo")); err != nil { t.Fatal(err) } - if _, err := db0.Exec(`INSERT INTO cttest.kv VALUES(3, $1)`, "foo"); err != nil { + if _, err := db0.Exec(fmt.Sprintf(`INSERT INTO %s.%s VALUES(3, '%s')`, + dbName, tableName, "foo")); err != nil { t.Fatal(err) } - // Start by ensuring that the values can be read from all replicas at ts. - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - baRead := makeReadBatchRequestForDesc(desc, ts) - testutils.SucceedsSoon(t, func() error { - return verifyCanReadFromAllRepls(ctx, t, baRead, repls, expectRows(2)) - }) // Manually split the table to have easier access to descriptors. - tableID, err := getTableID(db0, "cttest", "kv") + tableID, err := getTableID(db0, dbName, tableName) if err != nil { t.Fatalf("failed to lookup ids: %+v", err) } idxPrefix := keys.SystemSQLCodec.IndexPrefix(uint32(tableID), 1) - k, err := rowenc.EncodeTableKey(idxPrefix, tree.NewDInt(2), encoding.Ascending) + k, err := rowenc.EncodeTableKey(idxPrefix, tree.NewDInt(1), encoding.Ascending) if err != nil { t.Fatalf("failed to encode split key: %+v", err) } tcImpl := tc.(*testcluster.TestCluster) - leftDesc, rightDesc := tcImpl.SplitRangeOrFatal(t, k) - if err := tcImpl.WaitForFullReplication(); err != nil { - t.Fatal(err) + // Split at `k` so that the table has exactly two ranges: [1,2) and [2, Max). + // This split will never be merged by the merge queue so the expiration time + // doesn't matter here. + tcImpl.SplitRangeOrFatal(t, k) + idxPrefix = keys.SystemSQLCodec.IndexPrefix(uint32(tableID), 1) + k, err = rowenc.EncodeTableKey(idxPrefix, tree.NewDInt(2), encoding.Ascending) + if err != nil { + t.Fatalf("failed to encode split key: %+v", err) + } + leftDesc, rightDesc, err := tcImpl.SplitRangeWithExpiration(k, splitExpirationTime) + require.NoError(t, err) + + if tc.ReplicationMode() != base.ReplicationManual { + if err := tcImpl.WaitForFullReplication(); err != nil { + t.Fatal(err) + } } - return tc, db0, leftDesc, rightDesc + return leftDesc, rightDesc } func getCurrentMaxClosed( @@ -1092,100 +1132,84 @@ func aggressiveResolvedTimestampPushKnobs() *kvserver.StoreTestingKnobs { } } -// setupClusterForClosedTimestampTesting creates a test cluster that is prepared -// to exercise follower reads. The returned test cluster has follower reads -// enabled using the given targetDuration and testingCloseFraction. In addition -// to the newly minted test cluster, this function returns a db handle to node -// 0, a range descriptor for the range used by the table `cttest.kv` and the -// replica objects corresponding to the replicas for the range. It is the -// caller's responsibility to Stop the Stopper on the returned test cluster when -// done. -func setupClusterForClosedTimestampTesting( +// setupClusterForClosedTSTesting creates a test cluster that is prepared to +// exercise follower reads. The returned test cluster has follower reads enabled +// using the given targetDuration and testingCloseFraction. In addition to the +// newly minted test cluster, this function returns a db handle to node 0, a +// range descriptor for the range used by the table `{dbName}.{tableName}`. It +// is the caller's responsibility to Stop the Stopper on the returned test +// cluster when done. +func setupClusterForClosedTSTesting( ctx context.Context, t *testing.T, targetDuration time.Duration, closeFraction float64, clusterArgs base.TestClusterArgs, -) ( - tc serverutils.TestClusterInterface, - db0 *gosql.DB, - kvTableDesc roachpb.RangeDescriptor, - repls []*kvserver.Replica, -) { - tc = serverutils.StartNewTestCluster(t, numNodes, clusterArgs) - db0 = tc.ServerConn(0) + dbName, tableName string, +) (tc serverutils.TestClusterInterface, db0 *gosql.DB, kvTableDesc roachpb.RangeDescriptor) { + tc, desc := setupTestClusterWithDummyRange(t, clusterArgs, dbName, tableName, numNodes) + require.NoError(t, enableFollowerReadsForTesting(tc.ServerConn(0), targetDuration, closeFraction)) + return tc, tc.ServerConn(0), desc +} + +func enableFollowerReadsForTesting( + db *gosql.DB, targetDuration time.Duration, closeFraction float64, +) error { + if _, err := db.Exec(fmt.Sprintf(` +SET CLUSTER SETTING kv.closed_timestamp.target_duration = '%s'; +SET CLUSTER SETTING kv.closed_timestamp.close_fraction = %.3f; +SET CLUSTER SETTING kv.closed_timestamp.follower_reads_enabled = true; +`, targetDuration, closeFraction)); err != nil { + return err + } + return nil +} + +// setupTestClusterWithDummyRange creates a TestCluster with an empty table and +// returns a handle to the range descriptor corresponding to this table. +func setupTestClusterWithDummyRange( + t *testing.T, clusterArgs base.TestClusterArgs, dbName, tableName string, numNodes int, +) (serverutils.TestClusterInterface, roachpb.RangeDescriptor) { + tc := serverutils.StartNewTestCluster(t, numNodes, clusterArgs) + db0 := tc.ServerConn(0) if _, err := db0.Exec(fmt.Sprintf(` -- Set a timeout to get nicer test failures from these statements. Because of -- the aggressiveResolvedTimestampPushKnobs() these statements can restart -- forever under high load (testrace under high concurrency). SET statement_timeout='30s'; -SET CLUSTER SETTING kv.closed_timestamp.target_duration = '%s'; -SET CLUSTER SETTING kv.closed_timestamp.close_fraction = %.3f; -SET CLUSTER SETTING kv.closed_timestamp.follower_reads_enabled = true; -CREATE DATABASE cttest; -CREATE TABLE cttest.kv (id INT PRIMARY KEY, value STRING); +CREATE DATABASE %[1]s; +CREATE TABLE %[1]s.%[2]s (id INT PRIMARY KEY, value STRING); -- Reset the timeout set above. RESET statement_timeout; -`, targetDuration, closeFraction)); err != nil { +`, dbName, tableName)); err != nil { t.Fatal(err) } var rangeID roachpb.RangeID var startKey roachpb.Key var numReplicas int - testutils.SucceedsSoon(t, func() error { - if err := db0.QueryRow( - `SELECT range_id, start_key, array_length(replicas, 1) FROM crdb_internal.ranges WHERE table_name = 'kv' AND database_name = 'cttest'`, - ).Scan(&rangeID, &startKey, &numReplicas); err != nil { - return err - } - if numReplicas != 3 { - return errors.New("not fully replicated yet") - } - return nil - }) - - desc, err := tc.LookupRange(startKey) - require.Nil(t, err) - - // First, we perform an arbitrary lease transfer because that will turn the - // lease into an epoch based one (the initial lease is likely expiration based - // since the range just split off from the very first range which is expiration - // based). - var lh roachpb.ReplicationTarget - testutils.SucceedsSoon(t, func() error { - var err error - lh, err = tc.FindRangeLeaseHolder(desc, nil) - return err - }) - - for i := 0; i < numNodes; i++ { - target := tc.Target(i) - if target != lh { - if err := tc.TransferRangeLease(desc, target); err != nil { - t.Fatal(err) + // If replicate queue is not disabled, wait until the table's range is fully + // replicated. + if clusterArgs.ReplicationMode != base.ReplicationManual { + testutils.SucceedsSoon(t, func() error { + if err := db0.QueryRow( + fmt.Sprintf( + `SELECT range_id, start_key, array_length(replicas, 1) FROM crdb_internal.ranges WHERE table_name = '%s' AND database_name = '%s'`, + tableName, dbName), + ).Scan(&rangeID, &startKey, &numReplicas); err != nil { + return err } - break - } - } - repls = replsForRange(ctx, t, tc, desc, numNodes) - require.Equal(t, numReplicas, len(repls)) - // Wait until we see an epoch based lease on our chosen range. This should - // happen fairly quickly since we just transferred a lease (as a means to make - // it epoch based). If the lease transfer fails, we'll be sitting out the lease - // expiration, which is on the order of seconds. Not great, but good enough since - // the transfer basically always works. - for ok := false; !ok; time.Sleep(10 * time.Millisecond) { - for _, repl := range repls { - lease, _ := repl.GetLease() - if lease.Epoch != 0 { - ok = true - break + if numReplicas != numNodes { + return errors.New("not fully replicated yet") } - } + return nil + }) } - return tc, db0, desc, repls + + desc, err := tc.LookupRange(startKey) + require.Nil(t, err) + return tc, desc } type respFunc func(*roachpb.BatchResponse, *roachpb.Error) (shouldRetry bool, err error) diff --git a/pkg/kv/kvserver/consistency_queue.go b/pkg/kv/kvserver/consistency_queue.go index 18ac209aa90d..2b40d619136f 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -137,7 +137,7 @@ func consistencyQueueShouldQueueImpl( } } // Check if all replicas are live. - for _, rep := range data.desc.Replicas().All() { + for _, rep := range data.desc.Replicas().Descriptors() { if live, err := data.isNodeLive(rep.NodeID); err != nil { log.VErrEventf(ctx, 3, "node %d liveness failed: %s", rep.NodeID, err) return false, 0 diff --git a/pkg/kv/kvserver/constraint/analyzer.go b/pkg/kv/kvserver/constraint/analyzer.go index 41b249b82346..e5e191ec5e0f 100644 --- a/pkg/kv/kvserver/constraint/analyzer.go +++ b/pkg/kv/kvserver/constraint/analyzer.go @@ -82,7 +82,7 @@ func AnalyzeConstraints( // matches the conjunction if it matches all of them. func ConjunctionsCheck(store roachpb.StoreDescriptor, constraints []zonepb.Constraint) bool { for _, constraint := range constraints { - // StoreSatisfiesConstraint returns whether a store matches the given constraint. + // StoreMatchesConstraint returns whether a store matches the given constraint. hasConstraint := zonepb.StoreMatchesConstraint(store, constraint) if (constraint.Type == zonepb.Constraint_REQUIRED && !hasConstraint) || (constraint.Type == zonepb.Constraint_PROHIBITED && hasConstraint) { diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 0df29629dfd6..feb5885b6d04 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -287,41 +287,61 @@ func (mq *mergeQueue) process( return false, err } } - lhsReplicas, rhsReplicas := lhsDesc.Replicas().All(), rhsDesc.Replicas().All() - - // Defensive sanity check that everything is now a voter. - for i := range lhsReplicas { - if lhsReplicas[i].GetType() != roachpb.VOTER_FULL { - return false, errors.Errorf(`cannot merge non-voter replicas on lhs: %v`, lhsReplicas) + leftRepls, rightRepls := lhsDesc.Replicas().Descriptors(), rhsDesc.Replicas().Descriptors() + + // Defensive sanity check that the ranges involved only have either VOTER_FULL + // and NON_VOTER replicas. + for i := range leftRepls { + if typ := leftRepls[i].GetType(); !(typ == roachpb.VOTER_FULL || typ == roachpb.NON_VOTER) { + return false, + errors.AssertionFailedf( + `cannot merge because lhs is either in a joint state or has learner replicas: %v`, + leftRepls, + ) } } - for i := range rhsReplicas { - if rhsReplicas[i].GetType() != roachpb.VOTER_FULL { - return false, errors.Errorf(`cannot merge non-voter replicas on rhs: %v`, rhsReplicas) + for i := range rightRepls { + if typ := rightRepls[i].GetType(); !(typ == roachpb.VOTER_FULL || typ == roachpb.NON_VOTER) { + return false, + errors.AssertionFailedf( + `cannot merge because rhs is either in a joint state or has learner replicas: %v`, + rightRepls, + ) } } - if !replicaSetsEqual(lhsReplicas, rhsReplicas) { - var targets []roachpb.ReplicationTarget - for _, lhsReplDesc := range lhsReplicas { - targets = append(targets, roachpb.ReplicationTarget{ - NodeID: lhsReplDesc.NodeID, StoreID: lhsReplDesc.StoreID, - }) + // Range merges require that the set of stores that contain a replica for the + // RHS range be equal to the set of stores that contain a replica for the LHS + // range. The LHS and RHS ranges' leaseholders do not need to be co-located + // and types of the replicas (voting or non-voting) do not matter. + if !replicasCollocated(leftRepls, rightRepls) { + // TODO(aayush): We enable merges to proceed even when LHS and/or RHS are in + // violation of their constraints (by adding or removing replicas on the RHS + // as needed). We could instead choose to check constraints conformance of + // these ranges and only try to collocate them if they're not in violation, + // which would help us make better guarantees about not transiently + // violating constraints during a merge. + voterTargets, nonVoterTargets, err := GetTargetsToCollocateRHSForMerge(ctx, lhsDesc.Replicas(), rhsDesc.Replicas()) + if err != nil { + return false, err } + // AdminRelocateRange moves the lease to the first target in the list, so // sort the existing leaseholder there to leave it unchanged. lease, _ := lhsRepl.GetLease() - for i := range targets { - if targets[i].NodeID == lease.Replica.NodeID && targets[i].StoreID == lease.Replica.StoreID { + for i := range voterTargets { + if t := voterTargets[i]; t.NodeID == lease.Replica.NodeID && t.StoreID == lease.Replica.StoreID { if i > 0 { - targets[0], targets[i] = targets[i], targets[0] + voterTargets[0], voterTargets[i] = voterTargets[i], voterTargets[0] } break } } - // TODO(benesch): RelocateRange can sometimes fail if it needs to move a replica - // from one store to another store on the same node. - if err := mq.store.DB().AdminRelocateRange(ctx, rhsDesc.StartKey, targets); err != nil { + // The merge queue will only merge ranges that have the same zone config + // (see check inside mergeQueue.shouldQueue). + if err := mq.store.DB().AdminRelocateRange( + ctx, rhsDesc.StartKey, voterTargets, nonVoterTargets, + ); err != nil { return false, err } } diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index a7a562d93724..e342107eabd1 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -202,7 +202,7 @@ func newTruncateDecision(ctx context.Context, r *Replica) (truncateDecision, err log.Eventf(ctx, "raft status before lastUpdateTimes check: %+v", raftStatus.Progress) log.Eventf(ctx, "lastUpdateTimes: %+v", r.mu.lastUpdateTimes) updateRaftProgressFromActivity( - ctx, raftStatus.Progress, r.descRLocked().Replicas().All(), + ctx, raftStatus.Progress, r.descRLocked().Replicas().Descriptors(), func(replicaID roachpb.ReplicaID) bool { return r.mu.lastUpdateTimes.isFollowerActiveSince( ctx, replicaID, now, r.store.cfg.RangeLeaseActiveDuration()) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index ce1f23fddd67..f030793295f5 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -865,7 +865,7 @@ func maxReplicaIDOfAny(desc *roachpb.RangeDescriptor) roachpb.ReplicaID { return 0 } var maxID roachpb.ReplicaID - for _, repl := range desc.Replicas().All() { + for _, repl := range desc.Replicas().Descriptors() { if repl.ReplicaID > maxID { maxID = repl.ReplicaID } @@ -1066,7 +1066,7 @@ func (r *Replica) State() kvserverpb.RangeInfo { if desc := ri.ReplicaState.Desc; desc != nil { // Learner replicas don't serve follower reads, but they still receive // closed timestamp updates, so include them here. - allReplicas := desc.Replicas().All() + allReplicas := desc.Replicas().Descriptors() for i := range allReplicas { replDesc := &allReplicas[i] r.store.cfg.ClosedTimestamp.Storage.VisitDescending(replDesc.NodeID, func(e ctpb.Entry) (done bool) { diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 12cc8e03b326..e36a1ec5ba55 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -20,6 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -593,7 +594,7 @@ func (r *Replica) AdminMerge( // replica of the RHS too early. The comment on // TestStoreRangeMergeUninitializedLHSFollower explains the situation in full. if err := waitForReplicasInit( - ctx, r.store.cfg.NodeDialer, origLeftDesc.RangeID, origLeftDesc.Replicas().All(), + ctx, r.store.cfg.NodeDialer, origLeftDesc.RangeID, origLeftDesc.Replicas().Descriptors(), ); err != nil { return errors.Wrap(err, "waiting for all left-hand replicas to initialize") } @@ -627,19 +628,18 @@ func (r *Replica) AdminMerge( // queues should fix things up quickly). lReplicas, rReplicas := origLeftDesc.Replicas(), rightDesc.Replicas() - predFullVoter := func(rDesc roachpb.ReplicaDescriptor) bool { - return rDesc.GetType() == roachpb.VOTER_FULL + if len(lReplicas.VoterAndNonVoterDescriptors()) != len(lReplicas.Descriptors()) { + return errors.Errorf("cannot merge ranges when lhs is in a joint state or has learners: %s", + lReplicas) } - if len(lReplicas.Filter(predFullVoter)) != len(lReplicas.All()) { - return errors.Errorf("cannot merge range with non-voter replicas on lhs: %s", lReplicas) + if len(rReplicas.VoterAndNonVoterDescriptors()) != len(rReplicas.Descriptors()) { + return errors.Errorf("cannot merge ranges when rhs is in a joint state or has learners: %s", + rReplicas) } - if len(rReplicas.Filter(predFullVoter)) != len(rReplicas.All()) { - return errors.Errorf("cannot merge range with non-voter replicas on rhs: %s", rReplicas) - } - if !replicaSetsEqual(lReplicas.All(), rReplicas.All()) { + if !replicasCollocated(lReplicas.Descriptors(), rReplicas.Descriptors()) { return errors.Errorf("ranges not collocated; %s != %s", lReplicas, rReplicas) } - mergeReplicas := lReplicas.All() + mergeReplicas := lReplicas.Descriptors() updatedLeftDesc := *origLeftDesc // lhs.Generation = max(rhs.Generation, lhs.Generation)+1. @@ -992,7 +992,7 @@ func (r *Replica) changeReplicasImpl( } // Queue the replica up into the raft snapshot queue so that the non-voters // that were added receive their first snapshot relatively soon. See the - // comment block above ReplicaDescriptors.NonVoters() for why we do this. + // comment block above ReplicaSet.NonVoters() for why we do this. r.store.raftSnapshotQueue.AddAsync(ctx, r, raftSnapshotPriority) } @@ -1032,7 +1032,7 @@ func (r *Replica) changeReplicasImpl( // For all newly added nodes, first add raft learner replicas. They accept raft traffic // (so they can catch up) but don't get to vote (so they don't affect quorum and thus // don't introduce fragility into the system). For details see: - _ = roachpb.ReplicaDescriptors.Learners + _ = roachpb.ReplicaSet.LearnerDescriptors var err error desc, err = addRaftLearners(ctx, r.store, desc, reason, details, adds, internalChangeTypeAddLearner) if err != nil { @@ -1110,7 +1110,7 @@ func maybeLeaveAtomicChangeReplicasAndRemoveLearners( // Now the config isn't joint any more, but we may have demoted some voters // into learners. These learners should go as well. - learners := desc.Replicas().Learners() + learners := desc.Replicas().LearnerDescriptors() if len(learners) == 0 { return desc, nil } @@ -1178,7 +1178,7 @@ func validateReplicationChanges( // Then, check that we're not adding a second replica on nodes that already // have one, or "re-add" an existing replica. We delete from byNodeAndStoreID so that // after this loop, it contains only Nodes that we haven't seen in desc. - for _, rDesc := range desc.Replicas().All() { + for _, rDesc := range desc.Replicas().Descriptors() { byStoreID, ok := byNodeAndStoreID[rDesc.NodeID] if !ok { continue @@ -1231,7 +1231,7 @@ func validateReplicationChanges( // We're adding a replica that's already there. This isn't allowed, even // when the newly added one would be on a different store. if chg.ChangeType.IsAddition() { - if len(desc.Replicas().All()) > 1 { + if len(desc.Replicas().Descriptors()) > 1 { return errors.Mark( errors.Errorf("unable to add replica %v; node already has a replica in %s", chg.Target.StoreID, desc), errMarkInvalidReplicationChange) @@ -1626,7 +1626,7 @@ func prepareChangeReplicasTrigger( var isJoint bool // NB: the DeepCopy is needed or we'll skip over an entry every time we // call RemoveReplica below. - for _, rDesc := range updatedDesc.Replicas().DeepCopy().All() { + for _, rDesc := range updatedDesc.Replicas().DeepCopy().Descriptors() { switch rDesc.GetType() { case roachpb.VOTER_INCOMING: updatedDesc.SetReplicaType(rDesc.NodeID, rDesc.StoreID, roachpb.VOTER_FULL) @@ -1763,7 +1763,7 @@ func execChangeReplicasTxn( // See: // https://github.com/cockroachdb/cockroach/issues/54444#issuecomment-707706553 replicas := crt.Desc.Replicas() - liveReplicas, _ := args.liveAndDeadReplicas(replicas.All()) + liveReplicas, _ := args.liveAndDeadReplicas(replicas.Descriptors()) if !replicas.CanMakeProgress( func(rDesc roachpb.ReplicaDescriptor) bool { for _, inner := range liveReplicas { @@ -2133,9 +2133,9 @@ func (r *Replica) sendSnapshot( return nil } -// replicaSetsEqual is used in AdminMerge to ensure that the ranges are +// replicasCollocated is used in AdminMerge to ensure that the ranges are // all collocate on the same set of replicas. -func replicaSetsEqual(a, b []roachpb.ReplicaDescriptor) bool { +func replicasCollocated(a, b []roachpb.ReplicaDescriptor) bool { if len(a) != len(b) { return false } @@ -2158,6 +2158,100 @@ func replicaSetsEqual(a, b []roachpb.ReplicaDescriptor) bool { return true } +// GetTargetsToCollocateRHSForMerge decides the configuration of RHS replicas +// need before the rhs can be subsumed and then merged into the LHS range. The +// desired RHS voters and non-voters are returned; together they'll cover the +// same stores as LHS's replicas, but the configuration of replicas doesn't +// necessarily match (it doesn't need to match for the merge). +// +// We compute the new voter / non-voter targets for the RHS by first +// bootstrapping our result set with the replicas that are already collocated. +// We then step through RHS's non-collocated voters and try to move them to +// stores that already have a voter for LHS. If this is not possible for all the +// non-collocated voters of RHS (i.e. because the RHS has non-voter(s) on +// store(s) where the LHS has voter(s)), we may move some RHS voters to targets +// that have non-voters for LHS. Likewise, we do the same for the non-collocated +// non-voters of RHS: try to relocate them to stores where the LHS has +// non-voters, but resort to relocating them to stores where the LHS has voters. +// +// TODO(aayush): Can moving a voter replica from RHS to a store that has a +// non-voter for LHS (or vice versa) can lead to constraint violations? Justify +// why or why not. +func GetTargetsToCollocateRHSForMerge( + ctx context.Context, leftRepls, rightRepls roachpb.ReplicaSet, +) (voterTargets, nonVoterTargets []roachpb.ReplicationTarget, _ error) { + notInRight := func(desc roachpb.ReplicaDescriptor) bool { + return !rightRepls.Contains(desc) + } + + // Sets of replicas that exist on the LHS but not on the RHS + leftMinusRight := leftRepls.Filter(notInRight) + leftMinusRightVoters := leftMinusRight.Voters().Descriptors() + leftMinusRightNonVoters := leftMinusRight.NonVoters().Descriptors() + + // We bootstrap our result set by first including the replicas (voting and + // non-voting) that _are_ collocated, as these will stay unchanged and will + // be no-ops when passed through AdminRelocateRange. + finalRightVoters := rightRepls.Voters().Filter(leftRepls.Contains).DeepCopy() + finalRightNonVoters := rightRepls.NonVoters().Filter(leftRepls.Contains).DeepCopy() + + needMore := func() bool { + return len(finalRightVoters.Descriptors())+len(finalRightNonVoters.Descriptors()) < len(leftRepls.Descriptors()) + } + + numVoters := len(leftRepls.VoterDescriptors()) + // We loop through the set of non-collocated replicas and figure out a + // suitable configuration to relocate RHS's replicas to. At the end of these + // two loops, we will have exhausted `leftMinusRight`. + for len(finalRightVoters.Descriptors()) < numVoters && needMore() { + // Prefer to relocate voters for RHS to stores that have voters for LHS, but + // resort to relocating them to stores with non-voters for LHS if that's not + // possible. + if len(leftMinusRightVoters) != 0 { + finalRightVoters.AddReplica(leftMinusRightVoters[0]) + leftMinusRightVoters = leftMinusRightVoters[1:] + } else if len(leftMinusRightNonVoters) != 0 { + finalRightVoters.AddReplica(leftMinusRightNonVoters[0]) + leftMinusRightNonVoters = leftMinusRightNonVoters[1:] + } else { + log.Fatalf(ctx, "programming error: unexpectedly ran out of valid stores to relocate RHS"+ + " voters to; LHS: %s, RHS: %s", leftRepls.Descriptors(), rightRepls.Descriptors()) + } + } + + for needMore() { + // Like above, we try to relocate non-voters for RHS to stores that have + // non-voters for LHS, but resort to relocating them to stores with voters + // for LHS if that's not possible. + if len(leftMinusRightNonVoters) != 0 { + finalRightNonVoters.AddReplica(leftMinusRightNonVoters[0]) + leftMinusRightNonVoters = leftMinusRightNonVoters[1:] + } else if len(leftMinusRightVoters) != 0 { + finalRightNonVoters.AddReplica(leftMinusRightVoters[0]) + leftMinusRightVoters = leftMinusRightVoters[1:] + } else { + log.Fatalf(ctx, "programming error: unexpectedly ran out of valid stores to relocate RHS"+ + " non-voters to; LHS: %s, RHS: %s", leftRepls.Descriptors(), rightRepls.Descriptors()) + } + } + + if len(finalRightVoters.Descriptors()) == 0 { + // TODO(aayush): We can end up in this case for scenarios like the + // following (the digits represent StoreIDs): + // + // LHS-> voters: {1, 2, 3}, non-voters: {} + // RHS-> voters: {4}, non-voters: {1, 2, 3} + // + // Remove this error path once we support swapping voters and non-voters. + return nil, nil, + errors.UnimplementedErrorf(errors.IssueLink{IssueURL: build.MakeIssueURL(58499)}, + "unsupported configuration of RHS(%s) and LHS(%s) as it requires an atomic swap of a"+ + " voter and non-voter", rightRepls, leftRepls) + } + + return finalRightVoters.ReplicationTargets(), finalRightNonVoters.ReplicationTargets(), nil +} + func checkDescsEqual(desc *roachpb.RangeDescriptor) func(*roachpb.RangeDescriptor) bool { return func(desc2 *roachpb.RangeDescriptor) bool { return desc.Equal(desc2) @@ -2244,10 +2338,12 @@ func updateRangeDescriptor( // This is best-effort; it's possible that the replicate queue on the // leaseholder could take action at the same time, causing errors. func (s *Store) AdminRelocateRange( - ctx context.Context, rangeDesc roachpb.RangeDescriptor, targets []roachpb.ReplicationTarget, + ctx context.Context, + rangeDesc roachpb.RangeDescriptor, + voterTargets, nonVoterTargets []roachpb.ReplicationTarget, ) error { - // Step 0: Remove everything that's not a full voter so we don't have to think - // about them. + // Remove learners so we don't have to think about relocating them, and leave + // the joint config if we're in one. newDesc, err := maybeLeaveAtomicChangeReplicasAndRemoveLearners(ctx, s, &rangeDesc) if err != nil { log.Warningf(ctx, "%v", err) @@ -2255,6 +2351,40 @@ func (s *Store) AdminRelocateRange( } rangeDesc = *newDesc + rangeDesc, err = s.relocateReplicas(ctx, rangeDesc, voterTargets, nonVoterTargets) + if err != nil { + return err + } + return nil +} + +// relocateReplicas repeatedly adds and/or removes a replica until we reach +// the desired state. In an "atomic replication changes" world, this is +// conceptually easy: change from the old set of replicas to the new one. But +// there are two reasons that complicate this: +// 1. we can't remove the leaseholder, so if we ultimately want to do that +// the lease has to be moved first. If we start out with *only* the +// leaseholder, we will have to add a replica first. +// 2. this code is rewritten late in the cycle and it is both safer and +// closer to its previous incarnation to never issue atomic changes +// other than simple swaps. +// +// The loop below repeatedly calls relocateOne, which gives us either +// one or two ops that move the range towards the desired replication state. If +// it's one op, then a single add or remove is carried out (and it's only done +// when we can't swap instead). If it's two ops, then we're swapping (though +// this code doesn't concern itself with the details); and it's possible that we +// need to transfer the lease before we carry out the ops, determined via the +// leaseTarget variable. +// +// Transient errors returned from relocateOne are retried until things +// work out. +func (s *Store) relocateReplicas( + ctx context.Context, + rangeDesc roachpb.RangeDescriptor, + voterTargets, nonVoterTargets []roachpb.ReplicationTarget, +) (roachpb.RangeDescriptor, error) { + startKey := rangeDesc.StartKey.AsRawKey() canRetry := func(err error) bool { allowlist := []string{ snapshotApplySemBusyMsg, @@ -2268,8 +2398,6 @@ func (s *Store) AdminRelocateRange( } return false } - - startKey := rangeDesc.StartKey.AsRawKey() transferLease := func(target roachpb.ReplicationTarget) { // TODO(tbg): we ignore errors here, but it seems that in practice these // transfers "always work". Some of them are essential (we can't remove @@ -2283,37 +2411,16 @@ func (s *Store) AdminRelocateRange( } } - // Step 2: Repeatedly add and/or remove a replica until we reach the - // desired state. In an "atomic replication changes" world, this is - // conceptually easy: change from the old set of replicas to the new - // one. But there are two reasons that complicate this: - // 1. we can't remove the leaseholder, so if we ultimately want to do that - // the lease has to be moved first. If we start out with *only* the - // leaseholder, we will have to add a replica first. - // 2. this code is rewritten late in the cycle and it is both safer and - // closer to its previous incarnation to never issue atomic changes - // other than simple swaps. - // - // The loop below repeatedly calls relocateOne, which gives us either one or - // two ops that move the range towards the desired replication state. If - // it's one op, then a single add or remove is carried out (and it's only - // done when we can't swap instead). If it's two ops, then we're swapping - // (though this code doesn't concern itself with the details); and it's - // possible that we need to transfer the lease before we carry out the ops, - // determined via the leaseTarget variable. - // - // Transient errors returned from relocateOne are retried until things work - // out. every := log.Every(time.Minute) for { for re := retry.StartWithCtx(ctx, retry.Options{MaxBackoff: 5 * time.Second}); re.Next(); { if err := ctx.Err(); err != nil { - return err + return rangeDesc, err } - ops, leaseTarget, err := s.relocateOne(ctx, &rangeDesc, targets) + ops, leaseTarget, err := s.relocateOne(ctx, &rangeDesc, voterTargets, nonVoterTargets) if err != nil { - return err + return rangeDesc, err } if leaseTarget != nil { // NB: we may need to transfer even if there are no ops, to make @@ -2323,7 +2430,7 @@ func (s *Store) AdminRelocateRange( } if len(ops) == 0 { // Done. - return ctx.Err() + return rangeDesc, ctx.Err() } if fn := s.cfg.TestingKnobs.BeforeRelocateOne; fn != nil { fn(ops, leaseTarget, err) @@ -2341,7 +2448,7 @@ func (s *Store) AdminRelocateRange( if err != nil { returnErr := errors.Wrapf(err, "while carrying out changes %v", ops) if !canRetry(err) { - return returnErr + return rangeDesc, returnErr } if every.ShouldLog() { log.Infof(ctx, "%v", returnErr) @@ -2356,18 +2463,18 @@ func (s *Store) AdminRelocateRange( } } } - } func (s *Store) relocateOne( - ctx context.Context, desc *roachpb.RangeDescriptor, targets []roachpb.ReplicationTarget, + ctx context.Context, + desc *roachpb.RangeDescriptor, + voterTargets, nonVoterTargets []roachpb.ReplicationTarget, ) ([]roachpb.ReplicationChange, *roachpb.ReplicationTarget, error) { - rangeReplicas := desc.Replicas().All() - if len(rangeReplicas) != len(desc.Replicas().Voters()) { - // The caller removed all the learners, so there shouldn't be anything but - // voters. + if repls := desc.Replicas(); len(repls.VoterAndNonVoterDescriptors()) != len(repls.Descriptors()) { + // The caller removed all the learners and left the joint config, so there + // shouldn't be anything but voters and non_voters. return nil, nil, errors.AssertionFailedf( - `range %s had non-voter replicas: %v`, desc, desc.Replicas()) + `range %s was either in a joint configuration or had learner replicas: %v`, desc, desc.Replicas()) } sysCfg := s.cfg.Gossip.GetSystemConfig() @@ -2382,61 +2489,46 @@ func (s *Store) relocateOne( storeList, _, _ := s.allocator.storePool.getStoreList(storeFilterNone) storeMap := storeListToMap(storeList) - // Compute which replica to add and/or remove, respectively. We ask the allocator - // about this because we want to respect the constraints. For example, it would be - // unfortunate if we put two replicas into the same zone despite having a locality- - // preserving option available. + getTargetsToRelocate := func() (targetsToAdd, targetsToRemove []roachpb.ReplicaDescriptor, + addOp, removeOp roachpb.ReplicaChangeType, votersRelocated bool) { + votersToAdd := subtractTargets(voterTargets, desc.Replicas().Voters().ReplicationTargets()) + votersToRemove := subtractTargets(desc.Replicas().Voters().ReplicationTargets(), voterTargets) + // If there are no voters to relocate, we relocate the non-voters. + if len(votersToAdd) == 0 && len(votersToRemove) == 0 { + nonVotersToAdd := subtractTargets(nonVoterTargets, desc.Replicas().NonVoters().ReplicationTargets()) + nonVotersToRemove := subtractTargets(desc.Replicas().NonVoters().ReplicationTargets(), nonVoterTargets) + return nonVotersToAdd, nonVotersToRemove, roachpb.ADD_NON_VOTER, roachpb.REMOVE_NON_VOTER, true + } + return votersToAdd, votersToRemove, roachpb.ADD_VOTER, roachpb.REMOVE_VOTER, false + } + + // Compute which replica to add and/or remove, respectively. We then ask the + // allocator about this because we want to respect the constraints. For + // example, it would be unfortunate if we put two replicas into the same zone + // despite having a locality- preserving option available. // // TODO(radu): we can't have multiple replicas on different stores on the // same node, and this code doesn't do anything to specifically avoid that // case (although the allocator will avoid even trying to send snapshots to // such stores), so it could cause some failures. - - var addTargets []roachpb.ReplicaDescriptor - for _, t := range targets { - found := false - for _, replicaDesc := range rangeReplicas { - if replicaDesc.StoreID == t.StoreID && replicaDesc.NodeID == t.NodeID { - found = true - break - } - } - if !found { - addTargets = append(addTargets, roachpb.ReplicaDescriptor{ - NodeID: t.NodeID, - StoreID: t.StoreID, - }) - } - } - - var removeTargets []roachpb.ReplicaDescriptor - for _, replicaDesc := range rangeReplicas { - found := false - for _, t := range targets { - if replicaDesc.StoreID == t.StoreID && replicaDesc.NodeID == t.NodeID { - found = true - break - } - } - if !found { - removeTargets = append(removeTargets, roachpb.ReplicaDescriptor{ - NodeID: replicaDesc.NodeID, - StoreID: replicaDesc.StoreID, - }) - } + targetsToAdd, targetsToRemove, addOp, removeOp, votersRelocated := getTargetsToRelocate() + relocationTargets := voterTargets + existingReplicas := desc.Replicas().VoterDescriptors() + if votersRelocated { + relocationTargets = nonVoterTargets + existingReplicas = desc.Replicas().NonVoterDescriptors() } var ops roachpb.ReplicationChanges - - if len(addTargets) > 0 { + if len(targetsToAdd) > 0 { // Each iteration, pick the most desirable replica to add. However, // prefer the first target because it's the one that should hold the // lease in the end; it helps to add it early so that the lease doesn't // have to move too much. - candidateTargets := addTargets - if storeHasReplica(targets[0].StoreID, candidateTargets) { + candidateTargets := targetsToAdd + if !votersRelocated && storeHasReplica(relocationTargets[0].StoreID, candidateTargets) { candidateTargets = []roachpb.ReplicaDescriptor{ - {NodeID: targets[0].NodeID, StoreID: targets[0].StoreID}, + {NodeID: relocationTargets[0].NodeID, StoreID: relocationTargets[0].StoreID}, } } @@ -2452,27 +2544,27 @@ func (s *Store) relocateOne( } candidateDescs = append(candidateDescs, *store) } - storeList = makeStoreList(candidateDescs) + candidateStoreList := makeStoreList(candidateDescs) targetStore, _ := s.allocator.allocateTargetFromList( ctx, - storeList, + candidateStoreList, zone, - rangeReplicas, + existingReplicas, s.allocator.scorerOptions()) if targetStore == nil { - return nil, nil, fmt.Errorf("none of the remaining targets %v are legal additions to %v", - addTargets, desc.Replicas()) + return nil, nil, fmt.Errorf("none of the remaining relocationTargets %v are legal additions to %v", + targetsToAdd, desc.Replicas()) } target := roachpb.ReplicationTarget{ NodeID: targetStore.Node.NodeID, StoreID: targetStore.StoreID, } - ops = append(ops, roachpb.MakeReplicationChanges(roachpb.ADD_VOTER, target)...) - // Pretend the voter is already there so that the removal logic below will + ops = append(ops, roachpb.MakeReplicationChanges(addOp, target)...) + // Pretend the replica is already there so that the removal logic below will // take it into account when deciding which replica to remove. - rangeReplicas = append(rangeReplicas, roachpb.ReplicaDescriptor{ + existingReplicas = append(existingReplicas, roachpb.ReplicaDescriptor{ NodeID: target.NodeID, StoreID: target.StoreID, ReplicaID: desc.NextReplicaID, @@ -2481,19 +2573,19 @@ func (s *Store) relocateOne( } var transferTarget *roachpb.ReplicationTarget - if len(removeTargets) > 0 { - // Pick a replica to remove. Note that rangeReplicas may already reflect + if len(targetsToRemove) > 0 { + // Pick a replica to remove. Note that existingReplicas may already reflect // a replica we're adding in the current round. This is the right thing // to do. For example, consider relocating from (s1,s2,s3) to (s1,s2,s4) - // where addTargets will be (s4) and removeTargets is (s3). In this code, + // where targetsToAdd will be (s4) and targetsToRemove is (s3). In this code, // we'll want the allocator to see if s3 can be removed from // (s1,s2,s3,s4) which is a reasonable request; that replica set is // overreplicated. If we asked it instead to remove s3 from (s1,s2,s3) // it may not want to do that due to constraints. - targetStore, _, err := s.allocator.RemoveTarget(ctx, zone, removeTargets, rangeReplicas) + targetStore, _, err := s.allocator.RemoveTarget(ctx, zone, targetsToRemove, existingReplicas) if err != nil { return nil, nil, errors.Wrapf(err, "unable to select removal target from %v; current replicas %v", - removeTargets, rangeReplicas) + targetsToRemove, existingReplicas) } removalTarget := roachpb.ReplicationTarget{ NodeID: targetStore.NodeID, @@ -2519,11 +2611,11 @@ func (s *Store) relocateOne( // lease needs to be in the end. We also exclude the last replica if // it was added by the add branch above (in which case it doesn't // exist yet). - sortedTargetReplicas := append([]roachpb.ReplicaDescriptor(nil), rangeReplicas[:len(rangeReplicas)-len(ops)]...) + sortedTargetReplicas := append([]roachpb.ReplicaDescriptor(nil), existingReplicas[:len(existingReplicas)-len(ops)]...) sort.Slice(sortedTargetReplicas, func(i, j int) bool { sl := sortedTargetReplicas - // targets[0] goes to the front (if it's present). - return sl[i].StoreID == targets[0].StoreID + // relocationTargets[0] goes to the front (if it's present). + return sl[i].StoreID == relocationTargets[0].StoreID }) for _, rDesc := range sortedTargetReplicas { if rDesc.StoreID != curLeaseholder.StoreID { @@ -2544,7 +2636,7 @@ func (s *Store) relocateOne( // illegal). if ok { ops = append(ops, roachpb.MakeReplicationChanges( - roachpb.REMOVE_VOTER, + removeOp, removalTarget)...) } } @@ -2552,12 +2644,33 @@ func (s *Store) relocateOne( if len(ops) == 0 { // Make sure that the first target is the final leaseholder, as // AdminRelocateRange specifies. - transferTarget = &targets[0] + transferTarget = &voterTargets[0] } return ops, transferTarget, nil } +// subtractTargets returns the set of replication targets in `left` but not in +// `right` (i.e. left - right). +func subtractTargets(left, right []roachpb.ReplicationTarget) (diff []roachpb.ReplicaDescriptor) { + for _, t := range left { + found := false + for _, replicaDesc := range right { + if replicaDesc.StoreID == t.StoreID && replicaDesc.NodeID == t.NodeID { + found = true + break + } + } + if !found { + diff = append(diff, roachpb.ReplicaDescriptor{ + NodeID: t.NodeID, + StoreID: t.StoreID, + }) + } + } + return diff +} + // adminScatter moves replicas and leaseholders for a selection of ranges. func (r *Replica) adminScatter( ctx context.Context, args roachpb.AdminScatterRequest, @@ -2600,8 +2713,8 @@ func (r *Replica) adminScatter( if args.RandomizeLeases && r.OwnsValidLease(ctx, r.store.Clock().Now()) { desc := r.Desc() // Learner replicas aren't allowed to become the leaseholder or raft leader, - // so only consider the `Voters` replicas. - voterReplicas := desc.Replicas().Voters() + // so only consider the `VoterDescriptors` replicas. + voterReplicas := desc.Replicas().VoterDescriptors() newLeaseholderIdx := rand.Intn(len(voterReplicas)) targetStoreID := voterReplicas[newLeaseholderIdx].StoreID if targetStoreID != r.store.StoreID() { diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index f0dd22c55f17..6bb5c39d2d56 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -283,12 +283,12 @@ func (r *Replica) CheckConsistency( } // args.Terminate is a slice of properly redactable values, but // with %v `redact` will not realize that and will redact the - // whole thing. Wrap it as a ReplicaDescriptors which is a SafeFormatter + // whole thing. Wrap it as a ReplicaSet which is a SafeFormatter // and will get the job done. // // TODO(knz): clean up after https://github.com/cockroachdb/redact/issues/5. { - var tmp redact.SafeFormatter = roachpb.MakeReplicaDescriptors(args.Terminate) + var tmp redact.SafeFormatter = roachpb.MakeReplicaSet(args.Terminate) log.Errorf(ctx, "consistency check failed; fetching details and shutting down minority %v", tmp) } @@ -361,7 +361,7 @@ func (r *Replica) RunConsistencyCheck( // Move the local replica to the front (which makes it the "master" // we're comparing against). - orderedReplicas = append(orderedReplicas, desc.Replicas().All()...) + orderedReplicas = append(orderedReplicas, desc.Replicas().Descriptors()...) sort.Slice(orderedReplicas, func(i, j int) bool { return orderedReplicas[i] == localReplica diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index ad5cfbcefcc8..a32f807b0d30 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -172,8 +172,8 @@ func TestAddReplicaViaLearner(t *testing.T) { // added. <-blockUntilSnapshotCh desc := tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().Voters(), 1) - require.Len(t, desc.Replicas().Learners(), 1) + require.Len(t, desc.Replicas().VoterDescriptors(), 1) + require.Len(t, desc.Replicas().LearnerDescriptors(), 1) var voters, nonVoters string db.QueryRow(t, @@ -188,8 +188,8 @@ func TestAddReplicaViaLearner(t *testing.T) { require.NoError(t, g.Wait()) desc = tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().Voters(), 2) - require.Len(t, desc.Replicas().Learners(), 0) + require.Len(t, desc.Replicas().VoterDescriptors(), 2) + require.Len(t, desc.Replicas().LearnerDescriptors(), 0) require.Equal(t, int64(1), getFirstStoreMetric(t, tc.Server(1), `range.snapshots.applied-initial`)) } @@ -230,13 +230,13 @@ func TestAddRemoveNonVotingReplicasBasic(t *testing.T) { require.NoError(t, g.Wait()) desc := tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().NonVoters(), 1) + require.Len(t, desc.Replicas().NonVoterDescriptors(), 1) _, err := tc.RemoveNonVoters(scratchStartKey, tc.Target(1)) require.NoError(t, err) desc = tc.LookupRangeOrFatal(t, scratchStartKey) require.NoError(t, tc.WaitForFullReplication()) - require.Len(t, desc.Replicas().NonVoters(), 0) + require.Len(t, desc.Replicas().NonVoterDescriptors(), 0) } func TestLearnerRaftConfState(t *testing.T) { @@ -299,8 +299,8 @@ func TestLearnerRaftConfState(t *testing.T) { ltk.withStopAfterLearnerAtomic(func() { desc = tc.AddVotersOrFatal(t, scratchStartKey, tc.Target(1)) }) - require.Len(t, desc.Replicas().Learners(), 1) - learnerReplicaID := desc.Replicas().Learners()[0].ReplicaID + require.Len(t, desc.Replicas().LearnerDescriptors(), 1) + learnerReplicaID := desc.Replicas().LearnerDescriptors()[0].ReplicaID // Verify that raft on every node thinks it's a learner. This checks that we // use ConfChangeAddLearnerNode in the ConfChange and also checks that we @@ -355,7 +355,7 @@ func TestLearnerSnapshotFailsRollback(t *testing.T) { // Make sure we cleaned up after ourselves (by removing the learner). desc := tc.LookupRangeOrFatal(t, scratchStartKey) - require.Empty(t, desc.Replicas().Learners()) + require.Empty(t, desc.Replicas().LearnerDescriptors()) } func TestSplitWithLearnerOrJointConfig(t *testing.T) { @@ -381,8 +381,8 @@ func TestSplitWithLearnerOrJointConfig(t *testing.T) { // replication queue will eventually clean this up. left, right, err := tc.SplitRange(scratchStartKey.Next()) require.NoError(t, err) - require.Len(t, left.Replicas().Learners(), 1) - require.Len(t, right.Replicas().Learners(), 1) + require.Len(t, left.Replicas().LearnerDescriptors(), 1) + require.Len(t, right.Replicas().LearnerDescriptors(), 1) // Remove the learner on the RHS. right = tc.RemoveVotersOrFatal(t, right.StartKey.AsRawKey(), tc.Target(1)) @@ -402,7 +402,7 @@ func TestSplitWithLearnerOrJointConfig(t *testing.T) { } return err }) - require.Len(t, right.Replicas().Filter(predIncoming), 1) + require.Len(t, right.Replicas().FilterToDescriptors(predIncoming), 1) left, right, err = tc.SplitRange(right.StartKey.AsRawKey().Next()) require.NoError(t, err) require.False(t, left.Replicas().InAtomicReplicationChange(), left) @@ -440,11 +440,11 @@ func TestReplicateQueueSeesLearnerOrJointConfig(t *testing.T) { // Make sure it deleted the learner. desc := tc.LookupRangeOrFatal(t, scratchStartKey) - require.Empty(t, desc.Replicas().Learners()) + require.Empty(t, desc.Replicas().LearnerDescriptors()) // Bonus points: the replicate queue keeps processing until there is nothing // to do, so it should have upreplicated the range to 3. - require.Len(t, desc.Replicas().Voters(), 3) + require.Len(t, desc.Replicas().VoterDescriptors(), 3) } // Create a VOTER_OUTGOING, i.e. a joint configuration. @@ -465,7 +465,7 @@ func TestReplicateQueueSeesLearnerOrJointConfig(t *testing.T) { desc = tc.LookupRangeOrFatal(t, scratchStartKey) require.False(t, desc.Replicas().InAtomicReplicationChange(), desc) // Queue processed again, so we're back to three replicas. - require.Len(t, desc.Replicas().Voters(), 3) + require.Len(t, desc.Replicas().VoterDescriptors(), 3) }) } @@ -499,14 +499,14 @@ func TestReplicaGCQueueSeesLearnerOrJointConfig(t *testing.T) { } desc := checkNoGC() // Make sure it didn't collect the learner. - require.NotEmpty(t, desc.Replicas().Learners()) + require.NotEmpty(t, desc.Replicas().LearnerDescriptors()) // Now get the range into a joint config. tc.RemoveVotersOrFatal(t, scratchStartKey, tc.Target(1)) // remove learner ltk.withStopAfterJointConfig(func() { desc = tc.AddVotersOrFatal(t, scratchStartKey, tc.Target(1)) - require.Len(t, desc.Replicas().Filter(predIncoming), 1, desc) + require.Len(t, desc.Replicas().FilterToDescriptors(predIncoming), 1, desc) }) postDesc := checkNoGC() @@ -612,8 +612,8 @@ func TestLearnerAdminChangeReplicasRace(t *testing.T) { _, err := tc.RemoveVoters(scratchStartKey, tc.Target(1)) require.NoError(t, err) desc := tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().Voters(), 1) - require.Len(t, desc.Replicas().Learners(), 0) + require.Len(t, desc.Replicas().VoterDescriptors(), 1) + require.Len(t, desc.Replicas().LearnerDescriptors(), 0) // Unblock the snapshot, and surprise AddVoters. It should retry and error // that the descriptor has changed since the AdminChangeReplicas command @@ -627,8 +627,8 @@ func TestLearnerAdminChangeReplicasRace(t *testing.T) { t.Fatalf(`expected %q error got: %+v`, msgRE, err) } desc = tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().Voters(), 1) - require.Len(t, desc.Replicas().Learners(), 0) + require.Len(t, desc.Replicas().VoterDescriptors(), 1) + require.Len(t, desc.Replicas().LearnerDescriptors(), 0) } // This test verifies the result of a race between the replicate queue running @@ -703,16 +703,16 @@ func TestLearnerReplicateQueueRace(t *testing.T) { // leaving the 2 voters. desc, err := tc.RemoveVoters(scratchStartKey, tc.Target(2)) require.NoError(t, err) - require.Len(t, desc.Replicas().Voters(), 2) - require.Len(t, desc.Replicas().Learners(), 0) + require.Len(t, desc.Replicas().VoterDescriptors(), 2) + require.Len(t, desc.Replicas().LearnerDescriptors(), 0) // Unblock the snapshot, and surprise the replicate queue. It should retry, // get a descriptor changed error, and realize it should stop. close(blockSnapshotsCh) require.NoError(t, <-queue1ErrCh) desc = tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().Voters(), 2) - require.Len(t, desc.Replicas().Learners(), 0) + require.Len(t, desc.Replicas().VoterDescriptors(), 2) + require.Len(t, desc.Replicas().LearnerDescriptors(), 0) } func TestLearnerNoAcceptLease(t *testing.T) { @@ -840,7 +840,7 @@ func TestLearnerAndJointConfigFollowerRead(t *testing.T) { // Re-add the voter and remain in joint config. require.True(t, scratchDesc.Replicas().InAtomicReplicationChange(), scratchDesc) - require.Len(t, scratchDesc.Replicas().Filter(predIncoming), 1) + require.Len(t, scratchDesc.Replicas().FilterToDescriptors(predIncoming), 1) // Can't serve follower read from the VOTER_INCOMING. check() @@ -848,7 +848,7 @@ func TestLearnerAndJointConfigFollowerRead(t *testing.T) { // Remove the voter and remain in joint config. scratchDesc = tc.RemoveVotersOrFatal(t, scratchStartKey, tc.Target(1)) require.True(t, scratchDesc.Replicas().InAtomicReplicationChange(), scratchDesc) - require.Len(t, scratchDesc.Replicas().Filter(predDemoting), 1) + require.Len(t, scratchDesc.Replicas().FilterToDescriptors(predDemoting), 1) // Can't serve follower read from the VOTER_OUTGOING. check() @@ -875,19 +875,21 @@ func TestLearnerOrJointConfigAdminRelocateRange(t *testing.T) { _ = tc.AddVotersOrFatal(t, scratchStartKey, tc.Target(2)) }) - check := func(targets []roachpb.ReplicationTarget) { - require.NoError(t, tc.Server(0).DB().AdminRelocateRange(ctx, scratchStartKey, targets)) + check := func(voterTargets []roachpb.ReplicationTarget) { + require.NoError(t, tc.Server(0).DB().AdminRelocateRange( + ctx, scratchStartKey, voterTargets, []roachpb.ReplicationTarget{}, + )) desc := tc.LookupRangeOrFatal(t, scratchStartKey) - voters := desc.Replicas().Voters() - require.Len(t, voters, len(targets)) + voters := desc.Replicas().VoterDescriptors() + require.Len(t, voters, len(voterTargets)) sort.Slice(voters, func(i, j int) bool { return voters[i].NodeID < voters[j].NodeID }) for i := range voters { - require.Equal(t, targets[i].NodeID, voters[i].NodeID, `%v`, voters) - require.Equal(t, targets[i].StoreID, voters[i].StoreID, `%v`, voters) + require.Equal(t, voterTargets[i].NodeID, voters[i].NodeID, `%v`, voters) + require.Equal(t, voterTargets[i].StoreID, voters[i].StoreID, `%v`, voters) } - require.Empty(t, desc.Replicas().Learners()) - require.Empty(t, desc.Replicas().Filter(predIncoming)) - require.Empty(t, desc.Replicas().Filter(predOutgoing)) + require.Empty(t, desc.Replicas().LearnerDescriptors()) + require.Empty(t, desc.Replicas().FilterToDescriptors(predIncoming)) + require.Empty(t, desc.Replicas().FilterToDescriptors(predOutgoing)) } // Test AdminRelocateRange's treatment of learners by having one that it has @@ -905,7 +907,7 @@ func TestLearnerOrJointConfigAdminRelocateRange(t *testing.T) { atomic.StoreInt64(<k.replicationAlwaysUseJointConfig, 1) desc := tc.RemoveVotersOrFatal(t, scratchStartKey, tc.Target(3)) require.True(t, desc.Replicas().InAtomicReplicationChange(), desc) - require.Len(t, desc.Replicas().Filter(predDemoting), 1) + require.Len(t, desc.Replicas().FilterToDescriptors(predDemoting), 1) atomic.StoreInt64(<k.replicaAddStopAfterJointConfig, 0) check([]roachpb.ReplicationTarget{tc.Target(0), tc.Target(1), tc.Target(2)}) } @@ -943,11 +945,11 @@ func TestLearnerAndJointConfigAdminMerge(t *testing.T) { checkFails := func() { err := tc.Server(0).DB().AdminMerge(ctx, scratchStartKey) - if exp := `cannot merge range with non-voter replicas on`; !testutils.IsError(err, exp) { + if exp := `cannot merge ranges.*joint state`; !testutils.IsError(err, exp) { t.Fatalf(`expected "%s" error got: %+v`, exp, err) } err = tc.Server(0).DB().AdminMerge(ctx, splitKey1) - if exp := `cannot merge range with non-voter replicas on`; !testutils.IsError(err, exp) { + if exp := `cannot merge ranges.*joint state`; !testutils.IsError(err, exp) { t.Fatalf(`expected "%s" error got: %+v`, exp, err) } } @@ -961,10 +963,10 @@ func TestLearnerAndJointConfigAdminMerge(t *testing.T) { atomic.StoreInt64(<k.replicationAlwaysUseJointConfig, 1) desc1 = tc.RemoveVotersOrFatal(t, desc1.StartKey.AsRawKey(), tc.Target(1)) desc1 = tc.AddVotersOrFatal(t, desc1.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc1.Replicas().Filter(predIncoming), 1) + require.Len(t, desc1.Replicas().FilterToDescriptors(predIncoming), 1) desc3 = tc.RemoveVotersOrFatal(t, desc3.StartKey.AsRawKey(), tc.Target(1)) desc3 = tc.AddVotersOrFatal(t, desc3.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc1.Replicas().Filter(predIncoming), 1) + require.Len(t, desc1.Replicas().FilterToDescriptors(predIncoming), 1) // VOTER_INCOMING on the lhs or rhs should fail. // desc{1,2,3} = (VOTER_FULL, VOTER_INCOMING) (VOTER_FULL) (VOTER_FULL, VOTER_INCOMING) @@ -973,9 +975,9 @@ func TestLearnerAndJointConfigAdminMerge(t *testing.T) { // Turn the incoming voters on desc1 and desc3 into VOTER_DEMOTINGs. // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL) (VOTER_FULL, VOTER_DEMOTING) desc1 = tc.RemoveVotersOrFatal(t, desc1.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc1.Replicas().Filter(predDemoting), 1) + require.Len(t, desc1.Replicas().FilterToDescriptors(predDemoting), 1) desc3 = tc.RemoveVotersOrFatal(t, desc3.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc3.Replicas().Filter(predDemoting), 1) + require.Len(t, desc3.Replicas().FilterToDescriptors(predDemoting), 1) // VOTER_DEMOTING on the lhs or rhs should fail. checkFails() @@ -985,14 +987,14 @@ func TestLearnerAndJointConfigAdminMerge(t *testing.T) { // replica sets are equal). // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL, VOTER_INCOMING) (VOTER_FULL, VOTER_DEMOTING) desc2 := tc.AddVotersOrFatal(t, splitKey1, tc.Target(1)) - require.Len(t, desc2.Replicas().Filter(predIncoming), 1) + require.Len(t, desc2.Replicas().FilterToDescriptors(predIncoming), 1) checkFails() // Ditto VOTER_DEMOTING. // desc{1,2,3} = (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL, VOTER_DEMOTING) (VOTER_FULL, VOTER_DEMOTING) desc2 = tc.RemoveVotersOrFatal(t, desc2.StartKey.AsRawKey(), tc.Target(1)) - require.Len(t, desc2.Replicas().Filter(predDemoting), 1) + require.Len(t, desc2.Replicas().FilterToDescriptors(predDemoting), 1) checkFails() } @@ -1049,8 +1051,8 @@ func TestMergeQueueSeesLearnerOrJointConfig(t *testing.T) { require.Equal(t, origDesc.StartKey, desc.StartKey) require.Equal(t, origDesc.EndKey, desc.EndKey) // The merge removed the learner. - require.Len(t, desc.Replicas().Voters(), 1) - require.Empty(t, desc.Replicas().Learners()) + require.Len(t, desc.Replicas().VoterDescriptors(), 1) + require.Empty(t, desc.Replicas().LearnerDescriptors()) } // Create the RHS again and repeat the same game, except this time the LHS @@ -1062,7 +1064,7 @@ func TestMergeQueueSeesLearnerOrJointConfig(t *testing.T) { ltk.withStopAfterJointConfig(func() { desc = tc.AddVotersOrFatal(t, scratchStartKey, tc.Target(1)) }) - require.Len(t, desc.Replicas().Filter(predIncoming), 1, desc) + require.Len(t, desc.Replicas().FilterToDescriptors(predIncoming), 1, desc) checkTransitioningOut := func() { t.Helper() @@ -1082,7 +1084,7 @@ func TestMergeQueueSeesLearnerOrJointConfig(t *testing.T) { checkTransitioningOut() desc = tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().Voters(), 2) + require.Len(t, desc.Replicas().VoterDescriptors(), 2) require.False(t, desc.Replicas().InAtomicReplicationChange(), desc) // Repeat the game, except now we start with two replicas and we're @@ -1090,14 +1092,14 @@ func TestMergeQueueSeesLearnerOrJointConfig(t *testing.T) { desc = splitAndUnsplit() ltk.withStopAfterJointConfig(func() { descRight := tc.RemoveVotersOrFatal(t, desc.EndKey.AsRawKey(), tc.Target(1)) - require.Len(t, descRight.Replicas().Filter(predDemoting), 1, desc) + require.Len(t, descRight.Replicas().FilterToDescriptors(predDemoting), 1, desc) }) // This should transition out (i.e. remove the voter on s2 for the RHS) // and then do its thing, which means in the end we have two voters again. checkTransitioningOut() desc = tc.LookupRangeOrFatal(t, scratchStartKey) - require.Len(t, desc.Replicas().Voters(), 2) + require.Len(t, desc.Replicas().VoterDescriptors(), 2) require.False(t, desc.Replicas().InAtomicReplicationChange(), desc) } } diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index 7b49d85b58f8..9e87dc46ea2b 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -164,7 +164,7 @@ func calcRangeCounter( // It seems unlikely that a learner replica would be the first live one, but // there's no particular reason to exclude them. Note that `All` returns the // voters first. - for _, rd := range desc.Replicas().All() { + for _, rd := range desc.Replicas().Descriptors() { if livenessMap[rd.NodeID].IsLive { rangeCounter = rd.StoreID == storeID break @@ -193,7 +193,7 @@ func calcRangeCounter( // considered. func calcLiveVoterReplicas(desc *roachpb.RangeDescriptor, livenessMap liveness.IsLiveMap) int { var live int - for _, rd := range desc.Replicas().Voters() { + for _, rd := range desc.Replicas().VoterDescriptors() { if livenessMap[rd.NodeID].IsLive { live++ } @@ -207,7 +207,7 @@ func calcBehindCount( raftStatus *raft.Status, desc *roachpb.RangeDescriptor, livenessMap liveness.IsLiveMap, ) int64 { var behindCount int64 - for _, rd := range desc.Replicas().All() { + for _, rd := range desc.Replicas().Descriptors() { if progress, ok := raftStatus.Progress[uint64(rd.ReplicaID)]; ok { if progress.Match > 0 && progress.Match < raftStatus.Commit { diff --git a/pkg/kv/kvserver/replica_metrics_test.go b/pkg/kv/kvserver/replica_metrics_test.go index efaac4936022..42b36a531400 100644 --- a/pkg/kv/kvserver/replica_metrics_test.go +++ b/pkg/kv/kvserver/replica_metrics_test.go @@ -28,7 +28,7 @@ func TestCalcRangeCounterIsLiveMap(t *testing.T) { // https://github.com/cockroachdb/cockroach/pull/39936#pullrequestreview-359059629 desc := roachpb.NewRangeDescriptor(123, roachpb.RKeyMin, roachpb.RKeyMax, - roachpb.MakeReplicaDescriptors([]roachpb.ReplicaDescriptor{ + roachpb.MakeReplicaSet([]roachpb.ReplicaDescriptor{ {NodeID: 10, StoreID: 11, ReplicaID: 12, Type: roachpb.ReplicaTypeVoterFull()}, {NodeID: 100, StoreID: 110, ReplicaID: 120, Type: roachpb.ReplicaTypeVoterFull()}, {NodeID: 1000, StoreID: 1100, ReplicaID: 1200, Type: roachpb.ReplicaTypeVoterFull()}, diff --git a/pkg/kv/kvserver/replica_proposal_quota.go b/pkg/kv/kvserver/replica_proposal_quota.go index fbf63877f9c6..6ff64c06e107 100644 --- a/pkg/kv/kvserver/replica_proposal_quota.go +++ b/pkg/kv/kvserver/replica_proposal_quota.go @@ -103,7 +103,7 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // hands. r.mu.proposalQuota = quotapool.NewIntPool(r.rangeStr.String(), uint64(r.store.cfg.RaftProposalQuota)) r.mu.lastUpdateTimes = make(map[roachpb.ReplicaID]time.Time) - r.mu.lastUpdateTimes.updateOnBecomeLeader(r.mu.state.Desc.Replicas().All(), timeutil.Now()) + r.mu.lastUpdateTimes.updateOnBecomeLeader(r.mu.state.Desc.Replicas().Descriptors(), timeutil.Now()) } else if r.mu.proposalQuota != nil { // We're becoming a follower. // We unblock all ongoing and subsequent quota acquisition goroutines diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index d7255a74ca42..c649fe3e3d79 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -1805,7 +1805,7 @@ func maybeCampaignAfterConfChange( // If the leader is no longer in the descriptor but we are the first voter, // campaign. _, leaderStillThere := desc.GetReplicaDescriptorByID(roachpb.ReplicaID(st.Lead)) - if !leaderStillThere && storeID == desc.Replicas().Voters()[0].StoreID { + if !leaderStillThere && storeID == desc.Replicas().VoterDescriptors()[0].StoreID { log.VEventf(ctx, 3, "leader got removed by conf change; campaigning") _ = raftGroup.Campaign() } diff --git a/pkg/kv/kvserver/replica_raft_quiesce.go b/pkg/kv/kvserver/replica_raft_quiesce.go index 40a4899994c9..0a90f4d8c65b 100644 --- a/pkg/kv/kvserver/replica_raft_quiesce.go +++ b/pkg/kv/kvserver/replica_raft_quiesce.go @@ -69,7 +69,7 @@ func (r *Replica) unquiesceWithOptionsLocked(campaignOnWake bool) { } // NB: we know there's a non-nil RaftStatus because internalRaftGroup isn't nil. r.mu.lastUpdateTimes.updateOnUnquiesce( - r.mu.state.Desc.Replicas().All(), r.raftStatusRLocked().Progress, timeutil.Now(), + r.mu.state.Desc.Replicas().Descriptors(), r.raftStatusRLocked().Progress, timeutil.Now(), ) } } @@ -323,7 +323,7 @@ func shouldReplicaQuiesce( var foundSelf bool var lagging laggingReplicaSet - for _, rep := range q.descRLocked().Replicas().All() { + for _, rep := range q.descRLocked().Replicas().Descriptors() { if uint64(rep.ReplicaID) == status.ID { foundSelf = true } diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index c0b6330ecac7..0940d83491fa 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -772,8 +772,10 @@ func TestReplicaRangefeedPushesTransactions(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc, db, _, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, + testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) @@ -884,8 +886,10 @@ func TestReplicaRangefeedNudgeSlowClosedTimestamp(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - tc, db, desc, repls := setupClusterForClosedTimestampTesting(ctx, t, testingTargetDuration, testingCloseFraction, aggressiveResolvedTimestampClusterArgs) + tc, db, desc := setupClusterForClosedTSTesting(ctx, t, testingTargetDuration, + testingCloseFraction, aggressiveResolvedTimestampClusterArgs, "cttest", "kv") defer tc.Stopper().Stop(ctx) + repls := replsForRange(ctx, t, tc, desc, numNodes) sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 5cfbbc7d4a72..3d45f57a0f5b 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -592,7 +592,7 @@ func (r *Replica) executeAdminBatch( } case *roachpb.AdminRelocateRangeRequest: - err := r.store.AdminRelocateRange(ctx, *r.Desc(), tArgs.Targets) + err := r.store.AdminRelocateRange(ctx, *r.Desc(), tArgs.VoterTargets, tArgs.NonVoterTargets) pErr = roachpb.NewError(err) resp = &roachpb.AdminRelocateRangeResponse{} diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index eda36ced8ffc..880cddb718e8 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -6408,7 +6408,7 @@ func TestReplicaSetsEqual(t *testing.T) { {true, createReplicaSets([]roachpb.StoreID{1, 2, 3, 1, 2, 3}), createReplicaSets([]roachpb.StoreID{1, 1, 2, 2, 3, 3})}, } for _, test := range testData { - if replicaSetsEqual(test.a, test.b) != test.expected { + if replicasCollocated(test.a, test.b) != test.expected { t.Fatalf("unexpected replica intersection: %+v", test) } } @@ -9596,7 +9596,7 @@ func TestShouldReplicaQuiesce(t *testing.T) { if ok { // Any non-live replicas should be in the laggingReplicaSet. var expLagging laggingReplicaSet - for _, rep := range q.descRLocked().Replicas().All() { + for _, rep := range q.descRLocked().Replicas().Descriptors() { if l, ok := q.livenessMap[rep.NodeID]; ok && !l.IsLive { expLagging = append(expLagging, l.Liveness) } @@ -12833,7 +12833,7 @@ func TestPrepareChangeReplicasTrigger(t *testing.T) { }) } } - desc := roachpb.NewRangeDescriptor(roachpb.RangeID(10), roachpb.RKeyMin, roachpb.RKeyMax, roachpb.MakeReplicaDescriptors(rDescs)) + desc := roachpb.NewRangeDescriptor(roachpb.RangeID(10), roachpb.RKeyMin, roachpb.RKeyMax, roachpb.MakeReplicaSet(rDescs)) return testCase{ desc: desc, chgs: chgs, @@ -12934,7 +12934,7 @@ func TestRangeUnavailableMessage(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - var repls roachpb.ReplicaDescriptors + var repls roachpb.ReplicaSet 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) diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 049060f8685a..cf9b9f10a30b 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -265,7 +265,7 @@ func (r *Replica) executeWriteBatch( desc := r.Desc() // NB: waitForApplication already has a timeout. applicationErr := waitForApplication( - ctx, r.store.cfg.NodeDialer, desc.RangeID, desc.Replicas().All(), + ctx, r.store.cfg.NodeDialer, desc.RangeID, desc.Replicas().Descriptors(), uint64(maxLeaseIndex)) propResult.Err = roachpb.NewError(applicationErr) } @@ -305,7 +305,7 @@ func rangeUnavailableMessage( dur time.Duration, ) { var liveReplicas, otherReplicas []roachpb.ReplicaDescriptor - for _, rDesc := range desc.Replicas().All() { + for _, rDesc := range desc.Replicas().Descriptors() { if lm[rDesc.NodeID].IsLive { liveReplicas = append(liveReplicas, rDesc) } else { @@ -316,7 +316,7 @@ func rangeUnavailableMessage( // Ensure that these are going to redact nicely. var _ redact.SafeFormatter = ba var _ redact.SafeFormatter = desc - var _ redact.SafeFormatter = roachpb.ReplicaDescriptors{} + var _ redact.SafeFormatter = roachpb.ReplicaSet{} s.Printf(`have been waiting %.2fs for proposing command %s. This range is likely unavailable. @@ -337,8 +337,8 @@ support contract. Otherwise, please open an issue at: dur.Seconds(), ba, desc, - roachpb.MakeReplicaDescriptors(liveReplicas), - roachpb.MakeReplicaDescriptors(otherReplicas), + roachpb.MakeReplicaSet(liveReplicas), + roachpb.MakeReplicaSet(otherReplicas), redact.Safe(rs), // raft status contains no PII desc.RangeID, ) diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index c58de9aed872..2e9052a31e9c 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -222,7 +222,7 @@ func (rq *replicateQueue) shouldQueue( if action == AllocatorRemoveLearner { return true, priority } - voterReplicas := desc.Replicas().Voters() + voterReplicas := desc.Replicas().VoterDescriptors() if action == AllocatorNoop { log.VEventf(ctx, 2, "no action to take") @@ -321,7 +321,7 @@ func (rq *replicateQueue) processOneChange( // Avoid taking action if the range has too many dead replicas to make // quorum. - voterReplicas := desc.Replicas().Voters() + voterReplicas := desc.Replicas().VoterDescriptors() liveVoterReplicas, deadVoterReplicas := rq.allocator.storePool.liveAndDeadReplicas(voterReplicas) // NB: the replication layer ensures that the below operations don't cause @@ -710,7 +710,7 @@ func (rq *replicateQueue) removeDecommissioning( ctx context.Context, repl *Replica, dryRun bool, ) (requeue bool, _ error) { desc, _ := repl.DescAndZone() - decommissioningReplicas := rq.allocator.storePool.decommissioningReplicas(desc.Replicas().All()) + decommissioningReplicas := rq.allocator.storePool.decommissioningReplicas(desc.Replicas().Descriptors()) if len(decommissioningReplicas) == 0 { log.VEventf(ctx, 1, "range of replica %s was identified as having decommissioning replicas, "+ "but no decommissioning replicas were found", repl) @@ -784,7 +784,7 @@ func (rq *replicateQueue) removeLearner( ctx context.Context, repl *Replica, dryRun bool, ) (requeue bool, _ error) { desc := repl.Desc() - learnerReplicas := desc.Replicas().Learners() + learnerReplicas := desc.Replicas().LearnerDescriptors() if len(learnerReplicas) == 0 { log.VEventf(ctx, 1, "range of replica %s was identified as having learner replicas, "+ "but no learner replicas were found", repl) @@ -961,11 +961,11 @@ func (rq *replicateQueue) shedLease( opts transferLeaseOptions, ) (leaseTransferOutcome, error) { // Learner replicas aren't allowed to become the leaseholder or raft leader, - // so only consider the `Voters` replicas. + // so only consider the `VoterDescriptors` replicas. target := rq.allocator.TransferLeaseTarget( ctx, zone, - desc.Replicas().Voters(), + desc.Replicas().VoterDescriptors(), repl.store.StoreID(), repl.leaseholderStats, opts.checkTransferLeaseSource, diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index d4fc9da8fa54..0bdf609dcd39 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -85,7 +85,7 @@ func testReplicateQueueRebalanceInner(t *testing.T, atomic bool) { // queue is already hard at work. testutils.SucceedsSoon(t, func() error { desc := tc.LookupRangeOrFatal(t, splitKey) - if i > 0 && len(desc.Replicas().Voters()) > 3 { + if i > 0 && len(desc.Replicas().VoterDescriptors()) > 3 { // Some system ranges have five replicas but user ranges only three, // so we'll see downreplications early in the startup process which // we want to ignore. Delay the splits so that we don't create @@ -146,7 +146,7 @@ func testReplicateQueueRebalanceInner(t *testing.T, atomic bool) { infos, err := queryRangeLog(tc.Conns[0], `SELECT info FROM system.rangelog ORDER BY timestamp DESC`) require.NoError(t, err) for _, info := range infos { - if _, ok := trackedRanges[info.UpdatedDesc.RangeID]; !ok || len(info.UpdatedDesc.Replicas().Voters()) <= 3 { + if _, ok := trackedRanges[info.UpdatedDesc.RangeID]; !ok || len(info.UpdatedDesc.Replicas().VoterDescriptors()) <= 3 { continue } // If we have atomic changes enabled, we expect to never see four replicas @@ -270,7 +270,7 @@ func TestReplicateQueueDownReplicate(t *testing.T) { // starts up with 5 replicas. Since it's not a system range, its default zone // config asks for 3x replication, and the replication queue will // down-replicate it. - require.Len(t, desc.Replicas().All(), 5) + require.Len(t, desc.Replicas().Descriptors(), 5) // Re-enable the replication queue. tc.ToggleReplicateQueues(true) diff --git a/pkg/kv/kvserver/reports/constraint_stats_report_test.go b/pkg/kv/kvserver/reports/constraint_stats_report_test.go index 6c45cd351a99..dd13a63f7315 100644 --- a/pkg/kv/kvserver/reports/constraint_stats_report_test.go +++ b/pkg/kv/kvserver/reports/constraint_stats_report_test.go @@ -809,8 +809,8 @@ func compileTestCase(tc baseReportTestCase) (compiledTestCase, error) { storeDescs = append(storeDescs, sds...) } storeResolver := func(r *roachpb.RangeDescriptor) []roachpb.StoreDescriptor { - stores := make([]roachpb.StoreDescriptor, len(r.Replicas().Voters())) - for i, rep := range r.Replicas().Voters() { + stores := make([]roachpb.StoreDescriptor, len(r.Replicas().VoterDescriptors())) + for i, rep := range r.Replicas().VoterDescriptors() { for _, desc := range storeDescs { if rep.StoreID == desc.StoreID { stores[i] = desc diff --git a/pkg/kv/kvserver/reports/critical_localities_report.go b/pkg/kv/kvserver/reports/critical_localities_report.go index 41d1c90afa51..f64bb15d7983 100644 --- a/pkg/kv/kvserver/reports/critical_localities_report.go +++ b/pkg/kv/kvserver/reports/critical_localities_report.go @@ -375,7 +375,7 @@ func (v *criticalLocalitiesVisitor) countRange( // "region:us-east,dc=new-york", we collect both "region:us-east" and // "region:us-east,dc=new-york". dedupLocal := make(map[string]roachpb.Locality) - for _, rep := range r.Replicas().All() { + for _, rep := range r.Replicas().Descriptors() { for s, loc := range v.allLocalities[rep.NodeID] { if _, ok := dedupLocal[s]; ok { continue @@ -405,7 +405,7 @@ func processLocalityForRange( // Compute the required quorum and the number of live nodes. If the number of // live nodes gets lower than the required quorum then the range is already // unavailable. - quorumCount := len(r.Replicas().Voters())/2 + 1 + quorumCount := len(r.Replicas().VoterDescriptors())/2 + 1 liveNodeCount := len(storeDescs) for _, storeDesc := range storeDescs { isStoreLive := nodeChecker(storeDesc.Node.NodeID) diff --git a/pkg/kv/kvserver/reports/replication_stats_report.go b/pkg/kv/kvserver/reports/replication_stats_report.go index 29bff152e5c4..8124294d5389 100644 --- a/pkg/kv/kvserver/reports/replication_stats_report.go +++ b/pkg/kv/kvserver/reports/replication_stats_report.go @@ -388,9 +388,9 @@ func (v *replicationStatsVisitor) visitSameZone(ctx context.Context, r *roachpb. func (v *replicationStatsVisitor) countRange( key ZoneKey, replicationFactor int, r *roachpb.RangeDescriptor, ) { - voters := len(r.Replicas().Voters()) + voters := len(r.Replicas().VoterDescriptors()) var liveVoters int - for _, rep := range r.Replicas().Voters() { + for _, rep := range r.Replicas().VoterDescriptors() { if v.nodeChecker(rep.NodeID) { liveVoters++ } diff --git a/pkg/kv/kvserver/reports/reporter.go b/pkg/kv/kvserver/reports/reporter.go index 717b2caef2c8..78a1bd0eb0b0 100644 --- a/pkg/kv/kvserver/reports/reporter.go +++ b/pkg/kv/kvserver/reports/reporter.go @@ -182,13 +182,13 @@ func (stats *Reporter) update( var getStoresFromGossip StoreResolver = func( r *roachpb.RangeDescriptor, ) []roachpb.StoreDescriptor { - storeDescs := make([]roachpb.StoreDescriptor, len(r.Replicas().Voters())) + storeDescs := make([]roachpb.StoreDescriptor, len(r.Replicas().VoterDescriptors())) // We'll return empty descriptors for stores that gossip doesn't have a // descriptor for. These stores will be considered to satisfy all // constraints. // TODO(andrei): note down that some descriptors were missing from gossip // somewhere in the report. - for i, repl := range r.Replicas().Voters() { + for i, repl := range r.Replicas().VoterDescriptors() { storeDescs[i] = allStores[repl.StoreID] } return storeDescs diff --git a/pkg/kv/kvserver/reset_quorum_test.go b/pkg/kv/kvserver/reset_quorum_test.go index 673cb3f3dc68..158147d9309c 100644 --- a/pkg/kv/kvserver/reset_quorum_test.go +++ b/pkg/kv/kvserver/reset_quorum_test.go @@ -87,7 +87,7 @@ func TestResetQuorum(t *testing.T) { require.NoError(t, tc.TransferRangeLease(desc, tc.Target(n2))) desc, err = tc.RemoveVoters(k, tc.Target(n1)) require.NoError(t, err) - require.Len(t, desc.Replicas().All(), 3) + require.Len(t, desc.Replicas().Descriptors(), 3) srv := tc.Server(n1) @@ -153,11 +153,11 @@ func TestResetQuorum(t *testing.T) { } return errors.Errorf("range id %v not found after resetting quorum", rangeID) })) - if len(updatedDesc.Replicas().All()) != 1 { - t.Fatalf("found %v replicas found after resetting quorum, expected 1", len(updatedDesc.Replicas().All())) + if len(updatedDesc.Replicas().Descriptors()) != 1 { + t.Fatalf("found %v replicas found after resetting quorum, expected 1", len(updatedDesc.Replicas().Descriptors())) } - if updatedDesc.Replicas().All()[0].NodeID != srv.NodeID() { - t.Fatalf("replica found after resetting quorum is on node id %v, expected node id %v", updatedDesc.Replicas().All()[0].NodeID, srv.NodeID()) + if updatedDesc.Replicas().Descriptors()[0].NodeID != srv.NodeID() { + t.Fatalf("replica found after resetting quorum is on node id %v, expected node id %v", updatedDesc.Replicas().Descriptors()[0].NodeID, srv.NodeID()) } } diff --git a/pkg/kv/kvserver/split_delay_helper.go b/pkg/kv/kvserver/split_delay_helper.go index c68cdfee29f3..ba4c9cbb9c25 100644 --- a/pkg/kv/kvserver/split_delay_helper.go +++ b/pkg/kv/kvserver/split_delay_helper.go @@ -36,7 +36,7 @@ func (sdh *splitDelayHelper) RaftStatus(ctx context.Context) (roachpb.RangeID, * raftStatus := r.raftStatusRLocked() if raftStatus != nil { updateRaftProgressFromActivity( - ctx, raftStatus.Progress, r.descRLocked().Replicas().All(), + ctx, raftStatus.Progress, r.descRLocked().Replicas().Descriptors(), func(replicaID roachpb.ReplicaID) bool { return r.mu.lastUpdateTimes.isFollowerActiveSince( ctx, replicaID, timeutil.Now(), r.store.cfg.RangeLeaseActiveDuration()) diff --git a/pkg/kv/kvserver/split_queue_test.go b/pkg/kv/kvserver/split_queue_test.go index 5bc8d716708f..02386650723e 100644 --- a/pkg/kv/kvserver/split_queue_test.go +++ b/pkg/kv/kvserver/split_queue_test.go @@ -81,7 +81,7 @@ func TestSplitQueueShouldQueue(t *testing.T) { cpy := *tc.repl.Desc() cpy.StartKey = test.start cpy.EndKey = test.end - repl, err := newReplica(ctx, &cpy, tc.store, cpy.Replicas().Voters()[0].ReplicaID) + repl, err := newReplica(ctx, &cpy, tc.store, cpy.Replicas().VoterDescriptors()[0].ReplicaID) if err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index ab1bfabf3680..547a113df938 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -1119,7 +1119,7 @@ func (s *Store) SetDraining(drain bool, reporter func(int, redact.SafeString)) { // Learner replicas aren't allowed to become the leaseholder or raft // leader, so only consider the `Voters` replicas. - needsLeaseTransfer := len(r.Desc().Replicas().Voters()) > 1 && + needsLeaseTransfer := len(r.Desc().Replicas().VoterDescriptors()) > 1 && drainingLease.OwnedBy(s.StoreID()) && r.IsLeaseValid(ctx, drainingLease, s.Clock().Now()) diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index 1580744e3a58..8111a7052e93 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -170,7 +170,7 @@ func WriteInitialClusterData( ReplicaID: 1, }, } - desc.SetReplicas(roachpb.MakeReplicaDescriptors(replicas)) + desc.SetReplicas(roachpb.MakeReplicaSet(replicas)) if err := desc.Validate(); err != nil { return err } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index dd7f7f7cf2b6..2ac5efc66351 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -292,7 +292,7 @@ func (sr *StoreRebalancer) rebalanceStore( replicasToMaybeRebalance = append(replicasToMaybeRebalance, hottestRanges...) for localDesc.Capacity.QueriesPerSecond > qpsMaxThreshold { - replWithStats, targets := sr.chooseReplicaToRebalance( + replWithStats, voterTargets := sr.chooseReplicaToRebalance( ctx, &replicasToMaybeRebalance, localDesc, @@ -309,12 +309,13 @@ func (sr *StoreRebalancer) rebalanceStore( descBeforeRebalance := replWithStats.repl.Desc() log.VEventf(ctx, 1, "rebalancing r%d (%.2f qps) from %v to %v to better balance load", - replWithStats.repl.RangeID, replWithStats.qps, descBeforeRebalance.Replicas(), targets) + replWithStats.repl.RangeID, replWithStats.qps, descBeforeRebalance.Replicas(), voterTargets) timeout := sr.rq.processTimeoutFunc(sr.st, replWithStats.repl) if err := contextutil.RunWithTimeout(ctx, "relocate range", timeout, func(ctx context.Context) error { - return sr.rq.store.AdminRelocateRange(ctx, *descBeforeRebalance, targets) + // TODO(aayush): Fix when we can make decisions about rebalancing non-voting replicas. + return sr.rq.store.AdminRelocateRange(ctx, *descBeforeRebalance, voterTargets, []roachpb.ReplicationTarget{}) }); err != nil { - log.Errorf(ctx, "unable to relocate range to %v: %+v", targets, err) + log.Errorf(ctx, "unable to relocate range to %v: %+v", voterTargets, err) continue } sr.metrics.RangeRebalanceCount.Inc(1) @@ -326,7 +327,7 @@ func (sr *StoreRebalancer) rebalanceStore( // TODO(a-robinson): This just updates the copies used locally by the // storeRebalancer. We may also want to update the copies in the StorePool // itself. - replicasBeforeRebalance := descBeforeRebalance.Replicas().All() + replicasBeforeRebalance := descBeforeRebalance.Replicas().Descriptors() for i := range replicasBeforeRebalance { if storeDesc := storeMap[replicasBeforeRebalance[i].StoreID]; storeDesc != nil { storeDesc.Capacity.RangeCount-- @@ -334,8 +335,8 @@ func (sr *StoreRebalancer) rebalanceStore( } localDesc.Capacity.LeaseCount-- localDesc.Capacity.QueriesPerSecond -= replWithStats.qps - for i := range targets { - if storeDesc := storeMap[targets[i].StoreID]; storeDesc != nil { + for i := range voterTargets { + if storeDesc := storeMap[voterTargets[i].StoreID]; storeDesc != nil { storeDesc.Capacity.RangeCount++ if i == 0 { storeDesc.Capacity.LeaseCount++ @@ -398,7 +399,7 @@ func (sr *StoreRebalancer) chooseLeaseToTransfer( // Check all the other replicas in order of increasing qps. Learner replicas // aren't allowed to become the leaseholder or raft leader, so only consider // the `Voters` replicas. - candidates := desc.Replicas().DeepCopy().Voters() + candidates := desc.Replicas().DeepCopy().VoterDescriptors() sort.Slice(candidates, func(i, j int) bool { var iQPS, jQPS float64 if desc := storeMap[candidates[i].StoreID]; desc != nil { @@ -506,7 +507,7 @@ func (sr *StoreRebalancer) chooseReplicaToRebalance( desiredReplicas := GetNeededReplicas(*zone.NumReplicas, clusterNodes) targets := make([]roachpb.ReplicationTarget, 0, desiredReplicas) targetReplicas := make([]roachpb.ReplicaDescriptor, 0, desiredReplicas) - currentReplicas := desc.Replicas().All() + currentReplicas := desc.Replicas().Descriptors() // Check the range's existing diversity score, since we want to ensure we // don't hurt locality diversity just to improve QPS. diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index 038f2246a976..17bf196f0348 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -199,7 +199,7 @@ func splitPostApply( if rightReplOrNil != nil { r.store.splitQueue.MaybeAddAsync(ctx, rightReplOrNil, now) r.store.replicateQueue.MaybeAddAsync(ctx, rightReplOrNil, now) - if len(split.RightDesc.Replicas().All()) == 1 { + if len(split.RightDesc.Replicas().Descriptors()) == 1 { // TODO(peter): In single-node clusters, we enqueue the right-hand side of // the split (the new range) for Raft processing so that the corresponding // Raft group is created. This shouldn't be necessary for correctness, but diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index b5a8a063055c..7861efc52d08 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -74,7 +74,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -102,7 +102,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{1} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{1} } type ChecksumMode int32 @@ -149,7 +149,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{2} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -180,7 +180,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{3} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{3} } type ExternalStorageProvider int32 @@ -221,7 +221,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{4} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{4} } type MVCCFilter int32 @@ -244,7 +244,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{5} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{1, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{25, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -333,7 +333,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -405,7 +405,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{1} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,7 +439,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{2} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -482,7 +482,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{3} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -525,7 +525,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{4} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +559,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{5} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -648,7 +648,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{6} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -683,7 +683,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{7} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -729,7 +729,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{8} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -763,7 +763,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{9} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -803,7 +803,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{10} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -840,7 +840,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{11} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -874,7 +874,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{12} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -908,7 +908,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{13} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -960,7 +960,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{14} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -997,7 +997,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{15} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1052,7 +1052,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{16} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1086,7 +1086,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{17} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1128,7 +1128,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{18} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1162,7 +1162,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{19} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1213,7 +1213,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{20} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1265,7 +1265,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{21} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1316,7 +1316,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{22} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1368,7 +1368,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{23} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1421,7 +1421,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{24} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1458,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{25} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1502,7 +1502,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{25, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1550,7 +1550,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{26} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1586,7 +1586,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{27} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1697,7 +1697,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{28} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1743,7 +1743,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{29} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1804,7 +1804,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{30} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1839,7 +1839,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{31} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1878,7 +1878,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{32} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1913,7 +1913,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{33} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1956,7 +1956,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{34} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1991,7 +1991,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{35} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2029,7 +2029,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{36} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2062,7 +2062,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{37} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2097,7 +2097,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{38} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2155,7 +2155,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{39} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2190,7 +2190,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{40} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2219,15 +2219,16 @@ var xxx_messageInfo_AdminChangeReplicasResponse proto.InternalMessageInfo // method. Relocates the replicas for a range to the specified target stores. // The first store in the list of targets becomes the new leaseholder. type AdminRelocateRangeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` - Targets []ReplicationTarget `protobuf:"bytes,2,rep,name=targets,proto3" json:"targets"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + VoterTargets []ReplicationTarget `protobuf:"bytes,2,rep,name=voter_targets,json=voterTargets,proto3" json:"voter_targets"` + NonVoterTargets []ReplicationTarget `protobuf:"bytes,3,rep,name=non_voter_targets,json=nonVoterTargets,proto3" json:"non_voter_targets"` } func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeRequest{} } func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{41} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2260,7 +2261,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{42} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2302,7 +2303,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{43} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2339,7 +2340,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{44} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2377,7 +2378,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{45} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2411,7 +2412,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{45, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2445,7 +2446,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{46} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2514,7 +2515,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{47} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2557,7 +2558,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{48} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2604,7 +2605,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{49} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2640,7 +2641,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{50} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2684,7 +2685,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{51} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2728,7 +2729,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{52} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2788,7 +2789,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{53} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2824,7 +2825,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{54} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2871,7 +2872,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{55} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2906,7 +2907,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{56} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2957,7 +2958,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{57} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2992,7 +2993,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{58} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3029,7 +3030,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{59} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3063,7 +3064,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{60} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3108,7 +3109,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{61} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3142,7 +3143,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{62} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3186,7 +3187,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{63} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3235,7 +3236,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{64} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3272,7 +3273,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{65} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3309,7 +3310,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{66} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3344,7 +3345,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{67} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3399,7 +3400,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{68} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3436,7 +3437,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{69} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3476,7 +3477,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3510,7 +3511,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3543,7 +3544,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70, 1} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3585,7 +3586,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70, 2} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3624,7 +3625,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70, 3} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3660,7 +3661,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70, 4} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3699,7 +3700,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70, 5} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3741,7 +3742,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{70, 6} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3781,7 +3782,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{71} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3815,7 +3816,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{72} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3851,7 +3852,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{73} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3921,7 +3922,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{74} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3973,7 +3974,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{75} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4009,7 +4010,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{76} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4049,7 +4050,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{76, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4100,7 +4101,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{77} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4135,7 +4136,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{77, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4171,7 +4172,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{77, 1} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4206,7 +4207,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{78} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4244,7 +4245,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{79} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4281,7 +4282,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{80} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4314,7 +4315,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{80, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4359,7 +4360,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{81} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4397,7 +4398,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{82} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4450,7 +4451,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{83} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4484,7 +4485,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{84} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4528,7 +4529,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{85} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4562,7 +4563,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{86} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4601,7 +4602,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{87} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4635,7 +4636,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{88} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4684,7 +4685,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{89} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4733,7 +4734,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{90} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4768,7 +4769,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{91} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4810,7 +4811,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{92} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4853,7 +4854,7 @@ func (m *MigrateRequest) Reset() { *m = MigrateRequest{} } func (m *MigrateRequest) String() string { return proto.CompactTextString(m) } func (*MigrateRequest) ProtoMessage() {} func (*MigrateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{93} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{93} } func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4887,7 +4888,7 @@ func (m *MigrateResponse) Reset() { *m = MigrateResponse{} } func (m *MigrateResponse) String() string { return proto.CompactTextString(m) } func (*MigrateResponse) ProtoMessage() {} func (*MigrateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{94} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{94} } func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4971,7 +4972,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{95} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6452,7 +6453,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{96} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8017,7 +8018,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{97} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8055,7 +8056,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{98} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8091,7 +8092,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{99} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8128,7 +8129,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{100} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8207,7 +8208,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{100, 0} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8245,7 +8246,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{101} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8283,7 +8284,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{102} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8322,7 +8323,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{103} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8363,7 +8364,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{104} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8404,7 +8405,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{105} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8441,7 +8442,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{106} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8478,7 +8479,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{107} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8522,7 +8523,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{108} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8554,7 +8555,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{109} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8595,7 +8596,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{110} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8635,7 +8636,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{111} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8671,7 +8672,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{112} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8710,7 +8711,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{113} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8753,7 +8754,7 @@ func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (m *ContentionEvent) String() string { return proto.CompactTextString(m) } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e06a135f78303cad, []int{114} + return fileDescriptor_api_be6b7a2c0d1c6f44, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -10921,8 +10922,8 @@ func (m *AdminRelocateRangeRequest) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n57 - if len(m.Targets) > 0 { - for _, msg := range m.Targets { + if len(m.VoterTargets) > 0 { + for _, msg := range m.VoterTargets { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(msg.Size())) @@ -10933,6 +10934,18 @@ func (m *AdminRelocateRangeRequest) MarshalTo(dAtA []byte) (int, error) { i += n } } + if len(m.NonVoterTargets) > 0 { + for _, msg := range m.NonVoterTargets { + dAtA[i] = 0x1a + i++ + i = encodeVarintApi(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } return i, nil } @@ -16680,8 +16693,14 @@ func (m *AdminRelocateRangeRequest) Size() (n int) { _ = l l = m.RequestHeader.Size() n += 1 + l + sovApi(uint64(l)) - if len(m.Targets) > 0 { - for _, e := range m.Targets { + if len(m.VoterTargets) > 0 { + for _, e := range m.VoterTargets { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } + if len(m.NonVoterTargets) > 0 { + for _, e := range m.NonVoterTargets { l = e.Size() n += 1 + l + sovApi(uint64(l)) } @@ -24529,7 +24548,38 @@ func (m *AdminRelocateRangeRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Targets", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field VoterTargets", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VoterTargets = append(m.VoterTargets, ReplicationTarget{}) + if err := m.VoterTargets[len(m.VoterTargets)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NonVoterTargets", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -24553,8 +24603,8 @@ func (m *AdminRelocateRangeRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Targets = append(m.Targets, ReplicationTarget{}) - if err := m.Targets[len(m.Targets)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.NonVoterTargets = append(m.NonVoterTargets, ReplicationTarget{}) + if err := m.NonVoterTargets[len(m.NonVoterTargets)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -39119,523 +39169,525 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_e06a135f78303cad) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_be6b7a2c0d1c6f44) } -var fileDescriptor_api_e06a135f78303cad = []byte{ - // 8230 bytes of a gzipped FileDescriptorProto +var fileDescriptor_api_be6b7a2c0d1c6f44 = []byte{ + // 8257 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0xc9, - 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0x51, 0x69, 0x7e, 0xb8, 0xdc, 0xdd, 0xd1, 0x4c, - 0xcf, 0xff, 0xdc, 0x5d, 0x69, 0x67, 0xe6, 0xae, 0x77, 0xbd, 0xb3, 0xde, 0x6b, 0x89, 0xe2, 0x0c, - 0x29, 0x8d, 0x34, 0x9a, 0x26, 0x35, 0x93, 0x5d, 0x5f, 0xa7, 0x6f, 0xab, 0xbb, 0x44, 0xf5, 0x15, - 0xd9, 0xcd, 0xe9, 0x6e, 0xea, 0x67, 0x81, 0x00, 0x71, 0x6c, 0x38, 0x06, 0x02, 0x18, 0xf7, 0xc1, - 0x41, 0xae, 0xe1, 0x24, 0xbe, 0x8e, 0xe3, 0xf8, 0x21, 0x40, 0x12, 0x20, 0x41, 0x12, 0x04, 0x89, + 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0x51, 0x69, 0x7e, 0x38, 0xdc, 0xdd, 0xd1, 0x4c, + 0xcf, 0xff, 0x78, 0x57, 0xda, 0x99, 0xb9, 0xeb, 0x5d, 0xef, 0xac, 0xf7, 0x5a, 0xa2, 0x38, 0x43, + 0x4a, 0x23, 0x8d, 0xa6, 0x49, 0xcd, 0x64, 0xd7, 0xd7, 0xe9, 0xdb, 0xea, 0x2e, 0x51, 0x7d, 0x45, + 0x76, 0x73, 0xba, 0x9b, 0xfa, 0x59, 0x20, 0x40, 0x1c, 0x1b, 0xce, 0x05, 0x02, 0x18, 0xf7, 0xc1, + 0x41, 0xae, 0xe1, 0x24, 0xbe, 0x8e, 0x93, 0xf8, 0x21, 0x40, 0x12, 0x20, 0x41, 0x12, 0x04, 0x89, 0xfd, 0x62, 0x20, 0x17, 0x81, 0x93, 0x5c, 0x3f, 0xc5, 0x08, 0x10, 0xc5, 0xd6, 0xcd, 0x4b, 0x90, - 0xc0, 0x08, 0x82, 0x00, 0x06, 0xf6, 0x21, 0x08, 0xea, 0xa7, 0xff, 0xc8, 0xe6, 0x8f, 0x66, 0x7b, - 0xe3, 0x05, 0xee, 0x8b, 0xc4, 0x3e, 0x55, 0xe7, 0x74, 0xd5, 0xa9, 0xaa, 0x53, 0xe7, 0xab, 0x3a, - 0x55, 0x0d, 0xf3, 0x96, 0xa9, 0xa8, 0xfb, 0xdd, 0xdd, 0x65, 0xa5, 0xab, 0x2f, 0x75, 0x2d, 0xd3, - 0x31, 0xd1, 0xbc, 0x6a, 0xaa, 0x07, 0x94, 0xbc, 0xc4, 0x13, 0xcb, 0xf7, 0x0e, 0x0e, 0x97, 0x0f, - 0x0e, 0x6d, 0x6c, 0x1d, 0x62, 0x6b, 0x59, 0x35, 0x0d, 0xb5, 0x67, 0x59, 0xd8, 0x50, 0x4f, 0x96, - 0xdb, 0xa6, 0x7a, 0x40, 0xff, 0xe8, 0x46, 0x8b, 0xb1, 0x97, 0x91, 0x2b, 0x51, 0x53, 0x1c, 0x85, - 0xd3, 0x2e, 0xb8, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x25, 0x97, 0xda, 0xc1, 0x8e, 0x12, - 0xc8, 0xfd, 0xa6, 0xed, 0x98, 0x96, 0xd2, 0xc2, 0xcb, 0xd8, 0x68, 0xe9, 0x06, 0x26, 0x19, 0x0e, - 0x55, 0x95, 0x27, 0xbe, 0x15, 0x99, 0xf8, 0x90, 0xa7, 0x96, 0x7a, 0x8e, 0xde, 0x5e, 0xde, 0x6f, - 0xab, 0xcb, 0x8e, 0xde, 0xc1, 0xb6, 0xa3, 0x74, 0xba, 0x3c, 0xe5, 0x1e, 0x4d, 0x71, 0x2c, 0x45, - 0xd5, 0x8d, 0x96, 0xfb, 0xbf, 0xbb, 0xbb, 0x6c, 0x61, 0xd5, 0xb4, 0x34, 0xac, 0xc9, 0x76, 0x57, - 0x31, 0xdc, 0xe2, 0xb6, 0xcc, 0x96, 0x49, 0x7f, 0x2e, 0x93, 0x5f, 0x9c, 0x7a, 0xa5, 0x65, 0x9a, - 0xad, 0x36, 0x5e, 0xa6, 0x4f, 0xbb, 0xbd, 0xbd, 0x65, 0xad, 0x67, 0x29, 0x8e, 0x6e, 0x72, 0x2e, - 0xf1, 0x9f, 0x0b, 0x90, 0x93, 0xf0, 0xab, 0x1e, 0xb6, 0x9d, 0x1a, 0x56, 0x34, 0x6c, 0xa1, 0x37, - 0x20, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0xab, 0xc2, 0x9d, 0xb9, 0xd5, 0xd9, 0x2f, 0x4e, 0x17, 0x93, - 0x1b, 0xf8, 0x44, 0x22, 0x34, 0x74, 0x15, 0x66, 0xb1, 0xa1, 0xc9, 0x24, 0x39, 0x15, 0x4e, 0x9e, - 0xc1, 0x86, 0xb6, 0x81, 0x4f, 0xd0, 0xb7, 0x21, 0x6d, 0x13, 0x69, 0x86, 0x8a, 0x4b, 0xd3, 0x57, - 0x85, 0x3b, 0xd3, 0xab, 0x3f, 0xfb, 0xc5, 0xe9, 0xe2, 0xc7, 0x2d, 0xdd, 0xd9, 0xef, 0xed, 0x2e, - 0xa9, 0x66, 0x67, 0xd9, 0x6b, 0x27, 0x6d, 0xd7, 0xff, 0xbd, 0xdc, 0x3d, 0x68, 0x2d, 0xf7, 0xeb, - 0x68, 0xa9, 0x79, 0x6c, 0x34, 0xf0, 0x2b, 0xc9, 0x93, 0xb8, 0x9e, 0x4a, 0x0b, 0xc5, 0xc4, 0x7a, - 0x2a, 0x9d, 0x28, 0x26, 0xc5, 0xdf, 0x4e, 0x42, 0x5e, 0xc2, 0x76, 0xd7, 0x34, 0x6c, 0xcc, 0x4b, - 0xfe, 0x1e, 0x24, 0x9d, 0x63, 0x83, 0x96, 0x3c, 0xfb, 0xe0, 0xca, 0xd2, 0x40, 0x8f, 0x58, 0x6a, - 0x5a, 0x8a, 0x61, 0x2b, 0x2a, 0xa9, 0xbe, 0x44, 0xb2, 0xa2, 0x0f, 0x21, 0x6b, 0x61, 0xbb, 0xd7, - 0xc1, 0x54, 0x91, 0xb4, 0x52, 0xd9, 0x07, 0x97, 0x23, 0x38, 0x1b, 0x5d, 0xc5, 0x90, 0x80, 0xe5, - 0x25, 0xbf, 0xd1, 0x1b, 0x90, 0x36, 0x7a, 0x1d, 0xa2, 0x0a, 0x9b, 0x56, 0x34, 0x29, 0xcd, 0x1a, - 0xbd, 0xce, 0x06, 0x3e, 0xb1, 0xd1, 0x5f, 0x82, 0x4b, 0x1a, 0xee, 0x5a, 0x58, 0x55, 0x1c, 0xac, - 0xc9, 0x96, 0x62, 0xb4, 0xb0, 0xac, 0x1b, 0x7b, 0xa6, 0x5d, 0x9a, 0xb9, 0x9a, 0xbc, 0x93, 0x7d, - 0xf0, 0x56, 0x84, 0x7c, 0x89, 0xe4, 0xaa, 0x1b, 0x7b, 0xe6, 0x6a, 0xea, 0x87, 0xa7, 0x8b, 0x53, - 0xd2, 0x05, 0x5f, 0x82, 0x97, 0x64, 0xa3, 0x06, 0xe4, 0x78, 0x71, 0x2d, 0xac, 0xd8, 0xa6, 0x51, - 0x9a, 0xbd, 0x2a, 0xdc, 0xc9, 0x3f, 0x58, 0x8a, 0x12, 0x18, 0x52, 0x0d, 0x79, 0xec, 0x75, 0xb0, - 0x44, 0xb9, 0xa4, 0x39, 0x2b, 0xf0, 0x84, 0xde, 0x84, 0x0c, 0xa9, 0xc9, 0xee, 0x89, 0x83, 0xed, - 0x52, 0x9a, 0x56, 0x85, 0x54, 0x6d, 0x95, 0x3c, 0x8b, 0x9f, 0xc0, 0x5c, 0x90, 0x15, 0x21, 0xc8, - 0x4b, 0xd5, 0xc6, 0xce, 0x66, 0x55, 0xde, 0xd9, 0xda, 0xd8, 0x7a, 0xf6, 0x72, 0xab, 0x38, 0x85, - 0x2e, 0x40, 0x91, 0xd3, 0x36, 0xaa, 0x9f, 0xca, 0x4f, 0xeb, 0x9b, 0xf5, 0x66, 0x51, 0x28, 0xa7, - 0x7e, 0xe5, 0xb7, 0xaf, 0x4c, 0x89, 0xdb, 0x00, 0x4f, 0xb0, 0xc3, 0x3b, 0x18, 0x5a, 0x85, 0x99, - 0x7d, 0x5a, 0x9e, 0x92, 0x40, 0x35, 0x7d, 0x35, 0xb2, 0xe0, 0x81, 0xce, 0xb8, 0x9a, 0x26, 0xda, - 0xf8, 0xd1, 0xe9, 0xa2, 0x20, 0x71, 0x4e, 0xf1, 0xf7, 0x05, 0xc8, 0x52, 0x91, 0xac, 0x7e, 0xa8, - 0xd2, 0x27, 0xf3, 0xda, 0x58, 0x65, 0x0c, 0x0a, 0x45, 0x4b, 0x30, 0x7d, 0xa8, 0xb4, 0x7b, 0xb8, - 0x94, 0xa0, 0x32, 0x4a, 0x11, 0x32, 0x5e, 0x90, 0x74, 0x89, 0x65, 0x43, 0x8f, 0x60, 0x4e, 0x37, - 0x1c, 0x6c, 0x38, 0x32, 0x63, 0x4b, 0x8e, 0x61, 0xcb, 0xb2, 0xdc, 0xf4, 0x41, 0xfc, 0x67, 0x02, - 0xc0, 0x76, 0x2f, 0x4e, 0xa5, 0xa0, 0x6f, 0x4e, 0x58, 0x7e, 0xde, 0xbb, 0x78, 0x2d, 0x2e, 0xc1, - 0x8c, 0x6e, 0xb4, 0x75, 0x83, 0x95, 0x3f, 0x2d, 0xf1, 0x27, 0x74, 0x01, 0xa6, 0x77, 0xdb, 0xba, - 0xa1, 0xd1, 0xf1, 0x90, 0x96, 0xd8, 0x83, 0x28, 0x41, 0x96, 0x96, 0x3a, 0x46, 0xbd, 0x8b, 0xa7, - 0x09, 0xb8, 0x58, 0x31, 0x0d, 0x4d, 0x27, 0x43, 0x52, 0x69, 0x7f, 0x2d, 0xb4, 0xb2, 0x0e, 0x81, - 0xc1, 0x27, 0xe3, 0xe3, 0xee, 0x84, 0x6d, 0x8c, 0x7c, 0xae, 0xea, 0x71, 0x97, 0xd2, 0xa2, 0x35, - 0x89, 0xbe, 0x09, 0x97, 0x95, 0x76, 0xdb, 0x3c, 0x92, 0xf5, 0x3d, 0x59, 0x33, 0xb1, 0x2d, 0x1b, - 0xa6, 0x23, 0xe3, 0x63, 0xdd, 0x76, 0xa8, 0x29, 0x49, 0x4b, 0x0b, 0x34, 0xb9, 0xbe, 0xb7, 0x66, - 0x62, 0x7b, 0xcb, 0x74, 0xaa, 0x24, 0x89, 0x8c, 0x53, 0x52, 0x18, 0x36, 0x4e, 0x67, 0x88, 0xf9, - 0x95, 0xd2, 0xf8, 0xb8, 0x4b, 0xc7, 0x69, 0xa0, 0x29, 0x67, 0x83, 0x4d, 0x29, 0xfe, 0x3c, 0x5c, - 0xea, 0xd7, 0x6f, 0x9c, 0xed, 0xf7, 0x87, 0x02, 0xe4, 0xeb, 0x86, 0xee, 0x7c, 0x2d, 0x1a, 0xce, - 0x53, 0x76, 0x32, 0xa8, 0xec, 0x7b, 0x50, 0xdc, 0x53, 0xf4, 0xf6, 0x33, 0xa3, 0x69, 0x76, 0x76, - 0x6d, 0xc7, 0x34, 0xb0, 0xcd, 0x5b, 0x63, 0x80, 0x2e, 0xbe, 0x80, 0x82, 0x57, 0x9b, 0x38, 0xd5, - 0xe4, 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x05, 0x19, 0xdd, 0x95, - 0x4b, 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, 0x83, 0xf9, 0xc0, 0x5b, 0xe3, 0x34, 0x97, 0x64, 0xca, - 0xc0, 0x47, 0xb2, 0xdf, 0x46, 0x64, 0xca, 0xc0, 0x47, 0xcc, 0xbc, 0x35, 0x20, 0xb7, 0x86, 0xdb, - 0xd8, 0xc1, 0x71, 0x5a, 0xfd, 0x1d, 0xc8, 0xbb, 0x42, 0xe3, 0x6c, 0x98, 0xbf, 0x25, 0x00, 0xe2, - 0x72, 0xc9, 0x2c, 0x1b, 0x67, 0xdb, 0x2c, 0x12, 0xd7, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x11, 0x58, - 0x9f, 0x04, 0x46, 0xa2, 0x6e, 0x82, 0x3f, 0x64, 0x53, 0xc1, 0x21, 0xcb, 0xdd, 0x9b, 0x23, 0x58, - 0x08, 0x15, 0x2c, 0xde, 0xe6, 0x4b, 0xd1, 0x32, 0x25, 0xae, 0x26, 0x83, 0x3e, 0x1c, 0x25, 0x8a, - 0xdf, 0x17, 0x60, 0xbe, 0xd2, 0xc6, 0x8a, 0x15, 0xbb, 0x46, 0xbe, 0x05, 0x69, 0x0d, 0x2b, 0x1a, - 0xad, 0x32, 0x1b, 0xd8, 0x6f, 0x07, 0xa4, 0x10, 0x4f, 0x77, 0x69, 0xbf, 0xad, 0x2e, 0x35, 0x5d, - 0x1f, 0x98, 0x8f, 0x6e, 0x8f, 0x49, 0xfc, 0x14, 0x50, 0xb0, 0x64, 0x71, 0x76, 0x84, 0xff, 0x23, - 0x00, 0x92, 0xf0, 0x21, 0xb6, 0x9c, 0xd8, 0xab, 0xbd, 0x06, 0x59, 0x47, 0xb1, 0x5a, 0xd8, 0x91, - 0x89, 0x77, 0x7f, 0x9e, 0x9a, 0x03, 0xe3, 0x23, 0x64, 0xd4, 0x84, 0xdb, 0xd8, 0x50, 0x76, 0xdb, - 0x98, 0x4a, 0x91, 0x77, 0xcd, 0x9e, 0xa1, 0xc9, 0xba, 0x83, 0x2d, 0xc5, 0x31, 0x2d, 0xd9, 0xec, - 0x3a, 0x7a, 0x47, 0xff, 0x9c, 0x3a, 0xf6, 0xbc, 0xab, 0x5d, 0x67, 0xd9, 0x09, 0xf3, 0x2a, 0xc9, - 0x5c, 0xe7, 0x79, 0x9f, 0x05, 0xb2, 0x8a, 0x9f, 0xc1, 0x42, 0xa8, 0xd6, 0x71, 0xaa, 0xf4, 0x7f, - 0x09, 0x90, 0x6d, 0xa8, 0x8a, 0x11, 0xa7, 0x2e, 0x3f, 0x81, 0xac, 0xad, 0x2a, 0x86, 0xbc, 0x67, - 0x5a, 0x1d, 0xc5, 0xa1, 0x03, 0x27, 0x1f, 0xd2, 0xa5, 0xe7, 0xaf, 0xab, 0x8a, 0xf1, 0x98, 0x66, - 0x92, 0xc0, 0xf6, 0x7e, 0xa3, 0xe7, 0x90, 0x3d, 0xc0, 0x27, 0x32, 0xc7, 0x7e, 0x74, 0xb6, 0xcd, - 0x3f, 0x78, 0x2f, 0xc0, 0x7f, 0x70, 0xb8, 0xe4, 0x42, 0xc6, 0xa5, 0x00, 0x64, 0x5c, 0x22, 0x1c, - 0x4b, 0x0d, 0xc7, 0xc2, 0x46, 0xcb, 0xd9, 0x97, 0xe0, 0x00, 0x9f, 0x3c, 0x65, 0x32, 0xd8, 0x70, - 0x5d, 0x4f, 0xa5, 0x93, 0xc5, 0x94, 0xf8, 0xe7, 0x02, 0xcc, 0xb1, 0x2a, 0xc7, 0x39, 0x5c, 0xdf, - 0x87, 0x94, 0x65, 0x1e, 0xb1, 0xe1, 0x9a, 0x7d, 0xf0, 0x66, 0x84, 0x88, 0x0d, 0x7c, 0x12, 0x9c, - 0x0f, 0x69, 0x76, 0xb4, 0x0a, 0xdc, 0xeb, 0x94, 0x29, 0x77, 0x72, 0x52, 0x6e, 0x60, 0x5c, 0x12, - 0x91, 0x71, 0x1b, 0x0a, 0xbb, 0x8a, 0xa3, 0xee, 0xcb, 0x16, 0x2f, 0x24, 0x99, 0x3b, 0x93, 0x77, - 0xe6, 0xa4, 0x3c, 0x25, 0xbb, 0x45, 0xb7, 0x49, 0xcd, 0xd9, 0xf8, 0xb1, 0xf1, 0x4f, 0x58, 0x9b, - 0xff, 0x5f, 0x81, 0x8f, 0x21, 0xb7, 0xe6, 0x3f, 0x69, 0x4d, 0xff, 0xeb, 0x09, 0xb8, 0x5c, 0xd9, - 0xc7, 0xea, 0x41, 0xc5, 0x34, 0x6c, 0xdd, 0x76, 0x88, 0xee, 0xe2, 0x6c, 0xff, 0x37, 0x21, 0x73, - 0xa4, 0x3b, 0xfb, 0xb2, 0xa6, 0xef, 0xed, 0x51, 0xeb, 0x99, 0x96, 0xd2, 0x84, 0xb0, 0xa6, 0xef, - 0xed, 0xa1, 0x87, 0x90, 0xea, 0x98, 0x1a, 0x73, 0xce, 0xf3, 0x0f, 0x16, 0x23, 0xc4, 0xd3, 0xa2, - 0xd9, 0xbd, 0xce, 0xa6, 0xa9, 0x61, 0x89, 0x66, 0x46, 0x57, 0x00, 0x54, 0x42, 0xed, 0x9a, 0xba, - 0xe1, 0xf0, 0xd9, 0x37, 0x40, 0x41, 0x35, 0xc8, 0x38, 0xd8, 0xea, 0xe8, 0x86, 0xe2, 0xe0, 0xd2, - 0x34, 0x55, 0xde, 0x8d, 0xc8, 0x82, 0x77, 0xdb, 0xba, 0xaa, 0xac, 0x61, 0x5b, 0xb5, 0xf4, 0xae, - 0x63, 0x5a, 0x5c, 0x8b, 0x3e, 0xb3, 0xf8, 0xab, 0x29, 0x28, 0x0d, 0xea, 0x26, 0xce, 0x1e, 0xb2, - 0x0d, 0x33, 0x04, 0xcd, 0xb7, 0x1d, 0xde, 0x47, 0x1e, 0x0c, 0x53, 0x41, 0x44, 0x09, 0xe8, 0xaa, - 0x40, 0xdb, 0xe1, 0xc5, 0xe6, 0x72, 0xca, 0xff, 0x5a, 0x80, 0x19, 0x96, 0x80, 0xee, 0x43, 0x9a, - 0x2f, 0x5f, 0x68, 0xb4, 0x8c, 0xc9, 0xd5, 0x4b, 0x67, 0xa7, 0x8b, 0xb3, 0x6c, 0x45, 0x62, 0xed, - 0x0b, 0xff, 0xa7, 0x34, 0x4b, 0xf3, 0xd5, 0x35, 0xd2, 0x5a, 0xb6, 0xa3, 0x58, 0x0e, 0x5d, 0x24, - 0x4a, 0x30, 0x94, 0x42, 0x09, 0x1b, 0xf8, 0x04, 0xad, 0xc3, 0x8c, 0xed, 0x28, 0x4e, 0xcf, 0xe6, - 0xed, 0x75, 0xae, 0xc2, 0x36, 0x28, 0xa7, 0xc4, 0x25, 0x10, 0xf7, 0x49, 0xc3, 0x8e, 0xa2, 0xb7, - 0x69, 0x03, 0x66, 0x24, 0xfe, 0x24, 0xfe, 0x86, 0x00, 0x33, 0x2c, 0x2b, 0xba, 0x0c, 0x0b, 0xd2, - 0xca, 0xd6, 0x93, 0xaa, 0x5c, 0xdf, 0x5a, 0xab, 0x36, 0xab, 0xd2, 0x66, 0x7d, 0x6b, 0xa5, 0x59, - 0x2d, 0x4e, 0xa1, 0x4b, 0x80, 0xdc, 0x84, 0xca, 0xb3, 0xad, 0x46, 0xbd, 0xd1, 0xac, 0x6e, 0x35, - 0x8b, 0x02, 0x5d, 0xc9, 0xa0, 0xf4, 0x00, 0x35, 0x81, 0x6e, 0xc0, 0xd5, 0x7e, 0xaa, 0xdc, 0x68, - 0xae, 0x34, 0x1b, 0x72, 0xb5, 0xd1, 0xac, 0x6f, 0xae, 0x34, 0xab, 0x6b, 0xc5, 0xe4, 0x88, 0x5c, - 0xe4, 0x25, 0x92, 0x54, 0xad, 0x34, 0x8b, 0x29, 0xd1, 0x81, 0x8b, 0x12, 0x56, 0xcd, 0x4e, 0xb7, - 0xe7, 0x60, 0x52, 0x4a, 0x3b, 0xce, 0x91, 0x72, 0x19, 0x66, 0x35, 0xeb, 0x44, 0xb6, 0x7a, 0x06, - 0x1f, 0x27, 0x33, 0x9a, 0x75, 0x22, 0xf5, 0x0c, 0xf1, 0x9f, 0x08, 0x70, 0xa9, 0xff, 0xb5, 0x71, - 0x76, 0xc2, 0xe7, 0x90, 0x55, 0x34, 0x0d, 0x6b, 0xb2, 0x86, 0xdb, 0x8e, 0xc2, 0x5d, 0x9c, 0x7b, - 0x01, 0x49, 0x7c, 0x69, 0x6f, 0xc9, 0x5b, 0xda, 0xdb, 0x7c, 0x51, 0xa9, 0xd0, 0x82, 0xac, 0x11, - 0x0e, 0xd7, 0xfc, 0x50, 0x21, 0x94, 0x22, 0xfe, 0xcf, 0x14, 0xe4, 0xaa, 0x86, 0xd6, 0x3c, 0x8e, - 0x75, 0x2e, 0xb9, 0x04, 0x33, 0xaa, 0xd9, 0xe9, 0xe8, 0x8e, 0xab, 0x20, 0xf6, 0x84, 0x7e, 0x3a, - 0xe0, 0x9a, 0x26, 0x27, 0x70, 0xd0, 0x7c, 0xa7, 0x14, 0x7d, 0x07, 0x2e, 0x13, 0xab, 0x69, 0x19, - 0x4a, 0x5b, 0x66, 0xd2, 0x64, 0xc7, 0xd2, 0x5b, 0x2d, 0x6c, 0xf1, 0xe5, 0xc4, 0x3b, 0x11, 0xe5, - 0xac, 0x73, 0x8e, 0x0a, 0x65, 0x68, 0xb2, 0xfc, 0xd2, 0x45, 0x3d, 0x8a, 0x8c, 0x3e, 0x06, 0x20, - 0x53, 0x11, 0x5d, 0xa2, 0xb4, 0xb9, 0x3d, 0x1a, 0xb6, 0x46, 0xe9, 0x9a, 0x20, 0xc2, 0x40, 0x9e, - 0x6d, 0xb4, 0x4c, 0x70, 0xc8, 0xab, 0x9e, 0x6e, 0x61, 0xf9, 0x7e, 0x57, 0xa5, 0x0b, 0x07, 0xe9, - 0xd5, 0xfc, 0xd9, 0xe9, 0x22, 0x48, 0x8c, 0x7c, 0x7f, 0xbb, 0x42, 0x70, 0x09, 0xfb, 0xdd, 0x55, - 0xd1, 0x4b, 0xb8, 0x1b, 0x58, 0xff, 0x20, 0x33, 0x2f, 0xaf, 0x96, 0xe2, 0xc8, 0xfb, 0x7a, 0x6b, - 0x1f, 0x5b, 0xb2, 0xb7, 0x4c, 0x4d, 0xd7, 0x0b, 0xd3, 0xd2, 0x0d, 0x9f, 0xa1, 0xa2, 0x18, 0xac, - 0xf4, 0x2b, 0x4e, 0x8d, 0x66, 0xf6, 0x74, 0x46, 0x94, 0xdf, 0x35, 0x75, 0xdb, 0x34, 0x4a, 0x19, - 0xa6, 0x7c, 0xf6, 0x84, 0xee, 0x42, 0xd1, 0x39, 0x36, 0xe4, 0x7d, 0xac, 0x58, 0xce, 0x2e, 0x56, - 0x1c, 0x32, 0x4b, 0x03, 0xcd, 0x51, 0x70, 0x8e, 0x8d, 0x5a, 0x80, 0x8c, 0x9e, 0x43, 0x51, 0x37, - 0xe4, 0xbd, 0xb6, 0xde, 0xda, 0x77, 0xe4, 0x23, 0x4b, 0x77, 0xb0, 0x5d, 0x9a, 0xa7, 0x0a, 0x89, - 0xea, 0xb7, 0x0d, 0xbe, 0x6e, 0xac, 0xbd, 0x24, 0x39, 0xb9, 0x6a, 0xf2, 0xba, 0xf1, 0x98, 0xf2, - 0x53, 0xa2, 0xbd, 0x9e, 0x4a, 0xcf, 0x16, 0xd3, 0xe2, 0x7f, 0x11, 0x20, 0xef, 0x76, 0xb7, 0x38, - 0x47, 0xc6, 0x1d, 0x28, 0x9a, 0x06, 0x96, 0xbb, 0xfb, 0x8a, 0x8d, 0xb9, 0x1e, 0xf9, 0x84, 0x93, - 0x37, 0x0d, 0xbc, 0x4d, 0xc8, 0x4c, 0x5d, 0x68, 0x1b, 0xe6, 0x6d, 0x47, 0x69, 0xe9, 0x46, 0x2b, - 0xa0, 0xde, 0xe9, 0xc9, 0xc1, 0x42, 0x91, 0x73, 0x7b, 0xf4, 0x90, 0x97, 0xf2, 0x47, 0x02, 0xcc, - 0xaf, 0x68, 0x1d, 0xdd, 0x68, 0x74, 0xdb, 0x7a, 0xac, 0x6b, 0x10, 0x37, 0x20, 0x63, 0x13, 0x99, - 0xbe, 0xc1, 0xf7, 0x11, 0x65, 0x9a, 0xa6, 0x10, 0xcb, 0xff, 0x14, 0x0a, 0xf8, 0xb8, 0xab, 0xb3, - 0xad, 0x07, 0x06, 0x84, 0x52, 0x93, 0xd7, 0x2d, 0xef, 0xf3, 0x92, 0x24, 0x5e, 0xa7, 0x4f, 0x01, - 0x05, 0xab, 0x14, 0x27, 0x76, 0xf9, 0x14, 0x16, 0xa8, 0xe8, 0x1d, 0xc3, 0x8e, 0x59, 0x5f, 0xe2, - 0xcf, 0xc1, 0x85, 0xb0, 0xe8, 0x38, 0xcb, 0xfd, 0x92, 0xb7, 0xf2, 0x26, 0xb6, 0x62, 0x05, 0xb1, - 0x9e, 0xae, 0xb9, 0xe0, 0x38, 0xcb, 0xfc, 0x4b, 0x02, 0xbc, 0x41, 0x65, 0xd3, 0xdd, 0x99, 0x3d, - 0x6c, 0x3d, 0xc5, 0x8a, 0x1d, 0x2b, 0x02, 0xbf, 0x0e, 0x33, 0x0c, 0x49, 0xd3, 0xfe, 0x39, 0xbd, - 0x9a, 0x25, 0x9e, 0x4b, 0xc3, 0x31, 0x2d, 0xe2, 0xb9, 0xf0, 0x24, 0x51, 0x81, 0x72, 0x54, 0x29, - 0xe2, 0xac, 0xe9, 0xdf, 0x15, 0x60, 0x9e, 0x3b, 0x8d, 0xa4, 0x2b, 0x57, 0xf6, 0x89, 0xcf, 0x84, - 0xaa, 0x90, 0x55, 0xe9, 0x2f, 0xd9, 0x39, 0xe9, 0x62, 0x2a, 0x3f, 0x3f, 0xca, 0xdf, 0x64, 0x6c, - 0xcd, 0x93, 0x2e, 0x26, 0x4e, 0xab, 0xfb, 0x9b, 0x28, 0x2a, 0x50, 0xc9, 0x91, 0x1e, 0x2b, 0x1d, - 0x47, 0x34, 0xaf, 0xeb, 0xfa, 0x71, 0x1d, 0xfc, 0xd3, 0x24, 0x57, 0x02, 0x7b, 0x07, 0xcf, 0x1e, - 0xab, 0x8f, 0xf2, 0x59, 0x68, 0x73, 0x2c, 0x58, 0xf1, 0xc4, 0x39, 0x2a, 0x1e, 0x58, 0xa1, 0xf7, - 0xa9, 0xe8, 0x53, 0x08, 0xac, 0xc1, 0xcb, 0xac, 0x4e, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, 0xe6, 0x7d, + 0xc0, 0x08, 0x82, 0x00, 0x06, 0xf6, 0x21, 0x08, 0xea, 0xa7, 0xff, 0xc8, 0xe6, 0x8f, 0x66, 0xfb, + 0x26, 0x0b, 0xf8, 0x45, 0x62, 0x9f, 0xaa, 0x73, 0xba, 0xea, 0x54, 0xd5, 0xa9, 0xf3, 0x55, 0x9f, + 0xaa, 0x82, 0x79, 0xcb, 0x54, 0xd4, 0xfd, 0xee, 0xee, 0xb2, 0xd2, 0xd5, 0x97, 0xba, 0x96, 0xe9, + 0x98, 0x68, 0x5e, 0x35, 0xd5, 0x03, 0x4a, 0x5e, 0xe2, 0x89, 0xe5, 0xfb, 0x07, 0x87, 0xcb, 0x07, + 0x87, 0x36, 0xb6, 0x0e, 0xb1, 0xb5, 0xac, 0x9a, 0x86, 0xda, 0xb3, 0x2c, 0x6c, 0xa8, 0x27, 0xcb, + 0x6d, 0x53, 0x3d, 0xa0, 0x7f, 0x74, 0xa3, 0xc5, 0xd8, 0xcb, 0xc8, 0x95, 0xa8, 0x29, 0x8e, 0xc2, + 0x69, 0x17, 0x5c, 0x1a, 0xb6, 0x2c, 0xd3, 0xb2, 0x39, 0xf5, 0x92, 0x4b, 0xed, 0x60, 0x47, 0x09, + 0xe4, 0x7e, 0xcb, 0x76, 0x4c, 0x4b, 0x69, 0xe1, 0x65, 0x6c, 0xb4, 0x74, 0x03, 0x93, 0x0c, 0x87, + 0xaa, 0xca, 0x13, 0xdf, 0x8e, 0x4c, 0x7c, 0xc4, 0x53, 0x4b, 0x3d, 0x47, 0x6f, 0x2f, 0xef, 0xb7, + 0xd5, 0x65, 0x47, 0xef, 0x60, 0xdb, 0x51, 0x3a, 0x5d, 0x9e, 0x72, 0x9f, 0xa6, 0x38, 0x96, 0xa2, + 0xea, 0x46, 0xcb, 0xfd, 0xdf, 0xdd, 0x5d, 0xb6, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, 0xab, + 0x18, 0x6e, 0x71, 0x5b, 0x66, 0xcb, 0xa4, 0x3f, 0x97, 0xc9, 0x2f, 0x4e, 0xbd, 0xda, 0x32, 0xcd, + 0x56, 0x1b, 0x2f, 0xd3, 0xa7, 0xdd, 0xde, 0xde, 0xb2, 0xd6, 0xb3, 0x14, 0x47, 0x37, 0x39, 0x97, + 0xf8, 0xcf, 0x04, 0xc8, 0x49, 0xf8, 0x75, 0x0f, 0xdb, 0x4e, 0x0d, 0x2b, 0x1a, 0xb6, 0xd0, 0x15, + 0x48, 0x1e, 0xe0, 0x93, 0x52, 0xf2, 0x9a, 0x70, 0x77, 0x6e, 0x75, 0xf6, 0xcb, 0xd3, 0xc5, 0xe4, + 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x5d, 0x83, 0x59, 0x6c, 0x68, 0x32, 0x49, 0x4e, 0x85, 0x93, 0x67, + 0xb0, 0xa1, 0x6d, 0xe0, 0x13, 0xf4, 0x2d, 0x48, 0xdb, 0x44, 0x9a, 0xa1, 0xe2, 0xd2, 0xf4, 0x35, + 0xe1, 0xee, 0xf4, 0xea, 0xcf, 0x7d, 0x79, 0xba, 0xf8, 0x49, 0x4b, 0x77, 0xf6, 0x7b, 0xbb, 0x4b, + 0xaa, 0xd9, 0x59, 0xf6, 0xda, 0x49, 0xdb, 0xf5, 0x7f, 0x2f, 0x77, 0x0f, 0x5a, 0xcb, 0xfd, 0x3a, + 0x5a, 0x6a, 0x1e, 0x1b, 0x0d, 0xfc, 0x5a, 0xf2, 0x24, 0xae, 0xa7, 0xd2, 0x42, 0x31, 0xb1, 0x9e, + 0x4a, 0x27, 0x8a, 0x49, 0xf1, 0xb7, 0x93, 0x90, 0x97, 0xb0, 0xdd, 0x35, 0x0d, 0x1b, 0xf3, 0x92, + 0xbf, 0x0f, 0x49, 0xe7, 0xd8, 0xa0, 0x25, 0xcf, 0x3e, 0xbc, 0xba, 0x34, 0xd0, 0x23, 0x96, 0x9a, + 0x96, 0x62, 0xd8, 0x8a, 0x4a, 0xaa, 0x2f, 0x91, 0xac, 0xe8, 0x23, 0xc8, 0x5a, 0xd8, 0xee, 0x75, + 0x30, 0x55, 0x24, 0xad, 0x54, 0xf6, 0xe1, 0xe5, 0x08, 0xce, 0x46, 0x57, 0x31, 0x24, 0x60, 0x79, + 0xc9, 0x6f, 0x74, 0x05, 0xd2, 0x46, 0xaf, 0x43, 0x54, 0x61, 0xd3, 0x8a, 0x26, 0xa5, 0x59, 0xa3, + 0xd7, 0xd9, 0xc0, 0x27, 0x36, 0xfa, 0x0b, 0x70, 0x49, 0xc3, 0x5d, 0x0b, 0xab, 0x8a, 0x83, 0x35, + 0xd9, 0x52, 0x8c, 0x16, 0x96, 0x75, 0x63, 0xcf, 0xb4, 0x4b, 0x33, 0xd7, 0x92, 0x77, 0xb3, 0x0f, + 0xdf, 0x8e, 0x90, 0x2f, 0x91, 0x5c, 0x75, 0x63, 0xcf, 0x5c, 0x4d, 0xfd, 0xf0, 0x74, 0x71, 0x4a, + 0xba, 0xe0, 0x4b, 0xf0, 0x92, 0x6c, 0xd4, 0x80, 0x1c, 0x2f, 0xae, 0x85, 0x15, 0xdb, 0x34, 0x4a, + 0xb3, 0xd7, 0x84, 0xbb, 0xf9, 0x87, 0x4b, 0x51, 0x02, 0x43, 0xaa, 0x21, 0x8f, 0xbd, 0x0e, 0x96, + 0x28, 0x97, 0x34, 0x67, 0x05, 0x9e, 0xd0, 0x5b, 0x90, 0x21, 0x35, 0xd9, 0x3d, 0x71, 0xb0, 0x5d, + 0x4a, 0xd3, 0xaa, 0x90, 0xaa, 0xad, 0x92, 0x67, 0xf1, 0x53, 0x98, 0x0b, 0xb2, 0x22, 0x04, 0x79, + 0xa9, 0xda, 0xd8, 0xd9, 0xac, 0xca, 0x3b, 0x5b, 0x1b, 0x5b, 0xcf, 0x5f, 0x6d, 0x15, 0xa7, 0xd0, + 0x05, 0x28, 0x72, 0xda, 0x46, 0xf5, 0x33, 0xf9, 0x59, 0x7d, 0xb3, 0xde, 0x2c, 0x0a, 0xe5, 0xd4, + 0x77, 0x7f, 0xfb, 0xea, 0x94, 0xb8, 0x0d, 0xf0, 0x14, 0x3b, 0xbc, 0x83, 0xa1, 0x55, 0x98, 0xd9, + 0xa7, 0xe5, 0x29, 0x09, 0x54, 0xd3, 0xd7, 0x22, 0x0b, 0x1e, 0xe8, 0x8c, 0xab, 0x69, 0xa2, 0x8d, + 0x1f, 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0x7f, 0x4f, 0x80, 0x2c, 0x15, 0xc9, 0xea, 0x87, 0x2a, + 0x7d, 0x32, 0xaf, 0x8f, 0x55, 0xc6, 0xa0, 0x50, 0xb4, 0x04, 0xd3, 0x87, 0x4a, 0xbb, 0x87, 0x4b, + 0x09, 0x2a, 0xa3, 0x14, 0x21, 0xe3, 0x25, 0x49, 0x97, 0x58, 0x36, 0xf4, 0x18, 0xe6, 0x74, 0xc3, + 0xc1, 0x86, 0x23, 0x33, 0xb6, 0xe4, 0x18, 0xb6, 0x2c, 0xcb, 0x4d, 0x1f, 0xc4, 0x7f, 0x2a, 0x00, + 0x6c, 0xf7, 0xe2, 0x54, 0x0a, 0xfa, 0xc6, 0x84, 0xe5, 0xe7, 0xbd, 0x8b, 0xd7, 0xe2, 0x12, 0xcc, + 0xe8, 0x46, 0x5b, 0x37, 0x58, 0xf9, 0xd3, 0x12, 0x7f, 0x42, 0x17, 0x60, 0x7a, 0xb7, 0xad, 0x1b, + 0x1a, 0x1d, 0x0f, 0x69, 0x89, 0x3d, 0x88, 0x12, 0x64, 0x69, 0xa9, 0x63, 0xd4, 0xbb, 0x78, 0x9a, + 0x80, 0x8b, 0x15, 0xd3, 0xd0, 0x74, 0x32, 0x24, 0x95, 0xf6, 0xd7, 0x42, 0x2b, 0xeb, 0x10, 0x18, + 0x7c, 0x32, 0x3e, 0xee, 0x4e, 0xd8, 0xc6, 0xc8, 0xe7, 0xaa, 0x1e, 0x77, 0x29, 0x2d, 0x5a, 0x93, + 0xe8, 0x1b, 0x70, 0x59, 0x69, 0xb7, 0xcd, 0x23, 0x59, 0xdf, 0x93, 0x35, 0x13, 0xdb, 0xb2, 0x61, + 0x3a, 0x32, 0x3e, 0xd6, 0x6d, 0x87, 0x9a, 0x92, 0xb4, 0xb4, 0x40, 0x93, 0xeb, 0x7b, 0x6b, 0x26, + 0xb6, 0xb7, 0x4c, 0xa7, 0x4a, 0x92, 0xc8, 0x38, 0x25, 0x85, 0x61, 0xe3, 0x74, 0x86, 0x98, 0x5f, + 0x29, 0x8d, 0x8f, 0xbb, 0x74, 0x9c, 0x06, 0x9a, 0x72, 0x36, 0xd8, 0x94, 0xe2, 0x2f, 0xc0, 0xa5, + 0x7e, 0xfd, 0xc6, 0xd9, 0x7e, 0x7f, 0x20, 0x40, 0xbe, 0x6e, 0xe8, 0xce, 0xd7, 0xa2, 0xe1, 0x3c, + 0x65, 0x27, 0x83, 0xca, 0xbe, 0x0f, 0xc5, 0x3d, 0x45, 0x6f, 0x3f, 0x37, 0x9a, 0x66, 0x67, 0xd7, + 0x76, 0x4c, 0x03, 0xdb, 0xbc, 0x35, 0x06, 0xe8, 0xe2, 0x4b, 0x28, 0x78, 0xb5, 0x89, 0x53, 0x4d, + 0x0e, 0x14, 0xeb, 0x86, 0x6a, 0xe1, 0x0e, 0x36, 0x62, 0xd5, 0xd3, 0xdb, 0x90, 0xd1, 0x5d, 0xb9, + 0x54, 0x57, 0x49, 0xc9, 0x27, 0x88, 0x3d, 0x98, 0x0f, 0xbc, 0x35, 0x4e, 0x73, 0x49, 0xa6, 0x0c, + 0x7c, 0x24, 0xfb, 0x6d, 0x44, 0xa6, 0x0c, 0x7c, 0xc4, 0xcc, 0x5b, 0x03, 0x72, 0x6b, 0xb8, 0x8d, + 0x1d, 0x1c, 0xa7, 0xd5, 0xdf, 0x81, 0xbc, 0x2b, 0x34, 0xce, 0x86, 0xf9, 0x1b, 0x02, 0x20, 0x2e, + 0x97, 0xcc, 0xb2, 0x71, 0xb6, 0xcd, 0x22, 0x71, 0x2d, 0x9c, 0x9e, 0x65, 0x30, 0x1f, 0x81, 0xf5, + 0x49, 0x60, 0x24, 0xea, 0x26, 0xf8, 0x43, 0x36, 0x15, 0x1c, 0xb2, 0xdc, 0xbd, 0x39, 0x82, 0x85, + 0x50, 0xc1, 0xe2, 0x6d, 0xbe, 0x14, 0x2d, 0x53, 0xe2, 0x5a, 0x32, 0xe8, 0xc3, 0x51, 0xa2, 0xf8, + 0x7d, 0x01, 0xe6, 0x2b, 0x6d, 0xac, 0x58, 0xb1, 0x6b, 0xe4, 0x9b, 0x90, 0xd6, 0xb0, 0xa2, 0xd1, + 0x2a, 0xb3, 0x81, 0xfd, 0x4e, 0x40, 0x0a, 0xf1, 0x74, 0x97, 0xf6, 0xdb, 0xea, 0x52, 0xd3, 0xf5, + 0x81, 0xf9, 0xe8, 0xf6, 0x98, 0xc4, 0xcf, 0x00, 0x05, 0x4b, 0x16, 0x67, 0x47, 0xf8, 0xdf, 0x02, + 0x20, 0x09, 0x1f, 0x62, 0xcb, 0x89, 0xbd, 0xda, 0x6b, 0x90, 0x75, 0x14, 0xab, 0x85, 0x1d, 0x99, + 0x78, 0xf7, 0xe7, 0xa9, 0x39, 0x30, 0x3e, 0x42, 0x46, 0x4d, 0xb8, 0x83, 0x0d, 0x65, 0xb7, 0x8d, + 0xa9, 0x14, 0x79, 0xd7, 0xec, 0x19, 0x9a, 0xac, 0x3b, 0xd8, 0x52, 0x1c, 0xd3, 0x92, 0xcd, 0xae, + 0xa3, 0x77, 0xf4, 0x2f, 0xa8, 0x63, 0xcf, 0xbb, 0xda, 0x0d, 0x96, 0x9d, 0x30, 0xaf, 0x92, 0xcc, + 0x75, 0x9e, 0xf7, 0x79, 0x20, 0xab, 0xf8, 0x39, 0x2c, 0x84, 0x6a, 0x1d, 0xa7, 0x4a, 0xff, 0xa7, + 0x00, 0xd9, 0x86, 0xaa, 0x18, 0x71, 0xea, 0xf2, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, 0x7b, 0xa6, + 0xd5, 0x51, 0x1c, 0x3a, 0x70, 0xf2, 0x21, 0x5d, 0x7a, 0xfe, 0xba, 0xaa, 0x18, 0x4f, 0x68, 0x26, + 0x09, 0x6c, 0xef, 0x37, 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0xec, 0x47, 0x67, 0xdb, 0xfc, + 0xc3, 0xf7, 0x03, 0xfc, 0x07, 0x87, 0x4b, 0x2e, 0x64, 0x5c, 0x0a, 0x40, 0xc6, 0x25, 0xc2, 0xb1, + 0xd4, 0x70, 0x2c, 0x6c, 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, 0x0d, 0xd7, + 0xf5, 0x54, 0x3a, 0x59, 0x4c, 0x89, 0x7f, 0x26, 0xc0, 0x1c, 0xab, 0x72, 0x9c, 0xc3, 0xf5, 0x03, + 0x48, 0x59, 0xe6, 0x11, 0x1b, 0xae, 0xd9, 0x87, 0x6f, 0x45, 0x88, 0xd8, 0xc0, 0x27, 0xc1, 0xf9, + 0x90, 0x66, 0x47, 0xab, 0xc0, 0xbd, 0x4e, 0x99, 0x72, 0x27, 0x27, 0xe5, 0x06, 0xc6, 0x25, 0x11, + 0x19, 0x77, 0xa0, 0xb0, 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x42, 0x92, 0xb9, 0x33, 0x79, 0x77, + 0x4e, 0xca, 0x53, 0xb2, 0x5b, 0x74, 0x9b, 0xd4, 0x9c, 0x8d, 0x1f, 0x1b, 0xff, 0x39, 0x6b, 0xf3, + 0xff, 0x23, 0xf0, 0x31, 0xe4, 0xd6, 0xfc, 0xcf, 0x5b, 0xd3, 0xff, 0x7a, 0x02, 0x2e, 0x57, 0xf6, + 0xb1, 0x7a, 0x50, 0x31, 0x0d, 0x5b, 0xb7, 0x1d, 0xa2, 0xbb, 0x38, 0xdb, 0xff, 0x2d, 0xc8, 0x1c, + 0xe9, 0xce, 0xbe, 0xac, 0xe9, 0x7b, 0x7b, 0xd4, 0x7a, 0xa6, 0xa5, 0x34, 0x21, 0xac, 0xe9, 0x7b, + 0x7b, 0xe8, 0x11, 0xa4, 0x3a, 0xa6, 0xc6, 0x9c, 0xf3, 0xfc, 0xc3, 0xc5, 0x08, 0xf1, 0xb4, 0x68, + 0x76, 0xaf, 0xb3, 0x69, 0x6a, 0x58, 0xa2, 0x99, 0xd1, 0x55, 0x00, 0x95, 0x50, 0xbb, 0xa6, 0x6e, + 0x38, 0x7c, 0xf6, 0x0d, 0x50, 0x50, 0x0d, 0x32, 0x0e, 0xb6, 0x3a, 0xba, 0xa1, 0x38, 0xb8, 0x34, + 0x4d, 0x95, 0x77, 0x33, 0xb2, 0xe0, 0xdd, 0xb6, 0xae, 0x2a, 0x6b, 0xd8, 0x56, 0x2d, 0xbd, 0xeb, + 0x98, 0x16, 0xd7, 0xa2, 0xcf, 0x2c, 0xfe, 0x6a, 0x0a, 0x4a, 0x83, 0xba, 0x89, 0xb3, 0x87, 0x6c, + 0xc3, 0x0c, 0x41, 0xf3, 0x6d, 0x87, 0xf7, 0x91, 0x87, 0xc3, 0x54, 0x10, 0x51, 0x02, 0xba, 0x2a, + 0xd0, 0x76, 0x78, 0xb1, 0xb9, 0x9c, 0xf2, 0xbf, 0x12, 0x60, 0x86, 0x25, 0xa0, 0x07, 0x90, 0xe6, + 0xcb, 0x17, 0x1a, 0x2d, 0x63, 0x72, 0xf5, 0xd2, 0xd9, 0xe9, 0xe2, 0x2c, 0x5b, 0x91, 0x58, 0xfb, + 0xd2, 0xff, 0x29, 0xcd, 0xd2, 0x7c, 0x75, 0x8d, 0xb4, 0x96, 0xed, 0x28, 0x96, 0x43, 0x17, 0x89, + 0x12, 0x0c, 0xa5, 0x50, 0xc2, 0x06, 0x3e, 0x41, 0xeb, 0x30, 0x63, 0x3b, 0x8a, 0xd3, 0xb3, 0x79, + 0x7b, 0x9d, 0xab, 0xb0, 0x0d, 0xca, 0x29, 0x71, 0x09, 0xc4, 0x7d, 0xd2, 0xb0, 0xa3, 0xe8, 0x6d, + 0xda, 0x80, 0x19, 0x89, 0x3f, 0x89, 0xbf, 0x21, 0xc0, 0x0c, 0xcb, 0x8a, 0x2e, 0xc3, 0x82, 0xb4, + 0xb2, 0xf5, 0xb4, 0x2a, 0xd7, 0xb7, 0xd6, 0xaa, 0xcd, 0xaa, 0xb4, 0x59, 0xdf, 0x5a, 0x69, 0x56, + 0x8b, 0x53, 0xe8, 0x12, 0x20, 0x37, 0xa1, 0xf2, 0x7c, 0xab, 0x51, 0x6f, 0x34, 0xab, 0x5b, 0xcd, + 0xa2, 0x40, 0x57, 0x32, 0x28, 0x3d, 0x40, 0x4d, 0xa0, 0x9b, 0x70, 0xad, 0x9f, 0x2a, 0x37, 0x9a, + 0x2b, 0xcd, 0x86, 0x5c, 0x6d, 0x34, 0xeb, 0x9b, 0x2b, 0xcd, 0xea, 0x5a, 0x31, 0x39, 0x22, 0x17, + 0x79, 0x89, 0x24, 0x55, 0x2b, 0xcd, 0x62, 0x4a, 0x74, 0xe0, 0xa2, 0x84, 0x55, 0xb3, 0xd3, 0xed, + 0x39, 0x98, 0x94, 0xd2, 0x8e, 0x73, 0xa4, 0x5c, 0x86, 0x59, 0xcd, 0x3a, 0x91, 0xad, 0x9e, 0xc1, + 0xc7, 0xc9, 0x8c, 0x66, 0x9d, 0x48, 0x3d, 0x43, 0xfc, 0xc7, 0x02, 0x5c, 0xea, 0x7f, 0x6d, 0x9c, + 0x9d, 0xf0, 0x05, 0x64, 0x15, 0x4d, 0xc3, 0x9a, 0xac, 0xe1, 0xb6, 0xa3, 0x70, 0x17, 0xe7, 0x7e, + 0x40, 0x12, 0x5f, 0xda, 0x5b, 0xf2, 0x96, 0xf6, 0x36, 0x5f, 0x56, 0x2a, 0xb4, 0x20, 0x6b, 0x84, + 0xc3, 0x35, 0x3f, 0x54, 0x08, 0xa5, 0x88, 0xff, 0x23, 0x05, 0xb9, 0xaa, 0xa1, 0x35, 0x8f, 0x63, + 0x9d, 0x4b, 0x2e, 0xc1, 0x8c, 0x6a, 0x76, 0x3a, 0xba, 0xe3, 0x2a, 0x88, 0x3d, 0xa1, 0x9f, 0x09, + 0xb8, 0xa6, 0xc9, 0x09, 0x1c, 0x34, 0xdf, 0x29, 0x45, 0xdf, 0x86, 0xcb, 0xc4, 0x6a, 0x5a, 0x86, + 0xd2, 0x96, 0x99, 0x34, 0xd9, 0xb1, 0xf4, 0x56, 0x0b, 0x5b, 0x7c, 0x39, 0xf1, 0x6e, 0x44, 0x39, + 0xeb, 0x9c, 0xa3, 0x42, 0x19, 0x9a, 0x2c, 0xbf, 0x74, 0x51, 0x8f, 0x22, 0xa3, 0x4f, 0x00, 0xc8, + 0x54, 0x44, 0x97, 0x28, 0x6d, 0x6e, 0x8f, 0x86, 0xad, 0x51, 0xba, 0x26, 0x88, 0x30, 0x90, 0x67, + 0x1b, 0x2d, 0x13, 0x1c, 0xf2, 0xba, 0xa7, 0x5b, 0x58, 0x7e, 0xd0, 0x55, 0xe9, 0xc2, 0x41, 0x7a, + 0x35, 0x7f, 0x76, 0xba, 0x08, 0x12, 0x23, 0x3f, 0xd8, 0xae, 0x10, 0x5c, 0xc2, 0x7e, 0x77, 0x55, + 0xf4, 0x0a, 0xee, 0x05, 0xd6, 0x3f, 0xc8, 0xcc, 0xcb, 0xab, 0xa5, 0x38, 0xf2, 0xbe, 0xde, 0xda, + 0xc7, 0x96, 0xec, 0x2d, 0x53, 0xd3, 0xf5, 0xc2, 0xb4, 0x74, 0xd3, 0x67, 0xa8, 0x28, 0x06, 0x2b, + 0xfd, 0x8a, 0x53, 0xa3, 0x99, 0x3d, 0x9d, 0x11, 0xe5, 0x77, 0x4d, 0xdd, 0x36, 0x8d, 0x52, 0x86, + 0x29, 0x9f, 0x3d, 0xa1, 0x7b, 0x50, 0x74, 0x8e, 0x0d, 0x79, 0x1f, 0x2b, 0x96, 0xb3, 0x8b, 0x15, + 0x87, 0xcc, 0xd2, 0x40, 0x73, 0x14, 0x9c, 0x63, 0xa3, 0x16, 0x20, 0xa3, 0x17, 0x50, 0xd4, 0x0d, + 0x79, 0xaf, 0xad, 0xb7, 0xf6, 0x1d, 0xf9, 0xc8, 0xd2, 0x1d, 0x6c, 0x97, 0xe6, 0xa9, 0x42, 0xa2, + 0xfa, 0x6d, 0x83, 0xaf, 0x1b, 0x6b, 0xaf, 0x48, 0x4e, 0xae, 0x9a, 0xbc, 0x6e, 0x3c, 0xa1, 0xfc, + 0x94, 0x68, 0xaf, 0xa7, 0xd2, 0xb3, 0xc5, 0xb4, 0xf8, 0x9f, 0x05, 0xc8, 0xbb, 0xdd, 0x2d, 0xce, + 0x91, 0x71, 0x17, 0x8a, 0xa6, 0x81, 0xe5, 0xee, 0xbe, 0x62, 0x63, 0xae, 0x47, 0x3e, 0xe1, 0xe4, + 0x4d, 0x03, 0x6f, 0x13, 0x32, 0x53, 0x17, 0xda, 0x86, 0x79, 0xdb, 0x51, 0x5a, 0xba, 0xd1, 0x0a, + 0xa8, 0x77, 0x7a, 0x72, 0xb0, 0x50, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa1, 0x00, 0xf3, + 0x2b, 0x5a, 0x47, 0x37, 0x1a, 0xdd, 0xb6, 0x1e, 0xeb, 0x1a, 0xc4, 0x4d, 0xc8, 0xd8, 0x44, 0xa6, + 0x6f, 0xf0, 0x7d, 0x44, 0x99, 0xa6, 0x29, 0xc4, 0xf2, 0x3f, 0x83, 0x02, 0x3e, 0xee, 0xea, 0xec, + 0xd3, 0x03, 0x03, 0x42, 0xa9, 0xc9, 0xeb, 0x96, 0xf7, 0x79, 0x49, 0x12, 0xaf, 0xd3, 0x67, 0x80, + 0x82, 0x55, 0x8a, 0x13, 0xbb, 0x7c, 0x06, 0x0b, 0x54, 0xf4, 0x8e, 0x61, 0xc7, 0xac, 0x2f, 0xf1, + 0xe7, 0xe1, 0x42, 0x58, 0x74, 0x9c, 0xe5, 0x7e, 0xc5, 0x5b, 0x79, 0x13, 0x5b, 0xb1, 0x82, 0x58, + 0x4f, 0xd7, 0x5c, 0x70, 0x9c, 0x65, 0xfe, 0x65, 0x01, 0xae, 0x50, 0xd9, 0xf4, 0xeb, 0xcc, 0x1e, + 0xb6, 0x9e, 0x61, 0xc5, 0x8e, 0x15, 0x81, 0xdf, 0x80, 0x19, 0x86, 0xa4, 0x69, 0xff, 0x9c, 0x5e, + 0xcd, 0x12, 0xcf, 0xa5, 0xe1, 0x98, 0x16, 0xf1, 0x5c, 0x78, 0x92, 0xa8, 0x40, 0x39, 0xaa, 0x14, + 0x71, 0xd6, 0xf4, 0x6f, 0x0b, 0x30, 0xcf, 0x9d, 0x46, 0xd2, 0x95, 0x2b, 0xfb, 0xc4, 0x67, 0x42, + 0x55, 0xc8, 0xaa, 0xf4, 0x97, 0xec, 0x9c, 0x74, 0x31, 0x95, 0x9f, 0x1f, 0xe5, 0x6f, 0x32, 0xb6, + 0xe6, 0x49, 0x17, 0x13, 0xa7, 0xd5, 0xfd, 0x4d, 0x14, 0x15, 0xa8, 0xe4, 0x48, 0x8f, 0x95, 0x8e, + 0x23, 0x9a, 0xd7, 0x75, 0xfd, 0xb8, 0x0e, 0xfe, 0x49, 0x92, 0x2b, 0x81, 0xbd, 0x83, 0x67, 0x8f, + 0xd5, 0x47, 0xf9, 0x3c, 0xf4, 0x71, 0x2c, 0x58, 0xf1, 0xc4, 0x39, 0x2a, 0x1e, 0x58, 0xa1, 0xf7, + 0xa9, 0xe8, 0x33, 0x08, 0xac, 0xc1, 0xcb, 0xac, 0x4e, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, 0xe6, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, - 0xdb, 0xc5, 0x1b, 0xc0, 0x03, 0xb3, 0xf8, 0xb8, 0x4b, 0x88, 0x68, 0x87, 0xcc, 0x5e, 0xae, 0xab, - 0x40, 0x8b, 0x6d, 0x8f, 0x87, 0x17, 0x7e, 0x4f, 0xe1, 0xe2, 0x0a, 0x9e, 0x97, 0xc0, 0x44, 0x88, - 0x3f, 0x10, 0xe0, 0xcd, 0xc8, 0x56, 0x8b, 0x73, 0x22, 0xfb, 0x18, 0x52, 0xb4, 0xf2, 0x89, 0x73, - 0x56, 0x9e, 0x72, 0x89, 0xbf, 0xe3, 0x8e, 0x71, 0x09, 0xb7, 0x4d, 0xa2, 0xd8, 0xaf, 0x60, 0x95, - 0x6d, 0xd6, 0x6d, 0xf0, 0xc4, 0xb9, 0x1b, 0xdc, 0x65, 0xf5, 0x8c, 0x40, 0x5f, 0x31, 0xe3, 0x34, - 0x02, 0xdf, 0x13, 0x60, 0xc1, 0xf3, 0x69, 0x62, 0x76, 0x6f, 0xdf, 0x87, 0xa4, 0x61, 0x1e, 0x9d, - 0x67, 0x89, 0x91, 0xe4, 0x27, 0x53, 0x52, 0xb8, 0x44, 0x71, 0xd6, 0xf7, 0xdf, 0x24, 0x20, 0xf3, - 0xa4, 0x12, 0x67, 0x2d, 0x3f, 0xe6, 0xcb, 0xd7, 0x6c, 0x60, 0x47, 0x75, 0x45, 0xef, 0x7d, 0x4b, - 0x4f, 0x2a, 0x1b, 0xf8, 0xc4, 0xed, 0x8a, 0x84, 0x0b, 0xad, 0x40, 0xc6, 0xd9, 0xb7, 0xb0, 0xbd, - 0x6f, 0xb6, 0xb5, 0xf3, 0xf8, 0x20, 0x3e, 0x57, 0x19, 0xc3, 0x34, 0x95, 0xeb, 0x86, 0x4a, 0x08, - 0x11, 0xa1, 0x12, 0xe4, 0x35, 0x9e, 0x1b, 0x97, 0x38, 0xcf, 0x6b, 0x02, 0xfe, 0xdb, 0x74, 0x71, - 0x46, 0x7c, 0x0e, 0x40, 0xaa, 0x13, 0x67, 0x93, 0xfc, 0x72, 0x12, 0xf2, 0xdb, 0x3d, 0x7b, 0x3f, - 0xe6, 0xde, 0x57, 0x01, 0xe8, 0xf6, 0x6c, 0x8a, 0x0f, 0x8e, 0x0d, 0x5e, 0xe7, 0x31, 0x51, 0x18, - 0x6e, 0xa5, 0x19, 0x5f, 0xf3, 0xd8, 0x40, 0x35, 0x2e, 0x04, 0xcb, 0x7e, 0x28, 0xc7, 0xf5, 0x51, - 0x48, 0xb2, 0x79, 0x6c, 0x6c, 0x62, 0x0f, 0x42, 0x32, 0x49, 0x98, 0x48, 0xfa, 0x18, 0x66, 0xc9, - 0x83, 0xec, 0x98, 0xe7, 0x69, 0xe6, 0x19, 0xc2, 0xd3, 0x34, 0xd1, 0x23, 0xc8, 0x30, 0x6e, 0x32, - 0x35, 0xcd, 0xd0, 0xa9, 0x29, 0xaa, 0x2e, 0x5c, 0x8d, 0x74, 0x52, 0x4a, 0x53, 0x56, 0x32, 0x11, - 0x5d, 0x80, 0xe9, 0x3d, 0xd3, 0x52, 0xdd, 0xcd, 0x58, 0xf6, 0xc0, 0xda, 0x73, 0x3d, 0x95, 0x4e, - 0x17, 0x33, 0xeb, 0xa9, 0x74, 0xa6, 0x08, 0xe2, 0x6f, 0x08, 0x50, 0xf0, 0x1a, 0x22, 0x4e, 0x6b, - 0x5d, 0x09, 0x69, 0xf1, 0xfc, 0x4d, 0x41, 0x14, 0x28, 0xfe, 0x5b, 0xea, 0xae, 0xa8, 0xe6, 0x21, - 0x6d, 0x99, 0x38, 0x7b, 0xca, 0x23, 0x16, 0xa8, 0x93, 0x38, 0x6f, 0xeb, 0xd2, 0x98, 0x9d, 0xfb, - 0x70, 0x41, 0xef, 0x10, 0x3b, 0xae, 0x3b, 0xed, 0x13, 0x8e, 0xa9, 0x1c, 0xec, 0xee, 0xfa, 0x2e, - 0xf8, 0x69, 0x15, 0x37, 0x49, 0xfc, 0xfb, 0x74, 0x75, 0xda, 0xaf, 0x49, 0x9c, 0xaa, 0xae, 0x43, - 0xce, 0x62, 0xa2, 0x89, 0xcf, 0x71, 0x4e, 0x6d, 0xcf, 0x79, 0xac, 0x44, 0xe1, 0xbf, 0x95, 0x80, - 0xc2, 0xf3, 0x1e, 0xb6, 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x05, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, - 0xd3, 0x92, 0x7b, 0x5d, 0x4d, 0x71, 0xdc, 0x68, 0x91, 0x1c, 0x21, 0x3f, 0x36, 0xad, 0x1d, 0x4a, - 0x44, 0x18, 0xd0, 0x81, 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x25, 0xe3, - 0xd5, 0x0f, 0xfe, 0xf3, 0xe9, 0xe2, 0xc3, 0x89, 0x62, 0xc0, 0x68, 0xbc, 0x5b, 0xaf, 0xa7, 0x6b, - 0x4b, 0x3b, 0x3b, 0xf5, 0x35, 0xa9, 0x48, 0x45, 0xbe, 0x64, 0x12, 0x9b, 0xc7, 0x86, 0x2d, 0xfe, - 0x83, 0x04, 0x14, 0x7d, 0x1d, 0xc5, 0xd9, 0x90, 0x55, 0xc8, 0xbe, 0xea, 0x61, 0x4b, 0x7f, 0x8d, - 0x66, 0x04, 0xce, 0x48, 0xcc, 0xce, 0x67, 0x30, 0x17, 0xd2, 0x40, 0xf2, 0xcb, 0x69, 0x20, 0x7b, - 0xe4, 0x57, 0x1e, 0xdd, 0x83, 0x79, 0xe7, 0xd8, 0x90, 0x59, 0xf4, 0x1f, 0x8b, 0x18, 0x71, 0x83, - 0x19, 0x0a, 0x0e, 0xd1, 0x07, 0xa1, 0xd3, 0x68, 0x11, 0x5b, 0xfc, 0x7d, 0x01, 0x10, 0x55, 0x54, - 0x9d, 0xad, 0xe9, 0x7f, 0x5d, 0xfa, 0xd3, 0x1d, 0x28, 0xd2, 0x78, 0x4a, 0x59, 0xdf, 0x93, 0x3b, - 0xba, 0x6d, 0xeb, 0x46, 0x8b, 0x77, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0x2b, - 0xb0, 0x10, 0xaa, 0x40, 0x9c, 0x8d, 0x7d, 0x0d, 0xe6, 0xf6, 0xd8, 0x16, 0x2a, 0x15, 0xce, 0x97, - 0x03, 0xb3, 0x94, 0xc6, 0xde, 0x27, 0xfe, 0x8f, 0x04, 0x5c, 0x90, 0xb0, 0x6d, 0xb6, 0x0f, 0x71, - 0xfc, 0x2a, 0xac, 0x01, 0xdf, 0x6b, 0x91, 0x5f, 0x4b, 0x93, 0x19, 0xc6, 0xcc, 0xa6, 0xb9, 0xf0, - 0x9a, 0xfa, 0x8d, 0xd1, 0x3d, 0x76, 0x70, 0x15, 0x9d, 0xaf, 0xc9, 0xa5, 0x42, 0x6b, 0x72, 0x26, - 0x14, 0xf4, 0x96, 0x61, 0x12, 0x9b, 0x66, 0xe3, 0x57, 0x46, 0xaf, 0xe3, 0x22, 0x95, 0xa5, 0x51, - 0x85, 0xac, 0x33, 0x96, 0x06, 0x7e, 0xb5, 0xd5, 0xeb, 0x50, 0xdf, 0x79, 0xf5, 0x12, 0x29, 0xef, - 0xd9, 0xe9, 0x62, 0x3e, 0x94, 0x66, 0x4b, 0x79, 0xdd, 0x7b, 0x26, 0xd2, 0xc5, 0x6f, 0xc3, 0xc5, - 0x3e, 0x65, 0xc7, 0xe9, 0xf1, 0xfc, 0xab, 0x24, 0xbc, 0x11, 0x16, 0x1f, 0x37, 0xfe, 0xf8, 0xba, - 0x37, 0x68, 0x0d, 0x72, 0x1d, 0xdd, 0x78, 0xbd, 0xa5, 0xc5, 0xb9, 0x8e, 0x6e, 0xf8, 0xcb, 0xb8, - 0x11, 0x5d, 0x63, 0xe6, 0x2b, 0xed, 0x1a, 0x0a, 0x94, 0xa3, 0xda, 0x2e, 0xce, 0xfe, 0xf1, 0x2b, - 0x02, 0xcc, 0xc5, 0xbd, 0x66, 0xf6, 0x7a, 0x51, 0x6c, 0x62, 0x13, 0x72, 0x5f, 0xc1, 0x22, 0xdb, - 0x6f, 0x09, 0x80, 0x9a, 0x56, 0xcf, 0x20, 0xa0, 0xf6, 0xa9, 0xd9, 0x8a, 0xb3, 0x9a, 0x17, 0x60, - 0x5a, 0x37, 0x34, 0x7c, 0x4c, 0xab, 0x99, 0x92, 0xd8, 0x43, 0x68, 0xeb, 0x30, 0x39, 0xd1, 0xd6, - 0xa1, 0xf8, 0x19, 0x2c, 0x84, 0x8a, 0x18, 0x67, 0xfd, 0xff, 0x7b, 0x02, 0x16, 0x78, 0x45, 0x62, - 0x5f, 0x5e, 0xfc, 0x26, 0x4c, 0xb7, 0x89, 0xcc, 0x11, 0xed, 0x4c, 0xdf, 0xe9, 0xb6, 0x33, 0xcd, - 0x8c, 0x7e, 0x06, 0xa0, 0x6b, 0xe1, 0x43, 0x99, 0xb1, 0x26, 0x27, 0x62, 0xcd, 0x10, 0x0e, 0x4a, - 0x40, 0xdf, 0x17, 0xa0, 0x40, 0x06, 0x74, 0xd7, 0x32, 0xbb, 0xa6, 0x4d, 0x7c, 0x16, 0x7b, 0x32, - 0x98, 0xf3, 0xfc, 0xec, 0x74, 0x31, 0xb7, 0xa9, 0x1b, 0xdb, 0x9c, 0xb1, 0xd9, 0x98, 0x38, 0x40, - 0xdf, 0x3d, 0xa6, 0xb0, 0x54, 0x69, 0x9b, 0xea, 0x81, 0xbf, 0x19, 0x46, 0x2c, 0x8b, 0x27, 0xce, - 0x16, 0xff, 0xa3, 0x00, 0x17, 0xbe, 0xb2, 0xb5, 0xdc, 0xbf, 0x08, 0x65, 0x8b, 0x2f, 0xa0, 0x48, - 0x7f, 0xd4, 0x8d, 0x3d, 0x33, 0xce, 0x55, 0xf5, 0x5f, 0x15, 0x60, 0x3e, 0x20, 0x38, 0x4e, 0x07, - 0xe7, 0xb5, 0xf4, 0x24, 0xfe, 0x1c, 0x71, 0x79, 0x82, 0xa3, 0x24, 0xce, 0x31, 0xf8, 0x2f, 0x12, - 0x70, 0xa9, 0xc2, 0xf6, 0xa0, 0xdd, 0xa0, 0x8c, 0x38, 0x7b, 0x46, 0x09, 0x66, 0x0f, 0xb1, 0x65, - 0xeb, 0x26, 0x9b, 0x7e, 0x73, 0x92, 0xfb, 0x88, 0xca, 0x90, 0xb6, 0x0d, 0xa5, 0x6b, 0xef, 0x9b, - 0xee, 0xe6, 0x9b, 0xf7, 0xec, 0x05, 0x90, 0x4c, 0xbf, 0x7e, 0x00, 0xc9, 0xcc, 0xe8, 0x00, 0x92, - 0xd9, 0x2f, 0x11, 0x40, 0xc2, 0x77, 0xba, 0xfe, 0x9d, 0x00, 0x97, 0x07, 0x34, 0x17, 0x67, 0x6f, - 0xf9, 0x2e, 0x64, 0x55, 0x2e, 0x98, 0x18, 0x6c, 0xb6, 0x8d, 0x57, 0x27, 0xd9, 0x5e, 0x13, 0xb7, - 0x9c, 0x9d, 0x2e, 0x82, 0x5b, 0xd4, 0xfa, 0x1a, 0x57, 0x0e, 0xf9, 0xad, 0x89, 0xbf, 0x98, 0x83, - 0x42, 0xf5, 0x98, 0x2d, 0x61, 0x37, 0x98, 0x9b, 0x80, 0x1e, 0x43, 0xba, 0x6b, 0x99, 0x87, 0xba, - 0x5b, 0x8d, 0x7c, 0x28, 0x7a, 0xc0, 0xad, 0x46, 0x1f, 0xd7, 0x36, 0xe7, 0x90, 0x3c, 0x5e, 0xd4, - 0x84, 0xcc, 0x53, 0x53, 0x55, 0xda, 0x8f, 0xf5, 0xb6, 0xdb, 0xf3, 0xdf, 0x1b, 0x2f, 0x68, 0xc9, - 0xe3, 0xd9, 0x56, 0x9c, 0x7d, 0xb7, 0x11, 0x3c, 0x22, 0xaa, 0x43, 0xba, 0xe6, 0x38, 0x5d, 0x92, - 0xc8, 0x6d, 0xc7, 0xed, 0x09, 0x84, 0x12, 0x16, 0x37, 0x84, 0xd5, 0x65, 0x47, 0x4d, 0x98, 0x7f, - 0x42, 0x0f, 0x64, 0x55, 0xda, 0x66, 0x4f, 0xab, 0x98, 0xc6, 0x9e, 0xde, 0xe2, 0x76, 0xfb, 0xd6, - 0x04, 0x32, 0x9f, 0x54, 0x1a, 0xd2, 0xa0, 0x00, 0xb4, 0x02, 0xe9, 0xc6, 0x43, 0x2e, 0x8c, 0xf9, - 0x75, 0x37, 0x27, 0x10, 0xd6, 0x78, 0x28, 0x79, 0x6c, 0x68, 0x1d, 0xb2, 0x2b, 0x9f, 0xf7, 0x2c, - 0xcc, 0xa5, 0xcc, 0x0c, 0x0d, 0x5d, 0xe8, 0x97, 0x42, 0xb9, 0xa4, 0x20, 0x33, 0x6a, 0x40, 0xfe, - 0xa5, 0x69, 0x1d, 0xb4, 0x4d, 0xc5, 0xad, 0xe1, 0x2c, 0x15, 0xf7, 0x8d, 0x09, 0xc4, 0xb9, 0x8c, - 0x52, 0x9f, 0x08, 0xf4, 0x6d, 0x28, 0x90, 0xc6, 0x68, 0x2a, 0xbb, 0x6d, 0xb7, 0x90, 0x69, 0x2a, - 0xf5, 0x9d, 0x09, 0xa4, 0x7a, 0x9c, 0xee, 0x1e, 0x4a, 0x9f, 0xa8, 0xb2, 0x04, 0xb9, 0x50, 0x27, - 0x40, 0x08, 0x52, 0x5d, 0xd2, 0xde, 0x02, 0x0d, 0x2e, 0xa2, 0xbf, 0xd1, 0xbb, 0x30, 0x6b, 0x98, - 0x1a, 0x76, 0x47, 0x48, 0x6e, 0xf5, 0xc2, 0xd9, 0xe9, 0xe2, 0xcc, 0x96, 0xa9, 0x31, 0x8f, 0x86, - 0xff, 0x92, 0x66, 0x48, 0xa6, 0xba, 0x56, 0xbe, 0x0a, 0x29, 0xd2, 0xee, 0xc4, 0x30, 0xed, 0x2a, - 0x36, 0xde, 0xb1, 0x74, 0x2e, 0xcd, 0x7d, 0x2c, 0xff, 0xe3, 0x04, 0x24, 0x1a, 0x0f, 0x89, 0xcf, - 0xbe, 0xdb, 0x53, 0x0f, 0xb0, 0xc3, 0xd3, 0xf9, 0x13, 0xf5, 0xe5, 0x2d, 0xbc, 0xa7, 0x33, 0xd7, - 0x2a, 0x23, 0xf1, 0x27, 0xf4, 0x36, 0x80, 0xa2, 0xaa, 0xd8, 0xb6, 0x65, 0xf7, 0xa0, 0x5e, 0x46, - 0xca, 0x30, 0xca, 0x06, 0x3e, 0x21, 0x6c, 0x36, 0x56, 0x2d, 0xec, 0xb8, 0x91, 0x51, 0xec, 0x89, - 0xb0, 0x39, 0xb8, 0xd3, 0x95, 0x1d, 0xf3, 0x00, 0x1b, 0xb4, 0x9f, 0x64, 0x88, 0xa9, 0xe9, 0x74, - 0x9b, 0x84, 0x40, 0xac, 0x24, 0x36, 0x34, 0xdf, 0xa4, 0x65, 0x24, 0xef, 0x99, 0x88, 0xb4, 0x70, - 0x4b, 0xe7, 0x27, 0xce, 0x32, 0x12, 0x7f, 0x22, 0x5a, 0x52, 0x7a, 0xce, 0x3e, 0x6d, 0x89, 0x8c, - 0x44, 0x7f, 0xa3, 0x5b, 0x50, 0x60, 0xc1, 0x94, 0x32, 0x36, 0x54, 0x99, 0x1a, 0xd7, 0x0c, 0x4d, - 0xce, 0x31, 0x72, 0xd5, 0x50, 0x89, 0x29, 0x45, 0x0f, 0x81, 0x13, 0xe4, 0x83, 0x8e, 0x4d, 0x74, - 0x0a, 0x24, 0xd7, 0x6a, 0xe1, 0xec, 0x74, 0x31, 0xdb, 0xa0, 0x09, 0x1b, 0x9b, 0x8d, 0xfa, 0x9a, - 0x94, 0x65, 0xb9, 0x36, 0x3a, 0x76, 0x5d, 0x2b, 0xff, 0x9a, 0x00, 0xc9, 0x27, 0x95, 0xc6, 0xb9, - 0x55, 0xe6, 0x16, 0x34, 0x19, 0x28, 0xe8, 0x6d, 0x28, 0xec, 0xea, 0xed, 0xb6, 0x6e, 0xb4, 0x88, - 0x17, 0xf5, 0x5d, 0xac, 0xba, 0x0a, 0xcb, 0x73, 0xf2, 0x36, 0xa3, 0xa2, 0xab, 0x90, 0x55, 0x2d, - 0xac, 0x61, 0xc3, 0xd1, 0x95, 0xb6, 0xcd, 0x35, 0x17, 0x24, 0x95, 0x7f, 0x41, 0x80, 0x69, 0x3a, - 0x02, 0xd0, 0x5b, 0x90, 0x51, 0x4d, 0xc3, 0x51, 0x74, 0x83, 0x9b, 0xb2, 0x8c, 0xe4, 0x13, 0x86, - 0x16, 0xef, 0x1a, 0xcc, 0x29, 0xaa, 0x6a, 0xf6, 0x0c, 0x47, 0x36, 0x94, 0x0e, 0xe6, 0xc5, 0xcc, - 0x72, 0xda, 0x96, 0xd2, 0xc1, 0x68, 0x11, 0xdc, 0x47, 0xef, 0xfc, 0x65, 0x46, 0x02, 0x4e, 0xda, - 0xc0, 0x27, 0xe5, 0x3f, 0x10, 0x20, 0xed, 0x8e, 0x19, 0x52, 0x8c, 0x16, 0x36, 0x58, 0xc4, 0xb7, - 0x5b, 0x0c, 0x8f, 0xd0, 0x3f, 0x55, 0x66, 0xfc, 0xa9, 0xf2, 0x02, 0x4c, 0x3b, 0x64, 0x58, 0xf0, - 0x12, 0xb0, 0x07, 0xba, 0x9e, 0xdd, 0x56, 0x5a, 0x6c, 0x39, 0x2f, 0x23, 0xb1, 0x07, 0x52, 0x19, - 0x1e, 0x93, 0xcb, 0x34, 0xc2, 0x9f, 0x48, 0x49, 0x59, 0xe4, 0xe8, 0x2e, 0x6e, 0xe9, 0x06, 0xed, - 0x4b, 0x49, 0x09, 0x28, 0x69, 0x95, 0x50, 0xd0, 0x9b, 0x90, 0x61, 0x19, 0xb0, 0xa1, 0xd1, 0x0e, - 0x95, 0x94, 0xd2, 0x94, 0x50, 0x35, 0xb4, 0x32, 0x86, 0x8c, 0x37, 0x38, 0x49, 0xb3, 0xf5, 0x6c, - 0x4f, 0x91, 0xf4, 0x37, 0x7a, 0x0f, 0x2e, 0xbc, 0xea, 0x29, 0x6d, 0x7d, 0x8f, 0xae, 0xd4, 0xd1, - 0x90, 0x78, 0xaa, 0x33, 0x56, 0x13, 0xe4, 0xa5, 0x51, 0x09, 0x54, 0x75, 0xee, 0x58, 0x4e, 0xfa, - 0x63, 0x59, 0xfc, 0x5d, 0x01, 0xe6, 0x59, 0x58, 0x10, 0x8b, 0x66, 0x8d, 0xcf, 0x0f, 0xf9, 0x08, - 0x32, 0x9a, 0xe2, 0x28, 0xec, 0x44, 0x69, 0x62, 0xe4, 0x89, 0x52, 0xef, 0x84, 0x83, 0xe2, 0x28, - 0xf4, 0x54, 0x29, 0x82, 0x14, 0xf9, 0xcd, 0x0e, 0xdf, 0x4a, 0xf4, 0xb7, 0xf8, 0x29, 0xa0, 0x60, - 0x41, 0xe3, 0xf4, 0xc8, 0xee, 0xc2, 0x45, 0xa2, 0xeb, 0xaa, 0xa1, 0x5a, 0x27, 0x5d, 0x47, 0x37, - 0x8d, 0x67, 0xf4, 0xaf, 0x8d, 0x8a, 0x81, 0x8d, 0x2d, 0xba, 0x9f, 0x25, 0xfe, 0xde, 0x0c, 0xe4, - 0xaa, 0xc7, 0x5d, 0xd3, 0x8a, 0x75, 0x19, 0x6c, 0x15, 0x66, 0xf9, 0x4a, 0xc1, 0x88, 0x8d, 0xe5, - 0x3e, 0x63, 0xee, 0xee, 0xd9, 0x72, 0x46, 0xb4, 0x0a, 0xc0, 0x22, 0x4e, 0x69, 0x54, 0x51, 0xf2, - 0x1c, 0x5b, 0x6d, 0x94, 0x8d, 0x9e, 0xae, 0xd8, 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, 0xbd, - 0xed, 0xf0, 0xc0, 0xbd, 0xe8, 0x18, 0xf3, 0xcd, 0x17, 0x95, 0xca, 0x63, 0x9a, 0x89, 0xc5, 0xd0, - 0xf9, 0xcf, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x07, 0xf8, 0x49, 0x1f, 0xd9, 0x76, 0x0f, 0xf5, - 0xad, 0xe6, 0xce, 0x4e, 0x17, 0x33, 0x12, 0xa5, 0x36, 0x1a, 0x4d, 0x29, 0xc3, 0x32, 0x34, 0x6c, - 0x07, 0x5d, 0x87, 0x9c, 0xd9, 0xd1, 0x1d, 0xd9, 0x75, 0x92, 0xb8, 0x47, 0x39, 0x47, 0x88, 0xae, - 0x13, 0x75, 0x9e, 0x03, 0x20, 0xb3, 0x13, 0x1f, 0x00, 0x41, 0x7f, 0x5d, 0x80, 0x4b, 0x5c, 0x91, - 0xf2, 0x2e, 0x0d, 0x92, 0x57, 0xda, 0xba, 0x73, 0x22, 0x1f, 0x1c, 0x96, 0xd2, 0xd4, 0x6f, 0xfd, - 0xe9, 0xc8, 0x06, 0x09, 0xf4, 0x83, 0x25, 0xb7, 0x59, 0x4e, 0x9e, 0x72, 0xe6, 0x8d, 0xc3, 0xaa, - 0xe1, 0x58, 0x27, 0xab, 0x97, 0xcf, 0x4e, 0x17, 0x17, 0x06, 0x53, 0x5f, 0x48, 0x0b, 0xf6, 0x20, - 0x0b, 0xaa, 0x01, 0x60, 0xaf, 0x1f, 0xd2, 0x19, 0x23, 0xda, 0xff, 0x88, 0xec, 0xb0, 0x52, 0x80, - 0x17, 0xdd, 0x81, 0x22, 0x3f, 0x70, 0xb3, 0xa7, 0xb7, 0xb1, 0x6c, 0xeb, 0x9f, 0x63, 0x3a, 0xb7, - 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x0d, 0xfd, 0x73, 0x5c, 0xfe, 0x2e, 0x94, 0x86, 0x95, 0x3e, - 0x38, 0x04, 0x32, 0x6c, 0x4b, 0xf7, 0xc3, 0xf0, 0x7a, 0xce, 0x04, 0x5d, 0x95, 0xaf, 0xe9, 0x7c, - 0x94, 0xf8, 0x50, 0x10, 0xff, 0x61, 0x02, 0x72, 0xab, 0xbd, 0xf6, 0xc1, 0xb3, 0x6e, 0xa3, 0xd7, - 0xe9, 0x28, 0xd6, 0x09, 0x31, 0x83, 0xcc, 0x50, 0x90, 0x02, 0x0a, 0xcc, 0x0c, 0x52, 0x4b, 0xa0, - 0x7f, 0x8e, 0xc9, 0xe4, 0x14, 0x3c, 0x44, 0xce, 0x0e, 0x01, 0xd0, 0x3a, 0x04, 0x4e, 0x86, 0x9b, - 0x47, 0x36, 0xfa, 0x10, 0x4a, 0x81, 0x8c, 0x74, 0xf1, 0x45, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0x2d, - 0x20, 0x26, 0xa5, 0x40, 0xc0, 0x4d, 0x9d, 0x24, 0x57, 0x59, 0x2a, 0x6a, 0xc2, 0x1c, 0xc9, 0x78, - 0x22, 0xd3, 0x29, 0xc4, 0x5d, 0xe0, 0xbd, 0x1f, 0x51, 0xad, 0x50, 0xb9, 0x97, 0xa8, 0x7e, 0x2a, - 0x94, 0x87, 0xfe, 0x94, 0xb2, 0xd8, 0xa7, 0x94, 0x3f, 0x81, 0x62, 0x7f, 0x86, 0xa0, 0x2e, 0x53, - 0x4c, 0x97, 0x17, 0x82, 0xba, 0x4c, 0x06, 0xf4, 0xb4, 0x9e, 0x4a, 0xa7, 0x8a, 0xd3, 0xe2, 0x9f, - 0x26, 0x21, 0xef, 0x76, 0xb3, 0x38, 0x81, 0xce, 0x2a, 0x4c, 0x93, 0x4e, 0xe1, 0x06, 0x89, 0xdc, - 0x1a, 0xd1, 0xbb, 0x79, 0xd8, 0x39, 0xe9, 0x2c, 0x2e, 0x48, 0xa6, 0xac, 0x71, 0x18, 0x9c, 0xf2, - 0x2f, 0x24, 0x20, 0x45, 0xb1, 0xc5, 0x7d, 0x48, 0xd1, 0x89, 0x42, 0x98, 0x64, 0xa2, 0xa0, 0x59, - 0xbd, 0xe9, 0x2c, 0x11, 0x70, 0x4d, 0x89, 0xcf, 0xb7, 0xaf, 0xbc, 0x7f, 0xff, 0x01, 0x35, 0x36, - 0x73, 0x12, 0x7f, 0x42, 0xab, 0x34, 0x6e, 0xc9, 0xb4, 0x1c, 0xac, 0x71, 0x9f, 0xfe, 0xea, 0xb8, - 0xf6, 0x75, 0x27, 0x25, 0x97, 0x0f, 0xbd, 0x01, 0x49, 0x62, 0xc5, 0x66, 0x59, 0x94, 0xc3, 0xd9, - 0xe9, 0x62, 0x92, 0xd8, 0x2f, 0x42, 0x43, 0xcb, 0x90, 0x0d, 0x9b, 0x0c, 0xe2, 0xc1, 0x51, 0xc3, - 0x18, 0x18, 0xee, 0xd0, 0xf6, 0x86, 0x16, 0xc3, 0xb3, 0xbc, 0x8d, 0xff, 0x2c, 0x05, 0xb9, 0x7a, - 0x27, 0xee, 0x29, 0x65, 0x25, 0xdc, 0xc2, 0x51, 0x40, 0x28, 0xf4, 0xd2, 0x88, 0x06, 0x0e, 0xcd, - 0xe0, 0xc9, 0xf3, 0xcd, 0xe0, 0x75, 0xe2, 0x29, 0xf3, 0x5b, 0x21, 0x92, 0x43, 0x30, 0x4f, 0xf8, - 0xfd, 0xd4, 0x4f, 0x91, 0x08, 0x8f, 0x7f, 0x10, 0x83, 0x46, 0xaa, 0x7c, 0x42, 0x1d, 0x72, 0xd6, - 0xcb, 0x66, 0x26, 0xef, 0x65, 0xb3, 0xd8, 0xd0, 0xe8, 0xa4, 0x16, 0xb6, 0xa8, 0xb3, 0xaf, 0x6f, - 0x51, 0xcb, 0x0e, 0xef, 0xac, 0x1f, 0x41, 0x52, 0xd3, 0xdd, 0xc6, 0x99, 0x7c, 0xaa, 0x26, 0x4c, - 0x63, 0x7a, 0x6d, 0x2a, 0xd8, 0x6b, 0x59, 0x2f, 0x29, 0xd7, 0x01, 0x7c, 0xdd, 0xa0, 0xab, 0x30, - 0x63, 0xb6, 0x35, 0xf7, 0x24, 0x4a, 0x6e, 0x35, 0x73, 0x76, 0xba, 0x38, 0xfd, 0xac, 0xad, 0xd5, - 0xd7, 0xa4, 0x69, 0xb3, 0xad, 0xd5, 0x35, 0x7a, 0x25, 0x07, 0x3e, 0x92, 0xbd, 0x30, 0xb5, 0x39, - 0x69, 0xd6, 0xc0, 0x47, 0x6b, 0xd8, 0x56, 0x79, 0x87, 0xfb, 0x4d, 0x01, 0xf2, 0xae, 0xee, 0xe3, - 0x35, 0x2a, 0x69, 0xbd, 0xc3, 0x07, 0x59, 0xf2, 0x7c, 0x83, 0xcc, 0xe5, 0xe3, 0xa7, 0x7e, 0x7f, - 0x49, 0xe0, 0x81, 0xc7, 0x0d, 0x55, 0x71, 0x88, 0x53, 0x11, 0xe3, 0xc0, 0xb8, 0x0b, 0x45, 0x4b, - 0x31, 0x34, 0xb3, 0xa3, 0x7f, 0x8e, 0xd9, 0x42, 0xa8, 0xcd, 0xb7, 0x3d, 0x0b, 0x1e, 0x9d, 0xae, - 0xfa, 0xd9, 0xe2, 0x1f, 0x24, 0x78, 0x90, 0xb2, 0x57, 0x8c, 0x38, 0xd5, 0xf5, 0x1d, 0x98, 0xef, - 0xbf, 0x1f, 0xc5, 0x1d, 0xad, 0xef, 0x46, 0xc8, 0x8b, 0x2a, 0x08, 0x0b, 0x36, 0x74, 0x23, 0xdf, - 0xfb, 0xee, 0x4a, 0xb1, 0x51, 0x05, 0xb2, 0xc1, 0x6b, 0x57, 0x92, 0x13, 0x5f, 0xbb, 0x02, 0x96, - 0x77, 0xd9, 0x4a, 0xf9, 0x67, 0x61, 0x9a, 0x26, 0xbf, 0x86, 0x89, 0xe6, 0xad, 0xf9, 0x27, 0x09, - 0xb8, 0x41, 0x4b, 0xff, 0x02, 0x5b, 0xfa, 0xde, 0xc9, 0xb6, 0x65, 0x3a, 0x58, 0x75, 0xb0, 0xe6, - 0xaf, 0x9c, 0xc7, 0x6a, 0xf7, 0x32, 0x5d, 0xf7, 0x05, 0xe7, 0x0a, 0x38, 0xf3, 0xb8, 0xd0, 0x06, - 0x14, 0x78, 0x68, 0x81, 0xd2, 0xd6, 0x0f, 0xb1, 0xac, 0x38, 0xe7, 0x99, 0xdd, 0x72, 0x8c, 0x77, - 0x85, 0xb0, 0xae, 0x38, 0x48, 0x83, 0x0c, 0x17, 0xa6, 0x6b, 0xfc, 0xb6, 0xa0, 0x27, 0x5f, 0x6e, - 0x41, 0x31, 0xcd, 0xe2, 0x1b, 0xea, 0x6b, 0x52, 0x9a, 0x49, 0xae, 0x6b, 0xe2, 0x7f, 0x12, 0xe0, - 0xe6, 0x18, 0x15, 0xc7, 0xd9, 0x75, 0xcb, 0x90, 0x3e, 0x24, 0x2f, 0xd2, 0xb9, 0x8e, 0xd3, 0x92, - 0xf7, 0x8c, 0x36, 0x21, 0xb7, 0xa7, 0xe8, 0x6d, 0xbf, 0x4b, 0x0f, 0x8f, 0x4f, 0x8c, 0x0e, 0x95, - 0x9d, 0x63, 0xec, 0xac, 0x0f, 0x8b, 0xbf, 0x9e, 0x80, 0xf9, 0x15, 0x4d, 0x6b, 0x34, 0xb8, 0x0d, - 0x8c, 0xaf, 0xa7, 0xb8, 0x20, 0x33, 0xe1, 0x83, 0x4c, 0xf4, 0x2e, 0x20, 0x4d, 0xb7, 0xd9, 0xad, - 0x24, 0xf6, 0xbe, 0xa2, 0x99, 0x47, 0x7e, 0x5c, 0xc6, 0xbc, 0x9b, 0xd2, 0x70, 0x13, 0x50, 0x03, - 0x28, 0xda, 0x91, 0x6d, 0x47, 0xf1, 0xf6, 0x9d, 0x6e, 0x4e, 0x74, 0xde, 0x8b, 0xc1, 0x20, 0xef, - 0x51, 0xca, 0x10, 0x39, 0xf4, 0x27, 0xf1, 0xdb, 0x75, 0x52, 0x75, 0x47, 0x56, 0x6c, 0xf7, 0x70, - 0x0f, 0xbb, 0x0f, 0x25, 0xcf, 0xe8, 0x2b, 0x36, 0x3b, 0xb3, 0xc3, 0x4e, 0x23, 0xf8, 0xaa, 0x89, - 0x13, 0x12, 0xff, 0x3d, 0x01, 0xf2, 0x12, 0xde, 0xb3, 0xb0, 0x1d, 0xeb, 0xa2, 0xc0, 0x63, 0x98, - 0xb3, 0x98, 0x54, 0x79, 0xcf, 0x32, 0x3b, 0xe7, 0x19, 0x57, 0x59, 0xce, 0xf8, 0xd8, 0x32, 0x3b, - 0xdc, 0xb0, 0xbc, 0x80, 0x82, 0x57, 0xc6, 0x38, 0x2b, 0xff, 0xbb, 0xf4, 0x2c, 0x33, 0x13, 0x1c, - 0x77, 0x80, 0x44, 0xbc, 0x1a, 0xa0, 0x1b, 0x55, 0xc1, 0x82, 0xc6, 0xa9, 0x86, 0xff, 0x26, 0x40, - 0xbe, 0xd1, 0xdb, 0x65, 0xd7, 0x5e, 0xc5, 0xa7, 0x81, 0x2a, 0x64, 0xda, 0x78, 0xcf, 0x91, 0x5f, - 0x2b, 0x8e, 0x3e, 0x4d, 0x58, 0xe9, 0x29, 0x82, 0x27, 0x00, 0x16, 0x3d, 0xff, 0x46, 0xe5, 0x24, - 0xcf, 0x29, 0x27, 0x43, 0x79, 0x09, 0x59, 0xfc, 0xb3, 0x04, 0x14, 0xbc, 0x6a, 0xc6, 0x69, 0x25, - 0x5f, 0x86, 0xac, 0x43, 0xf2, 0x3c, 0xd6, 0x61, 0x9e, 0xc7, 0x84, 0x44, 0x5b, 0x88, 0x25, 0x58, - 0xa0, 0x8e, 0x8b, 0xac, 0x74, 0xbb, 0x6d, 0xdd, 0x85, 0xbb, 0xd4, 0xfe, 0xa4, 0xa4, 0x79, 0x9a, - 0xb4, 0xc2, 0x52, 0x28, 0xd0, 0x45, 0xbf, 0x2c, 0xc0, 0xdc, 0x9e, 0x85, 0xf1, 0xe7, 0x58, 0xa6, - 0xd0, 0x6b, 0xb2, 0xa0, 0x97, 0x35, 0x52, 0x86, 0x2f, 0xbd, 0x29, 0x9e, 0x65, 0x2f, 0x6e, 0x90, - 0xf7, 0xf2, 0x4e, 0xfb, 0x12, 0xe6, 0x69, 0xa3, 0xc4, 0x7d, 0xf0, 0x57, 0xfc, 0x3b, 0x09, 0x40, - 0x41, 0xc9, 0x5f, 0x5d, 0x63, 0x26, 0xe2, 0x6b, 0xcc, 0x77, 0x00, 0xb1, 0xb0, 0x49, 0x5b, 0xee, - 0x62, 0x4b, 0xb6, 0xb1, 0x6a, 0xf2, 0xbb, 0x9b, 0x04, 0xa9, 0xc8, 0x53, 0xb6, 0xb1, 0xd5, 0xa0, - 0x74, 0xf4, 0x08, 0xc0, 0x77, 0xe9, 0xf8, 0x8c, 0x33, 0xd2, 0xa3, 0x93, 0x32, 0x9e, 0x2f, 0x27, - 0x7e, 0x4f, 0x80, 0xfc, 0xa6, 0xde, 0xb2, 0x94, 0x58, 0x2f, 0x25, 0x42, 0x1f, 0x85, 0x97, 0xd1, - 0xb3, 0x0f, 0xca, 0x51, 0x21, 0x3e, 0x2c, 0x87, 0x0b, 0xce, 0x38, 0x03, 0x31, 0xe1, 0x5e, 0x89, - 0xe2, 0xb4, 0x5d, 0xff, 0xbe, 0x0c, 0x73, 0xbc, 0xdc, 0x3b, 0x86, 0x6e, 0x1a, 0xe8, 0x3e, 0x24, - 0x5b, 0x7c, 0x9b, 0x24, 0x1b, 0xb9, 0xa4, 0xe9, 0xdf, 0xcf, 0x57, 0x9b, 0x92, 0x48, 0x5e, 0xc2, - 0xd2, 0xed, 0x39, 0x11, 0x4e, 0xa4, 0x1f, 0xf5, 0x1e, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, 0x80, 0x82, - 0xea, 0xdf, 0x33, 0x26, 0x13, 0xf6, 0xe4, 0x50, 0xc0, 0x19, 0x79, 0xe3, 0x5b, 0x6d, 0x4a, 0xca, - 0xab, 0xa1, 0x04, 0x54, 0x09, 0x5e, 0x6f, 0x95, 0x1a, 0x08, 0xa9, 0xf3, 0x0f, 0x53, 0x87, 0xaf, - 0xd6, 0xaa, 0x4d, 0x05, 0x6e, 0xc1, 0x42, 0x1f, 0xc1, 0x8c, 0x46, 0x2f, 0x52, 0xe2, 0x83, 0x3f, - 0xaa, 0xa1, 0x43, 0xf7, 0x55, 0xd5, 0xa6, 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, 0x62, 0x2e, - 0x1d, 0x47, 0xe1, 0x37, 0x87, 0x4b, 0x08, 0x4c, 0x9a, 0xb5, 0x29, 0x29, 0xab, 0xf9, 0x54, 0xf4, - 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0xe2, 0xf0, 0x2b, 0x43, 0xee, 0x2f, 0xf1, 0x99, 0x69, 0x6e, 0xf4, - 0x88, 0xdd, 0xb8, 0xe9, 0x1c, 0xbb, 0x4b, 0xa2, 0x51, 0xc5, 0x0f, 0x9d, 0x93, 0x27, 0xc5, 0xc7, - 0x94, 0x80, 0x9e, 0x40, 0x56, 0x21, 0xbe, 0xb1, 0x4c, 0x4f, 0x9a, 0xd2, 0x35, 0xd0, 0xe8, 0x40, - 0x84, 0x81, 0x93, 0xc1, 0x35, 0x7a, 0x18, 0xdf, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, 0xe1, 0x52, - 0x76, 0xb4, 0xa0, 0x60, 0x20, 0x9d, 0x27, 0x88, 0x12, 0x89, 0x8f, 0xec, 0x1d, 0xf3, 0xa6, 0x95, - 0x9a, 0x1b, 0xba, 0xf5, 0x1d, 0x71, 0x46, 0xaa, 0x36, 0x25, 0xcd, 0xed, 0x07, 0xc8, 0x68, 0x09, - 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x6a, 0x0c, 0xbc, 0x73, 0x40, 0xb5, 0x29, 0x29, 0xd1, 0x52, 0xd1, - 0x27, 0x90, 0x66, 0x87, 0x3a, 0x8e, 0x8d, 0x52, 0x7e, 0xe8, 0x18, 0x0b, 0x1f, 0x8d, 0xa9, 0x4d, - 0x49, 0xf4, 0x1c, 0x09, 0x79, 0xdf, 0x36, 0xe4, 0x2d, 0x16, 0x89, 0xe8, 0xc6, 0x0d, 0x17, 0x87, - 0x86, 0x03, 0x44, 0x85, 0x0e, 0xd7, 0x28, 0x4a, 0x0a, 0xd0, 0xd1, 0x77, 0xe0, 0x42, 0x58, 0x22, - 0xef, 0x69, 0xf3, 0x43, 0xb7, 0xb6, 0x87, 0x86, 0xb1, 0xd6, 0xa6, 0x24, 0x64, 0x0d, 0x24, 0xa2, - 0x0f, 0x60, 0x9a, 0xb5, 0x1a, 0xa2, 0x22, 0xa3, 0x22, 0x5c, 0xfa, 0x1a, 0x8c, 0xe5, 0x27, 0x9d, - 0xdf, 0xe1, 0xe1, 0x78, 0x72, 0xdb, 0x6c, 0x95, 0x16, 0x86, 0x76, 0xfe, 0xc1, 0xc0, 0x42, 0xd2, - 0xf9, 0x1d, 0x9f, 0x4a, 0xda, 0xdd, 0x62, 0x29, 0x3c, 0x04, 0xeb, 0xc2, 0xd0, 0x76, 0x8f, 0x88, - 0xd2, 0xab, 0xd1, 0x83, 0x12, 0x3e, 0x99, 0x14, 0xcd, 0x62, 0x57, 0xee, 0xc8, 0x74, 0x4c, 0x5d, - 0x1c, 0x5a, 0xb4, 0xc1, 0x3b, 0x89, 0x6a, 0xd4, 0x9f, 0xf4, 0xa8, 0xe8, 0x05, 0x14, 0xf9, 0xc5, - 0x18, 0xfe, 0xfe, 0xcb, 0x25, 0x2a, 0xef, 0x6e, 0xa4, 0xe9, 0x8a, 0x8a, 0x5f, 0xaa, 0x4d, 0x49, - 0x05, 0x35, 0x9c, 0x82, 0x3e, 0x85, 0x79, 0x2a, 0x4f, 0x56, 0xfd, 0xbb, 0x4c, 0x4a, 0xa5, 0x81, - 0x9b, 0x31, 0x86, 0x5f, 0x7b, 0xe2, 0x4a, 0x2e, 0xaa, 0x7d, 0x49, 0xa4, 0x1b, 0xeb, 0x86, 0xee, - 0x50, 0x2b, 0x5b, 0x1e, 0xda, 0x8d, 0xc3, 0xf7, 0x32, 0x92, 0x6e, 0xac, 0x33, 0x0a, 0xe9, 0xc6, - 0x0e, 0x8f, 0xcf, 0xe3, 0xcd, 0xf1, 0xd6, 0xd0, 0x6e, 0x1c, 0x15, 0xc8, 0x47, 0xba, 0xb1, 0x13, - 0xa4, 0x93, 0x6e, 0xcc, 0x0c, 0x44, 0x9f, 0xdc, 0xb7, 0x87, 0x76, 0xe3, 0xa1, 0x27, 0xbe, 0x49, - 0x37, 0x56, 0x06, 0x12, 0xd1, 0x1a, 0x00, 0x73, 0xfd, 0xe8, 0xfc, 0x7f, 0x65, 0xe8, 0x64, 0xd0, - 0x1f, 0xa7, 0x47, 0x26, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, 0x02, 0x4b, 0x99, 0x6e, 0x37, 0x97, - 0x16, 0x87, 0x1a, 0xb2, 0x81, 0xad, 0x61, 0x62, 0xc8, 0x8e, 0x3c, 0x22, 0x99, 0x55, 0xd8, 0xfa, - 0x78, 0xe9, 0xea, 0x70, 0xb3, 0x1c, 0xdc, 0x26, 0xa3, 0x66, 0x99, 0x12, 0xd0, 0x0a, 0x64, 0x88, - 0x7b, 0x73, 0x42, 0xcd, 0xd0, 0xb5, 0xa1, 0xfe, 0x7b, 0xdf, 0x49, 0xa0, 0xda, 0x94, 0x94, 0x7e, - 0xc5, 0x49, 0xe4, 0xf5, 0x6c, 0xe5, 0xb0, 0x24, 0x0e, 0x7d, 0x7d, 0x68, 0x95, 0x99, 0xbc, 0x9e, - 0x71, 0x20, 0x15, 0x2e, 0xb2, 0xb6, 0xe2, 0x87, 0xaf, 0x2d, 0x7e, 0x5e, 0xb8, 0x74, 0x9d, 0x8a, - 0x1a, 0xba, 0x04, 0x17, 0x79, 0x26, 0xbc, 0x36, 0x25, 0x2d, 0x28, 0x83, 0xa9, 0x64, 0xc0, 0xf3, - 0xa9, 0x87, 0x2d, 0xdc, 0x95, 0x6e, 0x0c, 0x1d, 0xf0, 0x11, 0xeb, 0x9d, 0x64, 0xc0, 0x2b, 0x01, - 0x32, 0x9b, 0x80, 0x34, 0xd9, 0xb6, 0x59, 0x70, 0xc2, 0xcd, 0x11, 0x13, 0x50, 0xdf, 0x8a, 0x09, - 0x9b, 0x80, 0xb4, 0x06, 0xe3, 0x24, 0x82, 0xd4, 0x36, 0x56, 0x2c, 0x6e, 0x66, 0x6f, 0x0d, 0x15, - 0x34, 0x70, 0x05, 0x22, 0x11, 0xa4, 0x7a, 0x44, 0xe2, 0xf0, 0x58, 0xee, 0x8d, 0x3a, 0xdc, 0x37, - 0xbe, 0x3d, 0xd4, 0xe1, 0x89, 0xbc, 0xf2, 0x87, 0x38, 0x3c, 0x56, 0x28, 0x01, 0xfd, 0x0c, 0xcc, - 0x72, 0xa8, 0x5b, 0xba, 0x33, 0xc2, 0x05, 0x0c, 0xae, 0x4e, 0x90, 0x71, 0xcd, 0x79, 0x98, 0x95, - 0x65, 0x10, 0x9b, 0x55, 0xef, 0xee, 0x08, 0x2b, 0x3b, 0x80, 0xf2, 0x99, 0x95, 0xf5, 0xc9, 0xc4, - 0xca, 0xb2, 0x7e, 0xca, 0xe7, 0xba, 0x7b, 0x43, 0xad, 0xec, 0xe0, 0x39, 0x23, 0x62, 0x65, 0x5f, - 0xf9, 0x54, 0x52, 0x33, 0x9b, 0x41, 0xcd, 0xd2, 0x37, 0x86, 0xd6, 0x2c, 0x8c, 0xb9, 0x49, 0xcd, - 0x38, 0x0f, 0x69, 0x36, 0xe6, 0xfd, 0x33, 0x4d, 0xbf, 0x33, 0xfc, 0x86, 0x83, 0x7e, 0x7c, 0x55, - 0x73, 0x17, 0x75, 0x99, 0x86, 0x3d, 0x43, 0x65, 0xf1, 0x13, 0xde, 0x5c, 0x53, 0xef, 0x8e, 0x36, - 0x54, 0x51, 0xc7, 0xd6, 0x3d, 0x43, 0x15, 0x4a, 0xa4, 0x45, 0x65, 0x87, 0xfa, 0xe8, 0xf8, 0x5e, - 0x1a, 0x71, 0x19, 0x43, 0xdf, 0xd1, 0x4a, 0x5a, 0x54, 0x8f, 0xe8, 0x0f, 0xa1, 0x1e, 0xbb, 0x29, - 0xa4, 0xb4, 0x3c, 0x7a, 0x08, 0x85, 0xef, 0x2a, 0xf1, 0x86, 0x10, 0x27, 0x7b, 0x73, 0xa6, 0xeb, - 0x61, 0xbc, 0x37, 0x7a, 0xce, 0xec, 0x77, 0x2d, 0xd8, 0x9c, 0xc9, 0x7d, 0x8a, 0xbf, 0x26, 0xc0, - 0x55, 0x56, 0x36, 0xba, 0xfa, 0x79, 0x22, 0x7b, 0x6b, 0xc8, 0x81, 0x03, 0x26, 0xf7, 0xe9, 0x0b, - 0x3e, 0x18, 0x56, 0xdc, 0x31, 0x6b, 0xe2, 0xb5, 0x29, 0xe9, 0x6d, 0x65, 0x54, 0x3e, 0xd2, 0xa5, - 0x3a, 0x0c, 0x41, 0x95, 0x1e, 0x0c, 0xed, 0x52, 0x61, 0xd4, 0x47, 0xba, 0x14, 0xe7, 0x59, 0x9d, - 0xe5, 0x3b, 0xd0, 0xde, 0x81, 0xdc, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, 0x5f, 0x2e, 0x96, 0xd6, 0x53, - 0xe9, 0x37, 0x8a, 0xe5, 0xf5, 0x54, 0xfa, 0xcd, 0xe2, 0x5b, 0xe2, 0x3f, 0x2a, 0x43, 0xce, 0x05, - 0x5d, 0x0c, 0x50, 0x3d, 0x08, 0x02, 0xaa, 0x2b, 0xc3, 0x00, 0x15, 0x87, 0x69, 0x1c, 0x51, 0x3d, - 0x08, 0x22, 0xaa, 0x2b, 0xc3, 0x10, 0x95, 0xcf, 0x43, 0x20, 0x55, 0x73, 0x18, 0xa4, 0xba, 0x3b, - 0x01, 0xa4, 0xf2, 0x44, 0xf5, 0x63, 0xaa, 0xb5, 0x41, 0x4c, 0x75, 0x63, 0x34, 0xa6, 0xf2, 0x44, - 0x05, 0x40, 0xd5, 0xa3, 0x3e, 0x50, 0x75, 0x6d, 0x04, 0xa8, 0xf2, 0xf8, 0x5d, 0x54, 0xb5, 0x11, - 0x89, 0xaa, 0x6e, 0x8d, 0x43, 0x55, 0x9e, 0x9c, 0x10, 0xac, 0x7a, 0x3f, 0x04, 0xab, 0x16, 0x87, - 0xc2, 0x2a, 0x8f, 0x9b, 0xe1, 0xaa, 0x8f, 0xfb, 0x71, 0xd5, 0xb5, 0x11, 0xb8, 0xca, 0xaf, 0x01, - 0x07, 0x56, 0xb5, 0x28, 0x60, 0x75, 0x73, 0x0c, 0xb0, 0xf2, 0xa4, 0x04, 0x91, 0x55, 0x2d, 0x0a, - 0x59, 0xdd, 0x1c, 0x83, 0xac, 0xfa, 0x24, 0x31, 0x68, 0xb5, 0x15, 0x0d, 0xad, 0x6e, 0x8f, 0x85, - 0x56, 0x9e, 0xb4, 0x30, 0xb6, 0x5a, 0x0e, 0x60, 0xab, 0xb7, 0x87, 0x60, 0x2b, 0x8f, 0x95, 0x80, - 0xab, 0x6f, 0x0d, 0x80, 0x2b, 0x71, 0x14, 0xb8, 0xf2, 0x78, 0x3d, 0x74, 0xf5, 0x7c, 0x08, 0xba, - 0xba, 0x33, 0x1e, 0x5d, 0x79, 0xc2, 0xfa, 0xe0, 0x95, 0x32, 0x12, 0x5e, 0xbd, 0x3b, 0x21, 0xbc, - 0xf2, 0xa4, 0x47, 0xe1, 0xab, 0x0f, 0xc3, 0xf8, 0xea, 0xea, 0x70, 0x7c, 0xe5, 0x89, 0xe1, 0x00, - 0x6b, 0x23, 0x12, 0x60, 0xdd, 0x1a, 0x07, 0xb0, 0xfc, 0x71, 0x10, 0x44, 0x58, 0x5b, 0xd1, 0x08, - 0xeb, 0xf6, 0x58, 0x84, 0xe5, 0x37, 0x7f, 0x08, 0x62, 0x6d, 0x44, 0x42, 0xac, 0x5b, 0xe3, 0x20, - 0x96, 0x5f, 0xb8, 0x20, 0xc6, 0x7a, 0x39, 0x14, 0x63, 0xdd, 0x9b, 0x04, 0x63, 0x79, 0x42, 0x07, - 0x40, 0xd6, 0x67, 0xc3, 0x41, 0xd6, 0x37, 0xce, 0x71, 0xb7, 0x64, 0x24, 0xca, 0xfa, 0xd6, 0x00, - 0xca, 0x12, 0x47, 0xa1, 0x2c, 0xbf, 0x3f, 0xbb, 0x30, 0x4b, 0x19, 0x09, 0x8a, 0xde, 0x9d, 0x10, - 0x14, 0xf9, 0x9d, 0x2f, 0x02, 0x15, 0x55, 0x23, 0x50, 0xd1, 0x8d, 0xd1, 0xa8, 0xc8, 0x37, 0xe7, - 0x3e, 0x2c, 0xaa, 0x45, 0xc1, 0xa2, 0x9b, 0x63, 0x60, 0x91, 0x6f, 0x85, 0x02, 0xb8, 0xe8, 0x51, - 0x1f, 0x2e, 0xba, 0x36, 0x36, 0xc0, 0x2a, 0x00, 0x8c, 0x56, 0x07, 0x81, 0xd1, 0xf5, 0x91, 0xc0, - 0xc8, 0x93, 0xe0, 0x23, 0xa3, 0x47, 0x7d, 0xc8, 0xe8, 0xda, 0x08, 0x64, 0xe4, 0x17, 0x80, 0x43, - 0x23, 0x6d, 0x34, 0x34, 0x5a, 0x9a, 0x14, 0x1a, 0x79, 0x82, 0x23, 0xb1, 0xd1, 0x56, 0x34, 0x36, - 0xba, 0x3d, 0x61, 0xec, 0xc3, 0x00, 0x38, 0xaa, 0x45, 0x81, 0xa3, 0x9b, 0x63, 0xc0, 0x51, 0x70, - 0x0e, 0xf1, 0xd0, 0x51, 0x2d, 0x0a, 0x1d, 0xdd, 0x1c, 0x83, 0x8e, 0x7c, 0x49, 0x01, 0x78, 0xd4, - 0x1c, 0x06, 0x8f, 0xee, 0x4e, 0x00, 0x8f, 0x7c, 0xe7, 0xa5, 0x0f, 0x1f, 0x7d, 0xd2, 0x8f, 0x8f, - 0xc4, 0x51, 0xf8, 0xc8, 0x1f, 0x91, 0x2e, 0x40, 0xda, 0x8a, 0x06, 0x48, 0xb7, 0xc7, 0x02, 0xa4, - 0xa0, 0x91, 0x0c, 0x20, 0xa4, 0x8d, 0x48, 0x84, 0x74, 0x6b, 0x1c, 0x42, 0xf2, 0x8d, 0x64, 0x10, - 0x22, 0x7d, 0xd2, 0x0f, 0x91, 0xc4, 0x51, 0x10, 0xc9, 0xaf, 0x9c, 0x8b, 0x91, 0x6a, 0x51, 0x18, - 0xe9, 0xe6, 0x18, 0x8c, 0xe4, 0x37, 0x5e, 0x00, 0x24, 0x29, 0x23, 0x41, 0xd2, 0xbb, 0x13, 0x82, - 0xa4, 0x3e, 0xc3, 0x15, 0x46, 0x49, 0xb5, 0x28, 0x94, 0x74, 0x73, 0x0c, 0x4a, 0x0a, 0x14, 0xd6, - 0x87, 0x49, 0x5b, 0xd1, 0x30, 0xe9, 0xf6, 0x58, 0x98, 0xd4, 0x37, 0x9a, 0x5c, 0x9c, 0xb4, 0x11, - 0x89, 0x93, 0x6e, 0x8d, 0xc3, 0x49, 0x7d, 0x13, 0x1f, 0x77, 0x0e, 0x7e, 0x71, 0x72, 0xa0, 0xf4, - 0xe1, 0xf9, 0x81, 0x92, 0xf7, 0xce, 0x31, 0x48, 0xe9, 0x93, 0x7e, 0xa4, 0x24, 0x8e, 0x42, 0x4a, - 0x7e, 0xcf, 0x3a, 0x1f, 0x54, 0x5a, 0x4f, 0xa5, 0xdf, 0x2a, 0xbe, 0x2d, 0xfe, 0x8d, 0x59, 0x98, - 0xa9, 0x79, 0x21, 0x49, 0x7e, 0x2d, 0x85, 0xd7, 0xb9, 0x03, 0x0b, 0xad, 0x91, 0x11, 0x4f, 0xed, - 0xe6, 0xf8, 0x6b, 0x0d, 0x07, 0x2f, 0xde, 0xe3, 0xac, 0xaf, 0x71, 0x18, 0x1d, 0xbd, 0x0f, 0xb9, - 0x9e, 0x8d, 0x2d, 0xb9, 0x6b, 0xe9, 0xa6, 0xa5, 0x3b, 0xec, 0xc0, 0x8d, 0xb0, 0x5a, 0xfc, 0xe2, - 0x74, 0x71, 0x6e, 0xc7, 0xc6, 0xd6, 0x36, 0xa7, 0x4b, 0x73, 0xbd, 0xc0, 0x93, 0xfb, 0x09, 0xb2, - 0xe9, 0xc9, 0x3f, 0x41, 0xf6, 0x1c, 0x8a, 0x16, 0x56, 0xb4, 0x90, 0x07, 0xc3, 0xee, 0x9b, 0x8a, - 0xee, 0x73, 0xf4, 0x40, 0x9c, 0x9b, 0x93, 0xde, 0x3b, 0x55, 0xb0, 0xc2, 0x44, 0x74, 0x1f, 0x2e, - 0x76, 0x94, 0x63, 0x1a, 0xbe, 0x2a, 0xbb, 0x4e, 0x21, 0x0d, 0x49, 0x65, 0x5f, 0xf7, 0x42, 0x1d, - 0xe5, 0x98, 0x7e, 0xcf, 0x8c, 0x25, 0xd1, 0x8f, 0x91, 0xdc, 0x84, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, - 0xa8, 0x0e, 0xbf, 0x67, 0x98, 0xdd, 0xd1, 0x9b, 0x73, 0xa9, 0xec, 0x32, 0xe1, 0x7b, 0x30, 0xcf, - 0xcf, 0x35, 0x04, 0x36, 0x63, 0xf9, 0x5d, 0xbd, 0x2c, 0xc1, 0xdb, 0x7f, 0x45, 0x15, 0x28, 0xb4, - 0x14, 0x07, 0x1f, 0x29, 0x27, 0xb2, 0x7b, 0x6a, 0x2e, 0x4b, 0xaf, 0xdf, 0x7c, 0xf3, 0xec, 0x74, - 0x31, 0xf7, 0x84, 0x25, 0x0d, 0x1c, 0x9e, 0xcb, 0xb5, 0x02, 0x09, 0x1a, 0xba, 0x0d, 0x05, 0xc5, - 0x3e, 0x31, 0x54, 0xaa, 0x1e, 0x6c, 0xd8, 0x3d, 0x9b, 0x42, 0x92, 0xb4, 0x94, 0xa7, 0xe4, 0x8a, - 0x4b, 0x45, 0xd7, 0x60, 0x8e, 0x07, 0xfd, 0xb3, 0x0f, 0x24, 0x15, 0x68, 0x55, 0xf9, 0x97, 0x37, - 0xd8, 0x37, 0x92, 0x1e, 0x41, 0x99, 0x7f, 0x47, 0xe0, 0x48, 0xb1, 0x34, 0x99, 0x6a, 0xdd, 0xef, - 0x9f, 0x45, 0x2a, 0xf6, 0x32, 0xfb, 0x6e, 0x00, 0xc9, 0x40, 0x54, 0xed, 0x8f, 0xa1, 0x2d, 0x98, - 0x57, 0xdb, 0xba, 0x87, 0x20, 0x58, 0xcd, 0xe7, 0x87, 0x8e, 0xa6, 0x0a, 0xcd, 0xeb, 0x6f, 0x46, - 0x17, 0xd4, 0x30, 0x01, 0x35, 0x80, 0xde, 0xec, 0x23, 0x77, 0xcd, 0xb6, 0xae, 0x9e, 0x50, 0xf0, - 0x10, 0xbe, 0x0f, 0x7d, 0xe4, 0x57, 0x09, 0x5e, 0x2a, 0xba, 0xb3, 0x4d, 0x39, 0x25, 0x38, 0xf2, - 0x7e, 0xb3, 0xbb, 0x8c, 0xd7, 0x53, 0xe9, 0xb9, 0x62, 0x6e, 0x3d, 0x95, 0xce, 0x17, 0x0b, 0xe2, - 0xdf, 0x14, 0xa0, 0xd0, 0x57, 0x16, 0x54, 0x83, 0x8b, 0x9a, 0x37, 0x54, 0x64, 0x7e, 0x2a, 0x4c, - 0x37, 0x0d, 0x7e, 0xc5, 0xfb, 0xc2, 0x17, 0xa7, 0x8b, 0x05, 0x9a, 0xfb, 0x89, 0x97, 0x24, 0x5d, - 0xf0, 0x39, 0x7c, 0x2a, 0xfa, 0x10, 0xf2, 0xcc, 0xfd, 0xf4, 0xbe, 0xf9, 0x47, 0xc3, 0xf9, 0x57, - 0xe7, 0xbf, 0x38, 0x5d, 0xcc, 0x51, 0x9f, 0xd3, 0xbd, 0x82, 0x59, 0xca, 0xb5, 0x83, 0x8f, 0xe2, - 0xaf, 0x09, 0x30, 0x17, 0x3a, 0x79, 0xf5, 0xa8, 0x6f, 0xf3, 0xfb, 0x8d, 0x68, 0xdc, 0x3a, 0x2c, - 0xf6, 0x31, 0xcd, 0xfb, 0xb9, 0x1b, 0x48, 0xba, 0x38, 0x1c, 0xf7, 0xd0, 0x55, 0x1c, 0x37, 0xaa, - 0xc6, 0x65, 0xfb, 0x28, 0xf5, 0xfd, 0x1f, 0x2c, 0x4e, 0x89, 0x7f, 0x9e, 0x82, 0x5c, 0xf8, 0x9c, - 0x55, 0xbd, 0xaf, 0x5c, 0x51, 0xf3, 0x4a, 0x88, 0x63, 0x69, 0xc4, 0x15, 0x95, 0x19, 0xff, 0x5b, - 0x0a, 0xac, 0x98, 0x57, 0x47, 0x6c, 0xf1, 0x07, 0xcb, 0xe9, 0x33, 0x96, 0xff, 0x43, 0xd2, 0xb3, - 0xaf, 0x4b, 0x30, 0x4d, 0xaf, 0x4c, 0xe2, 0x45, 0x2b, 0xf5, 0xf7, 0x1e, 0xe2, 0x6b, 0x93, 0x74, - 0x89, 0x65, 0x23, 0xf6, 0xb8, 0xf9, 0x5a, 0x77, 0x12, 0xfa, 0xc3, 0xe0, 0xfc, 0x9f, 0x58, 0xec, - 0xb1, 0x3b, 0x29, 0xff, 0x3f, 0x46, 0xde, 0x90, 0xf7, 0xa1, 0x9f, 0x87, 0x82, 0x6a, 0xb6, 0xdb, - 0x6c, 0xae, 0x65, 0x16, 0x6d, 0xf0, 0x96, 0x1a, 0x5a, 0x04, 0xfe, 0x55, 0xcd, 0x25, 0xef, 0xeb, - 0x9a, 0x4b, 0x12, 0xff, 0xba, 0x66, 0x20, 0xac, 0x37, 0xef, 0x09, 0x63, 0x86, 0xb0, 0x2f, 0xc2, - 0x78, 0xf6, 0x75, 0x22, 0x8c, 0x59, 0x54, 0x3a, 0xef, 0x79, 0x7f, 0x24, 0xf0, 0x10, 0x9e, 0xa7, - 0xa6, 0x79, 0xd0, 0xf3, 0x22, 0x83, 0xcb, 0xc1, 0x1b, 0x26, 0xd3, 0x5f, 0x9c, 0x2e, 0xa6, 0x24, - 0xef, 0x8a, 0xc9, 0xa8, 0x99, 0x23, 0xf1, 0xe5, 0x66, 0x8e, 0x6b, 0x30, 0xd7, 0xb5, 0xf0, 0x1e, - 0x76, 0xd4, 0x7d, 0xd9, 0xe8, 0x75, 0xf8, 0x01, 0xa2, 0xac, 0x4b, 0xdb, 0xea, 0x75, 0xd0, 0x5d, - 0x28, 0x7a, 0x59, 0x38, 0xc6, 0x77, 0xaf, 0x20, 0x73, 0xe9, 0x7c, 0x45, 0x40, 0xfc, 0xdf, 0x02, - 0x2c, 0x84, 0xea, 0xc4, 0xc7, 0xd4, 0x3a, 0x64, 0x7d, 0x73, 0x62, 0x97, 0x84, 0x73, 0xc6, 0xc9, - 0x06, 0x99, 0x91, 0x0c, 0x97, 0xdc, 0xd7, 0xd2, 0xef, 0x0f, 0xf8, 0x62, 0x13, 0xe7, 0x14, 0x7b, - 0xd1, 0x97, 0xb3, 0x16, 0x78, 0x81, 0x37, 0xc8, 0x92, 0x13, 0x0d, 0x32, 0xf1, 0x37, 0x05, 0x28, - 0xd2, 0x17, 0x3c, 0xc6, 0x58, 0x8b, 0xc5, 0xba, 0xb9, 0xf1, 0xe7, 0x89, 0xc9, 0x8f, 0x08, 0x85, - 0xbe, 0x99, 0x92, 0x0c, 0x7f, 0x33, 0x45, 0xfc, 0x81, 0x00, 0x79, 0xaf, 0x84, 0xec, 0x3b, 0x85, - 0x23, 0x2e, 0x32, 0x7d, 0xbd, 0x6f, 0xf1, 0xb9, 0x17, 0xae, 0x4c, 0xf4, 0xe9, 0xc4, 0xe0, 0x85, - 0x2b, 0xec, 0x1b, 0x72, 0x7f, 0xdb, 0xed, 0x39, 0xa4, 0x88, 0x15, 0xff, 0x62, 0x8d, 0xd7, 0x38, - 0x2d, 0x25, 0xd1, 0x4f, 0xbc, 0x9a, 0xed, 0x43, 0x76, 0x47, 0xce, 0x44, 0x66, 0x0f, 0xf1, 0xc0, - 0x35, 0xe0, 0xab, 0x81, 0x5a, 0xb3, 0x41, 0x3f, 0xfe, 0xca, 0x7e, 0xdb, 0xe2, 0xe3, 0x80, 0x02, - 0x69, 0xe3, 0x13, 0x2d, 0x4d, 0x64, 0x8a, 0x5d, 0x2d, 0xb1, 0xbe, 0xf2, 0x87, 0xc1, 0x96, 0xa8, - 0x1e, 0x12, 0x14, 0xf8, 0x10, 0x92, 0x87, 0x4a, 0x7b, 0x54, 0x04, 0x58, 0xa8, 0xe5, 0x24, 0x92, - 0x1b, 0x3d, 0x0e, 0xdd, 0x47, 0x92, 0x18, 0x8e, 0x58, 0x06, 0x55, 0x1a, 0xba, 0xb7, 0xe4, 0x83, - 0x70, 0x5f, 0x1f, 0xf9, 0xfa, 0x60, 0xa7, 0xff, 0x28, 0xf5, 0xc3, 0x1f, 0x2c, 0x0a, 0xe2, 0xc7, - 0x80, 0x24, 0x6c, 0x63, 0xe7, 0x79, 0xcf, 0xb4, 0xfc, 0xbb, 0x5d, 0x6e, 0xf5, 0x7d, 0x44, 0x66, - 0x7a, 0x35, 0x7b, 0x16, 0x75, 0xfd, 0xd3, 0x45, 0x58, 0x08, 0x71, 0x33, 0x63, 0x21, 0x7e, 0x00, - 0x6f, 0x3c, 0x31, 0x6d, 0x5b, 0xef, 0x12, 0xe8, 0x4b, 0x47, 0x25, 0x99, 0x5a, 0x3c, 0xf3, 0x98, - 0xee, 0xd2, 0x45, 0x10, 0x83, 0x99, 0x91, 0x8c, 0xe4, 0x3d, 0x8b, 0xbf, 0x27, 0xc0, 0xe5, 0x41, - 0x4e, 0xa6, 0xe5, 0xa8, 0xc3, 0x9d, 0xb3, 0xaa, 0xe9, 0xdf, 0x05, 0x38, 0xbe, 0xb7, 0xba, 0xd9, - 0x89, 0x8b, 0xca, 0xdf, 0x29, 0x77, 0x14, 0x6a, 0x3e, 0xf8, 0x41, 0xf3, 0x3c, 0x27, 0x6f, 0x32, - 0xaa, 0x6f, 0x49, 0x52, 0x93, 0x59, 0x92, 0x26, 0x14, 0xd6, 0x4d, 0xdd, 0x20, 0x9e, 0xb0, 0x5b, - 0xdf, 0x15, 0xc8, 0xef, 0xea, 0x86, 0x62, 0x9d, 0xc8, 0x6e, 0xe0, 0xa1, 0x30, 0x2e, 0xf0, 0x50, - 0xca, 0x31, 0x0e, 0xfe, 0x28, 0xfe, 0x48, 0x80, 0xa2, 0x2f, 0x96, 0x5b, 0xe4, 0x77, 0x00, 0xd4, - 0x76, 0xcf, 0x76, 0xb0, 0xe5, 0xb6, 0xd2, 0x1c, 0x0b, 0xd4, 0xaf, 0x30, 0x6a, 0x7d, 0x4d, 0xca, - 0xf0, 0x0c, 0x75, 0x0d, 0x5d, 0x0f, 0xdf, 0x83, 0x31, 0xbd, 0x0a, 0x67, 0x03, 0xb7, 0x5f, 0x90, - 0x66, 0xb7, 0x1d, 0xd3, 0xf2, 0x30, 0x17, 0x6f, 0x76, 0xf7, 0xda, 0x7d, 0x7a, 0x7c, 0x9b, 0xe4, - 0x5b, 0x81, 0x3c, 0x71, 0x17, 0x0e, 0xb1, 0x57, 0xa5, 0xd4, 0xf8, 0x2a, 0x31, 0x0e, 0xb7, 0x4a, - 0xbf, 0x43, 0x9c, 0x5c, 0xd6, 0x1a, 0x5e, 0x0b, 0x8f, 0xb0, 0x68, 0x3f, 0x15, 0xbc, 0xbe, 0x72, - 0xb2, 0x0b, 0x3e, 0xa9, 0x27, 0xf3, 0x2d, 0x48, 0xbb, 0x1f, 0xcf, 0xe6, 0x03, 0xe4, 0x8d, 0x25, - 0xf6, 0x75, 0xed, 0x25, 0xf7, 0xeb, 0xda, 0x4b, 0x6b, 0x3c, 0x03, 0x33, 0xe3, 0xdf, 0xff, 0xaf, - 0x8b, 0x82, 0xe4, 0x31, 0xdd, 0x6b, 0x90, 0x1e, 0x3e, 0x30, 0x0b, 0xa3, 0x3c, 0x40, 0xe0, 0x7b, - 0x43, 0xfc, 0x7b, 0xca, 0x2b, 0x6b, 0xf2, 0xce, 0x56, 0xe5, 0xd9, 0xe6, 0x66, 0xbd, 0xd9, 0xac, - 0xae, 0x15, 0x05, 0x54, 0x84, 0xb9, 0xd0, 0xd7, 0x8a, 0x12, 0xec, 0x0b, 0xcb, 0xf7, 0x7e, 0x0a, - 0xc0, 0xff, 0xf0, 0x19, 0x91, 0xb5, 0x51, 0xfd, 0x54, 0x7e, 0xb1, 0xf2, 0x74, 0xa7, 0xda, 0x28, - 0x4e, 0x21, 0x04, 0xf9, 0xd5, 0x95, 0x66, 0xa5, 0x26, 0x4b, 0xd5, 0xc6, 0xf6, 0xb3, 0xad, 0x46, - 0xd5, 0xfd, 0x32, 0xf3, 0xbd, 0x35, 0x98, 0x0b, 0xde, 0x6c, 0x84, 0x16, 0xa0, 0x50, 0xa9, 0x55, - 0x2b, 0x1b, 0xf2, 0x8b, 0xfa, 0x8a, 0xfc, 0x7c, 0xa7, 0xba, 0x53, 0x2d, 0x4e, 0xd1, 0xa2, 0x51, - 0xe2, 0xe3, 0x9d, 0xa7, 0x4f, 0x8b, 0x02, 0x2a, 0x40, 0x96, 0x3d, 0xd3, 0x2f, 0x1b, 0x15, 0x13, - 0xf7, 0x36, 0x21, 0x1b, 0xb8, 0x02, 0x99, 0xbc, 0x6e, 0x7b, 0xa7, 0x51, 0x93, 0x9b, 0xf5, 0xcd, - 0x6a, 0xa3, 0xb9, 0xb2, 0xb9, 0xcd, 0x64, 0x50, 0xda, 0xca, 0xea, 0x33, 0xa9, 0x59, 0x14, 0xbc, - 0xe7, 0xe6, 0xb3, 0x9d, 0x4a, 0xcd, 0xad, 0x86, 0x98, 0x4a, 0x27, 0x8b, 0xc9, 0x7b, 0x7f, 0x55, - 0x80, 0xcb, 0x43, 0x6e, 0xf9, 0x41, 0x59, 0x98, 0xdd, 0x31, 0xe8, 0x7d, 0xb0, 0xc5, 0x29, 0x94, - 0x0b, 0x5c, 0xf4, 0x53, 0x14, 0x50, 0x9a, 0x5d, 0xb5, 0x52, 0x4c, 0xa0, 0x19, 0x48, 0x34, 0x1e, - 0x16, 0x93, 0xa4, 0xa4, 0x81, 0x7b, 0x72, 0x8a, 0x29, 0x94, 0xe1, 0x37, 0x74, 0x14, 0xa7, 0xd1, - 0x9c, 0x7f, 0x51, 0x46, 0x71, 0x86, 0x88, 0xf2, 0x2e, 0x9c, 0x28, 0xce, 0xde, 0xbb, 0x06, 0x81, - 0x43, 0xfd, 0x08, 0x60, 0xe6, 0xa9, 0xe2, 0x60, 0xdb, 0x29, 0x4e, 0xa1, 0x59, 0x48, 0xae, 0xb4, - 0xdb, 0x45, 0xe1, 0xc1, 0xbf, 0x4c, 0x41, 0xda, 0xfd, 0x82, 0x0f, 0x7a, 0x0a, 0xd3, 0x6c, 0xe9, - 0x7b, 0x71, 0x38, 0x32, 0xa0, 0x83, 0xb7, 0x7c, 0x75, 0x1c, 0x74, 0x10, 0xa7, 0xd0, 0x5f, 0x86, - 0x6c, 0xc0, 0x63, 0x42, 0x43, 0x97, 0xef, 0x42, 0x5e, 0x62, 0xf9, 0xd6, 0xb8, 0x6c, 0x9e, 0xfc, - 0x97, 0x90, 0xf1, 0x2c, 0x38, 0xba, 0x3e, 0xca, 0xbe, 0xbb, 0xb2, 0x47, 0x4f, 0x02, 0x64, 0xac, - 0x89, 0x53, 0xef, 0x09, 0xc8, 0x02, 0x34, 0x68, 0x6c, 0x51, 0x54, 0x40, 0xc5, 0x50, 0x6b, 0x5e, - 0xbe, 0x37, 0x51, 0x6e, 0xff, 0x9d, 0x44, 0x59, 0xfe, 0x8c, 0x11, 0xad, 0xac, 0x81, 0xf9, 0x28, - 0x5a, 0x59, 0x11, 0x13, 0xcf, 0x14, 0x7a, 0x0e, 0x29, 0x62, 0x29, 0x51, 0x94, 0x0f, 0xd9, 0x67, - 0x99, 0xcb, 0xd7, 0x47, 0xe6, 0x71, 0x45, 0xae, 0xde, 0xfd, 0xe1, 0x9f, 0x5e, 0x99, 0xfa, 0xe1, - 0xd9, 0x15, 0xe1, 0x47, 0x67, 0x57, 0x84, 0x3f, 0x3e, 0xbb, 0x22, 0xfc, 0xc9, 0xd9, 0x15, 0xe1, - 0x7b, 0x3f, 0xbe, 0x32, 0xf5, 0xa3, 0x1f, 0x5f, 0x99, 0xfa, 0xe3, 0x1f, 0x5f, 0x99, 0xfa, 0x6c, - 0x96, 0x73, 0xef, 0xce, 0x50, 0xa3, 0xf2, 0xf0, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x1e, 0x03, - 0x54, 0xaa, 0xed, 0x80, 0x00, 0x00, + 0xfb, 0x8a, 0x37, 0x80, 0x07, 0x66, 0xf1, 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x99, 0xbd, 0x5c, 0x57, + 0x81, 0x16, 0xdb, 0x1e, 0x0f, 0x2f, 0xfc, 0x9e, 0xc2, 0xc5, 0x15, 0x3c, 0x2f, 0x81, 0x89, 0x10, + 0x7f, 0x20, 0xc0, 0x5b, 0x91, 0xad, 0x16, 0xe7, 0x44, 0xf6, 0x09, 0xa4, 0x68, 0xe5, 0x13, 0xe7, + 0xac, 0x3c, 0xe5, 0x12, 0xbf, 0x9b, 0xe0, 0x63, 0x5c, 0xc2, 0x6d, 0x93, 0x28, 0x36, 0xf6, 0x55, + 0xb6, 0xe7, 0x90, 0x3b, 0x34, 0x1d, 0xe2, 0x9b, 0xf0, 0x66, 0x4f, 0x9c, 0xbb, 0xd9, 0xe7, 0xa8, + 0x00, 0xb7, 0xc5, 0x5f, 0xc2, 0xbc, 0x61, 0x1a, 0x72, 0x58, 0xe8, 0xf9, 0xfb, 0x52, 0xc1, 0x30, + 0x8d, 0x97, 0x01, 0xb9, 0x9e, 0x9d, 0xe9, 0xd3, 0x44, 0x9c, 0x76, 0xe6, 0x7b, 0x02, 0x2c, 0x78, + 0x6e, 0x53, 0xcc, 0x1e, 0xf4, 0x07, 0x90, 0x34, 0xcc, 0xa3, 0xf3, 0xac, 0x62, 0x92, 0xfc, 0x64, + 0xd6, 0x0b, 0x97, 0x28, 0xce, 0xfa, 0xfe, 0xeb, 0x04, 0x64, 0x9e, 0x56, 0xe2, 0xac, 0xe5, 0x27, + 0x7c, 0x85, 0x9c, 0xb5, 0x77, 0x54, 0x6f, 0xf7, 0xde, 0xb7, 0xf4, 0xb4, 0xb2, 0x81, 0x4f, 0xdc, + 0xde, 0x4e, 0xb8, 0xd0, 0x0a, 0x64, 0x9c, 0x7d, 0x0b, 0xdb, 0xfb, 0x66, 0x5b, 0x3b, 0x8f, 0x9b, + 0xe3, 0x73, 0x95, 0x31, 0x4c, 0x53, 0xb9, 0x6e, 0x34, 0x86, 0x10, 0x11, 0x8d, 0x41, 0x5e, 0xe3, + 0x79, 0x8a, 0x89, 0xf3, 0xbc, 0x26, 0xe0, 0x22, 0x4e, 0x17, 0x67, 0xc4, 0x17, 0x00, 0xa4, 0x3a, + 0x71, 0x36, 0xc9, 0xaf, 0x24, 0x21, 0xbf, 0xdd, 0xb3, 0xf7, 0x63, 0xee, 0x7d, 0x15, 0x80, 0x6e, + 0xcf, 0xa6, 0x10, 0xe4, 0xd8, 0xe0, 0x75, 0x1e, 0x13, 0xe8, 0xe1, 0x56, 0x9a, 0xf1, 0x35, 0x8f, + 0x0d, 0x54, 0xe3, 0x42, 0xb0, 0xec, 0x47, 0x8b, 0xdc, 0x18, 0x05, 0x56, 0x9b, 0xc7, 0xc6, 0x26, + 0xf6, 0x50, 0x2a, 0x93, 0x84, 0x89, 0xa4, 0x4f, 0x60, 0x96, 0x3c, 0xc8, 0x8e, 0x79, 0x9e, 0x66, + 0x9e, 0x21, 0x3c, 0x4d, 0x13, 0x3d, 0x86, 0x0c, 0xe3, 0x26, 0xb3, 0xdf, 0x0c, 0x9d, 0xfd, 0xa2, + 0xea, 0xc2, 0xd5, 0x48, 0xe7, 0xbd, 0x34, 0x65, 0x25, 0x73, 0xdd, 0x05, 0x98, 0xde, 0x33, 0x2d, + 0xd5, 0xfd, 0xde, 0xcb, 0x1e, 0x58, 0x7b, 0xae, 0xa7, 0xd2, 0xe9, 0x62, 0x66, 0x3d, 0x95, 0xce, + 0x14, 0x41, 0xfc, 0x0d, 0x01, 0x0a, 0x5e, 0x43, 0xc4, 0x39, 0x21, 0x54, 0x42, 0x5a, 0x3c, 0x7f, + 0x53, 0x10, 0x05, 0x8a, 0xff, 0x86, 0x7a, 0x44, 0xaa, 0x79, 0x48, 0x5b, 0x26, 0xce, 0x9e, 0xf2, + 0x98, 0xc5, 0x02, 0x25, 0xce, 0xdb, 0xba, 0x34, 0x2c, 0xe8, 0x01, 0x5c, 0xd0, 0x3b, 0xc4, 0x9e, + 0xeb, 0x4e, 0xfb, 0x84, 0xc3, 0x36, 0x07, 0xbb, 0x1f, 0x96, 0x17, 0xfc, 0xb4, 0x8a, 0x9b, 0x24, + 0xfe, 0x5d, 0xba, 0x00, 0xee, 0xd7, 0x24, 0x4e, 0x55, 0xd7, 0x21, 0x67, 0x31, 0xd1, 0xc4, 0xad, + 0x39, 0xa7, 0xb6, 0xe7, 0x3c, 0x56, 0xa2, 0xf0, 0xdf, 0x4a, 0x40, 0xe1, 0x45, 0x0f, 0x5b, 0x27, + 0x5f, 0x27, 0x75, 0xdf, 0x86, 0xc2, 0x91, 0xa2, 0x3b, 0xf2, 0x9e, 0x69, 0xc9, 0xbd, 0xae, 0xa6, + 0x38, 0x6e, 0x40, 0x4a, 0x8e, 0x90, 0x9f, 0x98, 0xd6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, 0xc0, 0x30, + 0x8f, 0x0c, 0x99, 0x90, 0x29, 0x50, 0x3e, 0x36, 0xf8, 0xaa, 0xf4, 0xea, 0x87, 0xff, 0xe9, 0x74, + 0xf1, 0xd1, 0x44, 0x61, 0x66, 0x34, 0xa4, 0xae, 0xd7, 0xd3, 0xb5, 0xa5, 0x9d, 0x9d, 0xfa, 0x9a, + 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x7e, 0x02, 0x8a, 0xbe, 0x8e, + 0xe2, 0x6c, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x33, 0x02, 0x67, 0x24, 0x66, + 0xe7, 0x73, 0x98, 0x0b, 0x69, 0x20, 0xf9, 0xd5, 0x34, 0x90, 0x3d, 0xf2, 0x2b, 0x8f, 0xee, 0xc3, + 0xbc, 0x73, 0x6c, 0xc8, 0x2c, 0xc0, 0x90, 0x05, 0xa5, 0xb8, 0xf1, 0x12, 0x05, 0x87, 0xe8, 0x83, + 0xd0, 0x69, 0x40, 0x8a, 0x2d, 0xfe, 0x9e, 0x00, 0x88, 0x2a, 0xaa, 0xce, 0x3e, 0x1b, 0x7c, 0x5d, + 0xfa, 0xd3, 0x5d, 0x28, 0xd2, 0x90, 0x4d, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, 0x8b, + 0x77, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0x4b, 0xb0, 0x10, 0xaa, 0x40, 0x9c, + 0x8d, 0x7d, 0x1d, 0xe6, 0xf6, 0xd8, 0x57, 0x5a, 0x2a, 0x9c, 0xaf, 0x38, 0x66, 0x29, 0x8d, 0xbd, + 0x4f, 0xfc, 0xef, 0x09, 0xb8, 0x20, 0x61, 0xdb, 0x6c, 0x1f, 0xe2, 0xf8, 0x55, 0x58, 0x03, 0xfe, + 0x39, 0x47, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x9b, 0xe6, 0xc2, 0xcb, 0xf6, 0x37, 0x47, 0xf7, + 0xd8, 0xc1, 0x85, 0x7a, 0xbe, 0xec, 0x97, 0x0a, 0x2d, 0xfb, 0x99, 0x50, 0xd0, 0x5b, 0x86, 0x49, + 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x0b, 0x86, 0x96, 0x46, 0x15, 0xb2, 0xce, 0x58, 0x1a, + 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xd5, 0x4b, 0xa4, 0xbc, 0x67, 0xa7, 0x8b, 0xf9, 0x50, + 0x9a, 0x2d, 0xe5, 0x75, 0xef, 0x99, 0x48, 0x17, 0xbf, 0x05, 0x17, 0xfb, 0x94, 0x1d, 0xa7, 0xc7, + 0xf3, 0x2f, 0x93, 0x70, 0x25, 0x2c, 0x3e, 0x6e, 0x88, 0xf3, 0x75, 0x6f, 0xd0, 0x1a, 0xe4, 0x3a, + 0xba, 0xf1, 0x66, 0xab, 0x97, 0x73, 0x1d, 0xdd, 0xf0, 0x57, 0x8a, 0x23, 0xba, 0xc6, 0xcc, 0x4f, + 0xb4, 0x6b, 0x28, 0x50, 0x8e, 0x6a, 0xbb, 0x38, 0xfb, 0xc7, 0x77, 0x05, 0x98, 0x8b, 0x7b, 0x59, + 0xee, 0xcd, 0x02, 0xe5, 0xc4, 0x26, 0xe4, 0x7e, 0x02, 0xeb, 0x78, 0xbf, 0x25, 0x00, 0x6a, 0x5a, + 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x6a, 0x5e, 0x80, 0x69, 0xdd, 0xd0, 0xf0, 0x31, + 0xad, 0x66, 0x4a, 0x62, 0x0f, 0xa1, 0xaf, 0x93, 0xc9, 0x89, 0xbe, 0x4e, 0x8a, 0x9f, 0xc3, 0x42, + 0xa8, 0x88, 0x71, 0xd6, 0xff, 0xbf, 0x25, 0x60, 0x81, 0x57, 0x24, 0xf6, 0x15, 0xcc, 0x6f, 0xc0, + 0x74, 0x9b, 0xc8, 0x1c, 0xd1, 0xce, 0xf4, 0x9d, 0x6e, 0x3b, 0xd3, 0xcc, 0xe8, 0x67, 0x01, 0xba, + 0x16, 0x3e, 0x94, 0x19, 0x6b, 0x72, 0x22, 0xd6, 0x0c, 0xe1, 0xa0, 0x04, 0xf4, 0x7d, 0x01, 0x0a, + 0x64, 0x40, 0x77, 0x2d, 0xb3, 0x6b, 0xda, 0xc4, 0x67, 0xb1, 0x27, 0x83, 0x39, 0x2f, 0xce, 0x4e, + 0x17, 0x73, 0x9b, 0xba, 0xb1, 0xcd, 0x19, 0x9b, 0x8d, 0x89, 0xf7, 0x00, 0xb8, 0x3b, 0x21, 0x96, + 0x2a, 0x6d, 0x53, 0x3d, 0xf0, 0xbf, 0xb7, 0x11, 0xcb, 0xe2, 0x89, 0xb3, 0xc5, 0xff, 0x20, 0xc0, + 0x85, 0x9f, 0xd8, 0x72, 0xf1, 0xff, 0x0f, 0x65, 0x8b, 0x2f, 0xa1, 0x48, 0x7f, 0xd4, 0x8d, 0x3d, + 0x33, 0xce, 0x85, 0xfb, 0x5f, 0x15, 0x60, 0x3e, 0x20, 0x38, 0x4e, 0x07, 0xe7, 0x8d, 0xf4, 0x24, + 0xfe, 0x3c, 0x71, 0x79, 0x82, 0xa3, 0x24, 0xce, 0x31, 0xf8, 0xcf, 0x13, 0x70, 0xa9, 0xc2, 0x3e, + 0x73, 0xbb, 0x71, 0x1f, 0x71, 0xf6, 0x8c, 0x12, 0xcc, 0x1e, 0x62, 0xcb, 0xd6, 0x4d, 0x36, 0xfd, + 0xe6, 0x24, 0xf7, 0x11, 0x95, 0x21, 0x6d, 0x1b, 0x4a, 0xd7, 0xde, 0x37, 0xdd, 0xef, 0x7b, 0xde, + 0xb3, 0x17, 0xa3, 0x32, 0xfd, 0xe6, 0x31, 0x2a, 0x33, 0xa3, 0x63, 0x54, 0x66, 0xbf, 0x42, 0x8c, + 0x0a, 0xff, 0x98, 0xf6, 0x6f, 0x05, 0xb8, 0x3c, 0xa0, 0xb9, 0x38, 0x7b, 0xcb, 0x77, 0x20, 0xab, + 0x72, 0xc1, 0xc4, 0x60, 0xb3, 0x2f, 0x85, 0x75, 0x92, 0xed, 0x0d, 0x71, 0xcb, 0xd9, 0xe9, 0x22, + 0xb8, 0x45, 0xad, 0xaf, 0x71, 0xe5, 0x90, 0xdf, 0x9a, 0xf8, 0x4b, 0x39, 0x28, 0x54, 0x8f, 0xd9, + 0x2a, 0x79, 0x83, 0xb9, 0x09, 0xe8, 0x09, 0xa4, 0xbb, 0x96, 0x79, 0xa8, 0xbb, 0xd5, 0xc8, 0x87, + 0x02, 0x14, 0xdc, 0x6a, 0xf4, 0x71, 0x6d, 0x73, 0x0e, 0xc9, 0xe3, 0x45, 0x4d, 0xc8, 0x3c, 0x33, + 0x55, 0xa5, 0xfd, 0x44, 0x6f, 0xbb, 0x3d, 0xff, 0xfd, 0xf1, 0x82, 0x96, 0x3c, 0x9e, 0x6d, 0xc5, + 0xd9, 0x77, 0x1b, 0xc1, 0x23, 0xa2, 0x3a, 0xa4, 0x6b, 0x8e, 0xd3, 0x25, 0x89, 0xdc, 0x76, 0xdc, + 0x99, 0x40, 0x28, 0x61, 0x71, 0xa3, 0x64, 0x5d, 0x76, 0xd4, 0x84, 0xf9, 0xa7, 0x74, 0xcf, 0x57, + 0xa5, 0x6d, 0xf6, 0xb4, 0x8a, 0x69, 0xec, 0xe9, 0x2d, 0x6e, 0xb7, 0x6f, 0x4f, 0x20, 0xf3, 0x69, + 0xa5, 0x21, 0x0d, 0x0a, 0x40, 0x2b, 0x90, 0x6e, 0x3c, 0xe2, 0xc2, 0x98, 0x5f, 0x77, 0x6b, 0x02, + 0x61, 0x8d, 0x47, 0x92, 0xc7, 0x86, 0xd6, 0x21, 0xbb, 0xf2, 0x45, 0xcf, 0xc2, 0x5c, 0xca, 0xcc, + 0xd0, 0xe8, 0x88, 0x7e, 0x29, 0x94, 0x4b, 0x0a, 0x32, 0xa3, 0x06, 0xe4, 0x5f, 0x99, 0xd6, 0x41, + 0xdb, 0x54, 0xdc, 0x1a, 0xce, 0x52, 0x71, 0x3f, 0x35, 0x81, 0x38, 0x97, 0x51, 0xea, 0x13, 0x81, + 0xbe, 0x05, 0x05, 0xd2, 0x18, 0x4d, 0x65, 0xb7, 0xed, 0x16, 0x32, 0x4d, 0xa5, 0xbe, 0x3b, 0x81, + 0x54, 0x8f, 0xd3, 0x5d, 0xf8, 0xef, 0x13, 0x55, 0x96, 0x20, 0x17, 0xea, 0x04, 0x08, 0x41, 0xaa, + 0x4b, 0xda, 0x5b, 0xa0, 0xf1, 0x4b, 0xf4, 0x37, 0x7a, 0x0f, 0x66, 0x0d, 0x53, 0xc3, 0xee, 0x08, + 0xc9, 0xad, 0x5e, 0x38, 0x3b, 0x5d, 0x9c, 0xd9, 0x32, 0x35, 0xe6, 0xd1, 0xf0, 0x5f, 0xd2, 0x0c, + 0xc9, 0x54, 0xd7, 0xca, 0xd7, 0x20, 0x45, 0xda, 0x9d, 0x18, 0xa6, 0x5d, 0xc5, 0xc6, 0x3b, 0x96, + 0xce, 0xa5, 0xb9, 0x8f, 0xe5, 0x7f, 0x94, 0x80, 0x44, 0xe3, 0x11, 0xf1, 0xd9, 0x77, 0x7b, 0xea, + 0x01, 0x76, 0x78, 0x3a, 0x7f, 0xa2, 0xbe, 0xbc, 0x85, 0xf7, 0x74, 0xe6, 0x5a, 0x65, 0x24, 0xfe, + 0x84, 0xde, 0x01, 0x50, 0x54, 0x15, 0xdb, 0xb6, 0xec, 0xee, 0x05, 0xcc, 0x48, 0x19, 0x46, 0xd9, + 0xc0, 0x27, 0x84, 0xcd, 0xc6, 0xaa, 0x85, 0x1d, 0x37, 0xf8, 0x8a, 0x3d, 0x11, 0x36, 0x07, 0x77, + 0xba, 0xb2, 0x63, 0x1e, 0x60, 0x83, 0xf6, 0x93, 0x0c, 0x31, 0x35, 0x9d, 0x6e, 0x93, 0x10, 0x88, + 0x95, 0xc4, 0x86, 0xe6, 0x9b, 0xb4, 0x8c, 0xe4, 0x3d, 0x13, 0x91, 0x16, 0x6e, 0xe9, 0x7c, 0x53, + 0x5b, 0x46, 0xe2, 0x4f, 0x44, 0x4b, 0x4a, 0xcf, 0xd9, 0xa7, 0x2d, 0x91, 0x91, 0xe8, 0x6f, 0x74, + 0x1b, 0x0a, 0x2c, 0x5e, 0x53, 0xc6, 0x86, 0x2a, 0x53, 0xe3, 0x9a, 0xa1, 0xc9, 0x39, 0x46, 0xae, + 0x1a, 0x2a, 0x31, 0xa5, 0xe8, 0x11, 0x70, 0x82, 0x7c, 0xd0, 0xb1, 0x89, 0x4e, 0x81, 0xe4, 0x5a, + 0x2d, 0x9c, 0x9d, 0x2e, 0x66, 0x1b, 0x34, 0x61, 0x63, 0xb3, 0x51, 0x5f, 0x93, 0xb2, 0x2c, 0xd7, + 0x46, 0xc7, 0xae, 0x6b, 0xe5, 0x5f, 0x13, 0x20, 0xf9, 0xb4, 0xd2, 0x38, 0xb7, 0xca, 0xdc, 0x82, + 0x26, 0x03, 0x05, 0xbd, 0x03, 0x85, 0x5d, 0xbd, 0xdd, 0xd6, 0x8d, 0x16, 0xf1, 0xa2, 0xbe, 0x83, + 0x55, 0x57, 0x61, 0x79, 0x4e, 0xde, 0x66, 0x54, 0x74, 0x0d, 0xb2, 0xaa, 0x85, 0x35, 0x6c, 0x38, + 0xba, 0xd2, 0xb6, 0xb9, 0xe6, 0x82, 0xa4, 0xf2, 0x2f, 0x0a, 0x30, 0x4d, 0x47, 0x00, 0x7a, 0x1b, + 0x32, 0xaa, 0x69, 0x38, 0x8a, 0x6e, 0x70, 0x53, 0x96, 0x91, 0x7c, 0xc2, 0xd0, 0xe2, 0x5d, 0x87, + 0x39, 0x45, 0x55, 0xcd, 0x9e, 0xe1, 0xc8, 0x86, 0xd2, 0xc1, 0xbc, 0x98, 0x59, 0x4e, 0xdb, 0x52, + 0x3a, 0x18, 0x2d, 0x82, 0xfb, 0xe8, 0x6d, 0xf1, 0xcc, 0x48, 0xc0, 0x49, 0x1b, 0xf8, 0xa4, 0xfc, + 0xfb, 0x02, 0xa4, 0xdd, 0x31, 0x43, 0x8a, 0xd1, 0xc2, 0x06, 0x0b, 0x2a, 0x77, 0x8b, 0xe1, 0x11, + 0xfa, 0xa7, 0xca, 0x8c, 0x3f, 0x55, 0x5e, 0x80, 0x69, 0x87, 0x0c, 0x0b, 0x5e, 0x02, 0xf6, 0x40, + 0xd7, 0xb3, 0xdb, 0x4a, 0x8b, 0x2d, 0xe7, 0x65, 0x24, 0xf6, 0x40, 0x2a, 0xc3, 0xc3, 0x7e, 0x99, + 0x46, 0xf8, 0x13, 0x29, 0x29, 0x0b, 0x4e, 0xdd, 0xc5, 0x2d, 0xdd, 0xa0, 0x7d, 0x29, 0x29, 0x01, + 0x25, 0xad, 0x12, 0x0a, 0x7a, 0x0b, 0x32, 0x2c, 0x03, 0x36, 0x34, 0xda, 0xa1, 0x92, 0x52, 0x9a, + 0x12, 0xaa, 0x86, 0x56, 0xc6, 0x90, 0xf1, 0x06, 0x27, 0x69, 0xb6, 0x9e, 0xed, 0x29, 0x92, 0xfe, + 0x46, 0xef, 0xc3, 0x85, 0xd7, 0x3d, 0xa5, 0xad, 0xef, 0xd1, 0x95, 0x3a, 0x1a, 0x75, 0x4f, 0x75, + 0xc6, 0x6a, 0x82, 0xbc, 0x34, 0x2a, 0x81, 0xaa, 0xce, 0x1d, 0xcb, 0x49, 0x7f, 0x2c, 0x8b, 0xbf, + 0x23, 0xc0, 0x3c, 0x8b, 0x3c, 0x62, 0x01, 0xb3, 0xf1, 0xf9, 0x21, 0x1f, 0x43, 0x46, 0x53, 0x1c, + 0x85, 0x6d, 0x5a, 0x4d, 0x8c, 0xdc, 0xb4, 0xea, 0x6d, 0xa2, 0x50, 0x1c, 0x85, 0x6e, 0x5c, 0x45, + 0x90, 0x22, 0xbf, 0xd9, 0xfe, 0x5e, 0x89, 0xfe, 0x16, 0x3f, 0x03, 0x14, 0x2c, 0x68, 0x9c, 0x1e, + 0xd9, 0x3d, 0xb8, 0x48, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, + 0xb5, 0x51, 0x31, 0xf0, 0x61, 0x8b, 0x7e, 0xcf, 0x12, 0x7f, 0x77, 0x06, 0x72, 0xd5, 0xe3, 0xae, + 0x69, 0xc5, 0xba, 0x0c, 0xb6, 0x0a, 0xb3, 0x7c, 0xa5, 0x60, 0xc4, 0xb7, 0xeb, 0x3e, 0x63, 0xee, + 0x7e, 0xb8, 0xe7, 0x8c, 0x68, 0x15, 0x80, 0x05, 0xb5, 0xd2, 0xc0, 0xa5, 0xe4, 0x39, 0x3e, 0xb5, + 0x51, 0x36, 0xba, 0x81, 0x63, 0x0b, 0xb2, 0x9d, 0x43, 0x55, 0x95, 0xf7, 0xf4, 0xb6, 0xc3, 0x63, + 0x03, 0xa3, 0xc3, 0xd8, 0x37, 0x5f, 0x56, 0x2a, 0x4f, 0x68, 0x26, 0x16, 0xa6, 0xe7, 0x3f, 0x4b, + 0x40, 0x24, 0xb0, 0xdf, 0xe8, 0x5d, 0xe0, 0x9b, 0x89, 0x64, 0xdb, 0xdd, 0x37, 0xb8, 0x9a, 0x3b, + 0x3b, 0x5d, 0xcc, 0x48, 0x94, 0xda, 0x68, 0x34, 0xa5, 0x0c, 0xcb, 0xd0, 0xb0, 0x1d, 0x74, 0x03, + 0x72, 0x66, 0x47, 0x77, 0x64, 0xd7, 0x49, 0xe2, 0x1e, 0xe5, 0x1c, 0x21, 0xba, 0x4e, 0xd4, 0x79, + 0xf6, 0x98, 0xcc, 0x4e, 0xbc, 0xc7, 0x04, 0xfd, 0x55, 0x01, 0x2e, 0x71, 0x45, 0xca, 0xbb, 0x34, + 0x0e, 0x5f, 0x69, 0xeb, 0xce, 0x89, 0x7c, 0x70, 0x58, 0x4a, 0x53, 0xbf, 0xf5, 0x67, 0x22, 0x1b, + 0x24, 0xd0, 0x0f, 0x96, 0xdc, 0x66, 0x39, 0x79, 0xc6, 0x99, 0x37, 0x0e, 0xab, 0x86, 0x63, 0x9d, + 0xac, 0x5e, 0x3e, 0x3b, 0x5d, 0x5c, 0x18, 0x4c, 0x7d, 0x29, 0x2d, 0xd8, 0x83, 0x2c, 0xa8, 0x06, + 0x80, 0xbd, 0x7e, 0x48, 0x67, 0x8c, 0x68, 0xff, 0x23, 0xb2, 0xc3, 0x4a, 0x01, 0x5e, 0x74, 0x17, + 0x8a, 0x7c, 0x4f, 0xcf, 0x9e, 0xde, 0xc6, 0xb2, 0xad, 0x7f, 0x81, 0xe9, 0xdc, 0x92, 0x94, 0xf2, + 0x8c, 0x4e, 0x44, 0x34, 0xf4, 0x2f, 0x70, 0xf9, 0x3b, 0x50, 0x1a, 0x56, 0xfa, 0xe0, 0x10, 0xc8, + 0xb0, 0x4f, 0xba, 0x1f, 0x85, 0xd7, 0x73, 0x26, 0xe8, 0xaa, 0x7c, 0x4d, 0xe7, 0xe3, 0xc4, 0x47, + 0x82, 0xf8, 0x0f, 0x12, 0x90, 0x5b, 0xed, 0xb5, 0x0f, 0x9e, 0x77, 0x1b, 0xbd, 0x4e, 0x47, 0xb1, + 0x4e, 0x88, 0x19, 0x64, 0x86, 0x82, 0x14, 0x50, 0x60, 0x66, 0x90, 0x5a, 0x02, 0xfd, 0x0b, 0x4c, + 0x26, 0xa7, 0xe0, 0x3e, 0x75, 0xb6, 0xcf, 0x80, 0xd6, 0x21, 0xb0, 0xf9, 0xdc, 0x3c, 0xb2, 0xd1, + 0x47, 0x50, 0x0a, 0x64, 0xa4, 0x8b, 0x2f, 0x32, 0x36, 0x1c, 0x4b, 0xc7, 0x6c, 0x01, 0x31, 0x29, + 0x05, 0x62, 0x7a, 0xea, 0x24, 0xb9, 0xca, 0x52, 0x51, 0x13, 0xe6, 0x48, 0xc6, 0x13, 0x99, 0x4e, + 0x21, 0xee, 0x02, 0xef, 0x83, 0x88, 0x6a, 0x85, 0xca, 0xbd, 0x44, 0xf5, 0x53, 0xa1, 0x3c, 0xf4, + 0xa7, 0x94, 0xc5, 0x3e, 0xa5, 0xfc, 0x29, 0x14, 0xfb, 0x33, 0x04, 0x75, 0x99, 0x62, 0xba, 0xbc, + 0x10, 0xd4, 0x65, 0x32, 0xa0, 0xa7, 0xf5, 0x54, 0x3a, 0x55, 0x9c, 0x16, 0xff, 0x24, 0x09, 0x79, + 0xb7, 0x9b, 0xc5, 0x09, 0x74, 0x56, 0x61, 0x9a, 0x74, 0x0a, 0x37, 0x02, 0xe5, 0xf6, 0x88, 0xde, + 0xcd, 0x23, 0xdb, 0x49, 0x67, 0x71, 0x41, 0x32, 0x65, 0x8d, 0xc3, 0xe0, 0x94, 0x7f, 0x31, 0x01, + 0x29, 0x8a, 0x2d, 0x1e, 0x40, 0x8a, 0x4e, 0x14, 0xc2, 0x24, 0x13, 0x05, 0xcd, 0xea, 0x4d, 0x67, + 0x89, 0x80, 0x6b, 0x4a, 0x7c, 0xbe, 0x7d, 0xe5, 0x83, 0x07, 0x0f, 0xa9, 0xb1, 0x99, 0x93, 0xf8, + 0x13, 0x5a, 0xa5, 0xa1, 0x51, 0xa6, 0xe5, 0x60, 0x8d, 0xfb, 0xf4, 0xd7, 0xc6, 0xb5, 0xaf, 0x3b, + 0x29, 0xb9, 0x7c, 0xe8, 0x0a, 0x24, 0x89, 0x15, 0x9b, 0x65, 0x51, 0x0e, 0x67, 0xa7, 0x8b, 0x49, + 0x62, 0xbf, 0x08, 0x0d, 0x2d, 0x43, 0x36, 0x6c, 0x32, 0x88, 0x07, 0x47, 0x0d, 0x63, 0x60, 0xb8, + 0x43, 0xdb, 0x1b, 0x5a, 0x0c, 0xcf, 0xf2, 0x36, 0xfe, 0xd3, 0x14, 0xe4, 0xea, 0x9d, 0xb8, 0xa7, + 0x94, 0x95, 0x70, 0x0b, 0x47, 0x01, 0xa1, 0xd0, 0x4b, 0x23, 0x1a, 0x38, 0x34, 0x83, 0x27, 0xcf, + 0x37, 0x83, 0xd7, 0x89, 0xa7, 0xcc, 0x0f, 0x9e, 0x48, 0x0e, 0xc1, 0x3c, 0xe1, 0xf7, 0x53, 0x3f, + 0x45, 0x22, 0x3c, 0xfe, 0x5e, 0x0f, 0x1a, 0xa9, 0xf2, 0x29, 0x75, 0xc8, 0x59, 0x2f, 0x9b, 0x99, + 0xbc, 0x97, 0xcd, 0x62, 0x43, 0xa3, 0x93, 0x5a, 0xd8, 0xa2, 0xce, 0xbe, 0xb9, 0x45, 0x2d, 0x3b, + 0xbc, 0xb3, 0x7e, 0x0c, 0x49, 0x4d, 0x77, 0x1b, 0x67, 0xf2, 0xa9, 0x9a, 0x30, 0x8d, 0xe9, 0xb5, + 0xa9, 0x60, 0xaf, 0x65, 0xbd, 0xa4, 0x5c, 0x07, 0xf0, 0x75, 0x83, 0xae, 0xc1, 0x8c, 0xd9, 0xd6, + 0xdc, 0xcd, 0x2e, 0xb9, 0xd5, 0xcc, 0xd9, 0xe9, 0xe2, 0xf4, 0xf3, 0xb6, 0x56, 0x5f, 0x93, 0xa6, + 0xcd, 0xb6, 0x56, 0xd7, 0xe8, 0xa9, 0x1f, 0xf8, 0x48, 0xf6, 0x22, 0xe1, 0xe6, 0xa4, 0x59, 0x03, + 0x1f, 0xad, 0x61, 0x5b, 0xe5, 0x1d, 0xee, 0x37, 0x05, 0xc8, 0xbb, 0xba, 0x8f, 0xd7, 0xa8, 0xa4, + 0xf5, 0x0e, 0x1f, 0x64, 0xc9, 0xf3, 0x0d, 0x32, 0x97, 0x8f, 0x6f, 0x2c, 0xfe, 0x65, 0x81, 0xc7, + 0x36, 0x37, 0x54, 0xc5, 0x21, 0x4e, 0x45, 0x8c, 0x03, 0xe3, 0x1e, 0x14, 0x2d, 0xc5, 0xd0, 0xcc, + 0x8e, 0xfe, 0x05, 0x66, 0x0b, 0xa1, 0x36, 0xff, 0xec, 0x59, 0xf0, 0xe8, 0x74, 0xd5, 0xcf, 0x16, + 0x7f, 0x3f, 0xc1, 0xe3, 0xa0, 0xbd, 0x62, 0xc4, 0xa9, 0xae, 0x6f, 0xc3, 0x7c, 0xff, 0x11, 0x2c, + 0xee, 0x68, 0x7d, 0x2f, 0x42, 0x5e, 0x54, 0x41, 0x58, 0x3c, 0xa3, 0x1b, 0x5c, 0xdf, 0x77, 0x1c, + 0x8b, 0x8d, 0x2a, 0x90, 0x0d, 0x9e, 0xec, 0x92, 0x9c, 0xf8, 0x64, 0x17, 0xb0, 0xbc, 0xf3, 0x5c, + 0xca, 0x3f, 0x07, 0xd3, 0x34, 0xf9, 0x0d, 0x4c, 0x34, 0x6f, 0xcd, 0x3f, 0x4e, 0xc0, 0x4d, 0x5a, + 0xfa, 0x97, 0xd8, 0xd2, 0xf7, 0x4e, 0xb6, 0x2d, 0xd3, 0xc1, 0xaa, 0x83, 0x35, 0x7f, 0xe5, 0x3c, + 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x2f, 0x38, 0x57, 0xc0, 0x99, 0xc7, 0x85, 0x36, 0xa0, 0xc0, 0x43, + 0x0b, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0x39, 0xcf, 0xec, 0x96, 0x63, 0xbc, 0x2b, 0x84, 0x75, + 0xc5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0x07, 0x12, 0x3d, 0xfd, 0x6a, 0x0b, 0x8a, 0x69, + 0x16, 0xdf, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, 0xa3, 0x00, 0xb7, 0xc6, 0xa8, + 0x38, 0xce, 0xae, 0x5b, 0x86, 0xf4, 0x21, 0x79, 0x91, 0xce, 0x75, 0x9c, 0x96, 0xbc, 0x67, 0xb4, + 0x09, 0xb9, 0x3d, 0x45, 0x6f, 0xfb, 0x5d, 0x7a, 0x78, 0x7c, 0x62, 0x74, 0x34, 0xee, 0x1c, 0x63, + 0x67, 0x7d, 0x58, 0xfc, 0xf5, 0x04, 0xcc, 0xaf, 0x68, 0x5a, 0xa3, 0xc1, 0x6d, 0x60, 0x7c, 0x3d, + 0xc5, 0x05, 0x99, 0x09, 0x1f, 0x64, 0xa2, 0xf7, 0x00, 0x69, 0xba, 0xcd, 0x0e, 0x3e, 0xb1, 0xf7, + 0x15, 0xcd, 0x3c, 0xf2, 0xe3, 0x32, 0xe6, 0xdd, 0x94, 0x86, 0x9b, 0x80, 0x1a, 0x40, 0xd1, 0x8e, + 0x6c, 0x3b, 0x8a, 0xf7, 0xdd, 0xe9, 0xd6, 0x44, 0x5b, 0xca, 0x18, 0x0c, 0xf2, 0x1e, 0xa5, 0x0c, + 0x91, 0x43, 0x7f, 0x12, 0xbf, 0x5d, 0x27, 0x55, 0x77, 0x64, 0xc5, 0x76, 0xf7, 0x0f, 0xb1, 0x23, + 0x57, 0xf2, 0x8c, 0xbe, 0x62, 0xb3, 0x6d, 0x41, 0x6c, 0xc3, 0x83, 0xaf, 0x9a, 0x38, 0x21, 0xf1, + 0xdf, 0x11, 0x20, 0x2f, 0xe1, 0x3d, 0x0b, 0xdb, 0xb1, 0x2e, 0x0a, 0x3c, 0x81, 0x39, 0x8b, 0x49, + 0x95, 0xf7, 0x2c, 0xb3, 0x73, 0x9e, 0x71, 0x95, 0xe5, 0x8c, 0x4f, 0x2c, 0xb3, 0xc3, 0x0d, 0xcb, + 0x4b, 0x28, 0x78, 0x65, 0x8c, 0xb3, 0xf2, 0xbf, 0x43, 0xb7, 0x4b, 0x33, 0xc1, 0x71, 0x07, 0x48, + 0xc4, 0xab, 0x01, 0xfa, 0xa1, 0x2a, 0x58, 0xd0, 0x38, 0xd5, 0xf0, 0x5f, 0x05, 0xc8, 0x37, 0x7a, + 0xbb, 0xec, 0x64, 0xad, 0xf8, 0x34, 0x50, 0x85, 0x4c, 0x1b, 0xef, 0x39, 0xf2, 0x1b, 0x85, 0xea, + 0xa7, 0x09, 0x2b, 0xdd, 0xa8, 0xf0, 0x14, 0xc0, 0xa2, 0x5b, 0xec, 0xa8, 0x9c, 0xe4, 0x39, 0xe5, + 0x64, 0x28, 0x2f, 0x21, 0x8b, 0x7f, 0x9a, 0x80, 0x82, 0x57, 0xcd, 0x38, 0xad, 0xe4, 0xab, 0x90, + 0x75, 0x48, 0x9e, 0xc7, 0x3a, 0xcc, 0xf3, 0x98, 0x90, 0x68, 0x0b, 0xb1, 0x04, 0x0b, 0xd4, 0x71, + 0x91, 0x95, 0x6e, 0xb7, 0xad, 0xbb, 0x70, 0x97, 0xda, 0x9f, 0x94, 0x34, 0x4f, 0x93, 0x56, 0x58, + 0x0a, 0x05, 0xba, 0xe8, 0x57, 0x04, 0x98, 0xdb, 0xb3, 0x30, 0xfe, 0x02, 0xcb, 0x14, 0x7a, 0x4d, + 0x16, 0xf4, 0xb2, 0x46, 0xca, 0xf0, 0x95, 0x3f, 0x8a, 0x67, 0xd9, 0x8b, 0x1b, 0xe4, 0xbd, 0xbc, + 0xd3, 0xbe, 0x82, 0x79, 0xda, 0x28, 0x71, 0xef, 0x2d, 0x16, 0xff, 0x56, 0x02, 0x50, 0x50, 0xf2, + 0x4f, 0xae, 0x31, 0x13, 0xf1, 0x35, 0xe6, 0xbb, 0x80, 0x58, 0xd8, 0xa4, 0x2d, 0x77, 0xb1, 0x25, + 0xdb, 0x58, 0x35, 0xf9, 0xf1, 0x50, 0x82, 0x54, 0xe4, 0x29, 0xdb, 0xd8, 0x6a, 0x50, 0x3a, 0x7a, + 0x0c, 0xe0, 0xbb, 0x74, 0x7c, 0xc6, 0x19, 0xe9, 0xd1, 0x49, 0x19, 0xcf, 0x97, 0x13, 0xbf, 0x27, + 0x40, 0x7e, 0x53, 0x6f, 0x59, 0x4a, 0xac, 0xe7, 0x1e, 0xa1, 0x8f, 0xc3, 0xcb, 0xe8, 0xd9, 0x87, + 0xe5, 0xa8, 0x10, 0x1f, 0x96, 0xc3, 0x05, 0x67, 0x9c, 0x81, 0x98, 0x70, 0xaf, 0x44, 0x71, 0xda, + 0xae, 0x7f, 0x57, 0x86, 0x39, 0x5e, 0xee, 0x1d, 0x43, 0x37, 0x0d, 0xf4, 0x00, 0x92, 0x2d, 0xfe, + 0x99, 0x24, 0x1b, 0xb9, 0xa4, 0xe9, 0x1f, 0x01, 0x58, 0x9b, 0x92, 0x48, 0x5e, 0xc2, 0xd2, 0xed, + 0x39, 0x11, 0x4e, 0xa4, 0x1f, 0xf5, 0x1e, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, 0x80, 0x82, 0xea, 0x1f, + 0x65, 0x26, 0x13, 0xf6, 0xe4, 0x50, 0xc0, 0x19, 0x79, 0xa8, 0x5c, 0x6d, 0x4a, 0xca, 0xab, 0xa1, + 0x04, 0x54, 0x09, 0x9e, 0xa0, 0x95, 0x1a, 0x08, 0xa9, 0xf3, 0xf7, 0x6b, 0x87, 0x4f, 0xef, 0xaa, + 0x4d, 0x05, 0x0e, 0xda, 0x42, 0x1f, 0xc3, 0x8c, 0x46, 0xcf, 0x6a, 0xe2, 0x83, 0x3f, 0xaa, 0xa1, + 0x43, 0x47, 0x62, 0xd5, 0xa6, 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, 0x62, 0x2e, 0x1d, 0x47, + 0xe1, 0xb7, 0x86, 0x4b, 0x08, 0x4c, 0x9a, 0xb5, 0x29, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x0d, 0x48, + 0xd9, 0xaa, 0xe2, 0xe2, 0xf0, 0xab, 0x43, 0x8e, 0x48, 0xf1, 0x99, 0x69, 0x6e, 0xf4, 0x98, 0x1d, + 0xea, 0xe9, 0x1c, 0xbb, 0x4b, 0xa2, 0x51, 0xc5, 0x0f, 0x6d, 0xc5, 0x27, 0xc5, 0xc7, 0x94, 0x80, + 0x9e, 0x42, 0x56, 0x21, 0xbe, 0xb1, 0x4c, 0x37, 0xb3, 0xd2, 0x35, 0xd0, 0xe8, 0x40, 0x84, 0x81, + 0xcd, 0xc7, 0x35, 0xba, 0xdf, 0xdf, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, 0xe1, 0x52, 0x76, 0xb4, + 0xa0, 0x60, 0x20, 0x9d, 0x27, 0x88, 0x12, 0x89, 0x8f, 0xec, 0xed, 0x24, 0xa7, 0x95, 0x9a, 0x1b, + 0xfa, 0xe9, 0x3b, 0x62, 0x8f, 0x54, 0x6d, 0x4a, 0x9a, 0xdb, 0x0f, 0x90, 0xd1, 0x12, 0x24, 0x5a, + 0x6a, 0x29, 0x37, 0xd4, 0x18, 0x78, 0xfb, 0x80, 0x6a, 0x53, 0x52, 0xa2, 0xa5, 0xa2, 0x4f, 0x21, + 0xcd, 0x36, 0x75, 0x1c, 0x1b, 0xa5, 0xfc, 0xd0, 0x31, 0x16, 0xde, 0x1a, 0x53, 0x9b, 0x92, 0xe8, + 0x3e, 0x12, 0xf2, 0xbe, 0x6d, 0xc8, 0x5b, 0x2c, 0x12, 0xd1, 0x8d, 0x1b, 0x2e, 0x0e, 0x0d, 0x07, + 0x88, 0x0a, 0x1d, 0xae, 0x51, 0x94, 0x14, 0xa0, 0xa3, 0x6f, 0xc3, 0x85, 0xb0, 0x44, 0xde, 0xd3, + 0xe6, 0x87, 0x7e, 0xda, 0x1e, 0x1a, 0xc6, 0x5a, 0x9b, 0x92, 0x90, 0x35, 0x90, 0x88, 0x3e, 0x84, + 0x69, 0xd6, 0x6a, 0x88, 0x8a, 0x8c, 0x8a, 0x70, 0xe9, 0x6b, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x87, + 0x87, 0xe3, 0xc9, 0x6d, 0xb3, 0x55, 0x5a, 0x18, 0xda, 0xf9, 0x07, 0x03, 0x0b, 0x49, 0xe7, 0x77, + 0x7c, 0x2a, 0x69, 0x77, 0x8b, 0xa5, 0xf0, 0x10, 0xac, 0x0b, 0x43, 0xdb, 0x3d, 0x22, 0x4a, 0xaf, + 0x46, 0x37, 0x4a, 0xf8, 0x64, 0x52, 0x34, 0x8b, 0x9d, 0xea, 0x23, 0xd3, 0x31, 0x75, 0x71, 0x68, + 0xd1, 0x06, 0x8f, 0x3d, 0xaa, 0x51, 0x7f, 0xd2, 0xa3, 0xa2, 0x97, 0x50, 0xe4, 0x67, 0x6f, 0xf8, + 0xdf, 0x5f, 0x2e, 0x51, 0x79, 0xf7, 0x22, 0x4d, 0x57, 0x54, 0xfc, 0x52, 0x6d, 0x4a, 0x2a, 0xa8, + 0xe1, 0x14, 0xf4, 0x19, 0xcc, 0x53, 0x79, 0xb2, 0xea, 0x1f, 0x97, 0x52, 0x2a, 0x0d, 0x1c, 0xbe, + 0x31, 0xfc, 0x64, 0x15, 0x57, 0x72, 0x51, 0xed, 0x4b, 0x22, 0xdd, 0x58, 0x37, 0x74, 0x87, 0x5a, + 0xd9, 0xf2, 0xd0, 0x6e, 0x1c, 0x3e, 0xfa, 0x91, 0x74, 0x63, 0x9d, 0x51, 0x48, 0x37, 0x76, 0x78, + 0x7c, 0x1e, 0x6f, 0x8e, 0xb7, 0x87, 0x76, 0xe3, 0xa8, 0x40, 0x3e, 0xd2, 0x8d, 0x9d, 0x20, 0x9d, + 0x74, 0x63, 0x66, 0x20, 0xfa, 0xe4, 0xbe, 0x33, 0xb4, 0x1b, 0x0f, 0xdd, 0x54, 0x4e, 0xba, 0xb1, + 0x32, 0x90, 0x88, 0xd6, 0x00, 0x98, 0xeb, 0x47, 0xe7, 0xff, 0xab, 0x43, 0x27, 0x83, 0xfe, 0x38, + 0x3d, 0x32, 0x19, 0xb4, 0x5d, 0x1a, 0x31, 0x64, 0x14, 0x58, 0xca, 0xf4, 0x73, 0x73, 0x69, 0x71, + 0xa8, 0x21, 0x1b, 0xf8, 0x34, 0x4c, 0x0c, 0xd9, 0x91, 0x47, 0x24, 0xb3, 0x0a, 0x5b, 0x1f, 0x2f, + 0x5d, 0x1b, 0x6e, 0x96, 0x83, 0x9f, 0xc9, 0xa8, 0x59, 0xa6, 0x04, 0xb4, 0x02, 0x19, 0xe2, 0xde, + 0x9c, 0x50, 0x33, 0x74, 0x7d, 0xa8, 0xff, 0xde, 0xb7, 0x13, 0xa8, 0x36, 0x25, 0xa5, 0x5f, 0x73, + 0x12, 0x79, 0x3d, 0x5b, 0x39, 0x2c, 0x89, 0x43, 0x5f, 0x1f, 0x5a, 0x65, 0x26, 0xaf, 0x67, 0x1c, + 0x48, 0x85, 0x8b, 0xac, 0xad, 0xf8, 0xfe, 0x6e, 0x8b, 0x6f, 0x49, 0x2e, 0xdd, 0xa0, 0xa2, 0x86, + 0x2e, 0xc1, 0x45, 0x6e, 0x3b, 0xaf, 0x4d, 0x49, 0x0b, 0xca, 0x60, 0x2a, 0x19, 0xf0, 0x7c, 0xea, + 0x61, 0x0b, 0x77, 0xa5, 0x9b, 0x43, 0x07, 0x7c, 0xc4, 0x7a, 0x27, 0x19, 0xf0, 0x4a, 0x80, 0xcc, + 0x26, 0x20, 0x4d, 0xb6, 0x6d, 0x16, 0x9c, 0x70, 0x6b, 0xc4, 0x04, 0xd4, 0xb7, 0x62, 0xc2, 0x26, + 0x20, 0xad, 0xc1, 0x38, 0x89, 0x20, 0xb5, 0x8d, 0x15, 0x8b, 0x9b, 0xd9, 0xdb, 0x43, 0x05, 0x0d, + 0x9c, 0xb2, 0x48, 0x04, 0xa9, 0x1e, 0x91, 0x38, 0x3c, 0x96, 0x7b, 0x68, 0x0f, 0xf7, 0x8d, 0xef, + 0x0c, 0x75, 0x78, 0x22, 0x4f, 0x15, 0x22, 0x0e, 0x8f, 0x15, 0x4a, 0x40, 0x3f, 0x0b, 0xb3, 0x1c, + 0xea, 0x96, 0xee, 0x8e, 0x70, 0x01, 0x83, 0xab, 0x13, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, + 0xc4, 0x66, 0xd5, 0xbb, 0x37, 0xc2, 0xca, 0x0e, 0xa0, 0x7c, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, + 0xac, 0x9f, 0xf2, 0xb9, 0xee, 0xfe, 0x50, 0x2b, 0x3b, 0xb8, 0xcf, 0x88, 0x58, 0xd9, 0xd7, 0x3e, + 0x95, 0xd4, 0xcc, 0x66, 0x50, 0xb3, 0xf4, 0x53, 0x43, 0x6b, 0x16, 0xc6, 0xdc, 0xa4, 0x66, 0x9c, + 0x87, 0x34, 0x1b, 0xf3, 0xfe, 0x99, 0xa6, 0xdf, 0x1d, 0x7e, 0x88, 0x42, 0x3f, 0xbe, 0xaa, 0xb9, + 0x8b, 0xba, 0x4c, 0xc3, 0x9e, 0xa1, 0xb2, 0xf8, 0x0e, 0x6f, 0xae, 0xa9, 0xf7, 0x46, 0x1b, 0xaa, + 0xa8, 0x9d, 0xf1, 0x9e, 0xa1, 0x0a, 0x25, 0xd2, 0xa2, 0xb2, 0x4d, 0x7d, 0x74, 0x7c, 0x2f, 0x8d, + 0x38, 0xef, 0xa1, 0x6f, 0x6b, 0x25, 0x2d, 0xaa, 0x47, 0xf4, 0x87, 0x50, 0x8f, 0x1d, 0x46, 0x52, + 0x5a, 0x1e, 0x3d, 0x84, 0xc2, 0xc7, 0xa1, 0x78, 0x43, 0x88, 0x93, 0xbd, 0x39, 0xd3, 0xf5, 0x30, + 0xde, 0x1f, 0x3d, 0x67, 0xf6, 0xbb, 0x16, 0x6c, 0xce, 0xe4, 0x3e, 0xc5, 0x5f, 0x11, 0xe0, 0x1a, + 0x2b, 0x1b, 0x5d, 0xfd, 0x3c, 0x91, 0xbd, 0x35, 0xe4, 0xc0, 0x06, 0x93, 0x07, 0xf4, 0x05, 0x1f, + 0x0e, 0x2b, 0xee, 0x98, 0x35, 0xf1, 0xda, 0x94, 0xf4, 0x8e, 0x32, 0x2a, 0x1f, 0xe9, 0x52, 0x1d, + 0x86, 0xa0, 0x4a, 0x0f, 0x87, 0x76, 0xa9, 0x30, 0xea, 0x23, 0x5d, 0x8a, 0xf3, 0xac, 0xce, 0xf2, + 0x2f, 0xd0, 0xde, 0x86, 0xdc, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, 0x5f, 0x2e, 0x96, 0xd6, 0x53, 0xe9, + 0x2b, 0xc5, 0xf2, 0x7a, 0x2a, 0xfd, 0x56, 0xf1, 0x6d, 0xf1, 0x1f, 0x96, 0x21, 0xe7, 0x82, 0x2e, + 0x06, 0xa8, 0x1e, 0x06, 0x01, 0xd5, 0xd5, 0x61, 0x80, 0x8a, 0xc3, 0x34, 0x8e, 0xa8, 0x1e, 0x06, + 0x11, 0xd5, 0xd5, 0x61, 0x88, 0xca, 0xe7, 0x21, 0x90, 0xaa, 0x39, 0x0c, 0x52, 0xdd, 0x9b, 0x00, + 0x52, 0x79, 0xa2, 0xfa, 0x31, 0xd5, 0xda, 0x20, 0xa6, 0xba, 0x39, 0x1a, 0x53, 0x79, 0xa2, 0x02, + 0xa0, 0xea, 0x71, 0x1f, 0xa8, 0xba, 0x3e, 0x02, 0x54, 0x79, 0xfc, 0x2e, 0xaa, 0xda, 0x88, 0x44, + 0x55, 0xb7, 0xc7, 0xa1, 0x2a, 0x4f, 0x4e, 0x08, 0x56, 0x7d, 0x10, 0x82, 0x55, 0x8b, 0x43, 0x61, + 0x95, 0xc7, 0xcd, 0x70, 0xd5, 0x27, 0xfd, 0xb8, 0xea, 0xfa, 0x08, 0x5c, 0xe5, 0xd7, 0x80, 0x03, + 0xab, 0x5a, 0x14, 0xb0, 0xba, 0x35, 0x06, 0x58, 0x79, 0x52, 0x82, 0xc8, 0xaa, 0x16, 0x85, 0xac, + 0x6e, 0x8d, 0x41, 0x56, 0x7d, 0x92, 0x18, 0xb4, 0xda, 0x8a, 0x86, 0x56, 0x77, 0xc6, 0x42, 0x2b, + 0x4f, 0x5a, 0x18, 0x5b, 0x2d, 0x07, 0xb0, 0xd5, 0x3b, 0x43, 0xb0, 0x95, 0xc7, 0x4a, 0xc0, 0xd5, + 0x37, 0x07, 0xc0, 0x95, 0x38, 0x0a, 0x5c, 0x79, 0xbc, 0x1e, 0xba, 0x7a, 0x31, 0x04, 0x5d, 0xdd, + 0x1d, 0x8f, 0xae, 0x3c, 0x61, 0x7d, 0xf0, 0x4a, 0x19, 0x09, 0xaf, 0xde, 0x9b, 0x10, 0x5e, 0x79, + 0xd2, 0xa3, 0xf0, 0xd5, 0x47, 0x61, 0x7c, 0x75, 0x6d, 0x38, 0xbe, 0xf2, 0xc4, 0x70, 0x80, 0xb5, + 0x11, 0x09, 0xb0, 0x6e, 0x8f, 0x03, 0x58, 0xfe, 0x38, 0x08, 0x22, 0xac, 0xad, 0x68, 0x84, 0x75, + 0x67, 0x2c, 0xc2, 0xf2, 0x9b, 0x3f, 0x04, 0xb1, 0x36, 0x22, 0x21, 0xd6, 0xed, 0x71, 0x10, 0xcb, + 0x2f, 0x5c, 0x10, 0x63, 0xbd, 0x1a, 0x8a, 0xb1, 0xee, 0x4f, 0x82, 0xb1, 0x3c, 0xa1, 0x03, 0x20, + 0xeb, 0xf3, 0xe1, 0x20, 0xeb, 0xa7, 0xce, 0x71, 0x7c, 0x65, 0x24, 0xca, 0xfa, 0xe6, 0x00, 0xca, + 0x12, 0x47, 0xa1, 0x2c, 0xbf, 0x3f, 0xbb, 0x30, 0x4b, 0x19, 0x09, 0x8a, 0xde, 0x9b, 0x10, 0x14, + 0xf9, 0x9d, 0x2f, 0x02, 0x15, 0x55, 0x23, 0x50, 0xd1, 0xcd, 0xd1, 0xa8, 0xc8, 0x37, 0xe7, 0x3e, + 0x2c, 0xaa, 0x45, 0xc1, 0xa2, 0x5b, 0x63, 0x60, 0x91, 0x6f, 0x85, 0x02, 0xb8, 0xe8, 0x71, 0x1f, + 0x2e, 0xba, 0x3e, 0x36, 0xc0, 0x2a, 0x00, 0x8c, 0x56, 0x07, 0x81, 0xd1, 0x8d, 0x91, 0xc0, 0xc8, + 0x93, 0xe0, 0x23, 0xa3, 0xc7, 0x7d, 0xc8, 0xe8, 0xfa, 0x08, 0x64, 0xe4, 0x17, 0x80, 0x43, 0x23, + 0x6d, 0x34, 0x34, 0x5a, 0x9a, 0x14, 0x1a, 0x79, 0x82, 0x23, 0xb1, 0xd1, 0x56, 0x34, 0x36, 0xba, + 0x33, 0x61, 0xec, 0xc3, 0x00, 0x38, 0xaa, 0x45, 0x81, 0xa3, 0x5b, 0x63, 0xc0, 0x51, 0x70, 0x0e, + 0xf1, 0xd0, 0x51, 0x2d, 0x0a, 0x1d, 0xdd, 0x1a, 0x83, 0x8e, 0x7c, 0x49, 0x01, 0x78, 0xd4, 0x1c, + 0x06, 0x8f, 0xee, 0x4d, 0x00, 0x8f, 0x7c, 0xe7, 0xa5, 0x0f, 0x1f, 0x7d, 0xda, 0x8f, 0x8f, 0xc4, + 0x51, 0xf8, 0xc8, 0x1f, 0x91, 0x2e, 0x40, 0xda, 0x8a, 0x06, 0x48, 0x77, 0xc6, 0x02, 0xa4, 0xa0, + 0x91, 0x0c, 0x20, 0xa4, 0x8d, 0x48, 0x84, 0x74, 0x7b, 0x1c, 0x42, 0xf2, 0x8d, 0x64, 0x10, 0x22, + 0x7d, 0xda, 0x0f, 0x91, 0xc4, 0x51, 0x10, 0xc9, 0xaf, 0x9c, 0x8b, 0x91, 0x6a, 0x51, 0x18, 0xe9, + 0xd6, 0x18, 0x8c, 0xe4, 0x37, 0x5e, 0x00, 0x24, 0x29, 0x23, 0x41, 0xd2, 0x7b, 0x13, 0x82, 0xa4, + 0x3e, 0xc3, 0x15, 0x46, 0x49, 0xb5, 0x28, 0x94, 0x74, 0x6b, 0x0c, 0x4a, 0x0a, 0x14, 0xd6, 0x87, + 0x49, 0x5b, 0xd1, 0x30, 0xe9, 0xce, 0x58, 0x98, 0xd4, 0x37, 0x9a, 0x5c, 0x9c, 0xb4, 0x11, 0x89, + 0x93, 0x6e, 0x8f, 0xc3, 0x49, 0x7d, 0x13, 0x1f, 0x77, 0x0e, 0x7e, 0x69, 0x72, 0xa0, 0xf4, 0xd1, + 0xf9, 0x81, 0x92, 0xf7, 0xce, 0x31, 0x48, 0xe9, 0xd3, 0x7e, 0xa4, 0x24, 0x8e, 0x42, 0x4a, 0x7e, + 0xcf, 0x3a, 0x1f, 0x54, 0x5a, 0x4f, 0xa5, 0xdf, 0x2e, 0xbe, 0x23, 0xfe, 0xb5, 0x59, 0x98, 0xa9, + 0x79, 0x21, 0x49, 0x7e, 0x2d, 0x85, 0x37, 0x39, 0x03, 0x0b, 0xad, 0x91, 0x11, 0x4f, 0xed, 0xe6, + 0xf8, 0x93, 0x13, 0x07, 0xcf, 0xf6, 0xe3, 0xac, 0x6f, 0xb0, 0x19, 0x1d, 0x7d, 0x00, 0xb9, 0x9e, + 0x8d, 0x2d, 0xb9, 0x6b, 0xe9, 0xa6, 0xa5, 0x3b, 0x6c, 0xc3, 0x8d, 0xb0, 0x5a, 0xfc, 0xf2, 0x74, + 0x71, 0x6e, 0xc7, 0xc6, 0xd6, 0x36, 0xa7, 0x4b, 0x73, 0xbd, 0xc0, 0x93, 0x7b, 0xcb, 0xd9, 0xf4, + 0xe4, 0xb7, 0x9c, 0xbd, 0x80, 0xa2, 0x85, 0x15, 0x2d, 0xe4, 0xc1, 0xb0, 0xf3, 0xa6, 0xa2, 0xfb, + 0x1c, 0xdd, 0x10, 0xe7, 0xe6, 0xa4, 0xe7, 0x4e, 0x15, 0xac, 0x30, 0x11, 0x3d, 0x80, 0x8b, 0x1d, + 0xe5, 0x98, 0x86, 0xaf, 0xca, 0xae, 0x53, 0x48, 0x43, 0x52, 0xd9, 0x05, 0x62, 0xa8, 0xa3, 0x1c, + 0xd3, 0x2b, 0xd3, 0x58, 0x12, 0xbd, 0xef, 0xe4, 0x16, 0xe4, 0x35, 0xdd, 0x76, 0x74, 0x43, 0x75, + 0xf8, 0x51, 0xc6, 0xec, 0x18, 0xe0, 0x9c, 0x4b, 0x65, 0xe7, 0x15, 0xdf, 0x87, 0x79, 0xbe, 0xaf, + 0x21, 0xf0, 0x31, 0x96, 0x1f, 0x07, 0xcc, 0x12, 0xbc, 0xef, 0xaf, 0xa8, 0x02, 0x85, 0x96, 0xe2, + 0xe0, 0x23, 0xe5, 0x44, 0x76, 0x77, 0xcd, 0x65, 0xe9, 0x09, 0x9f, 0x6f, 0x9d, 0x9d, 0x2e, 0xe6, + 0x9e, 0xb2, 0xa4, 0x81, 0xcd, 0x73, 0xb9, 0x56, 0x20, 0x41, 0x43, 0x77, 0xa0, 0xa0, 0xd8, 0x27, + 0x86, 0x4a, 0xd5, 0x83, 0x0d, 0xbb, 0x67, 0x53, 0x48, 0x92, 0x96, 0xf2, 0x94, 0x5c, 0x71, 0xa9, + 0xe8, 0x3a, 0xcc, 0xf1, 0xa0, 0x7f, 0x76, 0x07, 0x53, 0x81, 0x56, 0x95, 0x5f, 0xee, 0xc1, 0xae, + 0x61, 0x7a, 0x0c, 0x65, 0x7e, 0x55, 0xc1, 0x91, 0x62, 0x69, 0x32, 0xd5, 0xba, 0xdf, 0x3f, 0x8b, + 0x54, 0xec, 0x65, 0x76, 0x35, 0x01, 0xc9, 0x40, 0x54, 0xed, 0x8f, 0xa1, 0x2d, 0x98, 0x57, 0xdb, + 0xba, 0x87, 0x20, 0x58, 0xcd, 0xe7, 0x87, 0x8e, 0xa6, 0x0a, 0xcd, 0xeb, 0x7f, 0x8c, 0x2e, 0xa8, + 0x61, 0x02, 0x6a, 0x00, 0x3d, 0xd9, 0x47, 0xee, 0x9a, 0x6d, 0x5d, 0x3d, 0xa1, 0xe0, 0x21, 0x7c, + 0xe4, 0xfa, 0xc8, 0x8b, 0x0f, 0x5e, 0x29, 0xba, 0xb3, 0x4d, 0x39, 0x25, 0x38, 0xf2, 0x7e, 0xb3, + 0xe3, 0x92, 0xd7, 0x53, 0xe9, 0xb9, 0x62, 0x6e, 0x3d, 0x95, 0xce, 0x17, 0x0b, 0xe2, 0x5f, 0x17, + 0xa0, 0xd0, 0x57, 0x16, 0x54, 0x83, 0x8b, 0x9a, 0x37, 0x54, 0x64, 0xbe, 0x2b, 0x4c, 0x37, 0x0d, + 0x7e, 0x8a, 0xfc, 0xc2, 0x97, 0xa7, 0x8b, 0x05, 0x9a, 0xfb, 0xa9, 0x97, 0x24, 0x5d, 0xf0, 0x39, + 0x7c, 0x2a, 0xfa, 0x08, 0xf2, 0xcc, 0xfd, 0xf4, 0xae, 0x15, 0xa4, 0xe1, 0xfc, 0xab, 0xf3, 0x5f, + 0x9e, 0x2e, 0xe6, 0xa8, 0xcf, 0xe9, 0x9e, 0xf2, 0x2c, 0xe5, 0xda, 0xc1, 0x47, 0xf1, 0xd7, 0x04, + 0x98, 0x0b, 0xed, 0xbc, 0x7a, 0xdc, 0xf7, 0xf1, 0xfb, 0x4a, 0x34, 0x6e, 0x1d, 0x16, 0xfb, 0x98, + 0xe6, 0xfd, 0xdc, 0x0d, 0x24, 0x5d, 0x1c, 0x8e, 0x7b, 0xe8, 0x2a, 0x8e, 0x1b, 0x55, 0xe3, 0xb2, + 0x7d, 0x9c, 0xfa, 0xfe, 0x0f, 0x16, 0xa7, 0xc4, 0x3f, 0x4b, 0x41, 0x2e, 0xbc, 0xcf, 0xaa, 0xde, + 0x57, 0xae, 0xa8, 0x79, 0x25, 0xc4, 0xb1, 0x34, 0xe2, 0xae, 0x99, 0x8c, 0x7f, 0x5d, 0x03, 0x2b, + 0xe6, 0xb5, 0x11, 0x9f, 0xf8, 0x83, 0xe5, 0xf4, 0x19, 0xcb, 0xff, 0x3e, 0xe9, 0xd9, 0xd7, 0x25, + 0x98, 0xa6, 0x47, 0x26, 0xf1, 0xa2, 0x95, 0xfa, 0x7b, 0x0f, 0xf1, 0xb5, 0x49, 0xba, 0xc4, 0xb2, + 0x11, 0x7b, 0xdc, 0x7c, 0xa3, 0x33, 0x09, 0xfd, 0x61, 0x70, 0xfe, 0x5b, 0x1c, 0x7b, 0xec, 0x4c, + 0xca, 0xff, 0x87, 0x91, 0x37, 0xe4, 0x7d, 0xe8, 0x17, 0xa0, 0xa0, 0x9a, 0xed, 0x36, 0x9b, 0x6b, + 0x99, 0x45, 0x1b, 0x3c, 0xa5, 0x86, 0x16, 0x81, 0x5f, 0xdc, 0xb9, 0xe4, 0x5d, 0xe0, 0xb9, 0x24, + 0xf1, 0x0b, 0x3c, 0x03, 0x61, 0xbd, 0x79, 0x4f, 0x18, 0x33, 0x84, 0x7d, 0x11, 0xc6, 0xb3, 0x6f, + 0x12, 0x61, 0xcc, 0xa2, 0xd2, 0x79, 0xcf, 0xfb, 0x43, 0x81, 0x87, 0xf0, 0x3c, 0x33, 0xcd, 0x83, + 0x9e, 0x17, 0x19, 0x5c, 0x0e, 0x9e, 0x30, 0x99, 0xfe, 0xf2, 0x74, 0x31, 0x25, 0x79, 0x47, 0x4c, + 0x46, 0xcd, 0x1c, 0x89, 0xaf, 0x36, 0x73, 0x5c, 0x87, 0xb9, 0xae, 0x85, 0xf7, 0xb0, 0xa3, 0xee, + 0xcb, 0x46, 0xaf, 0xc3, 0x37, 0x10, 0x65, 0x5d, 0xda, 0x56, 0xaf, 0x83, 0xee, 0x41, 0xd1, 0xcb, + 0xc2, 0x31, 0xbe, 0x7b, 0x04, 0x99, 0x4b, 0xe7, 0x2b, 0x02, 0xe2, 0xff, 0x12, 0x60, 0x21, 0x54, + 0x27, 0x3e, 0xa6, 0xd6, 0x21, 0xeb, 0x9b, 0x13, 0xbb, 0x24, 0x9c, 0x33, 0x4e, 0x36, 0xc8, 0x8c, + 0x64, 0xb8, 0xe4, 0xbe, 0x96, 0x5e, 0x71, 0xe0, 0x8b, 0x4d, 0x9c, 0x53, 0xec, 0x45, 0x5f, 0xce, + 0x5a, 0xe0, 0x05, 0xde, 0x20, 0x4b, 0x4e, 0x34, 0xc8, 0xc4, 0xdf, 0x14, 0xa0, 0x48, 0x5f, 0xf0, + 0x04, 0x63, 0x2d, 0x16, 0xeb, 0xe6, 0xc6, 0x9f, 0x27, 0x26, 0xdf, 0x22, 0x14, 0xba, 0x96, 0x25, + 0x19, 0xbe, 0x96, 0x45, 0xfc, 0x81, 0x00, 0x79, 0xaf, 0x84, 0xec, 0x2a, 0xc4, 0x11, 0x07, 0x99, + 0xbe, 0xd9, 0x75, 0x7f, 0xee, 0x81, 0x2b, 0x13, 0xdd, 0xce, 0x18, 0x3c, 0x70, 0x85, 0x5d, 0x53, + 0xf7, 0x37, 0xdd, 0x9e, 0x43, 0x8a, 0x58, 0xf1, 0x0f, 0xd6, 0x78, 0x83, 0xdd, 0x52, 0x12, 0xbd, + 0x45, 0xd6, 0x6c, 0x1f, 0xb2, 0x33, 0x72, 0x26, 0x32, 0x7b, 0x88, 0x07, 0xae, 0x01, 0x5f, 0x0d, + 0xd4, 0x9a, 0x0d, 0x7a, 0xbf, 0x2c, 0xfb, 0x6d, 0x8b, 0x4f, 0x02, 0x0a, 0xa4, 0x8d, 0x4f, 0xb4, + 0x34, 0x91, 0x29, 0x76, 0xb5, 0xc4, 0xfa, 0xca, 0x1f, 0x04, 0x5b, 0xa2, 0x7a, 0x48, 0x50, 0xe0, + 0x23, 0x48, 0x1e, 0x2a, 0xed, 0x51, 0x11, 0x60, 0xa1, 0x96, 0x93, 0x48, 0x6e, 0xf4, 0x24, 0x74, + 0x1e, 0x49, 0x62, 0x38, 0x62, 0x19, 0x54, 0x69, 0xe8, 0xdc, 0x92, 0x0f, 0xc3, 0x7d, 0x7d, 0xe4, + 0xeb, 0x83, 0x9d, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, 0xb0, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, + 0x9d, 0x17, 0x3d, 0xd3, 0xf2, 0xcf, 0x76, 0xb9, 0xdd, 0x77, 0x4f, 0xcd, 0xf4, 0x6a, 0xf6, 0x2c, + 0xea, 0xf8, 0xa7, 0x8b, 0xb0, 0x10, 0xe2, 0x66, 0xc6, 0x42, 0xfc, 0x10, 0xae, 0x3c, 0x35, 0x6d, + 0x5b, 0xef, 0x12, 0xe8, 0x4b, 0x47, 0x25, 0x99, 0x5a, 0x3c, 0xf3, 0x98, 0xee, 0xd2, 0x45, 0x10, + 0x83, 0x99, 0x91, 0x8c, 0xe4, 0x3d, 0x8b, 0xbf, 0x2b, 0xc0, 0xe5, 0x41, 0x4e, 0xa6, 0xe5, 0xa8, + 0xcd, 0x9d, 0xb3, 0xaa, 0xe9, 0x9f, 0x05, 0x38, 0xbe, 0xb7, 0xba, 0xd9, 0x89, 0x8b, 0xca, 0xdf, + 0x29, 0x77, 0x14, 0x6a, 0x3e, 0xf8, 0x46, 0xf3, 0x3c, 0x27, 0x6f, 0x32, 0xaa, 0x6f, 0x49, 0x52, + 0x93, 0x59, 0x92, 0x26, 0x14, 0xd6, 0x4d, 0xdd, 0x20, 0x9e, 0xb0, 0x5b, 0xdf, 0x15, 0xc8, 0xef, + 0xea, 0x86, 0x62, 0x9d, 0xc8, 0x6e, 0xe0, 0xa1, 0x30, 0x2e, 0xf0, 0x50, 0xca, 0x31, 0x0e, 0xfe, + 0x28, 0xfe, 0x48, 0x80, 0xa2, 0x2f, 0x96, 0x5b, 0xe4, 0x77, 0x01, 0xd4, 0x76, 0xcf, 0x76, 0xb0, + 0xe5, 0xb6, 0xd2, 0x1c, 0x0b, 0xd4, 0xaf, 0x30, 0x6a, 0x7d, 0x4d, 0xca, 0xf0, 0x0c, 0x75, 0x0d, + 0xdd, 0x08, 0x9f, 0x83, 0x31, 0xbd, 0x0a, 0x67, 0x03, 0xa7, 0x5f, 0x90, 0x66, 0xb7, 0x1d, 0xd3, + 0xf2, 0x30, 0x17, 0x6f, 0x76, 0xf7, 0x64, 0x7f, 0xba, 0x7d, 0x9b, 0xe4, 0x5b, 0x81, 0x3c, 0x71, + 0x17, 0x0e, 0xb1, 0x57, 0xa5, 0xd4, 0xf8, 0x2a, 0x31, 0x0e, 0xb7, 0x4a, 0x7f, 0x8f, 0x38, 0xb9, + 0xac, 0x35, 0xbc, 0x16, 0x1e, 0x61, 0xd1, 0x7e, 0x3a, 0x78, 0x7c, 0xe5, 0x64, 0x07, 0x7c, 0x52, + 0x4f, 0xe6, 0x9b, 0x90, 0x76, 0xef, 0xe7, 0xe6, 0x03, 0xe4, 0xca, 0x12, 0xbb, 0xc0, 0x7b, 0xc9, + 0xbd, 0xc0, 0x7b, 0x69, 0x8d, 0x67, 0x60, 0x66, 0xfc, 0xfb, 0xff, 0x65, 0x51, 0x90, 0x3c, 0xa6, + 0xfb, 0x0d, 0xd2, 0xc3, 0x07, 0x66, 0x61, 0x94, 0x07, 0x08, 0x5c, 0x69, 0xc4, 0xaf, 0x6c, 0x5e, + 0x59, 0x93, 0x77, 0xb6, 0x2a, 0xcf, 0x37, 0x37, 0xeb, 0xcd, 0x66, 0x75, 0xad, 0x28, 0xa0, 0x22, + 0xcc, 0x85, 0x2e, 0x44, 0x4a, 0xb0, 0x4b, 0x9c, 0xef, 0xff, 0x34, 0x80, 0x7f, 0xb7, 0x1a, 0x91, + 0xb5, 0x51, 0xfd, 0x4c, 0x7e, 0xb9, 0xf2, 0x6c, 0xa7, 0xda, 0x28, 0x4e, 0x21, 0x04, 0xf9, 0xd5, + 0x95, 0x66, 0xa5, 0x26, 0x4b, 0xd5, 0xc6, 0xf6, 0xf3, 0xad, 0x46, 0xd5, 0xbd, 0xfc, 0xf9, 0xfe, + 0x1a, 0xcc, 0x05, 0x4f, 0x36, 0x42, 0x0b, 0x50, 0xa8, 0xd4, 0xaa, 0x95, 0x0d, 0xf9, 0x65, 0x7d, + 0x45, 0x7e, 0xb1, 0x53, 0xdd, 0xa9, 0x16, 0xa7, 0x68, 0xd1, 0x28, 0xf1, 0xc9, 0xce, 0xb3, 0x67, + 0x45, 0x01, 0x15, 0x20, 0xcb, 0x9e, 0xe9, 0xe5, 0x49, 0xc5, 0xc4, 0xfd, 0x4d, 0xc8, 0x06, 0x8e, + 0x40, 0x26, 0xaf, 0xdb, 0xde, 0x69, 0xd4, 0xe4, 0x66, 0x7d, 0xb3, 0xda, 0x68, 0xae, 0x6c, 0x6e, + 0x33, 0x19, 0x94, 0xb6, 0xb2, 0xfa, 0x5c, 0x6a, 0x16, 0x05, 0xef, 0xb9, 0xf9, 0x7c, 0xa7, 0x52, + 0x73, 0xab, 0x21, 0xa6, 0xd2, 0xc9, 0x62, 0xf2, 0xfe, 0x5f, 0x16, 0xe0, 0xf2, 0x90, 0x53, 0x7e, + 0x50, 0x16, 0x66, 0x77, 0x0c, 0x7a, 0x1e, 0x6c, 0x71, 0x0a, 0xe5, 0x02, 0x07, 0xfd, 0x14, 0x05, + 0x94, 0x66, 0x47, 0xad, 0x14, 0x13, 0x68, 0x06, 0x12, 0x8d, 0x47, 0xc5, 0x24, 0x29, 0x69, 0xe0, + 0x9c, 0x9c, 0x62, 0x0a, 0x65, 0xf8, 0x09, 0x1d, 0xc5, 0x69, 0x34, 0xe7, 0x1f, 0x94, 0x51, 0x9c, + 0x21, 0xa2, 0xbc, 0x03, 0x27, 0x8a, 0xb3, 0xf7, 0xaf, 0x43, 0x60, 0x53, 0x3f, 0x02, 0x98, 0x79, + 0xa6, 0x38, 0xd8, 0x76, 0x8a, 0x53, 0x68, 0x16, 0x92, 0x2b, 0xed, 0x76, 0x51, 0x78, 0xf8, 0x2f, + 0x52, 0x90, 0x76, 0x2f, 0x09, 0x42, 0xcf, 0x60, 0x9a, 0x2d, 0x7d, 0x2f, 0x0e, 0x47, 0x06, 0x74, + 0xf0, 0x96, 0xaf, 0x8d, 0x83, 0x0e, 0xe2, 0x14, 0xfa, 0x8b, 0x90, 0x0d, 0x78, 0x4c, 0x68, 0xe8, + 0xf2, 0x5d, 0xc8, 0x4b, 0x2c, 0xdf, 0x1e, 0x97, 0xcd, 0x93, 0xff, 0x0a, 0x32, 0x9e, 0x05, 0x47, + 0x37, 0x46, 0xd9, 0x77, 0x57, 0xf6, 0xe8, 0x49, 0x80, 0x8c, 0x35, 0x71, 0xea, 0x7d, 0x01, 0x59, + 0x80, 0x06, 0x8d, 0x2d, 0x8a, 0x0a, 0xa8, 0x18, 0x6a, 0xcd, 0xcb, 0xf7, 0x27, 0xca, 0xed, 0xbf, + 0x93, 0x28, 0xcb, 0x9f, 0x31, 0xa2, 0x95, 0x35, 0x30, 0x1f, 0x45, 0x2b, 0x2b, 0x62, 0xe2, 0x99, + 0x42, 0x2f, 0x20, 0x45, 0x2c, 0x25, 0x8a, 0xf2, 0x21, 0xfb, 0x2c, 0x73, 0xf9, 0xc6, 0xc8, 0x3c, + 0xae, 0xc8, 0xd5, 0x7b, 0x3f, 0xfc, 0x93, 0xab, 0x53, 0x3f, 0x3c, 0xbb, 0x2a, 0xfc, 0xe8, 0xec, + 0xaa, 0xf0, 0x47, 0x67, 0x57, 0x85, 0x3f, 0x3e, 0xbb, 0x2a, 0x7c, 0xef, 0xc7, 0x57, 0xa7, 0x7e, + 0xf4, 0xe3, 0xab, 0x53, 0x7f, 0xf4, 0xe3, 0xab, 0x53, 0x9f, 0xcf, 0x72, 0xee, 0xdd, 0x19, 0x6a, + 0x54, 0x1e, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd1, 0x04, 0x95, 0x46, 0x50, 0x81, 0x00, + 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index b3010ad6d95a..066f158b2980 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -824,7 +824,8 @@ message AdminChangeReplicasResponse { // The first store in the list of targets becomes the new leaseholder. message AdminRelocateRangeRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - repeated ReplicationTarget targets = 2 [(gogoproto.nullable) = false]; + repeated ReplicationTarget voter_targets = 2 [(gogoproto.nullable) = false]; + repeated ReplicationTarget non_voter_targets = 3 [(gogoproto.nullable) = false]; // TODO(a-robinson): Add "reason"/"details" string fields? } diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 32a71e4f2230..6ed133467bd1 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1496,7 +1496,7 @@ func writeTooOldRetryTimestamp(err *WriteTooOldError) hlc.Timestamp { // trigger applies. func (crt ChangeReplicasTrigger) Replicas() []ReplicaDescriptor { if crt.Desc != nil { - return crt.Desc.Replicas().All() + return crt.Desc.Replicas().Descriptors() } return crt.DeprecatedUpdatedReplicas } diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 3dfffeeab5c8..b5e1a88659da 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1732,7 +1732,7 @@ func TestChangeReplicasTrigger_String(t *testing.T) { crt.InternalRemovedReplicas = nil crt.InternalAddedReplicas = nil repl1.Type = ReplicaTypeVoterFull() - crt.Desc.SetReplicas(MakeReplicaDescriptors([]ReplicaDescriptor{repl1, learner})) + crt.Desc.SetReplicas(MakeReplicaSet([]ReplicaDescriptor{repl1, learner})) act = crt.String() require.Empty(t, crt.Added()) require.Empty(t, crt.Removed()) @@ -1781,7 +1781,7 @@ func TestChangeReplicasTrigger_ConfChange(t *testing.T) { m.ChangeReplicasTrigger.InternalAddedReplicas = in.add m.ChangeReplicasTrigger.InternalRemovedReplicas = in.del m.Desc = &RangeDescriptor{} - m.Desc.SetReplicas(MakeReplicaDescriptors(in.repls)) + m.Desc.SetReplicas(MakeReplicaSet(in.repls)) return m } diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index d77bb24cdc56..47d4b2bc2738 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -129,10 +129,8 @@ func (g RangeGeneration) String() string { func (g RangeGeneration) SafeValue() {} // NewRangeDescriptor returns a RangeDescriptor populated from the input. -func NewRangeDescriptor( - rangeID RangeID, start, end RKey, replicas ReplicaDescriptors, -) *RangeDescriptor { - repls := append([]ReplicaDescriptor(nil), replicas.All()...) +func NewRangeDescriptor(rangeID RangeID, start, end RKey, replicas ReplicaSet) *RangeDescriptor { + repls := append([]ReplicaDescriptor(nil), replicas.Descriptors()...) for i := range repls { repls[i].ReplicaID = ReplicaID(i + 1) } @@ -142,7 +140,7 @@ func NewRangeDescriptor( EndKey: end, NextReplicaID: ReplicaID(len(repls) + 1), } - desc.SetReplicas(MakeReplicaDescriptors(repls)) + desc.SetReplicas(MakeReplicaSet(repls)) return desc } @@ -209,13 +207,13 @@ func (r *RangeDescriptor) ContainsKeyRange(start, end RKey) bool { // Replicas returns the set of nodes/stores on which replicas of this range are // stored. -func (r *RangeDescriptor) Replicas() ReplicaDescriptors { - return MakeReplicaDescriptors(r.InternalReplicas) +func (r *RangeDescriptor) Replicas() ReplicaSet { + return MakeReplicaSet(r.InternalReplicas) } // SetReplicas overwrites the set of nodes/stores on which replicas of this // range are stored. -func (r *RangeDescriptor) SetReplicas(replicas ReplicaDescriptors) { +func (r *RangeDescriptor) SetReplicas(replicas ReplicaSet) { r.InternalReplicas = replicas.AsProto() } @@ -278,7 +276,7 @@ func (r *RangeDescriptor) RemoveReplica(nodeID NodeID, storeID StoreID) (Replica // GetReplicaDescriptor returns the replica which matches the specified store // ID. func (r *RangeDescriptor) GetReplicaDescriptor(storeID StoreID) (ReplicaDescriptor, bool) { - for _, repDesc := range r.Replicas().All() { + for _, repDesc := range r.Replicas().Descriptors() { if repDesc.StoreID == storeID { return repDesc, true } @@ -289,7 +287,7 @@ func (r *RangeDescriptor) GetReplicaDescriptor(storeID StoreID) (ReplicaDescript // GetReplicaDescriptorByID returns the replica which matches the specified store // ID. func (r *RangeDescriptor) GetReplicaDescriptorByID(replicaID ReplicaID) (ReplicaDescriptor, bool) { - for _, repDesc := range r.Replicas().All() { + for _, repDesc := range r.Replicas().Descriptors() { if repDesc.ReplicaID == replicaID { return repDesc, true } @@ -325,7 +323,7 @@ func (r *RangeDescriptor) Validate() error { } seen := map[ReplicaID]struct{}{} stores := map[StoreID]struct{}{} - for i, rep := range r.Replicas().All() { + for i, rep := range r.Replicas().Descriptors() { if err := rep.Validate(); err != nil { return errors.Errorf("replica %d is invalid: %s", i, err) } @@ -361,7 +359,7 @@ func (r RangeDescriptor) SafeFormat(w redact.SafePrinter, _ rune) { } w.SafeString(" [") - if allReplicas := r.Replicas().All(); len(allReplicas) > 0 { + if allReplicas := r.Replicas().Descriptors(); len(allReplicas) > 0 { for i, rep := range allReplicas { if i > 0 { w.SafeString(", ") diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index cf52d2d54227..f5f63773132c 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -52,31 +52,23 @@ func ReplicaTypeLearner() *ReplicaType { return &t } -// ReplicaDescriptors is a set of replicas, usually the nodes/stores on which +// ReplicaSet is a set of replicas, usually the nodes/stores on which // replicas of a range are stored. -type ReplicaDescriptors struct { +type ReplicaSet struct { wrapped []ReplicaDescriptor } -// MakeReplicaDescriptors creates a ReplicaDescriptors wrapper from a raw slice -// of individual descriptors. -// -// All construction of ReplicaDescriptors is required to go through this method -// so we can guarantee sortedness, which is used to speed up accessor -// operations. -// -// The function accepts a pointer to a slice instead of a slice directly to -// avoid an allocation when boxing the argument as a sort.Interface. This may -// cause the argument to escape to the heap for some callers, at which point -// we're trading one allocation for another. However, if the caller already has -// the slice header on the heap (which is the common case for *RangeDescriptors) -// then this is a net win. -func MakeReplicaDescriptors(replicas []ReplicaDescriptor) ReplicaDescriptors { - return ReplicaDescriptors{wrapped: replicas} +// TODO(aayush): Add a `Size` or `NumReplicas` method to ReplicaSet and amend +// usages that call `len(replicaSet.Descriptors())` + +// MakeReplicaSet creates a ReplicaSet wrapper from a raw slice of individual +// descriptors. +func MakeReplicaSet(replicas []ReplicaDescriptor) ReplicaSet { + return ReplicaSet{wrapped: replicas} } // SafeFormat implements redact.SafeFormatter. -func (d ReplicaDescriptors) SafeFormat(w redact.SafePrinter, _ rune) { +func (d ReplicaSet) SafeFormat(w redact.SafePrinter, _ rune) { for i, desc := range d.wrapped { if i > 0 { w.SafeRune(',') @@ -85,16 +77,26 @@ func (d ReplicaDescriptors) SafeFormat(w redact.SafePrinter, _ rune) { } } -func (d ReplicaDescriptors) String() string { +func (d ReplicaSet) String() string { return redact.StringWithoutMarkers(d) } -// All returns every replica in the set, including both voter replicas and -// learner replicas. Voter replicas are ordered first in the returned slice. -func (d ReplicaDescriptors) All() []ReplicaDescriptor { +// Descriptors returns every replica descriptor in the set, including both voter +// replicas and learner replicas. Voter replicas are ordered first in the +// returned slice. +func (d ReplicaSet) Descriptors() []ReplicaDescriptor { return d.wrapped } +func predVoterFull(rDesc ReplicaDescriptor) bool { + switch rDesc.GetType() { + case VOTER_FULL: + return true + default: + } + return false +} + func predVoterFullOrIncoming(rDesc ReplicaDescriptor) bool { switch rDesc.GetType() { case VOTER_FULL, VOTER_INCOMING: @@ -112,12 +114,15 @@ func predNonVoter(rDesc ReplicaDescriptor) bool { return rDesc.GetType() == NON_VOTER } -// Voters returns the current and future voter replicas in the set. This means -// that during an atomic replication change, only the replicas that will be -// voters once the change completes will be returned; "outgoing" voters will not -// be returned even though they do in the current state retain their voting -// rights. When no atomic membership change is ongoing, this is simply the set -// of all non-learners. +func predVoterOrNonVoter(rDesc ReplicaDescriptor) bool { + return predVoterFull(rDesc) || predNonVoter(rDesc) +} + +// Voters returns a ReplicaSet of current and future voter replicas in `d`. This +// means that during an atomic replication change, only the replicas that will +// be voters once the change completes will be returned; "outgoing" voters will +// not be returned even though they do in the current state retain their voting +// rights. // // This may allocate, but it also may return the underlying slice as a // performance optimization, so it's not safe to modify the returned value. @@ -125,13 +130,20 @@ func predNonVoter(rDesc ReplicaDescriptor) bool { // TODO(tbg): go through the callers and figure out the few which want a // different subset of voters. Consider renaming this method so that it's // more descriptive. -func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { +func (d ReplicaSet) Voters() ReplicaSet { return d.Filter(predVoterFullOrIncoming) } -// Learners returns the learner replicas in the set. This may allocate, but it -// also may return the underlying slice as a performance optimization, so it's -// not safe to modify the returned value. +// VoterDescriptors returns the descriptors of current and future voter replicas +// in the set. +func (d ReplicaSet) VoterDescriptors() []ReplicaDescriptor { + return d.FilterToDescriptors(predVoterFullOrIncoming) +} + +// LearnerDescriptors returns a slice of ReplicaDescriptors corresponding to +// learner replicas in `d`. This may allocate, but it also may return the +// underlying slice as a performance optimization, so it's not safe to modify +// the returned value. // // A learner is a participant in a raft group that accepts messages but doesn't // vote. This means it doesn't affect raft quorum and thus doesn't affect the @@ -211,17 +223,17 @@ func (d ReplicaDescriptors) Voters() []ReplicaDescriptor { // However, it means a slow learner can slow down regular traffic. // // For some related mega-comments, see Replica.sendSnapshot. -func (d ReplicaDescriptors) Learners() []ReplicaDescriptor { - return d.Filter(predLearner) +func (d ReplicaSet) LearnerDescriptors() []ReplicaDescriptor { + return d.FilterToDescriptors(predLearner) } -// NonVoters returns the non-voting replicas in the set. Non-voting replicas are -// treated differently from learner replicas. Learners are a temporary internal -// state used to make atomic replication changes less disruptive to the system. -// Even though learners and non-voting replicas are both etcd/raft LearnerNodes -// under the hood, non-voting replicas are meant to be a user-visible state and -// are explicitly chosen to be placed inside certain localities via zone -// configs. +// NonVoters returns a ReplicaSet containing only the non-voters in `d`. +// Non-voting replicas are treated differently from learner replicas. +// Learners are a temporary internal state used to make atomic +// replication changes less disruptive to the system. Even though learners and +// non-voting replicas are both etcd/raft LearnerNodes under the hood, +// non-voting replicas are meant to be a user-visible state and are explicitly +// chosen to be placed inside certain localities via zone configs. // // Key differences between how we treat (ephemeral) learners and (persistent) // non-voting replicas: - Non-voting replicas rely on the raft snapshot queue in @@ -238,13 +250,35 @@ func (d ReplicaDescriptors) Learners() []ReplicaDescriptor { // TODO(aayush): Expand this documentation once `AdminRelocateRange` knows how // to deal with such replicas & range merges no longer block due to the presence // of non-voting replicas. -func (d ReplicaDescriptors) NonVoters() []ReplicaDescriptor { +func (d ReplicaSet) NonVoters() ReplicaSet { return d.Filter(predNonVoter) } -// Filter returns only the replica descriptors for which the supplied method -// returns true. The memory returned may be shared with the receiver. -func (d ReplicaDescriptors) Filter(pred func(rDesc ReplicaDescriptor) bool) []ReplicaDescriptor { +// NonVoterDescriptors returns the non-voting replica descriptors in the set. +func (d ReplicaSet) NonVoterDescriptors() []ReplicaDescriptor { + return d.FilterToDescriptors(predNonVoter) +} + +// VoterAndNonVoterDescriptors returns the descriptors of VOTER_FULL/NON_VOTER +// replicas in the set. This set will not contain learners or, during an atomic +// replication change, incoming or outgoing voters. Notably, this set must +// encapsulate all replicas of a range for a range merge to proceed. +func (d ReplicaSet) VoterAndNonVoterDescriptors() []ReplicaDescriptor { + return d.FilterToDescriptors(predVoterOrNonVoter) +} + +// Filter returns a ReplicaSet corresponding to the replicas for which the +// supplied predicate returns true. +func (d ReplicaSet) Filter(pred func(rDesc ReplicaDescriptor) bool) ReplicaSet { + return MakeReplicaSet(d.FilterToDescriptors(pred)) +} + +// FilterToDescriptors returns only the replica descriptors for which the +// supplied method returns true. The memory returned may be shared with the +// receiver. +func (d ReplicaSet) FilterToDescriptors( + pred func(rDesc ReplicaDescriptor) bool, +) []ReplicaDescriptor { // Fast path when all or none match to avoid allocations. fastpath := true out := d.wrapped @@ -268,28 +302,38 @@ func (d ReplicaDescriptors) Filter(pred func(rDesc ReplicaDescriptor) bool) []Re // setting the InternalReplicas field of a RangeDescriptor. When possible the // SetReplicas method of RangeDescriptor should be used instead, this is only // here for the convenience of tests. -func (d ReplicaDescriptors) AsProto() []ReplicaDescriptor { +func (d ReplicaSet) AsProto() []ReplicaDescriptor { return d.wrapped } // DeepCopy returns a copy of this set of replicas. Modifications to the // returned set will not affect this one and vice-versa. -func (d ReplicaDescriptors) DeepCopy() ReplicaDescriptors { - return ReplicaDescriptors{ +func (d ReplicaSet) DeepCopy() ReplicaSet { + return ReplicaSet{ wrapped: append([]ReplicaDescriptor(nil), d.wrapped...), } } +// Contains returns true if the set contains rDesc. +func (d ReplicaSet) Contains(rDesc ReplicaDescriptor) bool { + descs := d.Descriptors() + for i := range descs { + repl := &descs[i] + if repl.StoreID == rDesc.StoreID && repl.NodeID == rDesc.NodeID { + return true + } + } + return false +} + // AddReplica adds the given replica to this set. -func (d *ReplicaDescriptors) AddReplica(r ReplicaDescriptor) { +func (d *ReplicaSet) AddReplica(r ReplicaDescriptor) { d.wrapped = append(d.wrapped, r) } // RemoveReplica removes the matching replica from this set. If it wasn't found // to remove, false is returned. -func (d *ReplicaDescriptors) RemoveReplica( - nodeID NodeID, storeID StoreID, -) (ReplicaDescriptor, bool) { +func (d *ReplicaSet) RemoveReplica(nodeID NodeID, storeID StoreID) (ReplicaDescriptor, bool) { idx := -1 for i := range d.wrapped { if d.wrapped[i].NodeID == nodeID && d.wrapped[i].StoreID == storeID { @@ -309,7 +353,7 @@ func (d *ReplicaDescriptors) RemoveReplica( // InAtomicReplicationChange returns true if the descriptor is in the middle of // an atomic replication change. -func (d ReplicaDescriptors) InAtomicReplicationChange() bool { +func (d ReplicaSet) InAtomicReplicationChange() bool { for _, rDesc := range d.wrapped { switch rDesc.GetType() { case VOTER_INCOMING, VOTER_OUTGOING, VOTER_DEMOTING: @@ -323,7 +367,7 @@ func (d ReplicaDescriptors) InAtomicReplicationChange() bool { } // ConfState returns the Raft configuration described by the set of replicas. -func (d ReplicaDescriptors) ConfState() raftpb.ConfState { +func (d ReplicaSet) ConfState() raftpb.ConfState { var cs raftpb.ConfState joint := d.InAtomicReplicationChange() // The incoming config is taken verbatim from the full voters when the @@ -359,7 +403,7 @@ func (d ReplicaDescriptors) ConfState() raftpb.ConfState { // CanMakeProgress reports whether the given descriptors can make progress at the // replication layer. This is more complicated than just counting the number // of replicas due to the existence of joint quorums. -func (d ReplicaDescriptors) CanMakeProgress(liveFunc func(descriptor ReplicaDescriptor) bool) bool { +func (d ReplicaSet) CanMakeProgress(liveFunc func(descriptor ReplicaDescriptor) bool) bool { isVoterOldConfig := func(rDesc ReplicaDescriptor) bool { switch rDesc.GetType() { case VOTER_FULL, VOTER_OUTGOING, VOTER_DEMOTING: @@ -385,8 +429,8 @@ func (d ReplicaDescriptors) CanMakeProgress(liveFunc func(descriptor ReplicaDesc } } - votersOldGroup := d.Filter(isVoterOldConfig) - liveVotersOldGroup := d.Filter(isBoth(isVoterOldConfig, liveFunc)) + votersOldGroup := d.FilterToDescriptors(isVoterOldConfig) + liveVotersOldGroup := d.FilterToDescriptors(isBoth(isVoterOldConfig, liveFunc)) n := len(votersOldGroup) // Empty groups succeed by default, to match the Raft implementation. @@ -394,13 +438,25 @@ func (d ReplicaDescriptors) CanMakeProgress(liveFunc func(descriptor ReplicaDesc return false } - votersNewGroup := d.Filter(isVoterNewConfig) - liveVotersNewGroup := d.Filter(isBoth(isVoterNewConfig, liveFunc)) + votersNewGroup := d.FilterToDescriptors(isVoterNewConfig) + liveVotersNewGroup := d.FilterToDescriptors(isBoth(isVoterNewConfig, liveFunc)) n = len(votersNewGroup) return len(liveVotersNewGroup) >= n/2+1 } +// ReplicationTargets returns a slice of ReplicationTargets corresponding to +// each of the replicas in the set. +func (d ReplicaSet) ReplicationTargets() (out []ReplicationTarget) { + descs := d.Descriptors() + out = make([]ReplicationTarget, len(descs)) + for i := range descs { + repl := &descs[i] + out[i].NodeID, out[i].StoreID = repl.NodeID, repl.StoreID + } + return out +} + // IsAddition returns true if `c` refers to a replica addition operation. func (c ReplicaChangeType) IsAddition() bool { switch c { diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index 9477016747da..8fac0c4dd3ff 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -61,9 +61,9 @@ func TestVotersLearnersAll(t *testing.T) { } for _, test := range tests { t.Run("", func(t *testing.T) { - r := MakeReplicaDescriptors(test) + r := MakeReplicaSet(test) seen := map[ReplicaDescriptor]struct{}{} - for _, voter := range r.Voters() { + for _, voter := range r.VoterDescriptors() { typ := voter.GetType() switch typ { case VOTER_FULL, VOTER_INCOMING: @@ -72,14 +72,14 @@ func TestVotersLearnersAll(t *testing.T) { assert.FailNow(t, "unexpectedly got a %s as Voter()", typ) } } - for _, learner := range r.Learners() { + for _, learner := range r.LearnerDescriptors() { seen[learner] = struct{}{} assert.Equal(t, LEARNER, learner.GetType()) } - all := r.All() + all := r.Descriptors() // Make sure that VOTER_OUTGOING is the only type that is skipped both - // by Learners() and Voters() + // by LearnerDescriptors() and VoterDescriptors() for _, rd := range all { typ := rd.GetType() if _, seen := seen[rd]; !seen { @@ -126,21 +126,21 @@ func TestReplicaDescriptorsRemove(t *testing.T) { }, } for i, test := range tests { - r := MakeReplicaDescriptors(test.replicas) - lenBefore := len(r.All()) + r := MakeReplicaSet(test.replicas) + lenBefore := len(r.Descriptors()) removedDesc, ok := r.RemoveReplica(test.remove.NodeID, test.remove.StoreID) assert.Equal(t, test.expected, ok, "testcase %d", i) if ok { assert.Equal(t, test.remove.NodeID, removedDesc.NodeID, "testcase %d", i) assert.Equal(t, test.remove.StoreID, removedDesc.StoreID, "testcase %d", i) - assert.Equal(t, lenBefore-1, len(r.All()), "testcase %d", i) + assert.Equal(t, lenBefore-1, len(r.Descriptors()), "testcase %d", i) } else { - assert.Equal(t, lenBefore, len(r.All()), "testcase %d", i) + assert.Equal(t, lenBefore, len(r.Descriptors()), "testcase %d", i) } - for _, voter := range r.Voters() { + for _, voter := range r.VoterDescriptors() { assert.Equal(t, VOTER_FULL, voter.GetType(), "testcase %d", i) } - for _, learner := range r.Learners() { + for _, learner := range r.LearnerDescriptors() { assert.Equal(t, LEARNER, learner.GetType(), "testcase %d", i) } } @@ -202,7 +202,7 @@ func TestReplicaDescriptorsConfState(t *testing.T) { for _, test := range tests { t.Run("", func(t *testing.T) { - r := MakeReplicaDescriptors(test.in) + r := MakeReplicaSet(test.in) cs := r.ConfState() require.Equal(t, test.out, raft.DescribeConfState(cs)) }) @@ -308,7 +308,7 @@ func TestReplicaDescriptorsCanMakeProgress(t *testing.T) { rds = append(rds, rDesc.ReplicaDescriptor) } - act := MakeReplicaDescriptors(rds).CanMakeProgress(func(rd ReplicaDescriptor) bool { + act := MakeReplicaSet(rds).CanMakeProgress(func(rd ReplicaDescriptor) bool { for _, rdi := range test.rds { if rdi.ReplicaID == rd.ReplicaID { return rdi.live @@ -346,7 +346,7 @@ func TestReplicaDescriptorsCanMakeProgressRandom(t *testing.T) { liveness[i] = (livenessBits >> i & 1) == 0 } - rng := MakeReplicaDescriptors(rds) + rng := MakeReplicaSet(rds) crdbCanMakeProgress := rng.CanMakeProgress(func(rd ReplicaDescriptor) bool { return liveness[rd.ReplicaID-1] diff --git a/pkg/server/admin.go b/pkg/server/admin.go index a792f1656afa..887f9f0035cc 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -774,7 +774,7 @@ func (s *adminServer) statsForSpan( if err := kv.Value.GetProto(&rng); err != nil { return nil, s.serverError(err) } - for _, repl := range rng.Replicas().All() { + for _, repl := range rng.Replicas().Descriptors() { nodeIDs[repl.NodeID] = struct{}{} } } @@ -1704,7 +1704,7 @@ func (s *adminServer) DecommissionStatus( if err := row.ValueProto(&rangeDesc); err != nil { return errors.Wrapf(err, "%s: unable to unmarshal range descriptor", row.Key) } - for _, r := range rangeDesc.Replicas().All() { + for _, r := range rangeDesc.Replicas().Descriptors() { if _, ok := replicaCounts[r.NodeID]; ok { replicaCounts[r.NodeID]++ } @@ -1891,7 +1891,7 @@ func (s *adminServer) DataDistribution( return err } - for _, replicaDesc := range rangeDesc.Replicas().All() { + for _, replicaDesc := range rangeDesc.Replicas().Descriptors() { tableInfo, ok := tableInfosByTableID[tableID] if !ok { // This is a database, skip. diff --git a/pkg/server/node.go b/pkg/server/node.go index 5b81b06aba85..f623dd125a76 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -1068,7 +1068,7 @@ func (n *Node) ResetQuorum( } // Update the range descriptor and update meta ranges for the descriptor, removing all replicas. - deadReplicas := append([]roachpb.ReplicaDescriptor(nil), desc.Replicas().All()...) + deadReplicas := append([]roachpb.ReplicaDescriptor(nil), desc.Replicas().Descriptors()...) for _, rd := range deadReplicas { desc.RemoveReplica(rd.NodeID, rd.StoreID) } diff --git a/pkg/server/status.go b/pkg/server/status.go index 5ce918a85926..366dc6ec9f1a 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -1416,7 +1416,7 @@ func (s *statusServer) RaftDebug( desc := node.Range.State.Desc // Check for whether replica should be GCed. containsNode := false - for _, replica := range desc.Replicas().All() { + for _, replica := range desc.Replicas().Descriptors() { if replica.NodeID == node.NodeID { containsNode = true } diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index b20fdc9849e9..3d1e679c089a 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -1126,15 +1126,16 @@ func (ts *TestServer) MergeRanges(leftKey roachpb.Key) (roachpb.RangeDescriptor, return ts.LookupRange(leftKey) } -// SplitRange splits the range containing splitKey. +// SplitRangeWithExpiration splits the range containing splitKey with a sticky +// bit expiring at expirationTime. // The right range created by the split starts at the split key and extends to the // original range's end key. // Returns the new descriptors of the left and right ranges. // // splitKey must correspond to a SQL table key (it must end with a family ID / // col ID). -func (ts *TestServer) SplitRange( - splitKey roachpb.Key, +func (ts *TestServer) SplitRangeWithExpiration( + splitKey roachpb.Key, expirationTime hlc.Timestamp, ) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { ctx := context.Background() splitRKey, err := keys.Addr(splitKey) @@ -1146,7 +1147,7 @@ func (ts *TestServer) SplitRange( Key: splitKey, }, SplitKey: splitKey, - ExpirationTime: hlc.MaxTimestamp, + ExpirationTime: expirationTime, } _, pErr := kv.SendWrapped(ctx, ts.DB().NonTransactionalSender(), &splitReq) if pErr != nil { @@ -1220,6 +1221,14 @@ func (ts *TestServer) SplitRange( return leftRangeDesc, rightRangeDesc, nil } +// SplitRange is exactly like SplitRangeWithExpiration, except that it creates a +// split with a sticky bit that never expires. +func (ts *TestServer) SplitRange( + splitKey roachpb.Key, +) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { + return ts.SplitRangeWithExpiration(splitKey, hlc.MaxTimestamp) +} + // GetRangeLease returns the current lease for the range containing key, and a // timestamp taken from the node. // diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index b7640b98d0e6..b803e6cb2f4b 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2554,9 +2554,9 @@ CREATE TABLE crdb_internal.ranges_no_leases ( return nil, err } - voterReplicas := append([]roachpb.ReplicaDescriptor(nil), desc.Replicas().Voters()...) + voterReplicas := append([]roachpb.ReplicaDescriptor(nil), desc.Replicas().VoterDescriptors()...) var learnerReplicaStoreIDs []int - for _, rd := range desc.Replicas().Learners() { + for _, rd := range desc.Replicas().LearnerDescriptors() { learnerReplicaStoreIDs = append(learnerReplicaStoreIDs, int(rd.StoreID)) } sort.Slice(voterReplicas, func(i, j int) bool { diff --git a/pkg/sql/relocate.go b/pkg/sql/relocate.go index 1a0cb88d2a90..1adabfcc0eac 100644 --- a/pkg/sql/relocate.go +++ b/pkg/sql/relocate.go @@ -52,6 +52,9 @@ func (n *relocateNode) startExec(runParams) error { return nil } +// TODO(aayush): Extend EXPERIMENTAL_RELOCATE syntax to support relocating +// non-voting replicas. + func (n *relocateNode) Next(params runParams) (bool, error) { // Each Next call relocates one range (corresponding to one row from n.rows). // TODO(radu): perform multiple relocations in parallel. @@ -131,7 +134,9 @@ func (n *relocateNode) Next(params runParams) (bool, error) { return false, err } } else { - if err := params.p.ExecCfg().DB.AdminRelocateRange(params.ctx, rowKey, relocationTargets); err != nil { + if err := params.p.ExecCfg().DB.AdminRelocateRange( + params.ctx, rowKey, relocationTargets, nil, /* nonVoterTargets */ + ); err != nil { return false, err } } diff --git a/pkg/testutils/serverutils/test_cluster_shim.go b/pkg/testutils/serverutils/test_cluster_shim.go index bf6cf7346364..ea334f956cfd 100644 --- a/pkg/testutils/serverutils/test_cluster_shim.go +++ b/pkg/testutils/serverutils/test_cluster_shim.go @@ -87,11 +87,22 @@ type TestClusterInterface interface { startKey roachpb.Key, targets ...roachpb.ReplicationTarget, ) (roachpb.RangeDescriptor, error) - // RemoveNonVoters removes one or more learners from a range. + // AddNonVotersOrFatal is the same as AddNonVoters but will fatal if it fails. + AddNonVotersOrFatal( + t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget, + ) roachpb.RangeDescriptor + + // RemoveNonVoters removes one or more non-voters from a range. RemoveNonVoters( startKey roachpb.Key, targets ...roachpb.ReplicationTarget, ) (roachpb.RangeDescriptor, error) + // RemoveNonVotersOrFatal is the same as RemoveNonVoters but will fatal if it + // fails. + RemoveNonVotersOrFatal( + t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget, + ) roachpb.RangeDescriptor + // FindRangeLeaseHolder returns the current lease holder for the given range. // In particular, it returns one particular node's (the hint, if specified) view // of the current lease. @@ -136,6 +147,10 @@ type TestClusterInterface interface { // as kv scratch space (it doesn't overlap system spans or SQL tables). The // range is lazily split off on the first call to ScratchRange. ScratchRange(t testing.TB) roachpb.Key + + // WaitForFullReplication waits until all stores in the cluster + // have no ranges with replication pending. + WaitForFullReplication() error } // TestClusterFactory encompasses the actual implementation of the shim diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index ec69a8745a58..db15b7a3f422 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -175,9 +175,7 @@ type TestServerInterface interface { ClusterSettings() *cluster.Settings // SplitRange splits the range containing splitKey. - SplitRange( - splitKey roachpb.Key, - ) (left roachpb.RangeDescriptor, right roachpb.RangeDescriptor, err error) + SplitRange(splitKey roachpb.Key) (left roachpb.RangeDescriptor, right roachpb.RangeDescriptor, err error) // MergeRanges merges the range containing leftKey with the following adjacent // range. diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index f3f164b424b6..5370ecb6cf98 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -488,6 +488,20 @@ func (tc *TestCluster) LookupRangeOrFatal(t testing.TB, key roachpb.Key) roachpb return desc } +// SplitRangeWithExpiration splits the range containing splitKey with a sticky +// bit expiring at expirationTime. +// The right range created by the split starts at the split key and extends to the +// original range's end key. +// Returns the new descriptors of the left and right ranges. +// +// splitKey must correspond to a SQL table key (it must end with a family ID / +// col ID). +func (tc *TestCluster) SplitRangeWithExpiration( + splitKey roachpb.Key, expirationTime hlc.Timestamp, +) (roachpb.RangeDescriptor, roachpb.RangeDescriptor, error) { + return tc.Servers[0].SplitRangeWithExpiration(splitKey, expirationTime) +} + // SplitRange splits the range containing splitKey. // The right range created by the split starts at the split key and extends to the // original range's end key. @@ -583,6 +597,17 @@ func (tc *TestCluster) AddNonVoters( return tc.addReplica(startKey, roachpb.ADD_NON_VOTER, targets...) } +// AddNonVotersOrFatal is part of TestClusterInterface. +func (tc *TestCluster) AddNonVotersOrFatal( + t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget, +) roachpb.RangeDescriptor { + desc, err := tc.addReplica(startKey, roachpb.ADD_NON_VOTER, targets...) + if err != nil { + t.Fatal(err) + } + return desc +} + // AddVotersMulti is part of TestClusterInterface. func (tc *TestCluster) AddVotersMulti( kts ...serverutils.KeyAndTargets, @@ -705,6 +730,18 @@ func (tc *TestCluster) RemoveNonVoters( return tc.changeReplicas(roachpb.REMOVE_NON_VOTER, keys.MustAddr(startKey), targets...) } +// RemoveNonVotersOrFatal is part of TestClusterInterface. +func (tc *TestCluster) RemoveNonVotersOrFatal( + t testing.TB, startKey roachpb.Key, targets ...roachpb.ReplicationTarget, +) roachpb.RangeDescriptor { + desc, err := tc.RemoveNonVoters(startKey, targets...) + if err != nil { + t.Fatalf(`could not remove %v replicas from range containing %s: %+v`, + targets, startKey, err) + } + return desc +} + // TransferRangeLease is part of the TestServerInterface. func (tc *TestCluster) TransferRangeLease( rangeDesc roachpb.RangeDescriptor, dest roachpb.ReplicationTarget, @@ -740,8 +777,8 @@ func (tc *TestCluster) FindRangeLease( } } else { hint = &roachpb.ReplicationTarget{ - NodeID: rangeDesc.Replicas().All()[0].NodeID, - StoreID: rangeDesc.Replicas().All()[0].StoreID} + NodeID: rangeDesc.Replicas().Descriptors()[0].NodeID, + StoreID: rangeDesc.Replicas().Descriptors()[0].StoreID} } // Find the server indicated by the hint and send a LeaseInfoRequest through @@ -816,7 +853,7 @@ func (tc *TestCluster) WaitForSplitAndInitialization(startKey roachpb.Key) error startKey, desc.StartKey) } // Once we've verified the split, make sure that replicas exist. - for _, rDesc := range desc.Replicas().All() { + for _, rDesc := range desc.Replicas().Descriptors() { store, err := tc.findMemberStore(rDesc.StoreID) if err != nil { return err @@ -901,6 +938,10 @@ func (tc *TestCluster) WaitForFullReplication() error { log.Infof(context.TODO(), "%s has %d underreplicated ranges", s, n) notReplicated = true } + if n := s.Metrics().OverReplicatedRangeCount.Value(); n > 0 { + log.Infof(context.TODO(), "%s has %d overreplicated ranges", s, n) + notReplicated = true + } return nil }) if err != nil {