From bfe698a6d3c91a0db31099cbf05b0e7ca25754d4 Mon Sep 17 00:00:00 2001 From: Aayush Shah Date: Wed, 18 Nov 2020 02:10:31 -0500 Subject: [PATCH] kvserver: make AdminRelocateRange work with non-voting replicas This commit teaches `RelocateRange` to work with non-voters. This lets the merge queue rebalance a range that has non-voters so the merge can actually proceed, as range merges require that replica sets of the LHS and RHS ranges must be aligned. Release note: None --- pkg/kv/batch.go | 7 +- pkg/kv/db.go | 4 +- pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/allocator.go | 18 +- pkg/kv/kvserver/allocator_scorer.go | 16 +- pkg/kv/kvserver/client_merge_test.go | 295 +++- pkg/kv/kvserver/client_relocate_range_test.go | 47 +- pkg/kv/kvserver/client_replica_test.go | 19 +- pkg/kv/kvserver/closed_timestamp_test.go | 12 +- pkg/kv/kvserver/merge_queue.go | 62 +- pkg/kv/kvserver/replica_command.go | 331 ++-- pkg/kv/kvserver/replica_learner_test.go | 16 +- pkg/kv/kvserver/replica_send.go | 2 +- pkg/kv/kvserver/replica_test.go | 2 +- pkg/kv/kvserver/store_rebalancer.go | 13 +- pkg/roachpb/api.pb.go | 1374 +++++++++-------- pkg/roachpb/api.proto | 3 +- pkg/roachpb/metadata_replicas.go | 90 +- pkg/sql/relocate.go | 7 +- 19 files changed, 1426 insertions(+), 893 deletions(-) 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/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 2dbe7c06a38d..ca3b3665d346 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -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/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index aabb52444ae4..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,7 +1990,7 @@ 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().VoterDescriptors(), 1) @@ -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_relocate_range_test.go b/pkg/kv/kvserver/client_relocate_range_test.go index 36dc40269544..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 } @@ -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_test.go b/pkg/kv/kvserver/client_replica_test.go index 37ace6142a7e..6cbc0ded37bd 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -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/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index 26a4028f1a96..db0a4699d038 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -886,7 +886,8 @@ func setupClusterForClosedTSTestingWithSplitRanges( dbName, tableName := "cttest", "kv" tc, _, _ := setupClusterForClosedTSTesting(ctx, t, targetDuration, closeFraction, clusterArgs, dbName, tableName) - leftDesc, rightDesc := splitDummyRangeInTestCluster(t, tc, dbName, tableName) + leftDesc, rightDesc := splitDummyRangeInTestCluster(t, tc, dbName, tableName, + hlc.Timestamp{} /* splitExpirationTime */) return tc, leftDesc, rightDesc } @@ -895,7 +896,10 @@ func setupClusterForClosedTSTestingWithSplitRanges( // 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, + 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')`, @@ -927,7 +931,9 @@ func splitDummyRangeInTestCluster( if err != nil { t.Fatalf("failed to encode split key: %+v", err) } - leftDesc, rightDesc := tcImpl.SplitRangeOrFatal(t, k) + 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) diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 7cd17218e7c4..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().Descriptors(), rhsDesc.Replicas().Descriptors() - - // 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/replica_command.go b/pkg/kv/kvserver/replica_command.go index ccd57ccfade1..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" @@ -627,16 +628,15 @@ 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.FilterToDescriptors(predFullVoter)) != len(lReplicas.Descriptors()) { - 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.FilterToDescriptors(predFullVoter)) != len(rReplicas.Descriptors()) { - return errors.Errorf("cannot merge range with non-voter replicas on rhs: %s", rReplicas) - } - if !replicaSetsEqual(lReplicas.Descriptors(), rReplicas.Descriptors()) { + if !replicasCollocated(lReplicas.Descriptors(), rReplicas.Descriptors()) { return errors.Errorf("ranges not collocated; %s != %s", lReplicas, rReplicas) } mergeReplicas := lReplicas.Descriptors() @@ -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 ReplicaSet.NonVoterDescriptors() for why we do this. + // comment block above ReplicaSet.NonVoters() for why we do this. r.store.raftSnapshotQueue.AddAsync(ctx, r, raftSnapshotPriority) } @@ -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().Descriptors() - if len(rangeReplicas) != len(desc.Replicas().VoterDescriptors()) { - // 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, diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index a8960e1e118d..a32f807b0d30 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -875,15 +875,17 @@ 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().VoterDescriptors() - require.Len(t, voters, len(targets)) + 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().LearnerDescriptors()) require.Empty(t, desc.Replicas().FilterToDescriptors(predIncoming)) @@ -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) } } 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 e3cc48f79398..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) } } diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index fcd63c695f1f..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) @@ -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++ 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/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index b08fb078f4bc..f5f63773132c 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -58,6 +58,9 @@ type ReplicaSet struct { wrapped []ReplicaDescriptor } +// 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 { @@ -85,6 +88,15 @@ 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: @@ -102,12 +114,15 @@ func predNonVoter(rDesc ReplicaDescriptor) bool { return rDesc.GetType() == NON_VOTER } -// VoterDescriptors returns the descriptors of 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. @@ -115,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 ReplicaSet) Voters() ReplicaSet { + return d.Filter(predVoterFullOrIncoming) +} + +// VoterDescriptors returns the descriptors of current and future voter replicas +// in the set. func (d ReplicaSet) VoterDescriptors() []ReplicaDescriptor { return d.FilterToDescriptors(predVoterFullOrIncoming) } -// LearnerDescriptors 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. +// 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 @@ -205,7 +227,7 @@ func (d ReplicaSet) LearnerDescriptors() []ReplicaDescriptor { return d.FilterToDescriptors(predLearner) } -// NonVoterDescriptors returns the non-voting replica descriptors in the set. +// 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 @@ -228,12 +250,32 @@ func (d ReplicaSet) LearnerDescriptors() []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 ReplicaSet) NonVoters() ReplicaSet { + return d.Filter(predNonVoter) +} + +// NonVoterDescriptors returns the non-voting replica descriptors in the set. func (d ReplicaSet) NonVoterDescriptors() []ReplicaDescriptor { return d.FilterToDescriptors(predNonVoter) } -// FilterToDescriptors returns only the replica descriptors for which the supplied method -// returns true. The memory returned may be shared with the receiver. +// 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 { @@ -272,6 +314,18 @@ func (d ReplicaSet) DeepCopy() ReplicaSet { } } +// 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 *ReplicaSet) AddReplica(r ReplicaDescriptor) { d.wrapped = append(d.wrapped, r) @@ -391,6 +445,18 @@ func (d ReplicaSet) CanMakeProgress(liveFunc func(descriptor ReplicaDescriptor) 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/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 } }