From b6a4a8d992a048bce7a6e837127473545cc3ec57 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Tue, 27 Jun 2017 16:08:23 -0400 Subject: [PATCH 1/3] storage: don't limit empty snapshot application Scattering is *much* more reliable when empty snapshots are not limited, and this doesn't seem to have any other adverse effects in my testing. It's also early in the release cycle, so we'll have time to fix any bugs this may introduce. --- pkg/storage/helpers_test.go | 2 +- pkg/storage/store.go | 37 ++++++++++++++----------------------- pkg/storage/store_test.go | 6 ++++-- 3 files changed, 19 insertions(+), 26 deletions(-) diff --git a/pkg/storage/helpers_test.go b/pkg/storage/helpers_test.go index 727978769c45..16e366ab9812 100644 --- a/pkg/storage/helpers_test.go +++ b/pkg/storage/helpers_test.go @@ -212,7 +212,7 @@ func (s *Store) ManualReplicaGC(repl *Replica) error { } func (s *Store) ReservationCount() int { - return len(s.emptySnapshotApplySem) + len(s.nonEmptySnapshotApplySem) + return len(s.snapshotApplySem) } func NewTestStorePool(cfg StoreConfig) *StorePool { diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 87752d025964..82eae67cff06 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -433,17 +433,7 @@ type Store struct { // Semaphore to limit concurrent non-empty snapshot application and replica // data destruction. - nonEmptySnapshotApplySem chan struct{} - - // Semaphore to limit empty snapshot application. Empty snapshots are limited - // separately from non-empty snapshots so that an store with empty ranges can - // transmit those empty ranges to an underfull store without getting stuck - // behind large snapshots from a store without empty ranges. - // - // Once we have rebalancing that balances bytes and not just the number of - // ranges on each store, this separation of empty and non-empty snapshots may - // be unnecessary. - emptySnapshotApplySem chan struct{} + snapshotApplySem chan struct{} // Are rebalances to this store allowed or prohibited. Rebalances are // prohibited while a store is catching up replicas (i.e. recovering) after @@ -943,8 +933,7 @@ func NewStore(cfg StoreConfig, eng engine.Engine, nodeDesc *roachpb.NodeDescript s.tsCacheMu.cache = newTimestampCache(s.cfg.Clock) s.tsCacheMu.Unlock() - s.nonEmptySnapshotApplySem = make(chan struct{}, cfg.concurrentSnapshotApplyLimit) - s.emptySnapshotApplySem = make(chan struct{}, cfg.concurrentSnapshotApplyLimit) + s.snapshotApplySem = make(chan struct{}, cfg.concurrentSnapshotApplyLimit) if s.cfg.Gossip != nil { // Add range scanner and configure with queues. @@ -2142,14 +2131,14 @@ func (s *Store) RemoveReplica( // Destroying replica state is moderately expensive, so we serialize such // operations with applying non-empty snapshots. select { - case s.nonEmptySnapshotApplySem <- struct{}{}: + case s.snapshotApplySem <- struct{}{}: case <-ctx.Done(): return ctx.Err() case <-s.stopper.ShouldStop(): return errors.Errorf("stopped") } defer func() { - <-s.nonEmptySnapshotApplySem + <-s.snapshotApplySem }() } @@ -2702,17 +2691,17 @@ func (s *Store) maybeWaitInPushTxnQueue( func (s *Store) reserveSnapshot( ctx context.Context, header *SnapshotRequest_Header, ) (func(), error) { - sem := s.nonEmptySnapshotApplySem if header.RangeSize == 0 { - sem = s.emptySnapshotApplySem - } - - if header.CanDecline { + // Empty snapshots are exempt from rate limits because they're so cheap to + // apply. This vastly speeds up rebalancing any empty ranges created by a + // RESTORE or manual SPLIT AT, since it prevents these empty snapshots from + // getting stuck behind large snapshots managed by the replicate queue. + } else if header.CanDecline { if atomic.LoadInt32(&s.rebalancesDisabled) == 1 { return nil, nil } select { - case sem <- struct{}{}: + case s.snapshotApplySem <- struct{}{}: case <-ctx.Done(): return nil, ctx.Err() case <-s.stopper.ShouldStop(): @@ -2722,7 +2711,7 @@ func (s *Store) reserveSnapshot( } } else { select { - case sem <- struct{}{}: + case s.snapshotApplySem <- struct{}{}: case <-ctx.Done(): return nil, ctx.Err() case <-s.stopper.ShouldStop(): @@ -2735,7 +2724,9 @@ func (s *Store) reserveSnapshot( return func() { s.metrics.ReservedReplicaCount.Dec(1) s.metrics.Reserved.Dec(header.RangeSize) - <-sem + if header.RangeSize != 0 { + <-s.snapshotApplySem + } }, nil } diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index c51ddc60b68b..92b034850b17 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -2607,8 +2607,10 @@ func TestReserveSnapshotThrottling(t *testing.T) { if err != nil { t.Fatal(err) } - if n := s.ReservationCount(); n != 2 { - t.Fatalf("expected 2 reservations, but found %d", n) + // Empty snapshots are not throttled and so do not increase the reservation + // count. + if n := s.ReservationCount(); n != 1 { + t.Fatalf("expected 1 reservations, but found %d", n) } cleanupEmpty() From 1a55f9e47591c861e40076ceb87d0e6daf625ab7 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Wed, 12 Jul 2017 23:18:13 -0400 Subject: [PATCH 2/3] storage: avoid range tombstones on small ranges Having upwards of 4000 RocksDB range tombstones in one SST renders a node useless, as operations that used to take microseconds take dozens of milliseconds. Under most workloads, this situation is rare: reads don't create tombstones, and inserting data will eventually cause a compaction that cleans up these tombstones. When presplitting for a 2TB restore on an otherwise idle cluster, however, up to 12k snapshots may be applied before any data is ingested. This quickly generates more range deletion tombstones than RocksDB can handle. As a quick fix, this commit avoids deletion tombstones when clearing ranges with less than 64 keys. Down the road, we may want to investigate teaching RocksDB to automatically compact any file that exceeds a certain number of range deletion tombstones, but this should do the trick for now. --- pkg/storage/replica.go | 5 +++-- pkg/storage/replica_raftstorage.go | 27 ++++++++++++++++++++------- 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 193baeeb6b63..dd5c5b0f3ec2 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -705,9 +705,11 @@ func (r *Replica) destroyDataRaftMuLocked( batch := r.store.Engine().NewWriteOnlyBatch() defer batch.Close() + ms := r.GetMVCCStats() + // NB: this uses the local descriptor instead of the consistent one to match // the data on disk. - if err := clearRangeData(ctx, r.Desc(), r.store.Engine(), batch); err != nil { + if err := clearRangeData(ctx, r.Desc(), ms.KeyCount, r.store.Engine(), batch); err != nil { return err } clearTime := timeutil.Now() @@ -739,7 +741,6 @@ func (r *Replica) destroyDataRaftMuLocked( } } - ms := r.GetMVCCStats() log.Infof(ctx, "removed %d (%d+%d) keys in %0.0fms [clear=%0.0fms commit=%0.0fms]", ms.KeyCount+ms.SysCount, ms.KeyCount, ms.SysCount, commitTime.Sub(startTime).Seconds()*1000, diff --git a/pkg/storage/replica_raftstorage.go b/pkg/storage/replica_raftstorage.go index de11fcd2cda5..8a5d7a2a31ec 100644 --- a/pkg/storage/replica_raftstorage.go +++ b/pkg/storage/replica_raftstorage.go @@ -611,20 +611,32 @@ const ( ) func clearRangeData( - ctx context.Context, desc *roachpb.RangeDescriptor, eng engine.Engine, batch engine.Batch, + ctx context.Context, + desc *roachpb.RangeDescriptor, + keyCount int64, + eng engine.Engine, + batch engine.Batch, ) error { iter := eng.NewIterator(false) defer iter.Close() + // It is expensive for there to be many range deletion tombstones in the same + // sstable because all of the tombstones in an sstable are loaded whenever the + // sstable is accessed. So we avoid using range deletion unless there is some + // minimum number of keys. The value here was pulled out of thin air. It might + // be better to make this dependent on the size of the data being deleted. Or + // perhaps we should fix RocksDB to handle large numbers of tombstones in an + // sstable better. + const clearRangeMinKeys = 64 const metadataRanges = 2 for i, keyRange := range makeAllKeyRanges(desc) { - // The metadata ranges have a relatively small number of keys making usage - // of range tombstones (as created by ClearRange) a pessimization. + // Metadata ranges always have too few keys to justify ClearRange (see + // above), but the data range's key count needs to be explicitly checked. var err error - if i < metadataRanges { - err = batch.ClearIterRange(iter, keyRange.start, keyRange.end) - } else { + if i >= metadataRanges && keyCount >= clearRangeMinKeys { err = batch.ClearRange(keyRange.start, keyRange.end) + } else { + err = batch.ClearIterRange(iter, keyRange.start, keyRange.end) } if err != nil { return err @@ -651,6 +663,7 @@ func (r *Replica) applySnapshot( r.mu.RLock() raftLogSize := r.mu.raftLogSize replicaID := r.mu.replicaID + keyCount := r.mu.state.Stats.KeyCount r.mu.RUnlock() snapType := inSnap.snapType @@ -708,7 +721,7 @@ func (r *Replica) applySnapshot( // Delete everything in the range and recreate it from the snapshot. // We need to delete any old Raft log entries here because any log entries // that predate the snapshot will be orphaned and never truncated or GC'd. - if err := clearRangeData(ctx, s.Desc, r.store.Engine(), batch); err != nil { + if err := clearRangeData(ctx, s.Desc, keyCount, r.store.Engine(), batch); err != nil { return err } stats.clear = timeutil.Now() From dbd90cf0c331c1f109985a22f96937e75e46aa52 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Thu, 25 May 2017 15:50:09 -0400 Subject: [PATCH 3/3] storage: teach scatter to use the allocator and zone config Replace the existing "toy" implementation of scatter with a real implementation that uses the zone configuration and the allocator's recommendations. --- pkg/ccl/sqlccl/restore.go | 17 +- pkg/roachpb/api.pb.go | 957 ++++++++++++++++----------------- pkg/roachpb/api.proto | 3 +- pkg/sql/split_at.go | 10 - pkg/sql/split_at_test.go | 88 ++- pkg/storage/allocator.go | 26 +- pkg/storage/allocator_test.go | 15 +- pkg/storage/replica_command.go | 172 ++++-- pkg/storage/replicate_queue.go | 3 + 9 files changed, 667 insertions(+), 624 deletions(-) diff --git a/pkg/ccl/sqlccl/restore.go b/pkg/ccl/sqlccl/restore.go index 4242ac80f7cb..449d67501887 100644 --- a/pkg/ccl/sqlccl/restore.go +++ b/pkg/ccl/sqlccl/restore.go @@ -748,23 +748,12 @@ func Restore( req := &roachpb.AdminScatterRequest{ Span: roachpb.Span{Key: span.Key, EndKey: span.EndKey}, } - res, pErr := client.SendWrapped(scatterCtx, db.GetSender(), req) - if pErr != nil { - return pErr.GoError() - } - // Scatter is best-effort, so log why any individual ranges - // didn't get scattered. - for _, r := range res.(*roachpb.AdminScatterResponse).Ranges { - if r.Error != nil { - log.Warningf(scatterCtx, "error scattering range [%s,%s): %+v", - r.Span.Key, r.Span.EndKey, r.Error.GoError()) - } - } - return nil + _, pErr := client.SendWrapped(scatterCtx, db.GetSender(), req) + return pErr.GoError() }) } if err := g.Wait(); err != nil { - return failed, errors.Wrapf(err, "scattering %d ranges", len(importRequests)) + log.Errorf(ctx, "failed scattering %d ranges: %s", len(importRequests), err) } log.Eventf(ctx, "scattered lease holders for %d key spans", len(newSpans)) } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index a759a8a5140c..bcab253e89d1 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -1522,8 +1522,6 @@ func (*AdminScatterResponse) Descriptor() ([]byte, []int) { return fileDescripto type AdminScatterResponse_Range struct { Span Span `protobuf:"bytes,1,opt,name=span" json:"span"` - // Error will be nil if this range was successfully scattered. - Error *Error `protobuf:"bytes,2,opt,name=error" json:"error,omitempty"` } func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse_Range{} } @@ -4901,16 +4899,6 @@ func (m *AdminScatterResponse_Range) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n112 - if m.Error != nil { - dAtA[i] = 0x12 - i++ - i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n113, err := m.Error.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n113 - } return i, nil } @@ -4932,11 +4920,11 @@ func (m *AddSSTableRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n114, err := m.Span.MarshalTo(dAtA[i:]) + n113, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n114 + i += n113 if m.Data != nil { dAtA[i] = 0x12 i++ @@ -4964,11 +4952,11 @@ func (m *AddSSTableResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n115, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n114, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n115 + i += n114 return i, nil } @@ -4991,151 +4979,151 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n116, err := m.Get.MarshalTo(dAtA[i:]) + n115, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n116 + i += n115 } if m.Put != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n117, err := m.Put.MarshalTo(dAtA[i:]) + n116, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n117 + i += n116 } if m.ConditionalPut != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n118, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n117, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n118 + i += n117 } if m.Increment != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n119, err := m.Increment.MarshalTo(dAtA[i:]) + n118, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n119 + i += n118 } if m.Delete != nil { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n120, err := m.Delete.MarshalTo(dAtA[i:]) + n119, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n120 + i += n119 } if m.DeleteRange != nil { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n121, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n120, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n121 + i += n120 } if m.Scan != nil { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n122, err := m.Scan.MarshalTo(dAtA[i:]) + n121, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n122 + i += n121 } if m.BeginTransaction != nil { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n123, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n122, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n123 + i += n122 } if m.EndTransaction != nil { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n124, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n123, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n124 + i += n123 } if m.AdminSplit != nil { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n125, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n124, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n125 + i += n124 } if m.AdminMerge != nil { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n126, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n125, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n126 + i += n125 } if m.HeartbeatTxn != nil { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n127, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n126, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n127 + i += n126 } if m.Gc != nil { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n128, err := m.Gc.MarshalTo(dAtA[i:]) + n127, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n128 + i += n127 } if m.PushTxn != nil { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n129, err := m.PushTxn.MarshalTo(dAtA[i:]) + n128, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n129 + i += n128 } if m.RangeLookup != nil { dAtA[i] = 0x7a i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeLookup.Size())) - n130, err := m.RangeLookup.MarshalTo(dAtA[i:]) + n129, err := m.RangeLookup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n130 + i += n129 } if m.ResolveIntent != nil { dAtA[i] = 0x82 @@ -5143,11 +5131,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n131, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n130, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n131 + i += n130 } if m.ResolveIntentRange != nil { dAtA[i] = 0x8a @@ -5155,11 +5143,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n132, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n131, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n132 + i += n131 } if m.Merge != nil { dAtA[i] = 0x92 @@ -5167,11 +5155,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n133, err := m.Merge.MarshalTo(dAtA[i:]) + n132, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n133 + i += n132 } if m.TruncateLog != nil { dAtA[i] = 0x9a @@ -5179,11 +5167,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n134, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n133, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n134 + i += n133 } if m.RequestLease != nil { dAtA[i] = 0xa2 @@ -5191,11 +5179,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n135, err := m.RequestLease.MarshalTo(dAtA[i:]) + n134, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n135 + i += n134 } if m.ReverseScan != nil { dAtA[i] = 0xaa @@ -5203,11 +5191,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n136, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n135, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n136 + i += n135 } if m.ComputeChecksum != nil { dAtA[i] = 0xb2 @@ -5215,11 +5203,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n137, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n136, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n137 + i += n136 } if m.DeprecatedVerifyChecksum != nil { dAtA[i] = 0xba @@ -5227,11 +5215,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeprecatedVerifyChecksum.Size())) - n138, err := m.DeprecatedVerifyChecksum.MarshalTo(dAtA[i:]) + n137, err := m.DeprecatedVerifyChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n138 + i += n137 } if m.CheckConsistency != nil { dAtA[i] = 0xc2 @@ -5239,11 +5227,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n139, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n138, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n139 + i += n138 } if m.Noop != nil { dAtA[i] = 0xca @@ -5251,11 +5239,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Noop.Size())) - n140, err := m.Noop.MarshalTo(dAtA[i:]) + n139, err := m.Noop.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n140 + i += n139 } if m.InitPut != nil { dAtA[i] = 0xd2 @@ -5263,11 +5251,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n141, err := m.InitPut.MarshalTo(dAtA[i:]) + n140, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n141 + i += n140 } if m.TransferLease != nil { dAtA[i] = 0xe2 @@ -5275,11 +5263,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size())) - n142, err := m.TransferLease.MarshalTo(dAtA[i:]) + n141, err := m.TransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n142 + i += n141 } if m.AdminTransferLease != nil { dAtA[i] = 0xea @@ -5287,11 +5275,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n143, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n142, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n143 + i += n142 } if m.LeaseInfo != nil { dAtA[i] = 0xf2 @@ -5299,11 +5287,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n144, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n143, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n144 + i += n143 } if m.WriteBatch != nil { dAtA[i] = 0xfa @@ -5311,11 +5299,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n145, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n144, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n145 + i += n144 } if m.Export != nil { dAtA[i] = 0x82 @@ -5323,11 +5311,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n146, err := m.Export.MarshalTo(dAtA[i:]) + n145, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n146 + i += n145 } if m.QueryTxn != nil { dAtA[i] = 0x8a @@ -5335,11 +5323,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n147, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n146, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n147 + i += n146 } if m.Import != nil { dAtA[i] = 0x92 @@ -5347,11 +5335,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n148, err := m.Import.MarshalTo(dAtA[i:]) + n147, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n148 + i += n147 } if m.AdminChangeReplicas != nil { dAtA[i] = 0x9a @@ -5359,11 +5347,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n149, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n148, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n149 + i += n148 } if m.AdminScatter != nil { dAtA[i] = 0xa2 @@ -5371,11 +5359,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n150, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n149, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n150 + i += n149 } if m.AddSstable != nil { dAtA[i] = 0xaa @@ -5383,11 +5371,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n151, err := m.AddSstable.MarshalTo(dAtA[i:]) + n150, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n151 + i += n150 } return i, nil } @@ -5411,151 +5399,151 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n152, err := m.Get.MarshalTo(dAtA[i:]) + n151, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n152 + i += n151 } if m.Put != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n153, err := m.Put.MarshalTo(dAtA[i:]) + n152, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n153 + i += n152 } if m.ConditionalPut != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n154, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n153, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n154 + i += n153 } if m.Increment != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n155, err := m.Increment.MarshalTo(dAtA[i:]) + n154, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n155 + i += n154 } if m.Delete != nil { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n156, err := m.Delete.MarshalTo(dAtA[i:]) + n155, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n156 + i += n155 } if m.DeleteRange != nil { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n157, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n156, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n157 + i += n156 } if m.Scan != nil { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n158, err := m.Scan.MarshalTo(dAtA[i:]) + n157, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n158 + i += n157 } if m.BeginTransaction != nil { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n159, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n158, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n159 + i += n158 } if m.EndTransaction != nil { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n160, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n159, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n160 + i += n159 } if m.AdminSplit != nil { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n161, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n160, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n161 + i += n160 } if m.AdminMerge != nil { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n162, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n161, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n162 + i += n161 } if m.HeartbeatTxn != nil { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n163, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n162, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n163 + i += n162 } if m.Gc != nil { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n164, err := m.Gc.MarshalTo(dAtA[i:]) + n163, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n164 + i += n163 } if m.PushTxn != nil { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n165, err := m.PushTxn.MarshalTo(dAtA[i:]) + n164, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n165 + i += n164 } if m.RangeLookup != nil { dAtA[i] = 0x7a i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeLookup.Size())) - n166, err := m.RangeLookup.MarshalTo(dAtA[i:]) + n165, err := m.RangeLookup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n166 + i += n165 } if m.ResolveIntent != nil { dAtA[i] = 0x82 @@ -5563,11 +5551,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n167, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n166, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n167 + i += n166 } if m.ResolveIntentRange != nil { dAtA[i] = 0x8a @@ -5575,11 +5563,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n168, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n167, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n168 + i += n167 } if m.Merge != nil { dAtA[i] = 0x92 @@ -5587,11 +5575,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n169, err := m.Merge.MarshalTo(dAtA[i:]) + n168, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n169 + i += n168 } if m.TruncateLog != nil { dAtA[i] = 0x9a @@ -5599,11 +5587,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n170, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n169, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n170 + i += n169 } if m.RequestLease != nil { dAtA[i] = 0xa2 @@ -5611,11 +5599,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n171, err := m.RequestLease.MarshalTo(dAtA[i:]) + n170, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n171 + i += n170 } if m.ReverseScan != nil { dAtA[i] = 0xaa @@ -5623,11 +5611,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n172, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n171, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n172 + i += n171 } if m.ComputeChecksum != nil { dAtA[i] = 0xb2 @@ -5635,11 +5623,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n173, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n172, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n173 + i += n172 } if m.DeprecatedVerifyChecksum != nil { dAtA[i] = 0xba @@ -5647,11 +5635,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeprecatedVerifyChecksum.Size())) - n174, err := m.DeprecatedVerifyChecksum.MarshalTo(dAtA[i:]) + n173, err := m.DeprecatedVerifyChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n174 + i += n173 } if m.CheckConsistency != nil { dAtA[i] = 0xc2 @@ -5659,11 +5647,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n175, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n174, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n175 + i += n174 } if m.Noop != nil { dAtA[i] = 0xca @@ -5671,11 +5659,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Noop.Size())) - n176, err := m.Noop.MarshalTo(dAtA[i:]) + n175, err := m.Noop.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n176 + i += n175 } if m.InitPut != nil { dAtA[i] = 0xd2 @@ -5683,11 +5671,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n177, err := m.InitPut.MarshalTo(dAtA[i:]) + n176, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n177 + i += n176 } if m.AdminTransferLease != nil { dAtA[i] = 0xea @@ -5695,11 +5683,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n178, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n177, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n178 + i += n177 } if m.LeaseInfo != nil { dAtA[i] = 0xf2 @@ -5707,11 +5695,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n179, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n178, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n179 + i += n178 } if m.WriteBatch != nil { dAtA[i] = 0xfa @@ -5719,11 +5707,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n180, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n179, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n180 + i += n179 } if m.Export != nil { dAtA[i] = 0x82 @@ -5731,11 +5719,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n181, err := m.Export.MarshalTo(dAtA[i:]) + n180, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n181 + i += n180 } if m.QueryTxn != nil { dAtA[i] = 0x8a @@ -5743,11 +5731,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n182, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n181, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n182 + i += n181 } if m.Import != nil { dAtA[i] = 0x92 @@ -5755,11 +5743,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n183, err := m.Import.MarshalTo(dAtA[i:]) + n182, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n183 + i += n182 } if m.AdminChangeReplicas != nil { dAtA[i] = 0x9a @@ -5767,11 +5755,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n184, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n183, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n184 + i += n183 } if m.AdminScatter != nil { dAtA[i] = 0xa2 @@ -5779,11 +5767,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n185, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n184, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n185 + i += n184 } if m.AddSstable != nil { dAtA[i] = 0xaa @@ -5791,11 +5779,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n186, err := m.AddSstable.MarshalTo(dAtA[i:]) + n185, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n186 + i += n185 } return i, nil } @@ -5818,19 +5806,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n187, err := m.Timestamp.MarshalTo(dAtA[i:]) + n186, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n187 + i += n186 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size())) - n188, err := m.Replica.MarshalTo(dAtA[i:]) + n187, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n188 + i += n187 dAtA[i] = 0x18 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeID)) @@ -5841,11 +5829,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n189, err := m.Txn.MarshalTo(dAtA[i:]) + n188, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n189 + i += n188 } dAtA[i] = 0x30 i++ @@ -5893,11 +5881,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n190, err := m.Header.MarshalTo(dAtA[i:]) + n189, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n190 + i += n189 if len(m.Requests) > 0 { for _, msg := range m.Requests { dAtA[i] = 0x12 @@ -5931,11 +5919,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size())) - n191, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) + n190, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n191 + i += n190 if len(m.Responses) > 0 { for _, msg := range m.Responses { dAtA[i] = 0x12 @@ -5970,38 +5958,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n192, err := m.Error.MarshalTo(dAtA[i:]) + n191, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n192 + i += n191 } dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n193, err := m.Timestamp.MarshalTo(dAtA[i:]) + n192, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n193 + i += n192 if m.Txn != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n194, err := m.Txn.MarshalTo(dAtA[i:]) + n193, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n194 + i += n193 } dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n195, err := m.Now.MarshalTo(dAtA[i:]) + n194, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n195 + i += n194 if len(m.CollectedSpans) > 0 { for _, msg := range m.CollectedSpans { dAtA[i] = 0x32 @@ -6959,10 +6947,6 @@ func (m *AdminScatterResponse_Range) Size() (n int) { _ = l l = m.Span.Size() n += 1 + l + sovApi(uint64(l)) - if m.Error != nil { - l = m.Error.Size() - n += 1 + l + sovApi(uint64(l)) - } return n } @@ -17009,39 +16993,6 @@ func (m *AdminScatterResponse_Range) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Error == nil { - m.Error = &Error{} - } - if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -20511,294 +20462,294 @@ var ( func init() { proto.RegisterFile("cockroach/pkg/roachpb/api.proto", fileDescriptorApi) } var fileDescriptorApi = []byte{ - // 4614 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5c, 0xcd, 0x73, 0xdb, 0x48, - 0x76, 0x37, 0xf8, 0x21, 0x91, 0x8f, 0xa4, 0x44, 0xb5, 0x65, 0x9b, 0xe6, 0x78, 0x44, 0x19, 0xfe, - 0x1e, 0xef, 0x50, 0x1e, 0x3b, 0xce, 0x8e, 0x27, 0x33, 0xb5, 0x23, 0x51, 0xb2, 0x4c, 0xcb, 0x96, - 0x35, 0x20, 0x35, 0x9e, 0x99, 0xdd, 0x2c, 0x02, 0x01, 0x2d, 0x12, 0x6b, 0x12, 0xa0, 0x01, 0xd0, - 0x92, 0x26, 0x87, 0xad, 0x4a, 0x6d, 0xd5, 0x26, 0x39, 0xa4, 0x92, 0xaa, 0x3d, 0x64, 0xf3, 0x39, - 0x95, 0x4d, 0xaa, 0xf2, 0x07, 0xa4, 0x2a, 0x87, 0x3d, 0xe4, 0x94, 0xd4, 0xe4, 0x36, 0xb7, 0xa4, - 0x92, 0x2a, 0x57, 0xe2, 0xbd, 0x6c, 0xe5, 0x9a, 0xca, 0x65, 0x4e, 0xa9, 0xfe, 0x00, 0x08, 0x90, - 0x00, 0x49, 0xc9, 0xd8, 0x9a, 0xe4, 0x24, 0xe8, 0x75, 0xbf, 0x1f, 0xba, 0x5f, 0x77, 0xbf, 0xfe, - 0xf5, 0xeb, 0x07, 0x42, 0x45, 0x35, 0xd5, 0x67, 0x96, 0xa9, 0xa8, 0xed, 0x95, 0xde, 0xb3, 0xd6, - 0x0a, 0x7d, 0xea, 0xed, 0xad, 0x28, 0x3d, 0xbd, 0xda, 0xb3, 0x4c, 0xc7, 0x44, 0x0b, 0x5e, 0x85, - 0x2a, 0x2f, 0x2c, 0x2f, 0x87, 0xeb, 0x68, 0x8a, 0xa3, 0x30, 0xa5, 0xb2, 0x18, 0x5e, 0x03, 0x5b, - 0x96, 0x69, 0xd9, 0xbc, 0xce, 0xe5, 0xf0, 0x3a, 0x5d, 0xec, 0x28, 0x3e, 0xa4, 0x5b, 0xc1, 0x5a, - 0xb6, 0x63, 0x5a, 0x4a, 0x0b, 0xaf, 0x60, 0xa3, 0xa5, 0x1b, 0xee, 0x1f, 0xa2, 0xf5, 0x42, 0x55, - 0xb9, 0xc6, 0xd5, 0xa0, 0x46, 0xdf, 0xd1, 0x3b, 0x2b, 0xed, 0x8e, 0xba, 0xe2, 0xe8, 0x5d, 0x6c, - 0x3b, 0x4a, 0xb7, 0xc7, 0xeb, 0x55, 0x43, 0xea, 0x39, 0x96, 0xa2, 0xea, 0x46, 0x6b, 0xc5, 0xc2, - 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0x9e, 0x62, 0xf0, 0xfa, 0x8b, 0x2d, 0xb3, 0x65, 0xd2, 0xc7, - 0x15, 0xf2, 0xc4, 0xa4, 0xe2, 0x0f, 0x21, 0x2b, 0x29, 0x46, 0x0b, 0xd7, 0x8d, 0x7d, 0x13, 0xbd, - 0x0f, 0x29, 0x0d, 0xdb, 0x6a, 0x49, 0x58, 0x16, 0xae, 0xe7, 0x6e, 0x8b, 0xd5, 0x11, 0xd3, 0x55, - 0x69, 0xdd, 0x75, 0x6c, 0xab, 0x96, 0xde, 0x73, 0x4c, 0x6b, 0x2d, 0xf5, 0xe5, 0xcb, 0xca, 0x29, - 0x89, 0x6a, 0xa1, 0x5f, 0x83, 0x74, 0x07, 0x2b, 0x36, 0x2e, 0x25, 0xa8, 0x7a, 0x29, 0x44, 0xfd, - 0x11, 0x29, 0xe7, 0x4a, 0xac, 0xb2, 0xf8, 0x4a, 0x80, 0x39, 0x09, 0xdb, 0x3d, 0xd3, 0xb0, 0xf1, - 0x03, 0xac, 0x68, 0xd8, 0x42, 0xb7, 0x20, 0xe9, 0x1c, 0x1a, 0xa5, 0x24, 0x85, 0x59, 0x0a, 0x81, - 0x69, 0x5a, 0x8a, 0x61, 0x2b, 0xaa, 0xa3, 0x9b, 0x86, 0x44, 0xaa, 0xa2, 0x77, 0x21, 0x67, 0x61, - 0xbb, 0xdf, 0xc5, 0xb4, 0xc3, 0xa5, 0x14, 0xd5, 0x3c, 0x17, 0xa2, 0xd9, 0xe8, 0x29, 0x86, 0x04, - 0xac, 0x2e, 0x79, 0x46, 0x15, 0xc8, 0x18, 0xfd, 0xae, 0xfc, 0x0c, 0x1f, 0xd9, 0xa5, 0xf4, 0xb2, - 0x70, 0x3d, 0xc9, 0x5b, 0x37, 0x6b, 0xf4, 0xbb, 0x5b, 0xf8, 0xc8, 0x46, 0x35, 0xc8, 0x59, 0xa4, - 0xd3, 0xb2, 0x6e, 0xec, 0x9b, 0x76, 0x69, 0x66, 0x39, 0x79, 0x3d, 0x77, 0xfb, 0x42, 0x94, 0x69, - 0x88, 0x19, 0x39, 0x02, 0x58, 0xae, 0xc0, 0x16, 0x37, 0x01, 0x36, 0xb1, 0x23, 0xe1, 0xe7, 0x7d, - 0x6c, 0x3b, 0xe8, 0x1e, 0xcc, 0xb4, 0x69, 0x4f, 0xb9, 0xa1, 0xa3, 0x1a, 0xba, 0x96, 0x21, 0x40, - 0x5f, 0xbd, 0xac, 0x08, 0x12, 0x57, 0x10, 0x7f, 0x47, 0x80, 0x1c, 0x45, 0x62, 0x06, 0x43, 0xb5, - 0x21, 0xa8, 0x8b, 0x61, 0x0d, 0x0b, 0x58, 0x77, 0x14, 0x14, 0x55, 0x21, 0xfd, 0x42, 0xe9, 0xf4, - 0xc7, 0x0d, 0xdc, 0xc7, 0xa4, 0x5c, 0x62, 0xd5, 0xc4, 0xbf, 0x17, 0x00, 0x76, 0xfa, 0x31, 0x74, - 0x87, 0x4c, 0x99, 0xa9, 0xde, 0xec, 0x4e, 0x19, 0x5a, 0x19, 0x5d, 0x80, 0x19, 0xdd, 0xe8, 0xe8, - 0x06, 0xa6, 0x53, 0x24, 0xc3, 0x0b, 0xb9, 0x0c, 0x95, 0x21, 0xbd, 0xd7, 0xd1, 0x0d, 0x8d, 0xce, - 0x02, 0xb7, 0x90, 0x89, 0x44, 0x09, 0x72, 0xb4, 0xe1, 0x31, 0x5a, 0x4f, 0xfc, 0x77, 0x01, 0xce, - 0xd4, 0x4c, 0x43, 0xd3, 0xc9, 0x74, 0x54, 0x3a, 0xdf, 0xa4, 0x61, 0xee, 0x42, 0x16, 0x1f, 0xf6, - 0x64, 0xa6, 0x99, 0x9c, 0x30, 0x98, 0x19, 0x7c, 0xd8, 0xa3, 0x4f, 0x63, 0x2d, 0xf6, 0x9b, 0x70, - 0x76, 0xb8, 0x73, 0x71, 0x1a, 0xef, 0xcf, 0x05, 0x98, 0xab, 0x1b, 0xba, 0xf3, 0x4d, 0x5a, 0xcd, - 0xeb, 0x7e, 0x72, 0xb4, 0xfb, 0x1f, 0xc3, 0xbc, 0xd7, 0xbc, 0x38, 0xfb, 0xfd, 0x1c, 0x8a, 0x75, - 0x43, 0xb5, 0x70, 0x17, 0x1b, 0x71, 0x74, 0x5c, 0x84, 0xac, 0xee, 0xc2, 0xd1, 0xce, 0xbb, 0x6e, - 0x6c, 0x20, 0x16, 0x7f, 0x1b, 0x16, 0x7c, 0xaf, 0x8c, 0xd3, 0x7f, 0x5c, 0x84, 0xac, 0x81, 0x0f, - 0xe4, 0x81, 0xe9, 0xdd, 0xb7, 0x67, 0x0c, 0x7c, 0x40, 0x0d, 0x2e, 0x3e, 0x84, 0xc2, 0x3a, 0xee, - 0x60, 0x07, 0xc7, 0xe0, 0x03, 0x77, 0x61, 0xce, 0xc5, 0x8a, 0x73, 0x48, 0xfe, 0x44, 0x00, 0xc4, - 0x71, 0x89, 0xe3, 0x8e, 0x61, 0x54, 0xae, 0x90, 0x5d, 0xc9, 0xe9, 0x5b, 0x06, 0xdb, 0x5e, 0xfc, - 0xd3, 0x0b, 0x58, 0x01, 0xdd, 0x61, 0x06, 0xee, 0x2c, 0x35, 0xea, 0xce, 0x1e, 0xa6, 0x32, 0x89, - 0x62, 0x52, 0x3c, 0x80, 0xd3, 0x81, 0xb6, 0xc5, 0x39, 0x7c, 0x6f, 0x40, 0x8a, 0xb6, 0x2f, 0xb1, - 0x9c, 0xbc, 0x9e, 0x5f, 0x9b, 0xfd, 0xfa, 0x65, 0x25, 0xb9, 0x85, 0x8f, 0x24, 0x2a, 0x14, 0xb7, - 0x21, 0xd7, 0x50, 0x15, 0xe3, 0xf5, 0xad, 0xc1, 0x3b, 0xf2, 0xfb, 0x02, 0xe4, 0x19, 0x60, 0x9c, - 0x5d, 0xb8, 0x0b, 0x29, 0xcb, 0x3c, 0x60, 0x5d, 0xc8, 0xdd, 0x7e, 0x23, 0x04, 0x62, 0x0b, 0x1f, - 0xf9, 0x97, 0x3e, 0xad, 0x2e, 0xee, 0x02, 0x92, 0xf0, 0x0b, 0x6c, 0xd9, 0x38, 0xd6, 0x3e, 0xfe, - 0x91, 0x00, 0xa7, 0x03, 0xb8, 0xff, 0x07, 0xba, 0x7a, 0x00, 0xe7, 0x6a, 0x6d, 0xac, 0x3e, 0xab, - 0x99, 0x86, 0xad, 0xdb, 0x0e, 0x36, 0xd4, 0xa3, 0x18, 0x66, 0xf8, 0x45, 0xc8, 0x1e, 0xe8, 0x4e, - 0x5b, 0xd6, 0xf4, 0xfd, 0x7d, 0xba, 0xf2, 0xdd, 0xd9, 0x9b, 0x21, 0xe2, 0x75, 0x7d, 0x7f, 0x5f, - 0x94, 0xa1, 0x34, 0xfa, 0xe2, 0x38, 0xd7, 0x6d, 0x13, 0xce, 0xad, 0xe1, 0x96, 0x6e, 0xf8, 0x49, - 0xe1, 0xeb, 0x3b, 0x19, 0x19, 0x4a, 0xa3, 0xa8, 0x71, 0x36, 0xfb, 0x47, 0x49, 0x38, 0xb3, 0x61, - 0x68, 0xb1, 0xb6, 0x9a, 0xb8, 0x12, 0xd5, 0xec, 0x76, 0x75, 0x27, 0x30, 0x18, 0x5c, 0x86, 0xee, - 0x41, 0x46, 0xc3, 0x8a, 0xe6, 0x31, 0xa7, 0xdc, 0xed, 0x37, 0x7d, 0xd0, 0xe4, 0x00, 0x51, 0x6d, - 0x77, 0xd4, 0x6a, 0xd3, 0x3d, 0x68, 0x48, 0x5e, 0x75, 0xf4, 0x5b, 0x70, 0x4e, 0x37, 0x1c, 0x6c, - 0x19, 0x4a, 0x47, 0x66, 0x68, 0xb2, 0x63, 0xe9, 0xad, 0x16, 0xb6, 0x38, 0xd9, 0xbe, 0x1e, 0xd2, - 0xc8, 0x3a, 0xd7, 0xa8, 0x51, 0x85, 0x26, 0xab, 0x2f, 0x9d, 0xd1, 0xc3, 0xc4, 0xe8, 0x43, 0xc8, - 0x93, 0x02, 0xc3, 0xa1, 0x14, 0x9e, 0x90, 0xf1, 0xe4, 0xb8, 0xbe, 0xb3, 0x9e, 0xe5, 0x98, 0x0a, - 0x91, 0xd8, 0xe8, 0x0e, 0x71, 0xb7, 0xcf, 0xfb, 0xba, 0x85, 0xe5, 0x77, 0x7a, 0x6a, 0x69, 0x86, - 0x5a, 0x00, 0x91, 0x7a, 0xaf, 0x5e, 0x56, 0x40, 0x62, 0x45, 0xef, 0xec, 0xd4, 0x88, 0xf3, 0x65, - 0xcf, 0x3d, 0x55, 0xfc, 0xa9, 0x00, 0x67, 0x87, 0x87, 0x21, 0x5e, 0x6e, 0x5d, 0x34, 0x0d, 0x2c, - 0xf7, 0xda, 0x8a, 0x8d, 0xb9, 0xe5, 0x02, 0x6e, 0x7e, 0xce, 0x34, 0xf0, 0x0e, 0x29, 0x64, 0xd6, - 0x60, 0x1e, 0xe4, 0x61, 0x2a, 0x93, 0x2c, 0xa6, 0x44, 0x07, 0x16, 0x56, 0xb5, 0xae, 0x6e, 0x34, - 0x7a, 0x1d, 0x3d, 0x0e, 0x96, 0x70, 0x19, 0xb2, 0x36, 0x81, 0x22, 0xdb, 0x11, 0x9d, 0x20, 0x3e, - 0x6f, 0x9f, 0xa1, 0x25, 0x5b, 0xf8, 0x48, 0xfc, 0x14, 0x90, 0xff, 0xad, 0x71, 0xce, 0xf9, 0x6d, - 0xde, 0xa1, 0xc7, 0xd8, 0x8a, 0x63, 0x83, 0xf5, 0x9a, 0xca, 0xf1, 0xe2, 0x6c, 0xea, 0x0f, 0xe1, - 0x3c, 0x85, 0xa6, 0x13, 0x63, 0x1f, 0x5b, 0xf4, 0xe4, 0x1a, 0xc3, 0x18, 0x5c, 0x83, 0x19, 0x47, - 0xb1, 0x5a, 0x98, 0xad, 0xd0, 0xf4, 0xda, 0x3c, 0xa9, 0xf1, 0xf5, 0xcb, 0xca, 0x6c, 0xc3, 0x31, - 0x2d, 0x5c, 0x5f, 0x97, 0x78, 0xb1, 0xa8, 0x40, 0x39, 0xac, 0x01, 0x71, 0xf6, 0xf1, 0xbf, 0x04, - 0xfe, 0x8e, 0x5a, 0x9b, 0xb1, 0x8a, 0x5e, 0x47, 0x57, 0x15, 0x3b, 0x86, 0x5e, 0x6e, 0x41, 0x4e, - 0xa5, 0x98, 0xb2, 0x73, 0xd4, 0x63, 0x9c, 0x70, 0xee, 0xf6, 0xe5, 0xd0, 0x36, 0xd2, 0x77, 0xb2, - 0x06, 0x34, 0x8f, 0x7a, 0xee, 0xa6, 0x05, 0xaa, 0x27, 0x41, 0xeb, 0x30, 0xcb, 0x6c, 0x42, 0x28, - 0x14, 0x71, 0x0a, 0x63, 0x80, 0xc8, 0x02, 0x6e, 0xd2, 0xca, 0xee, 0x39, 0x9e, 0xab, 0x8a, 0x7b, - 0xf0, 0x46, 0x68, 0x5f, 0xe3, 0xa6, 0x90, 0x94, 0xa0, 0x3d, 0x32, 0xcd, 0x67, 0xfd, 0x5e, 0x0c, - 0x86, 0xbc, 0x04, 0xd0, 0x55, 0x0e, 0x65, 0x1a, 0x4a, 0xb0, 0xf9, 0x94, 0xe1, 0xcc, 0xbe, 0xab, - 0x1c, 0xd2, 0x77, 0xd9, 0x68, 0x09, 0x66, 0x2d, 0x46, 0x37, 0x02, 0xae, 0xc5, 0x15, 0x72, 0x6f, - 0xf2, 0xdf, 0x84, 0x95, 0xf8, 0x1b, 0x17, 0xa7, 0x9b, 0xfb, 0x10, 0x66, 0xbc, 0x36, 0x26, 0x8f, - 0x15, 0x3b, 0xe2, 0x7a, 0x68, 0x17, 0x16, 0x7a, 0x16, 0xde, 0xc7, 0x8e, 0xda, 0xc6, 0x9a, 0xdb, - 0xe1, 0xe4, 0x31, 0xc1, 0x8a, 0x03, 0x08, 0x66, 0x1b, 0xf1, 0xc7, 0x02, 0x9c, 0x7e, 0x80, 0x15, - 0xcb, 0xd9, 0xc3, 0x8a, 0xd3, 0x3c, 0x8c, 0x63, 0x93, 0xbd, 0x0b, 0x49, 0xc3, 0x3c, 0xe0, 0x67, - 0xcc, 0xf1, 0x3b, 0x28, 0x6f, 0x16, 0xa9, 0x2f, 0x7e, 0x17, 0x16, 0x83, 0x0d, 0x89, 0x73, 0xe6, - 0xfd, 0x59, 0x12, 0xb2, 0x9b, 0xb5, 0x18, 0x3a, 0xf7, 0x3e, 0x3f, 0x0c, 0x44, 0x5b, 0xde, 0x7b, - 0x4d, 0x75, 0xb3, 0xb6, 0x85, 0x8f, 0x5c, 0x96, 0x49, 0xb4, 0xd0, 0x2a, 0x64, 0x9d, 0xb6, 0x85, - 0xed, 0xb6, 0xd9, 0xd1, 0x38, 0x31, 0x98, 0xca, 0x40, 0x03, 0x2d, 0xd4, 0x81, 0x33, 0xce, 0xa1, - 0x41, 0x49, 0x80, 0xdc, 0x52, 0xe5, 0x01, 0x5c, 0x7a, 0x1a, 0xb8, 0x32, 0xdf, 0xee, 0x51, 0xf3, - 0xd0, 0x20, 0x3d, 0xdc, 0xac, 0x35, 0x5d, 0x00, 0x09, 0x39, 0x5c, 0xa6, 0x7a, 0xb2, 0x32, 0x86, - 0x34, 0xed, 0x05, 0x3a, 0x0f, 0x49, 0xb2, 0x2b, 0x0a, 0xc1, 0x5d, 0x91, 0xc8, 0x68, 0xa7, 0xdc, - 0x17, 0x1c, 0x67, 0xd4, 0x07, 0x5a, 0xe2, 0x47, 0x00, 0xc4, 0x6c, 0x71, 0x8e, 0xf8, 0xcf, 0x93, - 0x30, 0xb7, 0xd3, 0xb7, 0xdb, 0xf1, 0xcc, 0xe9, 0x1a, 0x40, 0xaf, 0x6f, 0xb7, 0xb1, 0x25, 0x3b, - 0x87, 0x06, 0xef, 0xe4, 0x84, 0xc8, 0xab, 0xdb, 0x4b, 0xa6, 0xd7, 0x3c, 0x34, 0xd0, 0x13, 0x0e, - 0x82, 0xe5, 0x41, 0xf8, 0xf6, 0x2d, 0x1f, 0x08, 0x0f, 0x7e, 0x57, 0x59, 0xd4, 0xbb, 0xea, 0x06, - 0xbf, 0xab, 0xcd, 0x43, 0xe3, 0x31, 0x76, 0x94, 0x00, 0x20, 0x26, 0x80, 0xef, 0xc3, 0x2c, 0xf9, - 0x47, 0x76, 0xcc, 0xe3, 0x4c, 0xa6, 0x19, 0xa2, 0xd3, 0x34, 0xdd, 0x75, 0x9a, 0x3e, 0xde, 0x3a, - 0x25, 0xc3, 0xcd, 0x5e, 0x4a, 0x76, 0xae, 0x19, 0xba, 0x73, 0x85, 0x59, 0x82, 0xdb, 0xde, 0xb7, - 0x67, 0x65, 0xe8, 0x7b, 0xc9, 0x8e, 0x55, 0x86, 0xf4, 0xbe, 0x69, 0xa9, 0xb8, 0x34, 0xeb, 0x8f, - 0x28, 0x51, 0xd1, 0xc3, 0x54, 0x26, 0x53, 0xcc, 0x8a, 0x7f, 0x2a, 0xc0, 0xbc, 0x37, 0x7a, 0x71, - 0x3a, 0xe2, 0x5a, 0x60, 0x0c, 0x8e, 0x3f, 0x90, 0xc4, 0xee, 0xe2, 0xdf, 0x24, 0x60, 0xfe, 0xa3, - 0x3e, 0xb6, 0x8e, 0xe2, 0x99, 0x5c, 0x6b, 0x2c, 0x9e, 0x9f, 0x38, 0xe1, 0x84, 0xa0, 0x11, 0xfe, - 0x6f, 0xc1, 0xfc, 0x81, 0xa2, 0x3b, 0xf2, 0xbe, 0x69, 0xc9, 0xfd, 0x9e, 0xa6, 0x38, 0xc1, 0xe0, - 0x6f, 0x81, 0x14, 0xde, 0x37, 0xad, 0x5d, 0x5a, 0x84, 0x30, 0xa0, 0x67, 0x86, 0x79, 0x60, 0xc8, - 0x44, 0xac, 0x1b, 0x2d, 0x62, 0x0c, 0xbb, 0x94, 0xa2, 0x01, 0x8e, 0x6f, 0xff, 0xdb, 0xcb, 0xca, - 0x9d, 0x96, 0xee, 0xb4, 0xfb, 0x7b, 0x55, 0xd5, 0xec, 0xae, 0x78, 0xcd, 0xd1, 0xf6, 0x56, 0x42, - 0xae, 0x54, 0xfa, 0x7d, 0x5d, 0xab, 0xee, 0xee, 0xd6, 0xd7, 0xa5, 0x22, 0x85, 0x7c, 0xca, 0x10, - 0x9b, 0x87, 0x86, 0x2d, 0x7e, 0x2d, 0x40, 0x71, 0x60, 0xa7, 0x38, 0x87, 0x71, 0x03, 0x72, 0xcf, - 0xfb, 0xd8, 0xd2, 0xb1, 0x76, 0xec, 0x71, 0x04, 0xae, 0x48, 0x16, 0xd0, 0x67, 0x90, 0x0f, 0x58, - 0x20, 0xf9, 0x7a, 0x16, 0xc8, 0x1d, 0xf8, 0x3a, 0xff, 0xa3, 0x04, 0x2c, 0x4a, 0xd8, 0x36, 0x3b, - 0x2f, 0x70, 0x9d, 0x9e, 0xc2, 0x62, 0x98, 0x29, 0x4f, 0x00, 0xf8, 0x21, 0xf0, 0x75, 0x26, 0x4c, - 0x96, 0x61, 0x10, 0x03, 0xac, 0xc1, 0x8c, 0xed, 0x28, 0x4e, 0x9f, 0x45, 0xdf, 0xc2, 0x39, 0xa8, - 0xcf, 0x84, 0x0d, 0x5a, 0xd7, 0xf5, 0x23, 0x4c, 0x93, 0x1c, 0xaa, 0x7b, 0xa6, 0x6e, 0x9b, 0x46, - 0x30, 0x3e, 0xc7, 0x64, 0xe2, 0xf7, 0xe0, 0xcc, 0x90, 0x15, 0xe2, 0xf4, 0xf2, 0xbf, 0x97, 0x80, - 0xf3, 0x41, 0xf8, 0x98, 0x62, 0x93, 0xff, 0x0f, 0x2d, 0x3d, 0x07, 0xf9, 0x6d, 0xd3, 0xf4, 0x88, - 0xab, 0x58, 0x80, 0x1c, 0xfb, 0x9f, 0x1a, 0x83, 0x1c, 0x98, 0xc2, 0x2c, 0x15, 0xef, 0xa1, 0x30, - 0x1f, 0xd3, 0xd1, 0xf5, 0x64, 0x57, 0x15, 0x62, 0x13, 0x0a, 0xbf, 0x82, 0xb3, 0xee, 0x5f, 0x09, - 0x80, 0x9a, 0x56, 0xdf, 0x50, 0x15, 0x07, 0x3f, 0x32, 0x5b, 0x31, 0xf4, 0xae, 0x0c, 0x69, 0xdd, - 0xd0, 0xf0, 0x21, 0xed, 0x5d, 0xca, 0xed, 0x03, 0x15, 0xa1, 0xbb, 0x90, 0xe1, 0x17, 0xaa, 0xec, - 0xc6, 0x25, 0xe9, 0x91, 0xb6, 0x59, 0x76, 0x85, 0xba, 0xfe, 0xf5, 0xe0, 0x51, 0x9a, 0x65, 0xb7, - 0xa8, 0x9a, 0xf8, 0x19, 0x9c, 0x0e, 0xb4, 0x31, 0x4e, 0x03, 0xfc, 0x23, 0x0d, 0xd8, 0xd2, 0x5e, - 0xc7, 0x75, 0xce, 0x3f, 0xd1, 0x65, 0x38, 0xfa, 0x00, 0xa0, 0x67, 0xe1, 0x17, 0x32, 0x53, 0x4d, - 0x4e, 0xa5, 0x9a, 0x25, 0x1a, 0x54, 0x20, 0xfe, 0x93, 0x00, 0x8b, 0x71, 0x07, 0x2c, 0xbe, 0x91, - 0x8e, 0x3c, 0x86, 0x22, 0x7d, 0xa8, 0x1b, 0xfb, 0x66, 0x0c, 0x71, 0xa2, 0x3f, 0x10, 0x60, 0xc1, - 0x87, 0x17, 0xe7, 0x46, 0x7d, 0xb2, 0xa4, 0x87, 0xef, 0x92, 0xad, 0xd3, 0x3f, 0xdf, 0xe2, 0x9c, - 0xcd, 0x3f, 0x4e, 0xc0, 0xd9, 0x9a, 0xd9, 0xed, 0xf5, 0x1d, 0x4c, 0x23, 0xef, 0x76, 0xbf, 0x1b, - 0xc3, 0x3c, 0x58, 0x82, 0xd9, 0x17, 0xd8, 0xb2, 0x75, 0x93, 0xed, 0x16, 0x05, 0x37, 0xc8, 0xc0, - 0x85, 0xe8, 0x07, 0x90, 0x53, 0xf9, 0xdb, 0xdc, 0x95, 0x9d, 0x5f, 0xab, 0x93, 0x3a, 0x27, 0x64, - 0x1b, 0xaf, 0x5e, 0x56, 0xc0, 0x6d, 0x7f, 0x7d, 0x5d, 0x02, 0x17, 0xbd, 0xae, 0xa1, 0x65, 0xc8, - 0xd8, 0x86, 0xd2, 0xb3, 0xdb, 0x66, 0x30, 0x98, 0xea, 0x49, 0xdf, 0x4b, 0xfd, 0xf2, 0x8b, 0x8a, - 0x20, 0x7e, 0x1f, 0xce, 0x8d, 0x18, 0x22, 0x4e, 0x4b, 0x7f, 0x0f, 0x2a, 0xeb, 0xb8, 0x67, 0x61, - 0xe2, 0x94, 0xb4, 0x8f, 0xb1, 0xa5, 0xef, 0x1f, 0xc5, 0x67, 0x71, 0xb1, 0x05, 0xcb, 0xd1, 0xe8, - 0x71, 0x76, 0xe3, 0xef, 0x66, 0xa1, 0xb0, 0x71, 0xd8, 0x33, 0x2d, 0xa7, 0xc1, 0x76, 0x7d, 0xf4, - 0x10, 0x32, 0x3d, 0xcb, 0x7c, 0xa1, 0xbb, 0xc0, 0x73, 0xa1, 0xf1, 0xfd, 0x80, 0xce, 0x0e, 0xaf, - 0xef, 0x1d, 0x86, 0xf8, 0xff, 0x48, 0x82, 0xec, 0x23, 0x53, 0x55, 0x3a, 0xf7, 0xf5, 0x8e, 0xbb, - 0x4a, 0xaa, 0x93, 0xc0, 0xaa, 0x9e, 0xc6, 0x8e, 0xe2, 0xb4, 0x5d, 0xff, 0xe0, 0x09, 0xd1, 0x26, - 0x64, 0x1e, 0x38, 0x4e, 0x8f, 0x14, 0x72, 0xe7, 0x72, 0x65, 0x22, 0x24, 0x51, 0x70, 0x1b, 0xe7, - 0x2a, 0x23, 0x09, 0x16, 0x36, 0x4d, 0xb3, 0xd5, 0xc1, 0xb5, 0x8e, 0xd9, 0xd7, 0x6a, 0xa6, 0xb1, - 0xaf, 0xb7, 0xf8, 0x59, 0xf3, 0xf2, 0x44, 0xc4, 0xcd, 0x5a, 0x43, 0x1a, 0x55, 0x47, 0xdf, 0x81, - 0x4c, 0xe3, 0x0e, 0x87, 0x62, 0x87, 0xcf, 0x4b, 0x13, 0xa1, 0x1a, 0x77, 0x24, 0x4f, 0x09, 0x3d, - 0x80, 0xdc, 0xea, 0xe7, 0x7d, 0x0b, 0x73, 0x8c, 0x19, 0x8a, 0x71, 0x75, 0x22, 0x06, 0xd5, 0x91, - 0xfc, 0xaa, 0xe5, 0x1b, 0x50, 0x08, 0x58, 0x12, 0x95, 0x20, 0xd5, 0x23, 0x46, 0x23, 0x83, 0x9a, - 0x75, 0x43, 0x37, 0x44, 0x52, 0xbe, 0x0a, 0x29, 0x62, 0x15, 0xb2, 0xce, 0xf7, 0x14, 0x1b, 0xef, - 0x5a, 0x7a, 0xa0, 0x92, 0x2b, 0x2c, 0xff, 0xad, 0x00, 0x89, 0xc6, 0x1d, 0x42, 0xd5, 0xf6, 0xfa, - 0xea, 0x33, 0xec, 0x04, 0x6a, 0x71, 0x19, 0x25, 0x72, 0x16, 0xde, 0xd7, 0x19, 0x01, 0xf0, 0x4a, - 0x99, 0x0c, 0x5d, 0x02, 0x50, 0x54, 0x15, 0xdb, 0x36, 0xbd, 0x88, 0x48, 0xfa, 0x6a, 0x64, 0x99, - 0x7c, 0x0b, 0x1f, 0x11, 0x08, 0x1b, 0xab, 0x16, 0x66, 0x2b, 0xdc, 0x83, 0x60, 0x32, 0x02, 0xe1, - 0xe0, 0x6e, 0x4f, 0x76, 0xcc, 0x67, 0xd8, 0xa0, 0x56, 0xf6, 0x20, 0x88, 0xbc, 0x49, 0xc4, 0xe5, - 0x55, 0x48, 0x6e, 0xd6, 0x1a, 0xaf, 0xd3, 0xd4, 0xf2, 0x4f, 0x05, 0x48, 0x53, 0x83, 0x22, 0x11, - 0xb2, 0xaa, 0x69, 0x38, 0x8a, 0x6e, 0xf0, 0x35, 0xe1, 0xbd, 0xd0, 0x13, 0x4f, 0xe8, 0xf6, 0x35, - 0xc8, 0x2b, 0xaa, 0x6a, 0xf6, 0x0d, 0x47, 0x36, 0x94, 0x2e, 0x0e, 0x74, 0x3c, 0xc7, 0x4b, 0xb6, - 0x95, 0x2e, 0x46, 0x57, 0xc0, 0xfd, 0x97, 0x1a, 0xc8, 0xdf, 0x7f, 0xe0, 0x05, 0x5b, 0xf8, 0x48, - 0xfc, 0x0b, 0x01, 0x16, 0x9e, 0x5a, 0xba, 0x83, 0xd7, 0x14, 0x47, 0x6d, 0xc7, 0xe0, 0xe2, 0xdf, - 0x83, 0xac, 0xa6, 0x38, 0x0a, 0xcb, 0xa1, 0x4b, 0x8c, 0xd7, 0xe6, 0x0b, 0x89, 0xd4, 0xa7, 0x79, - 0x74, 0x08, 0x52, 0xe4, 0x99, 0xf9, 0x7d, 0x89, 0x3e, 0x8b, 0x9f, 0x02, 0xf2, 0xb7, 0x2f, 0x4e, - 0x97, 0xf5, 0x95, 0xe0, 0xba, 0xac, 0x18, 0xfa, 0xfd, 0x21, 0xcc, 0xf2, 0xe3, 0x0e, 0xef, 0xf5, - 0xf2, 0xa4, 0xb5, 0xe6, 0x2e, 0x0a, 0xae, 0x86, 0xd6, 0x00, 0x6c, 0x47, 0xb1, 0x1c, 0xd9, 0xd1, - 0xbb, 0xd3, 0xdd, 0xad, 0xba, 0x93, 0x87, 0xaa, 0x11, 0xa9, 0xf8, 0xd7, 0x02, 0x14, 0xd6, 0xfa, - 0x9d, 0x67, 0x4f, 0x7a, 0x8d, 0x7e, 0xb7, 0xab, 0x58, 0x47, 0xe8, 0xa2, 0x3b, 0x1e, 0xfa, 0xe7, - 0x98, 0xf6, 0x2a, 0x19, 0x30, 0xbb, 0xfe, 0x39, 0x26, 0xeb, 0x99, 0x67, 0x03, 0x0c, 0x4a, 0xa9, - 0x04, 0xdd, 0x80, 0x02, 0xe5, 0xdb, 0x32, 0x36, 0x1c, 0x4b, 0xc7, 0x36, 0xe7, 0xda, 0xac, 0x4a, - 0x9e, 0x16, 0x6d, 0xb0, 0x12, 0x74, 0x13, 0xe6, 0xec, 0x23, 0xdb, 0xc1, 0x5d, 0x99, 0xe5, 0x8d, - 0xda, 0x74, 0xca, 0xb9, 0x75, 0x0b, 0xac, 0x4c, 0x62, 0x45, 0xe2, 0xbf, 0x24, 0x60, 0xce, 0xb5, - 0x7c, 0x9c, 0x44, 0x6a, 0x0d, 0xd2, 0xfb, 0x7a, 0xc7, 0xbb, 0x40, 0x88, 0x76, 0x77, 0x2e, 0x52, - 0x95, 0x38, 0x35, 0x2f, 0xb6, 0x46, 0x54, 0xcb, 0x3f, 0x17, 0x20, 0x45, 0xf7, 0x87, 0x77, 0x20, - 0x45, 0x27, 0xb1, 0x30, 0xcd, 0x24, 0xa6, 0x55, 0x3d, 0xcf, 0x98, 0x18, 0xf6, 0x8c, 0xe8, 0x2c, - 0xcc, 0xd8, 0x6d, 0xe5, 0xee, 0x3b, 0xb7, 0xa9, 0x9f, 0xc9, 0x4b, 0xfc, 0x3f, 0xb4, 0x06, 0x19, - 0x4c, 0x5b, 0x84, 0x35, 0xee, 0xa3, 0xc3, 0xe6, 0x4d, 0x60, 0x48, 0xdd, 0xf1, 0x73, 0xf5, 0xd8, - 0xd5, 0xcc, 0xc3, 0x54, 0x26, 0x55, 0x4c, 0x8b, 0xbf, 0x48, 0x42, 0xa1, 0xde, 0x8d, 0x69, 0x4e, - 0xaf, 0x06, 0xcd, 0x19, 0xb6, 0x3d, 0x06, 0xde, 0x35, 0x6a, 0xcd, 0xa0, 0x3b, 0x48, 0x1e, 0xcf, - 0x1d, 0xd4, 0x61, 0xc6, 0xc2, 0x3c, 0xa9, 0x96, 0xbc, 0xff, 0xe6, 0xc4, 0xf7, 0x37, 0x95, 0xbd, - 0x0e, 0x96, 0x88, 0x8e, 0x77, 0x31, 0x44, 0x01, 0xca, 0x87, 0x7c, 0x4c, 0xdf, 0x85, 0xa4, 0xa6, - 0xbb, 0x96, 0x98, 0x76, 0x85, 0x12, 0x95, 0xa9, 0x86, 0x36, 0xe5, 0x1f, 0x5a, 0x36, 0x2c, 0xe5, - 0xc7, 0x00, 0x83, 0x56, 0xa1, 0xcb, 0x30, 0x63, 0x76, 0x34, 0xc2, 0x6d, 0x05, 0xca, 0x7f, 0x0b, - 0xfc, 0xd4, 0x9a, 0x7e, 0xd2, 0xd1, 0xea, 0xeb, 0x52, 0xda, 0xec, 0x68, 0x75, 0x0d, 0x9d, 0x87, - 0x8c, 0x81, 0x0f, 0x64, 0x9a, 0x46, 0x4d, 0xaf, 0xd8, 0xa5, 0x59, 0x03, 0x1f, 0xac, 0x63, 0x5b, - 0xe5, 0xa3, 0xfc, 0x97, 0x02, 0xcc, 0xb9, 0x3d, 0x8f, 0x77, 0xfd, 0x64, 0xf4, 0x2e, 0x9f, 0x8d, - 0xc9, 0xe3, 0xcd, 0x46, 0x57, 0x8f, 0xa7, 0x2f, 0xed, 0xc0, 0x69, 0x96, 0x00, 0xa0, 0x2a, 0x8e, - 0x83, 0xad, 0x18, 0x98, 0xec, 0x4f, 0x12, 0xb0, 0x18, 0x84, 0x8c, 0xb3, 0xe7, 0x5b, 0x43, 0x77, - 0x8f, 0x6f, 0x87, 0x80, 0x84, 0xbd, 0x9d, 0xdd, 0x21, 0x06, 0xaf, 0x21, 0xcb, 0x3f, 0x80, 0x34, - 0x15, 0x9f, 0xc4, 0x85, 0x54, 0x21, 0x4d, 0xbf, 0x10, 0x18, 0x73, 0x16, 0xdc, 0x20, 0xe5, 0x12, - 0xab, 0x26, 0xee, 0xc1, 0xc2, 0xaa, 0xa6, 0x35, 0x1a, 0x7c, 0x92, 0xbd, 0xf6, 0x9a, 0x77, 0xf7, - 0xe0, 0x44, 0x70, 0x0f, 0xf6, 0xbf, 0x23, 0xce, 0x3d, 0xf8, 0x9f, 0x17, 0x21, 0xcf, 0x5b, 0xbd, - 0x6b, 0x90, 0x23, 0xe0, 0x0a, 0x24, 0x5b, 0x9c, 0x65, 0x05, 0xb7, 0x3f, 0xef, 0xea, 0xd0, 0xcb, - 0x81, 0x97, 0x48, 0x4d, 0xa2, 0xd0, 0xeb, 0x3b, 0x21, 0x77, 0x6a, 0x83, 0x4b, 0x96, 0x81, 0x42, - 0xaf, 0xef, 0xa0, 0x8f, 0x60, 0x5e, 0x1d, 0x64, 0x23, 0xcb, 0x44, 0x39, 0x19, 0x99, 0x7e, 0x14, - 0x9a, 0x94, 0x2d, 0xcd, 0xa9, 0x01, 0x31, 0x5a, 0xf5, 0xa7, 0xce, 0xa6, 0x22, 0xe9, 0xfa, 0x70, - 0xb6, 0xae, 0x2f, 0xb3, 0x16, 0xbd, 0x0b, 0x33, 0x1a, 0x4d, 0xce, 0xe4, 0x74, 0x3f, 0x6c, 0xe1, - 0x05, 0xb2, 0x5f, 0x25, 0x5e, 0x1f, 0x3d, 0x80, 0x3c, 0x7b, 0x62, 0x17, 0xde, 0x7c, 0x1b, 0xb9, - 0x12, 0xad, 0xef, 0x8b, 0xfe, 0x4a, 0x39, 0x6d, 0x20, 0x43, 0xb7, 0x21, 0x65, 0xab, 0x8a, 0x41, - 0x6f, 0x9c, 0xc2, 0x6f, 0x0a, 0x7c, 0x29, 0x8e, 0x12, 0xad, 0x8b, 0x9e, 0xc2, 0xc2, 0x1e, 0x6e, - 0xe9, 0x86, 0xec, 0x0c, 0xe2, 0xb2, 0xa5, 0xcc, 0x48, 0x28, 0xd8, 0xf3, 0x1d, 0xe1, 0x59, 0x76, - 0x52, 0x71, 0x6f, 0xa8, 0x80, 0x0c, 0x13, 0x36, 0xb4, 0x00, 0x6c, 0x36, 0x72, 0x98, 0x42, 0x93, - 0xe0, 0xa4, 0x39, 0x1c, 0x10, 0xa3, 0x0d, 0xc8, 0x29, 0x64, 0x0d, 0xcb, 0x34, 0x4f, 0xa9, 0x04, - 0x91, 0x47, 0xb4, 0x91, 0x8c, 0x29, 0x09, 0x14, 0x4f, 0x34, 0x80, 0xe9, 0x62, 0xab, 0x85, 0x4b, - 0xb9, 0xf1, 0x30, 0xfe, 0x60, 0x2f, 0x87, 0xa1, 0x22, 0xb4, 0x05, 0x85, 0xb6, 0x7b, 0x97, 0x4f, - 0x03, 0xe8, 0xf9, 0xc8, 0x33, 0x5a, 0x48, 0xf2, 0x81, 0x94, 0x6f, 0xfb, 0x84, 0xe8, 0x5b, 0x90, - 0x68, 0xa9, 0xa5, 0x02, 0x45, 0xb8, 0x30, 0xee, 0xc2, 0x5d, 0x4a, 0xb4, 0x54, 0xf4, 0x3e, 0x64, - 0xd8, 0xf5, 0xe4, 0xa1, 0x51, 0x9a, 0x8b, 0x5c, 0xbc, 0xc1, 0x9b, 0x61, 0x89, 0x5e, 0xa3, 0x92, - 0x77, 0x3d, 0x80, 0x3c, 0x0b, 0xbe, 0x76, 0x68, 0x12, 0x48, 0x69, 0x3e, 0x72, 0xc2, 0x8d, 0xe6, - 0xb1, 0x48, 0xec, 0x43, 0x18, 0x26, 0x43, 0xdb, 0x30, 0x67, 0xb1, 0x70, 0xbb, 0xcc, 0x2e, 0x01, - 0x4a, 0x45, 0x8a, 0x75, 0x2d, 0xdc, 0x95, 0x8c, 0x5c, 0x13, 0x49, 0x05, 0xcb, 0x2f, 0x45, 0xdf, - 0x87, 0xc5, 0x20, 0x1e, 0x5f, 0x12, 0x0b, 0x14, 0xf5, 0x5b, 0x13, 0x51, 0xfd, 0x2b, 0x03, 0x59, - 0x23, 0x45, 0xe8, 0x2e, 0xa4, 0xd9, 0x98, 0x23, 0x0a, 0x58, 0x09, 0x01, 0x0c, 0x0c, 0x37, 0xab, - 0x4d, 0x0c, 0xe6, 0xf0, 0xb0, 0xb3, 0xdc, 0x31, 0x5b, 0xa5, 0xd3, 0x91, 0x06, 0x1b, 0x8d, 0xa0, - 0x4b, 0x39, 0x67, 0x20, 0x23, 0x73, 0xc6, 0x62, 0x72, 0x1e, 0x15, 0x5d, 0x8c, 0x9c, 0x33, 0x21, - 0xb1, 0x68, 0x29, 0x6f, 0xf9, 0x84, 0x74, 0x1c, 0x59, 0x76, 0x8f, 0x4c, 0x97, 0xfd, 0x99, 0xe8, - 0x71, 0x1c, 0x49, 0x70, 0x96, 0x72, 0xd6, 0x40, 0x86, 0x9a, 0x50, 0x54, 0x59, 0x8c, 0x4c, 0x76, - 0x23, 0x6c, 0xa5, 0xb3, 0x14, 0xed, 0x46, 0xa8, 0x4f, 0x0d, 0x8b, 0x2b, 0x4a, 0xf3, 0x6a, 0x50, - 0x8e, 0x7a, 0x50, 0xd6, 0xbc, 0xd8, 0x95, 0xfc, 0x82, 0x06, 0xaf, 0x06, 0xf8, 0xe7, 0x28, 0xfe, - 0xed, 0x50, 0x37, 0x37, 0x36, 0x9c, 0x26, 0x95, 0xb4, 0x88, 0x0a, 0xc4, 0x99, 0x51, 0x7c, 0x59, - 0x1d, 0x24, 0x1a, 0x97, 0x4a, 0x91, 0xce, 0x2c, 0x22, 0x19, 0x5a, 0x2a, 0xaa, 0x43, 0x05, 0xc4, - 0xb3, 0x1a, 0xa6, 0xd9, 0x2b, 0x9d, 0x8f, 0xf4, 0xac, 0xbe, 0x5b, 0x28, 0x89, 0xd6, 0x25, 0x8b, - 0x54, 0x37, 0x74, 0x87, 0x6e, 0x50, 0xe5, 0xc8, 0x45, 0x1a, 0xfc, 0xf0, 0x45, 0x9a, 0xd5, 0xd9, - 0xff, 0x64, 0x69, 0x39, 0x3c, 0x8a, 0xcf, 0xa7, 0xca, 0x85, 0xc8, 0xa5, 0x15, 0x16, 0xee, 0x97, - 0x0a, 0x8e, 0x5f, 0x4a, 0x96, 0x16, 0x73, 0x7a, 0x43, 0xa8, 0x6f, 0x46, 0x2e, 0xad, 0xc8, 0xd4, - 0x47, 0x09, 0x29, 0x23, 0x45, 0xe4, 0xf4, 0x4b, 0x01, 0xe9, 0x27, 0x72, 0xa5, 0xa5, 0xc8, 0x3d, - 0x74, 0x38, 0xa4, 0x2f, 0x65, 0x3b, 0xae, 0x84, 0x38, 0xe6, 0x03, 0x4b, 0x77, 0xb0, 0xbc, 0xa7, - 0x38, 0x6a, 0xbb, 0x54, 0x89, 0x74, 0xcc, 0x23, 0x11, 0x0f, 0x09, 0x0e, 0x3c, 0x11, 0xd9, 0x8a, - 0xd9, 0xd9, 0xaa, 0xb4, 0x3c, 0xe1, 0x9c, 0xe0, 0x6d, 0xc5, 0xac, 0x3e, 0xfa, 0x0e, 0x64, 0x9f, - 0xf7, 0xb1, 0x75, 0x44, 0x1d, 0xeb, 0xc5, 0xc8, 0x0f, 0x20, 0x87, 0xd2, 0x22, 0xa4, 0xcc, 0x73, - 0x2e, 0x20, 0xaf, 0x66, 0x44, 0xba, 0x24, 0x46, 0xbe, 0x3a, 0x70, 0xe4, 0x91, 0x78, 0x7d, 0xa4, - 0xc0, 0x19, 0x36, 0x3e, 0x3c, 0x67, 0xd2, 0xe2, 0xc9, 0x89, 0xa5, 0x4b, 0x14, 0x28, 0x92, 0xcf, - 0x86, 0xa6, 0x6d, 0x4a, 0xa7, 0x95, 0xd1, 0x32, 0xe2, 0x7c, 0xf8, 0xf6, 0xc9, 0x38, 0x70, 0xe9, - 0x72, 0xa4, 0xf3, 0x09, 0xe1, 0xfe, 0x52, 0x5e, 0xf1, 0x09, 0xd9, 0x26, 0xaa, 0xc9, 0xb6, 0xed, - 0x10, 0x52, 0x59, 0xba, 0x32, 0x66, 0x13, 0x1d, 0x62, 0xb7, 0x64, 0x13, 0xd5, 0x1a, 0x4c, 0xef, - 0xbd, 0xd4, 0x97, 0x5f, 0x54, 0x84, 0x87, 0xa9, 0xcc, 0x1b, 0xc5, 0x0b, 0xe2, 0xcf, 0x16, 0xa1, - 0xe0, 0x12, 0x4e, 0x46, 0x26, 0x6f, 0xf9, 0xc9, 0xe4, 0x52, 0x14, 0x99, 0x64, 0x1a, 0x8c, 0x4d, - 0xde, 0xf2, 0xb3, 0xc9, 0xa5, 0x28, 0x36, 0xe9, 0x6a, 0x10, 0x3a, 0x29, 0x45, 0xd1, 0xc9, 0x1b, - 0x53, 0xd0, 0x49, 0x0e, 0x34, 0xcc, 0x27, 0xd7, 0x46, 0xf9, 0xe4, 0xe5, 0xf1, 0x7c, 0x92, 0x03, - 0xf9, 0x08, 0xe5, 0xbd, 0x21, 0x42, 0x79, 0x71, 0x0c, 0xa1, 0xe4, 0xda, 0x2e, 0xa3, 0xac, 0x87, - 0x32, 0xca, 0xab, 0x93, 0x18, 0x25, 0x47, 0x09, 0x50, 0xca, 0x3b, 0x01, 0x4a, 0x59, 0x89, 0xa4, - 0x94, 0x5c, 0x97, 0x71, 0xca, 0x4f, 0xa2, 0x39, 0xe5, 0xcd, 0xa9, 0x38, 0x25, 0x47, 0x1b, 0x25, - 0x95, 0x52, 0x14, 0xa9, 0xbc, 0x31, 0x05, 0xa9, 0x74, 0x07, 0x6b, 0x88, 0x55, 0xde, 0x0f, 0x63, - 0x95, 0x57, 0x26, 0xb0, 0x4a, 0x8e, 0xe5, 0xa7, 0x95, 0xf7, 0xc3, 0x68, 0xe5, 0x95, 0x09, 0xb4, - 0x32, 0x80, 0xc3, 0x78, 0xe5, 0xa3, 0x70, 0x5e, 0x79, 0x6d, 0x22, 0xaf, 0xe4, 0x58, 0x41, 0x62, - 0xf9, 0xb6, 0x8f, 0x58, 0xbe, 0x19, 0x41, 0x2c, 0xb9, 0x22, 0x61, 0x96, 0x1f, 0x8c, 0x30, 0x4b, - 0x71, 0x1c, 0xb3, 0xe4, 0x9a, 0x1e, 0xb5, 0xac, 0x87, 0x52, 0xcb, 0xab, 0x93, 0xa8, 0xa5, 0x3b, - 0xf3, 0xfc, 0xdc, 0xf2, 0x49, 0x04, 0xb7, 0xbc, 0x3e, 0x99, 0x5b, 0x72, 0xb8, 0x21, 0x72, 0x29, - 0x8f, 0x25, 0x97, 0x6f, 0x4f, 0x49, 0x2e, 0x39, 0x76, 0x18, 0xbb, 0xfc, 0xf5, 0x20, 0xbb, 0x5c, - 0x8e, 0x66, 0x97, 0x1c, 0x84, 0xd3, 0xcb, 0x7a, 0x28, 0xbd, 0xbc, 0x3a, 0x89, 0x5e, 0xba, 0x46, - 0xf3, 0xf3, 0xcb, 0x47, 0xe1, 0xfc, 0xf2, 0xda, 0x44, 0x7e, 0xe9, 0xce, 0x9d, 0x00, 0xc1, 0xac, - 0x87, 0x12, 0xcc, 0xab, 0x93, 0x08, 0xa6, 0x37, 0x9a, 0x3e, 0x86, 0xb9, 0x1b, 0xc9, 0x30, 0xdf, - 0x9a, 0x86, 0x61, 0x72, 0xc8, 0x11, 0x8a, 0xf9, 0x7c, 0x0a, 0x8a, 0x79, 0xe7, 0x58, 0x14, 0x93, - 0xbf, 0x29, 0x9a, 0x63, 0x7e, 0x12, 0xcd, 0x31, 0x6f, 0x4e, 0xc5, 0x31, 0x5d, 0xe7, 0x36, 0x42, - 0x32, 0xef, 0x04, 0x48, 0x66, 0x25, 0x92, 0x64, 0xba, 0xbe, 0x96, 0xb2, 0xcc, 0x0f, 0x46, 0x58, - 0xa6, 0x38, 0x8e, 0x65, 0xba, 0x0b, 0xd6, 0xa5, 0x99, 0xf2, 0x58, 0x5a, 0xf8, 0xf6, 0x94, 0xb4, - 0xd0, 0x5d, 0x14, 0x21, 0xbc, 0xb0, 0x16, 0xc2, 0x0b, 0x2f, 0x8f, 0xe7, 0x85, 0xee, 0x5e, 0x38, - 0x20, 0x86, 0xf7, 0xc3, 0x88, 0xe1, 0x95, 0x09, 0xc4, 0xd0, 0x75, 0xad, 0x3e, 0x66, 0x78, 0x6f, - 0x88, 0x19, 0x5e, 0x9c, 0x78, 0xc1, 0xe0, 0x51, 0xc3, 0x0f, 0x47, 0xa9, 0xe1, 0xa5, 0xb1, 0xd4, - 0x90, 0xeb, 0x0f, 0xb8, 0xe1, 0xbd, 0x21, 0x6e, 0x78, 0x71, 0x0c, 0x37, 0x74, 0x5f, 0xce, 0xc9, - 0xe1, 0xde, 0x78, 0x72, 0x58, 0x9d, 0x96, 0x1c, 0x72, 0xd8, 0x50, 0x76, 0xf8, 0x28, 0x9c, 0x1d, - 0x5e, 0x9b, 0x32, 0x90, 0x3a, 0x44, 0x0f, 0xef, 0x87, 0xd1, 0xc3, 0x2b, 0x13, 0xe8, 0xe1, 0x60, - 0x33, 0x0c, 0xe3, 0x87, 0x0f, 0x53, 0x99, 0x0b, 0xc5, 0x37, 0xc5, 0xff, 0x49, 0xc1, 0xcc, 0x03, - 0xf7, 0x7e, 0xc3, 0x97, 0x94, 0x2f, 0x9c, 0x24, 0x29, 0x1f, 0xad, 0xc3, 0x2c, 0x37, 0x26, 0xe7, - 0x8c, 0x63, 0xbe, 0x2b, 0x1a, 0xf9, 0xd2, 0xc4, 0x55, 0x3d, 0x61, 0x3a, 0x1b, 0xba, 0x07, 0x85, - 0xbe, 0x8d, 0x2d, 0xb9, 0x67, 0xe9, 0xa6, 0xa5, 0x3b, 0xec, 0x96, 0x57, 0x58, 0x5b, 0xe4, 0x9f, - 0x83, 0xe5, 0x77, 0x6d, 0x6c, 0xed, 0xf0, 0x32, 0x29, 0xdf, 0xf7, 0xfd, 0xe7, 0xfe, 0x3c, 0x4a, - 0x7a, 0xfa, 0x9f, 0x47, 0x79, 0x0a, 0x45, 0x0b, 0x2b, 0x5a, 0xc0, 0x6d, 0xb1, 0xcc, 0xf6, 0x70, - 0x87, 0xae, 0x68, 0x3e, 0xdf, 0xe4, 0xcb, 0x70, 0x9f, 0xb7, 0x82, 0x45, 0xe8, 0xdb, 0x70, 0xa6, - 0xab, 0x1c, 0xb2, 0x8f, 0x35, 0xdc, 0xcd, 0x87, 0xde, 0xfa, 0x64, 0x7c, 0x17, 0x88, 0xa8, 0xab, - 0x1c, 0xd2, 0xdf, 0x5d, 0x61, 0x15, 0xe8, 0x37, 0xef, 0x37, 0x61, 0x4e, 0xd3, 0x6d, 0x47, 0x37, - 0x54, 0xf7, 0x7b, 0xcf, 0xac, 0x3f, 0x9b, 0xdb, 0x2d, 0x63, 0x1f, 0x76, 0xde, 0x82, 0x05, 0xfe, - 0x1d, 0xfd, 0xe0, 0x97, 0x58, 0x28, 0x57, 0xcb, 0x0c, 0xda, 0x45, 0x8a, 0x07, 0x3f, 0x64, 0xb3, - 0x09, 0xf3, 0x2d, 0xc5, 0xc1, 0x07, 0xca, 0x91, 0x6c, 0x98, 0x1a, 0x1d, 0x9b, 0x1c, 0xfd, 0x76, - 0xaa, 0xc2, 0xc7, 0xa6, 0xb0, 0xc9, 0x8a, 0xb7, 0x4d, 0x8d, 0x8d, 0xd0, 0x0c, 0x7b, 0x92, 0x0a, - 0x2d, 0x5f, 0x81, 0xf6, 0x30, 0x95, 0x99, 0x2d, 0x66, 0xc4, 0x9f, 0x08, 0x90, 0x0f, 0x5c, 0xb2, - 0xff, 0xc6, 0x50, 0xfc, 0xfc, 0x7c, 0x38, 0x41, 0x0b, 0xbf, 0xaf, 0x58, 0x85, 0x0c, 0xb7, 0x95, - 0x7b, 0x63, 0x51, 0x89, 0xde, 0xa3, 0xe9, 0x61, 0xc8, 0xbd, 0xa8, 0x71, 0xd5, 0xde, 0x4b, 0xfd, - 0xf1, 0x17, 0x95, 0x53, 0xe2, 0x2f, 0x93, 0x50, 0x08, 0xde, 0xad, 0xd7, 0x87, 0xda, 0x15, 0xb6, - 0x82, 0x03, 0x1a, 0xd1, 0xad, 0x5c, 0x87, 0xac, 0xc5, 0x2b, 0xb9, 0xcd, 0x5c, 0x1e, 0x73, 0x4b, - 0xe0, 0x6f, 0xe7, 0x40, 0xb1, 0xfc, 0x0f, 0x09, 0x6f, 0xc5, 0x7a, 0xb7, 0x23, 0xc2, 0x54, 0xb7, - 0x23, 0x64, 0x85, 0x37, 0x4f, 0xf4, 0xd9, 0x8d, 0x27, 0x38, 0xc1, 0x0f, 0x09, 0x9d, 0xf0, 0x9b, - 0x91, 0x06, 0x39, 0x48, 0x76, 0x3a, 0x58, 0x75, 0xf8, 0x6f, 0x2e, 0xb9, 0x3f, 0x14, 0x74, 0x79, - 0x18, 0x82, 0xff, 0x42, 0x53, 0x55, 0xe2, 0xbf, 0xd0, 0xe4, 0xbb, 0x44, 0x9a, 0xf3, 0x20, 0xe8, - 0xb4, 0x67, 0xf7, 0x85, 0x6c, 0xa8, 0xdf, 0x7a, 0x04, 0xa7, 0x43, 0x96, 0x25, 0x9a, 0x03, 0xa8, - 0x3d, 0xd9, 0x6e, 0xd4, 0x1b, 0xcd, 0x8d, 0xed, 0x66, 0xf1, 0x14, 0x2a, 0x40, 0x96, 0xfc, 0xbf, - 0xb1, 0xdd, 0xd8, 0x6d, 0x14, 0x05, 0x54, 0x84, 0x7c, 0x7d, 0xdb, 0x57, 0x21, 0x51, 0x4e, 0xfd, - 0xee, 0xcf, 0x96, 0x4e, 0xbd, 0xf5, 0x14, 0x72, 0xbe, 0xcf, 0x57, 0x10, 0x82, 0xb9, 0x9d, 0xdd, - 0xc6, 0x03, 0xb9, 0x59, 0x7f, 0xbc, 0xd1, 0x68, 0xae, 0x3e, 0xde, 0x29, 0x9e, 0x22, 0xc8, 0x54, - 0xb6, 0xba, 0xf6, 0x44, 0x6a, 0x16, 0x05, 0xef, 0xff, 0xe6, 0x93, 0xdd, 0xda, 0x83, 0x62, 0xc2, - 0xfb, 0xff, 0xa3, 0xdd, 0x0d, 0xe9, 0xd3, 0x62, 0x92, 0x03, 0x2b, 0x70, 0x26, 0x34, 0x29, 0x0c, - 0xe5, 0x60, 0x76, 0xd7, 0xa0, 0xdf, 0x4f, 0xb0, 0x56, 0x7a, 0x59, 0x49, 0x45, 0x01, 0x65, 0x58, - 0xe6, 0x51, 0x31, 0x81, 0x66, 0x20, 0xd1, 0xb8, 0x53, 0x4c, 0xa2, 0x79, 0xc8, 0xf9, 0xd2, 0xaa, - 0x8a, 0x29, 0x94, 0xe5, 0x59, 0x38, 0xc5, 0xf4, 0xed, 0x4f, 0x20, 0xe3, 0x7e, 0x57, 0x8e, 0x1e, - 0x41, 0x9a, 0x6d, 0xee, 0x95, 0xe8, 0x79, 0x4e, 0x57, 0x4c, 0x79, 0x79, 0xd2, 0x42, 0x10, 0x4f, - 0x11, 0xe4, 0x8d, 0xc3, 0x5f, 0x05, 0xf2, 0xda, 0xc5, 0x2f, 0xff, 0x73, 0xe9, 0xd4, 0x97, 0xaf, - 0x96, 0x84, 0xaf, 0x5e, 0x2d, 0x09, 0xff, 0xfa, 0x6a, 0x49, 0xf8, 0x8f, 0x57, 0x4b, 0xc2, 0x1f, - 0xfe, 0x62, 0xe9, 0xd4, 0x67, 0xb3, 0x5c, 0xe5, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x52, 0x4d, - 0x30, 0xfc, 0xc8, 0x4c, 0x00, 0x00, + // 4612 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x7c, 0xcf, 0x73, 0xdb, 0x48, + 0x76, 0xbf, 0xc1, 0x1f, 0x12, 0xf9, 0x48, 0x4a, 0x54, 0x5b, 0x1e, 0xd3, 0x1c, 0x8f, 0x28, 0xc3, + 0xbf, 0xc7, 0x33, 0x94, 0xc7, 0xfe, 0xfa, 0xbb, 0xe3, 0xc9, 0x4c, 0xad, 0x25, 0x4a, 0x96, 0x69, + 0xd9, 0xb2, 0x06, 0xa4, 0xc6, 0x33, 0xb3, 0x9b, 0x45, 0x20, 0xa0, 0x45, 0x22, 0x26, 0x01, 0x1a, + 0x00, 0x2d, 0x69, 0x72, 0xd8, 0xaa, 0xd4, 0x56, 0x6d, 0x92, 0x43, 0x2a, 0xa9, 0xca, 0x21, 0x9b, + 0x9f, 0x53, 0xd9, 0xa4, 0x2a, 0x7f, 0x40, 0xaa, 0x72, 0xd8, 0x43, 0x4e, 0x49, 0x4d, 0x6e, 0x73, + 0x4b, 0x2a, 0xa9, 0x72, 0x25, 0xde, 0xcb, 0xd6, 0x5e, 0x53, 0xb9, 0xcc, 0x29, 0xd5, 0x3f, 0x00, + 0x02, 0x24, 0x40, 0x52, 0x32, 0xb6, 0x26, 0x39, 0x09, 0x7a, 0xdd, 0xef, 0x83, 0xee, 0xd7, 0xdd, + 0xaf, 0x3f, 0xfd, 0xfa, 0x81, 0x50, 0x51, 0x4d, 0xf5, 0x99, 0x65, 0x2a, 0x6a, 0x7b, 0xa5, 0xf7, + 0xac, 0xb5, 0x42, 0x9f, 0x7a, 0x7b, 0x2b, 0x4a, 0x4f, 0xaf, 0xf6, 0x2c, 0xd3, 0x31, 0xd1, 0x82, + 0x57, 0xa1, 0xca, 0x0b, 0xcb, 0xcb, 0xe1, 0x3a, 0x9a, 0xe2, 0x28, 0x4c, 0xa9, 0x2c, 0x86, 0xd7, + 0xc0, 0x96, 0x65, 0x5a, 0x36, 0xaf, 0x73, 0x29, 0xbc, 0x4e, 0x17, 0x3b, 0x8a, 0x0f, 0xe9, 0x66, + 0xb0, 0x96, 0xed, 0x98, 0x96, 0xd2, 0xc2, 0x2b, 0xd8, 0x68, 0xe9, 0x86, 0xfb, 0x87, 0x68, 0xbd, + 0x50, 0x55, 0xae, 0x71, 0x25, 0xa8, 0xd1, 0x77, 0xf4, 0xce, 0x4a, 0xbb, 0xa3, 0xae, 0x38, 0x7a, + 0x17, 0xdb, 0x8e, 0xd2, 0xed, 0xf1, 0x7a, 0xd5, 0x90, 0x7a, 0x8e, 0xa5, 0xa8, 0xba, 0xd1, 0x5a, + 0xb1, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, 0xa7, 0x18, 0xbc, 0xfe, 0x62, 0xcb, 0x6c, 0x99, + 0xf4, 0x71, 0x85, 0x3c, 0x31, 0xa9, 0xf8, 0x43, 0xc8, 0x4a, 0x8a, 0xd1, 0xc2, 0x75, 0x63, 0xdf, + 0x44, 0x1f, 0x42, 0x4a, 0xc3, 0xb6, 0x5a, 0x12, 0x96, 0x85, 0x6b, 0xb9, 0x5b, 0x62, 0x75, 0xc4, + 0x74, 0x55, 0x5a, 0x77, 0x1d, 0xdb, 0xaa, 0xa5, 0xf7, 0x1c, 0xd3, 0x5a, 0x4b, 0x7d, 0xf5, 0xb2, + 0x72, 0x4a, 0xa2, 0x5a, 0xe8, 0xff, 0x41, 0xba, 0x83, 0x15, 0x1b, 0x97, 0x12, 0x54, 0xbd, 0x14, + 0xa2, 0xfe, 0x88, 0x94, 0x73, 0x25, 0x56, 0x59, 0x7c, 0x25, 0xc0, 0x9c, 0x84, 0xed, 0x9e, 0x69, + 0xd8, 0xf8, 0x01, 0x56, 0x34, 0x6c, 0xa1, 0x9b, 0x90, 0x74, 0x0e, 0x8d, 0x52, 0x92, 0xc2, 0x2c, + 0x85, 0xc0, 0x34, 0x2d, 0xc5, 0xb0, 0x15, 0xd5, 0xd1, 0x4d, 0x43, 0x22, 0x55, 0xd1, 0xfb, 0x90, + 0xb3, 0xb0, 0xdd, 0xef, 0x62, 0xda, 0xe1, 0x52, 0x8a, 0x6a, 0x9e, 0x0d, 0xd1, 0x6c, 0xf4, 0x14, + 0x43, 0x02, 0x56, 0x97, 0x3c, 0xa3, 0x0a, 0x64, 0x8c, 0x7e, 0x57, 0x7e, 0x86, 0x8f, 0xec, 0x52, + 0x7a, 0x59, 0xb8, 0x96, 0xe4, 0xad, 0x9b, 0x35, 0xfa, 0xdd, 0x2d, 0x7c, 0x64, 0xa3, 0x1a, 0xe4, + 0x2c, 0xd2, 0x69, 0x59, 0x37, 0xf6, 0x4d, 0xbb, 0x34, 0xb3, 0x9c, 0xbc, 0x96, 0xbb, 0x75, 0x3e, + 0xca, 0x34, 0xc4, 0x8c, 0x1c, 0x01, 0x2c, 0x57, 0x60, 0x8b, 0x9b, 0x00, 0x9b, 0xd8, 0x91, 0xf0, + 0xf3, 0x3e, 0xb6, 0x1d, 0x74, 0x17, 0x66, 0xda, 0xb4, 0xa7, 0xdc, 0xd0, 0x51, 0x0d, 0x5d, 0xcb, + 0x10, 0xa0, 0xaf, 0x5f, 0x56, 0x04, 0x89, 0x2b, 0x88, 0xbf, 0x2d, 0x40, 0x8e, 0x22, 0x31, 0x83, + 0xa1, 0xda, 0x10, 0xd4, 0x85, 0xb0, 0x86, 0x05, 0xac, 0x3b, 0x0a, 0x8a, 0xaa, 0x90, 0x7e, 0xa1, + 0x74, 0xfa, 0xe3, 0x06, 0xee, 0x13, 0x52, 0x2e, 0xb1, 0x6a, 0xe2, 0xdf, 0x0b, 0x00, 0x3b, 0xfd, + 0x18, 0xba, 0x43, 0xa6, 0xcc, 0x54, 0x6f, 0x76, 0xa7, 0x0c, 0xad, 0x8c, 0xce, 0xc3, 0x8c, 0x6e, + 0x74, 0x74, 0x03, 0xd3, 0x29, 0x92, 0xe1, 0x85, 0x5c, 0x86, 0xca, 0x90, 0xde, 0xeb, 0xe8, 0x86, + 0x46, 0x67, 0x81, 0x5b, 0xc8, 0x44, 0xa2, 0x04, 0x39, 0xda, 0xf0, 0x18, 0xad, 0x27, 0xfe, 0xbb, + 0x00, 0x67, 0x6a, 0xa6, 0xa1, 0xe9, 0x64, 0x3a, 0x2a, 0x9d, 0x6f, 0xd3, 0x30, 0x77, 0x20, 0x8b, + 0x0f, 0x7b, 0x32, 0xd3, 0x4c, 0x4e, 0x18, 0xcc, 0x0c, 0x3e, 0xec, 0xd1, 0xa7, 0xb1, 0x16, 0xfb, + 0x75, 0x78, 0x63, 0xb8, 0x73, 0x71, 0x1a, 0xef, 0xcf, 0x05, 0x98, 0xab, 0x1b, 0xba, 0xf3, 0x6d, + 0x5a, 0xcd, 0xeb, 0x7e, 0x72, 0xb4, 0xfb, 0x9f, 0xc0, 0xbc, 0xd7, 0xbc, 0x38, 0xfb, 0xfd, 0x1c, + 0x8a, 0x75, 0x43, 0xb5, 0x70, 0x17, 0x1b, 0x71, 0x74, 0x5c, 0x84, 0xac, 0xee, 0xc2, 0xd1, 0xce, + 0xbb, 0x6e, 0x6c, 0x20, 0x16, 0x7f, 0x0b, 0x16, 0x7c, 0xaf, 0x8c, 0xd3, 0x7f, 0x5c, 0x80, 0xac, + 0x81, 0x0f, 0xe4, 0x81, 0xe9, 0xdd, 0xb7, 0x67, 0x0c, 0x7c, 0x40, 0x0d, 0x2e, 0x3e, 0x84, 0xc2, + 0x3a, 0xee, 0x60, 0x07, 0xc7, 0xe0, 0x03, 0x77, 0x61, 0xce, 0xc5, 0x8a, 0x73, 0x48, 0xfe, 0x44, + 0x00, 0xc4, 0x71, 0x89, 0xe3, 0x8e, 0x61, 0x54, 0x2e, 0x93, 0x5d, 0xc9, 0xe9, 0x5b, 0x06, 0xdb, + 0x5e, 0xfc, 0xd3, 0x0b, 0x58, 0x01, 0xdd, 0x61, 0x06, 0xee, 0x2c, 0x35, 0xea, 0xce, 0x1e, 0xa6, + 0x32, 0x89, 0x62, 0x52, 0x3c, 0x80, 0xd3, 0x81, 0xb6, 0xc5, 0x39, 0x7c, 0x6f, 0x42, 0x8a, 0xb6, + 0x2f, 0xb1, 0x9c, 0xbc, 0x96, 0x5f, 0x9b, 0xfd, 0xe6, 0x65, 0x25, 0xb9, 0x85, 0x8f, 0x24, 0x2a, + 0x14, 0xb7, 0x21, 0xd7, 0x50, 0x15, 0xe3, 0xf5, 0xad, 0xc1, 0x3b, 0xf2, 0x7b, 0x02, 0xe4, 0x19, + 0x60, 0x9c, 0x5d, 0xb8, 0x03, 0x29, 0xcb, 0x3c, 0x60, 0x5d, 0xc8, 0xdd, 0x7a, 0x33, 0x04, 0x62, + 0x0b, 0x1f, 0xf9, 0x97, 0x3e, 0xad, 0x2e, 0xee, 0x02, 0x92, 0xf0, 0x0b, 0x6c, 0xd9, 0x38, 0xd6, + 0x3e, 0xfe, 0xa1, 0x00, 0xa7, 0x03, 0xb8, 0xff, 0x0b, 0xba, 0x7a, 0x00, 0x67, 0x6b, 0x6d, 0xac, + 0x3e, 0xab, 0x99, 0x86, 0xad, 0xdb, 0x0e, 0x36, 0xd4, 0xa3, 0x18, 0x66, 0xf8, 0x05, 0xc8, 0x1e, + 0xe8, 0x4e, 0x5b, 0xd6, 0xf4, 0xfd, 0x7d, 0xba, 0xf2, 0xdd, 0xd9, 0x9b, 0x21, 0xe2, 0x75, 0x7d, + 0x7f, 0x5f, 0x94, 0xa1, 0x34, 0xfa, 0xe2, 0x38, 0xd7, 0x6d, 0x13, 0xce, 0xae, 0xe1, 0x96, 0x6e, + 0xf8, 0x49, 0xe1, 0xeb, 0x3b, 0x19, 0x19, 0x4a, 0xa3, 0xa8, 0x71, 0x36, 0xfb, 0x47, 0x49, 0x38, + 0xb3, 0x61, 0x68, 0xb1, 0xb6, 0x9a, 0xb8, 0x12, 0xd5, 0xec, 0x76, 0x75, 0x27, 0x30, 0x18, 0x5c, + 0x86, 0xee, 0x42, 0x46, 0xc3, 0x8a, 0xe6, 0x31, 0xa7, 0xdc, 0xad, 0xb7, 0x7c, 0xd0, 0xe4, 0x00, + 0x51, 0x6d, 0x77, 0xd4, 0x6a, 0xd3, 0x3d, 0x68, 0x48, 0x5e, 0x75, 0xf4, 0x1b, 0x70, 0x56, 0x37, + 0x1c, 0x6c, 0x19, 0x4a, 0x47, 0x66, 0x68, 0xb2, 0x63, 0xe9, 0xad, 0x16, 0xb6, 0x38, 0xd9, 0xbe, + 0x16, 0xd2, 0xc8, 0x3a, 0xd7, 0xa8, 0x51, 0x85, 0x26, 0xab, 0x2f, 0x9d, 0xd1, 0xc3, 0xc4, 0xe8, + 0x1e, 0xe4, 0x49, 0x81, 0xe1, 0x50, 0x0a, 0x4f, 0xc8, 0x78, 0x72, 0x5c, 0xdf, 0x59, 0xcf, 0x72, + 0x4c, 0x85, 0x48, 0x6c, 0x74, 0x9b, 0xb8, 0xdb, 0xe7, 0x7d, 0xdd, 0xc2, 0xf2, 0x7b, 0x3d, 0xb5, + 0x34, 0x43, 0x2d, 0x80, 0x48, 0xbd, 0x57, 0x2f, 0x2b, 0x20, 0xb1, 0xa2, 0xf7, 0x76, 0x6a, 0xc4, + 0xf9, 0xb2, 0xe7, 0x9e, 0x2a, 0xfe, 0x44, 0x80, 0x37, 0x86, 0x87, 0x21, 0x5e, 0x6e, 0x5d, 0x34, + 0x0d, 0x2c, 0xf7, 0xda, 0x8a, 0x8d, 0xb9, 0xe5, 0x02, 0x6e, 0x7e, 0xce, 0x34, 0xf0, 0x0e, 0x29, + 0x64, 0xd6, 0x60, 0x1e, 0xe4, 0x61, 0x2a, 0x93, 0x2c, 0xa6, 0x44, 0x07, 0x16, 0x56, 0xb5, 0xae, + 0x6e, 0x34, 0x7a, 0x1d, 0x3d, 0x0e, 0x96, 0x70, 0x09, 0xb2, 0x36, 0x81, 0x22, 0xdb, 0x11, 0x9d, + 0x20, 0x3e, 0x6f, 0x9f, 0xa1, 0x25, 0x5b, 0xf8, 0x48, 0xfc, 0x0c, 0x90, 0xff, 0xad, 0x71, 0xce, + 0xf9, 0x6d, 0xde, 0xa1, 0xc7, 0xd8, 0x8a, 0x63, 0x83, 0xf5, 0x9a, 0xca, 0xf1, 0xe2, 0x6c, 0xea, + 0x0f, 0xe1, 0x1c, 0x85, 0xa6, 0x13, 0x63, 0x1f, 0x5b, 0xf4, 0xe4, 0x1a, 0xc3, 0x18, 0x5c, 0x85, + 0x19, 0x47, 0xb1, 0x5a, 0x98, 0xad, 0xd0, 0xf4, 0xda, 0x3c, 0xa9, 0xf1, 0xcd, 0xcb, 0xca, 0x6c, + 0xc3, 0x31, 0x2d, 0x5c, 0x5f, 0x97, 0x78, 0xb1, 0xa8, 0x40, 0x39, 0xac, 0x01, 0x71, 0xf6, 0xf1, + 0x97, 0x02, 0x7f, 0x47, 0xad, 0xcd, 0x58, 0x45, 0xaf, 0xa3, 0xab, 0x8a, 0x1d, 0x43, 0x2f, 0xb7, + 0x20, 0xa7, 0x52, 0x4c, 0xd9, 0x39, 0xea, 0x31, 0x4e, 0x38, 0x77, 0xeb, 0x52, 0x68, 0x1b, 0xe9, + 0x3b, 0x59, 0x03, 0x9a, 0x47, 0x3d, 0x77, 0xd3, 0x02, 0xd5, 0x93, 0xa0, 0x75, 0x98, 0x65, 0x36, + 0x21, 0x14, 0x8a, 0x38, 0x85, 0x31, 0x40, 0x64, 0x01, 0x37, 0x69, 0x65, 0xf7, 0x1c, 0xcf, 0x55, + 0xc5, 0x3d, 0x78, 0x33, 0xb4, 0xaf, 0x71, 0x53, 0x48, 0x4a, 0xd0, 0x1e, 0x99, 0xe6, 0xb3, 0x7e, + 0x2f, 0x06, 0x43, 0x5e, 0x04, 0xe8, 0x2a, 0x87, 0x32, 0x0d, 0x25, 0xd8, 0x7c, 0xca, 0x70, 0x66, + 0xdf, 0x55, 0x0e, 0xe9, 0xbb, 0x6c, 0xb4, 0x04, 0xb3, 0x16, 0xa3, 0x1b, 0x01, 0xd7, 0xe2, 0x0a, + 0xb9, 0x37, 0xf9, 0x2f, 0xc2, 0x4a, 0xfc, 0x8d, 0x8b, 0xd3, 0xcd, 0xdd, 0x83, 0x19, 0xaf, 0x8d, + 0xc9, 0x63, 0xc5, 0x8e, 0xb8, 0x1e, 0xda, 0x85, 0x85, 0x9e, 0x85, 0xf7, 0xb1, 0xa3, 0xb6, 0xb1, + 0xe6, 0x76, 0x38, 0x79, 0x4c, 0xb0, 0xe2, 0x00, 0x82, 0xd9, 0x46, 0xfc, 0xb1, 0x00, 0xa7, 0x1f, + 0x60, 0xc5, 0x72, 0xf6, 0xb0, 0xe2, 0x34, 0x0f, 0xe3, 0xd8, 0x64, 0xef, 0x40, 0xd2, 0x30, 0x0f, + 0xf8, 0x19, 0x73, 0xfc, 0x0e, 0xca, 0x9b, 0x45, 0xea, 0x8b, 0xdf, 0x83, 0xc5, 0x60, 0x43, 0xe2, + 0x9c, 0x79, 0x7f, 0x96, 0x84, 0xec, 0x66, 0x2d, 0x86, 0xce, 0x7d, 0xc8, 0x0f, 0x03, 0xd1, 0x96, + 0xf7, 0x5e, 0x53, 0xdd, 0xac, 0x6d, 0xe1, 0x23, 0x97, 0x65, 0x12, 0x2d, 0xb4, 0x0a, 0x59, 0xa7, + 0x6d, 0x61, 0xbb, 0x6d, 0x76, 0x34, 0x4e, 0x0c, 0xa6, 0x32, 0xd0, 0x40, 0x0b, 0x75, 0xe0, 0x8c, + 0x73, 0x68, 0x50, 0x12, 0x20, 0xb7, 0x54, 0x79, 0x00, 0x97, 0x9e, 0x06, 0xae, 0xcc, 0xb7, 0x7b, + 0xd4, 0x3c, 0x34, 0x48, 0x0f, 0x37, 0x6b, 0x4d, 0x17, 0x40, 0x42, 0x0e, 0x97, 0xa9, 0x9e, 0xac, + 0x8c, 0x21, 0x4d, 0x7b, 0x81, 0xce, 0x41, 0x92, 0xec, 0x8a, 0x42, 0x70, 0x57, 0x24, 0x32, 0xda, + 0x29, 0xf7, 0x05, 0xc7, 0x19, 0xf5, 0x81, 0x96, 0xf8, 0x31, 0x00, 0x31, 0x5b, 0x9c, 0x23, 0xfe, + 0xb3, 0x24, 0xcc, 0xed, 0xf4, 0xed, 0x76, 0x3c, 0x73, 0xba, 0x06, 0xd0, 0xeb, 0xdb, 0x6d, 0x6c, + 0xc9, 0xce, 0xa1, 0xc1, 0x3b, 0x39, 0x21, 0xf2, 0xea, 0xf6, 0x92, 0xe9, 0x35, 0x0f, 0x0d, 0xf4, + 0x84, 0x83, 0x60, 0x79, 0x10, 0xbe, 0x7d, 0xdb, 0x07, 0xc2, 0x83, 0xdf, 0x55, 0x16, 0xf5, 0xae, + 0xba, 0xc1, 0xef, 0x6a, 0xf3, 0xd0, 0x78, 0x8c, 0x1d, 0x25, 0x00, 0x88, 0x09, 0xe0, 0x87, 0x30, + 0x4b, 0xfe, 0x91, 0x1d, 0xf3, 0x38, 0x93, 0x69, 0x86, 0xe8, 0x34, 0x4d, 0x77, 0x9d, 0xa6, 0x8f, + 0xb7, 0x4e, 0xc9, 0x70, 0xb3, 0x97, 0x92, 0x9d, 0x6b, 0x86, 0xee, 0x5c, 0x61, 0x96, 0xe0, 0xb6, + 0xf7, 0xed, 0x59, 0x19, 0xfa, 0x5e, 0xb2, 0x63, 0x95, 0x21, 0xbd, 0x6f, 0x5a, 0x2a, 0x2e, 0xcd, + 0xfa, 0x23, 0x4a, 0x54, 0xf4, 0x30, 0x95, 0xc9, 0x14, 0xb3, 0xe2, 0x9f, 0x0a, 0x30, 0xef, 0x8d, + 0x5e, 0x9c, 0x8e, 0xb8, 0x16, 0x18, 0x83, 0xe3, 0x0f, 0x24, 0xb1, 0xbb, 0xf8, 0x37, 0x09, 0x98, + 0xff, 0xb8, 0x8f, 0xad, 0xa3, 0x78, 0x26, 0xd7, 0x1a, 0x8b, 0xe7, 0x27, 0x4e, 0x38, 0x21, 0x68, + 0x84, 0xff, 0x1d, 0x98, 0x3f, 0x50, 0x74, 0x47, 0xde, 0x37, 0x2d, 0xb9, 0xdf, 0xd3, 0x14, 0x27, + 0x18, 0xfc, 0x2d, 0x90, 0xc2, 0xfb, 0xa6, 0xb5, 0x4b, 0x8b, 0x10, 0x06, 0xf4, 0xcc, 0x30, 0x0f, + 0x0c, 0x99, 0x88, 0x75, 0xa3, 0x45, 0x8c, 0x61, 0x97, 0x52, 0x34, 0xc0, 0xf1, 0x9d, 0x7f, 0x7b, + 0x59, 0xb9, 0xdd, 0xd2, 0x9d, 0x76, 0x7f, 0xaf, 0xaa, 0x9a, 0xdd, 0x15, 0xaf, 0x39, 0xda, 0xde, + 0x4a, 0xc8, 0x95, 0x4a, 0xbf, 0xaf, 0x6b, 0xd5, 0xdd, 0xdd, 0xfa, 0xba, 0x54, 0xa4, 0x90, 0x4f, + 0x19, 0x62, 0xf3, 0xd0, 0xb0, 0xc5, 0x6f, 0x04, 0x28, 0x0e, 0xec, 0x14, 0xe7, 0x30, 0x6e, 0x40, + 0xee, 0x79, 0x1f, 0x5b, 0x3a, 0xd6, 0x8e, 0x3d, 0x8e, 0xc0, 0x15, 0xc9, 0x02, 0xfa, 0x1c, 0xf2, + 0x01, 0x0b, 0x24, 0x5f, 0xcf, 0x02, 0xb9, 0x03, 0x5f, 0xe7, 0x7f, 0x94, 0x80, 0x45, 0x09, 0xdb, + 0x66, 0xe7, 0x05, 0xae, 0xd3, 0x53, 0x58, 0x0c, 0x33, 0xe5, 0x09, 0x00, 0x3f, 0x04, 0xbe, 0xce, + 0x84, 0xc9, 0x32, 0x0c, 0x62, 0x80, 0x35, 0x98, 0xb1, 0x1d, 0xc5, 0xe9, 0xb3, 0xe8, 0x5b, 0x38, + 0x07, 0xf5, 0x99, 0xb0, 0x41, 0xeb, 0xba, 0x7e, 0x84, 0x69, 0x92, 0x43, 0x75, 0xcf, 0xd4, 0x6d, + 0xd3, 0x08, 0xc6, 0xe7, 0x98, 0x4c, 0xfc, 0x3e, 0x9c, 0x19, 0xb2, 0x42, 0x9c, 0x5e, 0xfe, 0x77, + 0x13, 0x70, 0x2e, 0x08, 0x1f, 0x53, 0x6c, 0xf2, 0xff, 0xa0, 0xa5, 0xe7, 0x20, 0xbf, 0x6d, 0x9a, + 0x1e, 0x71, 0x15, 0x0b, 0x90, 0x63, 0xff, 0x53, 0x63, 0x90, 0x03, 0x53, 0x98, 0xa5, 0xe2, 0x3d, + 0x14, 0xe6, 0x63, 0x3a, 0xba, 0x9e, 0xec, 0xaa, 0x42, 0x6c, 0x42, 0xe1, 0x57, 0x70, 0xd6, 0xfd, + 0x2b, 0x01, 0x50, 0xd3, 0xea, 0x1b, 0xaa, 0xe2, 0xe0, 0x47, 0x66, 0x2b, 0x86, 0xde, 0x95, 0x21, + 0xad, 0x1b, 0x1a, 0x3e, 0xa4, 0xbd, 0x4b, 0xb9, 0x7d, 0xa0, 0x22, 0x74, 0x07, 0x32, 0xfc, 0x42, + 0x95, 0xdd, 0xb8, 0x24, 0x3d, 0xd2, 0x36, 0xcb, 0xae, 0x50, 0xd7, 0xbf, 0x19, 0x3c, 0x4a, 0xb3, + 0xec, 0x16, 0x55, 0x13, 0x3f, 0x87, 0xd3, 0x81, 0x36, 0xc6, 0x69, 0x80, 0x7f, 0xa4, 0x01, 0x5b, + 0xda, 0xeb, 0xb8, 0xce, 0xf9, 0x27, 0xba, 0x0c, 0x47, 0x1f, 0x01, 0xf4, 0x2c, 0xfc, 0x42, 0x66, + 0xaa, 0xc9, 0xa9, 0x54, 0xb3, 0x44, 0x83, 0x0a, 0xc4, 0x7f, 0x12, 0x60, 0x31, 0xee, 0x80, 0xc5, + 0xb7, 0xd2, 0x91, 0xc7, 0x50, 0xa4, 0x0f, 0x75, 0x63, 0xdf, 0x8c, 0x21, 0x4e, 0xf4, 0xfb, 0x02, + 0x2c, 0xf8, 0xf0, 0xe2, 0xdc, 0xa8, 0x4f, 0x96, 0xf4, 0xf0, 0x3d, 0xb2, 0x75, 0xfa, 0xe7, 0x5b, + 0x9c, 0xb3, 0xf9, 0xc7, 0x09, 0x78, 0xa3, 0x66, 0x76, 0x7b, 0x7d, 0x07, 0xd3, 0xc8, 0xbb, 0xdd, + 0xef, 0xc6, 0x30, 0x0f, 0x96, 0x60, 0xf6, 0x05, 0xb6, 0x6c, 0xdd, 0x64, 0xbb, 0x45, 0xc1, 0x0d, + 0x32, 0x70, 0x21, 0xfa, 0x4d, 0xc8, 0xa9, 0xfc, 0x6d, 0xee, 0xca, 0xce, 0xaf, 0xd5, 0x49, 0x9d, + 0x13, 0xb2, 0x8d, 0x57, 0x2f, 0x2b, 0xe0, 0xb6, 0xbf, 0xbe, 0x2e, 0x81, 0x8b, 0x5e, 0xd7, 0xd0, + 0x32, 0x64, 0x6c, 0x43, 0xe9, 0xd9, 0x6d, 0x33, 0x18, 0x4c, 0xf5, 0xa4, 0x1f, 0xa4, 0x7e, 0xf1, + 0x65, 0x45, 0x10, 0x7f, 0x00, 0x67, 0x47, 0x0c, 0x11, 0xa7, 0xa5, 0xbf, 0x0f, 0x95, 0x75, 0xdc, + 0xb3, 0x30, 0x71, 0x4a, 0xda, 0x27, 0xd8, 0xd2, 0xf7, 0x8f, 0xe2, 0xb3, 0xb8, 0xd8, 0x82, 0xe5, + 0x68, 0xf4, 0x38, 0xbb, 0xf1, 0x77, 0xb3, 0x50, 0xd8, 0x38, 0xec, 0x99, 0x96, 0xd3, 0x60, 0xbb, + 0x3e, 0x7a, 0x08, 0x99, 0x9e, 0x65, 0xbe, 0xd0, 0x5d, 0xe0, 0xb9, 0xd0, 0xf8, 0x7e, 0x40, 0x67, + 0x87, 0xd7, 0xf7, 0x0e, 0x43, 0xfc, 0x7f, 0x24, 0x41, 0xf6, 0x91, 0xa9, 0x2a, 0x9d, 0xfb, 0x7a, + 0xc7, 0x5d, 0x25, 0xd5, 0x49, 0x60, 0x55, 0x4f, 0x63, 0x47, 0x71, 0xda, 0xae, 0x7f, 0xf0, 0x84, + 0x68, 0x13, 0x32, 0x0f, 0x1c, 0xa7, 0x47, 0x0a, 0xb9, 0x73, 0xb9, 0x3c, 0x11, 0x92, 0x28, 0xb8, + 0x8d, 0x73, 0x95, 0x91, 0x04, 0x0b, 0x9b, 0xa6, 0xd9, 0xea, 0xe0, 0x5a, 0xc7, 0xec, 0x6b, 0x35, + 0xd3, 0xd8, 0xd7, 0x5b, 0xfc, 0xac, 0x79, 0x69, 0x22, 0xe2, 0x66, 0xad, 0x21, 0x8d, 0xaa, 0xa3, + 0xef, 0x42, 0xa6, 0x71, 0x9b, 0x43, 0xb1, 0xc3, 0xe7, 0xc5, 0x89, 0x50, 0x8d, 0xdb, 0x92, 0xa7, + 0x84, 0x1e, 0x40, 0x6e, 0xf5, 0x8b, 0xbe, 0x85, 0x39, 0xc6, 0x0c, 0xc5, 0xb8, 0x32, 0x11, 0x83, + 0xea, 0x48, 0x7e, 0xd5, 0xf2, 0x75, 0x28, 0x04, 0x2c, 0x89, 0x4a, 0x90, 0xea, 0x11, 0xa3, 0x91, + 0x41, 0xcd, 0xba, 0xa1, 0x1b, 0x22, 0x29, 0x5f, 0x81, 0x14, 0xb1, 0x0a, 0x59, 0xe7, 0x7b, 0x8a, + 0x8d, 0x77, 0x2d, 0x3d, 0x50, 0xc9, 0x15, 0x96, 0xff, 0x56, 0x80, 0x44, 0xe3, 0x36, 0xa1, 0x6a, + 0x7b, 0x7d, 0xf5, 0x19, 0x76, 0x02, 0xb5, 0xb8, 0x8c, 0x12, 0x39, 0x0b, 0xef, 0xeb, 0x8c, 0x00, + 0x78, 0xa5, 0x4c, 0x86, 0x2e, 0x02, 0x28, 0xaa, 0x8a, 0x6d, 0x9b, 0x5e, 0x44, 0x24, 0x7d, 0x35, + 0xb2, 0x4c, 0xbe, 0x85, 0x8f, 0x08, 0x84, 0x8d, 0x55, 0x0b, 0xb3, 0x15, 0xee, 0x41, 0x30, 0x19, + 0x81, 0x70, 0x70, 0xb7, 0x27, 0x3b, 0xe6, 0x33, 0x6c, 0x50, 0x2b, 0x7b, 0x10, 0x44, 0xde, 0x24, + 0xe2, 0xf2, 0x2a, 0x24, 0x37, 0x6b, 0x8d, 0xd7, 0x69, 0x6a, 0xf9, 0x27, 0x02, 0xa4, 0xa9, 0x41, + 0x91, 0x08, 0x59, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xf8, 0x9a, 0xf0, 0x5e, 0xe8, 0x89, 0x27, 0x74, + 0xfb, 0x2a, 0xe4, 0x15, 0x55, 0x35, 0xfb, 0x86, 0x23, 0x1b, 0x4a, 0x17, 0x07, 0x3a, 0x9e, 0xe3, + 0x25, 0xdb, 0x4a, 0x17, 0xa3, 0xcb, 0xe0, 0xfe, 0x4b, 0x0d, 0xe4, 0xef, 0x3f, 0xf0, 0x82, 0x2d, + 0x7c, 0x24, 0xfe, 0x85, 0x00, 0x0b, 0x4f, 0x2d, 0xdd, 0xc1, 0x6b, 0x8a, 0xa3, 0xb6, 0x63, 0x70, + 0xf1, 0x1f, 0x40, 0x56, 0x53, 0x1c, 0x85, 0xe5, 0xd0, 0x25, 0xc6, 0x6b, 0xf3, 0x85, 0x44, 0xea, + 0xd3, 0x3c, 0x3a, 0x04, 0x29, 0xf2, 0xcc, 0xfc, 0xbe, 0x44, 0x9f, 0xc5, 0xcf, 0x00, 0xf9, 0xdb, + 0x17, 0xa7, 0xcb, 0xfa, 0x5a, 0x70, 0x5d, 0x56, 0x0c, 0xfd, 0xbe, 0x07, 0xb3, 0xfc, 0xb8, 0xc3, + 0x7b, 0xbd, 0x3c, 0x69, 0xad, 0xb9, 0x8b, 0x82, 0xab, 0xa1, 0x35, 0x00, 0xdb, 0x51, 0x2c, 0x47, + 0x76, 0xf4, 0xee, 0x74, 0x77, 0xab, 0xee, 0xe4, 0xa1, 0x6a, 0x44, 0x2a, 0xfe, 0xb5, 0x00, 0x85, + 0xb5, 0x7e, 0xe7, 0xd9, 0x93, 0x5e, 0xa3, 0xdf, 0xed, 0x2a, 0xd6, 0x11, 0xba, 0xe0, 0x8e, 0x87, + 0xfe, 0x05, 0xa6, 0xbd, 0x4a, 0x06, 0xcc, 0xae, 0x7f, 0x81, 0xc9, 0x7a, 0xe6, 0xd9, 0x00, 0x83, + 0x52, 0x2a, 0x41, 0xd7, 0xa1, 0x40, 0xf9, 0xb6, 0x8c, 0x0d, 0xc7, 0xd2, 0xb1, 0xcd, 0xb9, 0x36, + 0xab, 0x92, 0xa7, 0x45, 0x1b, 0xac, 0x04, 0xdd, 0x80, 0x39, 0xfb, 0xc8, 0x76, 0x70, 0x57, 0x66, + 0x79, 0xa3, 0x36, 0x9d, 0x72, 0x6e, 0xdd, 0x02, 0x2b, 0x93, 0x58, 0x91, 0xf8, 0x2f, 0x09, 0x98, + 0x73, 0x2d, 0x1f, 0x27, 0x91, 0x5a, 0x83, 0xf4, 0xbe, 0xde, 0xf1, 0x2e, 0x10, 0xa2, 0xdd, 0x9d, + 0x8b, 0x54, 0x25, 0x4e, 0xcd, 0x8b, 0xad, 0x11, 0xd5, 0xf2, 0xcf, 0x04, 0x48, 0xd1, 0xfd, 0xe1, + 0x3d, 0x48, 0xd1, 0x49, 0x2c, 0x4c, 0x33, 0x89, 0x69, 0x55, 0xcf, 0x33, 0x26, 0x86, 0x3d, 0x23, + 0x7a, 0x03, 0x66, 0xec, 0xb6, 0x72, 0xe7, 0xbd, 0x5b, 0xd4, 0xcf, 0xe4, 0x25, 0xfe, 0x1f, 0x5a, + 0x83, 0x0c, 0xa6, 0x2d, 0xc2, 0x1a, 0xf7, 0xd1, 0x61, 0xf3, 0x26, 0x30, 0xa4, 0xee, 0xf8, 0xb9, + 0x7a, 0xec, 0x6a, 0xe6, 0x61, 0x2a, 0x93, 0x2a, 0xa6, 0xc5, 0x9f, 0x27, 0xa1, 0x50, 0xef, 0xc6, + 0x34, 0xa7, 0x57, 0x83, 0xe6, 0x0c, 0xdb, 0x1e, 0x03, 0xef, 0x1a, 0xb5, 0x66, 0xd0, 0x1d, 0x24, + 0x8f, 0xe7, 0x0e, 0xea, 0x30, 0x63, 0x61, 0x9e, 0x54, 0x4b, 0xde, 0x7f, 0x63, 0xe2, 0xfb, 0x9b, + 0xca, 0x5e, 0x07, 0x4b, 0x44, 0xc7, 0xbb, 0x18, 0xa2, 0x00, 0xe5, 0x43, 0x3e, 0xa6, 0xef, 0x43, + 0x52, 0xd3, 0x5d, 0x4b, 0x4c, 0xbb, 0x42, 0x89, 0xca, 0x54, 0x43, 0x9b, 0xf2, 0x0f, 0x2d, 0x1b, + 0x96, 0xf2, 0x63, 0x80, 0x41, 0xab, 0xd0, 0x25, 0x98, 0x31, 0x3b, 0x1a, 0xe1, 0xb6, 0x02, 0xe5, + 0xbf, 0x05, 0x7e, 0x6a, 0x4d, 0x3f, 0xe9, 0x68, 0xf5, 0x75, 0x29, 0x6d, 0x76, 0xb4, 0xba, 0x86, + 0xce, 0x41, 0xc6, 0xc0, 0x07, 0x32, 0x4d, 0xa3, 0xa6, 0x57, 0xec, 0xd2, 0xac, 0x81, 0x0f, 0xd6, + 0xb1, 0xad, 0xf2, 0x51, 0xfe, 0x4b, 0x01, 0xe6, 0xdc, 0x9e, 0xc7, 0xbb, 0x7e, 0x32, 0x7a, 0x97, + 0xcf, 0xc6, 0xe4, 0xf1, 0x66, 0xa3, 0xab, 0xc7, 0xd3, 0x97, 0x76, 0xe0, 0x34, 0x4b, 0x00, 0x50, + 0x15, 0xc7, 0xc1, 0x56, 0x0c, 0x4c, 0xf6, 0x97, 0x02, 0x2c, 0x06, 0x21, 0xe3, 0xec, 0xf9, 0xd6, + 0xd0, 0xdd, 0xe3, 0xbb, 0x21, 0x20, 0x61, 0x6f, 0x67, 0x77, 0x88, 0xc1, 0x6b, 0xc8, 0xf2, 0x3d, + 0x48, 0x53, 0xf1, 0x09, 0x5c, 0x08, 0x37, 0xdf, 0x1e, 0x2c, 0xac, 0x6a, 0x5a, 0xa3, 0xc1, 0xa7, + 0xce, 0x6b, 0xaf, 0x64, 0x77, 0x67, 0x4d, 0x04, 0x77, 0x56, 0xff, 0x3b, 0xe2, 0xdc, 0x59, 0xff, + 0x79, 0x11, 0xf2, 0xbc, 0xd5, 0xbb, 0x06, 0x39, 0xd8, 0xad, 0x40, 0xb2, 0xc5, 0xb9, 0x53, 0x70, + 0x53, 0xf3, 0x2e, 0x04, 0xbd, 0xcc, 0x76, 0x89, 0xd4, 0x24, 0x0a, 0xbd, 0xbe, 0x13, 0x72, 0x53, + 0x36, 0xb8, 0x3a, 0x19, 0x28, 0xf4, 0xfa, 0x0e, 0xfa, 0x18, 0xe6, 0xd5, 0x41, 0x8e, 0xb1, 0x4c, + 0x94, 0x93, 0x91, 0x49, 0x45, 0xa1, 0xa9, 0xd6, 0xd2, 0x9c, 0x1a, 0x10, 0xa3, 0x55, 0x7f, 0x42, + 0x6c, 0x2a, 0x92, 0x84, 0x0f, 0xe7, 0xe0, 0xfa, 0xf2, 0x65, 0xd1, 0xfb, 0x30, 0xa3, 0xd1, 0x94, + 0x4b, 0x4e, 0xe2, 0xc3, 0x96, 0x53, 0x20, 0xa7, 0x55, 0xe2, 0xf5, 0xd1, 0x03, 0xc8, 0xb3, 0x27, + 0x76, 0x8d, 0xcd, 0x37, 0x87, 0xcb, 0xd1, 0xfa, 0xbe, 0x98, 0xae, 0x94, 0xd3, 0x06, 0x32, 0x74, + 0x0b, 0x52, 0xb6, 0xaa, 0x18, 0xf4, 0x1e, 0x29, 0x3c, 0xfe, 0xef, 0x4b, 0x5c, 0x94, 0x68, 0x5d, + 0xf4, 0x14, 0x16, 0xf6, 0x70, 0x4b, 0x37, 0x64, 0x67, 0x10, 0x6d, 0x2d, 0x65, 0x46, 0x02, 0xbc, + 0x9e, 0x47, 0x08, 0xcf, 0x9d, 0x93, 0x8a, 0x7b, 0x43, 0x05, 0x64, 0x98, 0xb0, 0xa1, 0x05, 0x60, + 0xb3, 0x91, 0xc3, 0x14, 0x9a, 0xda, 0x26, 0xcd, 0xe1, 0x80, 0x18, 0x6d, 0x40, 0x4e, 0x21, 0x2b, + 0x53, 0xa6, 0xd9, 0x47, 0x25, 0x88, 0x3c, 0x78, 0x8d, 0xe4, 0x41, 0x49, 0xa0, 0x78, 0xa2, 0x01, + 0x4c, 0x17, 0x5b, 0x2d, 0x5c, 0xca, 0x8d, 0x87, 0xf1, 0x87, 0x70, 0x39, 0x0c, 0x15, 0xa1, 0x2d, + 0x28, 0xb4, 0xdd, 0x1b, 0x7a, 0x1a, 0x16, 0xcf, 0x47, 0x9e, 0xbc, 0x42, 0x52, 0x0a, 0xa4, 0x7c, + 0xdb, 0x27, 0x44, 0xef, 0x40, 0xa2, 0xa5, 0x96, 0x0a, 0x14, 0xe1, 0xfc, 0xb8, 0x6b, 0x74, 0x29, + 0xd1, 0x52, 0xd1, 0x87, 0x90, 0x61, 0x97, 0x8e, 0x87, 0x46, 0x69, 0x2e, 0x72, 0xf1, 0x06, 0xef, + 0x7b, 0x25, 0x7a, 0x39, 0x4a, 0xde, 0xf5, 0x00, 0xf2, 0x2c, 0xa4, 0xda, 0xa1, 0xa9, 0x1d, 0xa5, + 0xf9, 0xc8, 0x09, 0x37, 0x9a, 0x9d, 0x22, 0xb1, 0xcf, 0x5b, 0x98, 0x0c, 0x6d, 0xc3, 0x9c, 0xc5, + 0x82, 0xe8, 0x32, 0x0b, 0xed, 0x97, 0x8a, 0x14, 0xeb, 0x6a, 0xb8, 0x2b, 0x19, 0xb9, 0xfc, 0x91, + 0x0a, 0x96, 0x5f, 0x8a, 0x7e, 0x00, 0x8b, 0x41, 0x3c, 0xbe, 0x24, 0x16, 0x28, 0xea, 0x3b, 0x13, + 0x51, 0xfd, 0x2b, 0x03, 0x59, 0x23, 0x45, 0xe8, 0x0e, 0xa4, 0xd9, 0x98, 0x23, 0x0a, 0x58, 0x09, + 0x01, 0x0c, 0x0c, 0x37, 0xab, 0x4d, 0x0c, 0xe6, 0xf0, 0x60, 0xb2, 0xdc, 0x31, 0x5b, 0xa5, 0xd3, + 0x91, 0x06, 0x1b, 0x8d, 0x8b, 0x4b, 0x39, 0x67, 0x20, 0x23, 0x73, 0xc6, 0x62, 0x72, 0x1e, 0xeb, + 0x5c, 0x8c, 0x9c, 0x33, 0x21, 0x11, 0x66, 0x29, 0x6f, 0xf9, 0x84, 0x74, 0x1c, 0x59, 0xce, 0x8e, + 0x4c, 0x97, 0xfd, 0x99, 0xe8, 0x71, 0x1c, 0x49, 0x5b, 0x96, 0x72, 0xd6, 0x40, 0x86, 0x9a, 0x50, + 0x54, 0x59, 0xe4, 0x4b, 0x76, 0xe3, 0x66, 0xa5, 0x37, 0x28, 0xda, 0xf5, 0x50, 0x9f, 0x1a, 0x16, + 0x2d, 0x94, 0xe6, 0xd5, 0xa0, 0x1c, 0xf5, 0xa0, 0xac, 0x79, 0x11, 0x29, 0xf9, 0x05, 0x0d, 0x49, + 0x0d, 0xf0, 0xcf, 0x52, 0xfc, 0x5b, 0xa1, 0x6e, 0x6e, 0x6c, 0x90, 0x4c, 0x2a, 0x69, 0x11, 0x15, + 0x88, 0x33, 0xa3, 0xf8, 0xb2, 0x3a, 0x48, 0x1f, 0x2e, 0x95, 0x22, 0x9d, 0x59, 0x44, 0x8a, 0xb3, + 0x54, 0x54, 0x87, 0x0a, 0x88, 0x67, 0x35, 0x4c, 0xb3, 0x57, 0x3a, 0x17, 0xe9, 0x59, 0x7d, 0x77, + 0x4b, 0x12, 0xad, 0x4b, 0x16, 0xa9, 0x6e, 0xe8, 0x0e, 0xdd, 0xa0, 0xca, 0x91, 0x8b, 0x34, 0xf8, + 0x39, 0x8b, 0x34, 0xab, 0xb3, 0xff, 0xc9, 0xd2, 0x72, 0x78, 0x6c, 0x9e, 0x4f, 0x95, 0xf3, 0x91, + 0x4b, 0x2b, 0x2c, 0x88, 0x2f, 0x15, 0x1c, 0xbf, 0x94, 0x2c, 0x2d, 0xe6, 0xf4, 0x86, 0x50, 0xdf, + 0x8a, 0x5c, 0x5a, 0x91, 0x09, 0x8d, 0x12, 0x52, 0x46, 0x8a, 0xc8, 0x99, 0x96, 0x02, 0xd2, 0x0f, + 0xdf, 0x4a, 0x4b, 0x91, 0x7b, 0xe8, 0x70, 0xa0, 0x5e, 0xca, 0x76, 0x5c, 0x09, 0x71, 0xcc, 0x07, + 0x96, 0xee, 0x60, 0x79, 0x4f, 0x71, 0xd4, 0x76, 0xa9, 0x12, 0xe9, 0x98, 0x47, 0xe2, 0x18, 0x12, + 0x1c, 0x78, 0x22, 0xb2, 0x15, 0xb3, 0x13, 0x53, 0x69, 0x79, 0x02, 0xfb, 0xf7, 0xb6, 0x62, 0x56, + 0x1f, 0x7d, 0x17, 0xb2, 0xcf, 0xfb, 0xd8, 0x3a, 0xa2, 0x8e, 0xf5, 0x42, 0xe4, 0x67, 0x8d, 0x43, + 0xc9, 0x0e, 0x52, 0xe6, 0x39, 0x17, 0x90, 0x57, 0x33, 0x7a, 0x5c, 0x12, 0x23, 0x5f, 0x1d, 0x38, + 0xc8, 0x48, 0xbc, 0x3e, 0x52, 0xe0, 0x0c, 0x1b, 0x1f, 0x9e, 0x09, 0x69, 0xf1, 0x94, 0xc3, 0xd2, + 0x45, 0x0a, 0x14, 0xc9, 0x52, 0x43, 0x93, 0x31, 0xa5, 0xd3, 0xca, 0x68, 0x19, 0x71, 0x3e, 0x7c, + 0xfb, 0x64, 0xcc, 0xb6, 0x74, 0x29, 0xd2, 0xf9, 0x84, 0x30, 0x7a, 0x29, 0xaf, 0xf8, 0x84, 0x6c, + 0x13, 0xd5, 0x64, 0xdb, 0x76, 0x08, 0xa9, 0x2c, 0x5d, 0x1e, 0xb3, 0x89, 0x0e, 0xb1, 0x5b, 0xb2, + 0x89, 0x6a, 0x0d, 0xa6, 0xf7, 0x41, 0xea, 0xab, 0x2f, 0x2b, 0xc2, 0xc3, 0x54, 0xe6, 0xcd, 0xe2, + 0x79, 0xf1, 0xa7, 0x8b, 0x50, 0x70, 0x09, 0x27, 0x23, 0x93, 0x37, 0xfd, 0x64, 0x72, 0x29, 0x8a, + 0x4c, 0x32, 0x0d, 0xc6, 0x26, 0x6f, 0xfa, 0xd9, 0xe4, 0x52, 0x14, 0x9b, 0x74, 0x35, 0x08, 0x9d, + 0x94, 0xa2, 0xe8, 0xe4, 0xf5, 0x29, 0xe8, 0x24, 0x07, 0x1a, 0xe6, 0x93, 0x6b, 0xa3, 0x7c, 0xf2, + 0xd2, 0x78, 0x3e, 0xc9, 0x81, 0x7c, 0x84, 0xf2, 0xee, 0x10, 0xa1, 0xbc, 0x30, 0x86, 0x50, 0x72, + 0x6d, 0x97, 0x51, 0xd6, 0x43, 0x19, 0xe5, 0x95, 0x49, 0x8c, 0x92, 0xa3, 0x04, 0x28, 0xe5, 0xed, + 0x00, 0xa5, 0xac, 0x44, 0x52, 0x4a, 0xae, 0xcb, 0x38, 0xe5, 0xa7, 0xd1, 0x9c, 0xf2, 0xc6, 0x54, + 0x9c, 0x92, 0xa3, 0x8d, 0x92, 0x4a, 0x29, 0x8a, 0x54, 0x5e, 0x9f, 0x82, 0x54, 0xba, 0x83, 0x35, + 0xc4, 0x2a, 0xef, 0x87, 0xb1, 0xca, 0xcb, 0x13, 0x58, 0x25, 0xc7, 0xf2, 0xd3, 0xca, 0xfb, 0x61, + 0xb4, 0xf2, 0xf2, 0x04, 0x5a, 0x19, 0xc0, 0x61, 0xbc, 0xf2, 0x51, 0x38, 0xaf, 0xbc, 0x3a, 0x91, + 0x57, 0x72, 0xac, 0x20, 0xb1, 0x7c, 0xd7, 0x47, 0x2c, 0xdf, 0x8a, 0x20, 0x96, 0x5c, 0x91, 0x30, + 0xcb, 0x8f, 0x46, 0x98, 0xa5, 0x38, 0x8e, 0x59, 0x72, 0x4d, 0x8f, 0x5a, 0xd6, 0x43, 0xa9, 0xe5, + 0x95, 0x49, 0xd4, 0xd2, 0x9d, 0x79, 0x7e, 0x6e, 0xf9, 0x24, 0x82, 0x5b, 0x5e, 0x9b, 0xcc, 0x2d, + 0x39, 0xdc, 0x10, 0xb9, 0x94, 0xc7, 0x92, 0xcb, 0x77, 0xa7, 0x24, 0x97, 0x1c, 0x3b, 0x8c, 0x5d, + 0xfe, 0xff, 0x20, 0xbb, 0x5c, 0x8e, 0x66, 0x97, 0x1c, 0x84, 0xd3, 0xcb, 0x7a, 0x28, 0xbd, 0xbc, + 0x32, 0x89, 0x5e, 0xba, 0x46, 0xf3, 0xf3, 0xcb, 0x47, 0xe1, 0xfc, 0xf2, 0xea, 0x44, 0x7e, 0xe9, + 0xce, 0x9d, 0x00, 0xc1, 0xac, 0x87, 0x12, 0xcc, 0x2b, 0x93, 0x08, 0xa6, 0x37, 0x9a, 0x3e, 0x86, + 0xb9, 0x1b, 0xc9, 0x30, 0xdf, 0x9e, 0x86, 0x61, 0x72, 0xc8, 0x11, 0x8a, 0xf9, 0x7c, 0x0a, 0x8a, + 0x79, 0xfb, 0x58, 0x14, 0x93, 0xbf, 0x29, 0x9a, 0x63, 0x7e, 0x1a, 0xcd, 0x31, 0x6f, 0x4c, 0xc5, + 0x31, 0x5d, 0xe7, 0x36, 0x42, 0x32, 0x6f, 0x07, 0x48, 0x66, 0x25, 0x92, 0x64, 0xba, 0xbe, 0x96, + 0xb2, 0xcc, 0x8f, 0x46, 0x58, 0xa6, 0x38, 0x8e, 0x65, 0xba, 0x0b, 0xd6, 0xa5, 0x99, 0xf2, 0x58, + 0x5a, 0xf8, 0xee, 0x94, 0xb4, 0xd0, 0x5d, 0x14, 0x21, 0xbc, 0xb0, 0x16, 0xc2, 0x0b, 0x2f, 0x8d, + 0xe7, 0x85, 0xee, 0x5e, 0x38, 0x20, 0x86, 0xf7, 0xc3, 0x88, 0xe1, 0xe5, 0x09, 0xc4, 0xd0, 0x75, + 0xad, 0x3e, 0x66, 0x78, 0x77, 0x88, 0x19, 0x5e, 0x98, 0x78, 0x6d, 0xe0, 0x51, 0xc3, 0x7b, 0xa3, + 0xd4, 0xf0, 0xe2, 0x58, 0x6a, 0xc8, 0xf5, 0x07, 0xdc, 0xf0, 0xee, 0x10, 0x37, 0xbc, 0x30, 0x86, + 0x1b, 0xba, 0x2f, 0xe7, 0xe4, 0x70, 0x6f, 0x3c, 0x39, 0xac, 0x4e, 0x4b, 0x0e, 0x39, 0x6c, 0x28, + 0x3b, 0x7c, 0x14, 0xce, 0x0e, 0xaf, 0x4e, 0x19, 0x1e, 0x1d, 0xa2, 0x87, 0xf7, 0xc3, 0xe8, 0xe1, + 0xe5, 0x09, 0xf4, 0x70, 0xb0, 0x19, 0x86, 0xf1, 0xc3, 0x87, 0xa9, 0xcc, 0xf9, 0xe2, 0x5b, 0xe2, + 0x7f, 0xa7, 0x60, 0xe6, 0x81, 0x7b, 0x6b, 0xe1, 0x4b, 0xb5, 0x17, 0x4e, 0x92, 0x6a, 0x8f, 0xd6, + 0x61, 0x96, 0x1b, 0x93, 0x73, 0xc6, 0x31, 0x5f, 0x0b, 0x8d, 0x7c, 0x3f, 0xe2, 0xaa, 0x9e, 0x30, + 0x49, 0x0d, 0xdd, 0x85, 0x42, 0xdf, 0xc6, 0x96, 0xdc, 0xb3, 0x74, 0xd3, 0xd2, 0x1d, 0x76, 0x77, + 0x2b, 0xac, 0x2d, 0xf2, 0x8f, 0xbc, 0xf2, 0xbb, 0x36, 0xb6, 0x76, 0x78, 0x99, 0x94, 0xef, 0xfb, + 0xfe, 0x73, 0x7f, 0xf4, 0x24, 0x3d, 0xfd, 0x8f, 0x9e, 0x3c, 0x85, 0xa2, 0x85, 0x15, 0x2d, 0xe0, + 0xb6, 0x58, 0xbe, 0x7a, 0xb8, 0x43, 0x57, 0x34, 0x9f, 0x6f, 0xf2, 0xe5, 0xad, 0xcf, 0x5b, 0xc1, + 0x22, 0xf4, 0x1d, 0x38, 0xd3, 0x55, 0x0e, 0xd9, 0x27, 0x18, 0xee, 0xe6, 0x43, 0xef, 0x72, 0x32, + 0xbe, 0x6b, 0x41, 0xd4, 0x55, 0x0e, 0xe9, 0xaf, 0xa9, 0xb0, 0x0a, 0xf4, 0x4b, 0xf6, 0x1b, 0x30, + 0xa7, 0xe9, 0xb6, 0xa3, 0x1b, 0xaa, 0xfb, 0x15, 0x67, 0xd6, 0x9f, 0xa3, 0xed, 0x96, 0xb1, 0xcf, + 0x35, 0x6f, 0xc2, 0x02, 0xff, 0x3a, 0x7e, 0xf0, 0xfb, 0x2a, 0x94, 0xab, 0x65, 0x06, 0xed, 0x22, + 0xc5, 0x83, 0x9f, 0xa7, 0xd9, 0x84, 0xf9, 0x96, 0xe2, 0xe0, 0x03, 0xe5, 0x48, 0x36, 0x4c, 0x8d, + 0x8e, 0x4d, 0x8e, 0x7e, 0x11, 0x55, 0xe1, 0x63, 0x53, 0xd8, 0x64, 0xc5, 0xdb, 0xa6, 0xc6, 0x46, + 0x68, 0x86, 0x3d, 0x49, 0x85, 0x96, 0xaf, 0x40, 0x7b, 0x98, 0xca, 0xcc, 0x16, 0x33, 0xe2, 0x1f, + 0x09, 0x90, 0x0f, 0x5c, 0x9d, 0xff, 0xda, 0x50, 0xfc, 0xfc, 0x5c, 0x38, 0x41, 0x0b, 0xbf, 0x85, + 0x58, 0x85, 0x0c, 0xb7, 0x95, 0x7b, 0x0f, 0x51, 0x89, 0xde, 0xa3, 0xe9, 0x61, 0xc8, 0xbd, 0x7e, + 0x71, 0xd5, 0x3e, 0x48, 0xfd, 0xf1, 0x97, 0x95, 0x53, 0xe2, 0x2f, 0x92, 0x50, 0x08, 0xde, 0x98, + 0xd7, 0x87, 0xda, 0x15, 0xb6, 0x82, 0x03, 0x1a, 0xd1, 0xad, 0x5c, 0x87, 0xac, 0xc5, 0x2b, 0xb9, + 0xcd, 0x5c, 0x1e, 0x73, 0x4b, 0xe0, 0x6f, 0xe7, 0x40, 0xb1, 0xfc, 0x0f, 0x09, 0x6f, 0xc5, 0x56, + 0x21, 0x4d, 0x7f, 0x15, 0x89, 0x37, 0x2d, 0x2c, 0xff, 0x6d, 0x83, 0x94, 0x4b, 0xac, 0x1a, 0x59, + 0xe1, 0xcd, 0x13, 0x7d, 0x4c, 0xe3, 0x09, 0x4e, 0xf0, 0xf3, 0x40, 0x27, 0xfc, 0x12, 0xa4, 0x41, + 0x0e, 0x92, 0x9d, 0x0e, 0x56, 0x1d, 0xfe, 0x4b, 0x4a, 0xee, 0xcf, 0xff, 0x5c, 0x1a, 0x86, 0xe0, + 0xbf, 0xbb, 0x54, 0x95, 0xf8, 0xef, 0x2e, 0xf9, 0xae, 0x86, 0xe6, 0x3c, 0x08, 0x3a, 0xed, 0xd9, + 0x2d, 0x20, 0x1b, 0xea, 0xb7, 0x1f, 0xc1, 0xe9, 0x90, 0x65, 0x89, 0xe6, 0x00, 0x6a, 0x4f, 0xb6, + 0x1b, 0xf5, 0x46, 0x73, 0x63, 0xbb, 0x59, 0x3c, 0x85, 0x0a, 0x90, 0x25, 0xff, 0x6f, 0x6c, 0x37, + 0x76, 0x1b, 0x45, 0x01, 0x15, 0x21, 0x5f, 0xdf, 0xf6, 0x55, 0x48, 0x94, 0x53, 0xbf, 0xf3, 0xd3, + 0xa5, 0x53, 0x6f, 0x3f, 0x85, 0x9c, 0xef, 0xa3, 0x14, 0x84, 0x60, 0x6e, 0x67, 0xb7, 0xf1, 0x40, + 0x6e, 0xd6, 0x1f, 0x6f, 0x34, 0x9a, 0xab, 0x8f, 0x77, 0x8a, 0xa7, 0x08, 0x32, 0x95, 0xad, 0xae, + 0x3d, 0x91, 0x9a, 0x45, 0xc1, 0xfb, 0xbf, 0xf9, 0x64, 0xb7, 0xf6, 0xa0, 0x98, 0xf0, 0xfe, 0xff, + 0x78, 0x77, 0x43, 0xfa, 0xac, 0x98, 0xe4, 0xc0, 0x0a, 0x9c, 0x09, 0x4d, 0xf5, 0x42, 0x39, 0x98, + 0xdd, 0x35, 0xe8, 0x57, 0x11, 0xac, 0x95, 0x5e, 0xae, 0x51, 0x51, 0x40, 0x19, 0x96, 0x4f, 0x54, + 0x4c, 0xa0, 0x19, 0x48, 0x34, 0x6e, 0x17, 0x93, 0x68, 0x1e, 0x72, 0xbe, 0x64, 0xa9, 0x62, 0x0a, + 0x65, 0x79, 0x6e, 0x4d, 0x31, 0x7d, 0xeb, 0x53, 0xc8, 0xb8, 0x5f, 0x8b, 0xa3, 0x47, 0x90, 0x66, + 0x9b, 0x7b, 0x25, 0x7a, 0x9e, 0xd3, 0x15, 0x53, 0x5e, 0x9e, 0xb4, 0x10, 0xc4, 0x53, 0x04, 0x79, + 0xe3, 0xf0, 0x57, 0x81, 0xbc, 0x76, 0xe1, 0xab, 0xff, 0x5c, 0x3a, 0xf5, 0xd5, 0xab, 0x25, 0xe1, + 0xeb, 0x57, 0x4b, 0xc2, 0xbf, 0xbe, 0x5a, 0x12, 0xfe, 0xe3, 0xd5, 0x92, 0xf0, 0x07, 0x3f, 0x5f, + 0x3a, 0xf5, 0xf9, 0x2c, 0x57, 0xf9, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x6c, 0x9c, 0xf1, 0xec, + 0x9e, 0x4c, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 94a567ada5e0..82a0f2699924 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -865,8 +865,7 @@ message AdminScatterResponse { message Range { optional Span span = 1 [(gogoproto.nullable) = false]; - // Error will be nil if this range was successfully scattered. - optional Error error = 2; + reserved 2; } repeated Range ranges = 2 [(gogoproto.nullable) = false]; } diff --git a/pkg/sql/split_at.go b/pkg/sql/split_at.go index b7104794061c..4f6260e97578 100644 --- a/pkg/sql/split_at.go +++ b/pkg/sql/split_at.go @@ -465,23 +465,13 @@ var scatterNodeColumns = sqlbase.ResultColumns{ Name: "pretty", Typ: parser.TypeString, }, - { - Name: "error", - Typ: parser.TypeString, - }, } func (n *scatterNode) Values() parser.Datums { r := n.ranges[n.rangeIdx] - dErr := parser.DNull - if r.Error != nil { - dErr = parser.NewDString(r.Error.String()) - } - return parser.Datums{ parser.NewDBytes(parser.DBytes(r.Span.Key)), parser.NewDString(keys.PrettyPrint(r.Span.Key)), - dErr, } } diff --git a/pkg/sql/split_at_test.go b/pkg/sql/split_at_test.go index 9baecf7523bb..2c87afbdb583 100644 --- a/pkg/sql/split_at_test.go +++ b/pkg/sql/split_at_test.go @@ -25,11 +25,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" - "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/pkg/errors" ) func TestSplitAt(t *testing.T) { @@ -147,68 +145,54 @@ func TestScatter(t *testing.T) { t.Skip("#14955") const numHosts = 4 - tc := serverutils.StartTestCluster(t, numHosts, base.TestClusterArgs{ - // TODO(radu): this test should be reliable in automatic mode as well; - // remove this when that is the case (#15003). - ReplicationMode: base.ReplicationManual, - }) + tc := serverutils.StartTestCluster(t, numHosts, base.TestClusterArgs{}) defer tc.Stopper().Stop(context.TODO()) sqlutils.CreateTable( t, tc.ServerConn(0), "t", "k INT PRIMARY KEY, v INT", - 100, + 1000, sqlutils.ToRowFn(sqlutils.RowIdxFn, sqlutils.RowModuloFn(10)), ) r := sqlutils.MakeSQLRunner(t, tc.ServerConn(0)) - // Introduce 9 splits to get 10 ranges. - r.Exec("ALTER TABLE test.t SPLIT AT (SELECT i*10 FROM GENERATE_SERIES(1, 9) as g(i))") - - // Scatter until each host has at least one leaseholder. - // The probability that a random distribution includes at most 3 hosts out of - // 4 is less than: 5 * (3/4)^10 = ~28% - // The probability of this happening (say) 20 times in a row is less than one - // in 100 billion. - testutils.SucceedsSoon(t, func() error { - r.Exec("ALTER TABLE test.t SCATTER") - rows := r.Query("SHOW TESTING_RANGES FROM TABLE test.t") - // See showRangesColumns for the schema. - if cols, err := rows.Columns(); err != nil { + // Introduce 99 splits to get 100 ranges. + r.Exec("ALTER TABLE test.t SPLIT AT (SELECT i*10 FROM generate_series(1, 99) AS g(i))") + + // Ensure that scattering leaves each node with at least 20% of the leases. + r.Exec("ALTER TABLE test.t SCATTER") + rows := r.Query("SHOW TESTING_RANGES FROM TABLE test.t") + // See showRangesColumns for the schema. + if cols, err := rows.Columns(); err != nil { + t.Fatal(err) + } else if len(cols) != 4 { + t.Fatalf("expected 4 columns, got %#v", cols) + } + vals := []interface{}{ + new(interface{}), + new(interface{}), + new(interface{}), + new(int), + } + leaseHolders := map[int]int{1: 0, 2: 0, 3: 0, 4: 0} + numRows := 0 + for ; rows.Next(); numRows++ { + if err := rows.Scan(vals...); err != nil { t.Fatal(err) - } else if len(cols) != 4 { - t.Fatalf("expected 4 columns, got %#v", cols) - } - vals := []interface{}{ - new(interface{}), - new(interface{}), - new(interface{}), - new(int), } - var leaseHolders []int - seenHost := make([]bool, numHosts) - numRows := 0 - for ; rows.Next(); numRows++ { - if err := rows.Scan(vals...); err != nil { - t.Fatal(err) - } - leaseHolder := *vals[3].(*int) - if leaseHolder < 1 || leaseHolder > numHosts { - t.Fatalf("invalid lease holder value: %d", leaseHolder) - } - leaseHolders = append(leaseHolders, leaseHolder) - seenHost[leaseHolder-1] = true + leaseHolder := *vals[3].(*int) + if leaseHolder < 1 || leaseHolder > numHosts { + t.Fatalf("invalid lease holder value: %d", leaseHolder) } - t.Logf("LeaseHolders: %v", leaseHolders) - if numRows != 10 { - t.Fatalf("expected 10 ranges, got %d", numRows) - } - for i, v := range seenHost { - if !v { - return errors.Errorf("no leaseholders on host %d", i+1) - } + leaseHolders[leaseHolder]++ + } + if numRows != 100 { + t.Fatalf("expected 100 ranges, got %d", numRows) + } + for i, count := range leaseHolders { + if count < 20 { + t.Errorf("less than 20 leaseholders on host %d (only %d)", i, count) } - return nil - }) + } } diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index 04ffb9fc1c20..afdd6bdf8b23 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -374,8 +374,9 @@ func (a Allocator) RebalanceTarget( constraints config.Constraints, existing []roachpb.ReplicaDescriptor, rangeID roachpb.RangeID, + filter storeFilter, ) *roachpb.StoreDescriptor { - sl, _, _ := a.storePool.getStoreList(rangeID, storeFilterThrottled) + sl, _, _ := a.storePool.getStoreList(rangeID, filter) existingCandidates, candidates := rebalanceCandidates( ctx, @@ -432,6 +433,7 @@ func (a *Allocator) TransferLeaseTarget( stats *replicaStats, checkTransferLeaseSource bool, checkCandidateFullness bool, + alwaysAllowDecisionWithoutStats bool, ) roachpb.ReplicaDescriptor { sl, _, _ := a.storePool.getStoreList(rangeID, storeFilterNone) sl = sl.filter(constraints) @@ -467,16 +469,21 @@ func (a *Allocator) TransferLeaseTarget( // Try to pick a replica to transfer the lease to while also determining // whether we actually should be transferring the lease. The transfer // decision is only needed if we've been asked to check the source. - transferDec, repl := a.shouldTransferLeaseUsingStats(ctx, sl, source, existing, stats) + transferDec, repl := a.shouldTransferLeaseUsingStats( + ctx, sl, source, existing, stats, + ) if checkTransferLeaseSource { switch transferDec { case shouldNotTransfer: - return roachpb.ReplicaDescriptor{} - case shouldTransfer: + if !alwaysAllowDecisionWithoutStats { + return roachpb.ReplicaDescriptor{} + } + fallthrough case decideWithoutStats: if !a.shouldTransferLeaseWithoutStats(ctx, sl, source, existing) { return roachpb.ReplicaDescriptor{} } + case shouldTransfer: default: log.Fatalf(ctx, "unexpected transfer decision %d with replica %+v", transferDec, repl) } @@ -570,11 +577,12 @@ func (a Allocator) shouldTransferLeaseUsingStats( qpsStats, qpsStatsDur := stats.perLocalityDecayingQPS() - // If we haven't yet accumulated enough data, avoid transferring for now. Do - // not fall back to the algorithm that doesn't use stats, since it can easily - // start fighting with the stats-based algorithm. This provides some amount of - // safety from lease thrashing, since leases cannot transfer more frequently - // than this threshold (because replica stats get reset upon lease transfer). + // If we haven't yet accumulated enough data, avoid transferring for now, + // unless we've been explicitly asked otherwise. Do not fall back to the + // algorithm that doesn't use stats, since it can easily start fighting with + // the stats-based algorithm. This provides some amount of safety from lease + // thrashing, since leases cannot transfer more frequently than this threshold + // (because replica stats get reset upon lease transfer). if qpsStatsDur < MinLeaseTransferStatsDuration { return shouldNotTransfer, roachpb.ReplicaDescriptor{} } diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index a8e2701d7d0a..64fda5e88bef 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -633,6 +633,7 @@ func TestAllocatorRebalance(t *testing.T) { config.Constraints{}, []roachpb.ReplicaDescriptor{{StoreID: 3}}, firstRange, + storeFilterThrottled, ) if result == nil { i-- // loop until we find 10 candidates @@ -724,7 +725,7 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { for _, c := range testCases { t.Run("", func(t *testing.T) { result := a.RebalanceTarget( - ctx, config.Constraints{}, c.existing, firstRange) + ctx, config.Constraints{}, c.existing, firstRange, storeFilterThrottled) if c.expected > 0 { if result == nil { t.Fatalf("expected %d, but found nil", c.expected) @@ -903,6 +904,7 @@ func TestAllocatorRebalanceByCount(t *testing.T) { config.Constraints{}, []roachpb.ReplicaDescriptor{{StoreID: stores[0].StoreID}}, firstRange, + storeFilterThrottled, ) if result != nil && result.StoreID != 4 { t.Errorf("expected store 4; got %d", result.StoreID) @@ -976,7 +978,8 @@ func TestAllocatorTransferLeaseTarget(t *testing.T) { 0, nil, /* replicaStats */ c.check, - true, /* checkCandidateFullness */ + true, /* checkCandidateFullness */ + false, /* !alwaysAllowDecisionWithoutStats */ ) if c.expected != target.StoreID { t.Fatalf("expected %d, but found %d", c.expected, target.StoreID) @@ -1029,7 +1032,8 @@ func TestAllocatorTransferLeaseTargetMultiStore(t *testing.T) { 0, nil, /* replicaStats */ c.check, - true, /* checkCandidateFullness */ + true, /* checkCandidateFullness */ + false, /* !alwaysAllowDecisionWithoutStats */ ) if c.expected != target.StoreID { t.Fatalf("expected %d, but found %d", c.expected, target.StoreID) @@ -1257,7 +1261,8 @@ func TestAllocatorTransferLeaseTargetLoadBased(t *testing.T) { 0, c.stats, c.check, - true, /* checkCandidateFullness */ + true, /* checkCandidateFullness */ + false, /* !alwaysAllowDecisionWithoutStats */ ) if c.expected != target.StoreID { t.Errorf("expected %d, got %d", c.expected, target.StoreID) @@ -2284,6 +2289,7 @@ func TestAllocatorRebalanceAway(t *testing.T) { constraints, existingReplicas, firstRange, + storeFilterThrottled, ) if tc.expected == nil && actual != nil { @@ -2396,6 +2402,7 @@ func Example_rebalancing() { config.Constraints{}, []roachpb.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}, firstRange, + storeFilterThrottled, ) if target != nil { testStores[j].rebalance(&testStores[int(target.StoreID)], alloc.randGen.Int63n(1<<20)) diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 917bd543ec62..fae38378cc50 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -37,6 +37,7 @@ import ( "golang.org/x/net/context" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -3889,47 +3890,158 @@ func TestingRelocateRange( } // adminScatter moves replicas and leaseholders for a selection of ranges. -// Scatter is best-effort; ranges that cannot be moved will include an error -// detail in the response and won't fail the request. func (r *Replica) adminScatter( ctx context.Context, args roachpb.AdminScatterRequest, ) (roachpb.AdminScatterResponse, error) { - db := r.store.DB() - rangeDesc := *r.Desc() + var desc *roachpb.RangeDescriptor + var zone config.ZoneConfig + var err error + + refreshDescAndZone := func() error { + desc = r.Desc() - rng := rand.New(rand.NewSource(rand.Int63())) + sysCfg, ok := r.store.cfg.Gossip.GetSystemConfig() + if !ok { + return errors.New("system config not yet available") + } + if zone, err = sysCfg.GetZoneConfigForKey(desc.StartKey); err != nil { + return err + } - sl, _, _ := r.store.cfg.StorePool.getStoreList(roachpb.RangeID(0), storeFilterNone) - stores := make([]roachpb.ReplicationTarget, len(sl.stores)) - for i, sd := range sl.stores { - stores[i].StoreID = sd.StoreID - stores[i].NodeID = sd.Node.NodeID + return nil } - // Choose three random stores. - // TODO(radu): this is a toy implementation; we need to get a real - // recommendation based on the zone config. - num := 3 - if num > len(stores) { - num = len(stores) + // Sleep for a random duration to avoid dumping too many leases or replicas on + // one underfull store when many ranges are scattered simultaneously. It's + // unfortunate this sleep is server-side instead of client-side, but since + // scatter is the only command that needs it, it's not worth building jitter + // support into DistSender. + const maxJitter = 3 * time.Second + jitter := time.Duration(rand.Int63n(maxJitter.Nanoseconds())) * time.Nanosecond + select { + case <-time.After(jitter): + case <-ctx.Done(): + return roachpb.AdminScatterResponse{}, ctx.Err() } - for i := 0; i < num; i++ { - j := i + rng.Intn(len(stores)-i) - stores[i], stores[j] = stores[j], stores[i] + + if err := refreshDescAndZone(); err != nil { + return roachpb.AdminScatterResponse{}, err } - targets := stores[:num] - relocateErr := TestingRelocateRange(ctx, db, rangeDesc, targets) + // Step 1. Rebalance by adding replicas of this range to the stores the + // allocator recommends, if any. It's unlikely that the allocator would + // suggest more than zone.NumReplicas rebalance targets--that would indicate + // the allocator had previously given us suggestions that did not balance the + // cluster--but we cap the number of replicas we'll try to add at + // zone.NumReplicas just in case. + // + // TODO(benesch): This causes overreplication. Ideally, we'd wait for the + // replicate queue to downreplicate after each ADD_REPLICA command, but this + // practically guarantees that, for at least some ranges, we'll remove our own + // replica first, after which we can no longer issue ADD_REPLICA commands. + for i := int32(0); i < zone.NumReplicas; i++ { + if err = refreshDescAndZone(); err != nil { + break + } + + targetStore := r.store.allocator.RebalanceTarget( + ctx, + zone.Constraints, + desc.Replicas, + desc.RangeID, + storeFilterNone, + ) + if targetStore == nil { + if log.V(2) { + log.Infof(ctx, "scatter: no rebalance targets found on try %d, moving on", i) + } + break + } else if log.V(2) { + log.Infof(ctx, "scatter: found rebalance target %d: %v", i, targetStore) + } + replicationTarget := roachpb.ReplicationTarget{ + NodeID: targetStore.Node.NodeID, + StoreID: targetStore.StoreID, + } - res := roachpb.AdminScatterResponse{ - Ranges: []roachpb.AdminScatterResponse_Range{{ - Span: roachpb.Span{ - Key: rangeDesc.StartKey.AsRawKey(), - EndKey: rangeDesc.EndKey.AsRawKey(), - }, - Error: roachpb.NewError(relocateErr), - }}, + retryOpts := retry.Options{ + InitialBackoff: 50 * time.Millisecond, + MaxRetries: 5, + RandomizationFactor: .3, + } + for re := retry.StartWithCtx(ctx, retryOpts); re.Next(); { + if err = r.changeReplicas( + ctx, roachpb.ADD_REPLICA, replicationTarget, desc, SnapshotRequest_REBALANCE, + ); err == nil { + break + } else if log.V(2) { + log.Infof(ctx, "scatter: unable to replicate to %v: %s", replicationTarget, err) + } + } + if err != nil { + switch errors.Cause(err).(type) { + case *roachpb.ConditionFailedError: + default: + return roachpb.AdminScatterResponse{}, err + } + } + if ctx.Err() != nil { + return roachpb.AdminScatterResponse{}, ctx.Err() + } + } + + // Step 2. Transfer our lease away, if the allocator wants us to. + retryOpts := retry.Options{ + InitialBackoff: 50 * time.Millisecond, + MaxBackoff: time.Second, + MaxRetries: 5, + RandomizationFactor: .3, + } + for re := retry.StartWithCtx(ctx, retryOpts); re.Next(); { + lease, _ := r.getLease() + if !r.IsLeaseValid(lease, r.store.Clock().Now()) { + // We assume that, if we no longer have the lease, the replicate queue has + // already transferred it away to balance the cluster, so we move on. + break + } + + if err = refreshDescAndZone(); err != nil { + continue + } + + candidates := filterBehindReplicas(r.RaftStatus(), desc.Replicas) + target := r.store.allocator.TransferLeaseTarget( + ctx, + zone.Constraints, + candidates, + r.store.StoreID(), + desc.RangeID, + r.leaseholderStats, + true, /* checkTransferLeaseSource */ + true, /* checkCandidateFullness */ + true, /* alwaysAllowDecisionWithoutStats */ + ) + + if target == (roachpb.ReplicaDescriptor{}) { + if log.V(2) { + log.Infof(ctx, "scatter: no lease transfer targets found, moving on") + } + r.store.replicateQueue.MaybeAdd(r, r.store.Clock().Now()) + break + } else if log.V(2) { + log.Infof(ctx, "scatter: attempting to transfer lease to s%d", target.StoreID) + } + + if err = r.AdminTransferLease(ctx, target.StoreID); err != nil && log.V(2) { + log.Infof(ctx, "scatter: unable to transfer lease to s%d: %s", target.StoreID, err) + } + } + if err != nil { + return roachpb.AdminScatterResponse{}, err + } + if ctx.Err() != nil { + return roachpb.AdminScatterResponse{}, ctx.Err() } - return res, nil + return roachpb.AdminScatterResponse{}, nil } diff --git a/pkg/storage/replicate_queue.go b/pkg/storage/replicate_queue.go index 8631c7175fd0..b3547fc83100 100644 --- a/pkg/storage/replicate_queue.go +++ b/pkg/storage/replicate_queue.go @@ -196,6 +196,7 @@ func (rq *replicateQueue) shouldQueue( zone.Constraints, desc.Replicas, desc.RangeID, + storeFilterThrottled, ) if log.V(2) { if target != nil { @@ -429,6 +430,7 @@ func (rq *replicateQueue) processOneChange( zone.Constraints, desc.Replicas, desc.RangeID, + storeFilterThrottled, ) if rebalanceStore == nil { if log.V(1) { @@ -474,6 +476,7 @@ func (rq *replicateQueue) transferLease( repl.leaseholderStats, checkTransferLeaseSource, checkCandidateFullness, + false, /* !alwaysAllowDecisionWithoutStats */ ); target != (roachpb.ReplicaDescriptor{}) { rq.metrics.TransferLeaseCount.Inc(1) if log.V(1) {