diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 098d9336d619..c4649d6bd5a0 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -685,7 +685,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) @@ -695,7 +697,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 89a2c001e695..d144d3c7aa25 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -608,10 +608,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/allocator.go b/pkg/kv/kvserver/allocator.go index 6ff9efa6ce23..888087e94a69 100644 --- a/pkg/kv/kvserver/allocator.go +++ b/pkg/kv/kvserver/allocator.go @@ -494,17 +494,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, ) @@ -559,17 +559,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 adc107d41c76..34503ee8cf22 100644 --- a/pkg/kv/kvserver/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator_scorer.go @@ -408,16 +408,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) { @@ -432,14 +432,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 eca67fe3c0f6..b19d08700ea4 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -3468,6 +3468,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) @@ -3548,37 +3564,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) { @@ -3587,7 +3587,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) { @@ -3600,18 +3600,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) @@ -3620,7 +3620,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 @@ -3629,13 +3629,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{ @@ -3647,33 +3647,158 @@ 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 + leftNonVoters, rightNonVoters, leftVoters, rightVoters []int + } + + // NB: The setup code for the "relocate-range" subtest places a single voter + // replica on (n1,s1) for both left and right range. + tests := []test{ + { + name: "collocated-per-type", + leftNonVoters: []int{1}, + rightNonVoters: []int{1}, + leftVoters: []int{2, 3, 4}, + rightVoters: []int{2, 3, 4}, + }, + { + name: "collocated-overall", + leftNonVoters: []int{1, 2}, + rightNonVoters: []int{3, 4}, + leftVoters: []int{3, 4}, + rightVoters: []int{1, 2}, + }, + { + name: "collocated-voters-only", + leftNonVoters: []int{2}, + rightNonVoters: []int{1}, + leftVoters: []int{3, 4}, + rightVoters: []int{3, 4}, + }, + { + name: "collocated-non-voters-only", + leftNonVoters: []int{1, 2}, + rightNonVoters: []int{1, 2}, + leftVoters: []int{3}, + rightVoters: []int{4}, + }, + { + name: "not-collocated", + leftNonVoters: []int{2}, + rightNonVoters: []int{1}, + leftVoters: []int{3}, + rightVoters: []int{4}, + }, + { + name: "partially-collocated-voters-only", + leftNonVoters: []int{1}, + rightNonVoters: []int{2}, + leftVoters: []int{2, 3}, + rightVoters: []int{1, 4}, + }, + { + name: "partially-collocated-non-voters-only", + leftNonVoters: []int{1, 3}, + rightNonVoters: []int{1, 2}, + leftVoters: []int{4}, + rightVoters: []int{4}, + }, + { + name: "partially-collocated", + leftNonVoters: []int{1, 3}, + rightNonVoters: []int{1, 2}, + leftVoters: []int{2}, + rightVoters: []int{4}, + }, + } + + 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. + tc.ServerConn(0).Exec(`SET CLUSTER SETTING kv.range_merge.queue_enabled=true`) + 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 _, id := range subtest.leftNonVoters { + tc.AddNonVotersOrFatal(t, leftDesc.StartKey.AsRawKey(), tc.Target(id)) + leftDesc = tc.LookupRangeOrFatal(t, leftDesc.StartKey.AsRawKey()) + } + for _, id := range subtest.rightNonVoters { + tc.AddNonVotersOrFatal(t, rightDesc.StartKey.AsRawKey(), tc.Target(id)) + rightDesc = tc.LookupRangeOrFatal(t, rightDesc.StartKey.AsRawKey()) + } + for _, id := range subtest.leftVoters { + tc.AddVotersOrFatal(t, leftDesc.StartKey.AsRawKey(), tc.Target(id)) + leftDesc = tc.LookupRangeOrFatal(t, leftDesc.StartKey.AsRawKey()) + } + for _, id := range subtest.rightVoters { + tc.AddVotersOrFatal(t, rightDesc.StartKey.AsRawKey(), tc.Target(id)) + rightDesc = tc.LookupRangeOrFatal(t, rightDesc.StartKey.AsRawKey()) + } + // Sanity check + verifyUnmerged(t, store, leftDesc.StartKey, rightDesc.StartKey) + + 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 d63dc416d1db..69d4d7aa54b6 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -2205,7 +2205,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 { @@ -2225,9 +2227,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) @@ -2711,7 +2716,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) @@ -2746,7 +2753,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 5dfcb9f5812f..90870afb61da 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -563,7 +563,8 @@ func TestClosedTimestampInactiveAfterSubsumption(t *testing.T) { // duration at the beginning of the test until we have the merge txn // suspended at its commit trigger, and then change it back down to // `testingTargetDuration`. - tc, leftDesc, rightDesc := setupClusterForClosedTsTestingWithSplitRanges(ctx, t, 5*time.Second, testingCloseFraction, clusterArgs) + tc, leftDesc, rightDesc := setupClusterForClosedTsTestingWithSplitRanges( + ctx, t, 5*time.Second, testingCloseFraction, clusterArgs) defer tc.Stopper().Stop(ctx) leftLeaseholder := getCurrentLeaseholder(t, tc, leftDesc) @@ -878,7 +879,7 @@ func setupClusterForClosedTsTestingWithSplitRanges( ) (serverutils.TestClusterInterface, roachpb.RangeDescriptor, roachpb.RangeDescriptor) { dbName := "cttest" tc, _, _ := setupClusterForClosedTsTesting(ctx, t, targetDuration, closeFraction, clusterArgs, dbName) - leftDesc, rightDesc := splitDummyRangeInTestCluster(t, tc, dbName) + leftDesc, rightDesc := splitDummyRangeInTestCluster(t, tc, dbName, "kv" /* tableName */, hlc.MaxTimestamp) return tc, leftDesc, rightDesc } @@ -887,13 +888,17 @@ func setupClusterForClosedTsTestingWithSplitRanges( // the {dbname}.kv 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 string, + t *testing.T, + tc serverutils.TestClusterInterface, + dbName string, + tableName string, + splitExpirationTime hlc.Timestamp, ) (roachpb.RangeDescriptor, roachpb.RangeDescriptor) { db0 := tc.ServerConn(0) - if _, err := db0.Exec(fmt.Sprintf(`INSERT INTO %s.kv VALUES(1, '%s')`, dbName, "foo")); err != nil { + if _, err := db0.Exec(fmt.Sprintf(`INSERT INTO %s.%s VALUES(1, '%s')`, dbName, tableName, "foo")); err != nil { t.Fatal(err) } - if _, err := db0.Exec(fmt.Sprintf(`INSERT INTO %s.kv VALUES(3, '%s')`, dbName, "foo")); err != nil { + if _, err := db0.Exec(fmt.Sprintf(`INSERT INTO %s.%s VALUES(3, '%s')`, dbName, tableName, "foo")); err != nil { t.Fatal(err) } // Manually split the table to have easier access to descriptors. @@ -911,13 +916,15 @@ func splitDummyRangeInTestCluster( // Split at `k` so that the `kv` table has exactly two ranges: [1,2) and [2, // Max). This split will never be merged by the merge queue so the expiration // time doesn't matter here. - tcImpl.SplitRangeOrFatal(t, k) + _, _, err = tcImpl.SplitRangeWithExpiration(k, splitExpirationTime) + require.NoError(t, err) idxPrefix = keys.SystemSQLCodec.IndexPrefix(uint32(tableID), 1) k, err = rowenc.EncodeTableKey(idxPrefix, tree.NewDInt(2), encoding.Ascending) if err != nil { t.Fatalf("failed to encode split key: %+v", err) } - leftDesc, rightDesc := 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 fdfe98952fd9..c2eb17eacb37 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -286,41 +286,52 @@ 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) { + voterTargets, nonVoterTargets := TargetsToCollocateRHSDuringMerge(lhsDesc.Replicas(), rhsDesc.Replicas()) + // 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 } } @@ -360,6 +371,60 @@ func (mq *mergeQueue) process( return true, nil } +// TargetsToCollocateRHSDuringMerge computes the stores which the voters and +// non-voters of an RHS (the right hand side range during a merge) should be +// relocated to. This method produces relocation targets that avoid superfluous +// data movement. +// +// 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 TargetsToCollocateRHSDuringMerge( + leftRepls roachpb.ReplicaSet, rightRepls roachpb.ReplicaSet, +) ([]roachpb.ReplicationTarget, []roachpb.ReplicationTarget) { + numVoters, numNonVoters := len(leftRepls.VoterDescriptors()), len(leftRepls.NonVoterDescriptors()) + notInRight := func(desc roachpb.ReplicaDescriptor) bool { + return !rightRepls.Contains(desc) + } + + // Sets of replicas that exist on the LHS but not on the RHS + nonCollocated := leftRepls.Filter(notInRight) + nonCollocatedVoters := nonCollocated.Voters().Descriptors() + nonCollocatedNonVoters := nonCollocated.NonVoters().Descriptors() + + // We bootstrap our result set by first including the replicas (voting and + // non-voting) that _are_ collocated, as these will stay unchanged. + finalVoters := rightRepls.Voters().Filter(leftRepls.Contains).DeepCopy() + finalNonVoters := rightRepls.NonVoters().Filter(leftRepls.Contains).DeepCopy() + + for len(finalVoters.Descriptors()) != numVoters { + // 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(nonCollocatedVoters) != 0 { + finalVoters.AddReplica(nonCollocatedVoters[0]) + nonCollocatedVoters = nonCollocatedVoters[1:] + } else if len(nonCollocatedNonVoters) != 0 { + finalVoters.AddReplica(nonCollocatedNonVoters[0]) + nonCollocatedNonVoters = nonCollocatedNonVoters[1:] + } + } + + for len(finalNonVoters.Descriptors()) != numNonVoters { + // 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(nonCollocatedNonVoters) != 0 { + finalNonVoters.AddReplica(nonCollocatedNonVoters[0]) + nonCollocatedNonVoters = nonCollocatedNonVoters[1:] + } else if len(nonCollocatedVoters) != 0 { + finalNonVoters.AddReplica(nonCollocatedVoters[0]) + nonCollocatedVoters = nonCollocatedVoters[1:] + } + } + return finalVoters.ReplicationTargets(), finalNonVoters.ReplicationTargets() +} + func (mq *mergeQueue) timer(time.Duration) time.Duration { return MergeQueueInterval.Get(&mq.store.ClusterSettings().SV) } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 2ebb5b58cc98..8b24d9f8c75b 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -626,16 +626,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() @@ -2132,9 +2131,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 } @@ -2256,10 +2255,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) @@ -2267,6 +2268,49 @@ func (s *Store) AdminRelocateRange( } rangeDesc = *newDesc + if len(nonVoterTargets) > 0 { + rangeDesc, err = s.relocateReplicas(ctx, rangeDesc, nonVoterTargets, false) + if err != nil { + return err + } + } + if len(voterTargets) > 0 { + _, err = s.relocateReplicas(ctx, rangeDesc, voterTargets, true) + 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, + targets []roachpb.ReplicationTarget, + areVoters bool, +) (roachpb.RangeDescriptor, error) { + startKey := rangeDesc.StartKey.AsRawKey() canRetry := func(err error) bool { allowlist := []string{ snapshotApplySemBusyMsg, @@ -2280,8 +2324,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 @@ -2295,39 +2337,18 @@ 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, targets, areVoters) if err != nil { - return err + return rangeDesc, err } - if leaseTarget != nil { + if areVoters && leaseTarget != nil { // NB: we may need to transfer even if there are no ops, to make // sure the attempt is made to make the first target the final // leaseholder. @@ -2335,7 +2356,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) @@ -2353,7 +2374,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) @@ -2368,18 +2389,19 @@ func (s *Store) AdminRelocateRange( } } } - } func (s *Store) relocateOne( - ctx context.Context, desc *roachpb.RangeDescriptor, targets []roachpb.ReplicationTarget, + ctx context.Context, + desc *roachpb.RangeDescriptor, + targets []roachpb.ReplicationTarget, + isVoter bool, ) ([]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() @@ -2394,58 +2416,37 @@ 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. + var rangeReplicas []roachpb.ReplicaDescriptor + var addOp, removeOp roachpb.ReplicaChangeType + if isVoter { + rangeReplicas = desc.Replicas().VoterDescriptors() + addOp, removeOp = roachpb.ADD_VOTER, roachpb.REMOVE_VOTER + } else { + rangeReplicas = desc.Replicas().NonVoterDescriptors() + addOp, removeOp = roachpb.ADD_NON_VOTER, roachpb.REMOVE_NON_VOTER + } + // 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, - }) - } - } + existingRepls := roachpb.MakeReplicaSet(rangeReplicas).ReplicationTargets() + targetsToAdd := diff(targets, existingRepls) // targets - existingRepls + targetsToRemove := diff(existingRepls, targets) // existingRepls - targets 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 + candidateTargets := targetsToAdd if storeHasReplica(targets[0].StoreID, candidateTargets) { candidateTargets = []roachpb.ReplicaDescriptor{ {NodeID: targets[0].NodeID, StoreID: targets[0].StoreID}, @@ -2464,25 +2465,25 @@ func (s *Store) relocateOne( } candidateDescs = append(candidateDescs, *store) } - storeList = makeStoreList(candidateDescs) + candidateStoreList := makeStoreList(candidateDescs) targetStore, _ := s.allocator.allocateTargetFromList( ctx, - storeList, + candidateStoreList, zone, rangeReplicas, 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()) + 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{ NodeID: target.NodeID, @@ -2493,19 +2494,19 @@ func (s *Store) relocateOne( } var transferTarget *roachpb.ReplicationTarget - if len(removeTargets) > 0 { + if len(targetsToRemove) > 0 { // Pick a replica to remove. Note that rangeReplicas 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, rangeReplicas) if err != nil { return nil, nil, errors.Wrapf(err, "unable to select removal target from %v; current replicas %v", - removeTargets, rangeReplicas) + targetsToRemove, rangeReplicas) } removalTarget := roachpb.ReplicationTarget{ NodeID: targetStore.NodeID, @@ -2556,7 +2557,7 @@ func (s *Store) relocateOne( // illegal). if ok { ops = append(ops, roachpb.MakeReplicationChanges( - roachpb.REMOVE_VOTER, + removeOp, removalTarget)...) } } @@ -2570,6 +2571,25 @@ func (s *Store) relocateOne( return ops, transferTarget, nil } +func diff(first, second []roachpb.ReplicationTarget ) (diff []roachpb.ReplicaDescriptor) { + for _, t := range first { + found := false + for _, replicaDesc := range second { + 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 82ad6aad7628..1e998648e9d2 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -594,7 +594,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 461a4bc1ecc0..a85c46164aaf 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -6413,7 +6413,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 9dd06160ced8..66efe71b6592 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -291,7 +291,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, @@ -308,12 +308,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) @@ -333,8 +334,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 8806b751f919..a7dd744c801e 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -73,7 +73,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -101,7 +101,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_fbe02bd09cf526bf, []int{1} } type ChecksumMode int32 @@ -148,7 +148,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_fbe02bd09cf526bf, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -179,7 +179,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_fbe02bd09cf526bf, []int{3} } type ExternalStorageProvider int32 @@ -220,7 +220,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_fbe02bd09cf526bf, []int{4} } type MVCCFilter int32 @@ -243,7 +243,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_fbe02bd09cf526bf, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -332,7 +332,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_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,7 +403,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_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_fbe02bd09cf526bf, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +437,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_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_fbe02bd09cf526bf, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +480,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_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_fbe02bd09cf526bf, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +523,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_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_fbe02bd09cf526bf, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -557,7 +557,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_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_fbe02bd09cf526bf, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -639,7 +639,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_290a2f113c3e6c17, []int{6} + return fileDescriptor_api_fbe02bd09cf526bf, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -674,7 +674,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_290a2f113c3e6c17, []int{7} + return fileDescriptor_api_fbe02bd09cf526bf, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +720,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_290a2f113c3e6c17, []int{8} + return fileDescriptor_api_fbe02bd09cf526bf, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +754,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_290a2f113c3e6c17, []int{9} + return fileDescriptor_api_fbe02bd09cf526bf, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +794,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_290a2f113c3e6c17, []int{10} + return fileDescriptor_api_fbe02bd09cf526bf, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -831,7 +831,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_290a2f113c3e6c17, []int{11} + return fileDescriptor_api_fbe02bd09cf526bf, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +865,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_290a2f113c3e6c17, []int{12} + return fileDescriptor_api_fbe02bd09cf526bf, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -899,7 +899,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_290a2f113c3e6c17, []int{13} + return fileDescriptor_api_fbe02bd09cf526bf, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -951,7 +951,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_290a2f113c3e6c17, []int{14} + return fileDescriptor_api_fbe02bd09cf526bf, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -988,7 +988,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_290a2f113c3e6c17, []int{15} + return fileDescriptor_api_fbe02bd09cf526bf, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1043,7 +1043,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_290a2f113c3e6c17, []int{16} + return fileDescriptor_api_fbe02bd09cf526bf, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1077,7 +1077,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_290a2f113c3e6c17, []int{17} + return fileDescriptor_api_fbe02bd09cf526bf, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1118,7 +1118,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_290a2f113c3e6c17, []int{18} + return fileDescriptor_api_fbe02bd09cf526bf, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1152,7 +1152,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_290a2f113c3e6c17, []int{19} + return fileDescriptor_api_fbe02bd09cf526bf, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1203,7 +1203,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_290a2f113c3e6c17, []int{20} + return fileDescriptor_api_fbe02bd09cf526bf, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1255,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_290a2f113c3e6c17, []int{21} + return fileDescriptor_api_fbe02bd09cf526bf, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1306,7 +1306,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_290a2f113c3e6c17, []int{22} + return fileDescriptor_api_fbe02bd09cf526bf, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1358,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_290a2f113c3e6c17, []int{23} + return fileDescriptor_api_fbe02bd09cf526bf, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1411,7 +1411,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_290a2f113c3e6c17, []int{24} + return fileDescriptor_api_fbe02bd09cf526bf, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1448,7 +1448,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_290a2f113c3e6c17, []int{25} + return fileDescriptor_api_fbe02bd09cf526bf, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1492,7 +1492,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_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1540,7 +1540,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_290a2f113c3e6c17, []int{26} + return fileDescriptor_api_fbe02bd09cf526bf, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1576,7 +1576,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_290a2f113c3e6c17, []int{27} + return fileDescriptor_api_fbe02bd09cf526bf, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1687,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_290a2f113c3e6c17, []int{28} + return fileDescriptor_api_fbe02bd09cf526bf, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1733,7 +1733,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_290a2f113c3e6c17, []int{29} + return fileDescriptor_api_fbe02bd09cf526bf, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1794,7 +1794,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_290a2f113c3e6c17, []int{30} + return fileDescriptor_api_fbe02bd09cf526bf, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1829,7 +1829,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_290a2f113c3e6c17, []int{31} + return fileDescriptor_api_fbe02bd09cf526bf, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1868,7 +1868,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_290a2f113c3e6c17, []int{32} + return fileDescriptor_api_fbe02bd09cf526bf, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1903,7 +1903,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_290a2f113c3e6c17, []int{33} + return fileDescriptor_api_fbe02bd09cf526bf, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1946,7 +1946,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_290a2f113c3e6c17, []int{34} + return fileDescriptor_api_fbe02bd09cf526bf, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +1981,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_290a2f113c3e6c17, []int{35} + return fileDescriptor_api_fbe02bd09cf526bf, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +2019,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_290a2f113c3e6c17, []int{36} + return fileDescriptor_api_fbe02bd09cf526bf, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2052,7 +2052,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_290a2f113c3e6c17, []int{37} + return fileDescriptor_api_fbe02bd09cf526bf, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2087,7 +2087,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_290a2f113c3e6c17, []int{38} + return fileDescriptor_api_fbe02bd09cf526bf, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2145,7 +2145,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_290a2f113c3e6c17, []int{39} + return fileDescriptor_api_fbe02bd09cf526bf, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2180,7 +2180,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_290a2f113c3e6c17, []int{40} + return fileDescriptor_api_fbe02bd09cf526bf, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2209,15 +2209,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_290a2f113c3e6c17, []int{41} + return fileDescriptor_api_fbe02bd09cf526bf, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2250,7 +2251,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_290a2f113c3e6c17, []int{42} + return fileDescriptor_api_fbe02bd09cf526bf, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,7 +2290,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_290a2f113c3e6c17, []int{43} + return fileDescriptor_api_fbe02bd09cf526bf, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2326,7 +2327,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_290a2f113c3e6c17, []int{44} + return fileDescriptor_api_fbe02bd09cf526bf, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2364,7 +2365,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_290a2f113c3e6c17, []int{45} + return fileDescriptor_api_fbe02bd09cf526bf, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2398,7 +2399,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_290a2f113c3e6c17, []int{45, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2432,7 +2433,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_290a2f113c3e6c17, []int{46} + return fileDescriptor_api_fbe02bd09cf526bf, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2501,7 +2502,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_290a2f113c3e6c17, []int{47} + return fileDescriptor_api_fbe02bd09cf526bf, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2544,7 +2545,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_290a2f113c3e6c17, []int{48} + return fileDescriptor_api_fbe02bd09cf526bf, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2591,7 +2592,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_290a2f113c3e6c17, []int{49} + return fileDescriptor_api_fbe02bd09cf526bf, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2627,7 +2628,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_290a2f113c3e6c17, []int{50} + return fileDescriptor_api_fbe02bd09cf526bf, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2671,7 +2672,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_290a2f113c3e6c17, []int{51} + return fileDescriptor_api_fbe02bd09cf526bf, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2715,7 +2716,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_290a2f113c3e6c17, []int{52} + return fileDescriptor_api_fbe02bd09cf526bf, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2775,7 +2776,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_290a2f113c3e6c17, []int{53} + return fileDescriptor_api_fbe02bd09cf526bf, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2811,7 +2812,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_290a2f113c3e6c17, []int{54} + return fileDescriptor_api_fbe02bd09cf526bf, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2858,7 +2859,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_290a2f113c3e6c17, []int{55} + return fileDescriptor_api_fbe02bd09cf526bf, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2893,7 +2894,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_290a2f113c3e6c17, []int{56} + return fileDescriptor_api_fbe02bd09cf526bf, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2944,7 +2945,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_290a2f113c3e6c17, []int{57} + return fileDescriptor_api_fbe02bd09cf526bf, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2979,7 +2980,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_290a2f113c3e6c17, []int{58} + return fileDescriptor_api_fbe02bd09cf526bf, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3016,7 +3017,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_290a2f113c3e6c17, []int{59} + return fileDescriptor_api_fbe02bd09cf526bf, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3050,7 +3051,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_290a2f113c3e6c17, []int{60} + return fileDescriptor_api_fbe02bd09cf526bf, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3095,7 +3096,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_290a2f113c3e6c17, []int{61} + return fileDescriptor_api_fbe02bd09cf526bf, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3129,7 +3130,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_290a2f113c3e6c17, []int{62} + return fileDescriptor_api_fbe02bd09cf526bf, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3173,7 +3174,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_290a2f113c3e6c17, []int{63} + return fileDescriptor_api_fbe02bd09cf526bf, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3222,7 +3223,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_290a2f113c3e6c17, []int{64} + return fileDescriptor_api_fbe02bd09cf526bf, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3259,7 +3260,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_290a2f113c3e6c17, []int{65} + return fileDescriptor_api_fbe02bd09cf526bf, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3296,7 +3297,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_290a2f113c3e6c17, []int{66} + return fileDescriptor_api_fbe02bd09cf526bf, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3331,7 +3332,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_290a2f113c3e6c17, []int{67} + return fileDescriptor_api_fbe02bd09cf526bf, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3386,7 +3387,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_290a2f113c3e6c17, []int{68} + return fileDescriptor_api_fbe02bd09cf526bf, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3423,7 +3424,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_290a2f113c3e6c17, []int{69} + return fileDescriptor_api_fbe02bd09cf526bf, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3463,7 +3464,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_290a2f113c3e6c17, []int{70} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3497,7 +3498,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_290a2f113c3e6c17, []int{70, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3530,7 +3531,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_290a2f113c3e6c17, []int{70, 1} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3572,7 +3573,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_290a2f113c3e6c17, []int{70, 2} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3611,7 +3612,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_290a2f113c3e6c17, []int{70, 3} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3647,7 +3648,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_290a2f113c3e6c17, []int{70, 4} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3686,7 +3687,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_290a2f113c3e6c17, []int{70, 5} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3728,7 +3729,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_290a2f113c3e6c17, []int{70, 6} + return fileDescriptor_api_fbe02bd09cf526bf, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3768,7 +3769,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_290a2f113c3e6c17, []int{71} + return fileDescriptor_api_fbe02bd09cf526bf, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3802,7 +3803,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_290a2f113c3e6c17, []int{72} + return fileDescriptor_api_fbe02bd09cf526bf, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3838,7 +3839,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_290a2f113c3e6c17, []int{73} + return fileDescriptor_api_fbe02bd09cf526bf, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3908,7 +3909,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_290a2f113c3e6c17, []int{74} + return fileDescriptor_api_fbe02bd09cf526bf, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3960,7 +3961,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_290a2f113c3e6c17, []int{75} + return fileDescriptor_api_fbe02bd09cf526bf, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3996,7 +3997,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_290a2f113c3e6c17, []int{76} + return fileDescriptor_api_fbe02bd09cf526bf, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4036,7 +4037,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_290a2f113c3e6c17, []int{76, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4087,7 +4088,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_290a2f113c3e6c17, []int{77} + return fileDescriptor_api_fbe02bd09cf526bf, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4122,7 +4123,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_290a2f113c3e6c17, []int{77, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4158,7 +4159,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_290a2f113c3e6c17, []int{77, 1} + return fileDescriptor_api_fbe02bd09cf526bf, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4193,7 +4194,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_290a2f113c3e6c17, []int{78} + return fileDescriptor_api_fbe02bd09cf526bf, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4231,7 +4232,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_290a2f113c3e6c17, []int{79} + return fileDescriptor_api_fbe02bd09cf526bf, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4268,7 +4269,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_290a2f113c3e6c17, []int{80} + return fileDescriptor_api_fbe02bd09cf526bf, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4301,7 +4302,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_290a2f113c3e6c17, []int{80, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4346,7 +4347,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_290a2f113c3e6c17, []int{81} + return fileDescriptor_api_fbe02bd09cf526bf, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4384,7 +4385,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_290a2f113c3e6c17, []int{82} + return fileDescriptor_api_fbe02bd09cf526bf, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4437,7 +4438,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_290a2f113c3e6c17, []int{83} + return fileDescriptor_api_fbe02bd09cf526bf, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4471,7 +4472,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_290a2f113c3e6c17, []int{84} + return fileDescriptor_api_fbe02bd09cf526bf, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4515,7 +4516,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_290a2f113c3e6c17, []int{85} + return fileDescriptor_api_fbe02bd09cf526bf, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4549,7 +4550,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_290a2f113c3e6c17, []int{86} + return fileDescriptor_api_fbe02bd09cf526bf, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4588,7 +4589,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_290a2f113c3e6c17, []int{87} + return fileDescriptor_api_fbe02bd09cf526bf, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4622,7 +4623,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_290a2f113c3e6c17, []int{88} + return fileDescriptor_api_fbe02bd09cf526bf, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4671,7 +4672,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_290a2f113c3e6c17, []int{89} + return fileDescriptor_api_fbe02bd09cf526bf, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4720,7 +4721,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_290a2f113c3e6c17, []int{90} + return fileDescriptor_api_fbe02bd09cf526bf, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4755,7 +4756,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_290a2f113c3e6c17, []int{91} + return fileDescriptor_api_fbe02bd09cf526bf, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4797,7 +4798,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_290a2f113c3e6c17, []int{92} + return fileDescriptor_api_fbe02bd09cf526bf, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4880,7 +4881,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_290a2f113c3e6c17, []int{93} + return fileDescriptor_api_fbe02bd09cf526bf, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6330,7 +6331,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_290a2f113c3e6c17, []int{94} + return fileDescriptor_api_fbe02bd09cf526bf, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7865,7 +7866,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_290a2f113c3e6c17, []int{95} + return fileDescriptor_api_fbe02bd09cf526bf, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7903,7 +7904,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_290a2f113c3e6c17, []int{96} + return fileDescriptor_api_fbe02bd09cf526bf, []int{96} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7939,7 +7940,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{97} + return fileDescriptor_api_fbe02bd09cf526bf, []int{97} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7976,7 +7977,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{98} + return fileDescriptor_api_fbe02bd09cf526bf, []int{98} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8055,7 +8056,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_290a2f113c3e6c17, []int{98, 0} + return fileDescriptor_api_fbe02bd09cf526bf, []int{98, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8093,7 +8094,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_290a2f113c3e6c17, []int{99} + return fileDescriptor_api_fbe02bd09cf526bf, []int{99} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8131,7 +8132,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_290a2f113c3e6c17, []int{100} + return fileDescriptor_api_fbe02bd09cf526bf, []int{100} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8170,7 +8171,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_290a2f113c3e6c17, []int{101} + return fileDescriptor_api_fbe02bd09cf526bf, []int{101} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8211,7 +8212,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_290a2f113c3e6c17, []int{102} + return fileDescriptor_api_fbe02bd09cf526bf, []int{102} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8252,7 +8253,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_290a2f113c3e6c17, []int{103} + return fileDescriptor_api_fbe02bd09cf526bf, []int{103} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8289,7 +8290,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_290a2f113c3e6c17, []int{104} + return fileDescriptor_api_fbe02bd09cf526bf, []int{104} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8326,7 +8327,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_290a2f113c3e6c17, []int{105} + return fileDescriptor_api_fbe02bd09cf526bf, []int{105} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8370,7 +8371,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_290a2f113c3e6c17, []int{106} + return fileDescriptor_api_fbe02bd09cf526bf, []int{106} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8402,7 +8403,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_290a2f113c3e6c17, []int{107} + return fileDescriptor_api_fbe02bd09cf526bf, []int{107} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8443,7 +8444,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_290a2f113c3e6c17, []int{108} + return fileDescriptor_api_fbe02bd09cf526bf, []int{108} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8483,7 +8484,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_290a2f113c3e6c17, []int{109} + return fileDescriptor_api_fbe02bd09cf526bf, []int{109} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8519,7 +8520,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_290a2f113c3e6c17, []int{110} + return fileDescriptor_api_fbe02bd09cf526bf, []int{110} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8558,7 +8559,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_290a2f113c3e6c17, []int{111} + return fileDescriptor_api_fbe02bd09cf526bf, []int{111} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8601,7 +8602,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_290a2f113c3e6c17, []int{112} + return fileDescriptor_api_fbe02bd09cf526bf, []int{112} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -10747,8 +10748,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())) @@ -10759,6 +10760,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 } @@ -16408,8 +16421,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)) } @@ -24169,7 +24188,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 { @@ -24193,8 +24243,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 @@ -38505,19 +38555,19 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_290a2f113c3e6c17) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_fbe02bd09cf526bf) } -var fileDescriptor_api_290a2f113c3e6c17 = []byte{ - // 8124 bytes of a gzipped FileDescriptorProto +var fileDescriptor_api_fbe02bd09cf526bf = []byte{ + // 8151 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0xd1, 0x95, 0xba, 0x9b, 0xad, 0x99, 0x69, 0xa9, 0xab, 0xff, 0x7b, 0x67, 0xa4, 0xe9, 0xee, 0x1d, 0xcf, 0x78, 0x7a, 0x3c, 0x6b, 0x91, 0x62, 0x37, 0x29, 0xb5, 0xd4, 0xea, 0x22, 0xd5, 0xed, 0x19, 0xaf, 0x53, 0x5b, 0xaa, 0xba, 0xa2, 0x6a, 0x45, - 0x56, 0xb1, 0xab, 0x8a, 0xfa, 0x19, 0x20, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x0b, 0x3f, 0x38, - 0xc8, 0x1a, 0x4e, 0xe2, 0x75, 0x1c, 0xc7, 0x01, 0x82, 0x20, 0x01, 0x12, 0xc4, 0x41, 0x90, 0xd8, - 0x2f, 0x46, 0xb2, 0x08, 0x0c, 0x64, 0xfd, 0x14, 0x23, 0x40, 0x14, 0x5b, 0x9b, 0xb7, 0x04, 0x46, - 0x90, 0x97, 0x00, 0xf3, 0x10, 0x04, 0xf7, 0xa7, 0xfe, 0xc8, 0x22, 0x45, 0xf5, 0xd6, 0x20, 0x03, - 0xec, 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, + 0x56, 0xb1, 0xab, 0x8a, 0xfa, 0x19, 0x20, 0x40, 0x1c, 0x1b, 0xc9, 0x02, 0x01, 0x16, 0x7e, 0x70, + 0x90, 0x35, 0x9c, 0xc4, 0xeb, 0x38, 0x89, 0x03, 0x04, 0x41, 0x02, 0x24, 0x88, 0x83, 0x20, 0xb1, + 0x5f, 0x8c, 0x64, 0x11, 0x18, 0xc8, 0xfa, 0x29, 0x46, 0x80, 0x28, 0xb6, 0x36, 0x6f, 0x09, 0x8c, + 0x20, 0x2f, 0x01, 0xe6, 0x21, 0x08, 0xee, 0x4f, 0xfd, 0x91, 0x45, 0x8a, 0xea, 0xad, 0x81, 0x07, + 0xf0, 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, 0x14, 0xcc, 0x5a, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x57, 0x94, 0xae, 0xbe, 0xdc, 0xb5, 0x4c, 0xc7, 0x44, 0xb3, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x32, 0xbf, 0xb9, 0x70, 0xff, 0xe0, 0x70, 0xe5, 0xe0, 0xd0, 0xc6, 0xd6, 0x21, 0xb6, 0x56, 0x54, 0xd3, 0x50, 0x7b, 0x96, 0x85, 0x0d, 0xf5, 0x64, 0xa5, @@ -38529,492 +38579,494 @@ var fileDescriptor_api_290a2f113c3e6c17 = []byte{ 0xaa, 0x1b, 0x2d, 0xf7, 0x7f, 0x77, 0x77, 0xc5, 0xc2, 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0xae, 0x62, 0xb8, 0xd5, 0x6d, 0x99, 0x2d, 0x93, 0xfe, 0x5c, 0x21, 0xbf, 0x38, 0xf5, 0x5a, 0xcb, 0x34, 0x5b, 0x6d, 0xbc, 0x42, 0xaf, 0x76, 0x7b, 0x7b, 0x2b, 0x5a, 0xcf, 0x52, 0x1c, 0xdd, 0xe4, 0x5c, - 0xe2, 0xbf, 0x12, 0x20, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42, 0x57, + 0xe2, 0xbf, 0x16, 0x20, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42, 0x57, 0x21, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x25, 0xe1, 0xee, 0x4c, 0x79, 0xfa, 0xcb, 0xd3, 0xc5, 0xe4, 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x2d, 0xc1, 0x34, 0x36, 0x34, 0x99, 0xdc, 0x4e, 0x85, 0x6f, 0x4f, 0x61, 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x25, 0xe1, 0xee, 0x64, 0xf9, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x65, 0xd5, 0xec, 0xac, 0x78, 0xfd, 0xa4, 0xed, 0xfa, 0xbf, 0x57, 0xba, 0x07, 0xad, 0x95, 0x7e, 0x1d, 0x2d, 0x37, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, - 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0xbb, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xcd, + 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0x3b, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xcd, 0xdf, 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x9a, 0x67, 0x1f, 0x5e, 0x5b, 0x1e, 0x18, 0x11, 0xcb, 0x4d, 0x4b, 0x31, 0x6c, 0x45, 0x25, 0xcd, 0x97, 0x48, 0x51, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, 0x98, 0x2a, 0x92, 0x36, 0x2a, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xb2, 0xe4, 0x37, 0xba, 0x0a, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x43, 0x93, 0xd2, 0xb4, 0xd1, 0xeb, 0x6c, 0xe0, 0x13, 0x1b, 0xfd, 0x02, 0x5c, 0xd6, 0x70, 0xd7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, 0xb6, 0x14, 0xa3, 0x85, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xd2, 0xd4, 0x52, 0xf2, 0x6e, 0xf6, 0xe1, - 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x0f, 0x4f, 0x17, 0x27, 0xa4, + 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x8f, 0x4e, 0x17, 0x27, 0xa4, 0x79, 0x5f, 0x82, 0x77, 0xcb, 0x46, 0x0d, 0xc8, 0xf1, 0xea, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, 0xbd, 0x24, 0xdc, 0xcd, 0x3f, 0x5c, 0x8e, 0x12, 0x18, 0x52, 0x0d, 0xb9, 0xec, 0x75, 0xb0, 0x44, 0xb9, 0xa4, 0x19, 0x2b, 0x70, 0x85, 0xde, 0x82, 0x0c, 0x69, 0xc9, 0xee, 0x89, 0x83, 0xed, 0x52, 0x9a, 0x36, 0x85, 0x34, 0xad, 0x4c, 0xae, 0xc5, 0x4f, 0x61, 0x26, 0xc8, 0x8a, 0x10, 0xe4, 0xa5, 0x6a, 0x63, 0x67, 0xb3, 0x2a, 0xef, 0x6c, 0x6d, 0x6c, 0x3d, 0x7f, 0xb5, 0x55, 0x9c, 0x40, 0xf3, - 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14, 0x16, 0x52, 0xbf, - 0xf6, 0xbb, 0xd7, 0x26, 0xc4, 0x6d, 0x80, 0xa7, 0xd8, 0xe1, 0x03, 0x0c, 0x95, 0x61, 0x6a, 0x9f, - 0xd6, 0xa7, 0x24, 0x50, 0x4d, 0x2f, 0x45, 0x56, 0x3c, 0x30, 0x18, 0xcb, 0x69, 0xa2, 0x8d, 0x1f, - 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0xff, 0x48, 0x80, 0x2c, 0x15, 0xc9, 0xda, 0x87, 0x2a, 0x7d, - 0x32, 0xaf, 0x9f, 0xab, 0x8c, 0x41, 0xa1, 0x68, 0x19, 0x26, 0x0f, 0x95, 0x76, 0x0f, 0x97, 0x12, - 0x54, 0x46, 0x29, 0x42, 0xc6, 0x4b, 0x72, 0x5f, 0x62, 0xc5, 0xd0, 0x63, 0x98, 0xd1, 0x0d, 0x07, - 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xe7, 0xb0, 0x65, 0x59, 0x69, 0x7a, 0x21, 0xfe, 0x4b, 0x01, 0x60, - 0xbb, 0x17, 0xa7, 0x52, 0xd0, 0x37, 0xc7, 0xac, 0x3f, 0x1f, 0x5d, 0xbc, 0x15, 0x97, 0x61, 0x4a, - 0x37, 0xda, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0xf8, 0x15, 0x9a, 0x87, 0xc9, 0xdd, 0xb6, 0x6e, 0x68, - 0x74, 0x3e, 0xa4, 0x25, 0x76, 0x21, 0x4a, 0x90, 0xa5, 0xb5, 0x8e, 0x51, 0xef, 0xe2, 0xbf, 0x4f, - 0xc0, 0xa5, 0x8a, 0x69, 0x68, 0x3a, 0x99, 0x92, 0x4a, 0xfb, 0x6b, 0xa1, 0x95, 0x75, 0x08, 0x4c, - 0x3e, 0x19, 0x1f, 0x77, 0xc7, 0xec, 0x63, 0xe4, 0x73, 0x55, 0x8f, 0xbb, 0x94, 0x16, 0xad, 0x49, - 0xf4, 0x4d, 0xb8, 0xa2, 0xb4, 0xdb, 0xe6, 0x91, 0xac, 0xef, 0xc9, 0x9a, 0x89, 0x6d, 0xd9, 0x30, - 0x1d, 0x19, 0x1f, 0xeb, 0xb6, 0x43, 0x4d, 0x49, 0x5a, 0x9a, 0xa3, 0xb7, 0xeb, 0x7b, 0x6b, 0x26, - 0xb6, 0xb7, 0x4c, 0xa7, 0x4a, 0x6e, 0x91, 0x79, 0x4a, 0x2a, 0xc3, 0xe6, 0xe9, 0x14, 0x31, 0xbf, - 0x52, 0x1a, 0x1f, 0x77, 0xd9, 0x3c, 0xfd, 0x25, 0xb8, 0xdc, 0xaf, 0xc7, 0x38, 0xfb, 0xe9, 0x4f, - 0x04, 0xc8, 0xd7, 0x0d, 0xdd, 0xf9, 0x5a, 0x74, 0x90, 0xa7, 0xd4, 0x64, 0x50, 0xa9, 0xf7, 0xa1, - 0xb8, 0xa7, 0xe8, 0xed, 0xe7, 0x46, 0xd3, 0xec, 0xec, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0x6b, 0x7d, - 0x80, 0x2e, 0xbe, 0x84, 0x82, 0xd7, 0x9a, 0x38, 0xd5, 0xe4, 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, - 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x0d, 0x19, 0xdd, 0x95, 0x4b, 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, - 0x83, 0xd9, 0xc0, 0x53, 0xe3, 0x34, 0x8b, 0x64, 0x69, 0xc0, 0x47, 0xb2, 0xdf, 0x47, 0x64, 0x69, - 0xc0, 0x47, 0xcc, 0x8c, 0x35, 0x20, 0xb7, 0x86, 0xdb, 0xd8, 0xc1, 0x71, 0x5a, 0xf7, 0x1d, 0xc8, - 0xbb, 0x42, 0xe3, 0xec, 0x98, 0xbf, 0x23, 0x00, 0xe2, 0x72, 0xc9, 0x6a, 0x1a, 0x67, 0xdf, 0x2c, - 0x12, 0x17, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x05, 0xd8, 0x98, 0x04, 0x46, 0xa2, 0xee, 0x80, 0x6f, - 0x65, 0x53, 0x41, 0x2b, 0xcb, 0xdd, 0x98, 0x23, 0x98, 0x0b, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, - 0x3a, 0x25, 0x96, 0x92, 0x41, 0x5f, 0x8d, 0x12, 0xc5, 0xef, 0x0b, 0x30, 0x5b, 0x69, 0x63, 0xc5, - 0x8a, 0x5d, 0x23, 0xdf, 0x82, 0xb4, 0x86, 0x15, 0x8d, 0x36, 0x99, 0x4d, 0xec, 0x77, 0x02, 0x52, - 0x88, 0x47, 0xbb, 0xbc, 0xdf, 0x56, 0x97, 0x9b, 0xae, 0xaf, 0xcb, 0x67, 0xb7, 0xc7, 0x24, 0x7e, - 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0xdf, 0x17, 0x00, 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, - 0xcd, 0x5e, 0x83, 0xac, 0xa3, 0x58, 0x2d, 0xec, 0xc8, 0xc4, 0x8b, 0xbf, 0x48, 0xcb, 0x81, 0xf1, - 0x11, 0xb2, 0xf8, 0x39, 0xcc, 0x85, 0xea, 0x17, 0x67, 0xe3, 0xff, 0x97, 0x00, 0xd9, 0x86, 0xaa, - 0x18, 0x71, 0xb6, 0xfa, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, 0x7b, 0xa6, 0xd5, 0x51, 0x1c, 0x3a, - 0xc4, 0xf3, 0xa1, 0x56, 0x7b, 0x1e, 0xb4, 0xaa, 0x18, 0x4f, 0x68, 0x21, 0x09, 0x6c, 0xef, 0x37, - 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0x34, 0x46, 0xd7, 0xbf, 0xfc, 0xc3, 0xf7, 0x03, 0xfc, - 0x07, 0x87, 0xcb, 0x2e, 0x88, 0x5b, 0x0e, 0x80, 0xb8, 0x65, 0xc2, 0xb1, 0xdc, 0x70, 0x2c, 0x6c, - 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, 0x4d, 0xac, 0xf5, 0x54, 0x3a, 0x59, - 0x4c, 0x89, 0xff, 0x47, 0x80, 0x19, 0xd6, 0xe4, 0x38, 0x27, 0xd6, 0x07, 0x90, 0xb2, 0xcc, 0x23, - 0x36, 0xb1, 0xb2, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x81, 0x4f, 0x82, 0x2b, 0x17, 0x2d, 0x8e, 0xca, - 0xc0, 0xfd, 0x40, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0x77, 0xa0, 0xb0, - 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x4a, 0x92, 0x55, 0x2e, 0x79, 0x77, 0x46, 0xca, 0x53, 0xb2, - 0x5b, 0x75, 0x9b, 0xb4, 0x9c, 0x8d, 0x74, 0x1b, 0xff, 0x94, 0xf5, 0xf9, 0xff, 0x15, 0xf8, 0x1c, - 0x72, 0x5b, 0xfe, 0xd3, 0xd6, 0xf5, 0xbf, 0x99, 0x80, 0x2b, 0x95, 0x7d, 0xac, 0x1e, 0x54, 0x4c, - 0xc3, 0xd6, 0x6d, 0x87, 0xe8, 0x2e, 0xce, 0xfe, 0x7f, 0x0b, 0x32, 0x47, 0xba, 0xb3, 0x2f, 0x6b, - 0xfa, 0xde, 0x1e, 0xb5, 0x73, 0x69, 0x29, 0x4d, 0x08, 0x6b, 0xfa, 0xde, 0x1e, 0x7a, 0x04, 0xa9, - 0x8e, 0xa9, 0x31, 0x77, 0x39, 0xff, 0x70, 0x31, 0x42, 0x3c, 0xad, 0x9a, 0xdd, 0xeb, 0x6c, 0x9a, - 0x1a, 0x96, 0x68, 0x61, 0x74, 0x0d, 0x40, 0x25, 0xd4, 0xae, 0xa9, 0x1b, 0x0e, 0x5f, 0x27, 0x03, - 0x14, 0x54, 0x83, 0x8c, 0x83, 0xad, 0x8e, 0x6e, 0x28, 0x0e, 0x2e, 0x4d, 0x52, 0xe5, 0xdd, 0x8c, - 0xac, 0x78, 0xb7, 0xad, 0xab, 0xca, 0x1a, 0xb6, 0x55, 0x4b, 0xef, 0x3a, 0xa6, 0xc5, 0xb5, 0xe8, - 0x33, 0x8b, 0xdf, 0x4b, 0x41, 0x69, 0x50, 0x37, 0x71, 0x8e, 0x90, 0x6d, 0x98, 0x22, 0xf8, 0xba, - 0xed, 0xf0, 0x31, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x6a, 0x40, 0x71, 0x7a, 0xdb, 0xe1, 0xd5, 0xe6, - 0x72, 0x16, 0xfe, 0xad, 0x00, 0x53, 0xec, 0x06, 0x7a, 0x00, 0x69, 0xbe, 0xa1, 0xa0, 0xd1, 0x3a, - 0x26, 0xcb, 0x97, 0xcf, 0x4e, 0x17, 0xa7, 0xd9, 0x1e, 0xc1, 0xda, 0x97, 0xfe, 0x4f, 0x69, 0x9a, - 0x96, 0xab, 0x6b, 0xa4, 0xb7, 0x6c, 0x47, 0xb1, 0x1c, 0xba, 0x6d, 0x93, 0x60, 0xb8, 0x81, 0x12, - 0x36, 0xf0, 0x09, 0x5a, 0x87, 0x29, 0xdb, 0x51, 0x9c, 0x9e, 0xcd, 0xfb, 0xeb, 0x42, 0x95, 0x6d, - 0x50, 0x4e, 0x89, 0x4b, 0x20, 0x8e, 0x8e, 0x86, 0x1d, 0x45, 0x6f, 0xd3, 0x0e, 0xcc, 0x48, 0xfc, - 0x4a, 0xfc, 0x2d, 0x01, 0xa6, 0x58, 0x51, 0x74, 0x05, 0xe6, 0xa4, 0xd5, 0xad, 0xa7, 0x55, 0xb9, - 0xbe, 0xb5, 0x56, 0x6d, 0x56, 0xa5, 0xcd, 0xfa, 0xd6, 0x6a, 0xb3, 0x5a, 0x9c, 0x40, 0x97, 0x01, - 0xb9, 0x37, 0x2a, 0xcf, 0xb7, 0x1a, 0xf5, 0x46, 0xb3, 0xba, 0xd5, 0x2c, 0x0a, 0x74, 0x6f, 0x81, - 0xd2, 0x03, 0xd4, 0x04, 0xba, 0x09, 0x4b, 0xfd, 0x54, 0xb9, 0xd1, 0x5c, 0x6d, 0x36, 0xe4, 0x6a, - 0xa3, 0x59, 0xdf, 0x5c, 0x6d, 0x56, 0xd7, 0x8a, 0xc9, 0x11, 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x5a, - 0x69, 0x16, 0x53, 0xa2, 0x03, 0x97, 0x24, 0xac, 0x9a, 0x9d, 0x6e, 0xcf, 0xc1, 0xa4, 0x96, 0x76, - 0x9c, 0x33, 0xe5, 0x0a, 0x4c, 0x6b, 0xd6, 0x89, 0x6c, 0xf5, 0x0c, 0x3e, 0x4f, 0xa6, 0x34, 0xeb, - 0x44, 0xea, 0x19, 0xe2, 0x3f, 0x17, 0xe0, 0x72, 0xff, 0x63, 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, - 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1d, 0x85, 0x3b, 0x23, 0xf7, 0x03, 0x92, 0xf8, 0x66, 0xdb, - 0xb2, 0xb7, 0xd9, 0xb6, 0xf9, 0xb2, 0x52, 0xa1, 0x15, 0x59, 0x23, 0x1c, 0xae, 0xf9, 0xa1, 0x42, - 0x28, 0x45, 0xfc, 0x9f, 0x29, 0xc8, 0x55, 0x0d, 0xad, 0x79, 0x1c, 0xeb, 0x5a, 0x72, 0x19, 0xa6, - 0x54, 0xb3, 0xd3, 0xd1, 0x1d, 0x57, 0x41, 0xec, 0x0a, 0xfd, 0x6c, 0xc0, 0x89, 0x4c, 0x8e, 0xe1, - 0x4a, 0xf9, 0xee, 0x23, 0xfa, 0x0e, 0x5c, 0x21, 0x56, 0xd3, 0x32, 0x94, 0xb6, 0xcc, 0xa4, 0xc9, - 0x8e, 0xa5, 0xb7, 0x5a, 0xd8, 0xe2, 0x1b, 0x7c, 0x77, 0x23, 0xea, 0x59, 0xe7, 0x1c, 0x15, 0xca, - 0xd0, 0x64, 0xe5, 0xa5, 0x4b, 0x7a, 0x14, 0x19, 0x7d, 0x02, 0x40, 0x96, 0x22, 0xba, 0x69, 0x68, - 0x73, 0x7b, 0x34, 0x6c, 0xd7, 0xd0, 0x35, 0x41, 0x84, 0x81, 0x5c, 0xdb, 0x68, 0x85, 0x20, 0x86, - 0xd7, 0x3d, 0xdd, 0xc2, 0xf2, 0x83, 0xae, 0x4a, 0xa1, 0x7c, 0xba, 0x9c, 0x3f, 0x3b, 0x5d, 0x04, - 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x82, 0x60, 0xbf, 0xbb, 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0x3b, - 0x12, 0x64, 0xe5, 0xe5, 0xcd, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xed, 0x63, 0x4b, 0xf6, 0x36, 0x8e, - 0xe9, 0x0e, 0x5e, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, 0xd5, 0x7e, 0xd5, 0xa9, 0xd1, 0xc2, - 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, 0x94, 0xcf, 0xae, 0xd0, 0x3d, - 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, 0x8a, 0x43, 0x56, 0x69, 0xa0, 0x25, - 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, - 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xb3, 0x54, 0x21, 0x51, 0xe3, 0xb6, 0xc1, 0x77, 0x72, - 0xb5, 0x57, 0xa4, 0x24, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd7, 0x53, 0xe9, - 0xe9, 0x62, 0x5a, 0xfc, 0xaf, 0x02, 0xe4, 0xdd, 0xe1, 0x16, 0xe7, 0xcc, 0xb8, 0x0b, 0x45, 0xd3, - 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x5f, 0x70, 0xf2, 0xa6, 0x81, 0xb7, 0x09, 0x99, - 0xa9, 0x0b, 0x6d, 0xc3, 0xac, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, 0xd4, 0x3b, 0x39, 0xbe, 0x5b, - 0x5f, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa9, 0x00, 0xb3, 0xab, 0x5a, 0x47, 0x37, 0x1a, - 0xdd, 0xb6, 0x1e, 0xeb, 0x6e, 0xc1, 0x4d, 0xc8, 0xd8, 0x44, 0xa6, 0x6f, 0xf0, 0x7d, 0xec, 0x97, - 0xa6, 0x77, 0x88, 0xe5, 0x7f, 0x06, 0x05, 0x7c, 0xdc, 0xd5, 0xd9, 0xcb, 0x00, 0x06, 0x59, 0x52, - 0xe3, 0xb7, 0x2d, 0xef, 0xf3, 0x92, 0x5b, 0xbc, 0x4d, 0x9f, 0x01, 0x0a, 0x36, 0x29, 0x4e, 0xec, - 0xf2, 0x19, 0xcc, 0x51, 0xd1, 0x3b, 0x86, 0x1d, 0xb3, 0xbe, 0xc4, 0x5f, 0x84, 0xf9, 0xb0, 0xe8, - 0x38, 0xeb, 0xfd, 0x8a, 0xf7, 0xf2, 0x26, 0xb6, 0x62, 0x85, 0x9b, 0x9e, 0xae, 0xb9, 0xe0, 0x38, - 0xeb, 0xfc, 0xab, 0x02, 0x5c, 0xa5, 0xb2, 0xe9, 0xfb, 0x92, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, - 0x2b, 0x56, 0xbe, 0x01, 0x53, 0x0c, 0xf3, 0xd2, 0xf1, 0x39, 0x59, 0xce, 0x12, 0xcf, 0xa5, 0xe1, - 0x98, 0x16, 0xf1, 0x5c, 0xf8, 0x2d, 0x51, 0x81, 0x85, 0xa8, 0x5a, 0xc4, 0xbc, 0x1d, 0x30, 0xcb, - 0x9d, 0x46, 0x32, 0x94, 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, 0x68, 0xe4, 0x48, 0x8f, 0x95, 0xce, 0x23, 0x5a, 0xd6, 0x75, 0xfd, 0xb8, 0x0e, - 0xfe, 0x45, 0x92, 0x2b, 0x81, 0x3d, 0x83, 0x17, 0x8f, 0xd5, 0x47, 0xf9, 0x3c, 0xf4, 0xba, 0x2a, - 0xd8, 0xf0, 0xc4, 0x05, 0x1a, 0x1e, 0xd8, 0x33, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xec, 0x8a, 0xcb, - 0xac, 0x4d, 0x2e, 0xfa, 0xb9, 0x88, 0x3a, 0x66, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, - 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, 0x7b, 0xaf, 0x36, 0x80, 0x07, 0xa6, 0xf1, - 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x59, 0xbd, 0x5c, 0x57, 0x81, 0x56, 0xdb, 0x3e, 0x1f, 0x5e, 0xf8, - 0x23, 0x85, 0x8b, 0x2b, 0x78, 0x5e, 0x02, 0x13, 0x21, 0xfe, 0x40, 0x80, 0xb7, 0x22, 0x7b, 0x2d, - 0xce, 0x85, 0xec, 0x13, 0x48, 0xd1, 0xc6, 0x27, 0x2e, 0xd8, 0x78, 0xca, 0x25, 0xfe, 0x9e, 0x3b, - 0xc7, 0x25, 0xdc, 0x36, 0x89, 0x62, 0xbf, 0x82, 0xfd, 0xb0, 0x69, 0xb7, 0xc3, 0x13, 0x17, 0xee, - 0x70, 0x97, 0xd5, 0x33, 0x02, 0x7d, 0xd5, 0x8c, 0xd3, 0x08, 0xfc, 0xba, 0x00, 0x73, 0x9e, 0x4f, - 0x13, 0xb3, 0x7b, 0xfb, 0x01, 0x24, 0x0d, 0xf3, 0xe8, 0x22, 0x9b, 0x81, 0xa4, 0x3c, 0x59, 0x92, - 0xc2, 0x35, 0x8a, 0xb3, 0xbd, 0xff, 0x2e, 0x01, 0x99, 0xa7, 0x95, 0x38, 0x5b, 0xf9, 0x09, 0xdf, - 0x68, 0x66, 0x13, 0x3b, 0x6a, 0x28, 0x7a, 0xcf, 0x5b, 0x7e, 0x5a, 0xd9, 0xc0, 0x27, 0xee, 0x50, - 0x24, 0x5c, 0x68, 0x15, 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, 0xb3, 0xad, 0x5d, 0xc4, 0x07, 0xf1, - 0xb9, 0x16, 0x30, 0x4c, 0x52, 0xb9, 0x6e, 0xf0, 0x82, 0x10, 0x11, 0xbc, 0x40, 0x1e, 0xe3, 0xb9, - 0x71, 0x89, 0x8b, 0x3c, 0x26, 0xe0, 0xbf, 0x4d, 0x16, 0xa7, 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, - 0x76, 0xc9, 0xdf, 0x48, 0x42, 0x7e, 0xbb, 0x67, 0xef, 0xc7, 0x3c, 0xfa, 0x2a, 0x00, 0xdd, 0x9e, - 0x4d, 0xf1, 0xc1, 0xb1, 0xc1, 0xdb, 0x7c, 0x4e, 0x5c, 0x84, 0xdb, 0x68, 0xc6, 0xd7, 0x3c, 0x36, - 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x5c, 0x71, 0x63, 0x14, 0x92, 0x6c, 0x1e, 0x1b, 0x9b, 0xd8, - 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x69, 0x72, 0x21, 0x3b, 0xe6, 0x45, 0xba, 0x79, - 0x8a, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, 0x4d, 0x53, 0x74, 0x69, 0x8a, 0x6a, - 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, 0xf3, 0x30, 0xb9, 0x67, 0x5a, 0x2a, - 0xa6, 0x11, 0x13, 0x69, 0x89, 0x5d, 0xb0, 0xfe, 0x5c, 0x4f, 0xa5, 0xd3, 0xc5, 0xcc, 0x7a, 0x2a, - 0x9d, 0x29, 0x82, 0xf8, 0x5b, 0x02, 0x14, 0xbc, 0x8e, 0x88, 0xd3, 0x5a, 0x57, 0x42, 0x5a, 0xbc, - 0x78, 0x57, 0x10, 0x05, 0x8a, 0xff, 0x81, 0xba, 0x2b, 0xaa, 0x79, 0x48, 0x7b, 0x26, 0xce, 0x91, - 0xf2, 0x98, 0x85, 0xce, 0x24, 0x2e, 0xda, 0xbb, 0x34, 0x8a, 0xe6, 0x01, 0xcc, 0xeb, 0x1d, 0x62, - 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xf7, 0xb3, 0x73, 0xfe, 0xbd, 0x8a, 0x7b, - 0x4b, 0xfc, 0x87, 0x74, 0x77, 0xda, 0x6f, 0x49, 0x9c, 0xaa, 0xae, 0x43, 0xce, 0x62, 0xa2, 0x89, - 0xcf, 0x71, 0x41, 0x6d, 0xcf, 0x78, 0xac, 0x44, 0xe1, 0xbf, 0x93, 0x80, 0xc2, 0x8b, 0x1e, 0xb6, - 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x0d, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, 0xd3, 0x92, 0x7b, 0x5d, - 0x4d, 0x71, 0xdc, 0xf8, 0x8d, 0x1c, 0x21, 0x3f, 0x31, 0xad, 0x1d, 0x4a, 0x44, 0x18, 0xd0, 0x81, - 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x2d, 0xe3, 0xf2, 0x87, 0xff, 0xe5, - 0x74, 0xf1, 0xd1, 0x58, 0x51, 0x59, 0x34, 0x02, 0xad, 0xd7, 0xd3, 0xb5, 0xe5, 0x9d, 0x9d, 0xfa, - 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x51, 0x02, 0x8a, 0xbe, - 0x8e, 0xe2, 0xec, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x37, 0x02, 0x67, 0x24, - 0x66, 0xe7, 0x73, 0x98, 0x09, 0x69, 0x20, 0xf9, 0x93, 0x69, 0x20, 0x7b, 0xe4, 0x37, 0x1e, 0xdd, - 0x87, 0x59, 0xe7, 0xd8, 0x90, 0x59, 0x3c, 0x1e, 0x8b, 0xe1, 0x70, 0xc3, 0x0e, 0x0a, 0x0e, 0xd1, - 0x07, 0xa1, 0xd3, 0xf8, 0x0d, 0x5b, 0xfc, 0x23, 0x01, 0x10, 0x55, 0x54, 0x9d, 0xed, 0xe9, 0x7f, - 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0x08, 0x47, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, - 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0xab, 0x30, 0x17, 0x6a, 0x40, - 0x9c, 0x9d, 0x7d, 0x1d, 0x66, 0xf6, 0xcc, 0x9e, 0xa1, 0xc9, 0xec, 0x8d, 0x07, 0xdf, 0x0e, 0xcc, - 0x52, 0x1a, 0x7b, 0x9e, 0xf8, 0x3f, 0x12, 0x30, 0x2f, 0x61, 0xdb, 0x6c, 0x1f, 0xe2, 0xf8, 0x55, - 0x58, 0x03, 0xfe, 0xae, 0x45, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x5b, 0xe6, 0xc2, 0x7b, 0xea, - 0x37, 0x47, 0x8f, 0xd8, 0xc1, 0x5d, 0x74, 0xbe, 0x27, 0x97, 0x0a, 0xed, 0xc9, 0x99, 0x50, 0xd0, - 0x5b, 0x86, 0x49, 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x8b, 0x54, 0x96, 0x47, 0x55, 0xb2, - 0xce, 0x58, 0x1a, 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xf2, 0x65, 0x52, 0xdf, 0xb3, 0xd3, - 0xc5, 0x7c, 0xe8, 0x9e, 0x2d, 0xe5, 0x75, 0xef, 0x9a, 0x48, 0x17, 0xbf, 0x0d, 0x97, 0xfa, 0x94, - 0x1d, 0xa7, 0xc7, 0xf3, 0x6f, 0x92, 0x70, 0x35, 0x2c, 0x3e, 0x6e, 0xfc, 0xf1, 0x75, 0xef, 0xd0, - 0x1a, 0xe4, 0x3a, 0xba, 0xf1, 0x66, 0x5b, 0x8b, 0x33, 0x1d, 0xdd, 0xf0, 0xb7, 0x71, 0x23, 0x86, - 0xc6, 0xd4, 0x57, 0x3a, 0x34, 0x14, 0x58, 0x88, 0xea, 0xbb, 0x38, 0xc7, 0xc7, 0xaf, 0x09, 0x30, - 0x13, 0xf7, 0x9e, 0xd9, 0x9b, 0xc5, 0x9b, 0x89, 0x4d, 0xc8, 0x7d, 0x05, 0x9b, 0x6c, 0xbf, 0x23, - 0x00, 0x6a, 0x5a, 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x66, 0xce, 0xc3, 0xa4, 0x6e, - 0x68, 0xf8, 0x98, 0x36, 0x33, 0x25, 0xb1, 0x8b, 0xd0, 0xab, 0xc3, 0xe4, 0x58, 0xaf, 0x0e, 0xc5, - 0xcf, 0x61, 0x2e, 0x54, 0xc5, 0x38, 0xdb, 0xff, 0x8f, 0x13, 0x30, 0xc7, 0x1b, 0x12, 0xfb, 0xf6, - 0xe2, 0x37, 0x61, 0xb2, 0x4d, 0x64, 0x8e, 0xe8, 0x67, 0xfa, 0x4c, 0xb7, 0x9f, 0x69, 0x61, 0xf4, - 0x73, 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x16, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, - 0x05, 0x28, 0x90, 0xf9, 0xdc, 0xb5, 0xcc, 0xae, 0x69, 0x13, 0x97, 0xc5, 0x1e, 0x0f, 0xe5, 0xcc, - 0x9e, 0x9d, 0x2e, 0xe6, 0x36, 0x75, 0x63, 0x9b, 0x33, 0x36, 0x1b, 0x12, 0x31, 0x0c, 0xde, 0xa5, - 0x2d, 0xfe, 0x27, 0x01, 0xe6, 0xbf, 0xb2, 0xad, 0xd8, 0xff, 0x1f, 0xba, 0x12, 0x5f, 0x42, 0x91, - 0xfe, 0xa8, 0x1b, 0x7b, 0x66, 0x9c, 0x9b, 0xe2, 0xdf, 0x13, 0x60, 0x36, 0x20, 0x38, 0x4e, 0xff, - 0xe4, 0x8d, 0xf4, 0x24, 0xfe, 0x22, 0xf1, 0x58, 0x82, 0x83, 0x3c, 0xce, 0x29, 0xf4, 0x07, 0x09, - 0xb8, 0x5c, 0x61, 0xaf, 0x90, 0xdd, 0x98, 0x8a, 0x38, 0x47, 0x46, 0x09, 0xa6, 0x0f, 0xb1, 0x65, - 0xeb, 0x26, 0x5b, 0x3d, 0x73, 0x92, 0x7b, 0x89, 0x16, 0x20, 0x6d, 0x1b, 0x4a, 0xd7, 0xde, 0x37, - 0xdd, 0x77, 0x67, 0xde, 0xb5, 0x17, 0xff, 0x31, 0xf9, 0xe6, 0xf1, 0x1f, 0x53, 0xa3, 0xe3, 0x3f, - 0xa6, 0x7f, 0x82, 0xf8, 0x0f, 0xfe, 0xa2, 0xea, 0x3f, 0x0a, 0x70, 0x65, 0x40, 0x73, 0x71, 0x8e, - 0x96, 0xef, 0x42, 0x56, 0xe5, 0x82, 0x89, 0xbd, 0x65, 0x6f, 0xe1, 0xea, 0xa4, 0xd8, 0x1b, 0xc2, - 0x8e, 0xb3, 0xd3, 0x45, 0x70, 0xab, 0x5a, 0x5f, 0xe3, 0xca, 0x21, 0xbf, 0x35, 0xf1, 0x57, 0x72, - 0x50, 0xa8, 0x1e, 0xb3, 0x1d, 0xe8, 0x06, 0x5b, 0xe5, 0xd1, 0x13, 0x48, 0x77, 0x2d, 0xf3, 0x50, - 0x77, 0x9b, 0x91, 0x0f, 0xbd, 0xfc, 0x77, 0x9b, 0xd1, 0xc7, 0xb5, 0xcd, 0x39, 0x24, 0x8f, 0x17, - 0x35, 0x21, 0xf3, 0xcc, 0x54, 0x95, 0xf6, 0x13, 0xbd, 0xed, 0x8e, 0xfc, 0xf7, 0xcf, 0x17, 0xb4, - 0xec, 0xf1, 0x6c, 0x2b, 0xce, 0xbe, 0xdb, 0x09, 0x1e, 0x11, 0xd5, 0x21, 0x5d, 0x73, 0x9c, 0x2e, - 0xb9, 0xc9, 0x6d, 0xc7, 0x9d, 0x31, 0x84, 0x12, 0x16, 0x37, 0x56, 0xd4, 0x65, 0x47, 0x4d, 0x98, - 0x7d, 0x4a, 0x4f, 0x38, 0x55, 0xda, 0x66, 0x4f, 0xab, 0x98, 0xc6, 0x9e, 0xde, 0xe2, 0x76, 0xf7, - 0xf6, 0x18, 0x32, 0x9f, 0x56, 0x1a, 0xd2, 0xa0, 0x00, 0xb4, 0x0a, 0xe9, 0xc6, 0x23, 0x2e, 0x8c, - 0xb9, 0x65, 0xb7, 0xc6, 0x10, 0xd6, 0x78, 0x24, 0x79, 0x6c, 0x68, 0x1d, 0xb2, 0xab, 0x5f, 0xf4, - 0x2c, 0xcc, 0xa5, 0x4c, 0x0d, 0x8d, 0x3c, 0xe8, 0x97, 0x42, 0xb9, 0xa4, 0x20, 0x33, 0x6a, 0x40, - 0xfe, 0x95, 0x69, 0x1d, 0xb4, 0x4d, 0xc5, 0x6d, 0xe1, 0x34, 0x15, 0xf7, 0x8d, 0x31, 0xc4, 0xb9, - 0x8c, 0x52, 0x9f, 0x08, 0xf4, 0x6d, 0x28, 0x90, 0xce, 0x68, 0x2a, 0xbb, 0x6d, 0xb7, 0x92, 0x69, - 0x2a, 0xf5, 0xdd, 0x31, 0xa4, 0x7a, 0x9c, 0xee, 0x2b, 0x90, 0x3e, 0x51, 0x0b, 0x12, 0xe4, 0x42, - 0x83, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7f, 0x0b, 0x34, 0x36, 0x88, 0xfe, 0x46, 0xef, 0xc1, 0xb4, - 0x61, 0x6a, 0xd8, 0x9d, 0x21, 0xb9, 0xf2, 0xfc, 0xd9, 0xe9, 0xe2, 0xd4, 0x96, 0xa9, 0x31, 0x87, - 0x84, 0xff, 0x92, 0xa6, 0x48, 0xa1, 0xba, 0xb6, 0xb0, 0x04, 0x29, 0xd2, 0xef, 0xc4, 0x30, 0xed, - 0x2a, 0x36, 0xde, 0xb1, 0x74, 0x2e, 0xcd, 0xbd, 0x5c, 0xf8, 0x67, 0x09, 0x48, 0x34, 0x1e, 0x11, - 0x97, 0x7b, 0xb7, 0xa7, 0x1e, 0x60, 0x87, 0xdf, 0xe7, 0x57, 0xd4, 0x15, 0xb7, 0xf0, 0x9e, 0xce, - 0x3c, 0xa3, 0x8c, 0xc4, 0xaf, 0xd0, 0x3b, 0x00, 0x8a, 0xaa, 0x62, 0xdb, 0x96, 0xdd, 0x93, 0x6f, - 0x19, 0x29, 0xc3, 0x28, 0x1b, 0xf8, 0x84, 0xb0, 0xd9, 0x58, 0xb5, 0xb0, 0xe3, 0x06, 0x36, 0xb1, - 0x2b, 0xc2, 0xe6, 0xe0, 0x4e, 0x57, 0x76, 0xcc, 0x03, 0x6c, 0xd0, 0x71, 0x92, 0x21, 0xa6, 0xa6, - 0xd3, 0x6d, 0x12, 0x02, 0xb1, 0x92, 0xd8, 0xd0, 0x7c, 0x93, 0x96, 0x91, 0xbc, 0x6b, 0x22, 0xd2, - 0xc2, 0x2d, 0x9d, 0x1f, 0xe1, 0xca, 0x48, 0xfc, 0x8a, 0x68, 0x49, 0xe9, 0x39, 0xfb, 0xb4, 0x27, - 0x32, 0x12, 0xfd, 0x8d, 0x6e, 0x43, 0x81, 0xc5, 0x42, 0xca, 0xd8, 0x50, 0x65, 0x6a, 0x5c, 0x33, - 0xf4, 0x76, 0x8e, 0x91, 0xab, 0x86, 0x4a, 0x4c, 0x29, 0x7a, 0x04, 0x9c, 0x20, 0x1f, 0x74, 0x6c, - 0xa2, 0x53, 0x20, 0xa5, 0xca, 0x85, 0xb3, 0xd3, 0xc5, 0x6c, 0x83, 0xde, 0xd8, 0xd8, 0x6c, 0xd4, - 0xd7, 0xa4, 0x2c, 0x2b, 0xb5, 0xd1, 0xb1, 0xeb, 0xda, 0xc2, 0x6f, 0x08, 0x90, 0x7c, 0x5a, 0x69, - 0x5c, 0x58, 0x65, 0x6e, 0x45, 0x93, 0x81, 0x8a, 0xde, 0x81, 0xc2, 0xae, 0xde, 0x6e, 0xeb, 0x46, - 0x8b, 0x78, 0x41, 0xdf, 0xc5, 0xaa, 0xab, 0xb0, 0x3c, 0x27, 0x6f, 0x33, 0x2a, 0x5a, 0x82, 0xac, - 0x6a, 0x61, 0x0d, 0x1b, 0x8e, 0xae, 0xb4, 0x6d, 0xae, 0xb9, 0x20, 0x69, 0xe1, 0x97, 0x05, 0x98, - 0xa4, 0x33, 0x00, 0xbd, 0x0d, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x29, 0xcb, 0x48, 0x3e, - 0x61, 0x68, 0xf5, 0xae, 0xc3, 0x8c, 0xa2, 0xaa, 0x66, 0xcf, 0x70, 0x64, 0x43, 0xe9, 0x60, 0x5e, - 0xcd, 0x2c, 0xa7, 0x6d, 0x29, 0x1d, 0x8c, 0x16, 0xc1, 0xbd, 0xf4, 0x0e, 0x34, 0x66, 0x24, 0xe0, - 0xa4, 0x0d, 0x7c, 0xb2, 0xf0, 0xc7, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, 0xd1, 0xc2, 0x06, 0xb6, - 0x14, 0xc7, 0xf4, 0xaa, 0xe1, 0x11, 0xfa, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0xce, 0xc3, 0xa4, 0x43, - 0xa6, 0x05, 0xaf, 0x01, 0xbb, 0xa0, 0xdb, 0xd1, 0x6d, 0xa5, 0xc5, 0x76, 0xe3, 0x32, 0x12, 0xbb, - 0x20, 0x8d, 0xe1, 0x21, 0xb5, 0x4c, 0x23, 0xfc, 0x8a, 0xd4, 0x94, 0x05, 0x7e, 0xee, 0xe2, 0x96, - 0x6e, 0xd0, 0xb1, 0x94, 0x94, 0x80, 0x92, 0xca, 0x84, 0x82, 0xde, 0x82, 0x0c, 0x2b, 0x80, 0x0d, - 0x8d, 0x0e, 0xa8, 0xa4, 0x94, 0xa6, 0x84, 0xaa, 0xa1, 0x2d, 0x60, 0xc8, 0x78, 0x93, 0x93, 0x74, - 0x5b, 0xcf, 0xf6, 0x14, 0x49, 0x7f, 0xa3, 0xf7, 0x61, 0xfe, 0x75, 0x4f, 0x69, 0xeb, 0x7b, 0x74, - 0xa3, 0x8d, 0x14, 0x63, 0x3a, 0x63, 0x2d, 0x41, 0xde, 0x3d, 0x2a, 0x81, 0xaa, 0xce, 0x9d, 0xcb, - 0x49, 0x7f, 0x2e, 0x8b, 0xbf, 0x2f, 0xc0, 0x2c, 0x8b, 0xea, 0x61, 0xc1, 0xa8, 0xf1, 0xf9, 0x21, - 0x1f, 0x43, 0x46, 0x53, 0x1c, 0x85, 0x1d, 0xd1, 0x4c, 0x8c, 0x3c, 0xa2, 0xe9, 0x1d, 0x25, 0x50, - 0x1c, 0x85, 0x1e, 0xd3, 0x44, 0x90, 0x22, 0xbf, 0xd9, 0x69, 0x56, 0x89, 0xfe, 0x16, 0x3f, 0x03, - 0x14, 0xac, 0x68, 0x9c, 0x1e, 0xd9, 0x3d, 0xb8, 0x44, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, - 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, 0xb5, 0x51, 0x31, 0xf0, 0x5e, 0x8a, 0xbe, 0x8e, 0x12, 0xff, 0x70, - 0x0a, 0x72, 0xd5, 0xe3, 0xae, 0x69, 0xc5, 0xba, 0x8b, 0x55, 0x86, 0x69, 0x0e, 0xf4, 0x47, 0xbc, - 0x17, 0xee, 0x33, 0xe6, 0xee, 0x2b, 0x57, 0xce, 0x88, 0xca, 0x00, 0x2c, 0x60, 0x94, 0x06, 0x05, - 0x25, 0x2f, 0xf0, 0xa6, 0x8c, 0xb2, 0x11, 0x2a, 0xda, 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, - 0xbd, 0xed, 0xf0, 0xb8, 0xbb, 0xe8, 0x10, 0xf1, 0xcd, 0x97, 0x95, 0xca, 0x13, 0x5a, 0x88, 0x85, - 0xc0, 0xf9, 0xd7, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x17, 0xf8, 0x91, 0x1a, 0xd9, 0x76, 0x4f, - 0xc9, 0x95, 0x73, 0x67, 0xa7, 0x8b, 0x19, 0x89, 0x52, 0x1b, 0x8d, 0xa6, 0x94, 0x61, 0x05, 0x1a, - 0xb6, 0x83, 0x6e, 0x40, 0xce, 0xec, 0xe8, 0x8e, 0xec, 0x3a, 0x49, 0xdc, 0xa3, 0x9c, 0x21, 0x44, - 0xd7, 0x89, 0x42, 0x4d, 0xb8, 0x83, 0x0d, 0x3a, 0xda, 0x49, 0x3b, 0xe5, 0x5d, 0xb6, 0xf9, 0xe8, - 0xb0, 0x19, 0x2d, 0x9b, 0x5d, 0x47, 0xef, 0xe8, 0x5f, 0xd0, 0x37, 0xd3, 0xfc, 0xa5, 0xd1, 0x0d, - 0x56, 0x9c, 0xb4, 0xaf, 0x4c, 0x77, 0x25, 0x79, 0xd9, 0xe7, 0x81, 0xa2, 0xe8, 0x6f, 0x0a, 0x70, - 0x99, 0x2b, 0x52, 0xde, 0xa5, 0x31, 0xee, 0x4a, 0x5b, 0x77, 0x4e, 0xe4, 0x83, 0xc3, 0x52, 0x9a, - 0xfa, 0xad, 0x3f, 0x1b, 0xd9, 0x21, 0x81, 0x71, 0xb0, 0xec, 0x76, 0xcb, 0xc9, 0x33, 0xce, 0xbc, - 0x71, 0x58, 0x35, 0x1c, 0xeb, 0xa4, 0x7c, 0xe5, 0xec, 0x74, 0x71, 0x6e, 0xf0, 0xee, 0x4b, 0x69, - 0xce, 0x1e, 0x64, 0x41, 0x35, 0x00, 0xec, 0x8d, 0x43, 0xba, 0x62, 0x44, 0xfb, 0x1f, 0x91, 0x03, - 0x56, 0x0a, 0xf0, 0xa2, 0xbb, 0x50, 0xe4, 0x27, 0x5b, 0xf6, 0xf4, 0x36, 0x96, 0x6d, 0xfd, 0x0b, - 0x4c, 0xd7, 0x96, 0xa4, 0x94, 0x67, 0x74, 0x22, 0xa2, 0xa1, 0x7f, 0x81, 0x17, 0xbe, 0x0b, 0xa5, - 0x61, 0xb5, 0x0f, 0x4e, 0x81, 0x0c, 0x7b, 0x23, 0xfb, 0x51, 0x78, 0x3b, 0x66, 0x8c, 0xa1, 0xca, - 0xb7, 0x64, 0x3e, 0x4e, 0x7c, 0x24, 0x88, 0xff, 0x24, 0x01, 0xb9, 0x72, 0xaf, 0x7d, 0xf0, 0xbc, - 0xdb, 0xe8, 0x75, 0x3a, 0x8a, 0x75, 0x42, 0xcc, 0x20, 0x33, 0x14, 0xa4, 0x82, 0x02, 0x33, 0x83, - 0xd4, 0x12, 0xe8, 0x5f, 0x60, 0xb2, 0x38, 0x05, 0x4f, 0x65, 0xb3, 0x18, 0x7e, 0xda, 0x86, 0xc0, - 0x51, 0x6b, 0xf3, 0xc8, 0x46, 0x1f, 0x41, 0x29, 0x50, 0x90, 0xee, 0x9d, 0xc8, 0xd8, 0x70, 0x2c, - 0x1d, 0xb3, 0xfd, 0xbf, 0xa4, 0x14, 0x88, 0x97, 0xa9, 0x93, 0xdb, 0x55, 0x76, 0x17, 0x35, 0x61, - 0x86, 0x14, 0x3c, 0x91, 0xe9, 0x12, 0xe2, 0xee, 0xcf, 0x3e, 0x88, 0x68, 0x56, 0xa8, 0xde, 0xcb, - 0x54, 0x3f, 0x15, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0x16, 0x3e, 0x85, 0x62, 0x7f, 0x81, - 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0xf3, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4f, 0xa5, 0x53, 0xc5, - 0x49, 0xf1, 0x2f, 0x92, 0x90, 0x77, 0x87, 0x59, 0x9c, 0x40, 0xa7, 0x0c, 0x93, 0x64, 0x50, 0xb8, - 0x31, 0x1e, 0xb7, 0x47, 0x8c, 0x6e, 0x1e, 0x35, 0x4e, 0x06, 0x8b, 0x0b, 0x92, 0x29, 0x6b, 0x1c, - 0x06, 0x67, 0xe1, 0x97, 0x13, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x42, 0x21, 0x8c, 0xb3, - 0x50, 0xd0, 0xa2, 0xde, 0x72, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x57, 0x3e, 0x78, 0xf0, - 0x90, 0x1a, 0x9b, 0x19, 0x89, 0x5f, 0xa1, 0x32, 0x0d, 0x3b, 0x32, 0x2d, 0x07, 0x6b, 0xdc, 0xa7, - 0x5f, 0x3a, 0xaf, 0x7f, 0xdd, 0x45, 0xc9, 0xe5, 0x43, 0x57, 0x21, 0x49, 0xac, 0xd8, 0x34, 0x0b, - 0x52, 0x38, 0x3b, 0x5d, 0x4c, 0x12, 0xfb, 0x45, 0x68, 0x68, 0x05, 0xb2, 0x61, 0x93, 0x41, 0x3c, - 0x38, 0x6a, 0x18, 0x03, 0xd3, 0x1d, 0xda, 0xde, 0xd4, 0x62, 0x78, 0x96, 0xf7, 0xf1, 0x5f, 0xa6, - 0x20, 0x57, 0xef, 0xc4, 0xbd, 0xa4, 0xac, 0x86, 0x7b, 0x38, 0x0a, 0x08, 0x85, 0x1e, 0x1a, 0xd1, - 0xc1, 0xa1, 0x15, 0x3c, 0x79, 0xb1, 0x15, 0xbc, 0x4e, 0x3c, 0x65, 0x9e, 0x66, 0x21, 0x39, 0x04, - 0xf3, 0x84, 0x9f, 0x4f, 0xfd, 0x14, 0x89, 0xf0, 0xf8, 0xe7, 0x28, 0x68, 0xa0, 0xc9, 0xa7, 0xd4, - 0x21, 0x67, 0xa3, 0x6c, 0x6a, 0xfc, 0x51, 0x36, 0x8d, 0x0d, 0x8d, 0x2e, 0x6a, 0x61, 0x8b, 0x3a, - 0xfd, 0xe6, 0x16, 0x75, 0xc1, 0xe1, 0x83, 0xf5, 0x63, 0x48, 0x6a, 0xba, 0xdb, 0x39, 0xe3, 0x2f, - 0xd5, 0x84, 0xe9, 0x9c, 0x51, 0x9b, 0x0a, 0x8e, 0x5a, 0x36, 0x4a, 0x16, 0xea, 0x00, 0xbe, 0x6e, - 0xd0, 0x12, 0x4c, 0x99, 0x6d, 0xcd, 0x3d, 0x48, 0x92, 0x2b, 0x67, 0xce, 0x4e, 0x17, 0x27, 0x9f, - 0xb7, 0xb5, 0xfa, 0x9a, 0x34, 0x69, 0xb6, 0xb5, 0xba, 0x46, 0x73, 0x5c, 0xe0, 0x23, 0xd9, 0x8b, - 0x32, 0x9b, 0x91, 0xa6, 0x0d, 0x7c, 0xb4, 0x86, 0x6d, 0x95, 0x0f, 0xb8, 0xdf, 0x16, 0x20, 0xef, - 0xea, 0x3e, 0x5e, 0xa3, 0x92, 0xd6, 0x3b, 0x7c, 0x92, 0x25, 0x2f, 0x36, 0xc9, 0x5c, 0x3e, 0x7e, - 0xbc, 0xf6, 0x57, 0x05, 0x1e, 0x37, 0xdc, 0x50, 0x15, 0x87, 0x38, 0x15, 0x31, 0x4e, 0x8c, 0x7b, - 0x50, 0xb4, 0x14, 0x43, 0x33, 0x3b, 0xfa, 0x17, 0x98, 0x6d, 0x84, 0xda, 0xfc, 0xad, 0x65, 0xc1, - 0xa3, 0xd3, 0x5d, 0x3f, 0x5b, 0xfc, 0xe3, 0x04, 0x8f, 0x31, 0xf6, 0xaa, 0x11, 0xa7, 0xba, 0xbe, - 0x03, 0xb3, 0xfd, 0x09, 0x47, 0xdc, 0xd9, 0xfa, 0x5e, 0x84, 0xbc, 0xa8, 0x8a, 0xb0, 0x58, 0x41, - 0x37, 0x70, 0xbd, 0x2f, 0xf9, 0x88, 0x8d, 0x2a, 0x90, 0x0d, 0xe6, 0x31, 0x49, 0x8e, 0x9d, 0xc7, - 0x04, 0x2c, 0x2f, 0x7b, 0xc9, 0xc2, 0xcf, 0xc3, 0x24, 0xbd, 0xfd, 0x06, 0x26, 0x9a, 0xf7, 0xe6, - 0x9f, 0x27, 0xe0, 0x26, 0xad, 0xfd, 0x4b, 0x6c, 0xe9, 0x7b, 0x27, 0xdb, 0x96, 0xe9, 0x60, 0xd5, - 0xc1, 0x9a, 0x7f, 0x0a, 0x24, 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x0f, 0xb8, 0x50, 0xbc, 0x98, 0xc7, - 0x85, 0x36, 0xa0, 0xc0, 0x23, 0x03, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0xb9, 0xc8, 0xea, 0x96, - 0x63, 0xbc, 0xab, 0x84, 0x75, 0xd5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0xe9, 0x77, 0x9e, - 0xfe, 0x64, 0x1b, 0x8a, 0x69, 0x16, 0x9e, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, - 0xb3, 0x00, 0xb7, 0xce, 0x51, 0x71, 0x9c, 0x43, 0x77, 0x01, 0xd2, 0x87, 0xe4, 0x41, 0x3a, 0xd7, - 0x71, 0x5a, 0xf2, 0xae, 0xd1, 0x26, 0xe4, 0xf6, 0x14, 0xbd, 0xed, 0x0f, 0xe9, 0xe1, 0xe1, 0x85, - 0xd1, 0x91, 0xae, 0x33, 0x8c, 0x9d, 0x8d, 0x61, 0xf1, 0x37, 0x13, 0x30, 0xbb, 0xaa, 0x69, 0x8d, - 0x06, 0xb7, 0x81, 0xf1, 0x8d, 0x14, 0x17, 0x64, 0x26, 0x7c, 0x90, 0x89, 0xde, 0x03, 0xa4, 0xe9, - 0x36, 0x4b, 0xf3, 0x61, 0xef, 0x2b, 0x9a, 0x79, 0xe4, 0x87, 0x55, 0xcc, 0xba, 0x77, 0x1a, 0xee, - 0x0d, 0xd4, 0x00, 0x8a, 0x76, 0x64, 0xdb, 0x51, 0xbc, 0xf7, 0x46, 0xb7, 0xc6, 0x3a, 0xae, 0xc5, - 0x60, 0x90, 0x77, 0x29, 0x65, 0x88, 0x1c, 0xfa, 0x93, 0xf8, 0xed, 0x3a, 0x69, 0xba, 0x23, 0x2b, - 0xb6, 0x7b, 0x36, 0x87, 0x25, 0x18, 0xc9, 0x33, 0xfa, 0xaa, 0xcd, 0x8e, 0xdc, 0xb0, 0xc3, 0x04, - 0xbe, 0x6a, 0xe2, 0x84, 0xc4, 0xff, 0x40, 0x80, 0xbc, 0x84, 0xf7, 0x2c, 0x6c, 0xc7, 0xba, 0x29, - 0xf0, 0x04, 0x66, 0x2c, 0x26, 0x55, 0xde, 0xb3, 0xcc, 0xce, 0x45, 0xe6, 0x55, 0x96, 0x33, 0x3e, - 0xb1, 0xcc, 0x0e, 0x37, 0x2c, 0x2f, 0xa1, 0xe0, 0xd5, 0x31, 0xce, 0xc6, 0xff, 0x3e, 0x3d, 0x8a, - 0xcc, 0x04, 0xc7, 0x1d, 0xdf, 0x10, 0xaf, 0x06, 0xe8, 0x8b, 0xaa, 0x60, 0x45, 0xe3, 0x54, 0xc3, - 0x7f, 0x17, 0x20, 0xdf, 0xe8, 0xed, 0xb2, 0x3c, 0x52, 0xf1, 0x69, 0xa0, 0x0a, 0x99, 0x36, 0xde, - 0x73, 0xe4, 0x37, 0x0a, 0x83, 0x4f, 0x13, 0x56, 0x7a, 0x08, 0xe0, 0x29, 0x80, 0x45, 0x8f, 0xaf, - 0x51, 0x39, 0xc9, 0x0b, 0xca, 0xc9, 0x50, 0x5e, 0x42, 0x16, 0xff, 0x69, 0x02, 0x0a, 0x5e, 0x33, - 0xe3, 0xb4, 0x92, 0xaf, 0x42, 0xd6, 0x21, 0x79, 0x11, 0xeb, 0x30, 0xcb, 0x43, 0x3a, 0xa2, 0x2d, - 0xc4, 0x32, 0xcc, 0x51, 0xc7, 0x45, 0x56, 0xba, 0xdd, 0xb6, 0xee, 0xc2, 0x5d, 0x6a, 0x7f, 0x52, - 0xd2, 0x2c, 0xbd, 0xb5, 0xca, 0xee, 0x50, 0xa0, 0x4b, 0xc6, 0xdc, 0x9e, 0x85, 0xf1, 0x17, 0x58, - 0xa6, 0xc8, 0xeb, 0x22, 0x21, 0x2b, 0x59, 0xc6, 0xd8, 0x20, 0x7c, 0x7c, 0xcc, 0xbd, 0x82, 0x59, - 0xaa, 0xd3, 0xb8, 0x8f, 0xdd, 0x8a, 0x7f, 0x2f, 0x01, 0x28, 0x28, 0xf9, 0xab, 0xeb, 0x8b, 0x44, - 0x7c, 0x7d, 0xf1, 0x2e, 0x20, 0x16, 0xb4, 0x68, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x9e, - 0xe3, 0x48, 0x90, 0x8a, 0xfc, 0xce, 0x36, 0xb6, 0x1a, 0x94, 0x8e, 0x1e, 0x03, 0xf8, 0x1e, 0x19, - 0x5f, 0x30, 0x46, 0x3a, 0x64, 0x52, 0xc6, 0x73, 0xc5, 0xc4, 0xef, 0x2d, 0xc0, 0x0c, 0xd7, 0xe1, - 0x8e, 0xa1, 0x9b, 0x06, 0x7a, 0x00, 0xc9, 0x16, 0xdf, 0xfa, 0xcf, 0x46, 0x6e, 0xd3, 0xf9, 0x49, - 0xdc, 0x6a, 0x13, 0x12, 0x29, 0x4b, 0x58, 0xba, 0x3d, 0x27, 0xc2, 0x31, 0xf2, 0x03, 0xb1, 0x83, - 0x2c, 0xdd, 0x9e, 0x83, 0x1a, 0x50, 0x50, 0xfd, 0x24, 0x55, 0x32, 0x61, 0x4f, 0x0e, 0x05, 0x51, - 0x91, 0x69, 0xc1, 0x6a, 0x13, 0x52, 0x5e, 0x0d, 0xdd, 0x40, 0x95, 0x60, 0x6e, 0xa4, 0xd4, 0x40, - 0x94, 0x97, 0x7f, 0xbe, 0x37, 0x9c, 0x97, 0xa9, 0x36, 0x11, 0x48, 0xa1, 0x84, 0x3e, 0x86, 0x29, - 0x8d, 0x66, 0xe1, 0xe1, 0x23, 0x3a, 0x6a, 0xd0, 0x85, 0x92, 0x1d, 0xd5, 0x26, 0x24, 0xce, 0x81, - 0xd6, 0x61, 0x86, 0xfd, 0x62, 0x6e, 0x0a, 0x47, 0x96, 0xb7, 0x86, 0x4b, 0x08, 0x2c, 0x04, 0xb5, - 0x09, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0x62, 0xcb, 0x6b, 0x43, 0x52, - 0x6a, 0xf8, 0xcc, 0xb4, 0x34, 0x7a, 0xcc, 0xd2, 0x32, 0x3a, 0xc7, 0xee, 0x36, 0x5f, 0x54, 0xf5, - 0x43, 0x47, 0xb7, 0x49, 0xf5, 0x31, 0x25, 0xa0, 0xa7, 0x90, 0x55, 0x88, 0xbf, 0x27, 0xd3, 0xc3, - 0x8f, 0x74, 0x5f, 0x2f, 0xfa, 0xe5, 0xfa, 0xc0, 0x61, 0xd5, 0x1a, 0x3d, 0x1f, 0xee, 0x12, 0x7d, - 0x41, 0x1d, 0x6c, 0xb5, 0x70, 0x29, 0x3b, 0x5a, 0x50, 0x30, 0xb6, 0xcb, 0x13, 0x44, 0x89, 0xc4, - 0xef, 0xf3, 0x4e, 0x1e, 0xd3, 0x46, 0xcd, 0x0c, 0x7d, 0x9d, 0x1b, 0x71, 0x6c, 0xa7, 0x36, 0x21, - 0xcd, 0xec, 0x07, 0xc8, 0x68, 0x19, 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x3a, 0x43, 0xbc, 0xa3, 0x29, - 0xb5, 0x09, 0x29, 0xd1, 0x52, 0xd1, 0xa7, 0x90, 0x66, 0xe7, 0x0c, 0x8e, 0x8d, 0x52, 0x7e, 0xa8, - 0x9d, 0x08, 0x9f, 0xd6, 0xa8, 0x4d, 0x48, 0xf4, 0x68, 0x03, 0x79, 0xde, 0x36, 0xe4, 0x2d, 0x16, - 0x1c, 0xe7, 0x86, 0xb2, 0x16, 0x87, 0xbe, 0xe2, 0x8e, 0x8a, 0x66, 0xad, 0x51, 0xcf, 0x3f, 0x40, - 0x47, 0xdf, 0x81, 0xf9, 0xb0, 0x44, 0x3e, 0xd2, 0x66, 0x87, 0xbe, 0xae, 0x1d, 0x1a, 0x59, 0x59, - 0x9b, 0x90, 0x90, 0x35, 0x70, 0x13, 0x7d, 0x08, 0x93, 0xac, 0xd7, 0x10, 0x15, 0x19, 0x15, 0xb5, - 0xd1, 0xd7, 0x61, 0xac, 0x3c, 0x19, 0xfc, 0x0e, 0x8f, 0x10, 0x93, 0xdb, 0x66, 0xab, 0x34, 0x37, - 0x74, 0xf0, 0x0f, 0xc6, 0xba, 0x91, 0xc1, 0xef, 0xf8, 0x54, 0xd2, 0xef, 0x16, 0xbb, 0xc3, 0xc3, - 0x8a, 0xe6, 0x87, 0xf6, 0x7b, 0x44, 0xe0, 0x58, 0x8d, 0xc6, 0xee, 0xfb, 0x64, 0x52, 0x35, 0x8b, - 0x65, 0x81, 0x91, 0xe9, 0x9c, 0xba, 0x34, 0xb4, 0x6a, 0x83, 0x69, 0x72, 0x6a, 0xd4, 0x47, 0xf2, - 0xa8, 0xe8, 0x25, 0x14, 0x79, 0xae, 0x06, 0xff, 0x9d, 0xc2, 0x65, 0x2a, 0xef, 0x5e, 0xa4, 0xe9, - 0x8a, 0x8a, 0xc9, 0xa9, 0x4d, 0x48, 0x05, 0x35, 0x7c, 0x07, 0x7d, 0x06, 0xb3, 0x54, 0x9e, 0xac, - 0xfa, 0xe9, 0x35, 0x4a, 0xa5, 0x81, 0x64, 0x0d, 0xc3, 0x33, 0x71, 0xb8, 0x92, 0x8b, 0x6a, 0xdf, - 0x2d, 0x32, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x95, 0x5d, 0x18, 0x3a, 0x8c, 0xc3, 0x49, 0xfd, 0xc8, - 0x30, 0xd6, 0x19, 0x85, 0x0c, 0x63, 0x87, 0xc7, 0x9c, 0xf1, 0xee, 0x78, 0x7b, 0xe8, 0x30, 0x8e, - 0x0a, 0x4e, 0x23, 0xc3, 0xd8, 0x09, 0xd2, 0xc9, 0x30, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, - 0x87, 0xf1, 0xd0, 0x43, 0xc8, 0x64, 0x18, 0x2b, 0x03, 0x37, 0xd1, 0x1a, 0x00, 0x73, 0x67, 0xe8, - 0xa2, 0x78, 0x6d, 0xe8, 0x62, 0xd0, 0x1f, 0x7b, 0x46, 0x16, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, - 0x82, 0x25, 0x99, 0xbe, 0x42, 0x2d, 0x2d, 0x0e, 0x35, 0x64, 0x03, 0xaf, 0x3b, 0x89, 0x21, 0x3b, - 0xf2, 0x88, 0x64, 0x55, 0x61, 0x7b, 0xbe, 0xa5, 0xa5, 0xe1, 0x66, 0x39, 0xf8, 0xea, 0x87, 0x9a, - 0x65, 0x4a, 0x40, 0xab, 0x90, 0x21, 0x6b, 0xfe, 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xa4, 0x7d, - 0x87, 0x53, 0x6a, 0x13, 0x52, 0xfa, 0x35, 0x27, 0x91, 0xc7, 0xb3, 0xdd, 0xb0, 0x92, 0x38, 0xf4, - 0xf1, 0xa1, 0x9d, 0x53, 0xf2, 0x78, 0xc6, 0x81, 0x54, 0xb8, 0xc4, 0xfa, 0x8a, 0x9f, 0x07, 0xb6, - 0xf8, 0x11, 0xd6, 0xd2, 0x0d, 0x2a, 0x6a, 0xe8, 0xb6, 0x52, 0xe4, 0x31, 0xe5, 0xda, 0x84, 0x34, - 0xa7, 0x0c, 0xde, 0x25, 0x13, 0x9e, 0x2f, 0x3d, 0x6c, 0x33, 0xaa, 0x74, 0x73, 0xe8, 0x84, 0x8f, - 0xd8, 0xc3, 0x23, 0x13, 0x5e, 0x09, 0x90, 0xd9, 0x02, 0xa4, 0xc9, 0xb6, 0xcd, 0x5e, 0xb8, 0xdf, - 0x1a, 0xb1, 0x00, 0xf5, 0xed, 0x02, 0xb0, 0x05, 0x48, 0x6b, 0x30, 0x4e, 0x22, 0x48, 0x6d, 0x63, - 0xc5, 0xe2, 0x66, 0xf6, 0xf6, 0x50, 0x41, 0x03, 0xf9, 0xf3, 0x88, 0x20, 0xd5, 0x23, 0x12, 0x87, - 0xc7, 0x72, 0x93, 0xbc, 0x70, 0x87, 0xf1, 0xce, 0x50, 0x87, 0x27, 0x32, 0x0b, 0x0d, 0x71, 0x78, - 0xac, 0xd0, 0x0d, 0xf4, 0x73, 0x30, 0xcd, 0xe1, 0x5b, 0xe9, 0xee, 0x08, 0x37, 0x36, 0x88, 0xb8, - 0xc9, 0xbc, 0xe6, 0x3c, 0xcc, 0xca, 0x32, 0xd8, 0xc8, 0x9a, 0x77, 0x6f, 0x84, 0x95, 0x1d, 0x40, - 0xae, 0xcc, 0xca, 0xfa, 0x64, 0x62, 0x65, 0xd9, 0x38, 0xe5, 0x6b, 0xdd, 0xfd, 0xa1, 0x56, 0x76, - 0xf0, 0xe8, 0x0b, 0xb1, 0xb2, 0xaf, 0x7d, 0x2a, 0x69, 0x99, 0xcd, 0xe0, 0x53, 0xe9, 0x1b, 0x43, - 0x5b, 0x16, 0xc6, 0x91, 0xa4, 0x65, 0x9c, 0x87, 0x74, 0x1b, 0x73, 0x89, 0x99, 0xa6, 0xdf, 0x1d, - 0x7e, 0xe8, 0xbe, 0x1f, 0x74, 0xd4, 0xdc, 0x8d, 0x4a, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, 0xd0, - 0x31, 0xd7, 0xd4, 0x7b, 0xa3, 0x0d, 0x55, 0xd4, 0x49, 0x6a, 0xcf, 0x50, 0x85, 0x6e, 0xd2, 0xaa, - 0xb2, 0x73, 0x66, 0x74, 0x7e, 0x2f, 0x8f, 0xc8, 0x0f, 0xd0, 0x77, 0xda, 0x8f, 0x56, 0xd5, 0x23, - 0xfa, 0x53, 0xa8, 0xc7, 0x92, 0x57, 0x94, 0x56, 0x46, 0x4f, 0xa1, 0x70, 0xfa, 0x0c, 0x6f, 0x0a, - 0x71, 0xb2, 0xb7, 0x66, 0xba, 0x1e, 0xc6, 0xfb, 0xa3, 0xd7, 0xcc, 0x7e, 0xd7, 0x82, 0xad, 0x99, - 0xdc, 0xa7, 0xf8, 0xeb, 0x02, 0x2c, 0xb1, 0xba, 0xd1, 0x1d, 0xbd, 0x13, 0xd9, 0xdb, 0x17, 0x0d, - 0x9c, 0x79, 0x78, 0x40, 0x1f, 0xf0, 0xe1, 0xb0, 0xea, 0x9e, 0xb3, 0xcf, 0x5b, 0x9b, 0x90, 0xde, - 0x51, 0x46, 0x95, 0x2b, 0x4f, 0xf3, 0xd7, 0xa2, 0xde, 0x21, 0xcf, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, - 0x5f, 0x29, 0x96, 0xd6, 0x53, 0xe9, 0xab, 0xc5, 0x85, 0xf5, 0x54, 0xfa, 0xad, 0xe2, 0xdb, 0xe2, - 0x5f, 0x5e, 0x85, 0x9c, 0x8b, 0xfc, 0x18, 0x22, 0x7a, 0x18, 0x44, 0x44, 0xd7, 0x86, 0x21, 0x22, - 0x8e, 0x15, 0x39, 0x24, 0x7a, 0x18, 0x84, 0x44, 0xd7, 0x86, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, - 0xe6, 0x30, 0x4c, 0x74, 0x6f, 0x0c, 0x4c, 0xe4, 0x89, 0xea, 0x07, 0x45, 0x6b, 0x83, 0xa0, 0xe8, - 0xe6, 0x68, 0x50, 0xe4, 0x89, 0x0a, 0xa0, 0xa2, 0xc7, 0x7d, 0xa8, 0xe8, 0xfa, 0x08, 0x54, 0xe4, - 0xf1, 0xbb, 0xb0, 0x68, 0x23, 0x12, 0x16, 0xdd, 0x3e, 0x0f, 0x16, 0x79, 0x72, 0x42, 0xb8, 0xe8, - 0x83, 0x10, 0x2e, 0x5a, 0x1c, 0x8a, 0x8b, 0x3c, 0x6e, 0x06, 0x8c, 0x3e, 0xe9, 0x07, 0x46, 0xd7, - 0x47, 0x00, 0x23, 0xbf, 0x05, 0x1c, 0x19, 0xd5, 0xa2, 0x90, 0xd1, 0xad, 0x73, 0x90, 0x91, 0x27, - 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xd6, 0x39, 0xd0, 0xa8, 0x4f, 0x12, 0xc3, 0x46, 0x5b, - 0xd1, 0xd8, 0xe8, 0xce, 0xb9, 0xd8, 0xc8, 0x93, 0x16, 0x06, 0x47, 0x2b, 0x01, 0x70, 0xf4, 0xce, - 0x10, 0x70, 0xe4, 0xb1, 0x12, 0x74, 0xf4, 0xad, 0x01, 0x74, 0x24, 0x8e, 0x42, 0x47, 0x1e, 0xaf, - 0x07, 0x8f, 0x5e, 0x0c, 0x81, 0x47, 0x77, 0xcf, 0x87, 0x47, 0x9e, 0xb0, 0x3e, 0x7c, 0xa4, 0x8c, - 0xc4, 0x47, 0xef, 0x8d, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, 0xe9, 0xa3, 0x30, 0x40, 0x5a, 0x1a, - 0x0e, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0x6d, 0x44, 0x22, 0xa4, 0xdb, 0xe7, 0x21, 0x24, 0x7f, 0x1e, - 0x04, 0x21, 0xd2, 0x56, 0x34, 0x44, 0xba, 0x73, 0x2e, 0x44, 0xf2, 0xbb, 0x3f, 0x84, 0x91, 0x36, - 0x22, 0x31, 0xd2, 0xed, 0xf3, 0x30, 0x92, 0x5f, 0xb9, 0x20, 0x48, 0x7a, 0x35, 0x14, 0x24, 0xdd, - 0x1f, 0x07, 0x24, 0x79, 0x42, 0x07, 0x50, 0xd2, 0xe7, 0xc3, 0x51, 0xd2, 0x37, 0x2e, 0x90, 0xaf, - 0x30, 0x12, 0x26, 0x7d, 0x6b, 0x00, 0x26, 0x89, 0xa3, 0x60, 0x92, 0x3f, 0x9e, 0x5d, 0x9c, 0xa4, - 0x8c, 0x44, 0x35, 0xef, 0x8d, 0x89, 0x6a, 0xfc, 0xc1, 0x17, 0x01, 0x6b, 0xaa, 0x11, 0xb0, 0xe6, - 0xe6, 0x68, 0x58, 0xe3, 0x9b, 0x73, 0x1f, 0xd7, 0xd4, 0xa2, 0x70, 0xcd, 0xad, 0x73, 0x70, 0x8d, - 0x6f, 0x85, 0x02, 0xc0, 0xe6, 0x71, 0x1f, 0xb0, 0xb9, 0x7e, 0x6e, 0xd4, 0x4f, 0x00, 0xd9, 0x94, - 0x07, 0x91, 0xcd, 0x8d, 0x91, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0xc7, 0x7d, 0xd0, 0xe6, 0xfa, - 0x08, 0x68, 0xe3, 0x57, 0x80, 0x63, 0x1b, 0x6d, 0x34, 0xb6, 0x59, 0x1e, 0x17, 0xdb, 0x78, 0x82, - 0x23, 0xc1, 0xcd, 0x56, 0x34, 0xb8, 0xb9, 0x33, 0xe6, 0x0b, 0xf9, 0x01, 0x74, 0x53, 0x8b, 0x42, - 0x37, 0xb7, 0xce, 0x41, 0x37, 0xc1, 0x35, 0xc4, 0x83, 0x37, 0xb5, 0x28, 0x78, 0x73, 0xeb, 0x1c, - 0x78, 0xe3, 0x4b, 0x0a, 0xe0, 0x9b, 0xe6, 0x30, 0x7c, 0x73, 0x6f, 0x0c, 0x7c, 0xe3, 0x3b, 0x2f, - 0x7d, 0x00, 0xe7, 0xd3, 0x7e, 0x80, 0x23, 0x8e, 0x02, 0x38, 0xfe, 0x8c, 0x74, 0x11, 0xce, 0x56, - 0x34, 0xc2, 0xb9, 0x73, 0x2e, 0xc2, 0x09, 0x1a, 0xc9, 0x00, 0xc4, 0xd9, 0x88, 0x84, 0x38, 0xb7, - 0xcf, 0x83, 0x38, 0xbe, 0x91, 0x0c, 0x62, 0x9c, 0x4f, 0xfb, 0x31, 0x8e, 0x38, 0x0a, 0xe3, 0xf8, - 0x8d, 0x73, 0x41, 0x4e, 0x2d, 0x0a, 0xe4, 0xdc, 0x3a, 0x07, 0xe4, 0xf8, 0x9d, 0x17, 0x40, 0x39, - 0xca, 0x48, 0x94, 0xf3, 0xde, 0x98, 0x28, 0xa7, 0xcf, 0x70, 0x85, 0x61, 0x4e, 0x2d, 0x0a, 0xe6, - 0xdc, 0x3a, 0x07, 0xe6, 0x04, 0x2a, 0xeb, 0xe3, 0x9c, 0xad, 0x68, 0x9c, 0x73, 0xe7, 0x5c, 0x9c, - 0xd3, 0x37, 0x9b, 0x5c, 0xa0, 0xb3, 0x11, 0x09, 0x74, 0x6e, 0x9f, 0x07, 0x74, 0xfa, 0x16, 0x3e, - 0xee, 0x1c, 0xfc, 0xca, 0xf8, 0x48, 0xe7, 0xa3, 0x8b, 0x23, 0x1d, 0xef, 0x99, 0xb1, 0x40, 0x9d, - 0xf5, 0x54, 0xfa, 0xed, 0xe2, 0x3b, 0xe2, 0xdf, 0x9a, 0x86, 0xa9, 0x9a, 0x17, 0xe7, 0xe2, 0xd7, - 0x52, 0x78, 0x93, 0xbc, 0x48, 0x68, 0x8d, 0xcc, 0x58, 0x6a, 0xf7, 0xce, 0x4f, 0x75, 0x37, 0x98, - 0x8c, 0x8d, 0xb3, 0xbe, 0xc1, 0x01, 0x65, 0xf4, 0x01, 0xe4, 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, - 0x9b, 0x96, 0xee, 0xb0, 0x53, 0x1c, 0x42, 0xb9, 0xf8, 0xe5, 0xe9, 0xe2, 0xcc, 0x8e, 0x8d, 0xad, - 0x6d, 0x4e, 0x97, 0x66, 0x7a, 0x81, 0x2b, 0xf7, 0x43, 0x51, 0x93, 0xe3, 0x7f, 0x28, 0xea, 0x05, - 0x14, 0x2d, 0xac, 0x68, 0x21, 0x0f, 0x84, 0xe5, 0x20, 0x8a, 0x1e, 0x33, 0xf4, 0x94, 0x95, 0x5b, - 0x92, 0xe6, 0x22, 0x2a, 0x58, 0x61, 0x22, 0x7a, 0x00, 0x97, 0x3a, 0xca, 0x31, 0x8d, 0x89, 0x94, - 0x5d, 0xa7, 0x8e, 0xc6, 0x39, 0xb2, 0x6f, 0x30, 0xa1, 0x8e, 0x72, 0x4c, 0xbf, 0x3a, 0xc5, 0x6e, - 0xd1, 0x4f, 0x49, 0xdc, 0x82, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0xcf, 0x3d, 0xcb, 0xf2, - 0xb6, 0xe6, 0x5c, 0x2a, 0x4b, 0x30, 0x7b, 0x1f, 0x66, 0x79, 0xb0, 0x7c, 0xe0, 0x15, 0x21, 0xcf, - 0xdf, 0xca, 0x6e, 0x78, 0x6f, 0x05, 0x51, 0x05, 0x0a, 0x2d, 0xc5, 0xc1, 0x47, 0xca, 0x89, 0xec, - 0x1e, 0xc5, 0xca, 0xd2, 0x94, 0x8c, 0x6f, 0x9d, 0x9d, 0x2e, 0xe6, 0x9e, 0xb2, 0x5b, 0x03, 0x27, - 0xb2, 0x72, 0xad, 0xc0, 0x0d, 0x0d, 0xdd, 0x81, 0x82, 0x62, 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, - 0xec, 0x9e, 0x4d, 0x21, 0x45, 0x5a, 0xca, 0x53, 0x72, 0xc5, 0xa5, 0xa2, 0xeb, 0x30, 0xc3, 0x23, - 0xc9, 0xd9, 0x67, 0x6c, 0x0a, 0xb4, 0xa9, 0xfc, 0xbb, 0x09, 0xf4, 0x4b, 0x36, 0xe8, 0x31, 0x2c, - 0xf0, 0xdc, 0xf2, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, 0xfe, 0xf8, 0x2c, 0x52, 0xb1, 0x57, 0x58, - 0x2e, 0x79, 0x52, 0x80, 0xa8, 0xda, 0xcf, 0x84, 0xb0, 0x05, 0xb3, 0x6a, 0x5b, 0xf7, 0x10, 0x00, - 0x6b, 0xf9, 0xec, 0x50, 0x3b, 0x5b, 0xa1, 0x65, 0xfd, 0x57, 0xa4, 0x05, 0x35, 0x4c, 0x40, 0x0d, - 0xa0, 0xd9, 0x5e, 0xe4, 0xae, 0xd9, 0xd6, 0xd5, 0x13, 0xea, 0xfc, 0x87, 0x73, 0x64, 0x8f, 0xcc, - 0x54, 0xff, 0x4a, 0xd1, 0x9d, 0x6d, 0xca, 0x29, 0xc1, 0x91, 0xf7, 0x9b, 0xe5, 0xb7, 0x5d, 0x4f, - 0xa5, 0x67, 0x8a, 0xb9, 0xf5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0x7f, 0x5b, 0x80, 0x42, 0x5f, 0x5d, - 0x50, 0x0d, 0x2e, 0x69, 0xde, 0x54, 0x91, 0xf9, 0x51, 0x23, 0xdd, 0x34, 0x78, 0xda, 0xef, 0xb9, - 0x2f, 0x4f, 0x17, 0x0b, 0xb4, 0xf4, 0x53, 0xef, 0x96, 0x34, 0xef, 0x73, 0xf8, 0x54, 0xf4, 0x11, - 0xe4, 0x99, 0xfb, 0xe8, 0x7d, 0x99, 0x8d, 0xc6, 0x88, 0x97, 0x67, 0xbf, 0x3c, 0x5d, 0xcc, 0x51, - 0x9f, 0xd1, 0x4d, 0xcb, 0x2b, 0xe5, 0xda, 0xc1, 0x4b, 0xf1, 0x37, 0x04, 0x98, 0x09, 0x1d, 0xe7, - 0x79, 0xdc, 0xf7, 0x06, 0xfd, 0x6a, 0x34, 0xee, 0x1c, 0x16, 0x50, 0x97, 0xe6, 0xe3, 0xdc, 0x8d, - 0x4e, 0x5c, 0x1c, 0x8e, 0x5b, 0xe8, 0x2e, 0x8c, 0x1b, 0xaa, 0xe1, 0xb2, 0x7d, 0x9c, 0xfa, 0xfe, - 0x0f, 0x16, 0x27, 0xc4, 0x3f, 0x48, 0x41, 0x2e, 0x7c, 0x78, 0xa7, 0xde, 0x57, 0xaf, 0xa8, 0x75, - 0x21, 0xc4, 0xb1, 0x3c, 0x22, 0x6d, 0x61, 0xc6, 0xcf, 0xaf, 0xcf, 0xaa, 0xb9, 0x34, 0x22, 0x4e, - 0x20, 0x58, 0x4f, 0x9f, 0x71, 0xe1, 0x7b, 0x49, 0xcf, 0xbe, 0x2e, 0xc3, 0x24, 0x4d, 0xa3, 0xc3, - 0xab, 0x56, 0xea, 0x1f, 0x3d, 0xc4, 0x57, 0x26, 0xf7, 0x25, 0x56, 0x8c, 0xd8, 0xe3, 0xe6, 0x1b, - 0xe5, 0xa9, 0xf3, 0xa7, 0xc1, 0xc5, 0x3f, 0x84, 0xc7, 0xf3, 0x14, 0x4e, 0x5e, 0x2c, 0x4f, 0x21, - 0xfa, 0x25, 0x28, 0xa8, 0x66, 0xbb, 0xcd, 0xd6, 0x3a, 0x66, 0x91, 0x06, 0x33, 0x8f, 0x50, 0x11, - 0xfc, 0xdb, 0x85, 0xcb, 0xde, 0x37, 0x0c, 0x97, 0x25, 0xfe, 0x0d, 0xc3, 0x40, 0xac, 0x67, 0xde, - 0x13, 0xc6, 0x0c, 0x59, 0x5f, 0xd8, 0xe9, 0xf4, 0x9b, 0x84, 0x9d, 0xb2, 0x50, 0x65, 0x3e, 0x72, - 0xfe, 0x54, 0xe0, 0x81, 0x21, 0xcf, 0x4c, 0xf3, 0xa0, 0xe7, 0x85, 0x8b, 0x2e, 0x04, 0xb3, 0x06, - 0xa6, 0xbf, 0x3c, 0x5d, 0x4c, 0x49, 0x5e, 0xda, 0xc0, 0x28, 0xcb, 0x9f, 0xf8, 0xc9, 0x2c, 0xff, - 0x75, 0x98, 0xe9, 0x5a, 0x78, 0x0f, 0x3b, 0xea, 0xbe, 0x6c, 0xf4, 0x3a, 0xfc, 0x54, 0x49, 0xd6, - 0xa5, 0x6d, 0xf5, 0x3a, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0x63, 0xbb, 0x69, 0xa5, 0x5c, 0x3a, - 0x47, 0xe4, 0xe2, 0xff, 0x16, 0x60, 0x2e, 0xd4, 0x26, 0x3e, 0x27, 0xd6, 0x21, 0xeb, 0x9b, 0x03, - 0xbb, 0x24, 0x5c, 0x30, 0x78, 0x32, 0xc8, 0x8c, 0x64, 0xb8, 0xec, 0x3e, 0x96, 0xe6, 0x94, 0xf7, - 0xc5, 0x26, 0x2e, 0x28, 0xf6, 0x92, 0x2f, 0x67, 0x2d, 0xf0, 0x00, 0x6f, 0x92, 0x24, 0xc7, 0x9a, - 0x24, 0xe2, 0x6f, 0x0b, 0x50, 0xa4, 0x0f, 0x78, 0x82, 0xb1, 0x16, 0x8b, 0x75, 0x72, 0x83, 0x92, - 0x13, 0xe3, 0x9f, 0x1b, 0x09, 0x7d, 0x07, 0x23, 0x19, 0xfe, 0x0e, 0x86, 0xf8, 0x03, 0x01, 0xf2, - 0x5e, 0x0d, 0xd9, 0xd7, 0xe0, 0x46, 0x24, 0xa7, 0x7c, 0xb3, 0x2f, 0xa1, 0xb9, 0x59, 0x38, 0xc6, - 0xfa, 0x40, 0x5d, 0x30, 0x0b, 0x07, 0xfb, 0x82, 0xd7, 0xdf, 0x75, 0x47, 0x0e, 0xa9, 0x62, 0xc5, - 0xcf, 0xb6, 0xf0, 0x06, 0x47, 0x68, 0x24, 0xfa, 0x21, 0x4d, 0xb3, 0x7d, 0xc8, 0x12, 0x9f, 0x8c, - 0x65, 0xb6, 0x10, 0x0f, 0x87, 0x02, 0xbe, 0x1b, 0xa7, 0x35, 0x1b, 0xf4, 0x13, 0x9b, 0xec, 0xb7, - 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x76, 0x3e, 0xd1, 0xd2, 0x58, 0xa6, 0xd4, 0xd5, 0x12, 0x1b, 0x2b, - 0x7f, 0x12, 0xec, 0x89, 0xea, 0x21, 0x41, 0x61, 0x8f, 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x85, 0x81, - 0x85, 0x7a, 0x4e, 0x22, 0xa5, 0xd1, 0x93, 0x50, 0x92, 0x8a, 0xc4, 0x70, 0xc4, 0x30, 0xa8, 0xd2, - 0x50, 0x32, 0x8b, 0x0f, 0xc3, 0x63, 0x7d, 0xe4, 0xe3, 0x83, 0x83, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, - 0xb0, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, 0x9d, 0x17, 0x3d, 0xd3, 0xf2, 0x13, 0x7e, 0xdc, - 0xee, 0xfb, 0x30, 0xc8, 0x64, 0x39, 0x7b, 0x16, 0x95, 0xd2, 0xe7, 0x12, 0xcc, 0x85, 0xb8, 0x99, - 0xb1, 0x10, 0x3f, 0x84, 0xab, 0x4f, 0x4d, 0xdb, 0xd6, 0xbb, 0x04, 0x7a, 0xd2, 0x59, 0x49, 0x96, - 0x06, 0xcf, 0x3c, 0xa6, 0xbb, 0x74, 0x13, 0xc2, 0x60, 0x66, 0x24, 0x23, 0x79, 0xd7, 0xe2, 0x1f, - 0x0a, 0x70, 0x65, 0x90, 0x93, 0x69, 0x39, 0xea, 0xc4, 0xdf, 0xb4, 0x6a, 0xfa, 0xf9, 0xdd, 0xce, - 0x1f, 0xad, 0x6e, 0x71, 0xe2, 0x62, 0xf2, 0x67, 0xca, 0x1d, 0x85, 0x9a, 0x0f, 0x7e, 0xfa, 0x38, - 0xcf, 0xc9, 0x9b, 0x8c, 0xea, 0x5b, 0x92, 0xd4, 0x78, 0x96, 0xa4, 0x09, 0x85, 0x75, 0x53, 0x37, - 0x88, 0x27, 0xeb, 0xb6, 0x77, 0x15, 0xf2, 0xbb, 0xba, 0xa1, 0x58, 0x27, 0xb2, 0x7b, 0xa8, 0x9b, - 0x0d, 0x94, 0x85, 0xa8, 0xca, 0xb2, 0x12, 0x52, 0x8e, 0x71, 0xf0, 0x4b, 0xf1, 0x47, 0x02, 0x14, - 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, 0xb6, 0x7b, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0x66, 0x58, - 0xf4, 0x76, 0x85, 0x51, 0xeb, 0x6b, 0x52, 0x86, 0x17, 0xa8, 0x6b, 0xe8, 0x46, 0x38, 0x39, 0xc2, - 0x64, 0x19, 0xce, 0x06, 0x52, 0x22, 0x90, 0x6e, 0xb7, 0x1d, 0xd3, 0xf2, 0x30, 0x13, 0xef, 0x76, - 0x37, 0x95, 0x3a, 0x3d, 0xd3, 0x4b, 0xca, 0xad, 0x42, 0x9e, 0x2c, 0xf7, 0x87, 0xd8, 0x6b, 0x52, - 0xea, 0xfc, 0x26, 0x31, 0x0e, 0xb7, 0x49, 0xbf, 0x47, 0x9c, 0x54, 0xd6, 0x1b, 0x5e, 0x0f, 0x8f, - 0xb0, 0x68, 0x3f, 0x13, 0x4c, 0x49, 0x38, 0x5e, 0xd2, 0x46, 0xea, 0x89, 0x7c, 0x0b, 0xd2, 0xee, - 0x27, 0x8a, 0xf9, 0x04, 0xb9, 0xba, 0xcc, 0xbe, 0x61, 0xbc, 0xec, 0x7e, 0xc3, 0x78, 0x79, 0x8d, - 0x17, 0x60, 0x66, 0xfc, 0xfb, 0xff, 0x6d, 0x51, 0x90, 0x3c, 0xa6, 0xfb, 0x0d, 0x32, 0xc2, 0x07, - 0x56, 0x61, 0x94, 0x07, 0x08, 0x7c, 0x43, 0x86, 0x7f, 0xb5, 0x76, 0x75, 0x4d, 0xde, 0xd9, 0xaa, - 0x3c, 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x13, 0xfa, 0x02, 0x4d, - 0x82, 0x7d, 0xc7, 0xf6, 0xfe, 0xcf, 0x00, 0xf8, 0x1f, 0xb3, 0x22, 0xb2, 0x36, 0xaa, 0x9f, 0xc9, - 0x2f, 0x57, 0x9f, 0xed, 0x54, 0x1b, 0xc5, 0x09, 0x84, 0x20, 0x5f, 0x5e, 0x6d, 0x56, 0x6a, 0xb2, - 0x54, 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0xef, 0xdf, 0xde, 0x5f, 0x83, 0x99, 0x60, 0xba, - 0x1b, 0x34, 0x07, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe5, 0x17, 0x3b, 0xd5, - 0x9d, 0x6a, 0x71, 0x82, 0x56, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, - 0xec, 0x9a, 0x7e, 0xad, 0xa6, 0x98, 0xb8, 0xbf, 0x09, 0xd9, 0x40, 0x5a, 0x5b, 0xf2, 0xb8, 0xed, - 0x9d, 0x46, 0x4d, 0x6e, 0xd6, 0x37, 0xab, 0x8d, 0xe6, 0xea, 0xe6, 0x36, 0x93, 0x41, 0x69, 0xab, - 0xe5, 0xe7, 0x52, 0xb3, 0x28, 0x78, 0xd7, 0xcd, 0xe7, 0x3b, 0x95, 0x9a, 0xdb, 0x0c, 0x31, 0x95, - 0x4e, 0x16, 0x93, 0xf7, 0xff, 0x9a, 0x00, 0x57, 0x86, 0xa4, 0x7e, 0x41, 0x59, 0x98, 0xde, 0x31, - 0x68, 0x8e, 0xcf, 0xe2, 0x04, 0xca, 0x05, 0xb2, 0xbf, 0x14, 0x05, 0x94, 0x66, 0xf9, 0x37, 0x8a, - 0x09, 0x34, 0x05, 0x89, 0xc6, 0xa3, 0x62, 0x92, 0xd4, 0x34, 0x90, 0x3c, 0xa5, 0x98, 0x42, 0x19, - 0x9e, 0xb6, 0xa1, 0x38, 0x89, 0x66, 0xfc, 0xec, 0x09, 0xc5, 0x29, 0x22, 0xca, 0xcb, 0x42, 0x50, - 0x9c, 0xbe, 0x7f, 0x1d, 0x02, 0x27, 0xbd, 0x11, 0xc0, 0xd4, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, - 0x40, 0xd3, 0x90, 0x5c, 0x6d, 0xb7, 0x8b, 0xc2, 0xc3, 0x7f, 0x9d, 0x82, 0xb4, 0xfb, 0x55, 0x16, - 0xf4, 0x0c, 0x26, 0xd9, 0xd6, 0xf3, 0xe2, 0x70, 0xcf, 0x9e, 0x4e, 0xde, 0x85, 0xa5, 0xf3, 0x5c, - 0x7f, 0x71, 0x02, 0xfd, 0x15, 0xc8, 0x06, 0x3c, 0x26, 0x34, 0x74, 0xfb, 0x2c, 0xe4, 0x25, 0x2e, - 0xdc, 0x3e, 0xaf, 0x98, 0x27, 0xff, 0x15, 0x64, 0x3c, 0x0b, 0x8e, 0x6e, 0x8c, 0xb2, 0xef, 0xae, - 0xec, 0xd1, 0x8b, 0x00, 0x99, 0x6b, 0xe2, 0xc4, 0xfb, 0x02, 0xb2, 0x00, 0x0d, 0x1a, 0x5b, 0x14, - 0x15, 0x91, 0x30, 0xd4, 0x9a, 0x2f, 0xdc, 0x1f, 0xab, 0xb4, 0xff, 0x4c, 0xa2, 0x2c, 0x7f, 0xc5, - 0x88, 0x56, 0xd6, 0xc0, 0x7a, 0x14, 0xad, 0xac, 0x88, 0x85, 0x67, 0x02, 0xbd, 0x80, 0x14, 0xb1, - 0x94, 0x28, 0xca, 0x87, 0xec, 0xb3, 0xcc, 0x0b, 0x37, 0x46, 0x96, 0x71, 0x45, 0x96, 0xef, 0xfd, - 0xf0, 0x2f, 0xae, 0x4d, 0xfc, 0xf0, 0xec, 0x9a, 0xf0, 0xa3, 0xb3, 0x6b, 0xc2, 0x9f, 0x9d, 0x5d, - 0x13, 0xfe, 0xfc, 0xec, 0x9a, 0xf0, 0xeb, 0x3f, 0xbe, 0x36, 0xf1, 0xa3, 0x1f, 0x5f, 0x9b, 0xf8, - 0xb3, 0x1f, 0x5f, 0x9b, 0xf8, 0x7c, 0x9a, 0x73, 0xef, 0x4e, 0x51, 0xa3, 0xf2, 0xe8, 0xff, 0x05, - 0x00, 0x00, 0xff, 0xff, 0x2a, 0xe5, 0xf1, 0xd1, 0x53, 0x7e, 0x00, 0x00, + 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14, 0x16, 0x52, 0xdf, + 0xfb, 0x9d, 0x6b, 0x13, 0xe2, 0x36, 0xc0, 0x53, 0xec, 0xf0, 0x01, 0x86, 0xca, 0x30, 0xb5, 0x4f, + 0xeb, 0x53, 0x12, 0xa8, 0xa6, 0x97, 0x22, 0x2b, 0x1e, 0x18, 0x8c, 0xe5, 0x34, 0xd1, 0xc6, 0x8f, + 0x4f, 0x17, 0x05, 0x89, 0x73, 0x8a, 0x7f, 0x20, 0x40, 0x96, 0x8a, 0x64, 0xed, 0x43, 0x95, 0x3e, + 0x99, 0xd7, 0xcf, 0x55, 0xc6, 0xa0, 0x50, 0xb4, 0x0c, 0x93, 0x87, 0x4a, 0xbb, 0x87, 0x4b, 0x09, + 0x2a, 0xa3, 0x14, 0x21, 0xe3, 0x25, 0xb9, 0x2f, 0xb1, 0x62, 0xe8, 0x31, 0xcc, 0xe8, 0x86, 0x83, + 0x0d, 0x47, 0x66, 0x6c, 0xc9, 0x73, 0xd8, 0xb2, 0xac, 0x34, 0xbd, 0x10, 0xff, 0x95, 0x00, 0xb0, + 0xdd, 0x8b, 0x53, 0x29, 0xe8, 0x9b, 0x63, 0xd6, 0x9f, 0x8f, 0x2e, 0xde, 0x8a, 0xcb, 0x30, 0xa5, + 0x1b, 0x6d, 0xdd, 0x60, 0xf5, 0x4f, 0x4b, 0xfc, 0x0a, 0xcd, 0xc3, 0xe4, 0x6e, 0x5b, 0x37, 0x34, + 0x3a, 0x1f, 0xd2, 0x12, 0xbb, 0x10, 0x25, 0xc8, 0xd2, 0x5a, 0xc7, 0xa8, 0x77, 0xf1, 0x3f, 0x24, + 0xe0, 0x52, 0xc5, 0x34, 0x34, 0x9d, 0x4c, 0x49, 0xa5, 0xfd, 0xb5, 0xd0, 0xca, 0x3a, 0x04, 0x26, + 0x9f, 0x8c, 0x8f, 0xbb, 0x63, 0xf6, 0x31, 0xf2, 0xb9, 0xaa, 0xc7, 0x5d, 0x4a, 0x8b, 0xd6, 0x24, + 0xfa, 0x26, 0x5c, 0x51, 0xda, 0x6d, 0xf3, 0x48, 0xd6, 0xf7, 0x64, 0xcd, 0xc4, 0xb6, 0x6c, 0x98, + 0x8e, 0x8c, 0x8f, 0x75, 0xdb, 0xa1, 0xa6, 0x24, 0x2d, 0xcd, 0xd1, 0xdb, 0xf5, 0xbd, 0x35, 0x13, + 0xdb, 0x5b, 0xa6, 0x53, 0x25, 0xb7, 0xc8, 0x3c, 0x25, 0x95, 0x61, 0xf3, 0x74, 0x8a, 0x98, 0x5f, + 0x29, 0x8d, 0x8f, 0xbb, 0x6c, 0x9e, 0xfe, 0x12, 0x5c, 0xee, 0xd7, 0x63, 0x9c, 0xfd, 0xf4, 0x47, + 0x02, 0xe4, 0xeb, 0x86, 0xee, 0x7c, 0x2d, 0x3a, 0xc8, 0x53, 0x6a, 0x32, 0xa8, 0xd4, 0xfb, 0x50, + 0xdc, 0x53, 0xf4, 0xf6, 0x73, 0xa3, 0x69, 0x76, 0x76, 0x6d, 0xc7, 0x34, 0xb0, 0xcd, 0xb5, 0x3e, + 0x40, 0x17, 0x5f, 0x42, 0xc1, 0x6b, 0x4d, 0x9c, 0x6a, 0x72, 0xa0, 0x58, 0x37, 0x54, 0x0b, 0x77, + 0xb0, 0x11, 0xab, 0x9e, 0xde, 0x86, 0x8c, 0xee, 0xca, 0xa5, 0xba, 0x4a, 0x4a, 0x3e, 0x41, 0xec, + 0xc1, 0x6c, 0xe0, 0xa9, 0x71, 0x9a, 0x45, 0xb2, 0x34, 0xe0, 0x23, 0xd9, 0xef, 0x23, 0xb2, 0x34, + 0xe0, 0x23, 0x66, 0xc6, 0x1a, 0x90, 0x5b, 0xc3, 0x6d, 0xec, 0xe0, 0x38, 0xad, 0xfb, 0x0e, 0xe4, + 0x5d, 0xa1, 0x71, 0x76, 0xcc, 0xdf, 0x11, 0x00, 0x71, 0xb9, 0x64, 0x35, 0x8d, 0xb3, 0x6f, 0x16, + 0x89, 0x0b, 0xe1, 0xf4, 0x2c, 0x83, 0xf9, 0x02, 0x6c, 0x4c, 0x02, 0x23, 0x51, 0x77, 0xc0, 0xb7, + 0xb2, 0xa9, 0xa0, 0x95, 0xe5, 0x6e, 0xcc, 0x11, 0xcc, 0x85, 0x2a, 0x16, 0x6f, 0xf7, 0xa5, 0x68, + 0x9d, 0x12, 0x4b, 0xc9, 0xa0, 0xaf, 0x46, 0x89, 0xe2, 0x0f, 0x04, 0x98, 0xad, 0xb4, 0xb1, 0x62, + 0xc5, 0xae, 0x91, 0x6f, 0x41, 0x5a, 0xc3, 0x8a, 0x46, 0x9b, 0xcc, 0x26, 0xf6, 0x3b, 0x01, 0x29, + 0xc4, 0xa3, 0x5d, 0xde, 0x6f, 0xab, 0xcb, 0x4d, 0xd7, 0xd7, 0xe5, 0xb3, 0xdb, 0x63, 0x12, 0x3f, + 0x03, 0x14, 0xac, 0x59, 0x9c, 0x03, 0xe1, 0xef, 0x0b, 0x80, 0x24, 0x7c, 0x88, 0x2d, 0x27, 0xf6, + 0x66, 0xaf, 0x41, 0xd6, 0x51, 0xac, 0x16, 0x76, 0x64, 0xe2, 0xc5, 0x5f, 0xa4, 0xe5, 0xc0, 0xf8, + 0x08, 0x59, 0xfc, 0x1c, 0xe6, 0x42, 0xf5, 0x8b, 0xb3, 0xf1, 0xff, 0x5b, 0x80, 0x6c, 0x43, 0x55, + 0x8c, 0x38, 0x5b, 0xfd, 0x29, 0x64, 0x6d, 0x55, 0x31, 0xe4, 0x3d, 0xd3, 0xea, 0x28, 0x0e, 0x1d, + 0xe2, 0xf9, 0x50, 0xab, 0x3d, 0x0f, 0x5a, 0x55, 0x8c, 0x27, 0xb4, 0x90, 0x04, 0xb6, 0xf7, 0x1b, + 0xbd, 0x80, 0xec, 0x01, 0x3e, 0x91, 0x39, 0x1a, 0xa3, 0xeb, 0x5f, 0xfe, 0xe1, 0xfb, 0x01, 0xfe, + 0x83, 0xc3, 0x65, 0x17, 0xc4, 0x2d, 0x07, 0x40, 0xdc, 0x32, 0xe1, 0x58, 0x6e, 0x38, 0x16, 0x36, + 0x5a, 0xce, 0xbe, 0x04, 0x07, 0xf8, 0xe4, 0x19, 0x93, 0xc1, 0x26, 0xd6, 0x7a, 0x2a, 0x9d, 0x2c, + 0xa6, 0xc4, 0xff, 0x2b, 0xc0, 0x0c, 0x6b, 0x72, 0x9c, 0x13, 0xeb, 0x03, 0x48, 0x59, 0xe6, 0x11, + 0x9b, 0x58, 0xd9, 0x87, 0x6f, 0x45, 0x88, 0xd8, 0xc0, 0x27, 0xc1, 0x95, 0x8b, 0x16, 0x47, 0x65, + 0xe0, 0x7e, 0xa0, 0x4c, 0xb9, 0x93, 0xe3, 0x72, 0x03, 0xe3, 0x92, 0x88, 0x8c, 0x3b, 0x50, 0xd8, + 0x55, 0x1c, 0x75, 0x5f, 0xb6, 0x78, 0x25, 0xc9, 0x2a, 0x97, 0xbc, 0x3b, 0x23, 0xe5, 0x29, 0xd9, + 0xad, 0xba, 0x4d, 0x5a, 0xce, 0x46, 0xba, 0x8d, 0xff, 0x92, 0xf5, 0xf9, 0xff, 0x13, 0xf8, 0x1c, + 0x72, 0x5b, 0xfe, 0x97, 0xad, 0xeb, 0x7f, 0x23, 0x01, 0x57, 0x2a, 0xfb, 0x58, 0x3d, 0xa8, 0x98, + 0x86, 0xad, 0xdb, 0x0e, 0xd1, 0x5d, 0x9c, 0xfd, 0xff, 0x16, 0x64, 0x8e, 0x74, 0x67, 0x5f, 0xd6, + 0xf4, 0xbd, 0x3d, 0x6a, 0xe7, 0xd2, 0x52, 0x9a, 0x10, 0xd6, 0xf4, 0xbd, 0x3d, 0xf4, 0x08, 0x52, + 0x1d, 0x53, 0x63, 0xee, 0x72, 0xfe, 0xe1, 0x62, 0x84, 0x78, 0x5a, 0x35, 0xbb, 0xd7, 0xd9, 0x34, + 0x35, 0x2c, 0xd1, 0xc2, 0xe8, 0x1a, 0x80, 0x4a, 0xa8, 0x5d, 0x53, 0x37, 0x1c, 0xbe, 0x4e, 0x06, + 0x28, 0xa8, 0x06, 0x19, 0x07, 0x5b, 0x1d, 0xdd, 0x50, 0x1c, 0x5c, 0x9a, 0xa4, 0xca, 0xbb, 0x19, + 0x59, 0xf1, 0x6e, 0x5b, 0x57, 0x95, 0x35, 0x6c, 0xab, 0x96, 0xde, 0x75, 0x4c, 0x8b, 0x6b, 0xd1, + 0x67, 0x16, 0xbf, 0x9f, 0x82, 0xd2, 0xa0, 0x6e, 0xe2, 0x1c, 0x21, 0xdb, 0x30, 0x45, 0xf0, 0x75, + 0xdb, 0xe1, 0x63, 0xe4, 0xe1, 0x30, 0x15, 0x44, 0xd4, 0x80, 0xe2, 0xf4, 0xb6, 0xc3, 0xab, 0xcd, + 0xe5, 0x2c, 0xfc, 0x3b, 0x01, 0xa6, 0xd8, 0x0d, 0xf4, 0x00, 0xd2, 0x7c, 0x43, 0x41, 0xa3, 0x75, + 0x4c, 0x96, 0x2f, 0x9f, 0x9d, 0x2e, 0x4e, 0xb3, 0x3d, 0x82, 0xb5, 0x2f, 0xfd, 0x9f, 0xd2, 0x34, + 0x2d, 0x57, 0xd7, 0x48, 0x6f, 0xd9, 0x8e, 0x62, 0x39, 0x74, 0xdb, 0x26, 0xc1, 0x70, 0x03, 0x25, + 0x6c, 0xe0, 0x13, 0xb4, 0x0e, 0x53, 0xb6, 0xa3, 0x38, 0x3d, 0x9b, 0xf7, 0xd7, 0x85, 0x2a, 0xdb, + 0xa0, 0x9c, 0x12, 0x97, 0x40, 0x1c, 0x1d, 0x0d, 0x3b, 0x8a, 0xde, 0xa6, 0x1d, 0x98, 0x91, 0xf8, + 0x95, 0xf8, 0x9b, 0x02, 0x4c, 0xb1, 0xa2, 0xe8, 0x0a, 0xcc, 0x49, 0xab, 0x5b, 0x4f, 0xab, 0x72, + 0x7d, 0x6b, 0xad, 0xda, 0xac, 0x4a, 0x9b, 0xf5, 0xad, 0xd5, 0x66, 0xb5, 0x38, 0x81, 0x2e, 0x03, + 0x72, 0x6f, 0x54, 0x9e, 0x6f, 0x35, 0xea, 0x8d, 0x66, 0x75, 0xab, 0x59, 0x14, 0xe8, 0xde, 0x02, + 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x13, 0x96, 0xfa, 0xa9, 0x72, 0xa3, 0xb9, 0xda, 0x6c, 0xc8, 0xd5, + 0x46, 0xb3, 0xbe, 0xb9, 0xda, 0xac, 0xae, 0x15, 0x93, 0x23, 0x4a, 0x91, 0x87, 0x48, 0x52, 0xb5, + 0xd2, 0x2c, 0xa6, 0x44, 0x07, 0x2e, 0x49, 0x58, 0x35, 0x3b, 0xdd, 0x9e, 0x83, 0x49, 0x2d, 0xed, + 0x38, 0x67, 0xca, 0x15, 0x98, 0xd6, 0xac, 0x13, 0xd9, 0xea, 0x19, 0x7c, 0x9e, 0x4c, 0x69, 0xd6, + 0x89, 0xd4, 0x33, 0xc4, 0x7f, 0x21, 0xc0, 0xe5, 0xfe, 0xc7, 0xc6, 0x39, 0x08, 0x5f, 0x40, 0x56, + 0xd1, 0x34, 0xac, 0xc9, 0x1a, 0x6e, 0x3b, 0x0a, 0x77, 0x46, 0xee, 0x07, 0x24, 0xf1, 0xcd, 0xb6, + 0x65, 0x6f, 0xb3, 0x6d, 0xf3, 0x65, 0xa5, 0x42, 0x2b, 0xb2, 0x46, 0x38, 0x5c, 0xf3, 0x43, 0x85, + 0x50, 0x8a, 0xf8, 0xbf, 0x52, 0x90, 0xab, 0x1a, 0x5a, 0xf3, 0x38, 0xd6, 0xb5, 0xe4, 0x32, 0x4c, + 0xa9, 0x66, 0xa7, 0xa3, 0x3b, 0xae, 0x82, 0xd8, 0x15, 0xfa, 0xd9, 0x80, 0x13, 0x99, 0x1c, 0xc3, + 0x95, 0xf2, 0xdd, 0x47, 0xf4, 0x1d, 0xb8, 0x42, 0xac, 0xa6, 0x65, 0x28, 0x6d, 0x99, 0x49, 0x93, + 0x1d, 0x4b, 0x6f, 0xb5, 0xb0, 0xc5, 0x37, 0xf8, 0xee, 0x46, 0xd4, 0xb3, 0xce, 0x39, 0x2a, 0x94, + 0xa1, 0xc9, 0xca, 0x4b, 0x97, 0xf4, 0x28, 0x32, 0xfa, 0x04, 0x80, 0x2c, 0x45, 0x74, 0xd3, 0xd0, + 0xe6, 0xf6, 0x68, 0xd8, 0xae, 0xa1, 0x6b, 0x82, 0x08, 0x03, 0xb9, 0xb6, 0xd1, 0x0a, 0x41, 0x0c, + 0xaf, 0x7b, 0xba, 0x85, 0xe5, 0x07, 0x5d, 0x95, 0x42, 0xf9, 0x74, 0x39, 0x7f, 0x76, 0xba, 0x08, + 0x12, 0x23, 0x3f, 0xd8, 0xae, 0x10, 0x04, 0xc1, 0x7e, 0x77, 0x55, 0xf4, 0x0a, 0xee, 0x05, 0x76, + 0x24, 0xc8, 0xca, 0xcb, 0x9b, 0xa5, 0x38, 0xf2, 0xbe, 0xde, 0xda, 0xc7, 0x96, 0xec, 0x6d, 0x1c, + 0xd3, 0x1d, 0xbc, 0xb4, 0x74, 0xd3, 0x67, 0xa8, 0x28, 0x06, 0xab, 0xfd, 0xaa, 0x53, 0xa3, 0x85, + 0x3d, 0x9d, 0x11, 0xe5, 0x77, 0x4d, 0xdd, 0x36, 0x8d, 0x52, 0x86, 0x29, 0x9f, 0x5d, 0xa1, 0x7b, + 0x50, 0x74, 0x8e, 0x0d, 0x79, 0x1f, 0x2b, 0x96, 0xb3, 0x8b, 0x15, 0x87, 0xac, 0xd2, 0x40, 0x4b, + 0x14, 0x9c, 0x63, 0xa3, 0x16, 0x20, 0xa3, 0x17, 0x50, 0xd4, 0x0d, 0x79, 0xaf, 0xad, 0xb7, 0xf6, + 0x1d, 0xf9, 0xc8, 0xd2, 0x1d, 0x6c, 0x97, 0x66, 0xa9, 0x42, 0xa2, 0xc6, 0x6d, 0x83, 0xef, 0xe4, + 0x6a, 0xaf, 0x48, 0x49, 0xae, 0x9a, 0xbc, 0x6e, 0x3c, 0xa1, 0xfc, 0x94, 0x68, 0xaf, 0xa7, 0xd2, + 0xd3, 0xc5, 0xb4, 0xf8, 0xdf, 0x04, 0xc8, 0xbb, 0xc3, 0x2d, 0xce, 0x99, 0x71, 0x17, 0x8a, 0xa6, + 0x81, 0xe5, 0xee, 0xbe, 0x62, 0x63, 0xae, 0x47, 0xbe, 0xe0, 0xe4, 0x4d, 0x03, 0x6f, 0x13, 0x32, + 0x53, 0x17, 0xda, 0x86, 0x59, 0xdb, 0x51, 0x5a, 0xba, 0xd1, 0x0a, 0xa8, 0x77, 0x72, 0x7c, 0xb7, + 0xbe, 0xc8, 0xb9, 0x3d, 0x7a, 0xc8, 0x4b, 0xf9, 0x63, 0x01, 0x66, 0x57, 0xb5, 0x8e, 0x6e, 0x34, + 0xba, 0x6d, 0x3d, 0xd6, 0xdd, 0x82, 0x9b, 0x90, 0xb1, 0x89, 0x4c, 0xdf, 0xe0, 0xfb, 0xd8, 0x2f, + 0x4d, 0xef, 0x10, 0xcb, 0xff, 0x0c, 0x0a, 0xf8, 0xb8, 0xab, 0xb3, 0x97, 0x01, 0x0c, 0xb2, 0xa4, + 0xc6, 0x6f, 0x5b, 0xde, 0xe7, 0x25, 0xb7, 0x78, 0x9b, 0x3e, 0x03, 0x14, 0x6c, 0x52, 0x9c, 0xd8, + 0xe5, 0x33, 0x98, 0xa3, 0xa2, 0x77, 0x0c, 0x3b, 0x66, 0x7d, 0x89, 0xbf, 0x08, 0xf3, 0x61, 0xd1, + 0x71, 0xd6, 0xfb, 0x15, 0xef, 0xe5, 0x4d, 0x6c, 0xc5, 0x0a, 0x37, 0x3d, 0x5d, 0x73, 0xc1, 0x71, + 0xd6, 0xf9, 0x57, 0x05, 0xb8, 0x4a, 0x65, 0xd3, 0xf7, 0x25, 0x7b, 0xd8, 0x7a, 0x86, 0x15, 0x3b, + 0x56, 0xac, 0x7c, 0x03, 0xa6, 0x18, 0xe6, 0xa5, 0xe3, 0x73, 0xb2, 0x9c, 0x25, 0x9e, 0x4b, 0xc3, + 0x31, 0x2d, 0xe2, 0xb9, 0xf0, 0x5b, 0xa2, 0x02, 0x0b, 0x51, 0xb5, 0x88, 0x79, 0x3b, 0x60, 0x96, + 0x3b, 0x8d, 0x64, 0x28, 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, 0xd0, 0xc8, 0x91, 0x1e, 0x2b, 0x9d, 0x47, 0xb4, 0xac, 0xeb, 0xfa, 0x71, 0x1d, + 0xfc, 0xcb, 0x24, 0x57, 0x02, 0x7b, 0x06, 0x2f, 0x1e, 0xab, 0x8f, 0xf2, 0x79, 0xe8, 0x75, 0x55, + 0xb0, 0xe1, 0x89, 0x0b, 0x34, 0x3c, 0xb0, 0x67, 0xee, 0x53, 0xd1, 0x67, 0x10, 0xd8, 0x15, 0x97, + 0x59, 0x9b, 0x5c, 0xf4, 0x73, 0x11, 0x75, 0xcc, 0xfa, 0x52, 0x18, 0xdd, 0x46, 0x15, 0x48, 0xe3, + 0xe3, 0xae, 0xac, 0x61, 0x5b, 0xe5, 0x86, 0x4b, 0x1c, 0xf6, 0x5e, 0x6d, 0x00, 0x0f, 0x4c, 0xe3, + 0xe3, 0x2e, 0x21, 0xa2, 0x1d, 0xb2, 0x7a, 0xb9, 0xae, 0x02, 0xad, 0xb6, 0x7d, 0x3e, 0xbc, 0xf0, + 0x47, 0x0a, 0x17, 0x57, 0xf0, 0xbc, 0x04, 0x26, 0x42, 0xfc, 0xa1, 0x00, 0x6f, 0x45, 0xf6, 0x5a, + 0x9c, 0x0b, 0xd9, 0x27, 0x90, 0xa2, 0x8d, 0x4f, 0x5c, 0xb0, 0xf1, 0x94, 0x4b, 0xfc, 0x5e, 0x82, + 0xcf, 0x71, 0x09, 0xb7, 0x4d, 0xa2, 0xd8, 0xd8, 0xf7, 0xc3, 0x9e, 0x43, 0xee, 0xd0, 0x74, 0x88, + 0x6f, 0xc2, 0xbb, 0x3d, 0x71, 0xe1, 0x6e, 0x9f, 0xa1, 0x02, 0xdc, 0x1e, 0x7f, 0x09, 0xb3, 0x86, + 0x69, 0xc8, 0x61, 0xa1, 0x17, 0x1f, 0x4b, 0x05, 0xc3, 0x34, 0x5e, 0x06, 0xe4, 0x7a, 0x76, 0xa6, + 0x4f, 0x13, 0x71, 0xda, 0x99, 0x5f, 0x13, 0x60, 0xce, 0x73, 0x9b, 0x62, 0xf6, 0xa0, 0x3f, 0x80, + 0xa4, 0x61, 0x1e, 0x5d, 0x64, 0xbf, 0x91, 0x94, 0x27, 0xab, 0x5e, 0xb8, 0x46, 0x71, 0xb6, 0xf7, + 0xdf, 0x27, 0x20, 0xf3, 0xb4, 0x12, 0x67, 0x2b, 0x3f, 0xe1, 0x7b, 0xd9, 0xac, 0xbf, 0xa3, 0x46, + 0xbb, 0xf7, 0xbc, 0xe5, 0xa7, 0x95, 0x0d, 0x7c, 0xe2, 0x8e, 0x76, 0xc2, 0x85, 0x56, 0x21, 0xe3, + 0xec, 0x5b, 0xd8, 0xde, 0x37, 0xdb, 0xda, 0x45, 0xdc, 0x1c, 0x9f, 0x6b, 0x01, 0xc3, 0x24, 0x95, + 0xeb, 0xc6, 0x47, 0x08, 0x11, 0xf1, 0x11, 0xe4, 0x31, 0x9e, 0xa7, 0x98, 0xb8, 0xc8, 0x63, 0x02, + 0x2e, 0xe2, 0x64, 0x71, 0x4a, 0x7c, 0x01, 0x40, 0x9a, 0x13, 0x67, 0x97, 0xfc, 0x8d, 0x24, 0xe4, + 0xb7, 0x7b, 0xf6, 0x7e, 0xcc, 0xa3, 0xaf, 0x02, 0xd0, 0xed, 0xd9, 0x14, 0x82, 0x1c, 0x1b, 0xbc, + 0xcd, 0xe7, 0x84, 0x5e, 0xb8, 0x8d, 0x66, 0x7c, 0xcd, 0x63, 0x03, 0xd5, 0xb8, 0x10, 0x2c, 0xfb, + 0xf1, 0x1b, 0x37, 0x46, 0x81, 0xd5, 0xe6, 0xb1, 0xb1, 0x89, 0x3d, 0x94, 0xca, 0x24, 0x61, 0x22, + 0xe9, 0x13, 0x98, 0x26, 0x17, 0xb2, 0x63, 0x5e, 0xa4, 0x9b, 0xa7, 0x08, 0x4f, 0xd3, 0x44, 0x8f, + 0x21, 0xc3, 0xb8, 0xc9, 0xea, 0x37, 0x45, 0x57, 0xbf, 0xa8, 0xb6, 0x70, 0x35, 0xd2, 0x75, 0x2f, + 0x4d, 0x59, 0xc9, 0x5a, 0x37, 0x0f, 0x93, 0x7b, 0xa6, 0xa5, 0x62, 0x1a, 0x94, 0x91, 0x96, 0xd8, + 0x05, 0xeb, 0xcf, 0xf5, 0x54, 0x3a, 0x5d, 0xcc, 0xac, 0xa7, 0xd2, 0x99, 0x22, 0x88, 0xbf, 0x29, + 0x40, 0xc1, 0xeb, 0x88, 0x38, 0x17, 0x84, 0x4a, 0x48, 0x8b, 0x17, 0xef, 0x0a, 0xa2, 0x40, 0xf1, + 0x3f, 0x52, 0x8f, 0x48, 0x35, 0x0f, 0x69, 0xcf, 0xc4, 0x39, 0x52, 0x1e, 0xb3, 0xe8, 0x9c, 0xc4, + 0x45, 0x7b, 0x97, 0x06, 0xea, 0x3c, 0x80, 0x79, 0xbd, 0x43, 0xec, 0xb9, 0xee, 0xb4, 0x4f, 0x38, + 0x6c, 0x73, 0xb0, 0xfb, 0x0a, 0x78, 0xce, 0xbf, 0x57, 0x71, 0x6f, 0x89, 0xff, 0x90, 0x6e, 0x80, + 0xfb, 0x2d, 0x89, 0x53, 0xd5, 0x75, 0xc8, 0x59, 0x4c, 0x34, 0x71, 0x6b, 0x2e, 0xa8, 0xed, 0x19, + 0x8f, 0x95, 0x28, 0xfc, 0xb7, 0x13, 0x50, 0x78, 0xd1, 0xc3, 0xd6, 0xc9, 0xd7, 0x49, 0xdd, 0xb7, + 0xa1, 0x70, 0xa4, 0xe8, 0x8e, 0xbc, 0x67, 0x5a, 0x72, 0xaf, 0xab, 0x29, 0x8e, 0x1b, 0x22, 0x92, + 0x23, 0xe4, 0x27, 0xa6, 0xb5, 0x43, 0x89, 0x08, 0x03, 0x3a, 0x30, 0xcc, 0x23, 0x43, 0x26, 0x64, + 0x0a, 0x94, 0x8f, 0x0d, 0xbe, 0x2b, 0x5d, 0xfe, 0xf0, 0xbf, 0x9e, 0x2e, 0x3e, 0x1a, 0x2b, 0xf0, + 0x8b, 0x06, 0xb9, 0xf5, 0x7a, 0xba, 0xb6, 0xbc, 0xb3, 0x53, 0x5f, 0x93, 0x8a, 0x54, 0xe4, 0x2b, + 0x26, 0xb1, 0x79, 0x6c, 0xd8, 0xe2, 0x3f, 0x4e, 0x40, 0xd1, 0xd7, 0x51, 0x9c, 0x1d, 0x59, 0x85, + 0xec, 0xeb, 0x1e, 0xb6, 0xf4, 0x37, 0xe8, 0x46, 0xe0, 0x8c, 0xc4, 0xec, 0x7c, 0x0e, 0x33, 0x21, + 0x0d, 0x24, 0x7f, 0x3a, 0x0d, 0x64, 0x8f, 0xfc, 0xc6, 0xa3, 0xfb, 0x30, 0xeb, 0x1c, 0x1b, 0x32, + 0x0b, 0xf9, 0x63, 0x61, 0x22, 0x6e, 0x64, 0x43, 0xc1, 0x21, 0xfa, 0x20, 0x74, 0x1a, 0x22, 0x62, + 0x8b, 0x7f, 0x20, 0x00, 0xa2, 0x8a, 0xaa, 0xb3, 0xd7, 0x06, 0x5f, 0x97, 0xf1, 0x74, 0x17, 0x8a, + 0x34, 0x88, 0x52, 0xd6, 0xf7, 0xe4, 0x8e, 0x6e, 0xdb, 0xba, 0xd1, 0xe2, 0x03, 0x2a, 0x4f, 0xe9, + 0xf5, 0xbd, 0x4d, 0x46, 0x15, 0xff, 0x2a, 0xcc, 0x85, 0x1a, 0x10, 0x67, 0x67, 0x5f, 0x87, 0x99, + 0x3d, 0xb3, 0x67, 0x68, 0x32, 0x7b, 0xa9, 0xc2, 0x77, 0x1c, 0xb3, 0x94, 0xc6, 0x9e, 0x27, 0xfe, + 0xcf, 0x04, 0xcc, 0x4b, 0xd8, 0x36, 0xdb, 0x87, 0x38, 0x7e, 0x15, 0xd6, 0x80, 0xbf, 0xce, 0x91, + 0xdf, 0x48, 0x93, 0x19, 0xc6, 0xcc, 0x96, 0xb9, 0xf0, 0xb6, 0xfd, 0xcd, 0xd1, 0x23, 0x76, 0x70, + 0xa3, 0x9e, 0x6f, 0xfb, 0xa5, 0x42, 0xdb, 0x7e, 0x26, 0x14, 0xf4, 0x96, 0x61, 0x12, 0x9b, 0x66, + 0xe3, 0xd7, 0x46, 0xaf, 0xe3, 0x82, 0xa1, 0xe5, 0x51, 0x95, 0xac, 0x33, 0x96, 0x06, 0x7e, 0xbd, + 0xd5, 0xeb, 0x50, 0xdf, 0xb9, 0x7c, 0x99, 0xd4, 0xf7, 0xec, 0x74, 0x31, 0x1f, 0xba, 0x67, 0x4b, + 0x79, 0xdd, 0xbb, 0x26, 0xd2, 0xc5, 0x6f, 0xc3, 0xa5, 0x3e, 0x65, 0xc7, 0xe9, 0xf1, 0xfc, 0xdb, + 0x24, 0x5c, 0x0d, 0x8b, 0x8f, 0x1b, 0xe2, 0x7c, 0xdd, 0x3b, 0xb4, 0x06, 0xb9, 0x8e, 0x6e, 0xbc, + 0xd9, 0xee, 0xe5, 0x4c, 0x47, 0x37, 0xfc, 0x9d, 0xe2, 0x88, 0xa1, 0x31, 0xf5, 0x95, 0x0e, 0x0d, + 0x05, 0x16, 0xa2, 0xfa, 0x2e, 0xce, 0xf1, 0xf1, 0x3d, 0x01, 0x66, 0xe2, 0xde, 0x96, 0x7b, 0xb3, + 0x90, 0x36, 0xb1, 0x09, 0xb9, 0xaf, 0x60, 0x1f, 0xef, 0xb7, 0x05, 0x40, 0x4d, 0xab, 0x67, 0x10, + 0x50, 0xfb, 0xcc, 0x6c, 0xc5, 0xd9, 0xcc, 0x79, 0x98, 0xd4, 0x0d, 0x0d, 0x1f, 0xd3, 0x66, 0xa6, + 0x24, 0x76, 0x11, 0x7a, 0x3b, 0x99, 0x1c, 0xeb, 0xed, 0xa4, 0xf8, 0x39, 0xcc, 0x85, 0xaa, 0x18, + 0x67, 0xfb, 0xff, 0x49, 0x02, 0xe6, 0x78, 0x43, 0x62, 0xdf, 0xc1, 0xfc, 0x26, 0x4c, 0xb6, 0x89, + 0xcc, 0x11, 0xfd, 0x4c, 0x9f, 0xe9, 0xf6, 0x33, 0x2d, 0x8c, 0x7e, 0x0e, 0xa0, 0x6b, 0xe1, 0x43, + 0x99, 0xb1, 0x26, 0xc7, 0x62, 0xcd, 0x10, 0x0e, 0x4a, 0x40, 0xbf, 0x00, 0x05, 0x32, 0x9f, 0xbb, + 0x96, 0xd9, 0x35, 0x6d, 0xe2, 0xb2, 0xd8, 0xe3, 0xa1, 0x9c, 0xd9, 0xb3, 0xd3, 0xc5, 0xdc, 0xa6, + 0x6e, 0x6c, 0x73, 0xc6, 0x66, 0x43, 0x22, 0x86, 0xc1, 0xbb, 0xb4, 0xc5, 0xff, 0x2c, 0xc0, 0xfc, + 0x57, 0xb6, 0xdb, 0xfb, 0x17, 0xa1, 0x2b, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, 0xb1, 0x67, 0xc6, + 0xb9, 0xef, 0xfe, 0x7d, 0x01, 0x66, 0x03, 0x82, 0xe3, 0xf4, 0x4f, 0xde, 0x48, 0x4f, 0xe2, 0x2f, + 0x12, 0x8f, 0x25, 0x38, 0xc8, 0xe3, 0x9c, 0x42, 0xbf, 0x97, 0x80, 0xcb, 0x15, 0xf6, 0x96, 0xda, + 0x0d, 0xdb, 0x88, 0x73, 0x64, 0x94, 0x60, 0xfa, 0x10, 0x5b, 0xb6, 0x6e, 0xb2, 0xd5, 0x33, 0x27, + 0xb9, 0x97, 0x68, 0x01, 0xd2, 0xb6, 0xa1, 0x74, 0xed, 0x7d, 0xd3, 0x7d, 0x3d, 0xe7, 0x5d, 0x7b, + 0x21, 0x26, 0x93, 0x6f, 0x1e, 0x62, 0x32, 0x35, 0x3a, 0xc4, 0x64, 0xfa, 0xa7, 0x08, 0x31, 0xe1, + 0xef, 0xc2, 0xfe, 0x93, 0x00, 0x57, 0x06, 0x34, 0x17, 0xe7, 0x68, 0xf9, 0x2e, 0x64, 0x55, 0x2e, + 0x98, 0xd8, 0x5b, 0xf6, 0xa2, 0xaf, 0x4e, 0x8a, 0xbd, 0x21, 0xec, 0x38, 0x3b, 0x5d, 0x04, 0xb7, + 0xaa, 0xf5, 0x35, 0xae, 0x1c, 0xf2, 0x5b, 0x13, 0x7f, 0x25, 0x07, 0x85, 0xea, 0x31, 0xdb, 0xe4, + 0x6e, 0xb0, 0x55, 0x1e, 0x3d, 0x81, 0x74, 0xd7, 0x32, 0x0f, 0x75, 0xb7, 0x19, 0xf9, 0x50, 0x7c, + 0x81, 0xdb, 0x8c, 0x3e, 0xae, 0x6d, 0xce, 0x21, 0x79, 0xbc, 0xa8, 0x09, 0x99, 0x67, 0xa6, 0xaa, + 0xb4, 0x9f, 0xe8, 0x6d, 0x77, 0xe4, 0xbf, 0x7f, 0xbe, 0xa0, 0x65, 0x8f, 0x67, 0x5b, 0x71, 0xf6, + 0xdd, 0x4e, 0xf0, 0x88, 0xa8, 0x0e, 0xe9, 0x9a, 0xe3, 0x74, 0xc9, 0x4d, 0x6e, 0x3b, 0xee, 0x8c, + 0x21, 0x94, 0xb0, 0xb8, 0xe1, 0xa8, 0x2e, 0x3b, 0x6a, 0xc2, 0xec, 0x53, 0x7a, 0x88, 0xaa, 0xd2, + 0x36, 0x7b, 0x5a, 0xc5, 0x34, 0xf6, 0xf4, 0x16, 0xb7, 0xbb, 0xb7, 0xc7, 0x90, 0xf9, 0xb4, 0xd2, + 0x90, 0x06, 0x05, 0xa0, 0x55, 0x48, 0x37, 0x1e, 0x71, 0x61, 0xcc, 0x2d, 0xbb, 0x35, 0x86, 0xb0, + 0xc6, 0x23, 0xc9, 0x63, 0x43, 0xeb, 0x90, 0x5d, 0xfd, 0xa2, 0x67, 0x61, 0x2e, 0x65, 0x6a, 0x68, + 0x70, 0x43, 0xbf, 0x14, 0xca, 0x25, 0x05, 0x99, 0x51, 0x03, 0xf2, 0xaf, 0x4c, 0xeb, 0xa0, 0x6d, + 0x2a, 0x6e, 0x0b, 0xa7, 0xa9, 0xb8, 0x6f, 0x8c, 0x21, 0xce, 0x65, 0x94, 0xfa, 0x44, 0xa0, 0x6f, + 0x43, 0x81, 0x74, 0x46, 0x53, 0xd9, 0x6d, 0xbb, 0x95, 0x4c, 0x53, 0xa9, 0xef, 0x8e, 0x21, 0xd5, + 0xe3, 0x74, 0xf7, 0xed, 0xfb, 0x44, 0x2d, 0x48, 0x90, 0x0b, 0x0d, 0x02, 0x84, 0x20, 0xd5, 0x25, + 0xfd, 0x2d, 0xd0, 0xf0, 0x23, 0xfa, 0x1b, 0xbd, 0x07, 0xd3, 0x86, 0xa9, 0x61, 0x77, 0x86, 0xe4, + 0xca, 0xf3, 0x67, 0xa7, 0x8b, 0x53, 0x5b, 0xa6, 0xc6, 0x1c, 0x12, 0xfe, 0x4b, 0x9a, 0x22, 0x85, + 0xea, 0xda, 0xc2, 0x12, 0xa4, 0x48, 0xbf, 0x13, 0xc3, 0xb4, 0xab, 0xd8, 0x78, 0xc7, 0xd2, 0xb9, + 0x34, 0xf7, 0x72, 0xe1, 0x9f, 0x27, 0x20, 0xd1, 0x78, 0x44, 0x5c, 0xee, 0xdd, 0x9e, 0x7a, 0x80, + 0x1d, 0x7e, 0x9f, 0x5f, 0x51, 0x57, 0xdc, 0xc2, 0x7b, 0x3a, 0xf3, 0x8c, 0x32, 0x12, 0xbf, 0x42, + 0xef, 0x00, 0x28, 0xaa, 0x8a, 0x6d, 0x5b, 0x76, 0x0f, 0xd7, 0x65, 0xa4, 0x0c, 0xa3, 0x6c, 0xe0, + 0x13, 0xc2, 0x66, 0x63, 0xd5, 0xc2, 0x8e, 0x1b, 0x3b, 0xc5, 0xae, 0x08, 0x9b, 0x83, 0x3b, 0x5d, + 0xd9, 0x31, 0x0f, 0xb0, 0x41, 0xc7, 0x49, 0x86, 0x98, 0x9a, 0x4e, 0xb7, 0x49, 0x08, 0xc4, 0x4a, + 0x62, 0x43, 0xf3, 0x4d, 0x5a, 0x46, 0xf2, 0xae, 0x89, 0x48, 0x0b, 0xb7, 0x74, 0x7e, 0x4a, 0x2c, + 0x23, 0xf1, 0x2b, 0xa2, 0x25, 0xa5, 0xe7, 0xec, 0xd3, 0x9e, 0xc8, 0x48, 0xf4, 0x37, 0xba, 0x0d, + 0x05, 0x16, 0x6e, 0x29, 0x63, 0x43, 0x95, 0xa9, 0x71, 0xcd, 0xd0, 0xdb, 0x39, 0x46, 0xae, 0x1a, + 0x2a, 0x31, 0xa5, 0xe8, 0x11, 0x70, 0x82, 0x7c, 0xd0, 0xb1, 0x89, 0x4e, 0x81, 0x94, 0x2a, 0x17, + 0xce, 0x4e, 0x17, 0xb3, 0x0d, 0x7a, 0x63, 0x63, 0xb3, 0x51, 0x5f, 0x93, 0xb2, 0xac, 0xd4, 0x46, + 0xc7, 0xae, 0x6b, 0x0b, 0xbf, 0x2e, 0x40, 0xf2, 0x69, 0xa5, 0x71, 0x61, 0x95, 0xb9, 0x15, 0x4d, + 0x06, 0x2a, 0x7a, 0x07, 0x0a, 0xbb, 0x7a, 0xbb, 0xad, 0x1b, 0x2d, 0xe2, 0x05, 0x7d, 0x17, 0xab, + 0xae, 0xc2, 0xf2, 0x9c, 0xbc, 0xcd, 0xa8, 0x68, 0x09, 0xb2, 0xaa, 0x85, 0x35, 0x6c, 0x38, 0xba, + 0xd2, 0xb6, 0xb9, 0xe6, 0x82, 0xa4, 0x85, 0x5f, 0x16, 0x60, 0x92, 0xce, 0x00, 0xf4, 0x36, 0x64, + 0x54, 0xd3, 0x70, 0x14, 0xdd, 0xe0, 0xa6, 0x2c, 0x23, 0xf9, 0x84, 0xa1, 0xd5, 0xbb, 0x0e, 0x33, + 0x8a, 0xaa, 0x9a, 0x3d, 0xc3, 0x91, 0x0d, 0xa5, 0x83, 0x79, 0x35, 0xb3, 0x9c, 0xb6, 0xa5, 0x74, + 0x30, 0x5a, 0x04, 0xf7, 0xd2, 0x3b, 0x33, 0x99, 0x91, 0x80, 0x93, 0x36, 0xf0, 0xc9, 0xc2, 0x1f, + 0x0a, 0x90, 0x76, 0xe7, 0x0c, 0xa9, 0x46, 0x0b, 0x1b, 0xd8, 0x52, 0x1c, 0xd3, 0xab, 0x86, 0x47, + 0xe8, 0x5f, 0x2a, 0x33, 0xfe, 0x52, 0x39, 0x0f, 0x93, 0x0e, 0x99, 0x16, 0xbc, 0x06, 0xec, 0x82, + 0x6e, 0x47, 0xb7, 0x95, 0x16, 0xdb, 0x8d, 0xcb, 0x48, 0xec, 0x82, 0x34, 0x86, 0x47, 0xed, 0x32, + 0x8d, 0xf0, 0x2b, 0x52, 0x53, 0x16, 0x5b, 0xba, 0x8b, 0x5b, 0xba, 0x41, 0xc7, 0x52, 0x52, 0x02, + 0x4a, 0x2a, 0x13, 0x0a, 0x7a, 0x0b, 0x32, 0xac, 0x00, 0x36, 0x34, 0x3a, 0xa0, 0x92, 0x52, 0x9a, + 0x12, 0xaa, 0x86, 0xb6, 0x80, 0x21, 0xe3, 0x4d, 0x4e, 0xd2, 0x6d, 0x3d, 0xdb, 0x53, 0x24, 0xfd, + 0x8d, 0xde, 0x87, 0xf9, 0xd7, 0x3d, 0xa5, 0xad, 0xef, 0xd1, 0x8d, 0x36, 0x52, 0x8c, 0xe9, 0x8c, + 0xb5, 0x04, 0x79, 0xf7, 0xa8, 0x04, 0xaa, 0x3a, 0x77, 0x2e, 0x27, 0xfd, 0xb9, 0x2c, 0xfe, 0xae, + 0x00, 0xb3, 0x2c, 0x70, 0x88, 0xc5, 0xbb, 0xc6, 0xe7, 0x87, 0x7c, 0x0c, 0x19, 0x4d, 0x71, 0x14, + 0x76, 0x0a, 0x34, 0x31, 0xf2, 0x14, 0xa8, 0x77, 0x5a, 0x41, 0x71, 0x14, 0x7a, 0x12, 0x14, 0x41, + 0x8a, 0xfc, 0x66, 0x07, 0x66, 0x25, 0xfa, 0x5b, 0xfc, 0x0c, 0x50, 0xb0, 0xa2, 0x71, 0x7a, 0x64, + 0xf7, 0xe0, 0x12, 0xd1, 0x75, 0xd5, 0x50, 0xad, 0x93, 0xae, 0xa3, 0x9b, 0xc6, 0x73, 0xfa, 0xd7, + 0x46, 0xc5, 0xc0, 0x7b, 0x29, 0xfa, 0x3a, 0x4a, 0xfc, 0xfd, 0x29, 0xc8, 0x55, 0x8f, 0xbb, 0xa6, + 0x15, 0xeb, 0x2e, 0x56, 0x19, 0xa6, 0x39, 0xd0, 0x1f, 0xf1, 0xea, 0xb9, 0xcf, 0x98, 0xbb, 0xef, + 0xdd, 0x39, 0x23, 0x2a, 0x03, 0xb0, 0x98, 0x54, 0x1a, 0x77, 0x94, 0xbc, 0xc0, 0x9b, 0x32, 0xca, + 0x46, 0xa8, 0x68, 0x0b, 0xb2, 0x9d, 0x43, 0x55, 0x95, 0xf7, 0xf4, 0xb6, 0xc3, 0x43, 0xfb, 0xa2, + 0xa3, 0xd0, 0x37, 0x5f, 0x56, 0x2a, 0x4f, 0x68, 0x21, 0x16, 0x65, 0xe7, 0x5f, 0x4b, 0x40, 0x24, + 0xb0, 0xdf, 0xe8, 0x5d, 0xe0, 0xa7, 0x76, 0x64, 0xdb, 0x3d, 0x88, 0x57, 0xce, 0x9d, 0x9d, 0x2e, + 0x66, 0x24, 0x4a, 0x6d, 0x34, 0x9a, 0x52, 0x86, 0x15, 0x68, 0xd8, 0x0e, 0xba, 0x01, 0x39, 0xb3, + 0xa3, 0x3b, 0xb2, 0xeb, 0x24, 0x71, 0x8f, 0x72, 0x86, 0x10, 0x5d, 0x27, 0x0a, 0x35, 0xe1, 0x0e, + 0x36, 0xe8, 0x68, 0x27, 0xed, 0x94, 0x77, 0xd9, 0xe6, 0xa3, 0xc3, 0x66, 0xb4, 0x6c, 0x76, 0x1d, + 0xbd, 0xa3, 0x7f, 0x41, 0xdf, 0x50, 0xf3, 0x97, 0x46, 0x37, 0x58, 0x71, 0xd2, 0xbe, 0x32, 0xdd, + 0x95, 0xe4, 0x65, 0x9f, 0x07, 0x8a, 0xa2, 0xbf, 0x29, 0xc0, 0x65, 0xae, 0x48, 0x79, 0x97, 0x86, + 0xd1, 0x2b, 0x6d, 0xdd, 0x39, 0x91, 0x0f, 0x0e, 0x4b, 0x69, 0xea, 0xb7, 0xfe, 0x6c, 0x64, 0x87, + 0x04, 0xc6, 0xc1, 0xb2, 0xdb, 0x2d, 0x27, 0xcf, 0x38, 0xf3, 0xc6, 0x61, 0xd5, 0x70, 0xac, 0x93, + 0xf2, 0x95, 0xb3, 0xd3, 0xc5, 0xb9, 0xc1, 0xbb, 0x2f, 0xa5, 0x39, 0x7b, 0x90, 0x05, 0xd5, 0x00, + 0xb0, 0x37, 0x0e, 0xe9, 0x8a, 0x11, 0xed, 0x7f, 0x44, 0x0e, 0x58, 0x29, 0xc0, 0x8b, 0xee, 0x42, + 0x91, 0x1f, 0x9e, 0xd9, 0xd3, 0xdb, 0x58, 0xb6, 0xf5, 0x2f, 0x30, 0x5d, 0x5b, 0x92, 0x52, 0x9e, + 0xd1, 0x89, 0x88, 0x86, 0xfe, 0x05, 0x5e, 0xf8, 0x2e, 0x94, 0x86, 0xd5, 0x3e, 0x38, 0x05, 0x32, + 0xec, 0x8d, 0xec, 0x47, 0xe1, 0xed, 0x98, 0x31, 0x86, 0x2a, 0xdf, 0x92, 0xf9, 0x38, 0xf1, 0x91, + 0x20, 0xfe, 0xd3, 0x04, 0xe4, 0xca, 0xbd, 0xf6, 0xc1, 0xf3, 0x6e, 0xa3, 0xd7, 0xe9, 0x28, 0xd6, + 0x09, 0x31, 0x83, 0xcc, 0x50, 0x90, 0x0a, 0x0a, 0xcc, 0x0c, 0x52, 0x4b, 0xa0, 0x7f, 0x81, 0xc9, + 0xe2, 0x14, 0x3c, 0xf8, 0xcd, 0x8e, 0x09, 0xd0, 0x36, 0x04, 0x4e, 0x73, 0x9b, 0x47, 0x36, 0xfa, + 0x08, 0x4a, 0x81, 0x82, 0x74, 0xef, 0x44, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0xed, 0xff, 0x25, 0xa5, + 0x40, 0x48, 0x4e, 0x9d, 0xdc, 0xae, 0xb2, 0xbb, 0xa8, 0x09, 0x33, 0xa4, 0xe0, 0x89, 0x4c, 0x97, + 0x10, 0x77, 0x7f, 0xf6, 0x41, 0x44, 0xb3, 0x42, 0xf5, 0x5e, 0xa6, 0xfa, 0xa9, 0x50, 0x1e, 0xfa, + 0x53, 0xca, 0x62, 0x9f, 0xb2, 0xf0, 0x29, 0x14, 0xfb, 0x0b, 0x04, 0x75, 0x99, 0x62, 0xba, 0x9c, + 0x0f, 0xea, 0x32, 0x19, 0xd0, 0xd3, 0x7a, 0x2a, 0x9d, 0x2a, 0x4e, 0x8a, 0x7f, 0x96, 0x84, 0xbc, + 0x3b, 0xcc, 0xe2, 0x04, 0x3a, 0x65, 0x98, 0x24, 0x83, 0xc2, 0x0d, 0x20, 0xb9, 0x3d, 0x62, 0x74, + 0xf3, 0xc0, 0x74, 0x32, 0x58, 0x5c, 0x90, 0x4c, 0x59, 0xe3, 0x30, 0x38, 0x0b, 0xbf, 0x9c, 0x80, + 0x14, 0xc5, 0x16, 0x0f, 0x20, 0x45, 0x17, 0x0a, 0x61, 0x9c, 0x85, 0x82, 0x16, 0xf5, 0x96, 0xb3, + 0x44, 0xc0, 0x35, 0x25, 0x3e, 0xdf, 0xbe, 0xf2, 0xc1, 0x83, 0x87, 0xd4, 0xd8, 0xcc, 0x48, 0xfc, + 0x0a, 0x95, 0x69, 0x64, 0x93, 0x69, 0x39, 0x58, 0xe3, 0x3e, 0xfd, 0xd2, 0x79, 0xfd, 0xeb, 0x2e, + 0x4a, 0x2e, 0x1f, 0xba, 0x0a, 0x49, 0x62, 0xc5, 0xa6, 0x59, 0x90, 0xc2, 0xd9, 0xe9, 0x62, 0x92, + 0xd8, 0x2f, 0x42, 0x43, 0x2b, 0x90, 0x0d, 0x9b, 0x0c, 0xe2, 0xc1, 0x51, 0xc3, 0x18, 0x98, 0xee, + 0xd0, 0xf6, 0xa6, 0x16, 0xc3, 0xb3, 0xbc, 0x8f, 0xff, 0x3c, 0x05, 0xb9, 0x7a, 0x27, 0xee, 0x25, + 0x65, 0x35, 0xdc, 0xc3, 0x51, 0x40, 0x28, 0xf4, 0xd0, 0x88, 0x0e, 0x0e, 0xad, 0xe0, 0xc9, 0x8b, + 0xad, 0xe0, 0x75, 0xe2, 0x29, 0xf3, 0x4c, 0x0e, 0xc9, 0x21, 0x98, 0x27, 0xfc, 0x7c, 0xea, 0xa7, + 0x48, 0x84, 0xc7, 0x3f, 0xaa, 0x41, 0x03, 0x4d, 0x3e, 0xa5, 0x0e, 0x39, 0x1b, 0x65, 0x53, 0xe3, + 0x8f, 0xb2, 0x69, 0x6c, 0x68, 0x74, 0x51, 0x0b, 0x5b, 0xd4, 0xe9, 0x37, 0xb7, 0xa8, 0x0b, 0x0e, + 0x1f, 0xac, 0x1f, 0x43, 0x52, 0xd3, 0xdd, 0xce, 0x19, 0x7f, 0xa9, 0x26, 0x4c, 0xe7, 0x8c, 0xda, + 0x54, 0x70, 0xd4, 0xb2, 0x51, 0xb2, 0x50, 0x07, 0xf0, 0x75, 0x83, 0x96, 0x60, 0xca, 0x6c, 0x6b, + 0xee, 0x59, 0x95, 0x5c, 0x39, 0x73, 0x76, 0xba, 0x38, 0xf9, 0xbc, 0xad, 0xd5, 0xd7, 0xa4, 0x49, + 0xb3, 0xad, 0xd5, 0x35, 0x9a, 0x46, 0x03, 0x1f, 0xc9, 0x5e, 0x20, 0xdb, 0x8c, 0x34, 0x6d, 0xe0, + 0xa3, 0x35, 0x6c, 0xab, 0x7c, 0xc0, 0xfd, 0x96, 0x00, 0x79, 0x57, 0xf7, 0xf1, 0x1a, 0x95, 0xb4, + 0xde, 0xe1, 0x93, 0x2c, 0x79, 0xb1, 0x49, 0xe6, 0xf2, 0xf1, 0x13, 0xbc, 0xbf, 0x2a, 0xf0, 0xd0, + 0xe4, 0x86, 0xaa, 0x38, 0xc4, 0xa9, 0x88, 0x71, 0x62, 0xdc, 0x83, 0xa2, 0xa5, 0x18, 0x9a, 0xd9, + 0xd1, 0xbf, 0xc0, 0x6c, 0x23, 0xd4, 0xe6, 0x6f, 0x2d, 0x0b, 0x1e, 0x9d, 0xee, 0xfa, 0xd9, 0xe2, + 0x1f, 0x26, 0x78, 0x18, 0xb3, 0x57, 0x8d, 0x38, 0xd5, 0xf5, 0x1d, 0x98, 0xed, 0xcf, 0x69, 0xe2, + 0xce, 0xd6, 0xf7, 0x22, 0xe4, 0x45, 0x55, 0x84, 0x85, 0x23, 0xba, 0xb1, 0xf1, 0x7d, 0xf9, 0x4d, + 0x6c, 0x54, 0x81, 0x6c, 0x30, 0x55, 0x4a, 0x72, 0xec, 0x54, 0x29, 0x60, 0x79, 0x09, 0x52, 0x16, + 0x7e, 0x1e, 0x26, 0xe9, 0xed, 0x37, 0x30, 0xd1, 0xbc, 0x37, 0xff, 0x34, 0x01, 0x37, 0x69, 0xed, + 0x5f, 0x62, 0x4b, 0xdf, 0x3b, 0xd9, 0xb6, 0x4c, 0x07, 0xab, 0x0e, 0xd6, 0xfc, 0x83, 0x26, 0xb1, + 0xda, 0xbd, 0x4c, 0xd7, 0x7d, 0xc0, 0x85, 0xe2, 0xc5, 0x3c, 0x2e, 0xb4, 0x01, 0x05, 0x1e, 0x19, + 0xa0, 0xb4, 0xf5, 0x43, 0x2c, 0x2b, 0xce, 0x45, 0x56, 0xb7, 0x1c, 0xe3, 0x5d, 0x25, 0xac, 0xab, + 0x0e, 0xd2, 0x20, 0xc3, 0x85, 0xe9, 0x1a, 0xcf, 0xf0, 0xf3, 0xf4, 0xa7, 0xdb, 0x50, 0x4c, 0xb3, + 0xf0, 0x84, 0xfa, 0x9a, 0x94, 0x66, 0x92, 0xeb, 0x9a, 0xf8, 0x5f, 0x04, 0xb8, 0x75, 0x8e, 0x8a, + 0xe3, 0x1c, 0xba, 0x0b, 0x90, 0x3e, 0x24, 0x0f, 0xd2, 0xb9, 0x8e, 0xd3, 0x92, 0x77, 0x8d, 0x36, + 0x21, 0xb7, 0xa7, 0xe8, 0x6d, 0x7f, 0x48, 0x0f, 0x0f, 0x2f, 0x8c, 0x0e, 0xa6, 0x9d, 0x61, 0xec, + 0x6c, 0x0c, 0x8b, 0xbf, 0x91, 0x80, 0xd9, 0x55, 0x4d, 0x6b, 0x34, 0xb8, 0x0d, 0x8c, 0x6f, 0xa4, + 0xb8, 0x20, 0x33, 0xe1, 0x83, 0x4c, 0xf4, 0x1e, 0x20, 0x4d, 0xb7, 0x59, 0x26, 0x11, 0x7b, 0x5f, + 0xd1, 0xcc, 0x23, 0x3f, 0xac, 0x62, 0xd6, 0xbd, 0xd3, 0x70, 0x6f, 0xa0, 0x06, 0x50, 0xb4, 0x23, + 0xdb, 0x8e, 0xe2, 0xbd, 0x37, 0xba, 0x35, 0xd6, 0x89, 0x30, 0x06, 0x83, 0xbc, 0x4b, 0x29, 0x43, + 0xe4, 0xd0, 0x9f, 0xc4, 0x6f, 0xd7, 0x49, 0xd3, 0x1d, 0x59, 0xb1, 0xdd, 0xe3, 0x3f, 0x2c, 0x87, + 0x49, 0x9e, 0xd1, 0x57, 0x6d, 0x76, 0xaa, 0x87, 0x9d, 0x57, 0xf0, 0x55, 0x13, 0x27, 0x24, 0xfe, + 0x07, 0x02, 0xe4, 0x25, 0xbc, 0x67, 0x61, 0x3b, 0xd6, 0x4d, 0x81, 0x27, 0x30, 0x63, 0x31, 0xa9, + 0xf2, 0x9e, 0x65, 0x76, 0x2e, 0x32, 0xaf, 0xb2, 0x9c, 0xf1, 0x89, 0x65, 0x76, 0xb8, 0x61, 0x79, + 0x09, 0x05, 0xaf, 0x8e, 0x71, 0x36, 0xfe, 0x77, 0xe9, 0x69, 0x67, 0x26, 0x38, 0xee, 0xf8, 0x86, + 0x78, 0x35, 0x40, 0x5f, 0x54, 0x05, 0x2b, 0x1a, 0xa7, 0x1a, 0xfe, 0x87, 0x00, 0xf9, 0x46, 0x6f, + 0x97, 0xa5, 0xaa, 0x8a, 0x4f, 0x03, 0x55, 0xc8, 0xb4, 0xf1, 0x9e, 0x23, 0xbf, 0x51, 0xa4, 0x7d, + 0x9a, 0xb0, 0xd2, 0x73, 0x06, 0x4f, 0x01, 0x2c, 0x7a, 0x42, 0x8e, 0xca, 0x49, 0x5e, 0x50, 0x4e, + 0x86, 0xf2, 0x12, 0xb2, 0xf8, 0xcf, 0x12, 0x50, 0xf0, 0x9a, 0x19, 0xa7, 0x95, 0x7c, 0x15, 0xb2, + 0x0e, 0xc9, 0x8b, 0x58, 0x87, 0x59, 0x1e, 0xd2, 0x11, 0x6d, 0x21, 0x96, 0x61, 0x8e, 0x3a, 0x2e, + 0xb2, 0xd2, 0xed, 0xb6, 0x75, 0x17, 0xee, 0x52, 0xfb, 0x93, 0x92, 0x66, 0xe9, 0xad, 0x55, 0x76, + 0x87, 0x02, 0x5d, 0x32, 0xe6, 0xf6, 0x2c, 0x8c, 0xbf, 0xc0, 0x32, 0x45, 0x5e, 0x17, 0x09, 0x59, + 0xc9, 0x32, 0xc6, 0x06, 0xe1, 0xe3, 0x63, 0xee, 0x15, 0xcc, 0x52, 0x9d, 0xc6, 0x7d, 0xb2, 0x57, + 0xfc, 0x7b, 0x09, 0x40, 0x41, 0xc9, 0x5f, 0x5d, 0x5f, 0x24, 0xe2, 0xeb, 0x8b, 0x77, 0x01, 0xb1, + 0xa0, 0x45, 0x5b, 0xee, 0x62, 0x4b, 0xb6, 0xb1, 0x6a, 0xf2, 0x34, 0x4a, 0x82, 0x54, 0xe4, 0x77, + 0xb6, 0xb1, 0xd5, 0xa0, 0x74, 0xf4, 0x18, 0xc0, 0xf7, 0xc8, 0xf8, 0x82, 0x31, 0xd2, 0x21, 0x93, + 0x32, 0x9e, 0x2b, 0x26, 0x7e, 0x7f, 0x01, 0x66, 0xb8, 0x0e, 0x77, 0x0c, 0xdd, 0x34, 0xd0, 0x03, + 0x48, 0xb6, 0xf8, 0xd6, 0x7f, 0x36, 0x72, 0x9b, 0xce, 0xcf, 0x13, 0x57, 0x9b, 0x90, 0x48, 0x59, + 0xc2, 0xd2, 0xed, 0x39, 0x11, 0x8e, 0x91, 0x1f, 0x88, 0x1d, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, 0x80, + 0x82, 0xea, 0xe7, 0xc1, 0x92, 0x09, 0x7b, 0x72, 0x28, 0x88, 0x8a, 0xcc, 0x3c, 0x56, 0x9b, 0x90, + 0xf2, 0x6a, 0xe8, 0x06, 0xaa, 0x04, 0xd3, 0x2f, 0xa5, 0x06, 0xa2, 0xbc, 0xfc, 0x23, 0xc4, 0xe1, + 0xd4, 0x4f, 0xb5, 0x89, 0x40, 0x96, 0x26, 0xf4, 0x31, 0x4c, 0x69, 0x34, 0xd1, 0x0f, 0x1f, 0xd1, + 0x51, 0x83, 0x2e, 0x94, 0x4f, 0xa9, 0x36, 0x21, 0x71, 0x0e, 0xb4, 0x0e, 0x33, 0xec, 0x17, 0x73, + 0x53, 0x38, 0xb2, 0xbc, 0x35, 0x5c, 0x42, 0x60, 0x21, 0xa8, 0x4d, 0x48, 0x59, 0xcd, 0xa7, 0xa2, + 0x6f, 0x42, 0xca, 0x56, 0x15, 0x17, 0x5b, 0x5e, 0x1b, 0x92, 0xb5, 0xc3, 0x67, 0xa6, 0xa5, 0xd1, + 0x63, 0x96, 0xf9, 0xd1, 0x39, 0x76, 0xb7, 0xf9, 0xa2, 0xaa, 0x1f, 0x3a, 0x1d, 0x4e, 0xaa, 0x8f, + 0x29, 0x01, 0x3d, 0x85, 0xac, 0x42, 0xfc, 0x3d, 0x99, 0x9e, 0xaf, 0xa4, 0xfb, 0x7a, 0xd1, 0x2f, + 0xd7, 0x07, 0xce, 0xc3, 0xd6, 0xe8, 0x11, 0x74, 0x97, 0xe8, 0x0b, 0xea, 0x60, 0xab, 0x85, 0x4b, + 0xd9, 0xd1, 0x82, 0x82, 0xb1, 0x5d, 0x9e, 0x20, 0x4a, 0x24, 0x7e, 0x9f, 0x77, 0xb8, 0x99, 0x36, + 0x6a, 0x66, 0xe8, 0xeb, 0xdc, 0x88, 0x63, 0x3b, 0xb5, 0x09, 0x69, 0x66, 0x3f, 0x40, 0x46, 0xcb, + 0x90, 0x68, 0xa9, 0xa5, 0xdc, 0xd0, 0x19, 0xe2, 0x1d, 0x4d, 0xa9, 0x4d, 0x48, 0x89, 0x96, 0x8a, + 0x3e, 0x85, 0x34, 0x3b, 0x67, 0x70, 0x6c, 0x94, 0xf2, 0x43, 0xed, 0x44, 0xf8, 0xb4, 0x46, 0x6d, + 0x42, 0xa2, 0x47, 0x1b, 0xc8, 0xf3, 0xb6, 0x21, 0x6f, 0xb1, 0xe0, 0x38, 0x37, 0x94, 0xb5, 0x38, + 0xf4, 0x15, 0x77, 0x54, 0x34, 0x6b, 0x8d, 0x7a, 0xfe, 0x01, 0x3a, 0xfa, 0x0e, 0xcc, 0x87, 0x25, + 0xf2, 0x91, 0x36, 0x3b, 0xf4, 0x75, 0xed, 0xd0, 0xc8, 0xca, 0xda, 0x84, 0x84, 0xac, 0x81, 0x9b, + 0xe8, 0x43, 0x98, 0x64, 0xbd, 0x86, 0xa8, 0xc8, 0xa8, 0xa8, 0x8d, 0xbe, 0x0e, 0x63, 0xe5, 0xc9, + 0xe0, 0x77, 0x78, 0x84, 0x98, 0xdc, 0x36, 0x5b, 0xa5, 0xb9, 0xa1, 0x83, 0x7f, 0x30, 0xd6, 0x8d, + 0x0c, 0x7e, 0xc7, 0xa7, 0x92, 0x7e, 0xb7, 0xd8, 0x1d, 0x1e, 0x56, 0x34, 0x3f, 0xb4, 0xdf, 0x23, + 0x02, 0xc7, 0x6a, 0x34, 0x76, 0xdf, 0x27, 0x93, 0xaa, 0x59, 0x2c, 0xd1, 0x8c, 0x4c, 0xe7, 0xd4, + 0xa5, 0xa1, 0x55, 0x1b, 0xcc, 0xc4, 0x53, 0xa3, 0x3e, 0x92, 0x47, 0x45, 0x2f, 0xa1, 0xc8, 0xd3, + 0x41, 0xf8, 0xef, 0x14, 0x2e, 0x53, 0x79, 0xf7, 0x22, 0x4d, 0x57, 0x54, 0x4c, 0x4e, 0x6d, 0x42, + 0x2a, 0xa8, 0xe1, 0x3b, 0xe8, 0x33, 0x98, 0xa5, 0xf2, 0x64, 0xd5, 0xcf, 0xe0, 0x51, 0x2a, 0x0d, + 0xe4, 0x83, 0x18, 0x9e, 0xec, 0xc3, 0x95, 0x5c, 0x54, 0xfb, 0x6e, 0x91, 0x61, 0xac, 0x1b, 0xba, + 0x43, 0xad, 0xec, 0xc2, 0xd0, 0x61, 0x1c, 0xce, 0x1b, 0x48, 0x86, 0xb1, 0xce, 0x28, 0x64, 0x18, + 0x3b, 0x3c, 0xe6, 0x8c, 0x77, 0xc7, 0xdb, 0x43, 0x87, 0x71, 0x54, 0x70, 0x1a, 0x19, 0xc6, 0x4e, + 0x90, 0x4e, 0x86, 0x31, 0x33, 0x10, 0x7d, 0x72, 0xdf, 0x19, 0x3a, 0x8c, 0x87, 0x9e, 0x73, 0x26, + 0xc3, 0x58, 0x19, 0xb8, 0x89, 0xd6, 0x00, 0x98, 0x3b, 0x43, 0x17, 0xc5, 0x6b, 0x43, 0x17, 0x83, + 0xfe, 0xd8, 0x33, 0xb2, 0x18, 0xb4, 0x5d, 0x1a, 0x31, 0x64, 0x14, 0x2c, 0xc9, 0xf4, 0x15, 0x6a, + 0x69, 0x71, 0xa8, 0x21, 0x1b, 0x78, 0xdd, 0x49, 0x0c, 0xd9, 0x91, 0x47, 0x24, 0xab, 0x0a, 0xdb, + 0xf3, 0x2d, 0x2d, 0x0d, 0x37, 0xcb, 0xc1, 0x57, 0x3f, 0xd4, 0x2c, 0x53, 0x02, 0x5a, 0x85, 0x0c, + 0x59, 0xf3, 0x4f, 0xa8, 0x19, 0xba, 0x3e, 0xd4, 0x27, 0xed, 0x3b, 0x9c, 0x52, 0x9b, 0x90, 0xd2, + 0xaf, 0x39, 0x89, 0x3c, 0x9e, 0xed, 0x86, 0x95, 0xc4, 0xa1, 0x8f, 0x0f, 0xed, 0x9c, 0x92, 0xc7, + 0x33, 0x0e, 0xa4, 0xc2, 0x25, 0xd6, 0x57, 0xfc, 0xc8, 0xb1, 0xc5, 0x4f, 0xc9, 0x96, 0x6e, 0x50, + 0x51, 0x43, 0xb7, 0x95, 0x22, 0x4f, 0x42, 0xd7, 0x26, 0xa4, 0x39, 0x65, 0xf0, 0x2e, 0x99, 0xf0, + 0x7c, 0xe9, 0x61, 0x9b, 0x51, 0xa5, 0x9b, 0x43, 0x27, 0x7c, 0xc4, 0x1e, 0x1e, 0x99, 0xf0, 0x4a, + 0x80, 0xcc, 0x16, 0x20, 0x4d, 0xb6, 0x6d, 0xf6, 0xc2, 0xfd, 0xd6, 0x88, 0x05, 0xa8, 0x6f, 0x17, + 0x80, 0x2d, 0x40, 0x5a, 0x83, 0x71, 0x12, 0x41, 0x6a, 0x1b, 0x2b, 0x16, 0x37, 0xb3, 0xb7, 0x87, + 0x0a, 0x1a, 0x48, 0xd1, 0x47, 0x04, 0xa9, 0x1e, 0x91, 0x38, 0x3c, 0x96, 0x9b, 0x47, 0x86, 0x3b, + 0x8c, 0x77, 0x86, 0x3a, 0x3c, 0x91, 0x89, 0x6e, 0x88, 0xc3, 0x63, 0x85, 0x6e, 0xa0, 0x9f, 0x83, + 0x69, 0x0e, 0xdf, 0x4a, 0x77, 0x47, 0xb8, 0xb1, 0x41, 0xc4, 0x4d, 0xe6, 0x35, 0xe7, 0x61, 0x56, + 0x96, 0xc1, 0x46, 0xd6, 0xbc, 0x7b, 0x23, 0xac, 0xec, 0x00, 0x72, 0x65, 0x56, 0xd6, 0x27, 0x13, + 0x2b, 0xcb, 0xc6, 0x29, 0x5f, 0xeb, 0xee, 0x0f, 0xb5, 0xb2, 0x83, 0x47, 0x5f, 0x88, 0x95, 0x7d, + 0xed, 0x53, 0x49, 0xcb, 0x6c, 0x06, 0x9f, 0x4a, 0xdf, 0x18, 0xda, 0xb2, 0x30, 0x8e, 0x24, 0x2d, + 0xe3, 0x3c, 0xa4, 0xdb, 0x98, 0x4b, 0xcc, 0x34, 0xfd, 0xee, 0xf0, 0x73, 0xfd, 0xfd, 0xa0, 0xa3, + 0xe6, 0x6e, 0x54, 0x32, 0x0d, 0x7b, 0x86, 0xca, 0xe2, 0x87, 0x8e, 0xb9, 0xa6, 0xde, 0x1b, 0x6d, + 0xa8, 0xa2, 0x0e, 0x6b, 0x7b, 0x86, 0x2a, 0x74, 0x93, 0x56, 0x95, 0x9d, 0x33, 0xa3, 0xf3, 0x7b, + 0x79, 0x44, 0x0a, 0x82, 0xbe, 0xd3, 0x7e, 0xb4, 0xaa, 0x1e, 0xd1, 0x9f, 0x42, 0x3d, 0x96, 0x1f, + 0xa3, 0xb4, 0x32, 0x7a, 0x0a, 0x85, 0x33, 0x74, 0x78, 0x53, 0x88, 0x93, 0xbd, 0x35, 0xd3, 0xf5, + 0x30, 0xde, 0x1f, 0xbd, 0x66, 0xf6, 0xbb, 0x16, 0x6c, 0xcd, 0xe4, 0x3e, 0xc5, 0x5f, 0x17, 0x60, + 0x89, 0xd5, 0x8d, 0xee, 0xe8, 0x9d, 0xc8, 0xde, 0xbe, 0x68, 0xe0, 0xcc, 0xc3, 0x03, 0xfa, 0x80, + 0x0f, 0x87, 0x55, 0xf7, 0x9c, 0x7d, 0xde, 0xda, 0x84, 0xf4, 0x8e, 0x32, 0xaa, 0x5c, 0x79, 0x9a, + 0xbf, 0x16, 0xf5, 0x0e, 0x79, 0x16, 0x8a, 0xc5, 0xf5, 0x54, 0xfa, 0x4a, 0xb1, 0xb4, 0x9e, 0x4a, + 0x5f, 0x2d, 0x2e, 0xac, 0xa7, 0xd2, 0x6f, 0x15, 0xdf, 0x16, 0xff, 0xfc, 0x2a, 0xe4, 0x5c, 0xe4, + 0xc7, 0x10, 0xd1, 0xc3, 0x20, 0x22, 0xba, 0x36, 0x0c, 0x11, 0x71, 0xac, 0xc8, 0x21, 0xd1, 0xc3, + 0x20, 0x24, 0xba, 0x36, 0x0c, 0x12, 0xf9, 0x3c, 0x04, 0x13, 0x35, 0x87, 0x61, 0xa2, 0x7b, 0x63, + 0x60, 0x22, 0x4f, 0x54, 0x3f, 0x28, 0x5a, 0x1b, 0x04, 0x45, 0x37, 0x47, 0x83, 0x22, 0x4f, 0x54, + 0x00, 0x15, 0x3d, 0xee, 0x43, 0x45, 0xd7, 0x47, 0xa0, 0x22, 0x8f, 0xdf, 0x85, 0x45, 0x1b, 0x91, + 0xb0, 0xe8, 0xf6, 0x79, 0xb0, 0xc8, 0x93, 0x13, 0xc2, 0x45, 0x1f, 0x84, 0x70, 0xd1, 0xe2, 0x50, + 0x5c, 0xe4, 0x71, 0x33, 0x60, 0xf4, 0x49, 0x3f, 0x30, 0xba, 0x3e, 0x02, 0x18, 0xf9, 0x2d, 0xe0, + 0xc8, 0xa8, 0x16, 0x85, 0x8c, 0x6e, 0x9d, 0x83, 0x8c, 0x3c, 0x29, 0x41, 0x68, 0x54, 0x8b, 0x82, + 0x46, 0xb7, 0xce, 0x81, 0x46, 0x7d, 0x92, 0x18, 0x36, 0xda, 0x8a, 0xc6, 0x46, 0x77, 0xce, 0xc5, + 0x46, 0x9e, 0xb4, 0x30, 0x38, 0x5a, 0x09, 0x80, 0xa3, 0x77, 0x86, 0x80, 0x23, 0x8f, 0x95, 0xa0, + 0xa3, 0x6f, 0x0d, 0xa0, 0x23, 0x71, 0x14, 0x3a, 0xf2, 0x78, 0x3d, 0x78, 0xf4, 0x62, 0x08, 0x3c, + 0xba, 0x7b, 0x3e, 0x3c, 0xf2, 0x84, 0xf5, 0xe1, 0x23, 0x65, 0x24, 0x3e, 0x7a, 0x6f, 0x4c, 0x7c, + 0xe4, 0x49, 0x8f, 0x02, 0x48, 0x1f, 0x85, 0x01, 0xd2, 0xd2, 0x70, 0x80, 0xe4, 0x89, 0xe1, 0x08, + 0x69, 0x23, 0x12, 0x21, 0xdd, 0x3e, 0x0f, 0x21, 0xf9, 0xf3, 0x20, 0x08, 0x91, 0xb6, 0xa2, 0x21, + 0xd2, 0x9d, 0x73, 0x21, 0x92, 0xdf, 0xfd, 0x21, 0x8c, 0xb4, 0x11, 0x89, 0x91, 0x6e, 0x9f, 0x87, + 0x91, 0xfc, 0xca, 0x05, 0x41, 0xd2, 0xab, 0xa1, 0x20, 0xe9, 0xfe, 0x38, 0x20, 0xc9, 0x13, 0x3a, + 0x80, 0x92, 0x3e, 0x1f, 0x8e, 0x92, 0xbe, 0x71, 0x81, 0x94, 0x88, 0x91, 0x30, 0xe9, 0x5b, 0x03, + 0x30, 0x49, 0x1c, 0x05, 0x93, 0xfc, 0xf1, 0xec, 0xe2, 0x24, 0x65, 0x24, 0xaa, 0x79, 0x6f, 0x4c, + 0x54, 0xe3, 0x0f, 0xbe, 0x08, 0x58, 0x53, 0x8d, 0x80, 0x35, 0x37, 0x47, 0xc3, 0x1a, 0xdf, 0x9c, + 0xfb, 0xb8, 0xa6, 0x16, 0x85, 0x6b, 0x6e, 0x9d, 0x83, 0x6b, 0x7c, 0x2b, 0x14, 0x00, 0x36, 0x8f, + 0xfb, 0x80, 0xcd, 0xf5, 0x73, 0xa3, 0x7e, 0x02, 0xc8, 0xa6, 0x3c, 0x88, 0x6c, 0x6e, 0x8c, 0x44, + 0x36, 0x9e, 0x04, 0x1f, 0xda, 0x3c, 0xee, 0x83, 0x36, 0xd7, 0x47, 0x40, 0x1b, 0xbf, 0x02, 0x1c, + 0xdb, 0x68, 0xa3, 0xb1, 0xcd, 0xf2, 0xb8, 0xd8, 0xc6, 0x13, 0x1c, 0x09, 0x6e, 0xb6, 0xa2, 0xc1, + 0xcd, 0x9d, 0x31, 0x5f, 0xc8, 0x0f, 0xa0, 0x9b, 0x5a, 0x14, 0xba, 0xb9, 0x75, 0x0e, 0xba, 0x09, + 0xae, 0x21, 0x1e, 0xbc, 0xa9, 0x45, 0xc1, 0x9b, 0x5b, 0xe7, 0xc0, 0x1b, 0x5f, 0x52, 0x00, 0xdf, + 0x34, 0x87, 0xe1, 0x9b, 0x7b, 0x63, 0xe0, 0x1b, 0xdf, 0x79, 0xe9, 0x03, 0x38, 0x9f, 0xf6, 0x03, + 0x1c, 0x71, 0x14, 0xc0, 0xf1, 0x67, 0xa4, 0x8b, 0x70, 0xb6, 0xa2, 0x11, 0xce, 0x9d, 0x73, 0x11, + 0x4e, 0xd0, 0x48, 0x06, 0x20, 0xce, 0x46, 0x24, 0xc4, 0xb9, 0x7d, 0x1e, 0xc4, 0xf1, 0x8d, 0x64, + 0x10, 0xe3, 0x7c, 0xda, 0x8f, 0x71, 0xc4, 0x51, 0x18, 0xc7, 0x6f, 0x9c, 0x0b, 0x72, 0x6a, 0x51, + 0x20, 0xe7, 0xd6, 0x39, 0x20, 0xc7, 0xef, 0xbc, 0x00, 0xca, 0x51, 0x46, 0xa2, 0x9c, 0xf7, 0xc6, + 0x44, 0x39, 0x7d, 0x86, 0x2b, 0x0c, 0x73, 0x6a, 0x51, 0x30, 0xe7, 0xd6, 0x39, 0x30, 0x27, 0x50, + 0x59, 0x1f, 0xe7, 0x6c, 0x45, 0xe3, 0x9c, 0x3b, 0xe7, 0xe2, 0x9c, 0xbe, 0xd9, 0xe4, 0x02, 0x9d, + 0x8d, 0x48, 0xa0, 0x73, 0xfb, 0x3c, 0xa0, 0xd3, 0xb7, 0xf0, 0x71, 0xe7, 0xe0, 0x57, 0xc6, 0x47, + 0x3a, 0x1f, 0x5d, 0x1c, 0xe9, 0x78, 0xcf, 0x8c, 0x05, 0xea, 0xac, 0xa7, 0xd2, 0x6f, 0x17, 0xdf, + 0x11, 0xff, 0xd6, 0x34, 0x4c, 0xd5, 0xbc, 0x38, 0x17, 0xbf, 0x96, 0xc2, 0x9b, 0xe4, 0x45, 0x42, + 0x6b, 0x64, 0xc6, 0x52, 0xbb, 0x77, 0x7e, 0x36, 0xbd, 0xc1, 0x7c, 0x6f, 0x9c, 0xf5, 0x0d, 0x0e, + 0x28, 0xa3, 0x0f, 0x20, 0xd7, 0xb3, 0xb1, 0x25, 0x77, 0x2d, 0xdd, 0xb4, 0x74, 0x87, 0x9d, 0xe2, + 0x10, 0xca, 0xc5, 0x2f, 0x4f, 0x17, 0x67, 0x76, 0x6c, 0x6c, 0x6d, 0x73, 0xba, 0x34, 0xd3, 0x0b, + 0x5c, 0xb9, 0xdf, 0xa2, 0x9a, 0x1c, 0xff, 0x5b, 0x54, 0x2f, 0xa0, 0x68, 0x61, 0x45, 0x0b, 0x79, + 0x20, 0x2c, 0x07, 0x51, 0xf4, 0x98, 0xa1, 0xa7, 0xac, 0xdc, 0x92, 0x34, 0x17, 0x51, 0xc1, 0x0a, + 0x13, 0xd1, 0x03, 0xb8, 0xd4, 0x51, 0x8e, 0x69, 0x4c, 0xa4, 0xec, 0x3a, 0x75, 0x34, 0xce, 0x91, + 0x7d, 0xe6, 0x09, 0x75, 0x94, 0x63, 0xfa, 0x61, 0x2b, 0x76, 0x8b, 0x7e, 0xad, 0xe2, 0x16, 0xe4, + 0x35, 0xdd, 0x76, 0x74, 0x43, 0x75, 0x78, 0x7a, 0x5b, 0x96, 0x1a, 0x36, 0xe7, 0x52, 0x59, 0x0e, + 0xdb, 0xfb, 0x30, 0xcb, 0x83, 0xe5, 0x03, 0xaf, 0x08, 0x79, 0x8a, 0x58, 0x76, 0xc3, 0x7b, 0x2b, + 0x88, 0x2a, 0x50, 0x68, 0x29, 0x0e, 0x3e, 0x52, 0x4e, 0x64, 0xf7, 0x28, 0x56, 0x96, 0x66, 0x7d, + 0x7c, 0xeb, 0xec, 0x74, 0x31, 0xf7, 0x94, 0xdd, 0x1a, 0x38, 0x91, 0x95, 0x6b, 0x05, 0x6e, 0x68, + 0xe8, 0x0e, 0x14, 0x14, 0xfb, 0xc4, 0x50, 0xa9, 0x7a, 0xb0, 0x61, 0xf7, 0x6c, 0x0a, 0x29, 0xd2, + 0x52, 0x9e, 0x92, 0x2b, 0x2e, 0x15, 0x5d, 0x87, 0x19, 0x1e, 0x49, 0xce, 0xbe, 0x94, 0x53, 0xa0, + 0x4d, 0xe5, 0x9f, 0x66, 0xa0, 0x1f, 0xcb, 0x41, 0x8f, 0x61, 0x81, 0xa7, 0xaf, 0x3f, 0x52, 0x2c, + 0x4d, 0xa6, 0x5a, 0xf7, 0xc7, 0x67, 0x91, 0x8a, 0xbd, 0xc2, 0xd2, 0xd5, 0x93, 0x02, 0x44, 0xd5, + 0x7e, 0x26, 0x84, 0x2d, 0x98, 0x55, 0xdb, 0xba, 0x87, 0x00, 0x58, 0xcb, 0x67, 0x87, 0xda, 0xd9, + 0x0a, 0x2d, 0xeb, 0xbf, 0x22, 0x2d, 0xa8, 0x61, 0x02, 0x6a, 0x00, 0xcd, 0xf6, 0x22, 0x77, 0xcd, + 0xb6, 0xae, 0x9e, 0x50, 0xe7, 0x3f, 0x9c, 0x86, 0x7b, 0x64, 0x32, 0xfc, 0x57, 0x8a, 0xee, 0x6c, + 0x53, 0x4e, 0x09, 0x8e, 0xbc, 0xdf, 0x2c, 0x85, 0xee, 0x7a, 0x2a, 0x3d, 0x53, 0xcc, 0xad, 0xa7, + 0xd2, 0xf9, 0x62, 0x41, 0xfc, 0xdb, 0x02, 0x14, 0xfa, 0xea, 0x82, 0x6a, 0x70, 0x49, 0xf3, 0xa6, + 0x8a, 0xcc, 0x8f, 0x1a, 0xe9, 0xa6, 0xc1, 0x33, 0x8b, 0xcf, 0x7d, 0x79, 0xba, 0x58, 0xa0, 0xa5, + 0x9f, 0x7a, 0xb7, 0xa4, 0x79, 0x9f, 0xc3, 0xa7, 0xa2, 0x8f, 0x20, 0xcf, 0xdc, 0x47, 0xef, 0xe3, + 0x6f, 0x34, 0x46, 0xbc, 0x3c, 0xfb, 0xe5, 0xe9, 0x62, 0x8e, 0xfa, 0x8c, 0x6e, 0xe6, 0x5f, 0x29, + 0xd7, 0x0e, 0x5e, 0x8a, 0xbf, 0x2e, 0xc0, 0x4c, 0xe8, 0x38, 0xcf, 0xe3, 0xbe, 0x37, 0xe8, 0x57, + 0xa3, 0x71, 0xe7, 0xb0, 0x80, 0xba, 0x34, 0x1f, 0xe7, 0x6e, 0x74, 0xe2, 0xe2, 0x70, 0xdc, 0x42, + 0x77, 0x61, 0xdc, 0x50, 0x0d, 0x97, 0xed, 0xe3, 0xd4, 0x0f, 0x7e, 0xb8, 0x38, 0x21, 0xfe, 0x5e, + 0x0a, 0x72, 0xe1, 0xc3, 0x3b, 0xf5, 0xbe, 0x7a, 0x45, 0xad, 0x0b, 0x21, 0x8e, 0xe5, 0x11, 0x5f, + 0x0a, 0xc9, 0xf8, 0x29, 0xfc, 0x59, 0x35, 0x97, 0x46, 0xc4, 0x09, 0x04, 0xeb, 0xe9, 0x33, 0x2e, + 0x7c, 0x3f, 0xe9, 0xd9, 0xd7, 0x65, 0x98, 0xa4, 0x69, 0x74, 0x78, 0xd5, 0x4a, 0xfd, 0xa3, 0x87, + 0xf8, 0xca, 0xe4, 0xbe, 0xc4, 0x8a, 0x11, 0x7b, 0xdc, 0x7c, 0xa3, 0x3c, 0x75, 0xfe, 0x34, 0xb8, + 0xf8, 0xb7, 0xf6, 0x78, 0x9e, 0xc2, 0xc9, 0x8b, 0xe5, 0x29, 0x44, 0xbf, 0x04, 0x05, 0xd5, 0x6c, + 0xb7, 0xd9, 0x5a, 0xc7, 0x2c, 0xd2, 0x60, 0xe6, 0x11, 0x2a, 0x82, 0x7f, 0x1e, 0x71, 0xd9, 0xfb, + 0x4c, 0xe2, 0xb2, 0xc4, 0x3f, 0x93, 0x18, 0x88, 0xf5, 0xcc, 0x7b, 0xc2, 0x98, 0x21, 0xeb, 0x0b, + 0x3b, 0x9d, 0x7e, 0x93, 0xb0, 0x53, 0x16, 0xaa, 0xcc, 0x47, 0xce, 0x1f, 0x0b, 0x3c, 0x30, 0xe4, + 0x99, 0x69, 0x1e, 0xf4, 0xbc, 0x70, 0xd1, 0x85, 0x60, 0xd6, 0xc0, 0xf4, 0x97, 0xa7, 0x8b, 0x29, + 0xc9, 0x4b, 0x1b, 0x18, 0x65, 0xf9, 0x13, 0x3f, 0x9d, 0xe5, 0xbf, 0x0e, 0x33, 0x5d, 0x0b, 0xef, + 0x61, 0x47, 0xdd, 0x97, 0x8d, 0x5e, 0x87, 0x9f, 0x2a, 0xc9, 0xba, 0xb4, 0xad, 0x5e, 0x07, 0xdd, + 0x83, 0xa2, 0x57, 0x84, 0x63, 0x6c, 0x37, 0xad, 0x94, 0x4b, 0xe7, 0x88, 0x5c, 0xfc, 0x3f, 0x02, + 0xcc, 0x85, 0xda, 0xc4, 0xe7, 0xc4, 0x3a, 0x64, 0x7d, 0x73, 0x60, 0x97, 0x84, 0x0b, 0x06, 0x4f, + 0x06, 0x99, 0x91, 0x0c, 0x97, 0xdd, 0xc7, 0xd2, 0xb4, 0xf5, 0xbe, 0xd8, 0xc4, 0x05, 0xc5, 0x5e, + 0xf2, 0xe5, 0xac, 0x05, 0x1e, 0xe0, 0x4d, 0x92, 0xe4, 0x58, 0x93, 0x44, 0xfc, 0x2d, 0x01, 0x8a, + 0xf4, 0x01, 0x4f, 0x30, 0xd6, 0x62, 0xb1, 0x4e, 0x6e, 0x50, 0x72, 0x62, 0xfc, 0x73, 0x23, 0xa1, + 0x4f, 0x6d, 0x24, 0xc3, 0x9f, 0xda, 0x10, 0x7f, 0x28, 0x40, 0xde, 0xab, 0x21, 0xfb, 0xe0, 0xdc, + 0x88, 0xe4, 0x94, 0x6f, 0xf6, 0xb1, 0x35, 0x37, 0x0b, 0xc7, 0x58, 0xdf, 0xc0, 0x0b, 0x66, 0xe1, + 0x60, 0x1f, 0x09, 0xfb, 0xbb, 0xee, 0xc8, 0x21, 0x55, 0xac, 0xf8, 0xd9, 0x16, 0xde, 0xe0, 0x08, + 0x8d, 0x44, 0xbf, 0xd5, 0x69, 0xb6, 0x0f, 0x59, 0xe2, 0x93, 0xb1, 0xcc, 0x16, 0xe2, 0xe1, 0x50, + 0xc0, 0x77, 0xe3, 0xb4, 0x66, 0x83, 0x7e, 0xc5, 0x93, 0xfd, 0xb6, 0xc5, 0x27, 0x01, 0x05, 0xd2, + 0xce, 0x27, 0x5a, 0x1a, 0xcb, 0x94, 0xba, 0x5a, 0x62, 0x63, 0xe5, 0x8f, 0x82, 0x3d, 0x51, 0x3d, + 0x24, 0x28, 0xec, 0x11, 0x24, 0x0f, 0x95, 0xf6, 0xa8, 0x30, 0xb0, 0x50, 0xcf, 0x49, 0xa4, 0x34, + 0x7a, 0x12, 0x4a, 0x52, 0x91, 0x18, 0x8e, 0x18, 0x06, 0x55, 0x1a, 0x4a, 0x66, 0xf1, 0x61, 0x78, + 0xac, 0x8f, 0x7c, 0x7c, 0x70, 0xd0, 0x7f, 0x9c, 0xfa, 0xd1, 0x0f, 0x17, 0x05, 0xf1, 0x13, 0x40, + 0x12, 0xb6, 0xb1, 0xf3, 0xa2, 0x67, 0x5a, 0x7e, 0xc2, 0x8f, 0xdb, 0x7d, 0xdf, 0x1e, 0x99, 0x2c, + 0x67, 0xcf, 0xa2, 0x52, 0xfa, 0x5c, 0x82, 0xb9, 0x10, 0x37, 0x33, 0x16, 0xe2, 0x87, 0x70, 0xf5, + 0xa9, 0x69, 0xdb, 0x7a, 0x97, 0x40, 0x4f, 0x3a, 0x2b, 0xc9, 0xd2, 0xe0, 0x99, 0xc7, 0x74, 0x97, + 0x6e, 0x42, 0x18, 0xcc, 0x8c, 0x64, 0x24, 0xef, 0x5a, 0xfc, 0x7d, 0x01, 0xae, 0x0c, 0x72, 0x32, + 0x2d, 0x47, 0x9d, 0xf8, 0x9b, 0x56, 0x4d, 0x3f, 0xbf, 0xdb, 0xf9, 0xa3, 0xd5, 0x2d, 0x4e, 0x5c, + 0x4c, 0xfe, 0x4c, 0xb9, 0xa3, 0x50, 0xf3, 0xc1, 0x4f, 0x1f, 0xe7, 0x39, 0x79, 0x93, 0x51, 0x7d, + 0x4b, 0x92, 0x1a, 0xcf, 0x92, 0x34, 0xa1, 0xb0, 0x6e, 0xea, 0x06, 0xf1, 0x64, 0xdd, 0xf6, 0xae, + 0x42, 0x7e, 0x57, 0x37, 0x14, 0xeb, 0x44, 0x76, 0x0f, 0x75, 0xb3, 0x81, 0xb2, 0x10, 0x55, 0x59, + 0x56, 0x42, 0xca, 0x31, 0x0e, 0x7e, 0x29, 0xfe, 0x58, 0x80, 0xa2, 0x2f, 0x96, 0x5b, 0xe4, 0x77, + 0x01, 0xd4, 0x76, 0xcf, 0x76, 0xb0, 0xe5, 0xf6, 0xd2, 0x0c, 0x8b, 0xde, 0xae, 0x30, 0x6a, 0x7d, + 0x4d, 0xca, 0xf0, 0x02, 0x75, 0x0d, 0xdd, 0x08, 0x27, 0x47, 0x98, 0x2c, 0xc3, 0xd9, 0x40, 0x4a, + 0x04, 0xd2, 0xed, 0xb6, 0x63, 0x5a, 0x1e, 0x66, 0xe2, 0xdd, 0xee, 0x66, 0x6b, 0xa7, 0x67, 0x7a, + 0x49, 0xb9, 0x55, 0xc8, 0x93, 0xe5, 0xfe, 0x10, 0x7b, 0x4d, 0x4a, 0x9d, 0xdf, 0x24, 0xc6, 0xe1, + 0x36, 0xe9, 0x1f, 0x11, 0x27, 0x95, 0xf5, 0x86, 0xd7, 0xc3, 0x23, 0x2c, 0xda, 0xcf, 0x04, 0x53, + 0x12, 0x8e, 0x97, 0xb4, 0x91, 0x7a, 0x22, 0xdf, 0x82, 0xb4, 0xfb, 0x15, 0x64, 0x3e, 0x41, 0xae, + 0x2e, 0xb3, 0xcf, 0x24, 0x2f, 0xbb, 0x9f, 0x49, 0x5e, 0x5e, 0xe3, 0x05, 0x98, 0x19, 0xff, 0xc1, + 0x7f, 0x5f, 0x14, 0x24, 0x8f, 0xe9, 0x7e, 0x83, 0x8c, 0xf0, 0x81, 0x55, 0x18, 0xe5, 0x01, 0x02, + 0x9f, 0xa9, 0xe1, 0x1f, 0xc6, 0x5d, 0x5d, 0x93, 0x77, 0xb6, 0x2a, 0xcf, 0x37, 0x37, 0xeb, 0xcd, + 0x66, 0x75, 0xad, 0x28, 0xa0, 0x22, 0xcc, 0x84, 0x3e, 0x72, 0x93, 0x60, 0x9f, 0xca, 0xbd, 0xff, + 0x33, 0x00, 0xfe, 0xf7, 0xb2, 0x88, 0xac, 0x8d, 0xea, 0x67, 0xf2, 0xcb, 0xd5, 0x67, 0x3b, 0xd5, + 0x46, 0x71, 0x02, 0x21, 0xc8, 0x97, 0x57, 0x9b, 0x95, 0x9a, 0x2c, 0x55, 0x1b, 0xdb, 0xcf, 0xb7, + 0x1a, 0x55, 0xf7, 0x13, 0xbb, 0xf7, 0xd7, 0x60, 0x26, 0x98, 0xee, 0x06, 0xcd, 0x41, 0xa1, 0x52, + 0xab, 0x56, 0x36, 0xe4, 0x97, 0xf5, 0x55, 0xf9, 0xc5, 0x4e, 0x75, 0xa7, 0x5a, 0x9c, 0xa0, 0x55, + 0xa3, 0xc4, 0x27, 0x3b, 0xcf, 0x9e, 0x15, 0x05, 0x54, 0x80, 0x2c, 0xbb, 0xa6, 0x1f, 0xc4, 0x29, + 0x26, 0xee, 0x6f, 0x42, 0x36, 0x90, 0xd6, 0x96, 0x3c, 0x6e, 0x7b, 0xa7, 0x51, 0x93, 0x9b, 0xf5, + 0xcd, 0x6a, 0xa3, 0xb9, 0xba, 0xb9, 0xcd, 0x64, 0x50, 0xda, 0x6a, 0xf9, 0xb9, 0xd4, 0x2c, 0x0a, + 0xde, 0x75, 0xf3, 0xf9, 0x4e, 0xa5, 0xe6, 0x36, 0x43, 0x4c, 0xa5, 0x93, 0xc5, 0xe4, 0xfd, 0xbf, + 0x26, 0xc0, 0x95, 0x21, 0xa9, 0x5f, 0x50, 0x16, 0xa6, 0x77, 0x0c, 0x9a, 0xe3, 0xb3, 0x38, 0x81, + 0x72, 0x81, 0xec, 0x2f, 0x45, 0x01, 0xa5, 0x59, 0xfe, 0x8d, 0x62, 0x02, 0x4d, 0x41, 0xa2, 0xf1, + 0xa8, 0x98, 0x24, 0x35, 0x0d, 0x24, 0x4f, 0x29, 0xa6, 0x50, 0x86, 0xa7, 0x6d, 0x28, 0x4e, 0xa2, + 0x19, 0x3f, 0x7b, 0x42, 0x71, 0x8a, 0x88, 0xf2, 0xb2, 0x10, 0x14, 0xa7, 0xef, 0x5f, 0x87, 0xc0, + 0x49, 0x6f, 0x04, 0x30, 0xf5, 0x4c, 0x71, 0xb0, 0xed, 0x14, 0x27, 0xd0, 0x34, 0x24, 0x57, 0xdb, + 0xed, 0xa2, 0xf0, 0xf0, 0xdf, 0xa4, 0x20, 0xed, 0x7e, 0xf8, 0x05, 0x3d, 0x83, 0x49, 0xb6, 0xf5, + 0xbc, 0x38, 0xdc, 0xb3, 0xa7, 0x93, 0x77, 0x61, 0xe9, 0x3c, 0xd7, 0x5f, 0x9c, 0x40, 0x7f, 0x05, + 0xb2, 0x01, 0x8f, 0x09, 0x0d, 0xdd, 0x3e, 0x0b, 0x79, 0x89, 0x0b, 0xb7, 0xcf, 0x2b, 0xe6, 0xc9, + 0x7f, 0x05, 0x19, 0xcf, 0x82, 0xa3, 0x1b, 0xa3, 0xec, 0xbb, 0x2b, 0x7b, 0xf4, 0x22, 0x40, 0xe6, + 0x9a, 0x38, 0xf1, 0xbe, 0x80, 0x2c, 0x40, 0x83, 0xc6, 0x16, 0x45, 0x45, 0x24, 0x0c, 0xb5, 0xe6, + 0x0b, 0xf7, 0xc7, 0x2a, 0xed, 0x3f, 0x93, 0x28, 0xcb, 0x5f, 0x31, 0xa2, 0x95, 0x35, 0xb0, 0x1e, + 0x45, 0x2b, 0x2b, 0x62, 0xe1, 0x99, 0x40, 0x2f, 0x20, 0x45, 0x2c, 0x25, 0x8a, 0xf2, 0x21, 0xfb, + 0x2c, 0xf3, 0xc2, 0x8d, 0x91, 0x65, 0x5c, 0x91, 0xe5, 0x7b, 0x3f, 0xfa, 0xb3, 0x6b, 0x13, 0x3f, + 0x3a, 0xbb, 0x26, 0xfc, 0xf8, 0xec, 0x9a, 0xf0, 0x27, 0x67, 0xd7, 0x84, 0x3f, 0x3d, 0xbb, 0x26, + 0xfc, 0xda, 0x4f, 0xae, 0x4d, 0xfc, 0xf8, 0x27, 0xd7, 0x26, 0xfe, 0xe4, 0x27, 0xd7, 0x26, 0x3e, + 0x9f, 0xe6, 0xdc, 0xbb, 0x53, 0xd4, 0xa8, 0x3c, 0xfa, 0xff, 0x01, 0x00, 0x00, 0xff, 0xff, 0x1a, + 0xc8, 0x90, 0x6a, 0xb6, 0x7e, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index ab86df7a67b8..a65f94909bb4 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -814,7 +814,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 28c0101c6b30..e229cde49871 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -96,6 +96,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: @@ -113,12 +122,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. @@ -126,11 +138,17 @@ 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 +// Learners returns a ReplicaSet containing the 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. // @@ -212,11 +230,17 @@ func (d ReplicaSet) VoterDescriptors() []ReplicaDescriptor { // However, it means a slow learner can slow down regular traffic. // // For some related mega-comments, see Replica.sendSnapshot. +func (d ReplicaSet) Learners() ReplicaSet { + return d.Filter(predLearner) +} + +// LearnerDescriptors returns a slice of ReplicaDescriptors corresponding to +// learner replicas in `d`. 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 @@ -239,12 +263,38 @@ 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. +// VotersAndNonVoters returns a ReplicaSet of VOTER_FULL/NON_VOTER replicas in +// `d`. 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) VotersAndNonVoters() ReplicaSet { + return d.Filter(predVoterOrNonVoter) +} + +// VoterAndNonVoterDescriptors returns the descriptors of VOTER_FULL/NON_VOTER +// replicas in the set. +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 { @@ -283,6 +333,16 @@ func (d ReplicaSet) DeepCopy() ReplicaSet { } } +// Contains returns true if the set contains rDesc. +func (d ReplicaSet) Contains(rDesc ReplicaDescriptor) bool { + for _, repl := range d.Descriptors() { + 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) @@ -402,6 +462,15 @@ 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) { + for _, r := range d.Descriptors() { + out = append(out, ReplicationTarget{NodeID: r.NodeID, StoreID: r.StoreID}) + } + return out +} + // IsAddition returns true if `c` refers to a replica addition operation. func (c ReplicaChangeType) IsAddition() bool { switch c { diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index 8d3738d0e515..4b964df525b0 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -308,7 +308,7 @@ func TestReplicaDescriptorsCanMakeProgress(t *testing.T) { rds = append(rds, rDesc.ReplicaDescriptor) } - act := MakeReplicaDescriptors(rds).CanMakeProgress(func(rd ReplicaDescriptor) bool { + act := MakeReplicaSet(rds).CanMakeProgress(func(rd ReplicaDescriptor) bool { for _, rdi := range test.rds { if rdi.ReplicaID == rd.ReplicaID { return rdi.live @@ -346,7 +346,7 @@ func TestReplicaDescriptorsCanMakeProgressRandom(t *testing.T) { liveness[i] = (livenessBits >> i & 1) == 0 } - rng := MakeReplicaDescriptors(rds) + rng := MakeReplicaSet(rds) crdbCanMakeProgress := rng.CanMakeProgress(func(rd ReplicaDescriptor) bool { return liveness[rd.ReplicaID-1] diff --git a/pkg/sql/relocate.go b/pkg/sql/relocate.go index 1a0cb88d2a90..ade2bc3545e3 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, + ); err != nil { return false, err } }