diff --git a/pkg/cmd/roachtest/clearrange.go b/pkg/cmd/roachtest/clearrange.go index e557cdd17b08..fc2fb09150e4 100644 --- a/pkg/cmd/roachtest/clearrange.go +++ b/pkg/cmd/roachtest/clearrange.go @@ -73,7 +73,7 @@ func runClearRange(ctx context.Context, t *test, c *cluster, aggressiveChecks bo if t.buildVersion.AtLeast(version.MustParse("v19.2.0")) { conn := c.Conn(ctx, 1) - if _, err := conn.ExecContext(ctx, `SET CLUSTER SETTING kv.bulk_io_write.concurrent_addsstable_requests = $1`, c.spec.NodeCount); err != nil { + if _, err := conn.ExecContext(ctx, `SET CLUSTER SETTING kv.bulk_io_write.concurrent_addsstable_requests = 8`); err != nil { t.Fatal(err) } conn.Close() @@ -114,6 +114,11 @@ func runClearRange(ctx context.Context, t *test, c *cluster, aggressiveChecks bo }() m := newMonitor(ctx, c) + m.Go(func(ctx context.Context) error { + c.Run(ctx, c.Node(1), `./cockroach workload init kv`) + c.Run(ctx, c.All(), `./cockroach workload run kv --concurrency=32 --duration=1h`) + return nil + }) m.Go(func(ctx context.Context) error { conn := c.Conn(ctx, 1) defer conn.Close() @@ -132,7 +137,7 @@ func runClearRange(ctx context.Context, t *test, c *cluster, aggressiveChecks bo // Set a low TTL so that the ClearRange-based cleanup mechanism can kick in earlier. // This could also be done after dropping the table. - if _, err := conn.ExecContext(ctx, `ALTER TABLE bigbank.bank CONFIGURE ZONE USING gc.ttlseconds = 30`); err != nil { + if _, err := conn.ExecContext(ctx, `ALTER TABLE bigbank.bank CONFIGURE ZONE USING gc.ttlseconds = 1200`); err != nil { return err } diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 2fdfc64790b4..f748fb290cd7 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -325,6 +325,7 @@ go_test( "//pkg/kv/kvserver/rditer", "//pkg/kv/kvserver/readsummary/rspb", "//pkg/kv/kvserver/spanset", + "//pkg/kv/kvserver/split", "//pkg/kv/kvserver/stateloader", "//pkg/kv/kvserver/tenantrate", "//pkg/kv/kvserver/tscache", diff --git a/pkg/kv/kvserver/batcheval/cmd_range_stats.go b/pkg/kv/kvserver/batcheval/cmd_range_stats.go index 968be1800cf5..d369aafbd776 100644 --- a/pkg/kv/kvserver/batcheval/cmd_range_stats.go +++ b/pkg/kv/kvserver/batcheval/cmd_range_stats.go @@ -42,7 +42,14 @@ func RangeStats( ) (result.Result, error) { reply := resp.(*roachpb.RangeStatsResponse) reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats() - reply.QueriesPerSecond = cArgs.EvalCtx.GetSplitQPS() + reply.DeprecatedLastQueriesPerSecond = cArgs.EvalCtx.GetLastSplitQPS() + if qps, ok := cArgs.EvalCtx.GetMaxSplitQPS(); ok { + reply.MaxQueriesPerSecond = qps + } else { + // See comment on MaxQueriesPerSecond. -1 means !ok. + reply.MaxQueriesPerSecond = -1 + } + reply.MaxQueriesPerSecondSet = true reply.RangeInfo = cArgs.EvalCtx.GetRangeInfo(ctx) return result.Result{}, nil } diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index 2f9c38e2c991..87dd73bcef97 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -86,11 +86,20 @@ type EvalContext interface { // results due to concurrent writes. GetMVCCStats() enginepb.MVCCStats - // GetSplitQPS returns the queries/s request rate for this range. + // GetMaxSplitQPS returns the Replicas maximum queries/s request rate over a + // configured retention period. // - // NOTE: This should not be used when the load based splitting cluster - // setting is disabled. - GetSplitQPS() float64 + // NOTE: This should not be used when the load based splitting cluster setting + // is disabled. + GetMaxSplitQPS() (float64, bool) + + // GetLastSplitQPS returns the Replica's most recent queries/s request rate. + // + // NOTE: This should not be used when the load based splitting cluster setting + // is disabled. + // + // TODO(nvanbenschoten): remove this method in v22.1. + GetLastSplitQPS() float64 GetGCThreshold() hlc.Timestamp GetLastReplicaGCTimestamp(context.Context) (hlc.Timestamp, error) @@ -215,7 +224,10 @@ func (m *mockEvalCtxImpl) ContainsKey(key roachpb.Key) bool { func (m *mockEvalCtxImpl) GetMVCCStats() enginepb.MVCCStats { return m.Stats } -func (m *mockEvalCtxImpl) GetSplitQPS() float64 { +func (m *mockEvalCtxImpl) GetMaxSplitQPS() (float64, bool) { + return m.QPS, true +} +func (m *mockEvalCtxImpl) GetLastSplitQPS() float64 { return m.QPS } func (m *mockEvalCtxImpl) CanCreateTxnRecord( diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 6e9a24959ac8..f5621ceed369 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -4148,9 +4148,18 @@ func TestMergeQueue(t *testing.T) { // setThresholds simulates a zone config update that updates the ranges' // minimum and maximum sizes. - setZones := func(zone zonepb.ZoneConfig) { - lhs().SetZoneConfig(&zone) - rhs().SetZoneConfig(&zone) + setZones := func(t *testing.T, zone zonepb.ZoneConfig) { + t.Helper() + if l := lhs(); l == nil { + t.Fatal("left-hand side range not found") + } else { + l.SetZoneConfig(&zone) + } + if r := rhs(); r == nil { + t.Fatal("right-hand side range not found") + } else { + r.SetZoneConfig(&zone) + } } reset := func(t *testing.T) { @@ -4161,7 +4170,11 @@ func TestMergeQueue(t *testing.T) { t.Fatal(err) } } - setZones(zoneConfig) + setZones(t, zoneConfig) + // Disable load-based splitting, so that the absence of sufficient QPS + // measurements do not prevent ranges from merging. Certain subtests + // re-enable the functionality. + kvserver.SplitByLoadEnabled.Override(sv, false) store.MustForceMergeScanAndProcess() // drain any merges that might already be queued split(t, rhsStartKey.AsRawKey(), hlc.Timestamp{} /* expirationTime */) } @@ -4192,7 +4205,7 @@ func TestMergeQueue(t *testing.T) { verifyMerged(t, store, lhsStartKey, rhsStartKey) }) - t.Run("combined-threshold", func(t *testing.T) { + t.Run("combined-size-threshold", func(t *testing.T) { reset(t) // The ranges are individually beneath the minimum size threshold, but @@ -4200,13 +4213,13 @@ func TestMergeQueue(t *testing.T) { zone := protoutil.Clone(&zoneConfig).(*zonepb.ZoneConfig) zone.RangeMinBytes = proto.Int64(rhs().GetMVCCStats().Total() + 1) zone.RangeMaxBytes = proto.Int64(lhs().GetMVCCStats().Total() + rhs().GetMVCCStats().Total() - 1) - setZones(*zone) + setZones(t, *zone) store.MustForceMergeScanAndProcess() verifyUnmerged(t, store, lhsStartKey, rhsStartKey) // Once the maximum size threshold is increased, the merge can occur. zone.RangeMaxBytes = proto.Int64(*zone.RangeMaxBytes + 1) - setZones(*zone) + setZones(t, *zone) l := lhs().RangeID r := rhs().RangeID log.Infof(ctx, "Left=%s, Right=%s", l, r) @@ -4229,8 +4242,88 @@ func TestMergeQueue(t *testing.T) { verifyMerged(t, store, lhsStartKey, rhsStartKey) }) - // TODO(jeffreyxiao): Add subtest to consider load when making merging - // decisions. + t.Run("load-based-merging", func(t *testing.T) { + const splitByLoadQPS = 10 + const mergeByLoadQPS = splitByLoadQPS / 2 // see conservativeLoadBasedSplitThreshold + const splitByLoadMergeDelay = 500 * time.Millisecond + + resetForLoadBasedSubtest := func(t *testing.T) { + reset(t) + + // Enable load-based splitting for these subtests, which also instructs + // the mergeQueue to consider load when making range merge decisions. When + // load is a consideration, the mergeQueue is fairly conservative. In an + // effort to avoid thrashing and to avoid overreacting to temporary + // fluctuations in load, the mergeQueue will only consider a merge when + // the combined load across the RHS and LHS ranges is below half the + // threshold required to split a range due to load. Furthermore, to ensure + // that transient drops in load do not trigger range merges, the + // mergeQueue will only consider a merge when it deems the maximum qps + // measurement from both sides to be sufficiently stable and reliable, + // meaning that it was a maximum measurement over some extended period of + // time. + kvserver.SplitByLoadEnabled.Override(sv, true) + kvserver.SplitByLoadQPSThreshold.Override(sv, splitByLoadQPS) + + // Drop the load-based splitting merge delay setting, which also dictates + // the duration that a leaseholder must measure QPS before considering its + // measurements to be reliable enough to base range merging decisions on. + kvserver.SplitByLoadMergeDelay.Override(sv, splitByLoadMergeDelay) + + // Reset both range's load-based splitters, so that QPS measurements do + // not leak over between subtests. Then, bump the manual clock so that + // both range's load-based splitters consider their measurements to be + // reliable. + lhs().LoadBasedSplitter().Reset(tc.Servers[0].Clock().PhysicalTime()) + rhs().LoadBasedSplitter().Reset(tc.Servers[1].Clock().PhysicalTime()) + manualClock.Increment(splitByLoadMergeDelay.Nanoseconds()) + } + + t.Run("unreliable-lhs-qps", func(t *testing.T) { + resetForLoadBasedSubtest(t) + + lhs().LoadBasedSplitter().Reset(tc.Servers[0].Clock().PhysicalTime()) + + clearRange(t, lhsStartKey, rhsEndKey) + store.MustForceMergeScanAndProcess() + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) + }) + + t.Run("unreliable-rhs-qps", func(t *testing.T) { + resetForLoadBasedSubtest(t) + + rhs().LoadBasedSplitter().Reset(tc.Servers[1].Clock().PhysicalTime()) + + clearRange(t, lhsStartKey, rhsEndKey) + store.MustForceMergeScanAndProcess() + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) + }) + + t.Run("combined-qps-above-threshold", func(t *testing.T) { + resetForLoadBasedSubtest(t) + + moreThanHalfQPS := mergeByLoadQPS/2 + 1 + rhs().LoadBasedSplitter().RecordMax(tc.Servers[0].Clock().PhysicalTime(), float64(moreThanHalfQPS)) + lhs().LoadBasedSplitter().RecordMax(tc.Servers[1].Clock().PhysicalTime(), float64(moreThanHalfQPS)) + + clearRange(t, lhsStartKey, rhsEndKey) + store.MustForceMergeScanAndProcess() + verifyUnmerged(t, store, lhsStartKey, rhsStartKey) + }) + + t.Run("combined-qps-below-threshold", func(t *testing.T) { + resetForLoadBasedSubtest(t) + + manualClock.Increment(splitByLoadMergeDelay.Nanoseconds()) + lessThanHalfQPS := mergeByLoadQPS/2 - 1 + rhs().LoadBasedSplitter().RecordMax(tc.Servers[0].Clock().PhysicalTime(), float64(lessThanHalfQPS)) + lhs().LoadBasedSplitter().RecordMax(tc.Servers[1].Clock().PhysicalTime(), float64(lessThanHalfQPS)) + + clearRange(t, lhsStartKey, rhsEndKey) + store.MustForceMergeScanAndProcess() + verifyMerged(t, store, lhsStartKey, rhsStartKey) + }) + }) t.Run("sticky-bit", func(t *testing.T) { reset(t) @@ -4404,6 +4497,15 @@ func TestMergeQueueSeesNonVoters(t *testing.T) { var clusterArgs = base.TestClusterArgs{ // We dont want the replicate queue mucking with our test, so disable it. ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + // Disable load-based splitting, so that the absence of sufficient QPS + // measurements do not prevent ranges from merging. + DisableLoadBasedSplitting: true, + }, + }, + }, } ctx := context.Background() @@ -4491,6 +4593,15 @@ func TestMergeQueueWithSlowNonVoterSnaps(t *testing.T) { var clusterArgs = base.TestClusterArgs{ // We dont want the replicate queue mucking with our test, so disable it. ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + // Disable load-based splitting, so that the absence of sufficient QPS + // measurements do not prevent ranges from merging. + DisableLoadBasedSplitting: true, + }, + }, + }, ServerArgsPerNode: map[int]base.TestServerArgs{ 1: { Knobs: base.TestingKnobs{ @@ -4503,6 +4614,8 @@ func TestMergeQueueWithSlowNonVoterSnaps(t *testing.T) { } return nil }, + // See above. + DisableLoadBasedSplitting: true, }, }, }, diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index f2ca98ac3721..903f8981517c 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/split" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/storage" @@ -416,6 +417,12 @@ func (r *Replica) LargestPreviousMaxRangeSizeBytes() int64 { return r.mu.largestPreviousMaxRangeSizeBytes } +// LoadBasedSplitter returns the replica's split.Decider, which is used to +// assist load-based split (and merge) decisions. +func (r *Replica) LoadBasedSplitter() *split.Decider { + return &r.loadBasedSplitter +} + func MakeSSTable(key, value string, ts hlc.Timestamp) ([]byte, storage.MVCCKeyValue) { sstFile := &storage.MemFile{} sst := storage.MakeIngestionSSTWriter(sstFile) diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 144ca7be5cae..f7196838453b 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -173,7 +173,7 @@ var _ purgatoryError = rangeMergePurgatoryError{} func (mq *mergeQueue) requestRangeStats( ctx context.Context, key roachpb.Key, -) (*roachpb.RangeDescriptor, enginepb.MVCCStats, float64, error) { +) (desc *roachpb.RangeDescriptor, stats enginepb.MVCCStats, qps float64, qpsOK bool, err error) { var ba roachpb.BatchRequest ba.Add(&roachpb.RangeStatsRequest{ @@ -182,10 +182,20 @@ func (mq *mergeQueue) requestRangeStats( br, pErr := mq.db.NonTransactionalSender().Send(ctx, ba) if pErr != nil { - return nil, enginepb.MVCCStats{}, 0, pErr.GoError() + return nil, enginepb.MVCCStats{}, 0, false, pErr.GoError() } res := br.Responses[0].GetInner().(*roachpb.RangeStatsResponse) - return &res.RangeInfo.Desc, res.MVCCStats, res.QueriesPerSecond, nil + + desc = &res.RangeInfo.Desc + stats = res.MVCCStats + if res.MaxQueriesPerSecondSet { + qps = res.MaxQueriesPerSecond + qpsOK = qps >= 0 + } else { + qps = res.DeprecatedLastQueriesPerSecond + qpsOK = true + } + return desc, stats, qps, qpsOK, nil } func (mq *mergeQueue) process( @@ -196,7 +206,9 @@ func (mq *mergeQueue) process( return false, nil } + lhsDesc := lhsRepl.Desc() lhsStats := lhsRepl.GetMVCCStats() + lhsQPS, lhsQPSOK := lhsRepl.GetMaxSplitQPS() minBytes := lhsRepl.GetMinBytes() if lhsStats.Total() >= minBytes { log.VEventf(ctx, 2, "skipping merge: LHS meets minimum size threshold %d with %d bytes", @@ -204,9 +216,7 @@ func (mq *mergeQueue) process( return false, nil } - lhsDesc := lhsRepl.Desc() - lhsQPS := lhsRepl.GetSplitQPS() - rhsDesc, rhsStats, rhsQPS, err := mq.requestRangeStats(ctx, lhsDesc.EndKey.AsRawKey()) + rhsDesc, rhsStats, rhsQPS, rhsQPSOK, err := mq.requestRangeStats(ctx, lhsDesc.EndKey.AsRawKey()) if err != nil { return false, err } @@ -234,6 +244,24 @@ func (mq *mergeQueue) process( var mergedQPS float64 if lhsRepl.SplitByLoadEnabled() { + // When load is a consideration for splits and, by extension, merges, the + // mergeQueue is fairly conservative. In an effort to avoid thrashing and to + // avoid overreacting to temporary fluctuations in load, the mergeQueue will + // only consider a merge when the combined load across the RHS and LHS + // ranges is below half the threshold required to split a range due to load. + // Furthermore, to ensure that transient drops in load do not trigger range + // merges, the mergeQueue will only consider a merge when it deems the + // maximum qps measurement from both sides to be sufficiently stable and + // reliable, meaning that it was a maximum measurement over some extended + // period of time. + if !lhsQPSOK { + log.VEventf(ctx, 2, "skipping merge: LHS QPS measurement not yet reliable") + return false, nil + } + if !rhsQPSOK { + log.VEventf(ctx, 2, "skipping merge: RHS QPS measurement not yet reliable") + return false, nil + } mergedQPS = lhsQPS + rhsQPS } @@ -363,6 +391,13 @@ func (mq *mergeQueue) process( log.Warningf(ctx, "%v", err) } } + + // Adjust the splitter to account for the additional load from the RHS. We + // could just Reset the splitter, but then we'd need to wait out a full + // measurement period (default of 5m) before merging this range again. + if mergedQPS != 0 { + lhsRepl.loadBasedSplitter.RecordMax(mq.store.Clock().PhysicalTime(), mergedQPS) + } return true, nil } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 6b047831c7d7..62b53614f348 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -50,7 +50,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -1035,14 +1034,26 @@ func (r *Replica) GetMVCCStats() enginepb.MVCCStats { return *r.mu.state.Stats } -// GetSplitQPS returns the Replica's queries/s request rate. +// GetMaxSplitQPS returns the Replica's maximum queries/s request rate over a +// configured measurement period. If the Replica has not been recording QPS for +// at least an entire measurement period, the method will return false. // // NOTE: This should only be used for load based splitting, only // works when the load based splitting cluster setting is enabled. // // Use QueriesPerSecond() for current QPS stats for all other purposes. -func (r *Replica) GetSplitQPS() float64 { - return r.loadBasedSplitter.LastQPS(timeutil.Now()) +func (r *Replica) GetMaxSplitQPS() (float64, bool) { + return r.loadBasedSplitter.MaxQPS(r.Clock().PhysicalTime()) +} + +// GetLastSplitQPS returns the Replica's most recent queries/s request rate. +// +// NOTE: This should only be used for load based splitting, only +// works when the load based splitting cluster setting is enabled. +// +// Use QueriesPerSecond() for current QPS stats for all other purposes. +func (r *Replica) GetLastSplitQPS() float64 { + return r.loadBasedSplitter.LastQPS(r.Clock().PhysicalTime()) } // ContainsKey returns whether this range contains the specified key. diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 1c009a24c9c0..0afcb0de2c15 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -149,9 +149,16 @@ func (rec SpanSetReplicaEvalContext) GetMVCCStats() enginepb.MVCCStats { return rec.i.GetMVCCStats() } -// GetSplitQPS returns the Replica's queries/s rate for splitting purposes. -func (rec SpanSetReplicaEvalContext) GetSplitQPS() float64 { - return rec.i.GetSplitQPS() +// GetMaxSplitQPS returns the Replica's maximum queries/s rate for splitting and +// merging purposes. +func (rec SpanSetReplicaEvalContext) GetMaxSplitQPS() (float64, bool) { + return rec.i.GetMaxSplitQPS() +} + +// GetLastSplitQPS returns the Replica's most recent queries/s rate for +// splitting and merging purposes. +func (rec SpanSetReplicaEvalContext) GetLastSplitQPS() float64 { + return rec.i.GetLastSplitQPS() } // CanCreateTxnRecord determines whether a transaction record can be created diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index d029f1b5f33d..14ee0c0000fa 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -94,6 +94,8 @@ func newUnloadedReplica( r.mu.replicaID = replicaID split.Init(&r.loadBasedSplitter, rand.Intn, func() float64 { return float64(SplitByLoadQPSThreshold.Get(&store.cfg.Settings.SV)) + }, func() time.Duration { + return SplitByLoadMergeDelay.Get(&store.cfg.Settings.SV) }) r.mu.proposals = map[kvserverbase.CmdIDKey]*ProposalData{} r.mu.checksums = map[uuid.UUID]ReplicaChecksum{} diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index e6835081cf23..6cc6121ebded 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -1074,6 +1074,9 @@ func TestMergeQueueSeesLearnerOrJointConfig(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() knobs, ltk := makeReplicationTestKnobs() + // Disable load-based splitting, so that the absence of sufficient QPS + // measurements do not prevent ranges from merging. + knobs.Store.(*kvserver.StoreTestingKnobs).DisableLoadBasedSplitting = true tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{Knobs: knobs}, ReplicationMode: base.ReplicationManual, diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index f4a78c888370..3f829285410f 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -426,11 +426,12 @@ func (r *Replica) leasePostApplyLocked( } applyReadSummaryToTimestampCache(r.store.tsCache, r.descRLocked(), sum) - // Reset the request counts used to make lease placement decisions whenever - // starting a new lease. + // Reset the request counts used to make lease placement decisions and + // load-based splitting/merging decisions whenever starting a new lease. if r.leaseholderStats != nil { r.leaseholderStats.resetRequestCounts() } + r.loadBasedSplitter.Reset(r.Clock().PhysicalTime()) } // Inform the concurrency manager that the lease holder has been updated. diff --git a/pkg/kv/kvserver/replica_split_load.go b/pkg/kv/kvserver/replica_split_load.go index 6b48607d5044..e0c0a7c8296b 100644 --- a/pkg/kv/kvserver/replica_split_load.go +++ b/pkg/kv/kvserver/replica_split_load.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" ) // SplitByLoadEnabled wraps "kv.range_split.by_load_enabled". @@ -39,7 +40,13 @@ var SplitByLoadMergeDelay = settings.RegisterDurationSetting( "kv.range_split.by_load_merge_delay", "the delay that range splits created due to load will wait before considering being merged away", 5*time.Minute, - settings.NonNegativeDuration, + func(v time.Duration) error { + const minDelay = 5 * time.Second + if v < minDelay { + return errors.Errorf("cannot be set to a value below %s", minDelay) + } + return nil + }, ) // SplitByLoadQPSThreshold returns the QPS request rate for a given replica. diff --git a/pkg/kv/kvserver/split/decider.go b/pkg/kv/kvserver/split/decider.go index 4c0a68d95669..c29e316d2eee 100644 --- a/pkg/kv/kvserver/split/decider.go +++ b/pkg/kv/kvserver/split/decider.go @@ -21,31 +21,50 @@ import ( ) const minSplitSuggestionInterval = time.Minute +const minQueriesPerSecondSampleDuration = time.Second // A Decider collects measurements about the activity (measured in qps) on a -// Replica and, assuming that qps thresholds are exceeded, tries to determine -// a split key that would approximately result in halving the load on each of -// the resultant ranges. +// Replica and, assuming that qps thresholds are exceeded, tries to determine a +// split key that would approximately result in halving the load on each of the +// resultant ranges. Similarly, these measurements are used to determine when a +// range is serving sufficiently little load, such that it should be allowed to +// merge with its left or right hand neighbor. // // Operations should call `Record` with a current timestamp. Operation counts -// are aggregated over a second and a qps computed. If the QPS is above threshold, -// a split finder is instantiated and the spans supplied to Record are sampled -// for a duration (on the order of ten seconds). Assuming that load consistently -// remains over threshold, and the workload touches a diverse enough set of keys -// to benefit from a split, sampling will eventually instruct a caller of Record -// to carry out a split. When the split is initiated, it can obtain the suggested -// split point from MaybeSplitKey (which may have disappeared either due to a drop -// in qps or a change in the workload). +// are aggregated over a second and a QPS is computed. +// +// If the QPS is above a threshold, a split finder is instantiated and the spans +// supplied to Record are sampled for a duration (on the order of ten seconds). +// Assuming that load consistently remains over threshold, and the workload +// touches a diverse enough set of keys to benefit from a split, sampling will +// eventually instruct a caller of Record to carry out a split. When the split +// is initiated, it can obtain the suggested split point from MaybeSplitKey +// (which may have disappeared either due to a drop in qps or a change in the +// workload). +// +// These second-long QPS samples are also aggregated together to track the +// maximum historical QPS over a configurable retention period. This maximum QPS +// measurement, which is accessible through the MaxQPS method, can be used to +// prevent load-based splits from being merged away until the resulting ranges +// have consistently remained below a certain QPS threshold for a sufficiently +// long period of time. type Decider struct { - intn func(n int) int // supplied to Init - qpsThreshold func() float64 // supplied to Init + intn func(n int) int // supplied to Init + qpsThreshold func() float64 // supplied to Init + qpsRetention func() time.Duration // supplied to Init mu struct { syncutil.Mutex + + // Fields tracking the current qps sample. lastQPSRollover time.Time // most recent time recorded by requests. - qps float64 // last reqs/s rate as of lastQPSRollover + lastQPS float64 // last reqs/s rate as of lastQPSRollover + count int64 // number of requests recorded since last rollover + + // Fields tracking historical qps samples. + maxQPS maxQPSTracker - count int64 // number of requests recorded since last rollover + // Fields tracking split key suggestions. splitFinder *Finder // populated when engaged or decided lastSplitSuggestion time.Time // last stipulation to client to carry out split } @@ -55,9 +74,15 @@ type Decider struct { // embedding the Decider into a larger struct outside of the scope of this package // without incurring a pointer reference. This is relevant since many Deciders // may exist in the system at any given point in time. -func Init(lbs *Decider, intn func(n int) int, qpsThreshold func() float64) { +func Init( + lbs *Decider, + intn func(n int) int, + qpsThreshold func() float64, + qpsRetention func() time.Duration, +) { lbs.intn = intn lbs.qpsThreshold = qpsThreshold + lbs.qpsRetention = qpsRetention } // Record notifies the Decider that 'n' operations are being carried out which @@ -79,25 +104,26 @@ func (d *Decider) recordLocked(now time.Time, n int, span func() roachpb.Span) b d.mu.count += int64(n) // First compute requests per second since the last check. + if d.mu.lastQPSRollover.IsZero() { + d.mu.lastQPSRollover = now + } elapsedSinceLastQPS := now.Sub(d.mu.lastQPSRollover) - if elapsedSinceLastQPS >= time.Second { - if elapsedSinceLastQPS > 2*time.Second { - // Force a QPS of zero; there wasn't any activity within the last - // second at all. - d.mu.count = 0 - } - // Update the QPS and reset the time and request counter. - d.mu.qps = (float64(d.mu.count) / float64(elapsedSinceLastQPS)) * 1e9 + if elapsedSinceLastQPS >= minQueriesPerSecondSampleDuration { + // Update the latest QPS and reset the time and request counter. + d.mu.lastQPS = (float64(d.mu.count) / float64(elapsedSinceLastQPS)) * 1e9 d.mu.lastQPSRollover = now d.mu.count = 0 + // Record the latest QPS sample in the historical tracker. + d.mu.maxQPS.record(now, d.qpsRetention(), d.mu.lastQPS) + // If the QPS for the range exceeds the threshold, start actively // tracking potential for splitting this range based on load. // This tracking will begin by initiating a splitFinder so it can // begin to Record requests so it can find a split point. If a // splitFinder already exists, we check if a split point is ready // to be used. - if d.mu.qps >= d.qpsThreshold() { + if d.mu.lastQPS >= d.qpsThreshold() { if d.mu.splitFinder == nil { d.mu.splitFinder = NewFinder(now) } @@ -119,14 +145,34 @@ func (d *Decider) recordLocked(now time.Time, n int, span func() roachpb.Span) b return false } +// RecordMax adds a QPS measurement directly into the Decider's historical QPS +// tracker. The QPS sample is considered to have been captured at the provided +// time. +func (d *Decider) RecordMax(now time.Time, qps float64) { + d.mu.Lock() + defer d.mu.Unlock() + + d.mu.maxQPS.record(now, d.qpsRetention(), qps) +} + // LastQPS returns the most recent QPS measurement. func (d *Decider) LastQPS(now time.Time) float64 { d.mu.Lock() - d.recordLocked(now, 0, nil) - qps := d.mu.qps - d.mu.Unlock() + defer d.mu.Unlock() - return qps + d.recordLocked(now, 0, nil) // force QPS computation + return d.mu.lastQPS +} + +// MaxQPS returns the maximum QPS measurement recorded over the retention +// period. If the Decider has not been recording for a full retention period, +// the method returns false. +func (d *Decider) MaxQPS(now time.Time) (float64, bool) { + d.mu.Lock() + defer d.mu.Unlock() + + d.recordLocked(now, 0, nil) // force QPS computation + return d.mu.maxQPS.maxQPS(now, d.qpsRetention()) } // MaybeSplitKey returns a key to perform a split at. The return value will be @@ -138,6 +184,8 @@ func (d *Decider) MaybeSplitKey(now time.Time) roachpb.Key { var key roachpb.Key d.mu.Lock() + defer d.mu.Unlock() + d.recordLocked(now, 0, nil) if d.mu.splitFinder != nil && d.mu.splitFinder.Ready(now) { // We've found a key to split at. This key might be in the middle of a @@ -177,15 +225,125 @@ func (d *Decider) MaybeSplitKey(now time.Time) roachpb.Key { key = safeKey } } - d.mu.Unlock() - return key } -// Reset deactivates any current attempt at determining a split key. -func (d *Decider) Reset() { +// Reset deactivates any current attempt at determining a split key. The method +// also discards any historical QPS tracking information. +func (d *Decider) Reset(now time.Time) { d.mu.Lock() - d.mu.splitFinder = nil + defer d.mu.Unlock() + + d.mu.lastQPSRollover = time.Time{} + d.mu.lastQPS = 0 d.mu.count = 0 - d.mu.Unlock() + d.mu.maxQPS.reset(now, d.qpsRetention()) + d.mu.splitFinder = nil + d.mu.lastSplitSuggestion = time.Time{} +} + +// maxQPSTracker collects a series of queries-per-second measurement samples and +// tracks the maximum observed over a period of time. +// +// The tracker internally uses a set of time windows in order to age out old +// partial aggregations without having to do hard resets. The `windows` array is +// a circular buffer of the last N windows of stats. We rotate through the +// circular buffer every so often as determined by `minRetention`. +// +// The tracker can be queried through its `maxQPS` method, which returns the +// maximum of all queries-per-second samples recorded over the retention period. +// If the tracker has not been recording for a full retention period, then the +// method returns false. +// +// The zero-value of a maxQPSTracker can be used immediately. +type maxQPSTracker struct { + windows [6]float64 + curIdx int + curStart time.Time + lastReset time.Time + minRetention time.Duration +} + +// record adds the qps sample to the tracker. +func (t *maxQPSTracker) record(now time.Time, minRetention time.Duration, qps float64) { + t.maybeReset(now, minRetention) + t.maybeRotate(now) + t.windows[t.curIdx] = max(t.windows[t.curIdx], qps) +} + +// reset clears the tracker. maxQPS will begin returning false until a full +// minRetention period has elapsed. +func (t *maxQPSTracker) reset(now time.Time, minRetention time.Duration) { + if minRetention <= 0 { + panic("minRetention must be positive") + } + t.windows = [6]float64{} + t.curIdx = 0 + t.curStart = now + t.lastReset = now + t.minRetention = minRetention +} + +func (t *maxQPSTracker) maybeReset(now time.Time, minRetention time.Duration) { + // If the retention period changes, simply reset the entire tracker. Merging + // or splitting windows would be a difficult task and could lead to samples + // either not being retained for long-enough, or being retained for too long. + // Resetting indicates to maxQPS that a new retention period needs to be + // measured before accurate results can be returned. + if minRetention != t.minRetention { + t.reset(now, minRetention) + } +} + +func (t *maxQPSTracker) maybeRotate(now time.Time) { + sinceLastRotate := now.Sub(t.curStart) + windowWidth := t.windowWidth() + if sinceLastRotate < windowWidth { + // Use the existing window. + return + } + + shift := int(sinceLastRotate / windowWidth) + if shift >= len(t.windows) { + // Clear all windows. We had a long period of inactivity. + t.windows = [6]float64{} + t.curIdx = 0 + t.curStart = now + return + } + for i := 0; i < shift; i++ { + t.curIdx = (t.curIdx + 1) % len(t.windows) + t.curStart = t.curStart.Add(windowWidth) + t.windows[t.curIdx] = 0 + } +} + +// maxQPS returns the maximum queries-per-second samples recorded over the last +// retention period. If the tracker has not been recording for a full retention +// period, then the method returns false. +func (t *maxQPSTracker) maxQPS(now time.Time, minRetention time.Duration) (float64, bool) { + t.record(now, minRetention, 0) // expire samples, if necessary + + if now.Sub(t.lastReset) < t.minRetention { + // The tracker has not been recording for long enough. + return 0, false + } + + qps := 0.0 + for _, v := range t.windows { + qps = max(qps, v) + } + return qps, true +} + +func (t *maxQPSTracker) windowWidth() time.Duration { + // NB: -1 because during a rotation, only len(t.windows)-1 windows survive. + return t.minRetention / time.Duration(len(t.windows)-1) +} + +func max(a, b float64) float64 { + if a > b { + return a + } + return b } diff --git a/pkg/kv/kvserver/split/decider_test.go b/pkg/kv/kvserver/split/decider_test.go index 494d8abc6755..3ea1821c3734 100644 --- a/pkg/kv/kvserver/split/decider_test.go +++ b/pkg/kv/kvserver/split/decider_test.go @@ -24,19 +24,21 @@ import ( "github.com/stretchr/testify/require" ) +func ms(i int) time.Time { + ts, err := time.Parse(time.RFC3339, "2000-01-01T00:00:00Z") + if err != nil { + panic(err) + } + return ts.Add(time.Duration(i) * time.Millisecond) +} + func TestDecider(t *testing.T) { defer leaktest.AfterTest(t)() intn := rand.New(rand.NewSource(12)).Intn var d Decider - Init(&d, intn, func() float64 { return 10.0 }) - - ms := func(i int) time.Time { - ts, err := time.Parse(time.RFC3339, "2000-01-01T00:00:00Z") - assert.NoError(t, err) - return ts.Add(time.Duration(i) * time.Millisecond) - } + Init(&d, intn, func() float64 { return 10.0 }, func() time.Duration { return 2 * time.Second }) op := func(s string) func() roachpb.Span { return func() roachpb.Span { return roachpb.Span{Key: roachpb.Key(s)} } @@ -48,27 +50,36 @@ func TestDecider(t *testing.T) { assert.Equal(t, expQPS, qps) } + assertMaxQPS := func(i int, expMaxQPS float64, expOK bool) { + t.Helper() + maxQPS, ok := d.MaxQPS(ms(i)) + assert.Equal(t, expMaxQPS, maxQPS) + assert.Equal(t, expOK, ok) + } + assert.Equal(t, false, d.Record(ms(100), 1, nil)) assertQPS(100, 0) + assertMaxQPS(100, 0, false) - // The first operation was interpreted as having happened after an eternity - // of no activity, and rolled over the qps to mark the beginning of a new - // second. The next qps computation is expected for timestamps >= 1100. assert.Equal(t, ms(100), d.mu.lastQPSRollover) - assert.EqualValues(t, 0, d.mu.count) + assert.EqualValues(t, 1, d.mu.count) - assert.Equal(t, false, d.Record(ms(400), 4, nil)) + assert.Equal(t, false, d.Record(ms(400), 3, nil)) assertQPS(100, 0) assertQPS(700, 0) + assertMaxQPS(400, 0, false) assert.Equal(t, false, d.Record(ms(300), 3, nil)) assertQPS(100, 0) + assertMaxQPS(300, 0, false) assert.Equal(t, false, d.Record(ms(900), 1, nil)) assertQPS(0, 0) + assertMaxQPS(900, 0, false) assert.Equal(t, false, d.Record(ms(1099), 1, nil)) assertQPS(0, 0) + assertMaxQPS(1099, 0, false) // Now 9 operations happened in the interval [100, 1099]. The next higher // timestamp will decide whether to engage the split finder. @@ -78,6 +89,7 @@ func TestDecider(t *testing.T) { assert.Equal(t, false, d.Record(ms(1200), 1, nil)) assertQPS(0, float64(10)/float64(1.1)) assert.Equal(t, ms(1200), d.mu.lastQPSRollover) + assertMaxQPS(1099, 0, false) var nilFinder *Finder @@ -90,6 +102,7 @@ func TestDecider(t *testing.T) { assert.Equal(t, false, d.Record(ms(2200), 1, op("a"))) assert.Equal(t, ms(2200), d.mu.lastQPSRollover) assertQPS(0, float64(13)) + assertMaxQPS(2200, 13, true) assert.NotNil(t, d.mu.splitFinder) assert.False(t, d.mu.splitFinder.Ready(ms(10))) @@ -123,7 +136,8 @@ func TestDecider(t *testing.T) { } // But after minSplitSuggestionInterval of ticks, we get another one. assert.True(t, d.Record(ms(tick), 11, op("a"))) - assert.True(t, d.LastQPS(ms(tick)) > 1.0) + assertQPS(tick, float64(11)) + assertMaxQPS(tick, 11, true) // Split key suggestion vanishes once qps drops. tick += 1000 @@ -170,17 +184,60 @@ func TestDecider(t *testing.T) { // back up at zero. assert.True(t, d.mu.splitFinder.Ready(ms(tick))) assert.Equal(t, roachpb.Key("z"), d.MaybeSplitKey(ms(tick))) - d.Reset() + d.Reset(ms(tick)) assert.Nil(t, d.MaybeSplitKey(ms(tick))) assert.Nil(t, d.mu.splitFinder) } +func TestDecider_MaxQPS(t *testing.T) { + defer leaktest.AfterTest(t)() + intn := rand.New(rand.NewSource(11)).Intn + + var d Decider + Init(&d, intn, func() float64 { return 100.0 }, func() time.Duration { return 10 * time.Second }) + + assertMaxQPS := func(i int, expMaxQPS float64, expOK bool) { + t.Helper() + maxQPS, ok := d.MaxQPS(ms(i)) + assert.Equal(t, expMaxQPS, maxQPS) + assert.Equal(t, expOK, ok) + } + + assertMaxQPS(1000, 0, false) + + // Record a large number of samples. + d.Record(ms(1500), 5, nil) + d.Record(ms(2000), 5, nil) + d.Record(ms(4500), 1, nil) + d.Record(ms(5000), 15, nil) + d.Record(ms(5500), 2, nil) + d.Record(ms(8000), 5, nil) + d.Record(ms(10000), 9, nil) + + assertMaxQPS(10000, 0, false) + assertMaxQPS(11000, 17, true) + + // Record more samples with a lower QPS. + d.Record(ms(12000), 1, nil) + d.Record(ms(13000), 4, nil) + d.Record(ms(15000), 2, nil) + d.Record(ms(19000), 3, nil) + + assertMaxQPS(20000, 4.5, true) + assertMaxQPS(21000, 4, true) + + // Add in a few QPS reading directly. + d.RecordMax(ms(24000), 6) + + assertMaxQPS(25000, 6, true) +} + func TestDeciderCallsEnsureSafeSplitKey(t *testing.T) { defer leaktest.AfterTest(t)() intn := rand.New(rand.NewSource(11)).Intn var d Decider - Init(&d, intn, func() float64 { return 1.0 }) + Init(&d, intn, func() float64 { return 1.0 }, func() time.Duration { return time.Second }) baseKey := keys.SystemSQLCodec.TablePrefix(51) for i := 0; i < 4; i++ { @@ -213,7 +270,7 @@ func TestDeciderIgnoresEnsureSafeSplitKeyOnError(t *testing.T) { intn := rand.New(rand.NewSource(11)).Intn var d Decider - Init(&d, intn, func() float64 { return 1.0 }) + Init(&d, intn, func() float64 { return 1.0 }, func() time.Duration { return time.Second }) baseKey := keys.SystemSQLCodec.TablePrefix(51) for i := 0; i < 4; i++ { @@ -244,3 +301,94 @@ func TestDeciderIgnoresEnsureSafeSplitKeyOnError(t *testing.T) { require.Equal(t, c1().Key, k) } + +func TestMaxQPSTracker(t *testing.T) { + defer leaktest.AfterTest(t)() + + tick := 100 + minRetention := time.Second + + var mt maxQPSTracker + mt.reset(ms(tick), minRetention) + require.Equal(t, 200*time.Millisecond, mt.windowWidth()) + + // Check the maxQPS returns false before any samples are recorded. + qps, ok := mt.maxQPS(ms(tick), minRetention) + require.Equal(t, 0.0, qps) + require.Equal(t, false, ok) + require.Equal(t, [6]float64{0, 0, 0, 0, 0, 0}, mt.windows) + require.Equal(t, 0, mt.curIdx) + + // Add some samples, but not for the full minRetention period. Each window + // should contain 4 samples. + for i := 0; i < 15; i++ { + tick += 50 + mt.record(ms(tick), minRetention, float64(10+i)) + } + + // maxQPS should still return false, but some windows should have samples. + qps, ok = mt.maxQPS(ms(tick), minRetention) + require.Equal(t, 0.0, qps) + require.Equal(t, false, ok) + require.Equal(t, [6]float64{12, 16, 20, 24, 0, 0}, mt.windows) + require.Equal(t, 3, mt.curIdx) + + // Add some more samples. + for i := 0; i < 15; i++ { + tick += 50 + mt.record(ms(tick), minRetention, float64(24+i)) + } + + // maxQPS should now return the maximum qps observed during the measurement + // period. + qps, ok = mt.maxQPS(ms(tick), minRetention) + require.Equal(t, 38.0, qps) + require.Equal(t, true, ok) + require.Equal(t, [6]float64{35, 38, 20, 24, 27, 31}, mt.windows) + require.Equal(t, 1, mt.curIdx) + + // Add another sample, this time with a small gap between it and the previous + // sample, so that 2 windows are skipped. + tick += 500 + mt.record(ms(tick), minRetention, float64(17)) + + qps, ok = mt.maxQPS(ms(tick), minRetention) + require.Equal(t, 38.0, qps) + require.Equal(t, true, ok) + require.Equal(t, [6]float64{35, 38, 0, 0, 17, 31}, mt.windows) + require.Equal(t, 4, mt.curIdx) + + // A query far in the future should return 0, because this indicates no + // recent activity. + tick += 1900 + qps, ok = mt.maxQPS(ms(tick), minRetention) + require.Equal(t, 0.0, qps) + require.Equal(t, true, ok) + require.Equal(t, [6]float64{0, 0, 0, 0, 0, 0}, mt.windows) + require.Equal(t, 0, mt.curIdx) + + // Add some new samples, then change the retention period, Should reset + // tracker. + for i := 0; i < 15; i++ { + tick += 50 + mt.record(ms(tick), minRetention, float64(33+i)) + } + + qps, ok = mt.maxQPS(ms(tick), minRetention) + require.Equal(t, 47.0, qps) + require.Equal(t, true, ok) + require.Equal(t, [6]float64{35, 39, 43, 47, 0, 0}, mt.windows) + require.Equal(t, 3, mt.curIdx) + + minRetention = 2 * time.Second + for i := 0; i < 15; i++ { + tick += 50 + mt.record(ms(tick), minRetention, float64(13+i)) + } + + qps, ok = mt.maxQPS(ms(tick), minRetention) + require.Equal(t, 0.0, qps) + require.Equal(t, false, ok) + require.Equal(t, [6]float64{20, 27, 0, 0, 0, 0}, mt.windows) + require.Equal(t, 1, mt.curIdx) +} diff --git a/pkg/kv/kvserver/split_queue.go b/pkg/kv/kvserver/split_queue.go index bb03a2a7772f..e5dca40d9908 100644 --- a/pkg/kv/kvserver/split_queue.go +++ b/pkg/kv/kvserver/split_queue.go @@ -231,11 +231,11 @@ func (sq *splitQueue) processAttempt( // Add a small delay (default of 5m) to any subsequent attempt to merge // this range split away. While the merge queue does takes into account // load to avoids merging ranges that would be immediately re-split due - // to load-based splitting, it doesn't take into account historical - // load. So this small delay is the only thing that prevents split - // points created due to load from being immediately merged away after - // load is stopped, which can be a problem for benchmarks where data is - // first imported and then the workload begins after a small delay. + // to load-based splitting, it did not used to take into account historical + // load. This has since been fixed by #64201, but we keep this small manual + // delay for compatibility reasons. + // TODO(nvanbenschoten): remove this entirely in v22.1 when it is no longer + // needed. var expTime hlc.Timestamp if expDelay := SplitByLoadMergeDelay.Get(&sq.store.cfg.Settings.SV); expDelay > 0 { expTime = sq.store.Clock().Now().Add(expDelay.Nanoseconds(), 0) @@ -259,7 +259,7 @@ func (sq *splitQueue) processAttempt( telemetry.Inc(sq.loadBasedCount) // Reset the splitter now that the bounds of the range changed. - r.loadBasedSplitter.Reset() + r.loadBasedSplitter.Reset(sq.store.Clock().PhysicalTime()) return true, nil } return false, nil diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 1df7042345a6..d8cae5d9d849 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -75,7 +75,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -103,7 +103,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{1} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{1} } type ChecksumMode int32 @@ -150,7 +150,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{2} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -181,7 +181,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{3} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{3} } type ExternalStorageProvider int32 @@ -222,7 +222,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{4} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{4} } type MVCCFilter int32 @@ -245,7 +245,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{5} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{5} } type ResponseHeader_ResumeReason int32 @@ -271,7 +271,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{1, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -313,7 +313,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{25, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -334,7 +334,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_3a63b0fb97c122b3, []int{0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -400,7 +400,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_3a63b0fb97c122b3, []int{1} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -440,7 +440,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_3a63b0fb97c122b3, []int{2} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -483,7 +483,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_3a63b0fb97c122b3, []int{3} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -526,7 +526,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_3a63b0fb97c122b3, []int{4} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -560,7 +560,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_3a63b0fb97c122b3, []int{5} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -649,7 +649,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_3a63b0fb97c122b3, []int{6} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -684,7 +684,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_3a63b0fb97c122b3, []int{7} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -730,7 +730,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_3a63b0fb97c122b3, []int{8} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -764,7 +764,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_3a63b0fb97c122b3, []int{9} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -804,7 +804,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_3a63b0fb97c122b3, []int{10} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -841,7 +841,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_3a63b0fb97c122b3, []int{11} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -875,7 +875,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_3a63b0fb97c122b3, []int{12} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -909,7 +909,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_3a63b0fb97c122b3, []int{13} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -961,7 +961,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_3a63b0fb97c122b3, []int{14} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -998,7 +998,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_3a63b0fb97c122b3, []int{15} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1053,7 +1053,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_3a63b0fb97c122b3, []int{16} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1087,7 +1087,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_3a63b0fb97c122b3, []int{17} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1137,7 +1137,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_3a63b0fb97c122b3, []int{18} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1171,7 +1171,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_3a63b0fb97c122b3, []int{19} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1222,7 +1222,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_3a63b0fb97c122b3, []int{20} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1274,7 +1274,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_3a63b0fb97c122b3, []int{21} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1325,7 +1325,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_3a63b0fb97c122b3, []int{22} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1377,7 +1377,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_3a63b0fb97c122b3, []int{23} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1430,7 +1430,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_3a63b0fb97c122b3, []int{24} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1467,7 +1467,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_3a63b0fb97c122b3, []int{25} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1511,7 +1511,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_3a63b0fb97c122b3, []int{25, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1559,7 +1559,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_3a63b0fb97c122b3, []int{26} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1595,7 +1595,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_3a63b0fb97c122b3, []int{27} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1706,7 +1706,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_3a63b0fb97c122b3, []int{28} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1752,7 +1752,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_3a63b0fb97c122b3, []int{29} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1813,7 +1813,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_3a63b0fb97c122b3, []int{30} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1848,7 +1848,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_3a63b0fb97c122b3, []int{31} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1887,7 +1887,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_3a63b0fb97c122b3, []int{32} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1922,7 +1922,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_3a63b0fb97c122b3, []int{33} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1965,7 +1965,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_3a63b0fb97c122b3, []int{34} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2000,7 +2000,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_3a63b0fb97c122b3, []int{35} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2038,7 +2038,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_3a63b0fb97c122b3, []int{36} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2071,7 +2071,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_3a63b0fb97c122b3, []int{37} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2106,7 +2106,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_3a63b0fb97c122b3, []int{38} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2164,7 +2164,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_3a63b0fb97c122b3, []int{39} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2199,7 +2199,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_3a63b0fb97c122b3, []int{40} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2237,7 +2237,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{41} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2270,7 +2270,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_3a63b0fb97c122b3, []int{42} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2312,7 +2312,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_3a63b0fb97c122b3, []int{43} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2349,7 +2349,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_3a63b0fb97c122b3, []int{44} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2387,7 +2387,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_3a63b0fb97c122b3, []int{45} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2421,7 +2421,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_3a63b0fb97c122b3, []int{45, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2455,7 +2455,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_3a63b0fb97c122b3, []int{46} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2524,7 +2524,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_3a63b0fb97c122b3, []int{47} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2567,7 +2567,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_3a63b0fb97c122b3, []int{48} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2614,7 +2614,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_3a63b0fb97c122b3, []int{49} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2650,7 +2650,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_3a63b0fb97c122b3, []int{50} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2694,7 +2694,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_3a63b0fb97c122b3, []int{51} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2738,7 +2738,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_3a63b0fb97c122b3, []int{52} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2806,7 +2806,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_3a63b0fb97c122b3, []int{53} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2842,7 +2842,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_3a63b0fb97c122b3, []int{54} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2889,7 +2889,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_3a63b0fb97c122b3, []int{55} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2924,7 +2924,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_3a63b0fb97c122b3, []int{56} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2975,7 +2975,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_3a63b0fb97c122b3, []int{57} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3010,7 +3010,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_3a63b0fb97c122b3, []int{58} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3047,7 +3047,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_3a63b0fb97c122b3, []int{59} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3081,7 +3081,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_3a63b0fb97c122b3, []int{60} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3126,7 +3126,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_3a63b0fb97c122b3, []int{61} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3160,7 +3160,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_3a63b0fb97c122b3, []int{62} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3204,7 +3204,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_3a63b0fb97c122b3, []int{63} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3253,7 +3253,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_3a63b0fb97c122b3, []int{64} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3292,7 +3292,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_3a63b0fb97c122b3, []int{65} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3342,7 +3342,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_3a63b0fb97c122b3, []int{66} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3377,7 +3377,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_3a63b0fb97c122b3, []int{67} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3432,7 +3432,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_3a63b0fb97c122b3, []int{68} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3469,7 +3469,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_3a63b0fb97c122b3, []int{69} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3509,7 +3509,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_3a63b0fb97c122b3, []int{70} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3543,7 +3543,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_3a63b0fb97c122b3, []int{70, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3576,7 +3576,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_3a63b0fb97c122b3, []int{70, 1} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3618,7 +3618,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_3a63b0fb97c122b3, []int{70, 2} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3657,7 +3657,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_3a63b0fb97c122b3, []int{70, 3} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3693,7 +3693,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_3a63b0fb97c122b3, []int{70, 4} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3732,7 +3732,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_3a63b0fb97c122b3, []int{70, 5} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3774,7 +3774,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_3a63b0fb97c122b3, []int{70, 6} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3814,7 +3814,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_3a63b0fb97c122b3, []int{71} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3848,7 +3848,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_3a63b0fb97c122b3, []int{72} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3884,7 +3884,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_3a63b0fb97c122b3, []int{73} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3954,7 +3954,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_3a63b0fb97c122b3, []int{74} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4006,7 +4006,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_3a63b0fb97c122b3, []int{75} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4042,7 +4042,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_3a63b0fb97c122b3, []int{76} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4082,7 +4082,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_3a63b0fb97c122b3, []int{76, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4133,7 +4133,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_3a63b0fb97c122b3, []int{77} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4168,7 +4168,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_3a63b0fb97c122b3, []int{77, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4204,7 +4204,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_3a63b0fb97c122b3, []int{77, 1} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4239,7 +4239,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_3a63b0fb97c122b3, []int{78} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4277,7 +4277,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_3a63b0fb97c122b3, []int{79} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4314,7 +4314,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_3a63b0fb97c122b3, []int{80} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4347,7 +4347,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_3a63b0fb97c122b3, []int{80, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4392,7 +4392,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_3a63b0fb97c122b3, []int{81} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4432,7 +4432,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_3a63b0fb97c122b3, []int{82} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4472,7 +4472,7 @@ func (m *AdminVerifyProtectedTimestampResponse_FailedRange) String() string { } func (*AdminVerifyProtectedTimestampResponse_FailedRange) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse_FailedRange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{82, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{82, 0} } func (m *AdminVerifyProtectedTimestampResponse_FailedRange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4525,7 +4525,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_3a63b0fb97c122b3, []int{83} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4559,7 +4559,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_3a63b0fb97c122b3, []int{84} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4603,7 +4603,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_3a63b0fb97c122b3, []int{85} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4637,7 +4637,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_3a63b0fb97c122b3, []int{86} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4676,7 +4676,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_3a63b0fb97c122b3, []int{87} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4710,7 +4710,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_3a63b0fb97c122b3, []int{88} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4759,7 +4759,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_3a63b0fb97c122b3, []int{89} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4837,7 +4837,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_3a63b0fb97c122b3, []int{90} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4872,7 +4872,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_3a63b0fb97c122b3, []int{91} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4903,8 +4903,29 @@ type RangeStatsResponse struct { // MVCCStats are the MVCC statistics for the range that processed the // request. MVCCStats enginepb.MVCCStats `protobuf:"bytes,2,opt,name=mvcc_stats,json=mvccStats,proto3" json:"mvcc_stats"` - // QueriesPerSecond is the rate of request/s or QPS for the range. - QueriesPerSecond float64 `protobuf:"fixed64,3,opt,name=queries_per_second,json=queriesPerSecond,proto3" json:"queries_per_second,omitempty"` + // DeprecatedLastQueriesPerSecond is the most recent rate of request/s or QPS + // for the range. The field is deprecated in favor of MaxQueriesPerSecond. + // + // TODO(nvanbenschoten): remove this field in v22.1 when all nodes in the + // cluster are guaranteed to return MaxQueriesPerSecond. + DeprecatedLastQueriesPerSecond float64 `protobuf:"fixed64,3,opt,name=deprecated_last_queries_per_second,json=deprecatedLastQueriesPerSecond,proto3" json:"deprecated_last_queries_per_second,omitempty"` + // MaxQueriesPerSecond is the maximum rate of request/s or QPS that the range + // has served over a configured measurement period. Set to -1 if the replica + // serving the RangeStats request has not been the leaseholder long enough to + // have recorded request rates for at least a full measurement period. In such + // cases, the recipient should not consider the QPS value reliable enough to + // base important decisions off of. + MaxQueriesPerSecond float64 `protobuf:"fixed64,5,opt,name=max_queries_per_second,json=maxQueriesPerSecond,proto3" json:"max_queries_per_second,omitempty"` + // MaxQueriesPerSecondSet indicates that the MaxQueriesPerSecond field is set + // by the server. Used to distinguish 0 qps set by a new server from the field + // not being set at all by an old server. + // + // TODO(nvanbenschoten): stop consulting this field on the receiver in v22.1 + // when all nodes in the cluster are guaranteed to return MaxQueriesPerSecond. + // + // TODO(nvanbenschoten): stop setting this field and remove it in v22.2 when + // no nodes in the cluster consult this field. + MaxQueriesPerSecondSet bool `protobuf:"varint,6,opt,name=max_queries_per_second_set,json=maxQueriesPerSecondSet,proto3" json:"max_queries_per_second_set,omitempty"` // range_info contains descriptor and lease information. RangeInfo RangeInfo `protobuf:"bytes,4,opt,name=range_info,json=rangeInfo,proto3" json:"range_info"` } @@ -4913,7 +4934,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_3a63b0fb97c122b3, []int{92} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4956,7 +4977,7 @@ func (m *MigrateRequest) Reset() { *m = MigrateRequest{} } func (m *MigrateRequest) String() string { return proto.CompactTextString(m) } func (*MigrateRequest) ProtoMessage() {} func (*MigrateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{93} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{93} } func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4990,7 +5011,7 @@ func (m *MigrateResponse) Reset() { *m = MigrateResponse{} } func (m *MigrateResponse) String() string { return proto.CompactTextString(m) } func (*MigrateResponse) ProtoMessage() {} func (*MigrateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{94} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{94} } func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5074,7 +5095,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_3a63b0fb97c122b3, []int{95} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6555,7 +6576,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_3a63b0fb97c122b3, []int{96} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8111,7 +8132,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_3a63b0fb97c122b3, []int{97} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8147,7 +8168,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{98} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{98} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8184,7 +8205,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{99} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{99} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8260,7 +8281,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_3a63b0fb97c122b3, []int{99, 0} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{99, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8298,7 +8319,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_3a63b0fb97c122b3, []int{100} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{100} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8336,7 +8357,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_3a63b0fb97c122b3, []int{101} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{101} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8375,7 +8396,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_3a63b0fb97c122b3, []int{102} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{102} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8416,7 +8437,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_3a63b0fb97c122b3, []int{103} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{103} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8457,7 +8478,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_3a63b0fb97c122b3, []int{104} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{104} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8494,7 +8515,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_3a63b0fb97c122b3, []int{105} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{105} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8531,7 +8552,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_3a63b0fb97c122b3, []int{106} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{106} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8575,7 +8596,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_3a63b0fb97c122b3, []int{107} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{107} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8607,7 +8628,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_3a63b0fb97c122b3, []int{108} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{108} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8648,7 +8669,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_3a63b0fb97c122b3, []int{109} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{109} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8688,7 +8709,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_3a63b0fb97c122b3, []int{110} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{110} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8724,7 +8745,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_3a63b0fb97c122b3, []int{111} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{111} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8763,7 +8784,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_3a63b0fb97c122b3, []int{112} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{112} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8803,7 +8824,7 @@ type ContentionEvent struct { func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_3a63b0fb97c122b3, []int{113} + return fileDescriptor_api_177e11c2c9d5aa4a, []int{113} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -13756,10 +13777,10 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n163 - if m.QueriesPerSecond != 0 { + if m.DeprecatedLastQueriesPerSecond != 0 { dAtA[i] = 0x19 i++ - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.DeprecatedLastQueriesPerSecond)))) i += 8 } dAtA[i] = 0x22 @@ -13770,6 +13791,22 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n164 + if m.MaxQueriesPerSecond != 0 { + dAtA[i] = 0x29 + i++ + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.MaxQueriesPerSecond)))) + i += 8 + } + if m.MaxQueriesPerSecondSet { + dAtA[i] = 0x30 + i++ + if m.MaxQueriesPerSecondSet { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -17991,11 +18028,17 @@ func (m *RangeStatsResponse) Size() (n int) { n += 1 + l + sovApi(uint64(l)) l = m.MVCCStats.Size() n += 1 + l + sovApi(uint64(l)) - if m.QueriesPerSecond != 0 { + if m.DeprecatedLastQueriesPerSecond != 0 { n += 9 } l = m.RangeInfo.Size() n += 1 + l + sovApi(uint64(l)) + if m.MaxQueriesPerSecond != 0 { + n += 9 + } + if m.MaxQueriesPerSecondSet { + n += 2 + } return n } @@ -33726,7 +33769,7 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field QueriesPerSecond", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedLastQueriesPerSecond", wireType) } var v uint64 if (iNdEx + 8) > l { @@ -33734,7 +33777,7 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { } v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 - m.QueriesPerSecond = float64(math.Float64frombits(v)) + m.DeprecatedLastQueriesPerSecond = float64(math.Float64frombits(v)) case 4: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field RangeInfo", wireType) @@ -33765,6 +33808,37 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 5: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxQueriesPerSecond", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.MaxQueriesPerSecond = float64(math.Float64frombits(v)) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxQueriesPerSecondSet", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.MaxQueriesPerSecondSet = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -39486,532 +39560,535 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_3a63b0fb97c122b3) } - -var fileDescriptor_api_3a63b0fb97c122b3 = []byte{ - // 8373 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0xbd, 0x5d, 0x6c, 0x23, 0x59, - 0x76, 0x1f, 0xae, 0x22, 0x29, 0x89, 0x3c, 0xa4, 0x48, 0xea, 0xaa, 0x3f, 0xd8, 0x9c, 0x99, 0x56, - 0x77, 0xf5, 0x77, 0xef, 0x0c, 0x35, 0xdd, 0xbd, 0xf3, 0x9f, 0xf1, 0xf4, 0x78, 0xd6, 0x12, 0xc5, - 0x6e, 0x52, 0x6a, 0xa9, 0xd5, 0x45, 0xaa, 0x1b, 0x33, 0x5e, 0xff, 0x6b, 0x4b, 0x55, 0x57, 0x54, - 0xad, 0xc8, 0x2a, 0x76, 0x55, 0x51, 0x1f, 0x03, 0x04, 0x88, 0x63, 0xc3, 0xd9, 0xa7, 0x60, 0x03, - 0x18, 0xf0, 0x1a, 0x0e, 0xe2, 0x75, 0xd6, 0x88, 0x1f, 0x02, 0x24, 0x01, 0x12, 0xe4, 0x0b, 0x89, - 0xfd, 0x62, 0x20, 0x8b, 0xc0, 0x89, 0xd7, 0x6f, 0x46, 0x80, 0x28, 0xb6, 0x36, 0x2f, 0x81, 0x83, - 0x20, 0xc8, 0x8b, 0x81, 0x01, 0x12, 0x04, 0xf7, 0xa3, 0xbe, 0xc8, 0x22, 0x45, 0xf5, 0xd4, 0x26, - 0x03, 0xf8, 0x45, 0x62, 0x9d, 0xba, 0xe7, 0xd4, 0xbd, 0xe7, 0xde, 0x7b, 0xee, 0xf9, 0xdd, 0x3a, - 0xf7, 0x14, 0xcc, 0x5b, 0xa6, 0xa2, 0xee, 0xf5, 0x76, 0x96, 0x94, 0x9e, 0x5e, 0xe9, 0x59, 0xa6, - 0x63, 0xa2, 0x79, 0xd5, 0x54, 0xf7, 0x29, 0xb9, 0xc2, 0x6f, 0x96, 0xef, 0xef, 0x1f, 0x2c, 0xed, - 0x1f, 0xd8, 0xd8, 0x3a, 0xc0, 0xd6, 0x92, 0x6a, 0x1a, 0x6a, 0xdf, 0xb2, 0xb0, 0xa1, 0x1e, 0x2f, - 0x75, 0x4c, 0x75, 0x9f, 0xfe, 0xd1, 0x8d, 0x36, 0x63, 0x0f, 0x97, 0xb5, 0xb0, 0xa2, 0xd9, 0xfd, - 0x6e, 0x57, 0xb1, 0x8e, 0x97, 0x2c, 0xbb, 0xb7, 0xb3, 0xc4, 0x2f, 0x78, 0x59, 0xe4, 0x3e, 0x5d, - 0x53, 0x1c, 0x85, 0xd3, 0x2e, 0xb8, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x25, 0x97, 0xda, - 0xc5, 0x8e, 0x12, 0x28, 0xfd, 0x96, 0xed, 0x98, 0x96, 0xd2, 0xc6, 0x4b, 0xd8, 0x68, 0xeb, 0x06, - 0x26, 0x05, 0x0e, 0x54, 0x95, 0xdf, 0x7c, 0x3b, 0xf2, 0xe6, 0x23, 0x7e, 0xb7, 0xd4, 0x77, 0xf4, - 0xce, 0xd2, 0x5e, 0x47, 0x5d, 0x72, 0xf4, 0x2e, 0xb6, 0x1d, 0xa5, 0xdb, 0x73, 0x9b, 0x40, 0xef, - 0x38, 0x96, 0xa2, 0xea, 0x46, 0xdb, 0xfd, 0xdf, 0xdb, 0x59, 0xb2, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, - 0x64, 0xbb, 0xa7, 0x18, 0x6e, 0x75, 0xdb, 0x66, 0xdb, 0xa4, 0x3f, 0x97, 0xc8, 0x2f, 0x4e, 0xbd, - 0xda, 0x36, 0xcd, 0x76, 0x07, 0x2f, 0xd1, 0xab, 0x9d, 0xfe, 0xee, 0x92, 0xd6, 0xb7, 0x14, 0x47, - 0x37, 0x39, 0x97, 0xf8, 0xcf, 0x04, 0x98, 0x93, 0xf0, 0xeb, 0x3e, 0xb6, 0x9d, 0x3a, 0x56, 0x34, - 0x6c, 0xa1, 0x2b, 0x90, 0xdc, 0xc7, 0xc7, 0xa5, 0xe4, 0x35, 0xe1, 0x6e, 0x6e, 0x65, 0xf6, 0xcb, - 0x93, 0xc5, 0xe4, 0x3a, 0x3e, 0x96, 0x08, 0x0d, 0x5d, 0x83, 0x59, 0x6c, 0x68, 0x32, 0xb9, 0x9d, - 0x0a, 0xdf, 0x9e, 0xc1, 0x86, 0xb6, 0x8e, 0x8f, 0xd1, 0xb7, 0x21, 0x6d, 0x13, 0x69, 0x86, 0x8a, - 0x4b, 0xd3, 0xd7, 0x84, 0xbb, 0xd3, 0x2b, 0xbf, 0xf0, 0xe5, 0xc9, 0xe2, 0x27, 0x6d, 0xdd, 0xd9, - 0xeb, 0xef, 0x54, 0x54, 0xb3, 0xbb, 0xe4, 0xf5, 0xa9, 0xb6, 0xe3, 0xff, 0x5e, 0xea, 0xed, 0xb7, - 0x97, 0x06, 0x75, 0x54, 0x69, 0x1d, 0x19, 0x4d, 0xfc, 0x5a, 0xf2, 0x24, 0xae, 0xa5, 0xd2, 0x42, - 0x31, 0xb1, 0x96, 0x4a, 0x27, 0x8a, 0x49, 0xf1, 0x8f, 0x13, 0x90, 0x97, 0xb0, 0xdd, 0x33, 0x0d, - 0x1b, 0xf3, 0x9a, 0xbf, 0x0f, 0x49, 0xe7, 0xc8, 0xa0, 0x35, 0xcf, 0x3e, 0xbc, 0x5a, 0x19, 0x1a, - 0x3d, 0x95, 0x96, 0xa5, 0x18, 0xb6, 0xa2, 0x92, 0xe6, 0x4b, 0xa4, 0x28, 0xfa, 0x08, 0xb2, 0x16, - 0xb6, 0xfb, 0x5d, 0x4c, 0x15, 0x49, 0x1b, 0x95, 0x7d, 0x78, 0x39, 0x82, 0xb3, 0xd9, 0x53, 0x0c, - 0x09, 0x58, 0x59, 0xf2, 0x1b, 0x5d, 0x81, 0xb4, 0xd1, 0xef, 0x12, 0x55, 0xd8, 0xb4, 0xa1, 0x49, - 0x69, 0xd6, 0xe8, 0x77, 0xd7, 0xf1, 0xb1, 0x8d, 0x9a, 0x30, 0xc7, 0x85, 0x5a, 0x58, 0xb1, 0x4d, - 0xa3, 0x34, 0x7b, 0x4d, 0xb8, 0x9b, 0x7f, 0x58, 0x89, 0x10, 0x1b, 0x6e, 0x00, 0xb9, 0xec, 0x77, - 0xb1, 0x44, 0xb9, 0xa4, 0x9c, 0x15, 0xb8, 0x42, 0x6f, 0x41, 0x86, 0x3c, 0x6f, 0xe7, 0xd8, 0xc1, - 0x76, 0x29, 0x4d, 0x1f, 0x48, 0x2a, 0xb0, 0x42, 0xae, 0xc5, 0x4f, 0x21, 0x17, 0x64, 0x45, 0x08, - 0xf2, 0x52, 0xad, 0xb9, 0xbd, 0x51, 0x93, 0xb7, 0x37, 0xd7, 0x37, 0x9f, 0xbf, 0xda, 0x2c, 0x4e, - 0xa1, 0x0b, 0x50, 0xe4, 0xb4, 0xf5, 0xda, 0x67, 0xf2, 0xb3, 0xc6, 0x46, 0xa3, 0x55, 0x14, 0xca, - 0xa9, 0xef, 0xfd, 0xe8, 0xea, 0xd4, 0x5a, 0x2a, 0x3d, 0x53, 0x9c, 0x15, 0x7f, 0x24, 0x00, 0x3c, - 0xc5, 0x0e, 0x1f, 0x0d, 0x68, 0x05, 0x66, 0xf6, 0x68, 0xb5, 0x4a, 0x02, 0x55, 0xcb, 0xb5, 0xc8, - 0xfa, 0x07, 0x46, 0xce, 0x4a, 0xfa, 0xc7, 0x27, 0x8b, 0x53, 0x3f, 0x39, 0x59, 0x14, 0x24, 0xce, - 0x89, 0x5e, 0x40, 0x76, 0x1f, 0x1f, 0xcb, 0x7c, 0x5e, 0x96, 0x12, 0x54, 0x11, 0xef, 0x07, 0x04, - 0xed, 0x1f, 0x54, 0xdc, 0x29, 0x5a, 0x09, 0x4c, 0xe7, 0x0a, 0xe1, 0xa8, 0x34, 0x1d, 0x0b, 0x1b, - 0x6d, 0x67, 0x4f, 0x82, 0x7d, 0x7c, 0xfc, 0x8c, 0xc9, 0x10, 0xff, 0x40, 0x80, 0x2c, 0xad, 0x25, - 0xd3, 0x1c, 0xaa, 0x0e, 0x54, 0xf3, 0xfa, 0x99, 0x6a, 0x8e, 0xa8, 0x67, 0x05, 0xa6, 0x0f, 0x94, - 0x4e, 0x1f, 0xd3, 0x1a, 0x66, 0x1f, 0x96, 0x22, 0x64, 0xbc, 0x24, 0xf7, 0x25, 0x56, 0x0c, 0x3d, - 0x86, 0x9c, 0x6e, 0x38, 0xd8, 0x70, 0x64, 0xc6, 0x96, 0x3c, 0x83, 0x2d, 0xcb, 0x4a, 0xd3, 0x0b, - 0xf1, 0x9f, 0x0a, 0x00, 0x5b, 0xfd, 0x58, 0xf5, 0xfc, 0xcd, 0x09, 0xeb, 0xbf, 0x92, 0x22, 0xac, - 0x6e, 0x2b, 0x2e, 0xc1, 0x8c, 0x6e, 0x74, 0x74, 0x83, 0xd5, 0x3f, 0x2d, 0xf1, 0x2b, 0x74, 0x01, - 0xa6, 0x77, 0x3a, 0xba, 0xa1, 0xd1, 0xf9, 0x90, 0x96, 0xd8, 0x85, 0x28, 0x41, 0x96, 0xd6, 0x3a, - 0x46, 0xbd, 0x8b, 0x27, 0x09, 0xb8, 0x58, 0x35, 0x0d, 0x4d, 0x27, 0x53, 0x52, 0xe9, 0x7c, 0x2d, - 0xb4, 0xb2, 0x06, 0x17, 0x34, 0xdc, 0xb3, 0xb0, 0xaa, 0x38, 0x58, 0x93, 0xf1, 0x51, 0x6f, 0xc2, - 0x3e, 0x46, 0x3e, 0x57, 0xed, 0xa8, 0x47, 0x69, 0xd1, 0x9a, 0x44, 0xdf, 0x84, 0xcb, 0x4a, 0xa7, - 0x63, 0x1e, 0xca, 0xfa, 0xae, 0xac, 0x99, 0xd8, 0x96, 0x0d, 0xd3, 0x91, 0xf1, 0x91, 0x6e, 0x3b, - 0xd4, 0x94, 0xa4, 0xa5, 0x05, 0x7a, 0xbb, 0xb1, 0xbb, 0x6a, 0x62, 0x7b, 0xd3, 0x74, 0x6a, 0xe4, - 0x16, 0xb1, 0x00, 0xa4, 0x32, 0xcc, 0x02, 0xcc, 0x10, 0xf3, 0x2b, 0xa5, 0xf1, 0x51, 0x8f, 0x5a, - 0x80, 0x40, 0x57, 0xce, 0x06, 0xbb, 0x52, 0xfc, 0x25, 0xb8, 0x34, 0xa8, 0xdf, 0x38, 0xfb, 0xef, - 0x8f, 0x04, 0xc8, 0x37, 0x0c, 0xdd, 0xf9, 0x5a, 0x74, 0x9c, 0xa7, 0xec, 0x64, 0x50, 0xd9, 0xf7, - 0xa1, 0xb8, 0xab, 0xe8, 0x9d, 0xe7, 0x46, 0xcb, 0xec, 0xee, 0xd8, 0x8e, 0x69, 0x60, 0x9b, 0xf7, - 0xc6, 0x10, 0x5d, 0x7c, 0x09, 0x05, 0xaf, 0x35, 0x71, 0xaa, 0xc9, 0x81, 0x62, 0xc3, 0x50, 0x2d, - 0xdc, 0xc5, 0x46, 0xac, 0x7a, 0x7a, 0x1b, 0x32, 0xba, 0x2b, 0x97, 0xea, 0x2a, 0x29, 0xf9, 0x04, - 0xb1, 0x0f, 0xf3, 0x81, 0xa7, 0xc6, 0x69, 0x2e, 0xc9, 0x62, 0x84, 0x0f, 0x65, 0xbf, 0x8f, 0xc8, - 0x62, 0x84, 0x0f, 0x99, 0x79, 0x6b, 0xc2, 0xdc, 0x2a, 0xee, 0x60, 0x07, 0xc7, 0xd8, 0x52, 0x71, - 0x1b, 0xf2, 0xae, 0xd0, 0x38, 0x3b, 0xe6, 0x37, 0x04, 0x40, 0x5c, 0xae, 0x62, 0xb4, 0xe3, 0xac, - 0x31, 0x5a, 0x24, 0xae, 0x85, 0xd3, 0xb7, 0x0c, 0xe6, 0x23, 0xb0, 0x31, 0x09, 0x8c, 0x44, 0xdd, - 0x04, 0x7f, 0xca, 0xa6, 0x82, 0x53, 0x96, 0xbb, 0x37, 0x87, 0xb0, 0x10, 0xaa, 0x58, 0xbc, 0xdd, - 0x97, 0xa2, 0x75, 0x4a, 0x5c, 0x4b, 0x06, 0x7d, 0x38, 0x4a, 0x14, 0x7f, 0x20, 0xc0, 0x7c, 0xb5, - 0x83, 0x15, 0x2b, 0x76, 0x8d, 0x7c, 0x0b, 0xd2, 0x1a, 0x56, 0x34, 0xda, 0x64, 0x36, 0xb1, 0xdf, - 0x09, 0x48, 0x21, 0x9e, 0x6e, 0x65, 0xaf, 0xa3, 0x56, 0x5a, 0xae, 0x0f, 0xcc, 0x67, 0xb7, 0xc7, - 0x24, 0x7e, 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0xef, 0x26, 0x00, 0x49, 0xf8, 0x00, 0x5b, - 0x4e, 0xec, 0xcd, 0x5e, 0x85, 0xac, 0xa3, 0x58, 0x6d, 0xec, 0xc8, 0xc4, 0xbb, 0x3f, 0x4f, 0xcb, - 0x81, 0xf1, 0x11, 0x32, 0x6a, 0xc1, 0x1d, 0x6c, 0x28, 0x3b, 0x1d, 0x4c, 0xa5, 0xc8, 0x3b, 0x66, - 0xdf, 0xd0, 0x64, 0xdd, 0xc1, 0x96, 0xe2, 0x98, 0x96, 0x6c, 0xf6, 0x1c, 0xbd, 0xab, 0x7f, 0x41, - 0x1d, 0x7b, 0x3e, 0xd4, 0x6e, 0xb0, 0xe2, 0x84, 0x79, 0x85, 0x14, 0x6e, 0xf0, 0xb2, 0xcf, 0x03, - 0x45, 0x51, 0x05, 0x16, 0xf4, 0xb6, 0x61, 0x5a, 0x58, 0x6e, 0xab, 0xb2, 0xb3, 0x67, 0x61, 0x7b, - 0xcf, 0xec, 0xb8, 0xab, 0xd5, 0x3c, 0xbb, 0xf5, 0x54, 0x6d, 0xb9, 0x37, 0xc4, 0xcf, 0x61, 0x21, - 0xa4, 0xa5, 0x38, 0xbb, 0xe0, 0x7f, 0x08, 0x90, 0x6d, 0xaa, 0x8a, 0x11, 0xa7, 0xee, 0x3f, 0x85, - 0xac, 0xad, 0x2a, 0x86, 0xbc, 0x6b, 0x5a, 0x5d, 0xc5, 0xa1, 0xed, 0xca, 0x87, 0x74, 0xef, 0xf9, - 0xf7, 0xaa, 0x62, 0x3c, 0xa1, 0x85, 0x24, 0xb0, 0xbd, 0xdf, 0x83, 0xfe, 0xeb, 0xf4, 0x57, 0xf7, - 0x5f, 0xd9, 0xf4, 0x5e, 0x4b, 0xa5, 0x93, 0xc5, 0x94, 0xf8, 0x97, 0x02, 0xe4, 0x58, 0x93, 0xe3, - 0x9c, 0xde, 0x1f, 0x40, 0xca, 0x32, 0x0f, 0xd9, 0xf4, 0xce, 0x3e, 0x7c, 0x2b, 0x42, 0xc4, 0x3a, - 0x3e, 0x0e, 0xae, 0x9f, 0xb4, 0x38, 0x5a, 0x01, 0xee, 0xa5, 0xca, 0x94, 0x3b, 0x39, 0x29, 0x37, - 0x30, 0x2e, 0x89, 0xc8, 0xb8, 0x03, 0x85, 0x1d, 0xc5, 0x51, 0xf7, 0x64, 0x8b, 0x57, 0x92, 0xac, - 0xb5, 0xc9, 0xbb, 0x39, 0x29, 0x4f, 0xc9, 0x6e, 0xd5, 0x6d, 0xd2, 0x72, 0x36, 0xdf, 0x6c, 0xfc, - 0x57, 0xac, 0xcf, 0xff, 0xb7, 0xc0, 0xe7, 0x90, 0xdb, 0xf2, 0xbf, 0x6a, 0x5d, 0xff, 0x9b, 0x09, - 0xb8, 0x5c, 0xdd, 0xc3, 0xea, 0x7e, 0xd5, 0x34, 0x6c, 0xdd, 0x76, 0x88, 0xee, 0xe2, 0xec, 0xff, - 0xb7, 0x20, 0x73, 0xa8, 0x3b, 0x7b, 0xb2, 0xa6, 0xef, 0xee, 0x52, 0x6b, 0x9b, 0x96, 0xd2, 0x84, - 0xb0, 0xaa, 0xef, 0xee, 0xa2, 0x47, 0x90, 0xea, 0x9a, 0x1a, 0x73, 0xe6, 0xf3, 0x0f, 0x17, 0x23, - 0xc4, 0xd3, 0xaa, 0xd9, 0xfd, 0xee, 0x86, 0xa9, 0x61, 0x89, 0x16, 0x46, 0x57, 0x01, 0x54, 0x42, - 0xed, 0x99, 0xba, 0xe1, 0x70, 0xe3, 0x18, 0xa0, 0xa0, 0x3a, 0x64, 0x1c, 0x6c, 0x75, 0x75, 0x43, - 0x71, 0x70, 0x69, 0x9a, 0x2a, 0xef, 0x66, 0x64, 0xc5, 0x7b, 0x1d, 0x5d, 0x55, 0x56, 0xb1, 0xad, - 0x5a, 0x7a, 0xcf, 0x31, 0x2d, 0xae, 0x45, 0x9f, 0x59, 0xfc, 0x5b, 0x29, 0x28, 0x0d, 0xeb, 0x26, - 0xce, 0x11, 0xb2, 0x05, 0x33, 0x16, 0xb6, 0xfb, 0x1d, 0x87, 0x8f, 0x91, 0x87, 0xa3, 0x54, 0x10, - 0x51, 0x03, 0xba, 0x3f, 0xd1, 0x71, 0x78, 0xb5, 0xb9, 0x9c, 0xf2, 0xbf, 0x16, 0x60, 0x86, 0xdd, - 0x40, 0x0f, 0x20, 0x6d, 0x91, 0x85, 0x41, 0xd6, 0x35, 0x5a, 0xc7, 0xe4, 0xca, 0xa5, 0xd3, 0x93, - 0xc5, 0x59, 0xba, 0x58, 0x34, 0x56, 0xbf, 0xf4, 0x7f, 0x4a, 0xb3, 0xb4, 0x5c, 0x43, 0x23, 0xbd, - 0x65, 0x3b, 0x8a, 0xe5, 0xd0, 0x4d, 0xa5, 0x04, 0x43, 0x35, 0x94, 0xb0, 0x8e, 0x8f, 0xd1, 0x1a, - 0xcc, 0xd8, 0x8e, 0xe2, 0xf4, 0x6d, 0xde, 0x5f, 0xe7, 0xaa, 0x6c, 0x93, 0x72, 0x4a, 0x5c, 0x02, - 0x71, 0xb7, 0x34, 0xec, 0x28, 0x7a, 0x87, 0x76, 0x60, 0x46, 0xe2, 0x57, 0xe2, 0x6f, 0x09, 0x30, - 0xc3, 0x8a, 0xa2, 0xcb, 0xb0, 0x20, 0x2d, 0x6f, 0x3e, 0xad, 0xc9, 0x8d, 0xcd, 0xd5, 0x5a, 0xab, - 0x26, 0x6d, 0x34, 0x36, 0x97, 0x5b, 0xb5, 0xe2, 0x14, 0xba, 0x04, 0xc8, 0xbd, 0x51, 0x7d, 0xbe, - 0xd9, 0x6c, 0x34, 0x5b, 0xb5, 0xcd, 0x56, 0x51, 0xa0, 0x7b, 0x2a, 0x94, 0x1e, 0xa0, 0x26, 0xd0, - 0x4d, 0xb8, 0x36, 0x48, 0x95, 0x9b, 0xad, 0xe5, 0x56, 0x53, 0xae, 0x35, 0x5b, 0x8d, 0x8d, 0xe5, - 0x56, 0x6d, 0xb5, 0x98, 0x1c, 0x53, 0x8a, 0x3c, 0x44, 0x92, 0x6a, 0xd5, 0x56, 0x31, 0x25, 0x3a, - 0x70, 0x51, 0xc2, 0xaa, 0xd9, 0xed, 0xf5, 0x1d, 0x4c, 0x6a, 0x69, 0xc7, 0x39, 0x53, 0x2e, 0xc3, - 0xac, 0x66, 0x1d, 0xcb, 0x56, 0xdf, 0xe0, 0xf3, 0x64, 0x46, 0xb3, 0x8e, 0xa5, 0xbe, 0x21, 0xfe, - 0x63, 0x01, 0x2e, 0x0d, 0x3e, 0x36, 0xce, 0x41, 0xf8, 0x02, 0xb2, 0x8a, 0xa6, 0x61, 0x4d, 0xd6, - 0x70, 0xc7, 0x51, 0xb8, 0x4b, 0x74, 0x3f, 0x20, 0x89, 0x6f, 0x05, 0x56, 0xbc, 0xad, 0xc0, 0x8d, - 0x97, 0xd5, 0x2a, 0xad, 0xc8, 0x2a, 0xe1, 0x70, 0xcd, 0x0f, 0x15, 0x42, 0x29, 0xe2, 0x7f, 0x4b, - 0xc1, 0x5c, 0xcd, 0xd0, 0x5a, 0x47, 0xb1, 0xae, 0x25, 0x97, 0x60, 0x46, 0x35, 0xbb, 0x5d, 0xdd, - 0x71, 0x15, 0xc4, 0xae, 0xd0, 0xcf, 0x05, 0x5c, 0xd9, 0xe4, 0x04, 0x0e, 0x9d, 0xef, 0xc4, 0xa2, - 0xef, 0xc0, 0x65, 0x62, 0x35, 0x2d, 0x43, 0xe9, 0xc8, 0x4c, 0x9a, 0xec, 0x58, 0x7a, 0xbb, 0x8d, - 0x2d, 0xbe, 0xfd, 0x78, 0x37, 0xa2, 0x9e, 0x0d, 0xce, 0x51, 0xa5, 0x0c, 0x2d, 0x56, 0x5e, 0xba, - 0xa8, 0x47, 0x91, 0xd1, 0x27, 0x00, 0x64, 0x29, 0xa2, 0x5b, 0x9a, 0x36, 0xb7, 0x47, 0xa3, 0xf6, - 0x34, 0x5d, 0x13, 0x44, 0x18, 0xc8, 0xb5, 0x8d, 0x96, 0x08, 0x6e, 0x79, 0xdd, 0xd7, 0x2d, 0x2c, - 0x3f, 0xe8, 0xa9, 0x74, 0xa3, 0x21, 0xbd, 0x92, 0x3f, 0x3d, 0x59, 0x04, 0x89, 0x91, 0x1f, 0x6c, - 0x55, 0x09, 0x8e, 0x61, 0xbf, 0x7b, 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0xfb, 0x25, 0x64, 0xe5, 0xe5, - 0xcd, 0x52, 0x1c, 0x79, 0x4f, 0x6f, 0xef, 0x61, 0x4b, 0xf6, 0xb6, 0xb5, 0xe9, 0xce, 0x65, 0x5a, - 0xba, 0xe9, 0x33, 0x54, 0x15, 0x83, 0xd5, 0x7e, 0xd9, 0xa9, 0xd3, 0xc2, 0x9e, 0xce, 0x88, 0xf2, - 0x7b, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, 0x94, 0xcf, 0xae, 0xd0, 0x3d, 0x28, 0x3a, 0x47, 0x86, - 0xbc, 0x87, 0x15, 0xcb, 0xd9, 0xc1, 0x8a, 0x43, 0x56, 0x69, 0xa0, 0x25, 0x0a, 0xce, 0x91, 0x51, - 0x0f, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, 0xbc, 0xdb, 0xd1, 0xdb, 0x7b, 0x8e, 0x7c, 0x68, 0xe9, - 0x0e, 0xb6, 0x4b, 0xf3, 0x54, 0x21, 0x51, 0xe3, 0xb6, 0xc9, 0xf7, 0x99, 0xb5, 0x57, 0xa4, 0x24, - 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd7, 0x52, 0xe9, 0xd9, 0x62, 0x5a, 0xfc, - 0x4f, 0x02, 0xe4, 0xdd, 0xe1, 0x16, 0xe7, 0xcc, 0xb8, 0x0b, 0x45, 0xd3, 0xc0, 0x72, 0x6f, 0x4f, - 0xb1, 0x31, 0xd7, 0x23, 0x5f, 0x70, 0xf2, 0xa6, 0x81, 0xb7, 0x08, 0x99, 0xa9, 0x0b, 0x6d, 0xc1, - 0xbc, 0xed, 0x28, 0x6d, 0xdd, 0x68, 0x07, 0xd4, 0x3b, 0x3d, 0x39, 0xb8, 0x28, 0x72, 0x6e, 0x8f, - 0x1e, 0xf2, 0x52, 0xfe, 0x44, 0x80, 0xf9, 0x65, 0xad, 0xab, 0x1b, 0xcd, 0x5e, 0x47, 0x8f, 0x75, - 0xcf, 0xe2, 0x26, 0x64, 0x6c, 0x22, 0xd3, 0x37, 0xf8, 0x3e, 0x02, 0x4d, 0xd3, 0x3b, 0xc4, 0xf2, - 0x3f, 0x83, 0x02, 0x3e, 0xea, 0xe9, 0xec, 0x55, 0x05, 0x03, 0x4e, 0xa9, 0xc9, 0xdb, 0x96, 0xf7, - 0x79, 0xc9, 0x2d, 0xde, 0xa6, 0xcf, 0x00, 0x05, 0x9b, 0x14, 0x27, 0x76, 0xf9, 0x0c, 0x16, 0xa8, - 0xe8, 0x6d, 0xc3, 0x8e, 0x59, 0x5f, 0xe2, 0x2f, 0xc2, 0x85, 0xb0, 0xe8, 0x38, 0xeb, 0xfd, 0x8a, - 0xf7, 0xf2, 0x06, 0xb6, 0x62, 0x05, 0xbd, 0x9e, 0xae, 0xb9, 0xe0, 0x38, 0xeb, 0xfc, 0xab, 0x02, - 0x5c, 0xa1, 0xb2, 0xe9, 0xdb, 0x9c, 0x5d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, 0x2b, 0x62, 0xbf, 0x01, - 0x33, 0x0c, 0x79, 0xd3, 0xf1, 0x39, 0xbd, 0x92, 0x25, 0x9e, 0x4b, 0xd3, 0x31, 0x2d, 0xe2, 0xb9, - 0xf0, 0x5b, 0xa2, 0x02, 0xe5, 0xa8, 0x5a, 0xc4, 0xd9, 0xd2, 0xbf, 0x2b, 0xc0, 0x3c, 0x77, 0x1a, - 0xc9, 0x50, 0xae, 0xee, 0x11, 0x9f, 0x09, 0xd5, 0x20, 0xab, 0xd2, 0x5f, 0xb2, 0x73, 0xdc, 0xc3, - 0x54, 0x7e, 0x7e, 0x9c, 0xbf, 0xc9, 0xd8, 0x5a, 0xc7, 0x3d, 0x4c, 0x9c, 0x56, 0xf7, 0x37, 0x51, - 0x54, 0xa0, 0x91, 0x63, 0x3d, 0x56, 0x3a, 0x8f, 0x68, 0x59, 0xd7, 0xf5, 0xe3, 0x3a, 0xf8, 0x27, - 0x49, 0xae, 0x04, 0xf6, 0x0c, 0x5e, 0x3c, 0x56, 0x1f, 0xe5, 0x73, 0xb8, 0x14, 0x5c, 0x5d, 0x02, - 0x0d, 0x4f, 0x9c, 0xa3, 0xe1, 0x81, 0x1d, 0x7d, 0x9f, 0x8a, 0x3e, 0x83, 0xc0, 0x9e, 0xbd, 0xcc, - 0xda, 0xe4, 0xa2, 0x9f, 0xf3, 0xa8, 0x63, 0xde, 0x97, 0xc2, 0xe8, 0x36, 0xaa, 0x42, 0x1a, 0x1f, - 0xf5, 0x64, 0x0d, 0xdb, 0x2a, 0x37, 0x5c, 0x62, 0x94, 0x40, 0x52, 0x95, 0x21, 0x3c, 0x30, 0x8b, - 0x8f, 0x7a, 0x84, 0x88, 0xb6, 0xc9, 0xea, 0xe5, 0xba, 0x0a, 0xb4, 0xda, 0xf6, 0xd9, 0xf0, 0xc2, - 0x1f, 0x29, 0x5c, 0x5c, 0xc1, 0xf3, 0x12, 0x98, 0x08, 0xf1, 0x87, 0x02, 0xbc, 0x15, 0xd9, 0x6b, - 0x71, 0x2e, 0x64, 0x9f, 0x40, 0x8a, 0x36, 0x3e, 0x71, 0xce, 0xc6, 0x53, 0x2e, 0xf1, 0x7b, 0x09, - 0x3e, 0xc7, 0x25, 0xdc, 0x31, 0x89, 0x62, 0x63, 0xdf, 0x95, 0x7b, 0x0e, 0x73, 0x07, 0xa6, 0x43, - 0x7c, 0x13, 0xde, 0xed, 0x89, 0x73, 0x77, 0x7b, 0x8e, 0x0a, 0x70, 0x7b, 0xfc, 0x25, 0xcc, 0x1b, - 0xa6, 0x21, 0x87, 0x85, 0x9e, 0x7f, 0x2c, 0x15, 0x0c, 0xd3, 0x78, 0x19, 0x90, 0xeb, 0xd9, 0x99, - 0x01, 0x4d, 0xc4, 0x69, 0x67, 0xbe, 0x2f, 0xc0, 0x82, 0xe7, 0x36, 0xc5, 0xec, 0x41, 0x7f, 0x00, - 0x49, 0xc3, 0x3c, 0x3c, 0xcf, 0xae, 0x27, 0x29, 0x4f, 0x56, 0xbd, 0x70, 0x8d, 0xe2, 0x6c, 0xef, - 0xbf, 0x49, 0x40, 0xe6, 0x69, 0x35, 0xce, 0x56, 0x7e, 0xc2, 0x77, 0xd4, 0x59, 0x7f, 0x47, 0x8d, - 0x76, 0xef, 0x79, 0x95, 0xa7, 0xd5, 0x75, 0x7c, 0xec, 0x8e, 0x76, 0xc2, 0x85, 0x96, 0x21, 0x13, - 0xde, 0x7b, 0x9d, 0x50, 0x53, 0x3e, 0x57, 0x19, 0xc3, 0x34, 0x95, 0xeb, 0x46, 0x6f, 0x08, 0x11, - 0xd1, 0x1b, 0xe4, 0x31, 0x9e, 0xa7, 0x98, 0x38, 0xcf, 0x63, 0x02, 0x2e, 0xe2, 0x74, 0x71, 0x46, - 0x7c, 0x01, 0x40, 0x9a, 0x13, 0x67, 0x97, 0xfc, 0x5a, 0x12, 0xf2, 0x5b, 0x7d, 0x7b, 0x2f, 0xe6, - 0xd1, 0x57, 0x05, 0xe8, 0xf5, 0x6d, 0x0a, 0x41, 0x8e, 0x0c, 0xde, 0xe6, 0x33, 0x02, 0x43, 0xdc, - 0x46, 0x33, 0xbe, 0xd6, 0x91, 0x81, 0xea, 0x5c, 0x08, 0x96, 0xfd, 0xe8, 0x92, 0x1b, 0xe3, 0xc0, - 0x6a, 0xeb, 0xc8, 0xd8, 0xc0, 0x1e, 0x4a, 0x65, 0x92, 0x30, 0x91, 0xf4, 0x09, 0xcc, 0x92, 0x0b, - 0xd9, 0x31, 0xcf, 0xd3, 0xcd, 0x33, 0x84, 0xa7, 0x65, 0xa2, 0xc7, 0x90, 0x61, 0xdc, 0x64, 0xf5, - 0x9b, 0xa1, 0xab, 0x5f, 0x54, 0x5b, 0xb8, 0x1a, 0xe9, 0xba, 0x97, 0xa6, 0xac, 0x64, 0xad, 0xbb, - 0x00, 0xd3, 0xbb, 0xa6, 0xa5, 0xba, 0xef, 0x87, 0xd9, 0x05, 0xeb, 0xcf, 0xb5, 0x54, 0x3a, 0x5d, - 0xcc, 0xac, 0xa5, 0xd2, 0x99, 0x22, 0x88, 0xbf, 0x25, 0x40, 0xc1, 0xeb, 0x88, 0x38, 0x17, 0x84, - 0x6a, 0x48, 0x8b, 0xe7, 0xef, 0x0a, 0xa2, 0x40, 0xf1, 0xdf, 0x52, 0x8f, 0x48, 0x35, 0x0f, 0x68, - 0xcf, 0xc4, 0x39, 0x52, 0x1e, 0xb3, 0xd8, 0xa1, 0xc4, 0x79, 0x7b, 0x97, 0x86, 0x11, 0x3d, 0x80, - 0x0b, 0x7a, 0x97, 0xd8, 0x73, 0xdd, 0xe9, 0x1c, 0x73, 0xd8, 0xe6, 0x60, 0xf7, 0x45, 0xf4, 0x82, - 0x7f, 0xaf, 0xea, 0xde, 0x12, 0x7f, 0x97, 0x6e, 0x80, 0xfb, 0x2d, 0x89, 0x53, 0xd5, 0x0d, 0x98, - 0xb3, 0x98, 0x68, 0xe2, 0xd6, 0x9c, 0x53, 0xdb, 0x39, 0x8f, 0x95, 0x28, 0xfc, 0x77, 0x12, 0x50, - 0x78, 0xd1, 0xc7, 0xd6, 0xf1, 0xd7, 0x49, 0xdd, 0xb7, 0xa1, 0x70, 0xa8, 0xe8, 0x8e, 0xbc, 0x6b, - 0x5a, 0x72, 0xbf, 0xa7, 0x29, 0x8e, 0x1b, 0xc0, 0x32, 0x47, 0xc8, 0x4f, 0x4c, 0x6b, 0x9b, 0x12, - 0x11, 0x06, 0xb4, 0x6f, 0x98, 0x87, 0x86, 0x4c, 0xc8, 0x14, 0x28, 0x1f, 0x19, 0x7c, 0x57, 0x7a, - 0xe5, 0xc3, 0xff, 0x78, 0xb2, 0xf8, 0x68, 0xa2, 0xb0, 0x34, 0x1a, 0x82, 0xd7, 0xef, 0xeb, 0x5a, - 0x65, 0x7b, 0xbb, 0xb1, 0x2a, 0x15, 0xa9, 0xc8, 0x57, 0x4c, 0x62, 0xeb, 0xc8, 0xb0, 0xc5, 0xbf, - 0x9f, 0x80, 0xa2, 0xaf, 0xa3, 0x38, 0x3b, 0xb2, 0x06, 0xd9, 0xd7, 0x7d, 0x6c, 0xe9, 0x6f, 0xd0, - 0x8d, 0xc0, 0x19, 0x89, 0xd9, 0xf9, 0x1c, 0x72, 0x21, 0x0d, 0x24, 0xbf, 0x9a, 0x06, 0xb2, 0x87, - 0x7e, 0xe3, 0xd1, 0x7d, 0x98, 0x77, 0x8e, 0x0c, 0x99, 0x05, 0x24, 0xb2, 0x20, 0x16, 0x37, 0xbe, - 0xa2, 0xe0, 0x10, 0x7d, 0x10, 0x3a, 0x0d, 0x60, 0xb1, 0xc5, 0x3f, 0x10, 0x00, 0x51, 0x45, 0x35, - 0xd8, 0x6b, 0x83, 0xaf, 0xcb, 0x78, 0xba, 0x0b, 0x45, 0x1a, 0xe2, 0x29, 0xeb, 0xbb, 0x72, 0x57, - 0xb7, 0x6d, 0xdd, 0x68, 0xf3, 0x01, 0x95, 0xa7, 0xf4, 0xc6, 0xee, 0x06, 0xa3, 0x8a, 0x7f, 0x0d, - 0x16, 0x42, 0x0d, 0x88, 0xb3, 0xb3, 0xaf, 0x43, 0x6e, 0x97, 0xbd, 0xd5, 0xa5, 0xc2, 0xf9, 0x8e, - 0x63, 0x96, 0xd2, 0xd8, 0xf3, 0xc4, 0xbf, 0x48, 0xc0, 0x05, 0x09, 0xdb, 0x66, 0xe7, 0x00, 0xc7, - 0xaf, 0xc2, 0x3a, 0xf0, 0xd7, 0x39, 0xf2, 0x1b, 0x69, 0x32, 0xc3, 0x98, 0xd9, 0x32, 0x17, 0xde, - 0xb6, 0xbf, 0x39, 0x7e, 0xc4, 0x0e, 0x6f, 0xd4, 0xf3, 0x6d, 0xbf, 0x54, 0x68, 0xdb, 0xcf, 0x84, - 0x02, 0x7b, 0x21, 0xad, 0xc9, 0x36, 0x7e, 0x6d, 0xf4, 0xbb, 0x2e, 0x18, 0xaa, 0x8c, 0xab, 0x64, - 0x83, 0xb1, 0x34, 0xf1, 0xeb, 0xcd, 0x7e, 0x97, 0xfa, 0xce, 0x2b, 0x97, 0x48, 0x7d, 0x4f, 0x4f, - 0x16, 0xf3, 0xa1, 0x7b, 0xb6, 0x94, 0xd7, 0xbd, 0x6b, 0x22, 0x5d, 0xfc, 0x36, 0x5c, 0x1c, 0x50, - 0x76, 0x9c, 0x1e, 0xcf, 0xbf, 0x4a, 0xc2, 0x95, 0xb0, 0xf8, 0xb8, 0x21, 0xce, 0xd7, 0xbd, 0x43, - 0xeb, 0x30, 0xd7, 0xd5, 0x8d, 0x37, 0xdb, 0xbd, 0xcc, 0x75, 0x75, 0xc3, 0xdf, 0x29, 0x8e, 0x18, - 0x1a, 0x33, 0x3f, 0xd3, 0xa1, 0xa1, 0x40, 0x39, 0xaa, 0xef, 0xe2, 0x1c, 0x1f, 0xdf, 0x13, 0x20, - 0x17, 0xf7, 0xb6, 0xdc, 0x9b, 0x05, 0xd6, 0x89, 0x2d, 0x98, 0xfb, 0x19, 0xec, 0xe3, 0xfd, 0x8e, - 0x00, 0xa8, 0x65, 0xf5, 0x0d, 0x02, 0x6a, 0x9f, 0x99, 0xed, 0x38, 0x9b, 0x79, 0x01, 0xa6, 0x75, - 0x43, 0xc3, 0x47, 0xb4, 0x99, 0x29, 0x89, 0x5d, 0x84, 0xde, 0x4e, 0x26, 0x27, 0x7a, 0x3b, 0x29, - 0x7e, 0x0e, 0x0b, 0xa1, 0x2a, 0xc6, 0xd9, 0xfe, 0xff, 0x9a, 0x80, 0x05, 0xde, 0x90, 0xd8, 0x77, - 0x30, 0xbf, 0x09, 0xd3, 0x1d, 0x22, 0x73, 0x4c, 0x3f, 0xd3, 0x67, 0xba, 0xfd, 0x4c, 0x0b, 0xa3, - 0x9f, 0x07, 0xe8, 0x59, 0xf8, 0x40, 0x66, 0xac, 0xc9, 0x89, 0x58, 0x33, 0x84, 0x83, 0x12, 0xd0, - 0x0f, 0x04, 0x28, 0x90, 0x09, 0xdd, 0xb3, 0xcc, 0x9e, 0x69, 0x13, 0x9f, 0xc5, 0x9e, 0x0c, 0xe6, - 0xbc, 0x38, 0x3d, 0x59, 0x9c, 0xdb, 0xd0, 0x8d, 0x2d, 0xce, 0xd8, 0x6a, 0x4e, 0x7c, 0x66, 0xc0, - 0x3d, 0x39, 0x51, 0xa9, 0x76, 0x4c, 0x75, 0xdf, 0x7f, 0xdf, 0x46, 0x2c, 0x8b, 0x27, 0xce, 0x16, - 0xff, 0x58, 0x80, 0x0b, 0x3f, 0xb3, 0xed, 0xe2, 0xff, 0x17, 0xca, 0x16, 0x5f, 0x42, 0x91, 0xfe, - 0x68, 0x18, 0xbb, 0x66, 0x9c, 0x1b, 0xf7, 0xff, 0x4b, 0x80, 0xf9, 0x80, 0xe0, 0x38, 0x1d, 0x9c, - 0x37, 0xd5, 0xd3, 0x1c, 0x8b, 0xb0, 0x71, 0x26, 0x53, 0x95, 0x94, 0xe3, 0xc5, 0xd9, 0xa0, 0xac, - 0x40, 0x0e, 0x13, 0x2b, 0x46, 0xb7, 0x78, 0x77, 0xd8, 0xb9, 0x95, 0x81, 0x1d, 0xfd, 0xac, 0x57, - 0x60, 0xe5, 0x58, 0xfc, 0x45, 0xe2, 0x61, 0x05, 0x27, 0x65, 0x9c, 0x53, 0xfe, 0x9f, 0x27, 0xe0, - 0x52, 0x95, 0xbd, 0x55, 0x77, 0xc3, 0x4c, 0xe2, 0x1c, 0x88, 0x25, 0x98, 0x3d, 0xc0, 0x96, 0xad, - 0x9b, 0x6c, 0xb5, 0x9f, 0x93, 0xdc, 0x4b, 0x54, 0x86, 0xb4, 0x6d, 0x28, 0x3d, 0x7b, 0xcf, 0x74, - 0x5f, 0x27, 0x7a, 0xd7, 0x5e, 0x48, 0xcc, 0xf4, 0x9b, 0x87, 0xc4, 0xcc, 0x8c, 0x0f, 0x89, 0x99, - 0xfd, 0x0a, 0x21, 0x31, 0xfc, 0xdd, 0xdd, 0xbf, 0x13, 0xe0, 0xf2, 0x90, 0xe6, 0xe2, 0x1c, 0x9c, - 0xdf, 0x85, 0xac, 0xca, 0x05, 0x93, 0xf5, 0x81, 0xbd, 0x98, 0x6c, 0x90, 0x62, 0x6f, 0x08, 0x93, - 0x4e, 0x4f, 0x16, 0xc1, 0xad, 0x6a, 0x63, 0x95, 0x2b, 0x87, 0xfc, 0xd6, 0xc4, 0x5f, 0x99, 0x83, - 0x42, 0xed, 0x88, 0x6d, 0xca, 0x37, 0x99, 0x57, 0x82, 0x9e, 0x40, 0xba, 0x67, 0x99, 0x07, 0xba, - 0xdb, 0x8c, 0x7c, 0x28, 0x1e, 0xc2, 0x6d, 0xc6, 0x00, 0xd7, 0x16, 0xe7, 0x90, 0x3c, 0x5e, 0xd4, - 0x82, 0xcc, 0x33, 0x53, 0x55, 0x3a, 0x4f, 0xf4, 0x8e, 0x3b, 0xd1, 0xde, 0x3f, 0x5b, 0x50, 0xc5, - 0xe3, 0xd9, 0x52, 0x9c, 0x3d, 0xb7, 0x13, 0x3c, 0x22, 0x6a, 0x40, 0xba, 0xee, 0x38, 0x3d, 0x72, - 0x93, 0xcf, 0xbf, 0x3b, 0x13, 0x08, 0x25, 0x2c, 0x6e, 0x10, 0xaf, 0xcb, 0x8e, 0x5a, 0x30, 0xff, - 0x94, 0x1e, 0x49, 0xab, 0x76, 0xcc, 0xbe, 0x56, 0x35, 0x8d, 0x5d, 0xbd, 0xcd, 0x97, 0x89, 0xdb, - 0x13, 0xc8, 0x7c, 0x5a, 0x6d, 0x4a, 0xc3, 0x02, 0xd0, 0x32, 0xa4, 0x9b, 0x8f, 0xb8, 0x30, 0xe6, - 0x46, 0xde, 0x9a, 0x40, 0x58, 0xf3, 0x91, 0xe4, 0xb1, 0xa1, 0x35, 0xc8, 0x2e, 0x7f, 0xd1, 0xb7, - 0x30, 0x97, 0x32, 0x33, 0x32, 0x18, 0x63, 0x50, 0x0a, 0xe5, 0x92, 0x82, 0xcc, 0xa8, 0x09, 0xf9, - 0x57, 0xa6, 0xb5, 0xdf, 0x31, 0x15, 0xb7, 0x85, 0xb3, 0x54, 0xdc, 0x37, 0x26, 0x10, 0xe7, 0x32, - 0x4a, 0x03, 0x22, 0xd0, 0xb7, 0xa1, 0x40, 0x3a, 0xa3, 0xa5, 0xec, 0x74, 0xdc, 0x4a, 0xa6, 0xa9, - 0xd4, 0x77, 0x27, 0x90, 0xea, 0x71, 0xba, 0xef, 0x19, 0x06, 0x44, 0x95, 0x25, 0x98, 0x0b, 0x0d, - 0x02, 0x84, 0x20, 0xd5, 0x23, 0xfd, 0x2d, 0xd0, 0x70, 0x29, 0xfa, 0x1b, 0xbd, 0x07, 0xb3, 0x86, - 0xa9, 0x61, 0x77, 0x86, 0xcc, 0xad, 0x5c, 0x38, 0x3d, 0x59, 0x9c, 0xd9, 0x34, 0x35, 0xe6, 0x40, - 0xf1, 0x5f, 0xd2, 0x0c, 0x29, 0xd4, 0xd0, 0xca, 0xd7, 0x20, 0x45, 0xfa, 0x9d, 0x18, 0xa6, 0x1d, - 0xc5, 0xc6, 0xdb, 0x96, 0xce, 0xa5, 0xb9, 0x97, 0xe5, 0x7f, 0x94, 0x80, 0x44, 0xf3, 0x11, 0x81, - 0x08, 0x3b, 0x7d, 0x75, 0x1f, 0x3b, 0xfc, 0x3e, 0xbf, 0xa2, 0xd0, 0xc1, 0xc2, 0xbb, 0x3a, 0xf3, - 0xe4, 0x32, 0x12, 0xbf, 0x42, 0xef, 0x00, 0x28, 0xaa, 0x8a, 0x6d, 0x5b, 0x76, 0x8f, 0x2a, 0x66, - 0xa4, 0x0c, 0xa3, 0xac, 0xe3, 0x63, 0xc2, 0x66, 0x63, 0xd5, 0xc2, 0x8e, 0x1b, 0xeb, 0xc5, 0xae, - 0x08, 0x9b, 0x83, 0xbb, 0x3d, 0xd9, 0x31, 0xf7, 0xb1, 0x41, 0xc7, 0x49, 0x86, 0x98, 0x9a, 0x6e, - 0xaf, 0x45, 0x08, 0xc4, 0x4a, 0x62, 0x43, 0xf3, 0x4d, 0x5a, 0x46, 0xf2, 0xae, 0x89, 0x48, 0x0b, - 0xb7, 0x75, 0x7e, 0x9a, 0x2f, 0x23, 0xf1, 0x2b, 0xa2, 0x25, 0xa5, 0xef, 0xec, 0xd1, 0x9e, 0xc8, - 0x48, 0xf4, 0x37, 0xba, 0x0d, 0x05, 0x16, 0x1e, 0x2a, 0x63, 0x43, 0x95, 0xa9, 0x71, 0xcd, 0xd0, - 0xdb, 0x73, 0x8c, 0x5c, 0x33, 0x54, 0x62, 0x4a, 0xd1, 0x23, 0xe0, 0x04, 0x79, 0xbf, 0x6b, 0x13, - 0x9d, 0x02, 0x29, 0xb5, 0x52, 0x38, 0x3d, 0x59, 0xcc, 0x36, 0xe9, 0x8d, 0xf5, 0x8d, 0x26, 0x59, - 0xa0, 0x58, 0xa9, 0xf5, 0xae, 0xdd, 0xd0, 0xca, 0xbf, 0x2e, 0x40, 0xf2, 0x69, 0xb5, 0x79, 0x6e, - 0x95, 0xb9, 0x15, 0x4d, 0x06, 0x2a, 0x7a, 0x07, 0x0a, 0x3b, 0x7a, 0xa7, 0xa3, 0x1b, 0x6d, 0xe2, - 0xb4, 0x7d, 0x17, 0xab, 0xae, 0xc2, 0xf2, 0x9c, 0xbc, 0xc5, 0xa8, 0xe8, 0x1a, 0x64, 0x55, 0x0b, - 0x6b, 0xd8, 0x70, 0x74, 0xa5, 0x63, 0x73, 0xcd, 0x05, 0x49, 0xe5, 0x5f, 0x16, 0x60, 0x9a, 0xce, - 0x00, 0xf4, 0x36, 0x64, 0x54, 0xd3, 0x70, 0x14, 0xdd, 0xe0, 0xa6, 0x2c, 0x23, 0xf9, 0x84, 0x91, - 0xd5, 0xbb, 0x0e, 0x39, 0x45, 0x55, 0xcd, 0xbe, 0xe1, 0xc8, 0x86, 0xd2, 0xc5, 0xbc, 0x9a, 0x59, - 0x4e, 0xdb, 0x54, 0xba, 0x18, 0x2d, 0x82, 0x7b, 0xe9, 0x9d, 0x40, 0xcd, 0x48, 0xc0, 0x49, 0xeb, - 0xf8, 0xb8, 0xfc, 0x87, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, 0xd1, 0xc6, 0x06, 0x8b, 0x79, 0x77, - 0xab, 0xe1, 0x11, 0x06, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0x5e, 0x80, 0x69, 0x87, 0x4c, 0x0b, 0x5e, - 0x03, 0x76, 0x41, 0xb7, 0xcf, 0x3b, 0x4a, 0x9b, 0xed, 0x1e, 0x66, 0x24, 0x76, 0x41, 0x1a, 0xc3, - 0xa3, 0x8c, 0x99, 0x46, 0xf8, 0x15, 0xa9, 0x29, 0x8b, 0x85, 0xdd, 0xc1, 0x6d, 0xdd, 0xa0, 0x63, - 0x29, 0x29, 0x01, 0x25, 0xad, 0x10, 0x0a, 0x7a, 0x0b, 0x32, 0xac, 0x00, 0x36, 0x34, 0x3a, 0xa0, - 0x92, 0x52, 0x9a, 0x12, 0x6a, 0x86, 0x56, 0xc6, 0x90, 0xf1, 0x26, 0x27, 0xe9, 0xb6, 0xbe, 0xed, - 0x29, 0x92, 0xfe, 0x46, 0xef, 0xc3, 0x85, 0xd7, 0x7d, 0xa5, 0xa3, 0xef, 0xd2, 0x8d, 0x41, 0x7a, - 0x28, 0x80, 0xea, 0x8c, 0xb5, 0x04, 0x79, 0xf7, 0xa8, 0x04, 0xaa, 0x3a, 0x77, 0x2e, 0x27, 0xfd, - 0xb9, 0x2c, 0xfe, 0x9e, 0x00, 0xf3, 0x2c, 0xd0, 0x89, 0xc5, 0xe7, 0xc6, 0xe7, 0x87, 0x7c, 0x0c, - 0x19, 0x4d, 0x71, 0x14, 0x76, 0xa6, 0x36, 0x31, 0xf6, 0x4c, 0xad, 0x77, 0xc6, 0x43, 0x71, 0x14, - 0x7a, 0xae, 0x16, 0x41, 0x8a, 0xfc, 0x66, 0xc7, 0x8f, 0x25, 0xfa, 0x5b, 0xfc, 0x0c, 0x50, 0xb0, - 0xa2, 0x71, 0x7a, 0x64, 0xf7, 0xe0, 0x22, 0xd1, 0x75, 0xcd, 0x50, 0xad, 0xe3, 0x9e, 0xa3, 0x9b, - 0xc6, 0x73, 0xfa, 0xd7, 0x46, 0xc5, 0xc0, 0x7b, 0x34, 0xfa, 0xfa, 0x4c, 0xfc, 0xfd, 0x19, 0x98, - 0xab, 0x1d, 0xf5, 0x4c, 0x2b, 0xd6, 0x5d, 0xb7, 0x15, 0x98, 0xe5, 0x1b, 0x13, 0x63, 0x5e, 0x95, - 0x0f, 0x18, 0x73, 0x37, 0x4e, 0x80, 0x33, 0xa2, 0x15, 0x00, 0x16, 0x43, 0x4b, 0xe3, 0xa4, 0x92, - 0xe7, 0x78, 0xb3, 0x47, 0xd9, 0xe8, 0xf9, 0x92, 0x4d, 0xc8, 0x76, 0x0f, 0x54, 0x55, 0xde, 0xd5, - 0x3b, 0x0e, 0x0f, 0x45, 0x8c, 0x8e, 0x9a, 0xdf, 0x78, 0x59, 0xad, 0x3e, 0xa1, 0x85, 0x58, 0x54, - 0xa0, 0x7f, 0x2d, 0x01, 0x91, 0xc0, 0x7e, 0xa3, 0x77, 0x81, 0x9f, 0x75, 0x92, 0x6d, 0xf7, 0x58, - 0xe3, 0xca, 0xdc, 0xe9, 0xc9, 0x62, 0x46, 0xa2, 0xd4, 0x66, 0xb3, 0x25, 0x65, 0x58, 0x81, 0xa6, - 0xed, 0xa0, 0x1b, 0x30, 0x67, 0x76, 0x75, 0x47, 0x76, 0x9d, 0x24, 0xee, 0x51, 0xe6, 0x08, 0xd1, - 0x75, 0xa2, 0xce, 0x73, 0x04, 0x66, 0x76, 0xf2, 0x23, 0x30, 0x7f, 0x53, 0x80, 0x4b, 0x5c, 0x91, - 0xf2, 0x0e, 0x0d, 0xfb, 0x57, 0x3a, 0xba, 0x73, 0x2c, 0xef, 0x1f, 0x94, 0xd2, 0xd4, 0x6f, 0xfd, - 0xb9, 0xc8, 0x0e, 0x09, 0x8c, 0x83, 0x8a, 0xdb, 0x2d, 0xc7, 0xcf, 0x38, 0xf3, 0xfa, 0x41, 0xcd, - 0x70, 0xac, 0xe3, 0x95, 0xcb, 0xa7, 0x27, 0x8b, 0x0b, 0xc3, 0x77, 0x5f, 0x4a, 0x0b, 0xf6, 0x30, - 0x0b, 0xaa, 0x03, 0x60, 0x6f, 0x1c, 0xd2, 0x15, 0x23, 0xda, 0xff, 0x88, 0x1c, 0xb0, 0x52, 0x80, - 0x17, 0xdd, 0x85, 0x22, 0x3f, 0x72, 0xb4, 0xab, 0x77, 0xb0, 0x6c, 0xeb, 0x5f, 0x60, 0xba, 0xb6, - 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x4d, 0xfd, 0x0b, 0x5c, 0xfe, 0x2e, 0x94, 0x46, 0xd5, 0x3e, - 0x38, 0x05, 0x32, 0xec, 0x0d, 0xf2, 0x47, 0xe1, 0xed, 0xa3, 0x09, 0x86, 0x2a, 0xdf, 0x42, 0xfa, - 0x38, 0xf1, 0x91, 0x20, 0xfe, 0x83, 0x04, 0xcc, 0xad, 0xf4, 0x3b, 0xfb, 0xcf, 0x7b, 0x4d, 0x96, - 0x7b, 0x81, 0x98, 0x41, 0x66, 0x28, 0x48, 0x05, 0x05, 0x66, 0x06, 0xa9, 0x25, 0xd0, 0xbf, 0xc0, - 0x64, 0x71, 0x0a, 0x44, 0xe7, 0xf0, 0x63, 0x0d, 0xb4, 0x0d, 0x3e, 0x99, 0x9e, 0x3c, 0xf8, 0x08, - 0x4a, 0x81, 0x82, 0x74, 0xaf, 0x47, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0xed, 0x57, 0x26, 0xa5, 0x40, - 0x08, 0x51, 0x83, 0xdc, 0xae, 0xb1, 0xbb, 0xa8, 0x05, 0x39, 0x52, 0xf0, 0x58, 0xa6, 0x4b, 0x88, - 0xbb, 0x9f, 0xfc, 0x20, 0xa2, 0x59, 0xa1, 0x7a, 0x57, 0xa8, 0x7e, 0xaa, 0x94, 0x87, 0xfe, 0x94, - 0xb2, 0xd8, 0xa7, 0x94, 0x3f, 0x85, 0xe2, 0x60, 0x81, 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0x17, 0x82, - 0xba, 0x4c, 0x06, 0xf4, 0xb4, 0x96, 0x4a, 0xa7, 0x8a, 0xd3, 0xe2, 0x9f, 0x27, 0x21, 0xef, 0x0e, - 0xb3, 0x38, 0x81, 0xce, 0x0a, 0x4c, 0x93, 0x41, 0xe1, 0x06, 0xbc, 0xdc, 0x1e, 0x33, 0xba, 0x79, - 0x20, 0x3d, 0x19, 0x2c, 0x2e, 0x26, 0xa7, 0xac, 0x71, 0x18, 0x9c, 0xf2, 0x2f, 0x27, 0x20, 0x45, - 0xb1, 0xc5, 0x03, 0x48, 0xd1, 0x85, 0x42, 0x98, 0x64, 0xa1, 0xa0, 0x45, 0xbd, 0xe5, 0x2c, 0x11, - 0x70, 0x4d, 0x89, 0xcf, 0xb7, 0xa7, 0x7c, 0xf0, 0xe0, 0x21, 0x35, 0x36, 0x39, 0x89, 0x5f, 0xa1, - 0x15, 0x1a, 0x89, 0x65, 0x5a, 0x0e, 0xd6, 0xb8, 0x4f, 0x7f, 0xed, 0xac, 0xfe, 0x75, 0x17, 0x25, - 0x97, 0x0f, 0x5d, 0x81, 0x24, 0xb1, 0x62, 0xb3, 0x2c, 0xa8, 0xe2, 0xf4, 0x64, 0x31, 0x49, 0xec, - 0x17, 0xa1, 0xa1, 0x25, 0xc8, 0x86, 0x4d, 0x06, 0xf1, 0xe0, 0xa8, 0x61, 0x0c, 0x4c, 0x77, 0xe8, - 0x78, 0x53, 0x8b, 0xe1, 0x59, 0xde, 0xc7, 0xff, 0x3d, 0x05, 0x73, 0x8d, 0x6e, 0xdc, 0x4b, 0xca, - 0x72, 0xb8, 0x87, 0xa3, 0x80, 0x50, 0xe8, 0xa1, 0x11, 0x1d, 0x1c, 0x5a, 0xc1, 0x93, 0xe7, 0x5b, - 0xc1, 0x1b, 0xc4, 0x53, 0xe6, 0x79, 0x31, 0x92, 0x23, 0x30, 0x4f, 0xf8, 0xf9, 0xd4, 0x4f, 0x91, - 0x08, 0x8f, 0x7f, 0xb4, 0x84, 0x06, 0xc6, 0x7c, 0x4a, 0x1d, 0x72, 0x36, 0xca, 0x66, 0x26, 0x1f, - 0x65, 0xb3, 0xd8, 0xd0, 0xe8, 0xa2, 0x16, 0xb6, 0xa8, 0xb3, 0x6f, 0x6e, 0x51, 0xcb, 0x0e, 0x1f, - 0xac, 0x1f, 0x43, 0x52, 0xd3, 0xdd, 0xce, 0x99, 0x7c, 0xa9, 0x26, 0x4c, 0x67, 0x8c, 0xda, 0x54, - 0x70, 0xd4, 0xb2, 0x51, 0x52, 0x6e, 0x00, 0xf8, 0xba, 0x41, 0xd7, 0x60, 0xc6, 0xec, 0x68, 0xee, - 0xd9, 0x9a, 0xb9, 0x95, 0xcc, 0xe9, 0xc9, 0xe2, 0xf4, 0xf3, 0x8e, 0xd6, 0x58, 0x95, 0xa6, 0xcd, - 0x8e, 0xd6, 0xd0, 0x68, 0x52, 0x12, 0x7c, 0x28, 0x7b, 0x81, 0x77, 0x39, 0x69, 0xd6, 0xc0, 0x87, - 0xab, 0xd8, 0x56, 0xf9, 0x80, 0xfb, 0x6d, 0x01, 0xf2, 0xae, 0xee, 0xe3, 0x35, 0x2a, 0x69, 0xbd, - 0xcb, 0x27, 0x59, 0xf2, 0x7c, 0x93, 0xcc, 0xe5, 0xe3, 0xe7, 0x9e, 0x7f, 0x55, 0xe0, 0xa1, 0xd4, - 0x4d, 0x55, 0x71, 0x88, 0x53, 0x11, 0xe3, 0xc4, 0xb8, 0x07, 0x45, 0x4b, 0x31, 0x34, 0xb3, 0xab, - 0x7f, 0x81, 0xd9, 0x66, 0xa2, 0xcd, 0xdf, 0xb2, 0x16, 0x3c, 0x3a, 0xdd, 0xf5, 0xb3, 0xc5, 0x3f, - 0x4c, 0xf0, 0xb0, 0x6b, 0xaf, 0x1a, 0x71, 0xaa, 0xeb, 0x3b, 0x30, 0x1f, 0x5c, 0xda, 0x58, 0x64, - 0x27, 0x9b, 0xad, 0xef, 0x45, 0xc8, 0x8b, 0xaa, 0x08, 0x0b, 0x9f, 0x74, 0x63, 0xf9, 0x03, 0x2b, - 0x22, 0x15, 0x86, 0xaa, 0x90, 0xe5, 0xef, 0x3a, 0x8c, 0x5d, 0xd3, 0x8d, 0x4b, 0x7b, 0x7b, 0x54, - 0x14, 0x66, 0xc3, 0xd8, 0x35, 0xdd, 0xb8, 0x01, 0xcb, 0x25, 0xd8, 0xe5, 0x5f, 0x80, 0x69, 0x7a, - 0xfb, 0x0d, 0x4c, 0x34, 0xef, 0xcd, 0x3f, 0x4b, 0xc0, 0x4d, 0x5a, 0xfb, 0x97, 0xd8, 0xd2, 0x77, - 0x8f, 0xb7, 0x2c, 0xd3, 0xc1, 0xaa, 0x83, 0x35, 0x7f, 0xa3, 0x3e, 0x56, 0xbb, 0x97, 0xe9, 0xb9, - 0x0f, 0x38, 0x57, 0x7c, 0x9b, 0xc7, 0x85, 0xd6, 0xa1, 0xc0, 0x23, 0x19, 0x94, 0x8e, 0x7e, 0x80, - 0x65, 0xc5, 0x39, 0xcf, 0xea, 0x36, 0xc7, 0x78, 0x97, 0x09, 0xeb, 0xb2, 0x83, 0x34, 0xc8, 0x70, - 0x61, 0xba, 0xc6, 0xf3, 0x25, 0x3d, 0xfd, 0x6a, 0x1b, 0x8a, 0x69, 0x16, 0x4e, 0xd1, 0x58, 0x95, - 0xd2, 0x4c, 0x72, 0x43, 0x13, 0xff, 0x76, 0x0a, 0x6e, 0x9d, 0xa1, 0xe2, 0x38, 0x87, 0x6e, 0x19, - 0xd2, 0x07, 0xe4, 0x41, 0x3a, 0xd7, 0x71, 0x5a, 0xf2, 0xae, 0xd1, 0x4e, 0xc8, 0x11, 0xdb, 0x55, - 0xf4, 0x8e, 0x3f, 0xba, 0x47, 0x47, 0x46, 0x46, 0xc7, 0x01, 0x07, 0x5c, 0xb6, 0x27, 0x54, 0x10, - 0x1f, 0xd8, 0xdf, 0x13, 0xa0, 0xcc, 0x1e, 0xc8, 0x82, 0x67, 0x07, 0x1e, 0x93, 0xa2, 0x8f, 0x59, - 0x1d, 0x35, 0x89, 0xce, 0xd2, 0x51, 0x25, 0xf0, 0x2c, 0x5e, 0x91, 0x52, 0xf0, 0x69, 0xc1, 0xaa, - 0x94, 0x7f, 0x43, 0x80, 0x6c, 0x80, 0x80, 0x6e, 0x0f, 0x9d, 0x7e, 0xcc, 0x9e, 0x46, 0x1d, 0x79, - 0xbc, 0x35, 0x74, 0xe4, 0x71, 0x25, 0xfd, 0xe5, 0xc9, 0x62, 0x4a, 0x62, 0x47, 0x60, 0xdc, 0xc3, - 0x8f, 0xd7, 0xfd, 0x64, 0x5b, 0xc9, 0x81, 0x42, 0x6e, 0xb6, 0x2d, 0xba, 0x29, 0xa5, 0xb8, 0x6f, - 0xd6, 0xe9, 0xa6, 0x14, 0xb9, 0x12, 0x7f, 0x33, 0x01, 0xf3, 0xcb, 0x9a, 0xd6, 0x6c, 0xf2, 0xd5, - 0x23, 0xbe, 0x39, 0xe6, 0xc2, 0xf3, 0x84, 0x0f, 0xcf, 0xd1, 0x7b, 0x80, 0x34, 0xdd, 0x66, 0x19, - 0x6d, 0xec, 0x3d, 0x45, 0x33, 0x0f, 0xfd, 0x00, 0x9a, 0x79, 0xf7, 0x4e, 0xd3, 0xbd, 0x81, 0x9a, - 0x40, 0x71, 0xa2, 0x6c, 0x3b, 0x8a, 0xf7, 0x82, 0xf0, 0xd6, 0x44, 0x67, 0xff, 0x18, 0x80, 0xf4, - 0x2e, 0xa5, 0x0c, 0x91, 0x43, 0x7f, 0x12, 0xc4, 0xa3, 0x93, 0x4e, 0x71, 0x64, 0xc5, 0x76, 0x0f, - 0x7a, 0xb1, 0x5c, 0x3a, 0x79, 0x46, 0x5f, 0xb6, 0xd9, 0xf9, 0x2d, 0x76, 0x32, 0xc5, 0x57, 0x4d, - 0x9c, 0x9b, 0x09, 0x7f, 0x4f, 0x80, 0xbc, 0x84, 0x77, 0x2d, 0x6c, 0xc7, 0xba, 0x9d, 0xf2, 0x04, - 0x72, 0x16, 0x93, 0x2a, 0xef, 0x5a, 0x66, 0xf7, 0x3c, 0x16, 0x29, 0xcb, 0x19, 0x9f, 0x58, 0x66, - 0x97, 0x9b, 0xe4, 0x97, 0x50, 0xf0, 0xea, 0x18, 0x67, 0xe3, 0x7f, 0x8f, 0x9e, 0x6b, 0x67, 0x82, - 0xe3, 0x8e, 0x64, 0x89, 0x57, 0x03, 0xf4, 0x15, 0x5f, 0xb0, 0xa2, 0x71, 0xaa, 0xe1, 0xbf, 0x08, - 0x90, 0x6f, 0xf6, 0x77, 0x58, 0x32, 0xb6, 0xf8, 0x34, 0x50, 0x83, 0x4c, 0x07, 0xef, 0x3a, 0xf2, - 0x1b, 0x9d, 0xa9, 0x48, 0x13, 0x56, 0x7a, 0xa2, 0xe4, 0x29, 0x80, 0x45, 0xcf, 0x42, 0x52, 0x39, - 0xc9, 0x73, 0xca, 0xc9, 0x50, 0x5e, 0x42, 0x26, 0xab, 0x4e, 0xc1, 0x6b, 0x66, 0x9c, 0xeb, 0xcb, - 0xab, 0x90, 0x75, 0x48, 0x9e, 0xc7, 0x3a, 0xcc, 0xf3, 0xe0, 0x9d, 0x68, 0x0b, 0x51, 0x81, 0x05, - 0xea, 0xf2, 0xc9, 0x4a, 0xaf, 0xd7, 0xd1, 0xdd, 0x8d, 0x02, 0x6a, 0x7f, 0x52, 0xd2, 0x3c, 0xbd, - 0xb5, 0xcc, 0xee, 0xd0, 0x2d, 0x02, 0xf4, 0x6b, 0x02, 0xe4, 0x76, 0x2d, 0x8c, 0xbf, 0xc0, 0x32, - 0x35, 0xc9, 0x93, 0x45, 0x27, 0xad, 0x92, 0x3a, 0x7c, 0xe5, 0xe8, 0x85, 0x2c, 0x7b, 0x70, 0x93, - 0x3c, 0x17, 0x6d, 0x42, 0x51, 0xed, 0xb0, 0x78, 0x0a, 0x2f, 0x52, 0xea, 0x1c, 0x60, 0xa8, 0xc0, - 0x98, 0xfd, 0x60, 0xa9, 0x17, 0x64, 0x32, 0x29, 0x9a, 0xcc, 0x13, 0x60, 0x72, 0x58, 0x54, 0x19, - 0x91, 0xe0, 0x22, 0x90, 0x37, 0xb3, 0x22, 0x61, 0x45, 0xe3, 0xde, 0x3b, 0x99, 0x57, 0xde, 0x05, - 0x9f, 0x57, 0xaf, 0x60, 0x9e, 0x8e, 0x9b, 0xb8, 0xcf, 0xa9, 0x8b, 0x3f, 0x4a, 0x00, 0x0a, 0x4a, - 0xfe, 0xd9, 0x8d, 0xb7, 0x44, 0x7c, 0xe3, 0xed, 0x5d, 0x40, 0x2c, 0x04, 0xd7, 0x96, 0x7b, 0xd8, - 0x92, 0x6d, 0xac, 0x9a, 0x3c, 0x35, 0x99, 0x20, 0x15, 0xf9, 0x9d, 0x2d, 0x6c, 0x35, 0x29, 0x1d, - 0x2d, 0x03, 0xf8, 0xfe, 0x3a, 0x5f, 0x14, 0x27, 0x71, 0xd7, 0x33, 0x9e, 0xbb, 0x2e, 0x7e, 0x5f, - 0x80, 0xfc, 0x86, 0xde, 0xb6, 0x94, 0x58, 0x33, 0x6f, 0xa1, 0x8f, 0xc3, 0x6f, 0x4a, 0xb2, 0x0f, - 0xcb, 0x51, 0x41, 0x63, 0xac, 0x84, 0x8b, 0xbf, 0x39, 0x03, 0x59, 0x6b, 0xbc, 0x1a, 0xc5, 0x69, - 0x64, 0xff, 0x7d, 0x19, 0x72, 0xbc, 0xde, 0xdb, 0x86, 0x6e, 0x1a, 0xe8, 0x01, 0x24, 0xdb, 0xfc, - 0x4d, 0x58, 0x36, 0x72, 0xd7, 0xda, 0xcf, 0x6b, 0x59, 0x9f, 0x92, 0x48, 0x59, 0xc2, 0xd2, 0xeb, - 0x3b, 0x11, 0x38, 0xc1, 0x3f, 0x47, 0x11, 0x64, 0xe9, 0xf5, 0x1d, 0xd4, 0x84, 0x82, 0xea, 0x27, - 0xd3, 0x93, 0x09, 0x7b, 0x72, 0xe4, 0x9e, 0x42, 0x64, 0x5a, 0xc3, 0xfa, 0x94, 0x94, 0x57, 0x43, - 0x37, 0x50, 0x35, 0x98, 0xc3, 0x2d, 0x35, 0x14, 0xa4, 0xe9, 0x67, 0x00, 0x08, 0xe7, 0x8f, 0xab, - 0x4f, 0x05, 0x52, 0xbd, 0xa1, 0x8f, 0x61, 0x46, 0xa3, 0xd9, 0xc2, 0xb8, 0x95, 0x8a, 0xea, 0xe8, - 0x50, 0x52, 0xb6, 0xfa, 0x94, 0xc4, 0x39, 0xd0, 0x1a, 0xe4, 0xd8, 0x2f, 0xe6, 0x43, 0x73, 0xdb, - 0x72, 0x6b, 0xb4, 0x84, 0xc0, 0xea, 0x5e, 0x9f, 0x92, 0xb2, 0x9a, 0x4f, 0x45, 0xdf, 0x84, 0x94, - 0xad, 0x2a, 0xee, 0x56, 0xcb, 0xd5, 0x11, 0x49, 0x77, 0x7c, 0x66, 0x5a, 0x1a, 0x3d, 0x66, 0x9e, - 0xae, 0x73, 0xe4, 0xee, 0x7a, 0x47, 0x55, 0x3f, 0x94, 0xdc, 0x81, 0x54, 0x1f, 0x53, 0x02, 0x7a, - 0x0a, 0x59, 0x85, 0xb8, 0xf6, 0x32, 0x3d, 0x1e, 0x4d, 0xb7, 0xb9, 0xa3, 0x63, 0x4d, 0x86, 0x8e, - 0xb3, 0xd7, 0x69, 0x06, 0x09, 0x97, 0xe8, 0x0b, 0xea, 0x62, 0xab, 0x8d, 0x4b, 0xd9, 0xf1, 0x82, - 0x82, 0xa1, 0x99, 0x9e, 0x20, 0x4a, 0x44, 0x1b, 0x30, 0xe7, 0xe5, 0x26, 0xa0, 0x8d, 0xca, 0x8d, - 0x8c, 0x6e, 0x88, 0x38, 0x75, 0x57, 0x9f, 0x92, 0x72, 0x7b, 0x01, 0x32, 0xaa, 0x40, 0xa2, 0xad, - 0x96, 0xe6, 0x46, 0x9a, 0x04, 0xef, 0x64, 0x59, 0x7d, 0x4a, 0x4a, 0xb4, 0x55, 0xf4, 0x29, 0xa4, - 0xd9, 0x31, 0xa1, 0x23, 0xa3, 0x94, 0x1f, 0x39, 0xc7, 0xc2, 0x87, 0xad, 0xea, 0x53, 0x12, 0x3d, - 0x99, 0x44, 0x9e, 0xb7, 0x05, 0x79, 0x8b, 0xc5, 0xb6, 0xba, 0x91, 0xe8, 0xc5, 0x91, 0x11, 0x1f, - 0x51, 0xc1, 0xe8, 0x75, 0x0a, 0x84, 0x03, 0x74, 0xf4, 0x1d, 0xb8, 0x10, 0x96, 0xc8, 0x47, 0xda, - 0xfc, 0xc8, 0xe8, 0x85, 0x91, 0x81, 0xd1, 0xf5, 0x29, 0x09, 0x59, 0x43, 0x37, 0xd1, 0x87, 0x30, - 0xcd, 0x7a, 0x0d, 0x51, 0x91, 0x51, 0x41, 0x4c, 0x03, 0x1d, 0xc6, 0xca, 0x93, 0xc1, 0xef, 0xf0, - 0x00, 0x4f, 0xb9, 0x63, 0xb6, 0x4b, 0x0b, 0x23, 0x07, 0xff, 0x70, 0xa8, 0x2a, 0x19, 0xfc, 0x8e, - 0x4f, 0x25, 0xfd, 0x6e, 0xb1, 0x3b, 0x3c, 0x52, 0xed, 0xc2, 0xc8, 0x7e, 0x8f, 0x88, 0xfb, 0xac, - 0xd3, 0xa3, 0x37, 0x3e, 0x99, 0x54, 0xcd, 0x62, 0x79, 0xa2, 0x64, 0x3a, 0xa7, 0x2e, 0x8e, 0xac, - 0xda, 0x70, 0x22, 0xad, 0x3a, 0x75, 0x7c, 0x3d, 0x2a, 0x7a, 0x09, 0x45, 0x9e, 0xcd, 0xc5, 0x7f, - 0xc5, 0x76, 0x89, 0xca, 0xbb, 0x17, 0x69, 0xba, 0xa2, 0x42, 0xd4, 0xea, 0xc4, 0x97, 0x08, 0xdf, - 0x41, 0x9f, 0xc1, 0x3c, 0x95, 0x27, 0xab, 0x7e, 0x02, 0x9e, 0x52, 0x69, 0x28, 0x9d, 0xcb, 0xe8, - 0x5c, 0x3d, 0xae, 0xe4, 0xa2, 0x3a, 0x70, 0x8b, 0x0c, 0x63, 0xdd, 0xd0, 0x1d, 0x6a, 0x65, 0xcb, - 0x23, 0x87, 0x71, 0x38, 0xf9, 0x28, 0x19, 0xc6, 0x3a, 0xa3, 0x90, 0x61, 0xec, 0xf0, 0x88, 0x4f, - 0xde, 0x1d, 0x6f, 0x8f, 0x1c, 0xc6, 0x51, 0xa1, 0xa1, 0x64, 0x18, 0x3b, 0x41, 0x3a, 0x19, 0xc6, - 0xcc, 0x40, 0x0c, 0xc8, 0x7d, 0x67, 0xe4, 0x30, 0x1e, 0x99, 0xa6, 0x80, 0x0c, 0x63, 0x65, 0xe8, - 0x26, 0x5a, 0x05, 0x60, 0x3e, 0x2a, 0xf5, 0x02, 0xae, 0x8e, 0x5c, 0x0c, 0x06, 0x23, 0x3f, 0xc9, - 0x62, 0xd0, 0x71, 0x69, 0xc4, 0x90, 0x51, 0x04, 0x2c, 0xd3, 0x88, 0x82, 0xd2, 0xe2, 0x48, 0x43, - 0x36, 0xf4, 0xf6, 0x9f, 0x18, 0xb2, 0x43, 0x8f, 0x48, 0x56, 0x15, 0xf6, 0x0a, 0xa4, 0x74, 0x6d, - 0xb4, 0x59, 0x0e, 0xbe, 0x09, 0xa5, 0x66, 0x99, 0x12, 0xd0, 0x32, 0x64, 0x88, 0x93, 0x73, 0x4c, - 0xcd, 0xd0, 0xf5, 0x91, 0x40, 0x63, 0xe0, 0x6c, 0x59, 0x7d, 0x4a, 0x4a, 0xbf, 0xe6, 0x24, 0xf2, - 0x78, 0xb6, 0x39, 0x5c, 0x12, 0x47, 0x3e, 0x3e, 0xf4, 0x22, 0x81, 0x3c, 0x9e, 0x71, 0x20, 0x15, - 0x2e, 0xb2, 0xbe, 0xe2, 0x19, 0x03, 0x2c, 0x7e, 0xc8, 0xbd, 0x74, 0x83, 0x8a, 0x1a, 0xb9, 0xcb, - 0x1a, 0x99, 0xc8, 0xa0, 0x3e, 0x25, 0x2d, 0x28, 0xc3, 0x77, 0xc9, 0x84, 0xe7, 0x4b, 0x0f, 0xdb, - 0x9b, 0x2d, 0xdd, 0x1c, 0x39, 0xe1, 0x23, 0xb6, 0xb4, 0xc9, 0x84, 0x57, 0x02, 0x64, 0xb6, 0x00, - 0x69, 0xb2, 0x6d, 0xb3, 0xf8, 0x93, 0x5b, 0x63, 0x16, 0xa0, 0x81, 0xad, 0x1d, 0xb6, 0x00, 0x69, - 0x4d, 0xc6, 0x49, 0x04, 0xa9, 0x1d, 0xac, 0x58, 0xdc, 0xcc, 0xde, 0x1e, 0x29, 0x68, 0x28, 0xcf, - 0x27, 0x11, 0xa4, 0x7a, 0x44, 0xe2, 0xf0, 0x58, 0x6e, 0x1a, 0x28, 0xee, 0x21, 0xdf, 0x19, 0xe9, - 0xf0, 0x44, 0xe6, 0xa9, 0x22, 0x0e, 0x8f, 0x15, 0xba, 0x81, 0x7e, 0x1e, 0x66, 0x39, 0x26, 0x2f, - 0xdd, 0x1d, 0xe3, 0x02, 0x06, 0xb7, 0x51, 0xc8, 0xbc, 0xe6, 0x3c, 0xcc, 0xca, 0xb2, 0xbd, 0x00, - 0xd6, 0xbc, 0x7b, 0x63, 0xac, 0xec, 0xd0, 0x76, 0x04, 0xb3, 0xb2, 0x3e, 0x99, 0x58, 0x59, 0x36, - 0x4e, 0xf9, 0x5a, 0x77, 0x7f, 0xa4, 0x95, 0x1d, 0x3e, 0xb9, 0x46, 0xac, 0xec, 0x6b, 0x9f, 0x4a, - 0x5a, 0x66, 0x33, 0x4c, 0x5c, 0xfa, 0xc6, 0xc8, 0x96, 0x85, 0x37, 0x07, 0x48, 0xcb, 0x38, 0x0f, - 0xe9, 0x36, 0x86, 0x01, 0x98, 0xa6, 0xdf, 0x1d, 0x9d, 0x96, 0x63, 0x10, 0x65, 0xd5, 0xdd, 0x7d, - 0x7b, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0x3c, 0x67, 0x00, 0xd7, 0xd4, 0x7b, 0xe3, 0x0d, 0x55, 0x54, - 0xae, 0x05, 0xcf, 0x50, 0x85, 0x6e, 0xd2, 0xaa, 0xb2, 0x63, 0xa2, 0x74, 0x7e, 0x57, 0xc6, 0x64, - 0x10, 0x19, 0x38, 0xac, 0x4b, 0xab, 0xea, 0x11, 0xfd, 0x29, 0xd4, 0x67, 0xe9, 0x6d, 0x4a, 0x4b, - 0xe3, 0xa7, 0x50, 0x38, 0xc1, 0x8e, 0x37, 0x85, 0x38, 0xd9, 0x5b, 0x33, 0x5d, 0x0f, 0xe3, 0xfd, - 0xf1, 0x6b, 0xe6, 0xa0, 0x6b, 0xc1, 0xd6, 0x4c, 0xee, 0x53, 0xfc, 0x0d, 0x01, 0xae, 0xb1, 0xba, - 0xd1, 0x1d, 0xe0, 0x63, 0xd9, 0x7b, 0x4d, 0x10, 0x00, 0xe2, 0x0f, 0xe8, 0x03, 0x3e, 0x3c, 0xff, - 0x7e, 0xb3, 0xfb, 0xc8, 0x77, 0x94, 0x71, 0xe5, 0xc8, 0x90, 0xea, 0x32, 0x04, 0x55, 0x7a, 0x38, - 0x72, 0x48, 0x85, 0x51, 0x1f, 0x19, 0x52, 0x9c, 0x67, 0x65, 0x96, 0x07, 0x19, 0x78, 0x47, 0xbc, - 0x0b, 0xc5, 0xe2, 0x5a, 0x2a, 0x7d, 0xb9, 0x58, 0x5a, 0x4b, 0xa5, 0xaf, 0x14, 0xcb, 0x6b, 0xa9, - 0xf4, 0x5b, 0xc5, 0xb7, 0xc5, 0x7f, 0x58, 0x86, 0x39, 0x17, 0x74, 0x31, 0x40, 0xf5, 0x30, 0x08, - 0xa8, 0xae, 0x8e, 0x02, 0x54, 0x1c, 0xa6, 0x71, 0x44, 0xf5, 0x30, 0x88, 0xa8, 0xae, 0x8e, 0x42, - 0x54, 0x3e, 0x0f, 0x81, 0x54, 0xad, 0x51, 0x90, 0xea, 0xde, 0x04, 0x90, 0xca, 0x13, 0x35, 0x88, - 0xa9, 0x56, 0x87, 0x31, 0xd5, 0xcd, 0xf1, 0x98, 0xca, 0x13, 0x15, 0x00, 0x55, 0x8f, 0x07, 0x40, - 0xd5, 0xf5, 0x31, 0xa0, 0xca, 0xe3, 0x77, 0x51, 0xd5, 0x7a, 0x24, 0xaa, 0xba, 0x7d, 0x16, 0xaa, - 0xf2, 0xe4, 0x84, 0x60, 0xd5, 0x07, 0x21, 0x58, 0xb5, 0x38, 0x12, 0x56, 0x79, 0xdc, 0x0c, 0x57, - 0x7d, 0x32, 0x88, 0xab, 0xae, 0x8f, 0xc1, 0x55, 0x7e, 0x0b, 0x38, 0xb0, 0xaa, 0x47, 0x01, 0xab, - 0x5b, 0x67, 0x00, 0x2b, 0x4f, 0x4a, 0x10, 0x59, 0xd5, 0xa3, 0x90, 0xd5, 0xad, 0x33, 0x90, 0xd5, - 0x80, 0x24, 0x06, 0xad, 0x36, 0xa3, 0xa1, 0xd5, 0x9d, 0x33, 0xa1, 0x95, 0x27, 0x2d, 0x8c, 0xad, - 0x96, 0x02, 0xd8, 0xea, 0x9d, 0x11, 0xd8, 0xca, 0x63, 0x25, 0xe0, 0xea, 0x5b, 0x43, 0xe0, 0x4a, - 0x1c, 0x07, 0xae, 0x3c, 0x5e, 0x0f, 0x5d, 0xbd, 0x18, 0x81, 0xae, 0xee, 0x9e, 0x8d, 0xae, 0x3c, - 0x61, 0x03, 0xf0, 0x4a, 0x19, 0x0b, 0xaf, 0xde, 0x9b, 0x10, 0x5e, 0x79, 0xd2, 0xa3, 0xf0, 0xd5, - 0x47, 0x61, 0x7c, 0x75, 0x6d, 0x34, 0xbe, 0xf2, 0xc4, 0x70, 0x80, 0xb5, 0x1e, 0x09, 0xb0, 0x6e, - 0x9f, 0x05, 0xb0, 0xfc, 0x79, 0x10, 0x44, 0x58, 0x9b, 0xd1, 0x08, 0xeb, 0xce, 0x99, 0x08, 0xcb, - 0xef, 0xfe, 0x10, 0xc4, 0x5a, 0x8f, 0x84, 0x58, 0xb7, 0xcf, 0x82, 0x58, 0x7e, 0xe5, 0x82, 0x18, - 0xeb, 0xd5, 0x48, 0x8c, 0x75, 0x7f, 0x12, 0x8c, 0xe5, 0x09, 0x1d, 0x02, 0x59, 0x9f, 0x8f, 0x06, - 0x59, 0xdf, 0x38, 0x47, 0x42, 0xd4, 0x48, 0x94, 0xf5, 0xad, 0x21, 0x94, 0x25, 0x8e, 0x43, 0x59, - 0xfe, 0x78, 0x76, 0x61, 0x96, 0x32, 0x16, 0x14, 0xbd, 0x37, 0x21, 0x28, 0xf2, 0x07, 0x5f, 0x04, - 0x2a, 0xaa, 0x45, 0xa0, 0xa2, 0x9b, 0xe3, 0x51, 0x91, 0x6f, 0xce, 0x7d, 0x58, 0x54, 0x8f, 0x82, - 0x45, 0xb7, 0xce, 0x80, 0x45, 0xbe, 0x15, 0x0a, 0xe0, 0xa2, 0xc7, 0x03, 0xb8, 0xe8, 0xfa, 0x99, - 0x31, 0x74, 0x01, 0x60, 0xb4, 0x32, 0x0c, 0x8c, 0x6e, 0x8c, 0x05, 0x46, 0x9e, 0x04, 0x1f, 0x19, - 0x3d, 0x1e, 0x40, 0x46, 0xd7, 0xc7, 0x20, 0x23, 0xbf, 0x02, 0x1c, 0x1a, 0x69, 0xe3, 0xa1, 0x51, - 0x65, 0x52, 0x68, 0xe4, 0x09, 0x8e, 0xc4, 0x46, 0x9b, 0xd1, 0xd8, 0xe8, 0xce, 0x84, 0xe1, 0x2d, - 0x43, 0xe0, 0xa8, 0x1e, 0x05, 0x8e, 0x6e, 0x9d, 0x01, 0x8e, 0x82, 0x6b, 0x88, 0x87, 0x8e, 0xea, - 0x51, 0xe8, 0xe8, 0xd6, 0x19, 0xe8, 0xc8, 0x97, 0x14, 0x80, 0x47, 0xad, 0x51, 0xf0, 0xe8, 0xde, - 0x04, 0xf0, 0xc8, 0x77, 0x5e, 0x06, 0xf0, 0xd1, 0xa7, 0x83, 0xf8, 0x48, 0x1c, 0x87, 0x8f, 0xfc, - 0x19, 0xe9, 0x02, 0xa4, 0xcd, 0x68, 0x80, 0x74, 0xe7, 0x4c, 0x80, 0x14, 0x34, 0x92, 0x01, 0x84, - 0xb4, 0x1e, 0x89, 0x90, 0x6e, 0x9f, 0x85, 0x90, 0x7c, 0x23, 0x19, 0x84, 0x48, 0x9f, 0x0e, 0x42, - 0x24, 0x71, 0x1c, 0x44, 0xf2, 0x1b, 0xe7, 0x62, 0xa4, 0x7a, 0x14, 0x46, 0xba, 0x75, 0x06, 0x46, - 0xf2, 0x3b, 0x2f, 0x00, 0x92, 0x94, 0xb1, 0x20, 0xe9, 0xbd, 0x09, 0x41, 0xd2, 0x80, 0xe1, 0x0a, - 0xa3, 0xa4, 0x7a, 0x14, 0x4a, 0xba, 0x75, 0x06, 0x4a, 0x0a, 0x54, 0xd6, 0x87, 0x49, 0x9b, 0xd1, - 0x30, 0xe9, 0xce, 0x99, 0x30, 0x69, 0x60, 0x36, 0xb9, 0x38, 0x69, 0x3d, 0x12, 0x27, 0xdd, 0x3e, - 0x0b, 0x27, 0x0d, 0x2c, 0x7c, 0xdc, 0x39, 0xf8, 0x95, 0xc9, 0x81, 0xd2, 0x47, 0x6f, 0x1a, 0x98, - 0x73, 0x36, 0x52, 0xfa, 0x74, 0x10, 0x29, 0x89, 0xe3, 0x90, 0x92, 0x3f, 0xb2, 0xce, 0x07, 0x95, - 0xd6, 0x52, 0xe9, 0xb7, 0x8b, 0xef, 0x88, 0x7f, 0x31, 0x03, 0x33, 0x75, 0x2f, 0xea, 0xcc, 0x6f, - 0xa5, 0xf0, 0x26, 0x59, 0xd5, 0xd0, 0x2a, 0x99, 0xf1, 0xd4, 0x6e, 0x9e, 0x9d, 0x8b, 0x73, 0x38, - 0x5b, 0x24, 0x67, 0x7d, 0x83, 0xf4, 0x06, 0xe8, 0x03, 0x98, 0xeb, 0xdb, 0xd8, 0x92, 0x7b, 0x96, - 0x6e, 0x5a, 0xba, 0xc3, 0xce, 0x54, 0x09, 0x2b, 0xc5, 0x2f, 0x4f, 0x16, 0x73, 0xdb, 0x36, 0xb6, - 0xb6, 0x38, 0x5d, 0xca, 0xf5, 0x03, 0x57, 0xee, 0x77, 0xf6, 0xa6, 0x27, 0xff, 0xce, 0xde, 0x0b, - 0x28, 0xd2, 0x77, 0xce, 0x41, 0x0f, 0x86, 0x65, 0x30, 0x8b, 0x1e, 0x73, 0xf4, 0xcc, 0xa3, 0x5b, - 0x92, 0x66, 0x32, 0x2b, 0x58, 0x61, 0x22, 0x7a, 0x00, 0x17, 0xbb, 0xca, 0x11, 0x8d, 0x50, 0x96, - 0x5d, 0xa7, 0x90, 0x46, 0x1d, 0xb3, 0x8f, 0xe3, 0xa1, 0xae, 0x72, 0x44, 0x3f, 0xda, 0xc7, 0x6e, - 0xd1, 0x2f, 0xee, 0xdc, 0x82, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0x4f, 0x8e, 0xcd, 0x12, - 0x4b, 0xcf, 0xb9, 0x54, 0x96, 0x01, 0xbb, 0x0a, 0x85, 0xb6, 0xe2, 0xe0, 0x43, 0xe5, 0x58, 0x76, - 0x0f, 0x3b, 0x66, 0xe9, 0xa9, 0xf1, 0xb7, 0x4e, 0x4f, 0x16, 0xe7, 0x9e, 0xb2, 0x5b, 0x43, 0x67, - 0x1e, 0xe7, 0xda, 0x81, 0x1b, 0x1a, 0xba, 0x03, 0x05, 0xc5, 0x3e, 0x36, 0x54, 0xda, 0x64, 0x6c, - 0xd8, 0x7d, 0x9b, 0xc2, 0x8c, 0xb4, 0x94, 0xa7, 0xe4, 0xaa, 0x4b, 0x45, 0xd7, 0x21, 0xc7, 0xcf, - 0x6a, 0xb0, 0x2f, 0x7b, 0x15, 0x68, 0xf5, 0xf9, 0x27, 0x63, 0xd8, 0xc7, 0xbd, 0x1e, 0x43, 0x99, - 0x7f, 0xd0, 0xe2, 0x50, 0xb1, 0x34, 0x99, 0x6a, 0xd2, 0x1f, 0x73, 0x45, 0x2a, 0xf6, 0x32, 0xfb, - 0x80, 0x05, 0x29, 0x40, 0xd4, 0xe7, 0xcf, 0x8b, 0x16, 0xcc, 0xab, 0x1d, 0xdd, 0x43, 0x05, 0xcc, - 0x89, 0x9a, 0x1f, 0x39, 0x43, 0xaa, 0xb4, 0xec, 0xe0, 0x6b, 0xe6, 0x82, 0x1a, 0x26, 0xa3, 0x26, - 0xd0, 0x2c, 0x50, 0x72, 0xcf, 0xec, 0xe8, 0xea, 0x31, 0x85, 0x05, 0xe1, 0xf4, 0xfc, 0x63, 0x3f, - 0x92, 0xf1, 0x4a, 0xd1, 0x9d, 0x2d, 0xca, 0x29, 0xc1, 0xa1, 0xf7, 0x9b, 0xa5, 0xd6, 0x5e, 0x4b, - 0xa5, 0xa1, 0x98, 0x5d, 0x4b, 0xa5, 0x73, 0xc5, 0xb9, 0xb5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0xbf, - 0x2e, 0x40, 0x2e, 0x74, 0x48, 0xed, 0xf1, 0xc0, 0x4b, 0xe4, 0x2b, 0xd1, 0xf8, 0x6f, 0x54, 0x98, - 0x68, 0x9a, 0x8f, 0x17, 0x37, 0xe6, 0x76, 0x71, 0x34, 0x7e, 0xa0, 0xbb, 0x21, 0x6e, 0x18, 0x8d, - 0xcb, 0xf6, 0x71, 0xea, 0x07, 0x3f, 0x5c, 0x9c, 0x12, 0xff, 0x32, 0x05, 0x73, 0xe1, 0x23, 0x69, - 0x8d, 0x81, 0x7a, 0x45, 0xd9, 0xe7, 0x10, 0x47, 0x65, 0xcc, 0x57, 0x83, 0x32, 0xfe, 0x87, 0x34, - 0x58, 0x35, 0xaf, 0x8d, 0x79, 0x55, 0x1e, 0xac, 0xa7, 0xcf, 0x58, 0xfe, 0x0f, 0x49, 0xcf, 0x4e, - 0x55, 0x60, 0x9a, 0x26, 0xb3, 0xe2, 0x55, 0x2b, 0x0d, 0xf6, 0x15, 0xf1, 0x59, 0xc9, 0x7d, 0x89, - 0x15, 0x23, 0x76, 0xad, 0xf5, 0x46, 0xd9, 0x22, 0xfd, 0xa1, 0x77, 0xfe, 0xef, 0x71, 0xf6, 0x59, - 0xb6, 0xd0, 0xff, 0x8b, 0xa1, 0x36, 0xe4, 0x79, 0xe8, 0x97, 0xa0, 0xa0, 0x9a, 0x9d, 0x0e, 0x5b, - 0xb3, 0x98, 0x65, 0x18, 0xce, 0x1f, 0x44, 0xab, 0xc0, 0x3f, 0xc1, 0x5a, 0xf1, 0x3e, 0xc5, 0x5a, - 0x91, 0xf8, 0xa7, 0x58, 0x03, 0x11, 0xd0, 0x79, 0x4f, 0x98, 0x6b, 0x50, 0x42, 0xc1, 0xd8, 0xb3, - 0x6f, 0x12, 0x8c, 0xcd, 0x02, 0xf8, 0xf9, 0xc8, 0xfb, 0x13, 0x81, 0x07, 0xc4, 0x3c, 0x33, 0xcd, - 0xfd, 0xbe, 0x17, 0x44, 0x5d, 0x0e, 0xe6, 0xfe, 0xf4, 0xa3, 0x45, 0xe9, 0x71, 0xa3, 0x28, 0x0b, - 0x9c, 0xf8, 0x6a, 0x16, 0xf8, 0x3a, 0xe4, 0x7a, 0x16, 0xde, 0xc5, 0x8e, 0xba, 0x27, 0x1b, 0xfd, - 0x2e, 0x3f, 0x6b, 0x95, 0x75, 0x69, 0x9b, 0xfd, 0x2e, 0xba, 0x07, 0x45, 0xaf, 0x08, 0xc7, 0xca, - 0x6e, 0x72, 0x38, 0x97, 0xce, 0x91, 0xb5, 0xf8, 0x3f, 0x05, 0x58, 0x08, 0xb5, 0x89, 0xcf, 0xa9, - 0x35, 0xc8, 0x6a, 0xde, 0x9a, 0x67, 0x97, 0x84, 0x73, 0xc6, 0x11, 0x07, 0x99, 0x91, 0x0c, 0x97, - 0xdc, 0xc7, 0xd2, 0x8f, 0x4f, 0xf8, 0x62, 0x13, 0xe7, 0x14, 0x7b, 0xd1, 0x97, 0xb3, 0x1a, 0x78, - 0x80, 0x37, 0xc9, 0x92, 0x13, 0x4d, 0x32, 0xf1, 0xb7, 0x05, 0x28, 0xd2, 0x07, 0x3c, 0xc1, 0x58, - 0x8b, 0xc5, 0xba, 0xb9, 0xa1, 0xfa, 0x89, 0xc9, 0x4f, 0x53, 0x85, 0x3e, 0x98, 0x93, 0x0c, 0x7f, - 0x30, 0x47, 0xfc, 0xa1, 0x00, 0x79, 0xaf, 0x86, 0xec, 0xa3, 0x96, 0x63, 0x52, 0xcc, 0xbe, 0xd9, - 0x87, 0x1b, 0xdd, 0x54, 0x38, 0x13, 0x7d, 0x67, 0x33, 0x98, 0x0a, 0x87, 0x7d, 0x70, 0xf0, 0xef, - 0xb8, 0x23, 0x87, 0x54, 0xb1, 0xea, 0xe7, 0x20, 0x79, 0x83, 0x83, 0x65, 0x12, 0xfd, 0x1e, 0xb0, - 0xd9, 0x39, 0x60, 0xd9, 0x8b, 0x26, 0x32, 0x7b, 0x88, 0x87, 0x81, 0x01, 0xdf, 0x55, 0xd3, 0x5a, - 0x4d, 0xfa, 0xa5, 0x60, 0xf6, 0xdb, 0x16, 0x9f, 0x04, 0x14, 0x48, 0x3b, 0x9f, 0x68, 0x69, 0x22, - 0x53, 0xec, 0x6a, 0x89, 0x8d, 0x95, 0x3f, 0x0a, 0xf6, 0x44, 0xed, 0x80, 0xa0, 0xa9, 0x47, 0x90, - 0x3c, 0x50, 0x3a, 0xe3, 0x22, 0xa9, 0x42, 0x3d, 0x27, 0x91, 0xd2, 0xe8, 0x49, 0x28, 0x75, 0x4b, - 0x62, 0xb4, 0xe7, 0x3f, 0xac, 0xd2, 0x50, 0x8a, 0x97, 0x0f, 0xc3, 0x63, 0x7d, 0xec, 0xe3, 0x83, - 0x83, 0xfe, 0xe3, 0xd4, 0x8f, 0x7f, 0xb8, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, 0x9d, 0x17, - 0x7d, 0xd3, 0xf2, 0xd3, 0xe0, 0x0c, 0xc6, 0xd0, 0x4f, 0x47, 0xc7, 0xd0, 0x8b, 0x17, 0x61, 0x21, - 0xc4, 0xcd, 0x8c, 0x85, 0xf8, 0x21, 0x5c, 0x79, 0x6a, 0xda, 0xb6, 0xde, 0x23, 0x10, 0x92, 0xce, - 0x4a, 0xb2, 0xb4, 0x78, 0xe6, 0x31, 0xdd, 0xa3, 0x9b, 0x09, 0x06, 0x33, 0x23, 0x19, 0xc9, 0xbb, - 0x16, 0x7f, 0x5f, 0x80, 0xcb, 0xc3, 0x9c, 0x4c, 0xcb, 0x51, 0xe7, 0x60, 0x67, 0x55, 0xd3, 0xcf, - 0xd2, 0x78, 0xf6, 0x68, 0x75, 0x8b, 0x13, 0xb7, 0x90, 0x3f, 0x53, 0xee, 0x2a, 0xd4, 0x7c, 0xf0, - 0x33, 0xf9, 0x79, 0x4e, 0xde, 0x60, 0x54, 0xdf, 0x92, 0xa4, 0x26, 0xb3, 0x24, 0x2d, 0x28, 0xac, - 0x99, 0xba, 0x41, 0xbc, 0x4f, 0xb7, 0xbd, 0xcb, 0x90, 0xdf, 0xd1, 0x0d, 0xc5, 0x3a, 0x96, 0xdd, - 0x00, 0x3e, 0xe1, 0xac, 0x00, 0x3e, 0x69, 0x8e, 0x71, 0xf0, 0x4b, 0xf1, 0x27, 0x02, 0x14, 0x7d, - 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, 0x76, 0xfa, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0x72, 0x2c, 0x32, - 0xbf, 0xca, 0xa8, 0x8d, 0x55, 0x29, 0xc3, 0x0b, 0x34, 0x34, 0x74, 0x23, 0x9c, 0x32, 0x64, 0x7a, - 0x05, 0x4e, 0x87, 0x12, 0x85, 0x90, 0x6e, 0xb7, 0x1d, 0xd3, 0xf2, 0xb0, 0x0b, 0xef, 0x76, 0x37, - 0x43, 0x13, 0x3d, 0xe9, 0x4e, 0xca, 0x2d, 0x43, 0x9e, 0xb8, 0x0b, 0x07, 0xd8, 0x6b, 0x52, 0xea, - 0xec, 0x26, 0x31, 0x0e, 0xb7, 0x49, 0xff, 0x52, 0x80, 0x42, 0x95, 0xf5, 0x86, 0xd7, 0xc3, 0x63, - 0x2c, 0xda, 0x2a, 0xa4, 0x9d, 0x23, 0x43, 0xee, 0x62, 0xef, 0x3b, 0x45, 0xe7, 0x48, 0xa1, 0x38, - 0xeb, 0xb0, 0x4b, 0xfa, 0xe9, 0x4b, 0xfe, 0xdd, 0x75, 0x3e, 0x5d, 0xae, 0x54, 0xd8, 0x87, 0xd9, - 0x2b, 0xee, 0x87, 0xd9, 0x2b, 0xab, 0xbc, 0x00, 0x33, 0xea, 0x3f, 0xf8, 0xcf, 0x8b, 0x82, 0xe4, - 0x31, 0xb1, 0x75, 0xff, 0x7e, 0x93, 0x8c, 0xfa, 0xa1, 0x95, 0x19, 0xe5, 0x01, 0x02, 0x1f, 0xa0, - 0xe2, 0x9f, 0xfa, 0x5e, 0x5e, 0x95, 0xb7, 0x37, 0xab, 0xcf, 0x37, 0x36, 0x1a, 0xad, 0x56, 0x6d, - 0xb5, 0x28, 0xa0, 0x22, 0xe4, 0x42, 0x9f, 0xaf, 0x4a, 0xb0, 0x8f, 0x7f, 0xdf, 0xff, 0xff, 0x00, - 0xfc, 0x2f, 0xe1, 0x11, 0x59, 0xeb, 0xb5, 0xcf, 0xe4, 0x97, 0xcb, 0xcf, 0xb6, 0x6b, 0xcd, 0xe2, - 0x14, 0x42, 0x90, 0x5f, 0x59, 0x6e, 0x55, 0xeb, 0xb2, 0x54, 0x6b, 0x6e, 0x3d, 0xdf, 0x6c, 0xd6, - 0xdc, 0x8f, 0x86, 0xdf, 0x5f, 0x85, 0x5c, 0x30, 0x31, 0x14, 0x5a, 0x80, 0x42, 0xb5, 0x5e, 0xab, - 0xae, 0xcb, 0x2f, 0x1b, 0xcb, 0xf2, 0x8b, 0xed, 0xda, 0x76, 0xad, 0x38, 0x45, 0xab, 0x46, 0x89, - 0x4f, 0xb6, 0x9f, 0x3d, 0x2b, 0x0a, 0xa8, 0x00, 0x59, 0x76, 0x4d, 0x3f, 0x75, 0x55, 0x4c, 0xdc, - 0xdf, 0x80, 0x6c, 0x20, 0x61, 0x35, 0x79, 0xdc, 0xd6, 0x76, 0xb3, 0x2e, 0xb7, 0x1a, 0x1b, 0xb5, - 0x66, 0x6b, 0x79, 0x63, 0x8b, 0xc9, 0xa0, 0xb4, 0xe5, 0x95, 0xe7, 0x52, 0xab, 0x28, 0x78, 0xd7, - 0xad, 0xe7, 0xdb, 0xd5, 0xba, 0xdb, 0x0c, 0x31, 0x95, 0x4e, 0x16, 0x93, 0xf7, 0xff, 0xba, 0x00, - 0x97, 0x47, 0x24, 0x49, 0x42, 0x59, 0x98, 0xdd, 0x36, 0x68, 0xf6, 0xde, 0xe2, 0x14, 0x9a, 0x0b, - 0xe4, 0x49, 0x2a, 0x0a, 0x28, 0xcd, 0x32, 0xd5, 0x14, 0x13, 0x68, 0x06, 0x12, 0xcd, 0x47, 0xc5, - 0x24, 0xa9, 0x69, 0x20, 0xcd, 0x50, 0x31, 0x85, 0x32, 0x3c, 0xc1, 0x49, 0x71, 0x1a, 0xe5, 0xfc, - 0x3c, 0x23, 0xc5, 0x19, 0x22, 0xca, 0xcb, 0xd7, 0x51, 0x9c, 0xbd, 0x7f, 0x1d, 0x02, 0x39, 0x11, - 0x10, 0xc0, 0xcc, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, 0x42, 0xb3, 0x90, 0x5c, 0xee, 0x74, 0x8a, - 0xc2, 0xc3, 0x7f, 0x91, 0x82, 0xb4, 0xfb, 0x49, 0x27, 0xf4, 0x0c, 0xa6, 0xd9, 0xb6, 0xf2, 0xe2, - 0x68, 0xb4, 0x40, 0x27, 0x74, 0xf9, 0xda, 0x59, 0x70, 0x42, 0x9c, 0x42, 0xff, 0x3f, 0x64, 0x03, - 0x5e, 0x14, 0x1a, 0xb9, 0x35, 0x16, 0xf2, 0x1c, 0xcb, 0xb7, 0xcf, 0x2a, 0xe6, 0xc9, 0x7f, 0x05, - 0x19, 0xcf, 0xaa, 0xa3, 0x1b, 0xe3, 0x6c, 0xbe, 0x2b, 0x7b, 0xfc, 0xc2, 0x40, 0xe6, 0x9f, 0x38, - 0xf5, 0xbe, 0x80, 0x2c, 0x40, 0xc3, 0x06, 0x18, 0x45, 0x05, 0x2b, 0x8c, 0xb4, 0xf0, 0xe5, 0xfb, - 0x13, 0x95, 0xf6, 0x9f, 0x49, 0x94, 0xe5, 0xaf, 0x22, 0xd1, 0xca, 0x1a, 0x5a, 0xa3, 0xa2, 0x95, - 0x15, 0xb1, 0x18, 0x4d, 0xa1, 0x17, 0x90, 0x22, 0xd6, 0x13, 0x45, 0xf9, 0x95, 0x03, 0xd6, 0xba, - 0x7c, 0x63, 0x6c, 0x19, 0x57, 0xe4, 0xca, 0xbd, 0x1f, 0xff, 0xf9, 0xd5, 0xa9, 0x1f, 0x9f, 0x5e, - 0x15, 0x7e, 0x72, 0x7a, 0x55, 0xf8, 0xd3, 0xd3, 0xab, 0xc2, 0x9f, 0x9d, 0x5e, 0x15, 0xbe, 0xff, - 0xd3, 0xab, 0x53, 0x3f, 0xf9, 0xe9, 0xd5, 0xa9, 0x3f, 0xfd, 0xe9, 0xd5, 0xa9, 0xcf, 0x67, 0x39, - 0xf7, 0xce, 0x0c, 0x35, 0x2d, 0x8f, 0xfe, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf1, 0x2a, 0x12, - 0x31, 0x5a, 0x83, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_177e11c2c9d5aa4a) } + +var fileDescriptor_api_177e11c2c9d5aa4a = []byte{ + // 8418 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, 0x90, 0x22, 0xa9, 0xab, 0xfe, 0x61, 0x73, 0x66, 0x5a, 0xdd, + 0xd5, 0xff, 0xbd, 0x3b, 0xd4, 0x74, 0xf7, 0x4e, 0x66, 0x3c, 0x3d, 0x9e, 0xb5, 0x44, 0xb1, 0x9b, + 0x94, 0x5a, 0x6a, 0x75, 0x91, 0xea, 0xc6, 0x8c, 0xd7, 0xa9, 0x2d, 0x55, 0x5d, 0x51, 0xb5, 0x22, + 0xab, 0xd8, 0x55, 0x45, 0xfd, 0x0c, 0x10, 0x20, 0x8e, 0x0d, 0x67, 0x9f, 0x82, 0x0d, 0x60, 0xc0, + 0x6b, 0x38, 0x88, 0xd7, 0xb1, 0x11, 0x3f, 0x04, 0x48, 0x82, 0x24, 0xc8, 0x1f, 0x12, 0xfb, 0xc5, + 0x40, 0x16, 0x81, 0x13, 0xaf, 0xdf, 0x8c, 0x00, 0x51, 0x6c, 0x6d, 0x5e, 0x02, 0x07, 0x41, 0x90, + 0x17, 0x03, 0x03, 0x24, 0x08, 0xee, 0x4f, 0xfd, 0x91, 0x45, 0x8a, 0xea, 0xa9, 0x49, 0x06, 0xf0, + 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, 0x14, + 0xcc, 0x5b, 0xa6, 0xa2, 0xee, 0xf5, 0x76, 0x96, 0x94, 0x9e, 0x5e, 0xe9, 0x59, 0xa6, 0x63, 0xa2, + 0x79, 0xd5, 0x54, 0xf7, 0x29, 0xb9, 0xc2, 0x6f, 0x96, 0xef, 0xef, 0x1f, 0x2c, 0xed, 0x1f, 0xd8, + 0xd8, 0x3a, 0xc0, 0xd6, 0x92, 0x6a, 0x1a, 0x6a, 0xdf, 0xb2, 0xb0, 0xa1, 0x1e, 0x2f, 0x75, 0x4c, + 0x75, 0x9f, 0xfe, 0xd1, 0x8d, 0x36, 0x63, 0x0f, 0x97, 0xb5, 0xb0, 0xa2, 0xd9, 0xfd, 0x6e, 0x57, + 0xb1, 0x8e, 0x97, 0x2c, 0xbb, 0xb7, 0xb3, 0xc4, 0x2f, 0x78, 0x59, 0xe4, 0x3e, 0x5d, 0x53, 0x1c, + 0x85, 0xd3, 0x2e, 0xb8, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x25, 0x97, 0xda, 0xc5, 0x8e, + 0x12, 0x28, 0xfd, 0x96, 0xed, 0x98, 0x96, 0xd2, 0xc6, 0x4b, 0xd8, 0x68, 0xeb, 0x06, 0x26, 0x05, + 0x0e, 0x54, 0x95, 0xdf, 0x7c, 0x3b, 0xf2, 0xe6, 0x23, 0x7e, 0xb7, 0xd4, 0x77, 0xf4, 0xce, 0xd2, + 0x5e, 0x47, 0x5d, 0x72, 0xf4, 0x2e, 0xb6, 0x1d, 0xa5, 0xdb, 0x73, 0x9b, 0x40, 0xef, 0x38, 0x96, + 0xa2, 0xea, 0x46, 0xdb, 0xfd, 0xdf, 0xdb, 0x59, 0xb2, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, + 0xa7, 0x18, 0x6e, 0x75, 0xdb, 0x66, 0xdb, 0xa4, 0x3f, 0x97, 0xc8, 0x2f, 0x4e, 0xbd, 0xda, 0x36, + 0xcd, 0x76, 0x07, 0x2f, 0xd1, 0xab, 0x9d, 0xfe, 0xee, 0x92, 0xd6, 0xb7, 0x14, 0x47, 0x37, 0x39, + 0x97, 0xf8, 0xcf, 0x05, 0x98, 0x93, 0xf0, 0xeb, 0x3e, 0xb6, 0x9d, 0x3a, 0x56, 0x34, 0x6c, 0xa1, + 0x2b, 0x90, 0xdc, 0xc7, 0xc7, 0xa5, 0xe4, 0x35, 0xe1, 0x6e, 0x6e, 0x65, 0xf6, 0x8b, 0x93, 0xc5, + 0xe4, 0x3a, 0x3e, 0x96, 0x08, 0x0d, 0x5d, 0x83, 0x59, 0x6c, 0x68, 0x32, 0xb9, 0x9d, 0x0a, 0xdf, + 0x9e, 0xc1, 0x86, 0xb6, 0x8e, 0x8f, 0xd1, 0x77, 0x20, 0x6d, 0x13, 0x69, 0x86, 0x8a, 0x4b, 0xd3, + 0xd7, 0x84, 0xbb, 0xd3, 0x2b, 0x3f, 0xf7, 0xc5, 0xc9, 0xe2, 0xc7, 0x6d, 0xdd, 0xd9, 0xeb, 0xef, + 0x54, 0x54, 0xb3, 0xbb, 0xe4, 0xf5, 0xa9, 0xb6, 0xe3, 0xff, 0x5e, 0xea, 0xed, 0xb7, 0x97, 0x06, + 0x75, 0x54, 0x69, 0x1d, 0x19, 0x4d, 0xfc, 0x5a, 0xf2, 0x24, 0xae, 0xa5, 0xd2, 0x42, 0x31, 0xb1, + 0x96, 0x4a, 0x27, 0x8a, 0x49, 0xf1, 0x8f, 0x12, 0x90, 0x97, 0xb0, 0xdd, 0x33, 0x0d, 0x1b, 0xf3, + 0x9a, 0xbf, 0x07, 0x49, 0xe7, 0xc8, 0xa0, 0x35, 0xcf, 0x3e, 0xbc, 0x5a, 0x19, 0x1a, 0x3d, 0x95, + 0x96, 0xa5, 0x18, 0xb6, 0xa2, 0x92, 0xe6, 0x4b, 0xa4, 0x28, 0xfa, 0x10, 0xb2, 0x16, 0xb6, 0xfb, + 0x5d, 0x4c, 0x15, 0x49, 0x1b, 0x95, 0x7d, 0x78, 0x39, 0x82, 0xb3, 0xd9, 0x53, 0x0c, 0x09, 0x58, + 0x59, 0xf2, 0x1b, 0x5d, 0x81, 0xb4, 0xd1, 0xef, 0x12, 0x55, 0xd8, 0xb4, 0xa1, 0x49, 0x69, 0xd6, + 0xe8, 0x77, 0xd7, 0xf1, 0xb1, 0x8d, 0x9a, 0x30, 0xc7, 0x85, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, + 0x7b, 0x4d, 0xb8, 0x9b, 0x7f, 0x58, 0x89, 0x10, 0x1b, 0x6e, 0x00, 0xb9, 0xec, 0x77, 0xb1, 0x44, + 0xb9, 0xa4, 0x9c, 0x15, 0xb8, 0x42, 0x6f, 0x41, 0x86, 0x3c, 0x6f, 0xe7, 0xd8, 0xc1, 0x76, 0x29, + 0x4d, 0x1f, 0x48, 0x2a, 0xb0, 0x42, 0xae, 0xc5, 0x4f, 0x20, 0x17, 0x64, 0x45, 0x08, 0xf2, 0x52, + 0xad, 0xb9, 0xbd, 0x51, 0x93, 0xb7, 0x37, 0xd7, 0x37, 0x9f, 0xbf, 0xda, 0x2c, 0x4e, 0xa1, 0x0b, + 0x50, 0xe4, 0xb4, 0xf5, 0xda, 0xa7, 0xf2, 0xb3, 0xc6, 0x46, 0xa3, 0x55, 0x14, 0xca, 0xa9, 0xef, + 0xff, 0xf6, 0xd5, 0xa9, 0xb5, 0x54, 0x7a, 0xa6, 0x38, 0x2b, 0xfe, 0xb6, 0x00, 0xf0, 0x14, 0x3b, + 0x7c, 0x34, 0xa0, 0x15, 0x98, 0xd9, 0xa3, 0xd5, 0x2a, 0x09, 0x54, 0x2d, 0xd7, 0x22, 0xeb, 0x1f, + 0x18, 0x39, 0x2b, 0xe9, 0x1f, 0x9f, 0x2c, 0x4e, 0xfd, 0xe4, 0x64, 0x51, 0x90, 0x38, 0x27, 0x7a, + 0x01, 0xd9, 0x7d, 0x7c, 0x2c, 0xf3, 0x79, 0x59, 0x4a, 0x50, 0x45, 0xbc, 0x17, 0x10, 0xb4, 0x7f, + 0x50, 0x71, 0xa7, 0x68, 0x25, 0x30, 0x9d, 0x2b, 0x84, 0xa3, 0xd2, 0x74, 0x2c, 0x6c, 0xb4, 0x9d, + 0x3d, 0x09, 0xf6, 0xf1, 0xf1, 0x33, 0x26, 0x43, 0xfc, 0x7d, 0x01, 0xb2, 0xb4, 0x96, 0x4c, 0x73, + 0xa8, 0x3a, 0x50, 0xcd, 0xeb, 0x67, 0xaa, 0x39, 0xa2, 0x9e, 0x15, 0x98, 0x3e, 0x50, 0x3a, 0x7d, + 0x4c, 0x6b, 0x98, 0x7d, 0x58, 0x8a, 0x90, 0xf1, 0x92, 0xdc, 0x97, 0x58, 0x31, 0xf4, 0x18, 0x72, + 0xba, 0xe1, 0x60, 0xc3, 0x91, 0x19, 0x5b, 0xf2, 0x0c, 0xb6, 0x2c, 0x2b, 0x4d, 0x2f, 0xc4, 0x7f, + 0x26, 0x00, 0x6c, 0xf5, 0x63, 0xd5, 0xf3, 0xb7, 0x26, 0xac, 0xff, 0x4a, 0x8a, 0xb0, 0xba, 0xad, + 0xb8, 0x04, 0x33, 0xba, 0xd1, 0xd1, 0x0d, 0x56, 0xff, 0xb4, 0xc4, 0xaf, 0xd0, 0x05, 0x98, 0xde, + 0xe9, 0xe8, 0x86, 0x46, 0xe7, 0x43, 0x5a, 0x62, 0x17, 0xa2, 0x04, 0x59, 0x5a, 0xeb, 0x18, 0xf5, + 0x2e, 0x9e, 0x24, 0xe0, 0x62, 0xd5, 0x34, 0x34, 0x9d, 0x4c, 0x49, 0xa5, 0xf3, 0xb5, 0xd0, 0xca, + 0x1a, 0x5c, 0xd0, 0x70, 0xcf, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, 0xc6, 0x47, 0xbd, 0x09, 0xfb, 0x18, + 0xf9, 0x5c, 0xb5, 0xa3, 0x1e, 0xa5, 0x45, 0x6b, 0x12, 0x7d, 0x0b, 0x2e, 0x2b, 0x9d, 0x8e, 0x79, + 0x28, 0xeb, 0xbb, 0xb2, 0x66, 0x62, 0x5b, 0x36, 0x4c, 0x47, 0xc6, 0x47, 0xba, 0xed, 0x50, 0x53, + 0x92, 0x96, 0x16, 0xe8, 0xed, 0xc6, 0xee, 0xaa, 0x89, 0xed, 0x4d, 0xd3, 0xa9, 0x91, 0x5b, 0xc4, + 0x02, 0x90, 0xca, 0x30, 0x0b, 0x30, 0x43, 0xcc, 0xaf, 0x94, 0xc6, 0x47, 0x3d, 0x6a, 0x01, 0x02, + 0x5d, 0x39, 0x1b, 0xec, 0x4a, 0xf1, 0x17, 0xe0, 0xd2, 0xa0, 0x7e, 0xe3, 0xec, 0xbf, 0x3f, 0x14, + 0x20, 0xdf, 0x30, 0x74, 0xe7, 0x6b, 0xd1, 0x71, 0x9e, 0xb2, 0x93, 0x41, 0x65, 0xdf, 0x87, 0xe2, + 0xae, 0xa2, 0x77, 0x9e, 0x1b, 0x2d, 0xb3, 0xbb, 0x63, 0x3b, 0xa6, 0x81, 0x6d, 0xde, 0x1b, 0x43, + 0x74, 0xf1, 0x25, 0x14, 0xbc, 0xd6, 0xc4, 0xa9, 0x26, 0x07, 0x8a, 0x0d, 0x43, 0xb5, 0x70, 0x17, + 0x1b, 0xb1, 0xea, 0xe9, 0x6d, 0xc8, 0xe8, 0xae, 0x5c, 0xaa, 0xab, 0xa4, 0xe4, 0x13, 0xc4, 0x3e, + 0xcc, 0x07, 0x9e, 0x1a, 0xa7, 0xb9, 0x24, 0x8b, 0x11, 0x3e, 0x94, 0xfd, 0x3e, 0x22, 0x8b, 0x11, + 0x3e, 0x64, 0xe6, 0xad, 0x09, 0x73, 0xab, 0xb8, 0x83, 0x1d, 0x1c, 0x63, 0x4b, 0xc5, 0x6d, 0xc8, + 0xbb, 0x42, 0xe3, 0xec, 0x98, 0x5f, 0x13, 0x00, 0x71, 0xb9, 0x8a, 0xd1, 0x8e, 0xb3, 0xc6, 0x68, + 0x91, 0xb8, 0x16, 0x4e, 0xdf, 0x32, 0x98, 0x8f, 0xc0, 0xc6, 0x24, 0x30, 0x12, 0x75, 0x13, 0xfc, + 0x29, 0x9b, 0x0a, 0x4e, 0x59, 0xee, 0xde, 0x1c, 0xc2, 0x42, 0xa8, 0x62, 0xf1, 0x76, 0x5f, 0x8a, + 0xd6, 0x29, 0x71, 0x2d, 0x19, 0xf4, 0xe1, 0x28, 0x51, 0xfc, 0xa1, 0x00, 0xf3, 0xd5, 0x0e, 0x56, + 0xac, 0xd8, 0x35, 0xf2, 0x6d, 0x48, 0x6b, 0x58, 0xd1, 0x68, 0x93, 0xd9, 0xc4, 0x7e, 0x27, 0x20, + 0x85, 0x78, 0xba, 0x95, 0xbd, 0x8e, 0x5a, 0x69, 0xb9, 0x3e, 0x30, 0x9f, 0xdd, 0x1e, 0x93, 0xf8, + 0x29, 0xa0, 0x60, 0xcd, 0xe2, 0x1c, 0x08, 0xbf, 0x93, 0x00, 0x24, 0xe1, 0x03, 0x6c, 0x39, 0xb1, + 0x37, 0x7b, 0x15, 0xb2, 0x8e, 0x62, 0xb5, 0xb1, 0x23, 0x13, 0xef, 0xfe, 0x3c, 0x2d, 0x07, 0xc6, + 0x47, 0xc8, 0xa8, 0x05, 0x77, 0xb0, 0xa1, 0xec, 0x74, 0x30, 0x95, 0x22, 0xef, 0x98, 0x7d, 0x43, + 0x93, 0x75, 0x07, 0x5b, 0x8a, 0x63, 0x5a, 0xb2, 0xd9, 0x73, 0xf4, 0xae, 0xfe, 0x39, 0x75, 0xec, + 0xf9, 0x50, 0xbb, 0xc1, 0x8a, 0x13, 0xe6, 0x15, 0x52, 0xb8, 0xc1, 0xcb, 0x3e, 0x0f, 0x14, 0x45, + 0x15, 0x58, 0xd0, 0xdb, 0x86, 0x69, 0x61, 0xb9, 0xad, 0xca, 0xce, 0x9e, 0x85, 0xed, 0x3d, 0xb3, + 0xe3, 0xae, 0x56, 0xf3, 0xec, 0xd6, 0x53, 0xb5, 0xe5, 0xde, 0x10, 0x3f, 0x83, 0x85, 0x90, 0x96, + 0xe2, 0xec, 0x82, 0xff, 0x29, 0x40, 0xb6, 0xa9, 0x2a, 0x46, 0x9c, 0xba, 0xff, 0x04, 0xb2, 0xb6, + 0xaa, 0x18, 0xf2, 0xae, 0x69, 0x75, 0x15, 0x87, 0xb6, 0x2b, 0x1f, 0xd2, 0xbd, 0xe7, 0xdf, 0xab, + 0x8a, 0xf1, 0x84, 0x16, 0x92, 0xc0, 0xf6, 0x7e, 0x0f, 0xfa, 0xaf, 0xd3, 0x5f, 0xde, 0x7f, 0x65, + 0xd3, 0x7b, 0x2d, 0x95, 0x4e, 0x16, 0x53, 0xe2, 0x5f, 0x08, 0x90, 0x63, 0x4d, 0x8e, 0x73, 0x7a, + 0xbf, 0x0f, 0x29, 0xcb, 0x3c, 0x64, 0xd3, 0x3b, 0xfb, 0xf0, 0xad, 0x08, 0x11, 0xeb, 0xf8, 0x38, + 0xb8, 0x7e, 0xd2, 0xe2, 0x68, 0x05, 0xb8, 0x97, 0x2a, 0x53, 0xee, 0xe4, 0xa4, 0xdc, 0xc0, 0xb8, + 0x24, 0x22, 0xe3, 0x0e, 0x14, 0x76, 0x14, 0x47, 0xdd, 0x93, 0x2d, 0x5e, 0x49, 0xb2, 0xd6, 0x26, + 0xef, 0xe6, 0xa4, 0x3c, 0x25, 0xbb, 0x55, 0xb7, 0x49, 0xcb, 0xd9, 0x7c, 0xb3, 0xf1, 0x5f, 0xb2, + 0x3e, 0xff, 0x3f, 0x02, 0x9f, 0x43, 0x6e, 0xcb, 0xff, 0xb2, 0x75, 0xfd, 0xaf, 0x27, 0xe0, 0x72, + 0x75, 0x0f, 0xab, 0xfb, 0x55, 0xd3, 0xb0, 0x75, 0xdb, 0x21, 0xba, 0x8b, 0xb3, 0xff, 0xdf, 0x82, + 0xcc, 0xa1, 0xee, 0xec, 0xc9, 0x9a, 0xbe, 0xbb, 0x4b, 0xad, 0x6d, 0x5a, 0x4a, 0x13, 0xc2, 0xaa, + 0xbe, 0xbb, 0x8b, 0x1e, 0x41, 0xaa, 0x6b, 0x6a, 0xcc, 0x99, 0xcf, 0x3f, 0x5c, 0x8c, 0x10, 0x4f, + 0xab, 0x66, 0xf7, 0xbb, 0x1b, 0xa6, 0x86, 0x25, 0x5a, 0x18, 0x5d, 0x05, 0x50, 0x09, 0xb5, 0x67, + 0xea, 0x86, 0xc3, 0x8d, 0x63, 0x80, 0x82, 0xea, 0x90, 0x71, 0xb0, 0xd5, 0xd5, 0x0d, 0xc5, 0xc1, + 0xa5, 0x69, 0xaa, 0xbc, 0x9b, 0x91, 0x15, 0xef, 0x75, 0x74, 0x55, 0x59, 0xc5, 0xb6, 0x6a, 0xe9, + 0x3d, 0xc7, 0xb4, 0xb8, 0x16, 0x7d, 0x66, 0xf1, 0x6f, 0xa5, 0xa0, 0x34, 0xac, 0x9b, 0x38, 0x47, + 0xc8, 0x16, 0xcc, 0x58, 0xd8, 0xee, 0x77, 0x1c, 0x3e, 0x46, 0x1e, 0x8e, 0x52, 0x41, 0x44, 0x0d, + 0xe8, 0xfe, 0x44, 0xc7, 0xe1, 0xd5, 0xe6, 0x72, 0xca, 0xff, 0x46, 0x80, 0x19, 0x76, 0x03, 0x3d, + 0x80, 0xb4, 0x45, 0x16, 0x06, 0x59, 0xd7, 0x68, 0x1d, 0x93, 0x2b, 0x97, 0x4e, 0x4f, 0x16, 0x67, + 0xe9, 0x62, 0xd1, 0x58, 0xfd, 0xc2, 0xff, 0x29, 0xcd, 0xd2, 0x72, 0x0d, 0x8d, 0xf4, 0x96, 0xed, + 0x28, 0x96, 0x43, 0x37, 0x95, 0x12, 0x0c, 0xd5, 0x50, 0xc2, 0x3a, 0x3e, 0x46, 0x6b, 0x30, 0x63, + 0x3b, 0x8a, 0xd3, 0xb7, 0x79, 0x7f, 0x9d, 0xab, 0xb2, 0x4d, 0xca, 0x29, 0x71, 0x09, 0xc4, 0xdd, + 0xd2, 0xb0, 0xa3, 0xe8, 0x1d, 0xda, 0x81, 0x19, 0x89, 0x5f, 0x89, 0xbf, 0x21, 0xc0, 0x0c, 0x2b, + 0x8a, 0x2e, 0xc3, 0x82, 0xb4, 0xbc, 0xf9, 0xb4, 0x26, 0x37, 0x36, 0x57, 0x6b, 0xad, 0x9a, 0xb4, + 0xd1, 0xd8, 0x5c, 0x6e, 0xd5, 0x8a, 0x53, 0xe8, 0x12, 0x20, 0xf7, 0x46, 0xf5, 0xf9, 0x66, 0xb3, + 0xd1, 0x6c, 0xd5, 0x36, 0x5b, 0x45, 0x81, 0xee, 0xa9, 0x50, 0x7a, 0x80, 0x9a, 0x40, 0x37, 0xe1, + 0xda, 0x20, 0x55, 0x6e, 0xb6, 0x96, 0x5b, 0x4d, 0xb9, 0xd6, 0x6c, 0x35, 0x36, 0x96, 0x5b, 0xb5, + 0xd5, 0x62, 0x72, 0x4c, 0x29, 0xf2, 0x10, 0x49, 0xaa, 0x55, 0x5b, 0xc5, 0x94, 0xe8, 0xc0, 0x45, + 0x09, 0xab, 0x66, 0xb7, 0xd7, 0x77, 0x30, 0xa9, 0xa5, 0x1d, 0xe7, 0x4c, 0xb9, 0x0c, 0xb3, 0x9a, + 0x75, 0x2c, 0x5b, 0x7d, 0x83, 0xcf, 0x93, 0x19, 0xcd, 0x3a, 0x96, 0xfa, 0x86, 0xf8, 0x8f, 0x05, + 0xb8, 0x34, 0xf8, 0xd8, 0x38, 0x07, 0xe1, 0x0b, 0xc8, 0x2a, 0x9a, 0x86, 0x35, 0x59, 0xc3, 0x1d, + 0x47, 0xe1, 0x2e, 0xd1, 0xfd, 0x80, 0x24, 0xbe, 0x15, 0x58, 0xf1, 0xb6, 0x02, 0x37, 0x5e, 0x56, + 0xab, 0xb4, 0x22, 0xab, 0x84, 0xc3, 0x35, 0x3f, 0x54, 0x08, 0xa5, 0x88, 0xff, 0x3d, 0x05, 0x73, + 0x35, 0x43, 0x6b, 0x1d, 0xc5, 0xba, 0x96, 0x5c, 0x82, 0x19, 0xd5, 0xec, 0x76, 0x75, 0xc7, 0x55, + 0x10, 0xbb, 0x42, 0x3f, 0x13, 0x70, 0x65, 0x93, 0x13, 0x38, 0x74, 0xbe, 0x13, 0x8b, 0xbe, 0x0b, + 0x97, 0x89, 0xd5, 0xb4, 0x0c, 0xa5, 0x23, 0x33, 0x69, 0xb2, 0x63, 0xe9, 0xed, 0x36, 0xb6, 0xf8, + 0xf6, 0xe3, 0xdd, 0x88, 0x7a, 0x36, 0x38, 0x47, 0x95, 0x32, 0xb4, 0x58, 0x79, 0xe9, 0xa2, 0x1e, + 0x45, 0x46, 0x1f, 0x03, 0x90, 0xa5, 0x88, 0x6e, 0x69, 0xda, 0xdc, 0x1e, 0x8d, 0xda, 0xd3, 0x74, + 0x4d, 0x10, 0x61, 0x20, 0xd7, 0x36, 0x5a, 0x22, 0xb8, 0xe5, 0x75, 0x5f, 0xb7, 0xb0, 0xfc, 0xa0, + 0xa7, 0xd2, 0x8d, 0x86, 0xf4, 0x4a, 0xfe, 0xf4, 0x64, 0x11, 0x24, 0x46, 0x7e, 0xb0, 0x55, 0x25, + 0x38, 0x86, 0xfd, 0xee, 0xa9, 0xe8, 0x15, 0xdc, 0x0b, 0xec, 0x97, 0x90, 0x95, 0x97, 0x37, 0x4b, + 0x71, 0xe4, 0x3d, 0xbd, 0xbd, 0x87, 0x2d, 0xd9, 0xdb, 0xd6, 0xa6, 0x3b, 0x97, 0x69, 0xe9, 0xa6, + 0xcf, 0x50, 0x55, 0x0c, 0x56, 0xfb, 0x65, 0xa7, 0x4e, 0x0b, 0x7b, 0x3a, 0x23, 0xca, 0xef, 0x99, + 0xba, 0x6d, 0x1a, 0xa5, 0x0c, 0x53, 0x3e, 0xbb, 0x42, 0xf7, 0xa0, 0xe8, 0x1c, 0x19, 0xf2, 0x1e, + 0x56, 0x2c, 0x67, 0x07, 0x2b, 0x0e, 0x59, 0xa5, 0x81, 0x96, 0x28, 0x38, 0x47, 0x46, 0x3d, 0x40, + 0x46, 0x2f, 0xa0, 0xa8, 0x1b, 0xf2, 0x6e, 0x47, 0x6f, 0xef, 0x39, 0xf2, 0xa1, 0xa5, 0x3b, 0xd8, + 0x2e, 0xcd, 0x53, 0x85, 0x44, 0x8d, 0xdb, 0x26, 0xdf, 0x67, 0xd6, 0x5e, 0x91, 0x92, 0x5c, 0x35, + 0x79, 0xdd, 0x78, 0x42, 0xf9, 0x29, 0xd1, 0x5e, 0x4b, 0xa5, 0x67, 0x8b, 0x69, 0xf1, 0x3f, 0x0b, + 0x90, 0x77, 0x87, 0x5b, 0x9c, 0x33, 0xe3, 0x2e, 0x14, 0x4d, 0x03, 0xcb, 0xbd, 0x3d, 0xc5, 0xc6, + 0x5c, 0x8f, 0x7c, 0xc1, 0xc9, 0x9b, 0x06, 0xde, 0x22, 0x64, 0xa6, 0x2e, 0xb4, 0x05, 0xf3, 0xb6, + 0xa3, 0xb4, 0x75, 0xa3, 0x1d, 0x50, 0xef, 0xf4, 0xe4, 0xe0, 0xa2, 0xc8, 0xb9, 0x3d, 0x7a, 0xc8, + 0x4b, 0xf9, 0x63, 0x01, 0xe6, 0x97, 0xb5, 0xae, 0x6e, 0x34, 0x7b, 0x1d, 0x3d, 0xd6, 0x3d, 0x8b, + 0x9b, 0x90, 0xb1, 0x89, 0x4c, 0xdf, 0xe0, 0xfb, 0x08, 0x34, 0x4d, 0xef, 0x10, 0xcb, 0xff, 0x0c, + 0x0a, 0xf8, 0xa8, 0xa7, 0xb3, 0x57, 0x15, 0x0c, 0x38, 0xa5, 0x26, 0x6f, 0x5b, 0xde, 0xe7, 0x25, + 0xb7, 0x78, 0x9b, 0x3e, 0x05, 0x14, 0x6c, 0x52, 0x9c, 0xd8, 0xe5, 0x53, 0x58, 0xa0, 0xa2, 0xb7, + 0x0d, 0x3b, 0x66, 0x7d, 0x89, 0x3f, 0x0f, 0x17, 0xc2, 0xa2, 0xe3, 0xac, 0xf7, 0x2b, 0xde, 0xcb, + 0x1b, 0xd8, 0x8a, 0x15, 0xf4, 0x7a, 0xba, 0xe6, 0x82, 0xe3, 0xac, 0xf3, 0x2f, 0x0b, 0x70, 0x85, + 0xca, 0xa6, 0x6f, 0x73, 0x76, 0xb1, 0xf5, 0x0c, 0x2b, 0x76, 0xac, 0x88, 0xfd, 0x06, 0xcc, 0x30, + 0xe4, 0x4d, 0xc7, 0xe7, 0xf4, 0x4a, 0x96, 0x78, 0x2e, 0x4d, 0xc7, 0xb4, 0x88, 0xe7, 0xc2, 0x6f, + 0x89, 0x0a, 0x94, 0xa3, 0x6a, 0x11, 0x67, 0x4b, 0xff, 0xae, 0x00, 0xf3, 0xdc, 0x69, 0x24, 0x43, + 0xb9, 0xba, 0x47, 0x7c, 0x26, 0x54, 0x83, 0xac, 0x4a, 0x7f, 0xc9, 0xce, 0x71, 0x0f, 0x53, 0xf9, + 0xf9, 0x71, 0xfe, 0x26, 0x63, 0x6b, 0x1d, 0xf7, 0x30, 0x71, 0x5a, 0xdd, 0xdf, 0x44, 0x51, 0x81, + 0x46, 0x8e, 0xf5, 0x58, 0xe9, 0x3c, 0xa2, 0x65, 0x5d, 0xd7, 0x8f, 0xeb, 0xe0, 0x9f, 0x26, 0xb9, + 0x12, 0xd8, 0x33, 0x78, 0xf1, 0x58, 0x7d, 0x94, 0xcf, 0xe0, 0x52, 0x70, 0x75, 0x09, 0x34, 0x3c, + 0x71, 0x8e, 0x86, 0x07, 0x76, 0xf4, 0x7d, 0x2a, 0xfa, 0x14, 0x02, 0x7b, 0xf6, 0x32, 0x6b, 0x93, + 0x8b, 0x7e, 0xce, 0xa3, 0x8e, 0x79, 0x5f, 0x0a, 0xa3, 0xdb, 0xa8, 0x0a, 0x69, 0x7c, 0xd4, 0x93, + 0x35, 0x6c, 0xab, 0xdc, 0x70, 0x89, 0x51, 0x02, 0x49, 0x55, 0x86, 0xf0, 0xc0, 0x2c, 0x3e, 0xea, + 0x11, 0x22, 0xda, 0x26, 0xab, 0x97, 0xeb, 0x2a, 0xd0, 0x6a, 0xdb, 0x67, 0xc3, 0x0b, 0x7f, 0xa4, + 0x70, 0x71, 0x05, 0xcf, 0x4b, 0x60, 0x22, 0xc4, 0x1f, 0x09, 0xf0, 0x56, 0x64, 0xaf, 0xc5, 0xb9, + 0x90, 0x7d, 0x0c, 0x29, 0xda, 0xf8, 0xc4, 0x39, 0x1b, 0x4f, 0xb9, 0xc4, 0xef, 0x27, 0xf8, 0x1c, + 0x97, 0x70, 0xc7, 0x24, 0x8a, 0x8d, 0x7d, 0x57, 0xee, 0x39, 0xcc, 0x1d, 0x98, 0x0e, 0xf1, 0x4d, + 0x78, 0xb7, 0x27, 0xce, 0xdd, 0xed, 0x39, 0x2a, 0xc0, 0xed, 0xf1, 0x97, 0x30, 0x6f, 0x98, 0x86, + 0x1c, 0x16, 0x7a, 0xfe, 0xb1, 0x54, 0x30, 0x4c, 0xe3, 0x65, 0x40, 0xae, 0x67, 0x67, 0x06, 0x34, + 0x11, 0xa7, 0x9d, 0xf9, 0x81, 0x00, 0x0b, 0x9e, 0xdb, 0x14, 0xb3, 0x07, 0xfd, 0x3e, 0x24, 0x0d, + 0xf3, 0xf0, 0x3c, 0xbb, 0x9e, 0xa4, 0x3c, 0x59, 0xf5, 0xc2, 0x35, 0x8a, 0xb3, 0xbd, 0xff, 0x36, + 0x01, 0x99, 0xa7, 0xd5, 0x38, 0x5b, 0xf9, 0x31, 0xdf, 0x51, 0x67, 0xfd, 0x1d, 0x35, 0xda, 0xbd, + 0xe7, 0x55, 0x9e, 0x56, 0xd7, 0xf1, 0xb1, 0x3b, 0xda, 0x09, 0x17, 0x5a, 0x86, 0x4c, 0x78, 0xef, + 0x75, 0x42, 0x4d, 0xf9, 0x5c, 0x65, 0x0c, 0xd3, 0x54, 0xae, 0x1b, 0xbd, 0x21, 0x44, 0x44, 0x6f, + 0x90, 0xc7, 0x78, 0x9e, 0x62, 0xe2, 0x3c, 0x8f, 0x09, 0xb8, 0x88, 0xd3, 0xc5, 0x19, 0xf1, 0x05, + 0x00, 0x69, 0x4e, 0x9c, 0x5d, 0xf2, 0x2b, 0x49, 0xc8, 0x6f, 0xf5, 0xed, 0xbd, 0x98, 0x47, 0x5f, + 0x15, 0xa0, 0xd7, 0xb7, 0x29, 0x04, 0x39, 0x32, 0x78, 0x9b, 0xcf, 0x08, 0x0c, 0x71, 0x1b, 0xcd, + 0xf8, 0x5a, 0x47, 0x06, 0xaa, 0x73, 0x21, 0x58, 0xf6, 0xa3, 0x4b, 0x6e, 0x8c, 0x03, 0xab, 0xad, + 0x23, 0x63, 0x03, 0x7b, 0x28, 0x95, 0x49, 0xc2, 0x44, 0xd2, 0xc7, 0x30, 0x4b, 0x2e, 0x64, 0xc7, + 0x3c, 0x4f, 0x37, 0xcf, 0x10, 0x9e, 0x96, 0x89, 0x1e, 0x43, 0x86, 0x71, 0x93, 0xd5, 0x6f, 0x86, + 0xae, 0x7e, 0x51, 0x6d, 0xe1, 0x6a, 0xa4, 0xeb, 0x5e, 0x9a, 0xb2, 0x92, 0xb5, 0xee, 0x02, 0x4c, + 0xef, 0x9a, 0x96, 0xea, 0xbe, 0x1f, 0x66, 0x17, 0xac, 0x3f, 0xd7, 0x52, 0xe9, 0x74, 0x31, 0xb3, + 0x96, 0x4a, 0x67, 0x8a, 0x20, 0xfe, 0x86, 0x00, 0x05, 0xaf, 0x23, 0xe2, 0x5c, 0x10, 0xaa, 0x21, + 0x2d, 0x9e, 0xbf, 0x2b, 0x88, 0x02, 0xc5, 0x7f, 0x47, 0x3d, 0x22, 0xd5, 0x3c, 0xa0, 0x3d, 0x13, + 0xe7, 0x48, 0x79, 0xcc, 0x62, 0x87, 0x12, 0xe7, 0xed, 0x5d, 0x1a, 0x46, 0xf4, 0x00, 0x2e, 0xe8, + 0x5d, 0x62, 0xcf, 0x75, 0xa7, 0x73, 0xcc, 0x61, 0x9b, 0x83, 0xdd, 0x17, 0xd1, 0x0b, 0xfe, 0xbd, + 0xaa, 0x7b, 0x4b, 0xfc, 0x1d, 0xba, 0x01, 0xee, 0xb7, 0x24, 0x4e, 0x55, 0x37, 0x60, 0xce, 0x62, + 0xa2, 0x89, 0x5b, 0x73, 0x4e, 0x6d, 0xe7, 0x3c, 0x56, 0xa2, 0xf0, 0xdf, 0x4a, 0x40, 0xe1, 0x45, + 0x1f, 0x5b, 0xc7, 0x5f, 0x27, 0x75, 0xdf, 0x86, 0xc2, 0xa1, 0xa2, 0x3b, 0xf2, 0xae, 0x69, 0xc9, + 0xfd, 0x9e, 0xa6, 0x38, 0x6e, 0x00, 0xcb, 0x1c, 0x21, 0x3f, 0x31, 0xad, 0x6d, 0x4a, 0x44, 0x18, + 0xd0, 0xbe, 0x61, 0x1e, 0x1a, 0x32, 0x21, 0x53, 0xa0, 0x7c, 0x64, 0xf0, 0x5d, 0xe9, 0x95, 0x0f, + 0xfe, 0xd3, 0xc9, 0xe2, 0xa3, 0x89, 0xc2, 0xd2, 0x68, 0x08, 0x5e, 0xbf, 0xaf, 0x6b, 0x95, 0xed, + 0xed, 0xc6, 0xaa, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xad, 0x23, 0xc3, 0x16, 0xff, 0x7e, 0x02, + 0x8a, 0xbe, 0x8e, 0xe2, 0xec, 0xc8, 0x1a, 0x64, 0x5f, 0xf7, 0xb1, 0xa5, 0xbf, 0x41, 0x37, 0x02, + 0x67, 0x24, 0x66, 0xe7, 0x33, 0xc8, 0x85, 0x34, 0x90, 0xfc, 0x72, 0x1a, 0xc8, 0x1e, 0xfa, 0x8d, + 0x47, 0xf7, 0x61, 0xde, 0x39, 0x32, 0x64, 0x16, 0x90, 0xc8, 0x82, 0x58, 0xdc, 0xf8, 0x8a, 0x82, + 0x43, 0xf4, 0x41, 0xe8, 0x34, 0x80, 0xc5, 0x16, 0x7f, 0x5f, 0x00, 0x44, 0x15, 0xd5, 0x60, 0xaf, + 0x0d, 0xbe, 0x2e, 0xe3, 0xe9, 0x2e, 0x14, 0x69, 0x88, 0xa7, 0xac, 0xef, 0xca, 0x5d, 0xdd, 0xb6, + 0x75, 0xa3, 0xcd, 0x07, 0x54, 0x9e, 0xd2, 0x1b, 0xbb, 0x1b, 0x8c, 0x2a, 0xfe, 0x35, 0x58, 0x08, + 0x35, 0x20, 0xce, 0xce, 0xbe, 0x0e, 0xb9, 0x5d, 0xf6, 0x56, 0x97, 0x0a, 0xe7, 0x3b, 0x8e, 0x59, + 0x4a, 0x63, 0xcf, 0x13, 0xff, 0x3c, 0x01, 0x17, 0x24, 0x6c, 0x9b, 0x9d, 0x03, 0x1c, 0xbf, 0x0a, + 0xeb, 0xc0, 0x5f, 0xe7, 0xc8, 0x6f, 0xa4, 0xc9, 0x0c, 0x63, 0x66, 0xcb, 0x5c, 0x78, 0xdb, 0xfe, + 0xe6, 0xf8, 0x11, 0x3b, 0xbc, 0x51, 0xcf, 0xb7, 0xfd, 0x52, 0xa1, 0x6d, 0x3f, 0x13, 0x0a, 0xec, + 0x85, 0xb4, 0x26, 0xdb, 0xf8, 0xb5, 0xd1, 0xef, 0xba, 0x60, 0xa8, 0x32, 0xae, 0x92, 0x0d, 0xc6, + 0xd2, 0xc4, 0xaf, 0x37, 0xfb, 0x5d, 0xea, 0x3b, 0xaf, 0x5c, 0x22, 0xf5, 0x3d, 0x3d, 0x59, 0xcc, + 0x87, 0xee, 0xd9, 0x52, 0x5e, 0xf7, 0xae, 0x89, 0x74, 0xf1, 0x3b, 0x70, 0x71, 0x40, 0xd9, 0x71, + 0x7a, 0x3c, 0xff, 0x3a, 0x09, 0x57, 0xc2, 0xe2, 0xe3, 0x86, 0x38, 0x5f, 0xf7, 0x0e, 0xad, 0xc3, + 0x5c, 0x57, 0x37, 0xde, 0x6c, 0xf7, 0x32, 0xd7, 0xd5, 0x0d, 0x7f, 0xa7, 0x38, 0x62, 0x68, 0xcc, + 0x7c, 0xa5, 0x43, 0x43, 0x81, 0x72, 0x54, 0xdf, 0xc5, 0x39, 0x3e, 0xbe, 0x2f, 0x40, 0x2e, 0xee, + 0x6d, 0xb9, 0x37, 0x0b, 0xac, 0x13, 0x5b, 0x30, 0xf7, 0x15, 0xec, 0xe3, 0xfd, 0x96, 0x00, 0xa8, + 0x65, 0xf5, 0x0d, 0x02, 0x6a, 0x9f, 0x99, 0xed, 0x38, 0x9b, 0x79, 0x01, 0xa6, 0x75, 0x43, 0xc3, + 0x47, 0xb4, 0x99, 0x29, 0x89, 0x5d, 0x84, 0xde, 0x4e, 0x26, 0x27, 0x7a, 0x3b, 0x29, 0x7e, 0x06, + 0x0b, 0xa1, 0x2a, 0xc6, 0xd9, 0xfe, 0xff, 0x96, 0x80, 0x05, 0xde, 0x90, 0xd8, 0x77, 0x30, 0xbf, + 0x05, 0xd3, 0x1d, 0x22, 0x73, 0x4c, 0x3f, 0xd3, 0x67, 0xba, 0xfd, 0x4c, 0x0b, 0xa3, 0x9f, 0x05, + 0xe8, 0x59, 0xf8, 0x40, 0x66, 0xac, 0xc9, 0x89, 0x58, 0x33, 0x84, 0x83, 0x12, 0xd0, 0x0f, 0x05, + 0x28, 0x90, 0x09, 0xdd, 0xb3, 0xcc, 0x9e, 0x69, 0x13, 0x9f, 0xc5, 0x9e, 0x0c, 0xe6, 0xbc, 0x38, + 0x3d, 0x59, 0x9c, 0xdb, 0xd0, 0x8d, 0x2d, 0xce, 0xd8, 0x6a, 0x4e, 0x7c, 0x66, 0xc0, 0x3d, 0x39, + 0x51, 0xa9, 0x76, 0x4c, 0x75, 0xdf, 0x7f, 0xdf, 0x46, 0x2c, 0x8b, 0x27, 0xce, 0x16, 0xff, 0x48, + 0x80, 0x0b, 0x5f, 0xd9, 0x76, 0xf1, 0xff, 0x0f, 0x65, 0x8b, 0x2f, 0xa1, 0x48, 0x7f, 0x34, 0x8c, + 0x5d, 0x33, 0xce, 0x8d, 0xfb, 0xff, 0x2d, 0xc0, 0x7c, 0x40, 0x70, 0x9c, 0x0e, 0xce, 0x9b, 0xea, + 0x69, 0x8e, 0x45, 0xd8, 0x38, 0x93, 0xa9, 0x4a, 0xca, 0xf1, 0xe2, 0x6c, 0x50, 0x56, 0x20, 0x87, + 0x89, 0x15, 0xa3, 0x5b, 0xbc, 0x3b, 0xec, 0xdc, 0xca, 0xc0, 0x8e, 0x7e, 0xd6, 0x2b, 0xb0, 0x72, + 0x2c, 0xfe, 0x3c, 0xf1, 0xb0, 0x82, 0x93, 0x32, 0xce, 0x29, 0xff, 0x2f, 0x12, 0x70, 0xa9, 0xca, + 0xde, 0xaa, 0xbb, 0x61, 0x26, 0x71, 0x0e, 0xc4, 0x12, 0xcc, 0x1e, 0x60, 0xcb, 0xd6, 0x4d, 0xb6, + 0xda, 0xcf, 0x49, 0xee, 0x25, 0x2a, 0x43, 0xda, 0x36, 0x94, 0x9e, 0xbd, 0x67, 0xba, 0xaf, 0x13, + 0xbd, 0x6b, 0x2f, 0x24, 0x66, 0xfa, 0xcd, 0x43, 0x62, 0x66, 0xc6, 0x87, 0xc4, 0xcc, 0x7e, 0x89, + 0x90, 0x18, 0xfe, 0xee, 0xee, 0xdf, 0x0b, 0x70, 0x79, 0x48, 0x73, 0x71, 0x0e, 0xce, 0xef, 0x41, + 0x56, 0xe5, 0x82, 0xc9, 0xfa, 0xc0, 0x5e, 0x4c, 0x36, 0x48, 0xb1, 0x37, 0x84, 0x49, 0xa7, 0x27, + 0x8b, 0xe0, 0x56, 0xb5, 0xb1, 0xca, 0x95, 0x43, 0x7e, 0x6b, 0xe2, 0x2f, 0xcd, 0x41, 0xa1, 0x76, + 0xc4, 0x36, 0xe5, 0x9b, 0xcc, 0x2b, 0x41, 0x4f, 0x20, 0xdd, 0xb3, 0xcc, 0x03, 0xdd, 0x6d, 0x46, + 0x3e, 0x14, 0x0f, 0xe1, 0x36, 0x63, 0x80, 0x6b, 0x8b, 0x73, 0x48, 0x1e, 0x2f, 0x6a, 0x41, 0xe6, + 0x99, 0xa9, 0x2a, 0x9d, 0x27, 0x7a, 0xc7, 0x9d, 0x68, 0xef, 0x9d, 0x2d, 0xa8, 0xe2, 0xf1, 0x6c, + 0x29, 0xce, 0x9e, 0xdb, 0x09, 0x1e, 0x11, 0x35, 0x20, 0x5d, 0x77, 0x9c, 0x1e, 0xb9, 0xc9, 0xe7, + 0xdf, 0x9d, 0x09, 0x84, 0x12, 0x16, 0x37, 0x88, 0xd7, 0x65, 0x47, 0x2d, 0x98, 0x7f, 0x4a, 0x8f, + 0xa4, 0x55, 0x3b, 0x66, 0x5f, 0xab, 0x9a, 0xc6, 0xae, 0xde, 0xe6, 0xcb, 0xc4, 0xed, 0x09, 0x64, + 0x3e, 0xad, 0x36, 0xa5, 0x61, 0x01, 0x68, 0x19, 0xd2, 0xcd, 0x47, 0x5c, 0x18, 0x73, 0x23, 0x6f, + 0x4d, 0x20, 0xac, 0xf9, 0x48, 0xf2, 0xd8, 0xd0, 0x1a, 0x64, 0x97, 0x3f, 0xef, 0x5b, 0x98, 0x4b, + 0x99, 0x19, 0x19, 0x8c, 0x31, 0x28, 0x85, 0x72, 0x49, 0x41, 0x66, 0xd4, 0x84, 0xfc, 0x2b, 0xd3, + 0xda, 0xef, 0x98, 0x8a, 0xdb, 0xc2, 0x59, 0x2a, 0xee, 0x1b, 0x13, 0x88, 0x73, 0x19, 0xa5, 0x01, + 0x11, 0xe8, 0x3b, 0x50, 0x20, 0x9d, 0xd1, 0x52, 0x76, 0x3a, 0x6e, 0x25, 0xd3, 0x54, 0xea, 0x37, + 0x27, 0x90, 0xea, 0x71, 0xba, 0xef, 0x19, 0x06, 0x44, 0x95, 0x25, 0x98, 0x0b, 0x0d, 0x02, 0x84, + 0x20, 0xd5, 0x23, 0xfd, 0x2d, 0xd0, 0x70, 0x29, 0xfa, 0x1b, 0xbd, 0x0b, 0xb3, 0x86, 0xa9, 0x61, + 0x77, 0x86, 0xcc, 0xad, 0x5c, 0x38, 0x3d, 0x59, 0x9c, 0xd9, 0x34, 0x35, 0xe6, 0x40, 0xf1, 0x5f, + 0xd2, 0x0c, 0x29, 0xd4, 0xd0, 0xca, 0xd7, 0x20, 0x45, 0xfa, 0x9d, 0x18, 0xa6, 0x1d, 0xc5, 0xc6, + 0xdb, 0x96, 0xce, 0xa5, 0xb9, 0x97, 0xe5, 0x7f, 0x94, 0x80, 0x44, 0xf3, 0x11, 0x81, 0x08, 0x3b, + 0x7d, 0x75, 0x1f, 0x3b, 0xfc, 0x3e, 0xbf, 0xa2, 0xd0, 0xc1, 0xc2, 0xbb, 0x3a, 0xf3, 0xe4, 0x32, + 0x12, 0xbf, 0x42, 0xef, 0x00, 0x28, 0xaa, 0x8a, 0x6d, 0x5b, 0x76, 0x8f, 0x2a, 0x66, 0xa4, 0x0c, + 0xa3, 0xac, 0xe3, 0x63, 0xc2, 0x66, 0x63, 0xd5, 0xc2, 0x8e, 0x1b, 0xeb, 0xc5, 0xae, 0x08, 0x9b, + 0x83, 0xbb, 0x3d, 0xd9, 0x31, 0xf7, 0xb1, 0x41, 0xc7, 0x49, 0x86, 0x98, 0x9a, 0x6e, 0xaf, 0x45, + 0x08, 0xc4, 0x4a, 0x62, 0x43, 0xf3, 0x4d, 0x5a, 0x46, 0xf2, 0xae, 0x89, 0x48, 0x0b, 0xb7, 0x75, + 0x7e, 0x9a, 0x2f, 0x23, 0xf1, 0x2b, 0xa2, 0x25, 0xa5, 0xef, 0xec, 0xd1, 0x9e, 0xc8, 0x48, 0xf4, + 0x37, 0xba, 0x0d, 0x05, 0x16, 0x1e, 0x2a, 0x63, 0x43, 0x95, 0xa9, 0x71, 0xcd, 0xd0, 0xdb, 0x73, + 0x8c, 0x5c, 0x33, 0x54, 0x62, 0x4a, 0xd1, 0x23, 0xe0, 0x04, 0x79, 0xbf, 0x6b, 0x13, 0x9d, 0x02, + 0x29, 0xb5, 0x52, 0x38, 0x3d, 0x59, 0xcc, 0x36, 0xe9, 0x8d, 0xf5, 0x8d, 0x26, 0x59, 0xa0, 0x58, + 0xa9, 0xf5, 0xae, 0xdd, 0xd0, 0xca, 0xbf, 0x2a, 0x40, 0xf2, 0x69, 0xb5, 0x79, 0x6e, 0x95, 0xb9, + 0x15, 0x4d, 0x06, 0x2a, 0x7a, 0x07, 0x0a, 0x3b, 0x7a, 0xa7, 0xa3, 0x1b, 0x6d, 0xe2, 0xb4, 0x7d, + 0x0f, 0xab, 0xae, 0xc2, 0xf2, 0x9c, 0xbc, 0xc5, 0xa8, 0xe8, 0x1a, 0x64, 0x55, 0x0b, 0x6b, 0xd8, + 0x70, 0x74, 0xa5, 0x63, 0x73, 0xcd, 0x05, 0x49, 0xe5, 0x5f, 0x14, 0x60, 0x9a, 0xce, 0x00, 0xf4, + 0x36, 0x64, 0x54, 0xd3, 0x70, 0x14, 0xdd, 0xe0, 0xa6, 0x2c, 0x23, 0xf9, 0x84, 0x91, 0xd5, 0xbb, + 0x0e, 0x39, 0x45, 0x55, 0xcd, 0xbe, 0xe1, 0xc8, 0x86, 0xd2, 0xc5, 0xbc, 0x9a, 0x59, 0x4e, 0xdb, + 0x54, 0xba, 0x18, 0x2d, 0x82, 0x7b, 0xe9, 0x9d, 0x40, 0xcd, 0x48, 0xc0, 0x49, 0xeb, 0xf8, 0xb8, + 0xfc, 0x07, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, 0xd1, 0xc6, 0x06, 0x8b, 0x79, 0x77, 0xab, 0xe1, + 0x11, 0x06, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0x5e, 0x80, 0x69, 0x87, 0x4c, 0x0b, 0x5e, 0x03, 0x76, + 0x41, 0xb7, 0xcf, 0x3b, 0x4a, 0x9b, 0xed, 0x1e, 0x66, 0x24, 0x76, 0x41, 0x1a, 0xc3, 0xa3, 0x8c, + 0x99, 0x46, 0xf8, 0x15, 0xa9, 0x29, 0x8b, 0x85, 0xdd, 0xc1, 0x6d, 0xdd, 0xa0, 0x63, 0x29, 0x29, + 0x01, 0x25, 0xad, 0x10, 0x0a, 0x7a, 0x0b, 0x32, 0xac, 0x00, 0x36, 0x34, 0x3a, 0xa0, 0x92, 0x52, + 0x9a, 0x12, 0x6a, 0x86, 0x56, 0xc6, 0x90, 0xf1, 0x26, 0x27, 0xe9, 0xb6, 0xbe, 0xed, 0x29, 0x92, + 0xfe, 0x46, 0xef, 0xc1, 0x85, 0xd7, 0x7d, 0xa5, 0xa3, 0xef, 0xd2, 0x8d, 0x41, 0x7a, 0x28, 0x80, + 0xea, 0x8c, 0xb5, 0x04, 0x79, 0xf7, 0xa8, 0x04, 0xaa, 0x3a, 0x77, 0x2e, 0x27, 0xfd, 0xb9, 0x2c, + 0xfe, 0xae, 0x00, 0xf3, 0x2c, 0xd0, 0x89, 0xc5, 0xe7, 0xc6, 0xe7, 0x87, 0x7c, 0x04, 0x19, 0x4d, + 0x71, 0x14, 0x76, 0xa6, 0x36, 0x31, 0xf6, 0x4c, 0xad, 0x77, 0xc6, 0x43, 0x71, 0x14, 0x7a, 0xae, + 0x16, 0x41, 0x8a, 0xfc, 0x66, 0xc7, 0x8f, 0x25, 0xfa, 0x5b, 0xfc, 0x14, 0x50, 0xb0, 0xa2, 0x71, + 0x7a, 0x64, 0xf7, 0xe0, 0x22, 0xd1, 0x75, 0xcd, 0x50, 0xad, 0xe3, 0x9e, 0xa3, 0x9b, 0xc6, 0x73, + 0xfa, 0xd7, 0x46, 0xc5, 0xc0, 0x7b, 0x34, 0xfa, 0xfa, 0x4c, 0xfc, 0xbd, 0x19, 0x98, 0xab, 0x1d, + 0xf5, 0x4c, 0x2b, 0xd6, 0x5d, 0xb7, 0x15, 0x98, 0xe5, 0x1b, 0x13, 0x63, 0x5e, 0x95, 0x0f, 0x18, + 0x73, 0x37, 0x4e, 0x80, 0x33, 0xa2, 0x15, 0x00, 0x16, 0x43, 0x4b, 0xe3, 0xa4, 0x92, 0xe7, 0x78, + 0xb3, 0x47, 0xd9, 0xe8, 0xf9, 0x92, 0x4d, 0xc8, 0x76, 0x0f, 0x54, 0x55, 0xde, 0xd5, 0x3b, 0x0e, + 0x0f, 0x45, 0x8c, 0x8e, 0x9a, 0xdf, 0x78, 0x59, 0xad, 0x3e, 0xa1, 0x85, 0x58, 0x54, 0xa0, 0x7f, + 0x2d, 0x01, 0x91, 0xc0, 0x7e, 0xa3, 0x6f, 0x02, 0x3f, 0xeb, 0x24, 0xdb, 0xee, 0xb1, 0xc6, 0x95, + 0xb9, 0xd3, 0x93, 0xc5, 0x8c, 0x44, 0xa9, 0xcd, 0x66, 0x4b, 0xca, 0xb0, 0x02, 0x4d, 0xdb, 0x41, + 0x37, 0x60, 0xce, 0xec, 0xea, 0x8e, 0xec, 0x3a, 0x49, 0xdc, 0xa3, 0xcc, 0x11, 0xa2, 0xeb, 0x44, + 0x9d, 0xe7, 0x08, 0xcc, 0xec, 0xe4, 0x47, 0x60, 0xfe, 0xa6, 0x00, 0x97, 0xb8, 0x22, 0xe5, 0x1d, + 0x1a, 0xf6, 0xaf, 0x74, 0x74, 0xe7, 0x58, 0xde, 0x3f, 0x28, 0xa5, 0xa9, 0xdf, 0xfa, 0x33, 0x91, + 0x1d, 0x12, 0x18, 0x07, 0x15, 0xb7, 0x5b, 0x8e, 0x9f, 0x71, 0xe6, 0xf5, 0x83, 0x9a, 0xe1, 0x58, + 0xc7, 0x2b, 0x97, 0x4f, 0x4f, 0x16, 0x17, 0x86, 0xef, 0xbe, 0x94, 0x16, 0xec, 0x61, 0x16, 0x54, + 0x07, 0xc0, 0xde, 0x38, 0xa4, 0x2b, 0x46, 0xb4, 0xff, 0x11, 0x39, 0x60, 0xa5, 0x00, 0x2f, 0xba, + 0x0b, 0x45, 0x7e, 0xe4, 0x68, 0x57, 0xef, 0x60, 0xd9, 0xd6, 0x3f, 0xc7, 0x74, 0x6d, 0x49, 0x4a, + 0x79, 0x46, 0x27, 0x22, 0x9a, 0xfa, 0xe7, 0xb8, 0xfc, 0x3d, 0x28, 0x8d, 0xaa, 0x7d, 0x70, 0x0a, + 0x64, 0xd8, 0x1b, 0xe4, 0x0f, 0xc3, 0xdb, 0x47, 0x13, 0x0c, 0x55, 0xbe, 0x85, 0xf4, 0x51, 0xe2, + 0x43, 0x41, 0xfc, 0x07, 0x09, 0x98, 0x5b, 0xe9, 0x77, 0xf6, 0x9f, 0xf7, 0x9a, 0x2c, 0xf7, 0x02, + 0x31, 0x83, 0xcc, 0x50, 0x90, 0x0a, 0x0a, 0xcc, 0x0c, 0x52, 0x4b, 0xa0, 0x7f, 0x8e, 0xc9, 0xe2, + 0x14, 0x88, 0xce, 0xe1, 0xc7, 0x1a, 0x68, 0x1b, 0x7c, 0x32, 0x3d, 0x79, 0xf0, 0x21, 0x94, 0x02, + 0x05, 0xe9, 0x5e, 0x8f, 0x8c, 0x0d, 0xc7, 0xd2, 0x31, 0xdb, 0xaf, 0x4c, 0x4a, 0x81, 0x10, 0xa2, + 0x06, 0xb9, 0x5d, 0x63, 0x77, 0x51, 0x0b, 0x72, 0xa4, 0xe0, 0xb1, 0x4c, 0x97, 0x10, 0x77, 0x3f, + 0xf9, 0x41, 0x44, 0xb3, 0x42, 0xf5, 0xae, 0x50, 0xfd, 0x54, 0x29, 0x0f, 0xfd, 0x29, 0x65, 0xb1, + 0x4f, 0x29, 0x7f, 0x02, 0xc5, 0xc1, 0x02, 0x41, 0x5d, 0xa6, 0x98, 0x2e, 0x2f, 0x04, 0x75, 0x99, + 0x0c, 0xe8, 0x69, 0x2d, 0x95, 0x4e, 0x15, 0xa7, 0xc5, 0x3f, 0x4b, 0x42, 0xde, 0x1d, 0x66, 0x71, + 0x02, 0x9d, 0x15, 0x98, 0x26, 0x83, 0xc2, 0x0d, 0x78, 0xb9, 0x3d, 0x66, 0x74, 0xf3, 0x40, 0x7a, + 0x32, 0x58, 0x5c, 0x4c, 0x4e, 0x59, 0xe3, 0x30, 0x38, 0xe5, 0x5f, 0x4c, 0x40, 0x8a, 0x62, 0x8b, + 0x07, 0x90, 0xa2, 0x0b, 0x85, 0x30, 0xc9, 0x42, 0x41, 0x8b, 0x7a, 0xcb, 0x59, 0x22, 0xe0, 0x9a, + 0x12, 0x9f, 0x6f, 0x4f, 0x79, 0xff, 0xc1, 0x43, 0x6a, 0x6c, 0x72, 0x12, 0xbf, 0x42, 0x2b, 0x34, + 0x12, 0xcb, 0xb4, 0x1c, 0xac, 0x71, 0x9f, 0xfe, 0xda, 0x59, 0xfd, 0xeb, 0x2e, 0x4a, 0x2e, 0x1f, + 0xba, 0x02, 0x49, 0x62, 0xc5, 0x66, 0x59, 0x50, 0xc5, 0xe9, 0xc9, 0x62, 0x92, 0xd8, 0x2f, 0x42, + 0x43, 0x4b, 0x90, 0x0d, 0x9b, 0x0c, 0xe2, 0xc1, 0x51, 0xc3, 0x18, 0x98, 0xee, 0xd0, 0xf1, 0xa6, + 0x16, 0xc3, 0xb3, 0xbc, 0x8f, 0xff, 0x47, 0x0a, 0xe6, 0x1a, 0xdd, 0xb8, 0x97, 0x94, 0xe5, 0x70, + 0x0f, 0x47, 0x01, 0xa1, 0xd0, 0x43, 0x23, 0x3a, 0x38, 0xb4, 0x82, 0x27, 0xcf, 0xb7, 0x82, 0x37, + 0x88, 0xa7, 0xcc, 0xf3, 0x62, 0x24, 0x47, 0x60, 0x9e, 0xf0, 0xf3, 0xa9, 0x9f, 0x22, 0x11, 0x1e, + 0xff, 0x68, 0x09, 0x0d, 0x8c, 0xf9, 0x84, 0x3a, 0xe4, 0x6c, 0x94, 0xcd, 0x4c, 0x3e, 0xca, 0x66, + 0xb1, 0xa1, 0xd1, 0x45, 0x2d, 0x6c, 0x51, 0x67, 0xdf, 0xdc, 0xa2, 0x96, 0x1d, 0x3e, 0x58, 0x3f, + 0x82, 0xa4, 0xa6, 0xbb, 0x9d, 0x33, 0xf9, 0x52, 0x4d, 0x98, 0xce, 0x18, 0xb5, 0xa9, 0xe0, 0xa8, + 0x65, 0xa3, 0xa4, 0xdc, 0x00, 0xf0, 0x75, 0x83, 0xae, 0xc1, 0x8c, 0xd9, 0xd1, 0xdc, 0xb3, 0x35, + 0x73, 0x2b, 0x99, 0xd3, 0x93, 0xc5, 0xe9, 0xe7, 0x1d, 0xad, 0xb1, 0x2a, 0x4d, 0x9b, 0x1d, 0xad, + 0xa1, 0xd1, 0xa4, 0x24, 0xf8, 0x50, 0xf6, 0x02, 0xef, 0x72, 0xd2, 0xac, 0x81, 0x0f, 0x57, 0xb1, + 0xad, 0xf2, 0x01, 0xf7, 0x9b, 0x02, 0xe4, 0x5d, 0xdd, 0xc7, 0x6b, 0x54, 0xd2, 0x7a, 0x97, 0x4f, + 0xb2, 0xe4, 0xf9, 0x26, 0x99, 0xcb, 0xc7, 0xcf, 0x3d, 0xff, 0xb2, 0xc0, 0x43, 0xa9, 0x9b, 0xaa, + 0xe2, 0x10, 0xa7, 0x22, 0xc6, 0x89, 0x71, 0x0f, 0x8a, 0x96, 0x62, 0x68, 0x66, 0x57, 0xff, 0x1c, + 0xb3, 0xcd, 0x44, 0x9b, 0xbf, 0x65, 0x2d, 0x78, 0x74, 0xba, 0xeb, 0x67, 0x8b, 0x7f, 0x90, 0xe0, + 0x61, 0xd7, 0x5e, 0x35, 0xe2, 0x54, 0xd7, 0x77, 0x61, 0x3e, 0xb8, 0xb4, 0xb1, 0xc8, 0x4e, 0x36, + 0x5b, 0xdf, 0x8d, 0x90, 0x17, 0x55, 0x11, 0x16, 0x3e, 0xe9, 0xc6, 0xf2, 0x07, 0x56, 0x44, 0x2a, + 0x0c, 0x55, 0x21, 0xcb, 0xdf, 0x75, 0x18, 0xbb, 0xa6, 0x1b, 0x97, 0xf6, 0xf6, 0xa8, 0x28, 0xcc, + 0x86, 0xb1, 0x6b, 0xba, 0x71, 0x03, 0x96, 0x4b, 0xb0, 0xcb, 0x3f, 0x07, 0xd3, 0xf4, 0xf6, 0x1b, + 0x98, 0x68, 0xde, 0x9b, 0x7f, 0x9a, 0x80, 0x9b, 0xb4, 0xf6, 0x2f, 0xb1, 0xa5, 0xef, 0x1e, 0x6f, + 0x59, 0xa6, 0x83, 0x55, 0x07, 0x6b, 0xfe, 0x46, 0x7d, 0xac, 0x76, 0x2f, 0xd3, 0x73, 0x1f, 0x70, + 0xae, 0xf8, 0x36, 0x8f, 0x0b, 0xad, 0x43, 0x81, 0x47, 0x32, 0x28, 0x1d, 0xfd, 0x00, 0xcb, 0x8a, + 0x73, 0x9e, 0xd5, 0x6d, 0x8e, 0xf1, 0x2e, 0x13, 0xd6, 0x65, 0x07, 0x69, 0x90, 0xe1, 0xc2, 0x74, + 0x8d, 0xe7, 0x4b, 0x7a, 0xfa, 0xe5, 0x36, 0x14, 0xd3, 0x2c, 0x9c, 0xa2, 0xb1, 0x2a, 0xa5, 0x99, + 0xe4, 0x86, 0x26, 0xfe, 0xed, 0x14, 0xdc, 0x3a, 0x43, 0xc5, 0x71, 0x0e, 0xdd, 0x32, 0xa4, 0x0f, + 0xc8, 0x83, 0x74, 0xae, 0xe3, 0xb4, 0xe4, 0x5d, 0xa3, 0x9d, 0x90, 0x23, 0xb6, 0xab, 0xe8, 0x1d, + 0x7f, 0x74, 0x8f, 0x8e, 0x8c, 0x8c, 0x8e, 0x03, 0x0e, 0xb8, 0x6c, 0x4f, 0xa8, 0x20, 0x3e, 0xb0, + 0xbf, 0x2f, 0x40, 0x99, 0x3d, 0x90, 0x05, 0xcf, 0x0e, 0x3c, 0x26, 0x45, 0x1f, 0xb3, 0x3a, 0x6a, + 0x12, 0x9d, 0xa5, 0xa3, 0x4a, 0xe0, 0x59, 0xbc, 0x22, 0xa5, 0xe0, 0xd3, 0x82, 0x55, 0x29, 0xff, + 0x9a, 0x00, 0xd9, 0x00, 0x01, 0xdd, 0x1e, 0x3a, 0xfd, 0x98, 0x3d, 0x8d, 0x3a, 0xf2, 0x78, 0x6b, + 0xe8, 0xc8, 0xe3, 0x4a, 0xfa, 0x8b, 0x93, 0xc5, 0x94, 0xc4, 0x8e, 0xc0, 0xb8, 0x87, 0x1f, 0xaf, + 0xfb, 0xc9, 0xb6, 0x92, 0x03, 0x85, 0xdc, 0x6c, 0x5b, 0x74, 0x53, 0x4a, 0x71, 0xdf, 0xac, 0xd3, + 0x4d, 0x29, 0x72, 0x25, 0xfe, 0x7a, 0x02, 0xe6, 0x97, 0x35, 0xad, 0xd9, 0xe4, 0xab, 0x47, 0x7c, + 0x73, 0xcc, 0x85, 0xe7, 0x09, 0x1f, 0x9e, 0xa3, 0x77, 0x01, 0x69, 0xba, 0xcd, 0x32, 0xda, 0xd8, + 0x7b, 0x8a, 0x66, 0x1e, 0xfa, 0x01, 0x34, 0xf3, 0xee, 0x9d, 0xa6, 0x7b, 0x03, 0x35, 0x81, 0xe2, + 0x44, 0xd9, 0x76, 0x14, 0xef, 0x05, 0xe1, 0xad, 0x89, 0xce, 0xfe, 0x31, 0x00, 0xe9, 0x5d, 0x4a, + 0x19, 0x22, 0x87, 0xfe, 0x24, 0x88, 0x47, 0x27, 0x9d, 0xe2, 0xc8, 0x8a, 0xed, 0x1e, 0xf4, 0x62, + 0xb9, 0x74, 0xf2, 0x8c, 0xbe, 0x6c, 0xb3, 0xf3, 0x5b, 0xec, 0x64, 0x8a, 0xaf, 0x9a, 0x38, 0x37, + 0x13, 0xfe, 0x9e, 0x00, 0x79, 0x09, 0xef, 0x5a, 0xd8, 0x8e, 0x75, 0x3b, 0xe5, 0x09, 0xe4, 0x2c, + 0x26, 0x55, 0xde, 0xb5, 0xcc, 0xee, 0x79, 0x2c, 0x52, 0x96, 0x33, 0x3e, 0xb1, 0xcc, 0x2e, 0x37, + 0xc9, 0x2f, 0xa1, 0xe0, 0xd5, 0x31, 0xce, 0xc6, 0xff, 0x2e, 0x3d, 0xd7, 0xce, 0x04, 0xc7, 0x1d, + 0xc9, 0x12, 0xaf, 0x06, 0xe8, 0x2b, 0xbe, 0x60, 0x45, 0xe3, 0x54, 0xc3, 0x7f, 0x15, 0x20, 0xdf, + 0xec, 0xef, 0xb0, 0x64, 0x6c, 0xf1, 0x69, 0xa0, 0x06, 0x99, 0x0e, 0xde, 0x75, 0xe4, 0x37, 0x3a, + 0x53, 0x91, 0x26, 0xac, 0xf4, 0x44, 0xc9, 0x53, 0x00, 0x8b, 0x9e, 0x85, 0xa4, 0x72, 0x92, 0xe7, + 0x94, 0x93, 0xa1, 0xbc, 0x84, 0x4c, 0x56, 0x9d, 0x82, 0xd7, 0xcc, 0x38, 0xd7, 0x97, 0x57, 0x21, + 0xeb, 0x90, 0x3c, 0x8f, 0x75, 0x98, 0xe7, 0xc1, 0x3b, 0xd1, 0x16, 0xa2, 0x02, 0x0b, 0xd4, 0xe5, + 0x93, 0x95, 0x5e, 0xaf, 0xa3, 0xbb, 0x1b, 0x05, 0xd4, 0xfe, 0xa4, 0xa4, 0x79, 0x7a, 0x6b, 0x99, + 0xdd, 0xa1, 0x5b, 0x04, 0xe8, 0x57, 0x04, 0xc8, 0xed, 0x5a, 0x18, 0x7f, 0x8e, 0x65, 0x6a, 0x92, + 0x27, 0x8b, 0x4e, 0x5a, 0x25, 0x75, 0xf8, 0xd2, 0xd1, 0x0b, 0x59, 0xf6, 0xe0, 0x26, 0x79, 0x2e, + 0xda, 0x84, 0xa2, 0xda, 0x61, 0xf1, 0x14, 0x5e, 0xa4, 0xd4, 0x39, 0xc0, 0x50, 0x81, 0x31, 0xfb, + 0xc1, 0x52, 0x2f, 0xc8, 0x64, 0x52, 0x34, 0x99, 0x27, 0xc0, 0xe4, 0xb0, 0xa8, 0x32, 0x22, 0xc1, + 0x45, 0x20, 0x6f, 0x66, 0x45, 0xc2, 0x8a, 0xc6, 0xbd, 0x77, 0x32, 0xaf, 0xbc, 0x0b, 0x3e, 0xaf, + 0x5e, 0xc1, 0x3c, 0x1d, 0x37, 0x71, 0x9f, 0x53, 0x17, 0xff, 0x49, 0x12, 0x50, 0x50, 0xf2, 0x57, + 0x37, 0xde, 0x12, 0xf1, 0x8d, 0xb7, 0x35, 0x10, 0x03, 0xce, 0x50, 0x47, 0xb1, 0x1d, 0x99, 0x85, + 0xe4, 0xda, 0x72, 0x0f, 0x5b, 0xb2, 0x8d, 0x55, 0x93, 0xa7, 0x2a, 0x13, 0xa4, 0xab, 0x7e, 0xc9, + 0x67, 0x8a, 0xed, 0xbc, 0x60, 0xe5, 0xb6, 0xb0, 0xd5, 0xa4, 0xa5, 0xd0, 0x32, 0x80, 0xef, 0xcd, + 0xf3, 0x25, 0x73, 0x12, 0x67, 0x3e, 0xe3, 0x39, 0xf3, 0xe8, 0x11, 0x5c, 0xea, 0x2a, 0x47, 0x51, + 0x55, 0x98, 0xa6, 0x55, 0x58, 0xe8, 0x2a, 0x47, 0x43, 0xcf, 0xfd, 0x08, 0xca, 0xd1, 0x4c, 0xb2, + 0x8d, 0xdd, 0xb7, 0xfe, 0x97, 0x22, 0x18, 0x9b, 0xd8, 0x11, 0x7f, 0x20, 0x40, 0x7e, 0x43, 0x6f, + 0x5b, 0x4a, 0xac, 0x89, 0xc0, 0xd0, 0x47, 0xe1, 0x17, 0x37, 0xd9, 0x87, 0xe5, 0xa8, 0x18, 0x36, + 0x56, 0xc2, 0xdd, 0x0e, 0xe0, 0x0c, 0x64, 0xe9, 0xf3, 0x6a, 0x14, 0xa7, 0xcd, 0xff, 0x0f, 0x65, + 0xc8, 0xf1, 0x7a, 0x6f, 0x1b, 0xba, 0x69, 0xa0, 0x07, 0x90, 0x6c, 0xf3, 0x17, 0x73, 0xd9, 0xc8, + 0x4d, 0x74, 0x3f, 0xcd, 0x66, 0x7d, 0x4a, 0x22, 0x65, 0x09, 0x4b, 0xaf, 0xef, 0x44, 0xc0, 0x16, + 0xff, 0x58, 0x47, 0x90, 0xa5, 0xd7, 0x77, 0x50, 0x13, 0x0a, 0xaa, 0x9f, 0xdb, 0x4f, 0x26, 0xec, + 0xc9, 0x91, 0x5b, 0x1c, 0x91, 0x59, 0x16, 0xeb, 0x53, 0x52, 0x5e, 0x0d, 0xdd, 0x40, 0xd5, 0x60, + 0x4a, 0xb9, 0xd4, 0x50, 0xcc, 0xa8, 0x9f, 0x90, 0x20, 0x9c, 0xce, 0xae, 0x3e, 0x15, 0xc8, 0x3c, + 0x87, 0x3e, 0x82, 0x19, 0x8d, 0x26, 0x2f, 0xe3, 0x46, 0x33, 0xaa, 0xa3, 0x43, 0x39, 0xe2, 0xea, + 0x53, 0x12, 0xe7, 0x40, 0x6b, 0x90, 0x63, 0xbf, 0x98, 0x4b, 0xcf, 0x4d, 0xdd, 0xad, 0xd1, 0x12, + 0x02, 0xce, 0x46, 0x7d, 0x4a, 0xca, 0x6a, 0x3e, 0x15, 0x7d, 0x0b, 0x52, 0xb6, 0xaa, 0xb8, 0x3b, + 0x3f, 0x57, 0x47, 0xe4, 0x00, 0xf2, 0x99, 0x69, 0x69, 0xf4, 0x98, 0x39, 0xde, 0xce, 0x91, 0xbb, + 0x09, 0x1f, 0x55, 0xfd, 0x50, 0xae, 0x09, 0x52, 0x7d, 0x4c, 0x09, 0xe8, 0x29, 0x64, 0x15, 0x82, + 0x34, 0x64, 0x7a, 0x5a, 0x9b, 0xee, 0xba, 0x47, 0x87, 0xbe, 0x0c, 0x9d, 0xae, 0xaf, 0xd3, 0x84, + 0x16, 0x2e, 0xd1, 0x17, 0xd4, 0xc5, 0x56, 0x1b, 0x97, 0xb2, 0xe3, 0x05, 0x05, 0x23, 0x45, 0x3d, + 0x41, 0x94, 0x88, 0x36, 0x60, 0xce, 0x4b, 0x95, 0x40, 0x1b, 0x95, 0x1b, 0x19, 0x6c, 0x11, 0x71, + 0x08, 0xb0, 0x3e, 0x25, 0xe5, 0xf6, 0x02, 0x64, 0x54, 0x81, 0x44, 0x5b, 0x2d, 0xcd, 0x8d, 0xb4, + 0x41, 0xde, 0x41, 0xb7, 0xfa, 0x94, 0x94, 0x68, 0xab, 0xe8, 0x13, 0x48, 0xb3, 0x53, 0x4b, 0x47, + 0x46, 0x29, 0x3f, 0x72, 0x8e, 0x85, 0xcf, 0x7e, 0xd5, 0xa7, 0x24, 0x7a, 0x50, 0x8a, 0x3c, 0x6f, + 0x0b, 0xf2, 0x16, 0x0b, 0xb5, 0x75, 0x03, 0xe3, 0x8b, 0x23, 0x03, 0x50, 0xa2, 0x62, 0xe3, 0xeb, + 0x14, 0x97, 0x07, 0xe8, 0xe8, 0xbb, 0x70, 0x21, 0x2c, 0x91, 0x8f, 0xb4, 0xf9, 0x91, 0xc1, 0x14, + 0x23, 0xe3, 0xb4, 0xeb, 0x53, 0x12, 0xb2, 0x86, 0x6e, 0xa2, 0x0f, 0x60, 0x9a, 0xf5, 0x1a, 0xa2, + 0x22, 0xa3, 0x62, 0xaa, 0x06, 0x3a, 0x8c, 0x95, 0x27, 0x83, 0xdf, 0xe1, 0xf1, 0xa6, 0x72, 0xc7, + 0x6c, 0x97, 0x16, 0x46, 0x0e, 0xfe, 0xe1, 0xc8, 0x59, 0x32, 0xf8, 0x1d, 0x9f, 0x4a, 0xfa, 0xdd, + 0x62, 0x77, 0x78, 0xe0, 0xdc, 0x85, 0x91, 0xfd, 0x1e, 0x11, 0x86, 0x5a, 0xa7, 0x27, 0x81, 0x7c, + 0x32, 0xa9, 0x9a, 0xc5, 0xd2, 0x56, 0xc9, 0x74, 0x4e, 0x5d, 0x1c, 0x59, 0xb5, 0xe1, 0xbc, 0x5e, + 0x75, 0xea, 0x87, 0x7b, 0x54, 0xf4, 0x12, 0x8a, 0x3c, 0xb9, 0x8c, 0xff, 0xc6, 0xef, 0x12, 0x95, + 0x77, 0x2f, 0xd2, 0x74, 0x45, 0x45, 0xcc, 0xd5, 0x89, 0x6b, 0x13, 0xbe, 0x83, 0x3e, 0x85, 0x79, + 0x2a, 0x4f, 0x56, 0xfd, 0x7c, 0x40, 0xa5, 0xd2, 0x50, 0x76, 0x99, 0xd1, 0xa9, 0x83, 0x5c, 0xc9, + 0x45, 0x75, 0xe0, 0x16, 0x19, 0xc6, 0xba, 0xa1, 0x3b, 0xd4, 0xca, 0x96, 0x47, 0x0e, 0xe3, 0x70, + 0x2e, 0x54, 0x32, 0x8c, 0x75, 0x46, 0x21, 0xc3, 0xd8, 0xe1, 0x01, 0xa8, 0xbc, 0x3b, 0xde, 0x1e, + 0x39, 0x8c, 0xa3, 0x22, 0x55, 0xc9, 0x30, 0x76, 0x82, 0x74, 0x32, 0x8c, 0x99, 0x81, 0x18, 0x90, + 0xfb, 0xce, 0xc8, 0x61, 0x3c, 0x32, 0x6b, 0x02, 0x19, 0xc6, 0xca, 0xd0, 0x4d, 0xb4, 0x0a, 0xc0, + 0x5c, 0x66, 0xea, 0x76, 0x5c, 0x1d, 0xb9, 0x18, 0x0c, 0x06, 0xa2, 0x92, 0xc5, 0xa0, 0xe3, 0xd2, + 0x88, 0x21, 0xa3, 0x80, 0x5c, 0xa6, 0x01, 0x0e, 0xa5, 0xc5, 0x91, 0x86, 0x6c, 0x28, 0x18, 0x81, + 0x18, 0xb2, 0x43, 0x8f, 0x48, 0x56, 0x15, 0xf6, 0x46, 0xa6, 0x74, 0x6d, 0xb4, 0x59, 0x0e, 0xbe, + 0x98, 0xa5, 0x66, 0x99, 0x12, 0xd0, 0x32, 0x64, 0x88, 0x17, 0x73, 0x4c, 0xcd, 0xd0, 0xf5, 0x91, + 0xb8, 0x67, 0xe0, 0xa8, 0x5b, 0x7d, 0x4a, 0x4a, 0xbf, 0xe6, 0x24, 0xf2, 0x78, 0xb6, 0x57, 0x5d, + 0x12, 0x47, 0x3e, 0x3e, 0xf4, 0x5e, 0x83, 0x3c, 0x9e, 0x71, 0x20, 0x15, 0x2e, 0xb2, 0xbe, 0xe2, + 0x09, 0x0c, 0x2c, 0x7e, 0xe6, 0xbe, 0x74, 0x83, 0x8a, 0x1a, 0xb9, 0xe9, 0x1b, 0x99, 0x57, 0xa1, + 0x3e, 0x25, 0x2d, 0x28, 0xc3, 0x77, 0xc9, 0x84, 0xe7, 0x4b, 0x0f, 0xdb, 0x2a, 0x2e, 0xdd, 0x1c, + 0x39, 0xe1, 0x23, 0x76, 0xd8, 0xc9, 0x84, 0x57, 0x02, 0x64, 0xb6, 0x00, 0x69, 0xb2, 0x6d, 0xb3, + 0x70, 0x98, 0x5b, 0x63, 0x16, 0xa0, 0x81, 0x9d, 0x26, 0xb6, 0x00, 0x69, 0x4d, 0xc6, 0x49, 0x04, + 0xa9, 0x1d, 0xac, 0x58, 0xdc, 0xcc, 0xde, 0x1e, 0x29, 0x68, 0x28, 0xed, 0x28, 0x11, 0xa4, 0x7a, + 0x44, 0xe2, 0xf0, 0x58, 0x6e, 0x56, 0x2a, 0xee, 0xb0, 0xdf, 0x19, 0xe9, 0xf0, 0x44, 0xa6, 0xcd, + 0x22, 0x0e, 0x8f, 0x15, 0xba, 0x81, 0x7e, 0x16, 0x66, 0xf9, 0x16, 0x41, 0xe9, 0xee, 0x18, 0x17, + 0x30, 0xb8, 0xab, 0x43, 0xe6, 0x35, 0xe7, 0x61, 0x56, 0x96, 0x6d, 0x4d, 0xb0, 0xe6, 0xdd, 0x1b, + 0x63, 0x65, 0x87, 0x76, 0x47, 0x98, 0x95, 0xf5, 0xc9, 0xc4, 0xca, 0xb2, 0x71, 0xca, 0xd7, 0xba, + 0xfb, 0x23, 0xad, 0xec, 0xf0, 0x41, 0x3a, 0x62, 0x65, 0x5f, 0xfb, 0x54, 0xd2, 0x32, 0x9b, 0x41, + 0xf4, 0xd2, 0x37, 0x46, 0xb6, 0x2c, 0xbc, 0x57, 0x41, 0x5a, 0xc6, 0x79, 0x48, 0xb7, 0x31, 0xd0, + 0xc1, 0x34, 0xfd, 0xcd, 0xd1, 0x59, 0x42, 0x06, 0x41, 0x5f, 0xdd, 0x7d, 0x8d, 0xc0, 0x34, 0xec, + 0x19, 0x2a, 0x8b, 0xa7, 0x30, 0xe0, 0x9a, 0x7a, 0x77, 0xbc, 0xa1, 0x8a, 0x4a, 0xfd, 0xe0, 0x19, + 0xaa, 0xd0, 0x4d, 0x5a, 0x55, 0x76, 0x6a, 0x95, 0xce, 0xef, 0xca, 0x98, 0x84, 0x26, 0x03, 0x67, + 0x87, 0x69, 0x55, 0x3d, 0xa2, 0x3f, 0x85, 0xfa, 0x2c, 0xdb, 0x4e, 0x69, 0x69, 0xfc, 0x14, 0x0a, + 0xe7, 0xfb, 0xf1, 0xa6, 0x10, 0x27, 0x7b, 0x6b, 0xa6, 0xeb, 0x61, 0xbc, 0x37, 0x7e, 0xcd, 0x1c, + 0x74, 0x2d, 0xd8, 0x9a, 0xc9, 0x7d, 0x8a, 0xbf, 0x21, 0xc0, 0x35, 0x56, 0x37, 0xba, 0x21, 0x7d, + 0x2c, 0x7b, 0x6f, 0x2d, 0x02, 0xfb, 0x02, 0x0f, 0xe8, 0x03, 0x3e, 0x38, 0xff, 0xf6, 0xb7, 0xfb, + 0xc8, 0x77, 0x94, 0x71, 0xe5, 0xc8, 0x90, 0xea, 0x32, 0x04, 0x55, 0x7a, 0x38, 0x72, 0x48, 0x85, + 0x51, 0x1f, 0x19, 0x52, 0x9c, 0x67, 0x65, 0x96, 0xc7, 0x3c, 0x78, 0x27, 0xce, 0x0b, 0xc5, 0xe2, + 0x5a, 0x2a, 0x7d, 0xb9, 0x58, 0x5a, 0x4b, 0xa5, 0xaf, 0x14, 0xcb, 0x6b, 0xa9, 0xf4, 0x5b, 0xc5, + 0xb7, 0xc5, 0x7f, 0x58, 0x86, 0x39, 0x17, 0x74, 0x31, 0x40, 0xf5, 0x30, 0x08, 0xa8, 0xae, 0x8e, + 0x02, 0x54, 0x1c, 0xa6, 0x71, 0x44, 0xf5, 0x30, 0x88, 0xa8, 0xae, 0x8e, 0x42, 0x54, 0x3e, 0x0f, + 0x81, 0x54, 0xad, 0x51, 0x90, 0xea, 0xde, 0x04, 0x90, 0xca, 0x13, 0x35, 0x88, 0xa9, 0x56, 0x87, + 0x31, 0xd5, 0xcd, 0xf1, 0x98, 0xca, 0x13, 0x15, 0x00, 0x55, 0x8f, 0x07, 0x40, 0xd5, 0xf5, 0x31, + 0xa0, 0xca, 0xe3, 0x77, 0x51, 0xd5, 0x7a, 0x24, 0xaa, 0xba, 0x7d, 0x16, 0xaa, 0xf2, 0xe4, 0x84, + 0x60, 0xd5, 0xfb, 0x21, 0x58, 0xb5, 0x38, 0x12, 0x56, 0x79, 0xdc, 0x0c, 0x57, 0x7d, 0x3c, 0x88, + 0xab, 0xae, 0x8f, 0xc1, 0x55, 0x7e, 0x0b, 0x38, 0xb0, 0xaa, 0x47, 0x01, 0xab, 0x5b, 0x67, 0x00, + 0x2b, 0x4f, 0x4a, 0x10, 0x59, 0xd5, 0xa3, 0x90, 0xd5, 0xad, 0x33, 0x90, 0xd5, 0x80, 0x24, 0x06, + 0xad, 0x36, 0xa3, 0xa1, 0xd5, 0x9d, 0x33, 0xa1, 0x95, 0x27, 0x2d, 0x8c, 0xad, 0x96, 0x02, 0xd8, + 0xea, 0x9d, 0x11, 0xd8, 0xca, 0x63, 0x25, 0xe0, 0xea, 0xdb, 0x43, 0xe0, 0x4a, 0x1c, 0x07, 0xae, + 0x3c, 0x5e, 0x0f, 0x5d, 0xbd, 0x18, 0x81, 0xae, 0xee, 0x9e, 0x8d, 0xae, 0x3c, 0x61, 0x03, 0xf0, + 0x4a, 0x19, 0x0b, 0xaf, 0xde, 0x9d, 0x10, 0x5e, 0x79, 0xd2, 0xa3, 0xf0, 0xd5, 0x87, 0x61, 0x7c, + 0x75, 0x6d, 0x34, 0xbe, 0xf2, 0xc4, 0x70, 0x80, 0xb5, 0x1e, 0x09, 0xb0, 0x6e, 0x9f, 0x05, 0xb0, + 0xfc, 0x79, 0x10, 0x44, 0x58, 0x9b, 0xd1, 0x08, 0xeb, 0xce, 0x99, 0x08, 0xcb, 0xef, 0xfe, 0x10, + 0xc4, 0x5a, 0x8f, 0x84, 0x58, 0xb7, 0xcf, 0x82, 0x58, 0x7e, 0xe5, 0x82, 0x18, 0xeb, 0xd5, 0x48, + 0x8c, 0x75, 0x7f, 0x12, 0x8c, 0xe5, 0x09, 0x1d, 0x02, 0x59, 0x9f, 0x8d, 0x06, 0x59, 0xdf, 0x38, + 0x47, 0x7e, 0xd6, 0x48, 0x94, 0xf5, 0xed, 0x21, 0x94, 0x25, 0x8e, 0x43, 0x59, 0xfe, 0x78, 0x76, + 0x61, 0x96, 0x32, 0x16, 0x14, 0xbd, 0x3b, 0x21, 0x28, 0xf2, 0x07, 0x5f, 0x04, 0x2a, 0xaa, 0x45, + 0xa0, 0xa2, 0x9b, 0xe3, 0x51, 0x91, 0x6f, 0xce, 0x7d, 0x58, 0x54, 0x8f, 0x82, 0x45, 0xb7, 0xce, + 0x80, 0x45, 0xbe, 0x15, 0x0a, 0xe0, 0xa2, 0xc7, 0x03, 0xb8, 0xe8, 0xfa, 0x99, 0x21, 0x7d, 0x01, + 0x60, 0xb4, 0x32, 0x0c, 0x8c, 0x6e, 0x8c, 0x05, 0x46, 0x9e, 0x04, 0x1f, 0x19, 0x3d, 0x1e, 0x40, + 0x46, 0xd7, 0xc7, 0x20, 0x23, 0xbf, 0x02, 0x1c, 0x1a, 0x69, 0xe3, 0xa1, 0x51, 0x65, 0x52, 0x68, + 0xe4, 0x09, 0x8e, 0xc4, 0x46, 0x9b, 0xd1, 0xd8, 0xe8, 0xce, 0x84, 0xd1, 0x36, 0x43, 0xe0, 0xa8, + 0x1e, 0x05, 0x8e, 0x6e, 0x9d, 0x01, 0x8e, 0x82, 0x6b, 0x88, 0x87, 0x8e, 0xea, 0x51, 0xe8, 0xe8, + 0xd6, 0x19, 0xe8, 0xc8, 0x97, 0x14, 0x80, 0x47, 0xad, 0x51, 0xf0, 0xe8, 0xde, 0x04, 0xf0, 0xc8, + 0x77, 0x5e, 0x06, 0xf0, 0xd1, 0x27, 0x83, 0xf8, 0x48, 0x1c, 0x87, 0x8f, 0xfc, 0x19, 0xe9, 0x02, + 0xa4, 0xcd, 0x68, 0x80, 0x74, 0xe7, 0x4c, 0x80, 0x14, 0x34, 0x92, 0x01, 0x84, 0xb4, 0x1e, 0x89, + 0x90, 0x6e, 0x9f, 0x85, 0x90, 0x7c, 0x23, 0x19, 0x84, 0x48, 0x9f, 0x0c, 0x42, 0x24, 0x71, 0x1c, + 0x44, 0xf2, 0x1b, 0xe7, 0x62, 0xa4, 0x7a, 0x14, 0x46, 0xba, 0x75, 0x06, 0x46, 0xf2, 0x3b, 0x2f, + 0x00, 0x92, 0x94, 0xb1, 0x20, 0xe9, 0xdd, 0x09, 0x41, 0xd2, 0x80, 0xe1, 0x0a, 0xa3, 0xa4, 0x7a, + 0x14, 0x4a, 0xba, 0x75, 0x06, 0x4a, 0x0a, 0x54, 0xd6, 0x87, 0x49, 0x9b, 0xd1, 0x30, 0xe9, 0xce, + 0x99, 0x30, 0x69, 0x60, 0x36, 0xb9, 0x38, 0x69, 0x3d, 0x12, 0x27, 0xdd, 0x3e, 0x0b, 0x27, 0x0d, + 0x2c, 0x7c, 0xdc, 0x39, 0xf8, 0xa5, 0xc9, 0x81, 0xd2, 0x87, 0x6f, 0x1a, 0x27, 0x74, 0x36, 0x52, + 0xfa, 0x64, 0x10, 0x29, 0x89, 0xe3, 0x90, 0x92, 0x3f, 0xb2, 0xce, 0x07, 0x95, 0xd6, 0x52, 0xe9, + 0xb7, 0x8b, 0xef, 0x88, 0x7f, 0x3e, 0x03, 0x33, 0x75, 0x2f, 0x08, 0xce, 0x6f, 0xa5, 0xf0, 0x26, + 0x49, 0xde, 0xd0, 0x2a, 0x99, 0xf1, 0xd4, 0x6e, 0x9e, 0x9d, 0x1a, 0x74, 0x38, 0x79, 0x25, 0x67, + 0x7d, 0x83, 0x6c, 0x0b, 0xe8, 0x7d, 0x98, 0xeb, 0xdb, 0xd8, 0x92, 0x7b, 0x96, 0x6e, 0x5a, 0xba, + 0xc3, 0x8e, 0x78, 0x09, 0x2b, 0xc5, 0x2f, 0x4e, 0x16, 0x73, 0xdb, 0x36, 0xb6, 0xb6, 0x38, 0x5d, + 0xca, 0xf5, 0x03, 0x57, 0xee, 0x67, 0xff, 0xa6, 0x27, 0xff, 0xec, 0xdf, 0x0b, 0x28, 0xd2, 0x57, + 0xe0, 0x41, 0x0f, 0x86, 0x25, 0x54, 0x8b, 0x1e, 0x73, 0xf4, 0x08, 0xa6, 0x5b, 0x92, 0x26, 0x56, + 0x2b, 0x58, 0x61, 0x22, 0x7a, 0x00, 0x17, 0xbb, 0xca, 0x11, 0x0d, 0x98, 0x96, 0x5d, 0xa7, 0x90, + 0x06, 0x41, 0xb3, 0x6f, 0xf5, 0xa1, 0xae, 0x72, 0x44, 0xbf, 0x21, 0xc8, 0x6e, 0xd1, 0x0f, 0x00, + 0xdd, 0x82, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0xcf, 0xd5, 0xcd, 0xf2, 0x5c, 0xcf, 0xb9, + 0x54, 0x96, 0x90, 0xbb, 0x0a, 0x85, 0xb6, 0xe2, 0xe0, 0x43, 0xe5, 0x58, 0x76, 0xcf, 0x5e, 0x66, + 0xe9, 0x21, 0xf6, 0xb7, 0x4e, 0x4f, 0x16, 0xe7, 0x9e, 0xb2, 0x5b, 0x43, 0x47, 0x30, 0xe7, 0xda, + 0x81, 0x1b, 0x1a, 0xba, 0x03, 0x05, 0xc5, 0x3e, 0x36, 0x54, 0xda, 0x64, 0x6c, 0xd8, 0x7d, 0x9b, + 0xc2, 0x8c, 0xb4, 0x94, 0xa7, 0xe4, 0xaa, 0x4b, 0x45, 0xd7, 0x21, 0xc7, 0x8f, 0x8e, 0xb0, 0x0f, + 0x8d, 0x15, 0x68, 0xf5, 0xf9, 0x17, 0x6c, 0xd8, 0xb7, 0xc6, 0x1e, 0x43, 0x99, 0x7f, 0x5f, 0xe3, + 0x50, 0xb1, 0x34, 0x99, 0x6a, 0xd2, 0x1f, 0x73, 0x45, 0x2a, 0xf6, 0x32, 0xfb, 0x9e, 0x06, 0x29, + 0x40, 0xd4, 0xe7, 0xcf, 0x8b, 0x16, 0xcc, 0xab, 0x1d, 0xdd, 0x43, 0x05, 0xcc, 0x89, 0x9a, 0x1f, + 0x39, 0x43, 0xaa, 0xb4, 0xec, 0xe0, 0x7b, 0xed, 0x82, 0x1a, 0x26, 0xa3, 0x26, 0xd0, 0xa4, 0x54, + 0x72, 0xcf, 0xec, 0xe8, 0xea, 0x31, 0x85, 0x05, 0xe1, 0xaf, 0x05, 0x8c, 0xfd, 0x66, 0xc7, 0x2b, + 0x45, 0x77, 0xb6, 0x28, 0xa7, 0x04, 0x87, 0xde, 0x6f, 0x96, 0xe9, 0x7b, 0x2d, 0x95, 0x86, 0x62, + 0x76, 0x2d, 0x95, 0xce, 0x15, 0xe7, 0xd6, 0x52, 0xe9, 0x7c, 0xb1, 0x20, 0xfe, 0xaa, 0x00, 0xb9, + 0xd0, 0x99, 0xb9, 0xc7, 0x03, 0x2f, 0x91, 0xaf, 0x44, 0xe3, 0xbf, 0x51, 0x51, 0xab, 0x69, 0x3e, + 0x5e, 0xdc, 0x10, 0xe0, 0xc5, 0xd1, 0xf8, 0x81, 0xee, 0x86, 0xb8, 0x51, 0x3d, 0x2e, 0xdb, 0x47, + 0xa9, 0x1f, 0xfe, 0x68, 0x71, 0x4a, 0xfc, 0x8b, 0x14, 0xcc, 0x85, 0x4f, 0xc8, 0x35, 0x06, 0xea, + 0x15, 0x65, 0x9f, 0x43, 0x1c, 0x95, 0x31, 0x1f, 0x31, 0xca, 0xf8, 0xdf, 0xf5, 0x60, 0xd5, 0xbc, + 0x36, 0xe6, 0x55, 0x79, 0xb0, 0x9e, 0x3e, 0x63, 0xf9, 0x3f, 0x26, 0x3d, 0x3b, 0x55, 0x81, 0x69, + 0x9a, 0x5b, 0x8b, 0x57, 0xad, 0x34, 0xd8, 0x57, 0xc4, 0x67, 0x25, 0xf7, 0x25, 0x56, 0x8c, 0xd8, + 0xb5, 0xd6, 0x1b, 0x25, 0xaf, 0xf4, 0x87, 0xde, 0xf9, 0x3f, 0x0f, 0xda, 0x67, 0xc9, 0x4b, 0xff, + 0x1f, 0x46, 0xfe, 0x90, 0xe7, 0xa1, 0x5f, 0x80, 0x82, 0x6a, 0x76, 0x3a, 0x6c, 0xcd, 0x62, 0x96, + 0x61, 0x38, 0x9d, 0x11, 0xad, 0x02, 0xff, 0x22, 0x6c, 0xc5, 0xfb, 0x32, 0x6c, 0x45, 0xe2, 0x5f, + 0x86, 0x0d, 0x04, 0x64, 0xe7, 0x3d, 0x61, 0xae, 0x41, 0x09, 0xc5, 0x86, 0xcf, 0xbe, 0x49, 0x6c, + 0x38, 0x3b, 0x4f, 0xc0, 0x47, 0xde, 0x1f, 0x0b, 0x3c, 0x3e, 0xe7, 0x99, 0x69, 0xee, 0xf7, 0xbd, + 0x98, 0xee, 0x72, 0x30, 0x15, 0xa9, 0x1f, 0xbc, 0x4a, 0x4f, 0x3f, 0x45, 0x59, 0xe0, 0xc4, 0x97, + 0xb3, 0xc0, 0xd7, 0x21, 0xd7, 0xb3, 0xf0, 0x2e, 0x76, 0xd4, 0x3d, 0xd9, 0xe8, 0x77, 0xf9, 0xd1, + 0xaf, 0xac, 0x4b, 0xdb, 0xec, 0x77, 0xd1, 0x3d, 0x28, 0x7a, 0x45, 0x38, 0x56, 0x76, 0x73, 0xd5, + 0xb9, 0x74, 0x8e, 0xac, 0xc5, 0xff, 0x25, 0xc0, 0x42, 0xa8, 0x4d, 0x7c, 0x4e, 0xad, 0x41, 0x56, + 0xf3, 0xd6, 0x3c, 0xbb, 0x24, 0x9c, 0x33, 0xac, 0x39, 0xc8, 0x8c, 0x64, 0xb8, 0xe4, 0x3e, 0x96, + 0x7e, 0x0b, 0xc3, 0x17, 0x9b, 0x38, 0xa7, 0xd8, 0x8b, 0xbe, 0x9c, 0xd5, 0xc0, 0x03, 0xbc, 0x49, + 0x96, 0x9c, 0x68, 0x92, 0x89, 0xbf, 0x29, 0x40, 0x91, 0x3e, 0xe0, 0x09, 0xc6, 0x5a, 0x2c, 0xd6, + 0xcd, 0x3d, 0x39, 0x90, 0x98, 0xfc, 0x70, 0x57, 0xe8, 0xfb, 0x3d, 0xc9, 0xf0, 0xf7, 0x7b, 0xc4, + 0x1f, 0x09, 0x90, 0xf7, 0x6a, 0xc8, 0xbe, 0xb1, 0x39, 0x26, 0xe3, 0xed, 0x9b, 0x7d, 0x47, 0xd2, + 0xcd, 0xcc, 0x33, 0xd1, 0x67, 0x3f, 0x83, 0x99, 0x79, 0xd8, 0xf7, 0x0f, 0xff, 0x8e, 0x3b, 0x72, + 0x48, 0x15, 0xab, 0x7e, 0x4a, 0x94, 0x37, 0x38, 0xe7, 0x26, 0xd1, 0xcf, 0x13, 0x9b, 0x9d, 0x03, + 0x96, 0x4c, 0x69, 0x22, 0xb3, 0x87, 0x78, 0x54, 0x1a, 0xf0, 0x5d, 0x35, 0xad, 0xd5, 0xa4, 0x1f, + 0x2e, 0x66, 0xbf, 0x6d, 0xf1, 0x49, 0x40, 0x81, 0xb4, 0xf3, 0x89, 0x96, 0x26, 0x32, 0xc5, 0xae, + 0x96, 0xd8, 0x58, 0xf9, 0xc3, 0x60, 0x4f, 0xd4, 0x0e, 0x08, 0x9a, 0x7a, 0x04, 0xc9, 0x03, 0xa5, + 0x33, 0x2e, 0x92, 0x2a, 0xd4, 0x73, 0x12, 0x29, 0x8d, 0x9e, 0x84, 0x32, 0xc9, 0x24, 0x46, 0x7b, + 0xfe, 0xc3, 0x2a, 0x0d, 0x65, 0x9c, 0xf9, 0x20, 0x3c, 0xd6, 0xc7, 0x3e, 0x3e, 0x38, 0xe8, 0x3f, + 0x4a, 0xfd, 0xf8, 0x47, 0x8b, 0x82, 0xf8, 0x31, 0x20, 0x09, 0xdb, 0xd8, 0x79, 0xd1, 0x37, 0x2d, + 0x3f, 0x2b, 0xcf, 0x60, 0x48, 0xff, 0x74, 0x74, 0x48, 0xbf, 0x78, 0x11, 0x16, 0x42, 0xdc, 0xcc, + 0x58, 0x88, 0x1f, 0xc0, 0x95, 0xa7, 0xa6, 0x6d, 0xeb, 0x3d, 0x02, 0x21, 0xe9, 0xac, 0x24, 0x4b, + 0x8b, 0x67, 0x1e, 0xd3, 0x3d, 0xba, 0x99, 0x60, 0x30, 0x33, 0x92, 0x91, 0xbc, 0x6b, 0xf1, 0xf7, + 0x04, 0xb8, 0x3c, 0xcc, 0xc9, 0xb4, 0x1c, 0x75, 0x2c, 0x77, 0x56, 0x35, 0xfd, 0xa4, 0x91, 0x67, + 0x8f, 0x56, 0xb7, 0x38, 0x71, 0x0b, 0xf9, 0x33, 0xe5, 0xae, 0x42, 0xcd, 0x07, 0x4f, 0x11, 0x90, + 0xe7, 0xe4, 0x0d, 0x46, 0xf5, 0x2d, 0x49, 0x6a, 0x32, 0x4b, 0xd2, 0x82, 0xc2, 0x9a, 0xa9, 0x1b, + 0xc4, 0xfb, 0x74, 0xdb, 0xbb, 0x0c, 0xf9, 0x1d, 0xdd, 0x50, 0xac, 0x63, 0xd9, 0x0d, 0xe0, 0x13, + 0xce, 0x0a, 0xe0, 0x93, 0xe6, 0x18, 0x07, 0xbf, 0x14, 0x7f, 0x22, 0x40, 0xd1, 0x17, 0xcb, 0x2d, + 0xf2, 0x37, 0x01, 0xd4, 0x4e, 0xdf, 0x76, 0xb0, 0xe5, 0xf6, 0x52, 0x8e, 0x1d, 0x14, 0xa8, 0x32, + 0x6a, 0x63, 0x55, 0xca, 0xf0, 0x02, 0x0d, 0x0d, 0xdd, 0x08, 0x67, 0x30, 0x99, 0x5e, 0x81, 0xd3, + 0xa1, 0xbc, 0x25, 0xa4, 0xdb, 0x6d, 0xc7, 0xb4, 0x3c, 0xec, 0xc2, 0xbb, 0xdd, 0x4d, 0x18, 0x45, + 0x0f, 0xde, 0x93, 0x72, 0xcb, 0x90, 0x27, 0xee, 0xc2, 0x01, 0xf6, 0x9a, 0x94, 0x3a, 0xbb, 0x49, + 0x8c, 0xc3, 0x6d, 0xd2, 0xbf, 0x12, 0xa0, 0x50, 0x65, 0xbd, 0xe1, 0xf5, 0xf0, 0x18, 0x8b, 0xb6, + 0x0a, 0x69, 0xe7, 0xc8, 0x90, 0xbb, 0xd8, 0xfb, 0x6c, 0xd2, 0x39, 0x32, 0x3a, 0xce, 0x3a, 0xec, + 0x92, 0x7e, 0x89, 0x93, 0x7f, 0x06, 0x9e, 0x4f, 0x97, 0x2b, 0x15, 0xf6, 0x9d, 0xf8, 0x8a, 0xfb, + 0x9d, 0xf8, 0xca, 0x2a, 0x2f, 0xc0, 0x8c, 0xfa, 0x0f, 0xff, 0xcb, 0xa2, 0x20, 0x79, 0x4c, 0x6c, + 0xdd, 0xbf, 0xdf, 0x24, 0xa3, 0x7e, 0x68, 0x65, 0x46, 0x79, 0x80, 0xc0, 0xf7, 0xb0, 0xf8, 0x97, + 0xc7, 0x97, 0x57, 0xe5, 0xed, 0xcd, 0xea, 0xf3, 0x8d, 0x8d, 0x46, 0xab, 0x55, 0x5b, 0x2d, 0x0a, + 0xa8, 0x08, 0xb9, 0xd0, 0xd7, 0xb4, 0x12, 0xec, 0x5b, 0xe4, 0xf7, 0xff, 0x0a, 0x80, 0xff, 0x61, + 0x3e, 0x22, 0x6b, 0xbd, 0xf6, 0xa9, 0xfc, 0x72, 0xf9, 0xd9, 0x76, 0xad, 0x59, 0x9c, 0x42, 0x08, + 0xf2, 0x2b, 0xcb, 0xad, 0x6a, 0x5d, 0x96, 0x6a, 0xcd, 0xad, 0xe7, 0x9b, 0xcd, 0x9a, 0xfb, 0x0d, + 0xf3, 0xfb, 0xab, 0x90, 0x0b, 0xe6, 0xa9, 0x42, 0x0b, 0x50, 0xa8, 0xd6, 0x6b, 0xd5, 0x75, 0xf9, + 0x65, 0x63, 0x59, 0x7e, 0xb1, 0x5d, 0xdb, 0xae, 0x15, 0xa7, 0x68, 0xd5, 0x28, 0xf1, 0xc9, 0xf6, + 0xb3, 0x67, 0x45, 0x01, 0x15, 0x20, 0xcb, 0xae, 0xe9, 0x97, 0xb7, 0x8a, 0x89, 0xfb, 0x1b, 0x90, + 0x0d, 0xe4, 0xcf, 0x26, 0x8f, 0xdb, 0xda, 0x6e, 0xd6, 0xe5, 0x56, 0x63, 0xa3, 0xd6, 0x6c, 0x2d, + 0x6f, 0x6c, 0x31, 0x19, 0x94, 0xb6, 0xbc, 0xf2, 0x5c, 0x6a, 0x15, 0x05, 0xef, 0xba, 0xf5, 0x7c, + 0xbb, 0x5a, 0x77, 0x9b, 0x21, 0xa6, 0xd2, 0xc9, 0x62, 0xf2, 0xfe, 0x5f, 0x17, 0xe0, 0xf2, 0x88, + 0x9c, 0x4d, 0x28, 0x0b, 0xb3, 0xdb, 0x06, 0x4d, 0x26, 0x5c, 0x9c, 0x42, 0x73, 0x81, 0xb4, 0x4d, + 0x45, 0x01, 0xa5, 0x59, 0xe2, 0x9c, 0x62, 0x02, 0xcd, 0x40, 0xa2, 0xf9, 0xa8, 0x98, 0x24, 0x35, + 0x0d, 0x64, 0x3d, 0x2a, 0xa6, 0x50, 0x86, 0xe7, 0x5b, 0x29, 0x4e, 0xa3, 0x9c, 0x9f, 0xf6, 0xa4, + 0x38, 0x43, 0x44, 0x79, 0xe9, 0x43, 0x8a, 0xb3, 0xf7, 0xaf, 0x43, 0x20, 0x45, 0x03, 0x02, 0x98, + 0x79, 0xa6, 0x38, 0xd8, 0x76, 0x8a, 0x53, 0x68, 0x16, 0x92, 0xcb, 0x9d, 0x4e, 0x51, 0x78, 0xf8, + 0x2f, 0x53, 0x90, 0x76, 0xbf, 0x30, 0x85, 0x9e, 0xc1, 0x34, 0xdb, 0x56, 0x5e, 0x1c, 0x8d, 0x16, + 0xe8, 0x84, 0x2e, 0x5f, 0x3b, 0x0b, 0x4e, 0x88, 0x53, 0xe8, 0xaf, 0x42, 0x36, 0xe0, 0x45, 0xa1, + 0x91, 0x5b, 0x63, 0x21, 0xcf, 0xb1, 0x7c, 0xfb, 0xac, 0x62, 0x9e, 0xfc, 0x57, 0x90, 0xf1, 0xac, + 0x3a, 0xba, 0x31, 0xce, 0xe6, 0xbb, 0xb2, 0xc7, 0x2f, 0x0c, 0x64, 0xfe, 0x89, 0x53, 0xef, 0x09, + 0xc8, 0x02, 0x34, 0x6c, 0x80, 0x51, 0x54, 0xb0, 0xc2, 0x48, 0x0b, 0x5f, 0xbe, 0x3f, 0x51, 0x69, + 0xff, 0x99, 0x44, 0x59, 0xfe, 0x2a, 0x12, 0xad, 0xac, 0xa1, 0x35, 0x2a, 0x5a, 0x59, 0x11, 0x8b, + 0xd1, 0x14, 0x7a, 0x01, 0x29, 0x62, 0x3d, 0x51, 0x94, 0x5f, 0x39, 0x60, 0xad, 0xcb, 0x37, 0xc6, + 0x96, 0x71, 0x45, 0xae, 0xdc, 0xfb, 0xf1, 0x9f, 0x5d, 0x9d, 0xfa, 0xf1, 0xe9, 0x55, 0xe1, 0x27, + 0xa7, 0x57, 0x85, 0x3f, 0x39, 0xbd, 0x2a, 0xfc, 0xe9, 0xe9, 0x55, 0xe1, 0x07, 0x3f, 0xbd, 0x3a, + 0xf5, 0x93, 0x9f, 0x5e, 0x9d, 0xfa, 0x93, 0x9f, 0x5e, 0x9d, 0xfa, 0x6c, 0x96, 0x73, 0xef, 0xcc, + 0x50, 0xd3, 0xf2, 0xe8, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x0b, 0x12, 0x83, 0xd9, 0xe9, 0x83, + 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index bdd30cb62b3a..49295aff5388 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1748,8 +1748,31 @@ message RangeStatsResponse { (gogoproto.customname) = "MVCCStats" ]; - // QueriesPerSecond is the rate of request/s or QPS for the range. - double queries_per_second = 3; + // DeprecatedLastQueriesPerSecond is the most recent rate of request/s or QPS + // for the range. The field is deprecated in favor of MaxQueriesPerSecond. + // + // TODO(nvanbenschoten): remove this field in v22.1 when all nodes in the + // cluster are guaranteed to return MaxQueriesPerSecond. + double deprecated_last_queries_per_second = 3; + + // MaxQueriesPerSecond is the maximum rate of request/s or QPS that the range + // has served over a configured measurement period. Set to -1 if the replica + // serving the RangeStats request has not been the leaseholder long enough to + // have recorded request rates for at least a full measurement period. In such + // cases, the recipient should not consider the QPS value reliable enough to + // base important decisions off of. + double max_queries_per_second = 5; + + // MaxQueriesPerSecondSet indicates that the MaxQueriesPerSecond field is set + // by the server. Used to distinguish 0 qps set by a new server from the field + // not being set at all by an old server. + // + // TODO(nvanbenschoten): stop consulting this field on the receiver in v22.1 + // when all nodes in the cluster are guaranteed to return MaxQueriesPerSecond. + // + // TODO(nvanbenschoten): stop setting this field and remove it in v22.2 when + // no nodes in the cluster consult this field. + bool max_queries_per_second_set = 6; // range_info contains descriptor and lease information. RangeInfo range_info = 4 [(gogoproto.nullable) = false];