From c6b040af49d5a2af1436986ec464950d8f565eea Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Wed, 8 Jul 2020 23:12:17 -0400 Subject: [PATCH] roachpb,gc,mvcc: add UseClearRange option to GCRequest_GCKey This commit adds an optimization to massively reduce the overhead of garbage collecting large numbers of versions. When running garbage collection, we currently iterate through the store to send (Key, Timestamp) pairs in a GCRequest to the store to be evaluated and applied. This can be rather slow when deleting large numbers of keys, particularly due to the need to paginate. The primary motivation for pagination is to ensure that we do not create raft commands with deletions that are too large. In practice, we find that for the tiny values in a sequence, we find that we can GC around 1800 versions/s with #51184 and around 900 without it (though note that in that PR the more versions exist, the worse the throughput will be). This remains abysmally slow. I imagine that using larger batches could be one approach to increase the throughput, but, as it stands, 256 KiB is not a tiny raft command. This instead turns to the ClearRange operation which can delete all of versions of a key with the replication overhead of just two copies. This approach is somewhat controversial because, as @petermattis puts it: ``` We need to be careful about this. Historically, adding many range tombstones was very bad for performance. I think we resolved most (all?) of those issues, but I'm still nervous about embracing using range tombstones below the level of a Range. ``` Nevertheless, the results are enticing. Rather than pinning a core at full utilization for minutes just to clear the versions written to a sequence over the course of a bit more than an hour, we can clear that in ~2 seconds. Release note (performance improvement): Improved performance of garbage collection in the face of large numbers of versions. --- c-deps/libroach/protos/roachpb/api.pb.cc | 35 +- c-deps/libroach/protos/roachpb/api.pb.h | 21 + docs/generated/settings/settings.html | 2 +- pkg/cli/debug.go | 7 +- pkg/clusterversion/cockroach_versions.go | 7 + pkg/kv/kvserver/gc/gc.go | 61 +- pkg/kv/kvserver/gc/gc_old_test.go | 1 + pkg/kv/kvserver/gc/gc_random_test.go | 14 +- pkg/kv/kvserver/gc_queue.go | 65 +- pkg/kv/kvserver/gc_queue_test.go | 13 +- pkg/roachpb/api.pb.go | 1236 +++++++++++----------- pkg/roachpb/api.proto | 10 + pkg/storage/mvcc.go | 15 +- 13 files changed, 828 insertions(+), 659 deletions(-) diff --git a/c-deps/libroach/protos/roachpb/api.pb.cc b/c-deps/libroach/protos/roachpb/api.pb.cc index 659c23146d87..62a45d313088 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.cc +++ b/c-deps/libroach/protos/roachpb/api.pb.cc @@ -14163,6 +14163,7 @@ void GCRequest_GCKey::clear_timestamp() { #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int GCRequest_GCKey::kKeyFieldNumber; const int GCRequest_GCKey::kTimestampFieldNumber; +const int GCRequest_GCKey::kUseClearRangeFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 GCRequest_GCKey::GCRequest_GCKey() @@ -14185,12 +14186,15 @@ GCRequest_GCKey::GCRequest_GCKey(const GCRequest_GCKey& from) } else { timestamp_ = NULL; } + use_clear_range_ = from.use_clear_range_; // @@protoc_insertion_point(copy_constructor:cockroach.roachpb.GCRequest.GCKey) } void GCRequest_GCKey::SharedCtor() { key_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - timestamp_ = NULL; + ::memset(×tamp_, 0, static_cast( + reinterpret_cast(&use_clear_range_) - + reinterpret_cast(×tamp_)) + sizeof(use_clear_range_)); } GCRequest_GCKey::~GCRequest_GCKey() { @@ -14223,6 +14227,7 @@ void GCRequest_GCKey::Clear() { delete timestamp_; } timestamp_ = NULL; + use_clear_range_ = false; _internal_metadata_.Clear(); } @@ -14264,6 +14269,20 @@ bool GCRequest_GCKey::MergePartialFromCodedStream( break; } + // bool use_clear_range = 3; + case 3: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(24u /* 24 & 0xFF */)) { + + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( + input, &use_clear_range_))); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -14300,6 +14319,11 @@ void GCRequest_GCKey::SerializeWithCachedSizes( 2, this->_internal_timestamp(), output); } + // bool use_clear_range = 3; + if (this->use_clear_range() != 0) { + ::google::protobuf::internal::WireFormatLite::WriteBool(3, this->use_clear_range(), output); + } + output->WriteRaw((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).data(), static_cast((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size())); // @@protoc_insertion_point(serialize_end:cockroach.roachpb.GCRequest.GCKey) @@ -14323,6 +14347,11 @@ size_t GCRequest_GCKey::ByteSizeLong() const { *timestamp_); } + // bool use_clear_range = 3; + if (this->use_clear_range() != 0) { + total_size += 1 + 1; + } + int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); SetCachedSize(cached_size); return total_size; @@ -14347,6 +14376,9 @@ void GCRequest_GCKey::MergeFrom(const GCRequest_GCKey& from) { if (from.has_timestamp()) { mutable_timestamp()->::cockroach::util::hlc::Timestamp::MergeFrom(from.timestamp()); } + if (from.use_clear_range() != 0) { + set_use_clear_range(from.use_clear_range()); + } } void GCRequest_GCKey::CopyFrom(const GCRequest_GCKey& from) { @@ -14369,6 +14401,7 @@ void GCRequest_GCKey::InternalSwap(GCRequest_GCKey* other) { key_.Swap(&other->key_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), GetArenaNoVirtual()); swap(timestamp_, other->timestamp_); + swap(use_clear_range_, other->use_clear_range_); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/c-deps/libroach/protos/roachpb/api.pb.h b/c-deps/libroach/protos/roachpb/api.pb.h index ca0a2208616a..50f3de98b6e7 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.h +++ b/c-deps/libroach/protos/roachpb/api.pb.h @@ -6361,12 +6361,19 @@ class GCRequest_GCKey : public ::google::protobuf::MessageLite /* @@protoc_inser ::cockroach::util::hlc::Timestamp* mutable_timestamp(); void set_allocated_timestamp(::cockroach::util::hlc::Timestamp* timestamp); + // bool use_clear_range = 3; + void clear_use_clear_range(); + static const int kUseClearRangeFieldNumber = 3; + bool use_clear_range() const; + void set_use_clear_range(bool value); + // @@protoc_insertion_point(class_scope:cockroach.roachpb.GCRequest.GCKey) private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; ::google::protobuf::internal::ArenaStringPtr key_; ::cockroach::util::hlc::Timestamp* timestamp_; + bool use_clear_range_; mutable ::google::protobuf::internal::CachedSize _cached_size_; friend struct ::protobuf_roachpb_2fapi_2eproto::TableStruct; }; @@ -21524,6 +21531,20 @@ inline void GCRequest_GCKey::set_allocated_timestamp(::cockroach::util::hlc::Tim // @@protoc_insertion_point(field_set_allocated:cockroach.roachpb.GCRequest.GCKey.timestamp) } +// bool use_clear_range = 3; +inline void GCRequest_GCKey::clear_use_clear_range() { + use_clear_range_ = false; +} +inline bool GCRequest_GCKey::use_clear_range() const { + // @@protoc_insertion_point(field_get:cockroach.roachpb.GCRequest.GCKey.use_clear_range) + return use_clear_range_; +} +inline void GCRequest_GCKey::set_use_clear_range(bool value) { + + use_clear_range_ = value; + // @@protoc_insertion_point(field_set:cockroach.roachpb.GCRequest.GCKey.use_clear_range) +} + // ------------------------------------------------------------------- // GCRequest diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 8b82f844cdfa..ec45f16d73b7 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -72,6 +72,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen in the /debug page trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versioncustom validation20.1-11set the active cluster version in the format '.' +versioncustom validation20.1-12set the active cluster version in the format '.' diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index bad72d144d68..cce4aa94bdf1 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -573,14 +573,11 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { policy := zonepb.GCPolicy{TTLSeconds: int32(gcTTLInSeconds)} now := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} thresh := gc.CalculateThreshold(now, policy) - info, err := gc.Run( - context.Background(), - &desc, snap, - now, thresh, policy, + info, err := gc.Run(context.Background(), &desc, snap, now, thresh, policy, gc.NoopGCer{}, func(_ context.Context, _ []roachpb.Intent) error { return nil }, func(_ context.Context, _ *roachpb.Transaction, _ []roachpb.LockUpdate) error { return nil }, - ) + false /* canUseClearRange */) if err != nil { return err } diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 151b4015e67e..a5214b9f1df9 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -71,6 +71,7 @@ const ( VersionNoOriginFKIndexes VersionClientRangeInfosOnBatchResponse VersionNodeMembershipStatus + VersionClearRangeForGC // Add new versions here (step one of two). ) @@ -539,6 +540,12 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionNodeMembershipStatus, Version: roachpb.Version{Major: 20, Minor: 1, Unstable: 11}, }, + { + // VersionClearRangeForGC gates the usage of UseClearRange in the + // GCRequest_GCKey proto. See comment on proto definition for more details. + Key: VersionClearRangeForGC, + Version: roachpb.Version{Major: 20, Minor: 1, Unstable: 12}, + }, // Add new versions here (step two of two). diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 71aa5a343254..86651f64e1d6 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -163,6 +163,7 @@ func Run( gcer GCer, cleanupIntentsFn CleanupIntentsFunc, cleanupTxnIntentsAsyncFn CleanupTxnIntentsAsyncFunc, + canUseClearRange bool, ) (Info, error) { txnExp := now.Add(-kvserverbase.TxnCleanupThreshold.Nanoseconds(), 0) @@ -182,7 +183,8 @@ func Run( // Maps from txn ID to txn and intent key slice. txnMap := map[uuid.UUID]*roachpb.Transaction{} intentKeyMap := map[uuid.UUID][]roachpb.Key{} - err := processReplicatedKeyRange(ctx, desc, snap, now, newThreshold, gcer, txnMap, intentKeyMap, &info) + err := processReplicatedKeyRange(ctx, desc, snap, now, newThreshold, gcer, + txnMap, intentKeyMap, &info, canUseClearRange) if err != nil { return Info{}, err } @@ -237,6 +239,7 @@ func processReplicatedKeyRange( txnMap map[uuid.UUID]*roachpb.Transaction, intentKeyMap map[uuid.UUID][]roachpb.Key, info *Info, + canUseClearRange bool, ) error { var alloc bufalloc.ByteAllocator // Compute intent expiration (intent age at which we attempt to resolve). @@ -284,11 +287,13 @@ func processReplicatedKeyRange( // version for a key has been reached, if haveGarbageForThisKey, we'll add the // current key to the batch with the gcTimestampForThisKey. var ( - batchGCKeys []roachpb.GCRequest_GCKey - batchGCKeysBytes int64 - haveGarbageForThisKey bool - gcTimestampForThisKey hlc.Timestamp - sentBatchForThisKey bool + batchGCKeys []roachpb.GCRequest_GCKey + batchGCKeysBytes int64 + haveGarbageForThisKey bool + gcTimestampForThisKey hlc.Timestamp + keyBytesForThisKey int64 + sentBatchForThisKey bool + useClearRangeForThisKey bool ) it := makeGCIterator(desc, snap) defer it.close() @@ -310,7 +315,12 @@ func processReplicatedKeyRange( isNewest := s.curIsNewest() if isGarbage(threshold, s.cur, s.next, isNewest) { keyBytes := int64(s.cur.Key.EncodedSize()) - batchGCKeysBytes += keyBytes + // If we have decided that we're going to use clear range for this key, + // we've already accounted for the overhead of those key bytes. + if !useClearRangeForThisKey { + batchGCKeysBytes += keyBytes + keyBytesForThisKey += keyBytes + } haveGarbageForThisKey = true gcTimestampForThisKey = s.cur.Key.Timestamp info.AffectedVersionsKeyBytes += keyBytes @@ -319,23 +329,48 @@ func processReplicatedKeyRange( if affected := isNewest && (sentBatchForThisKey || haveGarbageForThisKey); affected { info.NumKeysAffected++ } - shouldSendBatch := batchGCKeysBytes >= KeyVersionChunkBytes - if shouldSendBatch || isNewest && haveGarbageForThisKey { + + atBatchSizeLimit := batchGCKeysBytes >= KeyVersionChunkBytes + if atBatchSizeLimit && !useClearRangeForThisKey { + // We choose to use clear range for a key if we'd fill up an entire batch + // with just that key. + // + // TODO(ajwerner): Perhaps we should ensure that there are actually a + // large number of versions utilizing all of these bytes and not a small + // number of versions of a very large key. What's the right minimum number + // of keys? + useClearRangeForThisKey = canUseClearRange && len(batchGCKeys) == 0 + if useClearRangeForThisKey { + // Adjust the accounting for the size of this batch given that now + // we're going to deal with this key using clear range. + batchGCKeysBytes -= keyBytesForThisKey + batchGCKeysBytes += 2 * int64(s.cur.Key.EncodedSize()) + keyBytesForThisKey = 0 + } + } + + if addKeyToBatch := (atBatchSizeLimit && !useClearRangeForThisKey) || + (isNewest && haveGarbageForThisKey); addKeyToBatch { alloc, s.cur.Key.Key = alloc.Copy(s.cur.Key.Key, 0) batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{ - Key: s.cur.Key.Key, - Timestamp: gcTimestampForThisKey, + Key: s.cur.Key.Key, + Timestamp: gcTimestampForThisKey, + UseClearRange: useClearRangeForThisKey, }) haveGarbageForThisKey = false gcTimestampForThisKey = hlc.Timestamp{} + keyBytesForThisKey = 0 + useClearRangeForThisKey = false // Mark that we sent a batch for this key so we know that we had garbage // even if it turns out that there's no more garbage for this key. // We want to count a key as affected once even if we paginate the // deletion of its versions. - sentBatchForThisKey = shouldSendBatch && !isNewest + sentBatchForThisKey = atBatchSizeLimit && !isNewest } - if shouldSendBatch { + + if shouldSendBatch := (atBatchSizeLimit && !useClearRangeForThisKey) || + (isNewest && useClearRangeForThisKey); shouldSendBatch { if err := gcer.GC(ctx, batchGCKeys); err != nil { if errors.Is(err, ctx.Err()) { return err diff --git a/pkg/kv/kvserver/gc/gc_old_test.go b/pkg/kv/kvserver/gc/gc_old_test.go index 558036d27e80..be5b3f8af70c 100644 --- a/pkg/kv/kvserver/gc/gc_old_test.go +++ b/pkg/kv/kvserver/gc/gc_old_test.go @@ -48,6 +48,7 @@ func runGCOld( gcer GCer, cleanupIntentsFn CleanupIntentsFunc, cleanupTxnIntentsAsyncFn CleanupTxnIntentsAsyncFunc, + _ bool, ) (Info, error) { iter := rditer.NewReplicaDataIterator(desc, snap, diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index 3b30e242c2f7..9f2e15c51fca 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -96,19 +96,17 @@ func TestRunNewVsOld(t *testing.T) { oldGCer := makeFakeGCer() policy := zonepb.GCPolicy{TTLSeconds: tc.ttl} newThreshold := CalculateThreshold(tc.now, policy) + const useClearRange = false gcInfoOld, err := runGCOld(ctx, tc.ds.desc(), snap, tc.now, newThreshold, policy, &oldGCer, oldGCer.resolveIntents, - oldGCer.resolveIntentsAsync) + oldGCer.resolveIntentsAsync, + useClearRange) require.NoError(t, err) newGCer := makeFakeGCer() - gcInfoNew, err := Run(ctx, tc.ds.desc(), snap, tc.now, - newThreshold, policy, - &newGCer, - newGCer.resolveIntents, - newGCer.resolveIntentsAsync) + gcInfoNew, err := Run(ctx, tc.ds.desc(), snap, tc.now, newThreshold, policy, &newGCer, newGCer.resolveIntents, newGCer.resolveIntentsAsync, false) require.NoError(t, err) oldGCer.normalize() @@ -131,6 +129,7 @@ func BenchmarkRun(b *testing.B) { } snap := eng.NewSnapshot() policy := zonepb.GCPolicy{TTLSeconds: spec.ttl} + const useClearRange = false return runGCFunc(ctx, spec.ds.desc(), snap, spec.now, CalculateThreshold(spec.now, policy), policy, @@ -140,7 +139,8 @@ func BenchmarkRun(b *testing.B) { }, func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { return nil - }) + }, + useClearRange) } makeTest := func(old bool, spec randomRunGCTestSpec) func(b *testing.B) { return func(b *testing.B) { diff --git a/pkg/kv/kvserver/gc_queue.go b/pkg/kv/kvserver/gc_queue.go index 069f04450933..6faee3684e72 100644 --- a/pkg/kv/kvserver/gc_queue.go +++ b/pkg/kv/kvserver/gc_queue.go @@ -18,12 +18,14 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -48,6 +50,13 @@ const ( probablyLargeAbortSpanSysBytesThreshold = 16 * (1 << 20) // 16mb ) +// useClearRangeForGC is an experimental setting to utilize clear range +// operations in the face of a large number of versions of a key. +var useClearRangeForGC = settings.RegisterBoolSetting( + "kv.gc.use_clear_range.enabled", + "enables the use of clear range operations to delete large numbers of versions of a key", + false) + func probablyLargeAbortSpan(ms enginepb.MVCCStats) bool { // If there is "a lot" of data in Sys{Bytes,Count}, then we are likely // experiencing a large abort span. The abort span is not supposed to @@ -453,33 +462,37 @@ func (gcq *gcQueue) process( snap := repl.store.Engine().NewSnapshot() defer snap.Close() + cleanupIntentsFunc := func(ctx context.Context, intents []roachpb.Intent) error { + intentCount, err := repl.store.intentResolver. + CleanupIntents(ctx, intents, gcTimestamp, roachpb.PUSH_ABORT) + if err == nil { + gcq.store.metrics.GCResolveSuccess.Inc(int64(intentCount)) + } + return err + } + cleanupTxnIntentsAsyncFunc := func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { + err := repl.store.intentResolver. + CleanupTxnIntentsOnGCAsync(ctx, repl.RangeID, txn, intents, gcTimestamp, + func(pushed, succeeded bool) { + if pushed { + gcq.store.metrics.GCPushTxn.Inc(1) + } + if succeeded { + gcq.store.metrics.GCResolveSuccess.Inc(int64(len(intents))) + } + }) + if errors.Is(err, stop.ErrThrottled) { + log.Eventf(ctx, "processing txn %s: %s; skipping for future GC", txn.ID.Short(), err) + return nil + } + return err + } + canUseClearRange := useClearRangeForGC.Get(&repl.store.ClusterSettings().SV) && + gcq.store.ClusterSettings().Version.IsActive(ctx, clusterversion.VersionClearRangeForGC) + info, err := gc.Run(ctx, desc, snap, gcTimestamp, newThreshold, *zone.GC, - &replicaGCer{repl: repl}, - func(ctx context.Context, intents []roachpb.Intent) error { - intentCount, err := repl.store.intentResolver. - CleanupIntents(ctx, intents, gcTimestamp, roachpb.PUSH_ABORT) - if err == nil { - gcq.store.metrics.GCResolveSuccess.Inc(int64(intentCount)) - } - return err - }, - func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { - err := repl.store.intentResolver. - CleanupTxnIntentsOnGCAsync(ctx, repl.RangeID, txn, intents, gcTimestamp, - func(pushed, succeeded bool) { - if pushed { - gcq.store.metrics.GCPushTxn.Inc(1) - } - if succeeded { - gcq.store.metrics.GCResolveSuccess.Inc(int64(len(intents))) - } - }) - if errors.Is(err, stop.ErrThrottled) { - log.Eventf(ctx, "processing txn %s: %s; skipping for future GC", txn.ID.Short(), err) - return nil - } - return err - }) + &replicaGCer{repl: repl}, cleanupIntentsFunc, cleanupTxnIntentsAsyncFunc, + canUseClearRange) if err != nil { return false, err } diff --git a/pkg/kv/kvserver/gc_queue_test.go b/pkg/kv/kvserver/gc_queue_test.go index baf9d6fe0769..cc5520df38b1 100644 --- a/pkg/kv/kvserver/gc_queue_test.go +++ b/pkg/kv/kvserver/gc_queue_test.go @@ -555,14 +555,11 @@ func TestGCQueueProcess(t *testing.T) { now := tc.Clock().Now() newThreshold := gc.CalculateThreshold(now, *zone.GC) - return gc.Run(ctx, desc, snap, now, newThreshold, *zone.GC, - gc.NoopGCer{}, - func(ctx context.Context, intents []roachpb.Intent) error { - return nil - }, - func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { - return nil - }) + return gc.Run(ctx, desc, snap, now, newThreshold, *zone.GC, gc.NoopGCer{}, func(ctx context.Context, intents []roachpb.Intent) error { + return nil + }, func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { + return nil + }, false) }() if err != nil { t.Fatal(err) diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 8dbd75bdc246..76408082f4f2 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -72,7 +72,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{0} + return fileDescriptor_api_71616fea36bb2a95, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -100,7 +100,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{1} + return fileDescriptor_api_71616fea36bb2a95, []int{1} } type ChecksumMode int32 @@ -147,7 +147,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{2} + return fileDescriptor_api_71616fea36bb2a95, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -178,7 +178,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{3} + return fileDescriptor_api_71616fea36bb2a95, []int{3} } type ExternalStorageProvider int32 @@ -219,7 +219,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{4} + return fileDescriptor_api_71616fea36bb2a95, []int{4} } type MVCCFilter int32 @@ -242,7 +242,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{5} + return fileDescriptor_api_71616fea36bb2a95, []int{5} } type ResponseHeader_ResumeReason int32 @@ -268,7 +268,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{1, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -310,7 +310,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{25, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -331,7 +331,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{0} + return fileDescriptor_api_71616fea36bb2a95, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -402,7 +402,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{1} + return fileDescriptor_api_71616fea36bb2a95, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -436,7 +436,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{2} + return fileDescriptor_api_71616fea36bb2a95, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -479,7 +479,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{3} + return fileDescriptor_api_71616fea36bb2a95, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +522,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{4} + return fileDescriptor_api_71616fea36bb2a95, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -556,7 +556,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{5} + return fileDescriptor_api_71616fea36bb2a95, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -638,7 +638,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{6} + return fileDescriptor_api_71616fea36bb2a95, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -673,7 +673,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{7} + return fileDescriptor_api_71616fea36bb2a95, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -719,7 +719,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{8} + return fileDescriptor_api_71616fea36bb2a95, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -753,7 +753,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{9} + return fileDescriptor_api_71616fea36bb2a95, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -793,7 +793,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{10} + return fileDescriptor_api_71616fea36bb2a95, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -830,7 +830,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{11} + return fileDescriptor_api_71616fea36bb2a95, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -864,7 +864,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{12} + return fileDescriptor_api_71616fea36bb2a95, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -898,7 +898,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{13} + return fileDescriptor_api_71616fea36bb2a95, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -950,7 +950,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{14} + return fileDescriptor_api_71616fea36bb2a95, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -987,7 +987,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{15} + return fileDescriptor_api_71616fea36bb2a95, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1034,7 +1034,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{16} + return fileDescriptor_api_71616fea36bb2a95, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1068,7 +1068,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{17} + return fileDescriptor_api_71616fea36bb2a95, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1109,7 +1109,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{18} + return fileDescriptor_api_71616fea36bb2a95, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1143,7 +1143,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{19} + return fileDescriptor_api_71616fea36bb2a95, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1194,7 +1194,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{20} + return fileDescriptor_api_71616fea36bb2a95, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1246,7 +1246,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{21} + return fileDescriptor_api_71616fea36bb2a95, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1297,7 +1297,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{22} + return fileDescriptor_api_71616fea36bb2a95, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1349,7 +1349,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{23} + return fileDescriptor_api_71616fea36bb2a95, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1402,7 +1402,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{24} + return fileDescriptor_api_71616fea36bb2a95, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1439,7 +1439,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{25} + return fileDescriptor_api_71616fea36bb2a95, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1483,7 +1483,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{25, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1531,7 +1531,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{26} + return fileDescriptor_api_71616fea36bb2a95, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1567,7 +1567,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{27} + return fileDescriptor_api_71616fea36bb2a95, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1664,7 +1664,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{28} + return fileDescriptor_api_71616fea36bb2a95, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1710,7 +1710,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{29} + return fileDescriptor_api_71616fea36bb2a95, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1771,7 +1771,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{30} + return fileDescriptor_api_71616fea36bb2a95, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1806,7 +1806,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{31} + return fileDescriptor_api_71616fea36bb2a95, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1845,7 +1845,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{32} + return fileDescriptor_api_71616fea36bb2a95, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1880,7 +1880,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{33} + return fileDescriptor_api_71616fea36bb2a95, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1923,7 +1923,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{34} + return fileDescriptor_api_71616fea36bb2a95, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1958,7 +1958,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{35} + return fileDescriptor_api_71616fea36bb2a95, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1996,7 +1996,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{36} + return fileDescriptor_api_71616fea36bb2a95, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2029,7 +2029,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{37} + return fileDescriptor_api_71616fea36bb2a95, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2064,7 +2064,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{38} + return fileDescriptor_api_71616fea36bb2a95, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2122,7 +2122,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{39} + return fileDescriptor_api_71616fea36bb2a95, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2157,7 +2157,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{40} + return fileDescriptor_api_71616fea36bb2a95, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2194,7 +2194,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{41} + return fileDescriptor_api_71616fea36bb2a95, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2227,7 +2227,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{42} + return fileDescriptor_api_71616fea36bb2a95, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2266,7 +2266,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{43} + return fileDescriptor_api_71616fea36bb2a95, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2303,7 +2303,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{44} + return fileDescriptor_api_71616fea36bb2a95, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2341,7 +2341,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{45} + return fileDescriptor_api_71616fea36bb2a95, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2369,13 +2369,22 @@ var xxx_messageInfo_GCRequest proto.InternalMessageInfo type GCRequest_GCKey struct { Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + // UseClearRange indicates that the evaluation of deleting versions of this + // key should utilize a ClearRange operation instead of issuing point + // deletes. We delegate this decision to the caller which has the visibility + // to determine whether there are sufficient versions to justify this + // approach. This field was introduced in version 20.2 and is gated on that + // version. Nodes at earlier versions would happily ignore this value but + // it remains gated to avoid the negative consequences of sending a single + // key with an extremely large number of versions. + UseClearRange bool `protobuf:"varint,3,opt,name=use_clear_range,json=useClearRange,proto3" json:"use_clear_range,omitempty"` } func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{45, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2409,7 +2418,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{46} + return fileDescriptor_api_71616fea36bb2a95, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2478,7 +2487,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{47} + return fileDescriptor_api_71616fea36bb2a95, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2521,7 +2530,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{48} + return fileDescriptor_api_71616fea36bb2a95, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2568,7 +2577,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{49} + return fileDescriptor_api_71616fea36bb2a95, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2604,7 +2613,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{50} + return fileDescriptor_api_71616fea36bb2a95, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2648,7 +2657,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{51} + return fileDescriptor_api_71616fea36bb2a95, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2687,7 +2696,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{52} + return fileDescriptor_api_71616fea36bb2a95, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2747,7 +2756,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{53} + return fileDescriptor_api_71616fea36bb2a95, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2783,7 +2792,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{54} + return fileDescriptor_api_71616fea36bb2a95, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2829,7 +2838,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{55} + return fileDescriptor_api_71616fea36bb2a95, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2864,7 +2873,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{56} + return fileDescriptor_api_71616fea36bb2a95, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2914,7 +2923,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{57} + return fileDescriptor_api_71616fea36bb2a95, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2949,7 +2958,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{58} + return fileDescriptor_api_71616fea36bb2a95, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2986,7 +2995,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{59} + return fileDescriptor_api_71616fea36bb2a95, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3020,7 +3029,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{60} + return fileDescriptor_api_71616fea36bb2a95, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3065,7 +3074,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{61} + return fileDescriptor_api_71616fea36bb2a95, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3099,7 +3108,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{62} + return fileDescriptor_api_71616fea36bb2a95, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3143,7 +3152,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{63} + return fileDescriptor_api_71616fea36bb2a95, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3192,7 +3201,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{64} + return fileDescriptor_api_71616fea36bb2a95, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3229,7 +3238,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{65} + return fileDescriptor_api_71616fea36bb2a95, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3266,7 +3275,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{66} + return fileDescriptor_api_71616fea36bb2a95, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3301,7 +3310,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{67} + return fileDescriptor_api_71616fea36bb2a95, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3356,7 +3365,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{68} + return fileDescriptor_api_71616fea36bb2a95, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3393,7 +3402,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{69} + return fileDescriptor_api_71616fea36bb2a95, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3433,7 +3442,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70} + return fileDescriptor_api_71616fea36bb2a95, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3467,7 +3476,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3500,7 +3509,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70, 1} + return fileDescriptor_api_71616fea36bb2a95, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3540,7 +3549,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70, 2} + return fileDescriptor_api_71616fea36bb2a95, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3579,7 +3588,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70, 3} + return fileDescriptor_api_71616fea36bb2a95, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3615,7 +3624,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70, 4} + return fileDescriptor_api_71616fea36bb2a95, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3654,7 +3663,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70, 5} + return fileDescriptor_api_71616fea36bb2a95, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3694,7 +3703,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{70, 6} + return fileDescriptor_api_71616fea36bb2a95, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3734,7 +3743,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{71} + return fileDescriptor_api_71616fea36bb2a95, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3768,7 +3777,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{72} + return fileDescriptor_api_71616fea36bb2a95, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3802,7 +3811,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{73} + return fileDescriptor_api_71616fea36bb2a95, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3872,7 +3881,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{74} + return fileDescriptor_api_71616fea36bb2a95, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3924,7 +3933,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{75} + return fileDescriptor_api_71616fea36bb2a95, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3960,7 +3969,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{76} + return fileDescriptor_api_71616fea36bb2a95, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4000,7 +4009,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{76, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4051,7 +4060,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{77} + return fileDescriptor_api_71616fea36bb2a95, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4086,7 +4095,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{77, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4122,7 +4131,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{77, 1} + return fileDescriptor_api_71616fea36bb2a95, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4157,7 +4166,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{78} + return fileDescriptor_api_71616fea36bb2a95, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4195,7 +4204,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{79} + return fileDescriptor_api_71616fea36bb2a95, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4230,7 +4239,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{80} + return fileDescriptor_api_71616fea36bb2a95, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4263,7 +4272,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{80, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4308,7 +4317,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{81} + return fileDescriptor_api_71616fea36bb2a95, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4346,7 +4355,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{82} + return fileDescriptor_api_71616fea36bb2a95, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4399,7 +4408,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{83} + return fileDescriptor_api_71616fea36bb2a95, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4433,7 +4442,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{84} + return fileDescriptor_api_71616fea36bb2a95, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4477,7 +4486,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{85} + return fileDescriptor_api_71616fea36bb2a95, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4511,7 +4520,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{86} + return fileDescriptor_api_71616fea36bb2a95, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4550,7 +4559,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{87} + return fileDescriptor_api_71616fea36bb2a95, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4584,7 +4593,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{88} + return fileDescriptor_api_71616fea36bb2a95, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4633,7 +4642,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{89} + return fileDescriptor_api_71616fea36bb2a95, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4682,7 +4691,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{90} + return fileDescriptor_api_71616fea36bb2a95, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4717,7 +4726,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{91} + return fileDescriptor_api_71616fea36bb2a95, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4756,7 +4765,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{92} + return fileDescriptor_api_71616fea36bb2a95, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4839,7 +4848,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{93} + return fileDescriptor_api_71616fea36bb2a95, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6289,7 +6298,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{94} + return fileDescriptor_api_71616fea36bb2a95, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7804,7 +7813,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{95} + return fileDescriptor_api_71616fea36bb2a95, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7840,7 +7849,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{96} + return fileDescriptor_api_71616fea36bb2a95, []int{96} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7877,7 +7886,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{97} + return fileDescriptor_api_71616fea36bb2a95, []int{97} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7943,7 +7952,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{97, 0} + return fileDescriptor_api_71616fea36bb2a95, []int{97, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7982,7 +7991,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{98} + return fileDescriptor_api_71616fea36bb2a95, []int{98} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8023,7 +8032,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{99} + return fileDescriptor_api_71616fea36bb2a95, []int{99} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8064,7 +8073,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{100} + return fileDescriptor_api_71616fea36bb2a95, []int{100} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8101,7 +8110,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{101} + return fileDescriptor_api_71616fea36bb2a95, []int{101} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8138,7 +8147,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a06e0f3f58e22ede, []int{102} + return fileDescriptor_api_71616fea36bb2a95, []int{102} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -9066,6 +9075,9 @@ func (this *GCRequest_GCKey) Equal(that interface{}) bool { if !this.Timestamp.Equal(&that1.Timestamp) { return false } + if this.UseClearRange != that1.UseClearRange { + return false + } return true } func (this *PushTxnRequest) Equal(that interface{}) bool { @@ -12162,6 +12174,16 @@ func (m *GCRequest_GCKey) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n63 + if m.UseClearRange { + dAtA[i] = 0x18 + i++ + if m.UseClearRange { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -17296,6 +17318,9 @@ func (m *GCRequest_GCKey) Size() (n int) { } l = m.Timestamp.Size() n += 1 + l + sovApi(uint64(l)) + if m.UseClearRange { + n += 2 + } return n } @@ -25281,6 +25306,26 @@ func (m *GCRequest_GCKey) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UseClearRange", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.UseClearRange = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -37755,474 +37800,475 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a06e0f3f58e22ede) } - -var fileDescriptor_api_a06e0f3f58e22ede = []byte{ - // 7450 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x68, 0x24, 0xd9, - 0x75, 0xbf, 0xaa, 0xbb, 0x25, 0x75, 0x9f, 0x6e, 0xb5, 0x4a, 0x57, 0x9a, 0x99, 0x1e, 0xcd, 0xac, - 0xa4, 0xe9, 0x9d, 0xaf, 0x1d, 0xef, 0x4a, 0x3b, 0x33, 0xbb, 0xff, 0x5d, 0xef, 0xac, 0xd7, 0x96, - 0x5a, 0x3d, 0xd3, 0x2d, 0x8d, 0x34, 0x9a, 0xea, 0xd6, 0xac, 0x77, 0xed, 0xa5, 0x5c, 0xaa, 0xba, - 0x6a, 0x95, 0xd5, 0x5d, 0xd5, 0x53, 0x55, 0xad, 0x8f, 0x81, 0x3f, 0xf8, 0xff, 0x01, 0x0e, 0x26, - 0x2c, 0x79, 0x08, 0x21, 0xc4, 0x09, 0x5e, 0x70, 0xc0, 0x01, 0xb3, 0x21, 0xc9, 0x5b, 0x82, 0x43, - 0xf2, 0x90, 0xc0, 0xc6, 0x38, 0x60, 0x02, 0x89, 0x4d, 0x20, 0xc2, 0x1e, 0x83, 0x09, 0x7e, 0x08, - 0xe4, 0x25, 0x81, 0x85, 0x84, 0x70, 0x3f, 0xea, 0xa3, 0xbb, 0xab, 0x3f, 0x34, 0xae, 0x4d, 0x16, - 0xfc, 0xd2, 0x74, 0x9d, 0xba, 0xe7, 0xd4, 0xbd, 0xe7, 0x9e, 0x7b, 0xee, 0xf9, 0xdd, 0x3a, 0xf7, - 0x16, 0x4c, 0x59, 0xa6, 0xa2, 0xee, 0x35, 0x77, 0x96, 0x94, 0xa6, 0xbe, 0xd8, 0xb4, 0x4c, 0xc7, - 0x44, 0x53, 0xaa, 0xa9, 0xee, 0x53, 0xf2, 0x22, 0xbf, 0x39, 0x7b, 0x63, 0xff, 0x60, 0x69, 0xff, - 0xc0, 0xc6, 0xd6, 0x01, 0xb6, 0x96, 0x54, 0xd3, 0x50, 0x5b, 0x96, 0x85, 0x0d, 0xf5, 0x78, 0xa9, - 0x6e, 0xaa, 0xfb, 0xf4, 0x47, 0x37, 0x6a, 0x8c, 0x7d, 0x16, 0xb9, 0x12, 0x35, 0xc5, 0x51, 0x38, - 0x6d, 0xc6, 0xa5, 0x61, 0xcb, 0x32, 0x2d, 0x9b, 0x53, 0xcf, 0xba, 0xd4, 0x06, 0x76, 0x94, 0x40, - 0xe9, 0x0b, 0xb6, 0x63, 0x5a, 0x4a, 0x0d, 0x2f, 0x61, 0xa3, 0xa6, 0x1b, 0x98, 0x14, 0x38, 0x50, - 0x55, 0x7e, 0xf3, 0x62, 0xe8, 0xcd, 0xdb, 0xfc, 0x6e, 0xae, 0xe5, 0xe8, 0xf5, 0xa5, 0xbd, 0xba, - 0xba, 0xe4, 0xe8, 0x0d, 0x6c, 0x3b, 0x4a, 0xa3, 0xc9, 0xef, 0x2c, 0xd0, 0x3b, 0x8e, 0xa5, 0xa8, - 0xba, 0x51, 0x5b, 0xb2, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, 0xa9, 0x18, 0x6e, 0x25, 0x6b, - 0x66, 0xcd, 0xa4, 0x7f, 0x97, 0xc8, 0x3f, 0x46, 0xcd, 0x7f, 0x4f, 0x80, 0x09, 0x09, 0x3f, 0x6e, - 0x61, 0xdb, 0x29, 0x61, 0x45, 0xc3, 0x16, 0x3a, 0x0f, 0xf1, 0x7d, 0x7c, 0x9c, 0x8b, 0x2f, 0x08, - 0xd7, 0x33, 0x2b, 0xe3, 0x1f, 0x9f, 0xcc, 0xc7, 0xd7, 0xf1, 0xb1, 0x44, 0x68, 0x68, 0x01, 0xc6, - 0xb1, 0xa1, 0xc9, 0xe4, 0x76, 0xa2, 0xfd, 0xf6, 0x18, 0x36, 0xb4, 0x75, 0x7c, 0x8c, 0xbe, 0x0c, - 0x49, 0x9b, 0x48, 0x33, 0x54, 0x9c, 0x1b, 0x5d, 0x10, 0xae, 0x8f, 0xae, 0x7c, 0xe1, 0xe3, 0x93, - 0xf9, 0x37, 0x6b, 0xba, 0xb3, 0xd7, 0xda, 0x59, 0x54, 0xcd, 0xc6, 0x92, 0xa7, 0x7d, 0x6d, 0xc7, - 0xff, 0xbf, 0xd4, 0xdc, 0xaf, 0x2d, 0x75, 0xb6, 0x7c, 0xb1, 0x7a, 0x64, 0x54, 0xf0, 0x63, 0xc9, - 0x93, 0xf8, 0x46, 0xe2, 0x9f, 0x3f, 0x98, 0x17, 0xd6, 0x12, 0x49, 0x41, 0x8c, 0xad, 0x25, 0x92, - 0x31, 0x31, 0x9e, 0xff, 0x76, 0x1c, 0xb2, 0x12, 0xb6, 0x9b, 0xa6, 0x61, 0x63, 0x5e, 0xff, 0x97, - 0x21, 0xee, 0x1c, 0x19, 0xb4, 0xfe, 0xe9, 0x5b, 0x73, 0x8b, 0x5d, 0xbd, 0xbd, 0x58, 0xb5, 0x14, - 0xc3, 0x56, 0x54, 0x47, 0x37, 0x0d, 0x89, 0x14, 0x45, 0xaf, 0x43, 0xda, 0xc2, 0x76, 0xab, 0x81, - 0xa9, 0xba, 0x68, 0xd3, 0xd2, 0xb7, 0xce, 0x85, 0x70, 0x56, 0x9a, 0x8a, 0x21, 0x01, 0x2b, 0x4b, - 0xfe, 0xa3, 0xf3, 0x90, 0x34, 0x5a, 0x0d, 0xa2, 0x10, 0x9b, 0x36, 0x37, 0x2e, 0x8d, 0x1b, 0xad, - 0xc6, 0x3a, 0x3e, 0xb6, 0xd1, 0x17, 0xe1, 0xac, 0x86, 0x9b, 0x16, 0x56, 0x15, 0x07, 0x6b, 0xb2, - 0xa5, 0x18, 0x35, 0x2c, 0xeb, 0xc6, 0xae, 0x69, 0xe7, 0xc6, 0x16, 0xe2, 0xd7, 0xd3, 0xb7, 0x2e, - 0x86, 0xc8, 0x97, 0x48, 0xa9, 0xb2, 0xb1, 0x6b, 0xae, 0x24, 0x3e, 0x3a, 0x99, 0x1f, 0x91, 0x66, - 0x7c, 0x09, 0xde, 0x2d, 0x1b, 0x55, 0x60, 0x82, 0x57, 0xd7, 0xc2, 0x8a, 0x6d, 0x1a, 0xb9, 0xf1, - 0x05, 0xe1, 0x7a, 0xf6, 0xd6, 0x62, 0x98, 0xc0, 0x36, 0xd5, 0x90, 0xcb, 0x56, 0x03, 0x4b, 0x94, - 0x4b, 0xca, 0x58, 0x81, 0x2b, 0x74, 0x01, 0x52, 0xa4, 0x25, 0x3b, 0xc7, 0x0e, 0xb6, 0x73, 0x49, - 0xda, 0x14, 0xd2, 0xb4, 0x15, 0x72, 0x9d, 0x7f, 0x0b, 0x32, 0x41, 0x56, 0x84, 0x20, 0x2b, 0x15, - 0x2b, 0xdb, 0x1b, 0x45, 0x79, 0x7b, 0x73, 0x7d, 0xf3, 0xc1, 0xdb, 0x9b, 0xe2, 0x08, 0x9a, 0x01, - 0x91, 0xd3, 0xd6, 0x8b, 0xef, 0xc8, 0xf7, 0xcb, 0x1b, 0xe5, 0xaa, 0x28, 0xcc, 0x26, 0x7e, 0xed, - 0xdb, 0x73, 0x23, 0xf9, 0x47, 0x00, 0xf7, 0xb0, 0xc3, 0xcd, 0x0c, 0xad, 0xc0, 0xd8, 0x1e, 0xad, - 0x4f, 0x4e, 0xa0, 0x9a, 0x5e, 0x08, 0xad, 0x78, 0xc0, 0x24, 0x57, 0x92, 0x44, 0x1b, 0x3f, 0x3c, - 0x99, 0x17, 0x24, 0xce, 0xc9, 0x2c, 0x21, 0xff, 0x17, 0x02, 0xa4, 0xa9, 0x60, 0xd6, 0x4a, 0x54, - 0xe8, 0x90, 0x7c, 0x69, 0xa0, 0x4a, 0xba, 0x45, 0xa3, 0x45, 0x18, 0x3d, 0x50, 0xea, 0x2d, 0x9c, - 0x8b, 0x51, 0x19, 0xb9, 0x10, 0x19, 0x8f, 0xc8, 0x7d, 0x89, 0x15, 0x43, 0x77, 0x20, 0xa3, 0x1b, - 0x0e, 0x36, 0x1c, 0x99, 0xb1, 0xc5, 0x07, 0xb0, 0xa5, 0x59, 0x69, 0x7a, 0x91, 0xff, 0x33, 0x01, - 0x60, 0xab, 0x15, 0xa5, 0x6a, 0xd0, 0x2b, 0x43, 0xd6, 0x9f, 0xdb, 0x18, 0x6f, 0xc5, 0x59, 0x18, - 0xd3, 0x8d, 0xba, 0x6e, 0xb0, 0xfa, 0x27, 0x25, 0x7e, 0x85, 0x66, 0x60, 0x74, 0xa7, 0xae, 0x1b, - 0x1a, 0x1d, 0x15, 0x49, 0x89, 0x5d, 0x70, 0xf5, 0x4b, 0x90, 0xa6, 0x75, 0x8f, 0x50, 0xfb, 0xf9, - 0x1f, 0xc4, 0xe0, 0x4c, 0xc1, 0x34, 0x34, 0x9d, 0x0c, 0x4f, 0xa5, 0xfe, 0xa9, 0xd0, 0xcd, 0x1a, - 0x04, 0x06, 0xa2, 0x8c, 0x8f, 0x9a, 0x43, 0xf6, 0x34, 0xf2, 0xb9, 0x8a, 0x47, 0x4d, 0x4a, 0x0b, - 0xd7, 0x27, 0x7a, 0x05, 0xce, 0x29, 0xf5, 0xba, 0x79, 0x28, 0xeb, 0xbb, 0xb2, 0x66, 0x62, 0x5b, - 0x36, 0x4c, 0x47, 0xc6, 0x47, 0xba, 0xed, 0x50, 0xb7, 0x92, 0x94, 0xa6, 0xe9, 0xed, 0xf2, 0xee, - 0xaa, 0x89, 0xed, 0x4d, 0xd3, 0x29, 0x92, 0x5b, 0x64, 0xcc, 0x92, 0xca, 0xb0, 0x31, 0x3b, 0x46, - 0x1c, 0xb2, 0x94, 0xc4, 0x47, 0x4d, 0x3a, 0x66, 0x79, 0x17, 0xbd, 0x07, 0x67, 0x3b, 0xb5, 0x19, - 0x65, 0x6f, 0xfd, 0x9d, 0x00, 0xd9, 0xb2, 0xa1, 0x3b, 0x9f, 0x8a, 0x6e, 0xf2, 0x54, 0x1b, 0x0f, - 0xaa, 0xf6, 0x06, 0x88, 0xbb, 0x8a, 0x5e, 0x7f, 0x60, 0x54, 0xcd, 0xc6, 0x8e, 0xed, 0x98, 0x06, - 0xb6, 0xb9, 0xee, 0xbb, 0xe8, 0x5c, 0x67, 0x8f, 0x60, 0xd2, 0x6b, 0x53, 0x94, 0xca, 0x7a, 0x02, - 0x62, 0xd9, 0x50, 0x2d, 0xdc, 0xc0, 0x46, 0xa4, 0xda, 0xba, 0x08, 0x29, 0xdd, 0x95, 0x4b, 0x35, - 0x16, 0x97, 0x7c, 0x02, 0x6f, 0x53, 0x0b, 0xa6, 0x02, 0xcf, 0x8e, 0xd2, 0x5d, 0x92, 0x89, 0x03, - 0x1f, 0xca, 0x7e, 0x7f, 0x91, 0x89, 0x03, 0x1f, 0x32, 0xf7, 0xf6, 0x0e, 0x4c, 0xac, 0xe2, 0x3a, - 0x76, 0x70, 0xf4, 0xbe, 0x7f, 0x1b, 0xb2, 0xae, 0xe8, 0x28, 0x3b, 0xe9, 0xf7, 0x04, 0x40, 0x5c, - 0x2e, 0x99, 0x71, 0xa3, 0xec, 0xa7, 0x79, 0x12, 0x66, 0x38, 0x2d, 0xcb, 0x60, 0xf1, 0x02, 0xb3, - 0x52, 0x60, 0x24, 0x1a, 0x32, 0xf8, 0x3e, 0x38, 0x11, 0xf4, 0xc1, 0x5e, 0xd8, 0x43, 0x02, 0x9e, - 0x43, 0x98, 0x6e, 0xab, 0x5e, 0xb4, 0x5d, 0x99, 0xa0, 0x35, 0x8b, 0x2d, 0xc4, 0x83, 0xb1, 0x1d, - 0x25, 0xe6, 0xdf, 0x83, 0xa9, 0x42, 0x1d, 0x2b, 0x56, 0xd4, 0x6a, 0xe1, 0xdd, 0xf9, 0x0e, 0xa0, - 0xa0, 0xf8, 0x28, 0xbb, 0xf4, 0xf7, 0x05, 0x40, 0x12, 0x3e, 0xc0, 0x96, 0x13, 0x79, 0x97, 0xae, - 0x42, 0xda, 0x51, 0xac, 0x1a, 0x76, 0x64, 0x12, 0x8f, 0x73, 0x77, 0xf5, 0x5c, 0x40, 0x10, 0x89, - 0xca, 0x17, 0xf7, 0xea, 0xea, 0x62, 0xd5, 0x8d, 0xd7, 0xb9, 0xcf, 0x02, 0xc6, 0x47, 0xc8, 0x5c, - 0x03, 0xef, 0xc2, 0x74, 0x5b, 0x2d, 0xa3, 0x54, 0xc1, 0xbf, 0x09, 0x90, 0xae, 0xa8, 0x8a, 0x11, - 0x65, 0xdb, 0xdf, 0x82, 0xb4, 0xad, 0x2a, 0x86, 0xbc, 0x6b, 0x5a, 0x0d, 0xc5, 0xa1, 0x26, 0x9b, - 0x6d, 0x6b, 0xbb, 0x17, 0x35, 0xab, 0x8a, 0x71, 0x97, 0x16, 0x92, 0xc0, 0xf6, 0xfe, 0xa3, 0x87, - 0x90, 0xde, 0xc7, 0xc7, 0x32, 0x47, 0x57, 0x74, 0x9e, 0xcb, 0xde, 0x7a, 0x39, 0xc0, 0xbf, 0x7f, - 0xb0, 0xe8, 0x82, 0xb2, 0xc5, 0x00, 0x28, 0x5b, 0x24, 0x1c, 0x8b, 0x15, 0xc7, 0xc2, 0x46, 0xcd, - 0xd9, 0x93, 0x60, 0x1f, 0x1f, 0xdf, 0x67, 0x32, 0x82, 0x03, 0x65, 0x2d, 0x91, 0x8c, 0x8b, 0x89, - 0xfc, 0xbf, 0x0b, 0x90, 0x61, 0x0d, 0x8f, 0x72, 0xa0, 0xbc, 0x0a, 0x09, 0xcb, 0x3c, 0x64, 0x03, - 0x25, 0x7d, 0xeb, 0x42, 0x88, 0x88, 0x75, 0x7c, 0x1c, 0x9c, 0xa1, 0x68, 0x71, 0xb4, 0x02, 0x3c, - 0xf6, 0x93, 0x29, 0x77, 0x7c, 0x58, 0x6e, 0x60, 0x5c, 0x12, 0x91, 0x71, 0x0d, 0x26, 0x77, 0x14, - 0x47, 0xdd, 0x93, 0x2d, 0x5e, 0x49, 0x32, 0x9b, 0xc5, 0xaf, 0x67, 0xa4, 0x2c, 0x25, 0xbb, 0x55, - 0xb7, 0xf3, 0xff, 0xe1, 0x5a, 0xbd, 0x8d, 0x7f, 0x25, 0x7b, 0xfe, 0x3f, 0x05, 0x3e, 0x9e, 0xdc, - 0xf6, 0xff, 0xaa, 0x19, 0xc0, 0xb7, 0x62, 0x70, 0xae, 0xb0, 0x87, 0xd5, 0xfd, 0x82, 0x69, 0xd8, - 0xba, 0xed, 0x10, 0x0d, 0x46, 0x69, 0x05, 0x17, 0x20, 0x75, 0xa8, 0x3b, 0x7b, 0xb2, 0xa6, 0xef, - 0xee, 0x52, 0xcf, 0x97, 0x94, 0x92, 0x84, 0xb0, 0xaa, 0xef, 0xee, 0xa2, 0xdb, 0x90, 0x68, 0x98, - 0x1a, 0x0b, 0x91, 0xb3, 0xb7, 0xe6, 0x43, 0xc4, 0xd3, 0xaa, 0xd9, 0xad, 0xc6, 0x86, 0xa9, 0x61, - 0x89, 0x16, 0x46, 0x73, 0x00, 0x2a, 0xa1, 0x36, 0x4d, 0xdd, 0x70, 0xf8, 0x1c, 0x18, 0xa0, 0xa0, - 0x12, 0xa4, 0x1c, 0x6c, 0x35, 0x74, 0x43, 0x71, 0x70, 0x6e, 0x94, 0x2a, 0xef, 0x72, 0x68, 0xc5, - 0x9b, 0x75, 0x5d, 0x55, 0x56, 0xb1, 0xad, 0x5a, 0x7a, 0xd3, 0x31, 0x2d, 0xae, 0x45, 0x9f, 0x99, - 0x7b, 0xdc, 0xf7, 0x13, 0x90, 0xeb, 0xd6, 0x50, 0x94, 0x76, 0xb2, 0x05, 0x63, 0x04, 0x65, 0xd7, - 0x1d, 0x6e, 0x29, 0xb7, 0x7a, 0x29, 0x22, 0xa4, 0x06, 0x14, 0xad, 0xd7, 0x1d, 0x5e, 0x79, 0x2e, - 0x67, 0xf6, 0x7b, 0x02, 0x8c, 0xb1, 0x1b, 0xe8, 0x26, 0x24, 0xf9, 0xb2, 0x82, 0x46, 0xeb, 0x18, - 0x5f, 0x39, 0xfb, 0xf4, 0x64, 0x7e, 0x9c, 0xad, 0x14, 0xac, 0x7e, 0xec, 0xff, 0x95, 0xc6, 0x69, - 0xb9, 0xb2, 0x46, 0xfa, 0xcc, 0x76, 0x14, 0xcb, 0xa1, 0x4b, 0x38, 0x31, 0x86, 0x18, 0x28, 0x61, - 0x1d, 0x1f, 0xa3, 0x35, 0x18, 0xb3, 0x1d, 0xc5, 0x69, 0xd9, 0xbc, 0xd7, 0x4e, 0x55, 0xd9, 0x0a, - 0xe5, 0x94, 0xb8, 0x04, 0x12, 0xca, 0x68, 0xd8, 0x51, 0xf4, 0x3a, 0xed, 0xc6, 0x94, 0xc4, 0xaf, - 0xf2, 0xdf, 0x14, 0x60, 0x8c, 0x15, 0x45, 0xe7, 0x60, 0x5a, 0x5a, 0xde, 0xbc, 0x57, 0x94, 0xcb, - 0x9b, 0xab, 0xc5, 0x6a, 0x51, 0xda, 0x28, 0x6f, 0x2e, 0x57, 0x8b, 0xe2, 0x08, 0x3a, 0x0b, 0xc8, - 0xbd, 0x51, 0x78, 0xb0, 0x59, 0x29, 0x57, 0xaa, 0xc5, 0xcd, 0xaa, 0x28, 0xd0, 0x15, 0x06, 0x4a, - 0x0f, 0x50, 0x63, 0xe8, 0x32, 0x2c, 0x74, 0x52, 0xe5, 0x4a, 0x75, 0xb9, 0x5a, 0x91, 0x8b, 0x95, - 0x6a, 0x79, 0x63, 0xb9, 0x5a, 0x5c, 0x15, 0xe3, 0x7d, 0x4a, 0x91, 0x87, 0x48, 0x52, 0xb1, 0x50, - 0x15, 0x13, 0xf9, 0x27, 0x70, 0x46, 0xc2, 0xaa, 0xd9, 0x68, 0xb6, 0x1c, 0x4c, 0x6a, 0x69, 0x47, - 0x39, 0x5e, 0xce, 0xc1, 0xb8, 0x66, 0x1d, 0xcb, 0x56, 0xcb, 0xe0, 0xa3, 0x65, 0x4c, 0xb3, 0x8e, - 0xa5, 0x96, 0xc1, 0x8d, 0xf1, 0x8f, 0x04, 0x38, 0xdb, 0xf9, 0xf0, 0x28, 0x4d, 0xf1, 0x21, 0xa4, - 0x15, 0x4d, 0xc3, 0x9a, 0xac, 0xe1, 0xba, 0xa3, 0xf0, 0x50, 0xe5, 0x46, 0x40, 0x12, 0x5f, 0x7e, - 0x5b, 0xf4, 0x96, 0xdf, 0x36, 0x1e, 0x15, 0x0a, 0xb4, 0x22, 0xab, 0x84, 0xc3, 0x75, 0x45, 0x54, - 0x08, 0xa5, 0xe4, 0xff, 0x24, 0x01, 0x13, 0x45, 0x43, 0xab, 0x1e, 0x45, 0x3a, 0xbb, 0x9c, 0x85, - 0x31, 0xd5, 0x6c, 0x34, 0x74, 0xc7, 0x55, 0x13, 0xbb, 0x42, 0x9f, 0x85, 0xa4, 0x86, 0x15, 0xcd, - 0x5b, 0xa3, 0x18, 0x14, 0x68, 0x49, 0x5e, 0x71, 0xf4, 0x15, 0x38, 0x47, 0x3c, 0xa8, 0x65, 0x28, - 0x75, 0x99, 0x49, 0x93, 0x1d, 0x4b, 0xaf, 0xd5, 0xb0, 0xc5, 0x17, 0xfb, 0xae, 0x87, 0xd4, 0xb3, - 0xcc, 0x39, 0x0a, 0x94, 0xa1, 0xca, 0xca, 0x4b, 0x67, 0xf4, 0x30, 0x32, 0x7a, 0x13, 0x80, 0x4c, - 0x4e, 0x74, 0x01, 0xd1, 0xe6, 0xbe, 0xa9, 0xd7, 0x0a, 0xa2, 0xeb, 0x8e, 0x08, 0x03, 0xb9, 0xb6, - 0xd1, 0x12, 0x41, 0x06, 0x8f, 0x5b, 0xba, 0x85, 0xe5, 0x9b, 0x4d, 0x95, 0x42, 0xf9, 0xe4, 0x4a, - 0xf6, 0xe9, 0xc9, 0x3c, 0x48, 0x8c, 0x7c, 0x73, 0xab, 0x40, 0x90, 0x02, 0xfb, 0xdf, 0x54, 0xd1, - 0x0a, 0xcc, 0x91, 0x09, 0x98, 0xb7, 0x45, 0x71, 0xe4, 0x3d, 0xbd, 0xb6, 0x87, 0x2d, 0xd9, 0x5b, - 0x15, 0xa6, 0x4b, 0x78, 0x49, 0x69, 0x56, 0x55, 0x0c, 0x56, 0xd1, 0x65, 0xa7, 0x44, 0x8b, 0x78, - 0xea, 0x21, 0x7a, 0x6e, 0x9a, 0xba, 0x6d, 0x1a, 0xb9, 0x14, 0xd3, 0x33, 0xbb, 0x42, 0x0f, 0x41, - 0xd4, 0x0d, 0x79, 0xb7, 0xae, 0xd7, 0xf6, 0x1c, 0xf9, 0xd0, 0xd2, 0x1d, 0x6c, 0xe7, 0xa6, 0x68, - 0x83, 0xc2, 0xec, 0xae, 0xc2, 0xd7, 0x66, 0xb5, 0xb7, 0x49, 0x49, 0xde, 0xb4, 0xac, 0x6e, 0xdc, - 0xa5, 0xfc, 0x94, 0x68, 0x7b, 0xb3, 0xf3, 0xb8, 0x98, 0xcc, 0xff, 0x93, 0x00, 0x59, 0xd7, 0x68, - 0xa2, 0xb4, 0xef, 0xeb, 0x20, 0x9a, 0x06, 0x96, 0x9b, 0x7b, 0x8a, 0x8d, 0xb9, 0x62, 0xf8, 0x14, - 0x92, 0x35, 0x0d, 0xbc, 0x45, 0xc8, 0x4c, 0x13, 0x68, 0x0b, 0xa6, 0x6c, 0x47, 0xa9, 0xe9, 0x46, - 0x2d, 0xa0, 0xaf, 0xd1, 0xe1, 0x43, 0x77, 0x91, 0x73, 0x7b, 0xf4, 0xb6, 0xb8, 0xe3, 0x47, 0x02, - 0x4c, 0x2d, 0x6b, 0x0d, 0xdd, 0xa8, 0x34, 0xeb, 0x7a, 0xa4, 0x38, 0xff, 0x32, 0xa4, 0x6c, 0x22, - 0xd3, 0x77, 0xde, 0x3e, 0x46, 0x4b, 0xd2, 0x3b, 0xc4, 0x8b, 0xdf, 0x87, 0x49, 0x7c, 0xd4, 0xd4, - 0x2d, 0xc5, 0xd1, 0x4d, 0x83, 0xc1, 0x92, 0xc4, 0xf0, 0x6d, 0xcb, 0xfa, 0xbc, 0x3e, 0x34, 0xe1, - 0x2d, 0x7b, 0x07, 0x50, 0xb0, 0x61, 0x51, 0xe2, 0x13, 0x19, 0xa6, 0xa9, 0xe8, 0x6d, 0xc3, 0x8e, - 0x58, 0x6b, 0xdc, 0xbb, 0x7e, 0x09, 0x66, 0xda, 0x1f, 0x10, 0x65, 0xed, 0xdf, 0xe3, 0x3d, 0xbe, - 0x81, 0xad, 0x4f, 0x08, 0x1a, 0x07, 0xc5, 0x47, 0x59, 0xf3, 0x6f, 0x08, 0x70, 0x9e, 0xca, 0xa6, - 0xef, 0x44, 0x76, 0xb1, 0x75, 0x1f, 0x2b, 0x76, 0xa4, 0x08, 0xf9, 0x79, 0x18, 0x63, 0x48, 0x97, - 0x5a, 0xec, 0xe8, 0x4a, 0x9a, 0xc4, 0x25, 0x15, 0xc7, 0xb4, 0x48, 0x5c, 0xc2, 0x6f, 0xf1, 0x76, - 0x2a, 0x30, 0x1b, 0x56, 0x97, 0x88, 0x97, 0x02, 0xa6, 0x78, 0x78, 0x48, 0x4c, 0xbc, 0xb0, 0x47, - 0xe2, 0x22, 0x54, 0x84, 0xb4, 0x4a, 0xff, 0xc9, 0xce, 0x71, 0x13, 0x53, 0xf9, 0xd9, 0x7e, 0x91, - 0x25, 0x63, 0xab, 0x1e, 0x37, 0x31, 0x09, 0x4f, 0xdd, 0xff, 0x44, 0x5d, 0x81, 0xa6, 0xf6, 0x8d, - 0x4d, 0xe9, 0xf8, 0xa2, 0x65, 0xdd, 0xf0, 0xae, 0x4d, 0x13, 0x7f, 0x1a, 0xe7, 0xaa, 0x60, 0x4f, - 0xe2, 0x4c, 0x91, 0x46, 0x23, 0xef, 0xb6, 0xbd, 0x9e, 0x0a, 0x36, 0x3f, 0x76, 0x8a, 0xe6, 0x07, - 0xd6, 0xc5, 0x7d, 0x2a, 0x7a, 0x07, 0x02, 0x2b, 0xdf, 0x32, 0x6b, 0x99, 0x8b, 0x76, 0x4e, 0xa3, - 0x94, 0x29, 0x5f, 0x0a, 0xa3, 0xdb, 0xa8, 0x00, 0x49, 0x7c, 0xd4, 0x94, 0x35, 0x6c, 0xab, 0xdc, - 0xad, 0xe5, 0x7b, 0xbd, 0x47, 0xeb, 0x8a, 0xff, 0xc7, 0xf1, 0x51, 0x93, 0x10, 0xd1, 0x36, 0x99, - 0xe1, 0xdc, 0x70, 0x80, 0x56, 0xdb, 0x1e, 0x0c, 0x27, 0x7c, 0x7b, 0xe1, 0xe2, 0x26, 0xbd, 0x48, - 0x80, 0x89, 0xe0, 0x7d, 0xf7, 0x81, 0x00, 0x17, 0x42, 0xfb, 0x2e, 0xca, 0xc9, 0xee, 0x4d, 0x48, - 0x50, 0x15, 0xc4, 0x4e, 0xa9, 0x02, 0xca, 0x95, 0xff, 0xae, 0x3b, 0xea, 0x25, 0x5c, 0x37, 0x89, - 0x7a, 0x3f, 0x81, 0x75, 0xb1, 0x71, 0xb7, 0xdb, 0x63, 0xa7, 0xee, 0x76, 0x97, 0xb5, 0xc3, 0x2d, - 0x74, 0x54, 0x36, 0x4a, 0xb7, 0xf0, 0x5b, 0x02, 0x4c, 0x97, 0xb0, 0x62, 0x39, 0x3b, 0x58, 0x71, - 0x22, 0x0e, 0x67, 0x5f, 0x85, 0xb8, 0x61, 0x1e, 0x9e, 0x66, 0x69, 0x90, 0x94, 0xf7, 0xa7, 0xad, - 0xf6, 0x7a, 0x45, 0xd9, 0xea, 0xbf, 0x89, 0x41, 0xea, 0x5e, 0x21, 0xca, 0xb6, 0xbe, 0xc9, 0x17, - 0x90, 0xd9, 0x50, 0x0f, 0x33, 0x4b, 0xef, 0x79, 0x8b, 0xf7, 0x0a, 0xeb, 0xf8, 0xd8, 0x35, 0x4b, - 0xc2, 0x85, 0x96, 0x21, 0xe5, 0xec, 0x59, 0xd8, 0xde, 0x33, 0xeb, 0xda, 0x69, 0x62, 0x16, 0x9f, - 0x6b, 0x76, 0x1f, 0x46, 0xa9, 0x5c, 0x37, 0x89, 0x41, 0x08, 0x49, 0x62, 0x20, 0x8f, 0xf1, 0xc2, - 0xbe, 0xd8, 0x69, 0x1e, 0xe3, 0x12, 0x58, 0xe7, 0x78, 0xb1, 0xd1, 0xa8, 0x38, 0x96, 0x7f, 0x08, - 0x40, 0x9a, 0x16, 0x65, 0xf7, 0xfc, 0x7a, 0x1c, 0xb2, 0x5b, 0x2d, 0x7b, 0x2f, 0x62, 0x7b, 0x2c, - 0x00, 0x34, 0x5b, 0x36, 0x05, 0x0b, 0x47, 0x06, 0x6f, 0xff, 0x80, 0x2c, 0x09, 0x57, 0x01, 0x8c, - 0xaf, 0x7a, 0x64, 0xa0, 0x12, 0x17, 0x82, 0x65, 0x3f, 0xd5, 0xe2, 0xf9, 0x7e, 0x58, 0xb2, 0x7a, - 0x64, 0x6c, 0x60, 0x0f, 0x44, 0x32, 0x49, 0x98, 0x48, 0x7a, 0x13, 0xc6, 0xc9, 0x85, 0xec, 0x98, - 0xa7, 0xe9, 0xf2, 0x31, 0xc2, 0x53, 0x35, 0xd1, 0x1d, 0x48, 0x31, 0x6e, 0x32, 0x71, 0x8d, 0xd1, - 0x89, 0x2b, 0xac, 0x2d, 0x5c, 0x8d, 0x74, 0xca, 0x4a, 0x52, 0x56, 0x32, 0x4d, 0xcd, 0xc0, 0xe8, - 0xae, 0x69, 0xa9, 0x98, 0xe6, 0x4f, 0x24, 0x25, 0x76, 0x11, 0xec, 0xd5, 0xb5, 0x44, 0x32, 0x29, - 0xa6, 0xd6, 0x12, 0xc9, 0x94, 0x08, 0xf9, 0x6f, 0x0a, 0x30, 0xe9, 0x75, 0x47, 0x94, 0xbe, 0xbc, - 0xd0, 0xa6, 0xcb, 0xd3, 0x77, 0x08, 0x51, 0x63, 0xfe, 0x6f, 0x69, 0x60, 0xa3, 0x9a, 0x07, 0xb4, - 0x7f, 0xa2, 0xb4, 0x97, 0x3b, 0x2c, 0x9d, 0x26, 0x76, 0xda, 0x3e, 0xa6, 0x99, 0x35, 0x37, 0x61, - 0x46, 0x6f, 0x10, 0x2f, 0xaf, 0x3b, 0xf5, 0x63, 0x8e, 0xca, 0x1c, 0xec, 0xbe, 0xa1, 0x9d, 0xf6, - 0xef, 0x15, 0xdc, 0x5b, 0xdc, 0xf1, 0xb1, 0x77, 0x36, 0x7e, 0x7b, 0xa2, 0x54, 0x78, 0x19, 0x26, - 0x2c, 0x26, 0x9a, 0x44, 0x27, 0xa7, 0xd4, 0x79, 0xc6, 0x63, 0x25, 0x6a, 0xff, 0x4e, 0x0c, 0x26, - 0x1f, 0xb6, 0xb0, 0x75, 0xfc, 0x69, 0x52, 0xfa, 0x55, 0x98, 0x3c, 0x54, 0x74, 0x47, 0xde, 0x35, - 0x2d, 0xb9, 0xd5, 0xd4, 0x14, 0xc7, 0xcd, 0xe9, 0x98, 0x20, 0xe4, 0xbb, 0xa6, 0xb5, 0x4d, 0x89, - 0x08, 0x03, 0xda, 0x37, 0xcc, 0x43, 0x43, 0x26, 0x64, 0x8a, 0x86, 0x8f, 0x0c, 0xbe, 0x98, 0xbc, - 0xf2, 0xda, 0x3f, 0x9e, 0xcc, 0xdf, 0x1e, 0x2a, 0x6b, 0x8b, 0xe6, 0x9d, 0xb5, 0x5a, 0xba, 0xb6, - 0xb8, 0xbd, 0x5d, 0x5e, 0x95, 0x44, 0x2a, 0xf2, 0x6d, 0x26, 0xb1, 0x7a, 0x64, 0xb8, 0xb3, 0xf8, - 0xc7, 0x02, 0x88, 0xbe, 0xa6, 0xa2, 0xec, 0xce, 0x22, 0xa4, 0x1f, 0xb7, 0xb0, 0xa5, 0x3f, 0x43, - 0x67, 0x02, 0x67, 0x24, 0x8e, 0xe8, 0x5d, 0xc8, 0xb4, 0xe9, 0x21, 0xfe, 0xcb, 0xe9, 0x21, 0x7d, - 0xe8, 0xab, 0x20, 0xff, 0xd7, 0x02, 0x20, 0xda, 0xf8, 0x32, 0x5b, 0xc7, 0xff, 0xb4, 0x58, 0xca, - 0x75, 0x10, 0x69, 0xc6, 0xa2, 0xac, 0xef, 0xca, 0x0d, 0xdd, 0xb6, 0x75, 0xa3, 0xc6, 0x4d, 0x25, - 0x4b, 0xe9, 0xe5, 0xdd, 0x0d, 0x46, 0xe5, 0x9d, 0xf8, 0xbf, 0x61, 0xba, 0xad, 0x19, 0x51, 0x76, - 0xe3, 0x25, 0xc8, 0xec, 0x9a, 0x2d, 0x43, 0x93, 0xd9, 0xbb, 0x0e, 0xbe, 0xf8, 0x97, 0xa6, 0x34, - 0xf6, 0xbc, 0xfc, 0xbf, 0xc6, 0x60, 0x46, 0xc2, 0xb6, 0x59, 0x3f, 0xc0, 0xd1, 0x2b, 0xb2, 0x04, - 0xfc, 0x2d, 0x8b, 0xfc, 0x4c, 0xfa, 0x4c, 0x31, 0x66, 0x36, 0xa5, 0xb5, 0xaf, 0xa3, 0x5f, 0xee, - 0x6f, 0x8b, 0xdd, 0x2b, 0xe7, 0x7c, 0x59, 0x2e, 0xd1, 0xb6, 0x2c, 0x67, 0xc2, 0xa4, 0x5e, 0x33, - 0x4c, 0xe2, 0xb3, 0x6c, 0xfc, 0xd8, 0x68, 0x35, 0x5c, 0xcc, 0xb2, 0xd8, 0xaf, 0x92, 0x65, 0xc6, - 0x52, 0xc1, 0x8f, 0x37, 0x5b, 0x0d, 0x1a, 0x39, 0xaf, 0x9c, 0x25, 0xf5, 0x7d, 0x7a, 0x32, 0x9f, - 0x6d, 0xbb, 0x67, 0x4b, 0x59, 0xdd, 0xbb, 0x26, 0xd2, 0x79, 0x97, 0x7f, 0x19, 0xce, 0x74, 0xa8, - 0x3c, 0xca, 0x18, 0xe7, 0x2f, 0xe3, 0x70, 0xbe, 0x5d, 0x7c, 0xd4, 0x48, 0xe4, 0xd3, 0xde, 0xad, - 0x25, 0x98, 0x68, 0xe8, 0xc6, 0xb3, 0x2d, 0x44, 0x66, 0x1a, 0xba, 0xe1, 0xaf, 0xe7, 0x86, 0x18, - 0xc8, 0xd8, 0x7f, 0x83, 0x81, 0x28, 0x30, 0x1b, 0xd6, 0x83, 0x51, 0x5a, 0xc9, 0xfb, 0x02, 0x64, - 0xa2, 0x5e, 0x5b, 0x7b, 0xb6, 0x1c, 0x33, 0xde, 0xe6, 0x2a, 0x4c, 0x7c, 0x02, 0x8b, 0x71, 0xdf, - 0x11, 0x00, 0x55, 0xad, 0x96, 0x41, 0x40, 0xee, 0x7d, 0xb3, 0x16, 0x65, 0x63, 0x67, 0x60, 0x54, - 0x37, 0x34, 0x7c, 0x44, 0x1b, 0x9b, 0x90, 0xd8, 0x45, 0xdb, 0x0b, 0xc4, 0xf8, 0x50, 0x2f, 0x10, - 0xfd, 0x54, 0x95, 0xb6, 0x8a, 0x46, 0xa9, 0x85, 0x3f, 0x8c, 0xc1, 0x34, 0x6f, 0x4e, 0xe4, 0x8b, - 0x91, 0xaf, 0xc0, 0x68, 0x9d, 0xc8, 0xec, 0xd3, 0xe7, 0xf4, 0x99, 0x6e, 0x9f, 0xd3, 0xc2, 0xe8, - 0x73, 0x00, 0x4d, 0x0b, 0x1f, 0xc8, 0x8c, 0x35, 0x3e, 0x14, 0x6b, 0x8a, 0x70, 0x50, 0x02, 0xfa, - 0x22, 0x4c, 0x92, 0x11, 0xde, 0xb4, 0xcc, 0xa6, 0x69, 0x93, 0x20, 0xc5, 0x1e, 0x0e, 0xe9, 0x4c, - 0x3d, 0x3d, 0x99, 0x9f, 0xd8, 0xd0, 0x8d, 0x2d, 0xce, 0x58, 0xad, 0x48, 0xc4, 0x55, 0x78, 0x97, - 0xee, 0x00, 0xfc, 0x7b, 0x01, 0x66, 0x3e, 0xb1, 0xe5, 0xdb, 0xff, 0x09, 0x8d, 0x79, 0x33, 0x8f, - 0x48, 0x2f, 0xcb, 0xc6, 0xae, 0x19, 0xfd, 0xa2, 0xfa, 0xfb, 0x02, 0x4c, 0x05, 0xc4, 0x47, 0x19, - 0xc9, 0x3c, 0x93, 0xce, 0xf2, 0x5f, 0x22, 0xb1, 0x4d, 0xd0, 0xec, 0xa3, 0x1c, 0x54, 0x7f, 0x1e, - 0x83, 0xb3, 0x05, 0xf6, 0x6a, 0xd9, 0xcd, 0xbb, 0x88, 0xd2, 0x4a, 0x72, 0x30, 0x7e, 0x80, 0x2d, - 0x5b, 0x37, 0xd9, 0x0c, 0x3b, 0x21, 0xb9, 0x97, 0x68, 0x16, 0x92, 0xb6, 0xa1, 0x34, 0xed, 0x3d, - 0xd3, 0x7d, 0x1b, 0xe7, 0x5d, 0x7b, 0x39, 0x22, 0xa3, 0xcf, 0x9e, 0x23, 0x32, 0xd6, 0x3f, 0x47, - 0x64, 0xfc, 0x97, 0xce, 0x11, 0xe1, 0xaf, 0xbe, 0xbe, 0x2f, 0xc0, 0xb9, 0x2e, 0xfd, 0x45, 0x69, - 0x33, 0x5f, 0x85, 0xb4, 0xca, 0x05, 0x13, 0x6f, 0xcc, 0xde, 0xee, 0x95, 0x49, 0xb1, 0x67, 0x04, - 0x20, 0x4f, 0x4f, 0xe6, 0xc1, 0xad, 0x6a, 0x79, 0x95, 0xab, 0x88, 0xfc, 0xd7, 0xf2, 0xff, 0x90, - 0x81, 0xc9, 0xe2, 0x11, 0x5b, 0xbb, 0xae, 0xb0, 0x78, 0x00, 0xdd, 0x85, 0x64, 0xd3, 0x32, 0x0f, - 0x74, 0xb7, 0x19, 0xd9, 0xb6, 0xd4, 0x00, 0xb7, 0x19, 0x1d, 0x5c, 0x5b, 0x9c, 0x43, 0xf2, 0x78, - 0x51, 0x15, 0x52, 0xf7, 0x4d, 0x55, 0xa9, 0xdf, 0xd5, 0xeb, 0xae, 0xfd, 0xbf, 0x3c, 0x58, 0xd0, - 0xa2, 0xc7, 0xb3, 0xa5, 0x38, 0x7b, 0x6e, 0x57, 0x78, 0x44, 0x54, 0x86, 0x64, 0xc9, 0x71, 0x9a, - 0xe4, 0x26, 0xf7, 0x26, 0xd7, 0x86, 0x10, 0x4a, 0x58, 0xb8, 0x2c, 0x8f, 0x1d, 0x55, 0x61, 0xea, - 0x9e, 0x69, 0xd6, 0xea, 0xb8, 0x50, 0x37, 0x5b, 0x5a, 0xc1, 0x34, 0x76, 0xf5, 0x1a, 0xf7, 0xc7, - 0x57, 0x87, 0x90, 0x79, 0xaf, 0x50, 0x91, 0xba, 0x05, 0xa0, 0x65, 0x48, 0x56, 0x6e, 0x73, 0x61, - 0x2c, 0x80, 0xbb, 0x32, 0x84, 0xb0, 0xca, 0x6d, 0xc9, 0x63, 0x43, 0x6b, 0x90, 0x5e, 0x7e, 0xd2, - 0xb2, 0x30, 0x97, 0x32, 0xd6, 0x33, 0x2f, 0xa1, 0x53, 0x0a, 0xe5, 0x92, 0x82, 0xcc, 0xa8, 0x02, - 0xd9, 0xb7, 0x4d, 0x6b, 0xbf, 0x6e, 0x2a, 0x6e, 0x0b, 0xc7, 0xa9, 0xb8, 0xcf, 0x0c, 0x21, 0xce, - 0x65, 0x94, 0x3a, 0x44, 0xa0, 0x2f, 0xc3, 0x24, 0xe9, 0x8c, 0xaa, 0xb2, 0x53, 0x77, 0x2b, 0x99, - 0xa4, 0x52, 0x5f, 0x1c, 0x42, 0xaa, 0xc7, 0xe9, 0xbe, 0x3c, 0xe9, 0x10, 0x35, 0xfb, 0x45, 0x98, - 0x68, 0x33, 0x02, 0x84, 0x20, 0xd1, 0x24, 0xfd, 0x2d, 0xd0, 0xfc, 0x21, 0xfa, 0x1f, 0xbd, 0x04, - 0xe3, 0x86, 0xa9, 0x61, 0x77, 0x84, 0x4c, 0xac, 0xcc, 0x3c, 0x3d, 0x99, 0x1f, 0xdb, 0x34, 0x35, - 0x16, 0xae, 0xf0, 0x7f, 0xd2, 0x18, 0x29, 0xe4, 0x06, 0x2b, 0xb3, 0x57, 0x21, 0x41, 0x7a, 0x9f, - 0x38, 0xa9, 0x1d, 0xc5, 0xc6, 0xdb, 0x96, 0xce, 0x65, 0xba, 0x97, 0xbc, 0xdc, 0x8f, 0x05, 0x88, - 0x55, 0x6e, 0x93, 0x40, 0x7d, 0xa7, 0xa5, 0xee, 0x63, 0x87, 0x97, 0xe2, 0x57, 0x34, 0x80, 0xb7, - 0xf0, 0xae, 0xce, 0x62, 0xa8, 0x94, 0xc4, 0xaf, 0xd0, 0x73, 0x00, 0x8a, 0xaa, 0x62, 0xdb, 0x96, - 0xdd, 0x5d, 0x73, 0x29, 0x29, 0xc5, 0x28, 0xeb, 0xf8, 0x98, 0xb0, 0xd9, 0x58, 0xb5, 0xb0, 0xe3, - 0x26, 0x42, 0xb1, 0x2b, 0xc2, 0xe6, 0xe0, 0x46, 0x53, 0x76, 0xcc, 0x7d, 0x6c, 0x50, 0x9b, 0x49, - 0x11, 0xe7, 0xd3, 0x68, 0x56, 0x09, 0x81, 0xf8, 0x4d, 0x6c, 0x68, 0xbe, 0x93, 0x4b, 0x49, 0xde, - 0x35, 0x11, 0x69, 0xe1, 0x9a, 0xce, 0x37, 0x7e, 0xa5, 0x24, 0x7e, 0x45, 0x34, 0xa6, 0xb4, 0x9c, - 0x3d, 0xda, 0x2b, 0x29, 0x89, 0xfe, 0xe7, 0x4d, 0xfb, 0x1d, 0x01, 0xe2, 0xf7, 0x0a, 0x95, 0x53, - 0xb7, 0xcd, 0x95, 0x18, 0xf7, 0x25, 0xd2, 0xfc, 0x43, 0xbd, 0x5e, 0xd7, 0x8d, 0x1a, 0x09, 0x69, - 0xbe, 0x8a, 0x55, 0xb7, 0x65, 0x59, 0x4e, 0xde, 0x62, 0x54, 0xb4, 0x00, 0x69, 0xd5, 0xc2, 0x1a, - 0x36, 0x1c, 0x5d, 0xa9, 0xdb, 0xbc, 0x89, 0x41, 0x12, 0xaf, 0xdc, 0xd7, 0x05, 0x18, 0xa5, 0xc6, - 0x8b, 0x2e, 0x42, 0x4a, 0x35, 0x0d, 0x47, 0xd1, 0x0d, 0xee, 0x85, 0x52, 0x92, 0x4f, 0xe8, 0x59, - 0xc9, 0x4b, 0x90, 0x51, 0x54, 0xd5, 0x6c, 0x19, 0x8e, 0x6c, 0x28, 0x0d, 0xcc, 0x2b, 0x9b, 0xe6, - 0xb4, 0x4d, 0xa5, 0x81, 0xd1, 0x3c, 0xb8, 0x97, 0xde, 0xde, 0xc5, 0x94, 0x04, 0x9c, 0xb4, 0x8e, - 0x8f, 0x79, 0x4d, 0xbe, 0x2f, 0x40, 0xd2, 0x35, 0x7a, 0x52, 0x99, 0x1a, 0x36, 0xb0, 0xa5, 0x38, - 0xa6, 0x57, 0x19, 0x8f, 0xd0, 0x39, 0xe3, 0xa5, 0xfc, 0x19, 0x6f, 0x06, 0x46, 0x1d, 0x62, 0xd7, - 0xbc, 0x1e, 0xec, 0x82, 0xae, 0x35, 0xd7, 0x95, 0x1a, 0x5b, 0x5e, 0x4b, 0x49, 0xec, 0x82, 0x34, - 0x89, 0xe7, 0xd0, 0x32, 0xed, 0xf0, 0x2b, 0x52, 0x5f, 0x96, 0xe3, 0xb9, 0x83, 0x6b, 0xba, 0x41, - 0x0d, 0x20, 0x2e, 0x01, 0x25, 0xad, 0x10, 0x0a, 0xba, 0x00, 0x29, 0x56, 0x00, 0x1b, 0x1a, 0xb5, - 0x82, 0xb8, 0x94, 0xa4, 0x84, 0xa2, 0xbb, 0x39, 0x6b, 0x76, 0x1f, 0x52, 0xde, 0x18, 0x23, 0x1d, - 0xd9, 0xb2, 0x3d, 0xa5, 0xd2, 0xff, 0xe8, 0x65, 0x98, 0x79, 0xdc, 0x52, 0xea, 0xfa, 0x2e, 0x5d, - 0x39, 0x23, 0xc5, 0x98, 0xfe, 0x58, 0x7b, 0x90, 0x77, 0x8f, 0x4a, 0xa0, 0x6a, 0x74, 0x87, 0x64, - 0xdc, 0x1f, 0x92, 0xc1, 0x57, 0x21, 0xf9, 0x0f, 0x05, 0x98, 0x62, 0x69, 0x40, 0x2c, 0x13, 0x35, - 0xba, 0x00, 0xe3, 0x0d, 0x48, 0x69, 0x8a, 0xa3, 0xb0, 0xfd, 0x99, 0xb1, 0xbe, 0xfb, 0x33, 0x5d, - 0x8f, 0x4f, 0xca, 0xd3, 0x3d, 0x9a, 0x08, 0x12, 0xe4, 0x3f, 0xdb, 0xd0, 0x2a, 0xd1, 0xff, 0x7e, - 0x62, 0x45, 0xb0, 0xba, 0x51, 0x06, 0x5c, 0x4b, 0x70, 0x86, 0x68, 0xbf, 0x68, 0xa8, 0xd6, 0x71, - 0xd3, 0xd1, 0x4d, 0xe3, 0x01, 0xfd, 0xb5, 0x91, 0x18, 0x78, 0x31, 0x45, 0xdf, 0x47, 0xf1, 0xba, - 0xfc, 0xd5, 0x18, 0x4c, 0x14, 0x8f, 0x9a, 0xa6, 0x15, 0xe9, 0xa2, 0xd6, 0x0a, 0x8c, 0x73, 0xc4, - 0xdf, 0xe7, 0x55, 0x71, 0x87, 0xaf, 0x76, 0xdf, 0xc2, 0x72, 0x46, 0xb4, 0x02, 0xc0, 0x72, 0x46, - 0x69, 0x2e, 0x51, 0xfc, 0x14, 0x2f, 0xcc, 0x28, 0x1b, 0xa1, 0xa2, 0x4d, 0x48, 0x37, 0x0e, 0x54, - 0x55, 0xde, 0xd5, 0xeb, 0x0e, 0x4f, 0xba, 0x0b, 0xcf, 0x18, 0xdf, 0x78, 0x54, 0x28, 0xdc, 0xa5, - 0x85, 0x58, 0xfe, 0x9b, 0x7f, 0x2d, 0x01, 0x91, 0xc0, 0xfe, 0xa3, 0x17, 0x81, 0xef, 0x9b, 0x91, - 0x6d, 0x77, 0x8b, 0xdc, 0xca, 0xc4, 0xd3, 0x93, 0xf9, 0x94, 0x44, 0xa9, 0x95, 0x4a, 0x55, 0x4a, - 0xb1, 0x02, 0x15, 0xdb, 0x41, 0xcf, 0xc3, 0x84, 0xd9, 0xd0, 0x1d, 0xd9, 0x8d, 0x81, 0x78, 0xd8, - 0x98, 0x21, 0x44, 0x37, 0x46, 0x42, 0x55, 0xb8, 0x86, 0x0d, 0x3a, 0x0a, 0x48, 0x3b, 0xe5, 0x1d, - 0xb6, 0x16, 0xe9, 0xb0, 0xf1, 0x2e, 0x9b, 0x4d, 0x47, 0x6f, 0xe8, 0x4f, 0xe8, 0xcb, 0x6a, 0xfe, - 0xbe, 0xe8, 0x79, 0x56, 0x9c, 0xb4, 0x6f, 0x85, 0x2e, 0x52, 0xf2, 0xb2, 0x0f, 0x02, 0x45, 0xd1, - 0xd7, 0x05, 0x38, 0xcb, 0x15, 0x29, 0xef, 0xd0, 0x94, 0x77, 0xa5, 0xae, 0x3b, 0xc7, 0xf2, 0xfe, - 0x41, 0x2e, 0x49, 0x83, 0xd3, 0xcf, 0x86, 0x76, 0x48, 0xc0, 0x0e, 0x16, 0xdd, 0x6e, 0x39, 0xbe, - 0xcf, 0x99, 0xd7, 0x0f, 0x8a, 0x86, 0x63, 0x1d, 0xaf, 0x9c, 0x7b, 0x7a, 0x32, 0x3f, 0xdd, 0x7d, - 0xf7, 0x91, 0x34, 0x6d, 0x77, 0xb3, 0xa0, 0x12, 0x00, 0xf6, 0xac, 0x91, 0xa6, 0xfc, 0x85, 0x87, - 0x17, 0xa1, 0x66, 0x2b, 0x05, 0x78, 0xd1, 0x75, 0x10, 0xf9, 0xa6, 0x97, 0x5d, 0xbd, 0x8e, 0x65, - 0x5b, 0x7f, 0x82, 0x73, 0x40, 0x7d, 0x50, 0x96, 0xd1, 0x89, 0x88, 0x8a, 0xfe, 0x04, 0xcf, 0x7e, - 0x15, 0x72, 0xbd, 0x6a, 0x1f, 0x1c, 0x08, 0x29, 0xf6, 0x62, 0xf6, 0xf5, 0xf6, 0x15, 0x99, 0x21, - 0x4c, 0xd5, 0x5d, 0x95, 0x89, 0xbd, 0xee, 0xba, 0xa0, 0xef, 0xc6, 0x60, 0x62, 0xa5, 0x55, 0xdf, - 0x7f, 0xd0, 0xac, 0xb4, 0x1a, 0x0d, 0xc5, 0x3a, 0x26, 0xae, 0x92, 0xb9, 0x0e, 0x52, 0x4d, 0x81, - 0xb9, 0x4a, 0xea, 0x1b, 0xf4, 0x27, 0x98, 0x4c, 0x66, 0xc1, 0x4d, 0xda, 0x2c, 0xa5, 0x9f, 0xb6, - 0x24, 0xb0, 0xf3, 0xda, 0x3c, 0xb4, 0xd1, 0xeb, 0x90, 0x0b, 0x14, 0xa4, 0xcb, 0x27, 0x32, 0x36, - 0x1c, 0x4b, 0xc7, 0x6c, 0x39, 0x30, 0x2e, 0x05, 0xd2, 0x69, 0xca, 0xe4, 0x76, 0x91, 0xdd, 0x45, - 0x55, 0xc8, 0x90, 0x82, 0xc7, 0x32, 0x9d, 0x6c, 0xdc, 0x45, 0xdb, 0x9b, 0x21, 0x8d, 0x6b, 0xab, - 0xf7, 0x22, 0xd5, 0x52, 0x81, 0xf2, 0xd0, 0xbf, 0x52, 0x1a, 0xfb, 0x94, 0xd9, 0xb7, 0x40, 0xec, - 0x2c, 0x10, 0xd4, 0x68, 0x82, 0x69, 0x74, 0x26, 0xa8, 0xd1, 0x78, 0x40, 0x5b, 0x6b, 0x89, 0x64, - 0x42, 0x1c, 0xcd, 0xff, 0x34, 0x0e, 0x59, 0xd7, 0xd8, 0xa2, 0x44, 0x33, 0x2b, 0x30, 0x4a, 0x4c, - 0xc3, 0x4d, 0xfe, 0xb8, 0xda, 0xc7, 0xc6, 0x79, 0xfa, 0x38, 0x31, 0x19, 0x17, 0x0f, 0x53, 0xd6, - 0x28, 0xdc, 0xce, 0xec, 0xff, 0x89, 0x41, 0x82, 0x02, 0x88, 0x9b, 0x90, 0xa0, 0x53, 0x87, 0x30, - 0xcc, 0xd4, 0x41, 0x8b, 0x7a, 0x93, 0x5d, 0x2c, 0x10, 0x7f, 0x92, 0x60, 0x6e, 0x4f, 0x79, 0xf5, - 0xe6, 0x2d, 0xea, 0x72, 0x32, 0x12, 0xbf, 0x42, 0x2b, 0x34, 0x2b, 0xc9, 0xb4, 0x1c, 0xac, 0xf1, - 0xc0, 0x7d, 0x61, 0x50, 0xff, 0xba, 0xd3, 0x94, 0xcb, 0x87, 0xce, 0x43, 0x9c, 0xf8, 0xb2, 0x71, - 0x96, 0xb1, 0xf0, 0xf4, 0x64, 0x3e, 0x4e, 0xbc, 0x18, 0xa1, 0xa1, 0x25, 0x48, 0xb7, 0x3b, 0x0e, - 0xe1, 0x7a, 0x8a, 0xb9, 0xc7, 0xc0, 0xa0, 0x87, 0xba, 0x37, 0xc0, 0x18, 0x68, 0xe5, 0x7d, 0xfc, - 0xb5, 0x51, 0x98, 0x28, 0x37, 0xa2, 0x9e, 0x58, 0x96, 0xdb, 0x7b, 0x38, 0x0c, 0xed, 0xb4, 0x3d, - 0x34, 0xa4, 0x83, 0xdb, 0xe6, 0xf4, 0xf8, 0xe9, 0xe6, 0xf4, 0x32, 0x09, 0x81, 0xf9, 0xa9, 0x0b, - 0xf1, 0x1e, 0xc0, 0xa6, 0xfd, 0xf9, 0x34, 0x8a, 0x91, 0x08, 0x8f, 0xbf, 0xa1, 0x82, 0x66, 0x9d, - 0xbc, 0x45, 0x23, 0x6d, 0x66, 0x65, 0x63, 0xc3, 0x5b, 0xd9, 0x38, 0x36, 0x34, 0x3a, 0xb5, 0xb5, - 0xfb, 0xd5, 0xf1, 0x67, 0xf7, 0xab, 0xb3, 0x4f, 0xb8, 0xb1, 0xbe, 0x01, 0x71, 0x4d, 0x77, 0x3b, - 0x67, 0xf8, 0x09, 0x9b, 0x30, 0x0d, 0xb0, 0xda, 0x44, 0xd0, 0x6a, 0x83, 0x0b, 0x1c, 0xb3, 0x0f, - 0x00, 0x7c, 0x0d, 0xa1, 0x05, 0x18, 0x33, 0xeb, 0x9a, 0xbb, 0xaf, 0x64, 0x62, 0x25, 0xf5, 0xf4, - 0x64, 0x7e, 0xf4, 0x41, 0x5d, 0x2b, 0xaf, 0x4a, 0xa3, 0x66, 0x5d, 0x2b, 0x6b, 0xf4, 0xe0, 0x0b, - 0x7c, 0x28, 0x7b, 0x49, 0x68, 0x19, 0x69, 0xdc, 0xc0, 0x87, 0xab, 0xd8, 0x56, 0x3b, 0x92, 0x63, - 0x88, 0x09, 0x7e, 0x4b, 0x80, 0xac, 0xdb, 0x1b, 0xd1, 0xba, 0x99, 0xa4, 0xde, 0xe0, 0xc3, 0x2e, - 0x7e, 0xba, 0x61, 0xe7, 0xf2, 0xf1, 0x5d, 0xb5, 0xdf, 0x10, 0x78, 0x02, 0x72, 0x45, 0x55, 0x1c, - 0x12, 0x6c, 0x44, 0x38, 0x54, 0x5e, 0x00, 0xd1, 0x52, 0x0c, 0xcd, 0x6c, 0xe8, 0x4f, 0x30, 0x5b, - 0x11, 0xb5, 0xf9, 0xcb, 0xcd, 0x49, 0x8f, 0x4e, 0x97, 0xfc, 0xdc, 0x05, 0xdd, 0x5f, 0x08, 0x3c, - 0x59, 0xd9, 0xab, 0x4c, 0x94, 0x4a, 0x5b, 0x87, 0x31, 0x8b, 0xa5, 0x3c, 0xb2, 0xa1, 0xfb, 0x52, - 0x88, 0x90, 0xb0, 0xa7, 0xb3, 0x8c, 0x42, 0x6f, 0xf0, 0x50, 0x11, 0xb3, 0x5f, 0x80, 0x51, 0x4a, - 0x7e, 0x06, 0x07, 0xcb, 0x35, 0xff, 0xf3, 0x18, 0x5c, 0xa6, 0x8f, 0x7b, 0x84, 0x2d, 0x7d, 0xf7, - 0x78, 0xcb, 0x32, 0x1d, 0xac, 0x3a, 0x58, 0xf3, 0xb7, 0x71, 0x44, 0xea, 0xb5, 0x52, 0x4d, 0xf7, - 0x01, 0xa7, 0x4a, 0xfd, 0xf2, 0xb8, 0xd0, 0x3a, 0x4c, 0xb2, 0xc3, 0x75, 0x64, 0xa5, 0xae, 0x1f, - 0x60, 0x59, 0x71, 0x4e, 0x33, 0x37, 0x4d, 0x30, 0xde, 0x65, 0xc2, 0xba, 0xec, 0x20, 0x0d, 0x52, - 0x5c, 0x98, 0xae, 0xf1, 0x13, 0x75, 0xee, 0xfd, 0x72, 0x6b, 0x7e, 0x49, 0x89, 0xca, 0x2b, 0xaf, - 0x4a, 0x49, 0x26, 0xd9, 0x7b, 0x67, 0xf3, 0x23, 0x01, 0xae, 0x0c, 0x50, 0x74, 0x94, 0x66, 0x36, - 0x0b, 0xc9, 0x03, 0xf2, 0x20, 0x9d, 0x6b, 0x3a, 0x29, 0x79, 0xd7, 0x68, 0x03, 0x26, 0x76, 0x15, - 0xbd, 0xee, 0x1e, 0x8b, 0xd3, 0x2f, 0x5f, 0x30, 0x3c, 0x8d, 0x35, 0xc3, 0xd8, 0xe9, 0x4d, 0xba, - 0xd1, 0x71, 0x6a, 0x59, 0xd3, 0x2a, 0x15, 0xee, 0xc1, 0xa2, 0xb3, 0x17, 0x17, 0x3a, 0xc6, 0x7c, - 0xe8, 0x88, 0x5e, 0x02, 0xa4, 0xe9, 0x36, 0x3b, 0xad, 0xc3, 0xde, 0x53, 0x34, 0xf3, 0xd0, 0xcf, - 0x9a, 0x98, 0x72, 0xef, 0x54, 0xdc, 0x1b, 0xa8, 0x02, 0x14, 0xb7, 0xc8, 0xb6, 0xa3, 0x78, 0x2f, - 0x7e, 0xae, 0x0c, 0xb5, 0xeb, 0x8a, 0x01, 0x1a, 0xef, 0x52, 0x4a, 0x11, 0x39, 0xf4, 0x2f, 0x89, - 0xc0, 0x75, 0xd2, 0x74, 0x47, 0x56, 0x6c, 0x77, 0x8b, 0x0e, 0x3b, 0x27, 0x24, 0xcb, 0xe8, 0xcb, - 0x76, 0x70, 0xe7, 0x0d, 0xdb, 0x41, 0xe0, 0x2b, 0x28, 0x4a, 0xa0, 0xfb, 0x07, 0x02, 0x64, 0x25, - 0xbc, 0x6b, 0x61, 0x3b, 0x52, 0xc0, 0x7f, 0x17, 0x32, 0x16, 0x93, 0x2a, 0xef, 0x5a, 0x66, 0xe3, - 0x34, 0x63, 0x2c, 0xcd, 0x19, 0xef, 0x5a, 0x66, 0xa3, 0xed, 0xe8, 0x84, 0x47, 0x30, 0xe9, 0xd5, - 0x34, 0x4a, 0x15, 0x7c, 0x48, 0x77, 0x1a, 0x33, 0xc1, 0x51, 0xa7, 0x2f, 0x7c, 0x12, 0x7a, 0xa0, - 0x6f, 0x9a, 0x82, 0xd5, 0x8d, 0x52, 0x19, 0xbf, 0x10, 0x20, 0x5b, 0x69, 0xed, 0xb0, 0xc3, 0xa2, - 0xa2, 0xd3, 0x43, 0x11, 0x52, 0x75, 0xbc, 0xeb, 0xc8, 0xcf, 0x94, 0xf5, 0x9e, 0x24, 0xac, 0x34, - 0xf3, 0xff, 0x1e, 0x80, 0x45, 0xf7, 0xb5, 0x51, 0x39, 0xf1, 0x53, 0xca, 0x49, 0x51, 0x5e, 0x3f, - 0xc8, 0xc9, 0x7f, 0x18, 0x83, 0x49, 0xaf, 0xb1, 0x51, 0x7a, 0xcf, 0xb7, 0xdb, 0xbc, 0x46, 0xfc, - 0x34, 0x5e, 0x63, 0x8a, 0x67, 0x6f, 0x84, 0x7b, 0x8e, 0x45, 0x98, 0xa6, 0x21, 0x88, 0xac, 0x34, - 0x9b, 0x75, 0xdd, 0x85, 0xb2, 0xd4, 0x2f, 0x25, 0xa4, 0x29, 0x7a, 0x6b, 0x99, 0xdd, 0xa1, 0x20, - 0x96, 0xd8, 0xdf, 0xae, 0x85, 0xf1, 0x13, 0x2c, 0x53, 0x54, 0x75, 0x9a, 0xec, 0x94, 0x34, 0x63, - 0xac, 0x10, 0x3e, 0x6e, 0x79, 0xef, 0xc1, 0x14, 0xd5, 0x6c, 0xd4, 0x7b, 0x6b, 0x79, 0x77, 0xfc, - 0x44, 0x00, 0x14, 0x94, 0xff, 0xc9, 0xf5, 0x48, 0x2c, 0xba, 0x1e, 0x79, 0x11, 0x10, 0xcb, 0x42, - 0xb4, 0xe5, 0x26, 0xb6, 0x64, 0x1b, 0xab, 0x26, 0x3f, 0xc2, 0x48, 0x90, 0x44, 0x7e, 0x67, 0x0b, - 0x5b, 0x15, 0x4a, 0xcf, 0xbf, 0x3f, 0x0b, 0x19, 0xae, 0x8c, 0x6d, 0x43, 0x37, 0x0d, 0x74, 0x13, - 0xe2, 0x35, 0xbe, 0x9e, 0x9f, 0x0e, 0x5d, 0x51, 0xf3, 0x0f, 0x5e, 0x2b, 0x8d, 0x48, 0xa4, 0x2c, - 0x61, 0x69, 0xb6, 0x9c, 0x90, 0xf8, 0xc7, 0x4f, 0x97, 0x0e, 0xb2, 0x34, 0x5b, 0x0e, 0xaa, 0xc0, - 0xa4, 0xea, 0x1f, 0x23, 0x25, 0x13, 0xf6, 0x78, 0x4f, 0xa4, 0x13, 0x7a, 0x7c, 0x57, 0x69, 0x44, - 0xca, 0xaa, 0x6d, 0x37, 0x50, 0x21, 0x78, 0x6e, 0x51, 0xa2, 0x2b, 0x33, 0xcb, 0xdf, 0x87, 0xdb, - 0x7e, 0x66, 0x52, 0x69, 0x24, 0x70, 0xbc, 0x11, 0x7a, 0x03, 0xc6, 0x34, 0x7a, 0x1e, 0x0e, 0x37, - 0xcd, 0x30, 0xeb, 0x69, 0x3b, 0x82, 0xa8, 0x34, 0x22, 0x71, 0x0e, 0xb4, 0x06, 0x19, 0xf6, 0x8f, - 0xc5, 0x21, 0x1c, 0xfe, 0x5d, 0xe9, 0x2d, 0x21, 0xe0, 0xdd, 0x4b, 0x23, 0x52, 0x5a, 0xf3, 0xa9, - 0xe8, 0x15, 0x48, 0xd8, 0xaa, 0xe2, 0x02, 0xc0, 0xb9, 0x1e, 0x87, 0x61, 0xf8, 0xcc, 0xb4, 0x34, - 0xba, 0xc3, 0x0e, 0x54, 0x74, 0x8e, 0xdc, 0x15, 0xb9, 0xb0, 0xea, 0xb7, 0x6d, 0xb1, 0x26, 0xd5, - 0xc7, 0x94, 0x80, 0xee, 0x41, 0x5a, 0x21, 0x01, 0x9d, 0x4c, 0xb7, 0x34, 0xd2, 0x25, 0xb8, 0xf0, - 0x97, 0xdd, 0x5d, 0xdb, 0x51, 0x4b, 0x74, 0x1f, 0xb7, 0x4b, 0xf4, 0x05, 0x35, 0xb0, 0x55, 0xc3, - 0xb9, 0x74, 0x7f, 0x41, 0xc1, 0x4c, 0x2c, 0x4f, 0x10, 0x25, 0x92, 0xc0, 0x6e, 0xcf, 0xdd, 0xae, - 0x42, 0x1b, 0x95, 0xe9, 0xf9, 0x62, 0x35, 0x64, 0xbb, 0x4d, 0x69, 0x44, 0xca, 0xec, 0x05, 0xc8, - 0x68, 0x11, 0x62, 0x35, 0x35, 0x37, 0x41, 0x65, 0x5c, 0xec, 0xb7, 0x99, 0xa4, 0x34, 0x22, 0xc5, - 0x6a, 0x2a, 0x81, 0xf2, 0x6c, 0x37, 0xc0, 0x91, 0x91, 0xcb, 0xf6, 0x1c, 0xea, 0xed, 0x7b, 0x2a, - 0x4a, 0x23, 0x12, 0xdd, 0x80, 0x40, 0x9e, 0xb7, 0x05, 0x59, 0x8b, 0xa5, 0xb2, 0xb9, 0x49, 0xa8, - 0x62, 0xcf, 0x97, 0xcd, 0x61, 0x79, 0xa8, 0x25, 0x1a, 0xe0, 0x07, 0xe8, 0xe8, 0x2b, 0x30, 0xd3, - 0x2e, 0x91, 0x5b, 0xda, 0x54, 0xcf, 0x17, 0xa7, 0x3d, 0xb3, 0x21, 0x4b, 0x23, 0x12, 0xb2, 0xba, - 0x6e, 0xa2, 0xd7, 0x60, 0x94, 0xf5, 0x1a, 0xa2, 0x22, 0xc3, 0xb2, 0x28, 0x3a, 0x3a, 0x8c, 0x95, - 0x27, 0xc6, 0xef, 0xf0, 0x1c, 0x2e, 0xb9, 0x6e, 0xd6, 0x72, 0xd3, 0x3d, 0x8d, 0xbf, 0x3b, 0x27, - 0x8d, 0x18, 0xbf, 0xe3, 0x53, 0x49, 0xbf, 0x5b, 0xec, 0x0e, 0x4f, 0xf9, 0x99, 0xe9, 0xd9, 0xef, - 0x21, 0xa9, 0x5d, 0x25, 0x9a, 0x55, 0xef, 0x93, 0x49, 0xd5, 0x2c, 0x76, 0x72, 0x8b, 0x4c, 0xc7, - 0xd4, 0x99, 0x9e, 0x55, 0xeb, 0x3e, 0xe0, 0xa6, 0x44, 0x03, 0x1f, 0x8f, 0x8a, 0x1e, 0x81, 0xc8, - 0xcf, 0x54, 0xf0, 0x97, 0xff, 0xcf, 0x52, 0x79, 0x2f, 0x84, 0xba, 0xae, 0xb0, 0x1c, 0x99, 0xd2, - 0x88, 0x34, 0xa9, 0xb6, 0xdf, 0x41, 0xef, 0xc0, 0x14, 0x95, 0x27, 0xab, 0xfe, 0x61, 0x18, 0xb9, - 0x5c, 0xd7, 0xa1, 0x0a, 0xbd, 0xcf, 0xcd, 0x70, 0x25, 0x8b, 0x6a, 0xc7, 0x2d, 0x62, 0xc6, 0xba, - 0xa1, 0x3b, 0xd4, 0xcb, 0xce, 0xf6, 0x34, 0xe3, 0xf6, 0x63, 0xf7, 0x88, 0x19, 0xeb, 0x8c, 0x42, - 0xcc, 0xd8, 0xe1, 0xf9, 0x60, 0xbc, 0x3b, 0x2e, 0xf6, 0x34, 0xe3, 0xb0, 0xc4, 0x31, 0x62, 0xc6, - 0x4e, 0x90, 0x4e, 0xcc, 0x98, 0x39, 0x88, 0x0e, 0xb9, 0xcf, 0xf5, 0x34, 0xe3, 0x9e, 0x9b, 0x8a, - 0x89, 0x19, 0x2b, 0x5d, 0x37, 0xd1, 0x2a, 0x00, 0x8b, 0x4b, 0x74, 0x63, 0xd7, 0xcc, 0xcd, 0xf5, - 0x9c, 0x0c, 0x3a, 0x33, 0xc2, 0xc8, 0x64, 0x50, 0x77, 0x69, 0xc4, 0x91, 0x51, 0x34, 0x24, 0xd3, - 0x77, 0xa1, 0xb9, 0xf9, 0x9e, 0x8e, 0xac, 0xeb, 0x2d, 0x25, 0x71, 0x64, 0x87, 0x1e, 0x91, 0xcc, - 0x2a, 0x6c, 0x61, 0x36, 0xb7, 0xd0, 0xdb, 0x2d, 0x07, 0xdf, 0xd2, 0x50, 0xb7, 0x4c, 0x09, 0x68, - 0x19, 0x52, 0x64, 0xda, 0x3e, 0xa6, 0x6e, 0xe8, 0x52, 0xcf, 0x10, 0xb3, 0x63, 0xdb, 0x48, 0x69, - 0x44, 0x4a, 0x3e, 0xe6, 0x24, 0xf2, 0x78, 0xb6, 0x40, 0x95, 0xcb, 0xf7, 0x7c, 0x7c, 0xdb, 0xf2, - 0x26, 0x79, 0x3c, 0xe3, 0x40, 0x2a, 0x9c, 0x61, 0x7d, 0xc5, 0xf7, 0xf4, 0x5a, 0x7c, 0x03, 0x6a, - 0xee, 0x79, 0x2a, 0xaa, 0xe7, 0x72, 0x4f, 0xe8, 0x56, 0xe3, 0xd2, 0x88, 0x34, 0xad, 0x74, 0xdf, - 0x25, 0x03, 0x9e, 0x4f, 0x3d, 0x6c, 0x91, 0x28, 0x77, 0xb9, 0xe7, 0x80, 0x0f, 0x59, 0x56, 0x23, - 0x03, 0x5e, 0x09, 0x90, 0xd9, 0x04, 0xa4, 0xc9, 0xb6, 0xcd, 0xde, 0x9c, 0x5f, 0xe9, 0x33, 0x01, - 0x75, 0xc0, 0x7c, 0x36, 0x01, 0x69, 0x15, 0xc6, 0x49, 0x04, 0xa9, 0x75, 0xac, 0x58, 0xdc, 0xcd, - 0x5e, 0xed, 0x29, 0xa8, 0xeb, 0x28, 0x3b, 0x22, 0x48, 0xf5, 0x88, 0x24, 0xe0, 0xb1, 0xdc, 0xc3, - 0x58, 0x78, 0xcc, 0x77, 0xad, 0x67, 0xc0, 0x13, 0x7a, 0x66, 0x0c, 0x09, 0x78, 0xac, 0xb6, 0x1b, - 0xe8, 0x73, 0x30, 0xce, 0x31, 0x59, 0xee, 0x7a, 0x9f, 0x48, 0x34, 0x08, 0xa6, 0xc9, 0xb8, 0xe6, - 0x3c, 0xcc, 0xcb, 0x32, 0x2c, 0xc8, 0x9a, 0xf7, 0x42, 0x1f, 0x2f, 0xdb, 0x05, 0x47, 0x99, 0x97, - 0xf5, 0xc9, 0xc4, 0xcb, 0x32, 0x3b, 0xe5, 0x73, 0xdd, 0x8d, 0x9e, 0x5e, 0xb6, 0x7b, 0xeb, 0x0a, - 0xf1, 0xb2, 0x8f, 0x7d, 0x2a, 0x69, 0x99, 0xcd, 0x70, 0x50, 0xee, 0x33, 0x3d, 0x5b, 0xd6, 0x0e, - 0x0b, 0x49, 0xcb, 0x38, 0x0f, 0xe9, 0x36, 0x96, 0x88, 0xcc, 0x34, 0xfd, 0x62, 0xef, 0xed, 0xf3, - 0x9d, 0xe8, 0x81, 0x74, 0x9b, 0xe5, 0x11, 0x7d, 0x47, 0x65, 0xf1, 0xcd, 0xc2, 0x5c, 0x53, 0x2f, - 0xf5, 0x77, 0x54, 0x61, 0xfb, 0xa0, 0x3d, 0x47, 0xd5, 0x76, 0x93, 0x56, 0x95, 0xed, 0x00, 0xa3, - 0xe3, 0x7b, 0xb1, 0xcf, 0x4e, 0xff, 0x8e, 0xdd, 0x78, 0xb4, 0xaa, 0x1e, 0xd1, 0x1f, 0x42, 0x2d, - 0x76, 0x24, 0x45, 0x6e, 0xa9, 0xff, 0x10, 0x6a, 0x3f, 0x1a, 0xc3, 0x1b, 0x42, 0x9c, 0xec, 0xcd, - 0x99, 0x6e, 0x84, 0xf1, 0x72, 0xff, 0x39, 0xb3, 0x33, 0xb4, 0x60, 0x73, 0x26, 0x8f, 0x29, 0xfe, - 0xaf, 0x00, 0x0b, 0xac, 0x6e, 0x74, 0xc9, 0xee, 0x58, 0xf6, 0x96, 0x3f, 0x03, 0xfb, 0x14, 0x6e, - 0xd2, 0x07, 0xbc, 0xd6, 0xab, 0xba, 0x03, 0x96, 0x73, 0x4b, 0x23, 0xd2, 0x73, 0x4a, 0xbf, 0x72, - 0x2b, 0xe3, 0xfc, 0xdd, 0xa5, 0xb7, 0x09, 0x73, 0x52, 0x14, 0xd7, 0x12, 0xc9, 0x73, 0x62, 0x6e, - 0x2d, 0x91, 0x3c, 0x2f, 0xce, 0xae, 0x25, 0x92, 0x17, 0xc4, 0x8b, 0xf9, 0x7f, 0x39, 0x0f, 0x13, - 0x2e, 0x78, 0x63, 0x88, 0xe8, 0x56, 0x10, 0x11, 0xcd, 0xf5, 0x42, 0x44, 0x1c, 0xee, 0x71, 0x48, - 0x74, 0x2b, 0x08, 0x89, 0xe6, 0x7a, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, 0x6a, 0x2f, 0x4c, 0xf4, - 0xc2, 0x10, 0x98, 0xc8, 0x13, 0xd5, 0x09, 0x8a, 0x56, 0xbb, 0x41, 0xd1, 0xe5, 0xfe, 0xa0, 0xc8, - 0x13, 0x15, 0x40, 0x45, 0x77, 0x3a, 0x50, 0xd1, 0xa5, 0x3e, 0xa8, 0xc8, 0xe3, 0x77, 0x61, 0xd1, - 0x7a, 0x28, 0x2c, 0xba, 0x3a, 0x08, 0x16, 0x79, 0x72, 0xda, 0x70, 0xd1, 0xab, 0x6d, 0xb8, 0x68, - 0xbe, 0x27, 0x2e, 0xf2, 0xb8, 0x19, 0x30, 0x7a, 0xb3, 0x13, 0x18, 0x5d, 0xea, 0x03, 0x8c, 0xfc, - 0x16, 0x70, 0x64, 0x54, 0x0a, 0x43, 0x46, 0x57, 0x06, 0x20, 0x23, 0x4f, 0x4a, 0x10, 0x1a, 0x95, - 0xc2, 0xa0, 0xd1, 0x95, 0x01, 0xd0, 0xa8, 0x43, 0x12, 0xc3, 0x46, 0x9b, 0xe1, 0xd8, 0xe8, 0xda, - 0x40, 0x6c, 0xe4, 0x49, 0x6b, 0x07, 0x47, 0x4b, 0x01, 0x70, 0xf4, 0x5c, 0x0f, 0x70, 0xe4, 0xb1, - 0x12, 0x74, 0xf4, 0xf9, 0x2e, 0x74, 0x94, 0xef, 0x87, 0x8e, 0x3c, 0x5e, 0x0f, 0x1e, 0x3d, 0xec, - 0x01, 0x8f, 0xae, 0x0f, 0x86, 0x47, 0x9e, 0xb0, 0x0e, 0x7c, 0xa4, 0xf4, 0xc5, 0x47, 0x2f, 0x0d, - 0x89, 0x8f, 0x3c, 0xe9, 0x61, 0x00, 0xe9, 0xf5, 0x76, 0x80, 0xb4, 0xd0, 0x1b, 0x20, 0x79, 0x62, - 0x38, 0x42, 0x5a, 0x0f, 0x45, 0x48, 0x57, 0x07, 0x21, 0x24, 0x7f, 0x1c, 0x04, 0x21, 0xd2, 0x66, - 0x38, 0x44, 0xba, 0x36, 0x10, 0x22, 0xf9, 0xdd, 0xdf, 0x86, 0x91, 0xd6, 0x43, 0x31, 0xd2, 0xd5, - 0x41, 0x18, 0xc9, 0xaf, 0x5c, 0x10, 0x24, 0xbd, 0xdd, 0x13, 0x24, 0xdd, 0x18, 0x06, 0x24, 0x79, - 0x42, 0xbb, 0x50, 0xd2, 0xbb, 0xbd, 0x51, 0xd2, 0x67, 0x4e, 0x71, 0xba, 0x60, 0x28, 0x4c, 0xfa, - 0x7c, 0x17, 0x4c, 0xca, 0xf7, 0x83, 0x49, 0xbe, 0x3d, 0xbb, 0x38, 0x49, 0xe9, 0x8b, 0x6a, 0x5e, - 0x1a, 0x12, 0xd5, 0xf8, 0xc6, 0x17, 0x02, 0x6b, 0x8a, 0x21, 0xb0, 0xe6, 0x72, 0x7f, 0x58, 0xe3, - 0xbb, 0x73, 0x1f, 0xd7, 0x94, 0xc2, 0x70, 0xcd, 0x95, 0x01, 0xb8, 0xc6, 0xf7, 0x42, 0x01, 0x60, - 0x73, 0xa7, 0x03, 0xd8, 0x5c, 0x1a, 0x98, 0x9a, 0x13, 0x40, 0x36, 0x2b, 0xdd, 0xc8, 0xe6, 0xf9, - 0xbe, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0x3b, 0x1d, 0xd0, 0xe6, 0x52, 0x1f, 0x68, 0xe3, 0x57, - 0x80, 0x63, 0x1b, 0xad, 0x3f, 0xb6, 0x59, 0x1c, 0x16, 0xdb, 0x78, 0x82, 0x43, 0xc1, 0xcd, 0x66, - 0x38, 0xb8, 0xb9, 0x36, 0xe4, 0x8b, 0xf2, 0x2e, 0x74, 0x53, 0x0a, 0x43, 0x37, 0x57, 0x06, 0xa0, - 0x9b, 0xe0, 0x1c, 0xe2, 0xc1, 0x9b, 0x52, 0x18, 0xbc, 0xb9, 0x32, 0x00, 0xde, 0xf8, 0x92, 0x02, - 0xf8, 0xa6, 0xda, 0x0b, 0xdf, 0xbc, 0x30, 0x04, 0xbe, 0xf1, 0x83, 0x97, 0x0e, 0x80, 0xf3, 0x56, - 0x27, 0xc0, 0xc9, 0xf7, 0x03, 0x38, 0xfe, 0x88, 0x74, 0x11, 0xce, 0x66, 0x38, 0xc2, 0xb9, 0x36, - 0x10, 0xe1, 0x04, 0x9d, 0x64, 0x00, 0xe2, 0xac, 0x87, 0x42, 0x9c, 0xab, 0x83, 0x20, 0x8e, 0xef, - 0x24, 0x83, 0x18, 0xe7, 0xad, 0x4e, 0x8c, 0x93, 0xef, 0x87, 0x71, 0xfc, 0xc6, 0xb9, 0x20, 0xa7, - 0x14, 0x06, 0x72, 0xae, 0x0c, 0x00, 0x39, 0x7e, 0xe7, 0x05, 0x50, 0x8e, 0xd2, 0x17, 0xe5, 0xbc, - 0x34, 0x24, 0xca, 0xe9, 0x70, 0x5c, 0xed, 0x30, 0xa7, 0x14, 0x06, 0x73, 0xae, 0x0c, 0x80, 0x39, - 0x81, 0xca, 0xfa, 0x38, 0x67, 0x33, 0x1c, 0xe7, 0x5c, 0x1b, 0x88, 0x73, 0x3a, 0x46, 0x93, 0x0b, - 0x74, 0xd6, 0x43, 0x81, 0xce, 0xd5, 0x41, 0x40, 0xa7, 0x63, 0xe2, 0xe3, 0xc1, 0xc1, 0xff, 0x1b, - 0x1e, 0xe9, 0xbc, 0x7e, 0x7a, 0xa4, 0xe3, 0x3d, 0x33, 0x12, 0xa8, 0xb3, 0x96, 0x48, 0x5e, 0x14, - 0x9f, 0xcb, 0xff, 0x7c, 0x14, 0xc6, 0x4a, 0x5e, 0x3a, 0x8b, 0x5f, 0x4b, 0xe1, 0x59, 0x4e, 0x32, - 0x42, 0xab, 0x64, 0xc4, 0x52, 0xbf, 0x37, 0xf8, 0xd0, 0xba, 0xee, 0x03, 0xd5, 0x38, 0xeb, 0x33, - 0x6c, 0x24, 0x46, 0xaf, 0xc2, 0x44, 0xcb, 0xc6, 0x96, 0xdc, 0xb4, 0x74, 0xd3, 0xd2, 0x1d, 0xb6, - 0x29, 0x43, 0x58, 0x11, 0x3f, 0x3e, 0x99, 0xcf, 0x6c, 0xdb, 0xd8, 0xda, 0xe2, 0x74, 0x29, 0xd3, - 0x0a, 0x5c, 0xb9, 0x1f, 0x77, 0x1a, 0x1d, 0xfe, 0xe3, 0x4e, 0x0f, 0x41, 0xb4, 0xb0, 0xa2, 0xb5, - 0x45, 0x20, 0xec, 0xa4, 0xa0, 0x70, 0x9b, 0xa1, 0xfb, 0x9d, 0xdc, 0x92, 0xf4, 0xc4, 0xa0, 0x49, - 0xab, 0x9d, 0x88, 0x6e, 0xc2, 0x99, 0x86, 0x72, 0x44, 0x13, 0x17, 0x65, 0x37, 0xa8, 0xa3, 0xc9, - 0x88, 0xec, 0xbb, 0x49, 0xa8, 0xa1, 0x1c, 0xd1, 0x2f, 0x45, 0xb1, 0x5b, 0xf4, 0xd3, 0x0e, 0x57, - 0x20, 0xab, 0xe9, 0xb6, 0xa3, 0x1b, 0xaa, 0xc3, 0xcf, 0x88, 0x65, 0x87, 0xae, 0x4e, 0xb8, 0x54, - 0x76, 0x10, 0xec, 0x0d, 0x98, 0xe2, 0x79, 0xed, 0xfe, 0x07, 0xa3, 0x28, 0x7c, 0x49, 0x92, 0x5a, - 0x90, 0x1b, 0xde, 0x77, 0xa0, 0x50, 0x01, 0x26, 0x6b, 0x8a, 0x83, 0x0f, 0x95, 0x63, 0xd9, 0xdd, - 0x14, 0x95, 0xa6, 0x47, 0x2c, 0x5e, 0x78, 0x7a, 0x32, 0x3f, 0x71, 0x8f, 0xdd, 0xea, 0xda, 0x1b, - 0x35, 0x51, 0x0b, 0xdc, 0xd0, 0xd0, 0x35, 0x98, 0x54, 0xec, 0x63, 0x43, 0xa5, 0xea, 0xc1, 0x86, - 0xdd, 0xb2, 0x29, 0xa4, 0x48, 0x4a, 0x59, 0x4a, 0x2e, 0xb8, 0x54, 0x74, 0x09, 0x32, 0x3c, 0xe9, - 0x9b, 0x7d, 0x6e, 0x66, 0x92, 0x36, 0x95, 0x7f, 0xfd, 0x80, 0x7e, 0x71, 0x06, 0xdd, 0x81, 0x59, - 0x7e, 0x2a, 0xfc, 0xa1, 0x62, 0x69, 0x32, 0xd5, 0xba, 0x6f, 0x9f, 0x22, 0x15, 0x7b, 0x8e, 0x9d, - 0x02, 0x4f, 0x0a, 0x10, 0x55, 0x07, 0x8f, 0x50, 0x1d, 0x17, 0x93, 0x6b, 0x89, 0x64, 0x46, 0x9c, - 0x58, 0x4b, 0x24, 0xb3, 0xe2, 0x64, 0xfe, 0x37, 0x05, 0xc8, 0xb4, 0x6d, 0x24, 0xb9, 0xd3, 0xf1, - 0x1e, 0xf7, 0x7c, 0x38, 0x74, 0xea, 0x95, 0xfa, 0x95, 0xe4, 0x5d, 0xe5, 0x26, 0xbe, 0xcd, 0xf7, - 0x0e, 0xbd, 0xe9, 0x42, 0x82, 0x9b, 0x3c, 0xe0, 0xb2, 0xbd, 0x91, 0xf8, 0xed, 0x0f, 0xe6, 0x47, - 0xf2, 0x1f, 0x26, 0x60, 0xa2, 0x7d, 0xc3, 0x48, 0xb9, 0xa3, 0x5e, 0x61, 0xae, 0xad, 0x8d, 0x63, - 0xb1, 0xcf, 0xb9, 0x79, 0x29, 0xff, 0x58, 0x77, 0x56, 0xcd, 0x85, 0x3e, 0x6f, 0xab, 0x83, 0xf5, - 0xf4, 0x19, 0x67, 0xff, 0x7f, 0xdc, 0x73, 0x11, 0x8b, 0x30, 0x4a, 0x4f, 0x72, 0xe1, 0x55, 0x0b, - 0xdb, 0x8b, 0x5c, 0x24, 0xf7, 0x25, 0x56, 0x8c, 0xb8, 0x94, 0xea, 0x33, 0x1d, 0x8e, 0xe6, 0x9f, - 0x43, 0x71, 0xfa, 0xef, 0xaf, 0xf1, 0x23, 0xf2, 0x46, 0x4f, 0x77, 0x44, 0x1e, 0x7b, 0x29, 0x5d, - 0xaf, 0x33, 0x77, 0xcd, 0x06, 0xd5, 0x58, 0xd7, 0x86, 0x5f, 0x2a, 0x82, 0x7f, 0x16, 0x6f, 0x51, - 0xe2, 0x9f, 0xc5, 0x0b, 0xe4, 0x22, 0x66, 0x3d, 0x11, 0x6c, 0x04, 0x16, 0xdc, 0x59, 0x9a, 0x7d, - 0xab, 0x6d, 0x7c, 0xe8, 0x6f, 0xb5, 0xb1, 0x09, 0x9a, 0x7e, 0xa1, 0x8d, 0xa5, 0xbd, 0x72, 0x7b, - 0xf9, 0x96, 0x00, 0x22, 0x2d, 0x7b, 0x17, 0x63, 0x2d, 0x12, 0x53, 0x76, 0x73, 0x2d, 0x63, 0xc3, - 0x27, 0xb3, 0xb7, 0x9d, 0xd5, 0x1f, 0x6f, 0x3f, 0xab, 0x3f, 0xff, 0x81, 0x00, 0x59, 0xaf, 0x86, - 0xec, 0x2b, 0x55, 0x7d, 0x8e, 0xcf, 0x7b, 0xb6, 0x6f, 0x33, 0xb9, 0x27, 0x02, 0x0c, 0xf5, 0xe1, - 0xac, 0xe0, 0x89, 0x00, 0xec, 0x3b, 0x42, 0xbf, 0x2b, 0xc0, 0xb4, 0x57, 0xc5, 0x82, 0xbf, 0xdb, - 0xfb, 0x19, 0xf2, 0xfa, 0x25, 0xfa, 0xb1, 0x3f, 0xb3, 0x7e, 0xc0, 0x8e, 0x62, 0x18, 0xca, 0xc6, - 0x11, 0xcf, 0xe0, 0x00, 0xbe, 0xfa, 0xa0, 0x55, 0x2b, 0xf4, 0x33, 0x80, 0xec, 0xbf, 0x9d, 0xbf, - 0x1b, 0x50, 0x20, 0x1d, 0x4e, 0x44, 0x4b, 0x43, 0x8d, 0x3b, 0x57, 0x4b, 0xb4, 0x70, 0xfe, 0x07, - 0xc1, 0x9e, 0x28, 0x1e, 0x90, 0xa8, 0xf3, 0x36, 0xc4, 0x0f, 0x94, 0x7a, 0xbf, 0xcc, 0x95, 0xb6, - 0x9e, 0x93, 0x48, 0x69, 0x74, 0xb7, 0x6d, 0x93, 0x7c, 0xac, 0x77, 0x84, 0xd4, 0xad, 0xd2, 0xb6, - 0xcd, 0xf4, 0xaf, 0xb9, 0xad, 0x88, 0x0f, 0x7e, 0x7c, 0xd0, 0x8d, 0xbc, 0x91, 0xf8, 0xe8, 0x83, - 0x79, 0xe1, 0x46, 0x05, 0xa6, 0x43, 0xe6, 0x53, 0x94, 0x05, 0x08, 0x9c, 0xe0, 0xcf, 0xbf, 0x1c, - 0xb8, 0xbc, 0x2a, 0x6f, 0x6f, 0x16, 0x1e, 0x6c, 0x6c, 0x94, 0xab, 0xd5, 0xe2, 0xaa, 0x28, 0x20, - 0x11, 0x32, 0x6d, 0xe7, 0xff, 0xc7, 0xd8, 0xb7, 0x04, 0x6f, 0xfc, 0x2f, 0x00, 0xff, 0xb3, 0x22, - 0x44, 0xd6, 0x7a, 0xf1, 0x1d, 0xf9, 0xd1, 0xf2, 0xfd, 0xed, 0x62, 0x45, 0x1c, 0x41, 0x08, 0xb2, - 0x2b, 0xcb, 0xd5, 0x42, 0x49, 0x96, 0x8a, 0x95, 0xad, 0x07, 0x9b, 0x95, 0xa2, 0xfb, 0x0d, 0xc2, - 0x1b, 0xab, 0x90, 0x09, 0x1e, 0x27, 0x80, 0xa6, 0x61, 0xb2, 0x50, 0x2a, 0x16, 0xd6, 0xe5, 0x47, - 0xe5, 0x65, 0xf9, 0xe1, 0x76, 0x71, 0xbb, 0x28, 0x8e, 0xd0, 0xaa, 0x51, 0xe2, 0xdd, 0xed, 0xfb, - 0xf7, 0x45, 0x01, 0x4d, 0x42, 0x9a, 0x5d, 0xd3, 0x6f, 0x05, 0x88, 0xb1, 0x1b, 0x1b, 0x90, 0x0e, - 0x1c, 0x26, 0x48, 0x1e, 0xb7, 0xb5, 0x5d, 0x29, 0xc9, 0xd5, 0xf2, 0x46, 0xb1, 0x52, 0x5d, 0xde, - 0xd8, 0x62, 0x32, 0x28, 0x6d, 0x79, 0xe5, 0x81, 0x54, 0x15, 0x05, 0xef, 0xba, 0xfa, 0x60, 0xbb, - 0x50, 0x72, 0x9b, 0x91, 0x4f, 0x24, 0xe3, 0x62, 0xfc, 0xc6, 0xd7, 0x04, 0x38, 0xd7, 0x63, 0x53, - 0x3d, 0x4a, 0xc3, 0xf8, 0xb6, 0x41, 0x4f, 0x53, 0x13, 0x47, 0xd0, 0x44, 0x60, 0x5f, 0xbd, 0x28, - 0xa0, 0x24, 0xdb, 0xd3, 0x2c, 0xc6, 0xd0, 0x18, 0xc4, 0x2a, 0xb7, 0xc5, 0x38, 0xa9, 0x69, 0x60, - 0x5b, 0xba, 0x98, 0x40, 0x29, 0xbe, 0xab, 0x56, 0x1c, 0x45, 0x19, 0x7f, 0x5b, 0xab, 0x38, 0x46, - 0x44, 0x79, 0x1b, 0x43, 0xc5, 0xf1, 0x1b, 0x97, 0x20, 0xb0, 0xc9, 0x0e, 0x01, 0x8c, 0xdd, 0x57, - 0x1c, 0x6c, 0x3b, 0xe2, 0x08, 0x1a, 0x87, 0xf8, 0x72, 0xbd, 0x2e, 0x0a, 0xb7, 0xfe, 0x58, 0x80, - 0xa4, 0x7b, 0x1a, 0x3e, 0xba, 0x0f, 0xa3, 0x6c, 0x29, 0x61, 0xbe, 0xf7, 0x34, 0x47, 0x9d, 0xdc, - 0xec, 0xc2, 0xa0, 0x79, 0x30, 0x3f, 0x82, 0xde, 0x86, 0x94, 0x67, 0x41, 0xe8, 0xf9, 0x7e, 0xf6, - 0xe5, 0x4a, 0xed, 0x6f, 0x84, 0x64, 0xcc, 0xe4, 0x47, 0x5e, 0x16, 0x56, 0x5e, 0xf8, 0xe8, 0xa7, - 0x73, 0x23, 0x1f, 0x3d, 0x9d, 0x13, 0x7e, 0xf8, 0x74, 0x4e, 0xf8, 0xf1, 0xd3, 0x39, 0xe1, 0x27, - 0x4f, 0xe7, 0x84, 0xdf, 0xf8, 0xd9, 0xdc, 0xc8, 0x0f, 0x7f, 0x36, 0x37, 0xf2, 0xe3, 0x9f, 0xcd, - 0x8d, 0xbc, 0x3b, 0xce, 0xb9, 0x77, 0xc6, 0xe8, 0x97, 0x50, 0x6f, 0xff, 0x57, 0x00, 0x00, 0x00, - 0xff, 0xff, 0x96, 0xe9, 0x9b, 0x5b, 0x2c, 0x76, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_71616fea36bb2a95) } + +var fileDescriptor_api_71616fea36bb2a95 = []byte{ + // 7468 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6d, 0x68, 0x24, 0xc9, + 0x79, 0xbf, 0x7a, 0x66, 0x34, 0x9a, 0x79, 0x46, 0x1a, 0xb5, 0x4a, 0xda, 0xdd, 0x59, 0xed, 0x9e, + 0xa4, 0x9d, 0xdb, 0xb7, 0x5b, 0xdf, 0x49, 0xb7, 0xbb, 0x77, 0xff, 0x3b, 0xdf, 0x9e, 0xcf, 0x96, + 0x46, 0xb3, 0x3b, 0x23, 0xad, 0xb4, 0xda, 0x9e, 0xd1, 0x9e, 0xef, 0xec, 0xa3, 0xdd, 0xea, 0x2e, + 0x8d, 0xda, 0x9a, 0xe9, 0x9e, 0xed, 0xee, 0xd1, 0xcb, 0xc2, 0x1f, 0xfc, 0x7f, 0x01, 0xff, 0x31, + 0x7f, 0x8e, 0x40, 0x42, 0x08, 0x71, 0x82, 0x0f, 0x1c, 0x70, 0xc0, 0x5c, 0x48, 0xf2, 0x2d, 0xc1, + 0x21, 0xf9, 0x90, 0xc0, 0x61, 0x1c, 0x30, 0x81, 0xc4, 0x26, 0x10, 0x61, 0xaf, 0xc1, 0x04, 0x7f, + 0x08, 0x24, 0x1f, 0x12, 0x38, 0x48, 0x08, 0xf5, 0xd2, 0x2f, 0x33, 0xd3, 0xf3, 0xa2, 0x75, 0x5f, + 0x72, 0xe0, 0x2f, 0xc3, 0xf4, 0x53, 0xf5, 0x3c, 0x5d, 0xf5, 0x54, 0xd5, 0x53, 0xcf, 0xaf, 0xea, + 0xa9, 0x6a, 0x98, 0xb2, 0x4c, 0x45, 0xdd, 0x6b, 0xee, 0x2c, 0x29, 0x4d, 0x7d, 0xb1, 0x69, 0x99, + 0x8e, 0x89, 0xa6, 0x54, 0x53, 0xdd, 0xa7, 0xe4, 0x45, 0x9e, 0x38, 0x7b, 0x63, 0xff, 0x60, 0x69, + 0xff, 0xc0, 0xc6, 0xd6, 0x01, 0xb6, 0x96, 0x54, 0xd3, 0x50, 0x5b, 0x96, 0x85, 0x0d, 0xf5, 0x78, + 0xa9, 0x6e, 0xaa, 0xfb, 0xf4, 0x47, 0x37, 0x6a, 0x8c, 0x7d, 0x16, 0xb9, 0x12, 0x35, 0xc5, 0x51, + 0x38, 0x6d, 0xc6, 0xa5, 0x61, 0xcb, 0x32, 0x2d, 0x9b, 0x53, 0xcf, 0xba, 0xd4, 0x06, 0x76, 0x94, + 0x40, 0xee, 0x0b, 0xb6, 0x63, 0x5a, 0x4a, 0x0d, 0x2f, 0x61, 0xa3, 0xa6, 0x1b, 0x98, 0x64, 0x38, + 0x50, 0x55, 0x9e, 0x78, 0x31, 0x34, 0xf1, 0x36, 0x4f, 0xcd, 0xb5, 0x1c, 0xbd, 0xbe, 0xb4, 0x57, + 0x57, 0x97, 0x1c, 0xbd, 0x81, 0x6d, 0x47, 0x69, 0x34, 0x79, 0xca, 0x02, 0x4d, 0x71, 0x2c, 0x45, + 0xd5, 0x8d, 0xda, 0x92, 0x85, 0x55, 0xd3, 0xd2, 0xb0, 0x26, 0xdb, 0x4d, 0xc5, 0x70, 0x0b, 0x59, + 0x33, 0x6b, 0x26, 0xfd, 0xbb, 0x44, 0xfe, 0x31, 0x6a, 0xfe, 0x7b, 0x02, 0x4c, 0x48, 0xf8, 0x71, + 0x0b, 0xdb, 0x4e, 0x09, 0x2b, 0x1a, 0xb6, 0xd0, 0x79, 0x88, 0xef, 0xe3, 0xe3, 0x5c, 0x7c, 0x41, + 0xb8, 0x3e, 0xbe, 0x32, 0xf6, 0xf1, 0xc9, 0x7c, 0x7c, 0x1d, 0x1f, 0x4b, 0x84, 0x86, 0x16, 0x60, + 0x0c, 0x1b, 0x9a, 0x4c, 0x92, 0x13, 0xed, 0xc9, 0x49, 0x6c, 0x68, 0xeb, 0xf8, 0x18, 0x7d, 0x19, + 0x52, 0x36, 0x91, 0x66, 0xa8, 0x38, 0x37, 0xba, 0x20, 0x5c, 0x1f, 0x5d, 0xf9, 0xc2, 0xc7, 0x27, + 0xf3, 0x6f, 0xd6, 0x74, 0x67, 0xaf, 0xb5, 0xb3, 0xa8, 0x9a, 0x8d, 0x25, 0x4f, 0xfb, 0xda, 0x8e, + 0xff, 0x7f, 0xa9, 0xb9, 0x5f, 0x5b, 0xea, 0xac, 0xf9, 0x62, 0xf5, 0xc8, 0xa8, 0xe0, 0xc7, 0x92, + 0x27, 0xf1, 0x8d, 0xc4, 0x3f, 0x7e, 0x30, 0x2f, 0xac, 0x25, 0x52, 0x82, 0x18, 0x5b, 0x4b, 0xa4, + 0x62, 0x62, 0x3c, 0xff, 0xed, 0x38, 0x64, 0x25, 0x6c, 0x37, 0x4d, 0xc3, 0xc6, 0xbc, 0xfc, 0x2f, + 0x43, 0xdc, 0x39, 0x32, 0x68, 0xf9, 0x33, 0xb7, 0xe6, 0x16, 0xbb, 0x5a, 0x7b, 0xb1, 0x6a, 0x29, + 0x86, 0xad, 0xa8, 0x8e, 0x6e, 0x1a, 0x12, 0xc9, 0x8a, 0x5e, 0x87, 0x8c, 0x85, 0xed, 0x56, 0x03, + 0x53, 0x75, 0xd1, 0xaa, 0x65, 0x6e, 0x9d, 0x0b, 0xe1, 0xac, 0x34, 0x15, 0x43, 0x02, 0x96, 0x97, + 0xfc, 0x47, 0xe7, 0x21, 0x65, 0xb4, 0x1a, 0x44, 0x21, 0x36, 0xad, 0x6e, 0x5c, 0x1a, 0x33, 0x5a, + 0x8d, 0x75, 0x7c, 0x6c, 0xa3, 0x2f, 0xc2, 0x59, 0x0d, 0x37, 0x2d, 0xac, 0x2a, 0x0e, 0xd6, 0x64, + 0x4b, 0x31, 0x6a, 0x58, 0xd6, 0x8d, 0x5d, 0xd3, 0xce, 0x25, 0x17, 0xe2, 0xd7, 0x33, 0xb7, 0x2e, + 0x86, 0xc8, 0x97, 0x48, 0xae, 0xb2, 0xb1, 0x6b, 0xae, 0x24, 0x3e, 0x3a, 0x99, 0x1f, 0x91, 0x66, + 0x7c, 0x09, 0x5e, 0x92, 0x8d, 0x2a, 0x30, 0xc1, 0x8b, 0x6b, 0x61, 0xc5, 0x36, 0x8d, 0xdc, 0xd8, + 0x82, 0x70, 0x3d, 0x7b, 0x6b, 0x31, 0x4c, 0x60, 0x9b, 0x6a, 0xc8, 0x63, 0xab, 0x81, 0x25, 0xca, + 0x25, 0x8d, 0x5b, 0x81, 0x27, 0x74, 0x01, 0xd2, 0xa4, 0x26, 0x3b, 0xc7, 0x0e, 0xb6, 0x73, 0x29, + 0x5a, 0x15, 0x52, 0xb5, 0x15, 0xf2, 0x9c, 0x7f, 0x0b, 0xc6, 0x83, 0xac, 0x08, 0x41, 0x56, 0x2a, + 0x56, 0xb6, 0x37, 0x8a, 0xf2, 0xf6, 0xe6, 0xfa, 0xe6, 0x83, 0xb7, 0x37, 0xc5, 0x11, 0x34, 0x03, + 0x22, 0xa7, 0xad, 0x17, 0xdf, 0x91, 0xef, 0x97, 0x37, 0xca, 0x55, 0x51, 0x98, 0x4d, 0xfc, 0xbf, + 0x6f, 0xcf, 0x8d, 0xe4, 0x1f, 0x01, 0xdc, 0xc3, 0x0e, 0xef, 0x66, 0x68, 0x05, 0x92, 0x7b, 0xb4, + 0x3c, 0x39, 0x81, 0x6a, 0x7a, 0x21, 0xb4, 0xe0, 0x81, 0x2e, 0xb9, 0x92, 0x22, 0xda, 0xf8, 0xe1, + 0xc9, 0xbc, 0x20, 0x71, 0x4e, 0xd6, 0x13, 0xf2, 0x7f, 0x2e, 0x40, 0x86, 0x0a, 0x66, 0xb5, 0x44, + 0x85, 0x0e, 0xc9, 0x97, 0x06, 0xaa, 0xa4, 0x5b, 0x34, 0x5a, 0x84, 0xd1, 0x03, 0xa5, 0xde, 0xc2, + 0xb9, 0x18, 0x95, 0x91, 0x0b, 0x91, 0xf1, 0x88, 0xa4, 0x4b, 0x2c, 0x1b, 0xba, 0x03, 0xe3, 0xba, + 0xe1, 0x60, 0xc3, 0x91, 0x19, 0x5b, 0x7c, 0x00, 0x5b, 0x86, 0xe5, 0xa6, 0x0f, 0xf9, 0x3f, 0x15, + 0x00, 0xb6, 0x5a, 0x51, 0xaa, 0x06, 0xbd, 0x32, 0x64, 0xf9, 0x79, 0x1f, 0xe3, 0xb5, 0x38, 0x0b, + 0x49, 0xdd, 0xa8, 0xeb, 0x06, 0x2b, 0x7f, 0x4a, 0xe2, 0x4f, 0x68, 0x06, 0x46, 0x77, 0xea, 0xba, + 0xa1, 0xd1, 0x51, 0x91, 0x92, 0xd8, 0x03, 0x57, 0xbf, 0x04, 0x19, 0x5a, 0xf6, 0x08, 0xb5, 0x9f, + 0xff, 0x41, 0x0c, 0xce, 0x14, 0x4c, 0x43, 0xd3, 0xc9, 0xf0, 0x54, 0xea, 0x9f, 0x0a, 0xdd, 0xac, + 0x41, 0x60, 0x20, 0xca, 0xf8, 0xa8, 0x39, 0x64, 0x4b, 0x23, 0x9f, 0xab, 0x78, 0xd4, 0xa4, 0xb4, + 0x70, 0x7d, 0xa2, 0x57, 0xe0, 0x9c, 0x52, 0xaf, 0x9b, 0x87, 0xb2, 0xbe, 0x2b, 0x6b, 0x26, 0xb6, + 0x65, 0xc3, 0x74, 0x64, 0x7c, 0xa4, 0xdb, 0x0e, 0x35, 0x2b, 0x29, 0x69, 0x9a, 0x26, 0x97, 0x77, + 0x57, 0x4d, 0x6c, 0x6f, 0x9a, 0x4e, 0x91, 0x24, 0x91, 0x31, 0x4b, 0x0a, 0xc3, 0xc6, 0x6c, 0x92, + 0x18, 0x64, 0x29, 0x85, 0x8f, 0x9a, 0x74, 0xcc, 0xf2, 0x26, 0x7a, 0x0f, 0xce, 0x76, 0x6a, 0x33, + 0xca, 0xd6, 0xfa, 0x1b, 0x01, 0xb2, 0x65, 0x43, 0x77, 0x3e, 0x15, 0xcd, 0xe4, 0xa9, 0x36, 0x1e, + 0x54, 0xed, 0x0d, 0x10, 0x77, 0x15, 0xbd, 0xfe, 0xc0, 0xa8, 0x9a, 0x8d, 0x1d, 0xdb, 0x31, 0x0d, + 0x6c, 0x73, 0xdd, 0x77, 0xd1, 0xb9, 0xce, 0x1e, 0xc1, 0xa4, 0x57, 0xa7, 0x28, 0x95, 0xf5, 0x04, + 0xc4, 0xb2, 0xa1, 0x5a, 0xb8, 0x81, 0x8d, 0x48, 0xb5, 0x75, 0x11, 0xd2, 0xba, 0x2b, 0x97, 0x6a, + 0x2c, 0x2e, 0xf9, 0x04, 0x5e, 0xa7, 0x16, 0x4c, 0x05, 0xde, 0x1d, 0xa5, 0xb9, 0x24, 0x13, 0x07, + 0x3e, 0x94, 0xfd, 0xf6, 0x22, 0x13, 0x07, 0x3e, 0x64, 0xe6, 0xed, 0x1d, 0x98, 0x58, 0xc5, 0x75, + 0xec, 0xe0, 0xe8, 0x6d, 0xff, 0x36, 0x64, 0x5d, 0xd1, 0x51, 0x36, 0xd2, 0xef, 0x0a, 0x80, 0xb8, + 0x5c, 0x32, 0xe3, 0x46, 0xd9, 0x4e, 0xf3, 0xc4, 0xcd, 0x70, 0x5a, 0x96, 0xc1, 0xfc, 0x05, 0xd6, + 0x4b, 0x81, 0x91, 0xa8, 0xcb, 0xe0, 0xdb, 0xe0, 0x44, 0xd0, 0x06, 0x7b, 0x6e, 0x0f, 0x71, 0x78, + 0x0e, 0x61, 0xba, 0xad, 0x78, 0xd1, 0x36, 0x65, 0x82, 0x96, 0x2c, 0xb6, 0x10, 0x0f, 0xfa, 0x76, + 0x94, 0x98, 0x7f, 0x0f, 0xa6, 0x0a, 0x75, 0xac, 0x58, 0x51, 0xab, 0x85, 0x37, 0xe7, 0x3b, 0x80, + 0x82, 0xe2, 0xa3, 0x6c, 0xd2, 0xdf, 0x13, 0x00, 0x49, 0xf8, 0x00, 0x5b, 0x4e, 0xe4, 0x4d, 0xba, + 0x0a, 0x19, 0x47, 0xb1, 0x6a, 0xd8, 0x91, 0x89, 0x3f, 0xce, 0xcd, 0xd5, 0x73, 0x01, 0x41, 0xc4, + 0x2b, 0x5f, 0xdc, 0xab, 0xab, 0x8b, 0x55, 0xd7, 0x5f, 0xe7, 0x36, 0x0b, 0x18, 0x1f, 0x21, 0x73, + 0x0d, 0xbc, 0x0b, 0xd3, 0x6d, 0xa5, 0x8c, 0x52, 0x05, 0xff, 0x2a, 0x40, 0xa6, 0xa2, 0x2a, 0x46, + 0x94, 0x75, 0x7f, 0x0b, 0x32, 0xb6, 0xaa, 0x18, 0xf2, 0xae, 0x69, 0x35, 0x14, 0x87, 0x76, 0xd9, + 0x6c, 0x5b, 0xdd, 0x3d, 0xaf, 0x59, 0x55, 0x8c, 0xbb, 0x34, 0x93, 0x04, 0xb6, 0xf7, 0x1f, 0x3d, + 0x84, 0xcc, 0x3e, 0x3e, 0x96, 0x39, 0xba, 0xa2, 0xf3, 0x5c, 0xf6, 0xd6, 0xcb, 0x01, 0xfe, 0xfd, + 0x83, 0x45, 0x17, 0x94, 0x2d, 0x06, 0x40, 0xd9, 0x22, 0xe1, 0x58, 0xac, 0x38, 0x16, 0x36, 0x6a, + 0xce, 0x9e, 0x04, 0xfb, 0xf8, 0xf8, 0x3e, 0x93, 0x11, 0x1c, 0x28, 0x6b, 0x89, 0x54, 0x5c, 0x4c, + 0xe4, 0xff, 0x4d, 0x80, 0x71, 0x56, 0xf1, 0x28, 0x07, 0xca, 0xab, 0x90, 0xb0, 0xcc, 0x43, 0x36, + 0x50, 0x32, 0xb7, 0x2e, 0x84, 0x88, 0x58, 0xc7, 0xc7, 0xc1, 0x19, 0x8a, 0x66, 0x47, 0x2b, 0xc0, + 0x7d, 0x3f, 0x99, 0x72, 0xc7, 0x87, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0xd7, 0x60, 0x72, 0x47, + 0x71, 0xd4, 0x3d, 0xd9, 0xe2, 0x85, 0x24, 0xb3, 0x59, 0xfc, 0xfa, 0xb8, 0x94, 0xa5, 0x64, 0xb7, + 0xe8, 0x76, 0xfe, 0xdf, 0xdd, 0x5e, 0x6f, 0xe3, 0x5f, 0xc9, 0x96, 0xff, 0x0f, 0x81, 0x8f, 0x27, + 0xb7, 0xfe, 0xbf, 0x6a, 0x1d, 0xe0, 0x5b, 0x31, 0x38, 0x57, 0xd8, 0xc3, 0xea, 0x7e, 0xc1, 0x34, + 0x6c, 0xdd, 0x76, 0x88, 0x06, 0xa3, 0xec, 0x05, 0x17, 0x20, 0x7d, 0xa8, 0x3b, 0x7b, 0xb2, 0xa6, + 0xef, 0xee, 0x52, 0xcb, 0x97, 0x92, 0x52, 0x84, 0xb0, 0xaa, 0xef, 0xee, 0xa2, 0xdb, 0x90, 0x68, + 0x98, 0x1a, 0x73, 0x91, 0xb3, 0xb7, 0xe6, 0x43, 0xc4, 0xd3, 0xa2, 0xd9, 0xad, 0xc6, 0x86, 0xa9, + 0x61, 0x89, 0x66, 0x46, 0x73, 0x00, 0x2a, 0xa1, 0x36, 0x4d, 0xdd, 0x70, 0xf8, 0x1c, 0x18, 0xa0, + 0xa0, 0x12, 0xa4, 0x1d, 0x6c, 0x35, 0x74, 0x43, 0x71, 0x70, 0x6e, 0x94, 0x2a, 0xef, 0x72, 0x68, + 0xc1, 0x9b, 0x75, 0x5d, 0x55, 0x56, 0xb1, 0xad, 0x5a, 0x7a, 0xd3, 0x31, 0x2d, 0xae, 0x45, 0x9f, + 0x99, 0x5b, 0xdc, 0xf7, 0x13, 0x90, 0xeb, 0xd6, 0x50, 0x94, 0xfd, 0x64, 0x0b, 0x92, 0x04, 0x65, + 0xd7, 0x1d, 0xde, 0x53, 0x6e, 0xf5, 0x52, 0x44, 0x48, 0x09, 0x28, 0x5a, 0xaf, 0x3b, 0xbc, 0xf0, + 0x5c, 0xce, 0xec, 0xf7, 0x04, 0x48, 0xb2, 0x04, 0x74, 0x13, 0x52, 0x7c, 0x59, 0x41, 0xa3, 0x65, + 0x8c, 0xaf, 0x9c, 0x7d, 0x7a, 0x32, 0x3f, 0xc6, 0x56, 0x0a, 0x56, 0x3f, 0xf6, 0xff, 0x4a, 0x63, + 0x34, 0x5f, 0x59, 0x23, 0x6d, 0x66, 0x3b, 0x8a, 0xe5, 0xd0, 0x25, 0x9c, 0x18, 0x43, 0x0c, 0x94, + 0xb0, 0x8e, 0x8f, 0xd1, 0x1a, 0x24, 0x6d, 0x47, 0x71, 0x5a, 0x36, 0x6f, 0xb5, 0x53, 0x15, 0xb6, + 0x42, 0x39, 0x25, 0x2e, 0x81, 0xb8, 0x32, 0x1a, 0x76, 0x14, 0xbd, 0x4e, 0x9b, 0x31, 0x2d, 0xf1, + 0xa7, 0xfc, 0x37, 0x05, 0x48, 0xb2, 0xac, 0xe8, 0x1c, 0x4c, 0x4b, 0xcb, 0x9b, 0xf7, 0x8a, 0x72, + 0x79, 0x73, 0xb5, 0x58, 0x2d, 0x4a, 0x1b, 0xe5, 0xcd, 0xe5, 0x6a, 0x51, 0x1c, 0x41, 0x67, 0x01, + 0xb9, 0x09, 0x85, 0x07, 0x9b, 0x95, 0x72, 0xa5, 0x5a, 0xdc, 0xac, 0x8a, 0x02, 0x5d, 0x61, 0xa0, + 0xf4, 0x00, 0x35, 0x86, 0x2e, 0xc3, 0x42, 0x27, 0x55, 0xae, 0x54, 0x97, 0xab, 0x15, 0xb9, 0x58, + 0xa9, 0x96, 0x37, 0x96, 0xab, 0xc5, 0x55, 0x31, 0xde, 0x27, 0x17, 0x79, 0x89, 0x24, 0x15, 0x0b, + 0x55, 0x31, 0x91, 0x7f, 0x02, 0x67, 0x24, 0xac, 0x9a, 0x8d, 0x66, 0xcb, 0xc1, 0xa4, 0x94, 0x76, + 0x94, 0xe3, 0xe5, 0x1c, 0x8c, 0x69, 0xd6, 0xb1, 0x6c, 0xb5, 0x0c, 0x3e, 0x5a, 0x92, 0x9a, 0x75, + 0x2c, 0xb5, 0x0c, 0xde, 0x19, 0xff, 0x50, 0x80, 0xb3, 0x9d, 0x2f, 0x8f, 0xb2, 0x2b, 0x3e, 0x84, + 0x8c, 0xa2, 0x69, 0x58, 0x93, 0x35, 0x5c, 0x77, 0x14, 0xee, 0xaa, 0xdc, 0x08, 0x48, 0xe2, 0xcb, + 0x6f, 0x8b, 0xde, 0xf2, 0xdb, 0xc6, 0xa3, 0x42, 0x81, 0x16, 0x64, 0x95, 0x70, 0xb8, 0xa6, 0x88, + 0x0a, 0xa1, 0x94, 0xfc, 0x1f, 0x27, 0x60, 0xa2, 0x68, 0x68, 0xd5, 0xa3, 0x48, 0x67, 0x97, 0xb3, + 0x90, 0x54, 0xcd, 0x46, 0x43, 0x77, 0x5c, 0x35, 0xb1, 0x27, 0xf4, 0x59, 0x48, 0x69, 0x58, 0xd1, + 0xbc, 0x35, 0x8a, 0x41, 0x8e, 0x96, 0xe4, 0x65, 0x47, 0x5f, 0x81, 0x73, 0xc4, 0x82, 0x5a, 0x86, + 0x52, 0x97, 0x99, 0x34, 0xd9, 0xb1, 0xf4, 0x5a, 0x0d, 0x5b, 0x7c, 0xb1, 0xef, 0x7a, 0x48, 0x39, + 0xcb, 0x9c, 0xa3, 0x40, 0x19, 0xaa, 0x2c, 0xbf, 0x74, 0x46, 0x0f, 0x23, 0xa3, 0x37, 0x01, 0xc8, + 0xe4, 0x44, 0x17, 0x10, 0x6d, 0x6e, 0x9b, 0x7a, 0xad, 0x20, 0xba, 0xe6, 0x88, 0x30, 0x90, 0x67, + 0x1b, 0x2d, 0x11, 0x64, 0xf0, 0xb8, 0xa5, 0x5b, 0x58, 0xbe, 0xd9, 0x54, 0x29, 0x94, 0x4f, 0xad, + 0x64, 0x9f, 0x9e, 0xcc, 0x83, 0xc4, 0xc8, 0x37, 0xb7, 0x0a, 0x04, 0x29, 0xb0, 0xff, 0x4d, 0x15, + 0xad, 0xc0, 0x1c, 0x99, 0x80, 0x79, 0x5d, 0x14, 0x47, 0xde, 0xd3, 0x6b, 0x7b, 0xd8, 0x92, 0xbd, + 0x55, 0x61, 0xba, 0x84, 0x97, 0x92, 0x66, 0x55, 0xc5, 0x60, 0x05, 0x5d, 0x76, 0x4a, 0x34, 0x8b, + 0xa7, 0x1e, 0xa2, 0xe7, 0xa6, 0xa9, 0xdb, 0xa6, 0x91, 0x4b, 0x33, 0x3d, 0xb3, 0x27, 0xf4, 0x10, + 0x44, 0xdd, 0x90, 0x77, 0xeb, 0x7a, 0x6d, 0xcf, 0x91, 0x0f, 0x2d, 0xdd, 0xc1, 0x76, 0x6e, 0x8a, + 0x56, 0x28, 0xac, 0xdf, 0x55, 0xf8, 0xda, 0xac, 0xf6, 0x36, 0xc9, 0xc9, 0xab, 0x96, 0xd5, 0x8d, + 0xbb, 0x94, 0x9f, 0x12, 0x6d, 0x6f, 0x76, 0x1e, 0x13, 0x53, 0xf9, 0x7f, 0x10, 0x20, 0xeb, 0x76, + 0x9a, 0x28, 0xfb, 0xf7, 0x75, 0x10, 0x4d, 0x03, 0xcb, 0xcd, 0x3d, 0xc5, 0xc6, 0x5c, 0x31, 0x7c, + 0x0a, 0xc9, 0x9a, 0x06, 0xde, 0x22, 0x64, 0xa6, 0x09, 0xb4, 0x05, 0x53, 0xb6, 0xa3, 0xd4, 0x74, + 0xa3, 0x16, 0xd0, 0xd7, 0xe8, 0xf0, 0xae, 0xbb, 0xc8, 0xb9, 0x3d, 0x7a, 0x9b, 0xdf, 0xf1, 0x23, + 0x01, 0xa6, 0x96, 0xb5, 0x86, 0x6e, 0x54, 0x9a, 0x75, 0x3d, 0x52, 0x9c, 0x7f, 0x19, 0xd2, 0x36, + 0x91, 0xe9, 0x1b, 0x6f, 0x1f, 0xa3, 0xa5, 0x68, 0x0a, 0xb1, 0xe2, 0xf7, 0x61, 0x12, 0x1f, 0x35, + 0x75, 0x4b, 0x71, 0x74, 0xd3, 0x60, 0xb0, 0x24, 0x31, 0x7c, 0xdd, 0xb2, 0x3e, 0xaf, 0x0f, 0x4d, + 0x78, 0xcd, 0xde, 0x01, 0x14, 0xac, 0x58, 0x94, 0xf8, 0x44, 0x86, 0x69, 0x2a, 0x7a, 0xdb, 0xb0, + 0x23, 0xd6, 0x1a, 0xb7, 0xae, 0x5f, 0x82, 0x99, 0xf6, 0x17, 0x44, 0x59, 0xfa, 0xf7, 0x78, 0x8b, + 0x6f, 0x60, 0xeb, 0x13, 0x82, 0xc6, 0x41, 0xf1, 0x51, 0x96, 0xfc, 0x1b, 0x02, 0x9c, 0xa7, 0xb2, + 0xe9, 0x9e, 0xc8, 0x2e, 0xb6, 0xee, 0x63, 0xc5, 0x8e, 0x14, 0x21, 0x3f, 0x0f, 0x49, 0x86, 0x74, + 0x69, 0x8f, 0x1d, 0x5d, 0xc9, 0x10, 0xbf, 0xa4, 0xe2, 0x98, 0x16, 0xf1, 0x4b, 0x78, 0x12, 0xaf, + 0xa7, 0x02, 0xb3, 0x61, 0x65, 0x89, 0x78, 0x29, 0x60, 0x8a, 0xbb, 0x87, 0xa4, 0x8b, 0x17, 0xf6, + 0x88, 0x5f, 0x84, 0x8a, 0x90, 0x51, 0xe9, 0x3f, 0xd9, 0x39, 0x6e, 0x62, 0x2a, 0x3f, 0xdb, 0xcf, + 0xb3, 0x64, 0x6c, 0xd5, 0xe3, 0x26, 0x26, 0xee, 0xa9, 0xfb, 0x9f, 0xa8, 0x2b, 0x50, 0xd5, 0xbe, + 0xbe, 0x29, 0x1d, 0x5f, 0x34, 0xaf, 0xeb, 0xde, 0xb5, 0x69, 0xe2, 0x4f, 0xe2, 0x5c, 0x15, 0xec, + 0x4d, 0x9c, 0x29, 0x52, 0x6f, 0xe4, 0xdd, 0xb6, 0xed, 0xa9, 0x60, 0xf5, 0x63, 0xa7, 0xa8, 0x7e, + 0x60, 0x5d, 0xdc, 0xa7, 0xa2, 0x77, 0x20, 0xb0, 0xf2, 0x2d, 0xb3, 0x9a, 0xb9, 0x68, 0xe7, 0x34, + 0x4a, 0x99, 0xf2, 0xa5, 0x30, 0xba, 0x8d, 0x0a, 0x90, 0xc2, 0x47, 0x4d, 0x59, 0xc3, 0xb6, 0xca, + 0xcd, 0x5a, 0xbe, 0xd7, 0x3e, 0x5a, 0x97, 0xff, 0x3f, 0x86, 0x8f, 0x9a, 0x84, 0x88, 0xb6, 0xc9, + 0x0c, 0xe7, 0xba, 0x03, 0xb4, 0xd8, 0xf6, 0x60, 0x38, 0xe1, 0xf7, 0x17, 0x2e, 0x6e, 0xd2, 0xf3, + 0x04, 0x98, 0x08, 0xde, 0x76, 0x1f, 0x08, 0x70, 0x21, 0xb4, 0xed, 0xa2, 0x9c, 0xec, 0xde, 0x84, + 0x04, 0x55, 0x41, 0xec, 0x94, 0x2a, 0xa0, 0x5c, 0xf9, 0xef, 0xba, 0xa3, 0x5e, 0xc2, 0x75, 0x93, + 0xa8, 0xf7, 0x13, 0x58, 0x17, 0x1b, 0x73, 0x9b, 0x3d, 0x76, 0xea, 0x66, 0x77, 0x59, 0x3b, 0xcc, + 0x42, 0x47, 0x61, 0xa3, 0x34, 0x0b, 0xbf, 0x29, 0xc0, 0x74, 0x09, 0x2b, 0x96, 0xb3, 0x83, 0x15, + 0x27, 0x62, 0x77, 0xf6, 0x55, 0x88, 0x1b, 0xe6, 0xe1, 0x69, 0x96, 0x06, 0x49, 0x7e, 0x7f, 0xda, + 0x6a, 0x2f, 0x57, 0x94, 0xb5, 0xfe, 0x97, 0x18, 0xa4, 0xef, 0x15, 0xa2, 0xac, 0xeb, 0x9b, 0x7c, + 0x01, 0x99, 0x0d, 0xf5, 0xb0, 0x6e, 0xe9, 0xbd, 0x6f, 0xf1, 0x5e, 0x61, 0x1d, 0x1f, 0xbb, 0xdd, + 0x92, 0x70, 0xa1, 0x65, 0x48, 0x3b, 0x7b, 0x16, 0xb6, 0xf7, 0xcc, 0xba, 0x76, 0x1a, 0x9f, 0xc5, + 0xe7, 0x9a, 0xfd, 0x75, 0x01, 0x46, 0xa9, 0x60, 0x37, 0x8a, 0x41, 0x08, 0x89, 0x62, 0x20, 0xef, + 0xf1, 0xfc, 0xbe, 0xd8, 0x69, 0xde, 0xe3, 0xf9, 0xce, 0x57, 0x61, 0xb2, 0x45, 0xdc, 0xcc, 0x3a, + 0x56, 0x2c, 0xb6, 0xb7, 0xcf, 0x97, 0xf3, 0x27, 0x5a, 0x36, 0xf6, 0xd7, 0xb1, 0x59, 0x2b, 0x7a, + 0x4e, 0xd4, 0xa8, 0x98, 0xcc, 0x3f, 0x04, 0x20, 0x3a, 0x88, 0xb2, 0x1d, 0xff, 0x7f, 0x1c, 0xb2, + 0x5b, 0x2d, 0x7b, 0x2f, 0xe2, 0x8e, 0x5b, 0x00, 0x68, 0xb6, 0x6c, 0x8a, 0x2a, 0x8e, 0x0c, 0xae, + 0xa7, 0x01, 0xe1, 0x14, 0xae, 0xa2, 0x18, 0x5f, 0xf5, 0xc8, 0x40, 0x25, 0x2e, 0x04, 0xcb, 0x7e, + 0x4c, 0xc6, 0xf3, 0xfd, 0x40, 0x67, 0xf5, 0xc8, 0xd8, 0xc0, 0x1e, 0xda, 0x64, 0x92, 0x30, 0x91, + 0xf4, 0x26, 0x8c, 0x91, 0x07, 0xd9, 0x31, 0x4f, 0xd3, 0x37, 0x92, 0x84, 0xa7, 0x6a, 0xa2, 0x3b, + 0x90, 0x66, 0xdc, 0x64, 0x86, 0x4b, 0xd2, 0x19, 0x2e, 0xac, 0x2e, 0x5c, 0x8d, 0x74, 0x6e, 0x4b, + 0x51, 0x56, 0x32, 0x9f, 0xcd, 0xc0, 0xe8, 0xae, 0x69, 0xa9, 0x98, 0x06, 0x5a, 0xa4, 0x24, 0xf6, + 0x10, 0x6c, 0xd5, 0xb5, 0x44, 0x2a, 0x25, 0xa6, 0xd7, 0x12, 0xa9, 0xb4, 0x08, 0xf9, 0x6f, 0x0a, + 0x30, 0xe9, 0x35, 0x47, 0x94, 0x46, 0xbf, 0xd0, 0xa6, 0xcb, 0xd3, 0x37, 0x08, 0x51, 0x63, 0xfe, + 0xaf, 0xa9, 0x07, 0xa4, 0x9a, 0x07, 0xb4, 0x7d, 0xa2, 0xec, 0x2f, 0x77, 0x58, 0xdc, 0x4d, 0xec, + 0xb4, 0x6d, 0x4c, 0x43, 0x70, 0x6e, 0xc2, 0x8c, 0xde, 0x20, 0xd3, 0x81, 0xee, 0xd4, 0x8f, 0x39, + 0x7c, 0x73, 0xb0, 0xbb, 0x95, 0x3b, 0xed, 0xa7, 0x15, 0xdc, 0x24, 0x6e, 0x21, 0xd9, 0xe6, 0x8e, + 0x5f, 0x9f, 0x28, 0x15, 0x5e, 0x86, 0x09, 0x8b, 0x89, 0x26, 0x6e, 0xcc, 0x29, 0x75, 0x3e, 0xee, + 0xb1, 0x12, 0xb5, 0x7f, 0x27, 0x06, 0x93, 0x0f, 0x5b, 0xd8, 0x3a, 0xfe, 0x34, 0x29, 0xfd, 0x2a, + 0x4c, 0x1e, 0x2a, 0xba, 0x23, 0xef, 0x9a, 0x96, 0xdc, 0x6a, 0x6a, 0x8a, 0xe3, 0x59, 0x31, 0x42, + 0xbe, 0x6b, 0x5a, 0xdb, 0x94, 0x88, 0x30, 0xa0, 0x7d, 0xc3, 0x3c, 0x34, 0x64, 0x42, 0xa6, 0xb0, + 0xf9, 0xc8, 0xe0, 0xab, 0xce, 0x2b, 0xaf, 0xfd, 0xfd, 0xc9, 0xfc, 0xed, 0xa1, 0xc2, 0xbb, 0x68, + 0x80, 0x5a, 0xab, 0xa5, 0x6b, 0x8b, 0xdb, 0xdb, 0xe5, 0x55, 0x49, 0xa4, 0x22, 0xdf, 0x66, 0x12, + 0xab, 0x47, 0x86, 0x3b, 0xdd, 0x7f, 0x2c, 0x80, 0xe8, 0x6b, 0x2a, 0xca, 0xe6, 0x2c, 0x42, 0xe6, + 0x71, 0x0b, 0x5b, 0xfa, 0x33, 0x34, 0x26, 0x70, 0x46, 0x62, 0x88, 0xde, 0x85, 0xf1, 0x36, 0x3d, + 0xc4, 0x7f, 0x39, 0x3d, 0x64, 0x0e, 0x7d, 0x15, 0xe4, 0xff, 0x4a, 0x00, 0x44, 0x2b, 0x5f, 0x66, + 0x0b, 0xfe, 0x9f, 0x96, 0x9e, 0x72, 0x1d, 0x44, 0x1a, 0xda, 0x28, 0xeb, 0xbb, 0x72, 0x43, 0xb7, + 0x6d, 0xdd, 0xa8, 0xf1, 0xae, 0x92, 0xa5, 0xf4, 0xf2, 0xee, 0x06, 0xa3, 0xf2, 0x46, 0xfc, 0x9f, + 0x30, 0xdd, 0x56, 0x8d, 0x28, 0x9b, 0xf1, 0x12, 0x8c, 0xef, 0x9a, 0x2d, 0x43, 0x93, 0xd9, 0xa6, + 0x08, 0x5f, 0x25, 0xcc, 0x50, 0x1a, 0x7b, 0x5f, 0xfe, 0x9f, 0x63, 0x30, 0x23, 0x61, 0xdb, 0xac, + 0x1f, 0xe0, 0xe8, 0x15, 0x59, 0x02, 0xbe, 0x1d, 0x23, 0x3f, 0x93, 0x3e, 0xd3, 0x8c, 0x99, 0x4d, + 0x69, 0xed, 0x0b, 0xee, 0x97, 0xfb, 0xf7, 0xc5, 0xee, 0x25, 0x76, 0xbe, 0x7e, 0x97, 0x68, 0x5b, + 0xbf, 0x33, 0x61, 0x52, 0xaf, 0x19, 0x26, 0xb1, 0x59, 0x36, 0x7e, 0x6c, 0xb4, 0x1a, 0x2e, 0xb8, + 0x59, 0xec, 0x57, 0xc8, 0x32, 0x63, 0xa9, 0xe0, 0xc7, 0x9b, 0xad, 0x06, 0x75, 0x5e, 0x56, 0xce, + 0x92, 0xf2, 0x3e, 0x3d, 0x99, 0xcf, 0xb6, 0xa5, 0xd9, 0x52, 0x56, 0xf7, 0x9e, 0x89, 0x74, 0xde, + 0xe4, 0x5f, 0x86, 0x33, 0x1d, 0x2a, 0x8f, 0xd2, 0xc7, 0xf9, 0x8b, 0x38, 0x9c, 0x6f, 0x17, 0x1f, + 0x35, 0x64, 0xf9, 0xb4, 0x37, 0x6b, 0x09, 0x26, 0x1a, 0xba, 0xf1, 0x6c, 0x2b, 0x96, 0xe3, 0x0d, + 0xdd, 0xf0, 0x17, 0x7e, 0x43, 0x3a, 0x48, 0xf2, 0xbf, 0xa0, 0x83, 0x28, 0x30, 0x1b, 0xd6, 0x82, + 0x51, 0xf6, 0x92, 0xf7, 0x05, 0x18, 0x8f, 0x7a, 0x11, 0xee, 0xd9, 0x82, 0xd1, 0x78, 0x9d, 0xab, + 0x30, 0xf1, 0x09, 0xac, 0xda, 0x7d, 0x47, 0x00, 0x54, 0xb5, 0x5a, 0x06, 0x41, 0xc3, 0xf7, 0xcd, + 0x5a, 0x94, 0x95, 0x9d, 0x81, 0x51, 0xdd, 0xd0, 0xf0, 0x11, 0xad, 0x6c, 0x42, 0x62, 0x0f, 0x6d, + 0x3b, 0x8d, 0xf1, 0xa1, 0x76, 0x1a, 0xfd, 0x98, 0x96, 0xb6, 0x82, 0x46, 0xa9, 0x85, 0x3f, 0x88, + 0xc1, 0x34, 0xaf, 0x4e, 0xe4, 0xab, 0x96, 0xaf, 0xc0, 0x68, 0x9d, 0xc8, 0xec, 0xd3, 0xe6, 0xf4, + 0x9d, 0x6e, 0x9b, 0xd3, 0xcc, 0xe8, 0x73, 0x00, 0x4d, 0x0b, 0x1f, 0xc8, 0x8c, 0x35, 0x3e, 0x14, + 0x6b, 0x9a, 0x70, 0x50, 0x02, 0xfa, 0x22, 0x4c, 0x92, 0x11, 0xde, 0xb4, 0xcc, 0xa6, 0x69, 0x13, + 0x27, 0xc5, 0x1e, 0x0e, 0xe9, 0x4c, 0x3d, 0x3d, 0x99, 0x9f, 0xd8, 0xd0, 0x8d, 0x2d, 0xce, 0x58, + 0xad, 0x48, 0xc4, 0x54, 0x78, 0x8f, 0xee, 0x00, 0xfc, 0x5b, 0x01, 0x66, 0x3e, 0xb1, 0x75, 0xde, + 0xff, 0x0e, 0x8d, 0x79, 0x33, 0x8f, 0x48, 0x1f, 0xcb, 0xc6, 0xae, 0x19, 0xfd, 0xea, 0xfb, 0xfb, + 0x02, 0x4c, 0x05, 0xc4, 0x47, 0xe9, 0xc9, 0x3c, 0x93, 0xce, 0xf2, 0x5f, 0x22, 0xbe, 0x4d, 0xb0, + 0xdb, 0x47, 0x39, 0xa8, 0xfe, 0x2c, 0x06, 0x67, 0x0b, 0x6c, 0x0f, 0xda, 0x0d, 0xd0, 0x88, 0xb2, + 0x97, 0xe4, 0x60, 0xec, 0x00, 0x5b, 0xb6, 0x6e, 0xb2, 0x19, 0x76, 0x42, 0x72, 0x1f, 0xd1, 0x2c, + 0xa4, 0x6c, 0x43, 0x69, 0xda, 0x7b, 0xa6, 0xbb, 0x6d, 0xe7, 0x3d, 0x7b, 0xc1, 0x24, 0xa3, 0xcf, + 0x1e, 0x4c, 0x92, 0xec, 0x1f, 0x4c, 0x32, 0xf6, 0x4b, 0x07, 0x93, 0xf0, 0x3d, 0xb2, 0xef, 0x0b, + 0x70, 0xae, 0x4b, 0x7f, 0x51, 0xf6, 0x99, 0xaf, 0x42, 0x46, 0xe5, 0x82, 0x89, 0x35, 0x66, 0xdb, + 0x80, 0x65, 0x92, 0xed, 0x19, 0x01, 0xc8, 0xd3, 0x93, 0x79, 0x70, 0x8b, 0x5a, 0x5e, 0xe5, 0x2a, + 0x22, 0xff, 0xb5, 0xfc, 0xdf, 0x8d, 0xc3, 0x64, 0xf1, 0x88, 0x2d, 0x72, 0x57, 0x98, 0x3f, 0x80, + 0xee, 0x42, 0xaa, 0x69, 0x99, 0x07, 0xba, 0x5b, 0x8d, 0x6c, 0x5b, 0x0c, 0x81, 0x5b, 0x8d, 0x0e, + 0xae, 0x2d, 0xce, 0x21, 0x79, 0xbc, 0xa8, 0x0a, 0xe9, 0xfb, 0xa6, 0xaa, 0xd4, 0xef, 0xea, 0x75, + 0xb7, 0xff, 0xbf, 0x3c, 0x58, 0xd0, 0xa2, 0xc7, 0xb3, 0xa5, 0x38, 0x7b, 0x6e, 0x53, 0x78, 0x44, + 0x54, 0x86, 0x54, 0xc9, 0x71, 0x9a, 0x24, 0x91, 0x5b, 0x93, 0x6b, 0x43, 0x08, 0x25, 0x2c, 0x5c, + 0x96, 0xc7, 0x8e, 0xaa, 0x30, 0x75, 0xcf, 0x34, 0x6b, 0x75, 0x5c, 0xa8, 0x9b, 0x2d, 0xad, 0x60, + 0x1a, 0xbb, 0x7a, 0x8d, 0xdb, 0xe3, 0xab, 0x43, 0xc8, 0xbc, 0x57, 0xa8, 0x48, 0xdd, 0x02, 0xd0, + 0x32, 0xa4, 0x2a, 0xb7, 0xb9, 0x30, 0xe6, 0xc0, 0x5d, 0x19, 0x42, 0x58, 0xe5, 0xb6, 0xe4, 0xb1, + 0xa1, 0x35, 0xc8, 0x2c, 0x3f, 0x69, 0x59, 0x98, 0x4b, 0x49, 0xf6, 0x0c, 0x60, 0xe8, 0x94, 0x42, + 0xb9, 0xa4, 0x20, 0x33, 0xaa, 0x40, 0xf6, 0x6d, 0xd3, 0xda, 0xaf, 0x9b, 0x8a, 0x5b, 0xc3, 0x31, + 0x2a, 0xee, 0x33, 0x43, 0x88, 0x73, 0x19, 0xa5, 0x0e, 0x11, 0xe8, 0xcb, 0x30, 0x49, 0x1a, 0xa3, + 0xaa, 0xec, 0xd4, 0xdd, 0x42, 0xa6, 0xa8, 0xd4, 0x17, 0x87, 0x90, 0xea, 0x71, 0xba, 0xbb, 0x2c, + 0x1d, 0xa2, 0x66, 0xbf, 0x08, 0x13, 0x6d, 0x9d, 0x00, 0x21, 0x48, 0x34, 0x49, 0x7b, 0x0b, 0x34, + 0xd0, 0x88, 0xfe, 0x47, 0x2f, 0xc1, 0x98, 0x61, 0x6a, 0xd8, 0x1d, 0x21, 0x13, 0x2b, 0x33, 0x4f, + 0x4f, 0xe6, 0x93, 0x9b, 0xa6, 0xc6, 0xdc, 0x15, 0xfe, 0x4f, 0x4a, 0x92, 0x4c, 0xae, 0xb3, 0x32, + 0x7b, 0x15, 0x12, 0xa4, 0xf5, 0x89, 0x91, 0xda, 0x51, 0x6c, 0xbc, 0x6d, 0xe9, 0x5c, 0xa6, 0xfb, + 0xc8, 0xf3, 0xfd, 0x58, 0x80, 0x58, 0xe5, 0x36, 0x71, 0xd4, 0x77, 0x5a, 0xea, 0x3e, 0x76, 0x78, + 0x2e, 0xfe, 0x44, 0x1d, 0x78, 0x0b, 0xef, 0xea, 0xcc, 0x87, 0x4a, 0x4b, 0xfc, 0x09, 0x3d, 0x07, + 0xa0, 0xa8, 0x2a, 0xb6, 0x6d, 0xd9, 0x3d, 0x5e, 0x97, 0x96, 0xd2, 0x8c, 0xb2, 0x8e, 0x8f, 0x09, + 0x9b, 0x8d, 0x55, 0x0b, 0x3b, 0x6e, 0xc4, 0x14, 0x7b, 0x22, 0x6c, 0x0e, 0x6e, 0x34, 0x65, 0xc7, + 0xdc, 0xc7, 0x06, 0xed, 0x33, 0x69, 0x62, 0x7c, 0x1a, 0xcd, 0x2a, 0x21, 0x10, 0xbb, 0x89, 0x0d, + 0xcd, 0x37, 0x72, 0x69, 0xc9, 0x7b, 0x26, 0x22, 0x2d, 0x5c, 0xd3, 0xf9, 0x09, 0xb1, 0xb4, 0xc4, + 0x9f, 0x88, 0xc6, 0x94, 0x96, 0xb3, 0x47, 0x5b, 0x25, 0x2d, 0xd1, 0xff, 0xbc, 0x6a, 0xbf, 0x2d, + 0x40, 0xfc, 0x5e, 0xa1, 0x72, 0xea, 0xba, 0xb9, 0x12, 0xe3, 0xbe, 0x44, 0x1a, 0xa8, 0xa8, 0xd7, + 0xeb, 0xba, 0x51, 0x23, 0x2e, 0xcd, 0x57, 0xb1, 0xea, 0xd6, 0x2c, 0xcb, 0xc9, 0x5b, 0x8c, 0x8a, + 0x16, 0x20, 0xa3, 0x5a, 0x58, 0xc3, 0x86, 0xa3, 0x2b, 0x75, 0x9b, 0x57, 0x31, 0x48, 0xe2, 0x85, + 0xfb, 0xba, 0x00, 0xa3, 0xb4, 0xf3, 0xa2, 0x8b, 0x90, 0x56, 0x4d, 0xc3, 0x51, 0x74, 0x83, 0x5b, + 0xa1, 0xb4, 0xe4, 0x13, 0x7a, 0x16, 0xf2, 0x12, 0x8c, 0x2b, 0xaa, 0x6a, 0xb6, 0x0c, 0x47, 0x36, + 0x94, 0x06, 0xe6, 0x85, 0xcd, 0x70, 0xda, 0xa6, 0xd2, 0xc0, 0x68, 0x1e, 0xdc, 0x47, 0xef, 0x90, + 0x63, 0x5a, 0x02, 0x4e, 0x5a, 0xc7, 0xc7, 0xbc, 0x24, 0xdf, 0x17, 0x20, 0xe5, 0x76, 0x7a, 0x52, + 0x98, 0x1a, 0x36, 0xb0, 0xa5, 0x38, 0xa6, 0x57, 0x18, 0x8f, 0xd0, 0x39, 0xe3, 0xa5, 0xfd, 0x19, + 0x6f, 0x06, 0x46, 0x1d, 0xd2, 0xaf, 0x79, 0x39, 0xd8, 0x03, 0x5d, 0x6b, 0xae, 0x2b, 0x35, 0xb6, + 0xbc, 0x96, 0x96, 0xd8, 0x03, 0xa9, 0x12, 0x0f, 0xb6, 0x65, 0xda, 0xe1, 0x4f, 0xa4, 0xbc, 0x2c, + 0x18, 0x74, 0x07, 0xd7, 0x74, 0x83, 0x76, 0x80, 0xb8, 0x04, 0x94, 0xb4, 0x42, 0x28, 0xe8, 0x02, + 0xa4, 0x59, 0x06, 0x6c, 0x68, 0xb4, 0x17, 0xc4, 0xa5, 0x14, 0x25, 0x14, 0xdd, 0x53, 0x5c, 0xb3, + 0xfb, 0x90, 0xf6, 0xc6, 0x18, 0x69, 0xc8, 0x96, 0xed, 0x29, 0x95, 0xfe, 0x47, 0x2f, 0xc3, 0xcc, + 0xe3, 0x96, 0x52, 0xd7, 0x77, 0xe9, 0xca, 0x19, 0xc9, 0xc6, 0xf4, 0xc7, 0xea, 0x83, 0xbc, 0x34, + 0x2a, 0x81, 0xaa, 0xd1, 0x1d, 0x92, 0x71, 0x7f, 0x48, 0x06, 0xb7, 0x42, 0xf2, 0x1f, 0x0a, 0x30, + 0xc5, 0xe2, 0x85, 0x58, 0xc8, 0x6a, 0x74, 0x0e, 0xc6, 0x1b, 0x90, 0xd6, 0x14, 0x47, 0x61, 0x07, + 0x39, 0x63, 0x7d, 0x0f, 0x72, 0xba, 0x16, 0x9f, 0xe4, 0xa7, 0x87, 0x39, 0x11, 0x24, 0xc8, 0x7f, + 0x76, 0xf2, 0x55, 0xa2, 0xff, 0xfd, 0x08, 0x8c, 0x60, 0x71, 0xa3, 0x74, 0xb8, 0x96, 0xe0, 0x0c, + 0xd1, 0x7e, 0xd1, 0x50, 0xad, 0xe3, 0xa6, 0xa3, 0x9b, 0xc6, 0x03, 0xfa, 0x6b, 0x23, 0x31, 0xb0, + 0x81, 0x45, 0xf7, 0xad, 0x78, 0x59, 0xfe, 0x32, 0x09, 0x13, 0xc5, 0xa3, 0xa6, 0x69, 0x45, 0xba, + 0xa8, 0xb5, 0x02, 0x63, 0x1c, 0xf1, 0xf7, 0xd9, 0x53, 0xee, 0xb0, 0xd5, 0xee, 0x76, 0x2d, 0x67, + 0x44, 0x2b, 0x00, 0x2c, 0xb8, 0x94, 0x06, 0x1d, 0xc5, 0x4f, 0xb1, 0xb1, 0x46, 0xd9, 0x08, 0x15, + 0x6d, 0x42, 0xa6, 0x71, 0xa0, 0xaa, 0xf2, 0xae, 0x5e, 0x77, 0x78, 0x74, 0x5e, 0x78, 0x68, 0xf9, + 0xc6, 0xa3, 0x42, 0xe1, 0x2e, 0xcd, 0xc4, 0x02, 0xe5, 0xfc, 0x67, 0x09, 0x88, 0x04, 0xf6, 0x1f, + 0xbd, 0x08, 0xfc, 0x80, 0x8d, 0x6c, 0xbb, 0x67, 0xe9, 0x56, 0x26, 0x9e, 0x9e, 0xcc, 0xa7, 0x25, + 0x4a, 0xad, 0x54, 0xaa, 0x52, 0x9a, 0x65, 0xa8, 0xd8, 0x0e, 0x7a, 0x1e, 0x26, 0xcc, 0x86, 0xee, + 0xc8, 0xae, 0x0f, 0xc4, 0xdd, 0xc6, 0x71, 0x42, 0x74, 0x7d, 0x24, 0x54, 0x85, 0x6b, 0xd8, 0xa0, + 0xa3, 0x80, 0xd4, 0x53, 0xde, 0x61, 0x6b, 0x91, 0x0e, 0x1b, 0xef, 0xb2, 0xd9, 0x74, 0xf4, 0x86, + 0xfe, 0x84, 0xee, 0x6a, 0xf3, 0xfd, 0xa2, 0xe7, 0x59, 0x76, 0x52, 0xbf, 0x15, 0xba, 0x48, 0xc9, + 0xf3, 0x3e, 0x08, 0x64, 0x45, 0x5f, 0x17, 0xe0, 0x2c, 0x57, 0xa4, 0xbc, 0x43, 0x63, 0xe3, 0x95, + 0xba, 0xee, 0x1c, 0xcb, 0xfb, 0x07, 0xb9, 0x14, 0x75, 0x4e, 0x3f, 0x1b, 0xda, 0x20, 0x81, 0x7e, + 0xb0, 0xe8, 0x36, 0xcb, 0xf1, 0x7d, 0xce, 0xbc, 0x7e, 0x50, 0x34, 0x1c, 0xeb, 0x78, 0xe5, 0xdc, + 0xd3, 0x93, 0xf9, 0xe9, 0xee, 0xd4, 0x47, 0xd2, 0xb4, 0xdd, 0xcd, 0x82, 0x4a, 0x00, 0xd8, 0xeb, + 0x8d, 0x34, 0x36, 0x30, 0xdc, 0xbd, 0x08, 0xed, 0xb6, 0x52, 0x80, 0x17, 0x5d, 0x07, 0x91, 0x9f, + 0x8e, 0xd9, 0xd5, 0xeb, 0x58, 0xb6, 0xf5, 0x27, 0x38, 0x07, 0xd4, 0x06, 0x65, 0x19, 0x9d, 0x88, + 0xa8, 0xe8, 0x4f, 0xf0, 0xec, 0x57, 0x21, 0xd7, 0xab, 0xf4, 0xc1, 0x81, 0x90, 0x66, 0x1b, 0xb8, + 0xaf, 0xb7, 0xaf, 0xc8, 0x0c, 0xd1, 0x55, 0xdd, 0x55, 0x99, 0xd8, 0xeb, 0xae, 0x09, 0xfa, 0x6e, + 0x0c, 0x26, 0x56, 0x5a, 0xf5, 0xfd, 0x07, 0xcd, 0x4a, 0xab, 0xd1, 0x50, 0xac, 0x63, 0x62, 0x2a, + 0x99, 0xe9, 0x20, 0xc5, 0x14, 0x98, 0xa9, 0xa4, 0xb6, 0x41, 0x7f, 0x82, 0xc9, 0x64, 0x16, 0x3c, + 0xcd, 0xcd, 0x62, 0xff, 0x69, 0x4d, 0x02, 0x47, 0xb4, 0xcd, 0x43, 0x1b, 0xbd, 0x0e, 0xb9, 0x40, + 0x46, 0xba, 0x7c, 0x22, 0x63, 0xc3, 0xb1, 0x74, 0xcc, 0x96, 0x03, 0xe3, 0x52, 0x20, 0xee, 0xa6, + 0x4c, 0x92, 0x8b, 0x2c, 0x15, 0x55, 0x61, 0x9c, 0x64, 0x3c, 0x96, 0xe9, 0x64, 0xe3, 0x2e, 0xda, + 0xde, 0x0c, 0xa9, 0x5c, 0x5b, 0xb9, 0x17, 0xa9, 0x96, 0x0a, 0x94, 0x87, 0xfe, 0x95, 0x32, 0xd8, + 0xa7, 0xcc, 0xbe, 0x05, 0x62, 0x67, 0x86, 0xa0, 0x46, 0x13, 0x4c, 0xa3, 0x33, 0x41, 0x8d, 0xc6, + 0x03, 0xda, 0x5a, 0x4b, 0xa4, 0x12, 0xe2, 0x68, 0xfe, 0xa7, 0x71, 0xc8, 0xba, 0x9d, 0x2d, 0x4a, + 0x34, 0xb3, 0x02, 0xa3, 0xa4, 0x6b, 0xb8, 0x51, 0x22, 0x57, 0xfb, 0xf4, 0x71, 0x1e, 0x67, 0x4e, + 0xba, 0x8c, 0x8b, 0x87, 0x29, 0x6b, 0x14, 0x66, 0x67, 0xf6, 0x7f, 0xc5, 0x20, 0x41, 0x01, 0xc4, + 0x4d, 0x48, 0xd0, 0xa9, 0x43, 0x18, 0x66, 0xea, 0xa0, 0x59, 0xbd, 0xc9, 0x2e, 0x16, 0xf0, 0x3f, + 0x89, 0x33, 0xb7, 0xa7, 0xbc, 0x7a, 0xf3, 0x16, 0x35, 0x39, 0xe3, 0x12, 0x7f, 0x42, 0x2b, 0x34, + 0x7c, 0xc9, 0xb4, 0x1c, 0xac, 0x71, 0xc7, 0x7d, 0x61, 0x50, 0xfb, 0xba, 0xd3, 0x94, 0xcb, 0x87, + 0xce, 0x43, 0x9c, 0xd8, 0xb2, 0x31, 0x16, 0xd9, 0xf0, 0xf4, 0x64, 0x3e, 0x4e, 0xac, 0x18, 0xa1, + 0xa1, 0x25, 0xc8, 0xb4, 0x1b, 0x0e, 0xe1, 0x7a, 0x9a, 0x99, 0xc7, 0xc0, 0xa0, 0x87, 0xba, 0x37, + 0xc0, 0x18, 0x68, 0xe5, 0x6d, 0xfc, 0xb5, 0x51, 0x98, 0x28, 0x37, 0xa2, 0x9e, 0x58, 0x96, 0xdb, + 0x5b, 0x38, 0x0c, 0xed, 0xb4, 0xbd, 0x34, 0xa4, 0x81, 0xdb, 0xe6, 0xf4, 0xf8, 0xe9, 0xe6, 0xf4, + 0x32, 0x71, 0x81, 0xf9, 0xf5, 0x0c, 0xf1, 0x1e, 0xc0, 0xa6, 0xfd, 0xfd, 0xd4, 0x8b, 0x91, 0x08, + 0x8f, 0x7f, 0xf2, 0x82, 0x86, 0xa7, 0xbc, 0x45, 0x3d, 0x6d, 0xd6, 0xcb, 0x92, 0xc3, 0xf7, 0xb2, + 0x31, 0x6c, 0x68, 0x74, 0x6a, 0x6b, 0xb7, 0xab, 0x63, 0xcf, 0x6e, 0x57, 0x67, 0x9f, 0xf0, 0xce, + 0xfa, 0x06, 0xc4, 0x35, 0xdd, 0x6d, 0x9c, 0xe1, 0x27, 0x6c, 0xc2, 0x34, 0xa0, 0xd7, 0x26, 0x82, + 0xbd, 0x36, 0xb8, 0xc0, 0x31, 0xfb, 0x00, 0xc0, 0xd7, 0x10, 0x5a, 0x80, 0xa4, 0x59, 0xd7, 0xdc, + 0x03, 0x28, 0x13, 0x2b, 0xe9, 0xa7, 0x27, 0xf3, 0xa3, 0x0f, 0xea, 0x5a, 0x79, 0x55, 0x1a, 0x35, + 0xeb, 0x5a, 0x59, 0xa3, 0x37, 0x64, 0xe0, 0x43, 0xd9, 0x8b, 0x56, 0x1b, 0x97, 0xc6, 0x0c, 0x7c, + 0xb8, 0x8a, 0x6d, 0xb5, 0x23, 0x38, 0x86, 0x74, 0xc1, 0x6f, 0x09, 0x90, 0x75, 0x5b, 0x23, 0x5a, + 0x33, 0x93, 0xd2, 0x1b, 0x7c, 0xd8, 0xc5, 0x4f, 0x37, 0xec, 0x5c, 0x3e, 0x7e, 0xfc, 0xf6, 0x1b, + 0x02, 0x8f, 0x54, 0xae, 0xa8, 0x8a, 0x43, 0x9c, 0x8d, 0x08, 0x87, 0xca, 0x0b, 0x20, 0x5a, 0x8a, + 0xa1, 0x99, 0x0d, 0xfd, 0x09, 0x66, 0x2b, 0xa2, 0x36, 0xdf, 0xdc, 0x9c, 0xf4, 0xe8, 0x74, 0xc9, + 0xcf, 0x5d, 0xd0, 0xfd, 0x85, 0xc0, 0xa3, 0x9a, 0xbd, 0xc2, 0x44, 0xa9, 0xb4, 0x75, 0x48, 0x5a, + 0x2c, 0x36, 0x92, 0x0d, 0xdd, 0x97, 0x42, 0x84, 0x84, 0xbd, 0x9d, 0x85, 0x1e, 0x7a, 0x83, 0x87, + 0x8a, 0x98, 0xfd, 0x02, 0x8c, 0x52, 0xf2, 0x33, 0x18, 0x58, 0xae, 0xf9, 0x9f, 0xc7, 0xe0, 0x32, + 0x7d, 0xdd, 0x23, 0x6c, 0xe9, 0xbb, 0xc7, 0x5b, 0x96, 0xe9, 0x60, 0xd5, 0xc1, 0x9a, 0x7f, 0xde, + 0x23, 0x52, 0xab, 0x95, 0x6e, 0xba, 0x2f, 0x38, 0x55, 0x88, 0x98, 0xc7, 0x85, 0xd6, 0x61, 0x92, + 0xdd, 0xc2, 0x23, 0x2b, 0x75, 0xfd, 0x00, 0xcb, 0x8a, 0x73, 0x9a, 0xb9, 0x69, 0x82, 0xf1, 0x2e, + 0x13, 0xd6, 0x65, 0x07, 0x69, 0x90, 0xe6, 0xc2, 0x74, 0x8d, 0x5f, 0xbd, 0x73, 0xef, 0x97, 0x5b, + 0xf3, 0x4b, 0x49, 0x54, 0x5e, 0x79, 0x55, 0x4a, 0x31, 0xc9, 0xde, 0x9e, 0xcd, 0x8f, 0x04, 0xb8, + 0x32, 0x40, 0xd1, 0x51, 0x76, 0xb3, 0x59, 0x48, 0x1d, 0x90, 0x17, 0xe9, 0x5c, 0xd3, 0x29, 0xc9, + 0x7b, 0x46, 0x1b, 0x30, 0xb1, 0xab, 0xe8, 0x75, 0xf7, 0xfe, 0x9c, 0x7e, 0x81, 0x85, 0xe1, 0xf1, + 0xae, 0xe3, 0x8c, 0x9d, 0x26, 0xd2, 0x13, 0x91, 0x53, 0xcb, 0x9a, 0x56, 0xa9, 0x70, 0x0b, 0x16, + 0x5d, 0x7f, 0x71, 0xa1, 0x63, 0xcc, 0x87, 0x8e, 0xe8, 0x25, 0x40, 0x9a, 0x6e, 0xb3, 0x6b, 0x3d, + 0xec, 0x3d, 0x45, 0x33, 0x0f, 0xfd, 0xa8, 0x89, 0x29, 0x37, 0xa5, 0xe2, 0x26, 0xa0, 0x0a, 0x50, + 0xdc, 0x22, 0xdb, 0x8e, 0xe2, 0x6d, 0xfc, 0x5c, 0x19, 0xea, 0x78, 0x16, 0x03, 0x34, 0xde, 0xa3, + 0x94, 0x26, 0x72, 0xe8, 0x5f, 0xe2, 0x81, 0xeb, 0xa4, 0xea, 0x8e, 0xac, 0xd8, 0xee, 0x59, 0x1e, + 0x76, 0xa1, 0x48, 0x96, 0xd1, 0x97, 0xed, 0xe0, 0x11, 0x1d, 0x76, 0xd4, 0xc0, 0x57, 0x50, 0x94, + 0x40, 0xf7, 0xf7, 0x05, 0xc8, 0x4a, 0x78, 0xd7, 0xc2, 0x76, 0xa4, 0x80, 0xff, 0x2e, 0x8c, 0x5b, + 0x4c, 0xaa, 0xbc, 0x6b, 0x99, 0x8d, 0xd3, 0x8c, 0xb1, 0x0c, 0x67, 0xbc, 0x6b, 0x99, 0x8d, 0xb6, + 0x3b, 0x16, 0x1e, 0xc1, 0xa4, 0x57, 0xd2, 0x28, 0x55, 0xf0, 0x21, 0x3d, 0x92, 0xcc, 0x04, 0x47, + 0x1d, 0xbe, 0xf0, 0x49, 0xe8, 0x81, 0xee, 0x34, 0x05, 0x8b, 0x1b, 0xa5, 0x32, 0x7e, 0x21, 0x40, + 0xb6, 0xd2, 0xda, 0x61, 0xb7, 0x4a, 0x45, 0xa7, 0x87, 0x22, 0xa4, 0xeb, 0x78, 0xd7, 0x91, 0x9f, + 0x29, 0x3c, 0x3e, 0x45, 0x58, 0xe9, 0x11, 0x81, 0x7b, 0x00, 0x16, 0x3d, 0x00, 0x47, 0xe5, 0xc4, + 0x4f, 0x29, 0x27, 0x4d, 0x79, 0x7d, 0x27, 0x27, 0xff, 0x61, 0x0c, 0x26, 0xbd, 0xca, 0x46, 0x69, + 0x3d, 0xdf, 0x6e, 0xb3, 0x1a, 0xf1, 0xd3, 0x58, 0x8d, 0x29, 0x1e, 0xbd, 0x11, 0x6e, 0x39, 0x16, + 0x61, 0x9a, 0xba, 0x20, 0xb2, 0xd2, 0x6c, 0xd6, 0x75, 0x17, 0xca, 0x52, 0xbb, 0x94, 0x90, 0xa6, + 0x68, 0xd2, 0x32, 0x4b, 0xa1, 0x20, 0x96, 0xf4, 0xbf, 0x5d, 0x0b, 0xe3, 0x27, 0x58, 0xa6, 0xa8, + 0xea, 0x34, 0xd1, 0x29, 0x19, 0xc6, 0x58, 0x21, 0x7c, 0xbc, 0xe7, 0xbd, 0x07, 0x53, 0x54, 0xb3, + 0x51, 0x1f, 0xc2, 0xe5, 0xcd, 0xf1, 0x13, 0x01, 0x50, 0x50, 0xfe, 0x27, 0xd7, 0x22, 0xb1, 0xe8, + 0x5a, 0xe4, 0x45, 0x40, 0x2c, 0x0a, 0xd1, 0x96, 0x9b, 0xd8, 0x92, 0x6d, 0xac, 0x9a, 0xfc, 0xae, + 0x23, 0x41, 0x12, 0x79, 0xca, 0x16, 0xb6, 0x2a, 0x94, 0x9e, 0x7f, 0x7f, 0x16, 0xc6, 0xb9, 0x32, + 0xb6, 0x0d, 0xdd, 0x34, 0xd0, 0x4d, 0x88, 0xd7, 0xf8, 0x7a, 0x7e, 0x26, 0x74, 0x45, 0xcd, 0xbf, + 0xa1, 0xad, 0x34, 0x22, 0x91, 0xbc, 0x84, 0xa5, 0xd9, 0x72, 0x42, 0xfc, 0x1f, 0x3f, 0x5c, 0x3a, + 0xc8, 0xd2, 0x6c, 0x39, 0xa8, 0x02, 0x93, 0xaa, 0x7f, 0xdf, 0x94, 0x4c, 0xd8, 0xe3, 0x3d, 0x91, + 0x4e, 0xe8, 0x3d, 0x5f, 0xa5, 0x11, 0x29, 0xab, 0xb6, 0x25, 0xa0, 0x42, 0xf0, 0x82, 0xa3, 0x44, + 0x57, 0x64, 0x96, 0x7f, 0x60, 0xb7, 0xfd, 0x72, 0xa5, 0xd2, 0x48, 0xe0, 0x1e, 0x24, 0xf4, 0x06, + 0x24, 0x35, 0x7a, 0x71, 0x0e, 0xef, 0x9a, 0x61, 0xbd, 0xa7, 0xed, 0xae, 0xa2, 0xd2, 0x88, 0xc4, + 0x39, 0xd0, 0x1a, 0x8c, 0xb3, 0x7f, 0x3c, 0xd6, 0x3f, 0xd9, 0x73, 0xe7, 0xae, 0xfb, 0xea, 0xa0, + 0xd2, 0x88, 0x94, 0xd1, 0x7c, 0x2a, 0x7a, 0x05, 0x12, 0xb6, 0xaa, 0xb8, 0x00, 0x70, 0xae, 0xc7, + 0xad, 0x19, 0x3e, 0x33, 0xcd, 0x8d, 0xee, 0xb0, 0x9b, 0x17, 0x9d, 0x23, 0x77, 0x45, 0x2e, 0xac, + 0xf8, 0x6d, 0x67, 0xb1, 0x49, 0xf1, 0x31, 0x25, 0xa0, 0x7b, 0x90, 0x51, 0x88, 0x43, 0x27, 0xd3, + 0xb3, 0x8f, 0x74, 0x09, 0x2e, 0x7c, 0xb3, 0xbb, 0xeb, 0xdc, 0x6a, 0x89, 0x1e, 0xf8, 0x76, 0x89, + 0xbe, 0xa0, 0x06, 0xb6, 0x6a, 0x38, 0x97, 0xe9, 0x2f, 0x28, 0x18, 0x89, 0xe5, 0x09, 0xa2, 0x44, + 0xe2, 0xd8, 0xed, 0xb9, 0xe7, 0x5a, 0x68, 0xa5, 0xc6, 0x7b, 0x6e, 0xac, 0x86, 0x9c, 0xcb, 0x29, + 0x8d, 0x48, 0xe3, 0x7b, 0x01, 0x32, 0x5a, 0x84, 0x58, 0x4d, 0xcd, 0x4d, 0x50, 0x19, 0x17, 0xfb, + 0x9d, 0x3a, 0x29, 0x8d, 0x48, 0xb1, 0x9a, 0x4a, 0xa0, 0x3c, 0x3b, 0x0d, 0x70, 0x64, 0xe4, 0xb2, + 0x3d, 0x87, 0x7a, 0xfb, 0x99, 0x8a, 0xd2, 0x88, 0x44, 0x0f, 0x20, 0x90, 0xf7, 0x6d, 0x41, 0xd6, + 0x62, 0xa1, 0x6c, 0x6e, 0x10, 0xaa, 0xd8, 0x73, 0xb3, 0x39, 0x2c, 0x0e, 0xb5, 0x44, 0x1d, 0xfc, + 0x00, 0x1d, 0x7d, 0x05, 0x66, 0xda, 0x25, 0xf2, 0x9e, 0x36, 0xd5, 0x73, 0xe3, 0xb4, 0x67, 0x34, + 0x64, 0x69, 0x44, 0x42, 0x56, 0x57, 0x22, 0x7a, 0x0d, 0x46, 0x59, 0xab, 0x21, 0x2a, 0x32, 0x2c, + 0x8a, 0xa2, 0xa3, 0xc1, 0x58, 0x7e, 0xd2, 0xf9, 0x1d, 0x1e, 0xc3, 0x25, 0xd7, 0xcd, 0x5a, 0x6e, + 0xba, 0x67, 0xe7, 0xef, 0x8e, 0x49, 0x23, 0x9d, 0xdf, 0xf1, 0xa9, 0xa4, 0xdd, 0x2d, 0x96, 0xc2, + 0x43, 0x7e, 0x66, 0x7a, 0xb6, 0x7b, 0x48, 0x68, 0x57, 0x89, 0x46, 0xd5, 0xfb, 0x64, 0x52, 0x34, + 0x8b, 0x5d, 0xf1, 0x22, 0xd3, 0x31, 0x75, 0xa6, 0x67, 0xd1, 0xba, 0x6f, 0xc2, 0x29, 0x51, 0xc7, + 0xc7, 0xa3, 0xa2, 0x47, 0x20, 0xf2, 0xcb, 0x17, 0xfc, 0xe5, 0xff, 0xb3, 0x54, 0xde, 0x0b, 0xa1, + 0xa6, 0x2b, 0x2c, 0x46, 0xa6, 0x34, 0x22, 0x4d, 0xaa, 0xed, 0x29, 0xe8, 0x1d, 0x98, 0xa2, 0xf2, + 0x64, 0xd5, 0xbf, 0x35, 0x23, 0x97, 0xeb, 0xba, 0x7d, 0xa1, 0xf7, 0x05, 0x1b, 0xae, 0x64, 0x51, + 0xed, 0x48, 0x22, 0xdd, 0x58, 0x37, 0x74, 0x87, 0x5a, 0xd9, 0xd9, 0x9e, 0xdd, 0xb8, 0xfd, 0x7e, + 0x3e, 0xd2, 0x8d, 0x75, 0x46, 0x21, 0xdd, 0xd8, 0xe1, 0xf1, 0x60, 0xbc, 0x39, 0x2e, 0xf6, 0xec, + 0xc6, 0x61, 0x81, 0x63, 0xa4, 0x1b, 0x3b, 0x41, 0x3a, 0xe9, 0xc6, 0xcc, 0x40, 0x74, 0xc8, 0x7d, + 0xae, 0x67, 0x37, 0xee, 0x79, 0xfa, 0x98, 0x74, 0x63, 0xa5, 0x2b, 0x11, 0xad, 0x02, 0x30, 0xbf, + 0x44, 0x37, 0x76, 0xcd, 0xdc, 0x5c, 0xcf, 0xc9, 0xa0, 0x33, 0x22, 0x8c, 0x4c, 0x06, 0x75, 0x97, + 0x46, 0x0c, 0x19, 0x45, 0x43, 0x32, 0xdd, 0x0b, 0xcd, 0xcd, 0xf7, 0x34, 0x64, 0x5d, 0xbb, 0x94, + 0xc4, 0x90, 0x1d, 0x7a, 0x44, 0x32, 0xab, 0xb0, 0x85, 0xd9, 0xdc, 0x42, 0x6f, 0xb3, 0x1c, 0xdc, + 0xa5, 0xa1, 0x66, 0x99, 0x12, 0xd0, 0x32, 0xa4, 0xc9, 0xb4, 0x7d, 0x4c, 0xcd, 0xd0, 0xa5, 0x9e, + 0x2e, 0x66, 0xc7, 0xb1, 0x91, 0xd2, 0x88, 0x94, 0x7a, 0xcc, 0x49, 0xe4, 0xf5, 0x6c, 0x81, 0x2a, + 0x97, 0xef, 0xf9, 0xfa, 0xb6, 0xe5, 0x4d, 0xf2, 0x7a, 0xc6, 0x81, 0x54, 0x38, 0xc3, 0xda, 0x8a, + 0x1f, 0xfe, 0xb5, 0xf8, 0x49, 0xd5, 0xdc, 0xf3, 0x54, 0x54, 0xcf, 0xe5, 0x9e, 0xd0, 0x33, 0xc9, + 0xa5, 0x11, 0x69, 0x5a, 0xe9, 0x4e, 0x25, 0x03, 0x9e, 0x4f, 0x3d, 0x6c, 0x91, 0x28, 0x77, 0xb9, + 0xe7, 0x80, 0x0f, 0x59, 0x56, 0x23, 0x03, 0x5e, 0x09, 0x90, 0xd9, 0x04, 0xa4, 0xc9, 0xb6, 0xcd, + 0x76, 0xce, 0xaf, 0xf4, 0x99, 0x80, 0x3a, 0x60, 0x3e, 0x9b, 0x80, 0xb4, 0x0a, 0xe3, 0x24, 0x82, + 0x82, 0x87, 0xf7, 0xae, 0xf6, 0x14, 0xd4, 0x75, 0xe7, 0x1d, 0x11, 0xa4, 0x7a, 0x44, 0xe2, 0xf0, + 0x58, 0xee, 0xad, 0x2d, 0xdc, 0xe7, 0xbb, 0xd6, 0xd3, 0xe1, 0x09, 0xbd, 0x5c, 0x86, 0x38, 0x3c, + 0x56, 0x5b, 0x02, 0xfa, 0x1c, 0x8c, 0x71, 0x4c, 0x96, 0xbb, 0xde, 0xc7, 0x13, 0x0d, 0x82, 0x69, + 0x32, 0xae, 0x39, 0x0f, 0xb3, 0xb2, 0x0c, 0x0b, 0xb2, 0xea, 0xbd, 0xd0, 0xc7, 0xca, 0x76, 0xc1, + 0x51, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0xac, 0x9f, 0xf2, 0xb9, 0xee, 0x46, 0x4f, 0x2b, 0xdb, + 0x7d, 0x74, 0x85, 0x58, 0xd9, 0xc7, 0x3e, 0x95, 0xd4, 0xcc, 0x66, 0x38, 0x28, 0xf7, 0x99, 0x9e, + 0x35, 0x6b, 0x87, 0x85, 0xa4, 0x66, 0x9c, 0x87, 0x34, 0x1b, 0x0b, 0x44, 0x66, 0x9a, 0x7e, 0xb1, + 0xf7, 0x39, 0xfb, 0x4e, 0xf4, 0x40, 0x9a, 0xcd, 0xf2, 0x88, 0xbe, 0xa1, 0xb2, 0xf8, 0xa9, 0x62, + 0xae, 0xa9, 0x97, 0xfa, 0x1b, 0xaa, 0xb0, 0x03, 0xd3, 0x9e, 0xa1, 0x6a, 0x4b, 0xa4, 0x45, 0x65, + 0x27, 0xc0, 0xe8, 0xf8, 0x5e, 0xec, 0x73, 0x25, 0x40, 0xc7, 0x69, 0x3c, 0x5a, 0x54, 0x8f, 0xe8, + 0x0f, 0xa1, 0x16, 0xbb, 0xbb, 0x22, 0xb7, 0xd4, 0x7f, 0x08, 0xb5, 0xdf, 0xa1, 0xe1, 0x0d, 0x21, + 0x4e, 0xf6, 0xe6, 0x4c, 0xd7, 0xc3, 0x78, 0xb9, 0xff, 0x9c, 0xd9, 0xe9, 0x5a, 0xb0, 0x39, 0x93, + 0xfb, 0x14, 0xff, 0x5b, 0x80, 0x05, 0x56, 0x36, 0xba, 0x64, 0x77, 0x2c, 0x7b, 0xcb, 0x9f, 0x81, + 0x73, 0x0a, 0x37, 0xe9, 0x0b, 0x5e, 0xeb, 0x55, 0xdc, 0x01, 0xcb, 0xb9, 0xa5, 0x11, 0xe9, 0x39, + 0xa5, 0x5f, 0xbe, 0x95, 0x31, 0xbe, 0x77, 0xe9, 0x1d, 0xc2, 0x9c, 0x14, 0xc5, 0xb5, 0x44, 0xea, + 0x9c, 0x98, 0x5b, 0x4b, 0xa4, 0xce, 0x8b, 0xb3, 0x6b, 0x89, 0xd4, 0x05, 0xf1, 0x62, 0xfe, 0x9f, + 0xce, 0xc3, 0x84, 0x0b, 0xde, 0x18, 0x22, 0xba, 0x15, 0x44, 0x44, 0x73, 0xbd, 0x10, 0x11, 0x87, + 0x7b, 0x1c, 0x12, 0xdd, 0x0a, 0x42, 0xa2, 0xb9, 0x5e, 0x90, 0xc8, 0xe7, 0x21, 0x98, 0xa8, 0xda, + 0x0b, 0x13, 0xbd, 0x30, 0x04, 0x26, 0xf2, 0x44, 0x75, 0x82, 0xa2, 0xd5, 0x6e, 0x50, 0x74, 0xb9, + 0x3f, 0x28, 0xf2, 0x44, 0x05, 0x50, 0xd1, 0x9d, 0x0e, 0x54, 0x74, 0xa9, 0x0f, 0x2a, 0xf2, 0xf8, + 0x5d, 0x58, 0xb4, 0x1e, 0x0a, 0x8b, 0xae, 0x0e, 0x82, 0x45, 0x9e, 0x9c, 0x36, 0x5c, 0xf4, 0x6a, + 0x1b, 0x2e, 0x9a, 0xef, 0x89, 0x8b, 0x3c, 0x6e, 0x06, 0x8c, 0xde, 0xec, 0x04, 0x46, 0x97, 0xfa, + 0x00, 0x23, 0xbf, 0x06, 0x1c, 0x19, 0x95, 0xc2, 0x90, 0xd1, 0x95, 0x01, 0xc8, 0xc8, 0x93, 0x12, + 0x84, 0x46, 0xa5, 0x30, 0x68, 0x74, 0x65, 0x00, 0x34, 0xea, 0x90, 0xc4, 0xb0, 0xd1, 0x66, 0x38, + 0x36, 0xba, 0x36, 0x10, 0x1b, 0x79, 0xd2, 0xda, 0xc1, 0xd1, 0x52, 0x00, 0x1c, 0x3d, 0xd7, 0x03, + 0x1c, 0x79, 0xac, 0x04, 0x1d, 0x7d, 0xbe, 0x0b, 0x1d, 0xe5, 0xfb, 0xa1, 0x23, 0x8f, 0xd7, 0x83, + 0x47, 0x0f, 0x7b, 0xc0, 0xa3, 0xeb, 0x83, 0xe1, 0x91, 0x27, 0xac, 0x03, 0x1f, 0x29, 0x7d, 0xf1, + 0xd1, 0x4b, 0x43, 0xe2, 0x23, 0x4f, 0x7a, 0x18, 0x40, 0x7a, 0xbd, 0x1d, 0x20, 0x2d, 0xf4, 0x06, + 0x48, 0x9e, 0x18, 0x8e, 0x90, 0xd6, 0x43, 0x11, 0xd2, 0xd5, 0x41, 0x08, 0xc9, 0x1f, 0x07, 0x41, + 0x88, 0xb4, 0x19, 0x0e, 0x91, 0xae, 0x0d, 0x84, 0x48, 0x7e, 0xf3, 0xb7, 0x61, 0xa4, 0xf5, 0x50, + 0x8c, 0x74, 0x75, 0x10, 0x46, 0xf2, 0x0b, 0x17, 0x04, 0x49, 0x6f, 0xf7, 0x04, 0x49, 0x37, 0x86, + 0x01, 0x49, 0x9e, 0xd0, 0x2e, 0x94, 0xf4, 0x6e, 0x6f, 0x94, 0xf4, 0x99, 0x53, 0x5c, 0x43, 0x18, + 0x0a, 0x93, 0x3e, 0xdf, 0x05, 0x93, 0xf2, 0xfd, 0x60, 0x92, 0xdf, 0x9f, 0x5d, 0x9c, 0xa4, 0xf4, + 0x45, 0x35, 0x2f, 0x0d, 0x89, 0x6a, 0xfc, 0xce, 0x17, 0x02, 0x6b, 0x8a, 0x21, 0xb0, 0xe6, 0x72, + 0x7f, 0x58, 0xe3, 0x9b, 0x73, 0x1f, 0xd7, 0x94, 0xc2, 0x70, 0xcd, 0x95, 0x01, 0xb8, 0xc6, 0xb7, + 0x42, 0x01, 0x60, 0x73, 0xa7, 0x03, 0xd8, 0x5c, 0x1a, 0x18, 0x9a, 0x13, 0x40, 0x36, 0x2b, 0xdd, + 0xc8, 0xe6, 0xf9, 0xbe, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0x3b, 0x1d, 0xd0, 0xe6, 0x52, 0x1f, + 0x68, 0xe3, 0x17, 0x80, 0x63, 0x1b, 0xad, 0x3f, 0xb6, 0x59, 0x1c, 0x16, 0xdb, 0x78, 0x82, 0x43, + 0xc1, 0xcd, 0x66, 0x38, 0xb8, 0xb9, 0x36, 0xe4, 0x46, 0x79, 0x17, 0xba, 0x29, 0x85, 0xa1, 0x9b, + 0x2b, 0x03, 0xd0, 0x4d, 0x70, 0x0e, 0xf1, 0xe0, 0x4d, 0x29, 0x0c, 0xde, 0x5c, 0x19, 0x00, 0x6f, + 0x7c, 0x49, 0x01, 0x7c, 0x53, 0xed, 0x85, 0x6f, 0x5e, 0x18, 0x02, 0xdf, 0xf8, 0xce, 0x4b, 0x07, + 0xc0, 0x79, 0xab, 0x13, 0xe0, 0xe4, 0xfb, 0x01, 0x1c, 0x7f, 0x44, 0xba, 0x08, 0x67, 0x33, 0x1c, + 0xe1, 0x5c, 0x1b, 0x88, 0x70, 0x82, 0x46, 0x32, 0x00, 0x71, 0xd6, 0x43, 0x21, 0xce, 0xd5, 0x41, + 0x10, 0xc7, 0x37, 0x92, 0x41, 0x8c, 0xf3, 0x56, 0x27, 0xc6, 0xc9, 0xf7, 0xc3, 0x38, 0x7e, 0xe5, + 0x5c, 0x90, 0x53, 0x0a, 0x03, 0x39, 0x57, 0x06, 0x80, 0x1c, 0xbf, 0xf1, 0x02, 0x28, 0x47, 0xe9, + 0x8b, 0x72, 0x5e, 0x1a, 0x12, 0xe5, 0x74, 0x18, 0xae, 0x76, 0x98, 0x53, 0x0a, 0x83, 0x39, 0x57, + 0x06, 0xc0, 0x9c, 0x40, 0x61, 0x7d, 0x9c, 0xb3, 0x19, 0x8e, 0x73, 0xae, 0x0d, 0xc4, 0x39, 0x1d, + 0xa3, 0xc9, 0x05, 0x3a, 0xeb, 0xa1, 0x40, 0xe7, 0xea, 0x20, 0xa0, 0xd3, 0x31, 0xf1, 0x71, 0xe7, + 0xe0, 0xff, 0x0c, 0x8f, 0x74, 0x5e, 0x3f, 0x3d, 0xd2, 0xf1, 0xde, 0x19, 0x09, 0xd4, 0x59, 0x4b, + 0xa4, 0x2e, 0x8a, 0xcf, 0xe5, 0x7f, 0x3e, 0x0a, 0xc9, 0x92, 0x17, 0xce, 0xe2, 0x97, 0x52, 0x78, + 0xa6, 0x1b, 0x8f, 0x56, 0xc9, 0x88, 0xa5, 0x76, 0x6f, 0xf0, 0xed, 0x76, 0xdd, 0x37, 0xaf, 0x71, + 0xd6, 0x67, 0x38, 0x48, 0x8c, 0x5e, 0x85, 0x89, 0x96, 0x8d, 0x2d, 0xb9, 0x69, 0xe9, 0xa6, 0xa5, + 0x3b, 0xec, 0x50, 0x86, 0xb0, 0x22, 0x7e, 0x7c, 0x32, 0x3f, 0xbe, 0x6d, 0x63, 0x6b, 0x8b, 0xd3, + 0xa5, 0xf1, 0x56, 0xe0, 0xc9, 0xfd, 0x0a, 0xd4, 0xe8, 0xf0, 0x5f, 0x81, 0x7a, 0x08, 0xa2, 0x85, + 0x15, 0xad, 0xcd, 0x03, 0x61, 0x37, 0x05, 0x85, 0xf7, 0x19, 0x7a, 0xde, 0xc9, 0xcd, 0x49, 0x6f, + 0x0c, 0x9a, 0xb4, 0xda, 0x89, 0xe8, 0x26, 0x9c, 0x69, 0x28, 0x47, 0x34, 0x70, 0x51, 0x76, 0x9d, + 0x3a, 0x1a, 0x8c, 0xc8, 0x3e, 0xb0, 0x84, 0x1a, 0xca, 0x11, 0xfd, 0xa4, 0x14, 0x4b, 0xa2, 0xdf, + 0x80, 0xb8, 0x02, 0x59, 0x4d, 0xb7, 0x1d, 0xdd, 0x50, 0x1d, 0x7e, 0x99, 0x2c, 0xbb, 0x9d, 0x75, + 0xc2, 0xa5, 0xb2, 0x1b, 0x63, 0x6f, 0xc0, 0x14, 0x8f, 0x6b, 0xf7, 0xbf, 0x2c, 0x45, 0xe1, 0x4b, + 0x8a, 0x94, 0x82, 0x24, 0x78, 0x1f, 0x8c, 0x42, 0x05, 0x98, 0xac, 0x29, 0x0e, 0x3e, 0x54, 0x8e, + 0x65, 0xf7, 0x50, 0x54, 0x86, 0xde, 0xc5, 0x78, 0xe1, 0xe9, 0xc9, 0xfc, 0xc4, 0x3d, 0x96, 0xd4, + 0x75, 0x36, 0x6a, 0xa2, 0x16, 0x48, 0xd0, 0xd0, 0x35, 0x98, 0x54, 0xec, 0x63, 0x43, 0xa5, 0xea, + 0xc1, 0x86, 0xdd, 0xb2, 0x29, 0xa4, 0x48, 0x49, 0x59, 0x4a, 0x2e, 0xb8, 0x54, 0x74, 0x09, 0xc6, + 0x79, 0xd0, 0x37, 0xfb, 0x2e, 0xcd, 0x24, 0xad, 0x2a, 0xff, 0x4c, 0x02, 0xfd, 0x34, 0x0d, 0xba, + 0x03, 0xb3, 0xfc, 0xfa, 0xf8, 0x43, 0xc5, 0xd2, 0x64, 0xaa, 0x75, 0xbf, 0x7f, 0x8a, 0x54, 0xec, + 0x39, 0x76, 0x5d, 0x3c, 0xc9, 0x40, 0x54, 0x1d, 0xbc, 0x6b, 0x75, 0x4c, 0x4c, 0xad, 0x25, 0x52, + 0xe3, 0xe2, 0xc4, 0x5a, 0x22, 0x95, 0x15, 0x27, 0xf3, 0xbf, 0x21, 0xc0, 0x78, 0xdb, 0x41, 0x92, + 0x3b, 0x1d, 0xfb, 0xb8, 0xe7, 0xc3, 0xa1, 0x53, 0xaf, 0xd0, 0xaf, 0x14, 0x6f, 0x2a, 0x37, 0xf0, + 0x6d, 0xbe, 0xb7, 0xeb, 0x4d, 0x17, 0x12, 0xdc, 0xe0, 0x01, 0x97, 0xed, 0x8d, 0xc4, 0x6f, 0x7d, + 0x30, 0x3f, 0x92, 0xff, 0x30, 0x01, 0x13, 0xed, 0x07, 0x46, 0xca, 0x1d, 0xe5, 0x0a, 0x33, 0x6d, + 0x6d, 0x1c, 0x8b, 0x7d, 0x2e, 0xd8, 0x4b, 0xfb, 0xf7, 0xbf, 0xb3, 0x62, 0x2e, 0xf4, 0xd9, 0xad, + 0x0e, 0x96, 0xd3, 0x67, 0x9c, 0xfd, 0xbf, 0x71, 0xcf, 0x44, 0x2c, 0xc2, 0x28, 0xbd, 0xc9, 0x85, + 0x17, 0x2d, 0xec, 0x2c, 0x72, 0x91, 0xa4, 0x4b, 0x2c, 0x1b, 0x31, 0x29, 0xd5, 0x67, 0xba, 0x44, + 0xcd, 0xbf, 0x87, 0xe2, 0xf4, 0x1f, 0x6a, 0xe3, 0x77, 0xe9, 0x8d, 0x9e, 0xee, 0x2e, 0x3d, 0xb6, + 0x29, 0x5d, 0xaf, 0x33, 0x73, 0xcd, 0x06, 0x55, 0xb2, 0xeb, 0xc0, 0x2f, 0x15, 0xc1, 0xbf, 0x9f, + 0xb7, 0x28, 0xf1, 0xef, 0xe7, 0x05, 0x62, 0x11, 0xb3, 0x9e, 0x08, 0x36, 0x02, 0x0b, 0xee, 0x2c, + 0xcd, 0x3e, 0xea, 0x36, 0x36, 0xf4, 0x47, 0xdd, 0xd8, 0x04, 0x4d, 0x3f, 0xe5, 0xc6, 0xc2, 0x5e, + 0x79, 0x7f, 0xf9, 0x96, 0x00, 0x22, 0xcd, 0x7b, 0x17, 0x63, 0x2d, 0x92, 0xae, 0xec, 0xc6, 0x5a, + 0xc6, 0x86, 0x0f, 0x66, 0x6f, 0xbb, 0xd4, 0x3f, 0xde, 0x7e, 0xa9, 0x7f, 0xfe, 0x03, 0x01, 0xb2, + 0x5e, 0x09, 0xd9, 0xe7, 0xac, 0xfa, 0x5c, 0xb3, 0xf7, 0x6c, 0x1f, 0x71, 0x72, 0x6f, 0x04, 0x18, + 0xea, 0x0b, 0x5b, 0xc1, 0x1b, 0x01, 0xd8, 0x07, 0x87, 0x7e, 0x47, 0x80, 0x69, 0xaf, 0x88, 0x05, + 0xff, 0xb4, 0xf7, 0x33, 0xc4, 0xf5, 0x4b, 0xf4, 0xab, 0x80, 0x66, 0xfd, 0x80, 0x5d, 0xc5, 0x30, + 0x54, 0x1f, 0x47, 0x3c, 0x82, 0x03, 0xf8, 0xea, 0x83, 0x56, 0xad, 0xd0, 0xef, 0x05, 0xb2, 0xff, + 0x76, 0xfe, 0x6e, 0x40, 0x81, 0x74, 0x38, 0x11, 0x2d, 0x0d, 0x35, 0xee, 0x5c, 0x2d, 0xd1, 0xcc, + 0xf9, 0x1f, 0x04, 0x5b, 0xa2, 0x78, 0x40, 0xbc, 0xce, 0xdb, 0x10, 0x3f, 0x50, 0xea, 0xfd, 0x22, + 0x57, 0xda, 0x5a, 0x4e, 0x22, 0xb9, 0xd1, 0xdd, 0xb6, 0x43, 0xf2, 0xb1, 0xde, 0x1e, 0x52, 0xb7, + 0x4a, 0xdb, 0x0e, 0xd3, 0xbf, 0xe6, 0xd6, 0x22, 0x3e, 0xf8, 0xf5, 0x41, 0x33, 0xf2, 0x46, 0xe2, + 0xa3, 0x0f, 0xe6, 0x85, 0x1b, 0x15, 0x98, 0x0e, 0x99, 0x4f, 0x51, 0x16, 0x20, 0x70, 0xd5, 0x3f, + 0xff, 0xc4, 0xe0, 0xf2, 0xaa, 0xbc, 0xbd, 0x59, 0x78, 0xb0, 0xb1, 0x51, 0xae, 0x56, 0x8b, 0xab, + 0xa2, 0x80, 0x44, 0x18, 0x6f, 0xfb, 0x50, 0x40, 0x8c, 0x7d, 0x74, 0xf0, 0xc6, 0xff, 0x00, 0xf0, + 0xbf, 0x3f, 0x42, 0x64, 0xad, 0x17, 0xdf, 0x91, 0x1f, 0x2d, 0xdf, 0xdf, 0x2e, 0x56, 0xc4, 0x11, + 0x84, 0x20, 0xbb, 0xb2, 0x5c, 0x2d, 0x94, 0x64, 0xa9, 0x58, 0xd9, 0x7a, 0xb0, 0x59, 0x29, 0xba, + 0x1f, 0x2b, 0xbc, 0xb1, 0x0a, 0xe3, 0xc1, 0xeb, 0x04, 0xd0, 0x34, 0x4c, 0x16, 0x4a, 0xc5, 0xc2, + 0xba, 0xfc, 0xa8, 0xbc, 0x2c, 0x3f, 0xdc, 0x2e, 0x6e, 0x17, 0xc5, 0x11, 0x5a, 0x34, 0x4a, 0xbc, + 0xbb, 0x7d, 0xff, 0xbe, 0x28, 0xa0, 0x49, 0xc8, 0xb0, 0x67, 0xfa, 0x51, 0x01, 0x31, 0x76, 0x63, + 0x03, 0x32, 0x81, 0xcb, 0x04, 0xc9, 0xeb, 0xb6, 0xb6, 0x2b, 0x25, 0xb9, 0x5a, 0xde, 0x28, 0x56, + 0xaa, 0xcb, 0x1b, 0x5b, 0x4c, 0x06, 0xa5, 0x2d, 0xaf, 0x3c, 0x90, 0xaa, 0xa2, 0xe0, 0x3d, 0x57, + 0x1f, 0x6c, 0x17, 0x4a, 0x6e, 0x35, 0xf2, 0x89, 0x54, 0x5c, 0x8c, 0xdf, 0xf8, 0x9a, 0x00, 0xe7, + 0x7a, 0x1c, 0xaa, 0x47, 0x19, 0x18, 0xdb, 0x36, 0xe8, 0x6d, 0x6a, 0xe2, 0x08, 0x9a, 0x08, 0x9c, + 0xab, 0x17, 0x05, 0x94, 0x62, 0x67, 0x9a, 0xc5, 0x18, 0x4a, 0x42, 0xac, 0x72, 0x5b, 0x8c, 0x93, + 0x92, 0x06, 0x8e, 0xa5, 0x8b, 0x09, 0x94, 0xe6, 0xa7, 0x6a, 0xc5, 0x51, 0x34, 0xee, 0x1f, 0x6b, + 0x15, 0x93, 0x44, 0x94, 0x77, 0x30, 0x54, 0x1c, 0xbb, 0x71, 0x09, 0x02, 0x87, 0xec, 0x10, 0x40, + 0xf2, 0xbe, 0xe2, 0x60, 0xdb, 0x11, 0x47, 0xd0, 0x18, 0xc4, 0x97, 0xeb, 0x75, 0x51, 0xb8, 0xf5, + 0x47, 0x02, 0xa4, 0xdc, 0x6b, 0xf3, 0xd1, 0x7d, 0x18, 0x65, 0x4b, 0x09, 0xf3, 0xbd, 0xa7, 0x39, + 0x6a, 0xe4, 0x66, 0x17, 0x06, 0xcd, 0x83, 0xf9, 0x11, 0xf4, 0x36, 0xa4, 0xbd, 0x1e, 0x84, 0x9e, + 0xef, 0xd7, 0xbf, 0x5c, 0xa9, 0xfd, 0x3b, 0x21, 0x19, 0x33, 0xf9, 0x91, 0x97, 0x85, 0x95, 0x17, + 0x3e, 0xfa, 0xe9, 0xdc, 0xc8, 0x47, 0x4f, 0xe7, 0x84, 0x1f, 0x3e, 0x9d, 0x13, 0x7e, 0xfc, 0x74, + 0x4e, 0xf8, 0xc9, 0xd3, 0x39, 0xe1, 0xd7, 0x7e, 0x36, 0x37, 0xf2, 0xc3, 0x9f, 0xcd, 0x8d, 0xfc, + 0xf8, 0x67, 0x73, 0x23, 0xef, 0x8e, 0x71, 0xee, 0x9d, 0x24, 0xfd, 0x64, 0xea, 0xed, 0xff, 0x0c, + 0x00, 0x00, 0xff, 0xff, 0xb1, 0x53, 0xdb, 0xb6, 0x55, 0x76, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 8c62c84e906d..5073b6deca4b 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -874,6 +874,16 @@ message GCRequest { bytes key = 1 [(gogoproto.casttype) = "Key"]; util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + + // UseClearRange indicates that the evaluation of deleting versions of this + // key should utilize a ClearRange operation instead of issuing point + // deletes. We delegate this decision to the caller which has the visibility + // to determine whether there are sufficient versions to justify this + // approach. This field was introduced in version 20.2 and is gated on that + // version. Nodes at earlier versions would happily ignore this value but + // it remains gated to avoid the negative consequences of sending a single + // key with an extremely large number of versions. + bool use_clear_range = 3; } repeated GCKey keys = 3 [(gogoproto.nullable) = false]; // Threshold is the expiration timestamp. diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 6d555f860a7f..6be8fd9780d4 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -3164,7 +3164,7 @@ func MVCCGarbageCollect( var count int64 defer func(begin time.Time) { - log.Eventf(ctx, "done with GC evaluation for %d keys at %.2f keys/sec. Deleted %d entries", + log.VEventf(ctx, 2, "done with GC evaluation for %d keys at %.2f keys/sec. Deleted %d entries", len(keys), float64(len(keys))*1e9/float64(timeutil.Since(begin)), count) }(timeutil.Now()) @@ -3328,11 +3328,20 @@ func MVCCGarbageCollect( valSize, nil, fromNS)) } count++ - if err := rw.Clear(unsafeIterKey); err != nil { - return err + if !gcKey.UseClearRange { + if err := rw.Clear(unsafeIterKey); err != nil { + return err + } } prevNanos = unsafeIterKey.Timestamp.WallTime } + if gcKey.UseClearRange { + start := MVCCKey{Key: gcKey.Key, Timestamp: gcKey.Timestamp} + end := MVCCKey{Key: gcKey.Key, Timestamp: hlc.Timestamp{WallTime: 1}} + if err := rw.ClearRange(start, end); err != nil { + return err + } + } } return nil