From 6801349297f7a8c61f30770e816b1806d80fce96 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. The decision to use a ClearRange is controlled by whether an entire batch would be used to clear versions of a single key. This means that there we'll only send a clear range if there are at least ` * > 256 KiB`. There's any additional sanity check that the `` is greater than 32 in order to prevent issuing lots of `ClearRange` operations when the cluster has gigantic keys. This new functionality is gated behind both a version and an experimental cluster setting. I'm skipping the release note because of the experimental flag. Release note: None --- 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/clusterversion/versionkey_string.go | 5 +- pkg/kv/kvserver/gc/gc.go | 70 +- pkg/kv/kvserver/gc/gc_old_test.go | 1 + pkg/kv/kvserver/gc/gc_random_test.go | 32 +- pkg/kv/kvserver/gc/gc_test.go | 156 +++ pkg/kv/kvserver/gc_queue.go | 65 +- pkg/kv/kvserver/gc_queue_test.go | 3 +- pkg/roachpb/api.pb.go | 1274 +++++++++++----------- pkg/roachpb/api.proto | 10 + pkg/storage/mvcc.go | 15 +- pkg/storage/mvcc_test.go | 70 +- 16 files changed, 1074 insertions(+), 699 deletions(-) diff --git a/c-deps/libroach/protos/roachpb/api.pb.cc b/c-deps/libroach/protos/roachpb/api.pb.cc index 9fb86284838c..82713486b16f 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.cc +++ b/c-deps/libroach/protos/roachpb/api.pb.cc @@ -14248,6 +14248,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() @@ -14270,12 +14271,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() { @@ -14308,6 +14312,7 @@ void GCRequest_GCKey::Clear() { delete timestamp_; } timestamp_ = NULL; + use_clear_range_ = false; _internal_metadata_.Clear(); } @@ -14349,6 +14354,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) { @@ -14385,6 +14404,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) @@ -14408,6 +14432,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; @@ -14432,6 +14461,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) { @@ -14454,6 +14486,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 a7e1a1317cf6..47e83ff5b696 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.h +++ b/c-deps/libroach/protos/roachpb/api.pb.h @@ -6377,12 +6377,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; }; @@ -22079,6 +22086,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 ca82e08ce5a5..dfe102994b4a 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-13set the active cluster version in the format '.' +versioncustom validation20.1-14set 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 ce859fb84c6f..31ac9a8446c4 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -73,6 +73,7 @@ const ( VersionNodeMembershipStatus VersionRangeStatsRespHasDesc VersionMinPasswordLength + VersionClearRangeForGC // Add new versions here (step one of two). ) @@ -551,6 +552,12 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: VersionMinPasswordLength, Version: roachpb.Version{Major: 20, Minor: 1, Unstable: 13}, }, + { + // 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: 14}, + }, // Add new versions here (step two of two). diff --git a/pkg/clusterversion/versionkey_string.go b/pkg/clusterversion/versionkey_string.go index 5081c66b9b90..4c2dfdcfb3d1 100644 --- a/pkg/clusterversion/versionkey_string.go +++ b/pkg/clusterversion/versionkey_string.go @@ -49,11 +49,12 @@ func _() { _ = x[VersionNodeMembershipStatus-38] _ = x[VersionRangeStatsRespHasDesc-39] _ = x[VersionMinPasswordLength-40] + _ = x[VersionClearRangeForGC-41] } -const _VersionKey_name = "Version19_1VersionStart19_2VersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionRootPasswordVersionNoExplicitForeignKeyIndexIDsVersionHashShardedIndexesVersionCreateRolePrivilegeVersionStatementDiagnosticsSystemTablesVersionSchemaChangeJobVersionSavepointsVersionTimeTZTypeVersionTimePrecisionVersion20_1VersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLength" +const _VersionKey_name = "Version19_1VersionStart19_2VersionLearnerReplicasVersionTopLevelForeignKeysVersionAtomicChangeReplicasTriggerVersionAtomicChangeReplicasVersionTableDescModificationTimeFromMVCCVersionPartitionedBackupVersion19_2VersionStart20_1VersionContainsEstimatesCounterVersionChangeReplicasDemotionVersionSecondaryIndexColumnFamiliesVersionNamespaceTableWithSchemasVersionProtectedTimestampsVersionPrimaryKeyChangesVersionAuthLocalAndTrustRejectMethodsVersionPrimaryKeyColumnsOutOfFamilyZeroVersionRootPasswordVersionNoExplicitForeignKeyIndexIDsVersionHashShardedIndexesVersionCreateRolePrivilegeVersionStatementDiagnosticsSystemTablesVersionSchemaChangeJobVersionSavepointsVersionTimeTZTypeVersionTimePrecisionVersion20_1VersionStart20_2VersionGeospatialTypeVersionEnumsVersionRangefeedLeasesVersionAlterColumnTypeGeneralVersionAlterSystemJobsAddCreatedByColumnsVersionAddScheduledJobsTableVersionUserDefinedSchemasVersionNoOriginFKIndexesVersionClientRangeInfosOnBatchResponseVersionNodeMembershipStatusVersionRangeStatsRespHasDescVersionMinPasswordLengthVersionClearRangeForGC" -var _VersionKey_index = [...]uint16{0, 11, 27, 49, 75, 109, 136, 176, 200, 211, 227, 258, 287, 322, 354, 380, 404, 441, 480, 499, 534, 559, 585, 624, 646, 663, 680, 700, 711, 727, 748, 760, 782, 811, 852, 880, 905, 929, 967, 994, 1022, 1046} +var _VersionKey_index = [...]uint16{0, 11, 27, 49, 75, 109, 136, 176, 200, 211, 227, 258, 287, 322, 354, 380, 404, 441, 480, 499, 534, 559, 585, 624, 646, 663, 680, 700, 711, 727, 748, 760, 782, 811, 852, 880, 905, 929, 967, 994, 1022, 1046, 1068} func (i VersionKey) String() string { if i < 0 || i >= VersionKey(len(_VersionKey_index)-1) { diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 71aa5a343254..f711d8dcabec 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 + garbageVersionsForThisKey int + gcTimestampForThisKey hlc.Timestamp + keyBytesForThisKey int64 + sentBatchForThisKey bool + useClearRangeForThisKey bool ) it := makeGCIterator(desc, snap) defer it.close() @@ -310,32 +315,65 @@ func processReplicatedKeyRange( isNewest := s.curIsNewest() if isGarbage(threshold, s.cur, s.next, isNewest) { keyBytes := int64(s.cur.Key.EncodedSize()) - batchGCKeysBytes += keyBytes - haveGarbageForThisKey = true + // 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 + } + garbageVersionsForThisKey++ gcTimestampForThisKey = s.cur.Key.Timestamp info.AffectedVersionsKeyBytes += keyBytes info.AffectedVersionsValBytes += int64(len(s.cur.Value)) } - if affected := isNewest && (sentBatchForThisKey || haveGarbageForThisKey); affected { + if affected := isNewest && (sentBatchForThisKey || garbageVersionsForThisKey > 0); 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. We don't want to send a clear range if the key is + // very large and we might potentially be sending a clear range for a + // small number of versions. If the key size is more than + // KeyVersionChunkBytes / minGarbageVersionsForClearRange (4KiB) then + // ClearRange will not be used. + const minGarbageVersionsForClearRange = 32 + useClearRangeForThisKey = canUseClearRange && + len(batchGCKeys) == 0 && + !sentBatchForThisKey && + garbageVersionsForThisKey > minGarbageVersionsForClearRange + 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 && garbageVersionsForThisKey > 0); 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 + garbageVersionsForThisKey = 0 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..20e8339abe42 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -96,19 +96,19 @@ 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 /* canUseClearRange */) require.NoError(t, err) oldGCer.normalize() @@ -119,6 +119,16 @@ func TestRunNewVsOld(t *testing.T) { } } +func noopCleanupIntentsFunc(ctx context.Context, intents []roachpb.Intent) error { + return nil +} + +func noopCleanupIntentsAsyncFunc( + ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate, +) error { + return nil +} + // BenchmarkRun benchmarks the old and implementations of Run with different // data distributions. func BenchmarkRun(b *testing.B) { @@ -131,16 +141,14 @@ 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, NoopGCer{}, - func(ctx context.Context, intents []roachpb.Intent) error { - return nil - }, - func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { - return nil - }) + noopCleanupIntentsFunc, + noopCleanupIntentsAsyncFunc, + useClearRange) } makeTest := func(old bool, spec randomRunGCTestSpec) func(b *testing.B) { return func(b *testing.B) { diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index efc67fff2f77..5eab4c4bae48 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -13,13 +13,17 @@ package gc import ( "bytes" "context" + "strings" "testing" "time" "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -42,6 +46,10 @@ type collectingGCer struct { keys [][]roachpb.GCRequest_GCKey } +func (c *collectingGCer) SetGCThreshold(context.Context, Threshold) error { + return nil +} + func (c *collectingGCer) GC(_ context.Context, keys []roachpb.GCRequest_GCKey) error { c.keys = append(c.keys, keys) return nil @@ -89,3 +97,151 @@ func TestBatchingInlineGCer(t *testing.T) { require.Nil(t, m.gcKeys) require.Zero(t, m.size) } + +// TestUseClearRange tests that the GC logic properly issues request keys +// with UseClearRange set when Run is called with useClearRange and the +// appropriate situations occur. +func TestUseClearRange(t *testing.T) { + secondsToNanos := func(seconds int) (nanos int64) { + return (time.Second * time.Duration(seconds)).Nanoseconds() + } + mkTs := func(seconds int) hlc.Timestamp { + return hlc.Timestamp{WallTime: secondsToNanos(seconds)} + } + type versionsSpec struct { + start, end hlc.Timestamp + step time.Duration + } + type keyVersions struct { + key roachpb.Key + versions versionsSpec + expectClearRange bool + expectBatches int + } + type testCase struct { + name string + useClearRange bool + now hlc.Timestamp + ttl int + keys []keyVersions + } + oneHundredVersions := versionsSpec{start: mkTs(1), end: mkTs(100), step: time.Second} + tenThousandVersions := versionsSpec{start: mkTs(1), end: mkTs(10000), step: time.Second} + value := []byte("foo") + run := func(t *testing.T, tc testCase) { + eng := storage.NewDefaultInMem() + defer eng.Close() + for _, k := range tc.keys { + ts := k.versions.start + for ts.LessEq(k.versions.end) { + require.NoError(t, eng.Put(storage.MVCCKey{ + Key: k.key, + Timestamp: ts, + }, value)) + ts = ts.Add(k.versions.step.Nanoseconds(), 0) + } + } + desc := roachpb.RangeDescriptor{ + RangeID: 1, + StartKey: roachpb.RKey(keys.MinKey), + EndKey: roachpb.RKey(keys.MaxKey), + NextReplicaID: 1, + } + snap := eng.NewSnapshot() + defer snap.Close() + var gcer collectingGCer + newThreshold := tc.now.Add(secondsToNanos(tc.ttl), 0) + _, err := Run(context.Background(), &desc, snap, tc.now, newThreshold, + zonepb.GCPolicy{TTLSeconds: int32(tc.ttl)}, &gcer, + noopCleanupIntentsFunc, noopCleanupIntentsAsyncFunc, + tc.useClearRange) + require.NoError(t, err) + keyBatchesSeen := make(map[string]int) + keyClearRangesSeen := make(map[string]int) + for _, b := range gcer.keys { + for _, k := range b { + keyStr := string(k.Key) + keyBatchesSeen[keyStr]++ + if k.UseClearRange { + keyClearRangesSeen[keyStr]++ + } + } + } + for _, k := range tc.keys { + keyStr := string(k.key) + assert.Equal(t, k.expectBatches, keyBatchesSeen[keyStr], keyStr) + var expectedClearRanges int + if k.expectClearRange { + expectedClearRanges = 1 + } + assert.Equal(t, expectedClearRanges, keyClearRangesSeen[keyStr], keyStr) + } + } + tests := []testCase{ + { + name: "basic clear range enabled, one clear range batch", + useClearRange: true, + now: mkTs(1000000), + ttl: 1, + keys: []keyVersions{ + { + key: roachpb.Key("a"), + versions: oneHundredVersions, + expectBatches: 1, + }, + { + key: roachpb.Key(strings.Repeat("a", 256)), + versions: oneHundredVersions, + expectBatches: 1, + }, + { + key: roachpb.Key(strings.Repeat("b", 256)), + versions: tenThousandVersions, + expectClearRange: true, + expectBatches: 1, + }, + }, + }, + { + name: "no clear range because the keys are too large", + useClearRange: true, + now: mkTs(1000000), + ttl: 1, + keys: []keyVersions{ + { + key: roachpb.Key(strings.Repeat("a", 32<<10)), + versions: oneHundredVersions, + expectBatches: (((32 << 10) * 100) / (256 << 10)) + 1, // 12 + }, + }, + }, + { + name: "basic clear range disabled", + useClearRange: false, + now: mkTs(1000000), + ttl: 1, + keys: []keyVersions{ + { + key: roachpb.Key("a"), + versions: oneHundredVersions, + expectBatches: 1, + }, + { + key: roachpb.Key(strings.Repeat("a", 256)), + versions: oneHundredVersions, + expectBatches: 1, + }, + { + key: roachpb.Key(strings.Repeat("b", 256)), + versions: tenThousandVersions, + expectBatches: 11, + }, + }, + }, + } + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + run(t, tc) + }) + } +} 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 faa83d2be950..1641bef3a309 100644 --- a/pkg/kv/kvserver/gc_queue_test.go +++ b/pkg/kv/kvserver/gc_queue_test.go @@ -560,7 +560,8 @@ func TestGCQueueProcess(t *testing.T) { }, func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { return nil - }) + }, + false /* canUseClearRange */) }() if err != nil { t.Fatal(err) diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 7a3917269b7b..c9c27b63f1ad 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_bfad9ab5d8946a01, []int{0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{1} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{2} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{3} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{4} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{5} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{1, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{25, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{1} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{2} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{3} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{4} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{5} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{6} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{7} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{8} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{9} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{10} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{11} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{12} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{13} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{14} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{15} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{16} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{17} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{18} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{19} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{20} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{21} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{22} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{23} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{24} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{25} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{25, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{26} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{27} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{28} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{29} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{30} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{31} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{32} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{33} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{34} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{35} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{36} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{37} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{38} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{39} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{40} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{41} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{42} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{43} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{44} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{45} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{45, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{46} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{47} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{48} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{49} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{50} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{51} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{52} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{53} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{54} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{55} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{56} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{57} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{58} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{59} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{60} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{61} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{62} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{63} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{64} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{65} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{66} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{67} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{68} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{69} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70, 1} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70, 2} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70, 3} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70, 4} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70, 5} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{70, 6} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{71} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{72} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{73} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{74} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{75} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{76} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{76, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{77} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{77, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{77, 1} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{78} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{79} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{80} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{80, 0} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{81} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{82} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{83} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{84} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{85} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{86} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{87} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{88} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{89} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{90} + return fileDescriptor_api_f754b08a08b16d03, []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_bfad9ab5d8946a01, []int{91} + return fileDescriptor_api_f754b08a08b16d03, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4759,7 +4768,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_bfad9ab5d8946a01, []int{92} + return fileDescriptor_api_f754b08a08b16d03, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4842,7 +4851,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_bfad9ab5d8946a01, []int{93} + return fileDescriptor_api_f754b08a08b16d03, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6292,7 +6301,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_bfad9ab5d8946a01, []int{94} + return fileDescriptor_api_f754b08a08b16d03, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7807,7 +7816,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_bfad9ab5d8946a01, []int{95} + return fileDescriptor_api_f754b08a08b16d03, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7843,7 +7852,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_bfad9ab5d8946a01, []int{96} + return fileDescriptor_api_f754b08a08b16d03, []int{96} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7880,7 +7889,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_bfad9ab5d8946a01, []int{97} + return fileDescriptor_api_f754b08a08b16d03, []int{97} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7946,7 +7955,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_bfad9ab5d8946a01, []int{97, 0} + return fileDescriptor_api_f754b08a08b16d03, []int{97, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7984,7 +7993,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_bfad9ab5d8946a01, []int{98} + return fileDescriptor_api_f754b08a08b16d03, []int{98} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8022,7 +8031,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_bfad9ab5d8946a01, []int{99} + return fileDescriptor_api_f754b08a08b16d03, []int{99} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8061,7 +8070,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_bfad9ab5d8946a01, []int{100} + return fileDescriptor_api_f754b08a08b16d03, []int{100} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8102,7 +8111,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_bfad9ab5d8946a01, []int{101} + return fileDescriptor_api_f754b08a08b16d03, []int{101} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8143,7 +8152,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_bfad9ab5d8946a01, []int{102} + return fileDescriptor_api_f754b08a08b16d03, []int{102} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8180,7 +8189,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_bfad9ab5d8946a01, []int{103} + return fileDescriptor_api_f754b08a08b16d03, []int{103} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8217,7 +8226,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_bfad9ab5d8946a01, []int{104} + return fileDescriptor_api_f754b08a08b16d03, []int{104} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8258,7 +8267,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_bfad9ab5d8946a01, []int{105} + return fileDescriptor_api_f754b08a08b16d03, []int{105} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8298,7 +8307,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_bfad9ab5d8946a01, []int{106} + return fileDescriptor_api_f754b08a08b16d03, []int{106} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -9230,6 +9239,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 { @@ -12419,6 +12431,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 } @@ -17740,6 +17762,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 } @@ -25814,6 +25839,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:]) @@ -38855,489 +38900,490 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_bfad9ab5d8946a01) } - -var fileDescriptor_api_bfad9ab5d8946a01 = []byte{ - // 7688 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6b, 0x6c, 0x24, 0xc7, - 0xb5, 0x1e, 0x7b, 0x66, 0x48, 0xce, 0x9c, 0x19, 0x0e, 0x9b, 0x45, 0xee, 0xee, 0x2c, 0x57, 0x22, - 0xb9, 0xa3, 0x7d, 0x69, 0x2d, 0x91, 0xda, 0x5d, 0x29, 0x92, 0xb5, 0xb2, 0x6c, 0x72, 0x38, 0xbb, - 0x43, 0x72, 0xc9, 0xe5, 0xf6, 0x0c, 0x57, 0x96, 0x6c, 0xa5, 0xdd, 0xec, 0x2e, 0x0e, 0xdb, 0x9c, - 0xe9, 0x9e, 0xed, 0xee, 0xe1, 0x63, 0x81, 0x00, 0xce, 0x0b, 0x0e, 0x8c, 0x40, 0xc8, 0x8f, 0x20, - 0x08, 0xe2, 0x18, 0x16, 0xe0, 0x00, 0x0e, 0x60, 0x28, 0x08, 0xf2, 0x2f, 0x81, 0xf3, 0xf8, 0x91, - 0x00, 0x8a, 0xe1, 0x00, 0x4e, 0x80, 0xc4, 0x46, 0x80, 0x10, 0xf1, 0x1a, 0x30, 0x02, 0xff, 0x08, - 0x90, 0xfc, 0xb8, 0x17, 0x10, 0x70, 0x2f, 0x2e, 0xea, 0xd1, 0xaf, 0x99, 0x9e, 0x07, 0x57, 0xad, - 0x7b, 0x05, 0xf8, 0xcf, 0x60, 0xfa, 0x54, 0x9d, 0xd3, 0x55, 0xa7, 0xaa, 0x4e, 0x9d, 0xaf, 0xea, - 0x54, 0x35, 0x4c, 0x59, 0xa6, 0xa2, 0xee, 0xb7, 0x76, 0x97, 0x94, 0x96, 0xbe, 0xd8, 0xb2, 0x4c, - 0xc7, 0x44, 0x53, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x22, 0x4f, 0x9c, 0xbd, 0x79, 0x70, 0xb8, 0x74, - 0x70, 0x68, 0x63, 0xeb, 0x10, 0x5b, 0x4b, 0xaa, 0x69, 0xa8, 0x6d, 0xcb, 0xc2, 0x86, 0x7a, 0xb2, - 0xd4, 0x30, 0xd5, 0x03, 0xfa, 0xa3, 0x1b, 0x75, 0xc6, 0x3e, 0x8b, 0x5c, 0x89, 0x9a, 0xe2, 0x28, - 0x9c, 0x36, 0xe3, 0xd2, 0xb0, 0x65, 0x99, 0x96, 0xcd, 0xa9, 0xe7, 0x5d, 0x6a, 0x13, 0x3b, 0x4a, - 0x20, 0xf7, 0x25, 0xdb, 0x31, 0x2d, 0xa5, 0x8e, 0x97, 0xb0, 0x51, 0xd7, 0x0d, 0x4c, 0x32, 0x1c, - 0xaa, 0x2a, 0x4f, 0x7c, 0x21, 0x32, 0xf1, 0x0e, 0x4f, 0x2d, 0xb4, 0x1d, 0xbd, 0xb1, 0xb4, 0xdf, - 0x50, 0x97, 0x1c, 0xbd, 0x89, 0x6d, 0x47, 0x69, 0xb6, 0x78, 0xca, 0x02, 0x4d, 0x71, 0x2c, 0x45, - 0xd5, 0x8d, 0xfa, 0x92, 0x85, 0x55, 0xd3, 0xd2, 0xb0, 0x26, 0xdb, 0x2d, 0xc5, 0x70, 0x0b, 0x59, - 0x37, 0xeb, 0x26, 0xfd, 0xbb, 0x44, 0xfe, 0x31, 0x6a, 0xf1, 0xe7, 0x02, 0x4c, 0x48, 0xf8, 0x49, - 0x1b, 0xdb, 0x4e, 0x05, 0x2b, 0x1a, 0xb6, 0xd0, 0x45, 0x48, 0x1e, 0xe0, 0x93, 0x42, 0x72, 0x41, - 0xb8, 0x91, 0x5b, 0x19, 0xff, 0xec, 0x74, 0x3e, 0xb9, 0x81, 0x4f, 0x24, 0x42, 0x43, 0x0b, 0x30, - 0x8e, 0x0d, 0x4d, 0x26, 0xc9, 0xa9, 0x70, 0xf2, 0x18, 0x36, 0xb4, 0x0d, 0x7c, 0x82, 0xbe, 0x0d, - 0x69, 0x9b, 0x48, 0x33, 0x54, 0x5c, 0x18, 0x5d, 0x10, 0x6e, 0x8c, 0xae, 0x7c, 0xe3, 0xb3, 0xd3, - 0xf9, 0x77, 0xea, 0xba, 0xb3, 0xdf, 0xde, 0x5d, 0x54, 0xcd, 0xe6, 0x92, 0xa7, 0x7d, 0x6d, 0xd7, - 0xff, 0xbf, 0xd4, 0x3a, 0xa8, 0x2f, 0x75, 0xd6, 0x7c, 0xb1, 0x76, 0x6c, 0x54, 0xf1, 0x13, 0xc9, - 0x93, 0xf8, 0x76, 0xea, 0xff, 0x7c, 0x3c, 0x2f, 0xac, 0xa7, 0xd2, 0x82, 0x98, 0x58, 0x4f, 0xa5, - 0x13, 0x62, 0xb2, 0xf8, 0x93, 0x24, 0xe4, 0x25, 0x6c, 0xb7, 0x4c, 0xc3, 0xc6, 0xbc, 0xfc, 0xaf, - 0x41, 0xd2, 0x39, 0x36, 0x68, 0xf9, 0xb3, 0xb7, 0xe7, 0x16, 0xbb, 0x5a, 0x7b, 0xb1, 0x66, 0x29, - 0x86, 0xad, 0xa8, 0x8e, 0x6e, 0x1a, 0x12, 0xc9, 0x8a, 0xde, 0x82, 0xac, 0x85, 0xed, 0x76, 0x13, - 0x53, 0x75, 0xd1, 0xaa, 0x65, 0x6f, 0x5f, 0x88, 0xe0, 0xac, 0xb6, 0x14, 0x43, 0x02, 0x96, 0x97, - 0xfc, 0x47, 0x17, 0x21, 0x6d, 0xb4, 0x9b, 0x44, 0x21, 0x36, 0xad, 0x6e, 0x52, 0x1a, 0x37, 0xda, - 0xcd, 0x0d, 0x7c, 0x62, 0xa3, 0x6f, 0xc2, 0x79, 0x0d, 0xb7, 0x2c, 0xac, 0x2a, 0x0e, 0xd6, 0x64, - 0x4b, 0x31, 0xea, 0x58, 0xd6, 0x8d, 0x3d, 0xd3, 0x2e, 0x8c, 0x2d, 0x24, 0x6f, 0x64, 0x6f, 0xbf, - 0x10, 0x21, 0x5f, 0x22, 0xb9, 0xd6, 0x8c, 0x3d, 0x73, 0x25, 0xf5, 0xe9, 0xe9, 0xfc, 0x88, 0x34, - 0xe3, 0x4b, 0xf0, 0x92, 0x6c, 0x54, 0x85, 0x09, 0x5e, 0x5c, 0x0b, 0x2b, 0xb6, 0x69, 0x14, 0xc6, - 0x17, 0x84, 0x1b, 0xf9, 0xdb, 0x8b, 0x51, 0x02, 0x43, 0xaa, 0x21, 0x8f, 0xed, 0x26, 0x96, 0x28, - 0x97, 0x94, 0xb3, 0x02, 0x4f, 0xe8, 0x12, 0x64, 0x48, 0x4d, 0x76, 0x4f, 0x1c, 0x6c, 0x17, 0xd2, - 0xb4, 0x2a, 0xa4, 0x6a, 0x2b, 0xe4, 0xb9, 0xf8, 0x2e, 0xe4, 0x82, 0xac, 0x08, 0x41, 0x5e, 0x2a, - 0x57, 0x77, 0x36, 0xcb, 0xf2, 0xce, 0xd6, 0xc6, 0xd6, 0xc3, 0xf7, 0xb6, 0xc4, 0x11, 0x34, 0x03, - 0x22, 0xa7, 0x6d, 0x94, 0xdf, 0x97, 0x1f, 0xac, 0x6d, 0xae, 0xd5, 0x44, 0x61, 0x36, 0xf5, 0xf7, - 0x7e, 0x32, 0x37, 0x52, 0x7c, 0x0c, 0x70, 0x1f, 0x3b, 0xbc, 0x9b, 0xa1, 0x15, 0x18, 0xdb, 0xa7, - 0xe5, 0x29, 0x08, 0x54, 0xd3, 0x0b, 0x91, 0x05, 0x0f, 0x74, 0xc9, 0x95, 0x34, 0xd1, 0xc6, 0xaf, - 0x4e, 0xe7, 0x05, 0x89, 0x73, 0xb2, 0x9e, 0x50, 0xfc, 0xf7, 0x02, 0x64, 0xa9, 0x60, 0x56, 0x4b, - 0x54, 0xea, 0x90, 0x7c, 0x79, 0xa0, 0x4a, 0xba, 0x45, 0xa3, 0x45, 0x18, 0x3d, 0x54, 0x1a, 0x6d, - 0x5c, 0x48, 0x50, 0x19, 0x85, 0x08, 0x19, 0x8f, 0x49, 0xba, 0xc4, 0xb2, 0xa1, 0xbb, 0x90, 0xd3, - 0x0d, 0x07, 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0x03, 0xd8, 0xb2, 0x2c, 0x37, 0x7d, 0x28, 0xfe, 0x1b, - 0x01, 0x60, 0xbb, 0x1d, 0xa7, 0x6a, 0xd0, 0xeb, 0x43, 0x96, 0x9f, 0xf7, 0x31, 0x5e, 0x8b, 0xf3, - 0x30, 0xa6, 0x1b, 0x0d, 0xdd, 0x60, 0xe5, 0x4f, 0x4b, 0xfc, 0x09, 0xcd, 0xc0, 0xe8, 0x6e, 0x43, - 0x37, 0x34, 0x3a, 0x2a, 0xd2, 0x12, 0x7b, 0xe0, 0xea, 0x97, 0x20, 0x4b, 0xcb, 0x1e, 0xa3, 0xf6, - 0x8b, 0xbf, 0x4c, 0xc0, 0xb9, 0x92, 0x69, 0x68, 0x3a, 0x19, 0x9e, 0x4a, 0xe3, 0x4b, 0xa1, 0x9b, - 0x75, 0x08, 0x0c, 0x44, 0x19, 0x1f, 0xb7, 0x86, 0x6c, 0x69, 0xe4, 0x73, 0x95, 0x8f, 0x5b, 0x94, - 0x16, 0xad, 0x4f, 0xf4, 0x3a, 0x5c, 0x50, 0x1a, 0x0d, 0xf3, 0x48, 0xd6, 0xf7, 0x64, 0xcd, 0xc4, - 0xb6, 0x6c, 0x98, 0x8e, 0x8c, 0x8f, 0x75, 0xdb, 0xa1, 0x66, 0x25, 0x2d, 0x4d, 0xd3, 0xe4, 0xb5, - 0xbd, 0x55, 0x13, 0xdb, 0x5b, 0xa6, 0x53, 0x26, 0x49, 0x64, 0xcc, 0x92, 0xc2, 0xb0, 0x31, 0x3b, - 0x46, 0x0c, 0xb2, 0x94, 0xc6, 0xc7, 0x2d, 0x3a, 0x66, 0x79, 0x13, 0x7d, 0x08, 0xe7, 0x3b, 0xb5, - 0x19, 0x67, 0x6b, 0xfd, 0x37, 0x01, 0xf2, 0x6b, 0x86, 0xee, 0x7c, 0x29, 0x9a, 0xc9, 0x53, 0x6d, - 0x32, 0xa8, 0xda, 0x9b, 0x20, 0xee, 0x29, 0x7a, 0xe3, 0xa1, 0x51, 0x33, 0x9b, 0xbb, 0xb6, 0x63, - 0x1a, 0xd8, 0xe6, 0xba, 0xef, 0xa2, 0x73, 0x9d, 0x3d, 0x86, 0x49, 0xaf, 0x4e, 0x71, 0x2a, 0xeb, - 0x29, 0x88, 0x6b, 0x86, 0x6a, 0xe1, 0x26, 0x36, 0x62, 0xd5, 0xd6, 0x0b, 0x90, 0xd1, 0x5d, 0xb9, - 0x54, 0x63, 0x49, 0xc9, 0x27, 0xf0, 0x3a, 0xb5, 0x61, 0x2a, 0xf0, 0xee, 0x38, 0xcd, 0x25, 0x99, - 0x38, 0xf0, 0x91, 0xec, 0xb7, 0x17, 0x99, 0x38, 0xf0, 0x11, 0x33, 0x6f, 0xef, 0xc3, 0xc4, 0x2a, - 0x6e, 0x60, 0x07, 0xc7, 0x6f, 0xfb, 0x77, 0x20, 0xef, 0x8a, 0x8e, 0xb3, 0x91, 0x7e, 0x24, 0x00, - 0xe2, 0x72, 0xc9, 0x8c, 0x1b, 0x67, 0x3b, 0xcd, 0x13, 0x37, 0xc3, 0x69, 0x5b, 0x06, 0xf3, 0x17, - 0x58, 0x2f, 0x05, 0x46, 0xa2, 0x2e, 0x83, 0x6f, 0x83, 0x53, 0x41, 0x1b, 0xec, 0xb9, 0x3d, 0xc4, - 0xe1, 0x39, 0x82, 0xe9, 0x50, 0xf1, 0xe2, 0x6d, 0xca, 0x14, 0x2d, 0x59, 0x62, 0x21, 0x19, 0xf4, - 0xed, 0x28, 0xb1, 0xf8, 0x21, 0x4c, 0x95, 0x1a, 0x58, 0xb1, 0xe2, 0x56, 0x0b, 0x6f, 0xce, 0xf7, - 0x01, 0x05, 0xc5, 0xc7, 0xd9, 0xa4, 0xff, 0x4c, 0x00, 0x24, 0xe1, 0x43, 0x6c, 0x39, 0xb1, 0x37, - 0xe9, 0x2a, 0x64, 0x1d, 0xc5, 0xaa, 0x63, 0x47, 0x26, 0xfe, 0x38, 0x37, 0x57, 0x2f, 0x06, 0x04, - 0x11, 0xaf, 0x7c, 0x71, 0xbf, 0xa1, 0x2e, 0xd6, 0x5c, 0x7f, 0x9d, 0xdb, 0x2c, 0x60, 0x7c, 0x84, - 0xcc, 0x35, 0xf0, 0x01, 0x4c, 0x87, 0x4a, 0x19, 0xa7, 0x0a, 0xfe, 0x44, 0x80, 0x6c, 0x55, 0x55, - 0x8c, 0x38, 0xeb, 0xfe, 0x2e, 0x64, 0x6d, 0x55, 0x31, 0xe4, 0x3d, 0xd3, 0x6a, 0x2a, 0x0e, 0xed, - 0xb2, 0xf9, 0x50, 0xdd, 0x3d, 0xaf, 0x59, 0x55, 0x8c, 0x7b, 0x34, 0x93, 0x04, 0xb6, 0xf7, 0x1f, - 0x3d, 0x82, 0xec, 0x01, 0x3e, 0x91, 0x39, 0xba, 0xa2, 0xf3, 0x5c, 0xfe, 0xf6, 0x6b, 0x01, 0xfe, - 0x83, 0xc3, 0x45, 0x17, 0x94, 0x2d, 0x06, 0x40, 0xd9, 0x22, 0xe1, 0x58, 0xac, 0x3a, 0x16, 0x36, - 0xea, 0xce, 0xbe, 0x04, 0x07, 0xf8, 0xe4, 0x01, 0x93, 0x11, 0x1c, 0x28, 0xeb, 0xa9, 0x74, 0x52, - 0x4c, 0x15, 0xff, 0x54, 0x80, 0x1c, 0xab, 0x78, 0x9c, 0x03, 0xe5, 0x0d, 0x48, 0x59, 0xe6, 0x11, - 0x1b, 0x28, 0xd9, 0xdb, 0x97, 0x22, 0x44, 0x6c, 0xe0, 0x93, 0xe0, 0x0c, 0x45, 0xb3, 0xa3, 0x15, - 0xe0, 0xbe, 0x9f, 0x4c, 0xb9, 0x93, 0xc3, 0x72, 0x03, 0xe3, 0x92, 0x88, 0x8c, 0xeb, 0x30, 0xb9, - 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x42, 0x92, 0xd9, 0x2c, 0x79, 0x23, 0x27, 0xe5, 0x29, 0xd9, - 0x2d, 0xba, 0x5d, 0xfc, 0x33, 0xb7, 0xd7, 0xdb, 0xf8, 0x8f, 0xb2, 0xe5, 0xff, 0x5c, 0xe0, 0xe3, - 0xc9, 0xad, 0xff, 0x1f, 0x5b, 0x07, 0xf8, 0x71, 0x02, 0x2e, 0x94, 0xf6, 0xb1, 0x7a, 0x50, 0x32, - 0x0d, 0x5b, 0xb7, 0x1d, 0xa2, 0xc1, 0x38, 0x7b, 0xc1, 0x25, 0xc8, 0x1c, 0xe9, 0xce, 0xbe, 0xac, - 0xe9, 0x7b, 0x7b, 0xd4, 0xf2, 0xa5, 0xa5, 0x34, 0x21, 0xac, 0xea, 0x7b, 0x7b, 0xe8, 0x0e, 0xa4, - 0x9a, 0xa6, 0xc6, 0x5c, 0xe4, 0xfc, 0xed, 0xf9, 0x08, 0xf1, 0xb4, 0x68, 0x76, 0xbb, 0xb9, 0x69, - 0x6a, 0x58, 0xa2, 0x99, 0xd1, 0x1c, 0x80, 0x4a, 0xa8, 0x2d, 0x53, 0x37, 0x1c, 0x3e, 0x07, 0x06, - 0x28, 0xa8, 0x02, 0x19, 0x07, 0x5b, 0x4d, 0xdd, 0x50, 0x1c, 0x5c, 0x18, 0xa5, 0xca, 0xbb, 0x12, - 0x59, 0xf0, 0x56, 0x43, 0x57, 0x95, 0x55, 0x6c, 0xab, 0x96, 0xde, 0x72, 0x4c, 0x8b, 0x6b, 0xd1, - 0x67, 0xe6, 0x16, 0xf7, 0xa3, 0x14, 0x14, 0xba, 0x35, 0x14, 0x67, 0x3f, 0xd9, 0x86, 0x31, 0x82, - 0xb2, 0x1b, 0x0e, 0xef, 0x29, 0xb7, 0x7b, 0x29, 0x22, 0xa2, 0x04, 0x14, 0xad, 0x37, 0x1c, 0x5e, - 0x78, 0x2e, 0x67, 0xf6, 0xe7, 0x02, 0x8c, 0xb1, 0x04, 0x74, 0x0b, 0xd2, 0x7c, 0x59, 0x41, 0xa3, - 0x65, 0x4c, 0xae, 0x9c, 0x7f, 0x76, 0x3a, 0x3f, 0xce, 0x56, 0x0a, 0x56, 0x3f, 0xf3, 0xff, 0x4a, - 0xe3, 0x34, 0xdf, 0x9a, 0x46, 0xda, 0xcc, 0x76, 0x14, 0xcb, 0xa1, 0x4b, 0x38, 0x09, 0x86, 0x18, - 0x28, 0x61, 0x03, 0x9f, 0xa0, 0x75, 0x18, 0xb3, 0x1d, 0xc5, 0x69, 0xdb, 0xbc, 0xd5, 0xce, 0x54, - 0xd8, 0x2a, 0xe5, 0x94, 0xb8, 0x04, 0xe2, 0xca, 0x68, 0xd8, 0x51, 0xf4, 0x06, 0x6d, 0xc6, 0x8c, - 0xc4, 0x9f, 0x8a, 0x3f, 0x14, 0x60, 0x8c, 0x65, 0x45, 0x17, 0x60, 0x5a, 0x5a, 0xde, 0xba, 0x5f, - 0x96, 0xd7, 0xb6, 0x56, 0xcb, 0xb5, 0xb2, 0xb4, 0xb9, 0xb6, 0xb5, 0x5c, 0x2b, 0x8b, 0x23, 0xe8, - 0x3c, 0x20, 0x37, 0xa1, 0xf4, 0x70, 0xab, 0xba, 0x56, 0xad, 0x95, 0xb7, 0x6a, 0xa2, 0x40, 0x57, - 0x18, 0x28, 0x3d, 0x40, 0x4d, 0xa0, 0x2b, 0xb0, 0xd0, 0x49, 0x95, 0xab, 0xb5, 0xe5, 0x5a, 0x55, - 0x2e, 0x57, 0x6b, 0x6b, 0x9b, 0xcb, 0xb5, 0xf2, 0xaa, 0x98, 0xec, 0x93, 0x8b, 0xbc, 0x44, 0x92, - 0xca, 0xa5, 0x9a, 0x98, 0x2a, 0x3e, 0x85, 0x73, 0x12, 0x56, 0xcd, 0x66, 0xab, 0xed, 0x60, 0x52, - 0x4a, 0x3b, 0xce, 0xf1, 0x72, 0x01, 0xc6, 0x35, 0xeb, 0x44, 0xb6, 0xda, 0x06, 0x1f, 0x2d, 0x63, - 0x9a, 0x75, 0x22, 0xb5, 0x0d, 0xde, 0x19, 0xff, 0xa5, 0x00, 0xe7, 0x3b, 0x5f, 0x1e, 0x67, 0x57, - 0x7c, 0x04, 0x59, 0x45, 0xd3, 0xb0, 0x26, 0x6b, 0xb8, 0xe1, 0x28, 0xdc, 0x55, 0xb9, 0x19, 0x90, - 0xc4, 0x97, 0xdf, 0x16, 0xbd, 0xe5, 0xb7, 0xcd, 0xc7, 0xa5, 0x12, 0x2d, 0xc8, 0x2a, 0xe1, 0x70, - 0x4d, 0x11, 0x15, 0x42, 0x29, 0xc5, 0x7f, 0x95, 0x82, 0x89, 0xb2, 0xa1, 0xd5, 0x8e, 0x63, 0x9d, - 0x5d, 0xce, 0xc3, 0x98, 0x6a, 0x36, 0x9b, 0xba, 0xe3, 0xaa, 0x89, 0x3d, 0xa1, 0xaf, 0x42, 0x5a, - 0xc3, 0x8a, 0xe6, 0xad, 0x51, 0x0c, 0x72, 0xb4, 0x24, 0x2f, 0x3b, 0xfa, 0x0e, 0x5c, 0x20, 0x16, - 0xd4, 0x32, 0x94, 0x86, 0xcc, 0xa4, 0xc9, 0x8e, 0xa5, 0xd7, 0xeb, 0xd8, 0xe2, 0x8b, 0x7d, 0x37, - 0x22, 0xca, 0xb9, 0xc6, 0x39, 0x4a, 0x94, 0xa1, 0xc6, 0xf2, 0x4b, 0xe7, 0xf4, 0x28, 0x32, 0x7a, - 0x07, 0x80, 0x4c, 0x4e, 0x74, 0x01, 0xd1, 0xe6, 0xb6, 0xa9, 0xd7, 0x0a, 0xa2, 0x6b, 0x8e, 0x08, - 0x03, 0x79, 0xb6, 0xd1, 0x12, 0x41, 0x06, 0x4f, 0xda, 0xba, 0x85, 0xe5, 0x5b, 0x2d, 0x95, 0x42, - 0xf9, 0xf4, 0x4a, 0xfe, 0xd9, 0xe9, 0x3c, 0x48, 0x8c, 0x7c, 0x6b, 0xbb, 0x44, 0x90, 0x02, 0xfb, - 0xdf, 0x52, 0xd1, 0x0a, 0xcc, 0x91, 0x09, 0x98, 0xd7, 0x45, 0x71, 0xe4, 0x7d, 0xbd, 0xbe, 0x8f, - 0x2d, 0xd9, 0x5b, 0x15, 0xa6, 0x4b, 0x78, 0x69, 0x69, 0x56, 0x55, 0x0c, 0x56, 0xd0, 0x65, 0xa7, - 0x42, 0xb3, 0x78, 0xea, 0x21, 0x7a, 0x6e, 0x99, 0xba, 0x6d, 0x1a, 0x85, 0x0c, 0xd3, 0x33, 0x7b, - 0x42, 0x8f, 0x40, 0xd4, 0x0d, 0x79, 0xaf, 0xa1, 0xd7, 0xf7, 0x1d, 0xf9, 0xc8, 0xd2, 0x1d, 0x6c, - 0x17, 0xa6, 0x68, 0x85, 0xa2, 0xfa, 0x5d, 0x95, 0xaf, 0xcd, 0x6a, 0xef, 0x91, 0x9c, 0xbc, 0x6a, - 0x79, 0xdd, 0xb8, 0x47, 0xf9, 0x29, 0xd1, 0xf6, 0x66, 0xe7, 0x71, 0x31, 0x5d, 0xfc, 0x5f, 0x02, - 0xe4, 0xdd, 0x4e, 0x13, 0x67, 0xff, 0xbe, 0x01, 0xa2, 0x69, 0x60, 0xb9, 0xb5, 0xaf, 0xd8, 0x98, - 0x2b, 0x86, 0x4f, 0x21, 0x79, 0xd3, 0xc0, 0xdb, 0x84, 0xcc, 0x34, 0x81, 0xb6, 0x61, 0xca, 0x76, - 0x94, 0xba, 0x6e, 0xd4, 0x03, 0xfa, 0x1a, 0x1d, 0xde, 0x75, 0x17, 0x39, 0xb7, 0x47, 0x0f, 0xf9, - 0x1d, 0xbf, 0x16, 0x60, 0x6a, 0x59, 0x6b, 0xea, 0x46, 0xb5, 0xd5, 0xd0, 0x63, 0xc5, 0xf9, 0x57, - 0x20, 0x63, 0x13, 0x99, 0xbe, 0xf1, 0xf6, 0x31, 0x5a, 0x9a, 0xa6, 0x10, 0x2b, 0xfe, 0x00, 0x26, - 0xf1, 0x71, 0x4b, 0xb7, 0x14, 0x47, 0x37, 0x0d, 0x06, 0x4b, 0x52, 0xc3, 0xd7, 0x2d, 0xef, 0xf3, - 0xfa, 0xd0, 0x84, 0xd7, 0xec, 0x7d, 0x40, 0xc1, 0x8a, 0xc5, 0x89, 0x4f, 0x64, 0x98, 0xa6, 0xa2, - 0x77, 0x0c, 0x3b, 0x66, 0xad, 0x71, 0xeb, 0xfa, 0x2d, 0x98, 0x09, 0xbf, 0x20, 0xce, 0xd2, 0x7f, - 0xc8, 0x5b, 0x7c, 0x13, 0x5b, 0x5f, 0x10, 0x34, 0x0e, 0x8a, 0x8f, 0xb3, 0xe4, 0x3f, 0x10, 0xe0, - 0x22, 0x95, 0x4d, 0xf7, 0x44, 0xf6, 0xb0, 0xf5, 0x00, 0x2b, 0x76, 0xac, 0x08, 0xf9, 0x25, 0x18, - 0x63, 0x48, 0x97, 0xf6, 0xd8, 0xd1, 0x95, 0x2c, 0xf1, 0x4b, 0xaa, 0x8e, 0x69, 0x11, 0xbf, 0x84, - 0x27, 0xf1, 0x7a, 0x2a, 0x30, 0x1b, 0x55, 0x96, 0x98, 0x97, 0x02, 0xa6, 0xb8, 0x7b, 0x48, 0xba, - 0x78, 0x69, 0x9f, 0xf8, 0x45, 0xa8, 0x0c, 0x59, 0x95, 0xfe, 0x93, 0x9d, 0x93, 0x16, 0xa6, 0xf2, - 0xf3, 0xfd, 0x3c, 0x4b, 0xc6, 0x56, 0x3b, 0x69, 0x61, 0xe2, 0x9e, 0xba, 0xff, 0x89, 0xba, 0x02, - 0x55, 0xed, 0xeb, 0x9b, 0xd2, 0xf1, 0x45, 0xf3, 0xba, 0xee, 0x5d, 0x48, 0x13, 0xff, 0x3a, 0xc9, - 0x55, 0xc1, 0xde, 0xc4, 0x99, 0x62, 0xf5, 0x46, 0x3e, 0x08, 0x6d, 0x4f, 0x05, 0xab, 0x9f, 0x38, - 0x43, 0xf5, 0x03, 0xeb, 0xe2, 0x3e, 0x15, 0xbd, 0x0f, 0x81, 0x95, 0x6f, 0x99, 0xd5, 0xcc, 0x45, - 0x3b, 0x67, 0x51, 0xca, 0x94, 0x2f, 0x85, 0xd1, 0x6d, 0x54, 0x82, 0x34, 0x3e, 0x6e, 0xc9, 0x1a, - 0xb6, 0x55, 0x6e, 0xd6, 0x8a, 0xbd, 0xf6, 0xd1, 0xba, 0xfc, 0xff, 0x71, 0x7c, 0xdc, 0x22, 0x44, - 0xb4, 0x43, 0x66, 0x38, 0xd7, 0x1d, 0xa0, 0xc5, 0xb6, 0x07, 0xc3, 0x09, 0xbf, 0xbf, 0x70, 0x71, - 0x93, 0x9e, 0x27, 0xc0, 0x44, 0xf0, 0xb6, 0xfb, 0x58, 0x80, 0x4b, 0x91, 0x6d, 0x17, 0xe7, 0x64, - 0xf7, 0x0e, 0xa4, 0xa8, 0x0a, 0x12, 0x67, 0x54, 0x01, 0xe5, 0x2a, 0xfe, 0xcc, 0x1d, 0xf5, 0x12, - 0x6e, 0x98, 0x44, 0xbd, 0x5f, 0xc0, 0xba, 0xd8, 0xb8, 0xdb, 0xec, 0x89, 0x33, 0x37, 0xbb, 0xcb, - 0xda, 0x61, 0x16, 0x3a, 0x0a, 0x1b, 0xa7, 0x59, 0xf8, 0x47, 0x02, 0x4c, 0x57, 0xb0, 0x62, 0x39, - 0xbb, 0x58, 0x71, 0x62, 0x76, 0x67, 0xdf, 0x80, 0xa4, 0x61, 0x1e, 0x9d, 0x65, 0x69, 0x90, 0xe4, - 0xf7, 0xa7, 0xad, 0x70, 0xb9, 0xe2, 0xac, 0xf5, 0x7f, 0x4e, 0x40, 0xe6, 0x7e, 0x29, 0xce, 0xba, - 0xbe, 0xc3, 0x17, 0x90, 0xd9, 0x50, 0x8f, 0xea, 0x96, 0xde, 0xfb, 0x16, 0xef, 0x97, 0x36, 0xf0, - 0x89, 0xdb, 0x2d, 0x09, 0x17, 0x5a, 0x86, 0x8c, 0xb3, 0x6f, 0x61, 0x7b, 0xdf, 0x6c, 0x68, 0x67, - 0xf1, 0x59, 0x7c, 0xae, 0xd9, 0x03, 0x18, 0xa5, 0x72, 0xdd, 0x20, 0x06, 0x21, 0x22, 0x88, 0x81, - 0xbc, 0xc6, 0x73, 0xfb, 0x12, 0x67, 0x79, 0x8d, 0x4b, 0x60, 0x8d, 0xe3, 0xf9, 0x46, 0xa3, 0xe2, - 0x58, 0xf1, 0x11, 0x00, 0xa9, 0x5a, 0x9c, 0xcd, 0xf3, 0xf7, 0x93, 0x90, 0xdf, 0x6e, 0xdb, 0xfb, - 0x31, 0xf7, 0xc7, 0x12, 0x40, 0xab, 0x6d, 0x53, 0xb0, 0x70, 0x6c, 0xf0, 0xfa, 0x0f, 0x88, 0x92, - 0x70, 0x15, 0xc0, 0xf8, 0x6a, 0xc7, 0x06, 0xaa, 0x70, 0x21, 0x58, 0xf6, 0x43, 0x2d, 0x5e, 0xea, - 0x87, 0x25, 0x6b, 0xc7, 0xc6, 0x26, 0xf6, 0x40, 0x24, 0x93, 0x84, 0x89, 0xa4, 0x77, 0x60, 0x9c, - 0x3c, 0xc8, 0x8e, 0x79, 0x96, 0x26, 0x1f, 0x23, 0x3c, 0x35, 0x13, 0xdd, 0x85, 0x0c, 0xe3, 0x26, - 0x13, 0xd7, 0x18, 0x9d, 0xb8, 0xa2, 0xea, 0xc2, 0xd5, 0x48, 0xa7, 0xac, 0x34, 0x65, 0x25, 0xd3, - 0xd4, 0x0c, 0x8c, 0xee, 0x99, 0x96, 0x8a, 0x69, 0xfc, 0x44, 0x5a, 0x62, 0x0f, 0xc1, 0x56, 0x5d, - 0x4f, 0xa5, 0xd3, 0x62, 0x66, 0x3d, 0x95, 0xce, 0x88, 0x50, 0xfc, 0xa1, 0x00, 0x93, 0x5e, 0x73, - 0xc4, 0x69, 0xcb, 0x4b, 0x21, 0x5d, 0x9e, 0xbd, 0x41, 0x88, 0x1a, 0x8b, 0xff, 0x85, 0x3a, 0x36, - 0xaa, 0x79, 0x48, 0xdb, 0x27, 0xce, 0xfe, 0x72, 0x97, 0x85, 0xd3, 0x24, 0xce, 0xda, 0xc6, 0x34, - 0xb2, 0xe6, 0x16, 0xcc, 0xe8, 0x4d, 0x62, 0xe5, 0x75, 0xa7, 0x71, 0xc2, 0x51, 0x99, 0x83, 0xdd, - 0x1d, 0xda, 0x69, 0x3f, 0xad, 0xe4, 0x26, 0x71, 0xc3, 0xc7, 0xf6, 0x6c, 0xfc, 0xfa, 0xc4, 0xa9, - 0xf0, 0x35, 0x98, 0xb0, 0x98, 0x68, 0xe2, 0x9d, 0x9c, 0x51, 0xe7, 0x39, 0x8f, 0x95, 0xa8, 0xfd, - 0xa7, 0x09, 0x98, 0x7c, 0xd4, 0xc6, 0xd6, 0xc9, 0x97, 0x49, 0xe9, 0xd7, 0x60, 0xf2, 0x48, 0xd1, - 0x1d, 0x79, 0xcf, 0xb4, 0xe4, 0x76, 0x4b, 0x53, 0x1c, 0x37, 0xa6, 0x63, 0x82, 0x90, 0xef, 0x99, - 0xd6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, 0xc0, 0x30, 0x8f, 0x0c, 0x99, 0x90, 0x29, 0x1a, 0x3e, 0x36, - 0xf8, 0x62, 0xf2, 0xca, 0x9b, 0xff, 0xf3, 0x74, 0xfe, 0xce, 0x50, 0x51, 0x5b, 0x34, 0xee, 0xac, - 0xdd, 0xd6, 0xb5, 0xc5, 0x9d, 0x9d, 0xb5, 0x55, 0x49, 0xa4, 0x22, 0xdf, 0x63, 0x12, 0x6b, 0xc7, - 0x86, 0x3b, 0x8b, 0x7f, 0x26, 0x80, 0xe8, 0x6b, 0x2a, 0xce, 0xe6, 0x2c, 0x43, 0xf6, 0x49, 0x1b, - 0x5b, 0xfa, 0x73, 0x34, 0x26, 0x70, 0x46, 0x62, 0x88, 0x3e, 0x80, 0x5c, 0x48, 0x0f, 0xc9, 0xcf, - 0xa7, 0x87, 0xec, 0x91, 0xaf, 0x82, 0xe2, 0x7f, 0x12, 0x00, 0xd1, 0xca, 0xaf, 0xb1, 0x75, 0xfc, - 0x2f, 0x4b, 0x4f, 0xb9, 0x01, 0x22, 0x8d, 0x58, 0x94, 0xf5, 0x3d, 0xb9, 0xa9, 0xdb, 0xb6, 0x6e, - 0xd4, 0x79, 0x57, 0xc9, 0x53, 0xfa, 0xda, 0xde, 0x26, 0xa3, 0xf2, 0x46, 0xfc, 0x1b, 0x30, 0x1d, - 0xaa, 0x46, 0x9c, 0xcd, 0x78, 0x19, 0x72, 0x7b, 0x66, 0xdb, 0xd0, 0x64, 0xb6, 0xd7, 0xc1, 0x17, - 0xff, 0xb2, 0x94, 0xc6, 0xde, 0x57, 0xfc, 0x7f, 0x09, 0x98, 0x91, 0xb0, 0x6d, 0x36, 0x0e, 0x71, - 0xfc, 0x8a, 0xac, 0x00, 0xdf, 0x65, 0x91, 0x9f, 0x4b, 0x9f, 0x19, 0xc6, 0xcc, 0xa6, 0xb4, 0xf0, - 0x3a, 0xfa, 0x95, 0xfe, 0x7d, 0xb1, 0x7b, 0xe5, 0x9c, 0x2f, 0xcb, 0xa5, 0x42, 0xcb, 0x72, 0x26, - 0x4c, 0xea, 0x75, 0xc3, 0x24, 0x36, 0xcb, 0xc6, 0x4f, 0x8c, 0x76, 0xd3, 0xc5, 0x2c, 0x8b, 0xfd, - 0x0a, 0xb9, 0xc6, 0x58, 0xaa, 0xf8, 0xc9, 0x56, 0xbb, 0x49, 0x3d, 0xe7, 0x95, 0xf3, 0xa4, 0xbc, - 0xcf, 0x4e, 0xe7, 0xf3, 0xa1, 0x34, 0x5b, 0xca, 0xeb, 0xde, 0x33, 0x91, 0xce, 0x9b, 0xfc, 0xdb, - 0x70, 0xae, 0x43, 0xe5, 0x71, 0xfa, 0x38, 0xff, 0x21, 0x09, 0x17, 0xc3, 0xe2, 0xe3, 0x46, 0x22, - 0x5f, 0xf6, 0x66, 0xad, 0xc0, 0x44, 0x53, 0x37, 0x9e, 0x6f, 0x21, 0x32, 0xd7, 0xd4, 0x0d, 0x7f, - 0x3d, 0x37, 0xa2, 0x83, 0x8c, 0xfd, 0x25, 0x74, 0x10, 0x05, 0x66, 0xa3, 0x5a, 0x30, 0xce, 0x5e, - 0xf2, 0x91, 0x00, 0xb9, 0xb8, 0xd7, 0xd6, 0x9e, 0x2f, 0xc6, 0x8c, 0xd7, 0xb9, 0x06, 0x13, 0x5f, - 0xc0, 0x62, 0xdc, 0x4f, 0x05, 0x40, 0x35, 0xab, 0x6d, 0x10, 0x90, 0xfb, 0xc0, 0xac, 0xc7, 0x59, - 0xd9, 0x19, 0x18, 0xd5, 0x0d, 0x0d, 0x1f, 0xd3, 0xca, 0xa6, 0x24, 0xf6, 0x10, 0xda, 0x40, 0x4c, - 0x0e, 0xb5, 0x81, 0xe8, 0x87, 0xaa, 0x84, 0x0a, 0x1a, 0xa7, 0x16, 0xfe, 0x45, 0x02, 0xa6, 0x79, - 0x75, 0x62, 0x5f, 0x8c, 0x7c, 0x1d, 0x46, 0x1b, 0x44, 0x66, 0x9f, 0x36, 0xa7, 0xef, 0x74, 0xdb, - 0x9c, 0x66, 0x46, 0x5f, 0x03, 0x68, 0x59, 0xf8, 0x50, 0x66, 0xac, 0xc9, 0xa1, 0x58, 0x33, 0x84, - 0x83, 0x12, 0xd0, 0x37, 0x61, 0x92, 0x8c, 0xf0, 0x96, 0x65, 0xb6, 0x4c, 0x9b, 0x38, 0x29, 0xf6, - 0x70, 0x48, 0x67, 0xea, 0xd9, 0xe9, 0xfc, 0xc4, 0xa6, 0x6e, 0x6c, 0x73, 0xc6, 0x5a, 0x55, 0x22, - 0xa6, 0xc2, 0x7b, 0x74, 0x07, 0xe0, 0x7f, 0x17, 0x60, 0xe6, 0x0b, 0x5b, 0xbe, 0xfd, 0xab, 0xd0, - 0x98, 0x37, 0xf3, 0x88, 0xf4, 0x71, 0xcd, 0xd8, 0x33, 0xe3, 0x5f, 0x54, 0xff, 0x48, 0x80, 0xa9, - 0x80, 0xf8, 0x38, 0x3d, 0x99, 0xe7, 0xd2, 0x59, 0xf1, 0x5b, 0xc4, 0xb7, 0x09, 0x76, 0xfb, 0x38, - 0x07, 0xd5, 0xbf, 0x4d, 0xc0, 0xf9, 0x12, 0xdb, 0x5a, 0x76, 0xe3, 0x2e, 0xe2, 0xec, 0x25, 0x05, - 0x18, 0x3f, 0xc4, 0x96, 0xad, 0x9b, 0x6c, 0x86, 0x9d, 0x90, 0xdc, 0x47, 0x34, 0x0b, 0x69, 0xdb, - 0x50, 0x5a, 0xf6, 0xbe, 0xe9, 0xee, 0xc6, 0x79, 0xcf, 0x5e, 0x8c, 0xc8, 0xe8, 0xf3, 0xc7, 0x88, - 0x8c, 0xf5, 0x8f, 0x11, 0x19, 0xff, 0xdc, 0x31, 0x22, 0x7c, 0xeb, 0xeb, 0x17, 0x02, 0x5c, 0xe8, - 0xd2, 0x5f, 0x9c, 0x7d, 0xe6, 0xbb, 0x90, 0x55, 0xb9, 0x60, 0x62, 0x8d, 0xd9, 0xee, 0xde, 0x1a, - 0xc9, 0xf6, 0x9c, 0x00, 0xe4, 0xd9, 0xe9, 0x3c, 0xb8, 0x45, 0x5d, 0x5b, 0xe5, 0x2a, 0x22, 0xff, - 0xb5, 0xe2, 0xff, 0xc8, 0xc1, 0x64, 0xf9, 0x98, 0xad, 0x5d, 0x57, 0x99, 0x3f, 0x80, 0xee, 0x41, - 0xba, 0x65, 0x99, 0x87, 0xba, 0x5b, 0x8d, 0x7c, 0x28, 0x34, 0xc0, 0xad, 0x46, 0x07, 0xd7, 0x36, - 0xe7, 0x90, 0x3c, 0x5e, 0x54, 0x83, 0xcc, 0x03, 0x53, 0x55, 0x1a, 0xf7, 0xf4, 0x86, 0xdb, 0xff, - 0x5f, 0x1b, 0x2c, 0x68, 0xd1, 0xe3, 0xd9, 0x56, 0x9c, 0x7d, 0xb7, 0x29, 0x3c, 0x22, 0x5a, 0x83, - 0x74, 0xc5, 0x71, 0x5a, 0x24, 0x91, 0x5b, 0x93, 0xeb, 0x43, 0x08, 0x25, 0x2c, 0x5c, 0x96, 0xc7, - 0x8e, 0x6a, 0x30, 0x75, 0xdf, 0x34, 0xeb, 0x0d, 0x5c, 0x6a, 0x98, 0x6d, 0xad, 0x64, 0x1a, 0x7b, - 0x7a, 0x9d, 0xdb, 0xe3, 0x6b, 0x43, 0xc8, 0xbc, 0x5f, 0xaa, 0x4a, 0xdd, 0x02, 0xd0, 0x32, 0xa4, - 0xab, 0x77, 0xb8, 0x30, 0xe6, 0xc0, 0x5d, 0x1d, 0x42, 0x58, 0xf5, 0x8e, 0xe4, 0xb1, 0xa1, 0x75, - 0xc8, 0x2e, 0x3f, 0x6d, 0x5b, 0x98, 0x4b, 0x19, 0xeb, 0x19, 0x97, 0xd0, 0x29, 0x85, 0x72, 0x49, - 0x41, 0x66, 0x54, 0x85, 0xfc, 0x7b, 0xa6, 0x75, 0xd0, 0x30, 0x15, 0xb7, 0x86, 0xe3, 0x54, 0xdc, - 0x57, 0x86, 0x10, 0xe7, 0x32, 0x4a, 0x1d, 0x22, 0xd0, 0xb7, 0x61, 0x92, 0x34, 0x46, 0x4d, 0xd9, - 0x6d, 0xb8, 0x85, 0x4c, 0x53, 0xa9, 0xaf, 0x0c, 0x21, 0xd5, 0xe3, 0x74, 0x37, 0x4f, 0x3a, 0x44, - 0xcd, 0x7e, 0x13, 0x26, 0x42, 0x9d, 0x00, 0x21, 0x48, 0xb5, 0x48, 0x7b, 0x0b, 0x34, 0x7e, 0x88, - 0xfe, 0x47, 0xaf, 0xc2, 0xb8, 0x61, 0x6a, 0xd8, 0x1d, 0x21, 0x13, 0x2b, 0x33, 0xcf, 0x4e, 0xe7, - 0xc7, 0xb6, 0x4c, 0x8d, 0xb9, 0x2b, 0xfc, 0x9f, 0x34, 0x46, 0x32, 0xb9, 0xce, 0xca, 0xec, 0x35, - 0x48, 0x91, 0xd6, 0x27, 0x46, 0x6a, 0x57, 0xb1, 0xf1, 0x8e, 0xa5, 0x73, 0x99, 0xee, 0x23, 0xcf, - 0xf7, 0x1b, 0x01, 0x12, 0xd5, 0x3b, 0xc4, 0x51, 0xdf, 0x6d, 0xab, 0x07, 0xd8, 0xe1, 0xb9, 0xf8, - 0x13, 0x75, 0xe0, 0x2d, 0xbc, 0xa7, 0x33, 0x1f, 0x2a, 0x23, 0xf1, 0x27, 0xf4, 0x22, 0x80, 0xa2, - 0xaa, 0xd8, 0xb6, 0x65, 0xf7, 0xd4, 0x5c, 0x46, 0xca, 0x30, 0xca, 0x06, 0x3e, 0x21, 0x6c, 0x36, - 0x56, 0x2d, 0xec, 0xb8, 0x81, 0x50, 0xec, 0x89, 0xb0, 0x39, 0xb8, 0xd9, 0x92, 0x1d, 0xf3, 0x00, - 0x1b, 0xb4, 0xcf, 0x64, 0x88, 0xf1, 0x69, 0xb6, 0x6a, 0x84, 0x40, 0xec, 0x26, 0x36, 0x34, 0xdf, - 0xc8, 0x65, 0x24, 0xef, 0x99, 0x88, 0xb4, 0x70, 0x5d, 0xe7, 0x07, 0xbf, 0x32, 0x12, 0x7f, 0x22, - 0x1a, 0x53, 0xda, 0xce, 0x3e, 0x6d, 0x95, 0x8c, 0x44, 0xff, 0xf3, 0xaa, 0xfd, 0x13, 0x01, 0x92, - 0xf7, 0x4b, 0xd5, 0x33, 0xd7, 0xcd, 0x95, 0x98, 0xf4, 0x25, 0xd2, 0xf8, 0x43, 0xbd, 0xd1, 0xd0, - 0x8d, 0x3a, 0x71, 0x69, 0xbe, 0x8b, 0x55, 0xb7, 0x66, 0x79, 0x4e, 0xde, 0x66, 0x54, 0xb4, 0x00, - 0x59, 0xd5, 0xc2, 0x1a, 0x36, 0x1c, 0x5d, 0x69, 0xd8, 0xbc, 0x8a, 0x41, 0x12, 0x2f, 0xdc, 0xf7, - 0x05, 0x18, 0xa5, 0x9d, 0x17, 0xbd, 0x00, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x15, 0xca, - 0x48, 0x3e, 0xa1, 0x67, 0x21, 0x2f, 0x43, 0x4e, 0x51, 0x55, 0xb3, 0x6d, 0x38, 0xb2, 0xa1, 0x34, - 0x31, 0x2f, 0x6c, 0x96, 0xd3, 0xb6, 0x94, 0x26, 0x46, 0xf3, 0xe0, 0x3e, 0x7a, 0x67, 0x17, 0x33, - 0x12, 0x70, 0xd2, 0x06, 0x3e, 0xe1, 0x25, 0xf9, 0x85, 0x00, 0x69, 0xb7, 0xd3, 0x93, 0xc2, 0xd4, - 0xb1, 0x81, 0x2d, 0xc5, 0x31, 0xbd, 0xc2, 0x78, 0x84, 0xce, 0x19, 0x2f, 0xe3, 0xcf, 0x78, 0x33, - 0x30, 0xea, 0x90, 0x7e, 0xcd, 0xcb, 0xc1, 0x1e, 0xe8, 0x5a, 0x73, 0x43, 0xa9, 0xb3, 0xe5, 0xb5, - 0x8c, 0xc4, 0x1e, 0x48, 0x95, 0x78, 0x0c, 0x2d, 0xd3, 0x0e, 0x7f, 0x22, 0xe5, 0x65, 0x31, 0x9e, - 0xbb, 0xb8, 0xae, 0x1b, 0xb4, 0x03, 0x24, 0x25, 0xa0, 0xa4, 0x15, 0x42, 0x41, 0x97, 0x20, 0xc3, - 0x32, 0x60, 0x43, 0xa3, 0xbd, 0x20, 0x29, 0xa5, 0x29, 0xa1, 0xec, 0x1e, 0xce, 0x9a, 0x3d, 0x80, - 0x8c, 0x37, 0xc6, 0x48, 0x43, 0xb6, 0x6d, 0x4f, 0xa9, 0xf4, 0x3f, 0x7a, 0x0d, 0x66, 0x9e, 0xb4, - 0x95, 0x86, 0xbe, 0x47, 0x57, 0xce, 0x48, 0x36, 0xa6, 0x3f, 0x56, 0x1f, 0xe4, 0xa5, 0x51, 0x09, - 0x54, 0x8d, 0xee, 0x90, 0x4c, 0xfa, 0x43, 0x32, 0xb8, 0x15, 0x52, 0xfc, 0x44, 0x80, 0x29, 0x16, - 0x06, 0xc4, 0x22, 0x51, 0xe3, 0x73, 0x30, 0xde, 0x86, 0x8c, 0xa6, 0x38, 0x0a, 0x3b, 0x9f, 0x99, - 0xe8, 0x7b, 0x3e, 0xd3, 0xb5, 0xf8, 0x24, 0x3f, 0x3d, 0xa3, 0x89, 0x20, 0x45, 0xfe, 0xb3, 0x03, - 0xad, 0x12, 0xfd, 0xef, 0x07, 0x56, 0x04, 0x8b, 0x1b, 0xa7, 0xc3, 0xb5, 0x04, 0xe7, 0x88, 0xf6, - 0xcb, 0x86, 0x6a, 0x9d, 0xb4, 0x1c, 0xdd, 0x34, 0x1e, 0xd2, 0x5f, 0x1b, 0x89, 0x81, 0x8d, 0x29, - 0xba, 0x1f, 0xc5, 0xcb, 0xf2, 0x1f, 0xc7, 0x60, 0xa2, 0x7c, 0xdc, 0x32, 0xad, 0x58, 0x17, 0xb5, - 0x56, 0x60, 0x9c, 0x23, 0xfe, 0x3e, 0x5b, 0xc5, 0x1d, 0xb6, 0xda, 0xdd, 0x85, 0xe5, 0x8c, 0x68, - 0x05, 0x80, 0xc5, 0x8c, 0xd2, 0x58, 0xa2, 0xe4, 0x19, 0x36, 0xcc, 0x28, 0x1b, 0xa1, 0xa2, 0x2d, - 0xc8, 0x36, 0x0f, 0x55, 0x55, 0xde, 0xd3, 0x1b, 0x0e, 0x0f, 0xba, 0x8b, 0x8e, 0x18, 0xdf, 0x7c, - 0x5c, 0x2a, 0xdd, 0xa3, 0x99, 0x58, 0xfc, 0x9b, 0xff, 0x2c, 0x01, 0x91, 0xc0, 0xfe, 0xa3, 0x57, - 0x80, 0x9f, 0x9b, 0x91, 0x6d, 0xf7, 0x88, 0xdc, 0xca, 0xc4, 0xb3, 0xd3, 0xf9, 0x8c, 0x44, 0xa9, - 0xd5, 0x6a, 0x4d, 0xca, 0xb0, 0x0c, 0x55, 0xdb, 0x41, 0x2f, 0xc1, 0x84, 0xd9, 0xd4, 0x1d, 0xd9, - 0xf5, 0x81, 0xb8, 0xdb, 0x98, 0x23, 0x44, 0xd7, 0x47, 0x42, 0x35, 0xb8, 0x8e, 0x0d, 0x3a, 0x0a, - 0x48, 0x3d, 0xe5, 0x5d, 0xb6, 0x16, 0xe9, 0xb0, 0xf1, 0x2e, 0x9b, 0x2d, 0x47, 0x6f, 0xea, 0x4f, - 0xe9, 0x66, 0x35, 0xdf, 0x2f, 0x7a, 0x89, 0x65, 0x27, 0xf5, 0x5b, 0xa1, 0x8b, 0x94, 0x3c, 0xef, - 0xc3, 0x40, 0x56, 0xf4, 0x7d, 0x01, 0xce, 0x73, 0x45, 0xca, 0xbb, 0x34, 0xe4, 0x5d, 0x69, 0xe8, - 0xce, 0x89, 0x7c, 0x70, 0x58, 0x48, 0x53, 0xe7, 0xf4, 0xab, 0x91, 0x0d, 0x12, 0xe8, 0x07, 0x8b, - 0x6e, 0xb3, 0x9c, 0x3c, 0xe0, 0xcc, 0x1b, 0x87, 0x65, 0xc3, 0xb1, 0x4e, 0x56, 0x2e, 0x3c, 0x3b, - 0x9d, 0x9f, 0xee, 0x4e, 0x7d, 0x2c, 0x4d, 0xdb, 0xdd, 0x2c, 0xa8, 0x02, 0x80, 0xbd, 0xde, 0x48, - 0x43, 0xfe, 0xa2, 0xdd, 0x8b, 0xc8, 0x6e, 0x2b, 0x05, 0x78, 0xd1, 0x0d, 0x10, 0xf9, 0xa1, 0x97, - 0x3d, 0xbd, 0x81, 0x65, 0x5b, 0x7f, 0x8a, 0x0b, 0x40, 0x6d, 0x50, 0x9e, 0xd1, 0x89, 0x88, 0xaa, - 0xfe, 0x14, 0xcf, 0x7e, 0x17, 0x0a, 0xbd, 0x4a, 0x1f, 0x1c, 0x08, 0x19, 0xb6, 0x31, 0xfb, 0x56, - 0x78, 0x45, 0x66, 0x88, 0xae, 0xea, 0xae, 0xca, 0x24, 0xde, 0x72, 0x4d, 0xd0, 0xcf, 0x12, 0x30, - 0xb1, 0xd2, 0x6e, 0x1c, 0x3c, 0x6c, 0x55, 0xdb, 0xcd, 0xa6, 0x62, 0x9d, 0x10, 0x53, 0xc9, 0x4c, - 0x07, 0x29, 0xa6, 0xc0, 0x4c, 0x25, 0xb5, 0x0d, 0xfa, 0x53, 0x4c, 0x26, 0xb3, 0xe0, 0x21, 0x6d, - 0x16, 0xd2, 0x4f, 0x6b, 0x12, 0x38, 0x79, 0x6d, 0x1e, 0xd9, 0xe8, 0x2d, 0x28, 0x04, 0x32, 0xd2, - 0xe5, 0x13, 0x19, 0x1b, 0x8e, 0xa5, 0x63, 0xb6, 0x1c, 0x98, 0x94, 0x02, 0xe1, 0x34, 0x6b, 0x24, - 0xb9, 0xcc, 0x52, 0x51, 0x0d, 0x72, 0x24, 0xe3, 0x89, 0x4c, 0x27, 0x1b, 0x77, 0xd1, 0xf6, 0x56, - 0x44, 0xe5, 0x42, 0xe5, 0x5e, 0xa4, 0x5a, 0x2a, 0x51, 0x1e, 0xfa, 0x57, 0xca, 0x62, 0x9f, 0x32, - 0xfb, 0x2e, 0x88, 0x9d, 0x19, 0x82, 0x1a, 0x4d, 0x31, 0x8d, 0xce, 0x04, 0x35, 0x9a, 0x0c, 0x68, - 0x6b, 0x3d, 0x95, 0x4e, 0x89, 0xa3, 0xc5, 0xdf, 0x26, 0x21, 0xef, 0x76, 0xb6, 0x38, 0xd1, 0xcc, - 0x0a, 0x8c, 0x92, 0xae, 0xe1, 0x06, 0x7f, 0x5c, 0xeb, 0xd3, 0xc7, 0x79, 0xf8, 0x38, 0xe9, 0x32, - 0x2e, 0x1e, 0xa6, 0xac, 0x71, 0x98, 0x9d, 0xd9, 0xbf, 0x99, 0x80, 0x14, 0x05, 0x10, 0xb7, 0x20, - 0x45, 0xa7, 0x0e, 0x61, 0x98, 0xa9, 0x83, 0x66, 0xf5, 0x26, 0xbb, 0x44, 0xc0, 0xff, 0x24, 0xce, - 0xdc, 0xbe, 0xf2, 0xc6, 0xad, 0xdb, 0xd4, 0xe4, 0xe4, 0x24, 0xfe, 0x84, 0x56, 0x68, 0x54, 0x92, - 0x69, 0x39, 0x58, 0xe3, 0x8e, 0xfb, 0xc2, 0xa0, 0xf6, 0x75, 0xa7, 0x29, 0x97, 0x0f, 0x5d, 0x84, - 0x24, 0xb1, 0x65, 0xe3, 0x2c, 0x62, 0xe1, 0xd9, 0xe9, 0x7c, 0x92, 0x58, 0x31, 0x42, 0x43, 0x4b, - 0x90, 0x0d, 0x1b, 0x0e, 0xe1, 0x46, 0x86, 0x99, 0xc7, 0xc0, 0xa0, 0x87, 0x86, 0x37, 0xc0, 0x18, - 0x68, 0xe5, 0x6d, 0xfc, 0xbd, 0x51, 0x98, 0x58, 0x6b, 0xc6, 0x3d, 0xb1, 0x2c, 0x87, 0x5b, 0x38, - 0x0a, 0xed, 0x84, 0x5e, 0x1a, 0xd1, 0xc0, 0xa1, 0x39, 0x3d, 0x79, 0xb6, 0x39, 0x7d, 0x8d, 0xb8, - 0xc0, 0xfc, 0xd6, 0x85, 0x64, 0x0f, 0x60, 0x13, 0x7e, 0x3f, 0xf5, 0x62, 0x24, 0xc2, 0xe3, 0x1f, - 0xa8, 0xa0, 0x51, 0x27, 0xef, 0x52, 0x4f, 0x9b, 0xf5, 0xb2, 0xb1, 0xe1, 0x7b, 0xd9, 0x38, 0x36, - 0x34, 0x3a, 0xb5, 0x85, 0xed, 0xea, 0xf8, 0xf3, 0xdb, 0xd5, 0xd9, 0xa7, 0xbc, 0xb3, 0xbe, 0x0d, - 0x49, 0x4d, 0x77, 0x1b, 0x67, 0xf8, 0x09, 0x9b, 0x30, 0x0d, 0xe8, 0xb5, 0xa9, 0x60, 0xaf, 0x0d, - 0x2e, 0x70, 0xcc, 0x3e, 0x04, 0xf0, 0x35, 0x84, 0x16, 0x60, 0xcc, 0x6c, 0x68, 0xee, 0xb9, 0x92, - 0x89, 0x95, 0xcc, 0xb3, 0xd3, 0xf9, 0xd1, 0x87, 0x0d, 0x6d, 0x6d, 0x55, 0x1a, 0x35, 0x1b, 0xda, - 0x9a, 0x46, 0x2f, 0xbe, 0xc0, 0x47, 0xb2, 0x17, 0x84, 0x96, 0x93, 0xc6, 0x0d, 0x7c, 0xb4, 0x8a, - 0x6d, 0xb5, 0x23, 0x38, 0x86, 0x74, 0xc1, 0x1f, 0x0b, 0x90, 0x77, 0x5b, 0x23, 0x5e, 0x33, 0x93, - 0xd6, 0x9b, 0x7c, 0xd8, 0x25, 0xcf, 0x36, 0xec, 0x5c, 0x3e, 0x7e, 0xaa, 0xf6, 0x07, 0x02, 0x0f, - 0x40, 0xae, 0xaa, 0x8a, 0x43, 0x9c, 0x8d, 0x18, 0x87, 0xca, 0xcb, 0x20, 0x5a, 0x8a, 0xa1, 0x99, - 0x4d, 0xfd, 0x29, 0x66, 0x2b, 0xa2, 0x36, 0xdf, 0xdc, 0x9c, 0xf4, 0xe8, 0x74, 0xc9, 0xcf, 0x5d, - 0xd0, 0xfd, 0x83, 0xc0, 0x83, 0x95, 0xbd, 0xc2, 0xc4, 0xa9, 0xb4, 0x0d, 0x18, 0xb3, 0x58, 0xc8, - 0x23, 0x1b, 0xba, 0xaf, 0x46, 0x08, 0x89, 0x7a, 0x3b, 0x8b, 0x28, 0xf4, 0x06, 0x0f, 0x15, 0x31, - 0xfb, 0x0d, 0x18, 0xa5, 0xe4, 0xe7, 0x30, 0xb0, 0x5c, 0xf3, 0xbf, 0x4f, 0xc0, 0x15, 0xfa, 0xba, - 0xc7, 0xd8, 0xd2, 0xf7, 0x4e, 0xb6, 0x2d, 0xd3, 0xc1, 0xaa, 0x83, 0x35, 0xff, 0x18, 0x47, 0xac, - 0x56, 0x2b, 0xd3, 0x72, 0x5f, 0x70, 0xa6, 0xd0, 0x2f, 0x8f, 0x0b, 0x6d, 0xc0, 0x24, 0xbb, 0x5c, - 0x47, 0x56, 0x1a, 0xfa, 0x21, 0x96, 0x15, 0xe7, 0x2c, 0x73, 0xd3, 0x04, 0xe3, 0x5d, 0x26, 0xac, - 0xcb, 0x0e, 0xd2, 0x20, 0xc3, 0x85, 0xe9, 0x1a, 0xbf, 0x51, 0xe7, 0xfe, 0xe7, 0x5b, 0xf3, 0x4b, - 0x4b, 0x54, 0xde, 0xda, 0xaa, 0x94, 0x66, 0x92, 0xbd, 0x3d, 0x9b, 0x5f, 0x0b, 0x70, 0x75, 0x80, - 0xa2, 0xe3, 0xec, 0x66, 0xb3, 0x90, 0x3e, 0x24, 0x2f, 0xd2, 0xb9, 0xa6, 0xd3, 0x92, 0xf7, 0x8c, - 0x36, 0x61, 0x62, 0x4f, 0xd1, 0x1b, 0xee, 0xb5, 0x38, 0xfd, 0xe2, 0x05, 0xa3, 0xc3, 0x58, 0x73, - 0x8c, 0x9d, 0x26, 0xd2, 0x83, 0x8e, 0x53, 0xcb, 0x9a, 0x56, 0xad, 0x72, 0x0b, 0x16, 0x5f, 0x7f, - 0x71, 0xa1, 0x63, 0xc2, 0x87, 0x8e, 0xe8, 0x55, 0x40, 0x9a, 0x6e, 0xb3, 0xdb, 0x3a, 0xec, 0x7d, - 0x45, 0x33, 0x8f, 0xfc, 0xa8, 0x89, 0x29, 0x37, 0xa5, 0xea, 0x26, 0xa0, 0x2a, 0x50, 0xdc, 0x22, - 0xdb, 0x8e, 0xe2, 0x6d, 0xfc, 0x5c, 0x1d, 0xea, 0xd4, 0x15, 0x03, 0x34, 0xde, 0xa3, 0x94, 0x21, - 0x72, 0xe8, 0x5f, 0xe2, 0x81, 0xeb, 0xa4, 0xea, 0x8e, 0xac, 0xd8, 0xee, 0x11, 0x1d, 0x76, 0x4f, - 0x48, 0x9e, 0xd1, 0x97, 0xed, 0xe0, 0xc9, 0x1b, 0x76, 0x82, 0xc0, 0x57, 0x50, 0x9c, 0x40, 0xf7, - 0x9f, 0x0b, 0x90, 0x97, 0xf0, 0x9e, 0x85, 0xed, 0x58, 0x01, 0xff, 0x3d, 0xc8, 0x59, 0x4c, 0xaa, - 0xbc, 0x67, 0x99, 0xcd, 0xb3, 0x8c, 0xb1, 0x2c, 0x67, 0xbc, 0x67, 0x99, 0xcd, 0xd0, 0xd5, 0x09, - 0x8f, 0x61, 0xd2, 0x2b, 0x69, 0x9c, 0x2a, 0xf8, 0x84, 0x9e, 0x34, 0x66, 0x82, 0xe3, 0x0e, 0x5f, - 0xf8, 0x22, 0xf4, 0x40, 0x77, 0x9a, 0x82, 0xc5, 0x8d, 0x53, 0x19, 0x7f, 0x10, 0x20, 0x5f, 0x6d, - 0xef, 0xb2, 0xcb, 0xa2, 0xe2, 0xd3, 0x43, 0x19, 0x32, 0x0d, 0xbc, 0xe7, 0xc8, 0xcf, 0x15, 0xf5, - 0x9e, 0x26, 0xac, 0x34, 0xf2, 0xff, 0x3e, 0x80, 0x45, 0xcf, 0xb5, 0x51, 0x39, 0xc9, 0x33, 0xca, - 0xc9, 0x50, 0x5e, 0xdf, 0xc9, 0x29, 0x7e, 0x92, 0x80, 0x49, 0xaf, 0xb2, 0x71, 0x5a, 0xcf, 0xf7, - 0x42, 0x56, 0x23, 0x79, 0x16, 0xab, 0x31, 0xc5, 0xa3, 0x37, 0xa2, 0x2d, 0xc7, 0x22, 0x4c, 0x53, - 0x17, 0x44, 0x56, 0x5a, 0xad, 0x86, 0xee, 0x42, 0x59, 0x6a, 0x97, 0x52, 0xd2, 0x14, 0x4d, 0x5a, - 0x66, 0x29, 0x14, 0xc4, 0x92, 0xfe, 0xb7, 0x67, 0x61, 0xfc, 0x14, 0xcb, 0x14, 0x55, 0x9d, 0x25, - 0x3a, 0x25, 0xcb, 0x18, 0xab, 0x84, 0x8f, 0xf7, 0xbc, 0x0f, 0x61, 0x8a, 0x6a, 0x36, 0xee, 0xb3, - 0xb5, 0xbc, 0x39, 0x7e, 0x94, 0x00, 0x14, 0x94, 0xff, 0xc5, 0xb5, 0x48, 0x22, 0xbe, 0x16, 0x79, - 0x05, 0x10, 0x8b, 0x42, 0xb4, 0xe5, 0x16, 0xb6, 0x64, 0x1b, 0xab, 0x26, 0xbf, 0xc2, 0x48, 0x90, - 0x44, 0x9e, 0xb2, 0x8d, 0xad, 0x2a, 0xa5, 0xa3, 0xbb, 0x00, 0xfe, 0x55, 0x72, 0x7c, 0x3a, 0xe9, - 0x7b, 0x93, 0x9c, 0x94, 0xb1, 0xdc, 0xbf, 0xc5, 0x8f, 0x66, 0x21, 0xc7, 0x35, 0xb9, 0x63, 0xe8, - 0xa6, 0x81, 0x6e, 0x41, 0xb2, 0xce, 0x37, 0x03, 0xb2, 0x91, 0xcb, 0x71, 0xfe, 0xad, 0x6d, 0x95, - 0x11, 0x89, 0xe4, 0x25, 0x2c, 0xad, 0xb6, 0x13, 0xe1, 0x3c, 0xf9, 0xb1, 0xd6, 0x41, 0x96, 0x56, - 0xdb, 0x41, 0x55, 0x98, 0x54, 0xfd, 0x3b, 0xa8, 0x64, 0xc2, 0x9e, 0xec, 0x09, 0x93, 0x22, 0xef, - 0xfe, 0xaa, 0x8c, 0x48, 0x79, 0x35, 0x94, 0x80, 0x4a, 0xc1, 0x4b, 0x8f, 0x52, 0x5d, 0x61, 0x5d, - 0xfe, 0x21, 0xde, 0xf0, 0x85, 0x4b, 0x95, 0x91, 0xc0, 0xdd, 0x48, 0xe8, 0x6d, 0x18, 0xd3, 0xe8, - 0x65, 0x3a, 0xbc, 0x5f, 0x47, 0x75, 0xbd, 0xd0, 0xfd, 0x45, 0x95, 0x11, 0x89, 0x73, 0xa0, 0x75, - 0xc8, 0xb1, 0x7f, 0xcc, 0x89, 0xe1, 0xd8, 0xf1, 0x6a, 0x6f, 0x09, 0x81, 0xa9, 0xa1, 0x32, 0x22, - 0x65, 0x35, 0x9f, 0x8a, 0x5e, 0x87, 0x94, 0xad, 0x2a, 0x2e, 0x7a, 0x9c, 0xeb, 0x71, 0x93, 0x86, - 0xcf, 0x4c, 0x73, 0xa3, 0xbb, 0xec, 0x36, 0x46, 0xe7, 0xd8, 0x5d, 0xce, 0x8b, 0x2a, 0x7e, 0xe8, - 0x7c, 0x36, 0x29, 0x3e, 0xa6, 0x04, 0x74, 0x1f, 0xb2, 0x0a, 0xf1, 0x06, 0x65, 0x7a, 0x1e, 0x92, - 0xae, 0xdf, 0x45, 0xef, 0x94, 0x77, 0x9d, 0x65, 0xad, 0xd0, 0x43, 0xe0, 0x2e, 0xd1, 0x17, 0xd4, - 0xc4, 0x56, 0x1d, 0x17, 0xb2, 0xfd, 0x05, 0x05, 0xc3, 0xb8, 0x3c, 0x41, 0x94, 0x48, 0xbc, 0xc2, - 0x7d, 0xf7, 0xac, 0x0b, 0xad, 0x54, 0xae, 0xe7, 0xae, 0x6c, 0xc4, 0x59, 0x9d, 0xca, 0x88, 0x94, - 0xdb, 0x0f, 0x90, 0xd1, 0x22, 0x24, 0xea, 0x6a, 0x61, 0xa2, 0xe7, 0x08, 0xf1, 0x4e, 0xa2, 0x54, - 0x46, 0xa4, 0x44, 0x5d, 0x45, 0xef, 0x42, 0x9a, 0x1d, 0x25, 0x38, 0x36, 0x0a, 0xf9, 0x9e, 0x76, - 0x22, 0x7c, 0x20, 0xa3, 0x32, 0x22, 0xd1, 0xd3, 0x0b, 0xe4, 0x7d, 0xdb, 0x90, 0xb7, 0x58, 0x1c, - 0x9c, 0x1b, 0xc1, 0x2a, 0xf6, 0xdc, 0xa9, 0x8e, 0x0a, 0x62, 0xad, 0x50, 0x74, 0x10, 0xa0, 0xa3, - 0xef, 0xc0, 0x4c, 0x58, 0x22, 0xef, 0x69, 0x53, 0x3d, 0x77, 0x5d, 0x7b, 0x86, 0x52, 0x56, 0x46, - 0x24, 0x64, 0x75, 0x25, 0xa2, 0x37, 0x61, 0x94, 0xb5, 0x1a, 0xa2, 0x22, 0xa3, 0x42, 0x30, 0x3a, - 0x1a, 0x8c, 0xe5, 0x27, 0x9d, 0xdf, 0xe1, 0x01, 0x60, 0x72, 0xc3, 0xac, 0x17, 0xa6, 0x7b, 0x76, - 0xfe, 0xee, 0x80, 0x36, 0xd2, 0xf9, 0x1d, 0x9f, 0x4a, 0xda, 0xdd, 0x62, 0x29, 0x3c, 0x5e, 0x68, - 0xa6, 0x67, 0xbb, 0x47, 0xc4, 0x85, 0x55, 0x68, 0x48, 0xbe, 0x4f, 0x26, 0x45, 0xb3, 0xd8, 0xb5, - 0x2f, 0x32, 0x1d, 0x53, 0xe7, 0x7a, 0x16, 0xad, 0xfb, 0x76, 0x9c, 0x0a, 0xf5, 0x9a, 0x3c, 0x2a, - 0x7a, 0x0c, 0x22, 0xbf, 0x90, 0xc1, 0xdf, 0x3b, 0x38, 0x4f, 0xe5, 0xbd, 0x1c, 0x69, 0xba, 0xa2, - 0x02, 0x6c, 0x2a, 0x23, 0xd2, 0xa4, 0x1a, 0x4e, 0x41, 0xef, 0xc3, 0x14, 0x95, 0x27, 0xab, 0xfe, - 0x4d, 0x1a, 0x85, 0x42, 0xd7, 0x8d, 0x0c, 0xbd, 0x2f, 0xdd, 0x70, 0x25, 0x8b, 0x6a, 0x47, 0x12, - 0xe9, 0xc6, 0xba, 0xa1, 0x3b, 0xd4, 0xca, 0xce, 0xf6, 0xec, 0xc6, 0xe1, 0x3b, 0xfb, 0x48, 0x37, - 0xd6, 0x19, 0x85, 0x74, 0x63, 0x87, 0x07, 0x93, 0xf1, 0xe6, 0x78, 0xa1, 0x67, 0x37, 0x8e, 0x8a, - 0x3a, 0x23, 0xdd, 0xd8, 0x09, 0xd2, 0x49, 0x37, 0x66, 0x06, 0xa2, 0x43, 0xee, 0x8b, 0x3d, 0xbb, - 0x71, 0xcf, 0x13, 0xc9, 0xa4, 0x1b, 0x2b, 0x5d, 0x89, 0x68, 0x15, 0x80, 0x39, 0x35, 0x74, 0x52, - 0x9c, 0xeb, 0x39, 0x19, 0x74, 0x86, 0x93, 0x91, 0xc9, 0xa0, 0xe1, 0xd2, 0x88, 0x21, 0xa3, 0x50, - 0x4a, 0xa6, 0x1b, 0xa9, 0x85, 0xf9, 0x9e, 0x86, 0xac, 0x6b, 0x8b, 0x93, 0x18, 0xb2, 0x23, 0x8f, - 0x48, 0x66, 0x15, 0xb6, 0xaa, 0x5b, 0x58, 0xe8, 0x6d, 0x96, 0x83, 0x5b, 0x3c, 0xd4, 0x2c, 0x53, - 0x02, 0x5a, 0x86, 0x0c, 0x99, 0xf3, 0x4f, 0xa8, 0x19, 0xba, 0xdc, 0xd3, 0x3f, 0xed, 0x38, 0x73, - 0x52, 0x19, 0x91, 0xd2, 0x4f, 0x38, 0x89, 0xbc, 0x9e, 0xad, 0x6e, 0x15, 0x8a, 0x3d, 0x5f, 0x1f, - 0x5a, 0x1b, 0x25, 0xaf, 0x67, 0x1c, 0x48, 0x85, 0x73, 0xac, 0xad, 0xf8, 0x81, 0x60, 0x8b, 0x9f, - 0x5e, 0x2d, 0xbc, 0x44, 0x45, 0xf5, 0x5c, 0x2b, 0x8a, 0x3c, 0xa7, 0x5c, 0x19, 0x91, 0xa6, 0x95, - 0xee, 0x54, 0x32, 0xe0, 0xf9, 0xd4, 0xc3, 0x56, 0x98, 0x0a, 0x57, 0x7a, 0x0e, 0xf8, 0x88, 0x35, - 0x39, 0x32, 0xe0, 0x95, 0x00, 0x99, 0x4d, 0x40, 0x9a, 0x6c, 0xdb, 0x6c, 0xdb, 0xfd, 0x6a, 0x9f, - 0x09, 0xa8, 0x63, 0x8d, 0x80, 0x4d, 0x40, 0x5a, 0x95, 0x71, 0x12, 0x41, 0x6a, 0x03, 0x2b, 0x16, - 0x37, 0xb3, 0xd7, 0x7a, 0x0a, 0xea, 0xba, 0x07, 0x8f, 0x08, 0x52, 0x3d, 0x22, 0x71, 0x78, 0x2c, - 0xf7, 0x26, 0x17, 0xee, 0x30, 0x5e, 0xef, 0xe9, 0xf0, 0x44, 0x5e, 0x38, 0x43, 0x1c, 0x1e, 0x2b, - 0x94, 0x80, 0xbe, 0x06, 0xe3, 0x1c, 0xd0, 0x15, 0x6e, 0xf4, 0x71, 0x63, 0x83, 0x48, 0x9c, 0x8c, - 0x6b, 0xce, 0xc3, 0xac, 0x2c, 0x03, 0x92, 0xac, 0x7a, 0x2f, 0xf7, 0xb1, 0xb2, 0x5d, 0x58, 0x96, - 0x59, 0x59, 0x9f, 0x4c, 0xac, 0x2c, 0xeb, 0xa7, 0x7c, 0xae, 0xbb, 0xd9, 0xd3, 0xca, 0x76, 0x9f, - 0x7b, 0x21, 0x56, 0xf6, 0x89, 0x4f, 0x25, 0x35, 0xb3, 0x19, 0x88, 0x2a, 0x7c, 0xa5, 0x67, 0xcd, - 0xc2, 0x98, 0x92, 0xd4, 0x8c, 0xf3, 0x90, 0x66, 0x63, 0x2e, 0x31, 0xd3, 0xf4, 0x2b, 0xbd, 0xcf, - 0xde, 0x77, 0x42, 0x0f, 0xd2, 0x6c, 0x96, 0x47, 0xf4, 0x0d, 0x95, 0xc5, 0x4f, 0x1a, 0x73, 0x4d, - 0xbd, 0xda, 0xdf, 0x50, 0x45, 0x1d, 0xa2, 0xf6, 0x0c, 0x55, 0x28, 0x91, 0x16, 0x95, 0x1d, 0x1f, - 0xa3, 0xe3, 0x7b, 0xb1, 0xcf, 0x35, 0x01, 0x1d, 0x47, 0xf9, 0x68, 0x51, 0x3d, 0xa2, 0x3f, 0x84, - 0xda, 0xec, 0x3e, 0x8b, 0xc2, 0x52, 0xff, 0x21, 0x14, 0xbe, 0x57, 0xc3, 0x1b, 0x42, 0x9c, 0xec, - 0xcd, 0x99, 0xae, 0x87, 0xf1, 0x5a, 0xff, 0x39, 0xb3, 0xd3, 0xb5, 0x60, 0x73, 0x26, 0xf7, 0x29, - 0xfe, 0x96, 0x00, 0x0b, 0xac, 0x6c, 0x74, 0xbd, 0xef, 0x44, 0xf6, 0xd6, 0x4e, 0x03, 0x87, 0x1c, - 0x6e, 0xd1, 0x17, 0xbc, 0xd9, 0xab, 0xb8, 0x03, 0xd6, 0x82, 0x2b, 0x23, 0xd2, 0x8b, 0x4a, 0xbf, - 0x7c, 0x2b, 0xe3, 0x7c, 0xe3, 0xd3, 0x3b, 0xc1, 0x39, 0x29, 0x8a, 0xeb, 0xa9, 0xf4, 0x05, 0xb1, - 0xb0, 0x9e, 0x4a, 0x5f, 0x14, 0x67, 0xd7, 0x53, 0xe9, 0x4b, 0xe2, 0x0b, 0xc5, 0xff, 0x7b, 0x11, - 0x26, 0x5c, 0xe4, 0xc7, 0x10, 0xd1, 0xed, 0x20, 0x22, 0x9a, 0xeb, 0x85, 0x88, 0x38, 0x56, 0xe4, - 0x90, 0xe8, 0x76, 0x10, 0x12, 0xcd, 0xf5, 0x82, 0x44, 0x3e, 0x0f, 0xc1, 0x44, 0xb5, 0x5e, 0x98, - 0xe8, 0xe5, 0x21, 0x30, 0x91, 0x27, 0xaa, 0x13, 0x14, 0xad, 0x76, 0x83, 0xa2, 0x2b, 0xfd, 0x41, - 0x91, 0x27, 0x2a, 0x80, 0x8a, 0xee, 0x76, 0xa0, 0xa2, 0xcb, 0x7d, 0x50, 0x91, 0xc7, 0xef, 0xc2, - 0xa2, 0x8d, 0x48, 0x58, 0x74, 0x6d, 0x10, 0x2c, 0xf2, 0xe4, 0x84, 0x70, 0xd1, 0x1b, 0x21, 0x5c, - 0x34, 0xdf, 0x13, 0x17, 0x79, 0xdc, 0x0c, 0x18, 0xbd, 0xd3, 0x09, 0x8c, 0x2e, 0xf7, 0x01, 0x46, - 0x7e, 0x0d, 0x38, 0x32, 0xaa, 0x44, 0x21, 0xa3, 0xab, 0x03, 0x90, 0x91, 0x27, 0x25, 0x08, 0x8d, - 0x2a, 0x51, 0xd0, 0xe8, 0xea, 0x00, 0x68, 0xd4, 0x21, 0x89, 0x61, 0xa3, 0xad, 0x68, 0x6c, 0x74, - 0x7d, 0x20, 0x36, 0xf2, 0xa4, 0x85, 0xc1, 0xd1, 0x52, 0x00, 0x1c, 0xbd, 0xd8, 0x03, 0x1c, 0x79, - 0xac, 0x04, 0x1d, 0x7d, 0xbd, 0x0b, 0x1d, 0x15, 0xfb, 0xa1, 0x23, 0x8f, 0xd7, 0x83, 0x47, 0x8f, - 0x7a, 0xc0, 0xa3, 0x1b, 0x83, 0xe1, 0x91, 0x27, 0xac, 0x03, 0x1f, 0x29, 0x7d, 0xf1, 0xd1, 0xab, - 0x43, 0xe2, 0x23, 0x4f, 0x7a, 0x14, 0x40, 0x7a, 0x2b, 0x0c, 0x90, 0x16, 0x7a, 0x03, 0x24, 0x4f, - 0x0c, 0x47, 0x48, 0x1b, 0x91, 0x08, 0xe9, 0xda, 0x20, 0x84, 0xe4, 0x8f, 0x83, 0x20, 0x44, 0xda, - 0x8a, 0x86, 0x48, 0xd7, 0x07, 0x42, 0x24, 0xbf, 0xf9, 0x43, 0x18, 0x69, 0x23, 0x12, 0x23, 0x5d, - 0x1b, 0x84, 0x91, 0xfc, 0xc2, 0x05, 0x41, 0xd2, 0x7b, 0x3d, 0x41, 0xd2, 0xcd, 0x61, 0x40, 0x92, - 0x27, 0xb4, 0x0b, 0x25, 0x7d, 0xd0, 0x1b, 0x25, 0x7d, 0xe5, 0x0c, 0x57, 0x13, 0x46, 0xc2, 0xa4, - 0xaf, 0x77, 0xc1, 0xa4, 0x62, 0x3f, 0x98, 0xe4, 0xf7, 0x67, 0x17, 0x27, 0x29, 0x7d, 0x51, 0xcd, - 0xab, 0x43, 0xa2, 0x1a, 0xbf, 0xf3, 0x45, 0xc0, 0x9a, 0x72, 0x04, 0xac, 0xb9, 0xd2, 0x1f, 0xd6, - 0xf8, 0xe6, 0xdc, 0xc7, 0x35, 0x95, 0x28, 0x5c, 0x73, 0x75, 0x00, 0xae, 0xf1, 0xad, 0x50, 0x00, - 0xd8, 0xdc, 0xed, 0x00, 0x36, 0x97, 0x07, 0xc6, 0xf5, 0x04, 0x90, 0xcd, 0x4a, 0x37, 0xb2, 0x79, - 0xa9, 0x2f, 0xb2, 0xf1, 0x24, 0xf8, 0xd0, 0xe6, 0x6e, 0x07, 0xb4, 0xb9, 0xdc, 0x07, 0xda, 0xf8, - 0x05, 0xe0, 0xd8, 0x46, 0xeb, 0x8f, 0x6d, 0x16, 0x87, 0xc5, 0x36, 0x9e, 0xe0, 0x48, 0x70, 0xb3, - 0x15, 0x0d, 0x6e, 0xae, 0x0f, 0xb9, 0xcb, 0xde, 0x85, 0x6e, 0x2a, 0x51, 0xe8, 0xe6, 0xea, 0x00, - 0x74, 0x13, 0x9c, 0x43, 0x3c, 0x78, 0x53, 0x89, 0x82, 0x37, 0x57, 0x07, 0xc0, 0x1b, 0x5f, 0x52, - 0x00, 0xdf, 0xd4, 0x7a, 0xe1, 0x9b, 0x97, 0x87, 0xc0, 0x37, 0xbe, 0xf3, 0xd2, 0x01, 0x70, 0xde, - 0xed, 0x04, 0x38, 0xc5, 0x7e, 0x00, 0xc7, 0x1f, 0x91, 0x2e, 0xc2, 0xd9, 0x8a, 0x46, 0x38, 0xd7, - 0x07, 0x22, 0x9c, 0xa0, 0x91, 0x0c, 0x40, 0x9c, 0x8d, 0x48, 0x88, 0x73, 0x6d, 0x10, 0xc4, 0xf1, - 0x8d, 0x64, 0x10, 0xe3, 0xbc, 0xdb, 0x89, 0x71, 0x8a, 0xfd, 0x30, 0x8e, 0x5f, 0x39, 0x17, 0xe4, - 0x54, 0xa2, 0x40, 0xce, 0xd5, 0x01, 0x20, 0xc7, 0x6f, 0xbc, 0x00, 0xca, 0x51, 0xfa, 0xa2, 0x9c, - 0x57, 0x87, 0x44, 0x39, 0x1d, 0x86, 0x2b, 0x0c, 0x73, 0x2a, 0x51, 0x30, 0xe7, 0xea, 0x00, 0x98, - 0x13, 0x28, 0xac, 0x8f, 0x73, 0xb6, 0xa2, 0x71, 0xce, 0xf5, 0x81, 0x38, 0xa7, 0x63, 0x34, 0xb9, - 0x40, 0x67, 0x23, 0x12, 0xe8, 0x5c, 0x1b, 0x04, 0x74, 0x3a, 0x26, 0x3e, 0xee, 0x1c, 0xfc, 0xed, - 0xe1, 0x91, 0xce, 0x5b, 0x67, 0x47, 0x3a, 0xde, 0x3b, 0x63, 0x81, 0x3a, 0xeb, 0xa9, 0xf4, 0x0b, - 0xe2, 0x8b, 0xc5, 0xdf, 0x8f, 0xc2, 0x58, 0xc5, 0x8b, 0x85, 0xf1, 0x4b, 0x29, 0x3c, 0xcf, 0x35, - 0x48, 0x68, 0x95, 0x8c, 0x58, 0x6a, 0xf7, 0x06, 0xdf, 0x78, 0xd7, 0x7d, 0x1b, 0x1b, 0x67, 0x7d, - 0x8e, 0x53, 0xc8, 0xe8, 0x0d, 0x98, 0x68, 0xdb, 0xd8, 0x92, 0x5b, 0x96, 0x6e, 0x5a, 0xba, 0xc3, - 0x4e, 0x74, 0x08, 0x2b, 0xe2, 0x67, 0xa7, 0xf3, 0xb9, 0x1d, 0x1b, 0x5b, 0xdb, 0x9c, 0x2e, 0xe5, - 0xda, 0x81, 0x27, 0xf7, 0xcb, 0x50, 0xa3, 0xc3, 0x7f, 0x19, 0xea, 0x11, 0x88, 0x16, 0x56, 0xb4, - 0x90, 0x07, 0xc2, 0xae, 0x19, 0x8a, 0xee, 0x33, 0xf4, 0xb0, 0x94, 0x9b, 0x93, 0x5e, 0x37, 0x34, - 0x69, 0x85, 0x89, 0xe8, 0x16, 0x9c, 0x6b, 0x2a, 0xc7, 0x34, 0xea, 0x51, 0x76, 0x9d, 0x3a, 0x1a, - 0xc9, 0xc8, 0x3e, 0xba, 0x84, 0x9a, 0xca, 0x31, 0xfd, 0xcc, 0x14, 0x4b, 0xa2, 0xdf, 0x85, 0xb8, - 0x0a, 0x79, 0x4d, 0xb7, 0x1d, 0xdd, 0x50, 0x1d, 0x7e, 0xc1, 0x2c, 0xbb, 0xb1, 0x75, 0xc2, 0xa5, - 0xb2, 0x5b, 0x64, 0x6f, 0xc2, 0x14, 0x0f, 0x8a, 0x0f, 0x6c, 0x11, 0x02, 0x8f, 0x34, 0xa3, 0x09, - 0xde, 0xae, 0x20, 0x2a, 0xc1, 0x64, 0x5d, 0x71, 0xf0, 0x91, 0x72, 0x22, 0xbb, 0x27, 0xaa, 0xb2, - 0xf4, 0x7e, 0xc6, 0x4b, 0xcf, 0x4e, 0xe7, 0x27, 0xee, 0xb3, 0xa4, 0xae, 0x83, 0x55, 0x13, 0xf5, - 0x40, 0x82, 0x86, 0xae, 0xc3, 0xa4, 0x62, 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, 0xec, 0xb6, 0x4d, - 0x21, 0x45, 0x5a, 0xca, 0x53, 0x72, 0xc9, 0xa5, 0xa2, 0xcb, 0x90, 0xe3, 0x11, 0xe3, 0xec, 0x5b, - 0x35, 0x93, 0xb4, 0xaa, 0xfc, 0xd3, 0x09, 0xf4, 0x73, 0x35, 0xe8, 0x2e, 0xcc, 0xf2, 0x2b, 0xe5, - 0x8f, 0x14, 0x4b, 0x93, 0xa9, 0xd6, 0xfd, 0xfe, 0x29, 0x52, 0xb1, 0x17, 0xd8, 0x15, 0xf2, 0x24, - 0x03, 0x51, 0x75, 0xf0, 0xfe, 0xd5, 0x71, 0x31, 0xbd, 0x9e, 0x4a, 0xe7, 0xc4, 0x89, 0xf5, 0x54, - 0x3a, 0x2f, 0x4e, 0x16, 0xff, 0xa1, 0x00, 0xb9, 0xd0, 0x29, 0x94, 0xbb, 0x1d, 0x9b, 0xc0, 0x17, - 0xa3, 0xa1, 0x53, 0xaf, 0xb8, 0xb1, 0x34, 0x6f, 0x2a, 0x37, 0x6a, 0x6e, 0xbe, 0xb7, 0xeb, 0x4d, - 0x17, 0x12, 0xdc, 0xc8, 0x03, 0x97, 0xed, 0xed, 0xd4, 0x3f, 0xfe, 0x78, 0x7e, 0xa4, 0xf8, 0x49, - 0x0a, 0x26, 0xc2, 0xa7, 0x4d, 0xd6, 0x3a, 0xca, 0x15, 0x65, 0xda, 0x42, 0x1c, 0x8b, 0x7d, 0x2e, - 0xdd, 0xcb, 0xf8, 0x77, 0xc2, 0xb3, 0x62, 0x2e, 0xf4, 0xd9, 0xea, 0x0e, 0x96, 0xd3, 0x67, 0x9c, - 0xfd, 0x3b, 0x49, 0xcf, 0x44, 0x2c, 0xc2, 0x28, 0xbd, 0x06, 0x86, 0x17, 0x2d, 0xea, 0x20, 0x73, - 0x99, 0xa4, 0x4b, 0x2c, 0x1b, 0x31, 0x29, 0xb5, 0xe7, 0xba, 0x59, 0xcd, 0xbf, 0xc4, 0xe2, 0xec, - 0x1f, 0x6f, 0xe3, 0xf7, 0xeb, 0x8d, 0x9e, 0xed, 0x7e, 0x3d, 0xb6, 0x29, 0xdd, 0x68, 0x30, 0x73, - 0xcd, 0x06, 0xd5, 0x58, 0xd7, 0x69, 0x61, 0x2a, 0x82, 0x7f, 0x53, 0x6f, 0x51, 0xe2, 0xdf, 0xd4, - 0x0b, 0x04, 0x32, 0xe6, 0x3d, 0x11, 0x6c, 0x04, 0x96, 0xdc, 0x59, 0x9a, 0x7d, 0xe8, 0x6d, 0x7c, - 0xe8, 0x0f, 0xbd, 0x81, 0xb7, 0x49, 0x6f, 0xb3, 0x98, 0x59, 0xde, 0x5f, 0xfe, 0xab, 0xc0, 0x23, - 0x1a, 0x1e, 0x98, 0xe6, 0x41, 0xdb, 0x8b, 0x85, 0x9c, 0x0d, 0xde, 0x6e, 0x97, 0xfe, 0xec, 0x74, - 0x3e, 0x25, 0x79, 0xd7, 0xdb, 0x45, 0x99, 0xac, 0xc4, 0xe7, 0x33, 0x59, 0x97, 0x21, 0xd7, 0xb2, - 0xf0, 0x1e, 0x76, 0xd4, 0x7d, 0xd9, 0x68, 0x37, 0xf9, 0x81, 0x87, 0xac, 0x4b, 0xdb, 0x6a, 0x37, - 0xd1, 0xcb, 0x20, 0x7a, 0x59, 0x38, 0x38, 0xe4, 0xe7, 0xbd, 0x27, 0x5d, 0x3a, 0x87, 0x92, 0xc5, - 0xff, 0x2f, 0xc0, 0x74, 0xa8, 0x4e, 0x7c, 0x24, 0xac, 0x43, 0x56, 0xf3, 0x26, 0x09, 0xbb, 0x20, - 0x9c, 0x31, 0x26, 0x30, 0xc8, 0x8c, 0x64, 0x38, 0xef, 0xbe, 0x96, 0xde, 0x78, 0xee, 0x8b, 0x4d, - 0x9c, 0x51, 0xec, 0x39, 0x5f, 0xce, 0x6a, 0xe0, 0x05, 0xde, 0xd0, 0x48, 0x0e, 0x35, 0x34, 0x8a, - 0x3f, 0x16, 0x40, 0xa4, 0x2f, 0xb8, 0x87, 0xb1, 0x16, 0x8b, 0x4d, 0x72, 0x23, 0x6e, 0x13, 0xc3, - 0x1f, 0x69, 0x08, 0x7d, 0xb1, 0x21, 0x19, 0xfe, 0x62, 0x43, 0xf1, 0x63, 0x01, 0xf2, 0x5e, 0x09, - 0xd9, 0xb7, 0xca, 0xfa, 0x5c, 0xa2, 0xf8, 0x7c, 0x5f, 0xe8, 0x72, 0xef, 0x85, 0x18, 0xea, 0xf3, - 0x69, 0xc1, 0x7b, 0x21, 0xd8, 0xd7, 0xa4, 0xfe, 0xa9, 0xdb, 0x73, 0x48, 0x11, 0x4b, 0xfe, 0x99, - 0xff, 0xe7, 0x38, 0xdd, 0x21, 0xd1, 0x4f, 0x3e, 0x9a, 0x8d, 0x43, 0x76, 0x21, 0xc7, 0x50, 0xc6, - 0x0a, 0xf1, 0x38, 0x1e, 0xe0, 0xcb, 0x48, 0x5a, 0xad, 0x4a, 0x3f, 0x06, 0xc9, 0xfe, 0xdb, 0xc5, - 0x7b, 0x01, 0x05, 0xd2, 0xc6, 0x27, 0x5a, 0x1a, 0xca, 0x80, 0xba, 0x5a, 0x62, 0x7d, 0xe5, 0x97, - 0xc1, 0x96, 0x28, 0x1f, 0x12, 0xf8, 0x70, 0x07, 0x92, 0x87, 0x4a, 0xa3, 0x5f, 0xfc, 0x52, 0xa8, - 0xe5, 0x24, 0x92, 0x1b, 0xdd, 0x0b, 0x5d, 0x95, 0x90, 0xe8, 0xed, 0xea, 0x76, 0xab, 0x34, 0x74, - 0xa5, 0xc2, 0x9b, 0xe1, 0xbe, 0xde, 0xf7, 0xf5, 0xc1, 0x4e, 0xff, 0x76, 0xea, 0xd3, 0x8f, 0xe7, - 0x85, 0xe2, 0x9b, 0x70, 0xf1, 0xbe, 0x69, 0xdb, 0x7a, 0x8b, 0xc0, 0x1b, 0x3a, 0x80, 0x88, 0xed, - 0xf6, 0x2c, 0x59, 0xba, 0x45, 0x81, 0xae, 0xc1, 0x46, 0x7c, 0x46, 0xf2, 0x9e, 0x8b, 0xff, 0x4e, - 0x80, 0x0b, 0xdd, 0x9c, 0x4c, 0x21, 0x51, 0xa7, 0xc7, 0xc6, 0x55, 0xd3, 0xbf, 0x3a, 0x6c, 0x70, - 0xc7, 0x72, 0xb3, 0x13, 0x37, 0x86, 0xbf, 0x53, 0x6e, 0x2a, 0x74, 0xa4, 0xf3, 0x13, 0xae, 0x79, - 0x4e, 0xde, 0x64, 0x54, 0x7f, 0xd0, 0xa7, 0x86, 0x1a, 0xf4, 0x37, 0xab, 0x30, 0x1d, 0x61, 0x5f, - 0x51, 0x1e, 0x20, 0xf0, 0x05, 0x0b, 0xfe, 0xe5, 0xcc, 0xe5, 0x55, 0x79, 0x67, 0xab, 0xf4, 0x70, - 0x73, 0x73, 0xad, 0x56, 0x2b, 0xaf, 0x8a, 0x02, 0x12, 0x21, 0x17, 0xfa, 0xfe, 0x45, 0x82, 0x7d, - 0x4b, 0xf3, 0xe6, 0x5f, 0x03, 0xf0, 0x3f, 0xab, 0x43, 0x64, 0x6d, 0x94, 0xdf, 0x97, 0x1f, 0x2f, - 0x3f, 0xd8, 0x29, 0x57, 0xc5, 0x11, 0x84, 0x20, 0xbf, 0xb2, 0x5c, 0x2b, 0x55, 0x64, 0xa9, 0x5c, - 0xdd, 0x7e, 0xb8, 0x55, 0x2d, 0xbb, 0xdf, 0xe0, 0xbc, 0xb9, 0x0a, 0xb9, 0xe0, 0x75, 0x1a, 0x68, - 0x1a, 0x26, 0x4b, 0x95, 0x72, 0x69, 0x43, 0x7e, 0xbc, 0xb6, 0x2c, 0x3f, 0xda, 0x29, 0xef, 0x94, - 0xc5, 0x11, 0x5a, 0x34, 0x4a, 0xbc, 0xb7, 0xf3, 0xe0, 0x81, 0x28, 0xa0, 0x49, 0xc8, 0xb2, 0x67, - 0xfa, 0xad, 0x0c, 0x31, 0x71, 0x73, 0x13, 0xb2, 0x81, 0xcb, 0x34, 0xc9, 0xeb, 0xb6, 0x77, 0xaa, - 0x15, 0xb9, 0xb6, 0xb6, 0x59, 0xae, 0xd6, 0x96, 0x37, 0xb7, 0x99, 0x0c, 0x4a, 0x5b, 0x5e, 0x79, - 0x28, 0xd5, 0x44, 0xc1, 0x7b, 0xae, 0x3d, 0xdc, 0x29, 0x55, 0xdc, 0x6a, 0x14, 0x53, 0xe9, 0xa4, - 0x98, 0xbc, 0xf9, 0x3d, 0x01, 0x2e, 0xf4, 0xb8, 0x54, 0x02, 0x65, 0x61, 0x7c, 0xc7, 0xa0, 0xb7, - 0x09, 0x8a, 0x23, 0x68, 0x22, 0x70, 0xaf, 0x84, 0x28, 0xa0, 0x34, 0x3b, 0xd3, 0x2f, 0x26, 0xd0, - 0x18, 0x24, 0xaa, 0x77, 0xc4, 0x24, 0x29, 0x69, 0xe0, 0x5a, 0x06, 0x31, 0x85, 0x32, 0xfc, 0x54, - 0xb9, 0x38, 0x8a, 0x72, 0xfe, 0xb1, 0x6e, 0x71, 0x8c, 0x88, 0xf2, 0x0e, 0x46, 0x8b, 0xe3, 0x37, - 0x2f, 0x43, 0xe0, 0x90, 0x29, 0x02, 0x18, 0x7b, 0xa0, 0x38, 0xd8, 0x76, 0xc4, 0x11, 0x34, 0x0e, - 0xc9, 0xe5, 0x46, 0x43, 0x14, 0x6e, 0xff, 0xdd, 0x24, 0xa4, 0xdd, 0xaf, 0x41, 0xa0, 0x07, 0x30, - 0xca, 0x56, 0xc3, 0xe6, 0x7b, 0x7b, 0x6a, 0xb4, 0x6f, 0xcf, 0x2e, 0x0c, 0x72, 0xe5, 0x8a, 0x23, - 0xe8, 0xaf, 0x43, 0x36, 0x30, 0x17, 0xa2, 0x9e, 0x88, 0x3e, 0x34, 0xff, 0xcf, 0x5e, 0x1b, 0x94, - 0xcd, 0x93, 0xff, 0x1e, 0x64, 0xbc, 0xb1, 0x89, 0x5e, 0xea, 0x37, 0x72, 0x5d, 0xd9, 0xfd, 0x87, - 0x37, 0x19, 0x7c, 0xc5, 0x91, 0xd7, 0x04, 0x64, 0x01, 0xea, 0x1e, 0x9b, 0x28, 0x6a, 0x93, 0xb4, - 0xe7, 0xe0, 0x9f, 0xbd, 0x39, 0x54, 0x6e, 0xef, 0x9d, 0x2b, 0x2f, 0x7f, 0xfa, 0xdb, 0xb9, 0x91, - 0x4f, 0x9f, 0xcd, 0x09, 0xbf, 0x7a, 0x36, 0x27, 0xfc, 0xe6, 0xd9, 0x9c, 0xf0, 0xbf, 0x9f, 0xcd, - 0x09, 0xff, 0xe0, 0x77, 0x73, 0x23, 0xbf, 0xfa, 0xdd, 0xdc, 0xc8, 0x6f, 0x7e, 0x37, 0x37, 0xf2, - 0xc1, 0x38, 0x17, 0xb3, 0x3b, 0x46, 0xbf, 0x6e, 0x7c, 0xe7, 0x2f, 0x02, 0x00, 0x00, 0xff, 0xff, - 0x86, 0x8a, 0x7d, 0x48, 0x00, 0x7a, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_f754b08a08b16d03) } + +var fileDescriptor_api_f754b08a08b16d03 = []byte{ + // 7706 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x7d, 0x68, 0x24, 0x49, + 0x96, 0x9f, 0xb2, 0xaa, 0x54, 0xaa, 0x7a, 0x55, 0x2a, 0xa5, 0x42, 0xea, 0xee, 0x6a, 0xf5, 0x8c, + 0xa4, 0xae, 0xe9, 0xaf, 0xe9, 0x9d, 0x91, 0xa6, 0xbb, 0x67, 0x3c, 0x73, 0xd3, 0x73, 0x73, 0x27, + 0x95, 0xaa, 0xbb, 0x24, 0xb5, 0xd4, 0xea, 0xac, 0x52, 0xcf, 0xce, 0xdc, 0x8e, 0xf3, 0x52, 0x99, + 0xa1, 0x52, 0xae, 0xaa, 0x32, 0xab, 0x33, 0xb3, 0xf4, 0xd1, 0x60, 0x38, 0x7f, 0x71, 0xe6, 0x30, + 0x83, 0xc1, 0xc6, 0x18, 0x9f, 0x8f, 0x1d, 0x38, 0xc3, 0x19, 0x8e, 0x31, 0xc6, 0xff, 0xd9, 0xac, + 0x3f, 0xfe, 0xb0, 0x61, 0x58, 0xd6, 0xb0, 0x36, 0xd8, 0xbb, 0x18, 0x2c, 0xbc, 0xbd, 0xb0, 0x98, + 0xfd, 0xc3, 0xe0, 0xfd, 0xc3, 0x86, 0x01, 0x1b, 0x13, 0x1f, 0xf9, 0x55, 0x95, 0xf5, 0xa1, 0x9e, + 0x1c, 0x7b, 0x60, 0xff, 0x29, 0x2a, 0x5f, 0xc4, 0x7b, 0x19, 0xf1, 0x22, 0xe2, 0xc5, 0xfb, 0x45, + 0xbc, 0x88, 0x84, 0x69, 0xcb, 0x54, 0xd4, 0x83, 0xf6, 0xde, 0xb2, 0xd2, 0xd6, 0x97, 0xda, 0x96, + 0xe9, 0x98, 0x68, 0x5a, 0x35, 0xd5, 0x43, 0x4a, 0x5e, 0xe2, 0x89, 0x73, 0xb7, 0x0f, 0x8f, 0x96, + 0x0f, 0x8f, 0x6c, 0x6c, 0x1d, 0x61, 0x6b, 0x59, 0x35, 0x0d, 0xb5, 0x63, 0x59, 0xd8, 0x50, 0x4f, + 0x97, 0x9b, 0xa6, 0x7a, 0x48, 0x7f, 0x74, 0xa3, 0xc1, 0xd8, 0xe7, 0x90, 0x2b, 0x51, 0x53, 0x1c, + 0x85, 0xd3, 0x66, 0x5d, 0x1a, 0xb6, 0x2c, 0xd3, 0xb2, 0x39, 0xf5, 0xa2, 0x4b, 0x6d, 0x61, 0x47, + 0x09, 0xe4, 0xbe, 0x62, 0x3b, 0xa6, 0xa5, 0x34, 0xf0, 0x32, 0x36, 0x1a, 0xba, 0x81, 0x49, 0x86, + 0x23, 0x55, 0xe5, 0x89, 0xaf, 0x44, 0x26, 0xde, 0xe3, 0xa9, 0xc5, 0x8e, 0xa3, 0x37, 0x97, 0x0f, + 0x9a, 0xea, 0xb2, 0xa3, 0xb7, 0xb0, 0xed, 0x28, 0xad, 0x36, 0x4f, 0x59, 0xa4, 0x29, 0x8e, 0xa5, + 0xa8, 0xba, 0xd1, 0x58, 0xb6, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, 0xad, 0x18, 0x6e, 0x21, + 0x1b, 0x66, 0xc3, 0xa4, 0x7f, 0x97, 0xc9, 0x3f, 0x46, 0x2d, 0xfd, 0x50, 0x80, 0x49, 0x09, 0x3f, + 0xeb, 0x60, 0xdb, 0xa9, 0x62, 0x45, 0xc3, 0x16, 0xba, 0x0c, 0xc9, 0x43, 0x7c, 0x5a, 0x4c, 0x2e, + 0x0a, 0xb7, 0xf2, 0xab, 0x13, 0x5f, 0x9d, 0x2d, 0x24, 0x37, 0xf1, 0xa9, 0x44, 0x68, 0x68, 0x11, + 0x26, 0xb0, 0xa1, 0xc9, 0x24, 0x39, 0x15, 0x4e, 0x4e, 0x63, 0x43, 0xdb, 0xc4, 0xa7, 0xe8, 0x7b, + 0x90, 0xb1, 0x89, 0x34, 0x43, 0xc5, 0xc5, 0xf1, 0x45, 0xe1, 0xd6, 0xf8, 0xea, 0xef, 0x7e, 0x75, + 0xb6, 0xf0, 0x41, 0x43, 0x77, 0x0e, 0x3a, 0x7b, 0x4b, 0xaa, 0xd9, 0x5a, 0xf6, 0xb4, 0xaf, 0xed, + 0xf9, 0xff, 0x97, 0xdb, 0x87, 0x8d, 0xe5, 0xee, 0x9a, 0x2f, 0xd5, 0x4f, 0x8c, 0x1a, 0x7e, 0x26, + 0x79, 0x12, 0xdf, 0x4f, 0xfd, 0xb7, 0xcf, 0x17, 0x84, 0x8d, 0x54, 0x46, 0x10, 0x13, 0x1b, 0xa9, + 0x4c, 0x42, 0x4c, 0x96, 0xfe, 0x34, 0x09, 0x05, 0x09, 0xdb, 0x6d, 0xd3, 0xb0, 0x31, 0x2f, 0xff, + 0x5b, 0x90, 0x74, 0x4e, 0x0c, 0x5a, 0xfe, 0xdc, 0xdd, 0xf9, 0xa5, 0x9e, 0xd6, 0x5e, 0xaa, 0x5b, + 0x8a, 0x61, 0x2b, 0xaa, 0xa3, 0x9b, 0x86, 0x44, 0xb2, 0xa2, 0xf7, 0x20, 0x67, 0x61, 0xbb, 0xd3, + 0xc2, 0x54, 0x5d, 0xb4, 0x6a, 0xb9, 0xbb, 0x97, 0x22, 0x38, 0x6b, 0x6d, 0xc5, 0x90, 0x80, 0xe5, + 0x25, 0xff, 0xd1, 0x65, 0xc8, 0x18, 0x9d, 0x16, 0x51, 0x88, 0x4d, 0xab, 0x9b, 0x94, 0x26, 0x8c, + 0x4e, 0x6b, 0x13, 0x9f, 0xda, 0xe8, 0xbb, 0x70, 0x51, 0xc3, 0x6d, 0x0b, 0xab, 0x8a, 0x83, 0x35, + 0xd9, 0x52, 0x8c, 0x06, 0x96, 0x75, 0x63, 0xdf, 0xb4, 0x8b, 0xe9, 0xc5, 0xe4, 0xad, 0xdc, 0xdd, + 0x57, 0x22, 0xe4, 0x4b, 0x24, 0xd7, 0xba, 0xb1, 0x6f, 0xae, 0xa6, 0xbe, 0x3c, 0x5b, 0x18, 0x93, + 0x66, 0x7d, 0x09, 0x5e, 0x92, 0x8d, 0x6a, 0x30, 0xc9, 0x8b, 0x6b, 0x61, 0xc5, 0x36, 0x8d, 0xe2, + 0xc4, 0xa2, 0x70, 0xab, 0x70, 0x77, 0x29, 0x4a, 0x60, 0x48, 0x35, 0xe4, 0xb1, 0xd3, 0xc2, 0x12, + 0xe5, 0x92, 0xf2, 0x56, 0xe0, 0x09, 0x5d, 0x81, 0x2c, 0xa9, 0xc9, 0xde, 0xa9, 0x83, 0xed, 0x62, + 0x86, 0x56, 0x85, 0x54, 0x6d, 0x95, 0x3c, 0x97, 0x3e, 0x84, 0x7c, 0x90, 0x15, 0x21, 0x28, 0x48, + 0x95, 0xda, 0xee, 0x56, 0x45, 0xde, 0xdd, 0xde, 0xdc, 0x7e, 0xfc, 0xd1, 0xb6, 0x38, 0x86, 0x66, + 0x41, 0xe4, 0xb4, 0xcd, 0xca, 0xc7, 0xf2, 0xa3, 0xf5, 0xad, 0xf5, 0xba, 0x28, 0xcc, 0xa5, 0xfe, + 0xc6, 0x9f, 0xce, 0x8f, 0x95, 0x9e, 0x02, 0x3c, 0xc4, 0x0e, 0xef, 0x66, 0x68, 0x15, 0xd2, 0x07, + 0xb4, 0x3c, 0x45, 0x81, 0x6a, 0x7a, 0x31, 0xb2, 0xe0, 0x81, 0x2e, 0xb9, 0x9a, 0x21, 0xda, 0xf8, + 0xc9, 0xd9, 0x82, 0x20, 0x71, 0x4e, 0xd6, 0x13, 0x4a, 0xff, 0x4a, 0x80, 0x1c, 0x15, 0xcc, 0x6a, + 0x89, 0xca, 0x5d, 0x92, 0xaf, 0x0e, 0x55, 0x49, 0xaf, 0x68, 0xb4, 0x04, 0xe3, 0x47, 0x4a, 0xb3, + 0x83, 0x8b, 0x09, 0x2a, 0xa3, 0x18, 0x21, 0xe3, 0x29, 0x49, 0x97, 0x58, 0x36, 0x74, 0x1f, 0xf2, + 0xba, 0xe1, 0x60, 0xc3, 0x91, 0x19, 0x5b, 0x72, 0x08, 0x5b, 0x8e, 0xe5, 0xa6, 0x0f, 0xa5, 0x7f, + 0x2e, 0x00, 0xec, 0x74, 0xe2, 0x54, 0x0d, 0x7a, 0x7b, 0xc4, 0xf2, 0xf3, 0x3e, 0xc6, 0x6b, 0x71, + 0x11, 0xd2, 0xba, 0xd1, 0xd4, 0x0d, 0x56, 0xfe, 0x8c, 0xc4, 0x9f, 0xd0, 0x2c, 0x8c, 0xef, 0x35, + 0x75, 0x43, 0xa3, 0xa3, 0x22, 0x23, 0xb1, 0x07, 0xae, 0x7e, 0x09, 0x72, 0xb4, 0xec, 0x31, 0x6a, + 0xbf, 0xf4, 0xe3, 0x04, 0x5c, 0x28, 0x9b, 0x86, 0xa6, 0x93, 0xe1, 0xa9, 0x34, 0xbf, 0x15, 0xba, + 0xd9, 0x80, 0xc0, 0x40, 0x94, 0xf1, 0x49, 0x7b, 0xc4, 0x96, 0x46, 0x3e, 0x57, 0xe5, 0xa4, 0x4d, + 0x69, 0xd1, 0xfa, 0x44, 0x6f, 0xc3, 0x25, 0xa5, 0xd9, 0x34, 0x8f, 0x65, 0x7d, 0x5f, 0xd6, 0x4c, + 0x6c, 0xcb, 0x86, 0xe9, 0xc8, 0xf8, 0x44, 0xb7, 0x1d, 0x6a, 0x56, 0x32, 0xd2, 0x0c, 0x4d, 0x5e, + 0xdf, 0x5f, 0x33, 0xb1, 0xbd, 0x6d, 0x3a, 0x15, 0x92, 0x44, 0xc6, 0x2c, 0x29, 0x0c, 0x1b, 0xb3, + 0x69, 0x62, 0x90, 0xa5, 0x0c, 0x3e, 0x69, 0xd3, 0x31, 0xcb, 0x9b, 0xe8, 0x53, 0xb8, 0xd8, 0xad, + 0xcd, 0x38, 0x5b, 0xeb, 0x3f, 0x08, 0x50, 0x58, 0x37, 0x74, 0xe7, 0x5b, 0xd1, 0x4c, 0x9e, 0x6a, + 0x93, 0x41, 0xd5, 0xde, 0x06, 0x71, 0x5f, 0xd1, 0x9b, 0x8f, 0x8d, 0xba, 0xd9, 0xda, 0xb3, 0x1d, + 0xd3, 0xc0, 0x36, 0xd7, 0x7d, 0x0f, 0x9d, 0xeb, 0xec, 0x29, 0x4c, 0x79, 0x75, 0x8a, 0x53, 0x59, + 0xcf, 0x41, 0x5c, 0x37, 0x54, 0x0b, 0xb7, 0xb0, 0x11, 0xab, 0xb6, 0x5e, 0x81, 0xac, 0xee, 0xca, + 0xa5, 0x1a, 0x4b, 0x4a, 0x3e, 0x81, 0xd7, 0xa9, 0x03, 0xd3, 0x81, 0x77, 0xc7, 0x69, 0x2e, 0xc9, + 0xc4, 0x81, 0x8f, 0x65, 0xbf, 0xbd, 0xc8, 0xc4, 0x81, 0x8f, 0x99, 0x79, 0xfb, 0x18, 0x26, 0xd7, + 0x70, 0x13, 0x3b, 0x38, 0x7e, 0xdb, 0xbf, 0x0b, 0x05, 0x57, 0x74, 0x9c, 0x8d, 0xf4, 0x27, 0x02, + 0x20, 0x2e, 0x97, 0xcc, 0xb8, 0x71, 0xb6, 0xd3, 0x02, 0x71, 0x33, 0x9c, 0x8e, 0x65, 0x30, 0x7f, + 0x81, 0xf5, 0x52, 0x60, 0x24, 0xea, 0x32, 0xf8, 0x36, 0x38, 0x15, 0xb4, 0xc1, 0x9e, 0xdb, 0x43, + 0x1c, 0x9e, 0x63, 0x98, 0x09, 0x15, 0x2f, 0xde, 0xa6, 0x4c, 0xd1, 0x92, 0x25, 0x16, 0x93, 0x41, + 0xdf, 0x8e, 0x12, 0x4b, 0x9f, 0xc2, 0x74, 0xb9, 0x89, 0x15, 0x2b, 0x6e, 0xb5, 0xf0, 0xe6, 0xfc, + 0x18, 0x50, 0x50, 0x7c, 0x9c, 0x4d, 0xfa, 0x0f, 0x05, 0x40, 0x12, 0x3e, 0xc2, 0x96, 0x13, 0x7b, + 0x93, 0xae, 0x41, 0xce, 0x51, 0xac, 0x06, 0x76, 0x64, 0xe2, 0x8f, 0x73, 0x73, 0xf5, 0x6a, 0x40, + 0x10, 0xf1, 0xca, 0x97, 0x0e, 0x9a, 0xea, 0x52, 0xdd, 0xf5, 0xd7, 0xb9, 0xcd, 0x02, 0xc6, 0x47, + 0xc8, 0x5c, 0x03, 0x9f, 0xc0, 0x4c, 0xa8, 0x94, 0x71, 0xaa, 0xe0, 0x7f, 0x0a, 0x90, 0xab, 0xa9, + 0x8a, 0x11, 0x67, 0xdd, 0x3f, 0x84, 0x9c, 0xad, 0x2a, 0x86, 0xbc, 0x6f, 0x5a, 0x2d, 0xc5, 0xa1, + 0x5d, 0xb6, 0x10, 0xaa, 0xbb, 0xe7, 0x35, 0xab, 0x8a, 0xf1, 0x80, 0x66, 0x92, 0xc0, 0xf6, 0xfe, + 0xa3, 0x27, 0x90, 0x3b, 0xc4, 0xa7, 0x32, 0x47, 0x57, 0x74, 0x9e, 0x2b, 0xdc, 0x7d, 0x2b, 0xc0, + 0x7f, 0x78, 0xb4, 0xe4, 0x82, 0xb2, 0xa5, 0x00, 0x28, 0x5b, 0x22, 0x1c, 0x4b, 0x35, 0xc7, 0xc2, + 0x46, 0xc3, 0x39, 0x90, 0xe0, 0x10, 0x9f, 0x3e, 0x62, 0x32, 0x82, 0x03, 0x65, 0x23, 0x95, 0x49, + 0x8a, 0xa9, 0xd2, 0xff, 0x12, 0x20, 0xcf, 0x2a, 0x1e, 0xe7, 0x40, 0x79, 0x07, 0x52, 0x96, 0x79, + 0xcc, 0x06, 0x4a, 0xee, 0xee, 0x95, 0x08, 0x11, 0x9b, 0xf8, 0x34, 0x38, 0x43, 0xd1, 0xec, 0x68, + 0x15, 0xb8, 0xef, 0x27, 0x53, 0xee, 0xe4, 0xa8, 0xdc, 0xc0, 0xb8, 0x24, 0x22, 0xe3, 0x26, 0x4c, + 0xed, 0x29, 0x8e, 0x7a, 0x20, 0x5b, 0xbc, 0x90, 0x64, 0x36, 0x4b, 0xde, 0xca, 0x4b, 0x05, 0x4a, + 0x76, 0x8b, 0x6e, 0x97, 0xfe, 0xb7, 0xdb, 0xeb, 0x6d, 0xfc, 0x1b, 0xd9, 0xf2, 0xff, 0x47, 0xe0, + 0xe3, 0xc9, 0xad, 0xff, 0x6f, 0x5a, 0x07, 0xf8, 0x41, 0x02, 0x2e, 0x95, 0x0f, 0xb0, 0x7a, 0x58, + 0x36, 0x0d, 0x5b, 0xb7, 0x1d, 0xa2, 0xc1, 0x38, 0x7b, 0xc1, 0x15, 0xc8, 0x1e, 0xeb, 0xce, 0x81, + 0xac, 0xe9, 0xfb, 0xfb, 0xd4, 0xf2, 0x65, 0xa4, 0x0c, 0x21, 0xac, 0xe9, 0xfb, 0xfb, 0xe8, 0x1e, + 0xa4, 0x5a, 0xa6, 0xc6, 0x5c, 0xe4, 0xc2, 0xdd, 0x85, 0x08, 0xf1, 0xb4, 0x68, 0x76, 0xa7, 0xb5, + 0x65, 0x6a, 0x58, 0xa2, 0x99, 0xd1, 0x3c, 0x80, 0x4a, 0xa8, 0x6d, 0x53, 0x37, 0x1c, 0x3e, 0x07, + 0x06, 0x28, 0xa8, 0x0a, 0x59, 0x07, 0x5b, 0x2d, 0xdd, 0x50, 0x1c, 0x5c, 0x1c, 0xa7, 0xca, 0xbb, + 0x16, 0x59, 0xf0, 0x76, 0x53, 0x57, 0x95, 0x35, 0x6c, 0xab, 0x96, 0xde, 0x76, 0x4c, 0x8b, 0x6b, + 0xd1, 0x67, 0xe6, 0x16, 0xf7, 0xb3, 0x14, 0x14, 0x7b, 0x35, 0x14, 0x67, 0x3f, 0xd9, 0x81, 0x34, + 0x41, 0xd9, 0x4d, 0x87, 0xf7, 0x94, 0xbb, 0xfd, 0x14, 0x11, 0x51, 0x02, 0x8a, 0xd6, 0x9b, 0x0e, + 0x2f, 0x3c, 0x97, 0x33, 0xf7, 0x43, 0x01, 0xd2, 0x2c, 0x01, 0xdd, 0x81, 0x0c, 0x5f, 0x56, 0xd0, + 0x68, 0x19, 0x93, 0xab, 0x17, 0x5f, 0x9c, 0x2d, 0x4c, 0xb0, 0x95, 0x82, 0xb5, 0xaf, 0xfc, 0xbf, + 0xd2, 0x04, 0xcd, 0xb7, 0xae, 0x91, 0x36, 0xb3, 0x1d, 0xc5, 0x72, 0xe8, 0x12, 0x4e, 0x82, 0x21, + 0x06, 0x4a, 0xd8, 0xc4, 0xa7, 0x68, 0x03, 0xd2, 0xb6, 0xa3, 0x38, 0x1d, 0x9b, 0xb7, 0xda, 0xb9, + 0x0a, 0x5b, 0xa3, 0x9c, 0x12, 0x97, 0x40, 0x5c, 0x19, 0x0d, 0x3b, 0x8a, 0xde, 0xa4, 0xcd, 0x98, + 0x95, 0xf8, 0x53, 0xe9, 0x8f, 0x05, 0x48, 0xb3, 0xac, 0xe8, 0x12, 0xcc, 0x48, 0x2b, 0xdb, 0x0f, + 0x2b, 0xf2, 0xfa, 0xf6, 0x5a, 0xa5, 0x5e, 0x91, 0xb6, 0xd6, 0xb7, 0x57, 0xea, 0x15, 0x71, 0x0c, + 0x5d, 0x04, 0xe4, 0x26, 0x94, 0x1f, 0x6f, 0xd7, 0xd6, 0x6b, 0xf5, 0xca, 0x76, 0x5d, 0x14, 0xe8, + 0x0a, 0x03, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x0d, 0x16, 0xbb, 0xa9, 0x72, 0xad, 0xbe, 0x52, 0xaf, + 0xc9, 0x95, 0x5a, 0x7d, 0x7d, 0x6b, 0xa5, 0x5e, 0x59, 0x13, 0x93, 0x03, 0x72, 0x91, 0x97, 0x48, + 0x52, 0xa5, 0x5c, 0x17, 0x53, 0xa5, 0xe7, 0x70, 0x41, 0xc2, 0xaa, 0xd9, 0x6a, 0x77, 0x1c, 0x4c, + 0x4a, 0x69, 0xc7, 0x39, 0x5e, 0x2e, 0xc1, 0x84, 0x66, 0x9d, 0xca, 0x56, 0xc7, 0xe0, 0xa3, 0x25, + 0xad, 0x59, 0xa7, 0x52, 0xc7, 0xe0, 0x9d, 0xf1, 0x9f, 0x08, 0x70, 0xb1, 0xfb, 0xe5, 0x71, 0x76, + 0xc5, 0x27, 0x90, 0x53, 0x34, 0x0d, 0x6b, 0xb2, 0x86, 0x9b, 0x8e, 0xc2, 0x5d, 0x95, 0xdb, 0x01, + 0x49, 0x7c, 0xf9, 0x6d, 0xc9, 0x5b, 0x7e, 0xdb, 0x7a, 0x5a, 0x2e, 0xd3, 0x82, 0xac, 0x11, 0x0e, + 0xd7, 0x14, 0x51, 0x21, 0x94, 0x52, 0xfa, 0xa7, 0x29, 0x98, 0xac, 0x18, 0x5a, 0xfd, 0x24, 0xd6, + 0xd9, 0xe5, 0x22, 0xa4, 0x55, 0xb3, 0xd5, 0xd2, 0x1d, 0x57, 0x4d, 0xec, 0x09, 0xfd, 0x16, 0x64, + 0x34, 0xac, 0x68, 0xde, 0x1a, 0xc5, 0x30, 0x47, 0x4b, 0xf2, 0xb2, 0xa3, 0xdf, 0x87, 0x4b, 0xc4, + 0x82, 0x5a, 0x86, 0xd2, 0x94, 0x99, 0x34, 0xd9, 0xb1, 0xf4, 0x46, 0x03, 0x5b, 0x7c, 0xb1, 0xef, + 0x56, 0x44, 0x39, 0xd7, 0x39, 0x47, 0x99, 0x32, 0xd4, 0x59, 0x7e, 0xe9, 0x82, 0x1e, 0x45, 0x46, + 0x1f, 0x00, 0x90, 0xc9, 0x89, 0x2e, 0x20, 0xda, 0xdc, 0x36, 0xf5, 0x5b, 0x41, 0x74, 0xcd, 0x11, + 0x61, 0x20, 0xcf, 0x36, 0x5a, 0x26, 0xc8, 0xe0, 0x59, 0x47, 0xb7, 0xb0, 0x7c, 0xa7, 0xad, 0x52, + 0x28, 0x9f, 0x59, 0x2d, 0xbc, 0x38, 0x5b, 0x00, 0x89, 0x91, 0xef, 0xec, 0x94, 0x09, 0x52, 0x60, + 0xff, 0xdb, 0x2a, 0x5a, 0x85, 0x79, 0x32, 0x01, 0xf3, 0xba, 0x28, 0x8e, 0x7c, 0xa0, 0x37, 0x0e, + 0xb0, 0x25, 0x7b, 0xab, 0xc2, 0x74, 0x09, 0x2f, 0x23, 0xcd, 0xa9, 0x8a, 0xc1, 0x0a, 0xba, 0xe2, + 0x54, 0x69, 0x16, 0x4f, 0x3d, 0x44, 0xcf, 0x6d, 0x53, 0xb7, 0x4d, 0xa3, 0x98, 0x65, 0x7a, 0x66, + 0x4f, 0xe8, 0x09, 0x88, 0xba, 0x21, 0xef, 0x37, 0xf5, 0xc6, 0x81, 0x23, 0x1f, 0x5b, 0xba, 0x83, + 0xed, 0xe2, 0x34, 0xad, 0x50, 0x54, 0xbf, 0xab, 0xf1, 0xb5, 0x59, 0xed, 0x23, 0x92, 0x93, 0x57, + 0xad, 0xa0, 0x1b, 0x0f, 0x28, 0x3f, 0x25, 0xda, 0xde, 0xec, 0x3c, 0x21, 0x66, 0x4a, 0xff, 0x45, + 0x80, 0x82, 0xdb, 0x69, 0xe2, 0xec, 0xdf, 0xb7, 0x40, 0x34, 0x0d, 0x2c, 0xb7, 0x0f, 0x14, 0x1b, + 0x73, 0xc5, 0xf0, 0x29, 0xa4, 0x60, 0x1a, 0x78, 0x87, 0x90, 0x99, 0x26, 0xd0, 0x0e, 0x4c, 0xdb, + 0x8e, 0xd2, 0xd0, 0x8d, 0x46, 0x40, 0x5f, 0xe3, 0xa3, 0xbb, 0xee, 0x22, 0xe7, 0xf6, 0xe8, 0x21, + 0xbf, 0xe3, 0xa7, 0x02, 0x4c, 0xaf, 0x68, 0x2d, 0xdd, 0xa8, 0xb5, 0x9b, 0x7a, 0xac, 0x38, 0xff, + 0x1a, 0x64, 0x6d, 0x22, 0xd3, 0x37, 0xde, 0x3e, 0x46, 0xcb, 0xd0, 0x14, 0x62, 0xc5, 0x1f, 0xc1, + 0x14, 0x3e, 0x69, 0xeb, 0x96, 0xe2, 0xe8, 0xa6, 0xc1, 0x60, 0x49, 0x6a, 0xf4, 0xba, 0x15, 0x7c, + 0x5e, 0x1f, 0x9a, 0xf0, 0x9a, 0x7d, 0x0c, 0x28, 0x58, 0xb1, 0x38, 0xf1, 0x89, 0x0c, 0x33, 0x54, + 0xf4, 0xae, 0x61, 0xc7, 0xac, 0x35, 0x6e, 0x5d, 0x7f, 0x0f, 0x66, 0xc3, 0x2f, 0x88, 0xb3, 0xf4, + 0x9f, 0xf2, 0x16, 0xdf, 0xc2, 0xd6, 0x37, 0x04, 0x8d, 0x83, 0xe2, 0xe3, 0x2c, 0xf9, 0x1f, 0x09, + 0x70, 0x99, 0xca, 0xa6, 0x7b, 0x22, 0xfb, 0xd8, 0x7a, 0x84, 0x15, 0x3b, 0x56, 0x84, 0xfc, 0x1a, + 0xa4, 0x19, 0xd2, 0xa5, 0x3d, 0x76, 0x7c, 0x35, 0x47, 0xfc, 0x92, 0x9a, 0x63, 0x5a, 0xc4, 0x2f, + 0xe1, 0x49, 0xbc, 0x9e, 0x0a, 0xcc, 0x45, 0x95, 0x25, 0xe6, 0xa5, 0x80, 0x69, 0xee, 0x1e, 0x92, + 0x2e, 0x5e, 0x3e, 0x20, 0x7e, 0x11, 0xaa, 0x40, 0x4e, 0xa5, 0xff, 0x64, 0xe7, 0xb4, 0x8d, 0xa9, + 0xfc, 0xc2, 0x20, 0xcf, 0x92, 0xb1, 0xd5, 0x4f, 0xdb, 0x98, 0xb8, 0xa7, 0xee, 0x7f, 0xa2, 0xae, + 0x40, 0x55, 0x07, 0xfa, 0xa6, 0x74, 0x7c, 0xd1, 0xbc, 0xae, 0x7b, 0x17, 0xd2, 0xc4, 0x3f, 0x4b, + 0x72, 0x55, 0xb0, 0x37, 0x71, 0xa6, 0x58, 0xbd, 0x91, 0x4f, 0x42, 0xdb, 0x53, 0xc1, 0xea, 0x27, + 0xce, 0x51, 0xfd, 0xc0, 0xba, 0xb8, 0x4f, 0x45, 0x1f, 0x43, 0x60, 0xe5, 0x5b, 0x66, 0x35, 0x73, + 0xd1, 0xce, 0x79, 0x94, 0x32, 0xed, 0x4b, 0x61, 0x74, 0x1b, 0x95, 0x21, 0x83, 0x4f, 0xda, 0xb2, + 0x86, 0x6d, 0x95, 0x9b, 0xb5, 0x52, 0xbf, 0x7d, 0xb4, 0x1e, 0xff, 0x7f, 0x02, 0x9f, 0xb4, 0x09, + 0x11, 0xed, 0x92, 0x19, 0xce, 0x75, 0x07, 0x68, 0xb1, 0xed, 0xe1, 0x70, 0xc2, 0xef, 0x2f, 0x5c, + 0xdc, 0x94, 0xe7, 0x09, 0x30, 0x11, 0xbc, 0xed, 0x3e, 0x17, 0xe0, 0x4a, 0x64, 0xdb, 0xc5, 0x39, + 0xd9, 0x7d, 0x00, 0x29, 0xaa, 0x82, 0xc4, 0x39, 0x55, 0x40, 0xb9, 0x4a, 0x7f, 0xee, 0x8e, 0x7a, + 0x09, 0x37, 0x4d, 0xa2, 0xde, 0x6f, 0x60, 0x5d, 0x6c, 0xc2, 0x6d, 0xf6, 0xc4, 0xb9, 0x9b, 0xdd, + 0x65, 0xed, 0x32, 0x0b, 0x5d, 0x85, 0x8d, 0xd3, 0x2c, 0xfc, 0x5d, 0x01, 0x66, 0xaa, 0x58, 0xb1, + 0x9c, 0x3d, 0xac, 0x38, 0x31, 0xbb, 0xb3, 0xef, 0x40, 0xd2, 0x30, 0x8f, 0xcf, 0xb3, 0x34, 0x48, + 0xf2, 0xfb, 0xd3, 0x56, 0xb8, 0x5c, 0x71, 0xd6, 0xfa, 0xd7, 0x09, 0xc8, 0x3e, 0x2c, 0xc7, 0x59, + 0xd7, 0x0f, 0xf8, 0x02, 0x32, 0x1b, 0xea, 0x51, 0xdd, 0xd2, 0x7b, 0xdf, 0xd2, 0xc3, 0xf2, 0x26, + 0x3e, 0x75, 0xbb, 0x25, 0xe1, 0x42, 0x2b, 0x90, 0x75, 0x0e, 0x2c, 0x6c, 0x1f, 0x98, 0x4d, 0xed, + 0x3c, 0x3e, 0x8b, 0xcf, 0x35, 0xf7, 0xb7, 0x05, 0x18, 0xa7, 0x82, 0xdd, 0x28, 0x06, 0x21, 0x22, + 0x8a, 0x81, 0xbc, 0xc7, 0xf3, 0xfb, 0x12, 0xe7, 0x79, 0x8f, 0xe7, 0x3b, 0xdf, 0x80, 0xa9, 0x0e, + 0x71, 0x33, 0x9b, 0x58, 0xb1, 0xd8, 0xde, 0x3e, 0x5f, 0xce, 0x9f, 0xec, 0xd8, 0xd8, 0x5f, 0xc7, + 0x66, 0xad, 0xe8, 0x39, 0x51, 0xe3, 0x62, 0xba, 0xf4, 0x04, 0x80, 0xe8, 0x20, 0xce, 0x76, 0xfc, + 0x9b, 0x49, 0x28, 0xec, 0x74, 0xec, 0x83, 0x98, 0x3b, 0x6e, 0x19, 0xa0, 0xdd, 0xb1, 0x29, 0xaa, + 0x38, 0x31, 0xb8, 0x9e, 0x86, 0x84, 0x53, 0xb8, 0x8a, 0x62, 0x7c, 0xf5, 0x13, 0x03, 0x55, 0xb9, + 0x10, 0x2c, 0xfb, 0x31, 0x19, 0xaf, 0x0d, 0x02, 0x9d, 0xf5, 0x13, 0x63, 0x0b, 0x7b, 0x68, 0x93, + 0x49, 0xc2, 0x44, 0xd2, 0x07, 0x30, 0x41, 0x1e, 0x64, 0xc7, 0x3c, 0x4f, 0xdf, 0x48, 0x13, 0x9e, + 0xba, 0x89, 0xee, 0x43, 0x96, 0x71, 0x93, 0x19, 0x2e, 0x4d, 0x67, 0xb8, 0xa8, 0xba, 0x70, 0x35, + 0xd2, 0xb9, 0x2d, 0x43, 0x59, 0xc9, 0x7c, 0x36, 0x0b, 0xe3, 0xfb, 0xa6, 0xa5, 0x62, 0x1a, 0x68, + 0x91, 0x91, 0xd8, 0x43, 0xb0, 0x55, 0x37, 0x52, 0x99, 0x8c, 0x98, 0xdd, 0x48, 0x65, 0xb2, 0x22, + 0x94, 0xfe, 0x58, 0x80, 0x29, 0xaf, 0x39, 0xe2, 0x34, 0xfa, 0xe5, 0x90, 0x2e, 0xcf, 0xdf, 0x20, + 0x44, 0x8d, 0xa5, 0x7f, 0x47, 0x3d, 0x20, 0xd5, 0x3c, 0xa2, 0xed, 0x13, 0x67, 0x7f, 0xb9, 0xcf, + 0xe2, 0x6e, 0x12, 0xe7, 0x6d, 0x63, 0x1a, 0x82, 0x73, 0x07, 0x66, 0xf5, 0x16, 0x99, 0x0e, 0x74, + 0xa7, 0x79, 0xca, 0xe1, 0x9b, 0x83, 0xdd, 0xad, 0xdc, 0x19, 0x3f, 0xad, 0xec, 0x26, 0x71, 0x0b, + 0xc9, 0x36, 0x77, 0xfc, 0xfa, 0xc4, 0xa9, 0xf0, 0x75, 0x98, 0xb4, 0x98, 0x68, 0xe2, 0xc6, 0x9c, + 0x53, 0xe7, 0x79, 0x8f, 0x95, 0xa8, 0xfd, 0xcf, 0x12, 0x30, 0xf5, 0xa4, 0x83, 0xad, 0xd3, 0x6f, + 0x93, 0xd2, 0x6f, 0xc0, 0xd4, 0xb1, 0xa2, 0x3b, 0xf2, 0xbe, 0x69, 0xc9, 0x9d, 0xb6, 0xa6, 0x38, + 0x9e, 0x15, 0x23, 0xe4, 0x07, 0xa6, 0xb5, 0x4b, 0x89, 0x08, 0x03, 0x3a, 0x34, 0xcc, 0x63, 0x43, + 0x26, 0x64, 0x0a, 0x9b, 0x4f, 0x0c, 0xbe, 0xea, 0xbc, 0xfa, 0xee, 0x7f, 0x3e, 0x5b, 0xb8, 0x37, + 0x52, 0x78, 0x17, 0x0d, 0x50, 0xeb, 0x74, 0x74, 0x6d, 0x69, 0x77, 0x77, 0x7d, 0x4d, 0x12, 0xa9, + 0xc8, 0x8f, 0x98, 0xc4, 0xfa, 0x89, 0xe1, 0x4e, 0xf7, 0x5f, 0x09, 0x20, 0xfa, 0x9a, 0x8a, 0xb3, + 0x39, 0x2b, 0x90, 0x7b, 0xd6, 0xc1, 0x96, 0xfe, 0x12, 0x8d, 0x09, 0x9c, 0x91, 0x18, 0xa2, 0x4f, + 0x20, 0x1f, 0xd2, 0x43, 0xf2, 0xeb, 0xe9, 0x21, 0x77, 0xec, 0xab, 0xa0, 0xf4, 0x6f, 0x05, 0x40, + 0xb4, 0xf2, 0xeb, 0x6c, 0xc1, 0xff, 0xdb, 0xd2, 0x53, 0x6e, 0x81, 0x48, 0x43, 0x1b, 0x65, 0x7d, + 0x5f, 0x6e, 0xe9, 0xb6, 0xad, 0x1b, 0x0d, 0xde, 0x55, 0x0a, 0x94, 0xbe, 0xbe, 0xbf, 0xc5, 0xa8, + 0xbc, 0x11, 0xff, 0x12, 0xcc, 0x84, 0xaa, 0x11, 0x67, 0x33, 0x5e, 0x85, 0xfc, 0xbe, 0xd9, 0x31, + 0x34, 0x99, 0x6d, 0x8a, 0xf0, 0x55, 0xc2, 0x1c, 0xa5, 0xb1, 0xf7, 0x95, 0xfe, 0x47, 0x02, 0x66, + 0x25, 0x6c, 0x9b, 0xcd, 0x23, 0x1c, 0xbf, 0x22, 0xab, 0xc0, 0xb7, 0x63, 0xe4, 0x97, 0xd2, 0x67, + 0x96, 0x31, 0xb3, 0x29, 0x2d, 0xbc, 0xe0, 0x7e, 0x6d, 0x70, 0x5f, 0xec, 0x5d, 0x62, 0xe7, 0xeb, + 0x77, 0xa9, 0xd0, 0xfa, 0x9d, 0x09, 0x53, 0x7a, 0xc3, 0x30, 0x89, 0xcd, 0xb2, 0xf1, 0x33, 0xa3, + 0xd3, 0x72, 0xc1, 0xcd, 0xd2, 0xa0, 0x42, 0xae, 0x33, 0x96, 0x1a, 0x7e, 0xb6, 0xdd, 0x69, 0x51, + 0xe7, 0x65, 0xf5, 0x22, 0x29, 0xef, 0x8b, 0xb3, 0x85, 0x42, 0x28, 0xcd, 0x96, 0x0a, 0xba, 0xf7, + 0x4c, 0xa4, 0xf3, 0x26, 0xff, 0x1e, 0x5c, 0xe8, 0x52, 0x79, 0x9c, 0x3e, 0xce, 0xbf, 0x4e, 0xc2, + 0xe5, 0xb0, 0xf8, 0xb8, 0x21, 0xcb, 0xb7, 0xbd, 0x59, 0xab, 0x30, 0xd9, 0xd2, 0x8d, 0x97, 0x5b, + 0xb1, 0xcc, 0xb7, 0x74, 0xc3, 0x5f, 0xf8, 0x8d, 0xe8, 0x20, 0xe9, 0xff, 0x07, 0x1d, 0x44, 0x81, + 0xb9, 0xa8, 0x16, 0x8c, 0xb3, 0x97, 0x7c, 0x26, 0x40, 0x3e, 0xee, 0x45, 0xb8, 0x97, 0x0b, 0x46, + 0xe3, 0x75, 0xae, 0xc3, 0xe4, 0x37, 0xb0, 0x6a, 0xf7, 0x67, 0x02, 0xa0, 0xba, 0xd5, 0x31, 0x08, + 0x1a, 0x7e, 0x64, 0x36, 0xe2, 0xac, 0xec, 0x2c, 0x8c, 0xeb, 0x86, 0x86, 0x4f, 0x68, 0x65, 0x53, + 0x12, 0x7b, 0x08, 0xed, 0x34, 0x26, 0x47, 0xda, 0x69, 0xf4, 0x63, 0x5a, 0x42, 0x05, 0x8d, 0x53, + 0x0b, 0xff, 0x38, 0x01, 0x33, 0xbc, 0x3a, 0xb1, 0xaf, 0x5a, 0xbe, 0x0d, 0xe3, 0x4d, 0x22, 0x73, + 0x40, 0x9b, 0xd3, 0x77, 0xba, 0x6d, 0x4e, 0x33, 0xa3, 0xdf, 0x06, 0x68, 0x5b, 0xf8, 0x48, 0x66, + 0xac, 0xc9, 0x91, 0x58, 0xb3, 0x84, 0x83, 0x12, 0xd0, 0x77, 0x61, 0x8a, 0x8c, 0xf0, 0xb6, 0x65, + 0xb6, 0x4d, 0x9b, 0x38, 0x29, 0xf6, 0x68, 0x48, 0x67, 0xfa, 0xc5, 0xd9, 0xc2, 0xe4, 0x96, 0x6e, + 0xec, 0x70, 0xc6, 0x7a, 0x4d, 0x22, 0xa6, 0xc2, 0x7b, 0x74, 0x07, 0xe0, 0x7f, 0x14, 0x60, 0xf6, + 0x1b, 0x5b, 0xe7, 0xfd, 0xff, 0xa1, 0x31, 0x6f, 0xe6, 0x11, 0xe9, 0xe3, 0xba, 0xb1, 0x6f, 0xc6, + 0xbf, 0xfa, 0xfe, 0x99, 0x00, 0xd3, 0x01, 0xf1, 0x71, 0x7a, 0x32, 0x2f, 0xa5, 0xb3, 0xd2, 0xef, + 0x11, 0xdf, 0x26, 0xd8, 0xed, 0xe3, 0x1c, 0x54, 0xff, 0x22, 0x01, 0x17, 0xcb, 0x6c, 0x0f, 0xda, + 0x0d, 0xd0, 0x88, 0xb3, 0x97, 0x14, 0x61, 0xe2, 0x08, 0x5b, 0xb6, 0x6e, 0xb2, 0x19, 0x76, 0x52, + 0x72, 0x1f, 0xd1, 0x1c, 0x64, 0x6c, 0x43, 0x69, 0xdb, 0x07, 0xa6, 0xbb, 0x6d, 0xe7, 0x3d, 0x7b, + 0xc1, 0x24, 0xe3, 0x2f, 0x1f, 0x4c, 0x92, 0x1e, 0x1c, 0x4c, 0x32, 0xf1, 0xb5, 0x83, 0x49, 0xf8, + 0x1e, 0xd9, 0x8f, 0x04, 0xb8, 0xd4, 0xa3, 0xbf, 0x38, 0xfb, 0xcc, 0xf7, 0x21, 0xa7, 0x72, 0xc1, + 0xc4, 0x1a, 0xb3, 0x6d, 0xc0, 0x75, 0x92, 0xed, 0x25, 0x01, 0xc8, 0x8b, 0xb3, 0x05, 0x70, 0x8b, + 0xba, 0xbe, 0xc6, 0x55, 0x44, 0xfe, 0x6b, 0xa5, 0xff, 0x94, 0x87, 0xa9, 0xca, 0x09, 0x5b, 0xe4, + 0xae, 0x31, 0x7f, 0x00, 0x3d, 0x80, 0x4c, 0xdb, 0x32, 0x8f, 0x74, 0xb7, 0x1a, 0x85, 0x50, 0x0c, + 0x81, 0x5b, 0x8d, 0x2e, 0xae, 0x1d, 0xce, 0x21, 0x79, 0xbc, 0xa8, 0x0e, 0xd9, 0x47, 0xa6, 0xaa, + 0x34, 0x1f, 0xe8, 0x4d, 0xb7, 0xff, 0xbf, 0x35, 0x5c, 0xd0, 0x92, 0xc7, 0xb3, 0xa3, 0x38, 0x07, + 0x6e, 0x53, 0x78, 0x44, 0xb4, 0x0e, 0x99, 0xaa, 0xe3, 0xb4, 0x49, 0x22, 0xb7, 0x26, 0x37, 0x47, + 0x10, 0x4a, 0x58, 0xb8, 0x2c, 0x8f, 0x1d, 0xd5, 0x61, 0xfa, 0xa1, 0x69, 0x36, 0x9a, 0xb8, 0xdc, + 0x34, 0x3b, 0x5a, 0xd9, 0x34, 0xf6, 0xf5, 0x06, 0xb7, 0xc7, 0x37, 0x46, 0x90, 0xf9, 0xb0, 0x5c, + 0x93, 0x7a, 0x05, 0xa0, 0x15, 0xc8, 0xd4, 0xee, 0x71, 0x61, 0xcc, 0x81, 0xbb, 0x3e, 0x82, 0xb0, + 0xda, 0x3d, 0xc9, 0x63, 0x43, 0x1b, 0x90, 0x5b, 0x79, 0xde, 0xb1, 0x30, 0x97, 0x92, 0xee, 0x1b, + 0xc0, 0xd0, 0x2d, 0x85, 0x72, 0x49, 0x41, 0x66, 0x54, 0x83, 0xc2, 0x47, 0xa6, 0x75, 0xd8, 0x34, + 0x15, 0xb7, 0x86, 0x13, 0x54, 0xdc, 0x77, 0x46, 0x10, 0xe7, 0x32, 0x4a, 0x5d, 0x22, 0xd0, 0xf7, + 0x60, 0x8a, 0x34, 0x46, 0x5d, 0xd9, 0x6b, 0xba, 0x85, 0xcc, 0x50, 0xa9, 0x6f, 0x8c, 0x20, 0xd5, + 0xe3, 0x74, 0x77, 0x59, 0xba, 0x44, 0xcd, 0x7d, 0x17, 0x26, 0x43, 0x9d, 0x00, 0x21, 0x48, 0xb5, + 0x49, 0x7b, 0x0b, 0x34, 0xd0, 0x88, 0xfe, 0x47, 0x6f, 0xc2, 0x84, 0x61, 0x6a, 0xd8, 0x1d, 0x21, + 0x93, 0xab, 0xb3, 0x2f, 0xce, 0x16, 0xd2, 0xdb, 0xa6, 0xc6, 0xdc, 0x15, 0xfe, 0x4f, 0x4a, 0x93, + 0x4c, 0xae, 0xb3, 0x32, 0x77, 0x03, 0x52, 0xa4, 0xf5, 0x89, 0x91, 0xda, 0x53, 0x6c, 0xbc, 0x6b, + 0xe9, 0x5c, 0xa6, 0xfb, 0xc8, 0xf3, 0xfd, 0x4c, 0x80, 0x44, 0xed, 0x1e, 0x71, 0xd4, 0xf7, 0x3a, + 0xea, 0x21, 0x76, 0x78, 0x2e, 0xfe, 0x44, 0x1d, 0x78, 0x0b, 0xef, 0xeb, 0xcc, 0x87, 0xca, 0x4a, + 0xfc, 0x09, 0xbd, 0x0a, 0xa0, 0xa8, 0x2a, 0xb6, 0x6d, 0xd9, 0x3d, 0x5e, 0x97, 0x95, 0xb2, 0x8c, + 0xb2, 0x89, 0x4f, 0x09, 0x9b, 0x8d, 0x55, 0x0b, 0x3b, 0x6e, 0xc4, 0x14, 0x7b, 0x22, 0x6c, 0x0e, + 0x6e, 0xb5, 0x65, 0xc7, 0x3c, 0xc4, 0x06, 0xed, 0x33, 0x59, 0x62, 0x7c, 0x5a, 0xed, 0x3a, 0x21, + 0x10, 0xbb, 0x89, 0x0d, 0xcd, 0x37, 0x72, 0x59, 0xc9, 0x7b, 0x26, 0x22, 0x2d, 0xdc, 0xd0, 0xf9, + 0x09, 0xb1, 0xac, 0xc4, 0x9f, 0x88, 0xc6, 0x94, 0x8e, 0x73, 0x40, 0x5b, 0x25, 0x2b, 0xd1, 0xff, + 0xbc, 0x6a, 0x7f, 0x5f, 0x80, 0xe4, 0xc3, 0x72, 0xed, 0xdc, 0x75, 0x73, 0x25, 0x26, 0x7d, 0x89, + 0x34, 0x50, 0x51, 0x6f, 0x36, 0x75, 0xa3, 0x41, 0x5c, 0x9a, 0xef, 0x63, 0xd5, 0xad, 0x59, 0x81, + 0x93, 0x77, 0x18, 0x15, 0x2d, 0x42, 0x4e, 0xb5, 0xb0, 0x86, 0x0d, 0x47, 0x57, 0x9a, 0x36, 0xaf, + 0x62, 0x90, 0xc4, 0x0b, 0xf7, 0x87, 0x02, 0x8c, 0xd3, 0xce, 0x8b, 0x5e, 0x81, 0xac, 0x6a, 0x1a, + 0x8e, 0xa2, 0x1b, 0xdc, 0x0a, 0x65, 0x25, 0x9f, 0xd0, 0xb7, 0x90, 0x57, 0x21, 0xaf, 0xa8, 0xaa, + 0xd9, 0x31, 0x1c, 0xd9, 0x50, 0x5a, 0x98, 0x17, 0x36, 0xc7, 0x69, 0xdb, 0x4a, 0x0b, 0xa3, 0x05, + 0x70, 0x1f, 0xbd, 0x43, 0x8e, 0x59, 0x09, 0x38, 0x69, 0x13, 0x9f, 0xf2, 0x92, 0xfc, 0x48, 0x80, + 0x8c, 0xdb, 0xe9, 0x49, 0x61, 0x1a, 0xd8, 0xc0, 0x96, 0xe2, 0x98, 0x5e, 0x61, 0x3c, 0x42, 0xf7, + 0x8c, 0x97, 0xf5, 0x67, 0xbc, 0x59, 0x18, 0x77, 0x48, 0xbf, 0xe6, 0xe5, 0x60, 0x0f, 0x74, 0xad, + 0xb9, 0xa9, 0x34, 0xd8, 0xf2, 0x5a, 0x56, 0x62, 0x0f, 0xa4, 0x4a, 0x3c, 0xd8, 0x96, 0x69, 0x87, + 0x3f, 0x91, 0xf2, 0xb2, 0x60, 0xd0, 0x3d, 0xdc, 0xd0, 0x0d, 0xda, 0x01, 0x92, 0x12, 0x50, 0xd2, + 0x2a, 0xa1, 0xa0, 0x2b, 0x90, 0x65, 0x19, 0xb0, 0xa1, 0xd1, 0x5e, 0x90, 0x94, 0x32, 0x94, 0x50, + 0x71, 0x4f, 0x71, 0xcd, 0x1d, 0x42, 0xd6, 0x1b, 0x63, 0xa4, 0x21, 0x3b, 0xb6, 0xa7, 0x54, 0xfa, + 0x1f, 0xbd, 0x05, 0xb3, 0xcf, 0x3a, 0x4a, 0x53, 0xdf, 0xa7, 0x2b, 0x67, 0x24, 0x1b, 0xd3, 0x1f, + 0xab, 0x0f, 0xf2, 0xd2, 0xa8, 0x04, 0xaa, 0x46, 0x77, 0x48, 0x26, 0xfd, 0x21, 0x19, 0xdc, 0x0a, + 0x29, 0x7d, 0x21, 0xc0, 0x34, 0x8b, 0x17, 0x62, 0x21, 0xab, 0xf1, 0x39, 0x18, 0xef, 0x43, 0x56, + 0x53, 0x1c, 0x85, 0x1d, 0xe4, 0x4c, 0x0c, 0x3c, 0xc8, 0xe9, 0x5a, 0x7c, 0x92, 0x9f, 0x1e, 0xe6, + 0x44, 0x90, 0x22, 0xff, 0xd9, 0xc9, 0x57, 0x89, 0xfe, 0xf7, 0x23, 0x30, 0x82, 0xc5, 0x8d, 0xd3, + 0xe1, 0x5a, 0x86, 0x0b, 0x44, 0xfb, 0x15, 0x43, 0xb5, 0x4e, 0xdb, 0x8e, 0x6e, 0x1a, 0x8f, 0xe9, + 0xaf, 0x8d, 0xc4, 0xc0, 0x06, 0x16, 0xdd, 0xb7, 0xe2, 0x65, 0xf9, 0x37, 0x69, 0x98, 0xac, 0x9c, + 0xb4, 0x4d, 0x2b, 0xd6, 0x45, 0xad, 0x55, 0x98, 0xe0, 0x88, 0x7f, 0xc0, 0x9e, 0x72, 0x97, 0xad, + 0x76, 0xb7, 0x6b, 0x39, 0x23, 0x5a, 0x05, 0x60, 0xc1, 0xa5, 0x34, 0xe8, 0x28, 0x79, 0x8e, 0x8d, + 0x35, 0xca, 0x46, 0xa8, 0x68, 0x1b, 0x72, 0xad, 0x23, 0x55, 0x95, 0xf7, 0xf5, 0xa6, 0xc3, 0xa3, + 0xf3, 0xa2, 0x43, 0xcb, 0xb7, 0x9e, 0x96, 0xcb, 0x0f, 0x68, 0x26, 0x16, 0x28, 0xe7, 0x3f, 0x4b, + 0x40, 0x24, 0xb0, 0xff, 0xe8, 0x0d, 0xe0, 0x07, 0x6c, 0x64, 0xdb, 0x3d, 0x4b, 0xb7, 0x3a, 0xf9, + 0xe2, 0x6c, 0x21, 0x2b, 0x51, 0x6a, 0xad, 0x56, 0x97, 0xb2, 0x2c, 0x43, 0xcd, 0x76, 0xd0, 0x6b, + 0x30, 0x69, 0xb6, 0x74, 0x47, 0x76, 0x7d, 0x20, 0xee, 0x36, 0xe6, 0x09, 0xd1, 0xf5, 0x91, 0x50, + 0x1d, 0x6e, 0x62, 0x83, 0x8e, 0x02, 0x52, 0x4f, 0x79, 0x8f, 0xad, 0x45, 0x3a, 0x6c, 0xbc, 0xcb, + 0x66, 0xdb, 0xd1, 0x5b, 0xfa, 0x73, 0xba, 0xab, 0xcd, 0xf7, 0x8b, 0x5e, 0x63, 0xd9, 0x49, 0xfd, + 0x56, 0xe9, 0x22, 0x25, 0xcf, 0xfb, 0x38, 0x90, 0x15, 0xfd, 0xa1, 0x00, 0x17, 0xb9, 0x22, 0xe5, + 0x3d, 0x1a, 0x1b, 0xaf, 0x34, 0x75, 0xe7, 0x54, 0x3e, 0x3c, 0x2a, 0x66, 0xa8, 0x73, 0xfa, 0x5b, + 0x91, 0x0d, 0x12, 0xe8, 0x07, 0x4b, 0x6e, 0xb3, 0x9c, 0x3e, 0xe2, 0xcc, 0x9b, 0x47, 0x15, 0xc3, + 0xb1, 0x4e, 0x57, 0x2f, 0xbd, 0x38, 0x5b, 0x98, 0xe9, 0x4d, 0x7d, 0x2a, 0xcd, 0xd8, 0xbd, 0x2c, + 0xa8, 0x0a, 0x80, 0xbd, 0xde, 0x48, 0x63, 0x03, 0xa3, 0xdd, 0x8b, 0xc8, 0x6e, 0x2b, 0x05, 0x78, + 0xd1, 0x2d, 0x10, 0xf9, 0xe9, 0x98, 0x7d, 0xbd, 0x89, 0x65, 0x5b, 0x7f, 0x8e, 0x8b, 0x40, 0x6d, + 0x50, 0x81, 0xd1, 0x89, 0x88, 0x9a, 0xfe, 0x1c, 0xcf, 0x7d, 0x1f, 0x8a, 0xfd, 0x4a, 0x1f, 0x1c, + 0x08, 0x59, 0xb6, 0x81, 0xfb, 0x5e, 0x78, 0x45, 0x66, 0x84, 0xae, 0xea, 0xae, 0xca, 0x24, 0xde, + 0x73, 0x4d, 0xd0, 0x9f, 0x27, 0x60, 0x72, 0xb5, 0xd3, 0x3c, 0x7c, 0xdc, 0xae, 0x75, 0x5a, 0x2d, + 0xc5, 0x3a, 0x25, 0xa6, 0x92, 0x99, 0x0e, 0x52, 0x4c, 0x81, 0x99, 0x4a, 0x6a, 0x1b, 0xf4, 0xe7, + 0x98, 0x4c, 0x66, 0xc1, 0xd3, 0xdc, 0x2c, 0xf6, 0x9f, 0xd6, 0x24, 0x70, 0x44, 0xdb, 0x3c, 0xb6, + 0xd1, 0x7b, 0x50, 0x0c, 0x64, 0xa4, 0xcb, 0x27, 0x32, 0x36, 0x1c, 0x4b, 0xc7, 0x6c, 0x39, 0x30, + 0x29, 0x05, 0xe2, 0x6e, 0xd6, 0x49, 0x72, 0x85, 0xa5, 0xa2, 0x3a, 0xe4, 0x49, 0xc6, 0x53, 0x99, + 0x4e, 0x36, 0xee, 0xa2, 0xed, 0x9d, 0x88, 0xca, 0x85, 0xca, 0xbd, 0x44, 0xb5, 0x54, 0xa6, 0x3c, + 0xf4, 0xaf, 0x94, 0xc3, 0x3e, 0x65, 0xee, 0x43, 0x10, 0xbb, 0x33, 0x04, 0x35, 0x9a, 0x62, 0x1a, + 0x9d, 0x0d, 0x6a, 0x34, 0x19, 0xd0, 0xd6, 0x46, 0x2a, 0x93, 0x12, 0xc7, 0x4b, 0x3f, 0x4f, 0x42, + 0xc1, 0xed, 0x6c, 0x71, 0xa2, 0x99, 0x55, 0x18, 0x27, 0x5d, 0xc3, 0x8d, 0x12, 0xb9, 0x31, 0xa0, + 0x8f, 0xf3, 0x38, 0x73, 0xd2, 0x65, 0x5c, 0x3c, 0x4c, 0x59, 0xe3, 0x30, 0x3b, 0x73, 0x7f, 0x39, + 0x01, 0x29, 0x0a, 0x20, 0xee, 0x40, 0x8a, 0x4e, 0x1d, 0xc2, 0x28, 0x53, 0x07, 0xcd, 0xea, 0x4d, + 0x76, 0x89, 0x80, 0xff, 0x49, 0x9c, 0xb9, 0x03, 0xe5, 0x9d, 0x3b, 0x77, 0xa9, 0xc9, 0xc9, 0x4b, + 0xfc, 0x09, 0xad, 0xd2, 0xf0, 0x25, 0xd3, 0x72, 0xb0, 0xc6, 0x1d, 0xf7, 0xc5, 0x61, 0xed, 0xeb, + 0x4e, 0x53, 0x2e, 0x1f, 0xba, 0x0c, 0x49, 0x62, 0xcb, 0x26, 0x58, 0x64, 0xc3, 0x8b, 0xb3, 0x85, + 0x24, 0xb1, 0x62, 0x84, 0x86, 0x96, 0x21, 0x17, 0x36, 0x1c, 0xc2, 0xad, 0x2c, 0x33, 0x8f, 0x81, + 0x41, 0x0f, 0x4d, 0x6f, 0x80, 0x31, 0xd0, 0xca, 0xdb, 0xf8, 0x0f, 0xc6, 0x61, 0x72, 0xbd, 0x15, + 0xf7, 0xc4, 0xb2, 0x12, 0x6e, 0xe1, 0x28, 0xb4, 0x13, 0x7a, 0x69, 0x44, 0x03, 0x87, 0xe6, 0xf4, + 0xe4, 0xf9, 0xe6, 0xf4, 0x75, 0xe2, 0x02, 0xf3, 0xeb, 0x19, 0x92, 0x7d, 0x80, 0x4d, 0xf8, 0xfd, + 0xd4, 0x8b, 0x91, 0x08, 0x8f, 0x7f, 0xf2, 0x82, 0x86, 0xa7, 0x7c, 0x48, 0x3d, 0x6d, 0xd6, 0xcb, + 0xd2, 0xa3, 0xf7, 0xb2, 0x09, 0x6c, 0x68, 0x74, 0x6a, 0x0b, 0xdb, 0xd5, 0x89, 0x97, 0xb7, 0xab, + 0x73, 0xcf, 0x79, 0x67, 0x7d, 0x1f, 0x92, 0x9a, 0xee, 0x36, 0xce, 0xe8, 0x13, 0x36, 0x61, 0x1a, + 0xd2, 0x6b, 0x53, 0xc1, 0x5e, 0x1b, 0x5c, 0xe0, 0x98, 0x7b, 0x0c, 0xe0, 0x6b, 0x08, 0x2d, 0x42, + 0xda, 0x6c, 0x6a, 0xee, 0x01, 0x94, 0xc9, 0xd5, 0xec, 0x8b, 0xb3, 0x85, 0xf1, 0xc7, 0x4d, 0x6d, + 0x7d, 0x4d, 0x1a, 0x37, 0x9b, 0xda, 0xba, 0x46, 0x6f, 0xc8, 0xc0, 0xc7, 0xb2, 0x17, 0xad, 0x96, + 0x97, 0x26, 0x0c, 0x7c, 0xbc, 0x86, 0x6d, 0xb5, 0x2b, 0x38, 0x86, 0x74, 0xc1, 0x1f, 0x08, 0x50, + 0x70, 0x5b, 0x23, 0x5e, 0x33, 0x93, 0xd1, 0x5b, 0x7c, 0xd8, 0x25, 0xcf, 0x37, 0xec, 0x5c, 0x3e, + 0x7e, 0xfc, 0xf6, 0x8f, 0x04, 0x1e, 0xa9, 0x5c, 0x53, 0x15, 0x87, 0x38, 0x1b, 0x31, 0x0e, 0x95, + 0xd7, 0x41, 0xb4, 0x14, 0x43, 0x33, 0x5b, 0xfa, 0x73, 0xcc, 0x56, 0x44, 0x6d, 0xbe, 0xb9, 0x39, + 0xe5, 0xd1, 0xe9, 0x92, 0x9f, 0xbb, 0xa0, 0xfb, 0x2b, 0x81, 0x47, 0x35, 0x7b, 0x85, 0x89, 0x53, + 0x69, 0x9b, 0x90, 0xb6, 0x58, 0x6c, 0x24, 0x1b, 0xba, 0x6f, 0x46, 0x08, 0x89, 0x7a, 0x3b, 0x0b, + 0x3d, 0xf4, 0x06, 0x0f, 0x15, 0x31, 0xf7, 0xbb, 0x30, 0x4e, 0xc9, 0x2f, 0x61, 0x60, 0xb9, 0xe6, + 0x7f, 0x99, 0x80, 0x6b, 0xf4, 0x75, 0x4f, 0xb1, 0xa5, 0xef, 0x9f, 0xee, 0x58, 0xa6, 0x83, 0x55, + 0x07, 0x6b, 0xfe, 0x79, 0x8f, 0x58, 0xad, 0x56, 0xb6, 0xed, 0xbe, 0xe0, 0x5c, 0x21, 0x62, 0x1e, + 0x17, 0xda, 0x84, 0x29, 0x76, 0x0b, 0x8f, 0xac, 0x34, 0xf5, 0x23, 0x2c, 0x2b, 0xce, 0x79, 0xe6, + 0xa6, 0x49, 0xc6, 0xbb, 0x42, 0x58, 0x57, 0x1c, 0xa4, 0x41, 0x96, 0x0b, 0xd3, 0x35, 0x7e, 0xf5, + 0xce, 0xc3, 0xaf, 0xb7, 0xe6, 0x97, 0x91, 0xa8, 0xbc, 0xf5, 0x35, 0x29, 0xc3, 0x24, 0x7b, 0x7b, + 0x36, 0x3f, 0x15, 0xe0, 0xfa, 0x10, 0x45, 0xc7, 0xd9, 0xcd, 0xe6, 0x20, 0x73, 0x44, 0x5e, 0xa4, + 0x73, 0x4d, 0x67, 0x24, 0xef, 0x19, 0x6d, 0xc1, 0xe4, 0xbe, 0xa2, 0x37, 0xdd, 0xfb, 0x73, 0x06, + 0x05, 0x16, 0x46, 0xc7, 0xbb, 0xe6, 0x19, 0x3b, 0x4d, 0xa4, 0x27, 0x22, 0xa7, 0x57, 0x34, 0xad, + 0x56, 0xe3, 0x16, 0x2c, 0xbe, 0xfe, 0xe2, 0x42, 0xc7, 0x84, 0x0f, 0x1d, 0xd1, 0x9b, 0x80, 0x34, + 0xdd, 0x66, 0xd7, 0x7a, 0xd8, 0x07, 0x8a, 0x66, 0x1e, 0xfb, 0x51, 0x13, 0xd3, 0x6e, 0x4a, 0xcd, + 0x4d, 0x40, 0x35, 0xa0, 0xb8, 0x45, 0xb6, 0x1d, 0xc5, 0xdb, 0xf8, 0xb9, 0x3e, 0xd2, 0xf1, 0x2c, + 0x06, 0x68, 0xbc, 0x47, 0x29, 0x4b, 0xe4, 0xd0, 0xbf, 0xc4, 0x03, 0xd7, 0x49, 0xd5, 0x1d, 0x59, + 0xb1, 0xdd, 0xb3, 0x3c, 0xec, 0x42, 0x91, 0x02, 0xa3, 0xaf, 0xd8, 0xc1, 0x23, 0x3a, 0xec, 0xa8, + 0x81, 0xaf, 0xa0, 0x38, 0x81, 0xee, 0x3f, 0x12, 0xa0, 0x20, 0xe1, 0x7d, 0x0b, 0xdb, 0xb1, 0x02, + 0xfe, 0x07, 0x90, 0xb7, 0x98, 0x54, 0x79, 0xdf, 0x32, 0x5b, 0xe7, 0x19, 0x63, 0x39, 0xce, 0xf8, + 0xc0, 0x32, 0x5b, 0xa1, 0x3b, 0x16, 0x9e, 0xc2, 0x94, 0x57, 0xd2, 0x38, 0x55, 0xf0, 0x05, 0x3d, + 0x92, 0xcc, 0x04, 0xc7, 0x1d, 0xbe, 0xf0, 0x4d, 0xe8, 0x81, 0xee, 0x34, 0x05, 0x8b, 0x1b, 0xa7, + 0x32, 0x7e, 0x25, 0x40, 0xa1, 0xd6, 0xd9, 0x63, 0xb7, 0x4a, 0xc5, 0xa7, 0x87, 0x0a, 0x64, 0x9b, + 0x78, 0xdf, 0x91, 0x5f, 0x2a, 0x3c, 0x3e, 0x43, 0x58, 0xe9, 0x11, 0x81, 0x87, 0x00, 0x16, 0x3d, + 0x00, 0x47, 0xe5, 0x24, 0xcf, 0x29, 0x27, 0x4b, 0x79, 0x7d, 0x27, 0xa7, 0xf4, 0x45, 0x02, 0xa6, + 0xbc, 0xca, 0xc6, 0x69, 0x3d, 0x3f, 0x0a, 0x59, 0x8d, 0xe4, 0x79, 0xac, 0xc6, 0x34, 0x8f, 0xde, + 0x88, 0xb6, 0x1c, 0x4b, 0x30, 0x43, 0x5d, 0x10, 0x59, 0x69, 0xb7, 0x9b, 0xba, 0x0b, 0x65, 0xa9, + 0x5d, 0x4a, 0x49, 0xd3, 0x34, 0x69, 0x85, 0xa5, 0x50, 0x10, 0x4b, 0xfa, 0xdf, 0xbe, 0x85, 0xf1, + 0x73, 0x2c, 0x53, 0x54, 0x75, 0x9e, 0xe8, 0x94, 0x1c, 0x63, 0xac, 0x11, 0x3e, 0xde, 0xf3, 0x3e, + 0x85, 0x69, 0xaa, 0xd9, 0xb8, 0x0f, 0xe1, 0xf2, 0xe6, 0xf8, 0x93, 0x04, 0xa0, 0xa0, 0xfc, 0x6f, + 0xae, 0x45, 0x12, 0xf1, 0xb5, 0xc8, 0x1b, 0x80, 0x58, 0x14, 0xa2, 0x2d, 0xb7, 0xb1, 0x25, 0xdb, + 0x58, 0x35, 0xf9, 0x5d, 0x47, 0x82, 0x24, 0xf2, 0x94, 0x1d, 0x6c, 0xd5, 0x28, 0x1d, 0xdd, 0x07, + 0xf0, 0xef, 0x9c, 0xe3, 0xd3, 0xc9, 0xc0, 0x2b, 0xe7, 0xa4, 0xac, 0xe5, 0xfe, 0x2d, 0x7d, 0x36, + 0x07, 0x79, 0xae, 0xc9, 0x5d, 0x43, 0x37, 0x0d, 0x74, 0x07, 0x92, 0x0d, 0xbe, 0x19, 0x90, 0x8b, + 0x5c, 0x8e, 0xf3, 0xaf, 0x77, 0xab, 0x8e, 0x49, 0x24, 0x2f, 0x61, 0x69, 0x77, 0x9c, 0x08, 0xe7, + 0xc9, 0x8f, 0xb5, 0x0e, 0xb2, 0xb4, 0x3b, 0x0e, 0xaa, 0xc1, 0x94, 0xea, 0x5f, 0x56, 0x25, 0x13, + 0xf6, 0x64, 0x5f, 0x98, 0x14, 0x79, 0x49, 0x58, 0x75, 0x4c, 0x2a, 0xa8, 0xa1, 0x04, 0x54, 0x0e, + 0xde, 0x8e, 0x94, 0xea, 0x09, 0xeb, 0xf2, 0x4f, 0xfb, 0x86, 0x6f, 0x66, 0xaa, 0x8e, 0x05, 0x2e, + 0x51, 0x42, 0xef, 0x43, 0x5a, 0xa3, 0xb7, 0xee, 0xf0, 0x7e, 0x1d, 0xd5, 0xf5, 0x42, 0x17, 0x1d, + 0x55, 0xc7, 0x24, 0xce, 0x81, 0x36, 0x20, 0xcf, 0xfe, 0xf1, 0x83, 0x02, 0xe9, 0xbe, 0xdb, 0x7e, + 0xbd, 0xf7, 0x0e, 0x55, 0xc7, 0xa4, 0x9c, 0xe6, 0x53, 0xd1, 0xdb, 0x90, 0xb2, 0x55, 0xc5, 0x45, + 0x8f, 0xf3, 0x7d, 0xae, 0xdc, 0xf0, 0x99, 0x69, 0x6e, 0x74, 0x9f, 0x5d, 0xdb, 0xe8, 0x9c, 0xb8, + 0xcb, 0x79, 0x51, 0xc5, 0x0f, 0x1d, 0xe4, 0x26, 0xc5, 0xc7, 0x94, 0x80, 0x1e, 0x42, 0x4e, 0x21, + 0xde, 0xa0, 0x4c, 0x0f, 0x4e, 0xd2, 0xf5, 0xbb, 0xe8, 0x9d, 0xf2, 0x9e, 0x43, 0xaf, 0x55, 0x7a, + 0x5a, 0xdc, 0x25, 0xfa, 0x82, 0x5a, 0xd8, 0x6a, 0xe0, 0x62, 0x6e, 0xb0, 0xa0, 0x60, 0x18, 0x97, + 0x27, 0x88, 0x12, 0x89, 0x57, 0x78, 0xe0, 0x1e, 0x8a, 0xa1, 0x95, 0xca, 0xf7, 0xdd, 0x95, 0x8d, + 0x38, 0xd4, 0x53, 0x1d, 0x93, 0xf2, 0x07, 0x01, 0x32, 0x5a, 0x82, 0x44, 0x43, 0x2d, 0x4e, 0xf6, + 0x1d, 0x21, 0xde, 0x91, 0x95, 0xea, 0x98, 0x94, 0x68, 0xa8, 0xe8, 0x43, 0xc8, 0xb0, 0xa3, 0x04, + 0x27, 0x46, 0xb1, 0xd0, 0xd7, 0x4e, 0x84, 0x0f, 0x64, 0x54, 0xc7, 0x24, 0x7a, 0x7a, 0x81, 0xbc, + 0x6f, 0x07, 0x0a, 0x16, 0x8b, 0x83, 0x73, 0x23, 0x58, 0xc5, 0xbe, 0x3b, 0xd5, 0x51, 0x41, 0xac, + 0x55, 0x8a, 0x0e, 0x02, 0x74, 0xf4, 0xfb, 0x30, 0x1b, 0x96, 0xc8, 0x7b, 0xda, 0x74, 0xdf, 0x5d, + 0xd7, 0xbe, 0xa1, 0x94, 0xd5, 0x31, 0x09, 0x59, 0x3d, 0x89, 0xe8, 0x5d, 0x18, 0x67, 0xad, 0x86, + 0xa8, 0xc8, 0xa8, 0x10, 0x8c, 0xae, 0x06, 0x63, 0xf9, 0x49, 0xe7, 0x77, 0x78, 0x00, 0x98, 0xdc, + 0x34, 0x1b, 0xc5, 0x99, 0xbe, 0x9d, 0xbf, 0x37, 0xa0, 0x8d, 0x74, 0x7e, 0xc7, 0xa7, 0x92, 0x76, + 0xb7, 0x58, 0x0a, 0x8f, 0x17, 0x9a, 0xed, 0xdb, 0xee, 0x11, 0x71, 0x61, 0x55, 0x1a, 0x92, 0xef, + 0x93, 0x49, 0xd1, 0x2c, 0x76, 0x3f, 0x8c, 0x4c, 0xc7, 0xd4, 0x85, 0xbe, 0x45, 0xeb, 0xbd, 0x46, + 0xa7, 0x4a, 0xbd, 0x26, 0x8f, 0x8a, 0x9e, 0x82, 0xc8, 0x6f, 0x6e, 0xf0, 0xf7, 0x0e, 0x2e, 0x52, + 0x79, 0xaf, 0x47, 0x9a, 0xae, 0xa8, 0x00, 0x9b, 0xea, 0x98, 0x34, 0xa5, 0x86, 0x53, 0xd0, 0xc7, + 0x30, 0x4d, 0xe5, 0xc9, 0xaa, 0x7f, 0xe5, 0x46, 0xb1, 0xd8, 0x73, 0x75, 0x43, 0xff, 0xdb, 0x39, + 0x5c, 0xc9, 0xa2, 0xda, 0x95, 0x44, 0xba, 0xb1, 0x6e, 0xe8, 0x0e, 0xb5, 0xb2, 0x73, 0x7d, 0xbb, + 0x71, 0xf8, 0x72, 0x3f, 0xd2, 0x8d, 0x75, 0x46, 0x21, 0xdd, 0xd8, 0xe1, 0xc1, 0x64, 0xbc, 0x39, + 0x5e, 0xe9, 0xdb, 0x8d, 0xa3, 0xa2, 0xce, 0x48, 0x37, 0x76, 0x82, 0x74, 0xd2, 0x8d, 0x99, 0x81, + 0xe8, 0x92, 0xfb, 0x6a, 0xdf, 0x6e, 0xdc, 0xf7, 0xe8, 0x32, 0xe9, 0xc6, 0x4a, 0x4f, 0x22, 0x5a, + 0x03, 0x60, 0x4e, 0x0d, 0x9d, 0x14, 0xe7, 0xfb, 0x4e, 0x06, 0xdd, 0xe1, 0x64, 0x64, 0x32, 0x68, + 0xba, 0x34, 0x62, 0xc8, 0x28, 0x94, 0x92, 0xe9, 0x46, 0x6a, 0x71, 0xa1, 0xaf, 0x21, 0xeb, 0xd9, + 0xe2, 0x24, 0x86, 0xec, 0xd8, 0x23, 0x92, 0x59, 0x85, 0xad, 0xea, 0x16, 0x17, 0xfb, 0x9b, 0xe5, + 0xe0, 0x16, 0x0f, 0x35, 0xcb, 0x94, 0x80, 0x56, 0x20, 0x4b, 0xe6, 0xfc, 0x53, 0x6a, 0x86, 0xae, + 0xf6, 0xf5, 0x4f, 0xbb, 0xce, 0x9c, 0x54, 0xc7, 0xa4, 0xcc, 0x33, 0x4e, 0x22, 0xaf, 0x67, 0xab, + 0x5b, 0xc5, 0x52, 0xdf, 0xd7, 0x87, 0xd6, 0x46, 0xc9, 0xeb, 0x19, 0x07, 0x52, 0xe1, 0x02, 0x6b, + 0x2b, 0x7e, 0x72, 0xd8, 0xe2, 0xc7, 0x5c, 0x8b, 0xaf, 0x51, 0x51, 0x7d, 0xd7, 0x8a, 0x22, 0x0f, + 0x34, 0x57, 0xc7, 0xa4, 0x19, 0xa5, 0x37, 0x95, 0x0c, 0x78, 0x3e, 0xf5, 0xb0, 0x15, 0xa6, 0xe2, + 0xb5, 0xbe, 0x03, 0x3e, 0x62, 0x4d, 0x8e, 0x0c, 0x78, 0x25, 0x40, 0x66, 0x13, 0x90, 0x26, 0xdb, + 0x36, 0xdb, 0x76, 0xbf, 0x3e, 0x60, 0x02, 0xea, 0x5a, 0x23, 0x60, 0x13, 0x90, 0x56, 0x63, 0x9c, + 0x44, 0x50, 0xf0, 0xe4, 0xdf, 0x8d, 0xbe, 0x82, 0x7a, 0x2e, 0xcc, 0x23, 0x82, 0x54, 0x8f, 0x48, + 0x1c, 0x1e, 0xcb, 0xbd, 0xf2, 0x85, 0x3b, 0x8c, 0x37, 0xfb, 0x3a, 0x3c, 0x91, 0x37, 0xd3, 0x10, + 0x87, 0xc7, 0x0a, 0x25, 0xa0, 0xdf, 0x86, 0x09, 0x0e, 0xe8, 0x8a, 0xb7, 0x06, 0xb8, 0xb1, 0x41, + 0x24, 0x4e, 0xc6, 0x35, 0xe7, 0x61, 0x56, 0x96, 0x01, 0x49, 0x56, 0xbd, 0xd7, 0x07, 0x58, 0xd9, + 0x1e, 0x2c, 0xcb, 0xac, 0xac, 0x4f, 0x26, 0x56, 0x96, 0xf5, 0x53, 0x3e, 0xd7, 0xdd, 0xee, 0x6b, + 0x65, 0x7b, 0xcf, 0xbd, 0x10, 0x2b, 0xfb, 0xcc, 0xa7, 0x92, 0x9a, 0xd9, 0x0c, 0x44, 0x15, 0xbf, + 0xd3, 0xb7, 0x66, 0x61, 0x4c, 0x49, 0x6a, 0xc6, 0x79, 0x48, 0xb3, 0x31, 0x97, 0x98, 0x69, 0xfa, + 0x8d, 0xfe, 0x87, 0xf4, 0xbb, 0xa1, 0x07, 0x69, 0x36, 0xcb, 0x23, 0xfa, 0x86, 0xca, 0xe2, 0x47, + 0x92, 0xb9, 0xa6, 0xde, 0x1c, 0x6c, 0xa8, 0xa2, 0x4e, 0x5b, 0x7b, 0x86, 0x2a, 0x94, 0x48, 0x8b, + 0xca, 0x8e, 0x8f, 0xd1, 0xf1, 0xbd, 0x34, 0xe0, 0x3e, 0x81, 0xae, 0xa3, 0x7c, 0xb4, 0xa8, 0x1e, + 0xd1, 0x1f, 0x42, 0x1d, 0x76, 0xf1, 0x45, 0x71, 0x79, 0xf0, 0x10, 0x0a, 0x5f, 0xc0, 0xe1, 0x0d, + 0x21, 0x4e, 0xf6, 0xe6, 0x4c, 0xd7, 0xc3, 0x78, 0x6b, 0xf0, 0x9c, 0xd9, 0xed, 0x5a, 0xb0, 0x39, + 0x93, 0xfb, 0x14, 0x7f, 0x45, 0x80, 0x45, 0x56, 0x36, 0xba, 0xde, 0x77, 0x2a, 0x7b, 0x6b, 0xa7, + 0x81, 0x43, 0x0e, 0x77, 0xe8, 0x0b, 0xde, 0xed, 0x57, 0xdc, 0x21, 0x6b, 0xc1, 0xd5, 0x31, 0xe9, + 0x55, 0x65, 0x50, 0xbe, 0xd5, 0x09, 0xbe, 0xf1, 0xe9, 0x9d, 0xe0, 0x9c, 0x12, 0xc5, 0x8d, 0x54, + 0xe6, 0x92, 0x58, 0xdc, 0x48, 0x65, 0x2e, 0x8b, 0x73, 0x1b, 0xa9, 0xcc, 0x15, 0xf1, 0x95, 0xd2, + 0x7f, 0xbf, 0x0c, 0x93, 0x2e, 0xf2, 0x63, 0x88, 0xe8, 0x6e, 0x10, 0x11, 0xcd, 0xf7, 0x43, 0x44, + 0x1c, 0x2b, 0x72, 0x48, 0x74, 0x37, 0x08, 0x89, 0xe6, 0xfb, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, + 0x7a, 0x3f, 0x4c, 0xf4, 0xfa, 0x08, 0x98, 0xc8, 0x13, 0xd5, 0x0d, 0x8a, 0xd6, 0x7a, 0x41, 0xd1, + 0xb5, 0xc1, 0xa0, 0xc8, 0x13, 0x15, 0x40, 0x45, 0xf7, 0xbb, 0x50, 0xd1, 0xd5, 0x01, 0xa8, 0xc8, + 0xe3, 0x77, 0x61, 0xd1, 0x66, 0x24, 0x2c, 0xba, 0x31, 0x0c, 0x16, 0x79, 0x72, 0x42, 0xb8, 0xe8, + 0x9d, 0x10, 0x2e, 0x5a, 0xe8, 0x8b, 0x8b, 0x3c, 0x6e, 0x06, 0x8c, 0x3e, 0xe8, 0x06, 0x46, 0x57, + 0x07, 0x00, 0x23, 0xbf, 0x06, 0x1c, 0x19, 0x55, 0xa3, 0x90, 0xd1, 0xf5, 0x21, 0xc8, 0xc8, 0x93, + 0x12, 0x84, 0x46, 0xd5, 0x28, 0x68, 0x74, 0x7d, 0x08, 0x34, 0xea, 0x92, 0xc4, 0xb0, 0xd1, 0x76, + 0x34, 0x36, 0xba, 0x39, 0x14, 0x1b, 0x79, 0xd2, 0xc2, 0xe0, 0x68, 0x39, 0x00, 0x8e, 0x5e, 0xed, + 0x03, 0x8e, 0x3c, 0x56, 0x82, 0x8e, 0x7e, 0xa7, 0x07, 0x1d, 0x95, 0x06, 0xa1, 0x23, 0x8f, 0xd7, + 0x83, 0x47, 0x4f, 0xfa, 0xc0, 0xa3, 0x5b, 0xc3, 0xe1, 0x91, 0x27, 0xac, 0x0b, 0x1f, 0x29, 0x03, + 0xf1, 0xd1, 0x9b, 0x23, 0xe2, 0x23, 0x4f, 0x7a, 0x14, 0x40, 0x7a, 0x2f, 0x0c, 0x90, 0x16, 0xfb, + 0x03, 0x24, 0x4f, 0x0c, 0x47, 0x48, 0x9b, 0x91, 0x08, 0xe9, 0xc6, 0x30, 0x84, 0xe4, 0x8f, 0x83, + 0x20, 0x44, 0xda, 0x8e, 0x86, 0x48, 0x37, 0x87, 0x42, 0x24, 0xbf, 0xf9, 0x43, 0x18, 0x69, 0x33, + 0x12, 0x23, 0xdd, 0x18, 0x86, 0x91, 0xfc, 0xc2, 0x05, 0x41, 0xd2, 0x47, 0x7d, 0x41, 0xd2, 0xed, + 0x51, 0x40, 0x92, 0x27, 0xb4, 0x07, 0x25, 0x7d, 0xd2, 0x1f, 0x25, 0x7d, 0xe7, 0x1c, 0x77, 0x18, + 0x46, 0xc2, 0xa4, 0xdf, 0xe9, 0x81, 0x49, 0xa5, 0x41, 0x30, 0xc9, 0xef, 0xcf, 0x2e, 0x4e, 0x52, + 0x06, 0xa2, 0x9a, 0x37, 0x47, 0x44, 0x35, 0x7e, 0xe7, 0x8b, 0x80, 0x35, 0x95, 0x08, 0x58, 0x73, + 0x6d, 0x30, 0xac, 0xf1, 0xcd, 0xb9, 0x8f, 0x6b, 0xaa, 0x51, 0xb8, 0xe6, 0xfa, 0x10, 0x5c, 0xe3, + 0x5b, 0xa1, 0x00, 0xb0, 0xb9, 0xdf, 0x05, 0x6c, 0xae, 0x0e, 0x8d, 0xeb, 0x09, 0x20, 0x9b, 0xd5, + 0x5e, 0x64, 0xf3, 0xda, 0x40, 0x64, 0xe3, 0x49, 0xf0, 0xa1, 0xcd, 0xfd, 0x2e, 0x68, 0x73, 0x75, + 0x00, 0xb4, 0xf1, 0x0b, 0xc0, 0xb1, 0x8d, 0x36, 0x18, 0xdb, 0x2c, 0x8d, 0x8a, 0x6d, 0x3c, 0xc1, + 0x91, 0xe0, 0x66, 0x3b, 0x1a, 0xdc, 0xdc, 0x1c, 0x71, 0x97, 0xbd, 0x07, 0xdd, 0x54, 0xa3, 0xd0, + 0xcd, 0xf5, 0x21, 0xe8, 0x26, 0x38, 0x87, 0x78, 0xf0, 0xa6, 0x1a, 0x05, 0x6f, 0xae, 0x0f, 0x81, + 0x37, 0xbe, 0xa4, 0x00, 0xbe, 0xa9, 0xf7, 0xc3, 0x37, 0xaf, 0x8f, 0x80, 0x6f, 0x7c, 0xe7, 0xa5, + 0x0b, 0xe0, 0x7c, 0xd8, 0x0d, 0x70, 0x4a, 0x83, 0x00, 0x8e, 0x3f, 0x22, 0x5d, 0x84, 0xb3, 0x1d, + 0x8d, 0x70, 0x6e, 0x0e, 0x45, 0x38, 0x41, 0x23, 0x19, 0x80, 0x38, 0x9b, 0x91, 0x10, 0xe7, 0xc6, + 0x30, 0x88, 0xe3, 0x1b, 0xc9, 0x20, 0xc6, 0xf9, 0xb0, 0x1b, 0xe3, 0x94, 0x06, 0x61, 0x1c, 0xbf, + 0x72, 0x2e, 0xc8, 0xa9, 0x46, 0x81, 0x9c, 0xeb, 0x43, 0x40, 0x8e, 0xdf, 0x78, 0x01, 0x94, 0xa3, + 0x0c, 0x44, 0x39, 0x6f, 0x8e, 0x88, 0x72, 0xba, 0x0c, 0x57, 0x18, 0xe6, 0x54, 0xa3, 0x60, 0xce, + 0xf5, 0x21, 0x30, 0x27, 0x50, 0x58, 0x1f, 0xe7, 0x6c, 0x47, 0xe3, 0x9c, 0x9b, 0x43, 0x71, 0x4e, + 0xd7, 0x68, 0x72, 0x81, 0xce, 0x66, 0x24, 0xd0, 0xb9, 0x31, 0x0c, 0xe8, 0x74, 0x4d, 0x7c, 0xdc, + 0x39, 0xf8, 0xab, 0xa3, 0x23, 0x9d, 0xf7, 0xce, 0x8f, 0x74, 0xbc, 0x77, 0xc6, 0x02, 0x75, 0x36, + 0x52, 0x99, 0x57, 0xc4, 0x57, 0x4b, 0xbf, 0x1c, 0x87, 0x74, 0xd5, 0x8b, 0x85, 0xf1, 0x4b, 0x29, + 0xbc, 0xd4, 0x75, 0x49, 0x6b, 0x64, 0xc4, 0x52, 0xbb, 0x37, 0xfc, 0x6a, 0xbc, 0xde, 0x6b, 0xdb, + 0x38, 0xeb, 0x4b, 0x9c, 0x42, 0x46, 0xef, 0xc0, 0x64, 0xc7, 0xc6, 0x96, 0xdc, 0xb6, 0x74, 0xd3, + 0xd2, 0x1d, 0x76, 0xa2, 0x43, 0x58, 0x15, 0xbf, 0x3a, 0x5b, 0xc8, 0xef, 0xda, 0xd8, 0xda, 0xe1, + 0x74, 0x29, 0xdf, 0x09, 0x3c, 0xb9, 0x9f, 0x90, 0x1a, 0x1f, 0xfd, 0x13, 0x52, 0x4f, 0x40, 0xb4, + 0xb0, 0xa2, 0x85, 0x3c, 0x10, 0x76, 0xcd, 0x50, 0x74, 0x9f, 0xa1, 0x87, 0xa5, 0xdc, 0x9c, 0xf4, + 0xba, 0xa1, 0x29, 0x2b, 0x4c, 0x44, 0x77, 0xe0, 0x42, 0x4b, 0x39, 0xa1, 0x51, 0x8f, 0xb2, 0xeb, + 0xd4, 0xd1, 0x48, 0x46, 0xf6, 0x75, 0x26, 0xd4, 0x52, 0x4e, 0xe8, 0xf7, 0xa8, 0x58, 0x12, 0xfd, + 0x80, 0xc4, 0x75, 0x28, 0x68, 0xba, 0xed, 0xe8, 0x86, 0xea, 0xf0, 0x9b, 0x68, 0xd9, 0xd5, 0xae, + 0x93, 0x2e, 0x95, 0x5d, 0x37, 0x7b, 0x1b, 0xa6, 0x79, 0x50, 0x7c, 0x60, 0x8b, 0x10, 0x78, 0xa4, + 0x19, 0x4d, 0xf0, 0x76, 0x05, 0x51, 0x19, 0xa6, 0x1a, 0x8a, 0x83, 0x8f, 0x95, 0x53, 0xd9, 0x3d, + 0x51, 0x95, 0xa3, 0x17, 0x39, 0x5e, 0x79, 0x71, 0xb6, 0x30, 0xf9, 0x90, 0x25, 0xf5, 0x1c, 0xac, + 0x9a, 0x6c, 0x04, 0x12, 0x34, 0x74, 0x13, 0xa6, 0x14, 0xfb, 0xd4, 0x50, 0xa9, 0x7a, 0xb0, 0x61, + 0x77, 0x6c, 0x0a, 0x29, 0x32, 0x52, 0x81, 0x92, 0xcb, 0x2e, 0x15, 0x5d, 0x85, 0x3c, 0x8f, 0x18, + 0x67, 0x1f, 0xb5, 0x99, 0xa2, 0x55, 0xe5, 0xdf, 0x58, 0xa0, 0xdf, 0xb5, 0x41, 0xf7, 0x61, 0x8e, + 0xdf, 0x3d, 0x7f, 0xac, 0x58, 0x9a, 0x4c, 0xb5, 0xee, 0xf7, 0x4f, 0x91, 0x8a, 0xbd, 0xc4, 0xee, + 0x9a, 0x27, 0x19, 0x88, 0xaa, 0x83, 0x17, 0xb5, 0x4e, 0x88, 0x99, 0x8d, 0x54, 0x26, 0x2f, 0x4e, + 0x6e, 0xa4, 0x32, 0x05, 0x71, 0xaa, 0xf4, 0x77, 0x04, 0xc8, 0x87, 0x4e, 0xa1, 0xdc, 0xef, 0xda, + 0x04, 0xbe, 0x1c, 0x0d, 0x9d, 0xfa, 0xc5, 0x8d, 0x65, 0x78, 0x53, 0xb9, 0x51, 0x73, 0x0b, 0xfd, + 0x5d, 0x6f, 0xba, 0x90, 0xe0, 0x46, 0x1e, 0xb8, 0x6c, 0xef, 0xa7, 0xfe, 0xde, 0xe7, 0x0b, 0x63, + 0xa5, 0x2f, 0x52, 0x30, 0x19, 0x3e, 0x6d, 0xb2, 0xde, 0x55, 0xae, 0x28, 0xd3, 0x16, 0xe2, 0x58, + 0x1a, 0x70, 0x3b, 0x5f, 0xd6, 0xbf, 0x3c, 0x9e, 0x15, 0x73, 0x71, 0xc0, 0x56, 0x77, 0xb0, 0x9c, + 0x3e, 0xe3, 0xdc, 0x5f, 0x4b, 0x7a, 0x26, 0x62, 0x09, 0xc6, 0xe9, 0x35, 0x30, 0xbc, 0x68, 0x51, + 0x07, 0x99, 0x2b, 0x24, 0x5d, 0x62, 0xd9, 0x88, 0x49, 0xa9, 0xbf, 0xd4, 0x0d, 0x6c, 0xfe, 0x25, + 0x16, 0xe7, 0xff, 0xca, 0x1b, 0xbf, 0x88, 0x6f, 0xfc, 0x7c, 0x17, 0xf1, 0xb1, 0x4d, 0xe9, 0x66, + 0x93, 0x99, 0x6b, 0x36, 0xa8, 0xd2, 0x3d, 0xa7, 0x85, 0xa9, 0x08, 0xfe, 0xf1, 0xbd, 0x25, 0x89, + 0x7f, 0x7c, 0x2f, 0x10, 0xc8, 0x58, 0xf0, 0x44, 0xb0, 0x11, 0x58, 0x76, 0x67, 0x69, 0xf6, 0x45, + 0xb8, 0x89, 0x91, 0xbf, 0x08, 0x07, 0xde, 0x26, 0xbd, 0xcd, 0x62, 0x66, 0x79, 0x7f, 0xf9, 0xf7, + 0x02, 0x8f, 0x68, 0x78, 0x64, 0x9a, 0x87, 0x1d, 0x2f, 0x16, 0x72, 0x2e, 0x78, 0x0b, 0x5e, 0xe6, + 0xab, 0xb3, 0x85, 0x94, 0xe4, 0x5d, 0x83, 0x17, 0x65, 0xb2, 0x12, 0x5f, 0xcf, 0x64, 0x5d, 0x85, + 0x7c, 0xdb, 0xc2, 0xfb, 0xd8, 0x51, 0x0f, 0x64, 0xa3, 0xd3, 0xe2, 0x07, 0x1e, 0x72, 0x2e, 0x6d, + 0xbb, 0xd3, 0x42, 0xaf, 0x83, 0xe8, 0x65, 0xe1, 0xe0, 0x90, 0x9f, 0xf7, 0x9e, 0x72, 0xe9, 0x1c, + 0x4a, 0x96, 0x7e, 0x2d, 0xc0, 0x4c, 0xa8, 0x4e, 0x7c, 0x24, 0x6c, 0x40, 0x4e, 0xf3, 0x26, 0x09, + 0xbb, 0x28, 0x9c, 0x33, 0x26, 0x30, 0xc8, 0x8c, 0x64, 0xb8, 0xe8, 0xbe, 0x96, 0x5e, 0x8d, 0xee, + 0x8b, 0x4d, 0x9c, 0x53, 0xec, 0x05, 0x5f, 0xce, 0x5a, 0xe0, 0x05, 0xde, 0xd0, 0x48, 0x8e, 0x34, + 0x34, 0x4a, 0x3f, 0x10, 0x40, 0xa4, 0x2f, 0x78, 0x80, 0xb1, 0x16, 0x8b, 0x4d, 0x72, 0x23, 0x6e, + 0x13, 0xa3, 0x1f, 0x69, 0x08, 0x7d, 0xda, 0x21, 0x19, 0xfe, 0xb4, 0x43, 0xe9, 0x73, 0x01, 0x0a, + 0x5e, 0x09, 0xd9, 0x47, 0xcd, 0x06, 0x5c, 0xb6, 0xf8, 0x72, 0x9f, 0xf2, 0x72, 0xef, 0x85, 0x18, + 0xe9, 0x3b, 0x6b, 0xc1, 0x7b, 0x21, 0xd8, 0x67, 0xa7, 0xfe, 0x81, 0xdb, 0x73, 0x48, 0x11, 0xcb, + 0xfe, 0x99, 0xff, 0x97, 0x38, 0xdd, 0x21, 0xd1, 0x6f, 0x43, 0x9a, 0xcd, 0x23, 0x76, 0x21, 0xc7, + 0x48, 0xc6, 0x0a, 0xf1, 0x38, 0x1e, 0xe0, 0xcb, 0x48, 0x5a, 0xbd, 0x46, 0xbf, 0x1a, 0xc9, 0xfe, + 0xdb, 0xa5, 0x07, 0x01, 0x05, 0xd2, 0xc6, 0x27, 0x5a, 0x1a, 0xc9, 0x80, 0xba, 0x5a, 0x62, 0x7d, + 0xe5, 0xc7, 0xc1, 0x96, 0xa8, 0x1c, 0x11, 0xf8, 0x70, 0x0f, 0x92, 0x47, 0x4a, 0x73, 0x50, 0xfc, + 0x52, 0xa8, 0xe5, 0x24, 0x92, 0x1b, 0x3d, 0x08, 0x5d, 0x95, 0x90, 0xe8, 0xef, 0xea, 0xf6, 0xaa, + 0x34, 0x74, 0xa5, 0xc2, 0xbb, 0xe1, 0xbe, 0x3e, 0xf0, 0xf5, 0xc1, 0x4e, 0xff, 0x7e, 0xea, 0xcb, + 0xcf, 0x17, 0x84, 0xd2, 0xbb, 0x70, 0xf9, 0xa1, 0x69, 0xdb, 0x7a, 0x9b, 0xc0, 0x1b, 0x3a, 0x80, + 0x88, 0xed, 0xf6, 0x2c, 0x59, 0xa6, 0x4d, 0x81, 0xae, 0xc1, 0x46, 0x7c, 0x56, 0xf2, 0x9e, 0x4b, + 0xff, 0x52, 0x80, 0x4b, 0xbd, 0x9c, 0x4c, 0x21, 0x51, 0xa7, 0xc7, 0x26, 0x54, 0xd3, 0xbf, 0x3a, + 0x6c, 0x78, 0xc7, 0x72, 0xb3, 0x13, 0x37, 0x86, 0xbf, 0x53, 0x6e, 0x29, 0x74, 0xa4, 0xf3, 0x13, + 0xae, 0x05, 0x4e, 0xde, 0x62, 0x54, 0x7f, 0xd0, 0xa7, 0x46, 0x1a, 0xf4, 0xb7, 0x6b, 0x30, 0x13, + 0x61, 0x5f, 0x51, 0x01, 0x20, 0xf0, 0xa9, 0x0b, 0xfe, 0x89, 0xcd, 0x95, 0x35, 0x79, 0x77, 0xbb, + 0xfc, 0x78, 0x6b, 0x6b, 0xbd, 0x5e, 0xaf, 0xac, 0x89, 0x02, 0x12, 0x21, 0x1f, 0xfa, 0x50, 0x46, + 0x82, 0x7d, 0x74, 0xf3, 0xf6, 0x5f, 0x00, 0xf0, 0xbf, 0xbf, 0x43, 0x64, 0x6d, 0x56, 0x3e, 0x96, + 0x9f, 0xae, 0x3c, 0xda, 0xad, 0xd4, 0xc4, 0x31, 0x84, 0xa0, 0xb0, 0xba, 0x52, 0x2f, 0x57, 0x65, + 0xa9, 0x52, 0xdb, 0x79, 0xbc, 0x5d, 0xab, 0xb8, 0x1f, 0xeb, 0xbc, 0xbd, 0x06, 0xf9, 0xe0, 0x75, + 0x1a, 0x68, 0x06, 0xa6, 0xca, 0xd5, 0x4a, 0x79, 0x53, 0x7e, 0xba, 0xbe, 0x22, 0x3f, 0xd9, 0xad, + 0xec, 0x56, 0xc4, 0x31, 0x5a, 0x34, 0x4a, 0x7c, 0xb0, 0xfb, 0xe8, 0x91, 0x28, 0xa0, 0x29, 0xc8, + 0xb1, 0x67, 0xfa, 0x51, 0x0d, 0x31, 0x71, 0x7b, 0x0b, 0x72, 0x81, 0xcb, 0x34, 0xc9, 0xeb, 0x76, + 0x76, 0x6b, 0x55, 0xb9, 0xbe, 0xbe, 0x55, 0xa9, 0xd5, 0x57, 0xb6, 0x76, 0x98, 0x0c, 0x4a, 0x5b, + 0x59, 0x7d, 0x2c, 0xd5, 0x45, 0xc1, 0x7b, 0xae, 0x3f, 0xde, 0x2d, 0x57, 0xdd, 0x6a, 0x94, 0x52, + 0x99, 0xa4, 0x98, 0xbc, 0xfd, 0x07, 0x02, 0x5c, 0xea, 0x73, 0xa9, 0x04, 0xca, 0xc1, 0xc4, 0xae, + 0x41, 0x6f, 0x13, 0x14, 0xc7, 0xd0, 0x64, 0xe0, 0x5e, 0x09, 0x51, 0x40, 0x19, 0x76, 0xa6, 0x5f, + 0x4c, 0xa0, 0x34, 0x24, 0x6a, 0xf7, 0xc4, 0x24, 0x29, 0x69, 0xe0, 0x5a, 0x06, 0x31, 0x85, 0xb2, + 0xfc, 0x54, 0xb9, 0x38, 0x8e, 0xf2, 0xfe, 0xb1, 0x6e, 0x31, 0x4d, 0x44, 0x79, 0x07, 0xa3, 0xc5, + 0x89, 0xdb, 0x57, 0x21, 0x70, 0xc8, 0x14, 0x01, 0xa4, 0x1f, 0x29, 0x0e, 0xb6, 0x1d, 0x71, 0x0c, + 0x4d, 0x40, 0x72, 0xa5, 0xd9, 0x14, 0x85, 0xbb, 0x7f, 0x3d, 0x09, 0x19, 0xf7, 0xb3, 0x11, 0xe8, + 0x11, 0x8c, 0xb3, 0xd5, 0xb0, 0x85, 0xfe, 0x9e, 0x1a, 0xed, 0xdb, 0x73, 0x8b, 0xc3, 0x5c, 0xb9, + 0xd2, 0x18, 0xfa, 0x8b, 0x90, 0x0b, 0xcc, 0x85, 0xa8, 0x2f, 0xa2, 0x0f, 0xcd, 0xff, 0x73, 0x37, + 0x86, 0x65, 0xf3, 0xe4, 0x7f, 0x04, 0x59, 0x6f, 0x6c, 0xa2, 0xd7, 0x06, 0x8d, 0x5c, 0x57, 0xf6, + 0xe0, 0xe1, 0x4d, 0x06, 0x5f, 0x69, 0xec, 0x2d, 0x01, 0x59, 0x80, 0x7a, 0xc7, 0x26, 0x8a, 0xda, + 0x24, 0xed, 0x3b, 0xf8, 0xe7, 0x6e, 0x8f, 0x94, 0xdb, 0x7b, 0xe7, 0xea, 0xeb, 0x5f, 0xfe, 0x7c, + 0x7e, 0xec, 0xcb, 0x17, 0xf3, 0xc2, 0x4f, 0x5e, 0xcc, 0x0b, 0x3f, 0x7b, 0x31, 0x2f, 0xfc, 0xd7, + 0x17, 0xf3, 0xc2, 0xdf, 0xfa, 0xc5, 0xfc, 0xd8, 0x4f, 0x7e, 0x31, 0x3f, 0xf6, 0xb3, 0x5f, 0xcc, + 0x8f, 0x7d, 0x32, 0xc1, 0xc5, 0xec, 0xa5, 0xe9, 0x67, 0x90, 0xef, 0xfd, 0xdf, 0x00, 0x00, 0x00, + 0xff, 0xff, 0x75, 0xf3, 0x7b, 0x5e, 0x29, 0x7a, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 351fd5ee1ca7..54e30347fbea 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 325370aa0370..80dda2723873 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()) @@ -3350,11 +3350,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: prevNanos - 1}} + if err := rw.ClearRange(start, end); err != nil { + return err + } + } } return nil diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 78350b42e3a2..8693cfa204b2 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -5016,17 +5016,29 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { } } -// readWriterReturningSeekLTTrackingIterator is used in a test to inject errors -// and ensure that SeekLT is returned an appropriate number of times. -type readWriterReturningSeekLTTrackingIterator struct { - it seekLTTrackingIterator +// trackingReadWriter is used in a test to track the number of times which +// various methods are called on it and its returned iterator. +type trackingReadWriter struct { + it seekLTTrackingIterator + clearCalled int + clearRangeCalled int ReadWriter } +func (trw *trackingReadWriter) Clear(key MVCCKey) error { + trw.clearCalled++ + return trw.ReadWriter.Clear(key) +} + +func (trw *trackingReadWriter) ClearRange(start, end MVCCKey) error { + trw.clearRangeCalled++ + return trw.ReadWriter.ClearRange(start, end) +} + // NewIterator injects a seekLTTrackingIterator over the engine's real iterator. -func (rw *readWriterReturningSeekLTTrackingIterator) NewIterator(opts IterOptions) Iterator { - rw.it.Iterator = rw.ReadWriter.NewIterator(opts) - return &rw.it +func (trw *trackingReadWriter) NewIterator(opts IterOptions) Iterator { + trw.it.Iterator = trw.ReadWriter.NewIterator(opts) + return &trw.it } // seekLTTrackingIterator is used to determine the number of times seekLT is @@ -5047,10 +5059,11 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { defer leaktest.AfterTest(t)() type testCaseKey struct { - key string - timestamps []int - gcTimestamp int - expSeekLT bool + key string + timestamps []int + gcTimestamp int + expSeekLT bool + useClearRange bool } type testCase struct { name string @@ -5073,6 +5086,7 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { runTestCase := func(t *testing.T, tc testCase, engine Engine) { ctx := context.Background() ms := &enginepb.MVCCStats{} + for _, key := range tc.keys { for _, seconds := range key.timestamps { val := roachpb.MakeValueFromBytes(bytes) @@ -5089,11 +5103,28 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { var keys []roachpb.GCRequest_GCKey var expectedSeekLTs int + var expectedClears int + var expectedClearRanges int for _, key := range tc.keys { keys = append(keys, roachpb.GCRequest_GCKey{ - Key: roachpb.Key(key.key), - Timestamp: toHLC(key.gcTimestamp), + Key: roachpb.Key(key.key), + Timestamp: toHLC(key.gcTimestamp), + UseClearRange: key.useClearRange, }) + + var garbageForThisKey int + for _, ts := range key.timestamps { + if ts <= key.gcTimestamp { + garbageForThisKey++ + } + } + if garbageForThisKey > 0 { + if key.useClearRange { + expectedClearRanges++ + } else { + expectedClears += garbageForThisKey + } + } if supportsPrev && key.expSeekLT { expectedSeekLTs++ } @@ -5101,10 +5132,12 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { batch := engine.NewBatch() defer batch.Close() - rw := readWriterReturningSeekLTTrackingIterator{ReadWriter: batch} + rw := trackingReadWriter{ReadWriter: batch} require.NoError(t, MVCCGarbageCollect(ctx, &rw, ms, keys, toHLC(10))) require.Equal(t, expectedSeekLTs, rw.it.seekLTCalled) + require.Equal(t, expectedClears, rw.clearCalled) + require.Equal(t, expectedClearRanges, rw.clearRangeCalled) } cases := []testCase{ { @@ -5132,10 +5165,11 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { expSeekLT: true, }, { - key: "e", - timestamps: []int{1, 2, 3, 4, 5, 6, 7, 8, 9}, - gcTimestamp: 1, - expSeekLT: true, + key: "e", + timestamps: []int{1, 2, 3, 4, 5, 6, 7, 8, 9}, + gcTimestamp: 1, + expSeekLT: true, + useClearRange: true, }, { key: "f",