From 02c3c16343fa593d28f1c9c86dea39e618de5e47 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Thu, 23 Jan 2020 19:46:00 -0500 Subject: [PATCH] storage: leave intents behind after blind-writes experiencing write-too-old Before this patch, any write running into a write-too-old condition resulted in a WriteTooOldError being returned by the server. Returning an error implies that no intents are left behind. This is unfortunate; we'd like to leave intents (or, in the future, other types of locks) behind so keep away other transactions. We've observed this resulting in the starvation of a class of transactions in a user's workload. This patch makes it so that blind writes (i.e. Puts - used by UPDATE, not CPuts) don't return WriteTooOldErrors any more. Instead, they return the a txn proto with the WriteTooOld flag set. This is the behavior they had before #38668. This patch retains the goal of #38668, however: the client now eagerly refreshes the transactions when it sees a WriteTooOld flag, and if the refresh succeeds, it returns a WriteTooOldError to the higher layers (SQL), allowing for automatic retries where applicable. Unfortunately, CPuts (used by INSERT) continue to return WriteTooOldErrors without leaving locks behind. Dealing with them requires more tenderness because they imply a read, and the timestamp of a read cannot be bumped as easily as that of a write. Touches #44653 Release note (SQL change): UPDATEs returning a serialization failure error (code 40001) now leave behind a lock, helping the transaction succeed if it retries. This prevents starvation of transactions whose UPDATEs are prone to conflicts. --- c-deps/libroach/protos/roachpb/api.pb.cc | 55 +- c-deps/libroach/protos/roachpb/api.pb.h | 31 +- pkg/kv/dist_sender.go | 4 + pkg/kv/dist_sender_server_test.go | 69 +- pkg/kv/txn_interceptor_span_refresher.go | 60 +- pkg/kv/txn_test.go | 57 +- pkg/roachpb/api.pb.go | 1196 ++++++++--------- pkg/roachpb/api.proto | 16 +- pkg/roachpb/batch.go | 4 - .../intent_resolver_integration_test.go | 4 +- pkg/storage/replica_evaluate.go | 98 +- pkg/storage/replica_test.go | 36 +- pkg/storage/replica_write.go | 13 +- pkg/storage/store_test.go | 3 +- 14 files changed, 771 insertions(+), 875 deletions(-) diff --git a/c-deps/libroach/protos/roachpb/api.pb.cc b/c-deps/libroach/protos/roachpb/api.pb.cc index b727ef7cb038..4ce6686c18fa 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.cc +++ b/c-deps/libroach/protos/roachpb/api.pb.cc @@ -34826,7 +34826,6 @@ const int Header::kDistinctSpansFieldNumber; const int Header::kReturnRangeInfoFieldNumber; const int Header::kGatewayNodeIdFieldNumber; const int Header::kAsyncConsensusFieldNumber; -const int Header::kDeferWriteTooOldErrorFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 Header::Header() @@ -34856,15 +34855,15 @@ Header::Header(const Header& from) txn_ = NULL; } ::memcpy(&range_id_, &from.range_id_, - static_cast(reinterpret_cast(&defer_write_too_old_error_) - - reinterpret_cast(&range_id_)) + sizeof(defer_write_too_old_error_)); + static_cast(reinterpret_cast(&gateway_node_id_) - + reinterpret_cast(&range_id_)) + sizeof(gateway_node_id_)); // @@protoc_insertion_point(copy_constructor:cockroach.roachpb.Header) } void Header::SharedCtor() { ::memset(×tamp_, 0, static_cast( - reinterpret_cast(&defer_write_too_old_error_) - - reinterpret_cast(×tamp_)) + sizeof(defer_write_too_old_error_)); + reinterpret_cast(&gateway_node_id_) - + reinterpret_cast(×tamp_)) + sizeof(gateway_node_id_)); } Header::~Header() { @@ -34906,8 +34905,8 @@ void Header::Clear() { } txn_ = NULL; ::memset(&range_id_, 0, static_cast( - reinterpret_cast(&defer_write_too_old_error_) - - reinterpret_cast(&range_id_)) + sizeof(defer_write_too_old_error_)); + reinterpret_cast(&gateway_node_id_) - + reinterpret_cast(&range_id_)) + sizeof(gateway_node_id_)); _internal_metadata_.Clear(); } @@ -35071,20 +35070,6 @@ bool Header::MergePartialFromCodedStream( break; } - // bool defer_write_too_old_error = 14; - case 14: { - if (static_cast< ::google::protobuf::uint8>(tag) == - static_cast< ::google::protobuf::uint8>(112u /* 112 & 0xFF */)) { - - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( - input, &defer_write_too_old_error_))); - } else { - goto handle_unusual; - } - break; - } - default: { handle_unusual: if (tag == 0) { @@ -35165,11 +35150,6 @@ void Header::SerializeWithCachedSizes( ::google::protobuf::internal::WireFormatLite::WriteBool(13, this->async_consensus(), output); } - // bool defer_write_too_old_error = 14; - if (this->defer_write_too_old_error() != 0) { - ::google::protobuf::internal::WireFormatLite::WriteBool(14, this->defer_write_too_old_error(), 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.Header) @@ -35223,12 +35203,6 @@ size_t Header::ByteSizeLong() const { ::google::protobuf::internal::WireFormatLite::EnumSize(this->read_consistency()); } - if (this->gateway_node_id() != 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::Int32Size( - this->gateway_node_id()); - } - // bool distinct_spans = 9; if (this->distinct_spans() != 0) { total_size += 1 + 1; @@ -35244,9 +35218,10 @@ size_t Header::ByteSizeLong() const { total_size += 1 + 1; } - // bool defer_write_too_old_error = 14; - if (this->defer_write_too_old_error() != 0) { - total_size += 1 + 1; + if (this->gateway_node_id() != 0) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::Int32Size( + this->gateway_node_id()); } int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); @@ -35287,9 +35262,6 @@ void Header::MergeFrom(const Header& from) { if (from.read_consistency() != 0) { set_read_consistency(from.read_consistency()); } - if (from.gateway_node_id() != 0) { - set_gateway_node_id(from.gateway_node_id()); - } if (from.distinct_spans() != 0) { set_distinct_spans(from.distinct_spans()); } @@ -35299,8 +35271,8 @@ void Header::MergeFrom(const Header& from) { if (from.async_consensus() != 0) { set_async_consensus(from.async_consensus()); } - if (from.defer_write_too_old_error() != 0) { - set_defer_write_too_old_error(from.defer_write_too_old_error()); + if (from.gateway_node_id() != 0) { + set_gateway_node_id(from.gateway_node_id()); } } @@ -35328,11 +35300,10 @@ void Header::InternalSwap(Header* other) { swap(user_priority_, other->user_priority_); swap(max_span_request_keys_, other->max_span_request_keys_); swap(read_consistency_, other->read_consistency_); - swap(gateway_node_id_, other->gateway_node_id_); swap(distinct_spans_, other->distinct_spans_); swap(return_range_info_, other->return_range_info_); swap(async_consensus_, other->async_consensus_); - swap(defer_write_too_old_error_, other->defer_write_too_old_error_); + swap(gateway_node_id_, other->gateway_node_id_); _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 8539518424cf..559b3d242cc2 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.h +++ b/c-deps/libroach/protos/roachpb/api.pb.h @@ -15543,11 +15543,6 @@ class Header : public ::google::protobuf::MessageLite /* @@protoc_insertion_poin ::cockroach::roachpb::ReadConsistencyType read_consistency() const; void set_read_consistency(::cockroach::roachpb::ReadConsistencyType value); - void clear_gateway_node_id(); - static const int kGatewayNodeIdFieldNumber = 11; - ::google::protobuf::int32 gateway_node_id() const; - void set_gateway_node_id(::google::protobuf::int32 value); - // bool distinct_spans = 9; void clear_distinct_spans(); static const int kDistinctSpansFieldNumber = 9; @@ -15566,11 +15561,10 @@ class Header : public ::google::protobuf::MessageLite /* @@protoc_insertion_poin bool async_consensus() const; void set_async_consensus(bool value); - // bool defer_write_too_old_error = 14; - void clear_defer_write_too_old_error(); - static const int kDeferWriteTooOldErrorFieldNumber = 14; - bool defer_write_too_old_error() const; - void set_defer_write_too_old_error(bool value); + void clear_gateway_node_id(); + static const int kGatewayNodeIdFieldNumber = 11; + ::google::protobuf::int32 gateway_node_id() const; + void set_gateway_node_id(::google::protobuf::int32 value); // @@protoc_insertion_point(class_scope:cockroach.roachpb.Header) private: @@ -15583,11 +15577,10 @@ class Header : public ::google::protobuf::MessageLite /* @@protoc_insertion_poin double user_priority_; ::google::protobuf::int64 max_span_request_keys_; int read_consistency_; - ::google::protobuf::int32 gateway_node_id_; bool distinct_spans_; bool return_range_info_; bool async_consensus_; - bool defer_write_too_old_error_; + ::google::protobuf::int32 gateway_node_id_; mutable ::google::protobuf::internal::CachedSize _cached_size_; friend struct ::protobuf_roachpb_2fapi_2eproto::TableStruct; }; @@ -33012,20 +33005,6 @@ inline void Header::set_async_consensus(bool value) { // @@protoc_insertion_point(field_set:cockroach.roachpb.Header.async_consensus) } -// bool defer_write_too_old_error = 14; -inline void Header::clear_defer_write_too_old_error() { - defer_write_too_old_error_ = false; -} -inline bool Header::defer_write_too_old_error() const { - // @@protoc_insertion_point(field_get:cockroach.roachpb.Header.defer_write_too_old_error) - return defer_write_too_old_error_; -} -inline void Header::set_defer_write_too_old_error(bool value) { - - defer_write_too_old_error_ = value; - // @@protoc_insertion_point(field_set:cockroach.roachpb.Header.defer_write_too_old_error) -} - // ------------------------------------------------------------------- // BatchRequest diff --git a/pkg/kv/dist_sender.go b/pkg/kv/dist_sender.go index 70769f2d17fd..8a061ed547bb 100644 --- a/pkg/kv/dist_sender.go +++ b/pkg/kv/dist_sender.go @@ -648,6 +648,10 @@ func splitBatchAndCheckForRefreshSpans( // succeeded. Where possible, the caller may be able to update spans // encountered in the transaction and retry just the EndTxn request to // avoid client-side serializable txn retries. +// TODO(andrei,nvanbenschoten): Get rid of this returning both error and result, +// and change the span txnSpanRefresher to not look for partial results. It's +// probably better for the span refresher to refresh less (i.e. don't refresh +// the partial successes) and retry the whole batch. func (ds *DistSender) Send( ctx context.Context, ba roachpb.BatchRequest, ) (*roachpb.BatchResponse, *roachpb.Error) { diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index b4e816bb8735..1ed852e3c377 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1772,19 +1772,6 @@ func TestTxnCoordSenderRetries(t *testing.T) { }, txnCoordRetry: true, }, - { - name: "deferred write too old with put", - afterTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "put") - }, - retryable: func(ctx context.Context, txn *client.Txn) error { - b := txn.NewBatch() - b.Header.DeferWriteTooOldError = true - b.Put("a", "put") - return txn.Run(ctx, b) - }, - // This trivially succeeds as there are no refresh spans. - }, { name: "write too old with put timestamp leaked", afterTxnStart: func(ctx context.Context, db *client.DB) error { @@ -2149,23 +2136,6 @@ func TestTxnCoordSenderRetries(t *testing.T) { }, txnCoordRetry: true, }, - { - name: "multi-range batch with deferred write too old", - afterTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "c", "value") - }, - retryable: func(ctx context.Context, txn *client.Txn) error { - b := txn.NewBatch() - b.Header.DeferWriteTooOldError = true - b.Put("a", "put") - b.Put("c", "put") - // Both sub-batches will succeed, but the Put(a) will return a pushed - // timestamp, which is turned into a retriable error by the txnCommitter - // interceptor (because it's concurrent with writing the STAGING record). - return txn.CommitInBatch(ctx, b) - }, - txnCoordRetry: true, - }, { name: "multi-range batch with write too old and failed cput", beforeTxnStart: func(ctx context.Context, db *client.DB) error { @@ -2200,42 +2170,6 @@ func TestTxnCoordSenderRetries(t *testing.T) { // We expect the request to succeed after a server-side retry. txnCoordRetry: false, }, - { - name: "multi-range batch with deferred write too old and failed cput", - beforeTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "orig") - }, - afterTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "value") - }, - retryable: func(ctx context.Context, txn *client.Txn) error { - b := txn.NewBatch() - b.Header.DeferWriteTooOldError = true - b.CPut("a", "cput", strToValue("orig")) - b.Put("c", "put") - return txn.CommitInBatch(ctx, b) - }, - txnCoordRetry: false, // non-matching value means we fail txn coord retry - expFailure: "unexpected value", // the failure we get is a condition failed error - }, - { - name: "multi-range batch with deferred write too old and successful cput", - beforeTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "orig") - }, - afterTxnStart: func(ctx context.Context, db *client.DB) error { - return db.Put(ctx, "a", "orig") - }, - retryable: func(ctx context.Context, txn *client.Txn) error { - b := txn.NewBatch() - b.Header.DeferWriteTooOldError = true - b.CPut("a", "cput", strToValue("orig")) - b.Put("c", "put") - return txn.CommitInBatch(ctx, b) - }, - // We expect the request to succeed after a server-side retry. - txnCoordRetry: false, - }, { name: "cput within uncertainty interval", beforeTxnStart: func(ctx context.Context, db *client.DB) error { @@ -2470,7 +2404,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { metrics = txn.Sender().(*kv.TxnCoordSender).TxnCoordSenderFactory.Metrics() lastAutoRetries = metrics.AutoRetries.Count() - return tc.retryable(ctx, txn) + err := tc.retryable(ctx, txn) + return err }); err != nil { if len(tc.expFailure) == 0 || !testutils.IsError(err, tc.expFailure) { t.Fatal(err) diff --git a/pkg/kv/txn_interceptor_span_refresher.go b/pkg/kv/txn_interceptor_span_refresher.go index 828cdc990ef9..3f4148a4070a 100644 --- a/pkg/kv/txn_interceptor_span_refresher.go +++ b/pkg/kv/txn_interceptor_span_refresher.go @@ -228,9 +228,57 @@ func (sr *txnSpanRefresher) sendLockedWithRefreshAttempts( ctx context.Context, ba roachpb.BatchRequest, maxRefreshAttempts int, ) (_ *roachpb.BatchResponse, _ *roachpb.Error, largestRefreshTS hlc.Timestamp) { br, pErr := sr.sendHelper(ctx, ba) + if pErr == nil && br.Txn.WriteTooOld { + // If we got a response with the WriteTooOld flag set, then we pretend that + // we got a WriteTooOldError, which will cause us to attempt to refresh and + // propagate the error if we failed. When it can, the server prefers to + // return the WriteTooOld flag, rather than a WriteTooOldError because, in + // the former case, it can leave intents behind. We like refreshing eagerly + // when the WriteTooOld flag is set because it's likely that the refresh + // will fail (if we previously read the key that's now causing a WTO, then + // the refresh will surely fail). + // TODO(andrei): Implement a more discerning policy based on whether we've + // read that key before. + // + // If the refresh fails, we could continue running the transaction even + // though it will not be able to commit, in order for it to lay down more + // intents. Not doing so, though, gives the SQL a chance to auto-retry. + // TODO(andrei): Implement a more discerning policy based on whether + // auto-retries are still possible. + // + // For the refresh, we have two options: either refresh everything read + // *before* this batch, and then retry to batch, or refresh the current + // batch's reads too and then, if successful, there'd be nothing to refresh. + // We take the former option by setting br = nil below to minimized the + // chances that the refresh fails. + bumpedTxn := br.Txn.Clone() + bumpedTxn.WriteTooOld = false + bumpedTxn.ReadTimestamp = bumpedTxn.WriteTimestamp + pErr = roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(roachpb.RETRY_WRITE_TOO_OLD, ""), + bumpedTxn) + br = nil + } if pErr != nil && maxRefreshAttempts > 0 { br, pErr, largestRefreshTS = sr.maybeRetrySend(ctx, ba, br, pErr, maxRefreshAttempts) } + if pErr != nil { + // Don't confuse layers above with both a result and an error. This layer is + // the last one that benefits from looking at partial results. + br = nil + } else { + // Terminate the txn.WriteTooOld flag here. We failed to refresh it away, so + // turn it into a retriable error. + if br.Txn.WriteTooOld { + newTxn := br.Txn.Clone() + newTxn.WriteTooOld = false + newTxn.ReadTimestamp = newTxn.WriteTimestamp + pErr = roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(roachpb.RETRY_WRITE_TOO_OLD, ""), + newTxn) + br = nil + } + } return br, pErr, largestRefreshTS } @@ -256,9 +304,17 @@ func (sr *txnSpanRefresher) maybeRetrySend( // case is a batch split between everything up to but not including // the EndTxn. Requests up to the EndTxn succeed, but the EndTxn // fails with a retryable error. We want to retry only the EndTxn. + // TODO(andrei): This attempt to only retry part of the request is probably a bad idea. + // We try to refresh more (including the successful part of the current + // request) and retry less, but it'd probably be a better idea to refresh less + // and retry more - thereby trading the cost of some evaluation for a higher + // chance that the refresh succeeds. ba.UpdateTxn(retryTxn) retryBa := ba - if br != nil { + // If br came back with the WriteTooOld flag set, then we need to refresh the + // whole request; we don't know which part of the request encountered the + // write too old condition. + if br != nil && !br.Txn.WriteTooOld { doneBa := ba doneBa.Requests = ba.Requests[:len(br.Responses)] log.VEventf(ctx, 2, "collecting refresh spans after partial batch execution of %s", doneBa) @@ -273,7 +329,7 @@ func (sr *txnSpanRefresher) maybeRetrySend( // Try updating the txn spans so we can retry. if ok := sr.tryUpdatingTxnSpans(ctx, retryTxn); !ok { - return nil, pErr, hlc.Timestamp{} + return br, pErr, hlc.Timestamp{} } // We've refreshed all of the read spans successfully and set diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index 4d1755458158..d22af8e5f9e3 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -141,6 +141,7 @@ func TestLostUpdate(t *testing.T) { }) }() + firstAttempt := true if err := s.DB.Txn(context.TODO(), func(ctx context.Context, txn *client.Txn) error { // Issue a read to get initial value. gr, err := txn.Get(ctx, key) @@ -164,16 +165,14 @@ func TestLostUpdate(t *testing.T) { newVal = "oops!" } b := txn.NewBatch() - b.Header.DeferWriteTooOldError = true b.Put(key, newVal) - if err := txn.Run(ctx, b); err != nil { - t.Fatal(err) - } - // Verify that the WriteTooOld boolean is set on the txn. - proto := txn.TestingCloneTxn() - if (txn.Epoch() == 0) != proto.WriteTooOld { - t.Fatalf("expected write too old set (%t): got %t", (txn.Epoch() == 0), proto.WriteTooOld) + err = txn.Run(ctx, b) + if firstAttempt { + require.Error(t, err, "RETRY_WRITE_TOO_OLD") + firstAttempt = false + return err } + require.NoError(t, err) return nil }); err != nil { t.Fatal(err) @@ -648,3 +647,45 @@ func TestTxnCommitTimestampAdvancedByRefresh(t *testing.T) { }) require.NoError(t, err) } + +// Test that in some write too old situations (i.e. when the server returns the +// WriteTooOld flag set and then the client fails to refresh), intents are +// properly left behind. +func TestTxnLeavesIntentBehindAfterWriteTooOldError(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + s := createTestDB(t) + defer s.Stop() + + key := []byte("b") + + txn := s.DB.NewTxn(ctx, "test txn") + // Perform a Get so that the transaction can't refresh. + _, err := txn.Get(ctx, key) + require.NoError(t, err) + + // Another guy writes at a higher timestamp. + require.NoError(t, s.DB.Put(ctx, key, "newer value")) + + // Now we write and expect a WriteTooOld. + intentVal := []byte("test") + err = txn.Put(ctx, key, intentVal) + require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + require.Error(t, err, "WriteTooOld") + + // Check that the intent was left behind. + b := client.Batch{} + b.Header.ReadConsistency = roachpb.READ_UNCOMMITTED + b.Get(key) + require.NoError(t, s.DB.Run(ctx, &b)) + getResp := b.RawResponse().Responses[0].GetGet() + require.NotNil(t, getResp) + intent := getResp.IntentValue + require.NotNil(t, intent) + intentBytes, err := intent.GetBytes() + require.NoError(t, err) + require.Equal(t, intentVal, intentBytes) + + // Cleanup. + require.NoError(t, txn.Rollback(ctx)) +} diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index a44e67c8b9bf..5c4deaf7e1f8 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -71,7 +71,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{0} + return fileDescriptor_api_47826aa7f6400ef0, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -99,7 +99,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{1} + return fileDescriptor_api_47826aa7f6400ef0, []int{1} } type ChecksumMode int32 @@ -146,7 +146,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{2} + return fileDescriptor_api_47826aa7f6400ef0, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -177,7 +177,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{3} + return fileDescriptor_api_47826aa7f6400ef0, []int{3} } type ExternalStorageProvider int32 @@ -215,7 +215,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{4} + return fileDescriptor_api_47826aa7f6400ef0, []int{4} } type MVCCFilter int32 @@ -238,7 +238,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{5} + return fileDescriptor_api_47826aa7f6400ef0, []int{5} } type ResponseHeader_ResumeReason int32 @@ -264,7 +264,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{2, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -306,7 +306,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{26, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{26, 0} } // RangeInfo describes a range which executed a request. It contains @@ -320,7 +320,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{0} + return fileDescriptor_api_47826aa7f6400ef0, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -363,7 +363,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_0b609b42e437dfba, []int{1} + return fileDescriptor_api_47826aa7f6400ef0, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -425,7 +425,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_0b609b42e437dfba, []int{2} + return fileDescriptor_api_47826aa7f6400ef0, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -459,7 +459,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_0b609b42e437dfba, []int{3} + return fileDescriptor_api_47826aa7f6400ef0, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -497,7 +497,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_0b609b42e437dfba, []int{4} + return fileDescriptor_api_47826aa7f6400ef0, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -540,7 +540,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_0b609b42e437dfba, []int{5} + return fileDescriptor_api_47826aa7f6400ef0, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -574,7 +574,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_0b609b42e437dfba, []int{6} + return fileDescriptor_api_47826aa7f6400ef0, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -628,7 +628,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_0b609b42e437dfba, []int{7} + return fileDescriptor_api_47826aa7f6400ef0, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -663,7 +663,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_0b609b42e437dfba, []int{8} + return fileDescriptor_api_47826aa7f6400ef0, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -709,7 +709,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_0b609b42e437dfba, []int{9} + return fileDescriptor_api_47826aa7f6400ef0, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -743,7 +743,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_0b609b42e437dfba, []int{10} + return fileDescriptor_api_47826aa7f6400ef0, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -783,7 +783,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_0b609b42e437dfba, []int{11} + return fileDescriptor_api_47826aa7f6400ef0, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -820,7 +820,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_0b609b42e437dfba, []int{12} + return fileDescriptor_api_47826aa7f6400ef0, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -854,7 +854,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_0b609b42e437dfba, []int{13} + return fileDescriptor_api_47826aa7f6400ef0, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -888,7 +888,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_0b609b42e437dfba, []int{14} + return fileDescriptor_api_47826aa7f6400ef0, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -940,7 +940,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_0b609b42e437dfba, []int{15} + return fileDescriptor_api_47826aa7f6400ef0, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -977,7 +977,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_0b609b42e437dfba, []int{16} + return fileDescriptor_api_47826aa7f6400ef0, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1024,7 +1024,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_0b609b42e437dfba, []int{17} + return fileDescriptor_api_47826aa7f6400ef0, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1058,7 +1058,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_0b609b42e437dfba, []int{18} + return fileDescriptor_api_47826aa7f6400ef0, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1099,7 +1099,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_0b609b42e437dfba, []int{19} + return fileDescriptor_api_47826aa7f6400ef0, []int{19} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1133,7 +1133,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_0b609b42e437dfba, []int{20} + return fileDescriptor_api_47826aa7f6400ef0, []int{20} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1173,7 +1173,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_0b609b42e437dfba, []int{21} + return fileDescriptor_api_47826aa7f6400ef0, []int{21} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1220,7 +1220,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_0b609b42e437dfba, []int{22} + return fileDescriptor_api_47826aa7f6400ef0, []int{22} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1260,7 +1260,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_0b609b42e437dfba, []int{23} + return fileDescriptor_api_47826aa7f6400ef0, []int{23} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1307,7 +1307,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_0b609b42e437dfba, []int{24} + return fileDescriptor_api_47826aa7f6400ef0, []int{24} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1360,7 +1360,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_0b609b42e437dfba, []int{25} + return fileDescriptor_api_47826aa7f6400ef0, []int{25} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1397,7 +1397,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_0b609b42e437dfba, []int{26} + return fileDescriptor_api_47826aa7f6400ef0, []int{26} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1441,7 +1441,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_0b609b42e437dfba, []int{26, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{26, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1489,7 +1489,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_0b609b42e437dfba, []int{27} + return fileDescriptor_api_47826aa7f6400ef0, []int{27} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1525,7 +1525,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_0b609b42e437dfba, []int{28} + return fileDescriptor_api_47826aa7f6400ef0, []int{28} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1620,7 +1620,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_0b609b42e437dfba, []int{29} + return fileDescriptor_api_47826aa7f6400ef0, []int{29} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1666,7 +1666,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_0b609b42e437dfba, []int{30} + return fileDescriptor_api_47826aa7f6400ef0, []int{30} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1727,7 +1727,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_0b609b42e437dfba, []int{31} + return fileDescriptor_api_47826aa7f6400ef0, []int{31} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1762,7 +1762,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_0b609b42e437dfba, []int{32} + return fileDescriptor_api_47826aa7f6400ef0, []int{32} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1801,7 +1801,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_0b609b42e437dfba, []int{33} + return fileDescriptor_api_47826aa7f6400ef0, []int{33} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1836,7 +1836,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_0b609b42e437dfba, []int{34} + return fileDescriptor_api_47826aa7f6400ef0, []int{34} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1879,7 +1879,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_0b609b42e437dfba, []int{35} + return fileDescriptor_api_47826aa7f6400ef0, []int{35} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1914,7 +1914,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_0b609b42e437dfba, []int{36} + return fileDescriptor_api_47826aa7f6400ef0, []int{36} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1952,7 +1952,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_0b609b42e437dfba, []int{37} + return fileDescriptor_api_47826aa7f6400ef0, []int{37} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1985,7 +1985,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_0b609b42e437dfba, []int{38} + return fileDescriptor_api_47826aa7f6400ef0, []int{38} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2020,7 +2020,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_0b609b42e437dfba, []int{39} + return fileDescriptor_api_47826aa7f6400ef0, []int{39} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2078,7 +2078,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_0b609b42e437dfba, []int{40} + return fileDescriptor_api_47826aa7f6400ef0, []int{40} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2113,7 +2113,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_0b609b42e437dfba, []int{41} + return fileDescriptor_api_47826aa7f6400ef0, []int{41} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2150,7 +2150,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_0b609b42e437dfba, []int{42} + return fileDescriptor_api_47826aa7f6400ef0, []int{42} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2183,7 +2183,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_0b609b42e437dfba, []int{43} + return fileDescriptor_api_47826aa7f6400ef0, []int{43} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2222,7 +2222,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_0b609b42e437dfba, []int{44} + return fileDescriptor_api_47826aa7f6400ef0, []int{44} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2259,7 +2259,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_0b609b42e437dfba, []int{45} + return fileDescriptor_api_47826aa7f6400ef0, []int{45} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2297,7 +2297,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_0b609b42e437dfba, []int{46} + return fileDescriptor_api_47826aa7f6400ef0, []int{46} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2331,7 +2331,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{46, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{46, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2365,7 +2365,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_0b609b42e437dfba, []int{47} + return fileDescriptor_api_47826aa7f6400ef0, []int{47} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2434,7 +2434,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_0b609b42e437dfba, []int{48} + return fileDescriptor_api_47826aa7f6400ef0, []int{48} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2477,7 +2477,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_0b609b42e437dfba, []int{49} + return fileDescriptor_api_47826aa7f6400ef0, []int{49} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2524,7 +2524,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_0b609b42e437dfba, []int{50} + return fileDescriptor_api_47826aa7f6400ef0, []int{50} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2560,7 +2560,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_0b609b42e437dfba, []int{51} + return fileDescriptor_api_47826aa7f6400ef0, []int{51} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2604,7 +2604,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_0b609b42e437dfba, []int{52} + return fileDescriptor_api_47826aa7f6400ef0, []int{52} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2643,7 +2643,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_0b609b42e437dfba, []int{53} + return fileDescriptor_api_47826aa7f6400ef0, []int{53} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2703,7 +2703,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_0b609b42e437dfba, []int{54} + return fileDescriptor_api_47826aa7f6400ef0, []int{54} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2739,7 +2739,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_0b609b42e437dfba, []int{55} + return fileDescriptor_api_47826aa7f6400ef0, []int{55} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2785,7 +2785,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_0b609b42e437dfba, []int{56} + return fileDescriptor_api_47826aa7f6400ef0, []int{56} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2820,7 +2820,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_0b609b42e437dfba, []int{57} + return fileDescriptor_api_47826aa7f6400ef0, []int{57} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2870,7 +2870,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_0b609b42e437dfba, []int{58} + return fileDescriptor_api_47826aa7f6400ef0, []int{58} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2905,7 +2905,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_0b609b42e437dfba, []int{59} + return fileDescriptor_api_47826aa7f6400ef0, []int{59} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2942,7 +2942,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_0b609b42e437dfba, []int{60} + return fileDescriptor_api_47826aa7f6400ef0, []int{60} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2976,7 +2976,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_0b609b42e437dfba, []int{61} + return fileDescriptor_api_47826aa7f6400ef0, []int{61} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3021,7 +3021,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_0b609b42e437dfba, []int{62} + return fileDescriptor_api_47826aa7f6400ef0, []int{62} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3055,7 +3055,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_0b609b42e437dfba, []int{63} + return fileDescriptor_api_47826aa7f6400ef0, []int{63} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3099,7 +3099,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_0b609b42e437dfba, []int{64} + return fileDescriptor_api_47826aa7f6400ef0, []int{64} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3148,7 +3148,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_0b609b42e437dfba, []int{65} + return fileDescriptor_api_47826aa7f6400ef0, []int{65} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3185,7 +3185,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_0b609b42e437dfba, []int{66} + return fileDescriptor_api_47826aa7f6400ef0, []int{66} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3222,7 +3222,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_0b609b42e437dfba, []int{67} + return fileDescriptor_api_47826aa7f6400ef0, []int{67} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3257,7 +3257,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_0b609b42e437dfba, []int{68} + return fileDescriptor_api_47826aa7f6400ef0, []int{68} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3312,7 +3312,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_0b609b42e437dfba, []int{69} + return fileDescriptor_api_47826aa7f6400ef0, []int{69} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3349,7 +3349,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_0b609b42e437dfba, []int{70} + return fileDescriptor_api_47826aa7f6400ef0, []int{70} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3388,7 +3388,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_0b609b42e437dfba, []int{71} + return fileDescriptor_api_47826aa7f6400ef0, []int{71} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3422,7 +3422,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_0b609b42e437dfba, []int{71, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{71, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3455,7 +3455,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_0b609b42e437dfba, []int{71, 1} + return fileDescriptor_api_47826aa7f6400ef0, []int{71, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3495,7 +3495,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_0b609b42e437dfba, []int{71, 2} + return fileDescriptor_api_47826aa7f6400ef0, []int{71, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3534,7 +3534,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_0b609b42e437dfba, []int{71, 3} + return fileDescriptor_api_47826aa7f6400ef0, []int{71, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3570,7 +3570,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_0b609b42e437dfba, []int{71, 4} + return fileDescriptor_api_47826aa7f6400ef0, []int{71, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3609,7 +3609,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_0b609b42e437dfba, []int{71, 5} + return fileDescriptor_api_47826aa7f6400ef0, []int{71, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3649,7 +3649,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_0b609b42e437dfba, []int{72} + return fileDescriptor_api_47826aa7f6400ef0, []int{72} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3683,7 +3683,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_0b609b42e437dfba, []int{73} + return fileDescriptor_api_47826aa7f6400ef0, []int{73} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3745,7 +3745,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_0b609b42e437dfba, []int{74} + return fileDescriptor_api_47826aa7f6400ef0, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3781,7 +3781,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_0b609b42e437dfba, []int{75} + return fileDescriptor_api_47826aa7f6400ef0, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3817,7 +3817,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_0b609b42e437dfba, []int{76} + return fileDescriptor_api_47826aa7f6400ef0, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3857,7 +3857,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_0b609b42e437dfba, []int{76, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3907,7 +3907,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_0b609b42e437dfba, []int{77} + return fileDescriptor_api_47826aa7f6400ef0, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3942,7 +3942,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_0b609b42e437dfba, []int{77, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3978,7 +3978,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_0b609b42e437dfba, []int{77, 1} + return fileDescriptor_api_47826aa7f6400ef0, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4013,7 +4013,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_0b609b42e437dfba, []int{78} + return fileDescriptor_api_47826aa7f6400ef0, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4051,7 +4051,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_0b609b42e437dfba, []int{79} + return fileDescriptor_api_47826aa7f6400ef0, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4086,7 +4086,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_0b609b42e437dfba, []int{80} + return fileDescriptor_api_47826aa7f6400ef0, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4119,7 +4119,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_0b609b42e437dfba, []int{80, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4164,7 +4164,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_0b609b42e437dfba, []int{81} + return fileDescriptor_api_47826aa7f6400ef0, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4202,7 +4202,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_0b609b42e437dfba, []int{82} + return fileDescriptor_api_47826aa7f6400ef0, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4255,7 +4255,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_0b609b42e437dfba, []int{83} + return fileDescriptor_api_47826aa7f6400ef0, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4289,7 +4289,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_0b609b42e437dfba, []int{84} + return fileDescriptor_api_47826aa7f6400ef0, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4333,7 +4333,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_0b609b42e437dfba, []int{85} + return fileDescriptor_api_47826aa7f6400ef0, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4367,7 +4367,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_0b609b42e437dfba, []int{86} + return fileDescriptor_api_47826aa7f6400ef0, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4406,7 +4406,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_0b609b42e437dfba, []int{87} + return fileDescriptor_api_47826aa7f6400ef0, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4440,7 +4440,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_0b609b42e437dfba, []int{88} + return fileDescriptor_api_47826aa7f6400ef0, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4489,7 +4489,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_0b609b42e437dfba, []int{89} + return fileDescriptor_api_47826aa7f6400ef0, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4538,7 +4538,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_0b609b42e437dfba, []int{90} + return fileDescriptor_api_47826aa7f6400ef0, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4573,7 +4573,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_0b609b42e437dfba, []int{91} + return fileDescriptor_api_47826aa7f6400ef0, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4612,7 +4612,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_0b609b42e437dfba, []int{92} + return fileDescriptor_api_47826aa7f6400ef0, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4695,7 +4695,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_0b609b42e437dfba, []int{93} + return fileDescriptor_api_47826aa7f6400ef0, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6145,7 +6145,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_0b609b42e437dfba, []int{94} + return fileDescriptor_api_47826aa7f6400ef0, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7579,27 +7579,13 @@ type Header struct { // be much more straightforward if all transactional requests were // idempotent. We could just re-issue requests. See #26915. AsyncConsensus bool `protobuf:"varint,13,opt,name=async_consensus,json=asyncConsensus,proto3" json:"async_consensus,omitempty"` - // WriteTooOldErrors can either be returned immediately or deferred - // to the end of the transaction. By default they are returned - // immediately to maximize the SQL-level first-statement retry - // ability. Set this flag to request deferred errors, which can - // improve performance under heavy contention when client-side - // retries are already inevitable. - // - // Non-transactional requests are not allowed to set this flag (since there's - // nowhere to defer the error to). At the replica level, this flag is ignored - // when the batch contains an EndTxn (similarly to before, there's nowhere to - // defer the error to). However, the client is allowed to set the flag in a - // batch that contains the EndTxn, and it will have an effect for any - // sub-batches that are split off by the DistSender. - DeferWriteTooOldError bool `protobuf:"varint,14,opt,name=defer_write_too_old_error,json=deferWriteTooOldError,proto3" json:"defer_write_too_old_error,omitempty"` } 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_0b609b42e437dfba, []int{95} + return fileDescriptor_api_47826aa7f6400ef0, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7635,7 +7621,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{96} + return fileDescriptor_api_47826aa7f6400ef0, []int{96} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7672,7 +7658,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_0b609b42e437dfba, []int{97} + return fileDescriptor_api_47826aa7f6400ef0, []int{97} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7730,7 +7716,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_0b609b42e437dfba, []int{97, 0} + return fileDescriptor_api_47826aa7f6400ef0, []int{97, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7769,7 +7755,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_0b609b42e437dfba, []int{98} + return fileDescriptor_api_47826aa7f6400ef0, []int{98} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7810,7 +7796,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_0b609b42e437dfba, []int{99} + return fileDescriptor_api_47826aa7f6400ef0, []int{99} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7851,7 +7837,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_0b609b42e437dfba, []int{100} + return fileDescriptor_api_47826aa7f6400ef0, []int{100} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7888,7 +7874,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_0b609b42e437dfba, []int{101} + return fileDescriptor_api_47826aa7f6400ef0, []int{101} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7925,7 +7911,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_0b609b42e437dfba, []int{102} + return fileDescriptor_api_47826aa7f6400ef0, []int{102} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -15747,16 +15733,6 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { } i++ } - if m.DeferWriteTooOldError { - dAtA[i] = 0x70 - i++ - if m.DeferWriteTooOldError { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } return i, nil } @@ -19013,9 +18989,6 @@ func (m *Header) Size() (n int) { if m.AsyncConsensus { n += 2 } - if m.DeferWriteTooOldError { - n += 2 - } return n } @@ -35667,26 +35640,6 @@ func (m *Header) Unmarshal(dAtA []byte) error { } } m.AsyncConsensus = bool(v != 0) - case 14: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DeferWriteTooOldError", 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.DeferWriteTooOldError = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -36852,455 +36805,454 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_0b609b42e437dfba) } - -var fileDescriptor_api_0b609b42e437dfba = []byte{ - // 7149 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x5d, 0x5d, 0x6c, 0x24, 0xd9, - 0x55, 0x76, 0x75, 0xb7, 0xdb, 0xdd, 0xa7, 0x7f, 0xdc, 0xbe, 0x9e, 0x9f, 0x1e, 0xef, 0xae, 0xed, - 0xe9, 0xf9, 0xdd, 0xc9, 0xae, 0x67, 0x67, 0x26, 0xcb, 0x6e, 0x76, 0x37, 0x9b, 0xd8, 0xed, 0x9e, - 0xe9, 0x1e, 0x8f, 0x3d, 0xde, 0xea, 0xf6, 0x6c, 0x76, 0xc3, 0x52, 0xa9, 0xa9, 0xba, 0x6e, 0x57, - 0xa6, 0xbb, 0xaa, 0xa7, 0xaa, 0xda, 0x3f, 0x23, 0x21, 0x04, 0x3c, 0x04, 0x05, 0xb4, 0x42, 0x02, - 0x21, 0x94, 0x80, 0xb2, 0x52, 0x10, 0x20, 0x50, 0x22, 0x21, 0x21, 0x10, 0x11, 0x28, 0x0f, 0xbc, - 0x2c, 0x51, 0x84, 0x56, 0x08, 0x94, 0x08, 0x09, 0x8b, 0x38, 0x12, 0x44, 0x79, 0x40, 0xe2, 0x09, - 0x69, 0x1f, 0x00, 0xdd, 0x9f, 0xfa, 0xeb, 0xae, 0xfe, 0xb1, 0xb7, 0x16, 0x16, 0xf1, 0x64, 0xd7, - 0xa9, 0x7b, 0x4e, 0xdd, 0x7b, 0xee, 0xb9, 0xe7, 0x9e, 0xef, 0xde, 0x73, 0x6f, 0xc3, 0x8c, 0x69, - 0xc8, 0xca, 0x4e, 0xe7, 0xe1, 0x75, 0xb9, 0xa3, 0x2d, 0x75, 0x4c, 0xc3, 0x36, 0xd0, 0x8c, 0x62, - 0x28, 0x8f, 0x28, 0x79, 0x89, 0xbf, 0x9c, 0x43, 0x4e, 0x29, 0x55, 0xb6, 0x65, 0x56, 0x6c, 0xee, - 0x94, 0x43, 0xc3, 0xa6, 0x69, 0x98, 0x16, 0xa7, 0x9e, 0x71, 0xa8, 0x6d, 0x6c, 0xcb, 0xbe, 0xd2, - 0x25, 0xcb, 0x36, 0x4c, 0xb9, 0x89, 0xaf, 0x63, 0xbd, 0xa9, 0xe9, 0xce, 0x1f, 0x52, 0x6e, 0x57, - 0x51, 0x78, 0x99, 0x0b, 0xc3, 0xca, 0xdc, 0xe2, 0x85, 0x8a, 0x5d, 0x5b, 0x6b, 0x5d, 0xdf, 0x69, - 0x29, 0xd7, 0x6d, 0xad, 0x8d, 0x2d, 0x5b, 0x6e, 0x77, 0xf8, 0x9b, 0x45, 0xfa, 0xc6, 0x36, 0x65, - 0x45, 0xd3, 0x9b, 0xd7, 0x4d, 0xac, 0x18, 0xa6, 0x8a, 0x55, 0xc9, 0xea, 0xc8, 0xba, 0x53, 0xe5, - 0xa6, 0xd1, 0x34, 0xe8, 0xbf, 0xd7, 0xc9, 0x7f, 0x8c, 0x5a, 0xfa, 0x05, 0x48, 0x8b, 0xb2, 0xde, - 0xc4, 0x35, 0x7d, 0xdb, 0x40, 0xaf, 0x41, 0x42, 0xc5, 0x96, 0x52, 0x14, 0x16, 0x85, 0xab, 0x99, - 0x9b, 0xa5, 0xa5, 0x3e, 0x5d, 0x2c, 0xd1, 0xb2, 0xab, 0xd8, 0x52, 0x4c, 0xad, 0x63, 0x1b, 0xe6, - 0x4a, 0xe2, 0xfd, 0xc3, 0x85, 0x09, 0x91, 0x72, 0xa1, 0x4f, 0xc3, 0x64, 0x0b, 0xcb, 0x16, 0x2e, - 0xc6, 0x28, 0x7b, 0x31, 0x84, 0xfd, 0x1e, 0x79, 0xcf, 0x99, 0x58, 0xe1, 0xd2, 0x5f, 0x0b, 0x90, - 0x13, 0xf1, 0xe3, 0x2e, 0xb6, 0xec, 0x2a, 0x96, 0x55, 0x6c, 0xa2, 0x73, 0x10, 0x7f, 0x84, 0x0f, - 0x8a, 0xf1, 0x45, 0xe1, 0x6a, 0x76, 0x65, 0xea, 0xc3, 0xc3, 0x85, 0xf8, 0x1a, 0x3e, 0x10, 0x09, - 0x0d, 0x2d, 0xc2, 0x14, 0xd6, 0x55, 0x89, 0xbc, 0x4e, 0x04, 0x5f, 0x27, 0xb1, 0xae, 0xae, 0xe1, - 0x03, 0xa4, 0x40, 0xca, 0x22, 0xd2, 0x74, 0x05, 0x17, 0x27, 0x17, 0x85, 0xab, 0x93, 0x2b, 0x77, - 0x3e, 0x3c, 0x5c, 0x28, 0x37, 0x35, 0x7b, 0xa7, 0xfb, 0x70, 0x49, 0x31, 0xda, 0xd7, 0xdd, 0x5a, - 0xa9, 0x0f, 0xbd, 0xff, 0xaf, 0x77, 0x1e, 0x35, 0xaf, 0x0f, 0xe8, 0x81, 0xa5, 0xc6, 0xbe, 0x5e, - 0xc7, 0x8f, 0x45, 0x57, 0xf0, 0x2b, 0x89, 0x9f, 0xbc, 0xb7, 0x20, 0xdc, 0x4d, 0xa4, 0x84, 0x42, - 0xec, 0x6e, 0x22, 0x15, 0x2b, 0xc4, 0x4b, 0xff, 0x1e, 0x83, 0xbc, 0x88, 0xad, 0x8e, 0xa1, 0x5b, - 0x98, 0x37, 0xe3, 0x05, 0x88, 0xdb, 0xfb, 0x3a, 0x6d, 0x46, 0xe6, 0xe6, 0x7c, 0x88, 0x32, 0x1a, - 0xa6, 0xac, 0x5b, 0xb2, 0x62, 0x6b, 0x86, 0x2e, 0x92, 0xa2, 0xe8, 0x65, 0xc8, 0x98, 0xd8, 0xea, - 0xb6, 0x31, 0xed, 0x36, 0xda, 0xc2, 0xcc, 0xcd, 0xb3, 0x21, 0x9c, 0xf5, 0x8e, 0xac, 0x8b, 0xc0, - 0xca, 0x92, 0xff, 0xd1, 0x39, 0x48, 0xe9, 0xdd, 0x36, 0xd1, 0x8b, 0x45, 0x5b, 0x1d, 0x17, 0xa7, - 0xf4, 0x6e, 0x7b, 0x0d, 0x1f, 0x58, 0xa8, 0x0c, 0x19, 0x93, 0x74, 0x9a, 0xa4, 0xe9, 0xdb, 0x86, - 0x55, 0x4c, 0x2e, 0xc6, 0xaf, 0x66, 0x6e, 0x3e, 0x3d, 0xa8, 0x6b, 0x89, 0x19, 0xf0, 0xfe, 0x01, - 0xd3, 0x21, 0x58, 0xa8, 0x0e, 0x39, 0x5e, 0x33, 0x13, 0xcb, 0x96, 0xa1, 0x17, 0xa7, 0x16, 0x85, - 0xab, 0xf9, 0x9b, 0x4b, 0x61, 0x62, 0x02, 0x5a, 0x20, 0x8f, 0xdd, 0x36, 0x16, 0x29, 0x97, 0x98, - 0x35, 0x7d, 0x4f, 0xa5, 0xd7, 0x21, 0xeb, 0x7f, 0x8b, 0x10, 0xe4, 0xc5, 0x4a, 0x7d, 0x6b, 0xbd, - 0x22, 0x6d, 0x6d, 0xac, 0x6d, 0xdc, 0x7f, 0x73, 0xa3, 0x30, 0x81, 0x4e, 0x41, 0x81, 0xd3, 0xd6, - 0x2a, 0x6f, 0x49, 0xf7, 0x6a, 0xeb, 0xb5, 0x46, 0x41, 0x98, 0x4b, 0xfc, 0xca, 0x37, 0xe7, 0x27, - 0x4a, 0x0f, 0x00, 0xee, 0x60, 0x9b, 0xdb, 0x0e, 0x5a, 0x81, 0xe4, 0x0e, 0xfd, 0x24, 0xb7, 0xde, - 0xc5, 0xd0, 0xba, 0xf9, 0xec, 0x6c, 0x25, 0x45, 0x9a, 0xf9, 0xc1, 0xe1, 0x82, 0x20, 0x72, 0x4e, - 0xd6, 0xaf, 0xa5, 0xef, 0x0a, 0x90, 0xa1, 0x82, 0x59, 0x43, 0x50, 0xb9, 0x47, 0xf2, 0xf9, 0x91, - 0xad, 0xee, 0x17, 0x8d, 0x96, 0x60, 0x72, 0x57, 0x6e, 0x75, 0x87, 0x0d, 0x8e, 0x07, 0xe4, 0xbd, - 0xc8, 0x8a, 0xa1, 0x57, 0x21, 0xab, 0xe9, 0x36, 0xd6, 0x6d, 0x89, 0xb1, 0xc5, 0x47, 0xb0, 0x65, - 0x58, 0x69, 0xfa, 0x50, 0xfa, 0x8e, 0x00, 0xb0, 0xd9, 0x8d, 0x52, 0x35, 0x64, 0x70, 0x8f, 0x55, - 0x7f, 0x67, 0x70, 0xb3, 0x56, 0x9c, 0x81, 0xa4, 0xa6, 0xb7, 0x34, 0x9d, 0xd5, 0x3f, 0x25, 0xf2, - 0x27, 0x74, 0x0a, 0x26, 0x1f, 0xb6, 0x34, 0x5d, 0xa5, 0x36, 0x9e, 0x12, 0xd9, 0x03, 0x57, 0xbf, - 0x08, 0x19, 0x5a, 0xf7, 0x08, 0xb5, 0x5f, 0xfa, 0x5a, 0x0c, 0x4e, 0x97, 0x0d, 0x5d, 0xd5, 0xc8, - 0x60, 0x93, 0x5b, 0x9f, 0x08, 0xdd, 0xbc, 0x08, 0x69, 0xbc, 0xdf, 0x19, 0xb3, 0x7b, 0x53, 0x78, - 0xbf, 0x43, 0xff, 0x0b, 0x57, 0x1d, 0xfa, 0x34, 0x9c, 0x95, 0x5b, 0x2d, 0x63, 0x4f, 0xd2, 0xb6, - 0x25, 0xd5, 0xc0, 0x96, 0xa4, 0x1b, 0xb6, 0x84, 0xf7, 0x35, 0xcb, 0xa6, 0xfe, 0x20, 0x25, 0xce, - 0xd2, 0xd7, 0xb5, 0xed, 0x55, 0x03, 0x5b, 0x1b, 0x86, 0x5d, 0x21, 0xaf, 0xb8, 0xc2, 0xdf, 0x81, - 0x33, 0xbd, 0xba, 0x89, 0x52, 0xf7, 0x7f, 0x27, 0x40, 0xbe, 0xa6, 0x6b, 0xf6, 0x27, 0x42, 0xe9, - 0xae, 0xf6, 0xe2, 0x7e, 0xed, 0x5d, 0x83, 0xc2, 0xb6, 0xac, 0xb5, 0xee, 0xeb, 0x0d, 0xa3, 0xfd, - 0xd0, 0xb2, 0x0d, 0x1d, 0x5b, 0x5c, 0xbd, 0x7d, 0x74, 0xae, 0xb3, 0x07, 0x30, 0xed, 0xb6, 0x29, - 0x4a, 0x65, 0x3d, 0x81, 0x42, 0x4d, 0x57, 0x4c, 0xdc, 0xc6, 0x7a, 0xa4, 0xda, 0x7a, 0x1a, 0xd2, - 0x9a, 0x23, 0x97, 0x6a, 0x2c, 0x2e, 0x7a, 0x04, 0xde, 0xa6, 0x2e, 0xcc, 0xf8, 0xbe, 0x1d, 0xa5, - 0xf3, 0x7b, 0x0a, 0xd2, 0x3a, 0xde, 0x93, 0xbc, 0xfe, 0x8a, 0x8b, 0x29, 0x1d, 0xef, 0x31, 0x67, - 0xf5, 0x16, 0xe4, 0x56, 0x71, 0x0b, 0xdb, 0x38, 0x7a, 0x4f, 0xbe, 0x05, 0x79, 0x47, 0x74, 0x94, - 0x9d, 0xf4, 0xbb, 0x02, 0x20, 0x2e, 0x97, 0x4c, 0x91, 0x51, 0xf6, 0xd3, 0x02, 0x09, 0x01, 0xec, - 0xae, 0xa9, 0xb3, 0xb9, 0x9c, 0x59, 0x29, 0x30, 0x12, 0x9d, 0xce, 0x3d, 0x8f, 0x9a, 0xf0, 0x7b, - 0x54, 0x37, 0x24, 0x21, 0xc1, 0xc8, 0x1e, 0xcc, 0x06, 0xaa, 0x17, 0x6d, 0x57, 0x26, 0x68, 0xcd, - 0x62, 0x8b, 0x71, 0x7f, 0xf8, 0x45, 0x89, 0xa5, 0x77, 0x60, 0xa6, 0xdc, 0xc2, 0xb2, 0x19, 0xb5, - 0x5a, 0x78, 0x77, 0xbe, 0x05, 0xc8, 0x2f, 0x3e, 0xca, 0x2e, 0xfd, 0x3d, 0x01, 0x90, 0x88, 0x77, - 0xb1, 0x69, 0x47, 0xde, 0xa5, 0xab, 0x90, 0xb1, 0x65, 0xb3, 0x89, 0x6d, 0x89, 0xc4, 0xec, 0xdc, - 0x5d, 0x3d, 0xe3, 0x13, 0x44, 0x22, 0xf7, 0xa5, 0x9d, 0x96, 0xb2, 0xd4, 0x70, 0x62, 0x7a, 0x27, - 0x02, 0x63, 0x7c, 0x84, 0xcc, 0x35, 0xf0, 0x36, 0xcc, 0x06, 0x6a, 0x19, 0xa5, 0x0a, 0xbe, 0x29, - 0x40, 0xa6, 0xae, 0xc8, 0x7a, 0x94, 0x6d, 0x7f, 0x1d, 0x32, 0x96, 0x22, 0xeb, 0xd2, 0xb6, 0x61, - 0xb6, 0x65, 0x9b, 0x9a, 0x6c, 0x3e, 0xd0, 0x76, 0x37, 0xa2, 0x55, 0x64, 0xfd, 0x36, 0x2d, 0x24, - 0x82, 0xe5, 0xfe, 0xef, 0xb7, 0xea, 0xbb, 0x89, 0x54, 0xbc, 0x90, 0x28, 0xfd, 0x87, 0x00, 0x59, - 0x56, 0xcb, 0x28, 0xad, 0xfa, 0x45, 0x48, 0x98, 0xc6, 0x1e, 0xb3, 0xea, 0xcc, 0xcd, 0xa7, 0x42, - 0x44, 0xac, 0xe1, 0x03, 0xff, 0x74, 0x42, 0x8b, 0xa3, 0x15, 0xe0, 0x61, 0x97, 0x44, 0xb9, 0xe3, - 0xe3, 0x72, 0x03, 0xe3, 0x12, 0x89, 0x8c, 0x2b, 0x30, 0xfd, 0x50, 0xb6, 0x95, 0x1d, 0xc9, 0xe4, - 0x95, 0x24, 0x53, 0x4f, 0xfc, 0x6a, 0x56, 0xcc, 0x53, 0xb2, 0x53, 0x75, 0xab, 0xf4, 0x47, 0x8e, - 0x89, 0x5a, 0xf8, 0x93, 0xdf, 0x4d, 0xff, 0x29, 0x70, 0x4b, 0x75, 0x2a, 0xfb, 0xff, 0xad, 0xb7, - 0xbe, 0x11, 0x83, 0xb3, 0xe5, 0x1d, 0xac, 0x3c, 0x2a, 0x1b, 0xba, 0xa5, 0x59, 0x36, 0xd6, 0x95, - 0x83, 0x28, 0xbb, 0xec, 0x29, 0x48, 0xef, 0x69, 0xf6, 0x8e, 0xa4, 0x6a, 0xdb, 0xdb, 0xd4, 0xa7, - 0xa4, 0xc4, 0x14, 0x21, 0xac, 0x6a, 0xdb, 0xdb, 0xe8, 0x16, 0x24, 0xda, 0x86, 0xca, 0xa2, 0xca, - 0xfc, 0xcd, 0x85, 0x10, 0xf1, 0xb4, 0x6a, 0x56, 0xb7, 0xbd, 0x6e, 0xa8, 0x58, 0xa4, 0x85, 0xd1, - 0x3c, 0x80, 0x42, 0xa8, 0x1d, 0x43, 0xd3, 0x6d, 0x3e, 0xbb, 0xf8, 0x28, 0xa8, 0x0a, 0x69, 0x1b, - 0x9b, 0x6d, 0x4d, 0x97, 0x6d, 0x02, 0xad, 0x89, 0xf2, 0x2e, 0x86, 0x56, 0xbc, 0xd3, 0xd2, 0x14, - 0xb9, 0x6f, 0x8d, 0xc0, 0x63, 0xe6, 0xbe, 0xec, 0xdd, 0x04, 0x14, 0xfb, 0x35, 0x14, 0xa5, 0x9d, - 0x6c, 0x42, 0x92, 0x00, 0xce, 0x96, 0xcd, 0x2d, 0xe5, 0xe6, 0x20, 0x45, 0x84, 0xd4, 0x80, 0x02, - 0xd7, 0x96, 0xcd, 0x2b, 0xcf, 0xe5, 0xcc, 0xfd, 0xa5, 0x00, 0x49, 0xf6, 0x02, 0xdd, 0x80, 0x14, - 0xc7, 0xd5, 0x2a, 0xad, 0x63, 0x7c, 0xe5, 0xcc, 0xd1, 0xe1, 0xc2, 0x14, 0x43, 0xd1, 0xab, 0x1f, - 0x7a, 0xff, 0x8a, 0x53, 0x0c, 0x48, 0xab, 0xa4, 0xcf, 0x2c, 0x5b, 0x36, 0x6d, 0xba, 0x7e, 0x41, - 0xfa, 0x2c, 0x2b, 0xa6, 0x28, 0x61, 0x0d, 0x1f, 0xa0, 0xbb, 0x90, 0xb4, 0x6c, 0xd9, 0xee, 0x5a, - 0xbc, 0xd7, 0x8e, 0x55, 0xd9, 0x3a, 0xe5, 0x14, 0xb9, 0x04, 0x12, 0x24, 0xa8, 0xd8, 0x96, 0xb5, - 0x16, 0xed, 0xc6, 0xb4, 0xc8, 0x9f, 0x4a, 0x5f, 0x17, 0x20, 0xc9, 0x8a, 0xa2, 0xb3, 0x30, 0x2b, - 0x2e, 0x6f, 0xdc, 0xa9, 0x48, 0xb5, 0x8d, 0xd5, 0x4a, 0xa3, 0x22, 0xae, 0xd7, 0x36, 0x96, 0x1b, - 0x95, 0xc2, 0x04, 0x3a, 0x03, 0xc8, 0x79, 0x51, 0xbe, 0xbf, 0x51, 0xaf, 0xd5, 0x1b, 0x95, 0x8d, - 0x46, 0x41, 0xa0, 0x48, 0x9c, 0xd2, 0x7d, 0xd4, 0x18, 0xba, 0x08, 0x8b, 0xbd, 0x54, 0xa9, 0xde, - 0x58, 0x6e, 0xd4, 0xa5, 0x4a, 0xbd, 0x51, 0x5b, 0x5f, 0x6e, 0x54, 0x56, 0x0b, 0xf1, 0x21, 0xa5, - 0xc8, 0x47, 0x44, 0xb1, 0x52, 0x6e, 0x14, 0x12, 0xa5, 0x27, 0x70, 0x5a, 0xc4, 0x8a, 0xd1, 0xee, - 0x74, 0x6d, 0x4c, 0x6a, 0x69, 0x45, 0x39, 0x5e, 0xce, 0xc2, 0x94, 0x6a, 0x1e, 0x48, 0x66, 0x57, - 0xe7, 0xa3, 0x25, 0xa9, 0x9a, 0x07, 0x62, 0x57, 0xe7, 0xc6, 0xf8, 0xe7, 0x02, 0x9c, 0xe9, 0xfd, - 0x78, 0x94, 0xa6, 0xf8, 0x05, 0xc8, 0xc8, 0xaa, 0x8a, 0x55, 0x49, 0xc5, 0x2d, 0x5b, 0xe6, 0x41, - 0xc0, 0x0d, 0x9f, 0x24, 0xbe, 0xf6, 0xb4, 0xc4, 0x16, 0x9d, 0x96, 0xdc, 0xb5, 0xa7, 0xf5, 0x07, - 0xe5, 0x32, 0xad, 0xcf, 0x2a, 0x61, 0x74, 0x3c, 0x12, 0x95, 0x45, 0x29, 0xa5, 0x3f, 0x4b, 0x40, - 0xae, 0xa2, 0xab, 0x8d, 0xfd, 0x48, 0x67, 0x84, 0x33, 0x90, 0x54, 0x8c, 0x76, 0x5b, 0xb3, 0x1d, - 0x6d, 0xb1, 0x27, 0xf4, 0x19, 0x48, 0xa9, 0x58, 0x56, 0x5d, 0x48, 0x3f, 0x2a, 0x92, 0x11, 0xdd, - 0xe2, 0xe8, 0x4b, 0x70, 0x96, 0x38, 0x52, 0x53, 0x97, 0x5b, 0x12, 0x93, 0x26, 0xd9, 0xa6, 0xd6, - 0x6c, 0x62, 0x93, 0xaf, 0x74, 0x5d, 0x0d, 0xa9, 0x67, 0x8d, 0x73, 0x94, 0x29, 0x43, 0x83, 0x95, - 0x17, 0x4f, 0x6b, 0x61, 0x64, 0xf4, 0x79, 0x77, 0xcd, 0xc4, 0xea, 0xc8, 0xba, 0xc5, 0x9d, 0xd4, - 0xa0, 0x05, 0x34, 0xae, 0x4b, 0x3e, 0x27, 0x10, 0x8a, 0x85, 0xae, 0x93, 0xf0, 0xfb, 0x71, 0x57, - 0x33, 0xb1, 0x74, 0xa3, 0xa3, 0x14, 0x93, 0xa4, 0xed, 0x2b, 0xf9, 0xa3, 0xc3, 0x05, 0x10, 0x19, - 0xf9, 0xc6, 0x66, 0x99, 0x84, 0xe3, 0xec, 0xff, 0x8e, 0x82, 0x56, 0x60, 0x9e, 0x4c, 0x9c, 0xbc, - 0x3d, 0xb2, 0x2d, 0xed, 0x68, 0xcd, 0x1d, 0x6c, 0x4a, 0xee, 0xf2, 0x6c, 0x31, 0x45, 0xf5, 0x37, - 0xa7, 0xc8, 0x3a, 0xab, 0xec, 0xb2, 0x5d, 0xa5, 0x45, 0x5c, 0x15, 0x11, 0x5d, 0x77, 0x0c, 0xcd, - 0x32, 0xf4, 0x62, 0x9a, 0xe9, 0x9a, 0x3d, 0xa1, 0x37, 0xa0, 0xa0, 0xe9, 0xd2, 0x76, 0x4b, 0x6b, - 0xee, 0xd8, 0xd2, 0x9e, 0xa9, 0xd9, 0xd8, 0x2a, 0xce, 0xd0, 0x26, 0x85, 0x99, 0x60, 0x9d, 0x2f, - 0x4e, 0xaa, 0x6f, 0x92, 0x92, 0xbc, 0x71, 0x79, 0x4d, 0xbf, 0x4d, 0xf9, 0x29, 0xd1, 0x72, 0x27, - 0xea, 0xa9, 0x42, 0xaa, 0xf4, 0x4f, 0x02, 0xe4, 0x1d, 0xc3, 0x89, 0xd2, 0xd4, 0xaf, 0x42, 0xc1, - 0xd0, 0xb1, 0xd4, 0xd9, 0x91, 0x2d, 0xcc, 0x15, 0xc3, 0x67, 0x93, 0xbc, 0xa1, 0xe3, 0x4d, 0x42, - 0x66, 0x9a, 0x40, 0x9b, 0x30, 0x63, 0xd9, 0x72, 0x53, 0xd3, 0x9b, 0x3e, 0x7d, 0x4d, 0x8e, 0x1f, - 0x1f, 0x17, 0x38, 0xb7, 0x4b, 0x0f, 0x84, 0x20, 0x3f, 0x10, 0x60, 0x66, 0x59, 0x6d, 0x6b, 0x7a, - 0xbd, 0xd3, 0xd2, 0x22, 0x05, 0xd3, 0x17, 0x21, 0x6d, 0x11, 0x99, 0x9e, 0x1f, 0xf7, 0x80, 0x50, - 0x8a, 0xbe, 0x21, 0x0e, 0xfd, 0x1e, 0x4c, 0xe3, 0xfd, 0x8e, 0x66, 0xca, 0xb6, 0x66, 0xe8, 0x2c, - 0xf6, 0x4f, 0x8c, 0xdf, 0xb6, 0xbc, 0xc7, 0xeb, 0xc5, 0xff, 0xbc, 0x65, 0x6f, 0x01, 0xf2, 0x37, - 0x2c, 0x4a, 0x10, 0x20, 0xc1, 0x2c, 0x15, 0xbd, 0xa5, 0x5b, 0x11, 0x6b, 0x8d, 0x3b, 0xda, 0x2f, - 0xc2, 0xa9, 0xe0, 0x07, 0xa2, 0xac, 0xfd, 0x3b, 0xbc, 0xc7, 0xd7, 0xb1, 0xf9, 0x31, 0xe1, 0x4f, - 0xbf, 0xf8, 0x28, 0x6b, 0xfe, 0x55, 0x01, 0xce, 0x51, 0xd9, 0x74, 0x53, 0x60, 0x1b, 0x9b, 0x74, - 0xa7, 0x24, 0x4a, 0xa3, 0xbd, 0x00, 0x49, 0x06, 0x27, 0xa9, 0xc5, 0x4e, 0xae, 0x64, 0x48, 0x88, - 0x52, 0xb7, 0x0d, 0x93, 0x84, 0x28, 0xfc, 0x15, 0x6f, 0xa7, 0x0c, 0x73, 0x61, 0x75, 0x89, 0x18, - 0x6f, 0xcf, 0xf0, 0x48, 0x91, 0x98, 0x78, 0x79, 0x87, 0x84, 0x48, 0xa8, 0x02, 0x19, 0x85, 0xfe, - 0x27, 0xd9, 0x07, 0x1d, 0x4c, 0xe5, 0xe7, 0x87, 0x05, 0x99, 0x8c, 0xad, 0x71, 0xd0, 0xc1, 0x24, - 0x52, 0x75, 0xfe, 0x27, 0xea, 0xf2, 0x35, 0x75, 0x68, 0x98, 0x4a, 0xc7, 0x17, 0x2d, 0xeb, 0x44, - 0x7a, 0x01, 0x4d, 0xfc, 0x45, 0x9c, 0xab, 0x82, 0x7d, 0x89, 0x33, 0x45, 0x1a, 0x98, 0xbc, 0x0d, - 0x67, 0x54, 0xdc, 0x31, 0xb1, 0x22, 0xdb, 0x58, 0x95, 0xfc, 0xcd, 0x8f, 0x1d, 0xa3, 0xf9, 0xa7, - 0x3c, 0x19, 0x1e, 0x15, 0xbd, 0x05, 0xc8, 0x27, 0x9b, 0xb5, 0xcc, 0x01, 0x3e, 0xc7, 0x51, 0xca, - 0x8c, 0x27, 0x85, 0xd1, 0x2d, 0x54, 0x86, 0x14, 0xde, 0xef, 0x48, 0x74, 0xbb, 0x30, 0x71, 0xcc, - 0xed, 0xc2, 0x29, 0xbc, 0xdf, 0x21, 0x44, 0xb4, 0x45, 0x66, 0x38, 0x27, 0x24, 0xa0, 0xd5, 0xb6, - 0x46, 0x23, 0x0b, 0xcf, 0x5e, 0xb8, 0xb8, 0x69, 0x37, 0x1a, 0x60, 0x22, 0x78, 0xdf, 0xbd, 0x27, - 0xc0, 0x53, 0xa1, 0x7d, 0x17, 0xe5, 0x64, 0xe7, 0xec, 0x98, 0xc6, 0x4e, 0xb2, 0x63, 0x5a, 0xfa, - 0x63, 0x67, 0xd4, 0x8b, 0xb8, 0x65, 0x10, 0xf5, 0x7e, 0x0c, 0x8b, 0x4f, 0x53, 0x4e, 0xb7, 0xc7, - 0x8e, 0xdd, 0xed, 0x0e, 0x6b, 0x8f, 0x5b, 0xe8, 0xa9, 0x6c, 0x94, 0x6e, 0xe1, 0xb7, 0x04, 0x98, - 0xad, 0x62, 0xd9, 0xb4, 0x1f, 0x62, 0xd9, 0x8e, 0x38, 0xa4, 0x7d, 0x11, 0xe2, 0xba, 0xb1, 0x77, - 0x9c, 0xf5, 0x37, 0x52, 0xde, 0x9b, 0xb6, 0x82, 0xf5, 0x8a, 0xb2, 0xd5, 0x7f, 0x13, 0x83, 0xf4, - 0x9d, 0x72, 0x94, 0x6d, 0x7d, 0x8d, 0xaf, 0xd2, 0xb2, 0xa1, 0x1e, 0x66, 0x96, 0xee, 0xf7, 0x96, - 0xee, 0x94, 0xd7, 0xf0, 0x81, 0x63, 0x96, 0x84, 0x0b, 0x2d, 0x43, 0xda, 0xde, 0x31, 0xb1, 0xb5, - 0x63, 0xb4, 0xd4, 0xe3, 0xc4, 0x2c, 0x1e, 0xd7, 0xdc, 0x23, 0x98, 0xa4, 0x72, 0x9d, 0xcd, 0x7c, - 0x21, 0x64, 0x33, 0x9f, 0x7c, 0xc6, 0x0d, 0xfb, 0x62, 0xc7, 0xf9, 0x8c, 0x43, 0x60, 0x9d, 0xe3, - 0xc6, 0x46, 0x93, 0x85, 0x64, 0xe9, 0x0d, 0x00, 0xd2, 0xb4, 0x28, 0xbb, 0xe7, 0x37, 0xe2, 0x90, - 0xdf, 0xec, 0x5a, 0x3b, 0x11, 0xdb, 0x63, 0x19, 0xa0, 0xd3, 0xb5, 0x28, 0x58, 0xd8, 0xd7, 0x79, - 0xfb, 0x47, 0xa4, 0x09, 0x38, 0x0a, 0x60, 0x7c, 0x8d, 0x7d, 0x1d, 0xdd, 0xe7, 0x42, 0xb0, 0xe4, - 0xe5, 0x1a, 0x5c, 0x1b, 0x03, 0x56, 0x36, 0xf6, 0xf5, 0x75, 0xec, 0xe2, 0x49, 0x26, 0x10, 0x13, - 0x81, 0xaf, 0xc1, 0x14, 0x79, 0x90, 0x6c, 0xe3, 0x38, 0x3d, 0x9f, 0x24, 0x3c, 0x0d, 0x03, 0xbd, - 0x0a, 0x69, 0xc6, 0x4d, 0xe6, 0xaf, 0x24, 0x9d, 0xbf, 0xc2, 0x9a, 0xc4, 0xb5, 0x49, 0x67, 0xae, - 0x14, 0x65, 0x25, 0xb3, 0xd5, 0x29, 0x98, 0xdc, 0x36, 0x4c, 0x05, 0xd3, 0xe4, 0x82, 0x94, 0xc8, - 0x1e, 0xfc, 0x9d, 0x7b, 0x37, 0x91, 0x4a, 0x15, 0xd2, 0x77, 0x13, 0xa9, 0x74, 0x01, 0x4a, 0x5f, - 0x17, 0x60, 0xda, 0xed, 0x95, 0x28, 0x5d, 0x7a, 0x39, 0xa0, 0xd2, 0xe3, 0xf7, 0x0b, 0x51, 0x63, - 0xe9, 0xef, 0x69, 0x7c, 0xa3, 0x18, 0xbb, 0xb4, 0x9b, 0xa2, 0x34, 0x9b, 0x15, 0x96, 0x56, 0x12, - 0x3b, 0x61, 0x57, 0xd3, 0x44, 0x93, 0x1b, 0x70, 0x4a, 0x6b, 0x13, 0x9f, 0xaf, 0xd9, 0xad, 0x03, - 0x8e, 0xd1, 0x6c, 0xec, 0x6c, 0x8a, 0xce, 0x7a, 0xef, 0xca, 0xce, 0x2b, 0xee, 0x06, 0xd9, 0x36, - 0x89, 0xd7, 0xac, 0x28, 0xf5, 0x5e, 0x83, 0x9c, 0xc9, 0x44, 0x93, 0x58, 0xe5, 0x98, 0xaa, 0xcf, - 0xba, 0xac, 0x44, 0xfb, 0xdf, 0x8e, 0xc1, 0xf4, 0x1b, 0x5d, 0x6c, 0x1e, 0x7c, 0x02, 0x75, 0x7f, - 0x19, 0xa6, 0xf7, 0x64, 0xcd, 0x96, 0xb6, 0x0d, 0x53, 0xea, 0x76, 0x54, 0xd9, 0x76, 0x72, 0x23, - 0x72, 0x84, 0x7c, 0xdb, 0x30, 0xb7, 0x28, 0x11, 0x61, 0x40, 0x8f, 0x74, 0x63, 0x4f, 0x97, 0x08, - 0x99, 0x42, 0xe4, 0x7d, 0x9d, 0x2f, 0x36, 0xaf, 0xbc, 0xf4, 0x8f, 0x87, 0x0b, 0xb7, 0xc6, 0x4a, - 0x69, 0xa2, 0x59, 0x61, 0xdd, 0xae, 0xa6, 0x2e, 0x6d, 0x6d, 0xd5, 0x56, 0xc5, 0x02, 0x15, 0xf9, - 0x26, 0x93, 0xd8, 0xd8, 0xd7, 0x9d, 0xa9, 0xfd, 0x43, 0x01, 0x0a, 0x9e, 0xc2, 0xa2, 0xec, 0xd5, - 0x0a, 0x64, 0x1e, 0x77, 0xb1, 0xa9, 0x9d, 0xa0, 0x4f, 0x81, 0x33, 0x12, 0xb7, 0xf4, 0x36, 0x64, - 0x03, 0x7a, 0x88, 0x7f, 0x34, 0x3d, 0x64, 0xf6, 0x3c, 0x15, 0x94, 0xbe, 0x2f, 0x00, 0xa2, 0x8d, - 0xaf, 0xb1, 0x75, 0xfe, 0x4f, 0x98, 0xc1, 0x5c, 0x85, 0x02, 0x4d, 0x32, 0x94, 0xb4, 0x6d, 0xa9, - 0xad, 0x59, 0x96, 0xa6, 0x37, 0xb9, 0xc5, 0xe4, 0x29, 0xbd, 0xb6, 0xbd, 0xce, 0xa8, 0xbc, 0x2f, - 0x7f, 0x1e, 0x66, 0x03, 0xad, 0x89, 0xb2, 0x37, 0xcf, 0x43, 0x76, 0xdb, 0xe8, 0xea, 0xaa, 0xc4, - 0x16, 0xcd, 0xf8, 0xe2, 0x60, 0x86, 0xd2, 0xd8, 0xf7, 0x4a, 0xff, 0x15, 0x83, 0x53, 0x22, 0xb6, - 0x8c, 0xd6, 0x2e, 0x8e, 0x5e, 0x9f, 0xf7, 0x81, 0x6f, 0xc6, 0x48, 0x1f, 0x45, 0xad, 0x69, 0x26, - 0x83, 0x4d, 0x77, 0xc1, 0x55, 0xf7, 0x8b, 0xc3, 0x2d, 0xb3, 0x7f, 0x9d, 0x9d, 0xaf, 0xdc, 0x25, - 0x02, 0x2b, 0x77, 0xbb, 0x30, 0xad, 0x35, 0x75, 0x83, 0x38, 0x32, 0x0b, 0x3f, 0xd6, 0xbb, 0x6d, - 0x07, 0xd6, 0xbc, 0x38, 0x46, 0x5d, 0x6b, 0x8c, 0xb3, 0x8e, 0x1f, 0x6f, 0x74, 0xdb, 0x34, 0xc6, - 0x5e, 0x39, 0x43, 0xaa, 0x7d, 0x74, 0xb8, 0x90, 0x0f, 0xbc, 0xb3, 0xc4, 0xbc, 0xe6, 0x3e, 0x93, - 0x8f, 0x70, 0x03, 0xf8, 0x59, 0x38, 0xdd, 0xd3, 0x01, 0x51, 0x46, 0x43, 0x7f, 0x1b, 0x87, 0x73, - 0x41, 0xf1, 0x51, 0x63, 0x96, 0xff, 0x23, 0x9d, 0x5c, 0x85, 0x5c, 0x5b, 0xd3, 0x4f, 0xb6, 0x72, - 0x99, 0x6d, 0x6b, 0xba, 0xb7, 0x00, 0x1c, 0x62, 0x2e, 0xc9, 0xff, 0x39, 0x73, 0x91, 0x61, 0x2e, - 0xac, 0x3f, 0xa3, 0xb4, 0x99, 0x77, 0x05, 0xc8, 0x46, 0xbd, 0x26, 0x77, 0xb2, 0x04, 0x30, 0xde, - 0xe6, 0x06, 0xe4, 0x3e, 0x86, 0x45, 0xbc, 0x3f, 0x10, 0x00, 0x35, 0xcc, 0xae, 0x4e, 0xc0, 0xf1, - 0x3d, 0xa3, 0x19, 0x65, 0x63, 0x4f, 0xc1, 0xa4, 0xa6, 0xab, 0x78, 0x9f, 0x36, 0x36, 0x21, 0xb2, - 0x87, 0xc0, 0x1e, 0x64, 0x7c, 0xac, 0x3d, 0x48, 0x2f, 0x8f, 0x24, 0x50, 0xd1, 0x28, 0xb5, 0xf0, - 0xed, 0x18, 0xcc, 0xf2, 0xe6, 0x44, 0xbe, 0x88, 0x79, 0xa2, 0x14, 0x73, 0xf4, 0x59, 0x80, 0x8e, - 0x89, 0x77, 0x25, 0xc6, 0x1a, 0x1f, 0x8b, 0x35, 0x4d, 0x38, 0x28, 0x01, 0x7d, 0x01, 0xa6, 0xc9, - 0x40, 0xef, 0x98, 0x46, 0xc7, 0xb0, 0x48, 0x1c, 0x63, 0x8d, 0x07, 0x8d, 0x66, 0x8e, 0x0e, 0x17, - 0x72, 0xeb, 0x9a, 0xbe, 0xc9, 0x19, 0x1b, 0x75, 0x91, 0x78, 0x0c, 0xf7, 0xd1, 0x19, 0x80, 0xff, - 0x20, 0xc0, 0xa9, 0x8f, 0x6d, 0xd9, 0xf7, 0x7f, 0x43, 0x63, 0xee, 0x3c, 0x54, 0xa0, 0x8f, 0x35, - 0x7d, 0xdb, 0x88, 0x7e, 0x31, 0xfe, 0x5d, 0x01, 0x66, 0x7c, 0xe2, 0xa3, 0x8c, 0x72, 0x4e, 0x76, - 0x90, 0xe1, 0x8b, 0x24, 0xee, 0xf1, 0x9b, 0x7d, 0x94, 0x83, 0xea, 0xaf, 0x62, 0x70, 0xa6, 0xcc, - 0x76, 0xa7, 0x9d, 0xd4, 0x8d, 0x28, 0xad, 0xa4, 0x08, 0x53, 0xbb, 0xd8, 0xb4, 0x34, 0x83, 0xcd, - 0xb7, 0x39, 0xd1, 0x79, 0x44, 0x73, 0x90, 0xb2, 0x74, 0xb9, 0x63, 0xed, 0x18, 0xce, 0x2e, 0x9e, - 0xfb, 0xec, 0xa6, 0x99, 0x4c, 0x9e, 0x3c, 0xcd, 0x24, 0x39, 0x3c, 0xcd, 0x64, 0xea, 0x23, 0xa7, - 0x99, 0xf0, 0x2d, 0xb3, 0xef, 0x09, 0x70, 0xb6, 0x4f, 0x7f, 0x51, 0xda, 0xcc, 0x97, 0x21, 0xa3, - 0x70, 0xc1, 0xc4, 0x1b, 0xb3, 0x5d, 0xc1, 0x1a, 0x29, 0x76, 0x42, 0x8c, 0x72, 0x74, 0xb8, 0x00, - 0x4e, 0x55, 0x6b, 0xab, 0x5c, 0x45, 0xe4, 0x7f, 0xb5, 0xf4, 0xab, 0x19, 0x98, 0xae, 0xec, 0xb3, - 0x35, 0xef, 0x3a, 0x0b, 0x0b, 0xd0, 0x6d, 0x48, 0x75, 0x4c, 0x63, 0x57, 0x73, 0x9a, 0x91, 0x0f, - 0x84, 0x4c, 0x4e, 0x33, 0x7a, 0xb8, 0x36, 0x39, 0x87, 0xe8, 0xf2, 0xa2, 0x06, 0xa4, 0xef, 0x19, - 0x8a, 0xdc, 0xba, 0xad, 0xb5, 0x1c, 0xfb, 0x7f, 0x61, 0xb4, 0xa0, 0x25, 0x97, 0x67, 0x53, 0xb6, - 0x77, 0x9c, 0xae, 0x70, 0x89, 0xa8, 0x06, 0xa9, 0xaa, 0x6d, 0x77, 0xc8, 0x4b, 0xee, 0x4d, 0xae, - 0x8c, 0x21, 0x94, 0xb0, 0x70, 0x59, 0x2e, 0x3b, 0x6a, 0xc0, 0xcc, 0x1d, 0xc3, 0x68, 0xb6, 0x70, - 0xb9, 0x65, 0x74, 0xd5, 0xb2, 0xa1, 0x6f, 0x6b, 0x4d, 0xee, 0x8f, 0x2f, 0x8f, 0x21, 0xf3, 0x4e, - 0xb9, 0x2e, 0xf6, 0x0b, 0x40, 0xcb, 0x90, 0xaa, 0xdf, 0xe2, 0xc2, 0x58, 0x1c, 0x77, 0x69, 0x0c, - 0x61, 0xf5, 0x5b, 0xa2, 0xcb, 0x86, 0xee, 0x42, 0x66, 0xf9, 0x49, 0xd7, 0xc4, 0x5c, 0x4a, 0x72, - 0x60, 0x4e, 0x43, 0xaf, 0x14, 0xca, 0x25, 0xfa, 0x99, 0x51, 0x1d, 0xf2, 0x6f, 0x1a, 0xe6, 0xa3, - 0x96, 0x21, 0x3b, 0x2d, 0x9c, 0xa2, 0xe2, 0x3e, 0x35, 0x86, 0x38, 0x87, 0x51, 0xec, 0x11, 0x31, - 0xf7, 0x05, 0xc8, 0x05, 0xba, 0x09, 0x21, 0x48, 0x74, 0x48, 0x8f, 0x08, 0x34, 0x49, 0x88, 0xfe, - 0x8f, 0x9e, 0x87, 0x29, 0xdd, 0x50, 0xb1, 0x63, 0xc3, 0xb9, 0x95, 0x53, 0x47, 0x87, 0x0b, 0xc9, - 0x0d, 0x43, 0x65, 0x01, 0x05, 0xff, 0x4f, 0x4c, 0x92, 0x42, 0x4e, 0x38, 0x31, 0x77, 0x19, 0x12, - 0xa4, 0x7f, 0x88, 0x1b, 0x79, 0x28, 0x5b, 0x78, 0xcb, 0xd4, 0xb8, 0x4c, 0xe7, 0x91, 0x97, 0xfb, - 0xa1, 0x00, 0xb1, 0xfa, 0x2d, 0x12, 0x51, 0x3f, 0xec, 0x2a, 0x8f, 0xb0, 0xcd, 0x4b, 0xf1, 0x27, - 0x1a, 0x69, 0x9b, 0x78, 0x5b, 0x63, 0x51, 0x4e, 0x5a, 0xe4, 0x4f, 0xe8, 0x19, 0x00, 0x59, 0x51, - 0xb0, 0x65, 0x49, 0xce, 0xb9, 0xb0, 0xb4, 0x98, 0x66, 0x94, 0x35, 0x7c, 0x40, 0xd8, 0x2c, 0xac, - 0x98, 0xd8, 0x76, 0xb2, 0x9d, 0xd8, 0x13, 0x61, 0xb3, 0x71, 0xbb, 0x23, 0xd9, 0xc6, 0x23, 0xac, - 0xd3, 0x5e, 0x4d, 0x13, 0xf7, 0xd0, 0xee, 0x34, 0x08, 0x81, 0x78, 0x36, 0xac, 0xab, 0x9e, 0x1b, - 0x4a, 0x8b, 0xee, 0x33, 0x11, 0x69, 0xe2, 0xa6, 0xc6, 0x0f, 0x3a, 0xa5, 0x45, 0xfe, 0x44, 0x34, - 0x26, 0x77, 0xed, 0x1d, 0x9a, 0xd4, 0x91, 0x16, 0xe9, 0xff, 0xbc, 0x69, 0x5f, 0x13, 0x20, 0x7e, - 0xa7, 0x5c, 0x3f, 0x76, 0xdb, 0x1c, 0x89, 0x71, 0x4f, 0x22, 0x4d, 0x32, 0xd4, 0x5a, 0x2d, 0x4d, - 0x6f, 0x92, 0xa0, 0xe3, 0xcb, 0x58, 0x71, 0x5a, 0x96, 0xe7, 0xe4, 0x4d, 0x46, 0x45, 0x8b, 0x90, - 0x51, 0x4c, 0xac, 0x62, 0xdd, 0xd6, 0xe4, 0x96, 0xc5, 0x9b, 0xe8, 0x27, 0xf1, 0xca, 0x7d, 0x45, - 0x80, 0x49, 0x6a, 0x5e, 0xe8, 0x69, 0x48, 0x2b, 0x86, 0x6e, 0xcb, 0x9a, 0xce, 0xfd, 0x44, 0x5a, - 0xf4, 0x08, 0x03, 0x2b, 0x79, 0x1e, 0xb2, 0xb2, 0xa2, 0x18, 0x5d, 0xdd, 0x96, 0x74, 0xb9, 0x8d, - 0x79, 0x65, 0x33, 0x9c, 0xb6, 0x21, 0xb7, 0x31, 0x5a, 0x00, 0xe7, 0xd1, 0x3d, 0x9d, 0x97, 0x16, - 0x81, 0x93, 0xd6, 0xf0, 0x01, 0xaf, 0xc9, 0xf7, 0x04, 0x48, 0x39, 0x66, 0x49, 0x2a, 0xd3, 0xc4, - 0x3a, 0x36, 0x65, 0xdb, 0x70, 0x2b, 0xe3, 0x12, 0x7a, 0xe7, 0xa4, 0xb4, 0x37, 0x27, 0x9d, 0x82, - 0x49, 0x5b, 0x7e, 0xd8, 0x72, 0xea, 0xc1, 0x1e, 0xe8, 0xf2, 0x71, 0x4b, 0x6e, 0xb2, 0x35, 0xb2, - 0xb4, 0xc8, 0x1e, 0x48, 0x93, 0x78, 0x56, 0x2b, 0xd3, 0x0e, 0x7f, 0x22, 0xf5, 0x65, 0x89, 0x9c, - 0x0f, 0x71, 0x53, 0xd3, 0xa9, 0x01, 0xc4, 0x45, 0xa0, 0xa4, 0x15, 0x42, 0x41, 0x4f, 0x41, 0x9a, - 0x15, 0xc0, 0xba, 0x4a, 0xad, 0x20, 0x2e, 0xa6, 0x28, 0xa1, 0xe2, 0x9c, 0x54, 0xe2, 0x81, 0xc8, - 0xb7, 0x04, 0x98, 0x61, 0x39, 0x37, 0x2c, 0x03, 0x34, 0xba, 0x59, 0xf9, 0x15, 0x48, 0xab, 0xb2, - 0x2d, 0xb3, 0xd3, 0x80, 0xb1, 0xa1, 0xa7, 0x01, 0x1d, 0x37, 0x49, 0xca, 0xd3, 0x13, 0x81, 0x08, - 0x12, 0xe4, 0x7f, 0x76, 0x8a, 0x52, 0xa4, 0xff, 0x7b, 0x59, 0x0c, 0xfe, 0xea, 0x46, 0x19, 0xa5, - 0x7c, 0x30, 0x09, 0xb9, 0xca, 0x7e, 0xc7, 0x30, 0x23, 0x5e, 0x44, 0x9b, 0xe2, 0xe0, 0x77, 0xc8, - 0x36, 0x6b, 0x8f, 0x17, 0x74, 0x76, 0x30, 0x39, 0x23, 0x5a, 0x01, 0x60, 0xa9, 0x97, 0x34, 0x0f, - 0x27, 0x7e, 0x8c, 0xcd, 0x26, 0xca, 0x46, 0xa8, 0x68, 0x03, 0x32, 0xed, 0x5d, 0x45, 0x91, 0xb6, - 0xb5, 0x96, 0xcd, 0x93, 0xd6, 0xc2, 0xb3, 0xa4, 0xd7, 0x1f, 0x94, 0xcb, 0xb7, 0x69, 0x21, 0x96, - 0x3b, 0xe6, 0x3d, 0x8b, 0x40, 0x24, 0xb0, 0xff, 0xd1, 0x73, 0xc0, 0x0f, 0x76, 0x48, 0x96, 0x73, - 0x4c, 0x6b, 0x25, 0x77, 0x74, 0xb8, 0x90, 0x16, 0x29, 0xb5, 0x5e, 0x6f, 0x88, 0x69, 0x56, 0xa0, - 0x6e, 0xd9, 0xe8, 0x02, 0xe4, 0x8c, 0xb6, 0x66, 0x4b, 0x4e, 0x1c, 0xc0, 0x43, 0xa7, 0x2c, 0x21, - 0x3a, 0x71, 0x02, 0x6a, 0xc0, 0x15, 0xac, 0x93, 0xd1, 0x40, 0xdb, 0x29, 0x3d, 0x64, 0x6b, 0x75, - 0x36, 0x1b, 0x51, 0x92, 0xd1, 0xb1, 0xb5, 0xb6, 0xf6, 0x84, 0x6e, 0xf4, 0xf2, 0x4d, 0x96, 0x0b, - 0xac, 0x38, 0x69, 0xdf, 0x0a, 0x5d, 0xc4, 0xe3, 0x65, 0xef, 0xfb, 0x8a, 0xa2, 0xaf, 0x08, 0x70, - 0x86, 0x2b, 0x52, 0x7a, 0x78, 0x20, 0xb5, 0xc8, 0x24, 0xa2, 0xd9, 0x07, 0xd2, 0xa3, 0xdd, 0x62, - 0x8a, 0x06, 0x68, 0x9f, 0x09, 0xed, 0x10, 0x9f, 0x1d, 0x2c, 0x39, 0xdd, 0x72, 0x70, 0x8f, 0x33, - 0xaf, 0xed, 0x56, 0x74, 0xdb, 0x3c, 0x58, 0x39, 0x7b, 0x74, 0xb8, 0x30, 0xdb, 0xff, 0xf6, 0x81, - 0x38, 0x6b, 0xf5, 0xb3, 0xcc, 0x7d, 0x19, 0x8a, 0x83, 0x24, 0xa1, 0x82, 0xb7, 0xd3, 0x98, 0x66, - 0x1b, 0x8c, 0x2f, 0x07, 0x57, 0x08, 0xc6, 0x30, 0x1b, 0x67, 0x95, 0x20, 0xf6, 0xb2, 0x33, 0xba, - 0x7f, 0x4d, 0x80, 0xdc, 0x4a, 0xb7, 0xf5, 0xe8, 0x7e, 0xa7, 0xde, 0x6d, 0xb7, 0x65, 0xf3, 0x80, - 0x38, 0x06, 0x36, 0x2a, 0xb5, 0x27, 0x2c, 0x45, 0x25, 0xce, 0x87, 0x9d, 0xf6, 0x04, 0x93, 0x61, - 0xc7, 0x53, 0xd3, 0x09, 0x9d, 0xe5, 0x9d, 0x5f, 0x80, 0x1c, 0x85, 0xeb, 0x12, 0xd6, 0x6d, 0x53, - 0xc3, 0x6c, 0x15, 0x2a, 0x2e, 0x66, 0x29, 0xb1, 0xc2, 0x68, 0xe8, 0x12, 0xe4, 0xad, 0x03, 0xcb, - 0xc6, 0x6d, 0x89, 0x9d, 0xdd, 0x66, 0x18, 0x33, 0x2e, 0xe6, 0x18, 0x55, 0x64, 0xc4, 0xd2, 0x8f, - 0xe2, 0x90, 0x77, 0x34, 0x1b, 0x65, 0xf8, 0xba, 0x02, 0x93, 0xdb, 0x5a, 0x0b, 0x3b, 0x59, 0x02, - 0x97, 0x87, 0x74, 0x28, 0x4f, 0x39, 0x26, 0x51, 0x84, 0x03, 0x80, 0x28, 0x6b, 0x14, 0x63, 0x6c, - 0xee, 0x17, 0x63, 0x90, 0xa0, 0x11, 0xe3, 0x0d, 0x48, 0x50, 0xb7, 0x27, 0x8c, 0xe3, 0xf6, 0x68, - 0x51, 0x37, 0x9c, 0x89, 0xf9, 0xc2, 0x19, 0x12, 0x1b, 0xec, 0xc8, 0x2f, 0xde, 0xb8, 0x49, 0xc7, - 0x57, 0x56, 0xe4, 0x4f, 0x68, 0x85, 0xa6, 0xaf, 0x18, 0xa6, 0x8d, 0x55, 0x1e, 0xa9, 0x85, 0x79, - 0xa6, 0x40, 0xc7, 0x3b, 0x2e, 0xd6, 0xe1, 0x43, 0xe7, 0x20, 0x4e, 0x06, 0xee, 0x14, 0xdb, 0xda, - 0x3e, 0x3a, 0x5c, 0x88, 0x93, 0x21, 0x4b, 0x68, 0xe8, 0x3a, 0x64, 0x82, 0xa3, 0x44, 0xb8, 0x9a, - 0x66, 0xbe, 0xc0, 0x67, 0xe1, 0xd0, 0x72, 0x2d, 0x98, 0xa1, 0x94, 0xbb, 0x89, 0x54, 0xa2, 0x30, - 0x49, 0x33, 0x7a, 0x6b, 0xed, 0xa8, 0xbd, 0xe8, 0x72, 0xb0, 0x87, 0xc3, 0xc2, 0xdb, 0xc0, 0x47, - 0x43, 0x3a, 0x38, 0x30, 0x1f, 0xc5, 0x8f, 0x37, 0x1f, 0xd5, 0x48, 0x44, 0xc5, 0xcf, 0xa7, 0xc7, - 0x07, 0x44, 0xb2, 0xc1, 0xef, 0x37, 0x88, 0x63, 0x12, 0x09, 0x8f, 0x97, 0x84, 0x4f, 0xd3, 0x13, - 0x5e, 0xa7, 0x81, 0x1b, 0xb3, 0xb2, 0xe4, 0xf8, 0x56, 0x36, 0x85, 0x75, 0x95, 0xda, 0xd8, 0x13, - 0x6e, 0x62, 0xaf, 0x40, 0x5c, 0xd5, 0xcc, 0x21, 0x97, 0x1d, 0x84, 0xcf, 0x29, 0x84, 0x69, 0x84, - 0xad, 0x25, 0xfc, 0xb6, 0xe6, 0xc7, 0xa1, 0x73, 0xf7, 0x01, 0xbc, 0x76, 0xa1, 0x45, 0x48, 0x1a, - 0x2d, 0xd5, 0x39, 0x41, 0x90, 0x5b, 0x49, 0x1f, 0x1d, 0x2e, 0x4c, 0xde, 0x6f, 0xa9, 0xb5, 0x55, - 0x71, 0xd2, 0x68, 0xa9, 0x35, 0x95, 0x1e, 0xec, 0xc7, 0x7b, 0x92, 0x9b, 0x63, 0x94, 0x15, 0xa7, - 0x74, 0xbc, 0x47, 0x50, 0x6f, 0x4f, 0xee, 0x03, 0x31, 0x9c, 0x6f, 0x08, 0x90, 0x77, 0x74, 0x18, - 0xad, 0x73, 0x48, 0x69, 0x6d, 0x3e, 0x58, 0xe2, 0xc7, 0x1b, 0x2c, 0x0e, 0x1f, 0x3f, 0x99, 0xf8, - 0x55, 0x81, 0xe7, 0x97, 0xd6, 0x15, 0xd9, 0x26, 0xf3, 0x61, 0x84, 0x06, 0xfe, 0x2c, 0x14, 0x4c, - 0x59, 0x57, 0x8d, 0xb6, 0xf6, 0x04, 0xb3, 0x85, 0x2b, 0x8b, 0xef, 0x4f, 0x4d, 0xbb, 0x74, 0xba, - 0x32, 0xe3, 0xac, 0xbb, 0xfd, 0x54, 0xe0, 0xb9, 0xa8, 0x6e, 0x65, 0xa2, 0x54, 0xda, 0x1a, 0x24, - 0x4d, 0x96, 0xd1, 0xc6, 0x06, 0xdc, 0xf3, 0x21, 0x42, 0xc2, 0xbe, 0xce, 0x12, 0xc6, 0x5c, 0x93, - 0xa7, 0x22, 0xe6, 0x3e, 0x0f, 0x93, 0x94, 0x7c, 0x02, 0xb7, 0xc8, 0x35, 0xff, 0x2f, 0x31, 0xb8, - 0x48, 0x3f, 0xf7, 0x00, 0x9b, 0xda, 0xf6, 0xc1, 0xa6, 0x69, 0xd8, 0x58, 0xb1, 0xb1, 0xea, 0x65, - 0xea, 0x47, 0xea, 0x6b, 0xd2, 0x1d, 0xe7, 0x03, 0xc7, 0xca, 0xec, 0x71, 0xb9, 0xd0, 0x1a, 0x4c, - 0xb3, 0x89, 0x50, 0x92, 0x5b, 0xda, 0x2e, 0x96, 0x64, 0xfb, 0x38, 0x33, 0x4a, 0x8e, 0xf1, 0x2e, - 0x13, 0xd6, 0x65, 0x1b, 0xa9, 0x90, 0xe6, 0xc2, 0x34, 0x95, 0x5f, 0x1c, 0x72, 0xe7, 0xa3, 0x2d, - 0xcd, 0xa4, 0xd8, 0x84, 0x5c, 0x5b, 0x15, 0x53, 0x4c, 0xb2, 0xbb, 0xb4, 0xfe, 0x03, 0x01, 0x2e, - 0x8d, 0x50, 0x74, 0x94, 0x66, 0x36, 0x07, 0xa9, 0x5d, 0xf2, 0x21, 0x8d, 0x6b, 0x3a, 0x25, 0xba, - 0xcf, 0x68, 0x1d, 0x72, 0xdb, 0xb2, 0xd6, 0xc2, 0xaa, 0xc4, 0x2d, 0x71, 0x70, 0x3a, 0x58, 0x78, - 0x96, 0x62, 0x96, 0xb1, 0xd3, 0x97, 0x56, 0xe9, 0xf7, 0x63, 0x30, 0xb3, 0xac, 0xaa, 0xf5, 0x3a, - 0xf7, 0x60, 0xd1, 0xd9, 0x8b, 0x03, 0x56, 0x62, 0x1e, 0x58, 0x41, 0xcf, 0x03, 0x52, 0x35, 0x8b, - 0x5d, 0x6a, 0x60, 0xed, 0xc8, 0xaa, 0xb1, 0xe7, 0x6d, 0x7c, 0xcf, 0x38, 0x6f, 0xea, 0xce, 0x0b, - 0xf4, 0x36, 0xd0, 0xd0, 0x5a, 0xb2, 0x6c, 0xd9, 0x5d, 0x9f, 0x7f, 0xee, 0x38, 0xe7, 0x6b, 0x58, - 0xe8, 0xed, 0x3e, 0x8a, 0x69, 0x22, 0x8e, 0xfe, 0x8b, 0xae, 0x42, 0x41, 0x23, 0x1a, 0xb0, 0x25, - 0xd9, 0x72, 0x0e, 0x62, 0xb0, 0x5b, 0x15, 0xf2, 0x8c, 0xbe, 0x6c, 0xf9, 0xcf, 0x57, 0xb0, 0x3c, - 0x71, 0x4f, 0x4f, 0x51, 0x22, 0xac, 0x3f, 0x14, 0x20, 0x2f, 0xe2, 0x6d, 0x13, 0x5b, 0x91, 0x22, - 0xcd, 0xdb, 0x90, 0x35, 0x99, 0x54, 0x69, 0xdb, 0x34, 0xda, 0xc7, 0x19, 0x6a, 0x19, 0xce, 0x78, - 0xdb, 0x34, 0xda, 0x81, 0x53, 0xe8, 0x0f, 0x60, 0xda, 0xad, 0x69, 0x94, 0x2a, 0xf8, 0x16, 0x3d, - 0x5a, 0xca, 0x04, 0x47, 0xbd, 0xf5, 0xfc, 0x71, 0xe8, 0x81, 0xee, 0x0b, 0xf8, 0xab, 0x1b, 0xa5, - 0x32, 0x7e, 0x2a, 0x40, 0xbe, 0xde, 0x7d, 0xc8, 0x6e, 0xd1, 0x89, 0x4e, 0x0f, 0x15, 0x48, 0xb7, - 0xf0, 0xb6, 0x2d, 0x9d, 0x28, 0xb7, 0x39, 0x45, 0x58, 0x69, 0x7e, 0xf7, 0x1d, 0x00, 0x93, 0x9e, - 0x5e, 0xa2, 0x72, 0xe2, 0xc7, 0x94, 0x93, 0xa6, 0xbc, 0x5e, 0xac, 0x53, 0xfa, 0xd3, 0x18, 0x4c, - 0xbb, 0x8d, 0x8d, 0xd2, 0x89, 0xfe, 0x5c, 0xc0, 0x79, 0xc4, 0x4f, 0xe0, 0x3c, 0x66, 0xf8, 0x96, - 0x7b, 0xb8, 0x03, 0x59, 0x82, 0x59, 0x1a, 0x90, 0x48, 0x72, 0xa7, 0xd3, 0xd2, 0xb0, 0x2a, 0xb1, - 0xbd, 0xdc, 0x04, 0xdd, 0xcb, 0x9d, 0xa1, 0xaf, 0x96, 0xd9, 0x9b, 0x1a, 0xdd, 0xd7, 0xbd, 0x0d, - 0xd9, 0x6d, 0x13, 0xe3, 0x27, 0x58, 0xa2, 0xc8, 0xe8, 0x38, 0x99, 0x05, 0x19, 0xc6, 0x58, 0x27, - 0x7c, 0xdc, 0x00, 0xdf, 0x81, 0x19, 0xaa, 0xe0, 0xa8, 0xcf, 0x54, 0xf2, 0x5e, 0xf9, 0x57, 0x01, - 0x90, 0x5f, 0xfe, 0xc7, 0xd7, 0x31, 0xb1, 0xc8, 0x3b, 0xe6, 0x39, 0x40, 0x2c, 0xc9, 0xcc, 0x92, - 0x3a, 0xd8, 0x94, 0x2c, 0xac, 0x18, 0xfc, 0x6e, 0x18, 0x41, 0x2c, 0xf0, 0x37, 0x9b, 0xd8, 0xac, - 0x53, 0x7a, 0xe9, 0xdd, 0x39, 0xc8, 0x72, 0x9d, 0x6c, 0xe9, 0x9a, 0xa1, 0xa3, 0x1b, 0x10, 0x6f, - 0xf2, 0x95, 0xde, 0x4c, 0xe8, 0x4a, 0x90, 0x77, 0x3f, 0x55, 0x75, 0x42, 0x24, 0x65, 0x09, 0x4b, - 0xa7, 0x6b, 0x87, 0x04, 0x45, 0x5e, 0x6e, 0xac, 0x9f, 0xa5, 0xd3, 0xb5, 0x51, 0x1d, 0xa6, 0x15, - 0xef, 0x7e, 0x1e, 0x89, 0xb0, 0xc7, 0x07, 0x6e, 0x2e, 0x84, 0xde, 0x72, 0x54, 0x9d, 0x10, 0xf3, - 0x4a, 0xe0, 0x05, 0x2a, 0xfb, 0x2f, 0x84, 0x61, 0xd3, 0xe5, 0x85, 0xd0, 0xf3, 0x97, 0xc1, 0xcb, - 0x68, 0xaa, 0x13, 0xbe, 0x7b, 0x63, 0xd0, 0x2b, 0x90, 0x54, 0xe9, 0x45, 0x23, 0xdc, 0x42, 0xc3, - 0x8c, 0x28, 0x70, 0xb7, 0x4b, 0x75, 0x42, 0xe4, 0x1c, 0xe8, 0x2e, 0x64, 0xd9, 0x7f, 0x2c, 0x38, - 0xe1, 0x48, 0xee, 0xd2, 0x60, 0x09, 0x3e, 0x5f, 0x5f, 0x9d, 0x10, 0x33, 0xaa, 0x47, 0x45, 0x9f, - 0x86, 0x84, 0xa5, 0xc8, 0x3a, 0xdf, 0x24, 0x99, 0x1f, 0x70, 0x71, 0x81, 0xc7, 0x4c, 0x4b, 0xa3, - 0x57, 0xd9, 0x65, 0x72, 0xf6, 0x3e, 0x3b, 0x78, 0x19, 0x5e, 0xfd, 0xc0, 0xd1, 0x5a, 0x52, 0x7d, - 0x4c, 0x09, 0xe8, 0x0e, 0x64, 0x64, 0x12, 0xe5, 0x49, 0xf4, 0x18, 0x5b, 0x11, 0x06, 0x1e, 0x34, - 0xea, 0x3b, 0x82, 0x58, 0xa5, 0xe7, 0x77, 0x1d, 0xa2, 0x27, 0xa8, 0x8d, 0xcd, 0x26, 0x2e, 0x66, - 0x86, 0x0b, 0xf2, 0x67, 0xd1, 0xb8, 0x82, 0x28, 0x91, 0x44, 0x7b, 0x3b, 0xce, 0x11, 0x05, 0xda, - 0xa8, 0xec, 0xc0, 0x4d, 0xb1, 0x90, 0x23, 0x16, 0xd5, 0x09, 0x31, 0xbb, 0xe3, 0x23, 0xa3, 0x25, - 0x88, 0x35, 0x95, 0x62, 0x8e, 0xca, 0x78, 0x7a, 0xd8, 0x01, 0x82, 0xea, 0x84, 0x18, 0x6b, 0x2a, - 0x04, 0x95, 0xb3, 0xd4, 0xef, 0x7d, 0xbd, 0x98, 0x1f, 0x38, 0xe2, 0x83, 0x79, 0xf4, 0xd5, 0x09, - 0x91, 0x66, 0x9b, 0x93, 0xef, 0x6d, 0x42, 0xde, 0x64, 0x69, 0x48, 0x4e, 0x72, 0x61, 0x61, 0xe0, - 0x46, 0x61, 0x58, 0x7e, 0x61, 0x95, 0x46, 0xfd, 0x3e, 0x3a, 0xfa, 0x12, 0x9c, 0x0a, 0x4a, 0xe4, - 0x96, 0x36, 0xd3, 0xe7, 0x46, 0xc2, 0xe5, 0x06, 0x0d, 0x0e, 0x99, 0x7d, 0x2f, 0xd1, 0x4b, 0x30, - 0xc9, 0x7a, 0x0d, 0x51, 0x91, 0x61, 0x3b, 0xe0, 0x3d, 0x1d, 0xc6, 0xca, 0x13, 0xe3, 0xb7, 0x79, - 0xfe, 0x8d, 0xd4, 0x32, 0x9a, 0xc5, 0xd9, 0x81, 0xc6, 0xdf, 0x9f, 0x4f, 0x44, 0x8c, 0xdf, 0xf6, - 0xa8, 0xa4, 0xdf, 0x4d, 0xf6, 0x86, 0xa7, 0x6b, 0x9c, 0x1a, 0xd8, 0xef, 0x21, 0x69, 0x39, 0x55, - 0x9a, 0x3b, 0xed, 0x91, 0x49, 0xd5, 0x4c, 0x76, 0x71, 0x87, 0x44, 0xc7, 0xd4, 0xe9, 0x81, 0x55, - 0xeb, 0xbf, 0x8c, 0xa4, 0x4a, 0xc3, 0x20, 0x97, 0x8a, 0x1e, 0x40, 0x81, 0x1f, 0xa9, 0xf7, 0x96, - 0xad, 0xcf, 0x50, 0x79, 0xcf, 0x86, 0xba, 0xae, 0xb0, 0xfc, 0x86, 0xea, 0x84, 0x38, 0xad, 0x04, - 0xdf, 0xa0, 0xb7, 0x60, 0x86, 0xca, 0x93, 0x14, 0xef, 0x2e, 0x84, 0x62, 0xb1, 0x2f, 0x51, 0x70, - 0xf0, 0xb5, 0x09, 0x8e, 0xe4, 0x82, 0xd2, 0xf3, 0x8a, 0x98, 0xb1, 0xa6, 0x6b, 0x36, 0xf5, 0xb2, - 0x73, 0x03, 0xcd, 0x38, 0x78, 0x9f, 0x19, 0x31, 0x63, 0x8d, 0x51, 0x88, 0x19, 0xdb, 0x3c, 0x97, - 0x87, 0x77, 0xc7, 0xd3, 0x03, 0xcd, 0x38, 0x2c, 0xe9, 0x87, 0x98, 0xb1, 0xed, 0xa7, 0x13, 0x33, - 0x66, 0x0e, 0xa2, 0x47, 0xee, 0x33, 0x03, 0xcd, 0x78, 0xe0, 0x41, 0x52, 0x62, 0xc6, 0x72, 0xdf, - 0x4b, 0xb4, 0x0a, 0xc0, 0xc2, 0x13, 0x4d, 0xdf, 0x36, 0x8a, 0xf3, 0x03, 0x27, 0x83, 0xde, 0x6c, - 0x1e, 0x32, 0x19, 0xb4, 0x1c, 0x1a, 0x71, 0x64, 0x14, 0x1b, 0x49, 0x74, 0x97, 0xac, 0xb8, 0x30, - 0xd0, 0x91, 0xf5, 0x6d, 0x96, 0x11, 0x47, 0xb6, 0xe7, 0x12, 0xc9, 0xac, 0xc2, 0xd6, 0x58, 0x8b, - 0x8b, 0x83, 0xdd, 0xb2, 0x7f, 0x77, 0x81, 0xba, 0x65, 0x4a, 0x40, 0xcb, 0x90, 0x26, 0xd3, 0xf6, - 0x01, 0x75, 0x43, 0xe7, 0x07, 0x06, 0x9c, 0x3d, 0x87, 0x03, 0xaa, 0x13, 0x62, 0xea, 0x31, 0x27, - 0x91, 0xcf, 0xb3, 0x55, 0xab, 0x62, 0x69, 0xe0, 0xe7, 0x03, 0x2b, 0x95, 0xe4, 0xf3, 0x8c, 0x03, - 0x29, 0x70, 0x9a, 0xf5, 0x15, 0x3f, 0xc7, 0x69, 0xf2, 0x43, 0x87, 0xc5, 0x0b, 0x54, 0xd4, 0xc0, - 0x35, 0xa0, 0xd0, 0xe3, 0xa5, 0xd5, 0x09, 0x71, 0x56, 0xee, 0x7f, 0x4b, 0x06, 0x3c, 0x9f, 0x7a, - 0xd8, 0xca, 0x51, 0xf1, 0xe2, 0xc0, 0x01, 0x1f, 0xb2, 0xd6, 0x46, 0x06, 0xbc, 0xec, 0x23, 0xb3, - 0x09, 0x48, 0x95, 0x2c, 0x8b, 0xed, 0xa9, 0x5e, 0x1a, 0x32, 0x01, 0xf5, 0x60, 0x7f, 0x36, 0x01, - 0xa9, 0x75, 0xc6, 0x49, 0x04, 0x29, 0x2d, 0x2c, 0x9b, 0xdc, 0xcd, 0x5e, 0x1e, 0x28, 0xa8, 0xef, - 0x8e, 0x30, 0x22, 0x48, 0x71, 0x89, 0x24, 0xe0, 0x31, 0x9d, 0xbb, 0x38, 0x78, 0xe8, 0x77, 0x65, - 0x60, 0xc0, 0x13, 0x7a, 0x65, 0x08, 0x09, 0x78, 0xcc, 0xc0, 0x0b, 0xf4, 0x59, 0x98, 0xe2, 0x08, - 0xad, 0x78, 0x75, 0x48, 0x40, 0xea, 0x87, 0xd6, 0x64, 0x5c, 0x73, 0x1e, 0xe6, 0x65, 0x19, 0x32, - 0x64, 0xcd, 0x7b, 0x76, 0x88, 0x97, 0xed, 0x03, 0xa7, 0xcc, 0xcb, 0x7a, 0x64, 0xe2, 0x65, 0x99, - 0x9d, 0xf2, 0xb9, 0xee, 0xda, 0x40, 0x2f, 0xdb, 0x7f, 0x32, 0x81, 0x78, 0xd9, 0xc7, 0x1e, 0x95, - 0xb4, 0xcc, 0x62, 0xa8, 0xa8, 0xf8, 0xa9, 0x81, 0x2d, 0x0b, 0x82, 0x44, 0xd2, 0x32, 0xce, 0x43, - 0xba, 0x8d, 0x25, 0x91, 0x32, 0x4d, 0x3f, 0x37, 0xf8, 0xc8, 0x74, 0x2f, 0x88, 0xa8, 0x3a, 0x97, - 0xc4, 0x32, 0x0d, 0xbb, 0x8e, 0xca, 0xe4, 0x07, 0x44, 0xb9, 0xa6, 0x9e, 0x1f, 0xee, 0xa8, 0xc2, - 0xce, 0xbe, 0xba, 0x8e, 0x2a, 0xf0, 0x92, 0x56, 0x95, 0x9d, 0xf3, 0xa1, 0xe3, 0x7b, 0x69, 0xc8, - 0xe9, 0xee, 0x9e, 0xa3, 0x57, 0xb4, 0xaa, 0x2e, 0xd1, 0x1b, 0x42, 0x5d, 0x76, 0x0d, 0x41, 0xf1, - 0xfa, 0xf0, 0x21, 0x14, 0xbc, 0x0e, 0xc1, 0x1d, 0x42, 0x9c, 0xec, 0xce, 0x99, 0x4e, 0x84, 0xf1, - 0xc2, 0xf0, 0x39, 0xb3, 0x37, 0xb4, 0x60, 0x73, 0x26, 0x8f, 0x29, 0x7e, 0x49, 0x80, 0x45, 0x56, - 0x37, 0xba, 0x8e, 0x77, 0x20, 0xb9, 0x6b, 0xa2, 0xbe, 0x54, 0xf3, 0x1b, 0xf4, 0x03, 0x2f, 0x0d, - 0xaa, 0xee, 0x88, 0x35, 0xde, 0xea, 0x84, 0xf8, 0x8c, 0x3c, 0xac, 0xdc, 0xca, 0x14, 0xdf, 0x39, - 0x75, 0x4f, 0xdc, 0x4d, 0x17, 0x0a, 0x77, 0x13, 0xa9, 0xb3, 0x85, 0xe2, 0xdd, 0x44, 0xea, 0x5c, - 0x61, 0xee, 0x6e, 0x22, 0xf5, 0x54, 0xe1, 0xe9, 0xd2, 0xbf, 0x9d, 0x83, 0x9c, 0x83, 0xe1, 0x18, - 0x22, 0xba, 0xe9, 0x47, 0x44, 0xf3, 0x83, 0x10, 0x11, 0x47, 0x7d, 0x1c, 0x12, 0xdd, 0xf4, 0x43, - 0xa2, 0xf9, 0x41, 0x90, 0xc8, 0xe3, 0x21, 0x98, 0xa8, 0x31, 0x08, 0x13, 0x3d, 0x3b, 0x06, 0x26, - 0x72, 0x45, 0xf5, 0x82, 0xa2, 0xd5, 0x7e, 0x50, 0x74, 0x71, 0x38, 0x28, 0x72, 0x45, 0xf9, 0x50, - 0xd1, 0xab, 0x3d, 0xa8, 0xe8, 0xfc, 0x10, 0x54, 0xe4, 0xf2, 0x3b, 0xb0, 0x68, 0x2d, 0x14, 0x16, - 0x5d, 0x1e, 0x05, 0x8b, 0x5c, 0x39, 0x01, 0x5c, 0xf4, 0x62, 0x00, 0x17, 0x2d, 0x0c, 0xc4, 0x45, - 0x2e, 0x37, 0x03, 0x46, 0xaf, 0xf5, 0x02, 0xa3, 0xf3, 0x43, 0x80, 0x91, 0xd7, 0x02, 0x8e, 0x8c, - 0xaa, 0x61, 0xc8, 0xe8, 0xd2, 0x08, 0x64, 0xe4, 0x4a, 0xf1, 0x43, 0xa3, 0x6a, 0x18, 0x34, 0xba, - 0x34, 0x02, 0x1a, 0xf5, 0x48, 0x62, 0xd8, 0x68, 0x23, 0x1c, 0x1b, 0x5d, 0x19, 0x89, 0x8d, 0x5c, - 0x69, 0x41, 0x70, 0x74, 0xdd, 0x07, 0x8e, 0x9e, 0x19, 0x00, 0x8e, 0x5c, 0x56, 0x82, 0x8e, 0x3e, - 0xd7, 0x87, 0x8e, 0x4a, 0xc3, 0xd0, 0x91, 0xcb, 0xeb, 0xc2, 0xa3, 0x37, 0x06, 0xc0, 0xa3, 0xab, - 0xa3, 0xe1, 0x91, 0x2b, 0xac, 0x07, 0x1f, 0xc9, 0x43, 0xf1, 0xd1, 0xf3, 0x63, 0xe2, 0x23, 0x57, - 0x7a, 0x18, 0x40, 0x7a, 0x39, 0x08, 0x90, 0x16, 0x07, 0x03, 0x24, 0x57, 0x0c, 0x47, 0x48, 0x6b, - 0xa1, 0x08, 0xe9, 0xf2, 0x28, 0x84, 0xe4, 0x8d, 0x03, 0x3f, 0x44, 0xda, 0x08, 0x87, 0x48, 0x57, - 0x46, 0x42, 0x24, 0xaf, 0xfb, 0x03, 0x18, 0x69, 0x2d, 0x14, 0x23, 0x5d, 0x1e, 0x85, 0x91, 0xbc, - 0xca, 0xf9, 0x41, 0xd2, 0x9b, 0x03, 0x41, 0xd2, 0xb5, 0x71, 0x40, 0x92, 0x2b, 0xb4, 0x0f, 0x25, - 0xbd, 0x3d, 0x18, 0x25, 0x7d, 0xea, 0x18, 0x97, 0xcb, 0x85, 0xc2, 0xa4, 0xcf, 0xf5, 0xc1, 0xa4, - 0xd2, 0x30, 0x98, 0xe4, 0xd9, 0xb3, 0x83, 0x93, 0xe4, 0xa1, 0xa8, 0xe6, 0xf9, 0x31, 0x51, 0x8d, - 0x67, 0x7c, 0x21, 0xb0, 0xa6, 0x12, 0x02, 0x6b, 0x2e, 0x0e, 0x87, 0x35, 0x9e, 0x3b, 0xf7, 0x70, - 0x4d, 0x35, 0x0c, 0xd7, 0x5c, 0x1a, 0x81, 0x6b, 0x3c, 0x2f, 0xe4, 0x03, 0x36, 0xaf, 0xf6, 0x00, - 0x9b, 0xf3, 0x23, 0xb3, 0x6c, 0x7c, 0xc8, 0x66, 0xa5, 0x1f, 0xd9, 0x5c, 0x18, 0x8a, 0x6c, 0x5c, - 0x09, 0x1e, 0xb4, 0x79, 0xb5, 0x07, 0xda, 0x9c, 0x1f, 0x02, 0x6d, 0xbc, 0x0a, 0x70, 0x6c, 0xa3, - 0x0e, 0xc7, 0x36, 0x4b, 0xe3, 0x62, 0x1b, 0x57, 0x70, 0x28, 0xb8, 0xd9, 0x08, 0x07, 0x37, 0x57, - 0xc6, 0xdc, 0x3d, 0xef, 0x43, 0x37, 0xd5, 0x30, 0x74, 0x73, 0x69, 0x04, 0xba, 0xf1, 0xcf, 0x21, - 0x2e, 0xbc, 0xa9, 0x86, 0xc1, 0x9b, 0x4b, 0x23, 0xe0, 0x8d, 0x27, 0xc9, 0x87, 0x6f, 0x1a, 0x83, - 0xf0, 0xcd, 0xb3, 0x63, 0xe0, 0x1b, 0x2f, 0x78, 0xe9, 0x01, 0x38, 0xaf, 0xf7, 0x02, 0x9c, 0xd2, - 0x30, 0x80, 0xe3, 0x8d, 0x48, 0x07, 0xe1, 0x6c, 0x84, 0x23, 0x9c, 0x2b, 0x23, 0x11, 0x8e, 0xdf, - 0x49, 0xfa, 0x20, 0xce, 0x5a, 0x28, 0xc4, 0xb9, 0x3c, 0x0a, 0xe2, 0x78, 0x4e, 0xd2, 0x8f, 0x71, - 0x5e, 0xef, 0xc5, 0x38, 0xa5, 0x61, 0x18, 0xc7, 0x6b, 0x9c, 0x03, 0x72, 0xaa, 0x61, 0x20, 0xe7, - 0xd2, 0x08, 0x90, 0xe3, 0x75, 0x9e, 0x0f, 0xe5, 0xc8, 0x43, 0x51, 0xce, 0xf3, 0x63, 0xa2, 0x9c, - 0x1e, 0xc7, 0x15, 0x84, 0x39, 0xd5, 0x30, 0x98, 0x73, 0x69, 0x04, 0xcc, 0xf1, 0x55, 0xd6, 0xc3, - 0x39, 0x1b, 0xe1, 0x38, 0xe7, 0xca, 0x48, 0x9c, 0xd3, 0x33, 0x9a, 0x1c, 0xa0, 0xb3, 0x16, 0x0a, - 0x74, 0x2e, 0x8f, 0x02, 0x3a, 0x3d, 0x13, 0x1f, 0x0f, 0x0e, 0x7e, 0x79, 0x7c, 0xa4, 0xf3, 0xf2, - 0xf1, 0x91, 0x8e, 0xfb, 0xcd, 0x48, 0xa0, 0xce, 0xdd, 0x44, 0xea, 0xe9, 0xc2, 0x33, 0xa5, 0xef, - 0x4c, 0x42, 0xb2, 0xea, 0xe6, 0xb8, 0x78, 0xb5, 0x14, 0x4e, 0x72, 0x7b, 0x0d, 0x5a, 0x25, 0x23, - 0x96, 0xfa, 0xbd, 0xd1, 0x17, 0x95, 0xf5, 0x5f, 0xa2, 0xc5, 0x59, 0x4f, 0x70, 0x08, 0x14, 0xbd, - 0x08, 0xb9, 0xae, 0x85, 0x4d, 0xa9, 0x63, 0x6a, 0x86, 0xa9, 0xd9, 0x2c, 0x5d, 0x5f, 0x58, 0x29, - 0x7c, 0x78, 0xb8, 0x90, 0xdd, 0xb2, 0xb0, 0xb9, 0xc9, 0xe9, 0x62, 0xb6, 0xeb, 0x7b, 0x72, 0x7e, - 0xd1, 0x66, 0x72, 0xfc, 0x5f, 0xb4, 0x79, 0x03, 0x0a, 0x26, 0x96, 0xd5, 0x40, 0x04, 0xc2, 0xae, - 0x85, 0x09, 0xb7, 0x19, 0x7a, 0x56, 0xc5, 0x29, 0x49, 0xaf, 0x87, 0x99, 0x36, 0x83, 0x44, 0x74, - 0x03, 0x4e, 0xb7, 0xe5, 0x7d, 0x9a, 0x83, 0x28, 0x39, 0x41, 0x1d, 0xcd, 0x2b, 0x4c, 0xd1, 0x7c, - 0x59, 0xd4, 0x96, 0xf7, 0xe9, 0xcf, 0xe3, 0xb0, 0x57, 0xf4, 0xce, 0xfc, 0x4b, 0x90, 0x57, 0x35, - 0xcb, 0xd6, 0x74, 0xc5, 0xb9, 0x19, 0x94, 0x5d, 0xb4, 0x99, 0x73, 0xa8, 0xec, 0xf2, 0xcf, 0x6b, - 0x30, 0xc3, 0xf3, 0xb1, 0xbd, 0x1f, 0xcc, 0xa1, 0xf0, 0x25, 0x45, 0x6a, 0x41, 0x5e, 0x78, 0xbf, - 0x94, 0x54, 0x86, 0xe9, 0xa6, 0x6c, 0xe3, 0x3d, 0xf9, 0x40, 0x72, 0x8e, 0xcb, 0x64, 0xe8, 0xb5, - 0x7a, 0x4f, 0x1d, 0x1d, 0x2e, 0xe4, 0xee, 0xb0, 0x57, 0x7d, 0xa7, 0x66, 0x72, 0x4d, 0xdf, 0x0b, - 0x15, 0x5d, 0x81, 0x69, 0xd9, 0x3a, 0xd0, 0x15, 0xaa, 0x1e, 0xac, 0x5b, 0x5d, 0x8b, 0x42, 0x8a, - 0x94, 0x98, 0xa7, 0xe4, 0xb2, 0x43, 0x45, 0x2f, 0xc3, 0x39, 0x15, 0x93, 0x18, 0x89, 0x05, 0x22, - 0xb6, 0x61, 0x48, 0x46, 0x4b, 0x95, 0xe8, 0xf5, 0x0f, 0x14, 0x4e, 0xa4, 0xc4, 0xd3, 0xb4, 0x00, - 0x0d, 0x41, 0x1a, 0x86, 0x71, 0xbf, 0xa5, 0x56, 0xc8, 0x4b, 0x76, 0xd5, 0xe7, 0xdd, 0x44, 0x2a, - 0x5b, 0xc8, 0x95, 0x7e, 0x53, 0x80, 0x6c, 0xe0, 0x8c, 0xc2, 0xab, 0x3d, 0x5b, 0xb4, 0xe7, 0xc2, - 0xe1, 0xd0, 0xa0, 0x1c, 0xaf, 0x14, 0x57, 0xbf, 0x93, 0xe1, 0xb6, 0x30, 0x38, 0x9c, 0xa6, 0x8b, - 0x03, 0x4e, 0x7a, 0x80, 0xc3, 0xf6, 0x4a, 0xe2, 0xb7, 0xdf, 0x5b, 0x98, 0x28, 0xfd, 0x24, 0x0e, - 0xb9, 0xe0, 0x59, 0x84, 0x5a, 0x4f, 0xbd, 0xc2, 0xdc, 0x55, 0x80, 0x63, 0x69, 0xc8, 0xfd, 0x67, - 0x69, 0xef, 0xa6, 0x6e, 0x56, 0xcd, 0xc5, 0x21, 0x1b, 0xd1, 0xfe, 0x7a, 0x7a, 0x8c, 0x73, 0xdf, - 0x8d, 0xb9, 0xc3, 0x7e, 0x09, 0x26, 0x99, 0xda, 0x85, 0x81, 0x67, 0x43, 0xa9, 0xe6, 0x45, 0x56, - 0x8c, 0xb8, 0x89, 0xc6, 0x89, 0x2e, 0xb9, 0xf2, 0xae, 0x07, 0x38, 0xfe, 0x0f, 0x49, 0xf1, 0xab, - 0xce, 0x26, 0x8f, 0x77, 0xd5, 0x19, 0xdb, 0x68, 0x6e, 0xb5, 0x98, 0x0b, 0x66, 0x03, 0x25, 0xd9, - 0x77, 0x00, 0x93, 0x8a, 0xe0, 0xbf, 0x33, 0xb6, 0x24, 0xf2, 0xdf, 0x19, 0xf3, 0x25, 0x1d, 0xe6, - 0x5d, 0x11, 0x74, 0x54, 0xb1, 0xd4, 0x54, 0xde, 0xd5, 0xdf, 0x10, 0xa0, 0x40, 0xc7, 0xd0, 0x6d, - 0x8c, 0xd5, 0x48, 0xac, 0xd0, 0xc9, 0x87, 0x8c, 0x8d, 0x9f, 0x26, 0x1e, 0xb8, 0x39, 0x3d, 0x1e, - 0xbc, 0x39, 0xbd, 0xf4, 0x9e, 0x00, 0x79, 0xb7, 0x86, 0xec, 0x07, 0x77, 0x86, 0xdc, 0x60, 0x76, - 0xb2, 0xdf, 0xa0, 0x71, 0x0e, 0x57, 0x8f, 0xf5, 0xcb, 0x3f, 0xfe, 0xc3, 0xd5, 0xec, 0xf7, 0x52, - 0x7e, 0x47, 0x80, 0x59, 0xb7, 0x8a, 0x65, 0xef, 0xe0, 0xec, 0x09, 0x32, 0xe6, 0x45, 0xfa, 0x83, - 0x63, 0x04, 0xb4, 0xd3, 0x53, 0xed, 0x63, 0x99, 0x27, 0xe2, 0x09, 0x15, 0xc0, 0x17, 0x03, 0xd4, - 0x46, 0x9d, 0xfe, 0x14, 0x19, 0xfb, 0xdf, 0x2a, 0xdd, 0xf6, 0x29, 0x90, 0x8e, 0x04, 0xa2, 0xa5, - 0xb1, 0x86, 0x8c, 0xa3, 0x25, 0x5a, 0xb8, 0xf4, 0x7d, 0x7f, 0x4f, 0x54, 0x76, 0x49, 0x10, 0x78, - 0x0b, 0xe2, 0xbb, 0x72, 0x6b, 0x58, 0x3e, 0x49, 0xa0, 0xe7, 0x44, 0x52, 0x1a, 0xdd, 0x0e, 0x9c, - 0x37, 0x8e, 0x0d, 0x0e, 0x58, 0xfa, 0x55, 0x1a, 0x38, 0x97, 0xfc, 0x92, 0xd3, 0x8a, 0xf8, 0xe8, - 0xcf, 0xfb, 0x3d, 0xc0, 0x2b, 0x89, 0xf7, 0xdf, 0x5b, 0x10, 0xae, 0xd5, 0x61, 0x36, 0x64, 0x7a, - 0x43, 0x79, 0x00, 0xdf, 0x7d, 0xea, 0xfc, 0xf7, 0xce, 0x96, 0x57, 0xa5, 0xad, 0x8d, 0xf2, 0xfd, - 0xf5, 0xf5, 0x5a, 0xa3, 0x51, 0x59, 0x2d, 0x08, 0xa8, 0x00, 0xd9, 0xc0, 0x6d, 0xec, 0x31, 0xf6, - 0x0b, 0x68, 0xd7, 0x7e, 0x06, 0xc0, 0xfb, 0x45, 0x06, 0x22, 0x6b, 0xad, 0xf2, 0x96, 0xf4, 0x60, - 0xf9, 0xde, 0x56, 0xa5, 0x5e, 0x98, 0x40, 0x08, 0xf2, 0x2b, 0xcb, 0x8d, 0x72, 0x55, 0x12, 0x2b, - 0xf5, 0xcd, 0xfb, 0x1b, 0xf5, 0x8a, 0xf3, 0xcb, 0x69, 0xd7, 0x56, 0x21, 0xeb, 0x3f, 0x99, 0x8d, - 0x66, 0x61, 0xba, 0x5c, 0xad, 0x94, 0xd7, 0xa4, 0x07, 0xb5, 0x65, 0xe9, 0x8d, 0xad, 0xca, 0x56, - 0xa5, 0x30, 0x41, 0xab, 0x46, 0x89, 0xb7, 0xb7, 0xee, 0xdd, 0x2b, 0x08, 0x68, 0x1a, 0x32, 0xec, - 0x99, 0xde, 0xdc, 0x5e, 0x88, 0x5d, 0x5b, 0x87, 0x8c, 0xef, 0x22, 0x37, 0xf2, 0xb9, 0xcd, 0xad, - 0x7a, 0x55, 0x6a, 0xd4, 0xd6, 0x2b, 0xf5, 0xc6, 0xf2, 0xfa, 0x26, 0x93, 0x41, 0x69, 0xcb, 0x2b, - 0xf7, 0xc5, 0x46, 0x41, 0x70, 0x9f, 0x1b, 0xf7, 0xb7, 0xca, 0x55, 0xa7, 0x19, 0xa5, 0x44, 0x2a, - 0x5e, 0x88, 0x5f, 0x7b, 0x0c, 0x67, 0x07, 0x1c, 0x4f, 0x46, 0x19, 0x98, 0xda, 0xd2, 0xe9, 0xd5, - 0x55, 0x85, 0x09, 0x94, 0xf3, 0x9d, 0x50, 0x2e, 0x08, 0x28, 0xc5, 0xce, 0x9e, 0x16, 0x62, 0x28, - 0x09, 0xb1, 0xfa, 0xad, 0x42, 0x9c, 0x54, 0xd4, 0x77, 0xc0, 0xb7, 0x90, 0x40, 0x69, 0x7e, 0xfa, - 0xb1, 0x30, 0x89, 0xb2, 0xde, 0xf1, 0xc3, 0x42, 0xf2, 0xda, 0x79, 0xf0, 0x9d, 0xcd, 0x42, 0x00, - 0xc9, 0x7b, 0xb2, 0x8d, 0x2d, 0xbb, 0x30, 0x81, 0xa6, 0x20, 0xbe, 0xdc, 0x6a, 0x15, 0x84, 0x9b, - 0x7f, 0x22, 0x40, 0xca, 0xb9, 0x84, 0x1c, 0xdd, 0x83, 0x49, 0x86, 0xe4, 0x17, 0x06, 0xcf, 0x48, - 0xd4, 0xa9, 0xcd, 0x2d, 0x8e, 0x9a, 0xb2, 0x4a, 0x13, 0xe8, 0x4d, 0xfe, 0xd3, 0x8b, 0xc4, 0x62, - 0xd0, 0x85, 0x61, 0xf6, 0xe4, 0x48, 0x1d, 0x6e, 0x74, 0x64, 0x8c, 0x94, 0x26, 0x5e, 0x10, 0x56, - 0x9e, 0x7d, 0xff, 0x47, 0xf3, 0x13, 0xef, 0x1f, 0xcd, 0x0b, 0x1f, 0x1c, 0xcd, 0x0b, 0x3f, 0x3c, - 0x9a, 0x17, 0xfe, 0xf9, 0x68, 0x5e, 0xf8, 0xf5, 0x1f, 0xcf, 0x4f, 0x7c, 0xf0, 0xe3, 0xf9, 0x89, - 0x1f, 0xfe, 0x78, 0x7e, 0xe2, 0xed, 0x29, 0xce, 0xfd, 0x30, 0x49, 0x7f, 0x05, 0xf2, 0xd6, 0x7f, - 0x07, 0x00, 0x00, 0xff, 0xff, 0xf8, 0x98, 0xde, 0xeb, 0x0a, 0x73, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_47826aa7f6400ef0) } + +var fileDescriptor_api_47826aa7f6400ef0 = []byte{ + // 7125 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x24, 0xc7, + 0x75, 0x2e, 0x7b, 0x66, 0x38, 0x9c, 0x39, 0xf3, 0xc3, 0x61, 0x71, 0xb5, 0x3b, 0x4b, 0x49, 0x24, + 0x77, 0xf6, 0x57, 0x6b, 0x89, 0xab, 0xdd, 0xb5, 0xae, 0x64, 0x49, 0x96, 0x4d, 0x0e, 0x67, 0x77, + 0x66, 0xb9, 0xe4, 0x52, 0x3d, 0xc3, 0x95, 0x25, 0x5f, 0xdd, 0x76, 0xb3, 0xbb, 0x38, 0x6c, 0xef, + 0x4c, 0xf7, 0x6c, 0x77, 0x0f, 0x7f, 0x16, 0xb8, 0xb8, 0xb8, 0xf7, 0x02, 0x71, 0xe0, 0x04, 0x42, + 0x80, 0x04, 0x41, 0x60, 0x27, 0xb0, 0x00, 0x07, 0x49, 0x90, 0xc0, 0x06, 0x02, 0x04, 0x09, 0x12, + 0x24, 0xf0, 0x43, 0x5e, 0x14, 0xc3, 0x08, 0x84, 0x20, 0x81, 0x8d, 0x00, 0x21, 0x62, 0x1a, 0x48, + 0x0c, 0x3f, 0x04, 0xc8, 0x53, 0x00, 0x3d, 0x24, 0x41, 0xfd, 0xf4, 0xdf, 0x4c, 0xcf, 0x0f, 0xa9, + 0x56, 0xa2, 0x20, 0x4f, 0x64, 0x9f, 0xae, 0x73, 0xba, 0xea, 0xd4, 0xa9, 0x53, 0xe7, 0xab, 0x3a, + 0x55, 0x03, 0x33, 0xa6, 0x21, 0x2b, 0xbb, 0x9d, 0xed, 0x1b, 0x72, 0x47, 0x5b, 0xea, 0x98, 0x86, + 0x6d, 0xa0, 0x19, 0xc5, 0x50, 0x1e, 0x51, 0xf2, 0x12, 0x7f, 0x39, 0x87, 0x9c, 0x52, 0xaa, 0x6c, + 0xcb, 0xac, 0xd8, 0xdc, 0x19, 0x87, 0x86, 0x4d, 0xd3, 0x30, 0x2d, 0x4e, 0x3d, 0xeb, 0x50, 0xdb, + 0xd8, 0x96, 0x7d, 0xa5, 0x4b, 0x96, 0x6d, 0x98, 0x72, 0x13, 0xdf, 0xc0, 0x7a, 0x53, 0xd3, 0x9d, + 0x3f, 0xa4, 0xdc, 0x9e, 0xa2, 0xf0, 0x32, 0x17, 0x87, 0x95, 0xb9, 0xcd, 0x0b, 0x15, 0xbb, 0xb6, + 0xd6, 0xba, 0xb1, 0xdb, 0x52, 0x6e, 0xd8, 0x5a, 0x1b, 0x5b, 0xb6, 0xdc, 0xee, 0xf0, 0x37, 0x8b, + 0xf4, 0x8d, 0x6d, 0xca, 0x8a, 0xa6, 0x37, 0x6f, 0x98, 0x58, 0x31, 0x4c, 0x15, 0xab, 0x92, 0xd5, + 0x91, 0x75, 0xa7, 0xca, 0x4d, 0xa3, 0x69, 0xd0, 0x7f, 0x6f, 0x90, 0xff, 0x18, 0xb5, 0xf4, 0x7f, + 0x20, 0x2d, 0xca, 0x7a, 0x13, 0xd7, 0xf4, 0x1d, 0x03, 0xbd, 0x0e, 0x09, 0x15, 0x5b, 0x4a, 0x51, + 0x58, 0x14, 0xae, 0x65, 0x6e, 0x95, 0x96, 0xfa, 0x74, 0xb1, 0x44, 0xcb, 0xae, 0x62, 0x4b, 0x31, + 0xb5, 0x8e, 0x6d, 0x98, 0x2b, 0x89, 0x0f, 0x8e, 0x16, 0x26, 0x44, 0xca, 0x85, 0x3e, 0x0b, 0x93, + 0x2d, 0x2c, 0x5b, 0xb8, 0x18, 0xa3, 0xec, 0xc5, 0x10, 0xf6, 0xfb, 0xe4, 0x3d, 0x67, 0x62, 0x85, + 0x4b, 0x7f, 0x2e, 0x40, 0x4e, 0xc4, 0x8f, 0xbb, 0xd8, 0xb2, 0xab, 0x58, 0x56, 0xb1, 0x89, 0xce, + 0x43, 0xfc, 0x11, 0x3e, 0x2c, 0xc6, 0x17, 0x85, 0x6b, 0xd9, 0x95, 0xa9, 0x8f, 0x8e, 0x16, 0xe2, + 0x6b, 0xf8, 0x50, 0x24, 0x34, 0xb4, 0x08, 0x53, 0x58, 0x57, 0x25, 0xf2, 0x3a, 0x11, 0x7c, 0x9d, + 0xc4, 0xba, 0xba, 0x86, 0x0f, 0x91, 0x02, 0x29, 0x8b, 0x48, 0xd3, 0x15, 0x5c, 0x9c, 0x5c, 0x14, + 0xae, 0x4d, 0xae, 0xdc, 0xfd, 0xe8, 0x68, 0xa1, 0xdc, 0xd4, 0xec, 0xdd, 0xee, 0xf6, 0x92, 0x62, + 0xb4, 0x6f, 0xb8, 0xb5, 0x52, 0xb7, 0xbd, 0xff, 0x6f, 0x74, 0x1e, 0x35, 0x6f, 0x0c, 0xe8, 0x81, + 0xa5, 0xc6, 0x81, 0x5e, 0xc7, 0x8f, 0x45, 0x57, 0xf0, 0xab, 0x89, 0x9f, 0xbe, 0xbf, 0x20, 0xdc, + 0x4b, 0xa4, 0x84, 0x42, 0xec, 0x5e, 0x22, 0x15, 0x2b, 0xc4, 0x4b, 0xff, 0x1c, 0x83, 0xbc, 0x88, + 0xad, 0x8e, 0xa1, 0x5b, 0x98, 0x37, 0xe3, 0x45, 0x88, 0xdb, 0x07, 0x3a, 0x6d, 0x46, 0xe6, 0xd6, + 0x7c, 0x88, 0x32, 0x1a, 0xa6, 0xac, 0x5b, 0xb2, 0x62, 0x6b, 0x86, 0x2e, 0x92, 0xa2, 0xe8, 0x15, + 0xc8, 0x98, 0xd8, 0xea, 0xb6, 0x31, 0xed, 0x36, 0xda, 0xc2, 0xcc, 0xad, 0x73, 0x21, 0x9c, 0xf5, + 0x8e, 0xac, 0x8b, 0xc0, 0xca, 0x92, 0xff, 0xd1, 0x79, 0x48, 0xe9, 0xdd, 0x36, 0xd1, 0x8b, 0x45, + 0x5b, 0x1d, 0x17, 0xa7, 0xf4, 0x6e, 0x7b, 0x0d, 0x1f, 0x5a, 0xa8, 0x0c, 0x19, 0x93, 0x74, 0x9a, + 0xa4, 0xe9, 0x3b, 0x86, 0x55, 0x4c, 0x2e, 0xc6, 0xaf, 0x65, 0x6e, 0x3d, 0x33, 0xa8, 0x6b, 0x89, + 0x19, 0xf0, 0xfe, 0x01, 0xd3, 0x21, 0x58, 0xa8, 0x0e, 0x39, 0x5e, 0x33, 0x13, 0xcb, 0x96, 0xa1, + 0x17, 0xa7, 0x16, 0x85, 0x6b, 0xf9, 0x5b, 0x4b, 0x61, 0x62, 0x02, 0x5a, 0x20, 0x8f, 0xdd, 0x36, + 0x16, 0x29, 0x97, 0x98, 0x35, 0x7d, 0x4f, 0xa5, 0x37, 0x20, 0xeb, 0x7f, 0x8b, 0x10, 0xe4, 0xc5, + 0x4a, 0x7d, 0x6b, 0xbd, 0x22, 0x6d, 0x6d, 0xac, 0x6d, 0x3c, 0x78, 0x6b, 0xa3, 0x30, 0x81, 0xce, + 0x40, 0x81, 0xd3, 0xd6, 0x2a, 0x6f, 0x4b, 0xf7, 0x6b, 0xeb, 0xb5, 0x46, 0x41, 0x98, 0x4b, 0xfc, + 0xfc, 0xb7, 0xe7, 0x27, 0x4a, 0x0f, 0x01, 0xee, 0x62, 0x9b, 0xdb, 0x0e, 0x5a, 0x81, 0xe4, 0x2e, + 0xfd, 0x24, 0xb7, 0xde, 0xc5, 0xd0, 0xba, 0xf9, 0xec, 0x6c, 0x25, 0x45, 0x9a, 0xf9, 0xe1, 0xd1, + 0x82, 0x20, 0x72, 0x4e, 0xd6, 0xaf, 0xa5, 0xef, 0x09, 0x90, 0xa1, 0x82, 0x59, 0x43, 0x50, 0xb9, + 0x47, 0xf2, 0x85, 0x91, 0xad, 0xee, 0x17, 0x8d, 0x96, 0x60, 0x72, 0x4f, 0x6e, 0x75, 0x87, 0x0d, + 0x8e, 0x87, 0xe4, 0xbd, 0xc8, 0x8a, 0xa1, 0xd7, 0x20, 0xab, 0xe9, 0x36, 0xd6, 0x6d, 0x89, 0xb1, + 0xc5, 0x47, 0xb0, 0x65, 0x58, 0x69, 0xfa, 0x50, 0xfa, 0x13, 0x01, 0x60, 0xb3, 0x1b, 0xa5, 0x6a, + 0xc8, 0xe0, 0x1e, 0xab, 0xfe, 0xce, 0xe0, 0x66, 0xad, 0x38, 0x0b, 0x49, 0x4d, 0x6f, 0x69, 0x3a, + 0xab, 0x7f, 0x4a, 0xe4, 0x4f, 0xe8, 0x0c, 0x4c, 0x6e, 0xb7, 0x34, 0x5d, 0xa5, 0x36, 0x9e, 0x12, + 0xd9, 0x03, 0x57, 0xbf, 0x08, 0x19, 0x5a, 0xf7, 0x08, 0xb5, 0x5f, 0xfa, 0x46, 0x0c, 0x9e, 0x2a, + 0x1b, 0xba, 0xaa, 0x91, 0xc1, 0x26, 0xb7, 0x3e, 0x15, 0xba, 0x79, 0x09, 0xd2, 0xf8, 0xa0, 0x33, + 0x66, 0xf7, 0xa6, 0xf0, 0x41, 0x87, 0xfe, 0x17, 0xae, 0x3a, 0xf4, 0x59, 0x38, 0x27, 0xb7, 0x5a, + 0xc6, 0xbe, 0xa4, 0xed, 0x48, 0xaa, 0x81, 0x2d, 0x49, 0x37, 0x6c, 0x09, 0x1f, 0x68, 0x96, 0x4d, + 0xfd, 0x41, 0x4a, 0x9c, 0xa5, 0xaf, 0x6b, 0x3b, 0xab, 0x06, 0xb6, 0x36, 0x0c, 0xbb, 0x42, 0x5e, + 0x71, 0x85, 0xbf, 0x0b, 0x67, 0x7b, 0x75, 0x13, 0xa5, 0xee, 0xff, 0x4a, 0x80, 0x7c, 0x4d, 0xd7, + 0xec, 0x4f, 0x85, 0xd2, 0x5d, 0xed, 0xc5, 0xfd, 0xda, 0xbb, 0x0e, 0x85, 0x1d, 0x59, 0x6b, 0x3d, + 0xd0, 0x1b, 0x46, 0x7b, 0xdb, 0xb2, 0x0d, 0x1d, 0x5b, 0x5c, 0xbd, 0x7d, 0x74, 0xae, 0xb3, 0x87, + 0x30, 0xed, 0xb6, 0x29, 0x4a, 0x65, 0x3d, 0x81, 0x42, 0x4d, 0x57, 0x4c, 0xdc, 0xc6, 0x7a, 0xa4, + 0xda, 0x7a, 0x06, 0xd2, 0x9a, 0x23, 0x97, 0x6a, 0x2c, 0x2e, 0x7a, 0x04, 0xde, 0xa6, 0x2e, 0xcc, + 0xf8, 0xbe, 0x1d, 0xa5, 0xf3, 0x7b, 0x1a, 0xd2, 0x3a, 0xde, 0x97, 0xbc, 0xfe, 0x8a, 0x8b, 0x29, + 0x1d, 0xef, 0x33, 0x67, 0xf5, 0x36, 0xe4, 0x56, 0x71, 0x0b, 0xdb, 0x38, 0x7a, 0x4f, 0xbe, 0x05, + 0x79, 0x47, 0x74, 0x94, 0x9d, 0xf4, 0x1b, 0x02, 0x20, 0x2e, 0x97, 0x4c, 0x91, 0x51, 0xf6, 0xd3, + 0x02, 0x09, 0x01, 0xec, 0xae, 0xa9, 0xb3, 0xb9, 0x9c, 0x59, 0x29, 0x30, 0x12, 0x9d, 0xce, 0x3d, + 0x8f, 0x9a, 0xf0, 0x7b, 0x54, 0x37, 0x24, 0x21, 0xc1, 0xc8, 0x3e, 0xcc, 0x06, 0xaa, 0x17, 0x6d, + 0x57, 0x26, 0x68, 0xcd, 0x62, 0x8b, 0x71, 0x7f, 0xf8, 0x45, 0x89, 0xa5, 0x77, 0x61, 0xa6, 0xdc, + 0xc2, 0xb2, 0x19, 0xb5, 0x5a, 0x78, 0x77, 0xbe, 0x0d, 0xc8, 0x2f, 0x3e, 0xca, 0x2e, 0xfd, 0x4d, + 0x01, 0x90, 0x88, 0xf7, 0xb0, 0x69, 0x47, 0xde, 0xa5, 0xab, 0x90, 0xb1, 0x65, 0xb3, 0x89, 0x6d, + 0x89, 0xc4, 0xec, 0xdc, 0x5d, 0x3d, 0xeb, 0x13, 0x44, 0x22, 0xf7, 0xa5, 0xdd, 0x96, 0xb2, 0xd4, + 0x70, 0x62, 0x7a, 0x27, 0x02, 0x63, 0x7c, 0x84, 0xcc, 0x35, 0xf0, 0x0e, 0xcc, 0x06, 0x6a, 0x19, + 0xa5, 0x0a, 0xbe, 0x2d, 0x40, 0xa6, 0xae, 0xc8, 0x7a, 0x94, 0x6d, 0x7f, 0x03, 0x32, 0x96, 0x22, + 0xeb, 0xd2, 0x8e, 0x61, 0xb6, 0x65, 0x9b, 0x9a, 0x6c, 0x3e, 0xd0, 0x76, 0x37, 0xa2, 0x55, 0x64, + 0xfd, 0x0e, 0x2d, 0x24, 0x82, 0xe5, 0xfe, 0xef, 0xb7, 0xea, 0x7b, 0x89, 0x54, 0xbc, 0x90, 0x28, + 0xfd, 0x8b, 0x00, 0x59, 0x56, 0xcb, 0x28, 0xad, 0xfa, 0x25, 0x48, 0x98, 0xc6, 0x3e, 0xb3, 0xea, + 0xcc, 0xad, 0xa7, 0x43, 0x44, 0xac, 0xe1, 0x43, 0xff, 0x74, 0x42, 0x8b, 0xa3, 0x15, 0xe0, 0x61, + 0x97, 0x44, 0xb9, 0xe3, 0xe3, 0x72, 0x03, 0xe3, 0x12, 0x89, 0x8c, 0xab, 0x30, 0xbd, 0x2d, 0xdb, + 0xca, 0xae, 0x64, 0xf2, 0x4a, 0x92, 0xa9, 0x27, 0x7e, 0x2d, 0x2b, 0xe6, 0x29, 0xd9, 0xa9, 0xba, + 0x55, 0xfa, 0x5d, 0xc7, 0x44, 0x2d, 0xfc, 0xe9, 0xef, 0xa6, 0x7f, 0x15, 0xb8, 0xa5, 0x3a, 0x95, + 0xfd, 0xef, 0xd6, 0x5b, 0xdf, 0x8a, 0xc1, 0xb9, 0xf2, 0x2e, 0x56, 0x1e, 0x95, 0x0d, 0xdd, 0xd2, + 0x2c, 0x1b, 0xeb, 0xca, 0x61, 0x94, 0x5d, 0xf6, 0x34, 0xa4, 0xf7, 0x35, 0x7b, 0x57, 0x52, 0xb5, + 0x9d, 0x1d, 0xea, 0x53, 0x52, 0x62, 0x8a, 0x10, 0x56, 0xb5, 0x9d, 0x1d, 0x74, 0x1b, 0x12, 0x6d, + 0x43, 0x65, 0x51, 0x65, 0xfe, 0xd6, 0x42, 0x88, 0x78, 0x5a, 0x35, 0xab, 0xdb, 0x5e, 0x37, 0x54, + 0x2c, 0xd2, 0xc2, 0x68, 0x1e, 0x40, 0x21, 0xd4, 0x8e, 0xa1, 0xe9, 0x36, 0x9f, 0x5d, 0x7c, 0x14, + 0x54, 0x85, 0xb4, 0x8d, 0xcd, 0xb6, 0xa6, 0xcb, 0x36, 0x81, 0xd6, 0x44, 0x79, 0x97, 0x42, 0x2b, + 0xde, 0x69, 0x69, 0x8a, 0xdc, 0xb7, 0x46, 0xe0, 0x31, 0x73, 0x5f, 0xf6, 0x5e, 0x02, 0x8a, 0xfd, + 0x1a, 0x8a, 0xd2, 0x4e, 0x36, 0x21, 0x49, 0x00, 0x67, 0xcb, 0xe6, 0x96, 0x72, 0x6b, 0x90, 0x22, + 0x42, 0x6a, 0x40, 0x81, 0x6b, 0xcb, 0xe6, 0x95, 0xe7, 0x72, 0xe6, 0xfe, 0x54, 0x80, 0x24, 0x7b, + 0x81, 0x6e, 0x42, 0x8a, 0xe3, 0x6a, 0x95, 0xd6, 0x31, 0xbe, 0x72, 0xf6, 0xf8, 0x68, 0x61, 0x8a, + 0xa1, 0xe8, 0xd5, 0x8f, 0xbc, 0x7f, 0xc5, 0x29, 0x06, 0xa4, 0x55, 0xd2, 0x67, 0x96, 0x2d, 0x9b, + 0x36, 0x5d, 0xbf, 0x20, 0x7d, 0x96, 0x15, 0x53, 0x94, 0xb0, 0x86, 0x0f, 0xd1, 0x3d, 0x48, 0x5a, + 0xb6, 0x6c, 0x77, 0x2d, 0xde, 0x6b, 0x27, 0xaa, 0x6c, 0x9d, 0x72, 0x8a, 0x5c, 0x02, 0x09, 0x12, + 0x54, 0x6c, 0xcb, 0x5a, 0x8b, 0x76, 0x63, 0x5a, 0xe4, 0x4f, 0xa5, 0x6f, 0x0a, 0x90, 0x64, 0x45, + 0xd1, 0x39, 0x98, 0x15, 0x97, 0x37, 0xee, 0x56, 0xa4, 0xda, 0xc6, 0x6a, 0xa5, 0x51, 0x11, 0xd7, + 0x6b, 0x1b, 0xcb, 0x8d, 0x4a, 0x61, 0x02, 0x9d, 0x05, 0xe4, 0xbc, 0x28, 0x3f, 0xd8, 0xa8, 0xd7, + 0xea, 0x8d, 0xca, 0x46, 0xa3, 0x20, 0x50, 0x24, 0x4e, 0xe9, 0x3e, 0x6a, 0x0c, 0x5d, 0x82, 0xc5, + 0x5e, 0xaa, 0x54, 0x6f, 0x2c, 0x37, 0xea, 0x52, 0xa5, 0xde, 0xa8, 0xad, 0x2f, 0x37, 0x2a, 0xab, + 0x85, 0xf8, 0x90, 0x52, 0xe4, 0x23, 0xa2, 0x58, 0x29, 0x37, 0x0a, 0x89, 0xd2, 0x13, 0x78, 0x4a, + 0xc4, 0x8a, 0xd1, 0xee, 0x74, 0x6d, 0x4c, 0x6a, 0x69, 0x45, 0x39, 0x5e, 0xce, 0xc1, 0x94, 0x6a, + 0x1e, 0x4a, 0x66, 0x57, 0xe7, 0xa3, 0x25, 0xa9, 0x9a, 0x87, 0x62, 0x57, 0xe7, 0xc6, 0xf8, 0x47, + 0x02, 0x9c, 0xed, 0xfd, 0x78, 0x94, 0xa6, 0xf8, 0x25, 0xc8, 0xc8, 0xaa, 0x8a, 0x55, 0x49, 0xc5, + 0x2d, 0x5b, 0xe6, 0x41, 0xc0, 0x4d, 0x9f, 0x24, 0xbe, 0xf6, 0xb4, 0xc4, 0x16, 0x9d, 0x96, 0xdc, + 0xb5, 0xa7, 0xf5, 0x87, 0xe5, 0x32, 0xad, 0xcf, 0x2a, 0x61, 0x74, 0x3c, 0x12, 0x95, 0x45, 0x29, + 0xa5, 0x3f, 0x4c, 0x40, 0xae, 0xa2, 0xab, 0x8d, 0x83, 0x48, 0x67, 0x84, 0xb3, 0x90, 0x54, 0x8c, + 0x76, 0x5b, 0xb3, 0x1d, 0x6d, 0xb1, 0x27, 0xf4, 0x39, 0x48, 0xa9, 0x58, 0x56, 0x5d, 0x48, 0x3f, + 0x2a, 0x92, 0x11, 0xdd, 0xe2, 0xe8, 0x2b, 0x70, 0x8e, 0x38, 0x52, 0x53, 0x97, 0x5b, 0x12, 0x93, + 0x26, 0xd9, 0xa6, 0xd6, 0x6c, 0x62, 0x93, 0xaf, 0x74, 0x5d, 0x0b, 0xa9, 0x67, 0x8d, 0x73, 0x94, + 0x29, 0x43, 0x83, 0x95, 0x17, 0x9f, 0xd2, 0xc2, 0xc8, 0xe8, 0x8b, 0xee, 0x9a, 0x89, 0xd5, 0x91, + 0x75, 0x8b, 0x3b, 0xa9, 0x41, 0x0b, 0x68, 0x5c, 0x97, 0x7c, 0x4e, 0x20, 0x14, 0x0b, 0xdd, 0x20, + 0xe1, 0xf7, 0xe3, 0xae, 0x66, 0x62, 0xe9, 0x66, 0x47, 0x29, 0x26, 0x49, 0xdb, 0x57, 0xf2, 0xc7, + 0x47, 0x0b, 0x20, 0x32, 0xf2, 0xcd, 0xcd, 0x32, 0x09, 0xc7, 0xd9, 0xff, 0x1d, 0x05, 0xad, 0xc0, + 0x3c, 0x99, 0x38, 0x79, 0x7b, 0x64, 0x5b, 0xda, 0xd5, 0x9a, 0xbb, 0xd8, 0x94, 0xdc, 0xe5, 0xd9, + 0x62, 0x8a, 0xea, 0x6f, 0x4e, 0x91, 0x75, 0x56, 0xd9, 0x65, 0xbb, 0x4a, 0x8b, 0xb8, 0x2a, 0x22, + 0xba, 0xee, 0x18, 0x9a, 0x65, 0xe8, 0xc5, 0x34, 0xd3, 0x35, 0x7b, 0x42, 0x6f, 0x42, 0x41, 0xd3, + 0xa5, 0x9d, 0x96, 0xd6, 0xdc, 0xb5, 0xa5, 0x7d, 0x53, 0xb3, 0xb1, 0x55, 0x9c, 0xa1, 0x4d, 0x0a, + 0x33, 0xc1, 0x3a, 0x5f, 0x9c, 0x54, 0xdf, 0x22, 0x25, 0x79, 0xe3, 0xf2, 0x9a, 0x7e, 0x87, 0xf2, + 0x53, 0xa2, 0xe5, 0x4e, 0xd4, 0x53, 0x85, 0x54, 0xe9, 0xef, 0x04, 0xc8, 0x3b, 0x86, 0x13, 0xa5, + 0xa9, 0x5f, 0x83, 0x82, 0xa1, 0x63, 0xa9, 0xb3, 0x2b, 0x5b, 0x98, 0x2b, 0x86, 0xcf, 0x26, 0x79, + 0x43, 0xc7, 0x9b, 0x84, 0xcc, 0x34, 0x81, 0x36, 0x61, 0xc6, 0xb2, 0xe5, 0xa6, 0xa6, 0x37, 0x7d, + 0xfa, 0x9a, 0x1c, 0x3f, 0x3e, 0x2e, 0x70, 0x6e, 0x97, 0x1e, 0x08, 0x41, 0x7e, 0x28, 0xc0, 0xcc, + 0xb2, 0xda, 0xd6, 0xf4, 0x7a, 0xa7, 0xa5, 0x45, 0x0a, 0xa6, 0x2f, 0x41, 0xda, 0x22, 0x32, 0x3d, + 0x3f, 0xee, 0x01, 0xa1, 0x14, 0x7d, 0x43, 0x1c, 0xfa, 0x7d, 0x98, 0xc6, 0x07, 0x1d, 0xcd, 0x94, + 0x6d, 0xcd, 0xd0, 0x59, 0xec, 0x9f, 0x18, 0xbf, 0x6d, 0x79, 0x8f, 0xd7, 0x8b, 0xff, 0x79, 0xcb, + 0xde, 0x06, 0xe4, 0x6f, 0x58, 0x94, 0x20, 0x40, 0x82, 0x59, 0x2a, 0x7a, 0x4b, 0xb7, 0x22, 0xd6, + 0x1a, 0x77, 0xb4, 0x5f, 0x86, 0x33, 0xc1, 0x0f, 0x44, 0x59, 0xfb, 0x77, 0x79, 0x8f, 0xaf, 0x63, + 0xf3, 0x13, 0xc2, 0x9f, 0x7e, 0xf1, 0x51, 0xd6, 0xfc, 0xeb, 0x02, 0x9c, 0xa7, 0xb2, 0xe9, 0xa6, + 0xc0, 0x0e, 0x36, 0xe9, 0x4e, 0x49, 0x94, 0x46, 0x7b, 0x11, 0x92, 0x0c, 0x4e, 0x52, 0x8b, 0x9d, + 0x5c, 0xc9, 0x90, 0x10, 0xa5, 0x6e, 0x1b, 0x26, 0x09, 0x51, 0xf8, 0x2b, 0xde, 0x4e, 0x19, 0xe6, + 0xc2, 0xea, 0x12, 0x31, 0xde, 0x9e, 0xe1, 0x91, 0x22, 0x31, 0xf1, 0xf2, 0x2e, 0x09, 0x91, 0x50, + 0x05, 0x32, 0x0a, 0xfd, 0x4f, 0xb2, 0x0f, 0x3b, 0x98, 0xca, 0xcf, 0x0f, 0x0b, 0x32, 0x19, 0x5b, + 0xe3, 0xb0, 0x83, 0x49, 0xa4, 0xea, 0xfc, 0x4f, 0xd4, 0xe5, 0x6b, 0xea, 0xd0, 0x30, 0x95, 0x8e, + 0x2f, 0x5a, 0xd6, 0x89, 0xf4, 0x02, 0x9a, 0xf8, 0xe3, 0x38, 0x57, 0x05, 0xfb, 0x12, 0x67, 0x8a, + 0x34, 0x30, 0x79, 0x07, 0xce, 0xaa, 0xb8, 0x63, 0x62, 0x45, 0xb6, 0xb1, 0x2a, 0xf9, 0x9b, 0x1f, + 0x3b, 0x41, 0xf3, 0xcf, 0x78, 0x32, 0x3c, 0x2a, 0x7a, 0x1b, 0x90, 0x4f, 0x36, 0x6b, 0x99, 0x03, + 0x7c, 0x4e, 0xa2, 0x94, 0x19, 0x4f, 0x0a, 0xa3, 0x5b, 0xa8, 0x0c, 0x29, 0x7c, 0xd0, 0x91, 0xe8, + 0x76, 0x61, 0xe2, 0x84, 0xdb, 0x85, 0x53, 0xf8, 0xa0, 0x43, 0x88, 0x68, 0x8b, 0xcc, 0x70, 0x4e, + 0x48, 0x40, 0xab, 0x6d, 0x8d, 0x46, 0x16, 0x9e, 0xbd, 0x70, 0x71, 0xd3, 0x6e, 0x34, 0xc0, 0x44, + 0xf0, 0xbe, 0x7b, 0x5f, 0x80, 0xa7, 0x43, 0xfb, 0x2e, 0xca, 0xc9, 0xce, 0xd9, 0x31, 0x8d, 0x9d, + 0x66, 0xc7, 0xb4, 0xf4, 0x7b, 0xce, 0xa8, 0x17, 0x71, 0xcb, 0x20, 0xea, 0xfd, 0x04, 0x16, 0x9f, + 0xa6, 0x9c, 0x6e, 0x8f, 0x9d, 0xb8, 0xdb, 0x1d, 0xd6, 0x1e, 0xb7, 0xd0, 0x53, 0xd9, 0x28, 0xdd, + 0xc2, 0xaf, 0x0a, 0x30, 0x5b, 0xc5, 0xb2, 0x69, 0x6f, 0x63, 0xd9, 0x8e, 0x38, 0xa4, 0x7d, 0x09, + 0xe2, 0xba, 0xb1, 0x7f, 0x92, 0xf5, 0x37, 0x52, 0xde, 0x9b, 0xb6, 0x82, 0xf5, 0x8a, 0xb2, 0xd5, + 0x7f, 0x11, 0x83, 0xf4, 0xdd, 0x72, 0x94, 0x6d, 0x7d, 0x9d, 0xaf, 0xd2, 0xb2, 0xa1, 0x1e, 0x66, + 0x96, 0xee, 0xf7, 0x96, 0xee, 0x96, 0xd7, 0xf0, 0xa1, 0x63, 0x96, 0x84, 0x0b, 0x2d, 0x43, 0xda, + 0xde, 0x35, 0xb1, 0xb5, 0x6b, 0xb4, 0xd4, 0x93, 0xc4, 0x2c, 0x1e, 0xd7, 0xdc, 0x23, 0x98, 0xa4, + 0x72, 0x9d, 0xcd, 0x7c, 0x21, 0x64, 0x33, 0x9f, 0x7c, 0xc6, 0x0d, 0xfb, 0x62, 0x27, 0xf9, 0x8c, + 0x43, 0x60, 0x9d, 0xe3, 0xc6, 0x46, 0x93, 0x85, 0x64, 0xe9, 0x4d, 0x00, 0xd2, 0xb4, 0x28, 0xbb, + 0xe7, 0x97, 0xe3, 0x90, 0xdf, 0xec, 0x5a, 0xbb, 0x11, 0xdb, 0x63, 0x19, 0xa0, 0xd3, 0xb5, 0x28, + 0x58, 0x38, 0xd0, 0x79, 0xfb, 0x47, 0xa4, 0x09, 0x38, 0x0a, 0x60, 0x7c, 0x8d, 0x03, 0x1d, 0x3d, + 0xe0, 0x42, 0xb0, 0xe4, 0xe5, 0x1a, 0x5c, 0x1f, 0x03, 0x56, 0x36, 0x0e, 0xf4, 0x75, 0xec, 0xe2, + 0x49, 0x26, 0x10, 0x13, 0x81, 0xaf, 0xc3, 0x14, 0x79, 0x90, 0x6c, 0xe3, 0x24, 0x3d, 0x9f, 0x24, + 0x3c, 0x0d, 0x03, 0xbd, 0x06, 0x69, 0xc6, 0x4d, 0xe6, 0xaf, 0x24, 0x9d, 0xbf, 0xc2, 0x9a, 0xc4, + 0xb5, 0x49, 0x67, 0xae, 0x14, 0x65, 0x25, 0xb3, 0xd5, 0x19, 0x98, 0xdc, 0x31, 0x4c, 0x05, 0xd3, + 0xe4, 0x82, 0x94, 0xc8, 0x1e, 0xfc, 0x9d, 0x7b, 0x2f, 0x91, 0x4a, 0x15, 0xd2, 0xf7, 0x12, 0xa9, + 0x74, 0x01, 0x4a, 0xdf, 0x14, 0x60, 0xda, 0xed, 0x95, 0x28, 0x5d, 0x7a, 0x39, 0xa0, 0xd2, 0x93, + 0xf7, 0x0b, 0x51, 0x63, 0xe9, 0xaf, 0x69, 0x7c, 0xa3, 0x18, 0x7b, 0xb4, 0x9b, 0xa2, 0x34, 0x9b, + 0x15, 0x96, 0x56, 0x12, 0x3b, 0x65, 0x57, 0xd3, 0x44, 0x93, 0x9b, 0x70, 0x46, 0x6b, 0x13, 0x9f, + 0xaf, 0xd9, 0xad, 0x43, 0x8e, 0xd1, 0x6c, 0xec, 0x6c, 0x8a, 0xce, 0x7a, 0xef, 0xca, 0xce, 0x2b, + 0xee, 0x06, 0xd9, 0x36, 0x89, 0xd7, 0xac, 0x28, 0xf5, 0x5e, 0x83, 0x9c, 0xc9, 0x44, 0x93, 0x58, + 0xe5, 0x84, 0xaa, 0xcf, 0xba, 0xac, 0x44, 0xfb, 0xdf, 0x8d, 0xc1, 0xf4, 0x9b, 0x5d, 0x6c, 0x1e, + 0x7e, 0x0a, 0x75, 0x7f, 0x05, 0xa6, 0xf7, 0x65, 0xcd, 0x96, 0x76, 0x0c, 0x53, 0xea, 0x76, 0x54, + 0xd9, 0x76, 0x72, 0x23, 0x72, 0x84, 0x7c, 0xc7, 0x30, 0xb7, 0x28, 0x11, 0x61, 0x40, 0x8f, 0x74, + 0x63, 0x5f, 0x97, 0x08, 0x99, 0x42, 0xe4, 0x03, 0x9d, 0x2f, 0x36, 0xaf, 0xbc, 0xfc, 0xb7, 0x47, + 0x0b, 0xb7, 0xc7, 0x4a, 0x69, 0xa2, 0x59, 0x61, 0xdd, 0xae, 0xa6, 0x2e, 0x6d, 0x6d, 0xd5, 0x56, + 0xc5, 0x02, 0x15, 0xf9, 0x16, 0x93, 0xd8, 0x38, 0xd0, 0x9d, 0xa9, 0xfd, 0x23, 0x01, 0x0a, 0x9e, + 0xc2, 0xa2, 0xec, 0xd5, 0x0a, 0x64, 0x1e, 0x77, 0xb1, 0xa9, 0x9d, 0xa2, 0x4f, 0x81, 0x33, 0x12, + 0xb7, 0xf4, 0x0e, 0x64, 0x03, 0x7a, 0x88, 0x7f, 0x3c, 0x3d, 0x64, 0xf6, 0x3d, 0x15, 0x94, 0x7e, + 0x20, 0x00, 0xa2, 0x8d, 0xaf, 0xb1, 0x75, 0xfe, 0x4f, 0x99, 0xc1, 0x5c, 0x83, 0x02, 0x4d, 0x32, + 0x94, 0xb4, 0x1d, 0xa9, 0xad, 0x59, 0x96, 0xa6, 0x37, 0xb9, 0xc5, 0xe4, 0x29, 0xbd, 0xb6, 0xb3, + 0xce, 0xa8, 0xbc, 0x2f, 0xff, 0x37, 0xcc, 0x06, 0x5a, 0x13, 0x65, 0x6f, 0x5e, 0x80, 0xec, 0x8e, + 0xd1, 0xd5, 0x55, 0x89, 0x2d, 0x9a, 0xf1, 0xc5, 0xc1, 0x0c, 0xa5, 0xb1, 0xef, 0x95, 0xfe, 0x2d, + 0x06, 0x67, 0x44, 0x6c, 0x19, 0xad, 0x3d, 0x1c, 0xbd, 0x3e, 0x1f, 0x00, 0xdf, 0x8c, 0x91, 0x3e, + 0x8e, 0x5a, 0xd3, 0x4c, 0x06, 0x9b, 0xee, 0x82, 0xab, 0xee, 0x97, 0x86, 0x5b, 0x66, 0xff, 0x3a, + 0x3b, 0x5f, 0xb9, 0x4b, 0x04, 0x56, 0xee, 0xf6, 0x60, 0x5a, 0x6b, 0xea, 0x06, 0x71, 0x64, 0x16, + 0x7e, 0xac, 0x77, 0xdb, 0x0e, 0xac, 0x79, 0x69, 0x8c, 0xba, 0xd6, 0x18, 0x67, 0x1d, 0x3f, 0xde, + 0xe8, 0xb6, 0x69, 0x8c, 0xbd, 0x72, 0x96, 0x54, 0xfb, 0xf8, 0x68, 0x21, 0x1f, 0x78, 0x67, 0x89, + 0x79, 0xcd, 0x7d, 0x26, 0x1f, 0xe1, 0x06, 0xf0, 0x3f, 0xe1, 0xa9, 0x9e, 0x0e, 0x88, 0x32, 0x1a, + 0xfa, 0xcb, 0x38, 0x9c, 0x0f, 0x8a, 0x8f, 0x1a, 0xb3, 0xfc, 0x17, 0xe9, 0xe4, 0x2a, 0xe4, 0xda, + 0x9a, 0x7e, 0xba, 0x95, 0xcb, 0x6c, 0x5b, 0xd3, 0xbd, 0x05, 0xe0, 0x10, 0x73, 0x49, 0xfe, 0xc7, + 0x99, 0x8b, 0x0c, 0x73, 0x61, 0xfd, 0x19, 0xa5, 0xcd, 0xbc, 0x27, 0x40, 0x36, 0xea, 0x35, 0xb9, + 0xd3, 0x25, 0x80, 0xf1, 0x36, 0x37, 0x20, 0xf7, 0x09, 0x2c, 0xe2, 0xfd, 0xb6, 0x00, 0xa8, 0x61, + 0x76, 0x75, 0x02, 0x8e, 0xef, 0x1b, 0xcd, 0x28, 0x1b, 0x7b, 0x06, 0x26, 0x35, 0x5d, 0xc5, 0x07, + 0xb4, 0xb1, 0x09, 0x91, 0x3d, 0x04, 0xf6, 0x20, 0xe3, 0x63, 0xed, 0x41, 0x7a, 0x79, 0x24, 0x81, + 0x8a, 0x46, 0xa9, 0x85, 0xef, 0xc6, 0x60, 0x96, 0x37, 0x27, 0xf2, 0x45, 0xcc, 0x53, 0xa5, 0x98, + 0xa3, 0xcf, 0x03, 0x74, 0x4c, 0xbc, 0x27, 0x31, 0xd6, 0xf8, 0x58, 0xac, 0x69, 0xc2, 0x41, 0x09, + 0xe8, 0x4b, 0x30, 0x4d, 0x06, 0x7a, 0xc7, 0x34, 0x3a, 0x86, 0x45, 0xe2, 0x18, 0x6b, 0x3c, 0x68, + 0x34, 0x73, 0x7c, 0xb4, 0x90, 0x5b, 0xd7, 0xf4, 0x4d, 0xce, 0xd8, 0xa8, 0x8b, 0xc4, 0x63, 0xb8, + 0x8f, 0xce, 0x00, 0xfc, 0x1b, 0x01, 0xce, 0x7c, 0x62, 0xcb, 0xbe, 0xff, 0x19, 0x1a, 0x73, 0xe7, + 0xa1, 0x02, 0x7d, 0xac, 0xe9, 0x3b, 0x46, 0xf4, 0x8b, 0xf1, 0xef, 0x09, 0x30, 0xe3, 0x13, 0x1f, + 0x65, 0x94, 0x73, 0xba, 0x83, 0x0c, 0x5f, 0x26, 0x71, 0x8f, 0xdf, 0xec, 0xa3, 0x1c, 0x54, 0x7f, + 0x16, 0x83, 0xb3, 0x65, 0xb6, 0x3b, 0xed, 0xa4, 0x6e, 0x44, 0x69, 0x25, 0x45, 0x98, 0xda, 0xc3, + 0xa6, 0xa5, 0x19, 0x6c, 0xbe, 0xcd, 0x89, 0xce, 0x23, 0x9a, 0x83, 0x94, 0xa5, 0xcb, 0x1d, 0x6b, + 0xd7, 0x70, 0x76, 0xf1, 0xdc, 0x67, 0x37, 0xcd, 0x64, 0xf2, 0xf4, 0x69, 0x26, 0xc9, 0xe1, 0x69, + 0x26, 0x53, 0x1f, 0x3b, 0xcd, 0x84, 0x6f, 0x99, 0x7d, 0x5f, 0x80, 0x73, 0x7d, 0xfa, 0x8b, 0xd2, + 0x66, 0xbe, 0x0a, 0x19, 0x85, 0x0b, 0x26, 0xde, 0x98, 0xed, 0x0a, 0xd6, 0x48, 0xb1, 0x53, 0x62, + 0x94, 0xe3, 0xa3, 0x05, 0x70, 0xaa, 0x5a, 0x5b, 0xe5, 0x2a, 0x22, 0xff, 0xab, 0xa5, 0x5f, 0xc8, + 0xc0, 0x74, 0xe5, 0x80, 0xad, 0x79, 0xd7, 0x59, 0x58, 0x80, 0xee, 0x40, 0xaa, 0x63, 0x1a, 0x7b, + 0x9a, 0xd3, 0x8c, 0x7c, 0x20, 0x64, 0x72, 0x9a, 0xd1, 0xc3, 0xb5, 0xc9, 0x39, 0x44, 0x97, 0x17, + 0x35, 0x20, 0x7d, 0xdf, 0x50, 0xe4, 0xd6, 0x1d, 0xad, 0xe5, 0xd8, 0xff, 0x8b, 0xa3, 0x05, 0x2d, + 0xb9, 0x3c, 0x9b, 0xb2, 0xbd, 0xeb, 0x74, 0x85, 0x4b, 0x44, 0x35, 0x48, 0x55, 0x6d, 0xbb, 0x43, + 0x5e, 0x72, 0x6f, 0x72, 0x75, 0x0c, 0xa1, 0x84, 0x85, 0xcb, 0x72, 0xd9, 0x51, 0x03, 0x66, 0xee, + 0x1a, 0x46, 0xb3, 0x85, 0xcb, 0x2d, 0xa3, 0xab, 0x96, 0x0d, 0x7d, 0x47, 0x6b, 0x72, 0x7f, 0x7c, + 0x65, 0x0c, 0x99, 0x77, 0xcb, 0x75, 0xb1, 0x5f, 0x00, 0x5a, 0x86, 0x54, 0xfd, 0x36, 0x17, 0xc6, + 0xe2, 0xb8, 0xcb, 0x63, 0x08, 0xab, 0xdf, 0x16, 0x5d, 0x36, 0x74, 0x0f, 0x32, 0xcb, 0x4f, 0xba, + 0x26, 0xe6, 0x52, 0x92, 0x03, 0x73, 0x1a, 0x7a, 0xa5, 0x50, 0x2e, 0xd1, 0xcf, 0x8c, 0xea, 0x90, + 0x7f, 0xcb, 0x30, 0x1f, 0xb5, 0x0c, 0xd9, 0x69, 0xe1, 0x14, 0x15, 0xf7, 0x99, 0x31, 0xc4, 0x39, + 0x8c, 0x62, 0x8f, 0x88, 0xb9, 0x2f, 0x41, 0x2e, 0xd0, 0x4d, 0x08, 0x41, 0xa2, 0x43, 0x7a, 0x44, + 0xa0, 0x49, 0x42, 0xf4, 0x7f, 0xf4, 0x02, 0x4c, 0xe9, 0x86, 0x8a, 0x1d, 0x1b, 0xce, 0xad, 0x9c, + 0x39, 0x3e, 0x5a, 0x48, 0x6e, 0x18, 0x2a, 0x0b, 0x28, 0xf8, 0x7f, 0x62, 0x92, 0x14, 0x72, 0xc2, + 0x89, 0xb9, 0x2b, 0x90, 0x20, 0xfd, 0x43, 0xdc, 0xc8, 0xb6, 0x6c, 0xe1, 0x2d, 0x53, 0xe3, 0x32, + 0x9d, 0x47, 0x5e, 0xee, 0x47, 0x02, 0xc4, 0xea, 0xb7, 0x49, 0x44, 0xbd, 0xdd, 0x55, 0x1e, 0x61, + 0x9b, 0x97, 0xe2, 0x4f, 0x34, 0xd2, 0x36, 0xf1, 0x8e, 0xc6, 0xa2, 0x9c, 0xb4, 0xc8, 0x9f, 0xd0, + 0xb3, 0x00, 0xb2, 0xa2, 0x60, 0xcb, 0x92, 0x9c, 0x73, 0x61, 0x69, 0x31, 0xcd, 0x28, 0x6b, 0xf8, + 0x90, 0xb0, 0x59, 0x58, 0x31, 0xb1, 0xed, 0x64, 0x3b, 0xb1, 0x27, 0xc2, 0x66, 0xe3, 0x76, 0x47, + 0xb2, 0x8d, 0x47, 0x58, 0xa7, 0xbd, 0x9a, 0x26, 0xee, 0xa1, 0xdd, 0x69, 0x10, 0x02, 0xf1, 0x6c, + 0x58, 0x57, 0x3d, 0x37, 0x94, 0x16, 0xdd, 0x67, 0x22, 0xd2, 0xc4, 0x4d, 0x8d, 0x1f, 0x74, 0x4a, + 0x8b, 0xfc, 0x89, 0x68, 0x4c, 0xee, 0xda, 0xbb, 0x34, 0xa9, 0x23, 0x2d, 0xd2, 0xff, 0x79, 0xd3, + 0xbe, 0x21, 0x40, 0xfc, 0x6e, 0xb9, 0x7e, 0xe2, 0xb6, 0x39, 0x12, 0xe3, 0x9e, 0x44, 0x9a, 0x64, + 0xa8, 0xb5, 0x5a, 0x9a, 0xde, 0x24, 0x41, 0xc7, 0x57, 0xb1, 0xe2, 0xb4, 0x2c, 0xcf, 0xc9, 0x9b, + 0x8c, 0x8a, 0x16, 0x21, 0xa3, 0x98, 0x58, 0xc5, 0xba, 0xad, 0xc9, 0x2d, 0x8b, 0x37, 0xd1, 0x4f, + 0xe2, 0x95, 0xfb, 0x9a, 0x00, 0x93, 0xd4, 0xbc, 0xd0, 0x33, 0x90, 0x56, 0x0c, 0xdd, 0x96, 0x35, + 0x9d, 0xfb, 0x89, 0xb4, 0xe8, 0x11, 0x06, 0x56, 0xf2, 0x02, 0x64, 0x65, 0x45, 0x31, 0xba, 0xba, + 0x2d, 0xe9, 0x72, 0x1b, 0xf3, 0xca, 0x66, 0x38, 0x6d, 0x43, 0x6e, 0x63, 0xb4, 0x00, 0xce, 0xa3, + 0x7b, 0x3a, 0x2f, 0x2d, 0x02, 0x27, 0xad, 0xe1, 0x43, 0x5e, 0x93, 0xef, 0x0b, 0x90, 0x72, 0xcc, + 0x92, 0x54, 0xa6, 0x89, 0x75, 0x6c, 0xca, 0xb6, 0xe1, 0x56, 0xc6, 0x25, 0xf4, 0xce, 0x49, 0x69, + 0x6f, 0x4e, 0x3a, 0x03, 0x93, 0xb6, 0xbc, 0xdd, 0x72, 0xea, 0xc1, 0x1e, 0xe8, 0xf2, 0x71, 0x4b, + 0x6e, 0xb2, 0x35, 0xb2, 0xb4, 0xc8, 0x1e, 0x48, 0x93, 0x78, 0x56, 0x2b, 0xd3, 0x0e, 0x7f, 0x22, + 0xf5, 0x65, 0x89, 0x9c, 0xdb, 0xb8, 0xa9, 0xe9, 0xd4, 0x00, 0xe2, 0x22, 0x50, 0xd2, 0x0a, 0xa1, + 0xa0, 0xa7, 0x21, 0xcd, 0x0a, 0x60, 0x5d, 0xa5, 0x56, 0x10, 0x17, 0x53, 0x94, 0x50, 0x71, 0x4e, + 0x2a, 0xf1, 0x40, 0xe4, 0x3b, 0x02, 0xcc, 0xb0, 0x9c, 0x1b, 0x96, 0x01, 0x1a, 0xdd, 0xac, 0xfc, + 0x2a, 0xa4, 0x55, 0xd9, 0x96, 0xd9, 0x69, 0xc0, 0xd8, 0xd0, 0xd3, 0x80, 0x8e, 0x9b, 0x24, 0xe5, + 0xe9, 0x89, 0x40, 0x04, 0x09, 0xf2, 0x3f, 0x3b, 0x45, 0x29, 0xd2, 0xff, 0xbd, 0x2c, 0x06, 0x7f, + 0x75, 0xa3, 0x8c, 0x52, 0x3e, 0x9c, 0x84, 0x5c, 0xe5, 0xa0, 0x63, 0x98, 0x11, 0x2f, 0xa2, 0x4d, + 0x71, 0xf0, 0x3b, 0x64, 0x9b, 0xb5, 0xc7, 0x0b, 0x3a, 0x3b, 0x98, 0x9c, 0x11, 0xad, 0x00, 0xb0, + 0xd4, 0x4b, 0x9a, 0x87, 0x13, 0x3f, 0xc1, 0x66, 0x13, 0x65, 0x23, 0x54, 0xb4, 0x01, 0x99, 0xf6, + 0x9e, 0xa2, 0x48, 0x3b, 0x5a, 0xcb, 0xe6, 0x49, 0x6b, 0xe1, 0x59, 0xd2, 0xeb, 0x0f, 0xcb, 0xe5, + 0x3b, 0xb4, 0x10, 0xcb, 0x1d, 0xf3, 0x9e, 0x45, 0x20, 0x12, 0xd8, 0xff, 0xe8, 0x79, 0xe0, 0x07, + 0x3b, 0x24, 0xcb, 0x39, 0xa6, 0xb5, 0x92, 0x3b, 0x3e, 0x5a, 0x48, 0x8b, 0x94, 0x5a, 0xaf, 0x37, + 0xc4, 0x34, 0x2b, 0x50, 0xb7, 0x6c, 0x74, 0x11, 0x72, 0x46, 0x5b, 0xb3, 0x25, 0x27, 0x0e, 0xe0, + 0xa1, 0x53, 0x96, 0x10, 0x9d, 0x38, 0x01, 0x35, 0xe0, 0x2a, 0xd6, 0xc9, 0x68, 0xa0, 0xed, 0x94, + 0xb6, 0xd9, 0x5a, 0x9d, 0xcd, 0x46, 0x94, 0x64, 0x74, 0x6c, 0xad, 0xad, 0x3d, 0xa1, 0x1b, 0xbd, + 0x7c, 0x93, 0xe5, 0x22, 0x2b, 0x4e, 0xda, 0xb7, 0x42, 0x17, 0xf1, 0x78, 0xd9, 0x07, 0xbe, 0xa2, + 0xe8, 0x6b, 0x02, 0x9c, 0xe5, 0x8a, 0x94, 0xb6, 0x0f, 0xa5, 0x16, 0x99, 0x44, 0x34, 0xfb, 0x50, + 0x7a, 0xb4, 0x57, 0x4c, 0xd1, 0x00, 0xed, 0x73, 0xa1, 0x1d, 0xe2, 0xb3, 0x83, 0x25, 0xa7, 0x5b, + 0x0e, 0xef, 0x73, 0xe6, 0xb5, 0xbd, 0x8a, 0x6e, 0x9b, 0x87, 0x2b, 0xe7, 0x8e, 0x8f, 0x16, 0x66, + 0xfb, 0xdf, 0x3e, 0x14, 0x67, 0xad, 0x7e, 0x96, 0xb9, 0xaf, 0x42, 0x71, 0x90, 0x24, 0x54, 0xf0, + 0x76, 0x1a, 0xd3, 0x6c, 0x83, 0xf1, 0x95, 0xe0, 0x0a, 0xc1, 0x18, 0x66, 0xe3, 0xac, 0x12, 0xc4, + 0x5e, 0x71, 0x46, 0xf7, 0x2f, 0x0a, 0x90, 0x5b, 0xe9, 0xb6, 0x1e, 0x3d, 0xe8, 0xd4, 0xbb, 0xed, + 0xb6, 0x6c, 0x1e, 0x12, 0xc7, 0xc0, 0x46, 0xa5, 0xf6, 0x84, 0xa5, 0xa8, 0xc4, 0xf9, 0xb0, 0xd3, + 0x9e, 0x60, 0x32, 0xec, 0x78, 0x6a, 0x3a, 0xa1, 0xb3, 0xbc, 0xf3, 0x8b, 0x90, 0xa3, 0x70, 0x5d, + 0xc2, 0xba, 0x6d, 0x6a, 0x98, 0xad, 0x42, 0xc5, 0xc5, 0x2c, 0x25, 0x56, 0x18, 0x0d, 0x5d, 0x86, + 0xbc, 0x75, 0x68, 0xd9, 0xb8, 0x2d, 0xb1, 0xb3, 0xdb, 0x0c, 0x63, 0xc6, 0xc5, 0x1c, 0xa3, 0x8a, + 0x8c, 0x58, 0xfa, 0x71, 0x1c, 0xf2, 0x8e, 0x66, 0xa3, 0x0c, 0x5f, 0x57, 0x60, 0x72, 0x47, 0x6b, + 0x61, 0x27, 0x4b, 0xe0, 0xca, 0x90, 0x0e, 0xe5, 0x29, 0xc7, 0x24, 0x8a, 0x70, 0x00, 0x10, 0x65, + 0x8d, 0x62, 0x8c, 0xcd, 0xfd, 0xdf, 0x18, 0x24, 0x68, 0xc4, 0x78, 0x13, 0x12, 0xd4, 0xed, 0x09, + 0xe3, 0xb8, 0x3d, 0x5a, 0xd4, 0x0d, 0x67, 0x62, 0xbe, 0x70, 0x86, 0xc4, 0x06, 0xbb, 0xf2, 0x4b, + 0x37, 0x6f, 0xd1, 0xf1, 0x95, 0x15, 0xf9, 0x13, 0x5a, 0xa1, 0xe9, 0x2b, 0x86, 0x69, 0x63, 0x95, + 0x47, 0x6a, 0x61, 0x9e, 0x29, 0xd0, 0xf1, 0x8e, 0x8b, 0x75, 0xf8, 0xd0, 0x79, 0x88, 0x93, 0x81, + 0x3b, 0xc5, 0xb6, 0xb6, 0x8f, 0x8f, 0x16, 0xe2, 0x64, 0xc8, 0x12, 0x1a, 0xba, 0x01, 0x99, 0xe0, + 0x28, 0x11, 0xae, 0xa5, 0x99, 0x2f, 0xf0, 0x59, 0x38, 0xb4, 0x5c, 0x0b, 0x66, 0x28, 0xe5, 0x5e, + 0x22, 0x95, 0x28, 0x4c, 0xd2, 0x8c, 0xde, 0x5a, 0x3b, 0x6a, 0x2f, 0xba, 0x1c, 0xec, 0xe1, 0xb0, + 0xf0, 0x36, 0xf0, 0xd1, 0x90, 0x0e, 0x0e, 0xcc, 0x47, 0xf1, 0x93, 0xcd, 0x47, 0x35, 0x12, 0x51, + 0xf1, 0xf3, 0xe9, 0xf1, 0x01, 0x91, 0x6c, 0xf0, 0xfb, 0x0d, 0xe2, 0x98, 0x44, 0xc2, 0xe3, 0x25, + 0xe1, 0xd3, 0xf4, 0x84, 0x37, 0x68, 0xe0, 0xc6, 0xac, 0x2c, 0x39, 0xbe, 0x95, 0x4d, 0x61, 0x5d, + 0xa5, 0x36, 0xf6, 0x84, 0x9b, 0xd8, 0xab, 0x10, 0x57, 0x35, 0x73, 0xc8, 0x65, 0x07, 0xe1, 0x73, + 0x0a, 0x61, 0x1a, 0x61, 0x6b, 0x09, 0xbf, 0xad, 0xf9, 0x71, 0xe8, 0xdc, 0x03, 0x00, 0xaf, 0x5d, + 0x68, 0x11, 0x92, 0x46, 0x4b, 0x75, 0x4e, 0x10, 0xe4, 0x56, 0xd2, 0xc7, 0x47, 0x0b, 0x93, 0x0f, + 0x5a, 0x6a, 0x6d, 0x55, 0x9c, 0x34, 0x5a, 0x6a, 0x4d, 0xa5, 0x07, 0xfb, 0xf1, 0xbe, 0xe4, 0xe6, + 0x18, 0x65, 0xc5, 0x29, 0x1d, 0xef, 0x13, 0xd4, 0xdb, 0x93, 0xfb, 0x40, 0x0c, 0xe7, 0x5b, 0x02, + 0xe4, 0x1d, 0x1d, 0x46, 0xeb, 0x1c, 0x52, 0x5a, 0x9b, 0x0f, 0x96, 0xf8, 0xc9, 0x06, 0x8b, 0xc3, + 0xc7, 0x4f, 0x26, 0x7e, 0x5d, 0xe0, 0xf9, 0xa5, 0x75, 0x45, 0xb6, 0xc9, 0x7c, 0x18, 0xa1, 0x81, + 0x3f, 0x07, 0x05, 0x53, 0xd6, 0x55, 0xa3, 0xad, 0x3d, 0xc1, 0x6c, 0xe1, 0xca, 0xe2, 0xfb, 0x53, + 0xd3, 0x2e, 0x9d, 0xae, 0xcc, 0x38, 0xeb, 0x6e, 0x3f, 0x13, 0x78, 0x2e, 0xaa, 0x5b, 0x99, 0x28, + 0x95, 0xb6, 0x06, 0x49, 0x93, 0x65, 0xb4, 0xb1, 0x01, 0xf7, 0x42, 0x88, 0x90, 0xb0, 0xaf, 0xb3, + 0x84, 0x31, 0xd7, 0xe4, 0xa9, 0x88, 0xb9, 0x2f, 0xc2, 0x24, 0x25, 0x9f, 0xc2, 0x2d, 0x72, 0xcd, + 0xff, 0x43, 0x0c, 0x2e, 0xd1, 0xcf, 0x3d, 0xc4, 0xa6, 0xb6, 0x73, 0xb8, 0x69, 0x1a, 0x36, 0x56, + 0x6c, 0xac, 0x7a, 0x99, 0xfa, 0x91, 0xfa, 0x9a, 0x74, 0xc7, 0xf9, 0xc0, 0x89, 0x32, 0x7b, 0x5c, + 0x2e, 0xb4, 0x06, 0xd3, 0x6c, 0x22, 0x94, 0xe4, 0x96, 0xb6, 0x87, 0x25, 0xd9, 0x3e, 0xc9, 0x8c, + 0x92, 0x63, 0xbc, 0xcb, 0x84, 0x75, 0xd9, 0x46, 0x2a, 0xa4, 0xb9, 0x30, 0x4d, 0xe5, 0x17, 0x87, + 0xdc, 0xfd, 0x78, 0x4b, 0x33, 0x29, 0x36, 0x21, 0xd7, 0x56, 0xc5, 0x14, 0x93, 0xec, 0x2e, 0xad, + 0xff, 0x50, 0x80, 0xcb, 0x23, 0x14, 0x1d, 0xa5, 0x99, 0xcd, 0x41, 0x6a, 0x8f, 0x7c, 0x48, 0xe3, + 0x9a, 0x4e, 0x89, 0xee, 0x33, 0x5a, 0x87, 0xdc, 0x8e, 0xac, 0xb5, 0xb0, 0x2a, 0x71, 0x4b, 0x1c, + 0x9c, 0x0e, 0x16, 0x9e, 0xa5, 0x98, 0x65, 0xec, 0xf4, 0xa5, 0x55, 0xfa, 0xad, 0x18, 0xcc, 0x2c, + 0xab, 0x6a, 0xbd, 0xce, 0x3d, 0x58, 0x74, 0xf6, 0xe2, 0x80, 0x95, 0x98, 0x07, 0x56, 0xd0, 0x0b, + 0x80, 0x54, 0xcd, 0x62, 0x97, 0x1a, 0x58, 0xbb, 0xb2, 0x6a, 0xec, 0x7b, 0x1b, 0xdf, 0x33, 0xce, + 0x9b, 0xba, 0xf3, 0x02, 0xbd, 0x03, 0x34, 0xb4, 0x96, 0x2c, 0x5b, 0x76, 0xd7, 0xe7, 0x9f, 0x3f, + 0xc9, 0xf9, 0x1a, 0x16, 0x7a, 0xbb, 0x8f, 0x62, 0x9a, 0x88, 0xa3, 0xff, 0xa2, 0x6b, 0x50, 0xd0, + 0x88, 0x06, 0x6c, 0x49, 0xb6, 0x9c, 0x83, 0x18, 0xec, 0x56, 0x85, 0x3c, 0xa3, 0x2f, 0x5b, 0xfe, + 0xf3, 0x15, 0x2c, 0x4f, 0xdc, 0xd3, 0x53, 0x94, 0x08, 0xeb, 0x77, 0x04, 0xc8, 0x8b, 0x78, 0xc7, + 0xc4, 0x56, 0xa4, 0x48, 0xf3, 0x0e, 0x64, 0x4d, 0x26, 0x55, 0xda, 0x31, 0x8d, 0xf6, 0x49, 0x86, + 0x5a, 0x86, 0x33, 0xde, 0x31, 0x8d, 0x76, 0xe0, 0x14, 0xfa, 0x43, 0x98, 0x76, 0x6b, 0x1a, 0xa5, + 0x0a, 0xbe, 0x43, 0x8f, 0x96, 0x32, 0xc1, 0x51, 0x6f, 0x3d, 0x7f, 0x12, 0x7a, 0xa0, 0xfb, 0x02, + 0xfe, 0xea, 0x46, 0xa9, 0x8c, 0x9f, 0x09, 0x90, 0xaf, 0x77, 0xb7, 0xd9, 0x2d, 0x3a, 0xd1, 0xe9, + 0xa1, 0x02, 0xe9, 0x16, 0xde, 0xb1, 0xa5, 0x53, 0xe5, 0x36, 0xa7, 0x08, 0x2b, 0xcd, 0xef, 0xbe, + 0x0b, 0x60, 0xd2, 0xd3, 0x4b, 0x54, 0x4e, 0xfc, 0x84, 0x72, 0xd2, 0x94, 0xd7, 0x8b, 0x75, 0x4a, + 0x7f, 0x10, 0x83, 0x69, 0xb7, 0xb1, 0x51, 0x3a, 0xd1, 0xff, 0x15, 0x70, 0x1e, 0xf1, 0x53, 0x38, + 0x8f, 0x19, 0xbe, 0xe5, 0x1e, 0xee, 0x40, 0x96, 0x60, 0x96, 0x06, 0x24, 0x92, 0xdc, 0xe9, 0xb4, + 0x34, 0xac, 0x4a, 0x6c, 0x2f, 0x37, 0x41, 0xf7, 0x72, 0x67, 0xe8, 0xab, 0x65, 0xf6, 0xa6, 0x46, + 0xf7, 0x75, 0xef, 0x40, 0x76, 0xc7, 0xc4, 0xf8, 0x09, 0x96, 0x28, 0x32, 0x3a, 0x49, 0x66, 0x41, + 0x86, 0x31, 0xd6, 0x09, 0x1f, 0x37, 0xc0, 0x77, 0x61, 0x86, 0x2a, 0x38, 0xea, 0x33, 0x95, 0xbc, + 0x57, 0xfe, 0x51, 0x00, 0xe4, 0x97, 0xff, 0xc9, 0x75, 0x4c, 0x2c, 0xf2, 0x8e, 0x79, 0x1e, 0x10, + 0x4b, 0x32, 0xb3, 0xa4, 0x0e, 0x36, 0x25, 0x0b, 0x2b, 0x06, 0xbf, 0x1b, 0x46, 0x10, 0x0b, 0xfc, + 0xcd, 0x26, 0x36, 0xeb, 0x94, 0x5e, 0x7a, 0x6f, 0x0e, 0xb2, 0x5c, 0x27, 0x5b, 0xba, 0x66, 0xe8, + 0xe8, 0x26, 0xc4, 0x9b, 0x7c, 0xa5, 0x37, 0x13, 0xba, 0x12, 0xe4, 0xdd, 0x4f, 0x55, 0x9d, 0x10, + 0x49, 0x59, 0xc2, 0xd2, 0xe9, 0xda, 0x21, 0x41, 0x91, 0x97, 0x1b, 0xeb, 0x67, 0xe9, 0x74, 0x6d, + 0x54, 0x87, 0x69, 0xc5, 0xbb, 0x9f, 0x47, 0x22, 0xec, 0xf1, 0x81, 0x9b, 0x0b, 0xa1, 0xb7, 0x1c, + 0x55, 0x27, 0xc4, 0xbc, 0x12, 0x78, 0x81, 0xca, 0xfe, 0x0b, 0x61, 0xd8, 0x74, 0x79, 0x31, 0xf4, + 0xfc, 0x65, 0xf0, 0x32, 0x9a, 0xea, 0x84, 0xef, 0xde, 0x18, 0xf4, 0x2a, 0x24, 0x55, 0x7a, 0xd1, + 0x08, 0xb7, 0xd0, 0x30, 0x23, 0x0a, 0xdc, 0xed, 0x52, 0x9d, 0x10, 0x39, 0x07, 0xba, 0x07, 0x59, + 0xf6, 0x1f, 0x0b, 0x4e, 0x38, 0x92, 0xbb, 0x3c, 0x58, 0x82, 0xcf, 0xd7, 0x57, 0x27, 0xc4, 0x8c, + 0xea, 0x51, 0xd1, 0x67, 0x21, 0x61, 0x29, 0xb2, 0xce, 0x37, 0x49, 0xe6, 0x07, 0x5c, 0x5c, 0xe0, + 0x31, 0xd3, 0xd2, 0xe8, 0x35, 0x76, 0x99, 0x9c, 0x7d, 0xc0, 0x0e, 0x5e, 0x86, 0x57, 0x3f, 0x70, + 0xb4, 0x96, 0x54, 0x1f, 0x53, 0x02, 0xba, 0x0b, 0x19, 0x99, 0x44, 0x79, 0x12, 0x3d, 0xc6, 0x56, + 0x84, 0x81, 0x07, 0x8d, 0xfa, 0x8e, 0x20, 0x56, 0xe9, 0xf9, 0x5d, 0x87, 0xe8, 0x09, 0x6a, 0x63, + 0xb3, 0x89, 0x8b, 0x99, 0xe1, 0x82, 0xfc, 0x59, 0x34, 0xae, 0x20, 0x4a, 0x24, 0xd1, 0xde, 0xae, + 0x73, 0x44, 0x81, 0x36, 0x2a, 0x3b, 0x70, 0x53, 0x2c, 0xe4, 0x88, 0x45, 0x75, 0x42, 0xcc, 0xee, + 0xfa, 0xc8, 0x68, 0x09, 0x62, 0x4d, 0xa5, 0x98, 0xa3, 0x32, 0x9e, 0x19, 0x76, 0x80, 0xa0, 0x3a, + 0x21, 0xc6, 0x9a, 0x0a, 0x41, 0xe5, 0x2c, 0xf5, 0xfb, 0x40, 0x2f, 0xe6, 0x07, 0x8e, 0xf8, 0x60, + 0x1e, 0x7d, 0x75, 0x42, 0xa4, 0xd9, 0xe6, 0xe4, 0x7b, 0x9b, 0x90, 0x37, 0x59, 0x1a, 0x92, 0x93, + 0x5c, 0x58, 0x18, 0xb8, 0x51, 0x18, 0x96, 0x5f, 0x58, 0xa5, 0x51, 0xbf, 0x8f, 0x8e, 0xbe, 0x02, + 0x67, 0x82, 0x12, 0xb9, 0xa5, 0xcd, 0xf4, 0xb9, 0x91, 0x70, 0xb9, 0x41, 0x83, 0x43, 0x66, 0xdf, + 0x4b, 0xf4, 0x32, 0x4c, 0xb2, 0x5e, 0x43, 0x54, 0x64, 0xd8, 0x0e, 0x78, 0x4f, 0x87, 0xb1, 0xf2, + 0xc4, 0xf8, 0x6d, 0x9e, 0x7f, 0x23, 0xb5, 0x8c, 0x66, 0x71, 0x76, 0xa0, 0xf1, 0xf7, 0xe7, 0x13, + 0x11, 0xe3, 0xb7, 0x3d, 0x2a, 0xe9, 0x77, 0x93, 0xbd, 0xe1, 0xe9, 0x1a, 0x67, 0x06, 0xf6, 0x7b, + 0x48, 0x5a, 0x4e, 0x95, 0xe6, 0x4e, 0x7b, 0x64, 0x52, 0x35, 0x93, 0x5d, 0xdc, 0x21, 0xd1, 0x31, + 0xf5, 0xd4, 0xc0, 0xaa, 0xf5, 0x5f, 0x46, 0x52, 0xa5, 0x61, 0x90, 0x4b, 0x45, 0x0f, 0xa1, 0xc0, + 0x8f, 0xd4, 0x7b, 0xcb, 0xd6, 0x67, 0xa9, 0xbc, 0xe7, 0x42, 0x5d, 0x57, 0x58, 0x7e, 0x43, 0x75, + 0x42, 0x9c, 0x56, 0x82, 0x6f, 0xd0, 0xdb, 0x30, 0x43, 0xe5, 0x49, 0x8a, 0x77, 0x17, 0x42, 0xb1, + 0xd8, 0x97, 0x28, 0x38, 0xf8, 0xda, 0x04, 0x47, 0x72, 0x41, 0xe9, 0x79, 0x45, 0xcc, 0x58, 0xd3, + 0x35, 0x9b, 0x7a, 0xd9, 0xb9, 0x81, 0x66, 0x1c, 0xbc, 0xcf, 0x8c, 0x98, 0xb1, 0xc6, 0x28, 0xc4, + 0x8c, 0x6d, 0x9e, 0xcb, 0xc3, 0xbb, 0xe3, 0x99, 0x81, 0x66, 0x1c, 0x96, 0xf4, 0x43, 0xcc, 0xd8, + 0xf6, 0xd3, 0x89, 0x19, 0x33, 0x07, 0xd1, 0x23, 0xf7, 0xd9, 0x81, 0x66, 0x3c, 0xf0, 0x20, 0x29, + 0x31, 0x63, 0xb9, 0xef, 0x25, 0x5a, 0x05, 0x60, 0xe1, 0x89, 0xa6, 0xef, 0x18, 0xc5, 0xf9, 0x81, + 0x93, 0x41, 0x6f, 0x36, 0x0f, 0x99, 0x0c, 0x5a, 0x0e, 0x8d, 0x38, 0x32, 0x8a, 0x8d, 0x24, 0xba, + 0x4b, 0x56, 0x5c, 0x18, 0xe8, 0xc8, 0xfa, 0x36, 0xcb, 0x88, 0x23, 0xdb, 0x77, 0x89, 0x64, 0x56, + 0x61, 0x6b, 0xac, 0xc5, 0xc5, 0xc1, 0x6e, 0xd9, 0xbf, 0xbb, 0x40, 0xdd, 0x32, 0x25, 0xa0, 0x65, + 0x48, 0x93, 0x69, 0xfb, 0x90, 0xba, 0xa1, 0x0b, 0x03, 0x03, 0xce, 0x9e, 0xc3, 0x01, 0xd5, 0x09, + 0x31, 0xf5, 0x98, 0x93, 0xc8, 0xe7, 0xd9, 0xaa, 0x55, 0xb1, 0x34, 0xf0, 0xf3, 0x81, 0x95, 0x4a, + 0xf2, 0x79, 0xc6, 0x81, 0x14, 0x78, 0x8a, 0xf5, 0x15, 0x3f, 0xc7, 0x69, 0xf2, 0x43, 0x87, 0xc5, + 0x8b, 0x54, 0xd4, 0xc0, 0x35, 0xa0, 0xd0, 0xe3, 0xa5, 0xd5, 0x09, 0x71, 0x56, 0xee, 0x7f, 0x4b, + 0x06, 0x3c, 0x9f, 0x7a, 0xd8, 0xca, 0x51, 0xf1, 0xd2, 0xc0, 0x01, 0x1f, 0xb2, 0xd6, 0x46, 0x06, + 0xbc, 0xec, 0x23, 0xb3, 0x09, 0x48, 0x95, 0x2c, 0x8b, 0xed, 0xa9, 0x5e, 0x1e, 0x32, 0x01, 0xf5, + 0x60, 0x7f, 0x36, 0x01, 0xa9, 0x75, 0xc6, 0x49, 0x04, 0x29, 0x2d, 0x2c, 0x9b, 0xdc, 0xcd, 0x5e, + 0x19, 0x28, 0xa8, 0xef, 0x8e, 0x30, 0x22, 0x48, 0x71, 0x89, 0x24, 0xe0, 0x31, 0x9d, 0xbb, 0x38, + 0x78, 0xe8, 0x77, 0x75, 0x60, 0xc0, 0x13, 0x7a, 0x65, 0x08, 0x09, 0x78, 0xcc, 0xc0, 0x0b, 0xf4, + 0x79, 0x98, 0xe2, 0x08, 0xad, 0x78, 0x6d, 0x48, 0x40, 0xea, 0x87, 0xd6, 0x64, 0x5c, 0x73, 0x1e, + 0xe6, 0x65, 0x19, 0x32, 0x64, 0xcd, 0x7b, 0x6e, 0x88, 0x97, 0xed, 0x03, 0xa7, 0xcc, 0xcb, 0x7a, + 0x64, 0xe2, 0x65, 0x99, 0x9d, 0xf2, 0xb9, 0xee, 0xfa, 0x40, 0x2f, 0xdb, 0x7f, 0x32, 0x81, 0x78, + 0xd9, 0xc7, 0x1e, 0x95, 0xb4, 0xcc, 0x62, 0xa8, 0xa8, 0xf8, 0x99, 0x81, 0x2d, 0x0b, 0x82, 0x44, + 0xd2, 0x32, 0xce, 0x43, 0xba, 0x8d, 0x25, 0x91, 0x32, 0x4d, 0x3f, 0x3f, 0xf8, 0xc8, 0x74, 0x2f, + 0x88, 0xa8, 0x3a, 0x97, 0xc4, 0x32, 0x0d, 0xbb, 0x8e, 0xca, 0xe4, 0x07, 0x44, 0xb9, 0xa6, 0x5e, + 0x18, 0xee, 0xa8, 0xc2, 0xce, 0xbe, 0xba, 0x8e, 0x2a, 0xf0, 0x92, 0x56, 0x95, 0x9d, 0xf3, 0xa1, + 0xe3, 0x7b, 0x69, 0xc8, 0xe9, 0xee, 0x9e, 0xa3, 0x57, 0xb4, 0xaa, 0x2e, 0xd1, 0x1b, 0x42, 0x5d, + 0x76, 0x0d, 0x41, 0xf1, 0xc6, 0xf0, 0x21, 0x14, 0xbc, 0x0e, 0xc1, 0x1d, 0x42, 0x9c, 0xec, 0xce, + 0x99, 0x4e, 0x84, 0xf1, 0xe2, 0xf0, 0x39, 0xb3, 0x37, 0xb4, 0x60, 0x73, 0x26, 0x8f, 0x29, 0xfe, + 0x9f, 0x00, 0x8b, 0xac, 0x6e, 0x74, 0x1d, 0xef, 0x50, 0x72, 0xd7, 0x44, 0x7d, 0xa9, 0xe6, 0x37, + 0xe9, 0x07, 0x5e, 0x1e, 0x54, 0xdd, 0x11, 0x6b, 0xbc, 0xd5, 0x09, 0xf1, 0x59, 0x79, 0x58, 0xb9, + 0x95, 0x29, 0xbe, 0x73, 0xea, 0x9e, 0xb8, 0x9b, 0x2e, 0x14, 0xee, 0x25, 0x52, 0xe7, 0x0a, 0xc5, + 0x7b, 0x89, 0xd4, 0xf9, 0xc2, 0xdc, 0xbd, 0x44, 0xea, 0xe9, 0xc2, 0x33, 0xa5, 0x7f, 0x3a, 0x0f, + 0x39, 0x07, 0xc3, 0x31, 0x44, 0x74, 0xcb, 0x8f, 0x88, 0xe6, 0x07, 0x21, 0x22, 0x8e, 0xfa, 0x38, + 0x24, 0xba, 0xe5, 0x87, 0x44, 0xf3, 0x83, 0x20, 0x91, 0xc7, 0x43, 0x30, 0x51, 0x63, 0x10, 0x26, + 0x7a, 0x6e, 0x0c, 0x4c, 0xe4, 0x8a, 0xea, 0x05, 0x45, 0xab, 0xfd, 0xa0, 0xe8, 0xd2, 0x70, 0x50, + 0xe4, 0x8a, 0xf2, 0xa1, 0xa2, 0xd7, 0x7a, 0x50, 0xd1, 0x85, 0x21, 0xa8, 0xc8, 0xe5, 0x77, 0x60, + 0xd1, 0x5a, 0x28, 0x2c, 0xba, 0x32, 0x0a, 0x16, 0xb9, 0x72, 0x02, 0xb8, 0xe8, 0xa5, 0x00, 0x2e, + 0x5a, 0x18, 0x88, 0x8b, 0x5c, 0x6e, 0x06, 0x8c, 0x5e, 0xef, 0x05, 0x46, 0x17, 0x86, 0x00, 0x23, + 0xaf, 0x05, 0x1c, 0x19, 0x55, 0xc3, 0x90, 0xd1, 0xe5, 0x11, 0xc8, 0xc8, 0x95, 0xe2, 0x87, 0x46, + 0xd5, 0x30, 0x68, 0x74, 0x79, 0x04, 0x34, 0xea, 0x91, 0xc4, 0xb0, 0xd1, 0x46, 0x38, 0x36, 0xba, + 0x3a, 0x12, 0x1b, 0xb9, 0xd2, 0x82, 0xe0, 0xe8, 0x86, 0x0f, 0x1c, 0x3d, 0x3b, 0x00, 0x1c, 0xb9, + 0xac, 0x04, 0x1d, 0x7d, 0xa1, 0x0f, 0x1d, 0x95, 0x86, 0xa1, 0x23, 0x97, 0xd7, 0x85, 0x47, 0x6f, + 0x0e, 0x80, 0x47, 0xd7, 0x46, 0xc3, 0x23, 0x57, 0x58, 0x0f, 0x3e, 0x92, 0x87, 0xe2, 0xa3, 0x17, + 0xc6, 0xc4, 0x47, 0xae, 0xf4, 0x30, 0x80, 0xf4, 0x4a, 0x10, 0x20, 0x2d, 0x0e, 0x06, 0x48, 0xae, + 0x18, 0x8e, 0x90, 0xd6, 0x42, 0x11, 0xd2, 0x95, 0x51, 0x08, 0xc9, 0x1b, 0x07, 0x7e, 0x88, 0xb4, + 0x11, 0x0e, 0x91, 0xae, 0x8e, 0x84, 0x48, 0x5e, 0xf7, 0x07, 0x30, 0xd2, 0x5a, 0x28, 0x46, 0xba, + 0x32, 0x0a, 0x23, 0x79, 0x95, 0xf3, 0x83, 0xa4, 0xb7, 0x06, 0x82, 0xa4, 0xeb, 0xe3, 0x80, 0x24, + 0x57, 0x68, 0x1f, 0x4a, 0x7a, 0x67, 0x30, 0x4a, 0xfa, 0xcc, 0x09, 0x2e, 0x97, 0x0b, 0x85, 0x49, + 0x5f, 0xe8, 0x83, 0x49, 0xa5, 0x61, 0x30, 0xc9, 0xb3, 0x67, 0x07, 0x27, 0xc9, 0x43, 0x51, 0xcd, + 0x0b, 0x63, 0xa2, 0x1a, 0xcf, 0xf8, 0x42, 0x60, 0x4d, 0x25, 0x04, 0xd6, 0x5c, 0x1a, 0x0e, 0x6b, + 0x3c, 0x77, 0xee, 0xe1, 0x9a, 0x6a, 0x18, 0xae, 0xb9, 0x3c, 0x02, 0xd7, 0x78, 0x5e, 0xc8, 0x07, + 0x6c, 0x5e, 0xeb, 0x01, 0x36, 0x17, 0x46, 0x66, 0xd9, 0xf8, 0x90, 0xcd, 0x4a, 0x3f, 0xb2, 0xb9, + 0x38, 0x14, 0xd9, 0xb8, 0x12, 0x3c, 0x68, 0xf3, 0x5a, 0x0f, 0xb4, 0xb9, 0x30, 0x04, 0xda, 0x78, + 0x15, 0xe0, 0xd8, 0x46, 0x1d, 0x8e, 0x6d, 0x96, 0xc6, 0xc5, 0x36, 0xae, 0xe0, 0x50, 0x70, 0xb3, + 0x11, 0x0e, 0x6e, 0xae, 0x8e, 0xb9, 0x7b, 0xde, 0x87, 0x6e, 0xaa, 0x61, 0xe8, 0xe6, 0xf2, 0x08, + 0x74, 0xe3, 0x9f, 0x43, 0x5c, 0x78, 0x53, 0x0d, 0x83, 0x37, 0x97, 0x47, 0xc0, 0x1b, 0x4f, 0x92, + 0x0f, 0xdf, 0x34, 0x06, 0xe1, 0x9b, 0xe7, 0xc6, 0xc0, 0x37, 0x5e, 0xf0, 0xd2, 0x03, 0x70, 0xde, + 0xe8, 0x05, 0x38, 0xa5, 0x61, 0x00, 0xc7, 0x1b, 0x91, 0x0e, 0xc2, 0xd9, 0x08, 0x47, 0x38, 0x57, + 0x47, 0x22, 0x1c, 0xbf, 0x93, 0xf4, 0x41, 0x9c, 0xb5, 0x50, 0x88, 0x73, 0x65, 0x14, 0xc4, 0xf1, + 0x9c, 0xa4, 0x1f, 0xe3, 0xbc, 0xd1, 0x8b, 0x71, 0x4a, 0xc3, 0x30, 0x8e, 0xd7, 0x38, 0x07, 0xe4, + 0x54, 0xc3, 0x40, 0xce, 0xe5, 0x11, 0x20, 0xc7, 0xeb, 0x3c, 0x1f, 0xca, 0x91, 0x87, 0xa2, 0x9c, + 0x17, 0xc6, 0x44, 0x39, 0x3d, 0x8e, 0x2b, 0x08, 0x73, 0xaa, 0x61, 0x30, 0xe7, 0xf2, 0x08, 0x98, + 0xe3, 0xab, 0xac, 0x87, 0x73, 0x36, 0xc2, 0x71, 0xce, 0xd5, 0x91, 0x38, 0xa7, 0x67, 0x34, 0x39, + 0x40, 0x67, 0x2d, 0x14, 0xe8, 0x5c, 0x19, 0x05, 0x74, 0x7a, 0x26, 0x3e, 0x1e, 0x1c, 0xfc, 0xff, + 0xf1, 0x91, 0xce, 0x2b, 0x27, 0x47, 0x3a, 0xee, 0x37, 0x23, 0x81, 0x3a, 0xf7, 0x12, 0xa9, 0x67, + 0x0a, 0xcf, 0x96, 0x7e, 0x6e, 0x12, 0x92, 0x55, 0x37, 0xc7, 0xc5, 0xab, 0xa5, 0x70, 0x9a, 0xdb, + 0x6b, 0xd0, 0x2a, 0x19, 0xb1, 0xd4, 0xef, 0x8d, 0xbe, 0xa8, 0xac, 0xff, 0x12, 0x2d, 0xce, 0x7a, + 0x8a, 0x43, 0xa0, 0xe8, 0x25, 0xc8, 0x75, 0x2d, 0x6c, 0x4a, 0x1d, 0x53, 0x33, 0x4c, 0xcd, 0x66, + 0xe9, 0xfa, 0xc2, 0x4a, 0xe1, 0xa3, 0xa3, 0x85, 0xec, 0x96, 0x85, 0xcd, 0x4d, 0x4e, 0x17, 0xb3, + 0x5d, 0xdf, 0x93, 0xf3, 0x8b, 0x36, 0x93, 0xe3, 0xff, 0xa2, 0xcd, 0x9b, 0x50, 0x30, 0xb1, 0xac, + 0x06, 0x22, 0x10, 0x76, 0x2d, 0x4c, 0xb8, 0xcd, 0xd0, 0xb3, 0x2a, 0x4e, 0x49, 0x7a, 0x3d, 0xcc, + 0xb4, 0x19, 0x24, 0xa2, 0x9b, 0xf0, 0x54, 0x5b, 0x3e, 0xa0, 0x39, 0x88, 0x92, 0x13, 0xd4, 0xd1, + 0xbc, 0xc2, 0x14, 0xcd, 0x97, 0x45, 0x6d, 0xf9, 0x80, 0xfe, 0x3c, 0x0e, 0x7b, 0x45, 0xef, 0xcc, + 0xbf, 0x0c, 0x79, 0x55, 0xb3, 0x6c, 0x4d, 0x57, 0x9c, 0x9b, 0x41, 0xd9, 0x45, 0x9b, 0x39, 0x87, + 0xca, 0x2e, 0xff, 0xbc, 0x0e, 0x33, 0x3c, 0x1f, 0xdb, 0xfb, 0xc1, 0x1c, 0x0a, 0x5f, 0x52, 0xa4, + 0x16, 0xe4, 0x85, 0xf7, 0x4b, 0x49, 0x65, 0x98, 0x6e, 0xca, 0x36, 0xde, 0x97, 0x0f, 0x25, 0xe7, + 0xb8, 0x4c, 0x86, 0x5e, 0xab, 0xf7, 0xf4, 0xf1, 0xd1, 0x42, 0xee, 0x2e, 0x7b, 0xd5, 0x77, 0x6a, + 0x26, 0xd7, 0xf4, 0xbd, 0x50, 0xd1, 0x55, 0x98, 0x96, 0xad, 0x43, 0x5d, 0xa1, 0xea, 0xc1, 0xba, + 0xd5, 0xb5, 0x28, 0xa4, 0x48, 0x89, 0x79, 0x4a, 0x2e, 0x3b, 0x54, 0x76, 0x61, 0xe7, 0xbd, 0x44, + 0x2a, 0x5b, 0xc8, 0xdd, 0x4b, 0xa4, 0xf2, 0x85, 0xe9, 0xd2, 0xaf, 0x08, 0x90, 0x0d, 0x9c, 0x37, + 0x78, 0xad, 0x67, 0xbb, 0xf5, 0x7c, 0x38, 0xb4, 0x19, 0x94, 0xaf, 0x95, 0xe2, 0xaa, 0x74, 0xb2, + 0xd5, 0x16, 0x06, 0x87, 0xc6, 0x14, 0xe8, 0x3b, 0x5b, 0xfd, 0x0e, 0xdb, 0xab, 0x89, 0x5f, 0x7b, + 0x7f, 0x61, 0xa2, 0xf4, 0xd3, 0x38, 0xe4, 0x82, 0xe7, 0x0a, 0x6a, 0x3d, 0xf5, 0x0a, 0x73, 0x3d, + 0x01, 0x8e, 0xa5, 0x21, 0x77, 0x99, 0xa5, 0xbd, 0x5b, 0xb7, 0x59, 0x35, 0x17, 0x87, 0x6c, 0x2a, + 0xfb, 0xeb, 0xe9, 0x31, 0xce, 0x7d, 0x2f, 0xe6, 0x0e, 0xe1, 0x25, 0x98, 0xa4, 0x37, 0x68, 0xf0, + 0xaa, 0x85, 0x9d, 0xf3, 0xac, 0x90, 0xf7, 0x22, 0x2b, 0x46, 0x86, 0x7c, 0xe3, 0x54, 0x17, 0x56, + 0x79, 0x47, 0xfd, 0x4f, 0xfe, 0xa3, 0x50, 0xfc, 0xda, 0xb2, 0xc9, 0x93, 0x5d, 0x5b, 0xc6, 0x36, + 0x8d, 0x5b, 0x2d, 0xe6, 0x4e, 0x99, 0xd1, 0x27, 0xfb, 0x0e, 0x53, 0x52, 0x11, 0xfc, 0x37, 0xc3, + 0x96, 0x44, 0xfe, 0x9b, 0x61, 0xbe, 0x04, 0xc2, 0xbc, 0x2b, 0x82, 0x8e, 0x10, 0x96, 0x66, 0xca, + 0xbb, 0xfa, 0x5b, 0x02, 0x14, 0xe8, 0x78, 0xb8, 0x83, 0xb1, 0x1a, 0x89, 0x15, 0x3a, 0xb9, 0x8d, + 0xb1, 0xf1, 0x53, 0xbe, 0x03, 0xb7, 0xa0, 0xc7, 0x83, 0xb7, 0xa0, 0x97, 0xde, 0x17, 0x20, 0xef, + 0xd6, 0x90, 0xfd, 0x78, 0xce, 0x90, 0xdb, 0xc8, 0x4e, 0xf7, 0x7b, 0x32, 0xce, 0x41, 0xe9, 0xb1, + 0x7e, 0xc5, 0xc7, 0x7f, 0x50, 0x9a, 0xfd, 0xf6, 0xc9, 0xaf, 0x0b, 0x30, 0xeb, 0x56, 0xb1, 0xec, + 0x1d, 0x82, 0x3d, 0x45, 0xf6, 0xbb, 0x48, 0x7f, 0x3c, 0x8c, 0x00, 0x70, 0x7a, 0x42, 0x7d, 0x2c, + 0xf3, 0x44, 0x3c, 0x39, 0x02, 0x38, 0xb0, 0x57, 0x1b, 0x75, 0xfa, 0xb3, 0x62, 0xec, 0x7f, 0xab, + 0x74, 0xc7, 0xa7, 0x40, 0x3a, 0x12, 0x88, 0x96, 0xc6, 0x1a, 0x32, 0x8e, 0x96, 0x68, 0xe1, 0xd2, + 0x0f, 0xfc, 0x3d, 0x51, 0xd9, 0x23, 0x01, 0xdd, 0x6d, 0x88, 0xef, 0xc9, 0xad, 0x61, 0xb9, 0x21, + 0x81, 0x9e, 0x13, 0x49, 0x69, 0x74, 0x27, 0x70, 0x76, 0x38, 0x36, 0x38, 0xf8, 0xe8, 0x57, 0x69, + 0xe0, 0x8c, 0xf1, 0xcb, 0x4e, 0x2b, 0xe2, 0xa3, 0x3f, 0xef, 0xf7, 0x00, 0xaf, 0x26, 0x3e, 0x78, + 0x7f, 0x41, 0xb8, 0x5e, 0x87, 0xd9, 0x90, 0xa9, 0x0a, 0xe5, 0x01, 0x7c, 0x77, 0xa3, 0xf3, 0xdf, + 0x2e, 0x5b, 0x5e, 0x95, 0xb6, 0x36, 0xca, 0x0f, 0xd6, 0xd7, 0x6b, 0x8d, 0x46, 0x65, 0xb5, 0x20, + 0xa0, 0x02, 0x64, 0x03, 0x37, 0xab, 0xc7, 0xd8, 0xaf, 0x99, 0x5d, 0xff, 0x1f, 0x00, 0xde, 0xaf, + 0x2b, 0x10, 0x59, 0x6b, 0x95, 0xb7, 0xa5, 0x87, 0xcb, 0xf7, 0xb7, 0x2a, 0xf5, 0xc2, 0x04, 0x42, + 0x90, 0x5f, 0x59, 0x6e, 0x94, 0xab, 0x92, 0x58, 0xa9, 0x6f, 0x3e, 0xd8, 0xa8, 0x57, 0x9c, 0x5f, + 0x41, 0xbb, 0xbe, 0x0a, 0x59, 0xff, 0x29, 0x6b, 0x34, 0x0b, 0xd3, 0xe5, 0x6a, 0xa5, 0xbc, 0x26, + 0x3d, 0xac, 0x2d, 0x4b, 0x6f, 0x6e, 0x55, 0xb6, 0x2a, 0x85, 0x09, 0x5a, 0x35, 0x4a, 0xbc, 0xb3, + 0x75, 0xff, 0x7e, 0x41, 0x40, 0xd3, 0x90, 0x61, 0xcf, 0xf4, 0x16, 0xf6, 0x42, 0xec, 0xfa, 0x3a, + 0x64, 0x7c, 0x97, 0xb2, 0x91, 0xcf, 0x6d, 0x6e, 0xd5, 0xab, 0x52, 0xa3, 0xb6, 0x5e, 0xa9, 0x37, + 0x96, 0xd7, 0x37, 0x99, 0x0c, 0x4a, 0x5b, 0x5e, 0x79, 0x20, 0x36, 0x0a, 0x82, 0xfb, 0xdc, 0x78, + 0xb0, 0x55, 0xae, 0x3a, 0xcd, 0x28, 0x25, 0x52, 0xf1, 0x42, 0xfc, 0xfa, 0x63, 0x38, 0x37, 0xe0, + 0xa8, 0x31, 0xca, 0xc0, 0xd4, 0x96, 0x4e, 0xaf, 0xa1, 0x2a, 0x4c, 0xa0, 0x9c, 0xef, 0xb4, 0x71, + 0x41, 0x40, 0x29, 0x76, 0x8e, 0xb4, 0x10, 0x43, 0x49, 0x88, 0xd5, 0x6f, 0x17, 0xe2, 0xa4, 0xa2, + 0xbe, 0xc3, 0xba, 0x85, 0x04, 0x4a, 0xf3, 0x93, 0x8c, 0x85, 0x49, 0x94, 0xf5, 0x8e, 0x12, 0x16, + 0x92, 0xd7, 0x2f, 0x80, 0xef, 0x9c, 0x15, 0x02, 0x48, 0xde, 0x97, 0x6d, 0x6c, 0xd9, 0x85, 0x09, + 0x34, 0x05, 0xf1, 0xe5, 0x56, 0xab, 0x20, 0xdc, 0xfa, 0x7d, 0x01, 0x52, 0xce, 0x85, 0xe2, 0xe8, + 0x3e, 0x4c, 0x32, 0x54, 0xbe, 0x30, 0x78, 0x46, 0xa2, 0x4e, 0x6d, 0x6e, 0x71, 0xd4, 0x94, 0x55, + 0x9a, 0x40, 0x6f, 0xf1, 0x9f, 0x51, 0x24, 0x16, 0x83, 0x2e, 0x0e, 0xb3, 0x27, 0x47, 0xea, 0x70, + 0xa3, 0x23, 0x63, 0xa4, 0x34, 0xf1, 0xa2, 0xb0, 0xf2, 0xdc, 0x07, 0x3f, 0x9e, 0x9f, 0xf8, 0xe0, + 0x78, 0x5e, 0xf8, 0xf0, 0x78, 0x5e, 0xf8, 0xd1, 0xf1, 0xbc, 0xf0, 0xf7, 0xc7, 0xf3, 0xc2, 0x2f, + 0xfd, 0x64, 0x7e, 0xe2, 0xc3, 0x9f, 0xcc, 0x4f, 0xfc, 0xe8, 0x27, 0xf3, 0x13, 0xef, 0x4c, 0x71, + 0xee, 0xed, 0x24, 0xfd, 0x45, 0xc7, 0xdb, 0xff, 0x1e, 0x00, 0x00, 0xff, 0xff, 0x78, 0xad, 0x73, + 0x34, 0xd6, 0x72, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 8712f9475aba..e718e631649c 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1785,21 +1785,7 @@ message Header { // be much more straightforward if all transactional requests were // idempotent. We could just re-issue requests. See #26915. bool async_consensus = 13; - // WriteTooOldErrors can either be returned immediately or deferred - // to the end of the transaction. By default they are returned - // immediately to maximize the SQL-level first-statement retry - // ability. Set this flag to request deferred errors, which can - // improve performance under heavy contention when client-side - // retries are already inevitable. - // - // Non-transactional requests are not allowed to set this flag (since there's - // nowhere to defer the error to). At the replica level, this flag is ignored - // when the batch contains an EndTxn (similarly to before, there's nowhere to - // defer the error to). However, the client is allowed to set the flag in a - // batch that contains the EndTxn, and it will have an effect for any - // sub-batches that are split off by the DistSender. - bool defer_write_too_old_error = 14; - reserved 7,12; + reserved 7,12,14; } diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index f6d7dd3c07ee..cf543be43bb1 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -607,10 +607,6 @@ func (ba BatchRequest) ValidateForEvaluation() error { } else if ba.Replica.StoreID == 0 { return errors.AssertionFailedf("batch request missing store ID") } - if ba.Header.DeferWriteTooOldError && ba.Txn == nil { - return errors.AssertionFailedf( - "DeferWriteTooOldError can't be set on non-transactional requests") - } if _, ok := ba.GetArg(EndTxn); ok && ba.Txn == nil { return errors.AssertionFailedf("EndTxn request without transaction") } diff --git a/pkg/storage/intent_resolver_integration_test.go b/pkg/storage/intent_resolver_integration_test.go index 5aaad034efb9..b5069346fea4 100644 --- a/pkg/storage/intent_resolver_integration_test.go +++ b/pkg/storage/intent_resolver_integration_test.go @@ -384,7 +384,7 @@ func TestContendedIntentChangesOnRetry(t *testing.T) { putB := putArgs(keyB, []byte("value")) assignSeqNumsForReqs(txn1, &putB) repl, pErr := client.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ - Txn: txn1, DeferWriteTooOldError: true, + Txn: txn1, }, &putB) if pErr != nil { txnCh1 <- pErr.GoError() @@ -454,7 +454,7 @@ func TestContendedIntentPushedByHighPriorityScan(t *testing.T) { put := putArgs(keyA, []byte("value")) assignSeqNumsForReqs(txn2, &put) if _, pErr := client.SendWrappedWith(ctx, store.TestSender(), roachpb.Header{ - Txn: txn2, DeferWriteTooOldError: true, + Txn: txn2, }, &put); pErr != nil { txnCh2 <- pErr.GoError() return diff --git a/pkg/storage/replica_evaluate.go b/pkg/storage/replica_evaluate.go index b9ddd34d1024..11f2526d42e9 100644 --- a/pkg/storage/replica_evaluate.go +++ b/pkg/storage/replica_evaluate.go @@ -193,51 +193,23 @@ func evaluateBatch( var mergedResult result.Result - // WriteTooOldErrors are unique: When one is returned, we also lay - // down an intent at our new proposed timestamp. We have the option - // of continuing past a WriteTooOldError to the end of the - // transaction (at which point the txn.WriteTooOld flag will trigger - // a RefreshSpan and possibly a client-side retry). + // WriteTooOldErrors have particular handling. When a request encounters the + // error, we'd like to lay down an intent in order to avoid writers being + // starved. So, for blind writes, we swallow the error and instead we set the + // WriteTooOld flag on the response. For non-blind writes (e.g. CPut), we + // can't do that and so we just return the WriteTooOldError - see note on + // IsReadAndWrite() stanza below. Upon receiving either a WriteTooOldError or + // a response with the WriteTooOld flag set, the client will attempt to bump + // the txn's read timestamp through a refresh. If successful, the client will + // retry this batch (in both cases). // - // Within a batch, there's no downside to continuing past the - // WriteTooOldError, so we at least defer returning the error to the end of - // the batch so that we lay down more intents and we find out the final - // timestamp for the batch. - // - // Across batches, it's more complicated. We want to avoid - // client-side retries whenever possible. However, if a client-side - // retry is inevitable, it's probably best to continue and lay down - // as many intents as possible before that retry (this can avoid n^2 - // behavior in some scenarios with high contention on multiple keys, - // although we haven't verified this in practice). - // - // The SQL layer will transparently retry on the server side if - // we're in the first statement in a transaction. If we're in a - // first statement, we want to return WriteTooOldErrors immediately - // to take advantage of this. We don't have this information - // available at this level currently, so we err on the side of - // returning the WriteTooOldError immediately to get the server-side - // retry when it is available. - // - // TODO(bdarnell): Plumb the SQL CanAutoRetry field through to - // !baHeader.DeferWriteTooOldError. - // - // A more subtle heuristic is also possible: If we get a - // WriteTooOldError while writing to a key that we have already read - // (either earlier in the transaction, or as a part of the same - // operation for a ConditionalPut, Increment, or InitPut), a - // WriteTooOldError that is deferred to the end of the transaction - // is guarantee to result in a failed RefreshSpans and therefore a - // client-side retry. In some cases it may be possible to - // successfully retry at the TxnCoordSender, avoiding the - // client-side retry (this is likely for Increment, but unlikely for - // the others). In such cases, we may want to return the - // WriteTooOldError even if the SQL CanAutoRetry is false. As of - // this writing, nearly all writes issued by SQL are preceded by - // reads of the same key. + // In any case, evaluation of the current batch always continue after a + // WriteTooOldError in order to find out if there's more conflicts and chose a + // final write timestamp. var writeTooOldState struct { err *roachpb.WriteTooOldError - // cantDeferWTOE is set when a WriteTooOldError cannot be deferred. + // cantDeferWTOE is set when a WriteTooOldError cannot be deferred past the + // end of the current batch. cantDeferWTOE bool } @@ -268,8 +240,8 @@ func evaluateBatch( if ok && retErr.Reason == roachpb.RETRY_WRITE_TOO_OLD && args.Method() == roachpb.EndTxn && writeTooOldState.err != nil { pErr.SetDetail(writeTooOldState.err) - // Remember not to defer this error. Since it came from an EndTransaction, - // there's nowhere to defer it to. + // Don't defer this error. We could perhaps rely on the client observing + // the WriteTooOld flag and retry the batch, but we choose not too. writeTooOldState.cantDeferWTOE = true } @@ -303,11 +275,21 @@ func evaluateBatch( writeTooOldState.err = tErr } - // Requests which are both read and write are not currently - // accounted for in RefreshSpans, so they rely on eager - // returning of WriteTooOldErrors. - // TODO(bdarnell): add read+write requests to the read refresh spans - // in TxnCoordSender, and then I think this can go away. + // For requests that are both read and write, we don't have the option + // of leaving an intent behind when they encounter a WriteTooOldError, + // so we have to return an error instead of a response with the + // WriteTooOld flag set (which would also leave intents behind). These + // requests need to be re-evaluated at the bumped timestamp in order for + // their write to be valid. The current evaluation resulted in an intent + // that could well be different from what the request would write if it + // were evaluated at the bump timestamp, which would cause the request + // to be rejected if it were sent again with the same sequence number + // after a refresh. + // TODO(andrei): What we really want to do here is either speculatively + // evaluate the request at the bumped timestamp and return that + // speculative result, or leave behind a type of lock that wouldn't + // prevent the request for evaluating again at the same sequence number + // but at a bumped timestamp. if roachpb.IsReadAndWrite(args) { writeTooOldState.cantDeferWTOE = true } @@ -315,14 +297,15 @@ func evaluateBatch( if baHeader.Txn != nil { baHeader.Txn.WriteTimestamp.Forward(tErr.ActualTimestamp) baHeader.Txn.WriteTooOld = true + } else { + // For non-transactional requests, there's nowhere to defer the error + // to. And the request has to fail because non-transactional batches + // should read and write at the same timestamp. + writeTooOldState.cantDeferWTOE = true } - // Clear pErr; we're done processing it by having moved the - // batch or txn timestamps forward and set WriteTooOld if this - // is a transactional write. If we don't return the - // WriteTooOldError from this method, we will detect the - // pushed timestamp at commit time and refresh or retry the - // transaction. + // Clear pErr; we're done processing the WTOE for now and we'll return + // to considering it below after we've evaluated all requests. pErr = nil default: return nil, mergedResult, pErr @@ -357,9 +340,8 @@ func evaluateBatch( } } - // If there's a write too old error that we don't want to defer, return. - if writeTooOldState.err != nil && - (!baHeader.DeferWriteTooOldError || writeTooOldState.cantDeferWTOE) { + // If there's a write too old error that we can't defer, return it. + if writeTooOldState.cantDeferWTOE { return nil, mergedResult, roachpb.NewErrorWithTxn(writeTooOldState.err, baHeader.Txn) } diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index 86cbd4309a4b..5aa4741d1e8e 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -4471,30 +4471,17 @@ func TestBatchRetryCantCommitIntents(t *testing.T) { t.Errorf("expected transaction record to be cleared (%t): %+v", ok, err) } - // Now replay put for key A; by default this fails with a WriteTooOldError. - _, pErr = tc.Sender().Send(context.Background(), ba) - if _, ok := pErr.GetDetail().(*roachpb.WriteTooOldError); !ok { - t.Errorf("expected WriteTooOldError, got: %v", pErr) - } + // Now replay put for key A; this succeeds as there's nothing to detect + // the replay. The WriteTooOld flag will be set though. + br, pErr = tc.Sender().Send(context.Background(), ba) + require.NoError(t, pErr.GoError()) + require.True(t, br.Txn.WriteTooOld) - // Intent should not have been created. + // Intent should have been created. gArgs := getArgs(key) - if _, pErr = tc.SendWrapped(&gArgs); pErr != nil { - t.Errorf("unexpected error reading key: %s", pErr) - } - - // Send a put for keyB; by default this fails with a WriteTooOldError. - _, pErr = tc.SendWrappedWith(roachpb.Header{Txn: putTxn}, &putB) - if _, ok := pErr.GetDetail().(*roachpb.WriteTooOldError); !ok { - t.Errorf("expected WriteTooOldError, got: %v", pErr) - } - - // Try again with DeferWriteTooOldError. This currently succeeds - // (with a pushed timestamp) as there's nothing to detect the retry. - if _, pErr = tc.SendWrappedWith(roachpb.Header{ - Txn: putTxn, DeferWriteTooOldError: true, - }, &putB); pErr != nil { - t.Error(pErr) + _, pErr = tc.SendWrapped(&gArgs) + if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { + t.Errorf("expected WriteIntentError, got: %v", pErr) } // Heartbeat should fail with a TransactionAbortedError. @@ -4510,8 +4497,8 @@ func TestBatchRetryCantCommitIntents(t *testing.T) { t.Errorf("expected %s; got %v", expErr, pErr) } - // Expect that keyB intent got leaked. - gArgs = getArgs(keyB) + // Expect that key A intent got leaked. + gArgs = getArgs(key) _, pErr = tc.SendWrapped(&gArgs) if _, ok := pErr.GetDetail().(*roachpb.WriteIntentError); !ok { t.Errorf("expected WriteIntentError, got: %v", pErr) @@ -9609,6 +9596,7 @@ func TestReplicaServersideRefreshes(t *testing.T) { put := putArgs(roachpb.Key(key), []byte(val)) ba.Add(&put) return send(ba) + } testCases := []struct { diff --git a/pkg/storage/replica_write.go b/pkg/storage/replica_write.go index 0ceabdb8b589..e58b95d26127 100644 --- a/pkg/storage/replica_write.go +++ b/pkg/storage/replica_write.go @@ -366,8 +366,6 @@ func (r *Replica) evaluate1PC( // Try executing with transaction stripped. strippedBa := *ba strippedBa.Txn = nil - // strippedBa is non-transactional, so DeferWriteTooOldError cannot be set. - strippedBa.DeferWriteTooOldError = false strippedBa.Requests = ba.Requests[:len(ba.Requests)-1] // strip end txn req rec := NewReplicaEvalContext(r, spans) @@ -474,10 +472,19 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( batch, br, res, pErr = r.evaluateWriteBatchWrapper(ctx, idKey, rec, ms, ba, spans) + pErrOrWTO := pErr + if pErr == nil && br.Txn != nil && br.Txn.WriteTooOld { + bumpedTxn := br.Txn.Clone() + bumpedTxn.WriteTooOld = false + bumpedTxn.ReadTimestamp = bumpedTxn.WriteTimestamp + pErrOrWTO = roachpb.NewErrorWithTxn( + roachpb.NewTransactionRetryError(roachpb.RETRY_WRITE_TOO_OLD, ""), + bumpedTxn) + } // If we can retry, set a higher batch timestamp and continue. // Allow one retry only; a non-txn batch containing overlapping // spans will always experience WriteTooOldError. - if pErr == nil || retries > 0 || !canDoServersideRetry(ctx, pErr, ba, deadline) { + if pErrOrWTO == nil || retries > 0 || !canDoServersideRetry(ctx, pErrOrWTO, ba, deadline) { break } } diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index b36f16c68dbc..24002c1c3ddc 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -1545,7 +1545,6 @@ func TestStoreResolveWriteIntent(t *testing.T) { manual.Increment(100) // Now, try a put using the pusher's txn. h.Txn = pusher - h.DeferWriteTooOldError = true resultCh := make(chan *roachpb.Error, 1) go func() { _, pErr := client.SendWrappedWith(context.Background(), store.TestSender(), h, &pArgs) @@ -1733,7 +1732,7 @@ func TestStoreResolveWriteIntentPushOnRead(t *testing.T) { { args := putArgs(key, []byte("value2")) assignSeqNumsForReqs(pushee, &args) - h := roachpb.Header{Txn: pushee, DeferWriteTooOldError: true} + h := roachpb.Header{Txn: pushee} if _, pErr := client.SendWrappedWith(ctx, store.TestSender(), h, &args); pErr != nil { t.Fatal(pErr) }