From 6fe53e75583433b99173b472ea10823787bed315 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 4 Nov 2019 18:20:09 +0100 Subject: [PATCH] libroach,storage: extend MVCC to support ignored seqnum ranges The MVCC code already had rudimentary understanding of sequence numbers to allow reads to ignore writes at greater seqnums. To implement SQL savepoint rollbacks, we must also support ignoring writes that fall in ignored ranges of seqnums. To achieve this, this commit extends the `mvccScanner` for RocksDB (Pebble code remains to be done) to account for ignored seqnum ranges, and also extends `MVCCResolveWriteIntent` to collapse an intent to the last write that has not been marked to be ignored by a savepoint rollback. Release note: None --- c-deps/libroach/include/libroach.h | 14 + c-deps/libroach/mvcc.h | 98 +- c-deps/libroach/protos/roachpb/api.pb.cc | 99 +- c-deps/libroach/protos/roachpb/api.pb.h | 76 + c-deps/libroach/protos/roachpb/data.pb.cc | 95 +- c-deps/libroach/protos/roachpb/data.pb.h | 76 + c-deps/libroach/protos/roachpb/errors.pb.cc | 4 +- .../storage/engine/enginepb/mvcc3.pb.cc | 235 +++ .../protos/storage/engine/enginepb/mvcc3.pb.h | 164 ++- pkg/roachpb/api.go | 21 + pkg/roachpb/api.pb.go | 1255 +++++++++-------- pkg/roachpb/api.proto | 4 + pkg/roachpb/data.go | 43 +- pkg/roachpb/data.pb.go | 536 ++++--- pkg/roachpb/data.proto | 20 + pkg/roachpb/data_test.go | 23 + pkg/storage/batcheval/cmd_end_transaction.go | 2 +- .../batcheval/cmd_end_transaction_test.go | 85 ++ .../batcheval/cmd_refresh_range_test.go | 8 +- pkg/storage/batcheval/cmd_resolve_intent.go | 6 +- .../batcheval/cmd_resolve_intent_range.go | 6 +- .../batcheval/cmd_resolve_intent_test.go | 126 ++ pkg/storage/client_metrics_test.go | 8 +- pkg/storage/engine/enginepb/mvcc.go | 67 +- pkg/storage/engine/enginepb/mvcc3.pb.go | 381 +++-- pkg/storage/engine/enginepb/mvcc3.proto | 9 + pkg/storage/engine/enginepb/mvcc_test.go | 30 + pkg/storage/engine/mvcc.go | 270 +++- pkg/storage/engine/mvcc_history_test.go | 59 +- .../engine/mvcc_incremental_iterator.go | 8 +- .../engine/mvcc_incremental_iterator_test.go | 12 +- pkg/storage/engine/mvcc_logical_ops_test.go | 36 +- pkg/storage/engine/mvcc_stats_test.go | 58 +- pkg/storage/engine/mvcc_test.go | 131 +- pkg/storage/engine/rocksdb.go | 11 + .../mvcc_histories/conditional_put_with_txn | 46 +- .../mvcc_histories/idempotent_transactions | 2 +- .../testdata/mvcc_histories/ignored_seq_nums | 140 ++ .../mvcc_histories/ignored_seq_nums_commit | 80 ++ .../mvcc_histories/ignored_seq_nums_cput | 208 +++ .../intentresolver/contention_queue.go | 6 +- pkg/storage/intentresolver/intent_resolver.go | 27 +- .../intentresolver/intent_resolver_test.go | 123 +- pkg/storage/replica_test.go | 33 +- 44 files changed, 3504 insertions(+), 1237 deletions(-) create mode 100644 pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums create mode 100644 pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_commit create mode 100644 pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_cput diff --git a/c-deps/libroach/include/libroach.h b/c-deps/libroach/include/libroach.h index 0fbd3f0ca603..ea91f8eb7ee4 100644 --- a/c-deps/libroach/include/libroach.h +++ b/c-deps/libroach/include/libroach.h @@ -63,6 +63,19 @@ typedef struct { DBStatus status; } DBIterState; +// A DBIgnoredSeqNumRange is an alias for the Go struct +// IgnoredSeqNumRange. It must have exactly the same memory +// layout. +typedef struct { + int32_t start_seqnum; + int32_t end_seqnum; +} DBIgnoredSeqNumRange; + +typedef struct { + DBIgnoredSeqNumRange* ranges; + int len; +} DBIgnoredSeqNums; + typedef struct DBCache DBCache; typedef struct DBEngine DBEngine; typedef struct DBIterator DBIterator; @@ -328,6 +341,7 @@ typedef struct { uint32_t epoch; int32_t sequence; DBTimestamp max_timestamp; + DBIgnoredSeqNums ignored_seqnums; } DBTxn; typedef struct { diff --git a/c-deps/libroach/mvcc.h b/c-deps/libroach/mvcc.h index 90e35532a95a..7731dde0da01 100644 --- a/c-deps/libroach/mvcc.h +++ b/c-deps/libroach/mvcc.h @@ -59,6 +59,7 @@ template class mvccScanner { txn_epoch_(txn.epoch), txn_sequence_(txn.sequence), txn_max_timestamp_(txn.max_timestamp), + txn_ignored_seqnums_(txn.ignored_seqnums), inconsistent_(inconsistent), tombstones_(tombstones), check_uncertainty_(timestamp < txn.max_timestamp), @@ -163,27 +164,83 @@ template class mvccScanner { return results_; } + bool seqNumIsIgnored(int32_t sequence) const { + // The ignored seqnum ranges are guaranteed to be + // non-overlapping, non-contiguous, and guaranteed to be + // sorted in seqnum order. We're going to look from the end to + // see if the current intent seqnum is ignored. + // + // TODO(nathan): this can use use binary search to improve the complexity. + for (int i = txn_ignored_seqnums_.len - 1; i >= 0; i--) { + if (sequence < txn_ignored_seqnums_.ranges[i].start_seqnum) { + // The history entry's sequence number is lower/older than + // the current ignored range. Go to the previous range + // and try again. + continue; + } + + // Here we have a range where the start seqnum is lower than the current + // intent seqnum. Does it include it? + if (sequence > txn_ignored_seqnums_.ranges[i].end_seqnum) { + // Here we have a range where the current history entry's seqnum + // is higher than the range's end seqnum. Given that the + // ranges are storted, we're guaranteed that there won't + // be any further overlapping range at a lower value of i. + return false; + } + // Yes, it's included. We're going to skip over this + // intent seqnum and retry the search above. + return true; + } + + // Exhausted the ignore list. Not ignored. + return false; + } + bool getFromIntentHistory() { - cockroach::storage::engine::enginepb::MVCCMetadata_SequencedIntent readIntent; - readIntent.set_sequence(txn_sequence_); - // Look for the intent with the sequence number less than or equal to the - // read sequence. To do so, search using upper_bound, which returns an - // iterator pointing to the first element in the range [first, last) that is - // greater than value, or last if no such element is found. Then, return the - // previous value. - auto up = std::upper_bound( - meta_.intent_history().begin(), meta_.intent_history().end(), readIntent, - [](const cockroach::storage::engine::enginepb::MVCCMetadata_SequencedIntent& a, - const cockroach::storage::engine::enginepb::MVCCMetadata_SequencedIntent& b) -> bool { - return a.sequence() < b.sequence(); - }); - if (up == meta_.intent_history().begin()) { - // It is possible that no intent exists such that the sequence is less - // than the read sequence. In this case, we cannot read a value from the - // intent history. - return false; + cockroach::storage::engine::enginepb::MVCCMetadata_SequencedIntent readIntent; + readIntent.set_sequence(txn_sequence_); + + auto end = meta_.intent_history().end(); + cockroach::storage::engine::enginepb::MVCCMetadata_SequencedIntent intent; + while (true) { + // Look for the intent with the sequence number less than or equal to the + // read sequence. To do so, search using upper_bound, which returns an + // iterator pointing to the first element in the range [first, last) that is + // greater than value, or last if no such element is found. Then, return the + // previous value. + auto up = std::upper_bound( + meta_.intent_history().begin(), end, readIntent, + [](const cockroach::storage::engine::enginepb::MVCCMetadata_SequencedIntent& a, + const cockroach::storage::engine::enginepb::MVCCMetadata_SequencedIntent& b) -> bool { + return a.sequence() < b.sequence(); + }); + if (up == meta_.intent_history().begin()) { + // It is possible that no intent exists such that the sequence is less + // than the read sequence. In this case, we cannot read a value from the + // intent history. + return false; + } + + const auto intent_pos = up - 1; + // Here we have found a history entry with the highest seqnum that's + // equal or lower to the txn seqnum. + // + // However this entry may also be part of an ignored range + // (partially rolled back). We'll check this next. + // If it is, we'll try the search again. + if (seqNumIsIgnored(intent_pos->sequence())) { + // This entry was part of an ignored range. Skip it and + // try the search again, using the current position as new + // upper bound. + end = intent_pos; + continue; + } + // This history entry has not been ignored, so we're going to keep it. + intent = *intent_pos; + break; } - const auto intent = *(up - 1); + rocksdb::Slice value = intent.value(); if (value.size() > 0 || tombstones_) { kvs_->Put(cur_raw_key_, value); @@ -301,7 +358,7 @@ template class mvccScanner { } if (txn_epoch_ == meta_.txn().epoch()) { - if (txn_sequence_ >= meta_.txn().sequence()) { + if (txn_sequence_ >= meta_.txn().sequence() && !seqNumIsIgnored(meta_.txn().sequence())) { // 8. We're reading our own txn's intent at an equal or higher sequence. // Note that we read at the intent timestamp, not at our read timestamp // as the intent timestamp may have been pushed forward by another @@ -663,6 +720,7 @@ template class mvccScanner { const uint32_t txn_epoch_; const int32_t txn_sequence_; const DBTimestamp txn_max_timestamp_; + const DBIgnoredSeqNums txn_ignored_seqnums_; const bool inconsistent_; const bool tombstones_; const bool check_uncertainty_; diff --git a/c-deps/libroach/protos/roachpb/api.pb.cc b/c-deps/libroach/protos/roachpb/api.pb.cc index a886bbbac683..771c69276d4e 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.cc +++ b/c-deps/libroach/protos/roachpb/api.pb.cc @@ -106,7 +106,6 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobu extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_RecoverTxnResponse; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_RefreshRangeRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_RefreshRequest; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ResolveIntentRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ReverseScanResponse; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_RevertRangeRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ScanResponse; @@ -116,7 +115,7 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobu extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_ExportResponse; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_GCRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_RequestLeaseRequest; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_ResolveIntentRangeRequest; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_ResolveIntentRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_ResponseHeader; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_SubsumeResponse; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<42> scc_info_ResponseUnion; @@ -126,6 +125,7 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobu extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<4> scc_info_ExportRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<4> scc_info_Header; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<4> scc_info_PushTxnRequest; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<4> scc_info_ResolveIntentRangeRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<5> scc_info_EndTransactionRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<5> scc_info_ImportRequest; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fapi_2eproto ::google::protobuf::internal::SCCInfo<6> scc_info_ExternalStorage; @@ -137,7 +137,7 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protob extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_Value; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_Lease; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<5> scc_info_InternalCommitTrigger; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<5> scc_info_Transaction; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<6> scc_info_Transaction; } // namespace protobuf_roachpb_2fdata_2eproto namespace protobuf_roachpb_2ferrors_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2ferrors_2eproto ::google::protobuf::internal::SCCInfo<31> scc_info_AmbiguousResultError; @@ -148,6 +148,7 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fmetadata_2eproto ::google::pr extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fmetadata_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_RangeDescriptor; } // namespace protobuf_roachpb_2fmetadata_2eproto namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto { +extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_IgnoredSeqNumRange; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCStatsDelta; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_TxnMeta; } // namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto @@ -1759,10 +1760,11 @@ static void InitDefaultsResolveIntentRequest() { ::cockroach::roachpb::ResolveIntentRequest::InitAsDefaultInstance(); } -::google::protobuf::internal::SCCInfo<2> scc_info_ResolveIntentRequest = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsResolveIntentRequest}, { +::google::protobuf::internal::SCCInfo<3> scc_info_ResolveIntentRequest = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 3, InitDefaultsResolveIntentRequest}, { &protobuf_roachpb_2fapi_2eproto::scc_info_RequestHeader.base, - &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta.base,}}; + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta.base, + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_IgnoredSeqNumRange.base,}}; static void InitDefaultsResolveIntentResponse() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -1790,11 +1792,12 @@ static void InitDefaultsResolveIntentRangeRequest() { ::cockroach::roachpb::ResolveIntentRangeRequest::InitAsDefaultInstance(); } -::google::protobuf::internal::SCCInfo<3> scc_info_ResolveIntentRangeRequest = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 3, InitDefaultsResolveIntentRangeRequest}, { +::google::protobuf::internal::SCCInfo<4> scc_info_ResolveIntentRangeRequest = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 4, InitDefaultsResolveIntentRangeRequest}, { &protobuf_roachpb_2fapi_2eproto::scc_info_RequestHeader.base, &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta.base, - &protobuf_util_2fhlc_2ftimestamp_2eproto::scc_info_Timestamp.base,}}; + &protobuf_util_2fhlc_2ftimestamp_2eproto::scc_info_Timestamp.base, + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_IgnoredSeqNumRange.base,}}; static void InitDefaultsResolveIntentRangeResponse() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -17275,11 +17278,15 @@ void ResolveIntentRequest::clear_intent_txn() { } intent_txn_ = NULL; } +void ResolveIntentRequest::clear_ignored_seqnums() { + ignored_seqnums_.Clear(); +} #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int ResolveIntentRequest::kHeaderFieldNumber; const int ResolveIntentRequest::kIntentTxnFieldNumber; const int ResolveIntentRequest::kStatusFieldNumber; const int ResolveIntentRequest::kPoisonFieldNumber; +const int ResolveIntentRequest::kIgnoredSeqnumsFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 ResolveIntentRequest::ResolveIntentRequest() @@ -17291,7 +17298,8 @@ ResolveIntentRequest::ResolveIntentRequest() } ResolveIntentRequest::ResolveIntentRequest(const ResolveIntentRequest& from) : ::google::protobuf::MessageLite(), - _internal_metadata_(NULL) { + _internal_metadata_(NULL), + ignored_seqnums_(from.ignored_seqnums_) { _internal_metadata_.MergeFrom(from._internal_metadata_); if (from.has_header()) { header_ = new ::cockroach::roachpb::RequestHeader(*from.header_); @@ -17340,6 +17348,7 @@ void ResolveIntentRequest::Clear() { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; + ignored_seqnums_.Clear(); if (GetArenaNoVirtual() == NULL && header_ != NULL) { delete header_; } @@ -17421,6 +17430,17 @@ bool ResolveIntentRequest::MergePartialFromCodedStream( break; } + case 5: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(42u /* 42 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, add_ignored_seqnums())); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -17468,6 +17488,14 @@ void ResolveIntentRequest::SerializeWithCachedSizes( ::google::protobuf::internal::WireFormatLite::WriteBool(4, this->poison(), output); } + for (unsigned int i = 0, + n = static_cast(this->ignored_seqnums_size()); i < n; i++) { + ::google::protobuf::internal::WireFormatLite::WriteMessage( + 5, + this->ignored_seqnums(static_cast(i)), + 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.ResolveIntentRequest) @@ -17479,6 +17507,16 @@ size_t ResolveIntentRequest::ByteSizeLong() const { total_size += (::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size(); + { + unsigned int count = static_cast(this->ignored_seqnums_size()); + total_size += 1UL * count; + for (unsigned int i = 0; i < count; i++) { + total_size += + ::google::protobuf::internal::WireFormatLite::MessageSize( + this->ignored_seqnums(static_cast(i))); + } + } + if (this->has_header()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSize( @@ -17519,6 +17557,7 @@ void ResolveIntentRequest::MergeFrom(const ResolveIntentRequest& from) { ::google::protobuf::uint32 cached_has_bits = 0; (void) cached_has_bits; + ignored_seqnums_.MergeFrom(from.ignored_seqnums_); if (from.has_header()) { mutable_header()->::cockroach::roachpb::RequestHeader::MergeFrom(from.header()); } @@ -17550,6 +17589,7 @@ void ResolveIntentRequest::Swap(ResolveIntentRequest* other) { } void ResolveIntentRequest::InternalSwap(ResolveIntentRequest* other) { using std::swap; + CastToBase(&ignored_seqnums_)->InternalSwap(CastToBase(&other->ignored_seqnums_)); swap(header_, other->header_); swap(intent_txn_, other->intent_txn_); swap(status_, other->status_); @@ -17771,12 +17811,16 @@ void ResolveIntentRangeRequest::clear_min_timestamp() { } min_timestamp_ = NULL; } +void ResolveIntentRangeRequest::clear_ignored_seqnums() { + ignored_seqnums_.Clear(); +} #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int ResolveIntentRangeRequest::kHeaderFieldNumber; const int ResolveIntentRangeRequest::kIntentTxnFieldNumber; const int ResolveIntentRangeRequest::kStatusFieldNumber; const int ResolveIntentRangeRequest::kPoisonFieldNumber; const int ResolveIntentRangeRequest::kMinTimestampFieldNumber; +const int ResolveIntentRangeRequest::kIgnoredSeqnumsFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 ResolveIntentRangeRequest::ResolveIntentRangeRequest() @@ -17788,7 +17832,8 @@ ResolveIntentRangeRequest::ResolveIntentRangeRequest() } ResolveIntentRangeRequest::ResolveIntentRangeRequest(const ResolveIntentRangeRequest& from) : ::google::protobuf::MessageLite(), - _internal_metadata_(NULL) { + _internal_metadata_(NULL), + ignored_seqnums_(from.ignored_seqnums_) { _internal_metadata_.MergeFrom(from._internal_metadata_); if (from.has_header()) { header_ = new ::cockroach::roachpb::RequestHeader(*from.header_); @@ -17843,6 +17888,7 @@ void ResolveIntentRangeRequest::Clear() { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; + ignored_seqnums_.Clear(); if (GetArenaNoVirtual() == NULL && header_ != NULL) { delete header_; } @@ -17939,6 +17985,17 @@ bool ResolveIntentRangeRequest::MergePartialFromCodedStream( break; } + case 6: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(50u /* 50 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, add_ignored_seqnums())); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -17991,6 +18048,14 @@ void ResolveIntentRangeRequest::SerializeWithCachedSizes( 5, this->_internal_min_timestamp(), output); } + for (unsigned int i = 0, + n = static_cast(this->ignored_seqnums_size()); i < n; i++) { + ::google::protobuf::internal::WireFormatLite::WriteMessage( + 6, + this->ignored_seqnums(static_cast(i)), + 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.ResolveIntentRangeRequest) @@ -18002,6 +18067,16 @@ size_t ResolveIntentRangeRequest::ByteSizeLong() const { total_size += (::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size(); + { + unsigned int count = static_cast(this->ignored_seqnums_size()); + total_size += 1UL * count; + for (unsigned int i = 0; i < count; i++) { + total_size += + ::google::protobuf::internal::WireFormatLite::MessageSize( + this->ignored_seqnums(static_cast(i))); + } + } + if (this->has_header()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSize( @@ -18048,6 +18123,7 @@ void ResolveIntentRangeRequest::MergeFrom(const ResolveIntentRangeRequest& from) ::google::protobuf::uint32 cached_has_bits = 0; (void) cached_has_bits; + ignored_seqnums_.MergeFrom(from.ignored_seqnums_); if (from.has_header()) { mutable_header()->::cockroach::roachpb::RequestHeader::MergeFrom(from.header()); } @@ -18082,6 +18158,7 @@ void ResolveIntentRangeRequest::Swap(ResolveIntentRangeRequest* other) { } void ResolveIntentRangeRequest::InternalSwap(ResolveIntentRangeRequest* other) { using std::swap; + CastToBase(&ignored_seqnums_)->InternalSwap(CastToBase(&other->ignored_seqnums_)); swap(header_, other->header_); swap(intent_txn_, other->intent_txn_); swap(min_timestamp_, other->min_timestamp_); diff --git a/c-deps/libroach/protos/roachpb/api.pb.h b/c-deps/libroach/protos/roachpb/api.pb.h index 55965766e999..abf1bf0103dc 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.h +++ b/c-deps/libroach/protos/roachpb/api.pb.h @@ -7923,6 +7923,17 @@ class ResolveIntentRequest : public ::google::protobuf::MessageLite /* @@protoc_ // accessors ------------------------------------------------------- + int ignored_seqnums_size() const; + void clear_ignored_seqnums(); + static const int kIgnoredSeqnumsFieldNumber = 5; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* mutable_ignored_seqnums(int index); + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* + mutable_ignored_seqnums(); + const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& ignored_seqnums(int index) const; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* add_ignored_seqnums(); + const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& + ignored_seqnums() const; + bool has_header() const; void clear_header(); static const int kHeaderFieldNumber = 1; @@ -7961,6 +7972,7 @@ class ResolveIntentRequest : public ::google::protobuf::MessageLite /* @@protoc_ private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange > ignored_seqnums_; ::cockroach::roachpb::RequestHeader* header_; ::cockroach::storage::engine::enginepb::TxnMeta* intent_txn_; int status_; @@ -8161,6 +8173,17 @@ class ResolveIntentRangeRequest : public ::google::protobuf::MessageLite /* @@pr // accessors ------------------------------------------------------- + int ignored_seqnums_size() const; + void clear_ignored_seqnums(); + static const int kIgnoredSeqnumsFieldNumber = 6; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* mutable_ignored_seqnums(int index); + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* + mutable_ignored_seqnums(); + const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& ignored_seqnums(int index) const; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* add_ignored_seqnums(); + const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& + ignored_seqnums() const; + bool has_header() const; void clear_header(); static const int kHeaderFieldNumber = 1; @@ -8210,6 +8233,7 @@ class ResolveIntentRangeRequest : public ::google::protobuf::MessageLite /* @@pr private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange > ignored_seqnums_; ::cockroach::roachpb::RequestHeader* header_; ::cockroach::storage::engine::enginepb::TxnMeta* intent_txn_; ::cockroach::util::hlc::Timestamp* min_timestamp_; @@ -22546,6 +22570,32 @@ inline void ResolveIntentRequest::set_poison(bool value) { // @@protoc_insertion_point(field_set:cockroach.roachpb.ResolveIntentRequest.poison) } +inline int ResolveIntentRequest::ignored_seqnums_size() const { + return ignored_seqnums_.size(); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* ResolveIntentRequest::mutable_ignored_seqnums(int index) { + // @@protoc_insertion_point(field_mutable:cockroach.roachpb.ResolveIntentRequest.ignored_seqnums) + return ignored_seqnums_.Mutable(index); +} +inline ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* +ResolveIntentRequest::mutable_ignored_seqnums() { + // @@protoc_insertion_point(field_mutable_list:cockroach.roachpb.ResolveIntentRequest.ignored_seqnums) + return &ignored_seqnums_; +} +inline const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& ResolveIntentRequest::ignored_seqnums(int index) const { + // @@protoc_insertion_point(field_get:cockroach.roachpb.ResolveIntentRequest.ignored_seqnums) + return ignored_seqnums_.Get(index); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* ResolveIntentRequest::add_ignored_seqnums() { + // @@protoc_insertion_point(field_add:cockroach.roachpb.ResolveIntentRequest.ignored_seqnums) + return ignored_seqnums_.Add(); +} +inline const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& +ResolveIntentRequest::ignored_seqnums() const { + // @@protoc_insertion_point(field_list:cockroach.roachpb.ResolveIntentRequest.ignored_seqnums) + return ignored_seqnums_; +} + // ------------------------------------------------------------------- // ResolveIntentResponse @@ -22782,6 +22832,32 @@ inline void ResolveIntentRangeRequest::set_allocated_min_timestamp(::cockroach:: // @@protoc_insertion_point(field_set_allocated:cockroach.roachpb.ResolveIntentRangeRequest.min_timestamp) } +inline int ResolveIntentRangeRequest::ignored_seqnums_size() const { + return ignored_seqnums_.size(); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* ResolveIntentRangeRequest::mutable_ignored_seqnums(int index) { + // @@protoc_insertion_point(field_mutable:cockroach.roachpb.ResolveIntentRangeRequest.ignored_seqnums) + return ignored_seqnums_.Mutable(index); +} +inline ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* +ResolveIntentRangeRequest::mutable_ignored_seqnums() { + // @@protoc_insertion_point(field_mutable_list:cockroach.roachpb.ResolveIntentRangeRequest.ignored_seqnums) + return &ignored_seqnums_; +} +inline const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& ResolveIntentRangeRequest::ignored_seqnums(int index) const { + // @@protoc_insertion_point(field_get:cockroach.roachpb.ResolveIntentRangeRequest.ignored_seqnums) + return ignored_seqnums_.Get(index); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* ResolveIntentRangeRequest::add_ignored_seqnums() { + // @@protoc_insertion_point(field_add:cockroach.roachpb.ResolveIntentRangeRequest.ignored_seqnums) + return ignored_seqnums_.Add(); +} +inline const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& +ResolveIntentRangeRequest::ignored_seqnums() const { + // @@protoc_insertion_point(field_list:cockroach.roachpb.ResolveIntentRangeRequest.ignored_seqnums) + return ignored_seqnums_; +} + // ------------------------------------------------------------------- // ResolveIntentRangeResponse diff --git a/c-deps/libroach/protos/roachpb/data.pb.cc b/c-deps/libroach/protos/roachpb/data.pb.cc index 75fb25602820..8d8251173c97 100644 --- a/c-deps/libroach/protos/roachpb/data.pb.cc +++ b/c-deps/libroach/protos/roachpb/data.pb.cc @@ -26,13 +26,14 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protob extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_Value; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_ChangeReplicasTrigger; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_MergeTrigger; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<5> scc_info_Transaction; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<6> scc_info_Transaction; } // namespace protobuf_roachpb_2fdata_2eproto namespace protobuf_roachpb_2fmetadata_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fmetadata_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_ReplicaDescriptor; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fmetadata_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_RangeDescriptor; } // namespace protobuf_roachpb_2fmetadata_2eproto namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto { +extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_IgnoredSeqNumRange; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_TxnMeta; } // namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto namespace protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto { @@ -322,13 +323,14 @@ static void InitDefaultsTransaction() { ::cockroach::roachpb::Transaction::InitAsDefaultInstance(); } -::google::protobuf::internal::SCCInfo<5> scc_info_Transaction = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 5, InitDefaultsTransaction}, { +::google::protobuf::internal::SCCInfo<6> scc_info_Transaction = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 6, InitDefaultsTransaction}, { &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta.base, &protobuf_util_2fhlc_2ftimestamp_2eproto::scc_info_Timestamp.base, &protobuf_roachpb_2fdata_2eproto::scc_info_ObservedTimestamp.base, &protobuf_roachpb_2fdata_2eproto::scc_info_Span.base, - &protobuf_roachpb_2fdata_2eproto::scc_info_SequencedWrite.base,}}; + &protobuf_roachpb_2fdata_2eproto::scc_info_SequencedWrite.base, + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_IgnoredSeqNumRange.base,}}; static void InitDefaultsTransactionRecord() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -359,10 +361,11 @@ static void InitDefaultsIntent() { ::cockroach::roachpb::Intent::InitAsDefaultInstance(); } -::google::protobuf::internal::SCCInfo<2> scc_info_Intent = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsIntent}, { +::google::protobuf::internal::SCCInfo<3> scc_info_Intent = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 3, InitDefaultsIntent}, { &protobuf_roachpb_2fdata_2eproto::scc_info_Span.base, - &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta.base,}}; + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta.base, + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_IgnoredSeqNumRange.base,}}; static void InitDefaultsSequencedWrite() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -3467,6 +3470,9 @@ void Transaction::clear_max_timestamp() { } max_timestamp_ = NULL; } +void Transaction::clear_ignored_seqnums() { + ignored_seqnums_.Clear(); +} #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int Transaction::kMetaFieldNumber; const int Transaction::kNameFieldNumber; @@ -3480,6 +3486,7 @@ const int Transaction::kObservedTimestampsFieldNumber; const int Transaction::kWriteTooOldFieldNumber; const int Transaction::kIntentSpansFieldNumber; const int Transaction::kInFlightWritesFieldNumber; +const int Transaction::kIgnoredSeqnumsFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 Transaction::Transaction() @@ -3494,7 +3501,8 @@ Transaction::Transaction(const Transaction& from) _internal_metadata_(NULL), observed_timestamps_(from.observed_timestamps_), intent_spans_(from.intent_spans_), - in_flight_writes_(from.in_flight_writes_) { + in_flight_writes_(from.in_flight_writes_), + ignored_seqnums_(from.ignored_seqnums_) { _internal_metadata_.MergeFrom(from._internal_metadata_); name_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (from.name().size() > 0) { @@ -3570,6 +3578,7 @@ void Transaction::Clear() { observed_timestamps_.Clear(); intent_spans_.Clear(); in_flight_writes_.Clear(); + ignored_seqnums_.Clear(); name_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (GetArenaNoVirtual() == NULL && meta_ != NULL) { delete meta_; @@ -3760,6 +3769,17 @@ bool Transaction::MergePartialFromCodedStream( break; } + case 18: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(146u /* 146 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, add_ignored_seqnums())); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -3861,6 +3881,14 @@ void Transaction::SerializeWithCachedSizes( output); } + for (unsigned int i = 0, + n = static_cast(this->ignored_seqnums_size()); i < n; i++) { + ::google::protobuf::internal::WireFormatLite::WriteMessage( + 18, + this->ignored_seqnums(static_cast(i)), + 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.Transaction) @@ -3902,6 +3930,16 @@ size_t Transaction::ByteSizeLong() const { } } + { + unsigned int count = static_cast(this->ignored_seqnums_size()); + total_size += 2UL * count; + for (unsigned int i = 0; i < count; i++) { + total_size += + ::google::protobuf::internal::WireFormatLite::MessageSize( + this->ignored_seqnums(static_cast(i))); + } + } + // string name = 2; if (this->name().size() > 0) { total_size += 1 + @@ -3975,6 +4013,7 @@ void Transaction::MergeFrom(const Transaction& from) { observed_timestamps_.MergeFrom(from.observed_timestamps_); intent_spans_.MergeFrom(from.intent_spans_); in_flight_writes_.MergeFrom(from.in_flight_writes_); + ignored_seqnums_.MergeFrom(from.ignored_seqnums_); if (from.name().size() > 0) { name_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.name_); @@ -4025,6 +4064,7 @@ void Transaction::InternalSwap(Transaction* other) { CastToBase(&observed_timestamps_)->InternalSwap(CastToBase(&other->observed_timestamps_)); CastToBase(&intent_spans_)->InternalSwap(CastToBase(&other->intent_spans_)); CastToBase(&in_flight_writes_)->InternalSwap(CastToBase(&other->in_flight_writes_)); + CastToBase(&ignored_seqnums_)->InternalSwap(CastToBase(&other->ignored_seqnums_)); name_.Swap(&other->name_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), GetArenaNoVirtual()); swap(meta_, other->meta_); @@ -4399,10 +4439,14 @@ void Intent::clear_txn() { } txn_ = NULL; } +void Intent::clear_ignored_seqnums() { + ignored_seqnums_.Clear(); +} #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int Intent::kSpanFieldNumber; const int Intent::kTxnFieldNumber; const int Intent::kStatusFieldNumber; +const int Intent::kIgnoredSeqnumsFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 Intent::Intent() @@ -4414,7 +4458,8 @@ Intent::Intent() } Intent::Intent(const Intent& from) : ::google::protobuf::MessageLite(), - _internal_metadata_(NULL) { + _internal_metadata_(NULL), + ignored_seqnums_(from.ignored_seqnums_) { _internal_metadata_.MergeFrom(from._internal_metadata_); if (from.has_span()) { span_ = new ::cockroach::roachpb::Span(*from.span_); @@ -4461,6 +4506,7 @@ void Intent::Clear() { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; + ignored_seqnums_.Clear(); if (GetArenaNoVirtual() == NULL && span_ != NULL) { delete span_; } @@ -4526,6 +4572,17 @@ bool Intent::MergePartialFromCodedStream( break; } + case 4: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(34u /* 34 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, add_ignored_seqnums())); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -4568,6 +4625,14 @@ void Intent::SerializeWithCachedSizes( 3, this->status(), output); } + for (unsigned int i = 0, + n = static_cast(this->ignored_seqnums_size()); i < n; i++) { + ::google::protobuf::internal::WireFormatLite::WriteMessage( + 4, + this->ignored_seqnums(static_cast(i)), + 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.Intent) @@ -4579,6 +4644,16 @@ size_t Intent::ByteSizeLong() const { total_size += (::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size(); + { + unsigned int count = static_cast(this->ignored_seqnums_size()); + total_size += 1UL * count; + for (unsigned int i = 0; i < count; i++) { + total_size += + ::google::protobuf::internal::WireFormatLite::MessageSize( + this->ignored_seqnums(static_cast(i))); + } + } + if (this->has_span()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSize( @@ -4614,6 +4689,7 @@ void Intent::MergeFrom(const Intent& from) { ::google::protobuf::uint32 cached_has_bits = 0; (void) cached_has_bits; + ignored_seqnums_.MergeFrom(from.ignored_seqnums_); if (from.has_span()) { mutable_span()->::cockroach::roachpb::Span::MergeFrom(from.span()); } @@ -4642,6 +4718,7 @@ void Intent::Swap(Intent* other) { } void Intent::InternalSwap(Intent* other) { using std::swap; + CastToBase(&ignored_seqnums_)->InternalSwap(CastToBase(&other->ignored_seqnums_)); swap(span_, other->span_); swap(txn_, other->txn_); swap(status_, other->status_); diff --git a/c-deps/libroach/protos/roachpb/data.pb.h b/c-deps/libroach/protos/roachpb/data.pb.h index 49a5b1960549..d0c17f7172a6 100644 --- a/c-deps/libroach/protos/roachpb/data.pb.h +++ b/c-deps/libroach/protos/roachpb/data.pb.h @@ -1703,6 +1703,17 @@ class Transaction : public ::google::protobuf::MessageLite /* @@protoc_insertion const ::google::protobuf::RepeatedPtrField< ::cockroach::roachpb::SequencedWrite >& in_flight_writes() const; + int ignored_seqnums_size() const; + void clear_ignored_seqnums(); + static const int kIgnoredSeqnumsFieldNumber = 18; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* mutable_ignored_seqnums(int index); + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* + mutable_ignored_seqnums(); + const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& ignored_seqnums(int index) const; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* add_ignored_seqnums(); + const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& + ignored_seqnums() const; + // string name = 2; void clear_name(); static const int kNameFieldNumber = 2; @@ -1797,6 +1808,7 @@ class Transaction : public ::google::protobuf::MessageLite /* @@protoc_insertion ::google::protobuf::RepeatedPtrField< ::cockroach::roachpb::ObservedTimestamp > observed_timestamps_; ::google::protobuf::RepeatedPtrField< ::cockroach::roachpb::Span > intent_spans_; ::google::protobuf::RepeatedPtrField< ::cockroach::roachpb::SequencedWrite > in_flight_writes_; + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange > ignored_seqnums_; ::google::protobuf::internal::ArenaStringPtr name_; ::cockroach::storage::engine::enginepb::TxnMeta* meta_; ::cockroach::util::hlc::Timestamp* last_heartbeat_; @@ -2045,6 +2057,17 @@ class Intent : public ::google::protobuf::MessageLite /* @@protoc_insertion_poin // accessors ------------------------------------------------------- + int ignored_seqnums_size() const; + void clear_ignored_seqnums(); + static const int kIgnoredSeqnumsFieldNumber = 4; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* mutable_ignored_seqnums(int index); + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* + mutable_ignored_seqnums(); + const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& ignored_seqnums(int index) const; + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* add_ignored_seqnums(); + const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& + ignored_seqnums() const; + bool has_span() const; void clear_span(); static const int kSpanFieldNumber = 1; @@ -2077,6 +2100,7 @@ class Intent : public ::google::protobuf::MessageLite /* @@protoc_insertion_poin private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange > ignored_seqnums_; ::cockroach::roachpb::Span* span_; ::cockroach::storage::engine::enginepb::TxnMeta* txn_; int status_; @@ -4533,6 +4557,32 @@ Transaction::in_flight_writes() const { return in_flight_writes_; } +inline int Transaction::ignored_seqnums_size() const { + return ignored_seqnums_.size(); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* Transaction::mutable_ignored_seqnums(int index) { + // @@protoc_insertion_point(field_mutable:cockroach.roachpb.Transaction.ignored_seqnums) + return ignored_seqnums_.Mutable(index); +} +inline ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* +Transaction::mutable_ignored_seqnums() { + // @@protoc_insertion_point(field_mutable_list:cockroach.roachpb.Transaction.ignored_seqnums) + return &ignored_seqnums_; +} +inline const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& Transaction::ignored_seqnums(int index) const { + // @@protoc_insertion_point(field_get:cockroach.roachpb.Transaction.ignored_seqnums) + return ignored_seqnums_.Get(index); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* Transaction::add_ignored_seqnums() { + // @@protoc_insertion_point(field_add:cockroach.roachpb.Transaction.ignored_seqnums) + return ignored_seqnums_.Add(); +} +inline const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& +Transaction::ignored_seqnums() const { + // @@protoc_insertion_point(field_list:cockroach.roachpb.Transaction.ignored_seqnums) + return ignored_seqnums_; +} + // ------------------------------------------------------------------- // TransactionRecord @@ -4821,6 +4871,32 @@ inline void Intent::set_status(::cockroach::roachpb::TransactionStatus value) { // @@protoc_insertion_point(field_set:cockroach.roachpb.Intent.status) } +inline int Intent::ignored_seqnums_size() const { + return ignored_seqnums_.size(); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* Intent::mutable_ignored_seqnums(int index) { + // @@protoc_insertion_point(field_mutable:cockroach.roachpb.Intent.ignored_seqnums) + return ignored_seqnums_.Mutable(index); +} +inline ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >* +Intent::mutable_ignored_seqnums() { + // @@protoc_insertion_point(field_mutable_list:cockroach.roachpb.Intent.ignored_seqnums) + return &ignored_seqnums_; +} +inline const ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange& Intent::ignored_seqnums(int index) const { + // @@protoc_insertion_point(field_get:cockroach.roachpb.Intent.ignored_seqnums) + return ignored_seqnums_.Get(index); +} +inline ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* Intent::add_ignored_seqnums() { + // @@protoc_insertion_point(field_add:cockroach.roachpb.Intent.ignored_seqnums) + return ignored_seqnums_.Add(); +} +inline const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >& +Intent::ignored_seqnums() const { + // @@protoc_insertion_point(field_list:cockroach.roachpb.Intent.ignored_seqnums) + return ignored_seqnums_; +} + // ------------------------------------------------------------------- // SequencedWrite diff --git a/c-deps/libroach/protos/roachpb/errors.pb.cc b/c-deps/libroach/protos/roachpb/errors.pb.cc index e89c1ddade3e..ebdaf5efd4fb 100644 --- a/c-deps/libroach/protos/roachpb/errors.pb.cc +++ b/c-deps/libroach/protos/roachpb/errors.pb.cc @@ -19,9 +19,9 @@ namespace protobuf_roachpb_2fdata_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_ObservedTimestamp; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_Value; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_Intent; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_Lease; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<5> scc_info_Transaction; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<3> scc_info_Intent; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<6> scc_info_Transaction; } // namespace protobuf_roachpb_2fdata_2eproto namespace protobuf_roachpb_2ferrors_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2ferrors_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_ErrPosition; diff --git a/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.cc b/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.cc index 6d9250965f86..692f1605a77e 100644 --- a/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.cc +++ b/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.cc @@ -37,6 +37,11 @@ class TxnMetaDefaultTypeInternal { ::google::protobuf::internal::ExplicitlyConstructed _instance; } _TxnMeta_default_instance_; +class IgnoredSeqNumRangeDefaultTypeInternal { + public: + ::google::protobuf::internal::ExplicitlyConstructed + _instance; +} _IgnoredSeqNumRange_default_instance_; class MVCCStatsDeltaDefaultTypeInternal { public: ::google::protobuf::internal::ExplicitlyConstructed @@ -107,6 +112,20 @@ ::google::protobuf::internal::SCCInfo<1> scc_info_TxnMeta = {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsTxnMeta}, { &protobuf_util_2fhlc_2ftimestamp_2eproto::scc_info_Timestamp.base,}}; +static void InitDefaultsIgnoredSeqNumRange() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::cockroach::storage::engine::enginepb::_IgnoredSeqNumRange_default_instance_; + new (ptr) ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange(); + ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); + } + ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange::InitAsDefaultInstance(); +} + +::google::protobuf::internal::SCCInfo<0> scc_info_IgnoredSeqNumRange = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsIgnoredSeqNumRange}, {}}; + static void InitDefaultsMVCCStatsDelta() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -260,6 +279,7 @@ ::google::protobuf::internal::SCCInfo<6> scc_info_MVCCLogicalOp = void InitDefaults() { ::google::protobuf::internal::InitSCC(&scc_info_TxnMeta.base); + ::google::protobuf::internal::InitSCC(&scc_info_IgnoredSeqNumRange.base); ::google::protobuf::internal::InitSCC(&scc_info_MVCCStatsDelta.base); ::google::protobuf::internal::InitSCC(&scc_info_MVCCPersistentStats.base); ::google::protobuf::internal::InitSCC(&scc_info_RangeAppliedState.base); @@ -683,6 +703,218 @@ ::std::string TxnMeta::GetTypeName() const { } +// =================================================================== + +void IgnoredSeqNumRange::InitAsDefaultInstance() { +} +#if !defined(_MSC_VER) || _MSC_VER >= 1900 +const int IgnoredSeqNumRange::kStartFieldNumber; +const int IgnoredSeqNumRange::kEndFieldNumber; +#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 + +IgnoredSeqNumRange::IgnoredSeqNumRange() + : ::google::protobuf::MessageLite(), _internal_metadata_(NULL) { + ::google::protobuf::internal::InitSCC( + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_IgnoredSeqNumRange.base); + SharedCtor(); + // @@protoc_insertion_point(constructor:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) +} +IgnoredSeqNumRange::IgnoredSeqNumRange(const IgnoredSeqNumRange& from) + : ::google::protobuf::MessageLite(), + _internal_metadata_(NULL) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::memcpy(&start_, &from.start_, + static_cast(reinterpret_cast(&end_) - + reinterpret_cast(&start_)) + sizeof(end_)); + // @@protoc_insertion_point(copy_constructor:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) +} + +void IgnoredSeqNumRange::SharedCtor() { + ::memset(&start_, 0, static_cast( + reinterpret_cast(&end_) - + reinterpret_cast(&start_)) + sizeof(end_)); +} + +IgnoredSeqNumRange::~IgnoredSeqNumRange() { + // @@protoc_insertion_point(destructor:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + SharedDtor(); +} + +void IgnoredSeqNumRange::SharedDtor() { +} + +void IgnoredSeqNumRange::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const IgnoredSeqNumRange& IgnoredSeqNumRange::default_instance() { + ::google::protobuf::internal::InitSCC(&protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_IgnoredSeqNumRange.base); + return *internal_default_instance(); +} + + +void IgnoredSeqNumRange::Clear() { +// @@protoc_insertion_point(message_clear_start:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + ::google::protobuf::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + ::memset(&start_, 0, static_cast( + reinterpret_cast(&end_) - + reinterpret_cast(&start_)) + sizeof(end_)); + _internal_metadata_.Clear(); +} + +bool IgnoredSeqNumRange::MergePartialFromCodedStream( + ::google::protobuf::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!GOOGLE_PREDICT_TRUE(EXPRESSION)) goto failure + ::google::protobuf::uint32 tag; + ::google::protobuf::internal::LiteUnknownFieldSetter unknown_fields_setter( + &_internal_metadata_); + ::google::protobuf::io::StringOutputStream unknown_fields_output( + unknown_fields_setter.buffer()); + ::google::protobuf::io::CodedOutputStream unknown_fields_stream( + &unknown_fields_output, false); + // @@protoc_insertion_point(parse_start:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + for (;;) { + ::std::pair<::google::protobuf::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) { + case 1: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(8u /* 8 & 0xFF */)) { + + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>( + input, &start_))); + } else { + goto handle_unusual; + } + break; + } + + case 2: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(16u /* 16 & 0xFF */)) { + + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>( + input, &end_))); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::google::protobuf::internal::WireFormatLite::SkipField( + input, tag, &unknown_fields_stream)); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + return true; +failure: + // @@protoc_insertion_point(parse_failure:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + return false; +#undef DO_ +} + +void IgnoredSeqNumRange::SerializeWithCachedSizes( + ::google::protobuf::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (this->start() != 0) { + ::google::protobuf::internal::WireFormatLite::WriteInt32(1, this->start(), output); + } + + if (this->end() != 0) { + ::google::protobuf::internal::WireFormatLite::WriteInt32(2, this->end(), 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.storage.engine.enginepb.IgnoredSeqNumRange) +} + +size_t IgnoredSeqNumRange::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + size_t total_size = 0; + + total_size += (::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size(); + + if (this->start() != 0) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::Int32Size( + this->start()); + } + + if (this->end() != 0) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::Int32Size( + this->end()); + } + + int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void IgnoredSeqNumRange::CheckTypeAndMergeFrom( + const ::google::protobuf::MessageLite& from) { + MergeFrom(*::google::protobuf::down_cast(&from)); +} + +void IgnoredSeqNumRange::MergeFrom(const IgnoredSeqNumRange& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::google::protobuf::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.start() != 0) { + set_start(from.start()); + } + if (from.end() != 0) { + set_end(from.end()); + } +} + +void IgnoredSeqNumRange::CopyFrom(const IgnoredSeqNumRange& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool IgnoredSeqNumRange::IsInitialized() const { + return true; +} + +void IgnoredSeqNumRange::Swap(IgnoredSeqNumRange* other) { + if (other == this) return; + InternalSwap(other); +} +void IgnoredSeqNumRange::InternalSwap(IgnoredSeqNumRange* other) { + using std::swap; + swap(start_, other->start_); + swap(end_, other->end_); + _internal_metadata_.Swap(&other->_internal_metadata_); +} + +::std::string IgnoredSeqNumRange::GetTypeName() const { + return "cockroach.storage.engine.enginepb.IgnoredSeqNumRange"; +} + + // =================================================================== void MVCCStatsDelta::InitAsDefaultInstance() { @@ -4080,6 +4312,9 @@ namespace protobuf { template<> GOOGLE_PROTOBUF_ATTRIBUTE_NOINLINE ::cockroach::storage::engine::enginepb::TxnMeta* Arena::CreateMaybeMessage< ::cockroach::storage::engine::enginepb::TxnMeta >(Arena* arena) { return Arena::CreateInternal< ::cockroach::storage::engine::enginepb::TxnMeta >(arena); } +template<> GOOGLE_PROTOBUF_ATTRIBUTE_NOINLINE ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* Arena::CreateMaybeMessage< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >(Arena* arena) { + return Arena::CreateInternal< ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange >(arena); +} template<> GOOGLE_PROTOBUF_ATTRIBUTE_NOINLINE ::cockroach::storage::engine::enginepb::MVCCStatsDelta* Arena::CreateMaybeMessage< ::cockroach::storage::engine::enginepb::MVCCStatsDelta >(Arena* arena) { return Arena::CreateInternal< ::cockroach::storage::engine::enginepb::MVCCStatsDelta >(arena); } diff --git a/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.h b/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.h index a2e372a4df91..2c05597a0208 100644 --- a/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.h +++ b/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.h @@ -38,7 +38,7 @@ namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto { struct TableStruct { static const ::google::protobuf::internal::ParseTableField entries[]; static const ::google::protobuf::internal::AuxillaryParseTableField aux[]; - static const ::google::protobuf::internal::ParseTable schema[11]; + static const ::google::protobuf::internal::ParseTable schema[12]; static const ::google::protobuf::internal::FieldMetadata field_metadata[]; static const ::google::protobuf::internal::SerializationTable serialization_table[]; static const ::google::protobuf::uint32 offsets[]; @@ -48,6 +48,9 @@ namespace cockroach { namespace storage { namespace engine { namespace enginepb { +class IgnoredSeqNumRange; +class IgnoredSeqNumRangeDefaultTypeInternal; +extern IgnoredSeqNumRangeDefaultTypeInternal _IgnoredSeqNumRange_default_instance_; class MVCCAbortIntentOp; class MVCCAbortIntentOpDefaultTypeInternal; extern MVCCAbortIntentOpDefaultTypeInternal _MVCCAbortIntentOp_default_instance_; @@ -87,6 +90,7 @@ extern TxnMetaDefaultTypeInternal _TxnMeta_default_instance_; } // namespace cockroach namespace google { namespace protobuf { +template<> ::cockroach::storage::engine::enginepb::IgnoredSeqNumRange* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::IgnoredSeqNumRange>(Arena*); template<> ::cockroach::storage::engine::enginepb::MVCCAbortIntentOp* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::MVCCAbortIntentOp>(Arena*); template<> ::cockroach::storage::engine::enginepb::MVCCAbortTxnOp* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::MVCCAbortTxnOp>(Arena*); template<> ::cockroach::storage::engine::enginepb::MVCCCommitIntentOp* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::MVCCCommitIntentOp>(Arena*); @@ -272,6 +276,112 @@ class TxnMeta : public ::google::protobuf::MessageLite /* @@protoc_insertion_poi }; // ------------------------------------------------------------------- +class IgnoredSeqNumRange : public ::google::protobuf::MessageLite /* @@protoc_insertion_point(class_definition:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) */ { + public: + IgnoredSeqNumRange(); + virtual ~IgnoredSeqNumRange(); + + IgnoredSeqNumRange(const IgnoredSeqNumRange& from); + + inline IgnoredSeqNumRange& operator=(const IgnoredSeqNumRange& from) { + CopyFrom(from); + return *this; + } + #if LANG_CXX11 + IgnoredSeqNumRange(IgnoredSeqNumRange&& from) noexcept + : IgnoredSeqNumRange() { + *this = ::std::move(from); + } + + inline IgnoredSeqNumRange& operator=(IgnoredSeqNumRange&& from) noexcept { + if (GetArenaNoVirtual() == from.GetArenaNoVirtual()) { + if (this != &from) InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + #endif + static const IgnoredSeqNumRange& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const IgnoredSeqNumRange* internal_default_instance() { + return reinterpret_cast( + &_IgnoredSeqNumRange_default_instance_); + } + static constexpr int kIndexInFileMessages = + 1; + + void Swap(IgnoredSeqNumRange* other); + friend void swap(IgnoredSeqNumRange& a, IgnoredSeqNumRange& b) { + a.Swap(&b); + } + + // implements Message ---------------------------------------------- + + inline IgnoredSeqNumRange* New() const final { + return CreateMaybeMessage(NULL); + } + + IgnoredSeqNumRange* New(::google::protobuf::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CheckTypeAndMergeFrom(const ::google::protobuf::MessageLite& from) + final; + void CopyFrom(const IgnoredSeqNumRange& from); + void MergeFrom(const IgnoredSeqNumRange& from); + void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + bool MergePartialFromCodedStream( + ::google::protobuf::io::CodedInputStream* input) final; + void SerializeWithCachedSizes( + ::google::protobuf::io::CodedOutputStream* output) const final; + void DiscardUnknownFields(); + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const; + void InternalSwap(IgnoredSeqNumRange* other); + private: + inline ::google::protobuf::Arena* GetArenaNoVirtual() const { + return NULL; + } + inline void* MaybeArenaPtr() const { + return NULL; + } + public: + + ::std::string GetTypeName() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + void clear_start(); + static const int kStartFieldNumber = 1; + ::google::protobuf::int32 start() const; + void set_start(::google::protobuf::int32 value); + + void clear_end(); + static const int kEndFieldNumber = 2; + ::google::protobuf::int32 end() const; + void set_end(::google::protobuf::int32 value); + + // @@protoc_insertion_point(class_scope:cockroach.storage.engine.enginepb.IgnoredSeqNumRange) + private: + + ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; + ::google::protobuf::int32 start_; + ::google::protobuf::int32 end_; + mutable ::google::protobuf::internal::CachedSize _cached_size_; + friend struct ::protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::TableStruct; +}; +// ------------------------------------------------------------------- + class MVCCStatsDelta : public ::google::protobuf::MessageLite /* @@protoc_insertion_point(class_definition:cockroach.storage.engine.enginepb.MVCCStatsDelta) */ { public: MVCCStatsDelta(); @@ -306,7 +416,7 @@ class MVCCStatsDelta : public ::google::protobuf::MessageLite /* @@protoc_insert &_MVCCStatsDelta_default_instance_); } static constexpr int kIndexInFileMessages = - 1; + 2; void Swap(MVCCStatsDelta* other); friend void swap(MVCCStatsDelta& a, MVCCStatsDelta& b) { @@ -497,7 +607,7 @@ class MVCCPersistentStats : public ::google::protobuf::MessageLite /* @@protoc_i &_MVCCPersistentStats_default_instance_); } static constexpr int kIndexInFileMessages = - 2; + 3; void Swap(MVCCPersistentStats* other); friend void swap(MVCCPersistentStats& a, MVCCPersistentStats& b) { @@ -688,7 +798,7 @@ class RangeAppliedState : public ::google::protobuf::MessageLite /* @@protoc_ins &_RangeAppliedState_default_instance_); } static constexpr int kIndexInFileMessages = - 3; + 4; void Swap(RangeAppliedState* other); friend void swap(RangeAppliedState& a, RangeAppliedState& b) { @@ -808,7 +918,7 @@ class MVCCWriteValueOp : public ::google::protobuf::MessageLite /* @@protoc_inse &_MVCCWriteValueOp_default_instance_); } static constexpr int kIndexInFileMessages = - 4; + 5; void Swap(MVCCWriteValueOp* other); friend void swap(MVCCWriteValueOp& a, MVCCWriteValueOp& b) { @@ -959,7 +1069,7 @@ class MVCCWriteIntentOp : public ::google::protobuf::MessageLite /* @@protoc_ins &_MVCCWriteIntentOp_default_instance_); } static constexpr int kIndexInFileMessages = - 5; + 6; void Swap(MVCCWriteIntentOp* other); friend void swap(MVCCWriteIntentOp& a, MVCCWriteIntentOp& b) { @@ -1106,7 +1216,7 @@ class MVCCUpdateIntentOp : public ::google::protobuf::MessageLite /* @@protoc_in &_MVCCUpdateIntentOp_default_instance_); } static constexpr int kIndexInFileMessages = - 6; + 7; void Swap(MVCCUpdateIntentOp* other); friend void swap(MVCCUpdateIntentOp& a, MVCCUpdateIntentOp& b) { @@ -1226,7 +1336,7 @@ class MVCCCommitIntentOp : public ::google::protobuf::MessageLite /* @@protoc_in &_MVCCCommitIntentOp_default_instance_); } static constexpr int kIndexInFileMessages = - 7; + 8; void Swap(MVCCCommitIntentOp* other); friend void swap(MVCCCommitIntentOp& a, MVCCCommitIntentOp& b) { @@ -1391,7 +1501,7 @@ class MVCCAbortIntentOp : public ::google::protobuf::MessageLite /* @@protoc_ins &_MVCCAbortIntentOp_default_instance_); } static constexpr int kIndexInFileMessages = - 8; + 9; void Swap(MVCCAbortIntentOp* other); friend void swap(MVCCAbortIntentOp& a, MVCCAbortIntentOp& b) { @@ -1499,7 +1609,7 @@ class MVCCAbortTxnOp : public ::google::protobuf::MessageLite /* @@protoc_insert &_MVCCAbortTxnOp_default_instance_); } static constexpr int kIndexInFileMessages = - 9; + 10; void Swap(MVCCAbortTxnOp* other); friend void swap(MVCCAbortTxnOp& a, MVCCAbortTxnOp& b) { @@ -1607,7 +1717,7 @@ class MVCCLogicalOp : public ::google::protobuf::MessageLite /* @@protoc_inserti &_MVCCLogicalOp_default_instance_); } static constexpr int kIndexInFileMessages = - 10; + 11; void Swap(MVCCLogicalOp* other); friend void swap(MVCCLogicalOp& a, MVCCLogicalOp& b) { @@ -1994,6 +2104,36 @@ inline void TxnMeta::set_sequence(::google::protobuf::int32 value) { // ------------------------------------------------------------------- +// IgnoredSeqNumRange + +inline void IgnoredSeqNumRange::clear_start() { + start_ = 0; +} +inline ::google::protobuf::int32 IgnoredSeqNumRange::start() const { + // @@protoc_insertion_point(field_get:cockroach.storage.engine.enginepb.IgnoredSeqNumRange.start) + return start_; +} +inline void IgnoredSeqNumRange::set_start(::google::protobuf::int32 value) { + + start_ = value; + // @@protoc_insertion_point(field_set:cockroach.storage.engine.enginepb.IgnoredSeqNumRange.start) +} + +inline void IgnoredSeqNumRange::clear_end() { + end_ = 0; +} +inline ::google::protobuf::int32 IgnoredSeqNumRange::end() const { + // @@protoc_insertion_point(field_get:cockroach.storage.engine.enginepb.IgnoredSeqNumRange.end) + return end_; +} +inline void IgnoredSeqNumRange::set_end(::google::protobuf::int32 value) { + + end_ = value; + // @@protoc_insertion_point(field_set:cockroach.storage.engine.enginepb.IgnoredSeqNumRange.end) +} + +// ------------------------------------------------------------------- + // MVCCStatsDelta // int64 contains_estimates = 14; @@ -3716,6 +3856,8 @@ inline void MVCCLogicalOp::set_allocated_abort_txn(::cockroach::storage::engine: // ------------------------------------------------------------------- +// ------------------------------------------------------------------- + // @@protoc_insertion_point(namespace_scope) diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 8559af9ebdb1..f92ef3ede590 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1247,3 +1247,24 @@ func (acrr *AdminChangeReplicasRequest) Changes() []ReplicationChange { } return sl } + +// AsIntent creates an intent corresponding to the given resolve intent request. +func (rir *ResolveIntentRequest) AsIntent() Intent { + return Intent{ + Span: rir.Span(), + Txn: rir.IntentTxn, + Status: rir.Status, + IgnoredSeqNums: rir.IgnoredSeqNums, + } +} + +// AsIntent creates an intent corresponding to the given resolve +// intent range request. +func (rirr *ResolveIntentRangeRequest) AsIntent() Intent { + return Intent{ + Span: rirr.Span(), + Txn: rirr.IntentTxn, + Status: rirr.Status, + IgnoredSeqNums: rirr.IgnoredSeqNums, + } +} diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 53b3f223418d..6fa507e6889a 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_aaa0b4c509856715, []int{0} + return fileDescriptor_api_46e7e3331e4f911b, []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_aaa0b4c509856715, []int{1} + return fileDescriptor_api_46e7e3331e4f911b, []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_aaa0b4c509856715, []int{2} + return fileDescriptor_api_46e7e3331e4f911b, []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_aaa0b4c509856715, []int{3} + return fileDescriptor_api_46e7e3331e4f911b, []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_aaa0b4c509856715, []int{4} + return fileDescriptor_api_46e7e3331e4f911b, []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_aaa0b4c509856715, []int{5} + return fileDescriptor_api_46e7e3331e4f911b, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{2, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{27, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{27, 0} } // RangeInfo describes a range which executed a request. It contains @@ -326,7 +326,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_aaa0b4c509856715, []int{0} + return fileDescriptor_api_46e7e3331e4f911b, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -369,7 +369,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_aaa0b4c509856715, []int{1} + return fileDescriptor_api_46e7e3331e4f911b, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -431,7 +431,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_aaa0b4c509856715, []int{2} + return fileDescriptor_api_46e7e3331e4f911b, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -465,7 +465,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_aaa0b4c509856715, []int{3} + return fileDescriptor_api_46e7e3331e4f911b, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -503,7 +503,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_aaa0b4c509856715, []int{4} + return fileDescriptor_api_46e7e3331e4f911b, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -546,7 +546,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_aaa0b4c509856715, []int{5} + return fileDescriptor_api_46e7e3331e4f911b, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -580,7 +580,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_aaa0b4c509856715, []int{6} + return fileDescriptor_api_46e7e3331e4f911b, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -634,7 +634,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_aaa0b4c509856715, []int{7} + return fileDescriptor_api_46e7e3331e4f911b, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -669,7 +669,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_aaa0b4c509856715, []int{8} + return fileDescriptor_api_46e7e3331e4f911b, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -715,7 +715,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_aaa0b4c509856715, []int{9} + return fileDescriptor_api_46e7e3331e4f911b, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -749,7 +749,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_aaa0b4c509856715, []int{10} + return fileDescriptor_api_46e7e3331e4f911b, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -789,7 +789,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_aaa0b4c509856715, []int{11} + return fileDescriptor_api_46e7e3331e4f911b, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -826,7 +826,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_aaa0b4c509856715, []int{12} + return fileDescriptor_api_46e7e3331e4f911b, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -860,7 +860,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_aaa0b4c509856715, []int{13} + return fileDescriptor_api_46e7e3331e4f911b, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -894,7 +894,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_aaa0b4c509856715, []int{14} + return fileDescriptor_api_46e7e3331e4f911b, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -946,7 +946,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_aaa0b4c509856715, []int{15} + return fileDescriptor_api_46e7e3331e4f911b, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -983,7 +983,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_aaa0b4c509856715, []int{16} + return fileDescriptor_api_46e7e3331e4f911b, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1030,7 +1030,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_aaa0b4c509856715, []int{17} + return fileDescriptor_api_46e7e3331e4f911b, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1064,7 +1064,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_aaa0b4c509856715, []int{18} + return fileDescriptor_api_46e7e3331e4f911b, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1105,7 +1105,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_aaa0b4c509856715, []int{19} + return fileDescriptor_api_46e7e3331e4f911b, []int{19} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1139,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_aaa0b4c509856715, []int{20} + return fileDescriptor_api_46e7e3331e4f911b, []int{20} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1207,7 +1207,7 @@ func (m *ScanOptions) Reset() { *m = ScanOptions{} } func (m *ScanOptions) String() string { return proto.CompactTextString(m) } func (*ScanOptions) ProtoMessage() {} func (*ScanOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{21} + return fileDescriptor_api_46e7e3331e4f911b, []int{21} } func (m *ScanOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1247,7 +1247,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_aaa0b4c509856715, []int{22} + return fileDescriptor_api_46e7e3331e4f911b, []int{22} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1294,7 +1294,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_aaa0b4c509856715, []int{23} + return fileDescriptor_api_46e7e3331e4f911b, []int{23} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1334,7 +1334,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_aaa0b4c509856715, []int{24} + return fileDescriptor_api_46e7e3331e4f911b, []int{24} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1381,7 +1381,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_aaa0b4c509856715, []int{25} + return fileDescriptor_api_46e7e3331e4f911b, []int{25} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1434,7 +1434,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_aaa0b4c509856715, []int{26} + return fileDescriptor_api_46e7e3331e4f911b, []int{26} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1471,7 +1471,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_aaa0b4c509856715, []int{27} + return fileDescriptor_api_46e7e3331e4f911b, []int{27} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1515,7 +1515,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_aaa0b4c509856715, []int{27, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{27, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1563,7 +1563,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_aaa0b4c509856715, []int{28} + return fileDescriptor_api_46e7e3331e4f911b, []int{28} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1599,7 +1599,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_aaa0b4c509856715, []int{29} + return fileDescriptor_api_46e7e3331e4f911b, []int{29} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1687,7 @@ func (m *EndTransactionRequest) Reset() { *m = EndTransactionRequest{} } func (m *EndTransactionRequest) String() string { return proto.CompactTextString(m) } func (*EndTransactionRequest) ProtoMessage() {} func (*EndTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{30} + return fileDescriptor_api_46e7e3331e4f911b, []int{30} } func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1734,7 +1734,7 @@ func (m *EndTransactionResponse) Reset() { *m = EndTransactionResponse{} func (m *EndTransactionResponse) String() string { return proto.CompactTextString(m) } func (*EndTransactionResponse) ProtoMessage() {} func (*EndTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{31} + return fileDescriptor_api_46e7e3331e4f911b, []int{31} } func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1795,7 +1795,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_aaa0b4c509856715, []int{32} + return fileDescriptor_api_46e7e3331e4f911b, []int{32} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1830,7 +1830,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_aaa0b4c509856715, []int{33} + return fileDescriptor_api_46e7e3331e4f911b, []int{33} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1869,7 +1869,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_aaa0b4c509856715, []int{34} + return fileDescriptor_api_46e7e3331e4f911b, []int{34} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1904,7 +1904,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_aaa0b4c509856715, []int{35} + return fileDescriptor_api_46e7e3331e4f911b, []int{35} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1947,7 +1947,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_aaa0b4c509856715, []int{36} + return fileDescriptor_api_46e7e3331e4f911b, []int{36} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1982,7 +1982,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_aaa0b4c509856715, []int{37} + return fileDescriptor_api_46e7e3331e4f911b, []int{37} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2020,7 +2020,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_aaa0b4c509856715, []int{38} + return fileDescriptor_api_46e7e3331e4f911b, []int{38} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2053,7 +2053,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_aaa0b4c509856715, []int{39} + return fileDescriptor_api_46e7e3331e4f911b, []int{39} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2088,7 +2088,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_aaa0b4c509856715, []int{40} + return fileDescriptor_api_46e7e3331e4f911b, []int{40} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2146,7 +2146,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_aaa0b4c509856715, []int{41} + return fileDescriptor_api_46e7e3331e4f911b, []int{41} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2181,7 +2181,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_aaa0b4c509856715, []int{42} + return fileDescriptor_api_46e7e3331e4f911b, []int{42} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2218,7 +2218,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_aaa0b4c509856715, []int{43} + return fileDescriptor_api_46e7e3331e4f911b, []int{43} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2251,7 +2251,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_aaa0b4c509856715, []int{44} + return fileDescriptor_api_46e7e3331e4f911b, []int{44} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2290,7 +2290,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_aaa0b4c509856715, []int{45} + return fileDescriptor_api_46e7e3331e4f911b, []int{45} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2327,7 +2327,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_aaa0b4c509856715, []int{46} + return fileDescriptor_api_46e7e3331e4f911b, []int{46} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2365,7 +2365,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_aaa0b4c509856715, []int{47} + return fileDescriptor_api_46e7e3331e4f911b, []int{47} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2399,7 +2399,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_aaa0b4c509856715, []int{47, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{47, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2433,7 +2433,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_aaa0b4c509856715, []int{48} + return fileDescriptor_api_46e7e3331e4f911b, []int{48} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2508,7 +2508,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_aaa0b4c509856715, []int{49} + return fileDescriptor_api_46e7e3331e4f911b, []int{49} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2551,7 +2551,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_aaa0b4c509856715, []int{50} + return fileDescriptor_api_46e7e3331e4f911b, []int{50} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2598,7 +2598,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_aaa0b4c509856715, []int{51} + return fileDescriptor_api_46e7e3331e4f911b, []int{51} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2634,7 +2634,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_aaa0b4c509856715, []int{52} + return fileDescriptor_api_46e7e3331e4f911b, []int{52} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2678,7 +2678,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_aaa0b4c509856715, []int{53} + return fileDescriptor_api_46e7e3331e4f911b, []int{53} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2717,7 +2717,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_aaa0b4c509856715, []int{54} + return fileDescriptor_api_46e7e3331e4f911b, []int{54} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2777,7 +2777,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_aaa0b4c509856715, []int{55} + return fileDescriptor_api_46e7e3331e4f911b, []int{55} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2813,7 +2813,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_aaa0b4c509856715, []int{56} + return fileDescriptor_api_46e7e3331e4f911b, []int{56} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2851,13 +2851,15 @@ type ResolveIntentRequest struct { // Optionally poison the abort span for the transaction the intent's // range. Poison bool `protobuf:"varint,4,opt,name=poison,proto3" json:"poison,omitempty"` + // The list of ignored seqnum ranges as per the Transaction record. + IgnoredSeqNums []enginepb.IgnoredSeqNumRange `protobuf:"bytes,5,rep,name=ignored_seqnums,json=ignoredSeqnums,proto3" json:"ignored_seqnums"` } 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_aaa0b4c509856715, []int{57} + return fileDescriptor_api_46e7e3331e4f911b, []int{57} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2892,7 +2894,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_aaa0b4c509856715, []int{58} + return fileDescriptor_api_46e7e3331e4f911b, []int{58} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2934,13 +2936,15 @@ type ResolveIntentRangeRequest struct { // transaction. If present, this value can be used to optimize the // iteration over the span to find intents to resolve. MinTimestamp hlc.Timestamp `protobuf:"bytes,5,opt,name=min_timestamp,json=minTimestamp,proto3" json:"min_timestamp"` + // The list of ignored seqnum ranges as per the Transaction record. + IgnoredSeqNums []enginepb.IgnoredSeqNumRange `protobuf:"bytes,6,rep,name=ignored_seqnums,json=ignoredSeqnums,proto3" json:"ignored_seqnums"` } func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeRequest{} } func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{59} + return fileDescriptor_api_46e7e3331e4f911b, []int{59} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2975,7 +2979,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_aaa0b4c509856715, []int{60} + return fileDescriptor_api_46e7e3331e4f911b, []int{60} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3012,7 +3016,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_aaa0b4c509856715, []int{61} + return fileDescriptor_api_46e7e3331e4f911b, []int{61} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3046,7 +3050,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_aaa0b4c509856715, []int{62} + return fileDescriptor_api_46e7e3331e4f911b, []int{62} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3091,7 +3095,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_aaa0b4c509856715, []int{63} + return fileDescriptor_api_46e7e3331e4f911b, []int{63} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3125,7 +3129,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_aaa0b4c509856715, []int{64} + return fileDescriptor_api_46e7e3331e4f911b, []int{64} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3169,7 +3173,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_aaa0b4c509856715, []int{65} + return fileDescriptor_api_46e7e3331e4f911b, []int{65} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3218,7 +3222,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_aaa0b4c509856715, []int{66} + return fileDescriptor_api_46e7e3331e4f911b, []int{66} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3255,7 +3259,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_aaa0b4c509856715, []int{67} + return fileDescriptor_api_46e7e3331e4f911b, []int{67} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3292,7 +3296,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_aaa0b4c509856715, []int{68} + return fileDescriptor_api_46e7e3331e4f911b, []int{68} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3327,7 +3331,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_aaa0b4c509856715, []int{69} + return fileDescriptor_api_46e7e3331e4f911b, []int{69} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3382,7 +3386,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_aaa0b4c509856715, []int{70} + return fileDescriptor_api_46e7e3331e4f911b, []int{70} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3419,7 +3423,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_aaa0b4c509856715, []int{71} + return fileDescriptor_api_46e7e3331e4f911b, []int{71} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3458,7 +3462,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_aaa0b4c509856715, []int{72} + return fileDescriptor_api_46e7e3331e4f911b, []int{72} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3492,7 +3496,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_aaa0b4c509856715, []int{72, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{72, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3525,7 +3529,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_aaa0b4c509856715, []int{72, 1} + return fileDescriptor_api_46e7e3331e4f911b, []int{72, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3565,7 +3569,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_aaa0b4c509856715, []int{72, 2} + return fileDescriptor_api_46e7e3331e4f911b, []int{72, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3604,7 +3608,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_aaa0b4c509856715, []int{72, 3} + return fileDescriptor_api_46e7e3331e4f911b, []int{72, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3640,7 +3644,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_aaa0b4c509856715, []int{72, 4} + return fileDescriptor_api_46e7e3331e4f911b, []int{72, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3679,7 +3683,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_aaa0b4c509856715, []int{72, 5} + return fileDescriptor_api_46e7e3331e4f911b, []int{72, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3719,7 +3723,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_aaa0b4c509856715, []int{73} + return fileDescriptor_api_46e7e3331e4f911b, []int{73} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3753,7 +3757,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_aaa0b4c509856715, []int{74} + return fileDescriptor_api_46e7e3331e4f911b, []int{74} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3815,7 +3819,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_aaa0b4c509856715, []int{75} + return fileDescriptor_api_46e7e3331e4f911b, []int{75} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3851,7 +3855,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_aaa0b4c509856715, []int{76} + return fileDescriptor_api_46e7e3331e4f911b, []int{76} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3887,7 +3891,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_aaa0b4c509856715, []int{77} + return fileDescriptor_api_46e7e3331e4f911b, []int{77} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3927,7 +3931,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_aaa0b4c509856715, []int{77, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{77, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3977,7 +3981,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_aaa0b4c509856715, []int{78} + return fileDescriptor_api_46e7e3331e4f911b, []int{78} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4012,7 +4016,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_aaa0b4c509856715, []int{78, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{78, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4048,7 +4052,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_aaa0b4c509856715, []int{78, 1} + return fileDescriptor_api_46e7e3331e4f911b, []int{78, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4083,7 +4087,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_aaa0b4c509856715, []int{79} + return fileDescriptor_api_46e7e3331e4f911b, []int{79} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4121,7 +4125,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_aaa0b4c509856715, []int{80} + return fileDescriptor_api_46e7e3331e4f911b, []int{80} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4156,7 +4160,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_aaa0b4c509856715, []int{81} + return fileDescriptor_api_46e7e3331e4f911b, []int{81} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4189,7 +4193,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_aaa0b4c509856715, []int{81, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{81, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4242,7 +4246,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_aaa0b4c509856715, []int{82} + return fileDescriptor_api_46e7e3331e4f911b, []int{82} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4276,7 +4280,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_aaa0b4c509856715, []int{83} + return fileDescriptor_api_46e7e3331e4f911b, []int{83} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4325,7 +4329,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_aaa0b4c509856715, []int{84} + return fileDescriptor_api_46e7e3331e4f911b, []int{84} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4359,7 +4363,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_aaa0b4c509856715, []int{85} + return fileDescriptor_api_46e7e3331e4f911b, []int{85} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4402,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_aaa0b4c509856715, []int{86} + return fileDescriptor_api_46e7e3331e4f911b, []int{86} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4436,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_aaa0b4c509856715, []int{87} + return fileDescriptor_api_46e7e3331e4f911b, []int{87} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4485,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_aaa0b4c509856715, []int{88} + return fileDescriptor_api_46e7e3331e4f911b, []int{88} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4534,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_aaa0b4c509856715, []int{89} + return fileDescriptor_api_46e7e3331e4f911b, []int{89} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4569,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_aaa0b4c509856715, []int{90} + return fileDescriptor_api_46e7e3331e4f911b, []int{90} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4608,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_aaa0b4c509856715, []int{91} + return fileDescriptor_api_46e7e3331e4f911b, []int{91} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4690,7 +4694,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_aaa0b4c509856715, []int{92} + return fileDescriptor_api_46e7e3331e4f911b, []int{92} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6109,7 +6113,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_aaa0b4c509856715, []int{93} + return fileDescriptor_api_46e7e3331e4f911b, []int{93} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7526,7 +7530,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{94} + return fileDescriptor_api_46e7e3331e4f911b, []int{94} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7562,7 +7566,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{95} + return fileDescriptor_api_46e7e3331e4f911b, []int{95} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7599,7 +7603,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_aaa0b4c509856715, []int{96} + return fileDescriptor_api_46e7e3331e4f911b, []int{96} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7658,7 +7662,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_aaa0b4c509856715, []int{96, 0} + return fileDescriptor_api_46e7e3331e4f911b, []int{96, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7697,7 +7701,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_aaa0b4c509856715, []int{97} + return fileDescriptor_api_46e7e3331e4f911b, []int{97} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7738,7 +7742,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_aaa0b4c509856715, []int{98} + return fileDescriptor_api_46e7e3331e4f911b, []int{98} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7779,7 +7783,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_aaa0b4c509856715, []int{99} + return fileDescriptor_api_46e7e3331e4f911b, []int{99} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7816,7 +7820,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_aaa0b4c509856715, []int{100} + return fileDescriptor_api_46e7e3331e4f911b, []int{100} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7853,7 +7857,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_aaa0b4c509856715, []int{101} + return fileDescriptor_api_46e7e3331e4f911b, []int{101} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8969,6 +8973,14 @@ func (this *ResolveIntentRequest) Equal(that interface{}) bool { if this.Poison != that1.Poison { return false } + if len(this.IgnoredSeqNums) != len(that1.IgnoredSeqNums) { + return false + } + for i := range this.IgnoredSeqNums { + if !this.IgnoredSeqNums[i].Equal(&that1.IgnoredSeqNums[i]) { + return false + } + } return true } func (this *ResolveIntentRangeRequest) Equal(that interface{}) bool { @@ -9005,6 +9017,14 @@ func (this *ResolveIntentRangeRequest) Equal(that interface{}) bool { if !this.MinTimestamp.Equal(&that1.MinTimestamp) { return false } + if len(this.IgnoredSeqNums) != len(that1.IgnoredSeqNums) { + return false + } + for i := range this.IgnoredSeqNums { + if !this.IgnoredSeqNums[i].Equal(&that1.IgnoredSeqNums[i]) { + return false + } + } return true } func (this *MergeRequest) Equal(that interface{}) bool { @@ -12276,6 +12296,18 @@ func (m *ResolveIntentRequest) MarshalTo(dAtA []byte) (int, error) { } i++ } + if len(m.IgnoredSeqNums) > 0 { + for _, msg := range m.IgnoredSeqNums { + dAtA[i] = 0x2a + i++ + i = encodeVarintApi(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } return i, nil } @@ -12359,6 +12391,18 @@ func (m *ResolveIntentRangeRequest) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n89 + if len(m.IgnoredSeqNums) > 0 { + for _, msg := range m.IgnoredSeqNums { + dAtA[i] = 0x32 + i++ + i = encodeVarintApi(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } return i, nil } @@ -16909,6 +16953,12 @@ func (m *ResolveIntentRequest) Size() (n int) { if m.Poison { n += 2 } + if len(m.IgnoredSeqNums) > 0 { + for _, e := range m.IgnoredSeqNums { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } return n } @@ -16941,6 +16991,12 @@ func (m *ResolveIntentRangeRequest) Size() (n int) { } l = m.MinTimestamp.Size() n += 1 + l + sovApi(uint64(l)) + if len(m.IgnoredSeqNums) > 0 { + for _, e := range m.IgnoredSeqNums { + l = e.Size() + n += 1 + l + sovApi(uint64(l)) + } + } return n } @@ -26057,6 +26113,37 @@ func (m *ResolveIntentRequest) Unmarshal(dAtA []byte) error { } } m.Poison = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IgnoredSeqNums", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IgnoredSeqNums = append(m.IgnoredSeqNums, enginepb.IgnoredSeqNumRange{}) + if err := m.IgnoredSeqNums[len(m.IgnoredSeqNums)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -26316,6 +26403,37 @@ func (m *ResolveIntentRangeRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IgnoredSeqNums", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IgnoredSeqNums = append(m.IgnoredSeqNums, enginepb.IgnoredSeqNumRange{}) + if err := m.IgnoredSeqNums[len(m.IgnoredSeqNums)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -36395,448 +36513,451 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_aaa0b4c509856715) } - -var fileDescriptor_api_aaa0b4c509856715 = []byte{ - // 7032 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x5d, 0xcd, 0x6f, 0x23, 0xc9, - 0x75, 0x57, 0x93, 0x94, 0x44, 0x3e, 0x7e, 0xa8, 0x55, 0xd2, 0xcc, 0x70, 0xb4, 0xbb, 0x92, 0x86, - 0xf3, 0xb9, 0xe3, 0x5d, 0xcd, 0xce, 0x8c, 0x37, 0x5e, 0xef, 0xda, 0x6b, 0x4b, 0x14, 0x67, 0xc8, - 0xd1, 0xe8, 0x63, 0x9b, 0xd4, 0xac, 0x67, 0x9d, 0x4d, 0xbb, 0xd5, 0x5d, 0xa2, 0xda, 0x43, 0x76, - 0x73, 0xba, 0x9b, 0x23, 0x69, 0x80, 0x20, 0xc9, 0xc9, 0x81, 0x63, 0x18, 0x39, 0x04, 0x41, 0x60, - 0x27, 0xf0, 0x06, 0x36, 0x12, 0x20, 0x41, 0x0c, 0x04, 0x08, 0x92, 0x38, 0x48, 0xe0, 0x43, 0x2e, - 0x1b, 0xc3, 0x87, 0x45, 0x90, 0x0f, 0x23, 0x07, 0x21, 0x1e, 0x23, 0x88, 0xe1, 0x7f, 0xc0, 0xc0, - 0x1e, 0x82, 0xa0, 0x3e, 0xfa, 0x8b, 0x6c, 0x52, 0x94, 0xdc, 0x9b, 0x6c, 0x90, 0x13, 0xd9, 0xaf, - 0xea, 0xbd, 0xae, 0x7a, 0x55, 0xf5, 0xea, 0xfd, 0xaa, 0x5e, 0x55, 0xc3, 0xb4, 0x65, 0x2a, 0xea, - 0x5e, 0x67, 0xe7, 0x86, 0xd2, 0xd1, 0x97, 0x3a, 0x96, 0xe9, 0x98, 0x68, 0x5a, 0x35, 0xd5, 0x47, - 0x94, 0xbc, 0xc4, 0x13, 0xe7, 0x90, 0x9b, 0x4b, 0x53, 0x1c, 0x85, 0x65, 0x9b, 0x9b, 0x75, 0x69, - 0xd8, 0xb2, 0x4c, 0xcb, 0xe6, 0xd4, 0xb3, 0x2e, 0xb5, 0x8d, 0x1d, 0x25, 0x90, 0xbb, 0x64, 0x3b, - 0xa6, 0xa5, 0x34, 0xf1, 0x0d, 0x6c, 0x34, 0x75, 0xc3, 0xfd, 0x21, 0xf9, 0x9e, 0xa8, 0x2a, 0xcf, - 0x73, 0x71, 0x58, 0x9e, 0xdb, 0x3c, 0x53, 0xb1, 0xeb, 0xe8, 0xad, 0x1b, 0x7b, 0x2d, 0xf5, 0x86, - 0xa3, 0xb7, 0xb1, 0xed, 0x28, 0xed, 0x0e, 0x4f, 0x59, 0xa4, 0x29, 0x8e, 0xa5, 0xa8, 0xba, 0xd1, - 0xbc, 0x61, 0x61, 0xd5, 0xb4, 0x34, 0xac, 0xc9, 0x76, 0x47, 0x31, 0xdc, 0x22, 0x37, 0xcd, 0xa6, - 0x49, 0xff, 0xde, 0x20, 0xff, 0x18, 0xb5, 0xf4, 0x6b, 0x90, 0x91, 0x14, 0xa3, 0x89, 0x6b, 0xc6, - 0xae, 0x89, 0x3e, 0x03, 0x29, 0x0d, 0xdb, 0x6a, 0x51, 0x58, 0x14, 0xae, 0x65, 0x6f, 0x95, 0x96, - 0xfa, 0x74, 0xb1, 0x44, 0xf3, 0xae, 0x62, 0x5b, 0xb5, 0xf4, 0x8e, 0x63, 0x5a, 0x2b, 0xa9, 0xf7, - 0x8f, 0x16, 0xc6, 0x24, 0xca, 0x85, 0x3e, 0x09, 0xe3, 0x2d, 0xac, 0xd8, 0xb8, 0x98, 0xa0, 0xec, - 0xc5, 0x08, 0xf6, 0xfb, 0x24, 0x9d, 0x33, 0xb1, 0xcc, 0xa5, 0xbf, 0x17, 0x20, 0x2f, 0xe1, 0xc7, - 0x5d, 0x6c, 0x3b, 0x55, 0xac, 0x68, 0xd8, 0x42, 0xe7, 0x21, 0xf9, 0x08, 0x1f, 0x16, 0x93, 0x8b, - 0xc2, 0xb5, 0xdc, 0xca, 0xe4, 0x87, 0x47, 0x0b, 0xc9, 0x35, 0x7c, 0x28, 0x11, 0x1a, 0x5a, 0x84, - 0x49, 0x6c, 0x68, 0x32, 0x49, 0x4e, 0x85, 0x93, 0x27, 0xb0, 0xa1, 0xad, 0xe1, 0x43, 0xa4, 0x42, - 0xda, 0x26, 0xd2, 0x0c, 0x15, 0x17, 0xc7, 0x17, 0x85, 0x6b, 0xe3, 0x2b, 0x77, 0x3f, 0x3c, 0x5a, - 0x28, 0x37, 0x75, 0x67, 0xaf, 0xbb, 0xb3, 0xa4, 0x9a, 0xed, 0x1b, 0x5e, 0xa9, 0xb4, 0x1d, 0xff, - 0xff, 0x8d, 0xce, 0xa3, 0xe6, 0x8d, 0x01, 0x2d, 0xb0, 0xd4, 0x38, 0x30, 0xea, 0xf8, 0xb1, 0xe4, - 0x09, 0x7e, 0x3d, 0xf5, 0xd3, 0xf7, 0x16, 0x84, 0x7b, 0xa9, 0xb4, 0x20, 0x26, 0xee, 0xa5, 0xd2, - 0x09, 0x31, 0x59, 0xfa, 0x5a, 0x12, 0x0a, 0x12, 0xb6, 0x3b, 0xa6, 0x61, 0x63, 0x5e, 0x8d, 0x57, - 0x20, 0xe9, 0x1c, 0x18, 0xb4, 0x1a, 0xd9, 0x5b, 0xf3, 0x11, 0xca, 0x68, 0x58, 0x8a, 0x61, 0x2b, - 0xaa, 0xa3, 0x9b, 0x86, 0x44, 0xb2, 0xa2, 0xd7, 0x20, 0x6b, 0x61, 0xbb, 0xdb, 0xc6, 0xb4, 0xd9, - 0x68, 0x0d, 0xb3, 0xb7, 0xce, 0x45, 0x70, 0xd6, 0x3b, 0x8a, 0x21, 0x01, 0xcb, 0x4b, 0xfe, 0xa3, - 0xf3, 0x90, 0x36, 0xba, 0x6d, 0xa2, 0x17, 0x9b, 0xd6, 0x3a, 0x29, 0x4d, 0x1a, 0xdd, 0xf6, 0x1a, - 0x3e, 0xb4, 0x51, 0x19, 0xb2, 0x16, 0x69, 0x34, 0x59, 0x37, 0x76, 0x4d, 0xbb, 0x38, 0xb1, 0x98, - 0xbc, 0x96, 0xbd, 0xf5, 0xfc, 0xa0, 0xa6, 0x25, 0xdd, 0x80, 0xb7, 0x0f, 0x58, 0x2e, 0xc1, 0x46, - 0x75, 0xc8, 0xf3, 0x92, 0x59, 0x58, 0xb1, 0x4d, 0xa3, 0x38, 0xb9, 0x28, 0x5c, 0x2b, 0xdc, 0x5a, - 0x8a, 0x12, 0x13, 0xd2, 0x02, 0x79, 0xec, 0xb6, 0xb1, 0x44, 0xb9, 0xa4, 0x9c, 0x15, 0x78, 0x2a, - 0x3d, 0x84, 0x5c, 0x30, 0x15, 0x21, 0x28, 0x48, 0x95, 0xfa, 0xf6, 0x7a, 0x45, 0xde, 0xde, 0x58, - 0xdb, 0xd8, 0x7c, 0x7b, 0x43, 0x1c, 0x43, 0xb3, 0x20, 0x72, 0xda, 0x5a, 0xe5, 0xa1, 0x7c, 0xbf, - 0xb6, 0x5e, 0x6b, 0x88, 0x02, 0x3a, 0x0f, 0x67, 0x38, 0x55, 0x5a, 0xde, 0xb8, 0x5b, 0x91, 0x57, - 0x36, 0xb7, 0x37, 0x56, 0x97, 0xa5, 0x87, 0x62, 0x62, 0x2e, 0xf5, 0x9b, 0xdf, 0x9e, 0x1f, 0x2b, - 0x3d, 0x00, 0xb8, 0x8b, 0x1d, 0xde, 0xad, 0xd0, 0x0a, 0x4c, 0xec, 0xd1, 0xd2, 0xf0, 0x8e, 0xbd, - 0x18, 0x59, 0xec, 0x40, 0x17, 0x5c, 0x49, 0x13, 0x0d, 0x7c, 0x70, 0xb4, 0x20, 0x48, 0x9c, 0x93, - 0x35, 0x79, 0xe9, 0xfb, 0x02, 0x64, 0xa9, 0x60, 0x56, 0x47, 0x54, 0xee, 0x91, 0x7c, 0xe1, 0x58, - 0x85, 0xf4, 0x8b, 0x46, 0x4b, 0x30, 0xfe, 0x44, 0x69, 0x75, 0x87, 0x8d, 0x9b, 0x07, 0x24, 0x5d, - 0x62, 0xd9, 0xd0, 0x1b, 0x90, 0xd3, 0x0d, 0x07, 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xc7, 0xb0, 0x65, - 0x59, 0x6e, 0xfa, 0x50, 0xfa, 0x1b, 0x01, 0x60, 0xab, 0x1b, 0xa7, 0x6a, 0xc8, 0xb8, 0x1f, 0xa9, - 0xfc, 0xee, 0xb8, 0x67, 0xb5, 0x38, 0x0b, 0x13, 0xba, 0xd1, 0xd2, 0x0d, 0x56, 0xfe, 0xb4, 0xc4, - 0x9f, 0xd0, 0x2c, 0x8c, 0xef, 0xb4, 0x74, 0x43, 0xa3, 0xdd, 0x3f, 0x2d, 0xb1, 0x07, 0xae, 0x7e, - 0x09, 0xb2, 0xb4, 0xec, 0x31, 0x6a, 0xbf, 0xf4, 0x8d, 0x04, 0x9c, 0x29, 0x9b, 0x86, 0xa6, 0x93, - 0x71, 0xa8, 0xb4, 0x3e, 0x16, 0xba, 0x79, 0x15, 0x32, 0xf8, 0xa0, 0x33, 0x62, 0xf3, 0xa6, 0xf1, - 0x41, 0x87, 0xfe, 0x8b, 0x56, 0x1d, 0xfa, 0x24, 0x9c, 0x53, 0x5a, 0x2d, 0x73, 0x5f, 0xd6, 0x77, - 0x65, 0xcd, 0xc4, 0xb6, 0x6c, 0x98, 0x8e, 0x8c, 0x0f, 0x74, 0xdb, 0xa1, 0xa6, 0x22, 0x2d, 0xcd, - 0xd0, 0xe4, 0xda, 0xee, 0xaa, 0x89, 0xed, 0x0d, 0xd3, 0xa9, 0x90, 0x24, 0xae, 0xf0, 0x77, 0xe1, - 0x6c, 0xaf, 0x6e, 0xe2, 0xd4, 0xfd, 0x3f, 0x0a, 0x50, 0xa8, 0x19, 0xba, 0xf3, 0xb1, 0x50, 0xba, - 0xa7, 0xbd, 0x64, 0x50, 0x7b, 0xd7, 0x41, 0xdc, 0x55, 0xf4, 0xd6, 0xa6, 0xd1, 0x30, 0xdb, 0x3b, - 0xb6, 0x63, 0x1a, 0xd8, 0xe6, 0xea, 0xed, 0xa3, 0x73, 0x9d, 0x3d, 0x80, 0x29, 0xaf, 0x4e, 0x71, - 0x2a, 0xeb, 0x29, 0x88, 0x35, 0x43, 0xb5, 0x70, 0x1b, 0x1b, 0xb1, 0x6a, 0xeb, 0x79, 0xc8, 0xe8, - 0xae, 0x5c, 0xaa, 0xb1, 0xa4, 0xe4, 0x13, 0x78, 0x9d, 0xba, 0x30, 0x1d, 0x78, 0x77, 0x9c, 0xc6, - 0xef, 0x39, 0xc8, 0x18, 0x78, 0x5f, 0xf6, 0xdb, 0x2b, 0x29, 0xa5, 0x0d, 0xbc, 0xcf, 0x8c, 0xd5, - 0x43, 0xc8, 0xaf, 0xe2, 0x16, 0x76, 0x70, 0xfc, 0x96, 0x7c, 0x1b, 0x0a, 0xae, 0xe8, 0x38, 0x1b, - 0xe9, 0x0f, 0x04, 0x40, 0x5c, 0x2e, 0x99, 0x3d, 0xe3, 0x6c, 0xa7, 0x05, 0xe2, 0x1d, 0x38, 0x5d, - 0xcb, 0x60, 0xd3, 0x3c, 0xeb, 0xa5, 0xc0, 0x48, 0x74, 0xa6, 0xf7, 0x2d, 0x6a, 0x2a, 0x68, 0x51, - 0x3d, 0x6f, 0x85, 0xf8, 0x29, 0xfb, 0x30, 0x13, 0x2a, 0x5e, 0xbc, 0x4d, 0x99, 0xa2, 0x25, 0x4b, - 0x2c, 0x26, 0x83, 0x9e, 0x19, 0x25, 0x96, 0xde, 0x85, 0xe9, 0x72, 0x0b, 0x2b, 0x56, 0xdc, 0x6a, - 0xe1, 0xcd, 0xf9, 0x10, 0x50, 0x50, 0x7c, 0x9c, 0x4d, 0xfa, 0x1d, 0x01, 0x90, 0x84, 0x9f, 0x60, - 0xcb, 0x89, 0xbd, 0x49, 0x57, 0x21, 0xeb, 0x28, 0x56, 0x13, 0x3b, 0x32, 0x71, 0xe7, 0xb9, 0xb9, - 0x7a, 0x21, 0x20, 0x88, 0x38, 0xf5, 0x4b, 0x7b, 0x2d, 0x75, 0xa9, 0xe1, 0xba, 0xfb, 0xae, 0x73, - 0xc6, 0xf8, 0x08, 0x99, 0x6b, 0xe0, 0x1d, 0x98, 0x09, 0x95, 0x32, 0x4e, 0x15, 0xe8, 0x90, 0xad, - 0xab, 0x8a, 0xb1, 0xd9, 0x21, 0xf3, 0x80, 0x8d, 0x6e, 0xc3, 0x59, 0xdb, 0x31, 0x3b, 0xb2, 0xe2, - 0xc8, 0xcc, 0xb5, 0xdc, 0x31, 0xbb, 0x86, 0xa6, 0x58, 0x87, 0xf4, 0x1d, 0x69, 0x69, 0x86, 0xa4, - 0x2e, 0xb3, 0x82, 0xac, 0xf0, 0x24, 0xd2, 0x7d, 0xdb, 0xba, 0x21, 0x13, 0x0f, 0xb0, 0xe5, 0xd8, - 0x7c, 0xa8, 0x43, 0x5b, 0x37, 0x24, 0x46, 0xe1, 0xd5, 0xf8, 0xb6, 0xc0, 0xde, 0x15, 0xa7, 0x9a, - 0xdf, 0x84, 0xac, 0xad, 0x2a, 0x86, 0xbc, 0x6b, 0x5a, 0x6d, 0xc5, 0xa1, 0xa3, 0xa3, 0x10, 0x52, - 0xb3, 0xe7, 0x57, 0xab, 0x8a, 0x71, 0x87, 0x66, 0x92, 0xc0, 0xf6, 0xfe, 0x07, 0x07, 0xd0, 0xbd, - 0x54, 0x3a, 0x29, 0xa6, 0x4a, 0x3f, 0x17, 0x20, 0xc7, 0x4a, 0x19, 0xe7, 0x00, 0x7a, 0x15, 0x52, - 0x96, 0xb9, 0xcf, 0x06, 0x50, 0xf6, 0xd6, 0x73, 0x11, 0x22, 0xd6, 0xf0, 0x61, 0x70, 0xe6, 0xa2, - 0xd9, 0xd1, 0x0a, 0x70, 0x0f, 0x4f, 0xa6, 0xdc, 0xc9, 0x51, 0xb9, 0x81, 0x71, 0x49, 0x44, 0xc6, - 0x55, 0x98, 0xda, 0x51, 0x1c, 0x75, 0x8f, 0xb4, 0x0f, 0x2d, 0x24, 0x99, 0xe5, 0x92, 0xd7, 0x72, - 0x52, 0x81, 0x92, 0xdd, 0xa2, 0xdb, 0xa5, 0x3f, 0x71, 0x47, 0x83, 0x8d, 0x3f, 0xfe, 0xcd, 0xf4, - 0x5f, 0x02, 0x1f, 0x14, 0x6e, 0x61, 0xff, 0xbf, 0xb5, 0xd6, 0xb7, 0x12, 0x70, 0xae, 0xbc, 0x87, - 0xd5, 0x47, 0x65, 0xd3, 0xb0, 0x75, 0xdb, 0xc1, 0x86, 0x7a, 0x18, 0x67, 0x93, 0x3d, 0x07, 0x99, - 0x7d, 0xdd, 0xd9, 0x93, 0x35, 0x7d, 0x77, 0x97, 0x0e, 0xe9, 0xb4, 0x94, 0x26, 0x84, 0x55, 0x7d, - 0x77, 0x17, 0xdd, 0x86, 0x54, 0xdb, 0xd4, 0x98, 0x03, 0x5b, 0xb8, 0xb5, 0x10, 0x21, 0x9e, 0x16, - 0xcd, 0xee, 0xb6, 0xd7, 0x4d, 0x0d, 0x4b, 0x34, 0x33, 0x9a, 0x07, 0x50, 0x09, 0xb5, 0x63, 0xea, - 0x86, 0xc3, 0x27, 0xb2, 0x00, 0x05, 0x55, 0x21, 0xe3, 0x60, 0xab, 0xad, 0x1b, 0x8a, 0x43, 0x00, - 0x3e, 0x51, 0xde, 0xa5, 0xc8, 0x82, 0x77, 0x5a, 0xba, 0xaa, 0xf4, 0xad, 0x54, 0xf8, 0xcc, 0xdc, - 0xde, 0x7c, 0x3d, 0x05, 0xc5, 0x7e, 0x0d, 0xc5, 0xd9, 0x4f, 0xb6, 0x60, 0x82, 0x19, 0x3d, 0xde, - 0x53, 0x6e, 0x0d, 0x52, 0x44, 0x44, 0x09, 0x96, 0x98, 0x71, 0xe4, 0x85, 0xe7, 0x72, 0xe6, 0xfe, - 0x56, 0x80, 0x09, 0x96, 0x80, 0x6e, 0x42, 0x9a, 0xa3, 0x7b, 0x8d, 0x96, 0x31, 0xb9, 0x72, 0xf6, - 0xd9, 0xd1, 0xc2, 0x24, 0xc3, 0xf2, 0xab, 0x1f, 0xfa, 0x7f, 0xa5, 0x49, 0x06, 0xe7, 0x35, 0xd2, - 0x66, 0xb6, 0xa3, 0x58, 0x0e, 0x5d, 0x45, 0x21, 0x6d, 0x96, 0x93, 0xd2, 0x94, 0xb0, 0x86, 0x0f, - 0xd1, 0x3d, 0x98, 0xb0, 0x1d, 0xc5, 0xe9, 0xda, 0xbc, 0xd5, 0x4e, 0x54, 0xd8, 0x3a, 0xe5, 0x94, - 0xb8, 0x04, 0xe2, 0x8f, 0x68, 0xd8, 0x51, 0xf4, 0x16, 0x6d, 0xc6, 0x8c, 0xc4, 0x9f, 0x4a, 0xdf, - 0x14, 0x60, 0x82, 0x65, 0x45, 0xe7, 0x60, 0x86, 0x21, 0xf8, 0xda, 0xc6, 0x6a, 0xa5, 0x51, 0x91, - 0xd6, 0x6b, 0x1b, 0xcb, 0x8d, 0x8a, 0x38, 0x86, 0xce, 0x02, 0x72, 0x13, 0xca, 0x9b, 0x1b, 0xf5, - 0x5a, 0xbd, 0x51, 0xd9, 0x20, 0xc8, 0x7f, 0x16, 0x44, 0x46, 0x0f, 0x50, 0x13, 0xe8, 0x12, 0x2c, - 0xf6, 0x52, 0xe5, 0x7a, 0x63, 0xb9, 0x51, 0x97, 0x2b, 0xf5, 0x46, 0x6d, 0x7d, 0xb9, 0x51, 0x59, - 0x15, 0x93, 0x43, 0x72, 0x91, 0x97, 0x48, 0x52, 0xa5, 0xdc, 0x10, 0x53, 0xa5, 0xa7, 0x70, 0x46, - 0xc2, 0xaa, 0xd9, 0xee, 0x74, 0x1d, 0x4c, 0x4a, 0x69, 0xc7, 0x39, 0x5e, 0xce, 0xc1, 0xa4, 0x66, - 0x1d, 0xca, 0x56, 0xd7, 0xe0, 0xa3, 0x65, 0x42, 0xb3, 0x0e, 0xa5, 0xae, 0xc1, 0x3b, 0xe3, 0x5f, - 0x09, 0x70, 0xb6, 0xf7, 0xe5, 0x71, 0x76, 0xc5, 0x2f, 0x40, 0x56, 0xd1, 0x34, 0xac, 0xc9, 0x1a, - 0x6e, 0x39, 0x0a, 0xf7, 0x37, 0x6e, 0x06, 0x24, 0xf1, 0x15, 0xb0, 0x25, 0xb6, 0xf4, 0xb5, 0xe4, - 0xad, 0x80, 0xad, 0x3f, 0x28, 0x97, 0x69, 0x79, 0x56, 0x09, 0xa3, 0x6b, 0x91, 0xa8, 0x2c, 0x4a, - 0x29, 0x7d, 0x27, 0x05, 0x67, 0x2a, 0x86, 0x16, 0x5c, 0xd2, 0x8a, 0x51, 0x6d, 0x67, 0x61, 0x42, - 0x35, 0xdb, 0x6d, 0xdd, 0x71, 0xb5, 0xc6, 0x9e, 0xd0, 0xa7, 0x21, 0xad, 0x61, 0x45, 0xf3, 0x56, - 0x11, 0x8e, 0x73, 0x9e, 0x24, 0x2f, 0x3b, 0xfa, 0x12, 0x9c, 0x23, 0x06, 0xd5, 0x32, 0x94, 0x96, - 0xcc, 0xa4, 0xc9, 0x8e, 0xa5, 0x37, 0x9b, 0xd8, 0xe2, 0xeb, 0x6e, 0xd7, 0x22, 0xca, 0x59, 0xe3, - 0x1c, 0x65, 0xca, 0xd0, 0x60, 0xf9, 0xa5, 0x33, 0x7a, 0x14, 0x19, 0x7d, 0xde, 0x5b, 0xa6, 0xb1, - 0x3b, 0x8a, 0x61, 0x73, 0x63, 0x35, 0x68, 0x39, 0x8f, 0xeb, 0x94, 0xcf, 0x0d, 0x84, 0x62, 0xa3, - 0x1b, 0xc4, 0xe3, 0x7f, 0xdc, 0xd5, 0x2d, 0x2c, 0xdf, 0xec, 0xa8, 0xc5, 0x09, 0x52, 0xf7, 0x95, - 0xc2, 0xb3, 0xa3, 0x05, 0x90, 0x18, 0xf9, 0xe6, 0x56, 0x99, 0x20, 0x00, 0xf6, 0xbf, 0xa3, 0xa2, - 0x6b, 0x20, 0x1a, 0xa6, 0x6c, 0xe1, 0x5d, 0x0b, 0xdb, 0x7b, 0xfc, 0xb5, 0x69, 0xaa, 0xb1, 0x82, - 0x61, 0x4a, 0x8c, 0xcc, 0x44, 0x9f, 0x85, 0x89, 0x8e, 0xa9, 0xdb, 0xa6, 0x51, 0xcc, 0x30, 0x8d, - 0xb2, 0x27, 0xf4, 0x16, 0x88, 0xba, 0x21, 0xef, 0xb6, 0xf4, 0xe6, 0x9e, 0x23, 0xef, 0x5b, 0xba, - 0x83, 0xed, 0xe2, 0x34, 0x2d, 0x78, 0x54, 0x87, 0xab, 0xf3, 0x05, 0x51, 0xed, 0x6d, 0x92, 0x93, - 0x57, 0xa1, 0xa0, 0x1b, 0x77, 0x28, 0x3f, 0x25, 0xda, 0xde, 0xb4, 0x3c, 0x29, 0xa6, 0x4b, 0xff, - 0x21, 0xc0, 0xd9, 0xde, 0x6e, 0x12, 0x67, 0x07, 0xbf, 0x06, 0xa2, 0x69, 0x60, 0xb9, 0xb3, 0xa7, - 0xd8, 0x98, 0x37, 0x2b, 0x9f, 0x43, 0x0a, 0xa6, 0x81, 0xb7, 0x08, 0x99, 0x35, 0x12, 0xda, 0x82, - 0x69, 0xdb, 0x51, 0x9a, 0xba, 0xd1, 0x94, 0xbd, 0xa5, 0x74, 0xba, 0x1e, 0x32, 0xa2, 0x03, 0x2e, - 0x72, 0x6e, 0x8f, 0x1e, 0x72, 0x3c, 0xfe, 0x55, 0x80, 0xe9, 0x65, 0xad, 0xad, 0x1b, 0xf5, 0x4e, - 0x4b, 0x8f, 0x15, 0xad, 0x5f, 0x82, 0x8c, 0x4d, 0x64, 0xfa, 0xd6, 0xdb, 0x47, 0x5a, 0x69, 0x9a, - 0x42, 0xcc, 0xf8, 0x7d, 0x98, 0xc2, 0x07, 0x1d, 0xdd, 0x52, 0x88, 0x8a, 0x19, 0xb8, 0x48, 0x8d, - 0x5e, 0xb7, 0x82, 0xcf, 0xeb, 0x03, 0x0c, 0x5e, 0xb3, 0x87, 0x80, 0x82, 0x15, 0x8b, 0x13, 0x65, - 0xc8, 0x30, 0x43, 0x45, 0x6f, 0x1b, 0x76, 0xcc, 0x5a, 0xe3, 0xe6, 0xf5, 0x8b, 0x30, 0x1b, 0x7e, - 0x41, 0x9c, 0xa5, 0x7f, 0x97, 0xb7, 0xf8, 0x3a, 0xb6, 0x3e, 0x22, 0x80, 0x1b, 0x14, 0x1f, 0x67, - 0xc9, 0xbf, 0x2a, 0xc0, 0x79, 0x2a, 0x9b, 0x0e, 0xcb, 0x5d, 0x6c, 0xd1, 0x5d, 0x9a, 0x38, 0x3b, - 0xed, 0x45, 0x98, 0x60, 0x78, 0x95, 0xf6, 0xd8, 0xf1, 0x95, 0x2c, 0x71, 0x4c, 0xea, 0x8e, 0x69, - 0x11, 0xc7, 0x84, 0x27, 0xf1, 0x7a, 0x2a, 0x30, 0x17, 0x55, 0x96, 0x98, 0x01, 0xfd, 0x34, 0xf7, - 0x0f, 0x49, 0x17, 0x2f, 0xef, 0x11, 0xc7, 0x08, 0x55, 0x20, 0xab, 0xd2, 0x7f, 0xb2, 0x73, 0xd8, - 0xc1, 0x54, 0x7e, 0x61, 0x98, 0x6b, 0xc9, 0xd8, 0x1a, 0x87, 0x1d, 0x4c, 0xfc, 0x53, 0xf7, 0x3f, - 0x51, 0x57, 0xa0, 0xaa, 0x43, 0x9d, 0x53, 0x3a, 0xbe, 0x68, 0x5e, 0xd7, 0xbf, 0x0b, 0x69, 0xe2, - 0x7b, 0x49, 0xae, 0x0a, 0xf6, 0x26, 0xce, 0x14, 0xab, 0x3b, 0xf2, 0x0e, 0x9c, 0xd5, 0x70, 0xc7, - 0xc2, 0xaa, 0xe2, 0x60, 0x4d, 0x0e, 0x56, 0x3f, 0x71, 0x82, 0xea, 0xcf, 0xfa, 0x32, 0x7c, 0x2a, - 0x7a, 0x08, 0x28, 0x20, 0x9b, 0xd5, 0xcc, 0x85, 0x3b, 0x27, 0x51, 0xca, 0xb4, 0x2f, 0x85, 0xd1, - 0x6d, 0x54, 0x86, 0x34, 0x3e, 0xe8, 0xc8, 0x74, 0xab, 0x32, 0x75, 0xc2, 0xad, 0xca, 0x49, 0x7c, - 0xd0, 0x21, 0x44, 0xb4, 0x4d, 0x66, 0x3a, 0xd7, 0x01, 0xa0, 0xc5, 0xb6, 0x8f, 0xc7, 0x13, 0x7e, - 0x7f, 0xe1, 0xe2, 0xa6, 0xbc, 0xb9, 0x9f, 0x89, 0xe0, 0x6d, 0xf7, 0x9e, 0x00, 0xcf, 0x45, 0xb6, - 0x5d, 0x9c, 0x93, 0x9d, 0xbb, 0x5b, 0x9b, 0x38, 0xcd, 0x6e, 0x6d, 0xe9, 0x4f, 0xdd, 0x51, 0x2f, - 0xe1, 0x96, 0x49, 0xd4, 0xfb, 0x11, 0xac, 0x6e, 0x4d, 0xba, 0xcd, 0x9e, 0x38, 0x71, 0xb3, 0xbb, - 0xac, 0x3d, 0x66, 0xa1, 0xa7, 0xb0, 0x71, 0x9a, 0x85, 0xdf, 0x15, 0x60, 0xa6, 0x8a, 0x15, 0xcb, - 0xd9, 0xc1, 0x8a, 0xd3, 0x38, 0x88, 0xd5, 0x81, 0x7d, 0x15, 0x92, 0x86, 0xb9, 0x7f, 0x92, 0x05, - 0x3e, 0x92, 0xdf, 0x9f, 0xb6, 0xc2, 0xe5, 0x8a, 0xb3, 0xd6, 0xff, 0x90, 0x80, 0xcc, 0xdd, 0x72, - 0x9c, 0x75, 0xfd, 0x0c, 0x5f, 0x06, 0x66, 0x43, 0x3d, 0xaa, 0x5b, 0x7a, 0xef, 0x5b, 0xba, 0x5b, - 0x5e, 0xc3, 0x87, 0x6e, 0xb7, 0x24, 0x5c, 0x68, 0x19, 0x32, 0xce, 0x1e, 0xf1, 0x53, 0xcd, 0x96, - 0x76, 0x12, 0x9f, 0xc5, 0xe7, 0x9a, 0x7b, 0x04, 0xe3, 0x54, 0xae, 0x1b, 0x48, 0x20, 0x44, 0x04, - 0x12, 0x90, 0xd7, 0x78, 0x6e, 0x5f, 0xe2, 0x24, 0xaf, 0x71, 0x09, 0xac, 0x71, 0x3c, 0xdf, 0x68, - 0x5c, 0x9c, 0x28, 0xbd, 0x05, 0x40, 0xaa, 0x16, 0x67, 0xf3, 0x7c, 0x2f, 0x09, 0x85, 0xad, 0xae, - 0xbd, 0x17, 0x73, 0x7f, 0x2c, 0x03, 0x74, 0xba, 0xf6, 0x1e, 0xb6, 0x64, 0xe7, 0xc0, 0xe0, 0xf5, - 0x3f, 0x26, 0x44, 0xc1, 0x55, 0x00, 0xe3, 0x6b, 0x1c, 0x18, 0x68, 0x93, 0x0b, 0xc1, 0xb2, 0x1f, - 0xe7, 0x70, 0x7d, 0x04, 0x30, 0xd9, 0x38, 0x30, 0xd6, 0xb1, 0x87, 0x22, 0x99, 0x40, 0x4c, 0x04, - 0x7e, 0x06, 0x26, 0xc9, 0x83, 0xec, 0x98, 0x27, 0x69, 0xf9, 0x09, 0xc2, 0xd3, 0x30, 0xd1, 0x1b, - 0x90, 0x61, 0xdc, 0x64, 0xfe, 0x9a, 0xa0, 0xf3, 0x57, 0x54, 0x95, 0xb8, 0x36, 0xe9, 0xcc, 0x95, - 0xa6, 0xac, 0x64, 0xb6, 0x9a, 0x85, 0xf1, 0x5d, 0xd3, 0x52, 0x31, 0x0d, 0x6c, 0x48, 0x4b, 0xec, - 0x01, 0x5d, 0x87, 0x69, 0xdd, 0x50, 0x5b, 0x5d, 0x5b, 0x7f, 0x82, 0x65, 0xb7, 0x68, 0x0c, 0x30, - 0x4d, 0x79, 0x09, 0x54, 0xa0, 0x19, 0xec, 0x08, 0xf7, 0x52, 0xe9, 0xb4, 0x98, 0x29, 0x7d, 0x53, - 0x80, 0x29, 0xaf, 0xed, 0xe2, 0x34, 0xfc, 0xe5, 0x90, 0xe2, 0x4f, 0xde, 0x7a, 0x44, 0xd9, 0xa5, - 0x7f, 0xa2, 0x5e, 0x90, 0x6a, 0x3e, 0xa1, 0x8d, 0x19, 0x67, 0xe7, 0x5a, 0x61, 0x81, 0x2f, 0x89, - 0x53, 0x76, 0x08, 0x1a, 0x0a, 0x73, 0x13, 0x66, 0xf5, 0x36, 0x99, 0x19, 0x74, 0xa7, 0x75, 0xc8, - 0x91, 0x9c, 0x83, 0xdd, 0xbd, 0xd9, 0x19, 0x3f, 0xad, 0xec, 0x26, 0x71, 0x63, 0xc9, 0x76, 0x6b, - 0xfc, 0x6a, 0xc5, 0xa9, 0xf7, 0x1a, 0xe4, 0x2d, 0x26, 0x9a, 0x78, 0x34, 0x27, 0x54, 0x7d, 0xce, - 0x63, 0x25, 0xda, 0xff, 0x6e, 0x02, 0xa6, 0xde, 0xea, 0x62, 0xeb, 0xf0, 0x63, 0xa8, 0xfb, 0x2b, - 0x30, 0xb5, 0xaf, 0xe8, 0x8e, 0xbc, 0x6b, 0x5a, 0x72, 0xb7, 0xa3, 0x29, 0x8e, 0x1b, 0xa2, 0x91, - 0x27, 0xe4, 0x3b, 0xa6, 0xb5, 0x4d, 0x89, 0x08, 0x03, 0x7a, 0x64, 0x98, 0xfb, 0x86, 0x4c, 0xc8, - 0x14, 0x48, 0x1f, 0x18, 0x7c, 0x21, 0x7a, 0xe5, 0x53, 0xff, 0x76, 0xb4, 0x70, 0x7b, 0xa4, 0xa0, - 0x2b, 0x1a, 0xb7, 0xd6, 0xed, 0xea, 0xda, 0xd2, 0xf6, 0x76, 0x6d, 0x55, 0x12, 0xa9, 0xc8, 0xb7, - 0x99, 0xc4, 0xc6, 0x81, 0xe1, 0x3a, 0x00, 0x1f, 0x0a, 0x20, 0xfa, 0x0a, 0x8b, 0xb3, 0x55, 0x2b, - 0x90, 0x7d, 0xdc, 0xc5, 0x96, 0x7e, 0x8a, 0x36, 0x05, 0xce, 0x48, 0x8c, 0xd7, 0x3b, 0x90, 0x0b, - 0xe9, 0x21, 0xf9, 0x8b, 0xe9, 0x21, 0xbb, 0xef, 0xab, 0xa0, 0xf4, 0x43, 0x01, 0x10, 0xad, 0x7c, - 0x8d, 0xed, 0x01, 0x7c, 0xcc, 0x3a, 0xcc, 0x35, 0x10, 0x69, 0x18, 0xa4, 0xac, 0xef, 0xca, 0x6d, - 0xdd, 0xb6, 0x75, 0xa3, 0xc9, 0x7b, 0x4c, 0x81, 0xd2, 0x6b, 0xbb, 0xeb, 0x8c, 0xca, 0xdb, 0xf2, - 0x57, 0x61, 0x26, 0x54, 0x9b, 0x38, 0x5b, 0xf3, 0x02, 0xe4, 0x76, 0xcd, 0xae, 0xa1, 0xc9, 0x6c, - 0x21, 0x8d, 0x2f, 0x18, 0x66, 0x29, 0x8d, 0xbd, 0xaf, 0xf4, 0x95, 0x04, 0xcc, 0x4a, 0xd8, 0x36, - 0x5b, 0x4f, 0x70, 0xfc, 0xfa, 0xdc, 0x04, 0xbe, 0x51, 0x23, 0xff, 0x22, 0x6a, 0xcd, 0x30, 0x19, - 0x6c, 0x52, 0x0c, 0xaf, 0xc8, 0x5f, 0x1a, 0xde, 0x33, 0xfb, 0xd7, 0xe0, 0xf9, 0x3a, 0x5f, 0x2a, - 0xb8, 0xce, 0xc7, 0x1b, 0xe2, 0x97, 0xe1, 0x4c, 0x8f, 0x22, 0xe2, 0xf4, 0x5d, 0xfe, 0x25, 0x01, - 0xe7, 0xc3, 0xe2, 0xe3, 0x46, 0x18, 0xff, 0x37, 0x94, 0x8d, 0xaa, 0x90, 0x6f, 0xeb, 0xc6, 0xe9, - 0xd6, 0x19, 0x73, 0x6d, 0xdd, 0x68, 0x84, 0x7d, 0x4e, 0x02, 0x86, 0xa2, 0xf4, 0x1a, 0x67, 0xdb, - 0x7d, 0x5d, 0x80, 0x5c, 0xdc, 0x2b, 0x59, 0xa7, 0x8b, 0xcb, 0xe2, 0x75, 0x6e, 0x40, 0xfe, 0x23, - 0x58, 0xfa, 0xfa, 0x63, 0x01, 0x50, 0xc3, 0xea, 0x1a, 0x04, 0x52, 0xde, 0x37, 0x9b, 0x71, 0x56, - 0x76, 0x16, 0xc6, 0x75, 0x43, 0xc3, 0x07, 0xb4, 0xb2, 0x29, 0x89, 0x3d, 0x84, 0xf6, 0xeb, 0x92, - 0x23, 0xed, 0xd7, 0xf9, 0xe1, 0x1d, 0xa1, 0x82, 0xc6, 0xa9, 0x85, 0xef, 0x26, 0x60, 0x86, 0x57, - 0x27, 0xf6, 0xa5, 0xbf, 0x53, 0x05, 0x85, 0xa3, 0xcf, 0x02, 0x74, 0x2c, 0xfc, 0x44, 0x66, 0xac, - 0xc9, 0x91, 0x58, 0x33, 0x84, 0x83, 0x12, 0xd0, 0x17, 0x60, 0x8a, 0x0c, 0xb8, 0x8e, 0x65, 0x76, - 0x4c, 0x9b, 0xcc, 0xeb, 0xf6, 0x68, 0x80, 0x62, 0xfa, 0xd9, 0xd1, 0x42, 0x7e, 0x5d, 0x37, 0xb6, - 0x38, 0x63, 0xa3, 0x2e, 0x91, 0x91, 0xeb, 0x3d, 0xba, 0xce, 0xc8, 0x3f, 0x0b, 0x30, 0xfb, 0x91, - 0x2d, 0x96, 0xfe, 0x6f, 0x68, 0xcc, 0x9b, 0x0f, 0x44, 0xfa, 0x58, 0x33, 0x76, 0xcd, 0xf8, 0x97, - 0xb0, 0xbf, 0x2e, 0xc0, 0x74, 0x40, 0x7c, 0x9c, 0xb3, 0xfe, 0xe9, 0x8e, 0x1e, 0x7c, 0x91, 0xf8, - 0x01, 0xc1, 0x6e, 0x1f, 0xe7, 0xa0, 0xfa, 0xbb, 0x04, 0x9c, 0x2d, 0xb3, 0x9d, 0x5c, 0x37, 0xcc, - 0x21, 0xce, 0x5e, 0x52, 0x84, 0xc9, 0x27, 0xd8, 0xb2, 0x75, 0x93, 0xcd, 0x7b, 0x79, 0xc9, 0x7d, - 0x44, 0x73, 0x90, 0xb6, 0x0d, 0xa5, 0x63, 0xef, 0x99, 0xee, 0xde, 0x97, 0xf7, 0xec, 0x85, 0x64, - 0x8c, 0x9f, 0x3e, 0x24, 0x63, 0x62, 0x78, 0x48, 0xc6, 0xe4, 0x2f, 0x1c, 0x92, 0xc1, 0x37, 0x9a, - 0x7e, 0x20, 0xc0, 0xb9, 0x3e, 0xfd, 0xc5, 0xd9, 0x67, 0xbe, 0x0c, 0x59, 0x95, 0x0b, 0x26, 0xd6, - 0x98, 0xed, 0xa5, 0xd5, 0x48, 0xb6, 0x53, 0xfa, 0xec, 0xcf, 0x8e, 0x16, 0xc0, 0x2d, 0x6a, 0x6d, - 0x95, 0xab, 0x88, 0xfc, 0xd7, 0x4a, 0xbf, 0x95, 0x85, 0xa9, 0xca, 0x01, 0x5b, 0x29, 0xae, 0x33, - 0xa7, 0x04, 0xdd, 0x81, 0x74, 0xc7, 0x32, 0x9f, 0xe8, 0x6e, 0x35, 0x0a, 0x21, 0xd7, 0xc5, 0xad, - 0x46, 0x0f, 0xd7, 0x16, 0xe7, 0x90, 0x3c, 0x5e, 0xd4, 0x80, 0xcc, 0x7d, 0x53, 0x55, 0x5a, 0x77, - 0xf4, 0x96, 0xdb, 0xff, 0x5f, 0x39, 0x5e, 0xd0, 0x92, 0xc7, 0xb3, 0xa5, 0x38, 0x7b, 0x6e, 0x53, - 0x78, 0x44, 0x54, 0x83, 0x74, 0xd5, 0x71, 0x3a, 0x24, 0x91, 0x5b, 0x93, 0xab, 0x23, 0x08, 0x25, - 0x2c, 0x5c, 0x96, 0xc7, 0x8e, 0x1a, 0x30, 0x7d, 0xd7, 0x34, 0x9b, 0x2d, 0x5c, 0x6e, 0x99, 0x5d, - 0xad, 0x6c, 0x1a, 0xbb, 0x7a, 0x93, 0xdb, 0xe3, 0x2b, 0x23, 0xc8, 0xbc, 0x5b, 0xae, 0x4b, 0xfd, - 0x02, 0xd0, 0x32, 0xa4, 0xeb, 0xb7, 0xb9, 0x30, 0xe6, 0x4f, 0x5d, 0x1e, 0x41, 0x58, 0xfd, 0xb6, - 0xe4, 0xb1, 0xa1, 0x7b, 0x90, 0x5d, 0x7e, 0xda, 0xb5, 0x30, 0x97, 0x32, 0x31, 0x70, 0xdf, 0xbf, - 0x57, 0x0a, 0xe5, 0x92, 0x82, 0xcc, 0xa8, 0x0e, 0x85, 0xb7, 0x4d, 0xeb, 0x51, 0xcb, 0x54, 0xdc, - 0x1a, 0x4e, 0x52, 0x71, 0x9f, 0x18, 0x41, 0x9c, 0xcb, 0x28, 0xf5, 0x88, 0x98, 0xfb, 0x02, 0xe4, - 0x43, 0xcd, 0x84, 0x10, 0xa4, 0x3a, 0xa4, 0x45, 0x04, 0x1a, 0x50, 0x43, 0xff, 0xa3, 0x97, 0x61, - 0xd2, 0x30, 0x35, 0xec, 0xf6, 0xe1, 0xfc, 0xca, 0xec, 0xb3, 0xa3, 0x85, 0x89, 0x0d, 0x53, 0x63, - 0x0e, 0x05, 0xff, 0x27, 0x4d, 0x90, 0x4c, 0xae, 0x3b, 0x31, 0x77, 0x05, 0x52, 0xa4, 0x7d, 0x88, - 0x19, 0xd9, 0x51, 0x6c, 0xbc, 0x6d, 0xe9, 0x5c, 0xa6, 0xfb, 0xc8, 0xf3, 0xfd, 0x48, 0x80, 0x44, - 0xfd, 0x36, 0xf1, 0x6c, 0x77, 0xba, 0xea, 0x23, 0xec, 0xf0, 0x5c, 0xfc, 0x89, 0x7a, 0xbc, 0x16, - 0xde, 0xd5, 0x99, 0x97, 0x93, 0x91, 0xf8, 0x13, 0x7a, 0x01, 0x40, 0x51, 0x55, 0x6c, 0xdb, 0xb2, - 0x7b, 0x92, 0x2b, 0x23, 0x65, 0x18, 0x65, 0x0d, 0x1f, 0x12, 0x36, 0x1b, 0xab, 0x16, 0x76, 0xdc, - 0xc8, 0x20, 0xf6, 0x44, 0xd8, 0x1c, 0xdc, 0xee, 0xc8, 0x8e, 0xf9, 0x08, 0x1b, 0xb4, 0x55, 0x33, - 0xc4, 0x3c, 0xb4, 0x3b, 0x0d, 0x42, 0x20, 0x96, 0x0d, 0x1b, 0x9a, 0x6f, 0x86, 0x32, 0x92, 0xf7, - 0x4c, 0x44, 0x5a, 0xb8, 0xa9, 0xf3, 0xa3, 0x49, 0x19, 0x89, 0x3f, 0x11, 0x8d, 0x29, 0x5d, 0x67, - 0x8f, 0x86, 0x41, 0x64, 0x24, 0xfa, 0x9f, 0x57, 0xed, 0x1b, 0x02, 0x24, 0xef, 0x96, 0xeb, 0x27, - 0xae, 0x9b, 0x2b, 0x31, 0xe9, 0x4b, 0xa4, 0x01, 0x79, 0x7a, 0xab, 0x45, 0xc0, 0x7f, 0xc7, 0x32, - 0xbf, 0x8c, 0x55, 0xb7, 0x66, 0x05, 0x4e, 0xde, 0x62, 0x54, 0xb4, 0x08, 0x59, 0xd5, 0xc2, 0x1a, - 0x36, 0x1c, 0x5d, 0x69, 0xd9, 0xbc, 0x8a, 0x41, 0x12, 0x2f, 0xdc, 0x57, 0x04, 0x18, 0xa7, 0xdd, - 0x0b, 0x3d, 0x0f, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x9d, 0xc8, 0x48, 0x3e, 0x61, 0x60, - 0x21, 0x2f, 0x40, 0x4e, 0x51, 0x55, 0xb3, 0x6b, 0x38, 0xb2, 0xa1, 0xb4, 0x31, 0x2f, 0x6c, 0x96, - 0xd3, 0x36, 0x94, 0x36, 0x46, 0x0b, 0xe0, 0x3e, 0x7a, 0xe7, 0xe9, 0x32, 0x12, 0x70, 0xd2, 0x1a, - 0x3e, 0xe4, 0x25, 0xf9, 0x81, 0x00, 0x69, 0xb7, 0x5b, 0x92, 0xc2, 0x34, 0xb1, 0x81, 0x2d, 0xc5, - 0x31, 0xbd, 0xc2, 0x78, 0x84, 0xde, 0x39, 0x29, 0xe3, 0xcf, 0x49, 0xb3, 0x30, 0xee, 0x28, 0x3b, - 0x2d, 0xb7, 0x1c, 0xec, 0x81, 0x2e, 0xba, 0xb6, 0x94, 0x26, 0x5b, 0x33, 0xca, 0x48, 0xec, 0x81, - 0x54, 0x89, 0x47, 0x80, 0x32, 0xed, 0xf0, 0x27, 0x52, 0x5e, 0x16, 0xf4, 0xb8, 0x83, 0x9b, 0xba, - 0x41, 0x3b, 0x40, 0x52, 0x02, 0x4a, 0x5a, 0x21, 0x14, 0xf4, 0x1c, 0x64, 0x58, 0x06, 0x6c, 0x68, - 0xb4, 0x17, 0x24, 0xa5, 0x34, 0x25, 0x54, 0xdc, 0x03, 0x44, 0xdc, 0x11, 0xf9, 0x33, 0x01, 0xa6, - 0x59, 0xc4, 0x0a, 0x8b, 0x96, 0x8c, 0x6f, 0x56, 0x7e, 0x1d, 0x32, 0x9a, 0xe2, 0x28, 0xec, 0xfc, - 0x5e, 0x62, 0xe8, 0xf9, 0x3d, 0xd7, 0x4c, 0x92, 0xfc, 0xf4, 0x0c, 0x1f, 0x82, 0x14, 0xf9, 0xcf, - 0xce, 0x3d, 0x4a, 0xf4, 0xbf, 0xbf, 0xf7, 0x1f, 0x2c, 0x6e, 0x9c, 0x5e, 0xca, 0x07, 0xe3, 0x90, - 0xaf, 0x1c, 0x74, 0x4c, 0x2b, 0xe6, 0x45, 0xa5, 0x49, 0x0e, 0xbd, 0x87, 0x6c, 0x4e, 0xf6, 0x58, - 0x41, 0x77, 0xdf, 0x8f, 0x33, 0xa2, 0x15, 0x00, 0x16, 0xa6, 0x48, 0xa3, 0x57, 0x92, 0x27, 0xd8, - 0xa2, 0xa1, 0x6c, 0x84, 0x8a, 0x36, 0x20, 0xdb, 0x7e, 0xa2, 0xaa, 0xf2, 0xae, 0xde, 0x72, 0x78, - 0x60, 0x57, 0x74, 0x44, 0xf1, 0xfa, 0x83, 0x72, 0xf9, 0x0e, 0xcd, 0xc4, 0xe2, 0xab, 0xfc, 0x67, - 0x09, 0x88, 0x04, 0xf6, 0x1f, 0xbd, 0x04, 0xfc, 0xbc, 0x85, 0x6c, 0xbb, 0xa7, 0xa7, 0x56, 0xf2, - 0xcf, 0x8e, 0x16, 0x32, 0x12, 0xa5, 0xd6, 0xeb, 0x0d, 0x29, 0xc3, 0x32, 0xd4, 0x6d, 0x07, 0x5d, - 0x84, 0xbc, 0xd9, 0xd6, 0x1d, 0xd9, 0xf5, 0x03, 0xb8, 0xeb, 0x94, 0x23, 0x44, 0xd7, 0x4f, 0x40, - 0x0d, 0xb8, 0x8a, 0x0d, 0x32, 0x1a, 0x68, 0x3d, 0x59, 0x24, 0xbd, 0xac, 0x3b, 0x6c, 0x44, 0xc9, - 0x66, 0xc7, 0xd1, 0xdb, 0xfa, 0x53, 0xba, 0x3d, 0xca, 0xb7, 0x26, 0x2e, 0xb2, 0xec, 0xa4, 0x7e, - 0x34, 0xb6, 0xbe, 0xc6, 0xf3, 0x6e, 0x06, 0xb2, 0xa2, 0xaf, 0x08, 0x34, 0x44, 0x9f, 0x28, 0x52, - 0xde, 0x39, 0x94, 0x5b, 0x64, 0x12, 0xd1, 0x9d, 0x43, 0xf9, 0xd1, 0x93, 0x62, 0x9a, 0x3a, 0x68, - 0x9f, 0x8e, 0x6c, 0x90, 0x40, 0x3f, 0x58, 0x72, 0x9b, 0xe5, 0xf0, 0x3e, 0x67, 0x5e, 0x7b, 0x52, - 0x31, 0x1c, 0xeb, 0x70, 0xe5, 0xdc, 0xb3, 0xa3, 0x85, 0x99, 0xfe, 0xd4, 0x07, 0x34, 0xec, 0xbf, - 0x97, 0x65, 0xee, 0xcb, 0x50, 0x1c, 0x24, 0x09, 0x89, 0xfe, 0xfe, 0x5c, 0x86, 0x6d, 0xcb, 0xbd, - 0x16, 0x5e, 0x21, 0x18, 0xa1, 0xdb, 0xb8, 0xab, 0x04, 0x89, 0xd7, 0xdc, 0xd1, 0xfd, 0x35, 0x01, - 0xf2, 0x2b, 0xdd, 0xd6, 0xa3, 0xcd, 0x4e, 0xbd, 0xdb, 0x6e, 0x2b, 0xd6, 0x21, 0x31, 0x0c, 0x6c, - 0x54, 0xea, 0x4f, 0x59, 0x60, 0x47, 0x92, 0x0f, 0x3b, 0xfd, 0x29, 0x26, 0xc3, 0x8e, 0x87, 0x71, - 0x13, 0x3a, 0x8b, 0xd1, 0xbe, 0x08, 0x79, 0x0a, 0xd7, 0x65, 0x6c, 0x38, 0x96, 0x8e, 0xd9, 0x6a, - 0x50, 0x52, 0xca, 0x51, 0x62, 0x85, 0xd1, 0xd0, 0x65, 0x28, 0xd8, 0x87, 0xb6, 0x83, 0xdb, 0x32, - 0x3b, 0x6d, 0xcd, 0x30, 0x66, 0x52, 0xca, 0x33, 0xaa, 0xc4, 0x88, 0xa5, 0x1f, 0x27, 0xa1, 0xe0, - 0x6a, 0x36, 0x4e, 0xf7, 0x75, 0x05, 0xc6, 0x77, 0xf5, 0x16, 0x76, 0xf7, 0xd6, 0xaf, 0x0c, 0x69, - 0x50, 0x1e, 0x9e, 0x4b, 0xbc, 0x08, 0x17, 0x00, 0x51, 0xd6, 0x38, 0xc6, 0xd8, 0xdc, 0x6f, 0x24, - 0x20, 0x45, 0x3d, 0xc6, 0x9b, 0x90, 0xa2, 0x66, 0x4f, 0x18, 0xc5, 0xec, 0xd1, 0xac, 0x9e, 0x3b, - 0x93, 0x08, 0xb8, 0x33, 0xc4, 0x37, 0xd8, 0x53, 0x5e, 0xbd, 0x79, 0x8b, 0x8e, 0xaf, 0x9c, 0xc4, - 0x9f, 0xd0, 0x0a, 0x0d, 0xfa, 0x30, 0x2d, 0x07, 0x6b, 0xdc, 0x53, 0x8b, 0xb2, 0x4c, 0xa1, 0x86, - 0x77, 0x4d, 0xac, 0xcb, 0x87, 0xce, 0x43, 0x92, 0x0c, 0xdc, 0x49, 0xb6, 0x21, 0xfc, 0xec, 0x68, - 0x21, 0x49, 0x86, 0x2c, 0xa1, 0xa1, 0x1b, 0x90, 0x0d, 0x8f, 0x12, 0xe1, 0x5a, 0x86, 0xd9, 0x82, - 0x40, 0x0f, 0x87, 0x96, 0xd7, 0x83, 0x19, 0x4a, 0xb9, 0x97, 0x4a, 0xa7, 0xc4, 0xf1, 0xd2, 0x5f, - 0xa6, 0x20, 0x5f, 0x6b, 0xc7, 0x6d, 0x45, 0x97, 0xc3, 0x2d, 0x1c, 0xe5, 0xde, 0x86, 0x5e, 0x1a, - 0xd1, 0xc0, 0xa1, 0xf9, 0x28, 0x79, 0xb2, 0xf9, 0xa8, 0x46, 0x3c, 0x2a, 0x7e, 0xa2, 0x3c, 0x39, - 0xc0, 0x93, 0x0d, 0xbf, 0xbf, 0x41, 0x0c, 0x93, 0x44, 0x78, 0xfc, 0x80, 0x75, 0xba, 0xa9, 0xff, - 0x26, 0x75, 0xdc, 0x58, 0x2f, 0x9b, 0x18, 0xbd, 0x97, 0x4d, 0x62, 0x43, 0xa3, 0x7d, 0xec, 0x29, - 0xef, 0x62, 0xaf, 0x43, 0x52, 0xd3, 0xad, 0x21, 0xd7, 0x13, 0x44, 0xcf, 0x29, 0x84, 0xe9, 0x98, - 0xbe, 0x96, 0x0a, 0xf6, 0xb5, 0x20, 0x0e, 0x9d, 0xdb, 0x04, 0xf0, 0xeb, 0x85, 0x16, 0x61, 0xc2, - 0x6c, 0x69, 0x6e, 0xb4, 0x7d, 0x7e, 0x25, 0xf3, 0xec, 0x68, 0x61, 0x7c, 0xb3, 0xa5, 0xd5, 0x56, - 0xa5, 0x71, 0xb3, 0xa5, 0xd5, 0x34, 0x7a, 0x14, 0x1f, 0xef, 0xcb, 0x5e, 0x64, 0x4e, 0x4e, 0x9a, - 0x34, 0xf0, 0x3e, 0x41, 0xbd, 0x3d, 0x11, 0x03, 0xa4, 0xe3, 0x7c, 0x4b, 0x80, 0x82, 0xab, 0xc3, - 0x78, 0x8d, 0x43, 0x5a, 0x6f, 0xf3, 0xc1, 0x92, 0x3c, 0xd9, 0x60, 0x71, 0xf9, 0xf8, 0x81, 0xc1, - 0xaf, 0x0a, 0x3c, 0x2a, 0xb3, 0xae, 0x2a, 0x0e, 0x99, 0x0f, 0x63, 0xec, 0xe0, 0x2f, 0x82, 0x68, - 0x29, 0x86, 0x66, 0xb6, 0xf5, 0xa7, 0x98, 0x2d, 0x5c, 0xd9, 0x7c, 0xbf, 0x66, 0xca, 0xa3, 0xd3, - 0x95, 0x19, 0x77, 0xdd, 0xed, 0x67, 0x02, 0x8f, 0xe0, 0xf4, 0x0a, 0x13, 0xa7, 0xd2, 0xd6, 0x60, - 0xc2, 0x62, 0x71, 0x60, 0x6c, 0xc0, 0xbd, 0x1c, 0x21, 0x24, 0xea, 0xed, 0x2c, 0xcc, 0xca, 0xeb, - 0xf2, 0x54, 0xc4, 0xdc, 0xe7, 0x61, 0x9c, 0x92, 0x4f, 0x61, 0x16, 0xb9, 0xe6, 0xff, 0x28, 0x01, - 0xd3, 0xcb, 0x9a, 0x56, 0xaf, 0xf3, 0xee, 0x17, 0x9f, 0xde, 0x5d, 0x4f, 0x33, 0xe1, 0x7b, 0x9a, - 0xe8, 0x65, 0x40, 0x9a, 0x6e, 0xb3, 0x83, 0xe2, 0xf6, 0x9e, 0xa2, 0x99, 0xfb, 0xfe, 0x2e, 0xde, - 0xb4, 0x9b, 0x52, 0x77, 0x13, 0xd0, 0x3b, 0x40, 0xfd, 0x22, 0xd9, 0x76, 0x14, 0x6f, 0x71, 0xf5, - 0xa5, 0x93, 0x1c, 0x24, 0x60, 0x7e, 0x93, 0xf7, 0x28, 0x65, 0x88, 0x38, 0xfa, 0x17, 0x5d, 0x03, - 0x51, 0x27, 0x3a, 0x74, 0x64, 0xc5, 0x76, 0x63, 0xd0, 0xd9, 0x49, 0xf5, 0x02, 0xa3, 0x2f, 0xdb, - 0xc1, 0xd0, 0x72, 0x16, 0x1a, 0xeb, 0xeb, 0x29, 0x4e, 0xf7, 0xf8, 0xaf, 0x05, 0x28, 0xf0, 0xb8, - 0xf9, 0x98, 0xf7, 0x06, 0x68, 0xbd, 0x78, 0x6f, 0x67, 0x0f, 0xe8, 0x0e, 0xe4, 0xdc, 0xd0, 0xfd, - 0x5d, 0xcb, 0x6c, 0x9f, 0x64, 0x3e, 0xce, 0x72, 0xc6, 0x3b, 0x96, 0xd9, 0xf6, 0x8f, 0xa1, 0x7b, - 0x25, 0x8f, 0x75, 0x5d, 0x93, 0x9e, 0xa9, 0x63, 0x82, 0xe3, 0xde, 0xcf, 0xfb, 0x9f, 0xd0, 0x0b, - 0x5d, 0xf4, 0x0d, 0x16, 0x3f, 0x4e, 0xe5, 0xfc, 0x4c, 0x80, 0x42, 0xbd, 0xbb, 0xc3, 0x2e, 0x35, - 0x89, 0x4f, 0x2f, 0x15, 0xc8, 0xb4, 0xf0, 0xae, 0x23, 0x9f, 0x2a, 0xdc, 0x33, 0x4d, 0x58, 0x69, - 0xc8, 0xeb, 0x5d, 0x00, 0x8b, 0x1e, 0xec, 0xa0, 0x72, 0x92, 0x27, 0x94, 0x93, 0xa1, 0xbc, 0xfe, - 0x44, 0x56, 0xfa, 0x8b, 0x04, 0x4c, 0x79, 0x95, 0x8d, 0xd3, 0x10, 0xff, 0x4a, 0xc8, 0xb8, 0x24, - 0x4f, 0x61, 0x5c, 0xa6, 0x89, 0xcc, 0x81, 0x06, 0x66, 0x09, 0x66, 0xe8, 0x6c, 0x23, 0x2b, 0x9d, - 0x4e, 0x4b, 0xc7, 0x9a, 0xcc, 0x36, 0xea, 0x52, 0x74, 0xa3, 0x6e, 0x9a, 0x26, 0x2d, 0xb3, 0x94, - 0x1a, 0xdd, 0xb4, 0xbb, 0x03, 0xb9, 0x5d, 0x0b, 0xe3, 0xa7, 0x58, 0xa6, 0x6e, 0xef, 0x49, 0xb6, - 0x6f, 0xb3, 0x8c, 0xb1, 0x4e, 0xf8, 0xb8, 0x5d, 0x7f, 0x17, 0xa6, 0xa9, 0x82, 0xe3, 0x3e, 0x5c, - 0xc6, 0x5b, 0xe5, 0x3f, 0x05, 0x40, 0x41, 0xf9, 0x1f, 0x5d, 0xc3, 0x24, 0x62, 0x6f, 0x98, 0x97, - 0x00, 0xb1, 0x88, 0x1a, 0x5b, 0xee, 0x60, 0x4b, 0xb6, 0xb1, 0x6a, 0xf2, 0xfb, 0x38, 0x04, 0x49, - 0xe4, 0x29, 0x5b, 0xd8, 0xaa, 0x53, 0x7a, 0xe9, 0x0f, 0xcf, 0x43, 0x8e, 0xeb, 0x64, 0xdb, 0x20, - 0xa8, 0xf7, 0x26, 0x24, 0x9b, 0x7c, 0x19, 0x2f, 0x1b, 0x09, 0xf3, 0xfd, 0x3b, 0x81, 0xaa, 0x63, - 0x12, 0xc9, 0x4b, 0x58, 0x3a, 0x5d, 0x27, 0x22, 0x02, 0xd4, 0x0f, 0x17, 0x0c, 0xb2, 0x74, 0xba, - 0x0e, 0xaa, 0xc3, 0x94, 0xea, 0xdf, 0x89, 0x22, 0x13, 0xf6, 0xe4, 0xc0, 0x95, 0xe3, 0xc8, 0x9b, - 0x65, 0xaa, 0x63, 0x52, 0x41, 0x0d, 0x25, 0xa0, 0x72, 0xf0, 0x12, 0x0e, 0x36, 0x9d, 0x5e, 0x8c, - 0x3c, 0x80, 0x16, 0xbe, 0x00, 0xa4, 0x3a, 0x16, 0xb8, 0xab, 0x03, 0xbd, 0x0e, 0x13, 0x1a, 0xbd, - 0xdc, 0x81, 0xf7, 0xd0, 0xa8, 0x4e, 0x14, 0xba, 0x4f, 0xa3, 0x3a, 0x26, 0x71, 0x0e, 0x74, 0x0f, - 0x72, 0xec, 0x1f, 0x3b, 0xd2, 0xcf, 0xdd, 0xf4, 0xcb, 0x83, 0x25, 0x04, 0x6c, 0x7f, 0x75, 0x4c, - 0xca, 0x6a, 0x3e, 0x15, 0x7d, 0x12, 0x52, 0xb6, 0xaa, 0x18, 0x7c, 0x05, 0x7c, 0x7e, 0xc0, 0x09, - 0x6e, 0x9f, 0x99, 0xe6, 0x26, 0x7a, 0xa5, 0x20, 0xc1, 0x0f, 0xaf, 0xa0, 0xa1, 0x96, 0x03, 0x56, - 0xe4, 0xa3, 0xce, 0x1a, 0x12, 0xbd, 0xe2, 0x50, 0x02, 0xba, 0x0b, 0x59, 0x85, 0xb8, 0x6c, 0x32, - 0x3d, 0xf1, 0x53, 0x84, 0x81, 0x67, 0x32, 0xfa, 0x4e, 0x6b, 0x55, 0xe9, 0x01, 0x47, 0x97, 0xe8, - 0x0b, 0x6a, 0x63, 0xab, 0x89, 0x8b, 0xd9, 0xe1, 0x82, 0x82, 0xa1, 0x13, 0x9e, 0x20, 0x4a, 0x44, - 0xeb, 0x90, 0xdf, 0x73, 0xa3, 0xb9, 0x69, 0xd8, 0x4a, 0x6e, 0xe0, 0x4e, 0x48, 0x44, 0x34, 0x7a, - 0x75, 0x4c, 0xca, 0xed, 0x05, 0xc8, 0x68, 0x09, 0x12, 0x4d, 0xb5, 0x98, 0xa7, 0x32, 0x9e, 0x1f, - 0x16, 0x6b, 0x5d, 0x1d, 0x93, 0x12, 0x4d, 0x95, 0x40, 0x31, 0x16, 0xc8, 0x7a, 0x60, 0x14, 0x0b, - 0x03, 0x2d, 0x41, 0x38, 0xe4, 0xb8, 0x3a, 0x26, 0xd1, 0xc0, 0x5c, 0xf2, 0xbe, 0x2d, 0x28, 0x58, - 0x2c, 0xf6, 0xc4, 0x8d, 0xb0, 0x12, 0x07, 0xee, 0x0e, 0x45, 0x05, 0x59, 0x55, 0xc7, 0xa4, 0xbc, - 0x15, 0xa4, 0xa3, 0x2f, 0xc1, 0x6c, 0x58, 0x22, 0xef, 0x81, 0xd3, 0x7d, 0xe6, 0x25, 0x5a, 0x6e, - 0xb8, 0x23, 0x22, 0xab, 0x2f, 0x11, 0x7d, 0x0a, 0xc6, 0x59, 0xab, 0x21, 0x2a, 0x32, 0x6a, 0xdb, - 0xb3, 0xa7, 0xc1, 0x58, 0x7e, 0x32, 0x28, 0x1c, 0x1e, 0x74, 0x21, 0xb7, 0xcc, 0x66, 0x71, 0x66, - 0xe0, 0xa0, 0xe8, 0x0f, 0x22, 0x21, 0x83, 0xc2, 0xf1, 0xa9, 0xa4, 0xdd, 0x2d, 0x96, 0xc2, 0xf7, - 0xe8, 0x67, 0x07, 0xb6, 0x7b, 0x44, 0x2c, 0x46, 0x95, 0x06, 0x90, 0xfa, 0x64, 0x52, 0x34, 0x8b, - 0xdd, 0x6c, 0x20, 0xd3, 0xb1, 0x76, 0x66, 0x60, 0xd1, 0xfa, 0x6f, 0x6b, 0xa8, 0x52, 0xc7, 0xc8, - 0xa3, 0xa2, 0x07, 0x20, 0xf2, 0x33, 0xc7, 0xfe, 0x5a, 0xe5, 0x59, 0x2a, 0xef, 0xc5, 0x48, 0x93, - 0x16, 0xb5, 0xa9, 0x5d, 0x1d, 0x93, 0xa6, 0xd4, 0x70, 0x0a, 0x7a, 0x08, 0xd3, 0x54, 0x9e, 0xac, - 0xfa, 0x87, 0xc5, 0x8b, 0xc5, 0xbe, 0x28, 0xad, 0xc1, 0xe7, 0xca, 0x5d, 0xc9, 0xa2, 0xda, 0x93, - 0x44, 0xba, 0xb1, 0x6e, 0xe8, 0x0e, 0xb5, 0xbe, 0x73, 0x03, 0xbb, 0x71, 0xf8, 0x6e, 0x29, 0xd2, - 0x8d, 0x75, 0x46, 0x21, 0xdd, 0xd8, 0xe1, 0x01, 0x1c, 0xbc, 0x39, 0x9e, 0x1f, 0xd8, 0x8d, 0xa3, - 0x22, 0x3d, 0x48, 0x37, 0x76, 0x82, 0x74, 0xd2, 0x8d, 0x99, 0x81, 0xe8, 0x91, 0xfb, 0xc2, 0xc0, - 0x6e, 0x3c, 0xf0, 0xcc, 0x1d, 0xe9, 0xc6, 0x4a, 0x5f, 0x22, 0x5a, 0x05, 0x60, 0x6e, 0x8b, 0x6e, - 0xec, 0x9a, 0xc5, 0xf9, 0x81, 0x93, 0x44, 0x6f, 0x08, 0x07, 0x99, 0x24, 0x5a, 0x2e, 0x8d, 0x18, - 0x32, 0xea, 0x56, 0xcb, 0x74, 0x6b, 0xa4, 0xb8, 0x30, 0xd0, 0x90, 0xf5, 0xed, 0x90, 0x10, 0x43, - 0xb6, 0xef, 0x11, 0xc9, 0x6c, 0xc3, 0x16, 0xd6, 0x8a, 0x8b, 0x03, 0x67, 0x9b, 0xd0, 0x92, 0x32, - 0x99, 0x6d, 0x18, 0x07, 0x5a, 0x86, 0x0c, 0x99, 0xce, 0x0f, 0xa9, 0x19, 0xba, 0x30, 0xd0, 0x11, - 0xed, 0x89, 0x90, 0xae, 0x8e, 0x49, 0xe9, 0xc7, 0x9c, 0x44, 0x5e, 0xcf, 0x96, 0x2a, 0x8a, 0xa5, - 0x81, 0xaf, 0x0f, 0x2d, 0x4f, 0x91, 0xd7, 0x33, 0x0e, 0xa4, 0xc2, 0x19, 0xd6, 0x56, 0xfc, 0xc8, - 0x9b, 0xc5, 0xcf, 0x67, 0x15, 0x2f, 0x52, 0x51, 0x03, 0x81, 0x7f, 0xe4, 0x49, 0xbc, 0xea, 0x98, - 0x34, 0xa3, 0xf4, 0xa7, 0x92, 0x01, 0xcf, 0xa7, 0x1e, 0xb6, 0x5c, 0x50, 0xbc, 0x34, 0x70, 0xc0, - 0x47, 0x2c, 0xb0, 0x90, 0x01, 0xaf, 0x04, 0xc8, 0x6c, 0x02, 0xd2, 0x64, 0xdb, 0x66, 0x1b, 0x69, - 0x97, 0x87, 0x4c, 0x40, 0x3d, 0x6b, 0x06, 0x6c, 0x02, 0xd2, 0xea, 0x8c, 0x93, 0x08, 0x52, 0x5b, - 0x58, 0xb1, 0xb8, 0x99, 0xbd, 0x32, 0x50, 0x50, 0xdf, 0x7d, 0x4d, 0x44, 0x90, 0xea, 0x11, 0xc9, - 0x84, 0x6d, 0xb9, 0x97, 0x15, 0x70, 0x97, 0xf0, 0xea, 0xc0, 0x09, 0x3b, 0xf2, 0x4e, 0x05, 0x32, - 0x61, 0x5b, 0xa1, 0x04, 0xf4, 0x59, 0x98, 0xe4, 0x98, 0xad, 0x78, 0x6d, 0x88, 0xa3, 0x1a, 0x84, - 0xe4, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, 0x56, 0x64, 0xd5, 0x7b, 0x71, 0x88, 0x95, 0xed, - 0x03, 0xb1, 0xcc, 0xca, 0xfa, 0x64, 0x62, 0x65, 0x59, 0x3f, 0xe5, 0x73, 0xdd, 0xf5, 0x81, 0x56, - 0xb6, 0x3f, 0x3c, 0x9b, 0x58, 0xd9, 0xc7, 0x3e, 0x95, 0xd4, 0xcc, 0x66, 0x68, 0xa9, 0xf8, 0x89, - 0x81, 0x35, 0x0b, 0x83, 0x47, 0x52, 0x33, 0xce, 0x43, 0x9a, 0x8d, 0x45, 0x0e, 0x32, 0x4d, 0xbf, - 0x34, 0xf8, 0x74, 0x69, 0x2f, 0xb8, 0xa8, 0xba, 0x77, 0x79, 0x32, 0x0d, 0x7b, 0x86, 0xca, 0xe2, - 0x67, 0xe9, 0xb8, 0xa6, 0x5e, 0x1e, 0x6e, 0xa8, 0xa2, 0x8e, 0x09, 0x7a, 0x86, 0x2a, 0x94, 0x48, - 0x8b, 0xca, 0x0e, 0x3b, 0xd0, 0xf1, 0xbd, 0x34, 0xe4, 0x20, 0x6c, 0xcf, 0xf9, 0x13, 0x5a, 0x54, - 0x8f, 0xe8, 0x0f, 0xa1, 0x2e, 0x3b, 0xb1, 0x5d, 0xbc, 0x31, 0x7c, 0x08, 0x85, 0x4f, 0x8e, 0x7b, - 0x43, 0x88, 0x93, 0xbd, 0x39, 0xd3, 0xf5, 0x30, 0x5e, 0x19, 0x3e, 0x67, 0xf6, 0xba, 0x16, 0x6c, - 0xce, 0xe4, 0xd7, 0x5a, 0x4d, 0xf2, 0x9d, 0x2a, 0x76, 0xd6, 0xe7, 0x5e, 0x2a, 0x3d, 0x25, 0x8a, - 0xf7, 0x52, 0xe9, 0x73, 0x62, 0xf1, 0x5e, 0x2a, 0x7d, 0x5e, 0x9c, 0xbb, 0x97, 0x4a, 0x3f, 0x27, - 0x3e, 0x5f, 0xfa, 0xf5, 0xf3, 0x90, 0x77, 0x61, 0x15, 0x03, 0x29, 0xb7, 0x82, 0x20, 0x65, 0x7e, - 0x10, 0x48, 0xe1, 0x40, 0x8c, 0xa3, 0x94, 0x5b, 0x41, 0x94, 0x32, 0x3f, 0x08, 0xa5, 0xf8, 0x3c, - 0x04, 0xa6, 0x34, 0x06, 0xc1, 0x94, 0x17, 0x47, 0x80, 0x29, 0x9e, 0xa8, 0x5e, 0x9c, 0xb2, 0xda, - 0x8f, 0x53, 0x2e, 0x0d, 0xc7, 0x29, 0x9e, 0xa8, 0x00, 0x50, 0x79, 0xa3, 0x07, 0xa8, 0x5c, 0x18, - 0x02, 0x54, 0x3c, 0x7e, 0x17, 0xa9, 0xac, 0x45, 0x22, 0x95, 0x2b, 0xc7, 0x21, 0x15, 0x4f, 0x4e, - 0x08, 0xaa, 0xbc, 0x1a, 0x82, 0x2a, 0x0b, 0x03, 0xa1, 0x8a, 0xc7, 0xcd, 0xb0, 0x4a, 0x63, 0x10, - 0x56, 0x79, 0x71, 0x04, 0xac, 0xe2, 0x2b, 0xb7, 0x07, 0xac, 0x54, 0xa3, 0xc0, 0xca, 0xe5, 0x63, - 0xc0, 0x8a, 0x27, 0x2d, 0x88, 0x56, 0xaa, 0x51, 0x68, 0xe5, 0xf2, 0x31, 0x68, 0xa5, 0x47, 0x12, - 0x83, 0x2b, 0x1b, 0xd1, 0x70, 0xe5, 0xea, 0xb1, 0x70, 0xc5, 0x93, 0x16, 0xc6, 0x2b, 0x37, 0x02, - 0x78, 0xe5, 0x85, 0x01, 0x78, 0xc5, 0x63, 0x25, 0x80, 0xe5, 0x73, 0x7d, 0x80, 0xa5, 0x34, 0x0c, - 0xb0, 0x78, 0xbc, 0x1e, 0x62, 0x79, 0x6b, 0x00, 0x62, 0xb9, 0x76, 0x3c, 0x62, 0xf1, 0x84, 0xf5, - 0x40, 0x16, 0x65, 0x28, 0x64, 0x79, 0x79, 0x44, 0xc8, 0xe2, 0x49, 0x8f, 0xc2, 0x2c, 0xaf, 0x85, - 0x31, 0xcb, 0xe2, 0x60, 0xcc, 0xe2, 0x89, 0xe1, 0xa0, 0x65, 0x2d, 0x12, 0xb4, 0x5c, 0x39, 0x0e, - 0xb4, 0xf8, 0xe3, 0x23, 0x88, 0x5a, 0x36, 0xa2, 0x51, 0xcb, 0xd5, 0x63, 0x51, 0x8b, 0xdf, 0xfc, - 0x21, 0xd8, 0xb2, 0x16, 0x09, 0x5b, 0xae, 0x1c, 0x07, 0x5b, 0xfc, 0xc2, 0x05, 0x71, 0xcb, 0xdb, - 0x03, 0x71, 0xcb, 0xf5, 0x51, 0x70, 0x8b, 0x27, 0xb4, 0x0f, 0xb8, 0xbc, 0x33, 0x18, 0xb8, 0x7c, - 0xe2, 0x04, 0x17, 0x62, 0x45, 0x22, 0x97, 0xcf, 0xf5, 0x21, 0x97, 0xd2, 0x30, 0xe4, 0xe2, 0xf7, - 0x67, 0x17, 0xba, 0x28, 0x43, 0x81, 0xc6, 0xcb, 0x23, 0x02, 0x0d, 0xbf, 0xf3, 0x45, 0x20, 0x8d, - 0x4a, 0x04, 0xd2, 0xb8, 0x34, 0x1c, 0x69, 0xf8, 0x66, 0xde, 0x87, 0x1a, 0xd5, 0x28, 0xa8, 0x71, - 0xf9, 0x18, 0xa8, 0xe1, 0x5b, 0xa1, 0x00, 0xd6, 0x78, 0xa3, 0x07, 0x6b, 0x5c, 0x38, 0x36, 0xda, - 0x21, 0x00, 0x36, 0x56, 0xfa, 0xc1, 0xc6, 0xc5, 0xa1, 0x60, 0xc3, 0x93, 0xe0, 0xa3, 0x8d, 0x37, - 0x7a, 0xd0, 0xc6, 0x85, 0x21, 0x68, 0xc3, 0x2f, 0x00, 0x87, 0x1b, 0xda, 0x70, 0xb8, 0xb1, 0x34, - 0x2a, 0xdc, 0xf0, 0x04, 0x47, 0xe2, 0x8d, 0x8d, 0x68, 0xbc, 0x71, 0x75, 0xc4, 0x5d, 0xcc, 0x3e, - 0xc0, 0x51, 0x8d, 0x02, 0x1c, 0x97, 0x8f, 0x01, 0x1c, 0xc1, 0x39, 0xc4, 0x43, 0x1c, 0xd5, 0x28, - 0xc4, 0x71, 0xf9, 0x18, 0xc4, 0xe1, 0x4b, 0x0a, 0x40, 0x8e, 0xc6, 0x20, 0xc8, 0xf1, 0xe2, 0x08, - 0x90, 0xc3, 0x9f, 0x77, 0x7b, 0x30, 0xc7, 0x9b, 0xbd, 0x98, 0xa3, 0x34, 0x0c, 0x73, 0xf8, 0x23, - 0xd2, 0x05, 0x1d, 0x1b, 0xd1, 0xa0, 0xe3, 0xea, 0xb1, 0xa0, 0x23, 0x68, 0x24, 0x03, 0xa8, 0x63, - 0x2d, 0x12, 0x75, 0x5c, 0x39, 0x0e, 0x75, 0xf8, 0x46, 0x32, 0x08, 0x3b, 0xde, 0xec, 0x85, 0x1d, - 0xa5, 0x61, 0xb0, 0xc3, 0xaf, 0x9c, 0x8b, 0x3b, 0xaa, 0x51, 0xb8, 0xe3, 0xf2, 0x31, 0xb8, 0xc3, - 0x6f, 0xbc, 0x00, 0xf0, 0x50, 0x86, 0x02, 0x8f, 0x97, 0x47, 0x04, 0x1e, 0x3d, 0x86, 0x2b, 0x8c, - 0x3c, 0xaa, 0x51, 0xc8, 0xe3, 0xf2, 0x31, 0xc8, 0x23, 0x50, 0x58, 0x1f, 0x7a, 0x6c, 0x44, 0x43, - 0x8f, 0xab, 0xc7, 0x42, 0x8f, 0x9e, 0xd1, 0xe4, 0x62, 0x8f, 0xb5, 0x48, 0xec, 0x71, 0xe5, 0x38, - 0xec, 0xd1, 0x33, 0xf1, 0x9d, 0x08, 0x7c, 0xdc, 0x4b, 0xa5, 0x9f, 0x17, 0x5f, 0x28, 0xfd, 0x7c, - 0x1c, 0x26, 0xaa, 0x6e, 0x44, 0x51, 0xe0, 0xd6, 0x0b, 0xe1, 0x34, 0xb7, 0x5e, 0xa0, 0x55, 0x32, - 0x56, 0xa8, 0xc5, 0x39, 0xfe, 0x82, 0xa3, 0xfe, 0xcb, 0x77, 0x38, 0xeb, 0x29, 0x8e, 0xc1, 0xa1, - 0x57, 0x21, 0xdf, 0xb5, 0xb1, 0x25, 0x77, 0x2c, 0xdd, 0xb4, 0x74, 0x87, 0x05, 0x2c, 0x0b, 0x2b, - 0xe2, 0x87, 0x47, 0x0b, 0xb9, 0x6d, 0x1b, 0x5b, 0x5b, 0x9c, 0x2e, 0xe5, 0xba, 0x81, 0x27, 0xf7, - 0x2b, 0x1c, 0xe3, 0xa3, 0x7f, 0x85, 0xe3, 0x2d, 0x10, 0x2d, 0xac, 0x68, 0xa1, 0xb9, 0x9f, 0x5d, - 0x27, 0x11, 0xdd, 0x5a, 0x34, 0x5a, 0xdf, 0xcd, 0x49, 0xaf, 0x95, 0x98, 0xb2, 0xc2, 0x44, 0x74, - 0x13, 0xce, 0xb4, 0x95, 0x03, 0x1a, 0x85, 0x25, 0xbb, 0xee, 0x14, 0x8d, 0xac, 0x4a, 0xd3, 0x88, - 0x41, 0xd4, 0x56, 0x0e, 0xe8, 0x27, 0x3d, 0x58, 0x12, 0xbd, 0xcc, 0xfb, 0x32, 0x14, 0x34, 0xdd, - 0x76, 0x74, 0x43, 0x75, 0xef, 0x0f, 0x64, 0xf7, 0x4e, 0xe4, 0x5d, 0x2a, 0xbb, 0xc7, 0xef, 0x3a, - 0x4c, 0xf3, 0x88, 0x54, 0xff, 0x23, 0x1f, 0x14, 0x38, 0xa4, 0x49, 0x29, 0x48, 0x82, 0xff, 0x75, - 0x97, 0x32, 0x4c, 0x35, 0x15, 0x07, 0xef, 0x2b, 0x87, 0xb2, 0x7b, 0x60, 0x20, 0x4b, 0xaf, 0xe3, - 0x7a, 0xee, 0xd9, 0xd1, 0x42, 0xfe, 0x2e, 0x4b, 0xea, 0x3b, 0x37, 0x90, 0x6f, 0x06, 0x12, 0x34, - 0xb4, 0x0c, 0x39, 0x7a, 0x49, 0xaf, 0xc9, 0xee, 0x82, 0xe6, 0x70, 0x60, 0xd0, 0x1e, 0x0f, 0xbf, - 0x31, 0x5a, 0xa2, 0x17, 0xfb, 0xba, 0xd7, 0x47, 0x5f, 0x85, 0x29, 0xc5, 0x3e, 0x34, 0x54, 0xaa, - 0x61, 0x6c, 0xd8, 0x5d, 0x9b, 0xe2, 0x81, 0xb4, 0x54, 0xa0, 0xe4, 0xb2, 0x4b, 0x45, 0xaf, 0xc1, - 0x79, 0x0d, 0x13, 0x07, 0x87, 0x79, 0x11, 0x8e, 0x69, 0xca, 0x66, 0x4b, 0x93, 0xe9, 0x99, 0x72, - 0x8a, 0x05, 0xd2, 0xd2, 0x19, 0x9a, 0x81, 0xfa, 0x0f, 0x0d, 0xd3, 0xdc, 0x6c, 0x69, 0x15, 0x92, - 0xc8, 0x6f, 0x1b, 0xfc, 0x1d, 0x01, 0x72, 0xa1, 0x10, 0xef, 0x37, 0x7a, 0x36, 0x41, 0xcf, 0x47, - 0xa3, 0x98, 0x41, 0xe1, 0x78, 0x69, 0xde, 0x76, 0x6e, 0x80, 0xd0, 0xc2, 0x60, 0x2f, 0x98, 0x62, - 0x7d, 0x77, 0x03, 0xde, 0x65, 0x7b, 0x3d, 0xf5, 0x7b, 0xef, 0x2d, 0x8c, 0x95, 0x7e, 0x9a, 0x84, - 0x7c, 0x38, 0x94, 0xbb, 0xd6, 0x53, 0xae, 0x28, 0x2b, 0x13, 0xe2, 0x58, 0x1a, 0x72, 0xe9, 0x52, - 0xc6, 0xbf, 0x14, 0x98, 0x15, 0x73, 0x71, 0xc8, 0x56, 0x6f, 0xb0, 0x9c, 0x3e, 0xe3, 0xdc, 0xf7, - 0x13, 0x9e, 0xcd, 0x58, 0x82, 0x71, 0xa6, 0x70, 0x61, 0xe0, 0xd1, 0x3a, 0xaa, 0x73, 0x89, 0x65, - 0x23, 0x36, 0xa6, 0x71, 0xaa, 0x9b, 0x75, 0x3c, 0xc2, 0x29, 0xbe, 0x9c, 0xc3, 0xef, 0x57, 0x1a, - 0x3f, 0xd9, 0xfd, 0x4a, 0x6c, 0x2b, 0xb7, 0xd5, 0xc2, 0xaa, 0xc3, 0x3f, 0x95, 0xe4, 0x7e, 0x1f, - 0xe7, 0x52, 0xaf, 0x08, 0xfe, 0x61, 0xa5, 0x25, 0x89, 0x7f, 0x58, 0x29, 0x10, 0xb3, 0x55, 0xf0, - 0x44, 0xd0, 0x21, 0xc9, 0x22, 0xfb, 0x78, 0x53, 0x7f, 0x4b, 0x00, 0x91, 0x0e, 0xc0, 0x3b, 0x18, - 0x6b, 0xb1, 0xf4, 0x42, 0x37, 0x9c, 0x2c, 0x31, 0x7a, 0x94, 0x6d, 0xe8, 0x92, 0xe6, 0x64, 0xf8, - 0x92, 0xe6, 0xd2, 0x7b, 0x02, 0x14, 0xbc, 0x12, 0xb2, 0xcf, 0x88, 0x0c, 0xb9, 0x36, 0xe9, 0x74, - 0x5f, 0xd6, 0x70, 0xcf, 0xa6, 0x8e, 0xf4, 0x3d, 0x93, 0xe0, 0xd9, 0x54, 0xf6, 0x15, 0x88, 0xdf, - 0x17, 0x60, 0xc6, 0x2b, 0x62, 0xd9, 0x3f, 0x77, 0x78, 0x8a, 0x80, 0x63, 0x89, 0x7e, 0x61, 0x89, - 0x60, 0x6d, 0x7a, 0x28, 0x78, 0xa4, 0xee, 0x89, 0x78, 0xc8, 0x02, 0x70, 0x0c, 0xaf, 0x35, 0xea, - 0xf4, 0xdb, 0x4b, 0xec, 0xbf, 0x5d, 0xba, 0x13, 0x50, 0x20, 0x1d, 0x09, 0x44, 0x4b, 0x23, 0x0d, - 0x19, 0x57, 0x4b, 0x34, 0x73, 0xe9, 0x87, 0xc1, 0x96, 0xa8, 0x3c, 0x21, 0xbe, 0xdb, 0x6d, 0x48, - 0x3e, 0x51, 0x5a, 0xc3, 0x22, 0x36, 0x42, 0x2d, 0x27, 0x91, 0xdc, 0xe8, 0x4e, 0xe8, 0xb8, 0x66, - 0x62, 0xb0, 0x9f, 0xd1, 0xaf, 0xd2, 0xd0, 0xb1, 0xce, 0x4f, 0xb9, 0xb5, 0x48, 0x1e, 0xff, 0xfa, - 0xa0, 0x05, 0x78, 0x3d, 0xf5, 0xfe, 0x7b, 0x0b, 0xc2, 0xf5, 0x3a, 0xcc, 0x44, 0xcc, 0x8d, 0xa8, - 0x00, 0x10, 0xb8, 0xba, 0x99, 0x7f, 0xe0, 0x69, 0x79, 0x55, 0xde, 0xde, 0x28, 0x6f, 0xae, 0xaf, - 0xd7, 0x1a, 0x8d, 0xca, 0xaa, 0x28, 0x20, 0x11, 0x72, 0xa1, 0x8b, 0x9f, 0xf9, 0x77, 0x9d, 0xae, - 0xff, 0x12, 0x80, 0x7f, 0xf9, 0x3b, 0x91, 0xb5, 0x56, 0x79, 0x28, 0x3f, 0x58, 0xbe, 0xbf, 0x5d, - 0xa9, 0x8b, 0x63, 0x08, 0x41, 0x61, 0x65, 0xb9, 0x51, 0xae, 0xca, 0x52, 0xa5, 0xbe, 0xb5, 0xb9, - 0x51, 0xaf, 0x88, 0x02, 0xe7, 0x5b, 0x85, 0x5c, 0xf0, 0x60, 0x2b, 0x9a, 0x81, 0xa9, 0x72, 0xb5, - 0x52, 0x5e, 0x93, 0x1f, 0xd4, 0x96, 0xe5, 0xb7, 0xb6, 0x2b, 0xdb, 0x15, 0x71, 0x8c, 0x16, 0x8d, - 0x12, 0xef, 0x6c, 0xdf, 0xbf, 0x2f, 0x0a, 0x68, 0x0a, 0xb2, 0xec, 0x99, 0x5e, 0x12, 0x2d, 0x26, - 0xae, 0xaf, 0x43, 0x36, 0x70, 0x7b, 0x14, 0x79, 0xdd, 0xd6, 0x76, 0xbd, 0x2a, 0x37, 0x6a, 0xeb, - 0x95, 0x7a, 0x63, 0x79, 0x7d, 0x8b, 0xc9, 0xa0, 0xb4, 0xe5, 0x95, 0x4d, 0xa9, 0x21, 0x0a, 0xde, - 0x73, 0x63, 0x73, 0xbb, 0x5c, 0xf5, 0x3e, 0x4f, 0x95, 0x4a, 0x27, 0xc5, 0xe4, 0xf5, 0xc7, 0x70, - 0x6e, 0xc0, 0xe9, 0x4e, 0x94, 0x85, 0xc9, 0x6d, 0x83, 0xde, 0x84, 0x23, 0x8e, 0xa1, 0x7c, 0xe0, - 0x80, 0xa7, 0x28, 0xa0, 0x34, 0x3b, 0xba, 0x27, 0x26, 0xd0, 0x04, 0x24, 0xea, 0xb7, 0xc5, 0x24, - 0x29, 0x68, 0xe0, 0x7c, 0xa4, 0x98, 0x42, 0x19, 0x7e, 0x78, 0x4c, 0x1c, 0x47, 0x39, 0xff, 0xf4, - 0x96, 0x38, 0x71, 0xfd, 0x02, 0x04, 0x8e, 0xb6, 0x20, 0x80, 0x89, 0xfb, 0x8a, 0x83, 0x6d, 0x47, - 0x1c, 0x43, 0x93, 0x90, 0x5c, 0x6e, 0xb5, 0x44, 0xe1, 0xd6, 0x9f, 0x0b, 0x90, 0x76, 0xef, 0x39, - 0x46, 0xf7, 0x61, 0x9c, 0x01, 0xf0, 0x85, 0xc1, 0x33, 0x12, 0x35, 0x6a, 0x73, 0x8b, 0xc7, 0x4d, - 0x59, 0xa5, 0x31, 0xf4, 0x36, 0xff, 0xd6, 0x1c, 0xe9, 0x31, 0xe8, 0xe2, 0xb0, 0xfe, 0xe4, 0x4a, - 0x1d, 0xde, 0xe9, 0xc8, 0x18, 0x29, 0x8d, 0xbd, 0x22, 0xac, 0xbc, 0xf8, 0xfe, 0x8f, 0xe7, 0xc7, - 0xde, 0x7f, 0x36, 0x2f, 0x7c, 0xf0, 0x6c, 0x5e, 0xf8, 0xd1, 0xb3, 0x79, 0xe1, 0xdf, 0x9f, 0xcd, - 0x0b, 0xbf, 0xfd, 0x93, 0xf9, 0xb1, 0x0f, 0x7e, 0x32, 0x3f, 0xf6, 0xa3, 0x9f, 0xcc, 0x8f, 0xbd, - 0x33, 0xc9, 0xb9, 0x77, 0x26, 0xe8, 0x67, 0xef, 0x6e, 0xff, 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, - 0x54, 0xa0, 0xd3, 0x14, 0xfb, 0x6f, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_46e7e3331e4f911b) } + +var fileDescriptor_api_46e7e3331e4f911b = []byte{ + // 7087 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0xdb, 0x6f, 0x23, 0xc9, + 0x75, 0xb7, 0x9a, 0xa4, 0x28, 0xf2, 0x90, 0xa2, 0x5a, 0xa5, 0xb9, 0x70, 0xb4, 0xbb, 0x23, 0x0d, + 0xe7, 0xba, 0xe3, 0x5d, 0xcd, 0xce, 0x8c, 0xf7, 0xf3, 0x7a, 0xd7, 0x5e, 0x5b, 0xa2, 0x38, 0x43, + 0x8e, 0x46, 0x97, 0x6d, 0x52, 0xb3, 0x9e, 0xf5, 0xb7, 0x5f, 0xbb, 0xd5, 0x5d, 0xa2, 0xda, 0x43, + 0x76, 0x73, 0xba, 0x9b, 0xba, 0x0c, 0xf0, 0x21, 0xc9, 0x93, 0x03, 0xc7, 0x30, 0xf2, 0x10, 0x04, + 0x81, 0x9d, 0xc0, 0x1b, 0xd8, 0x48, 0x80, 0x04, 0x31, 0x10, 0x20, 0x48, 0xe2, 0x20, 0x81, 0x1f, + 0xf2, 0xb2, 0x31, 0x8c, 0x60, 0x11, 0x24, 0x88, 0x91, 0x07, 0x21, 0x96, 0x11, 0xc4, 0xf0, 0x3f, + 0x60, 0x60, 0x1f, 0x92, 0xa0, 0x2e, 0x7d, 0x23, 0x9b, 0x14, 0x25, 0xf7, 0x3a, 0x1b, 0xe4, 0x89, + 0xec, 0x53, 0x75, 0x4e, 0x57, 0x9d, 0xaa, 0x3a, 0x75, 0x7e, 0x55, 0xa7, 0xaa, 0x61, 0xda, 0x32, + 0x15, 0x75, 0xa7, 0xb3, 0x75, 0x4b, 0xe9, 0xe8, 0x0b, 0x1d, 0xcb, 0x74, 0x4c, 0x34, 0xad, 0x9a, + 0xea, 0x13, 0x4a, 0x5e, 0xe0, 0x89, 0xb3, 0xc8, 0xcd, 0xa5, 0x29, 0x8e, 0xc2, 0xb2, 0xcd, 0x9e, + 0x71, 0x69, 0xd8, 0xb2, 0x4c, 0xcb, 0xe6, 0xd4, 0x73, 0x2e, 0xb5, 0x8d, 0x1d, 0x25, 0x90, 0xbb, + 0x64, 0x3b, 0xa6, 0xa5, 0x34, 0xf1, 0x2d, 0x6c, 0x34, 0x75, 0xc3, 0xfd, 0x21, 0xf9, 0x76, 0x55, + 0x95, 0xe7, 0xb9, 0x3c, 0x2c, 0xcf, 0x5d, 0x9e, 0xa9, 0xd8, 0x75, 0xf4, 0xd6, 0xad, 0x9d, 0x96, + 0x7a, 0xcb, 0xd1, 0xdb, 0xd8, 0x76, 0x94, 0x76, 0x87, 0xa7, 0xcc, 0xd3, 0x14, 0xc7, 0x52, 0x54, + 0xdd, 0x68, 0xde, 0xb2, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, 0xbb, 0xa3, 0x18, 0x6e, 0x91, 0x9b, + 0x66, 0xd3, 0xa4, 0x7f, 0x6f, 0x91, 0x7f, 0x8c, 0x5a, 0xfa, 0x15, 0xc8, 0x4a, 0x8a, 0xd1, 0xc4, + 0x35, 0x63, 0xdb, 0x44, 0x9f, 0x81, 0x94, 0x86, 0x6d, 0xb5, 0x28, 0xcc, 0x0b, 0x37, 0x72, 0x77, + 0x4a, 0x0b, 0x7d, 0xba, 0x58, 0xa0, 0x79, 0x97, 0xb1, 0xad, 0x5a, 0x7a, 0xc7, 0x31, 0xad, 0xa5, + 0xd4, 0xfb, 0x87, 0x73, 0x63, 0x12, 0xe5, 0x42, 0x9f, 0x84, 0xf1, 0x16, 0x56, 0x6c, 0x5c, 0x4c, + 0x50, 0xf6, 0x62, 0x04, 0xfb, 0x43, 0x92, 0xce, 0x99, 0x58, 0xe6, 0xd2, 0xdf, 0x0a, 0x30, 0x29, + 0xe1, 0xa7, 0x5d, 0x6c, 0x3b, 0x55, 0xac, 0x68, 0xd8, 0x42, 0x17, 0x20, 0xf9, 0x04, 0x1f, 0x14, + 0x93, 0xf3, 0xc2, 0x8d, 0xfc, 0xd2, 0xc4, 0x87, 0x87, 0x73, 0xc9, 0x15, 0x7c, 0x20, 0x11, 0x1a, + 0x9a, 0x87, 0x09, 0x6c, 0x68, 0x32, 0x49, 0x4e, 0x85, 0x93, 0xd3, 0xd8, 0xd0, 0x56, 0xf0, 0x01, + 0x52, 0x21, 0x63, 0x13, 0x69, 0x86, 0x8a, 0x8b, 0xe3, 0xf3, 0xc2, 0x8d, 0xf1, 0xa5, 0xfb, 0x1f, + 0x1e, 0xce, 0x95, 0x9b, 0xba, 0xb3, 0xd3, 0xdd, 0x5a, 0x50, 0xcd, 0xf6, 0x2d, 0xaf, 0x54, 0xda, + 0x96, 0xff, 0xff, 0x56, 0xe7, 0x49, 0xf3, 0xd6, 0x80, 0x16, 0x58, 0x68, 0xec, 0x1b, 0x75, 0xfc, + 0x54, 0xf2, 0x04, 0xbf, 0x9e, 0xfa, 0xe9, 0x7b, 0x73, 0xc2, 0x83, 0x54, 0x46, 0x10, 0x13, 0x0f, + 0x52, 0x99, 0x84, 0x98, 0x2c, 0x7d, 0x2d, 0x09, 0x05, 0x09, 0xdb, 0x1d, 0xd3, 0xb0, 0x31, 0xaf, + 0xc6, 0x2b, 0x90, 0x74, 0xf6, 0x0d, 0x5a, 0x8d, 0xdc, 0x9d, 0x8b, 0x11, 0xca, 0x68, 0x58, 0x8a, + 0x61, 0x2b, 0xaa, 0xa3, 0x9b, 0x86, 0x44, 0xb2, 0xa2, 0xd7, 0x20, 0x67, 0x61, 0xbb, 0xdb, 0xc6, + 0xb4, 0xd9, 0x68, 0x0d, 0x73, 0x77, 0xce, 0x47, 0x70, 0xd6, 0x3b, 0x8a, 0x21, 0x01, 0xcb, 0x4b, + 0xfe, 0xa3, 0x0b, 0x90, 0x31, 0xba, 0x6d, 0xa2, 0x17, 0x9b, 0xd6, 0x3a, 0x29, 0x4d, 0x18, 0xdd, + 0xf6, 0x0a, 0x3e, 0xb0, 0x51, 0x19, 0x72, 0x16, 0x69, 0x34, 0x59, 0x37, 0xb6, 0x4d, 0xbb, 0x98, + 0x9e, 0x4f, 0xde, 0xc8, 0xdd, 0x79, 0x7e, 0x50, 0xd3, 0x92, 0x6e, 0xc0, 0xdb, 0x07, 0x2c, 0x97, + 0x60, 0xa3, 0x3a, 0x4c, 0xf2, 0x92, 0x59, 0x58, 0xb1, 0x4d, 0xa3, 0x38, 0x31, 0x2f, 0xdc, 0x28, + 0xdc, 0x59, 0x88, 0x12, 0x13, 0xd2, 0x02, 0x79, 0xec, 0xb6, 0xb1, 0x44, 0xb9, 0xa4, 0xbc, 0x15, + 0x78, 0x2a, 0x3d, 0x86, 0x7c, 0x30, 0x15, 0x21, 0x28, 0x48, 0x95, 0xfa, 0xe6, 0x6a, 0x45, 0xde, + 0x5c, 0x5b, 0x59, 0x5b, 0x7f, 0x7b, 0x4d, 0x1c, 0x43, 0x67, 0x40, 0xe4, 0xb4, 0x95, 0xca, 0x63, + 0xf9, 0x61, 0x6d, 0xb5, 0xd6, 0x10, 0x05, 0x74, 0x01, 0xce, 0x72, 0xaa, 0xb4, 0xb8, 0x76, 0xbf, + 0x22, 0x2f, 0xad, 0x6f, 0xae, 0x2d, 0x2f, 0x4a, 0x8f, 0xc5, 0xc4, 0x6c, 0xea, 0xd7, 0xbf, 0x7d, + 0x71, 0xac, 0xf4, 0x08, 0xe0, 0x3e, 0x76, 0x78, 0xb7, 0x42, 0x4b, 0x90, 0xde, 0xa1, 0xa5, 0xe1, + 0x1d, 0x7b, 0x3e, 0xb2, 0xd8, 0x81, 0x2e, 0xb8, 0x94, 0x21, 0x1a, 0xf8, 0xe0, 0x70, 0x4e, 0x90, + 0x38, 0x27, 0x6b, 0xf2, 0xd2, 0xf7, 0x05, 0xc8, 0x51, 0xc1, 0xac, 0x8e, 0xa8, 0xdc, 0x23, 0xf9, + 0xd2, 0xb1, 0x0a, 0xe9, 0x17, 0x8d, 0x16, 0x60, 0x7c, 0x57, 0x69, 0x75, 0x87, 0x8d, 0x9b, 0x47, + 0x24, 0x5d, 0x62, 0xd9, 0xd0, 0x1b, 0x90, 0xd7, 0x0d, 0x07, 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xc7, + 0xb0, 0xe5, 0x58, 0x6e, 0xfa, 0x50, 0xfa, 0x2b, 0x01, 0x60, 0xa3, 0x1b, 0xa7, 0x6a, 0xc8, 0xb8, + 0x1f, 0xa9, 0xfc, 0xee, 0xb8, 0x67, 0xb5, 0x38, 0x07, 0x69, 0xdd, 0x68, 0xe9, 0x06, 0x2b, 0x7f, + 0x46, 0xe2, 0x4f, 0xe8, 0x0c, 0x8c, 0x6f, 0xb5, 0x74, 0x43, 0xa3, 0xdd, 0x3f, 0x23, 0xb1, 0x07, + 0xae, 0x7e, 0x09, 0x72, 0xb4, 0xec, 0x31, 0x6a, 0xbf, 0xf4, 0x8d, 0x04, 0x9c, 0x2d, 0x9b, 0x86, + 0xa6, 0x93, 0x71, 0xa8, 0xb4, 0x3e, 0x16, 0xba, 0x79, 0x15, 0xb2, 0x78, 0xbf, 0x33, 0x62, 0xf3, + 0x66, 0xf0, 0x7e, 0x87, 0xfe, 0x8b, 0x56, 0x1d, 0xfa, 0x24, 0x9c, 0x57, 0x5a, 0x2d, 0x73, 0x4f, + 0xd6, 0xb7, 0x65, 0xcd, 0xc4, 0xb6, 0x6c, 0x98, 0x8e, 0x8c, 0xf7, 0x75, 0xdb, 0xa1, 0xa6, 0x22, + 0x23, 0xcd, 0xd0, 0xe4, 0xda, 0xf6, 0xb2, 0x89, 0xed, 0x35, 0xd3, 0xa9, 0x90, 0x24, 0xae, 0xf0, + 0x77, 0xe1, 0x5c, 0xaf, 0x6e, 0xe2, 0xd4, 0xfd, 0x3f, 0x08, 0x50, 0xa8, 0x19, 0xba, 0xf3, 0xb1, + 0x50, 0xba, 0xa7, 0xbd, 0x64, 0x50, 0x7b, 0x37, 0x41, 0xdc, 0x56, 0xf4, 0xd6, 0xba, 0xd1, 0x30, + 0xdb, 0x5b, 0xb6, 0x63, 0x1a, 0xd8, 0xe6, 0xea, 0xed, 0xa3, 0x73, 0x9d, 0x3d, 0x82, 0x29, 0xaf, + 0x4e, 0x71, 0x2a, 0xeb, 0x19, 0x88, 0x35, 0x43, 0xb5, 0x70, 0x1b, 0x1b, 0xb1, 0x6a, 0xeb, 0x79, + 0xc8, 0xea, 0xae, 0x5c, 0xaa, 0xb1, 0xa4, 0xe4, 0x13, 0x78, 0x9d, 0xba, 0x30, 0x1d, 0x78, 0x77, + 0x9c, 0xc6, 0xef, 0x39, 0xc8, 0x1a, 0x78, 0x4f, 0xf6, 0xdb, 0x2b, 0x29, 0x65, 0x0c, 0xbc, 0xc7, + 0x8c, 0xd5, 0x63, 0x98, 0x5c, 0xc6, 0x2d, 0xec, 0xe0, 0xf8, 0x2d, 0xf9, 0x26, 0x14, 0x5c, 0xd1, + 0x71, 0x36, 0xd2, 0xef, 0x09, 0x80, 0xb8, 0x5c, 0x32, 0x7b, 0xc6, 0xd9, 0x4e, 0x73, 0xc4, 0x3b, + 0x70, 0xba, 0x96, 0xc1, 0xa6, 0x79, 0xd6, 0x4b, 0x81, 0x91, 0xe8, 0x4c, 0xef, 0x5b, 0xd4, 0x54, + 0xd0, 0xa2, 0x7a, 0xde, 0x0a, 0xf1, 0x53, 0xf6, 0x60, 0x26, 0x54, 0xbc, 0x78, 0x9b, 0x32, 0x45, + 0x4b, 0x96, 0x98, 0x4f, 0x06, 0x3d, 0x33, 0x4a, 0x2c, 0xbd, 0x0b, 0xd3, 0xe5, 0x16, 0x56, 0xac, + 0xb8, 0xd5, 0xc2, 0x9b, 0xf3, 0x31, 0xa0, 0xa0, 0xf8, 0x38, 0x9b, 0xf4, 0x3b, 0x02, 0x20, 0x09, + 0xef, 0x62, 0xcb, 0x89, 0xbd, 0x49, 0x97, 0x21, 0xe7, 0x28, 0x56, 0x13, 0x3b, 0x32, 0x71, 0xe7, + 0xb9, 0xb9, 0x7a, 0x21, 0x20, 0x88, 0x38, 0xf5, 0x0b, 0x3b, 0x2d, 0x75, 0xa1, 0xe1, 0xba, 0xfb, + 0xae, 0x73, 0xc6, 0xf8, 0x08, 0x99, 0x6b, 0xe0, 0x1d, 0x98, 0x09, 0x95, 0x32, 0x4e, 0x15, 0xe8, + 0x90, 0xab, 0xab, 0x8a, 0xb1, 0xde, 0x21, 0xf3, 0x80, 0x8d, 0xee, 0xc2, 0x39, 0xdb, 0x31, 0x3b, + 0xb2, 0xe2, 0xc8, 0xcc, 0xb5, 0xdc, 0x32, 0xbb, 0x86, 0xa6, 0x58, 0x07, 0xf4, 0x1d, 0x19, 0x69, + 0x86, 0xa4, 0x2e, 0xb2, 0x82, 0x2c, 0xf1, 0x24, 0xd2, 0x7d, 0xdb, 0xba, 0x21, 0x13, 0x0f, 0xb0, + 0xe5, 0xd8, 0x7c, 0xa8, 0x43, 0x5b, 0x37, 0x24, 0x46, 0xe1, 0xd5, 0xf8, 0xb6, 0xc0, 0xde, 0x15, + 0xa7, 0x9a, 0xdf, 0x84, 0x9c, 0xad, 0x2a, 0x86, 0xbc, 0x6d, 0x5a, 0x6d, 0xc5, 0xa1, 0xa3, 0xa3, + 0x10, 0x52, 0xb3, 0xe7, 0x57, 0xab, 0x8a, 0x71, 0x8f, 0x66, 0x92, 0xc0, 0xf6, 0xfe, 0x07, 0x07, + 0xd0, 0x83, 0x54, 0x26, 0x29, 0xa6, 0x4a, 0x3f, 0x17, 0x20, 0xcf, 0x4a, 0x19, 0xe7, 0x00, 0x7a, + 0x15, 0x52, 0x96, 0xb9, 0xc7, 0x06, 0x50, 0xee, 0xce, 0x73, 0x11, 0x22, 0x56, 0xf0, 0x41, 0x70, + 0xe6, 0xa2, 0xd9, 0xd1, 0x12, 0x70, 0x0f, 0x4f, 0xa6, 0xdc, 0xc9, 0x51, 0xb9, 0x81, 0x71, 0x49, + 0x44, 0xc6, 0x75, 0x98, 0xda, 0x52, 0x1c, 0x75, 0x87, 0xb4, 0x0f, 0x2d, 0x24, 0x99, 0xe5, 0x92, + 0x37, 0xf2, 0x52, 0x81, 0x92, 0xdd, 0xa2, 0xdb, 0xa5, 0x3f, 0x72, 0x47, 0x83, 0x8d, 0x3f, 0xfe, + 0xcd, 0xf4, 0x1f, 0x02, 0x1f, 0x14, 0x6e, 0x61, 0xff, 0xb7, 0xb5, 0xd6, 0xb7, 0x12, 0x70, 0xbe, + 0xbc, 0x83, 0xd5, 0x27, 0x65, 0xd3, 0xb0, 0x75, 0xdb, 0xc1, 0x86, 0x7a, 0x10, 0x67, 0x93, 0x3d, + 0x07, 0xd9, 0x3d, 0xdd, 0xd9, 0x91, 0x35, 0x7d, 0x7b, 0x9b, 0x0e, 0xe9, 0x8c, 0x94, 0x21, 0x84, + 0x65, 0x7d, 0x7b, 0x1b, 0xdd, 0x85, 0x54, 0xdb, 0xd4, 0x98, 0x03, 0x5b, 0xb8, 0x33, 0x17, 0x21, + 0x9e, 0x16, 0xcd, 0xee, 0xb6, 0x57, 0x4d, 0x0d, 0x4b, 0x34, 0x33, 0xba, 0x08, 0xa0, 0x12, 0x6a, + 0xc7, 0xd4, 0x0d, 0x87, 0x4f, 0x64, 0x01, 0x0a, 0xaa, 0x42, 0xd6, 0xc1, 0x56, 0x5b, 0x37, 0x14, + 0x87, 0x00, 0x7c, 0xa2, 0xbc, 0x2b, 0x91, 0x05, 0xef, 0xb4, 0x74, 0x55, 0xe9, 0x5b, 0xa9, 0xf0, + 0x99, 0xb9, 0xbd, 0xf9, 0x7a, 0x0a, 0x8a, 0xfd, 0x1a, 0x8a, 0xb3, 0x9f, 0x6c, 0x40, 0x9a, 0x19, + 0x3d, 0xde, 0x53, 0xee, 0x0c, 0x52, 0x44, 0x44, 0x09, 0x16, 0x98, 0x71, 0xe4, 0x85, 0xe7, 0x72, + 0x66, 0xff, 0x5a, 0x80, 0x34, 0x4b, 0x40, 0xb7, 0x21, 0xc3, 0xd1, 0xbd, 0x46, 0xcb, 0x98, 0x5c, + 0x3a, 0x77, 0x74, 0x38, 0x37, 0xc1, 0xb0, 0xfc, 0xf2, 0x87, 0xfe, 0x5f, 0x69, 0x82, 0xc1, 0x79, + 0x8d, 0xb4, 0x99, 0xed, 0x28, 0x96, 0x43, 0x57, 0x51, 0x48, 0x9b, 0xe5, 0xa5, 0x0c, 0x25, 0xac, + 0xe0, 0x03, 0xf4, 0x00, 0xd2, 0xb6, 0xa3, 0x38, 0x5d, 0x9b, 0xb7, 0xda, 0x89, 0x0a, 0x5b, 0xa7, + 0x9c, 0x12, 0x97, 0x40, 0xfc, 0x11, 0x0d, 0x3b, 0x8a, 0xde, 0xa2, 0xcd, 0x98, 0x95, 0xf8, 0x53, + 0xe9, 0x9b, 0x02, 0xa4, 0x59, 0x56, 0x74, 0x1e, 0x66, 0x18, 0x82, 0xaf, 0xad, 0x2d, 0x57, 0x1a, + 0x15, 0x69, 0xb5, 0xb6, 0xb6, 0xd8, 0xa8, 0x88, 0x63, 0xe8, 0x1c, 0x20, 0x37, 0xa1, 0xbc, 0xbe, + 0x56, 0xaf, 0xd5, 0x1b, 0x95, 0x35, 0x82, 0xfc, 0xcf, 0x80, 0xc8, 0xe8, 0x01, 0x6a, 0x02, 0x5d, + 0x81, 0xf9, 0x5e, 0xaa, 0x5c, 0x6f, 0x2c, 0x36, 0xea, 0x72, 0xa5, 0xde, 0xa8, 0xad, 0x2e, 0x36, + 0x2a, 0xcb, 0x62, 0x72, 0x48, 0x2e, 0xf2, 0x12, 0x49, 0xaa, 0x94, 0x1b, 0x62, 0xaa, 0xf4, 0x0c, + 0xce, 0x4a, 0x58, 0x35, 0xdb, 0x9d, 0xae, 0x83, 0x49, 0x29, 0xed, 0x38, 0xc7, 0xcb, 0x79, 0x98, + 0xd0, 0xac, 0x03, 0xd9, 0xea, 0x1a, 0x7c, 0xb4, 0xa4, 0x35, 0xeb, 0x40, 0xea, 0x1a, 0xbc, 0x33, + 0xfe, 0x85, 0x00, 0xe7, 0x7a, 0x5f, 0x1e, 0x67, 0x57, 0xfc, 0x02, 0xe4, 0x14, 0x4d, 0xc3, 0x9a, + 0xac, 0xe1, 0x96, 0xa3, 0x70, 0x7f, 0xe3, 0x76, 0x40, 0x12, 0x5f, 0x01, 0x5b, 0x60, 0x4b, 0x5f, + 0x0b, 0xde, 0x0a, 0xd8, 0xea, 0xa3, 0x72, 0x99, 0x96, 0x67, 0x99, 0x30, 0xba, 0x16, 0x89, 0xca, + 0xa2, 0x94, 0xd2, 0x77, 0x52, 0x70, 0xb6, 0x62, 0x68, 0xc1, 0x25, 0xad, 0x18, 0xd5, 0x76, 0x0e, + 0xd2, 0xaa, 0xd9, 0x6e, 0xeb, 0x8e, 0xab, 0x35, 0xf6, 0x84, 0x3e, 0x0d, 0x19, 0x0d, 0x2b, 0x9a, + 0xb7, 0x8a, 0x70, 0x9c, 0xf3, 0x24, 0x79, 0xd9, 0xd1, 0x97, 0xe0, 0x3c, 0x31, 0xa8, 0x96, 0xa1, + 0xb4, 0x64, 0x26, 0x4d, 0x76, 0x2c, 0xbd, 0xd9, 0xc4, 0x16, 0x5f, 0x77, 0xbb, 0x11, 0x51, 0xce, + 0x1a, 0xe7, 0x28, 0x53, 0x86, 0x06, 0xcb, 0x2f, 0x9d, 0xd5, 0xa3, 0xc8, 0xe8, 0xf3, 0xde, 0x32, + 0x8d, 0xdd, 0x51, 0x0c, 0x9b, 0x1b, 0xab, 0x41, 0xcb, 0x79, 0x5c, 0xa7, 0x7c, 0x6e, 0x20, 0x14, + 0x1b, 0xdd, 0x22, 0x1e, 0xff, 0xd3, 0xae, 0x6e, 0x61, 0xf9, 0x76, 0x47, 0x2d, 0xa6, 0x49, 0xdd, + 0x97, 0x0a, 0x47, 0x87, 0x73, 0x20, 0x31, 0xf2, 0xed, 0x8d, 0x32, 0x41, 0x00, 0xec, 0x7f, 0x47, + 0x45, 0x37, 0x40, 0x34, 0x4c, 0xd9, 0xc2, 0xdb, 0x16, 0xb6, 0x77, 0xf8, 0x6b, 0x33, 0x54, 0x63, + 0x05, 0xc3, 0x94, 0x18, 0x99, 0x89, 0x3e, 0x07, 0xe9, 0x8e, 0xa9, 0xdb, 0xa6, 0x51, 0xcc, 0x32, + 0x8d, 0xb2, 0x27, 0xf4, 0x16, 0x88, 0xba, 0x21, 0x6f, 0xb7, 0xf4, 0xe6, 0x8e, 0x23, 0xef, 0x59, + 0xba, 0x83, 0xed, 0xe2, 0x34, 0x2d, 0x78, 0x54, 0x87, 0xab, 0xf3, 0x05, 0x51, 0xed, 0x6d, 0x92, + 0x93, 0x57, 0xa1, 0xa0, 0x1b, 0xf7, 0x28, 0x3f, 0x25, 0xda, 0xde, 0xb4, 0x3c, 0x21, 0x66, 0x4a, + 0xff, 0x26, 0xc0, 0xb9, 0xde, 0x6e, 0x12, 0x67, 0x07, 0xbf, 0x01, 0xa2, 0x69, 0x60, 0xb9, 0xb3, + 0xa3, 0xd8, 0x98, 0x37, 0x2b, 0x9f, 0x43, 0x0a, 0xa6, 0x81, 0x37, 0x08, 0x99, 0x35, 0x12, 0xda, + 0x80, 0x69, 0xdb, 0x51, 0x9a, 0xba, 0xd1, 0x94, 0xbd, 0xa5, 0x74, 0xba, 0x1e, 0x32, 0xa2, 0x03, + 0x2e, 0x72, 0x6e, 0x8f, 0x1e, 0x72, 0x3c, 0xfe, 0x59, 0x80, 0xe9, 0x45, 0xad, 0xad, 0x1b, 0xf5, + 0x4e, 0x4b, 0x8f, 0x15, 0xad, 0x5f, 0x81, 0xac, 0x4d, 0x64, 0xfa, 0xd6, 0xdb, 0x47, 0x5a, 0x19, + 0x9a, 0x42, 0xcc, 0xf8, 0x43, 0x98, 0xc2, 0xfb, 0x1d, 0xdd, 0x52, 0x88, 0x8a, 0x19, 0xb8, 0x48, + 0x8d, 0x5e, 0xb7, 0x82, 0xcf, 0xeb, 0x03, 0x0c, 0x5e, 0xb3, 0xc7, 0x80, 0x82, 0x15, 0x8b, 0x13, + 0x65, 0xc8, 0x30, 0x43, 0x45, 0x6f, 0x1a, 0x76, 0xcc, 0x5a, 0xe3, 0xe6, 0xf5, 0x8b, 0x70, 0x26, + 0xfc, 0x82, 0x38, 0x4b, 0xff, 0x2e, 0x6f, 0xf1, 0x55, 0x6c, 0x7d, 0x44, 0x00, 0x37, 0x28, 0x3e, + 0xce, 0x92, 0x7f, 0x55, 0x80, 0x0b, 0x54, 0x36, 0x1d, 0x96, 0xdb, 0xd8, 0xa2, 0xbb, 0x34, 0x71, + 0x76, 0xda, 0xcb, 0x90, 0x66, 0x78, 0x95, 0xf6, 0xd8, 0xf1, 0xa5, 0x1c, 0x71, 0x4c, 0xea, 0x8e, + 0x69, 0x11, 0xc7, 0x84, 0x27, 0xf1, 0x7a, 0x2a, 0x30, 0x1b, 0x55, 0x96, 0x98, 0x01, 0xfd, 0x34, + 0xf7, 0x0f, 0x49, 0x17, 0x2f, 0xef, 0x10, 0xc7, 0x08, 0x55, 0x20, 0xa7, 0xd2, 0x7f, 0xb2, 0x73, + 0xd0, 0xc1, 0x54, 0x7e, 0x61, 0x98, 0x6b, 0xc9, 0xd8, 0x1a, 0x07, 0x1d, 0x4c, 0xfc, 0x53, 0xf7, + 0x3f, 0x51, 0x57, 0xa0, 0xaa, 0x43, 0x9d, 0x53, 0x3a, 0xbe, 0x68, 0x5e, 0xd7, 0xbf, 0x0b, 0x69, + 0xe2, 0x7b, 0x49, 0xae, 0x0a, 0xf6, 0x26, 0xce, 0x14, 0xab, 0x3b, 0xf2, 0x0e, 0x9c, 0xd3, 0x70, + 0xc7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, 0x0e, 0x56, 0x3f, 0x71, 0x82, 0xea, 0x9f, 0xf1, 0x65, 0xf8, + 0x54, 0xf4, 0x18, 0x50, 0x40, 0x36, 0xab, 0x99, 0x0b, 0x77, 0x4e, 0xa2, 0x94, 0x69, 0x5f, 0x0a, + 0xa3, 0xdb, 0xa8, 0x0c, 0x19, 0xbc, 0xdf, 0x91, 0xe9, 0x56, 0x65, 0xea, 0x84, 0x5b, 0x95, 0x13, + 0x78, 0xbf, 0x43, 0x88, 0x68, 0x93, 0xcc, 0x74, 0xae, 0x03, 0x40, 0x8b, 0x6d, 0x1f, 0x8f, 0x27, + 0xfc, 0xfe, 0xc2, 0xc5, 0x4d, 0x79, 0x73, 0x3f, 0x13, 0xc1, 0xdb, 0xee, 0x3d, 0x01, 0x9e, 0x8b, + 0x6c, 0xbb, 0x38, 0x27, 0x3b, 0x77, 0xb7, 0x36, 0x71, 0x9a, 0xdd, 0xda, 0xd2, 0x1f, 0xbb, 0xa3, + 0x5e, 0xc2, 0x2d, 0x93, 0xa8, 0xf7, 0x23, 0x58, 0xdd, 0x9a, 0x70, 0x9b, 0x3d, 0x71, 0xe2, 0x66, + 0x77, 0x59, 0x7b, 0xcc, 0x42, 0x4f, 0x61, 0xe3, 0x34, 0x0b, 0xbf, 0x2d, 0xc0, 0x4c, 0x15, 0x2b, + 0x96, 0xb3, 0x85, 0x15, 0xa7, 0xb1, 0x1f, 0xab, 0x03, 0xfb, 0x2a, 0x24, 0x0d, 0x73, 0xef, 0x24, + 0x0b, 0x7c, 0x24, 0xbf, 0x3f, 0x6d, 0x85, 0xcb, 0x15, 0x67, 0xad, 0xff, 0x2e, 0x01, 0xd9, 0xfb, + 0xe5, 0x38, 0xeb, 0xfa, 0x19, 0xbe, 0x0c, 0xcc, 0x86, 0x7a, 0x54, 0xb7, 0xf4, 0xde, 0xb7, 0x70, + 0xbf, 0xbc, 0x82, 0x0f, 0xdc, 0x6e, 0x49, 0xb8, 0xd0, 0x22, 0x64, 0x9d, 0x1d, 0xe2, 0xa7, 0x9a, + 0x2d, 0xed, 0x24, 0x3e, 0x8b, 0xcf, 0x35, 0xfb, 0x04, 0xc6, 0xa9, 0x5c, 0x37, 0x90, 0x40, 0x88, + 0x08, 0x24, 0x20, 0xaf, 0xf1, 0xdc, 0xbe, 0xc4, 0x49, 0x5e, 0xe3, 0x12, 0x58, 0xe3, 0x78, 0xbe, + 0xd1, 0xb8, 0x98, 0x2e, 0xbd, 0x05, 0x40, 0xaa, 0x16, 0x67, 0xf3, 0x7c, 0x2f, 0x09, 0x85, 0x8d, + 0xae, 0xbd, 0x13, 0x73, 0x7f, 0x2c, 0x03, 0x74, 0xba, 0xf6, 0x0e, 0xb6, 0x64, 0x67, 0xdf, 0xe0, + 0xf5, 0x3f, 0x26, 0x44, 0xc1, 0x55, 0x00, 0xe3, 0x6b, 0xec, 0x1b, 0x68, 0x9d, 0x0b, 0xc1, 0xb2, + 0x1f, 0xe7, 0x70, 0x73, 0x04, 0x30, 0xd9, 0xd8, 0x37, 0x56, 0xb1, 0x87, 0x22, 0x99, 0x40, 0x4c, + 0x04, 0x7e, 0x06, 0x26, 0xc8, 0x83, 0xec, 0x98, 0x27, 0x69, 0xf9, 0x34, 0xe1, 0x69, 0x98, 0xe8, + 0x0d, 0xc8, 0x32, 0x6e, 0x32, 0x7f, 0xa5, 0xe9, 0xfc, 0x15, 0x55, 0x25, 0xae, 0x4d, 0x3a, 0x73, + 0x65, 0x28, 0x2b, 0x99, 0xad, 0xce, 0xc0, 0xf8, 0xb6, 0x69, 0xa9, 0x98, 0x06, 0x36, 0x64, 0x24, + 0xf6, 0x80, 0x6e, 0xc2, 0xb4, 0x6e, 0xa8, 0xad, 0xae, 0xad, 0xef, 0x62, 0xd9, 0x2d, 0x1a, 0x03, + 0x4c, 0x53, 0x5e, 0x02, 0x15, 0x68, 0x06, 0x3b, 0xc2, 0x83, 0x54, 0x26, 0x23, 0x66, 0x4b, 0xdf, + 0x14, 0x60, 0xca, 0x6b, 0xbb, 0x38, 0x0d, 0x7f, 0x39, 0xa4, 0xf8, 0x93, 0xb7, 0x1e, 0x51, 0x76, + 0xe9, 0x1f, 0xa9, 0x17, 0xa4, 0x9a, 0xbb, 0xb4, 0x31, 0xe3, 0xec, 0x5c, 0x4b, 0x2c, 0xf0, 0x25, + 0x71, 0xca, 0x0e, 0x41, 0x43, 0x61, 0x6e, 0xc3, 0x19, 0xbd, 0x4d, 0x66, 0x06, 0xdd, 0x69, 0x1d, + 0x70, 0x24, 0xe7, 0x60, 0x77, 0x6f, 0x76, 0xc6, 0x4f, 0x2b, 0xbb, 0x49, 0xdc, 0x58, 0xb2, 0xdd, + 0x1a, 0xbf, 0x5a, 0x71, 0xea, 0xbd, 0x06, 0x93, 0x16, 0x13, 0x4d, 0x3c, 0x9a, 0x13, 0xaa, 0x3e, + 0xef, 0xb1, 0x12, 0xed, 0x7f, 0x37, 0x01, 0x53, 0x6f, 0x75, 0xb1, 0x75, 0xf0, 0x31, 0xd4, 0xfd, + 0x35, 0x98, 0xda, 0x53, 0x74, 0x47, 0xde, 0x36, 0x2d, 0xb9, 0xdb, 0xd1, 0x14, 0xc7, 0x0d, 0xd1, + 0x98, 0x24, 0xe4, 0x7b, 0xa6, 0xb5, 0x49, 0x89, 0x08, 0x03, 0x7a, 0x62, 0x98, 0x7b, 0x86, 0x4c, + 0xc8, 0x14, 0x48, 0xef, 0x1b, 0x7c, 0x21, 0x7a, 0xe9, 0x53, 0xff, 0x72, 0x38, 0x77, 0x77, 0xa4, + 0xa0, 0x2b, 0x1a, 0xb7, 0xd6, 0xed, 0xea, 0xda, 0xc2, 0xe6, 0x66, 0x6d, 0x59, 0x12, 0xa9, 0xc8, + 0xb7, 0x99, 0xc4, 0xc6, 0xbe, 0xe1, 0x3a, 0x00, 0x1f, 0x0a, 0x20, 0xfa, 0x0a, 0x8b, 0xb3, 0x55, + 0x2b, 0x90, 0x7b, 0xda, 0xc5, 0x96, 0x7e, 0x8a, 0x36, 0x05, 0xce, 0x48, 0x8c, 0xd7, 0x3b, 0x90, + 0x0f, 0xe9, 0x21, 0xf9, 0x8b, 0xe9, 0x21, 0xb7, 0xe7, 0xab, 0xa0, 0xf4, 0x43, 0x01, 0x10, 0xad, + 0x7c, 0x8d, 0xed, 0x01, 0x7c, 0xcc, 0x3a, 0xcc, 0x0d, 0x10, 0x69, 0x18, 0xa4, 0xac, 0x6f, 0xcb, + 0x6d, 0xdd, 0xb6, 0x75, 0xa3, 0xc9, 0x7b, 0x4c, 0x81, 0xd2, 0x6b, 0xdb, 0xab, 0x8c, 0xca, 0xdb, + 0xf2, 0xff, 0xc3, 0x4c, 0xa8, 0x36, 0x71, 0xb6, 0xe6, 0x25, 0xc8, 0x6f, 0x9b, 0x5d, 0x43, 0x93, + 0xd9, 0x42, 0x1a, 0x5f, 0x30, 0xcc, 0x51, 0x1a, 0x7b, 0x5f, 0xe9, 0x3f, 0x13, 0x70, 0x46, 0xc2, + 0xb6, 0xd9, 0xda, 0xc5, 0xf1, 0xeb, 0x73, 0x1d, 0xf8, 0x46, 0x8d, 0xfc, 0x8b, 0xa8, 0x35, 0xcb, + 0x64, 0xb0, 0x49, 0x31, 0xbc, 0x22, 0x7f, 0x65, 0x78, 0xcf, 0xec, 0x5f, 0x83, 0xe7, 0xeb, 0x7c, + 0xa9, 0xd0, 0x3a, 0xdf, 0x2e, 0x4c, 0xe9, 0x4d, 0xc3, 0x24, 0x86, 0xcc, 0xc6, 0x4f, 0x8d, 0x6e, + 0xdb, 0x05, 0x3f, 0xaf, 0x8e, 0x50, 0xd6, 0x1a, 0xe3, 0xac, 0xe3, 0xa7, 0x6b, 0xdd, 0x36, 0xdb, + 0xe5, 0x3d, 0x47, 0x8a, 0x7d, 0x74, 0x38, 0x57, 0x08, 0xa5, 0xd9, 0x52, 0x41, 0xf7, 0x9e, 0xc9, + 0x4b, 0x78, 0x07, 0xf8, 0xbf, 0x70, 0xb6, 0xa7, 0x01, 0xe2, 0xf4, 0x99, 0xfe, 0x3e, 0x09, 0x17, + 0xc2, 0xe2, 0xe3, 0x46, 0x36, 0xff, 0x43, 0x1a, 0xb9, 0x0a, 0x93, 0x6d, 0xdd, 0x38, 0xdd, 0xfa, + 0x66, 0xbe, 0xad, 0x1b, 0x1e, 0x2d, 0xaa, 0xbb, 0xa4, 0x7f, 0x79, 0xdd, 0x45, 0x81, 0xd9, 0xa8, + 0xf6, 0x8c, 0xb3, 0xcf, 0x7c, 0x5d, 0x80, 0x7c, 0xdc, 0x2b, 0x77, 0xa7, 0x8b, 0x43, 0xe3, 0x75, + 0x6e, 0xc0, 0xe4, 0x47, 0xb0, 0xd4, 0xf7, 0x87, 0x02, 0xa0, 0x86, 0xd5, 0x35, 0x08, 0x84, 0x7e, + 0x68, 0x36, 0xe3, 0xac, 0xec, 0x19, 0x18, 0xd7, 0x0d, 0x0d, 0xef, 0xd3, 0xca, 0xa6, 0x24, 0xf6, + 0x10, 0xda, 0x9f, 0x4c, 0x8e, 0xb4, 0x3f, 0xe9, 0x87, 0xb3, 0x84, 0x0a, 0x1a, 0xa7, 0x16, 0xbe, + 0x9b, 0x80, 0x19, 0x5e, 0x9d, 0xd8, 0x97, 0x3a, 0x4f, 0x15, 0x04, 0x8f, 0x3e, 0x0b, 0xd0, 0xb1, + 0xf0, 0xae, 0xcc, 0x58, 0x93, 0x23, 0xb1, 0x66, 0x09, 0x07, 0x25, 0xa0, 0x2f, 0xc0, 0x14, 0x19, + 0xe8, 0x1d, 0xcb, 0xec, 0x98, 0x36, 0xf1, 0x63, 0xec, 0xd1, 0x00, 0xd4, 0xf4, 0xd1, 0xe1, 0xdc, + 0xe4, 0xaa, 0x6e, 0x6c, 0x70, 0xc6, 0x46, 0x5d, 0x22, 0x16, 0xc3, 0x7b, 0x74, 0x07, 0xe0, 0x3f, + 0x09, 0x70, 0xe6, 0x23, 0x5b, 0x1c, 0xfe, 0xef, 0xd0, 0x98, 0x37, 0x0f, 0x89, 0xf4, 0xb1, 0x66, + 0x6c, 0x9b, 0xf1, 0x2f, 0xd9, 0x7f, 0x5d, 0x80, 0xe9, 0x80, 0xf8, 0x38, 0xbd, 0x9c, 0xd3, 0x1d, + 0xb5, 0xf8, 0x22, 0xf1, 0x7b, 0x82, 0xdd, 0x3e, 0xce, 0x41, 0xf5, 0x37, 0x09, 0x38, 0x57, 0x66, + 0x3b, 0xd7, 0x6e, 0x58, 0x47, 0x9c, 0xbd, 0xa4, 0x08, 0x13, 0xbb, 0xd8, 0xb2, 0x75, 0x93, 0xcd, + 0xb7, 0x93, 0x92, 0xfb, 0x88, 0x66, 0x21, 0x63, 0x1b, 0x4a, 0xc7, 0xde, 0x31, 0xdd, 0xbd, 0x3e, + 0xef, 0xd9, 0x0b, 0x41, 0x19, 0x3f, 0x7d, 0x08, 0x4a, 0x7a, 0x78, 0x08, 0xca, 0xc4, 0x2f, 0x1c, + 0x82, 0xc2, 0x37, 0xd6, 0x7e, 0x20, 0xc0, 0xf9, 0x3e, 0xfd, 0xc5, 0xd9, 0x67, 0xbe, 0x0c, 0x39, + 0x95, 0x0b, 0x26, 0xd6, 0x98, 0xed, 0x1d, 0xd6, 0x48, 0xb6, 0x53, 0x62, 0x94, 0xa3, 0xc3, 0x39, + 0x70, 0x8b, 0x5a, 0x5b, 0xe6, 0x2a, 0x22, 0xff, 0xb5, 0xd2, 0x6f, 0xe4, 0x60, 0xaa, 0xb2, 0xcf, + 0x56, 0xc6, 0xeb, 0xcc, 0x2d, 0x40, 0xf7, 0x20, 0xd3, 0xb1, 0xcc, 0x5d, 0xdd, 0xad, 0x46, 0x21, + 0xe4, 0x32, 0xb9, 0xd5, 0xe8, 0xe1, 0xda, 0xe0, 0x1c, 0x92, 0xc7, 0x8b, 0x1a, 0x90, 0x7d, 0x68, + 0xaa, 0x4a, 0xeb, 0x9e, 0xde, 0x72, 0xfb, 0xff, 0x2b, 0xc7, 0x0b, 0x5a, 0xf0, 0x78, 0x36, 0x14, + 0x67, 0xc7, 0x6d, 0x0a, 0x8f, 0x88, 0x6a, 0x90, 0xa9, 0x3a, 0x4e, 0x87, 0x24, 0x72, 0x6b, 0x72, + 0x7d, 0x04, 0xa1, 0x84, 0x85, 0xcb, 0xf2, 0xd8, 0x51, 0x03, 0xa6, 0xef, 0x9b, 0x66, 0xb3, 0x85, + 0xcb, 0x2d, 0xb3, 0xab, 0x95, 0x4d, 0x63, 0x5b, 0x6f, 0x72, 0x7b, 0x7c, 0x6d, 0x04, 0x99, 0xf7, + 0xcb, 0x75, 0xa9, 0x5f, 0x00, 0x5a, 0x84, 0x4c, 0xfd, 0x2e, 0x17, 0xc6, 0xfc, 0xb8, 0xab, 0x23, + 0x08, 0xab, 0xdf, 0x95, 0x3c, 0x36, 0xf4, 0x00, 0x72, 0x8b, 0xcf, 0xba, 0x16, 0xe6, 0x52, 0xd2, + 0x03, 0xe3, 0x1c, 0x7a, 0xa5, 0x50, 0x2e, 0x29, 0xc8, 0x8c, 0xea, 0x50, 0x78, 0xdb, 0xb4, 0x9e, + 0xb4, 0x4c, 0xc5, 0xad, 0xe1, 0x04, 0x15, 0xf7, 0x89, 0x11, 0xc4, 0xb9, 0x8c, 0x52, 0x8f, 0x88, + 0xd9, 0x2f, 0xc0, 0x64, 0xa8, 0x99, 0x10, 0x82, 0x54, 0x87, 0xb4, 0x88, 0x40, 0x03, 0x88, 0xe8, + 0x7f, 0xf4, 0x32, 0x4c, 0x18, 0xa6, 0x86, 0xdd, 0x3e, 0x3c, 0xb9, 0x74, 0xe6, 0xe8, 0x70, 0x2e, + 0xbd, 0x66, 0x6a, 0xcc, 0xa1, 0xe0, 0xff, 0xa4, 0x34, 0xc9, 0xe4, 0xba, 0x13, 0xb3, 0xd7, 0x20, + 0x45, 0xda, 0x87, 0x98, 0x91, 0x2d, 0xc5, 0xc6, 0x9b, 0x96, 0xce, 0x65, 0xba, 0x8f, 0x3c, 0xdf, + 0x8f, 0x04, 0x48, 0xd4, 0xef, 0x12, 0x8f, 0x7a, 0xab, 0xab, 0x3e, 0xc1, 0x0e, 0xcf, 0xc5, 0x9f, + 0xa8, 0xa7, 0x6d, 0xe1, 0x6d, 0x9d, 0x79, 0x39, 0x59, 0x89, 0x3f, 0xa1, 0x17, 0x00, 0x14, 0x55, + 0xc5, 0xb6, 0x2d, 0xbb, 0x27, 0xd7, 0xb2, 0x52, 0x96, 0x51, 0x56, 0xf0, 0x01, 0x61, 0xb3, 0xb1, + 0x6a, 0x61, 0xc7, 0x8d, 0x84, 0x62, 0x4f, 0x84, 0xcd, 0xc1, 0xed, 0x8e, 0xec, 0x98, 0x4f, 0xb0, + 0x41, 0x5b, 0x35, 0x4b, 0xcc, 0x43, 0xbb, 0xd3, 0x20, 0x04, 0x62, 0xd9, 0xb0, 0xa1, 0xf9, 0x66, + 0x28, 0x2b, 0x79, 0xcf, 0x44, 0xa4, 0x85, 0x9b, 0x3a, 0x3f, 0x8a, 0x95, 0x95, 0xf8, 0x13, 0xd1, + 0x98, 0xd2, 0x75, 0x76, 0x68, 0xd8, 0x47, 0x56, 0xa2, 0xff, 0x79, 0xd5, 0xbe, 0x21, 0x40, 0xf2, + 0x7e, 0xb9, 0x7e, 0xe2, 0xba, 0xb9, 0x12, 0x93, 0xbe, 0x44, 0x1a, 0x80, 0xa8, 0xb7, 0x5a, 0xba, + 0xd1, 0x24, 0x4e, 0xc7, 0x97, 0xb1, 0xea, 0xd6, 0xac, 0xc0, 0xc9, 0x1b, 0x8c, 0x8a, 0xe6, 0x21, + 0xa7, 0x5a, 0x58, 0xc3, 0x86, 0xa3, 0x2b, 0x2d, 0x9b, 0x57, 0x31, 0x48, 0xe2, 0x85, 0xfb, 0x8a, + 0x00, 0xe3, 0xb4, 0x7b, 0xa1, 0xe7, 0x21, 0xab, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0xb7, 0x13, 0x59, + 0xc9, 0x27, 0x0c, 0x2c, 0xe4, 0x25, 0xc8, 0x2b, 0xaa, 0x6a, 0x76, 0x0d, 0x47, 0x36, 0x94, 0x36, + 0xe6, 0x85, 0xcd, 0x71, 0xda, 0x9a, 0xd2, 0xc6, 0x68, 0x0e, 0xdc, 0x47, 0xef, 0xfc, 0x60, 0x56, + 0x02, 0x4e, 0x5a, 0xc1, 0x07, 0xbc, 0x24, 0x3f, 0x10, 0x20, 0xe3, 0x76, 0x4b, 0x52, 0x98, 0x26, + 0x36, 0xb0, 0xa5, 0x38, 0xa6, 0x57, 0x18, 0x8f, 0xd0, 0x3b, 0x27, 0x65, 0xfd, 0x39, 0xe9, 0x0c, + 0x8c, 0x3b, 0xca, 0x56, 0xcb, 0x2d, 0x07, 0x7b, 0xa0, 0x8b, 0xcc, 0x2d, 0xa5, 0xc9, 0xd6, 0xc8, + 0xb2, 0x12, 0x7b, 0x20, 0x55, 0xe2, 0x11, 0xaf, 0x4c, 0x3b, 0xfc, 0x89, 0x94, 0x97, 0x05, 0x79, + 0x6e, 0xe1, 0xa6, 0x6e, 0xd0, 0x0e, 0x90, 0x94, 0x80, 0x92, 0x96, 0x08, 0x05, 0x3d, 0x07, 0x59, + 0x96, 0x01, 0x1b, 0x1a, 0xed, 0x05, 0x49, 0x29, 0x43, 0x09, 0x15, 0xf7, 0xc0, 0x14, 0x77, 0x44, + 0xfe, 0x44, 0x80, 0x69, 0x16, 0xa1, 0xc3, 0xa2, 0x43, 0xe3, 0x9b, 0x95, 0x5f, 0x87, 0xac, 0xa6, + 0x38, 0x0a, 0x3b, 0xaf, 0x98, 0x18, 0x7a, 0x5e, 0xd1, 0x35, 0x93, 0x24, 0x3f, 0x3d, 0xb3, 0x88, + 0x20, 0x45, 0xfe, 0xb3, 0x73, 0x9e, 0x12, 0xfd, 0xef, 0xc7, 0x3a, 0x04, 0x8b, 0x1b, 0xa7, 0x97, + 0xf2, 0xc1, 0x38, 0x4c, 0x56, 0xf6, 0x3b, 0xa6, 0x15, 0xf3, 0x22, 0xda, 0x04, 0x07, 0xbf, 0x43, + 0x36, 0x63, 0x7b, 0xac, 0xa0, 0xbb, 0xcf, 0xc9, 0x19, 0xd1, 0x12, 0x00, 0x0b, 0xcb, 0xa4, 0xd1, + 0x3a, 0xc9, 0x13, 0x6c, 0x49, 0x51, 0x36, 0x42, 0x45, 0x6b, 0x90, 0x6b, 0xef, 0xaa, 0xaa, 0xbc, + 0xad, 0xb7, 0x1c, 0x1e, 0xc8, 0x16, 0x1d, 0x41, 0xbd, 0xfa, 0xa8, 0x5c, 0xbe, 0x47, 0x33, 0xb1, + 0x78, 0x32, 0xff, 0x59, 0x02, 0x22, 0x81, 0xfd, 0x47, 0x2f, 0x01, 0x3f, 0x5f, 0x22, 0xdb, 0xee, + 0x69, 0xb1, 0xa5, 0xc9, 0xa3, 0xc3, 0xb9, 0xac, 0x44, 0xa9, 0xf5, 0x7a, 0x43, 0xca, 0xb2, 0x0c, + 0x75, 0xdb, 0x41, 0x97, 0x61, 0xd2, 0x6c, 0xeb, 0x8e, 0xec, 0xfa, 0x01, 0xdc, 0x75, 0xca, 0x13, + 0xa2, 0xeb, 0x27, 0xa0, 0x06, 0x5c, 0xc7, 0x06, 0x19, 0x0d, 0xb4, 0x9e, 0xec, 0xe4, 0x80, 0xac, + 0x3b, 0x6c, 0x44, 0xc9, 0x66, 0xc7, 0xd1, 0xdb, 0xfa, 0x33, 0xba, 0x1d, 0xcc, 0xb7, 0x62, 0x2e, + 0xb3, 0xec, 0xa4, 0x7e, 0xf4, 0x2c, 0x41, 0x8d, 0xe7, 0x5d, 0x0f, 0x64, 0x45, 0x5f, 0x11, 0xe8, + 0x91, 0x04, 0xa2, 0x48, 0x79, 0xeb, 0x40, 0x6e, 0x91, 0x49, 0x44, 0x77, 0x0e, 0xe4, 0x27, 0xbb, + 0xc5, 0x0c, 0x75, 0xd0, 0x3e, 0x1d, 0xd9, 0x20, 0x81, 0x7e, 0xb0, 0xe0, 0x36, 0xcb, 0xc1, 0x43, + 0xce, 0xbc, 0xb2, 0x5b, 0x31, 0x1c, 0xeb, 0x60, 0xe9, 0xfc, 0xd1, 0xe1, 0xdc, 0x4c, 0x7f, 0xea, + 0x23, 0x7a, 0xcc, 0xa1, 0x97, 0x65, 0xf6, 0xcb, 0x50, 0x1c, 0x24, 0x09, 0x89, 0xfe, 0x7e, 0x64, + 0x96, 0x6d, 0x43, 0xbe, 0x16, 0x5e, 0x21, 0x18, 0xa1, 0xdb, 0xb8, 0xab, 0x04, 0x89, 0xd7, 0xdc, + 0xd1, 0xfd, 0x35, 0x01, 0x26, 0x97, 0xba, 0xad, 0x27, 0xeb, 0x9d, 0x7a, 0xb7, 0xdd, 0x56, 0xac, + 0x03, 0x62, 0x18, 0xd8, 0xa8, 0xd4, 0x9f, 0xb1, 0x40, 0x96, 0x24, 0x1f, 0x76, 0xfa, 0x33, 0x4c, + 0x86, 0x1d, 0x0f, 0x5b, 0x27, 0x74, 0x16, 0x93, 0x7e, 0x19, 0x26, 0x29, 0x5c, 0x97, 0xb1, 0xe1, + 0x58, 0x3a, 0x66, 0xab, 0x50, 0x49, 0x29, 0x4f, 0x89, 0x15, 0x46, 0x43, 0x57, 0xa1, 0x60, 0x1f, + 0xd8, 0x0e, 0x6e, 0xcb, 0xec, 0x74, 0x39, 0xc3, 0x98, 0x49, 0x69, 0x92, 0x51, 0x25, 0x46, 0x2c, + 0xfd, 0x38, 0x09, 0x05, 0x57, 0xb3, 0x71, 0xba, 0xaf, 0x4b, 0x30, 0xbe, 0xad, 0xb7, 0xb0, 0x1b, + 0x4b, 0x70, 0x6d, 0x48, 0x83, 0xf2, 0x70, 0x64, 0xe2, 0x45, 0xb8, 0x00, 0x88, 0xb2, 0xc6, 0x31, + 0xc6, 0x66, 0x7f, 0x2d, 0x01, 0x29, 0xea, 0x31, 0xde, 0x86, 0x14, 0x35, 0x7b, 0xc2, 0x28, 0x66, + 0x8f, 0x66, 0xf5, 0xdc, 0x99, 0x44, 0xc0, 0x9d, 0x21, 0xbe, 0xc1, 0x8e, 0xf2, 0xea, 0xed, 0x3b, + 0x74, 0x7c, 0xe5, 0x25, 0xfe, 0x84, 0x96, 0x68, 0x90, 0x8b, 0x69, 0x39, 0x58, 0xe3, 0x9e, 0x5a, + 0x94, 0x65, 0x0a, 0x35, 0xbc, 0x6b, 0x62, 0x5d, 0x3e, 0x74, 0x01, 0x92, 0x64, 0xe0, 0x4e, 0xb0, + 0x0d, 0xf0, 0xa3, 0xc3, 0xb9, 0x24, 0x19, 0xb2, 0x84, 0x86, 0x6e, 0x41, 0x2e, 0x3c, 0x4a, 0x84, + 0x1b, 0x59, 0x66, 0x0b, 0x02, 0x3d, 0x1c, 0x5a, 0x5e, 0x0f, 0x66, 0x28, 0xe5, 0x41, 0x2a, 0x93, + 0x12, 0xc7, 0x4b, 0x7f, 0x9e, 0x82, 0xc9, 0x5a, 0x3b, 0x6e, 0x2b, 0xba, 0x18, 0x6e, 0xe1, 0x28, + 0xf7, 0x36, 0xf4, 0xd2, 0x88, 0x06, 0x0e, 0xcd, 0x47, 0xc9, 0x93, 0xcd, 0x47, 0x35, 0xe2, 0x51, + 0xf1, 0x13, 0xf4, 0xc9, 0x01, 0x9e, 0x6c, 0xf8, 0xfd, 0x0d, 0x62, 0x98, 0x24, 0xc2, 0xe3, 0x07, + 0xe8, 0xd3, 0x20, 0x86, 0x37, 0xa9, 0xe3, 0xc6, 0x7a, 0x59, 0x7a, 0xf4, 0x5e, 0x36, 0x81, 0x0d, + 0x8d, 0xf6, 0xb1, 0x67, 0xbc, 0x8b, 0xbd, 0x0e, 0x49, 0x4d, 0xb7, 0x86, 0x5c, 0xc7, 0x10, 0x3d, + 0xa7, 0x10, 0xa6, 0x63, 0xfa, 0x5a, 0x2a, 0xd8, 0xd7, 0x82, 0x38, 0x74, 0x76, 0x1d, 0xc0, 0xaf, + 0x17, 0x9a, 0x87, 0xb4, 0xd9, 0xd2, 0xdc, 0xd3, 0x05, 0x93, 0x4b, 0xd9, 0xa3, 0xc3, 0xb9, 0xf1, + 0xf5, 0x96, 0x56, 0x5b, 0x96, 0xc6, 0xcd, 0x96, 0x56, 0xd3, 0xe8, 0xd5, 0x03, 0x78, 0x4f, 0xf6, + 0x22, 0x91, 0xf2, 0xd2, 0x84, 0x81, 0xf7, 0x08, 0xea, 0xed, 0x89, 0x90, 0x20, 0x1d, 0xe7, 0x5b, + 0x02, 0x14, 0x5c, 0x1d, 0xc6, 0x6b, 0x1c, 0x32, 0x7a, 0x9b, 0x0f, 0x96, 0xe4, 0xc9, 0x06, 0x8b, + 0xcb, 0xc7, 0x0f, 0x48, 0x7e, 0x55, 0xe0, 0x51, 0xa8, 0x75, 0x55, 0x71, 0xc8, 0x7c, 0x18, 0x63, + 0x07, 0x7f, 0x11, 0x44, 0x4b, 0x31, 0x34, 0xb3, 0xad, 0x3f, 0xc3, 0x6c, 0xe1, 0xca, 0xe6, 0xfb, + 0x53, 0x53, 0x1e, 0x9d, 0xae, 0xcc, 0xb8, 0xeb, 0x6e, 0x3f, 0x13, 0x78, 0xc4, 0xaa, 0x57, 0x98, + 0x38, 0x95, 0xb6, 0x02, 0x69, 0x8b, 0xc5, 0xbd, 0xb1, 0x01, 0xf7, 0x72, 0x84, 0x90, 0xa8, 0xb7, + 0xb3, 0xb0, 0x32, 0xaf, 0xcb, 0x53, 0x11, 0xb3, 0x9f, 0x87, 0x71, 0x4a, 0x3e, 0x85, 0x59, 0xe4, + 0x9a, 0xff, 0x83, 0x04, 0x4c, 0x2f, 0x6a, 0x5a, 0xbd, 0xce, 0xbb, 0x5f, 0x7c, 0x7a, 0x77, 0x3d, + 0xcd, 0x84, 0xef, 0x69, 0xa2, 0x97, 0x01, 0x69, 0xba, 0xcd, 0x0e, 0xc6, 0xdb, 0x3b, 0x8a, 0x66, + 0xee, 0xf9, 0xbb, 0x96, 0xd3, 0x6e, 0x4a, 0xdd, 0x4d, 0x40, 0xef, 0x00, 0xf5, 0x8b, 0x64, 0xdb, + 0x51, 0xbc, 0xc5, 0xd5, 0x97, 0x4e, 0x72, 0x70, 0x82, 0xf9, 0x4d, 0xde, 0xa3, 0x94, 0x25, 0xe2, + 0xe8, 0x5f, 0x74, 0x03, 0x44, 0x9d, 0xe8, 0xd0, 0x91, 0x15, 0xdb, 0x8d, 0xb9, 0x67, 0x27, 0xf3, + 0x0b, 0x8c, 0xbe, 0x68, 0x07, 0x43, 0xe9, 0x59, 0x28, 0xb0, 0xaf, 0xa7, 0x38, 0xdd, 0xe3, 0xbf, + 0x14, 0xa0, 0xc0, 0xcf, 0x09, 0xc4, 0xbc, 0x37, 0x40, 0xeb, 0xc5, 0x7b, 0x3b, 0x7b, 0x40, 0xf7, + 0x20, 0xef, 0x1e, 0x55, 0xd8, 0xb6, 0xcc, 0xf6, 0x49, 0xe6, 0xe3, 0x1c, 0x67, 0xbc, 0x67, 0x99, + 0x6d, 0xff, 0xd8, 0xbd, 0x57, 0xf2, 0x58, 0xd7, 0x35, 0xe9, 0x19, 0x42, 0x26, 0x38, 0xee, 0x7d, + 0xc4, 0x5f, 0x86, 0x5e, 0xe8, 0xa2, 0x6f, 0xb0, 0xf8, 0x71, 0x2a, 0xe7, 0x67, 0x02, 0x14, 0xea, + 0xdd, 0x2d, 0x76, 0x89, 0x4b, 0x7c, 0x7a, 0xa9, 0x40, 0xb6, 0x85, 0xb7, 0x1d, 0xf9, 0x54, 0xe1, + 0xad, 0x19, 0xc2, 0x4a, 0x43, 0x7c, 0xef, 0x03, 0x58, 0xf4, 0x20, 0x0b, 0x95, 0x93, 0x3c, 0xa1, + 0x9c, 0x2c, 0xe5, 0xf5, 0x27, 0xb2, 0xd2, 0x9f, 0x25, 0x60, 0xca, 0xab, 0x6c, 0x9c, 0x86, 0xf8, + 0xff, 0x85, 0x8c, 0x4b, 0xf2, 0x14, 0xc6, 0x65, 0x9a, 0xef, 0xa7, 0x46, 0x1b, 0x98, 0x05, 0x98, + 0xa1, 0xb3, 0x8d, 0xac, 0x74, 0x3a, 0x2d, 0x1d, 0x6b, 0x32, 0xdb, 0xa8, 0x4b, 0xd1, 0x8d, 0xba, + 0x69, 0x9a, 0xb4, 0xc8, 0x52, 0x6a, 0x74, 0xd3, 0xee, 0x1e, 0xe4, 0xb7, 0x2d, 0x8c, 0x9f, 0x61, + 0x99, 0xba, 0xbd, 0x27, 0xd9, 0x36, 0xce, 0x31, 0xc6, 0x3a, 0xe1, 0xe3, 0x76, 0xfd, 0x5d, 0x98, + 0xa6, 0x0a, 0x8e, 0xfb, 0x30, 0x1d, 0x6f, 0x95, 0x7f, 0x17, 0x00, 0x05, 0xe5, 0x7f, 0x74, 0x0d, + 0x93, 0x88, 0xbd, 0x61, 0x5e, 0x02, 0xc4, 0x22, 0x88, 0x6c, 0xb9, 0x83, 0x2d, 0xd9, 0xc6, 0xaa, + 0xc9, 0xef, 0x1f, 0x11, 0x24, 0x91, 0xa7, 0x6c, 0x60, 0xab, 0x4e, 0xe9, 0xa5, 0xdf, 0xbf, 0x00, + 0x79, 0xae, 0x93, 0x4d, 0x83, 0xa0, 0xde, 0xdb, 0x90, 0x6c, 0xf2, 0x65, 0xbc, 0x5c, 0x24, 0xcc, + 0xf7, 0xef, 0x40, 0xaa, 0x8e, 0x49, 0x24, 0x2f, 0x61, 0xe9, 0x74, 0x9d, 0x88, 0x88, 0x57, 0x3f, + 0x3c, 0x32, 0xc8, 0xd2, 0xe9, 0x3a, 0xa8, 0x0e, 0x53, 0xaa, 0x7f, 0x07, 0x8c, 0x4c, 0xd8, 0x93, + 0x03, 0x57, 0x8e, 0x23, 0x6f, 0xd2, 0xa9, 0x8e, 0x49, 0x05, 0x35, 0x94, 0x80, 0xca, 0xc1, 0x4b, + 0x47, 0xd8, 0x74, 0x7a, 0x39, 0xf2, 0xc0, 0x5d, 0xf8, 0xc2, 0x93, 0xea, 0x58, 0xe0, 0x6e, 0x12, + 0xf4, 0x3a, 0xa4, 0x35, 0x7a, 0x99, 0x05, 0xef, 0xa1, 0x51, 0x9d, 0x28, 0x74, 0x7f, 0x48, 0x75, + 0x4c, 0xe2, 0x1c, 0xe8, 0x01, 0xe4, 0xd9, 0x3f, 0x76, 0x85, 0x01, 0x77, 0xd3, 0xaf, 0x0e, 0x96, + 0x10, 0xb0, 0xfd, 0xd5, 0x31, 0x29, 0xa7, 0xf9, 0x54, 0xf4, 0x49, 0x48, 0xd9, 0xaa, 0x62, 0xf0, + 0x15, 0xf0, 0x8b, 0x03, 0x4e, 0xac, 0xfb, 0xcc, 0x34, 0x37, 0xd1, 0x2b, 0x05, 0x09, 0x7e, 0x58, + 0x07, 0x0d, 0x2d, 0x1d, 0xb0, 0x22, 0x1f, 0x75, 0xb6, 0x92, 0xe8, 0x15, 0x87, 0x12, 0xd0, 0x7d, + 0xc8, 0x29, 0xc4, 0x65, 0x93, 0xe9, 0x09, 0xa7, 0x22, 0x0c, 0x3c, 0x83, 0xd2, 0x77, 0x3a, 0xad, + 0x4a, 0x0f, 0x74, 0xba, 0x44, 0x5f, 0x50, 0x1b, 0x5b, 0x4d, 0x5c, 0xcc, 0x0d, 0x17, 0x14, 0x0c, + 0x9d, 0xf0, 0x04, 0x51, 0x22, 0x5a, 0x85, 0xc9, 0x1d, 0x37, 0x7a, 0x9d, 0x86, 0xcb, 0xe4, 0x07, + 0xee, 0x84, 0x44, 0x44, 0xdf, 0x57, 0xc7, 0xa4, 0xfc, 0x4e, 0x80, 0x8c, 0x16, 0x20, 0xd1, 0x54, + 0x8b, 0x93, 0x54, 0xc6, 0xf3, 0xc3, 0x62, 0xcb, 0xab, 0x63, 0x52, 0xa2, 0xa9, 0x12, 0x28, 0xc6, + 0x02, 0x77, 0xf7, 0x8d, 0x62, 0x61, 0xa0, 0x25, 0x08, 0x87, 0x58, 0x57, 0xc7, 0x24, 0x1a, 0x88, + 0x4c, 0xde, 0xb7, 0x01, 0x05, 0x8b, 0xc5, 0x9e, 0xb8, 0x11, 0x65, 0xe2, 0xc0, 0xdd, 0xa1, 0xa8, + 0xa0, 0xb2, 0xea, 0x98, 0x34, 0x69, 0x05, 0xe9, 0xe8, 0x4b, 0x70, 0x26, 0x2c, 0x91, 0xf7, 0xc0, + 0xe9, 0x3e, 0xf3, 0x12, 0x2d, 0x37, 0xdc, 0x11, 0x91, 0xd5, 0x97, 0x88, 0x3e, 0x05, 0xe3, 0xac, + 0xd5, 0x10, 0x15, 0x19, 0xb5, 0xed, 0xd9, 0xd3, 0x60, 0x2c, 0x3f, 0x19, 0x14, 0x0e, 0x0f, 0xba, + 0x90, 0x5b, 0x66, 0xb3, 0x38, 0x33, 0x70, 0x50, 0xf4, 0x07, 0x91, 0x90, 0x41, 0xe1, 0xf8, 0x54, + 0xd2, 0xee, 0x16, 0x4b, 0xe1, 0x7b, 0xf4, 0x67, 0x06, 0xb6, 0x7b, 0x44, 0x2c, 0x46, 0x95, 0x06, + 0xcc, 0xfa, 0x64, 0x52, 0x34, 0x8b, 0xdd, 0xe4, 0x20, 0xd3, 0xb1, 0x76, 0x76, 0x60, 0xd1, 0xfa, + 0x6f, 0xa7, 0xa8, 0x52, 0xc7, 0xc8, 0xa3, 0xa2, 0x47, 0x20, 0xf2, 0x33, 0xd6, 0xfe, 0x5a, 0xe5, + 0x39, 0x2a, 0xef, 0xc5, 0x48, 0x93, 0x16, 0xb5, 0xa9, 0x5d, 0x1d, 0x93, 0xa6, 0xd4, 0x70, 0x0a, + 0x7a, 0x0c, 0xd3, 0x54, 0x9e, 0xac, 0xfa, 0x87, 0xe3, 0x8b, 0xc5, 0xbe, 0xe8, 0xb0, 0xc1, 0xe7, + 0xe8, 0x5d, 0xc9, 0xa2, 0xda, 0x93, 0x44, 0xba, 0xb1, 0x6e, 0xe8, 0x0e, 0xb5, 0xbe, 0xb3, 0x03, + 0xbb, 0x71, 0xf8, 0x2e, 0x2d, 0xd2, 0x8d, 0x75, 0x46, 0x21, 0xdd, 0xd8, 0xe1, 0x01, 0x1c, 0xbc, + 0x39, 0x9e, 0x1f, 0xd8, 0x8d, 0xa3, 0x22, 0x3d, 0x48, 0x37, 0x76, 0x82, 0x74, 0xd2, 0x8d, 0x99, + 0x81, 0xe8, 0x91, 0xfb, 0xc2, 0xc0, 0x6e, 0x3c, 0xf0, 0x8c, 0x21, 0xe9, 0xc6, 0x4a, 0x5f, 0x22, + 0x5a, 0x06, 0x60, 0x6e, 0x8b, 0x6e, 0x6c, 0x9b, 0xc5, 0x8b, 0x03, 0x27, 0x89, 0xde, 0x10, 0x0e, + 0x32, 0x49, 0xb4, 0x5c, 0x1a, 0x31, 0x64, 0xd4, 0xad, 0x96, 0xe9, 0xd6, 0x48, 0x71, 0x6e, 0xa0, + 0x21, 0xeb, 0xdb, 0x21, 0x21, 0x86, 0x6c, 0xcf, 0x23, 0x92, 0xd9, 0x86, 0x2d, 0xac, 0x15, 0xe7, + 0x07, 0xce, 0x36, 0xa1, 0x25, 0x65, 0x32, 0xdb, 0x30, 0x0e, 0xb4, 0x08, 0x59, 0x32, 0x9d, 0x1f, + 0x50, 0x33, 0x74, 0x69, 0xa0, 0x23, 0xda, 0x13, 0x11, 0x5e, 0x1d, 0x93, 0x32, 0x4f, 0x39, 0x89, + 0xbc, 0x9e, 0x2d, 0x55, 0x14, 0x4b, 0x03, 0x5f, 0x1f, 0x5a, 0x9e, 0x22, 0xaf, 0x67, 0x1c, 0x48, + 0x85, 0xb3, 0xac, 0xad, 0xf8, 0x11, 0x3f, 0x8b, 0x9f, 0x47, 0x2b, 0x5e, 0xa6, 0xa2, 0x06, 0x02, + 0xff, 0xc8, 0x93, 0x87, 0xd5, 0x31, 0x69, 0x46, 0xe9, 0x4f, 0x25, 0x03, 0x9e, 0x4f, 0x3d, 0x6c, + 0xb9, 0xa0, 0x78, 0x65, 0xe0, 0x80, 0x8f, 0x58, 0x60, 0x21, 0x03, 0x5e, 0x09, 0x90, 0xd9, 0x04, + 0xa4, 0xc9, 0xb6, 0xcd, 0x36, 0xd2, 0xae, 0x0e, 0x99, 0x80, 0x7a, 0xd6, 0x0c, 0xd8, 0x04, 0xa4, + 0xd5, 0x19, 0x27, 0x11, 0xa4, 0xb6, 0xb0, 0x62, 0x71, 0x33, 0x7b, 0x6d, 0xa0, 0xa0, 0xbe, 0xfb, + 0xa9, 0x88, 0x20, 0xd5, 0x23, 0x92, 0x09, 0xdb, 0x72, 0x2f, 0x67, 0xe0, 0x2e, 0xe1, 0xf5, 0x81, + 0x13, 0x76, 0xe4, 0x1d, 0x12, 0x64, 0xc2, 0xb6, 0x42, 0x09, 0xe8, 0xb3, 0x30, 0xc1, 0x31, 0x5b, + 0xf1, 0xc6, 0x10, 0x47, 0x35, 0x08, 0xc9, 0xc9, 0xb8, 0xe6, 0x3c, 0xcc, 0xca, 0x32, 0xac, 0xc8, + 0xaa, 0xf7, 0xe2, 0x10, 0x2b, 0xdb, 0x07, 0x62, 0x99, 0x95, 0xf5, 0xc9, 0xc4, 0xca, 0xb2, 0x7e, + 0xca, 0xe7, 0xba, 0x9b, 0x03, 0xad, 0x6c, 0x7f, 0x38, 0x3a, 0xb1, 0xb2, 0x4f, 0x7d, 0x2a, 0xa9, + 0x99, 0xcd, 0xd0, 0x52, 0xf1, 0x13, 0x03, 0x6b, 0x16, 0x06, 0x8f, 0xa4, 0x66, 0x9c, 0x87, 0x34, + 0x1b, 0x8b, 0x1c, 0x64, 0x9a, 0x7e, 0x69, 0xf0, 0x69, 0xda, 0x5e, 0x70, 0x51, 0x75, 0xef, 0x2e, + 0x65, 0x1a, 0xf6, 0x0c, 0x95, 0xc5, 0xcf, 0x0e, 0x72, 0x4d, 0xbd, 0x3c, 0xdc, 0x50, 0x45, 0x1d, + 0x8b, 0xf4, 0x0c, 0x55, 0x28, 0x91, 0x16, 0x95, 0x1d, 0xee, 0xa0, 0xe3, 0x7b, 0x61, 0xc8, 0xc1, + 0xdf, 0x9e, 0xf3, 0x36, 0xb4, 0xa8, 0x1e, 0xd1, 0x1f, 0x42, 0x5d, 0x76, 0x42, 0xbd, 0x78, 0x6b, + 0xf8, 0x10, 0x0a, 0x9f, 0x94, 0xf7, 0x86, 0x10, 0x27, 0x7b, 0x73, 0xa6, 0xeb, 0x61, 0xbc, 0x32, + 0x7c, 0xce, 0xec, 0x75, 0x2d, 0xd8, 0x9c, 0xc9, 0xaf, 0xf1, 0x9a, 0xe0, 0x3b, 0x55, 0xec, 0x6c, + 0xd3, 0x83, 0x54, 0x66, 0x4a, 0x14, 0x1f, 0xa4, 0x32, 0xe7, 0xc5, 0xe2, 0x83, 0x54, 0xe6, 0x82, + 0x38, 0xfb, 0x20, 0x95, 0x79, 0x4e, 0x7c, 0xbe, 0xf4, 0xab, 0x17, 0x60, 0xd2, 0x85, 0x55, 0x0c, + 0xa4, 0xdc, 0x09, 0x82, 0x94, 0x8b, 0x83, 0x40, 0x0a, 0x07, 0x62, 0x1c, 0xa5, 0xdc, 0x09, 0xa2, + 0x94, 0x8b, 0x83, 0x50, 0x8a, 0xcf, 0x43, 0x60, 0x4a, 0x63, 0x10, 0x4c, 0x79, 0x71, 0x04, 0x98, + 0xe2, 0x89, 0xea, 0xc5, 0x29, 0xcb, 0xfd, 0x38, 0xe5, 0xca, 0x70, 0x9c, 0xe2, 0x89, 0x0a, 0x00, + 0x95, 0x37, 0x7a, 0x80, 0xca, 0xa5, 0x21, 0x40, 0xc5, 0xe3, 0x77, 0x91, 0xca, 0x4a, 0x24, 0x52, + 0xb9, 0x76, 0x1c, 0x52, 0xf1, 0xe4, 0x84, 0xa0, 0xca, 0xab, 0x21, 0xa8, 0x32, 0x37, 0x10, 0xaa, + 0x78, 0xdc, 0x0c, 0xab, 0x34, 0x06, 0x61, 0x95, 0x17, 0x47, 0xc0, 0x2a, 0xbe, 0x72, 0x7b, 0xc0, + 0x4a, 0x35, 0x0a, 0xac, 0x5c, 0x3d, 0x06, 0xac, 0x78, 0xd2, 0x82, 0x68, 0xa5, 0x1a, 0x85, 0x56, + 0xae, 0x1e, 0x83, 0x56, 0x7a, 0x24, 0x31, 0xb8, 0xb2, 0x16, 0x0d, 0x57, 0xae, 0x1f, 0x0b, 0x57, + 0x3c, 0x69, 0x61, 0xbc, 0x72, 0x2b, 0x80, 0x57, 0x5e, 0x18, 0x80, 0x57, 0x3c, 0x56, 0x02, 0x58, + 0x3e, 0xd7, 0x07, 0x58, 0x4a, 0xc3, 0x00, 0x8b, 0xc7, 0xeb, 0x21, 0x96, 0xb7, 0x06, 0x20, 0x96, + 0x1b, 0xc7, 0x23, 0x16, 0x4f, 0x58, 0x0f, 0x64, 0x51, 0x86, 0x42, 0x96, 0x97, 0x47, 0x84, 0x2c, + 0x9e, 0xf4, 0x28, 0xcc, 0xf2, 0x5a, 0x18, 0xb3, 0xcc, 0x0f, 0xc6, 0x2c, 0x9e, 0x18, 0x0e, 0x5a, + 0x56, 0x22, 0x41, 0xcb, 0xb5, 0xe3, 0x40, 0x8b, 0x3f, 0x3e, 0x82, 0xa8, 0x65, 0x2d, 0x1a, 0xb5, + 0x5c, 0x3f, 0x16, 0xb5, 0xf8, 0xcd, 0x1f, 0x82, 0x2d, 0x2b, 0x91, 0xb0, 0xe5, 0xda, 0x71, 0xb0, + 0xc5, 0x2f, 0x5c, 0x10, 0xb7, 0xbc, 0x3d, 0x10, 0xb7, 0xdc, 0x1c, 0x05, 0xb7, 0x78, 0x42, 0xfb, + 0x80, 0xcb, 0x3b, 0x83, 0x81, 0xcb, 0x27, 0x4e, 0x70, 0x01, 0x58, 0x24, 0x72, 0xf9, 0x5c, 0x1f, + 0x72, 0x29, 0x0d, 0x43, 0x2e, 0x7e, 0x7f, 0x76, 0xa1, 0x8b, 0x32, 0x14, 0x68, 0xbc, 0x3c, 0x22, + 0xd0, 0xf0, 0x3b, 0x5f, 0x04, 0xd2, 0xa8, 0x44, 0x20, 0x8d, 0x2b, 0xc3, 0x91, 0x86, 0x6f, 0xe6, + 0x7d, 0xa8, 0x51, 0x8d, 0x82, 0x1a, 0x57, 0x8f, 0x81, 0x1a, 0xbe, 0x15, 0x0a, 0x60, 0x8d, 0x37, + 0x7a, 0xb0, 0xc6, 0xa5, 0x63, 0xa3, 0x1d, 0x02, 0x60, 0x63, 0xa9, 0x1f, 0x6c, 0x5c, 0x1e, 0x0a, + 0x36, 0x3c, 0x09, 0x3e, 0xda, 0x78, 0xa3, 0x07, 0x6d, 0x5c, 0x1a, 0x82, 0x36, 0xfc, 0x02, 0x70, + 0xb8, 0xa1, 0x0d, 0x87, 0x1b, 0x0b, 0xa3, 0xc2, 0x0d, 0x4f, 0x70, 0x24, 0xde, 0x58, 0x8b, 0xc6, + 0x1b, 0xd7, 0x47, 0xdc, 0xc5, 0xec, 0x03, 0x1c, 0xd5, 0x28, 0xc0, 0x71, 0xf5, 0x18, 0xc0, 0x11, + 0x9c, 0x43, 0x3c, 0xc4, 0x51, 0x8d, 0x42, 0x1c, 0x57, 0x8f, 0x41, 0x1c, 0xbe, 0xa4, 0x00, 0xe4, + 0x68, 0x0c, 0x82, 0x1c, 0x2f, 0x8e, 0x00, 0x39, 0xfc, 0x79, 0xb7, 0x07, 0x73, 0xbc, 0xd9, 0x8b, + 0x39, 0x4a, 0xc3, 0x30, 0x87, 0x3f, 0x22, 0x5d, 0xd0, 0xb1, 0x16, 0x0d, 0x3a, 0xae, 0x1f, 0x0b, + 0x3a, 0x82, 0x46, 0x32, 0x80, 0x3a, 0x56, 0x22, 0x51, 0xc7, 0xb5, 0xe3, 0x50, 0x87, 0x6f, 0x24, + 0x83, 0xb0, 0xe3, 0xcd, 0x5e, 0xd8, 0x51, 0x1a, 0x06, 0x3b, 0xfc, 0xca, 0xb9, 0xb8, 0xa3, 0x1a, + 0x85, 0x3b, 0xae, 0x1e, 0x83, 0x3b, 0xfc, 0xc6, 0x0b, 0x00, 0x0f, 0x65, 0x28, 0xf0, 0x78, 0x79, + 0x44, 0xe0, 0xd1, 0x63, 0xb8, 0xc2, 0xc8, 0xa3, 0x1a, 0x85, 0x3c, 0xae, 0x1e, 0x83, 0x3c, 0x02, + 0x85, 0xf5, 0xa1, 0xc7, 0x5a, 0x34, 0xf4, 0xb8, 0x7e, 0x2c, 0xf4, 0xe8, 0x19, 0x4d, 0x2e, 0xf6, + 0x58, 0x89, 0xc4, 0x1e, 0xd7, 0x8e, 0xc3, 0x1e, 0x3d, 0x13, 0xdf, 0x89, 0xc0, 0xc7, 0x83, 0x54, + 0xe6, 0x79, 0xf1, 0x85, 0xd2, 0xcf, 0xc7, 0x21, 0x5d, 0x75, 0x23, 0x8a, 0x02, 0xb7, 0x7c, 0x08, + 0xa7, 0xb9, 0xe5, 0x03, 0x2d, 0x93, 0xb1, 0x42, 0x2d, 0xce, 0xf1, 0x17, 0x3a, 0xf5, 0x5f, 0x36, + 0xc4, 0x59, 0x4f, 0x71, 0x0c, 0x0e, 0xbd, 0x0a, 0x93, 0x5d, 0x1b, 0x5b, 0x72, 0xc7, 0xd2, 0x4d, + 0x4b, 0x77, 0x58, 0xc0, 0xb2, 0xb0, 0x24, 0x7e, 0x78, 0x38, 0x97, 0xdf, 0xb4, 0xb1, 0xb5, 0xc1, + 0xe9, 0x52, 0xbe, 0x1b, 0x78, 0x72, 0xbf, 0x3a, 0x32, 0x3e, 0xfa, 0x57, 0x47, 0xde, 0x02, 0xd1, + 0xc2, 0x8a, 0x16, 0x9a, 0xfb, 0xd9, 0xf5, 0x19, 0xd1, 0xad, 0x45, 0xa3, 0xf5, 0xdd, 0x9c, 0xf4, + 0x1a, 0x8d, 0x29, 0x2b, 0x4c, 0x44, 0xb7, 0xe1, 0x6c, 0x5b, 0xd9, 0xa7, 0x51, 0x58, 0xb2, 0xeb, + 0x4e, 0xd1, 0xc8, 0xaa, 0x0c, 0x8d, 0x18, 0x44, 0x6d, 0x65, 0x9f, 0x7e, 0xc2, 0x84, 0x25, 0xd1, + 0xcb, 0xcb, 0xaf, 0x42, 0x41, 0xd3, 0x6d, 0x47, 0x37, 0x54, 0xf7, 0xbe, 0x44, 0x76, 0xcf, 0xc6, + 0xa4, 0x4b, 0x65, 0xf7, 0x16, 0xde, 0x84, 0x69, 0x1e, 0x91, 0xea, 0x7f, 0xd4, 0x84, 0x02, 0x87, + 0x0c, 0x29, 0x05, 0x49, 0xf0, 0xbf, 0x66, 0x53, 0x86, 0xa9, 0xa6, 0xe2, 0xe0, 0x3d, 0xe5, 0x40, + 0x76, 0x0f, 0x0c, 0xe4, 0xe8, 0xf5, 0x63, 0xcf, 0x1d, 0x1d, 0xce, 0x4d, 0xde, 0x67, 0x49, 0x7d, + 0xe7, 0x06, 0x26, 0x9b, 0x81, 0x04, 0x0d, 0x2d, 0x42, 0x9e, 0x5e, 0x4a, 0x6c, 0xb2, 0xbb, 0xaf, + 0x39, 0x1c, 0x18, 0xb4, 0xc7, 0xc3, 0x6f, 0xc8, 0x96, 0xe8, 0x45, 0xc6, 0xee, 0x75, 0xd9, 0xd7, + 0x61, 0x4a, 0xb1, 0x0f, 0x0c, 0x95, 0x6a, 0x18, 0x1b, 0x76, 0xd7, 0xa6, 0x78, 0x20, 0x23, 0x15, + 0x28, 0xb9, 0xec, 0x52, 0xd1, 0x6b, 0x70, 0x41, 0xc3, 0xc4, 0xc1, 0x61, 0x5e, 0x84, 0x63, 0x9a, + 0xb2, 0xd9, 0xd2, 0x64, 0x7a, 0x86, 0x9e, 0x62, 0x81, 0x8c, 0x74, 0x96, 0x66, 0xa0, 0xfe, 0x43, + 0xc3, 0x34, 0xd7, 0x5b, 0x5a, 0x85, 0x24, 0xf2, 0xdb, 0x15, 0x7f, 0x4b, 0x80, 0x7c, 0x28, 0xc4, + 0xfb, 0x8d, 0x9e, 0x4d, 0xd0, 0x0b, 0xd1, 0x28, 0x66, 0x50, 0x38, 0x5e, 0x86, 0xb7, 0x9d, 0x1b, + 0x20, 0x34, 0x37, 0xd8, 0x0b, 0xa6, 0x58, 0xdf, 0xdd, 0x80, 0x77, 0xd9, 0x5e, 0x4f, 0xfd, 0xce, + 0x7b, 0x73, 0x63, 0xa5, 0x9f, 0x26, 0x61, 0x32, 0x1c, 0xca, 0x5d, 0xeb, 0x29, 0x57, 0x94, 0x95, + 0x09, 0x71, 0x2c, 0x0c, 0xb9, 0x64, 0x2a, 0xeb, 0x5f, 0x82, 0xcc, 0x8a, 0x39, 0x3f, 0x64, 0xab, + 0x37, 0x58, 0x4e, 0x9f, 0x71, 0xf6, 0xfb, 0x09, 0xcf, 0x66, 0x2c, 0xc0, 0x38, 0x53, 0xb8, 0x30, + 0xf0, 0x68, 0x1d, 0xd5, 0xb9, 0xc4, 0xb2, 0x11, 0x1b, 0xd3, 0x38, 0xd5, 0x4d, 0x42, 0xfe, 0xe9, + 0xea, 0x93, 0x7f, 0x29, 0x88, 0xdf, 0x27, 0x35, 0x7e, 0xb2, 0xfb, 0xa4, 0xd8, 0x56, 0x6e, 0xab, + 0x85, 0x55, 0x87, 0x7f, 0x1a, 0xca, 0x3d, 0xc6, 0x7d, 0xa5, 0x57, 0x04, 0xff, 0x90, 0xd4, 0x82, + 0xc4, 0x3f, 0x24, 0x15, 0x88, 0xd9, 0x2a, 0x78, 0x22, 0xe8, 0x90, 0x64, 0x91, 0x7d, 0xbc, 0xa9, + 0xbf, 0x25, 0x80, 0x48, 0x07, 0xe0, 0x3d, 0x8c, 0xb5, 0x58, 0x7a, 0xa1, 0x1b, 0x4e, 0x96, 0x18, + 0x3d, 0xca, 0x36, 0x74, 0x29, 0x75, 0x32, 0x7c, 0x29, 0x75, 0xe9, 0x3d, 0x01, 0x0a, 0x5e, 0x09, + 0xd9, 0x67, 0x53, 0x86, 0x5c, 0x13, 0x75, 0xba, 0x2f, 0x89, 0xb8, 0x67, 0x53, 0x47, 0xfa, 0x7e, + 0x4b, 0xf0, 0x6c, 0x2a, 0xfb, 0xea, 0xc5, 0xef, 0x0a, 0x30, 0xe3, 0x15, 0xb1, 0xec, 0x9f, 0x3b, + 0x3c, 0x45, 0xc0, 0xb1, 0x44, 0xbf, 0x28, 0x45, 0xb0, 0x36, 0x3d, 0x14, 0x3c, 0x52, 0xf7, 0x44, + 0x3c, 0x64, 0x01, 0x38, 0x86, 0xd7, 0x1a, 0x75, 0xfa, 0xad, 0x29, 0xf6, 0xdf, 0x2e, 0xdd, 0x0b, + 0x28, 0x90, 0x8e, 0x04, 0xa2, 0xa5, 0x91, 0x86, 0x8c, 0xab, 0x25, 0x9a, 0xb9, 0xf4, 0xc3, 0x60, + 0x4b, 0x54, 0x76, 0x89, 0xef, 0x76, 0x17, 0x92, 0xbb, 0x4a, 0x6b, 0x58, 0xc4, 0x46, 0xa8, 0xe5, + 0x24, 0x92, 0x1b, 0xdd, 0x0b, 0x1d, 0xd7, 0x4c, 0x0c, 0xf6, 0x33, 0xfa, 0x55, 0x1a, 0x3a, 0xd6, + 0xf9, 0x29, 0xb7, 0x16, 0xc9, 0xe3, 0x5f, 0x1f, 0xb4, 0x00, 0xaf, 0xa7, 0xde, 0x7f, 0x6f, 0x4e, + 0xb8, 0x59, 0x87, 0x99, 0x88, 0xb9, 0x11, 0x15, 0x00, 0x02, 0x57, 0x55, 0xf3, 0x0f, 0x5a, 0x2d, + 0x2e, 0xcb, 0x9b, 0x6b, 0xe5, 0xf5, 0xd5, 0xd5, 0x5a, 0xa3, 0x51, 0x59, 0x16, 0x05, 0x24, 0x42, + 0x3e, 0x74, 0xd1, 0x35, 0xff, 0x8e, 0xd5, 0xcd, 0xff, 0x03, 0xe0, 0x5f, 0x76, 0x4f, 0x64, 0xad, + 0x54, 0x1e, 0xcb, 0x8f, 0x16, 0x1f, 0x6e, 0x56, 0xea, 0xe2, 0x18, 0x42, 0x50, 0x58, 0x5a, 0x6c, + 0x94, 0xab, 0xb2, 0x54, 0xa9, 0x6f, 0xac, 0xaf, 0xd5, 0x2b, 0xa2, 0xc0, 0xf9, 0x96, 0x21, 0x1f, + 0x3c, 0xd8, 0x8a, 0x66, 0x60, 0xaa, 0x5c, 0xad, 0x94, 0x57, 0xe4, 0x47, 0xb5, 0x45, 0xf9, 0xad, + 0xcd, 0xca, 0x66, 0x45, 0x1c, 0xa3, 0x45, 0xa3, 0xc4, 0x7b, 0x9b, 0x0f, 0x1f, 0x8a, 0x02, 0x9a, + 0x82, 0x1c, 0x7b, 0xa6, 0x97, 0x62, 0x8b, 0x89, 0x9b, 0xab, 0x90, 0x0b, 0xdc, 0x96, 0x45, 0x5e, + 0xb7, 0xb1, 0x59, 0xaf, 0xca, 0x8d, 0xda, 0x6a, 0xa5, 0xde, 0x58, 0x5c, 0xdd, 0x60, 0x32, 0x28, + 0x6d, 0x71, 0x69, 0x5d, 0x6a, 0x88, 0x82, 0xf7, 0xdc, 0x58, 0xdf, 0x2c, 0x57, 0xbd, 0xcf, 0x71, + 0xa5, 0x32, 0x49, 0x31, 0x79, 0xf3, 0x29, 0x9c, 0x1f, 0x70, 0xba, 0x13, 0xe5, 0x60, 0x62, 0xd3, + 0xa0, 0x37, 0xff, 0x88, 0x63, 0x68, 0x32, 0x70, 0xc0, 0x53, 0x14, 0x50, 0x86, 0x1d, 0xdd, 0x13, + 0x13, 0x28, 0x0d, 0x89, 0xfa, 0x5d, 0x31, 0x49, 0x0a, 0x1a, 0x38, 0x1f, 0x29, 0xa6, 0x50, 0x96, + 0x1f, 0x1e, 0x13, 0xc7, 0x51, 0xde, 0x3f, 0xbd, 0x25, 0xa6, 0x6f, 0x5e, 0x82, 0xc0, 0xd1, 0x16, + 0x04, 0x90, 0x7e, 0xa8, 0x38, 0xd8, 0x76, 0xc4, 0x31, 0x34, 0x01, 0xc9, 0xc5, 0x56, 0x4b, 0x14, + 0xee, 0xfc, 0xa9, 0x00, 0x19, 0xf7, 0x5e, 0x67, 0xf4, 0x10, 0xc6, 0x19, 0x00, 0x9f, 0x1b, 0x3c, + 0x23, 0x51, 0xa3, 0x36, 0x3b, 0x7f, 0xdc, 0x94, 0x55, 0x1a, 0x43, 0x6f, 0xf3, 0x6f, 0xeb, 0x91, + 0x1e, 0x83, 0x2e, 0x0f, 0xeb, 0x4f, 0xae, 0xd4, 0xe1, 0x9d, 0x8e, 0x8c, 0x91, 0xd2, 0xd8, 0x2b, + 0xc2, 0xd2, 0x8b, 0xef, 0xff, 0xf8, 0xe2, 0xd8, 0xfb, 0x47, 0x17, 0x85, 0x0f, 0x8e, 0x2e, 0x0a, + 0x3f, 0x3a, 0xba, 0x28, 0xfc, 0xeb, 0xd1, 0x45, 0xe1, 0x37, 0x7f, 0x72, 0x71, 0xec, 0x83, 0x9f, + 0x5c, 0x1c, 0xfb, 0xd1, 0x4f, 0x2e, 0x8e, 0xbd, 0x33, 0xc1, 0xb9, 0xb7, 0xd2, 0xf4, 0x33, 0x7f, + 0x77, 0xff, 0x2b, 0x00, 0x00, 0xff, 0xff, 0xd5, 0x8a, 0x59, 0x79, 0xeb, 0x70, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index af73a3d64aa5..ff795becd42f 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1046,6 +1046,8 @@ message ResolveIntentRequest { // Optionally poison the abort span for the transaction the intent's // range. bool poison = 4; + // The list of ignored seqnum ranges as per the Transaction record. + repeated storage.engine.enginepb.IgnoredSeqNumRange ignored_seqnums = 5 [(gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums"]; } // A ResolveIntentResponse is the return value from the @@ -1073,6 +1075,8 @@ message ResolveIntentRangeRequest { // transaction. If present, this value can be used to optimize the // iteration over the span to find intents to resolve. util.hlc.Timestamp min_timestamp = 5 [(gogoproto.nullable) = false]; + // The list of ignored seqnum ranges as per the Transaction record. + repeated storage.engine.enginepb.IgnoredSeqNumRange ignored_seqnums = 6 [(gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums"]; } // A ResolveIntentRangeResponse is the return value from the diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index caa85849538b..e31be383f7a9 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -935,6 +935,7 @@ func (t *Transaction) Restart( t.CommitTimestampFixed = false t.IntentSpans = nil t.InFlightWrites = nil + t.IgnoredSeqNums = nil } // BumpEpoch increments the transaction's epoch, allowing for an in-place @@ -984,6 +985,7 @@ func (t *Transaction) Update(o *Transaction) { t.Sequence = o.Sequence t.IntentSpans = o.IntentSpans t.InFlightWrites = o.InFlightWrites + t.IgnoredSeqNums = o.IgnoredSeqNums } else if t.Epoch == o.Epoch { // Forward all epoch-scoped state. switch t.Status { @@ -1020,6 +1022,9 @@ func (t *Transaction) Update(o *Transaction) { if len(o.InFlightWrites) > 0 { t.InFlightWrites = o.InFlightWrites } + if len(o.IgnoredSeqNums) > 0 { + t.IgnoredSeqNums = o.IgnoredSeqNums + } } else /* t.Epoch > o.Epoch */ { // Ignore epoch-specific state from previous epoch. if o.Status == COMMITTED { @@ -1084,6 +1089,9 @@ func (t Transaction) String() string { if nw := len(t.InFlightWrites); t.Status != PENDING && nw > 0 { fmt.Fprintf(&buf, " ifw=%d", nw) } + if ni := len(t.IgnoredSeqNums); ni > 0 { + fmt.Fprintf(&buf, " isn=%d", ni) + } return buf.String() } @@ -1104,6 +1112,9 @@ func (t Transaction) SafeMessage() string { if nw := len(t.InFlightWrites); t.Status != PENDING && nw > 0 { fmt.Fprintf(&buf, " ifw=%d", nw) } + if ni := len(t.IgnoredSeqNums); ni > 0 { + fmt.Fprintf(&buf, " isn=%d", ni) + } return buf.String() } @@ -1772,11 +1783,8 @@ func (l *Lease) Equal(that interface{}) bool { func AsIntents(spans []Span, txn *Transaction) []Intent { ret := make([]Intent, len(spans)) for i := range spans { - ret[i] = Intent{ - Span: spans[i], - Txn: txn.TxnMeta, - Status: txn.Status, - } + ret[i] = Intent{Span: spans[i]} + ret[i].SetTxn(*txn) } return ret } @@ -2117,3 +2125,28 @@ func init() { enginepb.FormatBytesAsKey = func(k []byte) string { return Key(k).String() } enginepb.FormatBytesAsValue = func(v []byte) string { return Value{RawBytes: v}.PrettyPrint() } } + +// MakeIntent makes an intent from the given span and txn. +func MakeIntent(txn Transaction, span Span) Intent { + intent := Intent{Span: span} + intent.SetTxn(txn) + return intent +} + +// MakeErrorIntent makes an intent in the pending state with the given +// span and txn. This is suitable for use when constructing +// WriteIntentError. +func MakeErrorIntent(txn enginepb.TxnMeta, span Span) Intent { + return Intent{ + Span: span, + Txn: txn, + Status: PENDING, + } +} + +// SetTxn updates the transaction details in the intent. +func (i *Intent) SetTxn(txn Transaction) { + i.Txn = txn.TxnMeta + i.Status = txn.Status + i.IgnoredSeqNums = txn.IgnoredSeqNums +} diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 84347552f63c..f3b9ef173932 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -90,7 +90,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{0} + return fileDescriptor_data_1743ea2cc9b724e3, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -114,7 +114,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{1} + return fileDescriptor_data_1743ea2cc9b724e3, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -166,7 +166,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{2} + return fileDescriptor_data_1743ea2cc9b724e3, []int{2} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -183,7 +183,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{0} + return fileDescriptor_data_1743ea2cc9b724e3, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -235,7 +235,7 @@ func (m *Value) Reset() { *m = Value{} } func (m *Value) String() string { return proto.CompactTextString(m) } func (*Value) ProtoMessage() {} func (*Value) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{1} + return fileDescriptor_data_1743ea2cc9b724e3, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -271,7 +271,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{2} + return fileDescriptor_data_1743ea2cc9b724e3, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -309,7 +309,7 @@ func (m *StoreIdent) Reset() { *m = StoreIdent{} } func (m *StoreIdent) String() string { return proto.CompactTextString(m) } func (*StoreIdent) ProtoMessage() {} func (*StoreIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{3} + return fileDescriptor_data_1743ea2cc9b724e3, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -349,7 +349,7 @@ func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } func (*SplitTrigger) ProtoMessage() {} func (*SplitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{4} + return fileDescriptor_data_1743ea2cc9b724e3, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -397,7 +397,7 @@ func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } func (*MergeTrigger) ProtoMessage() {} func (*MergeTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{5} + return fileDescriptor_data_1743ea2cc9b724e3, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -467,7 +467,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{6} + return fileDescriptor_data_1743ea2cc9b724e3, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -509,7 +509,7 @@ func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } func (*ModifiedSpanTrigger) ProtoMessage() {} func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{7} + return fileDescriptor_data_1743ea2cc9b724e3, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -555,7 +555,7 @@ func (m *StickyBitTrigger) Reset() { *m = StickyBitTrigger{} } func (m *StickyBitTrigger) String() string { return proto.CompactTextString(m) } func (*StickyBitTrigger) ProtoMessage() {} func (*StickyBitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{8} + return fileDescriptor_data_1743ea2cc9b724e3, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -594,7 +594,7 @@ func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } func (*InternalCommitTrigger) ProtoMessage() {} func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{9} + return fileDescriptor_data_1743ea2cc9b724e3, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -663,7 +663,7 @@ func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } func (*ObservedTimestamp) ProtoMessage() {} func (*ObservedTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{10} + return fileDescriptor_data_1743ea2cc9b724e3, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -868,12 +868,19 @@ type Transaction struct { // treated as immutable and all updates should be performed on a copy of the // slice. InFlightWrites []SequencedWrite `protobuf:"bytes,17,rep,name=in_flight_writes,json=inFlightWrites,proto3" json:"in_flight_writes"` + // A list of ignored seqnum ranges. + // + // The user code must guarantee this list to be non-overlapping, + // non-contiguous (i.e. it must coalesce ranges to avoid situations + // where a range's end seqnum is equal to the next range's start + // seqnum), and sorted in seqnum order. + IgnoredSeqNums []enginepb.IgnoredSeqNumRange `protobuf:"bytes,18,rep,name=ignored_seqnums,json=ignoredSeqnums,proto3" json:"ignored_seqnums"` } func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{11} + return fileDescriptor_data_1743ea2cc9b724e3, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -923,7 +930,7 @@ func (m *TransactionRecord) Reset() { *m = TransactionRecord{} } func (m *TransactionRecord) String() string { return proto.CompactTextString(m) } func (*TransactionRecord) ProtoMessage() {} func (*TransactionRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{12} + return fileDescriptor_data_1743ea2cc9b724e3, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -949,17 +956,29 @@ func (m *TransactionRecord) XXX_DiscardUnknown() { var xxx_messageInfo_TransactionRecord proto.InternalMessageInfo // A Intent is a Span together with a Transaction metadata and its status. +// +// Intents are used for two separate purposes: +// - on the return path of e.g. scans, to report the existence of a +// write intent on a key; +// - as input argument to intent resolution, to pass the current txn status, +// timestamps and ignored seqnum ranges to the resolution algorithm. +// Only in the latter case are the TxnMeta, status and ignored seqnum +// ranges guaranteed to be consistent with the latest txn's state. +// +// Note: avoid constructing Intent directly; consider using +// MakeIntent() or MakeErrorIntent() instead. type Intent struct { - Span `protobuf:"bytes,1,opt,name=span,proto3,embedded=span" json:"span"` - Txn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn,proto3" json:"txn"` - Status TransactionStatus `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` + Span `protobuf:"bytes,1,opt,name=span,proto3,embedded=span" json:"span"` + Txn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn,proto3" json:"txn"` + Status TransactionStatus `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` + IgnoredSeqNums []enginepb.IgnoredSeqNumRange `protobuf:"bytes,4,rep,name=ignored_seqnums,json=ignoredSeqnums,proto3" json:"ignored_seqnums"` } func (m *Intent) Reset() { *m = Intent{} } func (m *Intent) String() string { return proto.CompactTextString(m) } func (*Intent) ProtoMessage() {} func (*Intent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{13} + return fileDescriptor_data_1743ea2cc9b724e3, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -996,7 +1015,7 @@ func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } func (*SequencedWrite) ProtoMessage() {} func (*SequencedWrite) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{14} + return fileDescriptor_data_1743ea2cc9b724e3, []int{14} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1060,7 +1079,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{15} + return fileDescriptor_data_1743ea2cc9b724e3, []int{15} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1105,7 +1124,7 @@ func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } func (*AbortSpanEntry) ProtoMessage() {} func (*AbortSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{16} + return fileDescriptor_data_1743ea2cc9b724e3, []int{16} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1152,7 +1171,7 @@ func (m *LeafTxnInputState) Reset() { *m = LeafTxnInputState{} } func (m *LeafTxnInputState) String() string { return proto.CompactTextString(m) } func (*LeafTxnInputState) ProtoMessage() {} func (*LeafTxnInputState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{17} + return fileDescriptor_data_1743ea2cc9b724e3, []int{17} } func (m *LeafTxnInputState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1218,7 +1237,7 @@ func (m *LeafTxnFinalState) Reset() { *m = LeafTxnFinalState{} } func (m *LeafTxnFinalState) String() string { return proto.CompactTextString(m) } func (*LeafTxnFinalState) ProtoMessage() {} func (*LeafTxnFinalState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_485fa13f222a40c5, []int{18} + return fileDescriptor_data_1743ea2cc9b724e3, []int{18} } func (m *LeafTxnFinalState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1622,6 +1641,14 @@ func (this *Transaction) Equal(that interface{}) bool { return false } } + if len(this.IgnoredSeqNums) != len(that1.IgnoredSeqNums) { + return false + } + for i := range this.IgnoredSeqNums { + if !this.IgnoredSeqNums[i].Equal(&that1.IgnoredSeqNums[i]) { + return false + } + } return true } func (this *TransactionRecord) Equal(that interface{}) bool { @@ -1698,6 +1725,14 @@ func (this *Intent) Equal(that interface{}) bool { if this.Status != that1.Status { return false } + if len(this.IgnoredSeqNums) != len(that1.IgnoredSeqNums) { + return false + } + for i := range this.IgnoredSeqNums { + if !this.IgnoredSeqNums[i].Equal(&that1.IgnoredSeqNums[i]) { + return false + } + } return true } func (this *SequencedWrite) Equal(that interface{}) bool { @@ -2342,6 +2377,20 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) { i += n } } + if len(m.IgnoredSeqNums) > 0 { + for _, msg := range m.IgnoredSeqNums { + dAtA[i] = 0x92 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintData(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } return i, nil } @@ -2446,6 +2495,18 @@ func (m *Intent) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintData(dAtA, i, uint64(m.Status)) } + if len(m.IgnoredSeqNums) > 0 { + for _, msg := range m.IgnoredSeqNums { + dAtA[i] = 0x22 + i++ + i = encodeVarintData(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } return i, nil } @@ -2769,6 +2830,14 @@ func NewPopulatedTransaction(r randyData, easy bool) *Transaction { this.InFlightWrites[i] = *v14 } } + if r.Intn(10) != 0 { + v15 := r.Intn(5) + this.IgnoredSeqNums = make([]enginepb.IgnoredSeqNumRange, v15) + for i := 0; i < v15; i++ { + v16 := enginepb.NewPopulatedIgnoredSeqNumRange(r, easy) + this.IgnoredSeqNums[i] = *v16 + } + } if !easy && r.Intn(10) != 0 { } return this @@ -2776,25 +2845,25 @@ func NewPopulatedTransaction(r randyData, easy bool) *Transaction { func NewPopulatedTransactionRecord(r randyData, easy bool) *TransactionRecord { this := &TransactionRecord{} - v15 := enginepb.NewPopulatedTxnMeta(r, easy) - this.TxnMeta = *v15 + v17 := enginepb.NewPopulatedTxnMeta(r, easy) + this.TxnMeta = *v17 this.Status = TransactionStatus([]int32{0, 3, 1, 2}[r.Intn(4)]) - v16 := hlc.NewPopulatedTimestamp(r, easy) - this.LastHeartbeat = *v16 + v18 := hlc.NewPopulatedTimestamp(r, easy) + this.LastHeartbeat = *v18 if r.Intn(10) != 0 { - v17 := r.Intn(5) - this.IntentSpans = make([]Span, v17) - for i := 0; i < v17; i++ { - v18 := NewPopulatedSpan(r, easy) - this.IntentSpans[i] = *v18 + v19 := r.Intn(5) + this.IntentSpans = make([]Span, v19) + for i := 0; i < v19; i++ { + v20 := NewPopulatedSpan(r, easy) + this.IntentSpans[i] = *v20 } } if r.Intn(10) != 0 { - v19 := r.Intn(5) - this.InFlightWrites = make([]SequencedWrite, v19) - for i := 0; i < v19; i++ { - v20 := NewPopulatedSequencedWrite(r, easy) - this.InFlightWrites[i] = *v20 + v21 := r.Intn(5) + this.InFlightWrites = make([]SequencedWrite, v21) + for i := 0; i < v21; i++ { + v22 := NewPopulatedSequencedWrite(r, easy) + this.InFlightWrites[i] = *v22 } } if !easy && r.Intn(10) != 0 { @@ -2804,9 +2873,9 @@ func NewPopulatedTransactionRecord(r randyData, easy bool) *TransactionRecord { func NewPopulatedSequencedWrite(r randyData, easy bool) *SequencedWrite { this := &SequencedWrite{} - v21 := r.Intn(100) - this.Key = make(Key, v21) - for i := 0; i < v21; i++ { + v23 := r.Intn(100) + this.Key = make(Key, v23) + for i := 0; i < v23; i++ { this.Key[i] = byte(r.Intn(256)) } this.Sequence = github_com_cockroachdb_cockroach_pkg_storage_engine_enginepb.TxnSeq(r.Int31()) @@ -2820,13 +2889,13 @@ func NewPopulatedSequencedWrite(r randyData, easy bool) *SequencedWrite { func NewPopulatedLease(r randyData, easy bool) *Lease { this := &Lease{} - v22 := hlc.NewPopulatedTimestamp(r, easy) - this.Start = *v22 + v24 := hlc.NewPopulatedTimestamp(r, easy) + this.Start = *v24 if r.Intn(10) != 0 { this.Expiration = hlc.NewPopulatedTimestamp(r, easy) } - v23 := NewPopulatedReplicaDescriptor(r, easy) - this.Replica = *v23 + v25 := NewPopulatedReplicaDescriptor(r, easy) + this.Replica = *v25 if r.Intn(10) != 0 { this.DeprecatedStartStasis = hlc.NewPopulatedTimestamp(r, easy) } @@ -2848,13 +2917,13 @@ func NewPopulatedLease(r randyData, easy bool) *Lease { func NewPopulatedAbortSpanEntry(r randyData, easy bool) *AbortSpanEntry { this := &AbortSpanEntry{} - v24 := r.Intn(100) - this.Key = make(Key, v24) - for i := 0; i < v24; i++ { + v26 := r.Intn(100) + this.Key = make(Key, v26) + for i := 0; i < v26; i++ { this.Key[i] = byte(r.Intn(256)) } - v25 := hlc.NewPopulatedTimestamp(r, easy) - this.Timestamp = *v25 + v27 := hlc.NewPopulatedTimestamp(r, easy) + this.Timestamp = *v27 this.Priority = github_com_cockroachdb_cockroach_pkg_storage_engine_enginepb.TxnPriority(r.Int31()) if r.Intn(2) == 0 { this.Priority *= -1 @@ -2883,9 +2952,9 @@ func randUTF8RuneData(r randyData) rune { return rune(ru + 61) } func randStringData(r randyData) string { - v26 := r.Intn(100) - tmps := make([]rune, v26) - for i := 0; i < v26; i++ { + v28 := r.Intn(100) + tmps := make([]rune, v28) + for i := 0; i < v28; i++ { tmps[i] = randUTF8RuneData(r) } return string(tmps) @@ -2907,11 +2976,11 @@ func randFieldData(dAtA []byte, r randyData, fieldNumber int, wire int) []byte { switch wire { case 0: dAtA = encodeVarintPopulateData(dAtA, uint64(key)) - v27 := r.Int63() + v29 := r.Int63() if r.Intn(2) == 0 { - v27 *= -1 + v29 *= -1 } - dAtA = encodeVarintPopulateData(dAtA, uint64(v27)) + dAtA = encodeVarintPopulateData(dAtA, uint64(v29)) case 1: dAtA = encodeVarintPopulateData(dAtA, uint64(key)) dAtA = append(dAtA, byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256)), byte(r.Intn(256))) @@ -3186,6 +3255,12 @@ func (m *Transaction) Size() (n int) { n += 2 + l + sovData(uint64(l)) } } + if len(m.IgnoredSeqNums) > 0 { + for _, e := range m.IgnoredSeqNums { + l = e.Size() + n += 2 + l + sovData(uint64(l)) + } + } return n } @@ -3230,6 +3305,12 @@ func (m *Intent) Size() (n int) { if m.Status != 0 { n += 1 + sovData(uint64(m.Status)) } + if len(m.IgnoredSeqNums) > 0 { + for _, e := range m.IgnoredSeqNums { + l = e.Size() + n += 1 + l + sovData(uint64(l)) + } + } return n } @@ -5187,6 +5268,37 @@ func (m *Transaction) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 18: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IgnoredSeqNums", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthData + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IgnoredSeqNums = append(m.IgnoredSeqNums, enginepb.IgnoredSeqNumRange{}) + if err := m.IgnoredSeqNums[len(m.IgnoredSeqNums)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipData(dAtA[iNdEx:]) @@ -5507,6 +5619,37 @@ func (m *Intent) Unmarshal(dAtA []byte) error { break } } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IgnoredSeqNums", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthData + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IgnoredSeqNums = append(m.IgnoredSeqNums, enginepb.IgnoredSeqNumRange{}) + if err := m.IgnoredSeqNums[len(m.IgnoredSeqNums)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipData(dAtA[iNdEx:]) @@ -6391,142 +6534,145 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_485fa13f222a40c5) } - -var fileDescriptor_data_485fa13f222a40c5 = []byte{ - // 2129 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcd, 0x6f, 0xdb, 0xc8, - 0x15, 0x37, 0x4d, 0x4a, 0xa2, 0x9e, 0x64, 0x99, 0x9e, 0xc4, 0x89, 0x92, 0x45, 0xa5, 0x54, 0x5b, - 0xb4, 0x69, 0xb0, 0x2b, 0xa3, 0xc9, 0x76, 0xd1, 0x06, 0x45, 0x51, 0x7d, 0x25, 0xa1, 0x62, 0xd9, - 0xd9, 0x11, 0x9d, 0x45, 0xb2, 0x5d, 0xb0, 0x14, 0x39, 0x96, 0xd9, 0x48, 0xa4, 0x96, 0xa4, 0x1c, - 0xab, 0x7f, 0xc1, 0xa2, 0x97, 0xee, 0xb1, 0xe8, 0x29, 0x40, 0x6f, 0xbd, 0xf4, 0xda, 0x53, 0x2f, - 0x3d, 0x34, 0xa7, 0x22, 0xb7, 0x2e, 0x7a, 0x10, 0x5a, 0x05, 0x05, 0x7a, 0xee, 0x31, 0x40, 0x81, - 0x62, 0x66, 0x48, 0x8a, 0x89, 0x15, 0x43, 0x82, 0x77, 0x81, 0xbd, 0x48, 0xe4, 0x9b, 0xf7, 0x7e, - 0x6f, 0xde, 0xf7, 0x0c, 0x01, 0x79, 0xae, 0x61, 0x1e, 0x8d, 0x7a, 0x3b, 0x96, 0x11, 0x18, 0xd5, - 0x91, 0xe7, 0x06, 0x2e, 0xda, 0x32, 0x5d, 0xf3, 0x09, 0xa3, 0x57, 0xc3, 0xd5, 0xab, 0x97, 0x22, - 0xb6, 0x21, 0x09, 0x8c, 0x39, 0xeb, 0xd5, 0x8a, 0x1f, 0xb8, 0x9e, 0xd1, 0x27, 0x3b, 0xc4, 0xe9, - 0xdb, 0x4e, 0xf4, 0x47, 0xf9, 0x8e, 0x4d, 0x33, 0xe4, 0x79, 0xf7, 0x2c, 0x9e, 0x5b, 0x21, 0x53, - 0x71, 0x1c, 0xd8, 0x83, 0x9d, 0xa3, 0x81, 0xb9, 0x13, 0xd8, 0x43, 0xe2, 0x07, 0xc6, 0x70, 0x14, - 0xae, 0x5c, 0xec, 0xbb, 0x7d, 0x97, 0x3d, 0xee, 0xd0, 0x27, 0x4e, 0xad, 0x7c, 0x0a, 0x52, 0x77, - 0x64, 0x38, 0xe8, 0x0a, 0x88, 0x4f, 0xc8, 0xa4, 0x28, 0x5e, 0x13, 0xae, 0xe7, 0xeb, 0x99, 0x57, - 0xd3, 0xb2, 0x78, 0x9f, 0x4c, 0x30, 0xa5, 0xa1, 0x6b, 0x90, 0x21, 0x8e, 0xa5, 0xd3, 0x65, 0xe9, - 0xf5, 0xe5, 0x34, 0x71, 0xac, 0xfb, 0x64, 0x72, 0x3b, 0xff, 0xdb, 0x67, 0xe5, 0xb5, 0x3f, 0x3d, - 0x2b, 0x0b, 0xff, 0x79, 0x56, 0x16, 0xda, 0x92, 0x2c, 0x28, 0xeb, 0x6d, 0x49, 0x5e, 0x57, 0xc4, - 0xca, 0x10, 0x52, 0x0f, 0x8d, 0xc1, 0x98, 0xa0, 0x77, 0x20, 0xeb, 0x19, 0x4f, 0xf5, 0xde, 0x24, - 0x20, 0x7e, 0x51, 0xa0, 0x30, 0x58, 0xf6, 0x8c, 0xa7, 0x75, 0xfa, 0x8e, 0x6a, 0x90, 0x8d, 0x77, - 0x5b, 0x5c, 0xbf, 0x26, 0x5c, 0xcf, 0xdd, 0xfc, 0x56, 0x75, 0xee, 0x3c, 0x6a, 0x52, 0xf5, 0x68, - 0x60, 0x56, 0xb5, 0x88, 0xa9, 0x2e, 0x3d, 0x9f, 0x96, 0xd7, 0xf0, 0x5c, 0xea, 0xb6, 0x44, 0x55, - 0x57, 0x3e, 0x01, 0xf9, 0x3e, 0x99, 0x70, 0x8d, 0xa1, 0x45, 0xc2, 0x02, 0x8b, 0x3e, 0x80, 0xd4, - 0x31, 0xe5, 0x09, 0x75, 0x15, 0xab, 0xa7, 0x02, 0x55, 0x65, 0x18, 0xa1, 0x1a, 0xce, 0x5c, 0xf9, - 0xbb, 0x00, 0xd0, 0x0d, 0x5c, 0x8f, 0xa8, 0x16, 0x71, 0x02, 0xd4, 0x07, 0x30, 0x07, 0x63, 0x3f, - 0x20, 0x9e, 0x6e, 0x5b, 0xa1, 0x9a, 0x7b, 0x94, 0xff, 0x1f, 0xd3, 0xf2, 0xad, 0xbe, 0x1d, 0x1c, - 0x8d, 0x7b, 0x55, 0xd3, 0x1d, 0xee, 0xc4, 0xd8, 0x56, 0x6f, 0xfe, 0xbc, 0x33, 0x7a, 0xd2, 0xdf, - 0x61, 0xa1, 0x1a, 0x8f, 0x6d, 0xab, 0x7a, 0x70, 0xa0, 0x36, 0x67, 0xd3, 0x72, 0xb6, 0xc1, 0x01, - 0xd5, 0x26, 0xce, 0x86, 0xd8, 0xaa, 0x85, 0xde, 0x87, 0x8c, 0xe3, 0x5a, 0x84, 0x6a, 0xa1, 0xfb, - 0x4d, 0xd5, 0x2f, 0xce, 0xa6, 0xe5, 0xf4, 0x9e, 0x6b, 0x11, 0xb5, 0xf9, 0x2a, 0x7e, 0xc2, 0x69, - 0xca, 0xa4, 0x5a, 0xe8, 0x07, 0x20, 0xd3, 0x44, 0x61, 0xfc, 0x22, 0xe3, 0xbf, 0x34, 0x9b, 0x96, - 0x33, 0x7c, 0xe7, 0x54, 0x20, 0x7a, 0xc4, 0x19, 0x9f, 0x5b, 0x53, 0xf9, 0x83, 0x00, 0xf9, 0xee, - 0x68, 0x60, 0x07, 0x9a, 0x67, 0xf7, 0xfb, 0xc4, 0x43, 0x2d, 0xc8, 0x0e, 0xc8, 0x61, 0xa0, 0x5b, - 0xc4, 0x37, 0x99, 0x69, 0xb9, 0x9b, 0x95, 0x05, 0x4e, 0xc2, 0x86, 0xd3, 0x27, 0x4d, 0xe2, 0x9b, - 0x9e, 0x3d, 0x0a, 0x5c, 0x2f, 0x74, 0x97, 0x4c, 0x45, 0x29, 0x15, 0xdd, 0x05, 0xf0, 0xec, 0xfe, - 0x51, 0x88, 0xb3, 0xbe, 0x22, 0x4e, 0x96, 0xc9, 0x52, 0x32, 0x8f, 0x6e, 0x5b, 0x92, 0x45, 0x45, - 0xaa, 0xcc, 0xd6, 0x21, 0xdf, 0x21, 0x5e, 0x9f, 0x7c, 0x43, 0x37, 0x8b, 0x1c, 0x50, 0x38, 0x10, - 0xad, 0x4b, 0xdd, 0x0f, 0x8c, 0xc0, 0x67, 0x85, 0x93, 0xbb, 0xf9, 0x5e, 0x02, 0x2e, 0x2c, 0xe6, - 0x2a, 0xaf, 0xe2, 0x6a, 0x54, 0xcc, 0xd5, 0xce, 0xc3, 0x46, 0xa3, 0x4b, 0x65, 0xea, 0x97, 0x28, - 0xf0, 0x6c, 0x5a, 0x2e, 0x60, 0x8a, 0x16, 0xd3, 0x71, 0x81, 0xa1, 0x77, 0x8e, 0x4d, 0x93, 0xbd, - 0xa3, 0x3b, 0x90, 0x3f, 0xf4, 0x08, 0xf9, 0x15, 0xa1, 0xba, 0xbc, 0xa0, 0x98, 0x5a, 0xbe, 0x80, - 0x72, 0x5c, 0xb0, 0x4b, 0xe5, 0x5e, 0x73, 0xf2, 0x1f, 0x53, 0xb0, 0xdd, 0x38, 0xa2, 0x96, 0x62, - 0x32, 0x1a, 0xd8, 0xa6, 0xe1, 0x47, 0xde, 0x7e, 0x0c, 0x97, 0x2c, 0x32, 0xf2, 0x88, 0x69, 0x04, - 0xc4, 0xd2, 0x4d, 0xc6, 0xa3, 0x07, 0x93, 0x11, 0x61, 0xae, 0x2f, 0xdc, 0xfc, 0xce, 0x22, 0x97, - 0x71, 0x0c, 0x0e, 0xa8, 0x4d, 0x46, 0x04, 0x5f, 0x9c, 0x63, 0xcc, 0xa9, 0xe8, 0x11, 0xa0, 0x04, - 0xb6, 0xc7, 0xa5, 0xc2, 0x50, 0x9c, 0x81, 0x7b, 0x2a, 0x18, 0x5b, 0x73, 0x94, 0x90, 0x05, 0xfd, - 0x12, 0xde, 0x49, 0x40, 0x8f, 0x47, 0x56, 0x52, 0x85, 0x5f, 0x14, 0xaf, 0x89, 0x2b, 0xea, 0xb8, - 0x32, 0x87, 0x3b, 0xe0, 0x68, 0x91, 0xa7, 0x10, 0x81, 0xab, 0x09, 0x5d, 0x0e, 0x39, 0x09, 0x22, - 0x45, 0xb4, 0x26, 0x25, 0x56, 0x93, 0xd7, 0x67, 0xd3, 0xf2, 0xe5, 0x66, 0xcc, 0xb5, 0x47, 0x4e, - 0x82, 0x50, 0x9e, 0xd5, 0x68, 0x36, 0x7e, 0xc1, 0x97, 0xad, 0x85, 0x5c, 0x16, 0xfa, 0x10, 0x24, - 0x96, 0xaa, 0xa9, 0x65, 0x53, 0x15, 0x33, 0x7e, 0xd4, 0x83, 0xcb, 0xb6, 0x13, 0x10, 0xcf, 0x31, - 0x06, 0xba, 0x61, 0x59, 0x49, 0x37, 0xa4, 0x57, 0x76, 0xc3, 0x76, 0x04, 0x55, 0xa3, 0x48, 0xb1, - 0x0b, 0x0e, 0xe1, 0x4a, 0xac, 0xc3, 0x23, 0x43, 0xf7, 0x38, 0xa9, 0x25, 0xb3, 0xb2, 0x96, 0x78, - 0xc3, 0x98, 0x63, 0x45, 0x7a, 0x6e, 0xcb, 0x74, 0xf2, 0xb0, 0xd6, 0xff, 0x85, 0x00, 0x17, 0x3a, - 0xae, 0x65, 0x1f, 0xda, 0xc4, 0xa2, 0x13, 0x2d, 0xca, 0xd7, 0xf7, 0x00, 0xf9, 0x13, 0x3f, 0x20, - 0x43, 0xdd, 0x74, 0x9d, 0x43, 0xbb, 0xaf, 0xfb, 0x23, 0xc3, 0x61, 0xb9, 0x2a, 0x63, 0x85, 0xaf, - 0x34, 0xd8, 0x02, 0x1b, 0x83, 0x2d, 0x40, 0xac, 0xd7, 0x0e, 0xec, 0x63, 0xe2, 0x10, 0xdf, 0xe7, - 0xdc, 0x3c, 0x03, 0x2f, 0x2f, 0xd8, 0x30, 0x15, 0xc2, 0x0a, 0x15, 0xd9, 0x0d, 0x25, 0x28, 0x25, - 0x9c, 0x46, 0x3f, 0x07, 0xa5, 0x1b, 0xd8, 0xe6, 0x93, 0x49, 0x7d, 0xde, 0x59, 0xeb, 0x00, 0x3e, - 0xa3, 0xe9, 0x3d, 0x3b, 0x08, 0xbb, 0xd5, 0x72, 0xb3, 0xce, 0x8f, 0xa0, 0x42, 0xf4, 0x3f, 0x8b, - 0xb0, 0xad, 0x86, 0x6e, 0x69, 0xb8, 0xc3, 0xe1, 0x5c, 0x47, 0x13, 0x36, 0x7c, 0xda, 0xcd, 0xf5, - 0x80, 0x13, 0x42, 0x35, 0xe5, 0x85, 0xfb, 0x9f, 0x77, 0x7d, 0x9c, 0xf7, 0x93, 0x33, 0xa0, 0x09, - 0x1b, 0x43, 0xda, 0x66, 0x63, 0x94, 0xf5, 0xb7, 0xa2, 0x24, 0xdb, 0x31, 0xce, 0x0f, 0x93, 0xcd, - 0xf9, 0x17, 0x70, 0x39, 0xec, 0x11, 0x51, 0xf8, 0x63, 0x3c, 0x91, 0xe1, 0x5d, 0x5f, 0x80, 0xb7, - 0xb0, 0xf3, 0xe0, 0x6d, 0xf3, 0x2d, 0x0d, 0x69, 0x7b, 0x18, 0xc6, 0x9d, 0x45, 0x2b, 0xc6, 0xe7, - 0x3d, 0xf7, 0xbb, 0x8b, 0xf6, 0x7b, 0x3a, 0x4f, 0xf0, 0x85, 0xe1, 0x82, 0xe4, 0xf9, 0x08, 0xd0, - 0x3c, 0x5a, 0x31, 0x30, 0x2f, 0xb8, 0x77, 0x17, 0xb9, 0xf3, 0x8d, 0x70, 0x63, 0xc5, 0x7f, 0x83, - 0x72, 0x5b, 0xfe, 0x3c, 0x3c, 0x27, 0x55, 0x7e, 0x23, 0xc0, 0xd6, 0x7e, 0xcf, 0x27, 0xde, 0x31, - 0xb1, 0xe2, 0x68, 0x27, 0xa7, 0xbd, 0xb0, 0xc4, 0xb4, 0xff, 0x0a, 0x8e, 0x4e, 0x72, 0x74, 0x72, - 0xab, 0x4c, 0xd3, 0x90, 0xd3, 0x3c, 0xc3, 0xf1, 0x0d, 0x33, 0xb0, 0x5d, 0x07, 0xdd, 0x03, 0x89, - 0x9e, 0x53, 0xc3, 0xfc, 0xb9, 0xb1, 0xc4, 0xf4, 0xd2, 0x4e, 0x9c, 0x0e, 0x09, 0x8c, 0xba, 0x4c, - 0x95, 0xbc, 0x98, 0x96, 0x05, 0xcc, 0x10, 0x10, 0x02, 0xc9, 0x31, 0x86, 0xfc, 0xc0, 0x95, 0xc5, - 0xec, 0x19, 0xfd, 0x04, 0xd2, 0x74, 0x38, 0x8e, 0xf9, 0x74, 0x5c, 0x3c, 0x39, 0x12, 0xbb, 0xe9, - 0x32, 0x5e, 0x1c, 0xca, 0xa0, 0x36, 0x14, 0x06, 0x86, 0x1f, 0xe8, 0x47, 0xc4, 0xf0, 0x82, 0x1e, - 0x31, 0x56, 0x9a, 0x7b, 0x1b, 0x54, 0xf4, 0x5e, 0x24, 0x89, 0x74, 0x48, 0x74, 0x73, 0xdd, 0xf5, - 0xec, 0xbe, 0x3e, 0x77, 0x6a, 0x7a, 0x79, 0xd8, 0x44, 0xab, 0xde, 0xf7, 0xec, 0xfe, 0x3c, 0xa8, - 0xf7, 0x60, 0x63, 0x68, 0x9c, 0x24, 0x40, 0x33, 0xcb, 0x83, 0xe6, 0x87, 0xc6, 0xc9, 0x1c, 0xe9, - 0x13, 0xb8, 0xe0, 0x86, 0x39, 0x33, 0x87, 0xf3, 0x8b, 0xf2, 0x5b, 0x5b, 0xea, 0xa9, 0x0c, 0x0b, - 0x61, 0x91, 0xfb, 0xe6, 0x82, 0x8f, 0x7e, 0x06, 0x79, 0xda, 0x68, 0x9d, 0x80, 0x15, 0x92, 0x5f, - 0xcc, 0x31, 0xd4, 0xb7, 0xf5, 0xbd, 0xe8, 0x0c, 0xc1, 0x45, 0x28, 0xc5, 0x47, 0x15, 0xd8, 0x78, - 0xea, 0xd9, 0x01, 0xd1, 0x03, 0xd7, 0xd5, 0xdd, 0x81, 0x55, 0xcc, 0xb3, 0x46, 0x9b, 0x63, 0x44, - 0xcd, 0x75, 0xf7, 0x07, 0x16, 0x8d, 0x9c, 0x47, 0x8c, 0xc4, 0xf6, 0x8b, 0x9b, 0x2b, 0x44, 0x8e, - 0x8a, 0xce, 0xdd, 0xf1, 0x01, 0x5c, 0x32, 0x59, 0xef, 0x9b, 0xa3, 0xe9, 0x87, 0xf6, 0x09, 0xb1, - 0x8a, 0x0a, 0x53, 0x7c, 0x91, 0xaf, 0xc6, 0x02, 0x77, 0xe8, 0x1a, 0xfa, 0x08, 0x14, 0xdb, 0xd1, - 0x0f, 0x07, 0xec, 0x94, 0xc6, 0xb6, 0xe6, 0x17, 0xb7, 0x98, 0xad, 0xdf, 0x5e, 0x64, 0x2b, 0xf9, - 0x6c, 0x4c, 0x1c, 0x93, 0x58, 0x1f, 0x53, 0xce, 0x70, 0x1f, 0x05, 0xdb, 0xb9, 0xc3, 0xe4, 0x19, - 0xd1, 0x3f, 0x75, 0x05, 0x12, 0x15, 0xa9, 0x2d, 0xc9, 0x59, 0x05, 0xda, 0x92, 0xbc, 0xa1, 0x14, - 0xda, 0x92, 0x5c, 0x50, 0x36, 0x2b, 0x7f, 0x13, 0x61, 0x2b, 0x91, 0xd2, 0x98, 0x98, 0xae, 0x67, - 0x7d, 0x85, 0x65, 0xf6, 0xcd, 0x29, 0xa9, 0xf3, 0xa7, 0xd2, 0xd7, 0x10, 0x24, 0x39, 0x11, 0xa0, - 0x75, 0x45, 0x8c, 0xc3, 0x94, 0x56, 0x32, 0x6d, 0x49, 0xce, 0x28, 0x72, 0x5b, 0x92, 0x65, 0x25, - 0x1b, 0x07, 0x0e, 0x94, 0x5c, 0x5b, 0x92, 0xf3, 0xca, 0x46, 0x32, 0x88, 0x6d, 0x49, 0xde, 0x54, - 0x94, 0xb6, 0x24, 0x2b, 0xca, 0x56, 0xe5, 0xaf, 0x02, 0xa4, 0x55, 0xb6, 0x69, 0xf4, 0x43, 0x90, - 0xe2, 0xa3, 0xc5, 0x19, 0x96, 0x26, 0x42, 0x46, 0xd9, 0x51, 0x1d, 0xc4, 0xe0, 0x24, 0x3a, 0x62, - 0xac, 0x12, 0x7b, 0x6e, 0x22, 0x15, 0x4e, 0x84, 0x5d, 0x5c, 0x3d, 0xec, 0xe1, 0x71, 0xe2, 0x77, - 0x02, 0x14, 0x5e, 0x77, 0xe2, 0x59, 0x37, 0x68, 0x13, 0x64, 0x3f, 0x64, 0x0e, 0x2f, 0xa5, 0x77, - 0x5f, 0x4d, 0xcb, 0x8d, 0xa5, 0xae, 0xbd, 0x6f, 0xf9, 0x8c, 0x41, 0x0d, 0xeb, 0x92, 0xcf, 0x70, - 0x0c, 0x9c, 0x18, 0x4c, 0xff, 0x13, 0x21, 0xb5, 0x4b, 0x0c, 0x9f, 0xa0, 0x1f, 0x43, 0x8a, 0xdf, - 0x72, 0x56, 0x38, 0x3a, 0x71, 0x09, 0xf4, 0x29, 0x00, 0x39, 0x19, 0xd9, 0x9e, 0x41, 0x7d, 0xb0, - 0xdc, 0xac, 0x2c, 0xfd, 0x77, 0x5a, 0xbe, 0x9a, 0xb0, 0xe4, 0x76, 0xc5, 0x33, 0x1c, 0xcb, 0x19, - 0x0f, 0x06, 0x46, 0x6f, 0x40, 0x2a, 0x38, 0x01, 0x88, 0x9a, 0x90, 0x89, 0x6e, 0x2c, 0xe2, 0xca, - 0x37, 0x96, 0x48, 0x14, 0x8d, 0x21, 0x31, 0x44, 0xf8, 0x85, 0x8e, 0xfe, 0xfa, 0x76, 0x74, 0x87, - 0x3c, 0xe7, 0x8e, 0xb7, 0xe7, 0xe8, 0xec, 0xd6, 0xd7, 0x65, 0xd8, 0x68, 0x0f, 0x72, 0x23, 0xcf, - 0x1d, 0xb9, 0x3e, 0x1d, 0x2b, 0xfe, 0x72, 0x75, 0x5f, 0x98, 0x4d, 0xcb, 0xf0, 0x20, 0x94, 0xd2, - 0xba, 0x18, 0x22, 0x04, 0xcd, 0x47, 0x17, 0x21, 0x45, 0x46, 0xae, 0x79, 0xc4, 0xa6, 0xa7, 0x88, - 0xf9, 0x0b, 0x7a, 0x3f, 0x91, 0x35, 0x74, 0x02, 0x8a, 0xf5, 0xad, 0x57, 0xd3, 0xf2, 0x06, 0x8b, - 0x6c, 0x94, 0x7b, 0xc9, 0xf8, 0x47, 0x3d, 0xb5, 0x32, 0x13, 0xa0, 0x50, 0xeb, 0xb9, 0x1e, 0x6b, - 0x0d, 0x2d, 0x27, 0xf0, 0x26, 0x67, 0x25, 0xe7, 0xf9, 0xcf, 0x44, 0xe8, 0x08, 0xe4, 0x91, 0x67, - 0xbb, 0x9e, 0x1d, 0x4c, 0xc2, 0x8f, 0x28, 0xbb, 0xaf, 0xa6, 0xe5, 0x7b, 0xe7, 0xcd, 0xef, 0x07, - 0x21, 0x26, 0x8e, 0xd1, 0x13, 0x49, 0xfe, 0x6f, 0x01, 0xb6, 0x76, 0x89, 0x71, 0xa8, 0x9d, 0x38, - 0xaa, 0x33, 0x1a, 0xd3, 0xd0, 0x04, 0x04, 0x7d, 0xc8, 0xfb, 0x03, 0x4f, 0xf7, 0xd2, 0xd9, 0x85, - 0x9d, 0xec, 0x09, 0xdf, 0x83, 0x4d, 0x8f, 0x1c, 0x7a, 0xc4, 0x3f, 0xd2, 0x6d, 0xe7, 0xd8, 0x18, - 0xd8, 0x16, 0x73, 0xb9, 0x8c, 0x0b, 0x21, 0x59, 0xe5, 0xd4, 0x85, 0x7d, 0x56, 0x3e, 0x57, 0x9f, - 0x7d, 0xa3, 0xbb, 0x4a, 0x4a, 0xaa, 0x2d, 0xc9, 0x29, 0x25, 0xcd, 0x3b, 0x6d, 0xe5, 0xd7, 0xeb, - 0xb1, 0x9d, 0x77, 0x6c, 0xc7, 0x18, 0x9c, 0xcf, 0xce, 0x1f, 0x41, 0x31, 0xf9, 0x3d, 0xc2, 0x1d, - 0x0e, 0x0d, 0x87, 0xfe, 0x8f, 0x9d, 0x80, 0x47, 0x0e, 0x27, 0xbe, 0x57, 0x34, 0xf8, 0x72, 0x83, - 0xae, 0xa2, 0x3a, 0x6c, 0x44, 0x1e, 0xa2, 0x87, 0x0a, 0x5a, 0x60, 0x4b, 0xcc, 0xa8, 0x7c, 0x28, - 0x83, 0xa9, 0xc8, 0xd2, 0x5e, 0x8e, 0x5d, 0x12, 0xbb, 0x81, 0x8f, 0x9a, 0x1b, 0x7f, 0x11, 0x20, - 0xcb, 0xbe, 0x35, 0xb2, 0x0f, 0x20, 0x39, 0xc8, 0x1c, 0xec, 0xdd, 0xdf, 0xdb, 0xff, 0x78, 0x4f, - 0x59, 0x43, 0x19, 0x10, 0xd5, 0x3d, 0x4d, 0x11, 0x50, 0x16, 0x52, 0x77, 0x76, 0xf7, 0x6b, 0x9a, - 0xb2, 0x4e, 0x1f, 0xeb, 0x8f, 0xb4, 0x56, 0x57, 0x11, 0xd1, 0x05, 0xd8, 0x6c, 0xb6, 0x76, 0xd5, - 0x8e, 0xaa, 0xb5, 0x9a, 0x3a, 0x27, 0xca, 0x48, 0x06, 0x49, 0x53, 0x3b, 0x2d, 0x45, 0xa2, 0x50, - 0xcd, 0x56, 0x43, 0xed, 0xd4, 0x76, 0x95, 0x14, 0xda, 0x86, 0xad, 0x39, 0x6f, 0x44, 0xce, 0xa2, - 0x3c, 0xc8, 0xcd, 0x03, 0x5c, 0xd3, 0xd4, 0xfd, 0x3d, 0x25, 0x8d, 0x00, 0xd2, 0x54, 0x56, 0x7b, - 0xac, 0xe4, 0xa9, 0x1e, 0xed, 0xe0, 0xc1, 0x6e, 0x4b, 0x01, 0xca, 0x54, 0x57, 0xb5, 0x1a, 0xc6, - 0xb5, 0x47, 0x4a, 0x0e, 0x15, 0x00, 0x28, 0x53, 0xb7, 0x85, 0xd5, 0x56, 0x57, 0xb1, 0x2a, 0x74, - 0x6c, 0x66, 0x6e, 0xfc, 0x14, 0xb6, 0x4e, 0x7d, 0xe3, 0x41, 0x9b, 0x90, 0xab, 0x35, 0x9b, 0x3a, - 0x6e, 0x3d, 0xd8, 0x55, 0x1b, 0x35, 0x65, 0x0d, 0x21, 0x28, 0xe0, 0x56, 0x67, 0xff, 0x61, 0x2b, - 0xa6, 0x09, 0x57, 0xa5, 0xcf, 0x7f, 0x5f, 0x5a, 0xbb, 0xb1, 0xff, 0xda, 0xb1, 0x88, 0xcf, 0x27, - 0x6a, 0xc1, 0x83, 0xd6, 0x5e, 0x53, 0xdd, 0xbb, 0xab, 0xac, 0xd1, 0x97, 0xae, 0x56, 0xbb, 0x4b, - 0x5f, 0x44, 0xb4, 0x01, 0xd9, 0xc6, 0x7e, 0xa7, 0xa3, 0x6a, 0x5a, 0xab, 0xa9, 0x08, 0x74, 0xad, - 0x56, 0xdf, 0xc7, 0xf4, 0x65, 0x9d, 0x03, 0xd6, 0xbf, 0xff, 0xfc, 0x5f, 0xa5, 0xb5, 0xe7, 0xb3, - 0x92, 0xf0, 0x62, 0x56, 0x12, 0xbe, 0x9c, 0x95, 0x84, 0x7f, 0xce, 0x4a, 0xc2, 0x17, 0x2f, 0x4b, - 0x6b, 0x2f, 0x5e, 0x96, 0xd6, 0xbe, 0x7c, 0x59, 0x5a, 0x7b, 0x9c, 0x09, 0x63, 0xdb, 0x4b, 0xb3, - 0x0f, 0xe1, 0xb7, 0xfe, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x5b, 0x66, 0x0e, 0x27, 0xc2, 0x17, 0x00, - 0x00, +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_1743ea2cc9b724e3) } + +var fileDescriptor_data_1743ea2cc9b724e3 = []byte{ + // 2181 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcd, 0x8f, 0x1b, 0x49, + 0x15, 0x9f, 0x9e, 0x6e, 0xdb, 0xed, 0x67, 0x8f, 0xa7, 0xa7, 0x92, 0x49, 0x9c, 0xac, 0xb0, 0x83, + 0x17, 0x41, 0x88, 0x76, 0x3d, 0x22, 0xd9, 0x5d, 0x41, 0x84, 0x10, 0xfe, 0x4a, 0xd2, 0xce, 0xd8, + 0x93, 0x6d, 0xf7, 0x64, 0x95, 0x2c, 0xab, 0xa6, 0xdd, 0x5d, 0x63, 0x37, 0xb1, 0xbb, 0x9d, 0xee, + 0xf6, 0x64, 0xcc, 0x5f, 0xb0, 0xe2, 0xc2, 0x1e, 0x11, 0xa7, 0x48, 0xdc, 0xb8, 0x70, 0xe5, 0x04, + 0x07, 0x2e, 0x39, 0xa1, 0x9c, 0x60, 0xc5, 0xc1, 0x02, 0x47, 0x48, 0x9c, 0x39, 0x46, 0x42, 0x42, + 0x55, 0xd5, 0x5f, 0xc9, 0x38, 0x83, 0x87, 0x89, 0x56, 0x7b, 0xb1, 0xbb, 0x5f, 0xbd, 0xf7, 0x7b, + 0x55, 0xef, 0xbb, 0x1a, 0x90, 0xeb, 0xe8, 0xc6, 0x70, 0xd2, 0xdf, 0x31, 0x75, 0x5f, 0xaf, 0x4e, + 0x5c, 0xc7, 0x77, 0xd0, 0x96, 0xe1, 0x18, 0x8f, 0x28, 0xbd, 0x1a, 0xac, 0x5e, 0xbe, 0x10, 0xb2, + 0x8d, 0xb1, 0xaf, 0xc7, 0xac, 0x97, 0x2b, 0x9e, 0xef, 0xb8, 0xfa, 0x00, 0xef, 0x60, 0x7b, 0x60, + 0xd9, 0xe1, 0x1f, 0xe1, 0x3b, 0x34, 0x8c, 0x80, 0xe7, 0xdd, 0x93, 0x78, 0x6e, 0x04, 0x4c, 0xc5, + 0xa9, 0x6f, 0x8d, 0x76, 0x86, 0x23, 0x63, 0xc7, 0xb7, 0xc6, 0xd8, 0xf3, 0xf5, 0xf1, 0x24, 0x58, + 0x39, 0x3f, 0x70, 0x06, 0x0e, 0x7d, 0xdc, 0x21, 0x4f, 0x8c, 0x5a, 0xf9, 0x0c, 0x84, 0xde, 0x44, + 0xb7, 0xd1, 0x25, 0xe0, 0x1f, 0xe1, 0x59, 0x91, 0xbf, 0xc2, 0x5d, 0xcd, 0xd7, 0x33, 0x2f, 0xe7, + 0x65, 0xfe, 0x2e, 0x9e, 0x29, 0x84, 0x86, 0xae, 0x40, 0x06, 0xdb, 0xa6, 0x46, 0x96, 0x85, 0x57, + 0x97, 0xd3, 0xd8, 0x36, 0xef, 0xe2, 0xd9, 0xcd, 0xfc, 0xaf, 0x9e, 0x96, 0xd7, 0x7e, 0xff, 0xb4, + 0xcc, 0xfd, 0xeb, 0x69, 0x99, 0x6b, 0x0b, 0x22, 0x27, 0xad, 0xb7, 0x05, 0x71, 0x5d, 0xe2, 0x2b, + 0x63, 0x48, 0xdd, 0xd7, 0x47, 0x53, 0x8c, 0xde, 0x81, 0xac, 0xab, 0x3f, 0xd1, 0xfa, 0x33, 0x1f, + 0x7b, 0x45, 0x8e, 0xc0, 0x28, 0xa2, 0xab, 0x3f, 0xa9, 0x93, 0x77, 0x54, 0x83, 0x6c, 0xb4, 0xdb, + 0xe2, 0xfa, 0x15, 0xee, 0x6a, 0xee, 0xfa, 0x37, 0xaa, 0xb1, 0xf1, 0xc8, 0x91, 0xaa, 0xc3, 0x91, + 0x51, 0x55, 0x43, 0xa6, 0xba, 0xf0, 0x6c, 0x5e, 0x5e, 0x53, 0x62, 0xa9, 0x9b, 0x02, 0x51, 0x5d, + 0xf9, 0x14, 0xc4, 0xbb, 0x78, 0xc6, 0x34, 0x06, 0x27, 0xe2, 0x96, 0x9c, 0xe8, 0x03, 0x48, 0x1d, + 0x12, 0x9e, 0x40, 0x57, 0xb1, 0x7a, 0xcc, 0x51, 0x55, 0x8a, 0x11, 0xa8, 0x61, 0xcc, 0x95, 0xbf, + 0x72, 0x00, 0x3d, 0xdf, 0x71, 0xb1, 0x6c, 0x62, 0xdb, 0x47, 0x03, 0x00, 0x63, 0x34, 0xf5, 0x7c, + 0xec, 0x6a, 0x96, 0x19, 0xa8, 0xb9, 0x43, 0xf8, 0xff, 0x36, 0x2f, 0xdf, 0x18, 0x58, 0xfe, 0x70, + 0xda, 0xaf, 0x1a, 0xce, 0x78, 0x27, 0xc2, 0x36, 0xfb, 0xf1, 0xf3, 0xce, 0xe4, 0xd1, 0x60, 0x87, + 0xba, 0x6a, 0x3a, 0xb5, 0xcc, 0xea, 0xfe, 0xbe, 0xdc, 0x5c, 0xcc, 0xcb, 0xd9, 0x06, 0x03, 0x94, + 0x9b, 0x4a, 0x36, 0xc0, 0x96, 0x4d, 0xf4, 0x3e, 0x64, 0x6c, 0xc7, 0xc4, 0x44, 0x0b, 0xd9, 0x6f, + 0xaa, 0x7e, 0x7e, 0x31, 0x2f, 0xa7, 0xbb, 0x8e, 0x89, 0xe5, 0xe6, 0xcb, 0xe8, 0x49, 0x49, 0x13, + 0x26, 0xd9, 0x44, 0xdf, 0x03, 0x91, 0x04, 0x0a, 0xe5, 0xe7, 0x29, 0xff, 0x85, 0xc5, 0xbc, 0x9c, + 0x61, 0x3b, 0x27, 0x02, 0xe1, 0xa3, 0x92, 0xf1, 0xd8, 0x69, 0x2a, 0xbf, 0xe5, 0x20, 0xdf, 0x9b, + 0x8c, 0x2c, 0x5f, 0x75, 0xad, 0xc1, 0x00, 0xbb, 0xa8, 0x05, 0xd9, 0x11, 0x3e, 0xf0, 0x35, 0x13, + 0x7b, 0x06, 0x3d, 0x5a, 0xee, 0x7a, 0x65, 0x89, 0x91, 0x14, 0xdd, 0x1e, 0xe0, 0x26, 0xf6, 0x0c, + 0xd7, 0x9a, 0xf8, 0x8e, 0x1b, 0x98, 0x4b, 0x24, 0xa2, 0x84, 0x8a, 0x6e, 0x03, 0xb8, 0xd6, 0x60, + 0x18, 0xe0, 0xac, 0x9f, 0x12, 0x27, 0x4b, 0x65, 0x09, 0x99, 0x79, 0xb7, 0x2d, 0x88, 0xbc, 0x24, + 0x54, 0x16, 0xeb, 0x90, 0xef, 0x60, 0x77, 0x80, 0xbf, 0xa6, 0x9b, 0x45, 0x36, 0x48, 0x0c, 0x88, + 0xe4, 0xa5, 0xe6, 0xf9, 0xba, 0xef, 0xd1, 0xc4, 0xc9, 0x5d, 0x7f, 0x2f, 0x01, 0x17, 0x24, 0x73, + 0x95, 0x65, 0x71, 0x35, 0x4c, 0xe6, 0x6a, 0xe7, 0x7e, 0xa3, 0xd1, 0x23, 0x32, 0xf5, 0x0b, 0x04, + 0x78, 0x31, 0x2f, 0x17, 0x14, 0x82, 0x16, 0xd1, 0x95, 0x02, 0x45, 0xef, 0x1c, 0x1a, 0x06, 0x7d, + 0x47, 0xb7, 0x20, 0x7f, 0xe0, 0x62, 0xfc, 0x73, 0x4c, 0x74, 0xb9, 0x7e, 0x31, 0xb5, 0x7a, 0x02, + 0xe5, 0x98, 0x60, 0x8f, 0xc8, 0xbd, 0x62, 0xe4, 0xdf, 0xa5, 0x60, 0xbb, 0x31, 0x24, 0x27, 0x55, + 0xf0, 0x64, 0x64, 0x19, 0xba, 0x17, 0x5a, 0xfb, 0x21, 0x5c, 0x30, 0xf1, 0xc4, 0xc5, 0x86, 0xee, + 0x63, 0x53, 0x33, 0x28, 0x8f, 0xe6, 0xcf, 0x26, 0x98, 0x9a, 0xbe, 0x70, 0xfd, 0x5b, 0xcb, 0x4c, + 0xc6, 0x30, 0x18, 0xa0, 0x3a, 0x9b, 0x60, 0xe5, 0x7c, 0x8c, 0x11, 0x53, 0xd1, 0x03, 0x40, 0x09, + 0x6c, 0x97, 0x49, 0x05, 0xae, 0x38, 0x01, 0xf7, 0x98, 0x33, 0xb6, 0x62, 0x94, 0x80, 0x05, 0xfd, + 0x0c, 0xde, 0x49, 0x40, 0x4f, 0x27, 0x66, 0x52, 0x85, 0x57, 0xe4, 0xaf, 0xf0, 0xa7, 0xd4, 0x71, + 0x29, 0x86, 0xdb, 0x67, 0x68, 0xa1, 0xa5, 0x10, 0x86, 0xcb, 0x09, 0x5d, 0x36, 0x3e, 0xf2, 0x43, + 0x45, 0x24, 0x27, 0x05, 0x9a, 0x93, 0x57, 0x17, 0xf3, 0xf2, 0xc5, 0x66, 0xc4, 0xd5, 0xc5, 0x47, + 0x7e, 0x20, 0x4f, 0x73, 0x34, 0x1b, 0xbd, 0x28, 0x17, 0xcd, 0xa5, 0x5c, 0x26, 0xfa, 0x08, 0x04, + 0x1a, 0xaa, 0xa9, 0x55, 0x43, 0x55, 0xa1, 0xfc, 0xa8, 0x0f, 0x17, 0x2d, 0xdb, 0xc7, 0xae, 0xad, + 0x8f, 0x34, 0xdd, 0x34, 0x93, 0x66, 0x48, 0x9f, 0xda, 0x0c, 0xdb, 0x21, 0x54, 0x8d, 0x20, 0x45, + 0x26, 0x38, 0x80, 0x4b, 0x91, 0x0e, 0x17, 0x8f, 0x9d, 0xc3, 0xa4, 0x96, 0xcc, 0xa9, 0xb5, 0x44, + 0x1b, 0x56, 0x18, 0x56, 0xa8, 0xe7, 0xa6, 0x48, 0x3a, 0x0f, 0x2d, 0xfd, 0x5f, 0x70, 0x70, 0xae, + 0xe3, 0x98, 0xd6, 0x81, 0x85, 0x4d, 0xd2, 0xd1, 0xc2, 0x78, 0x7d, 0x0f, 0x90, 0x37, 0xf3, 0x7c, + 0x3c, 0xd6, 0x0c, 0xc7, 0x3e, 0xb0, 0x06, 0x9a, 0x37, 0xd1, 0x6d, 0x1a, 0xab, 0xa2, 0x22, 0xb1, + 0x95, 0x06, 0x5d, 0xa0, 0x6d, 0xb0, 0x05, 0x88, 0xd6, 0xda, 0x91, 0x75, 0x88, 0x6d, 0xec, 0x79, + 0x8c, 0x9b, 0x45, 0xe0, 0xc5, 0x25, 0x1b, 0x26, 0x42, 0x8a, 0x44, 0x44, 0x76, 0x03, 0x09, 0x42, + 0x09, 0xba, 0xd1, 0x4f, 0x40, 0xea, 0xf9, 0x96, 0xf1, 0x68, 0x56, 0x8f, 0x2b, 0x6b, 0x1d, 0xc0, + 0xa3, 0x34, 0xad, 0x6f, 0xf9, 0x41, 0xb5, 0x5a, 0xad, 0xd7, 0x79, 0x21, 0x54, 0x80, 0xfe, 0x07, + 0x1e, 0xb6, 0xe5, 0xc0, 0x2c, 0x0d, 0x67, 0x3c, 0x8e, 0x75, 0x34, 0x61, 0xc3, 0x23, 0xd5, 0x5c, + 0xf3, 0x19, 0x21, 0x50, 0x53, 0x5e, 0xba, 0xff, 0xb8, 0xea, 0x2b, 0x79, 0x2f, 0xd9, 0x03, 0x9a, + 0xb0, 0x31, 0x26, 0x65, 0x36, 0x42, 0x59, 0x7f, 0x23, 0x4a, 0xb2, 0x1c, 0x2b, 0xf9, 0x71, 0xb2, + 0x38, 0xff, 0x14, 0x2e, 0x06, 0x35, 0x22, 0x74, 0x7f, 0x84, 0xc7, 0x53, 0xbc, 0xab, 0x4b, 0xf0, + 0x96, 0x56, 0x1e, 0x65, 0xdb, 0x78, 0x43, 0x41, 0xda, 0x1e, 0x07, 0x7e, 0xa7, 0xde, 0x8a, 0xf0, + 0x59, 0xcd, 0xfd, 0xf6, 0xb2, 0xfd, 0x1e, 0x8f, 0x13, 0xe5, 0xdc, 0x78, 0x49, 0xf0, 0x7c, 0x0c, + 0x28, 0xf6, 0x56, 0x04, 0xcc, 0x12, 0xee, 0xdd, 0x65, 0xe6, 0x7c, 0xcd, 0xdd, 0x8a, 0xe4, 0xbd, + 0x46, 0xb9, 0x29, 0x7e, 0x1e, 0xcc, 0x49, 0x95, 0x5f, 0x72, 0xb0, 0xb5, 0xd7, 0xf7, 0xb0, 0x7b, + 0x88, 0xcd, 0xc8, 0xdb, 0xc9, 0x6e, 0xcf, 0xad, 0xd0, 0xed, 0xdf, 0xc2, 0xe8, 0x24, 0x86, 0x93, + 0x5b, 0xe5, 0x2f, 0x19, 0xc8, 0xa9, 0xae, 0x6e, 0x7b, 0xba, 0xe1, 0x5b, 0x8e, 0x8d, 0xee, 0x80, + 0x40, 0xe6, 0xd4, 0x20, 0x7e, 0xae, 0xad, 0xd0, 0xbd, 0xd4, 0x23, 0xbb, 0x83, 0x7d, 0xbd, 0x2e, + 0x12, 0x25, 0xcf, 0xe7, 0x65, 0x4e, 0xa1, 0x08, 0x08, 0x81, 0x60, 0xeb, 0x63, 0x36, 0x70, 0x65, + 0x15, 0xfa, 0x8c, 0x7e, 0x08, 0x69, 0xd2, 0x1c, 0xa7, 0xac, 0x3b, 0x2e, 0xef, 0x1c, 0x89, 0xdd, + 0xf4, 0x28, 0xaf, 0x12, 0xc8, 0xa0, 0x36, 0x14, 0x46, 0xba, 0xe7, 0x6b, 0x43, 0xac, 0xbb, 0x7e, + 0x1f, 0xeb, 0xa7, 0xea, 0x7b, 0x1b, 0x44, 0xf4, 0x4e, 0x28, 0x89, 0x34, 0x48, 0x54, 0x73, 0xcd, + 0x71, 0xad, 0x81, 0x16, 0x1b, 0x35, 0xbd, 0x3a, 0x6c, 0xa2, 0x54, 0xef, 0xb9, 0xd6, 0x20, 0x76, + 0xea, 0x1d, 0xd8, 0x18, 0xeb, 0x47, 0x09, 0xd0, 0xcc, 0xea, 0xa0, 0xf9, 0xb1, 0x7e, 0x14, 0x23, + 0x7d, 0x0a, 0xe7, 0x9c, 0x20, 0x66, 0x62, 0x38, 0xaf, 0x28, 0xbe, 0xb1, 0xa4, 0x1e, 0x8b, 0xb0, + 0x00, 0x16, 0x39, 0xaf, 0x2f, 0x78, 0xe8, 0xc7, 0x90, 0x27, 0x85, 0xd6, 0xf6, 0x69, 0x22, 0x79, + 0xc5, 0x1c, 0x45, 0x7d, 0x53, 0xdd, 0x0b, 0x67, 0x08, 0x26, 0x42, 0x28, 0x1e, 0xaa, 0xc0, 0xc6, + 0x13, 0xd7, 0xf2, 0xb1, 0xe6, 0x3b, 0x8e, 0xe6, 0x8c, 0xcc, 0x62, 0x9e, 0x16, 0xda, 0x1c, 0x25, + 0xaa, 0x8e, 0xb3, 0x37, 0x32, 0x89, 0xe7, 0x5c, 0xac, 0x27, 0xb6, 0x5f, 0xdc, 0x3c, 0x85, 0xe7, + 0x88, 0x68, 0x6c, 0x8e, 0x0f, 0xe0, 0x82, 0x41, 0x6b, 0x5f, 0x8c, 0xa6, 0x1d, 0x58, 0x47, 0xd8, + 0x2c, 0x4a, 0x54, 0xf1, 0x79, 0xb6, 0x1a, 0x09, 0xdc, 0x22, 0x6b, 0xe8, 0x63, 0x90, 0x2c, 0x5b, + 0x3b, 0x18, 0xd1, 0x29, 0x8d, 0x6e, 0xcd, 0x2b, 0x6e, 0xd1, 0xb3, 0x7e, 0x73, 0xd9, 0x59, 0xf1, + 0xe3, 0x29, 0xb6, 0x0d, 0x6c, 0x7e, 0x42, 0x38, 0x83, 0x7d, 0x14, 0x2c, 0xfb, 0x16, 0x95, 0xa7, + 0x44, 0x0f, 0x1d, 0xc2, 0xa6, 0x35, 0xb0, 0x1d, 0x97, 0x14, 0x21, 0xfc, 0xd8, 0x9e, 0x8e, 0xbd, + 0x22, 0xa2, 0x88, 0x1f, 0xae, 0x90, 0x35, 0x32, 0x93, 0xec, 0xe1, 0xc7, 0xdd, 0xe9, 0x98, 0x36, + 0xed, 0x78, 0xf8, 0x7b, 0x65, 0xcd, 0x53, 0x0a, 0x56, 0xf4, 0x4e, 0x94, 0x1c, 0xbb, 0x7a, 0xf1, + 0x92, 0xd0, 0x16, 0xc4, 0xac, 0x04, 0x6d, 0x41, 0xdc, 0x90, 0x0a, 0x6d, 0x41, 0x2c, 0x48, 0x9b, + 0x95, 0x3f, 0xf3, 0xb0, 0x95, 0x48, 0x25, 0x05, 0x1b, 0x8e, 0x6b, 0xbe, 0xc5, 0xf4, 0xfe, 0xfa, + 0xa4, 0xf2, 0xd9, 0x43, 0xf8, 0xed, 0x07, 0x47, 0x5c, 0x61, 0xd9, 0xad, 0x38, 0x72, 0x53, 0x5a, + 0xca, 0xb4, 0x05, 0x31, 0x23, 0x89, 0x6d, 0x41, 0x14, 0xa5, 0x6c, 0xe4, 0x38, 0x90, 0x72, 0x6d, + 0x41, 0xcc, 0x4b, 0x1b, 0x49, 0x27, 0xb6, 0x05, 0x71, 0x53, 0x92, 0xda, 0x82, 0x28, 0x49, 0x5b, + 0x95, 0x3f, 0xae, 0x43, 0x5a, 0xa6, 0x9b, 0x46, 0x1f, 0x82, 0x10, 0x8d, 0x34, 0x27, 0x9c, 0x34, + 0xe1, 0x32, 0xc2, 0x8e, 0xea, 0xc0, 0xfb, 0x47, 0xe1, 0x68, 0x73, 0x1a, 0xdf, 0xb3, 0x23, 0x12, + 0xe1, 0x84, 0xdb, 0xf9, 0xff, 0xc3, 0xed, 0x4b, 0x52, 0x46, 0xf8, 0x2a, 0x52, 0x86, 0x8d, 0x4f, + 0xbf, 0xe6, 0xa0, 0xf0, 0xaa, 0xf3, 0x4e, 0xfa, 0x62, 0x60, 0x80, 0xe8, 0x05, 0xcc, 0xc1, 0x25, + 0xfc, 0xf6, 0xcb, 0x79, 0xb9, 0xb1, 0xd2, 0x35, 0xff, 0x0d, 0x9f, 0x6d, 0x88, 0x41, 0x7b, 0xf8, + 0xb1, 0x12, 0x01, 0x27, 0x1a, 0xf1, 0x7f, 0x78, 0x48, 0xed, 0x62, 0xdd, 0xc3, 0xe8, 0x07, 0x90, + 0x62, 0xb7, 0xba, 0x53, 0x8c, 0x8a, 0x4c, 0x02, 0x7d, 0x06, 0x80, 0x8f, 0x26, 0x96, 0xab, 0x13, + 0xdb, 0xaf, 0x36, 0x1b, 0x94, 0xfe, 0x3d, 0x2f, 0x5f, 0x4e, 0x9c, 0xe4, 0x66, 0xc5, 0xd5, 0x6d, + 0xd3, 0x9e, 0x8e, 0x46, 0x7a, 0x7f, 0x84, 0x2b, 0x4a, 0x02, 0x10, 0x35, 0x21, 0x13, 0xde, 0xd0, + 0xf8, 0x53, 0xdf, 0xd0, 0x42, 0x51, 0x34, 0x85, 0x44, 0xd3, 0x64, 0x17, 0x58, 0xf2, 0xeb, 0x59, + 0xe1, 0x9d, 0xf9, 0x8c, 0x3b, 0xde, 0x8e, 0xd1, 0xe9, 0x2d, 0xb7, 0x47, 0xb1, 0x51, 0x17, 0x72, + 0x13, 0xd7, 0x99, 0x38, 0x1e, 0x69, 0xa3, 0xde, 0x6a, 0xf5, 0xa6, 0xb0, 0x98, 0x97, 0xe1, 0x5e, + 0x20, 0xa5, 0xf6, 0x14, 0x08, 0x11, 0x54, 0x0f, 0x9d, 0x87, 0x14, 0x9e, 0x38, 0xc6, 0x90, 0x4e, + 0x0b, 0xbc, 0xc2, 0x5e, 0xd0, 0xfb, 0x89, 0xa8, 0x21, 0x1d, 0x9f, 0xaf, 0x6f, 0xbd, 0x9c, 0x97, + 0x37, 0xa8, 0x67, 0xc3, 0xd8, 0x4b, 0xfa, 0x3f, 0xac, 0xe5, 0x95, 0x05, 0x07, 0x85, 0x5a, 0xdf, + 0x71, 0x69, 0x49, 0x6a, 0xd9, 0xbe, 0x3b, 0x3b, 0x29, 0x38, 0xcf, 0x3e, 0x03, 0xa2, 0x21, 0x88, + 0x13, 0xd7, 0x72, 0x5c, 0xcb, 0x9f, 0x05, 0x1f, 0x8d, 0x76, 0x5f, 0xce, 0xcb, 0x77, 0xce, 0x1a, + 0xdf, 0xf7, 0x02, 0x4c, 0x25, 0x42, 0x4f, 0x04, 0xf9, 0x3f, 0x39, 0xd8, 0xda, 0xc5, 0xfa, 0x81, + 0x7a, 0x64, 0xcb, 0xf6, 0x64, 0x4a, 0x5c, 0xe3, 0x63, 0xf4, 0x11, 0xab, 0x4b, 0x2c, 0xdc, 0x4b, + 0x27, 0x17, 0x94, 0x64, 0x2d, 0xfa, 0x0e, 0x6c, 0xba, 0xf8, 0xc0, 0xc5, 0xde, 0x50, 0xb3, 0xec, + 0x43, 0x7d, 0x64, 0x99, 0xd4, 0xe4, 0xa2, 0x52, 0x08, 0xc8, 0x32, 0xa3, 0x2e, 0xad, 0xef, 0xe2, + 0x99, 0xea, 0xfb, 0x6b, 0x55, 0x5d, 0x90, 0x52, 0x6d, 0x41, 0x4c, 0x49, 0x69, 0x56, 0xe1, 0x2b, + 0xbf, 0x58, 0x8f, 0xce, 0x79, 0xcb, 0xb2, 0xf5, 0xd1, 0xd9, 0xce, 0xf9, 0x7d, 0x28, 0x26, 0xbf, + 0xbf, 0x38, 0xe3, 0xb1, 0x6e, 0x93, 0xff, 0xa9, 0xed, 0x33, 0xcf, 0x29, 0x89, 0xef, 0x33, 0x0d, + 0xb6, 0xdc, 0x20, 0xab, 0xa8, 0x0e, 0x1b, 0xa1, 0x85, 0xc8, 0x10, 0x15, 0x56, 0xdb, 0xff, 0xd1, + 0x1b, 0xf3, 0x81, 0x8c, 0x42, 0x44, 0x56, 0xb6, 0x72, 0x64, 0x92, 0xc8, 0x0c, 0xac, 0xc5, 0x5d, + 0xfb, 0x13, 0x07, 0x59, 0xfa, 0x6d, 0x95, 0x7e, 0xf0, 0xc9, 0x41, 0x66, 0xbf, 0x7b, 0xb7, 0xbb, + 0xf7, 0x49, 0x57, 0x5a, 0x43, 0x19, 0xe0, 0xe5, 0xae, 0x2a, 0x71, 0x28, 0x0b, 0xa9, 0x5b, 0xbb, + 0x7b, 0x35, 0x55, 0x5a, 0x27, 0x8f, 0xf5, 0x07, 0x6a, 0xab, 0x27, 0xf1, 0xe8, 0x1c, 0x6c, 0x36, + 0x5b, 0xbb, 0x72, 0x47, 0x56, 0x5b, 0x4d, 0x8d, 0x11, 0x45, 0x24, 0x82, 0xa0, 0xca, 0x9d, 0x96, + 0x24, 0x10, 0xa8, 0x66, 0xab, 0x21, 0x77, 0x6a, 0xbb, 0x52, 0x0a, 0x6d, 0xc3, 0x56, 0xcc, 0x1b, + 0x92, 0xb3, 0x28, 0x0f, 0x62, 0x73, 0x5f, 0xa9, 0xa9, 0xf2, 0x5e, 0x57, 0x4a, 0x23, 0x80, 0x34, + 0x91, 0x55, 0x1f, 0x4a, 0x79, 0xa2, 0x47, 0xdd, 0xbf, 0xb7, 0xdb, 0x92, 0x80, 0x30, 0xd5, 0x65, + 0xb5, 0xa6, 0x28, 0xb5, 0x07, 0x52, 0x0e, 0x15, 0x00, 0x08, 0x53, 0xaf, 0xa5, 0xc8, 0xad, 0x9e, + 0x64, 0x56, 0x48, 0xbb, 0xce, 0x5c, 0xfb, 0x11, 0x6c, 0x1d, 0xfb, 0xa6, 0x85, 0x36, 0x21, 0x57, + 0x6b, 0x36, 0x35, 0xa5, 0x75, 0x6f, 0x57, 0x6e, 0xd4, 0xa4, 0x35, 0x84, 0xa0, 0xa0, 0xb4, 0x3a, + 0x7b, 0xf7, 0x5b, 0x11, 0x8d, 0xbb, 0x2c, 0x7c, 0xfe, 0x9b, 0xd2, 0xda, 0xb5, 0xbd, 0x57, 0xc6, + 0x31, 0xd6, 0x17, 0xc9, 0x09, 0xee, 0xb5, 0xba, 0x4d, 0xb9, 0x7b, 0x5b, 0x5a, 0x23, 0x2f, 0x3d, + 0xb5, 0x76, 0x9b, 0xbc, 0xf0, 0x68, 0x03, 0xb2, 0x8d, 0xbd, 0x4e, 0x47, 0x56, 0xd5, 0x56, 0x53, + 0xe2, 0xc8, 0x5a, 0xad, 0xbe, 0xa7, 0x90, 0x97, 0x75, 0x06, 0x58, 0xff, 0xee, 0xb3, 0x7f, 0x94, + 0xd6, 0x9e, 0x2d, 0x4a, 0xdc, 0xf3, 0x45, 0x89, 0xfb, 0x72, 0x51, 0xe2, 0xfe, 0xbe, 0x28, 0x71, + 0x5f, 0xbc, 0x28, 0xad, 0x3d, 0x7f, 0x51, 0x5a, 0xfb, 0xf2, 0x45, 0x69, 0xed, 0x61, 0x26, 0xf0, + 0x6d, 0x3f, 0x4d, 0x3f, 0xfc, 0xdf, 0xf8, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x70, 0x89, 0x9f, + 0xff, 0xb2, 0x18, 0x00, 0x00, } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 05ef4f5232b8..32d70cdf37e2 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -480,6 +480,14 @@ message Transaction { // treated as immutable and all updates should be performed on a copy of the // slice. repeated SequencedWrite in_flight_writes = 17 [(gogoproto.nullable) = false]; + // A list of ignored seqnum ranges. + // + // The user code must guarantee this list to be non-overlapping, + // non-contiguous (i.e. it must coalesce ranges to avoid situations + // where a range's end seqnum is equal to the next range's start + // seqnum), and sorted in seqnum order. + repeated storage.engine.enginepb.IgnoredSeqNumRange ignored_seqnums = 18 + [(gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums"]; reserved 3, 9, 13, 14; } @@ -512,12 +520,24 @@ message TransactionRecord { } // A Intent is a Span together with a Transaction metadata and its status. +// +// Intents are used for two separate purposes: +// - on the return path of e.g. scans, to report the existence of a +// write intent on a key; +// - as input argument to intent resolution, to pass the current txn status, +// timestamps and ignored seqnum ranges to the resolution algorithm. +// Only in the latter case are the TxnMeta, status and ignored seqnum +// ranges guaranteed to be consistent with the latest txn's state. +// +// Note: avoid constructing Intent directly; consider using +// MakeIntent() or MakeErrorIntent() instead. message Intent { option (gogoproto.equal) = true; Span span = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; storage.engine.enginepb.TxnMeta txn = 2 [(gogoproto.nullable) = false]; TransactionStatus status = 3; + repeated storage.engine.enginepb.IgnoredSeqNumRange ignored_seqnums = 4 [(gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums"]; } // A SequencedWrite is a point write to a key with a certain sequence number. diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 1ed286494491..2a8556f3324a 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -490,6 +490,7 @@ var nonZeroTxn = Transaction{ IntentSpans: []Span{{Key: []byte("a"), EndKey: []byte("b")}}, InFlightWrites: []SequencedWrite{{Key: []byte("c"), Sequence: 1}}, CommitTimestampFixed: true, + IgnoredSeqNums: []enginepb.IgnoredSeqNumRange{{Start: 888, End: 999}}, } func TestTransactionUpdate(t *testing.T) { @@ -550,6 +551,7 @@ func TestTransactionUpdate(t *testing.T) { expTxn5.Sequence = txn.Sequence - 10 expTxn5.IntentSpans = nil expTxn5.InFlightWrites = nil + expTxn5.IgnoredSeqNums = nil expTxn5.WriteTooOld = false expTxn5.CommitTimestampFixed = false require.Equal(t, expTxn5, txn5) @@ -636,6 +638,7 @@ func TestTransactionClone(t *testing.T) { // listed below. If this test fails, please update the list below and/or // Transaction.Clone(). expFields := []string{ + "IgnoredSeqNums", "InFlightWrites", "InFlightWrites.Key", "IntentSpans", @@ -666,6 +669,7 @@ func TestTransactionRestart(t *testing.T) { expTxn.CommitTimestampFixed = false expTxn.IntentSpans = nil expTxn.InFlightWrites = nil + expTxn.IgnoredSeqNums = nil require.Equal(t, expTxn, txn) } @@ -1877,3 +1881,22 @@ func TestChangeReplicasTrigger_ConfChange(t *testing.T) { }) } } + +// TestAsIntents verifies that AsIntents propagates all the important +// fields from a txn to each intent. +func TestAsIntents(t *testing.T) { + defer leaktest.AfterTest(t)() + + ts := hlc.Timestamp{WallTime: 1} + txn := MakeTransaction("hello", Key("k"), 0, ts, 0) + + txn.Status = COMMITTED + txn.IgnoredSeqNums = []enginepb.IgnoredSeqNumRange{{Start: 0, End: 0}} + + spans := []Span{{Key: Key("a"), EndKey: Key("b")}} + for _, intent := range AsIntents(spans, &txn) { + require.Equal(t, intent.Status, txn.Status) + require.Equal(t, intent.IgnoredSeqNums, txn.IgnoredSeqNums) + require.Equal(t, intent.Txn, txn.TxnMeta) + } +} diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index 68717c9dd1b0..8432b45a43cf 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -482,7 +482,7 @@ func resolveLocalIntents( externalIntents = append(externalIntents, span) return nil } - intent := roachpb.Intent{Span: span, Txn: txn.TxnMeta, Status: txn.Status} + intent := roachpb.MakeIntent(*txn, span) if len(span.EndKey) == 0 { // For single-key intents, do a KeyAddress-aware check of // whether it's contained in our Range. diff --git a/pkg/storage/batcheval/cmd_end_transaction_test.go b/pkg/storage/batcheval/cmd_end_transaction_test.go index b71c44330cf5..76a60de69892 100644 --- a/pkg/storage/batcheval/cmd_end_transaction_test.go +++ b/pkg/storage/batcheval/cmd_end_transaction_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/engine" + "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -1054,3 +1055,87 @@ func TestEndTransactionUpdatesTransactionRecord(t *testing.T) { }) } } + +func TestPartialRollbackOnEndTransaction(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + k := roachpb.Key("a") + ts := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} + txn := roachpb.MakeTransaction("test", k, 0, ts, 0) + endKey := roachpb.Key("z") + desc := roachpb.RangeDescriptor{ + RangeID: 99, + StartKey: roachpb.RKey(k), + EndKey: roachpb.RKey(endKey), + } + intents := []roachpb.Span{{Key: k}} + + db := engine.NewDefaultInMem() + defer db.Close() + batch := db.NewBatch() + defer batch.Close() + + var v roachpb.Value + + // Write a first value at key. + v.SetString("a") + txn.Sequence = 0 + if err := engine.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + t.Fatal(err) + } + // Write another value. + v.SetString("b") + txn.Sequence = 1 + if err := engine.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + t.Fatal(err) + } + + // Partially revert the store above. + txn.IgnoredSeqNums = []enginepb.IgnoredSeqNumRange{{Start: 1, End: 1}} + + // Issue the end txn command. + req := roachpb.EndTransactionRequest{ + RequestHeader: roachpb.RequestHeader{Key: txn.Key}, + Commit: true, + NoRefreshSpans: true, + IntentSpans: intents, + } + var resp roachpb.EndTransactionResponse + if _, err := EndTransaction(ctx, batch, CommandArgs{ + EvalCtx: &mockEvalCtx{ + desc: &desc, + canCreateTxnFn: func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { + //t.Fatal("CanCreateTxnRecord unexpectedly called") + return true, ts, 0 + }, + }, + Args: &req, + Header: roachpb.Header{ + Timestamp: ts, + Txn: &txn, + }, + }, &resp); err != nil { + t.Fatal(err) + } + + // The second write has been rolled back; verify that the remaining + // value is from the first write. + res, i, err := engine.MVCCGet(ctx, batch, k, ts2, engine.MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if i != nil { + t.Errorf("found intent, expected none: %+v", i) + } + if res == nil { + t.Errorf("no value found, expected one") + } else { + s, err := res.GetBytes() + if err != nil { + t.Fatal(err) + } + require.Equal(t, "a", string(s)) + } +} diff --git a/pkg/storage/batcheval/cmd_refresh_range_test.go b/pkg/storage/batcheval/cmd_refresh_range_test.go index 34f56d6a0dd6..2466a452d936 100644 --- a/pkg/storage/batcheval/cmd_refresh_range_test.go +++ b/pkg/storage/batcheval/cmd_refresh_range_test.go @@ -83,11 +83,9 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { // (committed). The sstable also has a second write at a different (older) // timestamp, because if it were empty other than the deletion tombstone, it // would not have any timestamp bounds and would be selected for every read. - if err := engine.MVCCResolveWriteIntent(ctx, db, nil, roachpb.Intent{ - Span: roachpb.Span{Key: k}, - Txn: txn.TxnMeta, - Status: roachpb.COMMITTED, - }); err != nil { + intent := roachpb.MakeIntent(*txn, roachpb.Span{Key: k}) + intent.Status = roachpb.COMMITTED + if err := engine.MVCCResolveWriteIntent(ctx, db, nil, intent); err != nil { t.Fatal(err) } if err := engine.MVCCPut(ctx, db, nil, roachpb.Key("unused2"), ts1, v, nil); err != nil { diff --git a/pkg/storage/batcheval/cmd_resolve_intent.go b/pkg/storage/batcheval/cmd_resolve_intent.go index b88dbd998db6..c8ec642872b7 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent.go +++ b/pkg/storage/batcheval/cmd_resolve_intent.go @@ -77,11 +77,7 @@ func ResolveIntent( return result.Result{}, ErrTransactionUnsupported } - intent := roachpb.Intent{ - Span: args.Span(), - Txn: args.IntentTxn, - Status: args.Status, - } + intent := args.AsIntent() if err := engine.MVCCResolveWriteIntent(ctx, batch, ms, intent); err != nil { return result.Result{}, err } diff --git a/pkg/storage/batcheval/cmd_resolve_intent_range.go b/pkg/storage/batcheval/cmd_resolve_intent_range.go index fce07a5c5ab3..64f5ca803fa3 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent_range.go +++ b/pkg/storage/batcheval/cmd_resolve_intent_range.go @@ -42,11 +42,7 @@ func ResolveIntentRange( return result.Result{}, ErrTransactionUnsupported } - intent := roachpb.Intent{ - Span: args.Span(), - Txn: args.IntentTxn, - Status: args.Status, - } + intent := args.AsIntent() iterAndBuf := engine.GetIterAndBuf(batch, engine.IterOptions{UpperBound: args.EndKey}) defer iterAndBuf.Cleanup() diff --git a/pkg/storage/batcheval/cmd_resolve_intent_test.go b/pkg/storage/batcheval/cmd_resolve_intent_test.go index 28421bf55a73..466b7ce8b840 100644 --- a/pkg/storage/batcheval/cmd_resolve_intent_test.go +++ b/pkg/storage/batcheval/cmd_resolve_intent_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/stretchr/testify/require" ) type mockEvalCtx struct { @@ -234,3 +235,128 @@ func TestDeclareKeysResolveIntent(t *testing.T) { } }) } + +func TestResolveIntentAfterPartialRollback(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + k := roachpb.Key("a") + ts := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} + endKey := roachpb.Key("z") + txn := roachpb.MakeTransaction("test", k, 0, ts, 0) + desc := roachpb.RangeDescriptor{ + RangeID: 99, + StartKey: roachpb.RKey(k), + EndKey: roachpb.RKey(endKey), + } + + testutils.RunTrueAndFalse(t, "ranged", func(t *testing.T, ranged bool) { + db := engine.NewDefaultInMem() + defer db.Close() + batch := db.NewBatch() + defer batch.Close() + + var v roachpb.Value + // Write a first value at key. + v.SetString("a") + txn.Sequence = 0 + if err := engine.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + t.Fatal(err) + } + // Write another value. + v.SetString("b") + txn.Sequence = 1 + if err := engine.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + t.Fatal(err) + } + if err := batch.Commit(true); err != nil { + t.Fatal(err) + } + + // Partially revert the 2nd store above. + ignoredSeqNums := []enginepb.IgnoredSeqNumRange{{Start: 1, End: 1}} + + h := roachpb.Header{ + RangeID: desc.RangeID, + Timestamp: ts, + } + + var spans spanset.SpanSet + rbatch := db.NewBatch() + rbatch = spanset.NewBatch(rbatch, &spans) + defer rbatch.Close() + + if !ranged { + // Resolve a point intent. + ri := roachpb.ResolveIntentRequest{ + IntentTxn: txn.TxnMeta, + Status: roachpb.COMMITTED, + IgnoredSeqNums: ignoredSeqNums, + } + ri.Key = k + + declareKeysResolveIntent(&desc, h, &ri, &spans) + + if _, err := ResolveIntent(ctx, rbatch, + CommandArgs{ + Header: h, + EvalCtx: &mockEvalCtx{}, + Args: &ri, + }, + &roachpb.ResolveIntentResponse{}, + ); err != nil { + t.Fatal(err) + } + } else { + // Resolve an intent range. + rir := roachpb.ResolveIntentRangeRequest{ + IntentTxn: txn.TxnMeta, + Status: roachpb.COMMITTED, + IgnoredSeqNums: ignoredSeqNums, + } + rir.Key = k + rir.EndKey = endKey + + declareKeysResolveIntentRange(&desc, h, &rir, &spans) + + if _, err := ResolveIntentRange(ctx, rbatch, + CommandArgs{ + Header: h, + EvalCtx: &mockEvalCtx{}, + Args: &rir, + MaxKeys: 10, + }, + &roachpb.ResolveIntentRangeResponse{}, + ); err != nil { + t.Fatal(err) + } + } + + if err := rbatch.Commit(true); err != nil { + t.Fatal(err) + } + + batch = db.NewBatch() + defer batch.Close() + + // The second write has been rolled back; verify that the remaining + // value is from the first write. + res, i, err := engine.MVCCGet(ctx, batch, k, ts2, engine.MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if i != nil { + t.Errorf("%s: found intent, expected none: %+v", k, i) + } + if res == nil { + t.Errorf("%s: no value found, expected one", k) + } else { + s, err := res.GetBytes() + if err != nil { + t.Fatal(err) + } + require.Equal(t, "a", string(s), "at key %s", k) + } + }) +} diff --git a/pkg/storage/client_metrics_test.go b/pkg/storage/client_metrics_test.go index c91d386c9e81..684a3fa5801f 100644 --- a/pkg/storage/client_metrics_test.go +++ b/pkg/storage/client_metrics_test.go @@ -196,14 +196,18 @@ func TestStoreResolveMetrics(t *testing.T) { endKey := span.EndKey if i > n/2 { req := &roachpb.ResolveIntentRangeRequest{ - IntentTxn: txn.TxnMeta, Status: status, Poison: poison, + IntentTxn: txn.TxnMeta, + Status: status, + Poison: poison, } req.Key, req.EndKey = key, endKey ba.Add(req) continue } req := &roachpb.ResolveIntentRequest{ - IntentTxn: txn.TxnMeta, Status: status, Poison: poison, + IntentTxn: txn.TxnMeta, + Status: status, + Poison: poison, } req.Key = key ba.Add(req) diff --git a/pkg/storage/engine/enginepb/mvcc.go b/pkg/storage/engine/enginepb/mvcc.go index 243f161770da..f92b2f196cc5 100644 --- a/pkg/storage/engine/enginepb/mvcc.go +++ b/pkg/storage/engine/enginepb/mvcc.go @@ -48,6 +48,39 @@ const ( MaxTxnPriority TxnPriority = math.MaxInt32 ) +// TxnSeqIsIgnored returns true iff the sequence number overlaps with +// any range in the ignored array. +func TxnSeqIsIgnored(seq TxnSeq, ignored []IgnoredSeqNumRange) bool { + // The ignored seqnum ranges are guaranteed to be + // non-overlapping, non-contiguous, and guaranteed to be + // sorted in seqnum order. We're going to look from the end to + // see if the current intent seqnum is ignored. + for i := len(ignored) - 1; i >= 0; i-- { + if seq < ignored[i].Start { + // The history entry's sequence number is lower/older than + // the current ignored range. Go to the previous range + // and try again. + continue + } + + // Here we have a range where the start seqnum is lower than the current + // intent seqnum. Does it include it? + if seq > ignored[i].End { + // Here we have a range where the current history entry's seqnum + // is higher than the range's end seqnum. Given that the + // ranges are sorted, we're guaranteed that there won't + // be any further overlapping range at a lower value of i. + return false + } + // Yes, it's included. We're going to skip over this + // intent seqnum and retry the search above. + return true + } + + // Exhausted the ignore list. Not ignored. + return false +} + // Short returns a prefix of the transaction's ID. func (t TxnMeta) Short() string { return t.ID.Short() @@ -174,14 +207,34 @@ func (meta *MVCCMetadata) AddToIntentHistory(seq TxnSeq, val []byte) { // GetPrevIntentSeq goes through the intent history and finds the previous // intent's sequence number given the current sequence. -func (meta *MVCCMetadata) GetPrevIntentSeq(seq TxnSeq) (TxnSeq, bool) { - index := sort.Search(len(meta.IntentHistory), func(i int) bool { - return meta.IntentHistory[i].Sequence >= seq - }) - if index > 0 && index < len(meta.IntentHistory) { - return meta.IntentHistory[index-1].Sequence, true +func (meta *MVCCMetadata) GetPrevIntentSeq( + seq TxnSeq, ignored []IgnoredSeqNumRange, +) (MVCCMetadata_SequencedIntent, bool) { + end := len(meta.IntentHistory) + found := 0 + for { + index := sort.Search(end, func(i int) bool { + return meta.IntentHistory[i].Sequence >= seq + }) + if index == 0 { + // It is possible that no intent exists such that the sequence is less + // than the read sequence. In this case, we cannot read a value from the + // intent history. + return MVCCMetadata_SequencedIntent{}, false + } + candidate := index - 1 + if TxnSeqIsIgnored(meta.IntentHistory[candidate].Sequence, ignored) { + // This entry was part of an ignored range. Skip it and + // try the search again, using the current position as new + // upper bound. + end = candidate + continue + } + // This history entry has not been ignored, so we're going to keep it. + found = candidate + break } - return 0, false + return meta.IntentHistory[found], true } // GetIntentValue goes through the intent history and finds the value diff --git a/pkg/storage/engine/enginepb/mvcc3.pb.go b/pkg/storage/engine/enginepb/mvcc3.pb.go index c0822fb0ad60..9d36ea5015e1 100644 --- a/pkg/storage/engine/enginepb/mvcc3.pb.go +++ b/pkg/storage/engine/enginepb/mvcc3.pb.go @@ -123,7 +123,7 @@ type TxnMeta struct { func (m *TxnMeta) Reset() { *m = TxnMeta{} } func (*TxnMeta) ProtoMessage() {} func (*TxnMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{0} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{0} } func (m *TxnMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -148,6 +148,42 @@ func (m *TxnMeta) XXX_DiscardUnknown() { var xxx_messageInfo_TxnMeta proto.InternalMessageInfo +// IgnoredSeqNumRange describes a range of ignored seqnums. +// The range is inclusive on both ends. +type IgnoredSeqNumRange struct { + Start TxnSeq `protobuf:"varint,1,opt,name=start,proto3,casttype=TxnSeq" json:"start,omitempty"` + End TxnSeq `protobuf:"varint,2,opt,name=end,proto3,casttype=TxnSeq" json:"end,omitempty"` +} + +func (m *IgnoredSeqNumRange) Reset() { *m = IgnoredSeqNumRange{} } +func (m *IgnoredSeqNumRange) String() string { return proto.CompactTextString(m) } +func (*IgnoredSeqNumRange) ProtoMessage() {} +func (*IgnoredSeqNumRange) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{1} +} +func (m *IgnoredSeqNumRange) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IgnoredSeqNumRange) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *IgnoredSeqNumRange) XXX_Merge(src proto.Message) { + xxx_messageInfo_IgnoredSeqNumRange.Merge(dst, src) +} +func (m *IgnoredSeqNumRange) XXX_Size() int { + return m.Size() +} +func (m *IgnoredSeqNumRange) XXX_DiscardUnknown() { + xxx_messageInfo_IgnoredSeqNumRange.DiscardUnknown(m) +} + +var xxx_messageInfo_IgnoredSeqNumRange proto.InternalMessageInfo + // MVCCStatsDelta is convertible to MVCCStats, but uses signed variable width // encodings for most fields that make it more efficient to store negative // values. This makes the encodings incompatible. @@ -172,7 +208,7 @@ func (m *MVCCStatsDelta) Reset() { *m = MVCCStatsDelta{} } func (m *MVCCStatsDelta) String() string { return proto.CompactTextString(m) } func (*MVCCStatsDelta) ProtoMessage() {} func (*MVCCStatsDelta) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{1} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{2} } func (m *MVCCStatsDelta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -222,7 +258,7 @@ func (m *MVCCPersistentStats) Reset() { *m = MVCCPersistentStats{} } func (m *MVCCPersistentStats) String() string { return proto.CompactTextString(m) } func (*MVCCPersistentStats) ProtoMessage() {} func (*MVCCPersistentStats) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{2} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{3} } func (m *MVCCPersistentStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -267,7 +303,7 @@ func (m *RangeAppliedState) Reset() { *m = RangeAppliedState{} } func (m *RangeAppliedState) String() string { return proto.CompactTextString(m) } func (*RangeAppliedState) ProtoMessage() {} func (*RangeAppliedState) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{3} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{4} } func (m *RangeAppliedState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -305,7 +341,7 @@ func (m *MVCCWriteValueOp) Reset() { *m = MVCCWriteValueOp{} } func (m *MVCCWriteValueOp) String() string { return proto.CompactTextString(m) } func (*MVCCWriteValueOp) ProtoMessage() {} func (*MVCCWriteValueOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{4} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{5} } func (m *MVCCWriteValueOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -343,7 +379,7 @@ func (m *MVCCWriteIntentOp) Reset() { *m = MVCCWriteIntentOp{} } func (m *MVCCWriteIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCWriteIntentOp) ProtoMessage() {} func (*MVCCWriteIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{5} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{6} } func (m *MVCCWriteIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -379,7 +415,7 @@ func (m *MVCCUpdateIntentOp) Reset() { *m = MVCCUpdateIntentOp{} } func (m *MVCCUpdateIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCUpdateIntentOp) ProtoMessage() {} func (*MVCCUpdateIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{6} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{7} } func (m *MVCCUpdateIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -418,7 +454,7 @@ func (m *MVCCCommitIntentOp) Reset() { *m = MVCCCommitIntentOp{} } func (m *MVCCCommitIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCCommitIntentOp) ProtoMessage() {} func (*MVCCCommitIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{7} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{8} } func (m *MVCCCommitIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +494,7 @@ func (m *MVCCAbortIntentOp) Reset() { *m = MVCCAbortIntentOp{} } func (m *MVCCAbortIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCAbortIntentOp) ProtoMessage() {} func (*MVCCAbortIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{8} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{9} } func (m *MVCCAbortIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -494,7 +530,7 @@ func (m *MVCCAbortTxnOp) Reset() { *m = MVCCAbortTxnOp{} } func (m *MVCCAbortTxnOp) String() string { return proto.CompactTextString(m) } func (*MVCCAbortTxnOp) ProtoMessage() {} func (*MVCCAbortTxnOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{9} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{10} } func (m *MVCCAbortTxnOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -533,7 +569,7 @@ func (m *MVCCLogicalOp) Reset() { *m = MVCCLogicalOp{} } func (m *MVCCLogicalOp) String() string { return proto.CompactTextString(m) } func (*MVCCLogicalOp) ProtoMessage() {} func (*MVCCLogicalOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_975116ebc9352733, []int{10} + return fileDescriptor_mvcc3_e29d424eb6100aca, []int{11} } func (m *MVCCLogicalOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -560,6 +596,7 @@ var xxx_messageInfo_MVCCLogicalOp proto.InternalMessageInfo func init() { proto.RegisterType((*TxnMeta)(nil), "cockroach.storage.engine.enginepb.TxnMeta") + proto.RegisterType((*IgnoredSeqNumRange)(nil), "cockroach.storage.engine.enginepb.IgnoredSeqNumRange") proto.RegisterType((*MVCCStatsDelta)(nil), "cockroach.storage.engine.enginepb.MVCCStatsDelta") proto.RegisterType((*MVCCPersistentStats)(nil), "cockroach.storage.engine.enginepb.MVCCPersistentStats") proto.RegisterType((*RangeAppliedState)(nil), "cockroach.storage.engine.enginepb.RangeAppliedState") @@ -613,6 +650,33 @@ func (this *TxnMeta) Equal(that interface{}) bool { } return true } +func (this *IgnoredSeqNumRange) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*IgnoredSeqNumRange) + if !ok { + that2, ok := that.(IgnoredSeqNumRange) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Start != that1.Start { + return false + } + if this.End != that1.End { + return false + } + return true +} func (this *MVCCStatsDelta) Equal(that interface{}) bool { if that == nil { return this == nil @@ -832,6 +896,34 @@ func (m *TxnMeta) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *IgnoredSeqNumRange) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *IgnoredSeqNumRange) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Start != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintMvcc3(dAtA, i, uint64(m.Start)) + } + if m.End != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintMvcc3(dAtA, i, uint64(m.End)) + } + return i, nil +} + func (m *MVCCStatsDelta) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1397,6 +1489,21 @@ func NewPopulatedTxnMeta(r randyMvcc3, easy bool) *TxnMeta { return this } +func NewPopulatedIgnoredSeqNumRange(r randyMvcc3, easy bool) *IgnoredSeqNumRange { + this := &IgnoredSeqNumRange{} + this.Start = TxnSeq(r.Int31()) + if r.Intn(2) == 0 { + this.Start *= -1 + } + this.End = TxnSeq(r.Int31()) + if r.Intn(2) == 0 { + this.End *= -1 + } + if !easy && r.Intn(10) != 0 { + } + return this +} + func NewPopulatedMVCCPersistentStats(r randyMvcc3, easy bool) *MVCCPersistentStats { this := &MVCCPersistentStats{} this.LastUpdateNanos = int64(r.Int63()) @@ -1571,6 +1678,21 @@ func (m *TxnMeta) Size() (n int) { return n } +func (m *IgnoredSeqNumRange) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Start != 0 { + n += 1 + sovMvcc3(uint64(m.Start)) + } + if m.End != 0 { + n += 1 + sovMvcc3(uint64(m.End)) + } + return n +} + func (m *MVCCStatsDelta) Size() (n int) { if m == nil { return 0 @@ -2107,6 +2229,94 @@ func (m *TxnMeta) Unmarshal(dAtA []byte) error { } return nil } +func (m *IgnoredSeqNumRange) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMvcc3 + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: IgnoredSeqNumRange: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: IgnoredSeqNumRange: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) + } + m.Start = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMvcc3 + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Start |= (TxnSeq(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field End", wireType) + } + m.End = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMvcc3 + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.End |= (TxnSeq(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipMvcc3(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMvcc3 + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *MVCCStatsDelta) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -3994,80 +4204,83 @@ var ( ) func init() { - proto.RegisterFile("storage/engine/enginepb/mvcc3.proto", fileDescriptor_mvcc3_975116ebc9352733) + proto.RegisterFile("storage/engine/enginepb/mvcc3.proto", fileDescriptor_mvcc3_e29d424eb6100aca) } -var fileDescriptor_mvcc3_975116ebc9352733 = []byte{ - // 1135 bytes of a gzipped FileDescriptorProto +var fileDescriptor_mvcc3_e29d424eb6100aca = []byte{ + // 1175 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0x4f, 0x6f, 0x1b, 0x45, - 0x14, 0xf7, 0x7a, 0xd7, 0x89, 0x3d, 0x76, 0x12, 0x7b, 0x5a, 0x09, 0xab, 0xa8, 0xde, 0xd4, 0x48, - 0x28, 0xe2, 0xcf, 0x1a, 0x12, 0xe0, 0x90, 0x9b, 0x9d, 0x54, 0xe0, 0xd2, 0x34, 0x65, 0xeb, 0xb4, - 0x12, 0x08, 0xad, 0xc6, 0xeb, 0x61, 0x33, 0xca, 0x7a, 0x76, 0xbb, 0x3b, 0x76, 0xd7, 0xdf, 0x82, - 0x23, 0x48, 0x20, 0xe5, 0xc2, 0x37, 0xe0, 0xc0, 0x47, 0xc8, 0xb1, 0xc7, 0xaa, 0x07, 0x0b, 0x9c, - 0x0b, 0x1f, 0x80, 0x53, 0x10, 0x12, 0x9a, 0x99, 0xdd, 0xb5, 0x13, 0x51, 0x27, 0x69, 0x44, 0xd4, - 0x53, 0x66, 0xde, 0xef, 0xbd, 0xdf, 0x7b, 0x7e, 0xef, 0xe7, 0x79, 0x0e, 0x78, 0x27, 0x64, 0x5e, - 0x80, 0x1c, 0xdc, 0xc0, 0xd4, 0x21, 0x34, 0xf9, 0xe3, 0x77, 0x1b, 0xfd, 0xa1, 0x6d, 0x6f, 0x18, - 0x7e, 0xe0, 0x31, 0x0f, 0xde, 0xb1, 0x3d, 0xfb, 0x20, 0xf0, 0x90, 0xbd, 0x6f, 0xc4, 0xee, 0x86, - 0xf4, 0x33, 0x12, 0xf7, 0x5b, 0xd5, 0x01, 0x23, 0x6e, 0x63, 0xdf, 0xb5, 0x1b, 0x8c, 0xf4, 0x71, - 0xc8, 0x50, 0xdf, 0x97, 0xc1, 0xb7, 0x6e, 0x3a, 0x9e, 0xe3, 0x89, 0x63, 0x83, 0x9f, 0xa4, 0xb5, - 0xfe, 0xa3, 0x0a, 0x16, 0x3b, 0x11, 0xdd, 0xc1, 0x0c, 0xc1, 0xaf, 0x40, 0x96, 0xf4, 0xaa, 0xca, - 0xaa, 0xb2, 0x56, 0x6a, 0x35, 0x8f, 0xc6, 0x7a, 0xe6, 0xe5, 0x58, 0xdf, 0x70, 0x08, 0xdb, 0x1f, - 0x74, 0x0d, 0xdb, 0xeb, 0x37, 0xd2, 0xec, 0xbd, 0xee, 0xf4, 0xdc, 0xf0, 0x0f, 0x9c, 0x86, 0x48, - 0x3a, 0x18, 0x90, 0x9e, 0xb1, 0xb7, 0xd7, 0xde, 0x9e, 0x8c, 0xf5, 0x6c, 0x7b, 0xdb, 0xcc, 0x92, - 0x1e, 0x2c, 0x03, 0xf5, 0x00, 0x8f, 0xaa, 0x2a, 0xe7, 0x34, 0xf9, 0x11, 0xd6, 0x41, 0x0e, 0xfb, - 0x9e, 0xbd, 0x5f, 0xd5, 0x56, 0x95, 0xb5, 0x5c, 0xab, 0x74, 0x32, 0xd6, 0xf3, 0x9d, 0x88, 0xde, - 0xe5, 0x36, 0x53, 0x42, 0xf0, 0x3e, 0x58, 0x79, 0x16, 0x10, 0x86, 0xad, 0xf4, 0x33, 0x54, 0x73, - 0xab, 0xca, 0x5a, 0x71, 0xfd, 0xb6, 0x31, 0xed, 0x00, 0xcf, 0x69, 0xec, 0xbb, 0xb6, 0xd1, 0x49, - 0x9c, 0x5a, 0x1a, 0x2f, 0xda, 0x5c, 0x16, 0xb1, 0xa9, 0x15, 0xbe, 0x0f, 0xf2, 0x7e, 0x40, 0xbc, - 0x80, 0xb0, 0x51, 0x75, 0x41, 0x24, 0x5d, 0x39, 0x19, 0xeb, 0xc5, 0x4e, 0x44, 0x1f, 0xc6, 0x66, - 0x33, 0x75, 0x80, 0xef, 0x82, 0x7c, 0x88, 0x9f, 0x0e, 0x30, 0xb5, 0x71, 0x75, 0x51, 0x38, 0x83, - 0x93, 0xb1, 0xbe, 0xd0, 0x89, 0xe8, 0x23, 0xfc, 0xd4, 0x4c, 0x31, 0xf8, 0x05, 0x58, 0xea, 0x13, - 0x3a, 0x53, 0x60, 0xe1, 0xe2, 0x05, 0x96, 0xfa, 0x84, 0xa6, 0xb6, 0xcd, 0xd2, 0x0f, 0x87, 0x7a, - 0xe6, 0xb7, 0x43, 0x5d, 0xf9, 0xf3, 0x50, 0x57, 0xee, 0x69, 0xf9, 0x6c, 0x59, 0xbd, 0xa7, 0xe5, - 0xf3, 0xe5, 0x42, 0xfd, 0x2f, 0x15, 0x2c, 0xef, 0x3c, 0xde, 0xda, 0x7a, 0xc4, 0x10, 0x0b, 0xb7, - 0xb1, 0xcb, 0x10, 0x7c, 0x0f, 0x54, 0x5c, 0x14, 0x32, 0x6b, 0xe0, 0xf7, 0x10, 0xc3, 0x16, 0x45, - 0xd4, 0x0b, 0xc5, 0xc4, 0xca, 0xe6, 0x0a, 0x07, 0xf6, 0x84, 0xfd, 0x01, 0x37, 0xc3, 0xdb, 0x00, - 0x10, 0xca, 0x30, 0x65, 0x16, 0x72, 0x70, 0x35, 0x2b, 0x9c, 0x0a, 0xd2, 0xd2, 0x74, 0x30, 0xfc, - 0x08, 0x94, 0x1c, 0xdb, 0xea, 0x8e, 0x18, 0x0e, 0x85, 0x03, 0x9f, 0x51, 0xb9, 0xb5, 0x3c, 0x19, - 0xeb, 0xe0, 0xf3, 0xad, 0x16, 0x37, 0x37, 0x1d, 0x6c, 0x02, 0xc7, 0x4e, 0xce, 0x9c, 0xd0, 0x25, - 0x43, 0x2c, 0x63, 0xc4, 0xfc, 0xa0, 0x59, 0xe0, 0x16, 0xe1, 0x91, 0xc2, 0xb6, 0x37, 0xa0, 0x4c, - 0x0c, 0x2c, 0x86, 0xb7, 0xb8, 0x01, 0xbe, 0x0d, 0x0a, 0x07, 0x78, 0x14, 0x07, 0x2f, 0x08, 0x34, - 0x7f, 0x80, 0x47, 0x32, 0x36, 0x06, 0x65, 0xe8, 0x62, 0x0a, 0xa6, 0x91, 0x43, 0xe4, 0xc6, 0x91, - 0x79, 0x09, 0x0e, 0x91, 0x9b, 0x46, 0x72, 0x50, 0x46, 0x16, 0x52, 0x50, 0x46, 0xde, 0x01, 0xa5, - 0xb8, 0x05, 0x32, 0x18, 0x08, 0xbc, 0x28, 0x6d, 0x32, 0x7e, 0xea, 0x22, 0x29, 0x8a, 0xb3, 0x2e, - 0x69, 0xfe, 0x70, 0x14, 0xc6, 0x14, 0x25, 0x99, 0x22, 0x1c, 0x85, 0x69, 0x7e, 0x0e, 0xca, 0xe0, - 0xa5, 0x14, 0x94, 0x91, 0x1f, 0x02, 0x68, 0x7b, 0x94, 0x21, 0x42, 0x43, 0x0b, 0x87, 0x8c, 0xf4, - 0x11, 0xa7, 0x58, 0x5e, 0x55, 0xd6, 0x54, 0xb3, 0x92, 0x20, 0x77, 0x13, 0x60, 0x53, 0xe3, 0x12, - 0xa8, 0xff, 0xa3, 0x82, 0x1b, 0x7c, 0xec, 0x0f, 0x71, 0x10, 0x92, 0x90, 0x97, 0x21, 0x04, 0xf0, - 0xa6, 0xcd, 0x5e, 0x9d, 0x3f, 0x7b, 0x75, 0xee, 0xec, 0xd5, 0x79, 0xb3, 0x57, 0xe7, 0xcd, 0x5e, - 0x9d, 0x37, 0x7b, 0xf5, 0x9c, 0xd9, 0xab, 0xe7, 0xcf, 0x5e, 0x3d, 0x67, 0xf6, 0xea, 0xbc, 0xd9, - 0xab, 0xaf, 0x3f, 0xfb, 0x7c, 0xf2, 0x04, 0xd4, 0x5f, 0x2a, 0xa0, 0x62, 0x22, 0xea, 0xe0, 0xa6, - 0xef, 0xbb, 0x04, 0xf7, 0xf8, 0xf4, 0x31, 0xfc, 0x00, 0xc0, 0x00, 0x7d, 0xc7, 0x2c, 0x24, 0x8d, - 0x16, 0xa1, 0x3d, 0x1c, 0x89, 0xf1, 0x6b, 0x66, 0x99, 0x23, 0xb1, 0x77, 0x9b, 0xdb, 0xa1, 0x01, - 0x6e, 0xb8, 0x18, 0x85, 0xf8, 0x8c, 0x7b, 0x56, 0xb8, 0x57, 0x04, 0x74, 0xca, 0xff, 0x5b, 0x50, - 0x0c, 0x78, 0x4a, 0x2b, 0xe4, 0x52, 0x13, 0x7a, 0x28, 0xae, 0x7f, 0x66, 0x9c, 0xbb, 0x6f, 0x8c, - 0xff, 0x10, 0x6a, 0xfc, 0xca, 0x01, 0x41, 0x28, 0x2c, 0x33, 0x1f, 0xee, 0x27, 0x05, 0x94, 0x79, - 0xcc, 0x13, 0xfe, 0x46, 0x3f, 0x46, 0xee, 0x00, 0xef, 0xfa, 0xc9, 0x96, 0x50, 0xa6, 0x5b, 0xa2, - 0x09, 0x0a, 0xd3, 0xa7, 0x35, 0x7b, 0xf1, 0xa7, 0x75, 0x1a, 0x05, 0x6f, 0x82, 0xdc, 0x90, 0xf3, - 0xc7, 0xcb, 0x47, 0x5e, 0xb8, 0x50, 0xfd, 0x00, 0x0f, 0x2d, 0x09, 0x69, 0x02, 0x2a, 0x70, 0x8b, - 0xa8, 0xa5, 0xfe, 0x73, 0x16, 0x54, 0xd2, 0xf2, 0xda, 0x42, 0x07, 0xbb, 0x3e, 0xfc, 0x06, 0x2c, - 0xb0, 0x88, 0x5a, 0xe9, 0x72, 0xdc, 0xbe, 0xda, 0x72, 0xcc, 0x75, 0x22, 0xda, 0xde, 0x36, 0x73, - 0x2c, 0xa2, 0xed, 0x1e, 0x7c, 0x0b, 0x2c, 0x72, 0x72, 0xde, 0x80, 0xac, 0x28, 0x87, 0xe7, 0xfa, - 0xf2, 0x6c, 0x0f, 0xd4, 0xd7, 0xea, 0xc1, 0x2e, 0xa8, 0x70, 0xee, 0xd3, 0x9b, 0x4a, 0xbb, 0x38, - 0xd5, 0x0a, 0x8b, 0xe8, 0xce, 0xcc, 0xb2, 0xaa, 0xff, 0xaa, 0x00, 0xc8, 0xfb, 0x23, 0xdf, 0x9a, - 0xeb, 0x69, 0xd0, 0xd5, 0xb5, 0x50, 0xff, 0x3b, 0x2e, 0x7b, 0xcb, 0xeb, 0xf7, 0x09, 0xbb, 0x9e, - 0xb2, 0x63, 0x51, 0x67, 0x5f, 0x21, 0x6a, 0xf5, 0x6a, 0xa2, 0xd6, 0x5e, 0x2d, 0xea, 0xdc, 0x59, - 0x51, 0xfb, 0x52, 0xd3, 0xcd, 0xae, 0x17, 0x5c, 0xcf, 0x67, 0xaf, 0xf7, 0xe5, 0x0f, 0x17, 0x91, - 0xb1, 0x13, 0xd1, 0xff, 0x3b, 0xdd, 0x2f, 0x1a, 0x58, 0xe2, 0xf9, 0xee, 0x7b, 0x0e, 0xb1, 0x91, - 0xbb, 0xeb, 0xc3, 0x0e, 0x28, 0xca, 0x5f, 0x90, 0xb2, 0x25, 0x8a, 0x68, 0xf6, 0xc6, 0x05, 0xdf, - 0xb3, 0xd9, 0xb7, 0xc9, 0x04, 0xcf, 0xd2, 0x1b, 0x7c, 0x02, 0x4a, 0x92, 0x55, 0x6e, 0x88, 0x58, - 0x8c, 0x9f, 0x5c, 0x86, 0x36, 0xe9, 0xbf, 0x29, 0xeb, 0x93, 0x57, 0xf8, 0x35, 0x58, 0x8a, 0xb7, - 0x7a, 0xcc, 0x2c, 0xd5, 0xf1, 0xe9, 0x05, 0x99, 0x4f, 0x7f, 0x1b, 0xcd, 0xd2, 0x60, 0xe6, 0xce, - 0xb9, 0x6d, 0x21, 0xfb, 0x84, 0x5b, 0xbb, 0x14, 0xf7, 0xe9, 0xaf, 0x8c, 0x59, 0xb2, 0x67, 0xee, - 0xbc, 0x21, 0x88, 0xcf, 0x38, 0xa1, 0xce, 0x5d, 0xaa, 0x21, 0xa7, 0x04, 0x69, 0x16, 0xd1, 0xf4, - 0x0a, 0x1f, 0x80, 0x82, 0x24, 0x66, 0x11, 0x15, 0x3f, 0x18, 0x8a, 0xeb, 0x1f, 0x5f, 0x86, 0x55, - 0x88, 0xce, 0xcc, 0xa3, 0xf8, 0xbc, 0xa9, 0x1d, 0x1d, 0xea, 0x4a, 0x6b, 0xf5, 0xe8, 0x8f, 0x5a, - 0xe6, 0x68, 0x52, 0x53, 0x9e, 0x4f, 0x6a, 0xca, 0x8b, 0x49, 0x4d, 0xf9, 0x7d, 0x52, 0x53, 0xbe, - 0x3f, 0xae, 0x65, 0x9e, 0x1f, 0xd7, 0x32, 0x2f, 0x8e, 0x6b, 0x99, 0xee, 0x82, 0xf8, 0xaf, 0x68, - 0xe3, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x52, 0x3f, 0xe8, 0x2e, 0x8f, 0x0d, 0x00, 0x00, + 0x14, 0xf7, 0x7a, 0xd7, 0x89, 0xfd, 0xec, 0x24, 0xf6, 0xb4, 0x12, 0x56, 0xa1, 0xb6, 0x6b, 0x24, + 0x14, 0xf1, 0x67, 0x0d, 0x09, 0x70, 0xc8, 0xcd, 0x4e, 0x2a, 0x70, 0x69, 0x92, 0xb2, 0x71, 0x5a, + 0x09, 0x04, 0xd6, 0x78, 0x3d, 0x6c, 0x56, 0x59, 0xcf, 0x6e, 0x76, 0xc7, 0xae, 0xfd, 0x2d, 0x38, + 0x82, 0x04, 0x52, 0x2e, 0x7c, 0x03, 0x0e, 0x7c, 0x84, 0x1c, 0x7b, 0xac, 0x7a, 0xb0, 0xc0, 0xb9, + 0xf0, 0x01, 0x38, 0x05, 0x21, 0xa1, 0x99, 0x59, 0xaf, 0xed, 0x40, 0x9d, 0xa4, 0x11, 0x11, 0xa7, + 0xcc, 0xbc, 0xdf, 0x7b, 0xbf, 0xf7, 0xfc, 0xde, 0xcf, 0xf3, 0x1c, 0x78, 0x33, 0x60, 0xae, 0x8f, + 0x2d, 0x52, 0x21, 0xd4, 0xb2, 0xe9, 0xf8, 0x8f, 0xd7, 0xaa, 0x74, 0x7a, 0xa6, 0xb9, 0xae, 0x7b, + 0xbe, 0xcb, 0x5c, 0x74, 0xcf, 0x74, 0xcd, 0x43, 0xdf, 0xc5, 0xe6, 0x81, 0x1e, 0xba, 0xeb, 0xd2, + 0x4f, 0x1f, 0xbb, 0xdf, 0xc9, 0x77, 0x99, 0xed, 0x54, 0x0e, 0x1c, 0xb3, 0xc2, 0xec, 0x0e, 0x09, + 0x18, 0xee, 0x78, 0x32, 0xf8, 0xce, 0x6d, 0xcb, 0xb5, 0x5c, 0x71, 0xac, 0xf0, 0x93, 0xb4, 0x96, + 0xbf, 0x57, 0x61, 0xb1, 0xd1, 0xa7, 0xdb, 0x84, 0x61, 0xf4, 0x39, 0xc4, 0xed, 0x76, 0x5e, 0x29, + 0x29, 0xab, 0x99, 0x5a, 0xf5, 0x64, 0x58, 0x8c, 0xbd, 0x18, 0x16, 0xd7, 0x2d, 0x9b, 0x1d, 0x74, + 0x5b, 0xba, 0xe9, 0x76, 0x2a, 0x51, 0xf6, 0x76, 0x6b, 0x72, 0xae, 0x78, 0x87, 0x56, 0x45, 0x24, + 0xed, 0x76, 0xed, 0xb6, 0xbe, 0xbf, 0x5f, 0xdf, 0x1a, 0x0d, 0x8b, 0xf1, 0xfa, 0x96, 0x11, 0xb7, + 0xdb, 0x28, 0x0b, 0xea, 0x21, 0x19, 0xe4, 0x55, 0xce, 0x69, 0xf0, 0x23, 0x2a, 0x43, 0x82, 0x78, + 0xae, 0x79, 0x90, 0xd7, 0x4a, 0xca, 0x6a, 0xa2, 0x96, 0x39, 0x1b, 0x16, 0x93, 0x8d, 0x3e, 0xbd, + 0xcf, 0x6d, 0x86, 0x84, 0xd0, 0x43, 0x58, 0x79, 0xea, 0xdb, 0x8c, 0x34, 0xa3, 0xcf, 0x90, 0x4f, + 0x94, 0x94, 0xd5, 0xf4, 0xda, 0x5d, 0x7d, 0xd2, 0x01, 0x9e, 0x53, 0x3f, 0x70, 0x4c, 0xbd, 0x31, + 0x76, 0xaa, 0x69, 0xbc, 0x68, 0x63, 0x59, 0xc4, 0x46, 0x56, 0xf4, 0x0e, 0x24, 0x3d, 0xdf, 0x76, + 0x7d, 0x9b, 0x0d, 0xf2, 0x0b, 0x22, 0xe9, 0xca, 0xd9, 0xb0, 0x98, 0x6e, 0xf4, 0xe9, 0xa3, 0xd0, + 0x6c, 0x44, 0x0e, 0xe8, 0x2d, 0x48, 0x06, 0xe4, 0xa8, 0x4b, 0xa8, 0x49, 0xf2, 0x8b, 0xc2, 0x19, + 0xce, 0x86, 0xc5, 0x85, 0x46, 0x9f, 0xee, 0x91, 0x23, 0x23, 0xc2, 0xd0, 0xa7, 0xb0, 0xd4, 0xb1, + 0xe9, 0x54, 0x81, 0xa9, 0xcb, 0x17, 0x98, 0xe9, 0xd8, 0x34, 0xb2, 0x6d, 0x64, 0xbe, 0x3b, 0x2e, + 0xc6, 0x7e, 0x39, 0x2e, 0x2a, 0xbf, 0x1f, 0x17, 0x95, 0x07, 0x5a, 0x32, 0x9e, 0x55, 0x1f, 0x68, + 0xc9, 0x64, 0x36, 0x55, 0xfe, 0x1a, 0x50, 0xdd, 0xa2, 0xae, 0x4f, 0xda, 0x7b, 0xe4, 0x68, 0xa7, + 0xdb, 0x31, 0x30, 0xb5, 0x08, 0x2a, 0x41, 0x22, 0x60, 0xd8, 0x67, 0x62, 0x50, 0xb3, 0xe5, 0x49, + 0x00, 0xbd, 0x01, 0x2a, 0xa1, 0xed, 0x7c, 0xfc, 0x1f, 0x38, 0x37, 0x6f, 0x24, 0xc7, 0xb9, 0xca, + 0x7f, 0xa8, 0xb0, 0xbc, 0xfd, 0x78, 0x73, 0x73, 0x8f, 0x61, 0x16, 0x6c, 0x11, 0x87, 0x61, 0xf4, + 0x36, 0xe4, 0x1c, 0x1c, 0xb0, 0x66, 0xd7, 0x6b, 0x63, 0x46, 0x9a, 0x14, 0x53, 0x37, 0x10, 0x89, + 0xb2, 0xc6, 0x0a, 0x07, 0xf6, 0x85, 0x7d, 0x87, 0x9b, 0xd1, 0x5d, 0x00, 0x9b, 0x32, 0x42, 0x59, + 0x13, 0x5b, 0x44, 0x64, 0xcb, 0x1a, 0x29, 0x69, 0xa9, 0x5a, 0x04, 0xbd, 0x0f, 0x19, 0xcb, 0x6c, + 0xb6, 0x06, 0x8c, 0x04, 0xc2, 0x81, 0x6b, 0x20, 0x5b, 0x5b, 0x1e, 0x0d, 0x8b, 0xf0, 0xc9, 0x66, + 0x8d, 0x9b, 0xab, 0x16, 0x31, 0xc0, 0x32, 0xc7, 0x67, 0x4e, 0xe8, 0xd8, 0x3d, 0x22, 0x63, 0x84, + 0x3e, 0x90, 0x91, 0xe2, 0x16, 0xe1, 0x11, 0xc1, 0xa6, 0xdb, 0xa5, 0x4c, 0x08, 0x22, 0x84, 0x37, + 0xb9, 0x01, 0xbd, 0x0e, 0xa9, 0x43, 0x32, 0x08, 0x83, 0x17, 0x04, 0x9a, 0x3c, 0x24, 0x03, 0x19, + 0x1b, 0x82, 0x32, 0x74, 0x31, 0x02, 0xa3, 0xc8, 0x1e, 0x76, 0xc2, 0xc8, 0xa4, 0x04, 0x7b, 0xd8, + 0x89, 0x22, 0x39, 0x28, 0x23, 0x53, 0x11, 0x28, 0x23, 0xef, 0x41, 0x26, 0x6c, 0x81, 0x0c, 0x06, + 0x81, 0xa7, 0xa5, 0x4d, 0xc6, 0x4f, 0x5c, 0x24, 0x45, 0x7a, 0xda, 0x25, 0xca, 0x1f, 0x0c, 0x82, + 0x90, 0x22, 0x23, 0x53, 0x04, 0x83, 0x20, 0xca, 0xcf, 0x41, 0x19, 0xbc, 0x14, 0x81, 0x32, 0xf2, + 0x3d, 0x40, 0xa6, 0x4b, 0x19, 0xb6, 0x69, 0xd0, 0x24, 0x01, 0xb3, 0x3b, 0x98, 0x53, 0x2c, 0x97, + 0x94, 0x55, 0xd5, 0xc8, 0x8d, 0x91, 0xfb, 0x63, 0x60, 0x43, 0x13, 0x63, 0xff, 0x4b, 0x85, 0x5b, + 0x7c, 0xec, 0x8f, 0x88, 0x1f, 0xd8, 0x01, 0x2f, 0x43, 0x08, 0xe0, 0xff, 0x36, 0x7b, 0x75, 0xfe, + 0xec, 0xd5, 0xb9, 0xb3, 0x57, 0xe7, 0xcd, 0x5e, 0x9d, 0x37, 0x7b, 0x75, 0xde, 0xec, 0xd5, 0x0b, + 0x66, 0xaf, 0x5e, 0x3c, 0x7b, 0xf5, 0x82, 0xd9, 0xab, 0xf3, 0x66, 0xaf, 0xbe, 0xfa, 0xec, 0x27, + 0x5f, 0xfb, 0x17, 0x0a, 0xe4, 0xc4, 0x53, 0x52, 0xf5, 0x3c, 0xc7, 0x26, 0x6d, 0x3e, 0x7d, 0x82, + 0xde, 0x05, 0xe4, 0xe3, 0x6f, 0x58, 0x13, 0x4b, 0x63, 0xd3, 0xa6, 0x6d, 0xd2, 0x17, 0xe3, 0xd7, + 0x8c, 0x2c, 0x47, 0x42, 0xef, 0x3a, 0xb7, 0x23, 0x1d, 0x6e, 0x39, 0x04, 0x07, 0xe4, 0x9c, 0x7b, + 0x5c, 0xb8, 0xe7, 0x04, 0x34, 0xe3, 0xff, 0x15, 0xa4, 0x7d, 0x9e, 0xb2, 0x19, 0x70, 0xa9, 0x09, + 0x3d, 0xa4, 0xd7, 0x3e, 0xd6, 0x2f, 0xdc, 0x67, 0xfa, 0xbf, 0x08, 0x35, 0x7c, 0x45, 0x41, 0x10, + 0x0a, 0xcb, 0xd4, 0x87, 0xfb, 0x41, 0x81, 0x2c, 0x8f, 0x79, 0xc2, 0x77, 0xc0, 0x63, 0xec, 0x74, + 0xc9, 0xae, 0x37, 0xde, 0x42, 0xca, 0x64, 0x0b, 0x55, 0x21, 0x35, 0x79, 0xba, 0xe3, 0x97, 0x7f, + 0xba, 0x27, 0x51, 0xe8, 0x36, 0x24, 0x7a, 0x9c, 0x3f, 0x5c, 0x6e, 0xf2, 0xc2, 0x85, 0xea, 0xf9, + 0xa4, 0xd7, 0x94, 0x90, 0x26, 0xa0, 0x14, 0xb7, 0x88, 0x5a, 0xca, 0x3f, 0xc6, 0x21, 0x17, 0x95, + 0x57, 0x17, 0x3a, 0xd8, 0xf5, 0xd0, 0x97, 0xb0, 0xc0, 0xfa, 0xb4, 0x19, 0x2d, 0xdf, 0xad, 0xeb, + 0x2d, 0xdf, 0x44, 0xa3, 0x4f, 0xeb, 0x5b, 0x46, 0x82, 0xf5, 0x69, 0xbd, 0x8d, 0x5e, 0x83, 0x45, + 0x4e, 0xce, 0x1b, 0x10, 0x17, 0xe5, 0xf0, 0x5c, 0x9f, 0x9d, 0xef, 0x81, 0xfa, 0x4a, 0x3d, 0xd8, + 0x85, 0x1c, 0xe7, 0x9e, 0xdd, 0x84, 0xda, 0xe5, 0xa9, 0x56, 0x58, 0x9f, 0x6e, 0x4f, 0x2d, 0xc3, + 0xf2, 0xcf, 0x0a, 0x20, 0xde, 0x1f, 0xf9, 0xd6, 0xdc, 0x4c, 0x83, 0xae, 0xaf, 0x85, 0xf2, 0x9f, + 0x61, 0xd9, 0x9b, 0x6e, 0xa7, 0x63, 0xb3, 0x9b, 0x29, 0x3b, 0x14, 0x75, 0xfc, 0x25, 0xa2, 0x56, + 0xaf, 0x27, 0x6a, 0xed, 0xe5, 0xa2, 0x4e, 0x9c, 0x17, 0xb5, 0x27, 0x35, 0x5d, 0x6d, 0xb9, 0xfe, + 0xcd, 0x7c, 0xf6, 0x72, 0x47, 0xfe, 0x70, 0x11, 0x19, 0x1b, 0x7d, 0xfa, 0x5f, 0xa7, 0xfb, 0x49, + 0x83, 0x25, 0x9e, 0xef, 0xa1, 0x6b, 0xd9, 0x26, 0x76, 0x76, 0x3d, 0xd4, 0x80, 0xb4, 0xfc, 0x85, + 0x2a, 0x5b, 0xa2, 0x88, 0x66, 0xaf, 0x5f, 0xf2, 0x3d, 0x9b, 0x7e, 0x9b, 0x0c, 0x78, 0x1a, 0xdd, + 0xd0, 0x13, 0xc8, 0x48, 0x56, 0xb9, 0x21, 0x42, 0x31, 0x7e, 0x78, 0x15, 0xda, 0x71, 0xff, 0x0d, + 0x59, 0x9f, 0xbc, 0xa2, 0x2f, 0x60, 0x29, 0xdc, 0xea, 0x21, 0xb3, 0x54, 0xc7, 0x47, 0x97, 0x64, + 0x9e, 0xfd, 0x36, 0x1a, 0x99, 0xee, 0xd4, 0x9d, 0x73, 0x9b, 0x42, 0xf6, 0x63, 0x6e, 0xed, 0x4a, + 0xdc, 0xb3, 0x5f, 0x19, 0x23, 0x63, 0x4e, 0xdd, 0x79, 0x43, 0x30, 0x9f, 0xf1, 0x98, 0x3a, 0x71, + 0xa5, 0x86, 0xcc, 0x08, 0xd2, 0x48, 0xe3, 0xc9, 0x15, 0xed, 0x40, 0x4a, 0x12, 0xb3, 0x3e, 0x15, + 0x3f, 0x18, 0xd2, 0x6b, 0x1f, 0x5c, 0x85, 0x55, 0x88, 0xce, 0x48, 0xe2, 0xf0, 0xbc, 0xa1, 0x9d, + 0x1c, 0x17, 0x95, 0x5a, 0xe9, 0xe4, 0xb7, 0x42, 0xec, 0x64, 0x54, 0x50, 0x9e, 0x8d, 0x0a, 0xca, + 0xf3, 0x51, 0x41, 0xf9, 0x75, 0x54, 0x50, 0xbe, 0x3d, 0x2d, 0xc4, 0x9e, 0x9d, 0x16, 0x62, 0xcf, + 0x4f, 0x0b, 0xb1, 0xd6, 0x82, 0xf8, 0xaf, 0x6b, 0xfd, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x4d, + 0x60, 0x0d, 0x7f, 0xef, 0x0d, 0x00, 0x00, } diff --git a/pkg/storage/engine/enginepb/mvcc3.proto b/pkg/storage/engine/enginepb/mvcc3.proto index 9ecb68a6b9d4..e902af25f3e2 100644 --- a/pkg/storage/engine/enginepb/mvcc3.proto +++ b/pkg/storage/engine/enginepb/mvcc3.proto @@ -116,6 +116,15 @@ message TxnMeta { reserved 8; } +// IgnoredSeqNumRange describes a range of ignored seqnums. +// The range is inclusive on both ends. +message IgnoredSeqNumRange { + option (gogoproto.equal) = true; + option (gogoproto.populate) = true; + int32 start = 1 [(gogoproto.casttype) = "TxnSeq"]; + int32 end = 2 [(gogoproto.casttype) = "TxnSeq"]; +} + // MVCCStatsDelta is convertible to MVCCStats, but uses signed variable width // encodings for most fields that make it more efficient to store negative // values. This makes the encodings incompatible. diff --git a/pkg/storage/engine/enginepb/mvcc_test.go b/pkg/storage/engine/enginepb/mvcc_test.go index 865d8c8ce8b2..147080177a66 100644 --- a/pkg/storage/engine/enginepb/mvcc_test.go +++ b/pkg/storage/engine/enginepb/mvcc_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/stretchr/testify/assert" ) func TestFormatMVCCMetadata(t *testing.T) { @@ -71,3 +72,32 @@ func TestFormatMVCCMetadata(t *testing.T) { expV, str) } } + +func TestTxnSeqIsIgnored(t *testing.T) { + type s = enginepb.TxnSeq + type r = enginepb.IgnoredSeqNumRange + mr := func(a, b s) r { + return r{Start: a, End: b} + } + + testData := []struct { + list []r + ignored []s + notIgnored []s + }{ + {[]r{}, nil, []s{0, 1, 10}}, + {[]r{mr(1, 1)}, []s{1}, []s{0, 2, 10}}, + {[]r{mr(1, 1), mr(2, 3)}, []s{1, 2, 3}, []s{0, 4, 10}}, + {[]r{mr(1, 2), mr(4, 8), mr(9, 10)}, []s{1, 2, 5, 10}, []s{0, 3, 11}}, + {[]r{mr(0, 10)}, []s{0, 1, 2, 3, 10}, []s{11, 100}}, + } + + for _, tc := range testData { + for _, ign := range tc.ignored { + assert.True(t, enginepb.TxnSeqIsIgnored(ign, tc.list)) + } + for _, notIgn := range tc.notIgnored { + assert.False(t, enginepb.TxnSeqIsIgnored(notIgn, tc.list)) + } + } +} diff --git a/pkg/storage/engine/mvcc.go b/pkg/storage/engine/mvcc.go index aac3b7a8baf0..b7025ff88517 100644 --- a/pkg/storage/engine/mvcc.go +++ b/pkg/storage/engine/mvcc.go @@ -935,11 +935,8 @@ func mvccGetInternal( // ignore the intent by insisting that the timestamp we're reading // at is a historical timestamp < the intent timestamp. However, we // return the intent separately; the caller may want to resolve it. - ignoredIntent = &roachpb.Intent{ - Span: roachpb.Span{Key: metaKey.Key}, - Status: roachpb.PENDING, - Txn: *meta.Txn, - } + intent := roachpb.MakeErrorIntent(*meta.Txn, roachpb.Span{Key: metaKey.Key}) + ignoredIntent = &intent timestamp = metaTimestamp.Prev() } @@ -957,9 +954,9 @@ func mvccGetInternal( } if !maxVisibleTimestamp.Less(metaTimestamp) { return nil, nil, safeValue, &roachpb.WriteIntentError{ - Intents: []roachpb.Intent{{ - Span: roachpb.Span{Key: metaKey.Key}, Status: roachpb.PENDING, Txn: *meta.Txn, - }}, + Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(*meta.Txn, roachpb.Span{Key: metaKey.Key}), + }, } } } @@ -1323,36 +1320,39 @@ func replayTransactionalWrite( // If the valueFn is specified, we must apply it to the would-be value at the key. if valueFn != nil { - prevSeq, prevValueWritten := meta.GetPrevIntentSeq(txn.Sequence) + var exVal *roachpb.Value + + // If there's an intent history, use that. + prevIntent, prevValueWritten := meta.GetPrevIntentSeq(txn.Sequence, txn.IgnoredSeqNums) if prevValueWritten { // If the previous value was found in the IntentHistory, // simply apply the value function to the historic value // to get the would-be value. - prevVal, _ := meta.GetIntentValue(prevSeq) - value, err = valueFn(&roachpb.Value{RawBytes: prevVal}) - if err != nil { - return err - } - } else { - // If the previous value at the key wasn't written by this transaction, - // we must apply the value function to the last committed value on the key. + prevVal := prevIntent.Value + + exVal = &roachpb.Value{RawBytes: prevVal} + } + if exVal == nil { + // If the previous value at the key wasn't written by this + // transaction, or it was hidden by a rolled back seqnum, we + // look at last committed value on the key. getBuf := newGetBuffer() defer getBuf.release() getBuf.meta = buf.meta - var exVal *roachpb.Value - var err error // Since we want the last committed value on the key, we must make // an inconsistent read so we ignore our previous intents here. - if exVal, _, _, err = mvccGetInternal( - ctx, iter, metaKey, timestamp, false /* consistent */, unsafeValue, nil /* txn */, getBuf); err != nil { - return err - } - value, err = valueFn(exVal) + exVal, _, _, err = mvccGetInternal( + ctx, iter, metaKey, timestamp, false /* consistent */, unsafeValue, nil /* txn */, getBuf) if err != nil { return err } } + + value, err = valueFn(exVal) + if err != nil { + return err + } } // To ensure the transaction is idempotent, we must assert that the @@ -1496,7 +1496,9 @@ func mvccPutInternal( if txn == nil || meta.Txn.ID != txn.ID { // The current Put operation does not come from the same // transaction. - return &roachpb.WriteIntentError{Intents: []roachpb.Intent{{Span: roachpb.Span{Key: key}, Status: roachpb.PENDING, Txn: *meta.Txn}}} + return &roachpb.WriteIntentError{Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(*meta.Txn, roachpb.Span{Key: key}), + }} } else if txn.Epoch < meta.Txn.Epoch { return errors.Errorf("put with epoch %d came after put with epoch %d in txn %s", txn.Epoch, meta.Txn.Epoch, txn.ID) @@ -1508,29 +1510,65 @@ func mvccPutInternal( } // We're overwriting the intent that was present at this key, before we do - // that though - we must record the older intent in the IntentHistory. - var prevIntentValBytes []byte - getBuf := newGetBuffer() - // Release the buffer after using the existing value. - defer getBuf.release() - getBuf.meta = buf.meta // initialize get metadata from what we've already read + // that though - we must record the older value in the IntentHistory. + + // But where to find the older value? There are 4 cases: + // - last write inside txn, same epoch, seqnum of last write is not ignored: value at key. + // => read the value associated with the intent with consistent mvccGetInternal(). + // (This is the common case.) + // - last write inside txn, same epoch, seqnum of last write is ignored: cannot use value at key. + // => try reading from intent history. + // => if all intent history entries are rolled back, fall back to last case below. + // - last write outside txn or at different epoch: use inconsistent mvccGetInternal, + // which will find it outside. + // + // (Note that _some_ value is guaranteed to be found, as indicated by ok == true above.) + + var existingVal *roachpb.Value + if txn.Epoch == meta.Txn.Epoch /* last write inside txn */ { + if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, txn.IgnoredSeqNums) { + // Seqnum of last write is not ignored. Retrieve the value + // using a consistent read. + getBuf := newGetBuffer() + // Release the buffer after using the existing value. + defer getBuf.release() + getBuf.meta = buf.meta // initialize get metadata from what we've already read + + existingVal, _, _, err = mvccGetInternal( + ctx, iter, metaKey, readTimestamp, true /* consistent */, safeValue, txn, getBuf) + if err != nil { + return err + } + } else { + // Seqnum of last write was ignored. Try retrieving the value from the history. + prevIntent, prevValueWritten := meta.GetPrevIntentSeq(txn.Sequence, txn.IgnoredSeqNums) + if prevValueWritten { + existingVal = &roachpb.Value{RawBytes: prevIntent.Value} + } + } - existingVal, _, _, err := mvccGetInternal( - ctx, iter, metaKey, readTimestamp, true /* consistent */, safeValue, txn, getBuf) - if err != nil { - return err } - // It's possible that the existing value is nil if the intent on the key - // has a lower epoch. We don't have to deal with this as a special case - // because in this case, the value isn't written to the intent history. - // Instead, the intent history is blown away completely. - if existingVal != nil { - prevIntentValBytes = existingVal.RawBytes + if existingVal == nil { + + // "last write inside txn && seqnum of last write is not ignored" + // OR + // "last write outside txn" + // => use inconsistent mvccGetInternal to retrieve the last committed value at key. + getBuf := newGetBuffer() + defer getBuf.release() + getBuf.meta = buf.meta + + // Since we want the last committed value on the key, we must make + // an inconsistent read so we ignore our previous intents here. + existingVal, _, _, err = mvccGetInternal( + ctx, iter, metaKey, readTimestamp, false /* consistent */, unsafeValue, nil /* txn */, getBuf) + if err != nil { + return err + } } - prevIntentSequence := meta.Txn.Sequence // Make sure we process valueFn before clearing any earlier - // version. For example, a conditional put within same + // version. For example, a conditional put within same // transaction should read previous write. if valueFn != nil { value, err = valueFn(existingVal) @@ -1539,6 +1577,16 @@ func mvccPutInternal( } } + // It's possible that the existing value is nil if the intent on the key + // has a lower epoch. We don't have to deal with this as a special case + // because in this case, the value isn't written to the intent history. + // Instead, the intent history is blown away completely. + var prevIntentValBytes []byte + if existingVal != nil { + prevIntentValBytes = existingVal.RawBytes + } + prevIntentSequence := meta.Txn.Sequence + // We are replacing our own write intent. If we are writing at // the same timestamp (see comments in else block) we can // overwrite the existing intent; otherwise we must manually @@ -1581,10 +1629,11 @@ func mvccPutInternal( // If the epoch of the transaction doesn't match the epoch of the // intent, blow away the intent history. if txn.Epoch == meta.Txn.Epoch { - // This case shouldn't pop up, but it is worth asserting - // that it doesn't. We shouldn't write invalid intents - // to the history + if existingVal == nil { + // This case shouldn't pop up, but it is worth asserting + // that it doesn't. We shouldn't write invalid intents + // to the history. return errors.Errorf( "previous intent of the transaction with the same epoch not found for %s (%+v)", metaKey, txn) @@ -1654,6 +1703,7 @@ func mvccPutInternal( } } } + { var txnMeta *enginepb.TxnMeta if txn != nil { @@ -2139,9 +2189,9 @@ func MVCCClearTimeRange( ts := hlc.Timestamp(meta.Timestamp) if meta.Txn != nil && startTime.Less(ts) && !endTime.Less(ts) { err := &roachpb.WriteIntentError{ - Intents: []roachpb.Intent{{Span: roachpb.Span{Key: append([]byte{}, k.Key...)}, - Status: roachpb.PENDING, Txn: *meta.Txn, - }}} + Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(*meta.Txn, roachpb.Span{Key: append([]byte{}, k.Key...)}), + }} return nil, err } } @@ -2347,11 +2397,7 @@ func buildScanIntents(data []byte) ([]roachpb.Intent, error) { if err := protoutil.Unmarshal(reader.Value(), &meta); err != nil { return nil, err } - intents = append(intents, roachpb.Intent{ - Span: roachpb.Span{Key: key.Key}, - Status: roachpb.PENDING, - Txn: *meta.Txn, - }) + intents = append(intents, roachpb.MakeErrorIntent(*meta.Txn, roachpb.Span{Key: key.Key})) } if err := reader.Error(); err != nil { @@ -2705,11 +2751,29 @@ func mvccResolveWriteIntent( // testing. inProgress := !intent.Status.IsFinalized() && meta.Txn.Epoch >= intent.Txn.Epoch pushed := inProgress && hlc.Timestamp(meta.Timestamp).Less(intent.Txn.WriteTimestamp) + collapsedIntent := false + var rolledBackVal []byte + latestKey := MVCCKey{Key: intent.Key, Timestamp: hlc.Timestamp(meta.Timestamp)} + + // Handle partial txn rollbacks. If the current txn sequence + // is part of a rolled back (ignored) seqnum range, we're going + // to erase that MVCC write and reveal the previous value. + // If _all_ the writes get removed in this way, the intent + // "collapses" and should be considered empty (i.e. can be removed altogether). + // If only part of the intent history was rolled back, but the intent still + // remains, updatedIntent is set to true. + if len(intent.IgnoredSeqNums) > 0 { + collapsedIntent, rolledBackVal, err = mvccMaybeRewriteIntentHistory(ctx, rw, intent.IgnoredSeqNums, meta, latestKey) + if err != nil { + return false, err + } + } // There's nothing to do if meta's epoch is greater than or equal txn's // epoch and the state is still in progress but the intent was not pushed - // to a larger timestamp. - if inProgress && !pushed { + // to a larger timestamp, and if the rollback code did not modify or collapse + // the intent. + if inProgress && !pushed && rolledBackVal == nil && !collapsedIntent { return false, nil } @@ -2717,15 +2781,21 @@ func mvccResolveWriteIntent( // the proposed epoch matches the existing epoch: update the meta.Txn. For commit, it's set to // nil; otherwise, we update its value. We may have to update the actual version value (remove old // and create new with proper timestamp-encoded key) if timestamp changed. - if commit || pushed { + // + // We also use the commit path when the intent is being collapsed, + // to ensure the intent meta gets deleted and the stats updated. + if commit || pushed || (rolledBackVal != nil) { buf.newMeta = *meta // Set the timestamp for upcoming write (or at least the stats update). buf.newMeta.Timestamp = hlc.LegacyTimestamp(intent.Txn.WriteTimestamp) // Update or remove the metadata key. var metaKeySize, metaValSize int64 - if pushed { - // Keep existing intent if we're pushing timestamp. We keep the + // Note that commit takes precedence over {updated, collapsed}Intent; as in, + // if we're committing, we don't care whether mvccMaybeRewriteIntentHistory + // updated the intent or collapsed it. + if pushed || (rolledBackVal != nil && !commit) { + // Keep existing intent if we're updating it. We keep the // existing metadata instead of using the supplied intent meta // to avoid overwriting a newer epoch (see comments above). The // pusher's job isn't to do anything to update the intent but @@ -2739,9 +2809,12 @@ func mvccResolveWriteIntent( return false, err } - // If we're moving the intent's timestamp, adjust stats and rewrite it. + // If we're moving the intent's timestamp, adjust stats and + // rewrite it. However this work needs not be done if the intent + // was collapsed (mvccMaybeRewriteIntentHistory already rewrote + // the key at the correct timestamp and adjusted the stats). var prevValSize int64 - if buf.newMeta.Timestamp != meta.Timestamp { + if buf.newMeta.Timestamp != meta.Timestamp && !collapsedIntent { oldKey := MVCCKey{Key: intent.Key, Timestamp: hlc.Timestamp(meta.Timestamp)} newKey := MVCCKey{Key: intent.Key, Timestamp: intent.Txn.WriteTimestamp} @@ -2752,7 +2825,16 @@ func mvccResolveWriteIntent( } else if !valid || !iter.UnsafeKey().Equal(oldKey) { return false, errors.Errorf("existing intent value missing: %s", oldKey) } - if err = rw.Put(newKey, iter.UnsafeValue()); err != nil { + value := iter.UnsafeValue() + // Special case: If mvccMaybeRewriteIntentHistory rolled back to a value + // in the intent history and wrote that at oldKey, iter would not be able + // to "see" the value since it was created before that value was written + // to the engine. In this case, reuse the value returned by + // mvccMaybeRewriteIntentHistory. + if rolledBackVal != nil { + value = rolledBackVal + } + if err = rw.Put(newKey, value); err != nil { return false, err } if err = rw.Clear(oldKey); err != nil { @@ -2808,7 +2890,6 @@ func mvccResolveWriteIntent( // - ResolveIntent with epoch 0 aborts intent from epoch 1. // First clear the intent value. - latestKey := MVCCKey{Key: intent.Key, Timestamp: hlc.Timestamp(meta.Timestamp)} if err := rw.Clear(latestKey); err != nil { return false, err } @@ -2860,6 +2941,59 @@ func mvccResolveWriteIntent( return true, nil } +// mvccMaybeRewriteIntentHistory rewrites the intent to reveal the latest +// stored value, ignoring all values from the history that have an +// ignored seqnum. +// The cleared return value, when true, indicates that +// all the writes in the intent are ignored and the intent should +// not be considered to exist any more. +// The updated return value, when true, indicates that the intent was updated +// and should be overwritten in engine. +func mvccMaybeRewriteIntentHistory( + ctx context.Context, + engine ReadWriter, + ignoredSeqNums []enginepb.IgnoredSeqNumRange, + meta *enginepb.MVCCMetadata, + latestKey MVCCKey, +) (cleared bool, updatedVal []byte, err error) { + if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, ignoredSeqNums) { + // The latest write was not ignored. Nothing to do here. We'll + // proceed with the intent as usual. + return false, nil, nil + } + // Find the latest historical write before that that was not + // ignored. + var i int + for i = len(meta.IntentHistory) - 1; i >= 0; i-- { + e := &meta.IntentHistory[i] + if !enginepb.TxnSeqIsIgnored(e.Sequence, ignoredSeqNums) { + break + } + } + + // If i < 0, we don't have an intent any more: everything + // has been rolled back. + if i < 0 { + err := engine.Clear(latestKey) + // For stats recomputation in the caller, flatten the + // value size so there's nothing left attributed to this intent. + meta.ValBytes = 0 + return true, nil, err + } + + // Otherwise, we place back the write at that history entry + // back into the intent. + restoredVal := meta.IntentHistory[i].Value + meta.Txn.Sequence = meta.IntentHistory[i].Sequence + meta.IntentHistory = meta.IntentHistory[:i] + meta.Deleted = len(restoredVal) == 0 + meta.ValBytes = int64(len(restoredVal)) + // And also overwrite whatever was there in storage. + err = engine.Put(latestKey, restoredVal) + + return false, restoredVal, err +} + // IterAndBuf used to pass iterators and buffers between MVCC* calls, allowing // reuse without the callers needing to know the particulars. type IterAndBuf struct { @@ -3513,11 +3647,11 @@ func checkForKeyCollisionsGo( // encounter many intents during IMPORT INTO as we lock the key space we // are importing into. Older write intents could however be found in the // target key space, which will require appropriate resolution logic. - var writeIntentErr roachpb.WriteIntentError - var intent roachpb.Intent - intent.Txn = *mvccMeta.Txn - intent.Key = existingIter.Key().Key - writeIntentErr.Intents = append(writeIntentErr.Intents, intent) + writeIntentErr := roachpb.WriteIntentError{ + Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(*mvccMeta.Txn, roachpb.Span{Key: existingIter.Key().Key}), + }, + } return enginepb.MVCCStats{}, &writeIntentErr } else { diff --git a/pkg/storage/engine/mvcc_history_test.go b/pkg/storage/engine/mvcc_history_test.go index f914ba4ff81d..c820da2a6431 100644 --- a/pkg/storage/engine/mvcc_history_test.go +++ b/pkg/storage/engine/mvcc_history_test.go @@ -368,13 +368,14 @@ const ( // commands is the list of all supported script commands. var commands = map[string]cmd{ - "txn_advance": {typTxnUpdate, cmdTxnAdvance}, - "txn_begin": {typTxnUpdate, cmdTxnBegin}, - "txn_remove": {typTxnUpdate, cmdTxnRemove}, - "txn_restart": {typTxnUpdate, cmdTxnRestart}, - "txn_status": {typTxnUpdate, cmdTxnSetStatus}, - "txn_step": {typTxnUpdate, cmdTxnStep}, - "txn_update": {typTxnUpdate, cmdTxnUpdate}, + "txn_advance": {typTxnUpdate, cmdTxnAdvance}, + "txn_begin": {typTxnUpdate, cmdTxnBegin}, + "txn_ignore_seqs": {typTxnUpdate, cmdTxnIgnoreSeqs}, + "txn_remove": {typTxnUpdate, cmdTxnRemove}, + "txn_restart": {typTxnUpdate, cmdTxnRestart}, + "txn_status": {typTxnUpdate, cmdTxnSetStatus}, + "txn_step": {typTxnUpdate, cmdTxnStep}, + "txn_update": {typTxnUpdate, cmdTxnUpdate}, "resolve_intent": {typDataUpdate, cmdResolveIntent}, "check_intent": {typReadOnly, cmdCheckIntent}, @@ -414,6 +415,30 @@ func cmdTxnBegin(e *evalCtx) error { return err } +func cmdTxnIgnoreSeqs(e *evalCtx) error { + txn := e.getTxn(mandatory) + seql := e.getList("seqs") + is := []enginepb.IgnoredSeqNumRange{} + for _, s := range seql { + parts := strings.Split(s, "-") + if len(parts) != 2 { + e.Fatalf("syntax error: expected 'a-b', got: '%s'", s) + } + a, err := strconv.ParseInt(parts[0], 10, 32) + if err != nil { + e.Fatalf("%v", err) + } + b, err := strconv.ParseInt(parts[1], 10, 32) + if err != nil { + e.Fatalf("%v", err) + } + is = append(is, enginepb.IgnoredSeqNumRange{Start: enginepb.TxnSeq(a), End: enginepb.TxnSeq(b)}) + } + txn.IgnoredSeqNums = is + e.results.txn = txn + return nil +} + func cmdTxnRemove(e *evalCtx) error { txn := e.getTxn(mandatory) delete(e.txns, txn.Name) @@ -478,11 +503,9 @@ func cmdResolveIntent(e *evalCtx) error { func (e *evalCtx) resolveIntent( rw ReadWriter, key roachpb.Key, txn *roachpb.Transaction, resolveStatus roachpb.TransactionStatus, ) error { - return MVCCResolveWriteIntent(e.ctx, rw, nil, roachpb.Intent{ - Span: roachpb.Span{Key: key}, - Status: resolveStatus, - Txn: txn.TxnMeta, - }) + intent := roachpb.MakeIntent(*txn, roachpb.Span{Key: key}) + intent.Status = resolveStatus + return MVCCResolveWriteIntent(e.ctx, rw, nil, intent) } func cmdCheckIntent(e *evalCtx) error { @@ -501,7 +524,7 @@ func cmdCheckIntent(e *evalCtx) error { return errors.Newf("meta: %v -> expected intent, found none", key) } if ok { - fmt.Fprintf(e.results.buf, "meta: %v -> %s\n", key, &meta) + fmt.Fprintf(e.results.buf, "meta: %v -> %+v\n", key, &meta) if !wantIntent { return errors.Newf("meta: %v -> expected no intent, found one", key) } @@ -792,6 +815,16 @@ const ( mandatory ) +func (e *evalCtx) getList(argName string) []string { + for _, c := range e.td.CmdArgs { + if c.Key == argName { + return c.Vals + } + } + e.Fatalf("missing argument: %s", argName) + return nil +} + func (e *evalCtx) getTxn(opt optArg) *roachpb.Transaction { e.t.Helper() if opt == optional && (e.hasArg("notxn") || !e.hasArg("t")) { diff --git a/pkg/storage/engine/mvcc_incremental_iterator.go b/pkg/storage/engine/mvcc_incremental_iterator.go index 259bff770a90..75ecbfb5615b 100644 --- a/pkg/storage/engine/mvcc_incremental_iterator.go +++ b/pkg/storage/engine/mvcc_incremental_iterator.go @@ -192,11 +192,9 @@ func (i *MVCCIncrementalIterator) advance() { if i.meta.Txn != nil { if i.startTime.Less(metaTimestamp) && !i.endTime.Less(metaTimestamp) { i.err = &roachpb.WriteIntentError{ - Intents: []roachpb.Intent{{ - Span: roachpb.Span{Key: i.iter.Key().Key}, - Status: roachpb.PENDING, - Txn: *i.meta.Txn, - }}, + Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(*i.meta.Txn, roachpb.Span{Key: i.iter.Key().Key}), + }, } i.valid = false return diff --git a/pkg/storage/engine/mvcc_incremental_iterator_test.go b/pkg/storage/engine/mvcc_incremental_iterator_test.go index 5eeaf1199efe..76a391869eab 100644 --- a/pkg/storage/engine/mvcc_incremental_iterator_test.go +++ b/pkg/storage/engine/mvcc_incremental_iterator_test.go @@ -206,11 +206,13 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, ts4, tsMax, kvs())) t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, ts4.Next(), tsMax, kvs())) - intent1 := roachpb.Intent{Span: roachpb.Span{Key: testKey1}, Txn: txn1.TxnMeta, Status: roachpb.COMMITTED} + intent1 := roachpb.MakeIntent(txn1, roachpb.Span{Key: testKey1}) + intent1.Status = roachpb.COMMITTED if err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { t.Fatal(err) } - intent2 := roachpb.Intent{Span: roachpb.Span{Key: testKey2}, Txn: txn2.TxnMeta, Status: roachpb.ABORTED} + intent2 := roachpb.MakeIntent(txn2, roachpb.Span{Key: testKey2}) + intent2.Status = roachpb.ABORTED if err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { t.Fatal(err) } @@ -294,11 +296,13 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, ts4, tsMax, kvs())) t.Run("intents", assertEqualKVs(e, fn, keyMin, keyMax, ts4.Next(), tsMax, kvs())) - intent1 := roachpb.Intent{Span: roachpb.Span{Key: testKey1}, Txn: txn1.TxnMeta, Status: roachpb.COMMITTED} + intent1 := roachpb.MakeIntent(txn1, roachpb.Span{Key: testKey1}) + intent1.Status = roachpb.COMMITTED if err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { t.Fatal(err) } - intent2 := roachpb.Intent{Span: roachpb.Span{Key: testKey2}, Txn: txn2.TxnMeta, Status: roachpb.ABORTED} + intent2 := roachpb.MakeIntent(txn2, roachpb.Span{Key: testKey2}) + intent2.Status = roachpb.ABORTED if err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { t.Fatal(err) } diff --git a/pkg/storage/engine/mvcc_logical_ops_test.go b/pkg/storage/engine/mvcc_logical_ops_test.go index e9514faa0e8b..84d3b3fd586d 100644 --- a/pkg/storage/engine/mvcc_logical_ops_test.go +++ b/pkg/storage/engine/mvcc_logical_ops_test.go @@ -75,18 +75,18 @@ func TestMVCCOpLogWriter(t *testing.T) { // Resolve all three intent. txn1CommitTS := *txn1Commit txn1CommitTS.WriteTimestamp = hlc.Timestamp{Logical: 4} - if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}, - Txn: txn1CommitTS.TxnMeta, - Status: txn1CommitTS.Status, - }, math.MaxInt64); err != nil { + if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, + roachpb.MakeIntent( + txn1CommitTS, + roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}), + math.MaxInt64); err != nil { t.Fatal(err) } - if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, roachpb.Intent{ - Span: roachpb.Span{Key: localKey, EndKey: localKey.Next()}, - Txn: txn1CommitTS.TxnMeta, - Status: txn1CommitTS.Status, - }, math.MaxInt64); err != nil { + if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, + roachpb.MakeIntent( + txn1CommitTS, + roachpb.Span{Key: localKey, EndKey: localKey.Next()}), + math.MaxInt64); err != nil { t.Fatal(err) } @@ -97,20 +97,16 @@ func TestMVCCOpLogWriter(t *testing.T) { } txn2Pushed := *txn2 txn2Pushed.WriteTimestamp = hlc.Timestamp{Logical: 6} - if err := MVCCResolveWriteIntent(ctx, ol, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey3}, - Txn: txn2Pushed.TxnMeta, - Status: txn2Pushed.Status, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, ol, nil, + roachpb.MakeIntent(txn2Pushed, roachpb.Span{Key: testKey3}), + ); err != nil { t.Fatal(err) } txn2Abort := txn2Pushed txn2Abort.Status = roachpb.ABORTED - if err := MVCCResolveWriteIntent(ctx, ol, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey3}, - Txn: txn2Abort.TxnMeta, - Status: txn2Abort.Status, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, ol, nil, + roachpb.MakeIntent(txn2Abort, roachpb.Span{Key: testKey3}), + ); err != nil { t.Fatal(err) } diff --git a/pkg/storage/engine/mvcc_stats_test.go b/pkg/storage/engine/mvcc_stats_test.go index addb28c7bb5d..c526f3a9d8eb 100644 --- a/pkg/storage/engine/mvcc_stats_test.go +++ b/pkg/storage/engine/mvcc_stats_test.go @@ -112,9 +112,9 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { ts4 := hlc.Timestamp{WallTime: 4 * 1e9} txn.Status = roachpb.COMMITTED txn.WriteTimestamp.Forward(ts4) - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ - Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeIntent(*txn, roachpb.Span{Key: key}), + ); err != nil { t.Fatal(err) } @@ -192,9 +192,9 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { ts4 := hlc.Timestamp{WallTime: 4 * 1e9} txn.Status = roachpb.COMMITTED txn.WriteTimestamp.Forward(ts4) - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ - Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeIntent(*txn, roachpb.Span{Key: key}), + ); err != nil { t.Fatal(err) } @@ -270,9 +270,9 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { // push as it would happen for a SNAPSHOT txn) ts4 := hlc.Timestamp{WallTime: 4 * 1e9} txn.WriteTimestamp.Forward(ts4) - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ - Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeIntent(*txn, roachpb.Span{Key: key}), + ); err != nil { t.Fatal(err) } @@ -605,9 +605,9 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { txnCommit := txn.Clone() txnCommit.Status = roachpb.COMMITTED txnCommit.WriteTimestamp.Forward(ts3) - if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ - Span: roachpb.Span{Key: key}, Status: txnCommit.Status, Txn: txnCommit.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, + roachpb.MakeIntent(*txnCommit, roachpb.Span{Key: key}), + ); err != nil { t.Fatal(err) } @@ -634,9 +634,9 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { txnAbort := txn.Clone() txnAbort.Status = roachpb.ABORTED txnAbort.WriteTimestamp.Forward(ts3) - if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ - Span: roachpb.Span{Key: key}, Status: txnAbort.Status, Txn: txnAbort.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, + roachpb.MakeIntent(*txnAbort, roachpb.Span{Key: key}), + ); err != nil { t.Fatal(err) } @@ -764,9 +764,9 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { txnAbort := txn.Clone() txnAbort.Status = roachpb.ABORTED // doesn't change m2ValSize, fortunately - if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, roachpb.Intent{ - Span: roachpb.Span{Key: key}, Status: txnAbort.Status, Txn: txnAbort.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, &aggMS, + roachpb.MakeIntent(*txnAbort, roachpb.Span{Key: key}), + ); err != nil { t.Fatal(err) } @@ -1235,9 +1235,9 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { // Now abort the intent. txn.Status = roachpb.ABORTED - if err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.Intent{ - Span: roachpb.Span{Key: key}, Status: txn.Status, Txn: txn.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeIntent(*txn, roachpb.Span{Key: key}), + ); err != nil { t.Fatal(err) } @@ -1340,19 +1340,15 @@ type state struct { } func (s *state) intent(status roachpb.TransactionStatus) roachpb.Intent { - return roachpb.Intent{ - Span: roachpb.Span{Key: s.key}, - Txn: s.Txn.TxnMeta, - Status: status, - } + intent := roachpb.MakeIntent(*s.Txn, roachpb.Span{Key: s.key}) + intent.Status = status + return intent } func (s *state) intentRange(status roachpb.TransactionStatus) roachpb.Intent { - return roachpb.Intent{ - Span: roachpb.Span{Key: roachpb.KeyMin, EndKey: roachpb.KeyMax}, - Txn: s.Txn.TxnMeta, - Status: status, - } + intent := roachpb.MakeIntent(*s.Txn, roachpb.Span{Key: roachpb.KeyMin, EndKey: roachpb.KeyMax}) + intent.Status = status + return intent } func (s *state) rngVal() roachpb.Value { diff --git a/pkg/storage/engine/mvcc_test.go b/pkg/storage/engine/mvcc_test.go index f6097312801c..db0954406774 100644 --- a/pkg/storage/engine/mvcc_test.go +++ b/pkg/storage/engine/mvcc_test.go @@ -2721,11 +2721,8 @@ func TestMVCCInitPutWithTxn(t *testing.T) { txnCommit := txn txnCommit.Status = roachpb.COMMITTED txnCommit.WriteTimestamp = clock.Now().Add(1, 0) - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txnCommit.Status, - Txn: txnCommit.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(txnCommit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -3039,11 +3036,8 @@ func TestMVCCResolveTxn(t *testing.T) { } // Resolve will write with txn1's timestamp which is 0,1. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Txn: txn1Commit.TxnMeta, - Status: txn1Commit.Status, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn1Commit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -3084,11 +3078,8 @@ func TestMVCCResolveNewerIntent(t *testing.T) { } // Resolve will succeed but should remove the intent. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Txn: txn1Commit.TxnMeta, - Status: txn1Commit.Status, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn1Commit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -3121,15 +3112,12 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { t.Fatal(err) } - intent := roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: roachpb.PENDING, - // The Timestamp within is equal to that of txn.Meta even though - // the intent sits at tsEarly. The bug was looking at the former - // instead of the latter (and so we could also tickle it with - // smaller timestamps in Txn). - Txn: txn.TxnMeta, - } + // The Timestamp within is equal to that of txn.Meta even though + // the intent sits at tsEarly. The bug was looking at the former + // instead of the latter (and so we could also tickle it with + // smaller timestamps in Txn). + intent := roachpb.MakeIntent(*txn, roachpb.Span{Key: testKey1}) + intent.Status = roachpb.PENDING // A bug (see #7654) caused intents to just stay where they were instead // of being moved forward in the situation set up above. @@ -3350,11 +3338,9 @@ func TestMVCCAbortTxn(t *testing.T) { txn1AbortWithTS := txn1Abort.Clone() txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{Logical: 1} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Txn: txn1AbortWithTS.TxnMeta, - Status: txn1AbortWithTS.Status, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn1AbortWithTS, roachpb.Span{Key: testKey1}), + ); err != nil { t.Fatal(err) } @@ -3397,11 +3383,9 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { txn1AbortWithTS := txn1Abort.Clone() txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 2} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txn1AbortWithTS.Status, - Txn: txn1AbortWithTS.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn1AbortWithTS, roachpb.Span{Key: testKey1}), + ); err != nil { t.Fatal(err) } @@ -3467,11 +3451,8 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { txne2Commit := txne2 txne2Commit.Status = roachpb.COMMITTED txne2Commit.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txne2Commit.Status, - Txn: txne2Commit.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(txne2Commit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -3800,11 +3781,8 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { } // Resolve the intent, pushing its timestamp forward. txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txn.Status, - Txn: txn.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } // Attempt to read using naive txn's previous timestamp. @@ -3835,11 +3813,8 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, value2, txn1e2); err != nil { t.Fatal(err) } - num, _, err := MVCCResolveWriteIntentRange(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}, - Txn: txn1e2Commit.TxnMeta, - Status: txn1e2Commit.Status, - }, 2) + num, _, err := MVCCResolveWriteIntentRange(ctx, engine, nil, + roachpb.MakeIntent(*txn1e2Commit, roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}), 2) if err != nil { t.Fatal(err) } @@ -3894,11 +3869,8 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { // Resolve with a higher commit timestamp -- this should rewrite the // intent when making it permanent. txn := makeTxn(*txn1Commit, hlc.Timestamp{WallTime: 1}) - if err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txn.Status, - Txn: txn.TxnMeta, - }); err != nil { + if err = MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -3948,11 +3920,8 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { // Resolve with a higher commit timestamp, but with still-pending transaction. // This represents a straightforward push (i.e. from a read/write conflict). txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txn.Status, - Txn: txn.TxnMeta, - }); err != nil { + if err = MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -3989,11 +3958,8 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { defer engine.Close() // Resolve a non existent key; noop. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txn1Commit.Status, - Txn: txn1Commit.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn1Commit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -4001,11 +3967,8 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txn2Commit.Status, - Txn: txn2Commit.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn2Commit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } @@ -4016,11 +3979,8 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { txn1CommitWithTS := txn2Commit.Clone() txn1CommitWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey2}, - Status: txn1CommitWithTS.Status, - Txn: txn1CommitWithTS.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn1CommitWithTS, roachpb.Span{Key: testKey2})); err != nil { t.Fatal(err) } }) @@ -4049,11 +4009,9 @@ func TestMVCCResolveTxnRange(t *testing.T) { t.Fatal(err) } - num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1, EndKey: testKey4.Next()}, - Txn: txn1Commit.TxnMeta, - Status: txn1Commit.Status, - }, math.MaxInt64) + num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, + roachpb.MakeIntent(*txn1Commit, roachpb.Span{Key: testKey1, EndKey: testKey4.Next()}), + math.MaxInt64) if err != nil { t.Fatal(err) } @@ -4135,11 +4093,9 @@ func TestMVCCResolveTxnRangeResume(t *testing.T) { } // Resolve up to 5 intents. - num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: roachpb.Key("00"), EndKey: roachpb.Key("30")}, - Txn: txn1Commit.TxnMeta, - Status: txn1Commit.Status, - }, 5) + num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, + roachpb.MakeIntent(*txn1Commit, roachpb.Span{Key: roachpb.Key("00"), EndKey: roachpb.Key("30")}), + 5) if err != nil { t.Fatal(err) } @@ -4894,11 +4850,8 @@ func TestResolveIntentWithLowerEpoch(t *testing.T) { t.Fatal(err) } // Resolve the intent with a low epoch. - if err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.Intent{ - Span: roachpb.Span{Key: testKey1}, - Status: txn1.Status, - Txn: txn1.TxnMeta, - }); err != nil { + if err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeIntent(*txn1, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } diff --git a/pkg/storage/engine/rocksdb.go b/pkg/storage/engine/rocksdb.go index 232b46b25dab..7f89db74879f 100644 --- a/pkg/storage/engine/rocksdb.go +++ b/pkg/storage/engine/rocksdb.go @@ -2611,6 +2611,16 @@ func goToCSlice(b []byte) C.DBSlice { } } +func goToCIgnoredSeqNums(b []enginepb.IgnoredSeqNumRange) C.DBIgnoredSeqNums { + if len(b) == 0 { + return C.DBIgnoredSeqNums{ranges: nil, len: 0} + } + return C.DBIgnoredSeqNums{ + ranges: (*C.DBIgnoredSeqNumRange)(unsafe.Pointer(&b[0])), + len: C.int(len(b)), + } +} + func goToCKey(key MVCCKey) C.DBKey { return C.DBKey{ key: goToCSlice(key.Key), @@ -2698,6 +2708,7 @@ func goToCTxn(txn *roachpb.Transaction) C.DBTxn { r.epoch = C.uint32_t(txn.Epoch) r.sequence = C.int32_t(txn.Sequence) r.max_timestamp = goToCTimestamp(txn.MaxTimestamp) + r.ignored_seqnums = goToCIgnoredSeqNums(txn.IgnoredSeqNums) } return r } diff --git a/pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn b/pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn index 572a85db37ab..7c55b7777240 100644 --- a/pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn +++ b/pkg/storage/engine/testdata/mvcc_histories/conditional_put_with_txn @@ -48,9 +48,9 @@ run ok with t=A txn_advance ts=124 resolve_intent k=k + txn_remove ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000124,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000123,0 wto=false max=0.000000000,0 data: "k"/0.000000124,0 -> /BYTES/v3 # Write value4 with an old timestamp without txn...should get a write @@ -63,3 +63,47 @@ cput k=k v=v4 cond=v3 ts=123 data: "k"/0.000000124,1 -> /BYTES/v4 data: "k"/0.000000124,0 -> /BYTES/v3 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000123,0 too old; wrote at 0.000000124,1 + +# Reset for next test + +run ok +clear_range k=k end=-k +---- +>> at end: + + +# From TxnCoordSenderRetries, +# "multi-range batch with forwarded timestamp and cput and delete range" + +# First txn attempt + +run ok +# Before txn start: +put k=c v=value ts=1 +with t=A + txn_begin ts=2 + txn_step + cput k=c v=cput cond=value +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000002,0 wto=false max=0.000000000,0 +meta: "c"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0.000000000,0 seq=1} ts=0.000000002,0 del=false klen=12 vlen=9 +data: "c"/0.000000002,0 -> /BYTES/cput +data: "c"/0.000000001,0 -> /BYTES/value + +# Restart and retry cput. It should succeed. + +run trace ok +with t=A + txn_restart ts=3 + txn_step + cput k=c v=cput cond=value +---- +>> txn_restart ts=3 t=A +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0.000000000,0 seq=0} rw=true stat=PENDING rts=0.000000003,0 wto=false max=0.000000000,0 +>> txn_step t=A +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000003,0 wto=false max=0.000000000,0 +>> cput k=c v=cput cond=value t=A +meta: "c"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0.000000000,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=9 +data: "c"/0.000000003,0 -> /BYTES/cput +data: "c"/0.000000001,0 -> /BYTES/value diff --git a/pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions b/pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions index 3ed1d12eae6b..dcdad745ef10 100644 --- a/pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions +++ b/pkg/storage/engine/testdata/mvcc_histories/idempotent_transactions @@ -34,7 +34,7 @@ with t=a k=a txn_step check_intent ---- -meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 nih=1 +meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} >> at end: txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 meta: "a"/0.000000000,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} diff --git a/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums b/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums new file mode 100644 index 000000000000..dd5ff8731ac7 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums @@ -0,0 +1,140 @@ +# Pebble does not support ignored seqnums for now. +skip pebble +---- + +# Perform some writes at various sequence numbers. + +run ok +with t=A + txn_begin ts=11 + txn_step seq=10 + put k=k v=a + put k=k/10 v=10 + txn_step seq=20 + put k=k v=b + put k=k/20 v=20 + txn_step seq=30 + put k=k v=c + put k=k/30 v=30 + txn_step seq=40 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/0.000000011,0 -> /BYTES/c +meta: "k/10"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 +data: "k/10"/0.000000011,0 -> /BYTES/10 +meta: "k/20"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 +data: "k/20"/0.000000011,0 -> /BYTES/20 +meta: "k/30"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 +data: "k/30"/0.000000011,0 -> /BYTES/30 + +# Mask a single write. +# The third write should be hidden now. + +run ok +with t=A + txn_ignore_seqs seqs=(25-35) + scan k=k end=-k + get k=k +---- +scan: "k" -> /BYTES/b @0.000000000,0 +scan: "k/10" -> /BYTES/10 @0.000000011,0 +scan: "k/20" -> /BYTES/20 @0.000000011,0 +get: "k" -> /BYTES/b @0.000000000,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 + +# Mask a write in the middle. + +run ok +with t=A + txn_ignore_seqs seqs=(15-25) + txn_step seq=40 + scan k=k end=-k + get k=k +---- +scan: "k" -> /BYTES/c @0.000000011,0 +scan: "k/10" -> /BYTES/10 @0.000000011,0 +scan: "k/30" -> /BYTES/30 @0.000000011,0 +get: "k" -> /BYTES/c @0.000000011,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 + +# Mask all the writes. + +run ok +with t=A + txn_ignore_seqs seqs=(1-35) + txn_step seq=40 + scan k=k end=-k + get k=k +---- +scan: "k"-"l" -> +get: "k" -> +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 + +# Disjoint masks. + +run ok +with t=A + txn_ignore_seqs seqs=(1-15,25-35) + txn_step seq=40 + scan k=k end=-k + get k=k +---- +scan: "k" -> /BYTES/b @0.000000000,0 +scan: "k/20" -> /BYTES/20 @0.000000011,0 +get: "k" -> /BYTES/b @0.000000000,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=2 + +# A historical read before the ignored range should retrieve the +# historical value visible at that point. + +run ok +with t=A + txn_ignore_seqs seqs=(15-25) + txn_step seq=12 + scan k=k end=-k + get k=k +---- +scan: "k" -> /BYTES/a @0.000000000,0 +scan: "k/10" -> /BYTES/10 @0.000000011,0 +get: "k" -> /BYTES/a @0.000000000,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=12} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 + +# A historical read with an ignored range before it should hide +# the historical value hidden at that point. + +run ok +with t=A + txn_ignore_seqs seqs=(5-15) + txn_step seq=22 + scan k=k end=-k + get k=k +---- +scan: "k" -> /BYTES/b @0.000000000,0 +scan: "k/20" -> /BYTES/20 @0.000000011,0 +get: "k" -> /BYTES/b @0.000000000,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=22} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 + +# A historical read with an ignored range that overlaps should hide +# the historical value hidden at that point. + +run ok +with t=A + txn_ignore_seqs seqs=(25-35) + txn_step seq=32 + scan k=k end=-k + get k=k +---- +scan: "k" -> /BYTES/b @0.000000000,0 +scan: "k/10" -> /BYTES/10 @0.000000011,0 +scan: "k/20" -> /BYTES/20 @0.000000011,0 +get: "k" -> /BYTES/b @0.000000000,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=32} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 diff --git a/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_commit b/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_commit new file mode 100644 index 000000000000..00389c255ca6 --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_commit @@ -0,0 +1,80 @@ +# Pebble does not support ignored seqnums for now. +skip pebble +---- + +# Perform some writes at various sequence numbers. + +run ok +with t=A + txn_begin ts=11 + txn_step seq=10 + put k=k v=a + put k=k/10 v=10 + txn_step seq=20 + put k=k v=b + put k=k/20 v=20 + txn_step seq=30 + put k=k v=c + put k=k/30 v=30 + txn_step seq=40 +# Mask a single write. +# The third write should be hidden now. + txn_ignore_seqs seqs=(25-35) + resolve_intent k=k + resolve_intent k=k/10 + resolve_intent k=k/20 + resolve_intent k=k/30 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 +data: "k"/0.000000011,0 -> /BYTES/b +data: "k/10"/0.000000011,0 -> /BYTES/10 +data: "k/20"/0.000000011,0 -> /BYTES/20 + +run ok +scan k=k end=-k +get k=k +---- +scan: "k"-"l" -> +get: "k" -> + +run ok +clear_range k=k end=-k +txn_remove t=A +---- +>> at end: + + +run ok +with t=A + txn_begin ts=11 + txn_step seq=10 + put k=k v=a + put k=k/10 v=10 + txn_step seq=20 + put k=k v=b + put k=k/20 v=20 + txn_step seq=30 + put k=k v=c + put k=k/30 v=30 + txn_step seq=40 +# Mask a single write. +# The middle write should be hidden now. + txn_ignore_seqs seqs=(15-25) + resolve_intent k=k + resolve_intent k=k/10 + resolve_intent k=k/20 + resolve_intent k=k/30 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 +data: "k"/0.000000011,0 -> /BYTES/c +data: "k/10"/0.000000011,0 -> /BYTES/10 +data: "k/30"/0.000000011,0 -> /BYTES/30 + +run ok +scan k=k end=-k +get k=k +---- +scan: "k"-"l" -> +get: "k" -> diff --git a/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_cput b/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_cput new file mode 100644 index 000000000000..731f866b2c7e --- /dev/null +++ b/pkg/storage/engine/testdata/mvcc_histories/ignored_seq_nums_cput @@ -0,0 +1,208 @@ +# Pebble does not support ignored seqnums for now. +skip pebble +---- + + +## We'll check the behavior of cput in the following circumstances: +## A. last write is ignored, no intent history (need to go to store) +## B. last write is ignored, intent history not ignored (need to pick last from history) +## C. last write is ignored, last entry in intent history is ignored, other entry available (need to pick next-to-last from history) +## D. last write is ignored, all intent history ignored (need to go to store) + +# Case A: just 1 put in txn before CPut (no intent history). Then ignore that put. +# Expect cput to find original value (first). + +run ok +put k=k v=first ts=1 +with t=A + txn_begin ts=11 + txn_step seq=10 + put k=k v=a + txn_ignore_seqs seqs=(5-15) + txn_step seq=20 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=6 +data: "k"/0.000000011,0 -> /BYTES/a +data: "k"/0.000000001,0 -> /BYTES/first + +# Condition must fail to find the last write a. + +run error +cput t=A k=k cond=a v=c +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=6 +data: "k"/0.000000011,0 -> /BYTES/a +data: "k"/0.000000001,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: + +# Condition succeeds to find the original value. + +run ok +cput t=A k=k cond=first v=b +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/first}} +data: "k"/0.000000011,0 -> /BYTES/b +data: "k"/0.000000001,0 -> /BYTES/first + +run ok +clear_range k=k end=-k +---- +>> at end: + + +# Case B: two writes, ignore last (check cput picks from history). + +run ok +put k=k v=first ts=1 +with t=B + txn_begin ts=11 + txn_step seq=10 + put k=k v=a + txn_step seq=20 + put k=k v=b + txn_ignore_seqs seqs=(15-25) + txn_step seq=30 +---- +>> at end: +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/0.000000011,0 -> /BYTES/b +data: "k"/0.000000001,0 -> /BYTES/first + +# Condition must fail to find the last write b. + +run error +cput t=B k=k cond=b v=c +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/0.000000011,0 -> /BYTES/b +data: "k"/0.000000001,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> + +# However it succeeds to find the write before that. + +run ok +cput t=B k=k cond=a v=c +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/a}} +data: "k"/0.000000011,0 -> /BYTES/c +data: "k"/0.000000001,0 -> /BYTES/first + +run ok +clear_range k=k end=-k +---- +>> at end: + + +# Case C: two or more history entries, last is ignored. + +run ok +put k=k v=first ts=1 +with t=C + txn_begin ts=11 + txn_step seq=10 + put k=k v=a + txn_step seq=20 + put k=k v=b + txn_step seq=30 + put k=k v=c + txn_ignore_seqs seqs=(15-35) + txn_step seq=40 +---- +>> at end: +txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/0.000000011,0 -> /BYTES/c +data: "k"/0.000000001,0 -> /BYTES/first + +# Condition must fail to find the last write b and c. + +run error +cput t=C k=k cond=c v=d +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/0.000000011,0 -> /BYTES/c +data: "k"/0.000000001,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> + +run error +cput t=C k=k cond=b v=d +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/0.000000011,0 -> /BYTES/c +data: "k"/0.000000001,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> + +# However it succeeds to find the write before that. + +run ok +cput t=C k=k cond=a v=c +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=40} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}{30 /BYTES/a}} +data: "k"/0.000000011,0 -> /BYTES/c +data: "k"/0.000000001,0 -> /BYTES/first + +run ok +clear_range k=k end=-k +---- +>> at end: + + +# Case D: last write + intent history all ignored. +# We need just two writes to have a non-empty intent history, then we can ignore that. + +run ok +put k=k v=first ts=1 +with t=D + txn_begin ts=11 + txn_step seq=10 + put k=k v=a + txn_step seq=20 + put k=k v=b + txn_ignore_seqs seqs=(5-25) + txn_step seq=30 +---- +>> at end: +txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} rw=true stat=PENDING rts=0.000000011,0 wto=false max=0.000000000,0 isn=1 +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/0.000000011,0 -> /BYTES/b +data: "k"/0.000000001,0 -> /BYTES/first + +# Condition must fail to find the last writes a and b. + +run error +cput t=D k=k cond=a v=c +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/0.000000011,0 -> /BYTES/b +data: "k"/0.000000001,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: + +run error +cput t=D k=k cond=b v=c +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/0.000000011,0 -> /BYTES/b +data: "k"/0.000000001,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: + +# However it succeeds to find the write before that. + +run ok +cput t=D k=k cond=first v=c +---- +>> at end: +meta: "k"/0.000000000,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0.000000000,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/first}} +data: "k"/0.000000011,0 -> /BYTES/c +data: "k"/0.000000001,0 -> /BYTES/first diff --git a/pkg/storage/intentresolver/contention_queue.go b/pkg/storage/intentresolver/contention_queue.go index 5cda4bf84f2c..d478d8137579 100644 --- a/pkg/storage/intentresolver/contention_queue.go +++ b/pkg/storage/intentresolver/contention_queue.go @@ -221,11 +221,7 @@ func (cq *contentionQueue) add( log.VEventf(ctx, 3, "%s exiting contention queue to push %s", txnID(curPusher.txn), txnMeta.ID.Short()) wiErrCopy := *wiErr wiErrCopy.Intents = []roachpb.Intent{ - { - Span: intent.Span, - Txn: *txnMeta, - Status: roachpb.PENDING, - }, + roachpb.MakeErrorIntent(*txnMeta, intent.Span), } wiErr = &wiErrCopy } else { diff --git a/pkg/storage/intentresolver/intent_resolver.go b/pkg/storage/intentresolver/intent_resolver.go index 51038776bc9f..55f67f1927bf 100644 --- a/pkg/storage/intentresolver/intent_resolver.go +++ b/pkg/storage/intentresolver/intent_resolver.go @@ -367,8 +367,7 @@ func updateIntentTxnStatus( // It must have been skipped. continue } - intent.Txn = pushee.TxnMeta - intent.Status = pushee.Status + intent.SetTxn(pushee) results = append(results, intent) } return results @@ -705,8 +704,7 @@ func (ir *IntentResolver) CleanupTxnIntentsOnGCAsync( // Get the pushed txn and update the intents slice. txn = &b.RawResponse().Responses[0].GetInner().(*roachpb.PushTxnResponse).PusheeTxn for i := range intents { - intents[i].Txn = txn.TxnMeta - intents[i].Status = txn.Status + intents[i].SetTxn(*txn) } } var onCleanupComplete func(error) @@ -883,24 +881,27 @@ func (ir *IntentResolver) ResolveIntents( var resolveRangeReqs []roachpb.Request for i := range intents { intent := intents[i] // avoids a race in `i, intent := range ...` + if len(intent.EndKey) == 0 { resolveReqs = append(resolveReqs, resolveReq{ rangeID: ir.lookupRangeID(ctx, intent.Key), req: &roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), - IntentTxn: intent.Txn, - Status: intent.Status, - Poison: opts.Poison, + RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), + IntentTxn: intent.Txn, + Status: intent.Status, + Poison: opts.Poison, + IgnoredSeqNums: intent.IgnoredSeqNums, }, }) } else { resolveRangeReqs = append(resolveRangeReqs, &roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), - IntentTxn: intent.Txn, - Status: intent.Status, - Poison: opts.Poison, - MinTimestamp: opts.MinTimestamp, + RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), + IntentTxn: intent.Txn, + Status: intent.Status, + Poison: opts.Poison, + MinTimestamp: opts.MinTimestamp, + IgnoredSeqNums: intent.IgnoredSeqNums, }) } } diff --git a/pkg/storage/intentresolver/intent_resolver_test.go b/pkg/storage/intentresolver/intent_resolver_test.go index 256d4dbbdaf1..709990037550 100644 --- a/pkg/storage/intentresolver/intent_resolver_test.go +++ b/pkg/storage/intentresolver/intent_resolver_test.go @@ -13,6 +13,7 @@ package intentresolver import ( "context" "fmt" + "reflect" "sync" "sync/atomic" "testing" @@ -145,14 +146,8 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { { txn: txn1, intents: []roachpb.Intent{ - { - Span: roachpb.Span{Key: key}, - Txn: txn1.TxnMeta, - }, - { - Span: roachpb.Span{Key: key, EndKey: roachpb.Key("b")}, - Txn: txn1.TxnMeta, - }, + roachpb.MakeIntent(*txn1, roachpb.Span{Key: key}), + roachpb.MakeIntent(*txn1, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), }, sendFuncs: newSendFuncs(t, singlePushTxnSendFunc(t), @@ -171,9 +166,9 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { { txn: txn1, intents: []roachpb.Intent{ - {Span: roachpb.Span{Key: key}, Txn: txn1.TxnMeta}, - {Span: roachpb.Span{Key: roachpb.Key("aa")}, Txn: txn1.TxnMeta}, - {Span: roachpb.Span{Key: key, EndKey: roachpb.Key("b")}, Txn: txn1.TxnMeta}, + roachpb.MakeIntent(*txn1, roachpb.Span{Key: key}), + roachpb.MakeIntent(*txn1, roachpb.Span{Key: roachpb.Key("aa")}), + roachpb.MakeIntent(*txn1, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), }, sendFuncs: func() *sendFuncs { s := newSendFuncs(t) @@ -210,14 +205,8 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { { txn: txn3, intents: []roachpb.Intent{ - { - Span: roachpb.Span{Key: key}, - Txn: txn3.TxnMeta, - }, - { - Span: roachpb.Span{Key: key, EndKey: roachpb.Key("b")}, - Txn: txn3.TxnMeta, - }, + roachpb.MakeIntent(*txn3, roachpb.Span{Key: key}), + roachpb.MakeIntent(*txn3, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), }, sendFuncs: newSendFuncs(t, singlePushTxnSendFunc(t), @@ -236,9 +225,9 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { { txn: txn3, intents: []roachpb.Intent{ - {Span: roachpb.Span{Key: key}, Txn: txn3.TxnMeta}, - {Span: roachpb.Span{Key: roachpb.Key("aa")}, Txn: txn3.TxnMeta}, - {Span: roachpb.Span{Key: key, EndKey: roachpb.Key("b")}, Txn: txn3.TxnMeta}, + roachpb.MakeIntent(*txn3, roachpb.Span{Key: key}), + roachpb.MakeIntent(*txn3, roachpb.Span{Key: roachpb.Key("aa")}), + roachpb.MakeIntent(*txn3, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), }, sendFuncs: func() *sendFuncs { s := newSendFuncs(t) @@ -430,10 +419,8 @@ func TestContendedIntent(t *testing.T) { defer cancel() testCases[i].cancelFunc = cancel t.Run(tc.pusher.ID.String(), func(t *testing.T) { - wiErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{{ - Txn: origTxn.TxnMeta, - Span: roachpb.Span{Key: keyA}, - }}} + wiErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(origTxn.TxnMeta, roachpb.Span{Key: keyA})}} h := roachpb.Header{Txn: tc.pusher} wg.Add(1) go func(idx int) { @@ -508,10 +495,8 @@ func TestContendedIntent(t *testing.T) { "max priority": newTransaction("max-txn", keyA, roachpb.MaxUserPriority, clock), } { t.Run(name, func(t *testing.T) { - wiErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{{ - Txn: origTxn.TxnMeta, - Span: roachpb.Span{Key: keyA}, - }}} + wiErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(origTxn.TxnMeta, roachpb.Span{Key: keyA})}} h := roachpb.Header{Txn: pusher} cleanupFunc, pErr := ir.ProcessWriteIntentError(ctx, roachpb.NewError(wiErr), nil, h, roachpb.PUSH_ABORT) if pErr != nil { @@ -539,18 +524,14 @@ func TestContendedIntent(t *testing.T) { // Call the CleanupFunc with a new WriteIntentError with a different // transaction. This should lean to a new push on the new transaction and // an intent resolution of the original intent. - f(&roachpb.WriteIntentError{Intents: []roachpb.Intent{{ - Span: roachpb.Span{Key: keyA}, - Txn: unrelatedRWTxn.TxnMeta, - }}}, nil) + f(&roachpb.WriteIntentError{Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(unrelatedRWTxn.TxnMeta, roachpb.Span{Key: keyA})}}, nil) verifyPushTxn(<-reqChan, rwTxn2.ID, unrelatedRWTxn.ID) verifyResolveIntent(<-reqChan, rwTxn1.Key) case 5: verifyPushTxn(<-reqChan, rwTxn3.ID, unrelatedRWTxn.ID) - f(&roachpb.WriteIntentError{Intents: []roachpb.Intent{{ - Span: roachpb.Span{Key: keyB}, - Txn: rwTxn1.TxnMeta, - }}}, nil) + f(&roachpb.WriteIntentError{Intents: []roachpb.Intent{ + roachpb.MakeErrorIntent(rwTxn1.TxnMeta, roachpb.Span{Key: keyB})}}, nil) case 6: f(nil, &testCases[idx].pusher.TxnMeta) default: @@ -592,7 +573,7 @@ func TestCleanupIntentsAsyncThrottled(t *testing.T) { wg.Wait() testIntentsWithArg := []result.IntentsWithArg{ {Intents: []roachpb.Intent{ - {Span: roachpb.Span{Key: roachpb.Key("a")}, Txn: txn.TxnMeta}, + roachpb.MakeIntent(*txn, roachpb.Span{Key: roachpb.Key("a")}), }}, } // Running with allowSyncProcessing = false should result in an error and no @@ -619,7 +600,7 @@ func TestCleanupIntentsAsync(t *testing.T) { txn := newTransaction("txn", roachpb.Key("a"), 1, clock) testIntentsWithArg := []result.IntentsWithArg{ {Intents: []roachpb.Intent{ - {Span: roachpb.Span{Key: roachpb.Key("a")}, Txn: txn.TxnMeta}, + roachpb.MakeIntent(*txn, roachpb.Span{Key: roachpb.Key("a")}), }}, } cases := []testCase{ @@ -695,6 +676,64 @@ func (sf *sendFuncs) popLocked() sendFunc { return ret } +// TestTxnCleanupIntentsAsyncWithPartialRollback verifies that +// CleanupIntentsAsync properly forwards the ignored seqnum list in +// the resolve intent requests. +func TestCleanupTxnIntentsAsyncWithPartialRollback(t *testing.T) { + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + txn := newTransaction("txn", roachpb.Key("a"), 1, clock) + txn.IntentSpans = []roachpb.Span{ + {Key: roachpb.Key("a")}, + {Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, + } + txn.IgnoredSeqNums = []enginepb.IgnoredSeqNumRange{{Start: 1, End: 1}} + + var gotResolveIntent, gotResolveIntentRange int32 + check := func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { + for _, r := range ba.Requests { + if ri, ok := r.GetInner().(*roachpb.ResolveIntentRequest); ok { + atomic.StoreInt32(&gotResolveIntent, 1) + if !reflect.DeepEqual(ri.IgnoredSeqNums, txn.IgnoredSeqNums) { + t.Errorf("expected ignored list %v, got %v", txn.IgnoredSeqNums, ri.IgnoredSeqNums) + } + } else if rir, ok := r.GetInner().(*roachpb.ResolveIntentRangeRequest); ok { + atomic.StoreInt32(&gotResolveIntentRange, 1) + if !reflect.DeepEqual(rir.IgnoredSeqNums, txn.IgnoredSeqNums) { + t.Errorf("expected ignored list %v, got %v", txn.IgnoredSeqNums, rir.IgnoredSeqNums) + } + } + } + return respForResolveIntentBatch(t, ba), nil + } + sf := newSendFuncs(t, + sendFunc(check), + sendFunc(check), + gcSendFunc(t), + ) + stopper := stop.NewStopper() + defer stopper.Stop(context.Background()) + cfg := Config{ + Stopper: stopper, + Clock: clock, + } + ir := newIntentResolverWithSendFuncs(cfg, sf) + + intents := []result.EndTxnIntents{{Txn: *txn}} + + if err := ir.CleanupTxnIntentsAsync(context.Background(), 1, intents, true /*allowAsyncProcessing*/); err != nil { + t.Fatal(err) + } + testutils.SucceedsSoon(t, func() error { + if atomic.LoadInt32(&gotResolveIntent) == 0 { + return errors.New("still waiting for resolve intent req") + } + if atomic.LoadInt32(&gotResolveIntentRange) == 0 { + return errors.New("still waiting for resolve intent range req") + } + return nil + }) +} + // TestCleanupTxnIntentsAsync verifies that CleanupTxnIntentsAsync sends the // expected requests. func TestCleanupTxnIntentsAsync(t *testing.T) { @@ -788,7 +827,7 @@ func TestCleanupIntents(t *testing.T) { // Set txn.ID to a very small value so it's sorted deterministically first. txn.ID = uuid.UUID{15: 0x01} testIntents := []roachpb.Intent{ - {Span: roachpb.Span{Key: roachpb.Key("a")}, Txn: txn.TxnMeta}, + roachpb.MakeIntent(*txn, roachpb.Span{Key: roachpb.Key("a")}), } type testCase struct { intents []roachpb.Intent @@ -872,7 +911,7 @@ func makeTxnIntents(t *testing.T, clock *hlc.Clock, numIntents int) []roachpb.In for i := 0; i < numIntents; i++ { txn := newTransaction("test", roachpb.Key("a"), 1, clock) ret = append(ret, - roachpb.Intent{Span: roachpb.Span{Key: txn.Key}, Txn: txn.TxnMeta}) + roachpb.MakeIntent(*txn, roachpb.Span{Key: txn.Key})) } return ret } diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index a5da91dc93e8..e8ea6c0f7365 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -4379,11 +4379,9 @@ func TestEndTransactionRollbackAbortedTransaction(t *testing.T) { } if pErr := tc.store.intentResolver.ResolveIntents(context.TODO(), - []roachpb.Intent{{ - Span: roachpb.Span{Key: key}, - Txn: txnRecord.TxnMeta, - Status: txnRecord.Status, - }}, intentresolver.ResolveOptions{Wait: true, Poison: true}); pErr != nil { + []roachpb.Intent{ + roachpb.MakeIntent(txnRecord, roachpb.Span{Key: key}), + }, intentresolver.ResolveOptions{Wait: true, Poison: true}); pErr != nil { t.Fatal(pErr) } } @@ -5048,11 +5046,11 @@ func TestReplicaResolveIntentNoWait(t *testing.T) { txn := newTransaction("name", key, 1, tc.Clock()) txn.Status = roachpb.COMMITTED if pErr := tc.store.intentResolver.ResolveIntents(context.Background(), - []roachpb.Intent{{ - Span: roachpb.Span{Key: key}, - Txn: txn.TxnMeta, - Status: txn.Status, - }}, intentresolver.ResolveOptions{Wait: false, Poison: true /* irrelevant */}); pErr != nil { + []roachpb.Intent{ + roachpb.MakeIntent(*txn, roachpb.Span{Key: key}), + }, + intentresolver.ResolveOptions{Wait: false, Poison: true /* irrelevant */}, + ); pErr != nil { t.Fatal(pErr) } testutils.SucceedsSoon(t, func() error { @@ -6005,8 +6003,9 @@ func TestReplicaResolveIntentRange(t *testing.T) { Key: roachpb.Key("a"), EndKey: roachpb.Key("c"), }, - IntentTxn: txn.TxnMeta, - Status: roachpb.COMMITTED, + IntentTxn: txn.TxnMeta, + Status: roachpb.COMMITTED, + IgnoredSeqNums: txn.IgnoredSeqNums, } if _, pErr := tc.SendWrapped(rArgs); pErr != nil { t.Fatal(pErr) @@ -6119,8 +6118,9 @@ func TestRangeStatsComputation(t *testing.T) { RequestHeader: roachpb.RequestHeader{ Key: pArgs.Key, }, - IntentTxn: txn.TxnMeta, - Status: roachpb.COMMITTED, + IntentTxn: txn.TxnMeta, + Status: roachpb.COMMITTED, + IgnoredSeqNums: txn.IgnoredSeqNums, } if _, pErr := tc.SendWrapped(rArgs); pErr != nil { @@ -6520,8 +6520,9 @@ func TestReplicaLookupUseReverseScan(t *testing.T) { Key: keys.RangeMetaKey(roachpb.RKey("a")).AsRawKey(), EndKey: keys.RangeMetaKey(roachpb.RKey("z")).AsRawKey(), }, - IntentTxn: txn.TxnMeta, - Status: roachpb.COMMITTED, + IntentTxn: txn.TxnMeta, + Status: roachpb.COMMITTED, + IgnoredSeqNums: txn.IgnoredSeqNums, } if _, pErr := tc.SendWrapped(rArgs); pErr != nil { t.Fatal(pErr)