From eea660864312e66737d4f9ab213587f985843e2e 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. The code is still incomplete wrt. `CPut` which should also skip writes at ignored seqnums to evaluate its condition. Release note: None --- c-deps/libroach/include/libroach.h | 14 + c-deps/libroach/mvcc.h | 96 +++- c-deps/libroach/protos/roachpb/api.pb.cc | 2 +- c-deps/libroach/protos/roachpb/data.pb.cc | 2 +- .../protos/storage/engine/enginepb/mvcc.pb.cc | 2 +- .../storage/engine/enginepb/mvcc3.pb.cc | 279 ++++++++++- .../protos/storage/engine/enginepb/mvcc3.pb.h | 209 +++++++- pkg/roachpb/data.go | 10 + pkg/roachpb/data_test.go | 17 +- pkg/storage/below_raft_protos_test.go | 2 +- pkg/storage/engine/enginepb/mvcc3.pb.go | 472 ++++++++++++++---- pkg/storage/engine/enginepb/mvcc3.proto | 14 + pkg/storage/engine/mvcc.go | 86 +++- pkg/storage/engine/rocksdb.go | 11 + 14 files changed, 1074 insertions(+), 142 deletions(-) diff --git a/c-deps/libroach/include/libroach.h b/c-deps/libroach/include/libroach.h index a46ca49edc5a..d96651580ed3 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 +// TxnMeta_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; @@ -323,6 +336,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 0a2907c28680..3638e2201b34 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), ignore_sequence_(ignore_sequence), @@ -164,27 +165,81 @@ 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. + 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); @@ -302,7 +357,7 @@ template class mvccScanner { } if (txn_epoch_ == meta_.txn().epoch()) { - if ((ignore_sequence_) || (txn_sequence_ >= meta_.txn().sequence())) { + if ((ignore_sequence_) || (txn_sequence_ >= meta_.txn().sequence() && !seqNumIsIgnored(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 @@ -664,6 +719,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 ignore_sequence_; diff --git a/c-deps/libroach/protos/roachpb/api.pb.cc b/c-deps/libroach/protos/roachpb/api.pb.cc index c676c7043233..2de6ed354d26 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.cc +++ b/c-deps/libroach/protos/roachpb/api.pb.cc @@ -151,7 +151,7 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fmetadata_2eproto ::google::pr } // 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_MVCCStatsDelta; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_TxnMeta; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_TxnMeta; } // namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto namespace protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCStats; diff --git a/c-deps/libroach/protos/roachpb/data.pb.cc b/c-deps/libroach/protos/roachpb/data.pb.cc index 1ec8abd1d5b3..0d924fd3c2ea 100644 --- a/c-deps/libroach/protos/roachpb/data.pb.cc +++ b/c-deps/libroach/protos/roachpb/data.pb.cc @@ -33,7 +33,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<1> scc_info_TxnMeta; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_TxnMeta; } // namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto namespace protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCStats; diff --git a/c-deps/libroach/protos/storage/engine/enginepb/mvcc.pb.cc b/c-deps/libroach/protos/storage/engine/enginepb/mvcc.pb.cc index d72c610f95c0..8eb806674f2b 100644 --- a/c-deps/libroach/protos/storage/engine/enginepb/mvcc.pb.cc +++ b/c-deps/libroach/protos/storage/engine/enginepb/mvcc.pb.cc @@ -17,7 +17,7 @@ // @@protoc_insertion_point(includes) namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto { -extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_TxnMeta; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_TxnMeta; } // namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto namespace protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCMetadata_SequencedIntent; 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 a532907c0af5..d6141390a6e9 100644 --- a/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.cc +++ b/c-deps/libroach/protos/storage/engine/enginepb/mvcc3.pb.cc @@ -20,6 +20,7 @@ namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCAbortIntentOp; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCAbortTxnOp; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCPersistentStats; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_TxnMeta_IgnoredSeqNumRange; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_MVCCCommitIntentOp; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_MVCCUpdateIntentOp; extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_MVCCWriteIntentOp; @@ -32,6 +33,11 @@ namespace cockroach { namespace storage { namespace engine { namespace enginepb { +class TxnMeta_IgnoredSeqNumRangeDefaultTypeInternal { + public: + ::google::protobuf::internal::ExplicitlyConstructed + _instance; +} _TxnMeta_IgnoredSeqNumRange_default_instance_; class TxnMetaDefaultTypeInternal { public: ::google::protobuf::internal::ExplicitlyConstructed @@ -92,6 +98,20 @@ class MVCCLogicalOpDefaultTypeInternal { } // namespace storage } // namespace cockroach namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto { +static void InitDefaultsTxnMeta_IgnoredSeqNumRange() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::cockroach::storage::engine::enginepb::_TxnMeta_IgnoredSeqNumRange_default_instance_; + new (ptr) ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange(); + ::google::protobuf::internal::OnShutdownDestroyMessage(ptr); + } + ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange::InitAsDefaultInstance(); +} + +::google::protobuf::internal::SCCInfo<0> scc_info_TxnMeta_IgnoredSeqNumRange = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsTxnMeta_IgnoredSeqNumRange}, {}}; + static void InitDefaultsTxnMeta() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -103,9 +123,10 @@ static void InitDefaultsTxnMeta() { ::cockroach::storage::engine::enginepb::TxnMeta::InitAsDefaultInstance(); } -::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,}}; +::google::protobuf::internal::SCCInfo<2> scc_info_TxnMeta = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsTxnMeta}, { + &protobuf_util_2fhlc_2ftimestamp_2eproto::scc_info_Timestamp.base, + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta_IgnoredSeqNumRange.base,}}; static void InitDefaultsMVCCStatsDelta() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -259,6 +280,7 @@ ::google::protobuf::internal::SCCInfo<6> scc_info_MVCCLogicalOp = &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_MVCCAbortTxnOp.base,}}; void InitDefaults() { + ::google::protobuf::internal::InitSCC(&scc_info_TxnMeta_IgnoredSeqNumRange.base); ::google::protobuf::internal::InitSCC(&scc_info_TxnMeta.base); ::google::protobuf::internal::InitSCC(&scc_info_MVCCStatsDelta.base); ::google::protobuf::internal::InitSCC(&scc_info_MVCCPersistentStats.base); @@ -278,6 +300,218 @@ namespace storage { namespace engine { namespace enginepb { +// =================================================================== + +void TxnMeta_IgnoredSeqNumRange::InitAsDefaultInstance() { +} +#if !defined(_MSC_VER) || _MSC_VER >= 1900 +const int TxnMeta_IgnoredSeqNumRange::kStartFieldNumber; +const int TxnMeta_IgnoredSeqNumRange::kEndFieldNumber; +#endif // !defined(_MSC_VER) || _MSC_VER >= 1900 + +TxnMeta_IgnoredSeqNumRange::TxnMeta_IgnoredSeqNumRange() + : ::google::protobuf::MessageLite(), _internal_metadata_(NULL) { + ::google::protobuf::internal::InitSCC( + &protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta_IgnoredSeqNumRange.base); + SharedCtor(); + // @@protoc_insertion_point(constructor:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange) +} +TxnMeta_IgnoredSeqNumRange::TxnMeta_IgnoredSeqNumRange(const TxnMeta_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.TxnMeta.IgnoredSeqNumRange) +} + +void TxnMeta_IgnoredSeqNumRange::SharedCtor() { + ::memset(&start_, 0, static_cast( + reinterpret_cast(&end_) - + reinterpret_cast(&start_)) + sizeof(end_)); +} + +TxnMeta_IgnoredSeqNumRange::~TxnMeta_IgnoredSeqNumRange() { + // @@protoc_insertion_point(destructor:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange) + SharedDtor(); +} + +void TxnMeta_IgnoredSeqNumRange::SharedDtor() { +} + +void TxnMeta_IgnoredSeqNumRange::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const TxnMeta_IgnoredSeqNumRange& TxnMeta_IgnoredSeqNumRange::default_instance() { + ::google::protobuf::internal::InitSCC(&protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto::scc_info_TxnMeta_IgnoredSeqNumRange.base); + return *internal_default_instance(); +} + + +void TxnMeta_IgnoredSeqNumRange::Clear() { +// @@protoc_insertion_point(message_clear_start:cockroach.storage.engine.enginepb.TxnMeta.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 TxnMeta_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.TxnMeta.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.TxnMeta.IgnoredSeqNumRange) + return true; +failure: + // @@protoc_insertion_point(parse_failure:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange) + return false; +#undef DO_ +} + +void TxnMeta_IgnoredSeqNumRange::SerializeWithCachedSizes( + ::google::protobuf::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:cockroach.storage.engine.enginepb.TxnMeta.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.TxnMeta.IgnoredSeqNumRange) +} + +size_t TxnMeta_IgnoredSeqNumRange::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:cockroach.storage.engine.enginepb.TxnMeta.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 TxnMeta_IgnoredSeqNumRange::CheckTypeAndMergeFrom( + const ::google::protobuf::MessageLite& from) { + MergeFrom(*::google::protobuf::down_cast(&from)); +} + +void TxnMeta_IgnoredSeqNumRange::MergeFrom(const TxnMeta_IgnoredSeqNumRange& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:cockroach.storage.engine.enginepb.TxnMeta.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 TxnMeta_IgnoredSeqNumRange::CopyFrom(const TxnMeta_IgnoredSeqNumRange& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool TxnMeta_IgnoredSeqNumRange::IsInitialized() const { + return true; +} + +void TxnMeta_IgnoredSeqNumRange::Swap(TxnMeta_IgnoredSeqNumRange* other) { + if (other == this) return; + InternalSwap(other); +} +void TxnMeta_IgnoredSeqNumRange::InternalSwap(TxnMeta_IgnoredSeqNumRange* other) { + using std::swap; + swap(start_, other->start_); + swap(end_, other->end_); + _internal_metadata_.Swap(&other->_internal_metadata_); +} + +::std::string TxnMeta_IgnoredSeqNumRange::GetTypeName() const { + return "cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange"; +} + + // =================================================================== void TxnMeta::InitAsDefaultInstance() { @@ -306,6 +540,7 @@ const int TxnMeta::kTimestampFieldNumber; const int TxnMeta::kMinTimestampFieldNumber; const int TxnMeta::kPriorityFieldNumber; const int TxnMeta::kSequenceFieldNumber; +const int TxnMeta::kIgnoredSeqnumsFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 TxnMeta::TxnMeta() @@ -317,7 +552,8 @@ TxnMeta::TxnMeta() } TxnMeta::TxnMeta(const TxnMeta& from) : ::google::protobuf::MessageLite(), - _internal_metadata_(NULL) { + _internal_metadata_(NULL), + ignored_seqnums_(from.ignored_seqnums_) { _internal_metadata_.MergeFrom(from._internal_metadata_); id_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (from.id().size() > 0) { @@ -378,6 +614,7 @@ void TxnMeta::Clear() { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; + ignored_seqnums_.Clear(); id_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); key_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (GetArenaNoVirtual() == NULL && timestamp_ != NULL) { @@ -494,6 +731,17 @@ bool TxnMeta::MergePartialFromCodedStream( break; } + case 10: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(82u /* 82 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, add_ignored_seqnums())); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -553,6 +801,14 @@ void TxnMeta::SerializeWithCachedSizes( 9, 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( + 10, + 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.storage.engine.enginepb.TxnMeta) @@ -564,6 +820,16 @@ size_t TxnMeta::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->id().size() > 0) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::BytesSize( @@ -624,6 +890,7 @@ void TxnMeta::MergeFrom(const TxnMeta& from) { ::google::protobuf::uint32 cached_has_bits = 0; (void) cached_has_bits; + ignored_seqnums_.MergeFrom(from.ignored_seqnums_); if (from.id().size() > 0) { id_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.id_); @@ -666,6 +933,7 @@ void TxnMeta::Swap(TxnMeta* other) { } void TxnMeta::InternalSwap(TxnMeta* other) { using std::swap; + CastToBase(&ignored_seqnums_)->InternalSwap(CastToBase(&other->ignored_seqnums_)); id_.Swap(&other->id_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), GetArenaNoVirtual()); key_.Swap(&other->key_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), @@ -3999,6 +4267,9 @@ ::std::string MVCCLogicalOp::GetTypeName() const { } // namespace cockroach namespace google { namespace protobuf { +template<> GOOGLE_PROTOBUF_ATTRIBUTE_NOINLINE ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange* Arena::CreateMaybeMessage< ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange >(Arena* arena) { + return Arena::CreateInternal< ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange >(arena); +} 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); } 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 271594d68924..6aa565ce58b1 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[]; @@ -81,6 +81,9 @@ extern RangeAppliedStateDefaultTypeInternal _RangeAppliedState_default_instance_ class TxnMeta; class TxnMetaDefaultTypeInternal; extern TxnMetaDefaultTypeInternal _TxnMeta_default_instance_; +class TxnMeta_IgnoredSeqNumRange; +class TxnMeta_IgnoredSeqNumRangeDefaultTypeInternal; +extern TxnMeta_IgnoredSeqNumRangeDefaultTypeInternal _TxnMeta_IgnoredSeqNumRange_default_instance_; } // namespace enginepb } // namespace engine } // namespace storage @@ -98,6 +101,7 @@ template<> ::cockroach::storage::engine::enginepb::MVCCWriteIntentOp* Arena::Cre template<> ::cockroach::storage::engine::enginepb::MVCCWriteValueOp* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::MVCCWriteValueOp>(Arena*); template<> ::cockroach::storage::engine::enginepb::RangeAppliedState* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::RangeAppliedState>(Arena*); template<> ::cockroach::storage::engine::enginepb::TxnMeta* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::TxnMeta>(Arena*); +template<> ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange* Arena::CreateMaybeMessage<::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange>(Arena*); } // namespace protobuf } // namespace google namespace cockroach { @@ -107,6 +111,112 @@ namespace enginepb { // =================================================================== +class TxnMeta_IgnoredSeqNumRange : public ::google::protobuf::MessageLite /* @@protoc_insertion_point(class_definition:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange) */ { + public: + TxnMeta_IgnoredSeqNumRange(); + virtual ~TxnMeta_IgnoredSeqNumRange(); + + TxnMeta_IgnoredSeqNumRange(const TxnMeta_IgnoredSeqNumRange& from); + + inline TxnMeta_IgnoredSeqNumRange& operator=(const TxnMeta_IgnoredSeqNumRange& from) { + CopyFrom(from); + return *this; + } + #if LANG_CXX11 + TxnMeta_IgnoredSeqNumRange(TxnMeta_IgnoredSeqNumRange&& from) noexcept + : TxnMeta_IgnoredSeqNumRange() { + *this = ::std::move(from); + } + + inline TxnMeta_IgnoredSeqNumRange& operator=(TxnMeta_IgnoredSeqNumRange&& from) noexcept { + if (GetArenaNoVirtual() == from.GetArenaNoVirtual()) { + if (this != &from) InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + #endif + static const TxnMeta_IgnoredSeqNumRange& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const TxnMeta_IgnoredSeqNumRange* internal_default_instance() { + return reinterpret_cast( + &_TxnMeta_IgnoredSeqNumRange_default_instance_); + } + static constexpr int kIndexInFileMessages = + 0; + + void Swap(TxnMeta_IgnoredSeqNumRange* other); + friend void swap(TxnMeta_IgnoredSeqNumRange& a, TxnMeta_IgnoredSeqNumRange& b) { + a.Swap(&b); + } + + // implements Message ---------------------------------------------- + + inline TxnMeta_IgnoredSeqNumRange* New() const final { + return CreateMaybeMessage(NULL); + } + + TxnMeta_IgnoredSeqNumRange* New(::google::protobuf::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CheckTypeAndMergeFrom(const ::google::protobuf::MessageLite& from) + final; + void CopyFrom(const TxnMeta_IgnoredSeqNumRange& from); + void MergeFrom(const TxnMeta_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(TxnMeta_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.TxnMeta.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 TxnMeta : public ::google::protobuf::MessageLite /* @@protoc_insertion_point(class_definition:cockroach.storage.engine.enginepb.TxnMeta) */ { public: TxnMeta(); @@ -141,7 +251,7 @@ class TxnMeta : public ::google::protobuf::MessageLite /* @@protoc_insertion_poi &_TxnMeta_default_instance_); } static constexpr int kIndexInFileMessages = - 0; + 1; void Swap(TxnMeta* other); friend void swap(TxnMeta& a, TxnMeta& b) { @@ -190,8 +300,21 @@ class TxnMeta : public ::google::protobuf::MessageLite /* @@protoc_insertion_poi // nested types ---------------------------------------------------- + typedef TxnMeta_IgnoredSeqNumRange IgnoredSeqNumRange; + // accessors ------------------------------------------------------- + int ignored_seqnums_size() const; + void clear_ignored_seqnums(); + static const int kIgnoredSeqnumsFieldNumber = 10; + ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange* mutable_ignored_seqnums(int index); + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange >* + mutable_ignored_seqnums(); + const ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange& ignored_seqnums(int index) const; + ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange* add_ignored_seqnums(); + const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange >& + ignored_seqnums() const; + void clear_id(); static const int kIdFieldNumber = 1; const ::std::string& id() const; @@ -260,6 +383,7 @@ class TxnMeta : public ::google::protobuf::MessageLite /* @@protoc_insertion_poi private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; + ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange > ignored_seqnums_; ::google::protobuf::internal::ArenaStringPtr id_; ::google::protobuf::internal::ArenaStringPtr key_; ::cockroach::util::hlc::Timestamp* timestamp_; @@ -306,7 +430,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 +621,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 +812,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 +932,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) { @@ -944,7 +1068,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) { @@ -1091,7 +1215,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) { @@ -1211,7 +1335,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) { @@ -1361,7 +1485,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) { @@ -1469,7 +1593,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) { @@ -1577,7 +1701,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) { @@ -1722,6 +1846,36 @@ class MVCCLogicalOp : public ::google::protobuf::MessageLite /* @@protoc_inserti #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wstrict-aliasing" #endif // __GNUC__ +// TxnMeta_IgnoredSeqNumRange + +inline void TxnMeta_IgnoredSeqNumRange::clear_start() { + start_ = 0; +} +inline ::google::protobuf::int32 TxnMeta_IgnoredSeqNumRange::start() const { + // @@protoc_insertion_point(field_get:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange.start) + return start_; +} +inline void TxnMeta_IgnoredSeqNumRange::set_start(::google::protobuf::int32 value) { + + start_ = value; + // @@protoc_insertion_point(field_set:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange.start) +} + +inline void TxnMeta_IgnoredSeqNumRange::clear_end() { + end_ = 0; +} +inline ::google::protobuf::int32 TxnMeta_IgnoredSeqNumRange::end() const { + // @@protoc_insertion_point(field_get:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange.end) + return end_; +} +inline void TxnMeta_IgnoredSeqNumRange::set_end(::google::protobuf::int32 value) { + + end_ = value; + // @@protoc_insertion_point(field_set:cockroach.storage.engine.enginepb.TxnMeta.IgnoredSeqNumRange.end) +} + +// ------------------------------------------------------------------- + // TxnMeta inline void TxnMeta::clear_id() { @@ -1962,6 +2116,35 @@ inline void TxnMeta::set_sequence(::google::protobuf::int32 value) { // @@protoc_insertion_point(field_set:cockroach.storage.engine.enginepb.TxnMeta.sequence) } +inline int TxnMeta::ignored_seqnums_size() const { + return ignored_seqnums_.size(); +} +inline void TxnMeta::clear_ignored_seqnums() { + ignored_seqnums_.Clear(); +} +inline ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange* TxnMeta::mutable_ignored_seqnums(int index) { + // @@protoc_insertion_point(field_mutable:cockroach.storage.engine.enginepb.TxnMeta.ignored_seqnums) + return ignored_seqnums_.Mutable(index); +} +inline ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange >* +TxnMeta::mutable_ignored_seqnums() { + // @@protoc_insertion_point(field_mutable_list:cockroach.storage.engine.enginepb.TxnMeta.ignored_seqnums) + return &ignored_seqnums_; +} +inline const ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange& TxnMeta::ignored_seqnums(int index) const { + // @@protoc_insertion_point(field_get:cockroach.storage.engine.enginepb.TxnMeta.ignored_seqnums) + return ignored_seqnums_.Get(index); +} +inline ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange* TxnMeta::add_ignored_seqnums() { + // @@protoc_insertion_point(field_add:cockroach.storage.engine.enginepb.TxnMeta.ignored_seqnums) + return ignored_seqnums_.Add(); +} +inline const ::google::protobuf::RepeatedPtrField< ::cockroach::storage::engine::enginepb::TxnMeta_IgnoredSeqNumRange >& +TxnMeta::ignored_seqnums() const { + // @@protoc_insertion_point(field_list:cockroach.storage.engine.enginepb.TxnMeta.ignored_seqnums) + return ignored_seqnums_; +} + // ------------------------------------------------------------------- // MVCCStatsDelta @@ -3580,6 +3763,8 @@ inline void MVCCLogicalOp::set_allocated_abort_txn(::cockroach::storage::engine: // ------------------------------------------------------------------- +// ------------------------------------------------------------------- + // @@protoc_insertion_point(namespace_scope) diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index aa9591df331a..e8b9bbd03cde 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1001,6 +1001,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 { @@ -1037,6 +1038,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 { @@ -1110,6 +1114,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() } @@ -1134,6 +1141,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() } diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 115a232b529b..cc7893d0b712 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -476,13 +476,14 @@ func TestFastPathObservedTimestamp(t *testing.T) { var nonZeroTxn = Transaction{ TxnMeta: enginepb.TxnMeta{ - Key: Key("foo"), - ID: uuid.MakeV4(), - Epoch: 2, - Timestamp: makeTS(20, 21), - MinTimestamp: makeTS(10, 11), - Priority: 957356782, - Sequence: 123, + Key: Key("foo"), + ID: uuid.MakeV4(), + Epoch: 2, + Timestamp: makeTS(20, 21), + MinTimestamp: makeTS(10, 11), + Priority: 957356782, + Sequence: 123, + IgnoredSeqNums: []enginepb.TxnMeta_IgnoredSeqNumRange{{Start: 888, End: 999}}, }, Name: "name", Status: COMMITTED, @@ -556,6 +557,7 @@ func TestTransactionUpdate(t *testing.T) { expTxn5.Sequence = txn.Sequence - 10 expTxn5.IntentSpans = nil expTxn5.InFlightWrites = nil + expTxn5.IgnoredSeqNums = nil expTxn5.WriteTooOld = false expTxn5.OrigTimestampWasObserved = false require.Equal(t, expTxn5, txn5) @@ -655,6 +657,7 @@ func TestTransactionClone(t *testing.T) { "IntentSpans.EndKey", "IntentSpans.Key", "ObservedTimestamps", + "TxnMeta.IgnoredSeqNums", "TxnMeta.Key", } if !reflect.DeepEqual(expFields, fields) { diff --git a/pkg/storage/below_raft_protos_test.go b/pkg/storage/below_raft_protos_test.go index f47044cfa9f6..ddb8274fa7e1 100644 --- a/pkg/storage/below_raft_protos_test.go +++ b/pkg/storage/below_raft_protos_test.go @@ -59,7 +59,7 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ return m }, emptySum: 7551962144604783939, - populatedSum: 12720006657210437557, + populatedSum: 4789115409959538678, }, reflect.TypeOf(&enginepb.RangeAppliedState{}): { populatedConstructor: func(r *rand.Rand) protoutil.Message { diff --git a/pkg/storage/engine/enginepb/mvcc3.pb.go b/pkg/storage/engine/enginepb/mvcc3.pb.go index 65a973b121c0..078ca3944e4c 100644 --- a/pkg/storage/engine/enginepb/mvcc3.pb.go +++ b/pkg/storage/engine/enginepb/mvcc3.pb.go @@ -118,13 +118,20 @@ type TxnMeta struct { // last request. Used to provide idempotency and to protect against // out-of-order application (by means of a transaction retry). Sequence TxnSeq `protobuf:"varint,7,opt,name=sequence,proto3,casttype=TxnSeq" json:"sequence,omitempty"` + // A list of ignored seqnum ranges. + // + // The user code must guarantee this list to be non-overlapping, + // non-continuous (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 []TxnMeta_IgnoredSeqNumRange `protobuf:"bytes,10,rep,name=ignored_seqnums,json=ignoredSeqnums,proto3" json:"ignored_seqnums"` } func (m *TxnMeta) Reset() { *m = TxnMeta{} } func (m *TxnMeta) String() string { return proto.CompactTextString(m) } func (*TxnMeta) ProtoMessage() {} func (*TxnMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_d6ebeaa8ad44885c, []int{0} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{0} } func (m *TxnMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -149,6 +156,40 @@ func (m *TxnMeta) XXX_DiscardUnknown() { var xxx_messageInfo_TxnMeta proto.InternalMessageInfo +type TxnMeta_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 *TxnMeta_IgnoredSeqNumRange) Reset() { *m = TxnMeta_IgnoredSeqNumRange{} } +func (m *TxnMeta_IgnoredSeqNumRange) String() string { return proto.CompactTextString(m) } +func (*TxnMeta_IgnoredSeqNumRange) ProtoMessage() {} +func (*TxnMeta_IgnoredSeqNumRange) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{0, 0} +} +func (m *TxnMeta_IgnoredSeqNumRange) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxnMeta_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 *TxnMeta_IgnoredSeqNumRange) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnMeta_IgnoredSeqNumRange.Merge(dst, src) +} +func (m *TxnMeta_IgnoredSeqNumRange) XXX_Size() int { + return m.Size() +} +func (m *TxnMeta_IgnoredSeqNumRange) XXX_DiscardUnknown() { + xxx_messageInfo_TxnMeta_IgnoredSeqNumRange.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnMeta_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. @@ -173,7 +214,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_d6ebeaa8ad44885c, []int{1} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{1} } func (m *MVCCStatsDelta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -223,7 +264,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_d6ebeaa8ad44885c, []int{2} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{2} } func (m *MVCCPersistentStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -268,7 +309,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_d6ebeaa8ad44885c, []int{3} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{3} } func (m *RangeAppliedState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -305,7 +346,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_d6ebeaa8ad44885c, []int{4} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{4} } func (m *MVCCWriteValueOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -343,7 +384,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_d6ebeaa8ad44885c, []int{5} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{5} } func (m *MVCCWriteIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -379,7 +420,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_d6ebeaa8ad44885c, []int{6} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{6} } func (m *MVCCUpdateIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -417,7 +458,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_d6ebeaa8ad44885c, []int{7} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{7} } func (m *MVCCCommitIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -457,7 +498,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_d6ebeaa8ad44885c, []int{8} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{8} } func (m *MVCCAbortIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -493,7 +534,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_d6ebeaa8ad44885c, []int{9} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{9} } func (m *MVCCAbortTxnOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -532,7 +573,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_d6ebeaa8ad44885c, []int{10} + return fileDescriptor_mvcc3_351ae7cbe7e93b17, []int{10} } func (m *MVCCLogicalOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,6 +600,7 @@ var xxx_messageInfo_MVCCLogicalOp proto.InternalMessageInfo func init() { proto.RegisterType((*TxnMeta)(nil), "cockroach.storage.engine.enginepb.TxnMeta") + proto.RegisterType((*TxnMeta_IgnoredSeqNumRange)(nil), "cockroach.storage.engine.enginepb.TxnMeta.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") @@ -610,6 +652,41 @@ func (this *TxnMeta) Equal(that interface{}) bool { if this.Sequence != that1.Sequence { 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 *TxnMeta_IgnoredSeqNumRange) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*TxnMeta_IgnoredSeqNumRange) + if !ok { + that2, ok := that.(TxnMeta_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 { @@ -828,6 +905,46 @@ func (m *TxnMeta) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n3 + if len(m.IgnoredSeqNums) > 0 { + for _, msg := range m.IgnoredSeqNums { + dAtA[i] = 0x52 + i++ + i = encodeVarintMvcc3(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *TxnMeta_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 *TxnMeta_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 } @@ -1379,6 +1496,29 @@ func NewPopulatedTxnMeta(r randyMvcc3, easy bool) *TxnMeta { } v4 := hlc.NewPopulatedTimestamp(r, easy) this.MinTimestamp = *v4 + if r.Intn(10) != 0 { + v5 := r.Intn(5) + this.IgnoredSeqNums = make([]TxnMeta_IgnoredSeqNumRange, v5) + for i := 0; i < v5; i++ { + v6 := NewPopulatedTxnMeta_IgnoredSeqNumRange(r, easy) + this.IgnoredSeqNums[i] = *v6 + } + } + if !easy && r.Intn(10) != 0 { + } + return this +} + +func NewPopulatedTxnMeta_IgnoredSeqNumRange(r randyMvcc3, easy bool) *TxnMeta_IgnoredSeqNumRange { + this := &TxnMeta_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 @@ -1451,8 +1591,8 @@ func NewPopulatedRangeAppliedState(r randyMvcc3, easy bool) *RangeAppliedState { this := &RangeAppliedState{} this.RaftAppliedIndex = uint64(uint64(r.Uint32())) this.LeaseAppliedIndex = uint64(uint64(r.Uint32())) - v5 := NewPopulatedMVCCPersistentStats(r, easy) - this.RangeStats = *v5 + v7 := NewPopulatedMVCCPersistentStats(r, easy) + this.RangeStats = *v7 if !easy && r.Intn(10) != 0 { } return this @@ -1477,9 +1617,9 @@ func randUTF8RuneMvcc3(r randyMvcc3) rune { return rune(ru + 61) } func randStringMvcc3(r randyMvcc3) string { - v6 := r.Intn(100) - tmps := make([]rune, v6) - for i := 0; i < v6; i++ { + v8 := r.Intn(100) + tmps := make([]rune, v8) + for i := 0; i < v8; i++ { tmps[i] = randUTF8RuneMvcc3(r) } return string(tmps) @@ -1501,11 +1641,11 @@ func randFieldMvcc3(dAtA []byte, r randyMvcc3, fieldNumber int, wire int) []byte switch wire { case 0: dAtA = encodeVarintPopulateMvcc3(dAtA, uint64(key)) - v7 := r.Int63() + v9 := r.Int63() if r.Intn(2) == 0 { - v7 *= -1 + v9 *= -1 } - dAtA = encodeVarintPopulateMvcc3(dAtA, uint64(v7)) + dAtA = encodeVarintPopulateMvcc3(dAtA, uint64(v9)) case 1: dAtA = encodeVarintPopulateMvcc3(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))) @@ -1555,6 +1695,27 @@ func (m *TxnMeta) Size() (n int) { } l = m.MinTimestamp.Size() n += 1 + l + sovMvcc3(uint64(l)) + if len(m.IgnoredSeqNums) > 0 { + for _, e := range m.IgnoredSeqNums { + l = e.Size() + n += 1 + l + sovMvcc3(uint64(l)) + } + } + return n +} + +func (m *TxnMeta_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 } @@ -2065,6 +2226,125 @@ func (m *TxnMeta) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 10: + 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 ErrIntOverflowMvcc3 + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMvcc3 + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IgnoredSeqNums = append(m.IgnoredSeqNums, TxnMeta_IgnoredSeqNumRange{}) + if err := m.IgnoredSeqNums[len(m.IgnoredSeqNums)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 *TxnMeta_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:]) @@ -3911,78 +4191,84 @@ var ( ) func init() { - proto.RegisterFile("storage/engine/enginepb/mvcc3.proto", fileDescriptor_mvcc3_d6ebeaa8ad44885c) -} - -var fileDescriptor_mvcc3_d6ebeaa8ad44885c = []byte{ - // 1101 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0x4f, 0x6f, 0x1b, 0x45, - 0x14, 0xcf, 0x7a, 0xd7, 0x89, 0xfd, 0xec, 0x24, 0xf6, 0xb4, 0x12, 0x56, 0x51, 0xed, 0xd4, 0x48, - 0x28, 0xe2, 0xcf, 0x1a, 0x12, 0xe0, 0x90, 0x9b, 0x9d, 0x54, 0xe0, 0x42, 0x9a, 0xb2, 0x75, 0x5a, - 0x09, 0x84, 0x56, 0xe3, 0xf5, 0xb0, 0x19, 0x65, 0x3d, 0xbb, 0xf5, 0x8e, 0xd3, 0xf5, 0x81, 0x2f, - 0xc0, 0x89, 0x2f, 0x80, 0x94, 0x0b, 0xdf, 0x80, 0x03, 0x1f, 0x21, 0xc7, 0x1e, 0xab, 0x0a, 0x59, - 0xe0, 0x5c, 0xf8, 0x00, 0x9c, 0x22, 0x21, 0xa1, 0x99, 0x59, 0xaf, 0xed, 0x0a, 0x9c, 0x7f, 0x22, - 0xe2, 0x94, 0x99, 0xf7, 0x7b, 0xef, 0xf7, 0xde, 0xcc, 0xef, 0xed, 0xbc, 0x18, 0xde, 0x0a, 0xb9, - 0xdf, 0xc3, 0x2e, 0xa9, 0x11, 0xe6, 0x52, 0x36, 0xfe, 0x13, 0xb4, 0x6b, 0xdd, 0x23, 0xc7, 0xd9, - 0x34, 0x83, 0x9e, 0xcf, 0x7d, 0x74, 0xcf, 0xf1, 0x9d, 0xc3, 0x9e, 0x8f, 0x9d, 0x03, 0x33, 0x76, - 0x37, 0x95, 0x9f, 0x39, 0x76, 0xbf, 0x53, 0xea, 0x73, 0xea, 0xd5, 0x0e, 0x3c, 0xa7, 0xc6, 0x69, - 0x97, 0x84, 0x1c, 0x77, 0x03, 0x15, 0x7c, 0xe7, 0xb6, 0xeb, 0xbb, 0xbe, 0x5c, 0xd6, 0xc4, 0x4a, - 0x59, 0xab, 0xdf, 0xeb, 0xb0, 0xd4, 0x8a, 0xd8, 0x2e, 0xe1, 0x18, 0x7d, 0x09, 0x29, 0xda, 0x29, - 0x69, 0x6b, 0xda, 0x7a, 0xbe, 0x51, 0x3f, 0x19, 0x56, 0x16, 0x5e, 0x0d, 0x2b, 0x9b, 0x2e, 0xe5, - 0x07, 0xfd, 0xb6, 0xe9, 0xf8, 0xdd, 0x5a, 0x92, 0xbd, 0xd3, 0x9e, 0xac, 0x6b, 0xc1, 0xa1, 0x5b, - 0x93, 0x49, 0xfb, 0x7d, 0xda, 0x31, 0xf7, 0xf7, 0x9b, 0x3b, 0xa3, 0x61, 0x25, 0xd5, 0xdc, 0xb1, - 0x52, 0xb4, 0x83, 0x0a, 0xa0, 0x1f, 0x92, 0x41, 0x49, 0x17, 0x9c, 0x96, 0x58, 0xa2, 0x2a, 0xa4, - 0x49, 0xe0, 0x3b, 0x07, 0x25, 0x63, 0x4d, 0x5b, 0x4f, 0x37, 0xf2, 0x67, 0xc3, 0x4a, 0xa6, 0x15, - 0xb1, 0xfb, 0xc2, 0x66, 0x29, 0x08, 0xd5, 0x21, 0x9b, 0x54, 0x5f, 0x4a, 0xaf, 0x69, 0xeb, 0xb9, - 0x8d, 0xbb, 0xe6, 0xe4, 0xec, 0x22, 0x9b, 0x79, 0xe0, 0x39, 0x66, 0x6b, 0xec, 0xd4, 0x30, 0x44, - 0xb9, 0xd6, 0x24, 0x0a, 0xbd, 0x0b, 0x99, 0xa0, 0x47, 0xfd, 0x1e, 0xe5, 0x83, 0xd2, 0xa2, 0xcc, - 0xb4, 0x7a, 0x36, 0xac, 0xe4, 0x5a, 0x11, 0x7b, 0x14, 0x9b, 0xad, 0xc4, 0x01, 0xbd, 0x0d, 0x99, - 0x90, 0x3c, 0xeb, 0x13, 0xe6, 0x90, 0xd2, 0x92, 0x74, 0x86, 0xb3, 0x61, 0x65, 0xb1, 0x15, 0xb1, - 0xc7, 0xe4, 0x99, 0x95, 0x60, 0xe8, 0x33, 0x58, 0xee, 0x52, 0x66, 0x4f, 0x6a, 0xcb, 0x5e, 0xbc, - 0xb6, 0x7c, 0x97, 0xb2, 0xc4, 0xb6, 0x95, 0xf9, 0xe5, 0xb8, 0xa2, 0xfd, 0x71, 0x5c, 0xd1, 0x1e, - 0x18, 0x99, 0x54, 0x41, 0x7f, 0x60, 0x64, 0x32, 0x85, 0x6c, 0xf5, 0x4f, 0x1d, 0x56, 0x76, 0x9f, - 0x6c, 0x6f, 0x3f, 0xe6, 0x98, 0x87, 0x3b, 0xc4, 0xe3, 0x18, 0xbd, 0x03, 0x45, 0x0f, 0x87, 0xdc, - 0xee, 0x07, 0x1d, 0xcc, 0x89, 0xcd, 0x30, 0xf3, 0x43, 0x29, 0x51, 0xc1, 0x5a, 0x15, 0xc0, 0xbe, - 0xb4, 0x3f, 0x14, 0x66, 0x74, 0x17, 0x80, 0x32, 0x4e, 0x18, 0xb7, 0xb1, 0x4b, 0x4a, 0x29, 0xe9, - 0x94, 0x55, 0x96, 0xba, 0x4b, 0xd0, 0x07, 0x90, 0x77, 0x1d, 0xbb, 0x3d, 0xe0, 0x24, 0x94, 0x0e, - 0x42, 0x94, 0x42, 0x63, 0x65, 0x34, 0xac, 0xc0, 0xa7, 0xdb, 0x0d, 0x61, 0xae, 0xbb, 0xc4, 0x02, - 0xd7, 0x19, 0xaf, 0x05, 0xa1, 0x47, 0x8f, 0x88, 0x8a, 0x91, 0x82, 0x21, 0x2b, 0x2b, 0x2c, 0xd2, - 0x23, 0x81, 0x1d, 0xbf, 0xcf, 0xb8, 0xd4, 0x29, 0x86, 0xb7, 0x85, 0x01, 0xbd, 0x09, 0xd9, 0x43, - 0x32, 0x88, 0x83, 0x17, 0x25, 0x9a, 0x39, 0x24, 0x03, 0x15, 0x1b, 0x83, 0x2a, 0x74, 0x29, 0x01, - 0x93, 0xc8, 0x23, 0xec, 0xc5, 0x91, 0x19, 0x05, 0x1e, 0x61, 0x2f, 0x89, 0x14, 0xa0, 0x8a, 0xcc, - 0x26, 0xa0, 0x8a, 0xbc, 0x07, 0xf9, 0xf8, 0x0a, 0x54, 0x30, 0x48, 0x3c, 0xa7, 0x6c, 0x2a, 0x7e, - 0xe2, 0xa2, 0x28, 0x72, 0xd3, 0x2e, 0x49, 0xfe, 0x70, 0x10, 0xc6, 0x14, 0x79, 0x95, 0x22, 0x1c, - 0x84, 0x49, 0x7e, 0x01, 0xaa, 0xe0, 0xe5, 0x04, 0x54, 0x91, 0xef, 0x03, 0x72, 0x7c, 0xc6, 0x31, - 0x65, 0xa1, 0x4d, 0x42, 0x4e, 0xbb, 0x58, 0x50, 0xac, 0xac, 0x69, 0xeb, 0xba, 0x55, 0x1c, 0x23, - 0xf7, 0xc7, 0xc0, 0x96, 0x21, 0x5a, 0xa0, 0xfa, 0x97, 0x0e, 0xb7, 0x84, 0xec, 0x8f, 0x48, 0x2f, - 0xa4, 0xa1, 0x28, 0x43, 0x36, 0xc0, 0xff, 0x4d, 0x7b, 0x7d, 0xbe, 0xf6, 0xfa, 0x5c, 0xed, 0xf5, - 0x79, 0xda, 0xeb, 0xf3, 0xb4, 0xd7, 0xe7, 0x69, 0xaf, 0x9f, 0xa3, 0xbd, 0x7e, 0xbe, 0xf6, 0xfa, - 0x39, 0xda, 0xeb, 0xf3, 0xb4, 0xd7, 0xaf, 0xae, 0x7d, 0xf2, 0x04, 0x54, 0x5f, 0x69, 0x50, 0xb4, - 0x30, 0x73, 0x49, 0x3d, 0x08, 0x3c, 0x4a, 0x3a, 0x42, 0x7d, 0x82, 0xde, 0x03, 0xd4, 0xc3, 0xdf, - 0x72, 0x1b, 0x2b, 0xa3, 0x4d, 0x59, 0x87, 0x44, 0x52, 0x7e, 0xc3, 0x2a, 0x08, 0x24, 0xf6, 0x6e, - 0x0a, 0x3b, 0x32, 0xe1, 0x96, 0x47, 0x70, 0x48, 0x5e, 0x73, 0x4f, 0x49, 0xf7, 0xa2, 0x84, 0x66, - 0xfc, 0xbf, 0x81, 0x5c, 0x4f, 0xa4, 0xb4, 0x43, 0xd1, 0x6a, 0xb2, 0x1f, 0x72, 0x1b, 0x9f, 0x98, - 0xe7, 0x0e, 0x18, 0xf3, 0x1f, 0x1a, 0x35, 0x7e, 0xe1, 0x40, 0x12, 0x4a, 0xcb, 0xd4, 0xe1, 0xbe, - 0x83, 0x82, 0x08, 0x79, 0xda, 0xa3, 0x9c, 0x3c, 0xc1, 0x5e, 0x9f, 0xec, 0x05, 0xe3, 0xa9, 0xa0, - 0x4d, 0xa6, 0xc2, 0xcc, 0x8b, 0x9f, 0xba, 0xd2, 0x8b, 0x7f, 0x1b, 0xd2, 0x47, 0x82, 0x3f, 0x1e, - 0x36, 0x6a, 0x53, 0xfd, 0x31, 0x05, 0xc5, 0x24, 0x7f, 0x53, 0xea, 0xbc, 0x17, 0xa0, 0xaf, 0x61, - 0x91, 0x47, 0xcc, 0x4e, 0xa6, 0xdd, 0xce, 0xf5, 0xa6, 0x5d, 0xba, 0x15, 0xb1, 0xe6, 0x8e, 0x95, - 0xe6, 0x11, 0x6b, 0x76, 0xd0, 0x1b, 0xb0, 0x24, 0xc8, 0xc5, 0x09, 0x53, 0xb2, 0x14, 0x91, 0xeb, - 0xf3, 0xd7, 0x0f, 0xa9, 0x5f, 0xe9, 0x90, 0x7b, 0x50, 0x14, 0xdc, 0xb3, 0x53, 0xc8, 0xb8, 0x38, - 0xd5, 0x2a, 0x8f, 0xd8, 0xee, 0xd4, 0x20, 0xaa, 0xfe, 0xac, 0x01, 0x12, 0xf7, 0xa3, 0xde, 0x92, - 0x9b, 0xb9, 0xa0, 0xeb, 0x8b, 0x5d, 0xfd, 0x35, 0x2e, 0x7b, 0xdb, 0xef, 0x76, 0x29, 0xbf, 0x99, - 0xb2, 0xe3, 0xae, 0x4d, 0xfd, 0x4b, 0xd7, 0xea, 0xd7, 0xeb, 0x5a, 0x63, 0xba, 0x6b, 0x03, 0xd5, - 0xb4, 0xf5, 0xb6, 0xdf, 0xbb, 0x99, 0xc3, 0x55, 0xbb, 0xea, 0x3f, 0x0f, 0x99, 0xb1, 0x15, 0xb1, - 0xff, 0x3a, 0xdd, 0x4f, 0x06, 0x2c, 0x8b, 0x7c, 0x5f, 0xf8, 0x2e, 0x75, 0xb0, 0xb7, 0x17, 0xa0, - 0x16, 0xe4, 0x9e, 0x8b, 0x6f, 0xd4, 0x56, 0xd7, 0xa1, 0xc9, 0xdb, 0xdc, 0xbc, 0xe0, 0x83, 0x34, - 0xfd, 0xba, 0x58, 0xf0, 0x3c, 0xd9, 0xa1, 0xa7, 0x90, 0x57, 0xac, 0xea, 0x89, 0x8f, 0xbb, 0xed, - 0xa3, 0xcb, 0xd0, 0x8e, 0xef, 0xdf, 0x52, 0xf5, 0xa9, 0x2d, 0xfa, 0x0a, 0x96, 0xe3, 0xb1, 0x1c, - 0x33, 0x2b, 0xf9, 0x3f, 0xbe, 0x20, 0xf3, 0xec, 0xe7, 0x66, 0xe5, 0xfb, 0x53, 0x7b, 0xc1, 0xed, - 0xc8, 0xbe, 0x1e, 0x73, 0x1b, 0x97, 0xe2, 0x9e, 0xfd, 0x26, 0xac, 0xbc, 0x33, 0xb5, 0x17, 0x17, - 0x82, 0x85, 0xc6, 0x63, 0xea, 0xf4, 0xa5, 0x2e, 0x64, 0xa6, 0x21, 0xad, 0x1c, 0x9e, 0x6c, 0xd1, - 0x43, 0xc8, 0x2a, 0x62, 0x1e, 0x31, 0x39, 0xf1, 0x73, 0x1b, 0x1f, 0x5e, 0x86, 0x55, 0x36, 0x9d, - 0x95, 0xc1, 0xf1, 0x7a, 0xcb, 0x38, 0x39, 0xae, 0x68, 0x8d, 0xb5, 0x93, 0xdf, 0xcb, 0x0b, 0x27, - 0xa3, 0xb2, 0xf6, 0x62, 0x54, 0xd6, 0x5e, 0x8e, 0xca, 0xda, 0x6f, 0xa3, 0xb2, 0xf6, 0xc3, 0x69, - 0x79, 0xe1, 0xc5, 0x69, 0x79, 0xe1, 0xe5, 0x69, 0x79, 0xa1, 0xbd, 0x28, 0x7f, 0xc7, 0x6c, 0xfe, - 0x1d, 0x00, 0x00, 0xff, 0xff, 0x45, 0x60, 0x94, 0x6c, 0x41, 0x0d, 0x00, 0x00, + proto.RegisterFile("storage/engine/enginepb/mvcc3.proto", fileDescriptor_mvcc3_351ae7cbe7e93b17) +} + +var fileDescriptor_mvcc3_351ae7cbe7e93b17 = []byte{ + // 1188 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0xcf, 0x6f, 0xe3, 0xc4, + 0x17, 0xaf, 0x63, 0xa7, 0x4d, 0x26, 0x69, 0x9b, 0xcc, 0xae, 0xbe, 0xdf, 0x68, 0x61, 0x93, 0x6c, + 0x90, 0x50, 0xc5, 0x0f, 0x07, 0x5a, 0xe0, 0x50, 0x89, 0x43, 0xd2, 0xae, 0x20, 0x0b, 0x6d, 0x17, + 0x37, 0xdd, 0x95, 0x40, 0x60, 0x4d, 0x9c, 0xc1, 0x1d, 0xd5, 0x1e, 0xbb, 0xf6, 0xa4, 0xeb, 0x1c, + 0xe0, 0x6f, 0xe0, 0x1f, 0x40, 0xea, 0x85, 0xff, 0x80, 0x03, 0x7f, 0x42, 0x8f, 0x7b, 0x5c, 0xad, + 0x50, 0x04, 0xe9, 0x05, 0x71, 0xe6, 0x54, 0x09, 0x09, 0xcd, 0x8c, 0xe3, 0x24, 0xbb, 0x90, 0xfe, + 0x12, 0x15, 0xa7, 0xce, 0xbc, 0xcf, 0x7b, 0x9f, 0xf7, 0x66, 0x3e, 0xcf, 0xf3, 0x1a, 0xf0, 0x5a, + 0xc8, 0xbc, 0x00, 0xd9, 0xb8, 0x8e, 0xa9, 0x4d, 0xe8, 0xe8, 0x8f, 0xdf, 0xa9, 0xbb, 0x47, 0x96, + 0xb5, 0xa6, 0xfb, 0x81, 0xc7, 0x3c, 0x78, 0xcf, 0xf2, 0xac, 0x83, 0xc0, 0x43, 0xd6, 0xbe, 0x1e, + 0xbb, 0xeb, 0xd2, 0x4f, 0x1f, 0xb9, 0xdf, 0x29, 0xf5, 0x18, 0x71, 0xea, 0xfb, 0x8e, 0x55, 0x67, + 0xc4, 0xc5, 0x21, 0x43, 0xae, 0x2f, 0x83, 0xef, 0xdc, 0xb6, 0x3d, 0xdb, 0x13, 0xcb, 0x3a, 0x5f, + 0x49, 0x6b, 0xed, 0x77, 0x0d, 0x2c, 0xb4, 0x23, 0xba, 0x85, 0x19, 0x82, 0x9f, 0x81, 0x14, 0xe9, + 0x96, 0x94, 0xaa, 0xb2, 0x92, 0x6f, 0x36, 0x4e, 0x06, 0x95, 0xb9, 0xe7, 0x83, 0xca, 0x9a, 0x4d, + 0xd8, 0x7e, 0xaf, 0xa3, 0x5b, 0x9e, 0x5b, 0x4f, 0xb2, 0x77, 0x3b, 0xe3, 0x75, 0xdd, 0x3f, 0xb0, + 0xeb, 0x22, 0x69, 0xaf, 0x47, 0xba, 0xfa, 0xde, 0x5e, 0x6b, 0x73, 0x38, 0xa8, 0xa4, 0x5a, 0x9b, + 0x46, 0x8a, 0x74, 0x61, 0x01, 0xa8, 0x07, 0xb8, 0x5f, 0x52, 0x39, 0xa7, 0xc1, 0x97, 0xb0, 0x06, + 0xd2, 0xd8, 0xf7, 0xac, 0xfd, 0x92, 0x56, 0x55, 0x56, 0xd2, 0xcd, 0xfc, 0xd9, 0xa0, 0x92, 0x69, + 0x47, 0xf4, 0x3e, 0xb7, 0x19, 0x12, 0x82, 0x0d, 0x90, 0x4d, 0xaa, 0x2f, 0xa5, 0xab, 0xca, 0x4a, + 0x6e, 0xf5, 0xae, 0x3e, 0x3e, 0x3b, 0xcf, 0xa6, 0xef, 0x3b, 0x96, 0xde, 0x1e, 0x39, 0x35, 0x35, + 0x5e, 0xae, 0x31, 0x8e, 0x82, 0x6f, 0x82, 0x8c, 0x1f, 0x10, 0x2f, 0x20, 0xac, 0x5f, 0x9a, 0x17, + 0x99, 0x96, 0xcf, 0x06, 0x95, 0x5c, 0x3b, 0xa2, 0x0f, 0x63, 0xb3, 0x91, 0x38, 0xc0, 0xd7, 0x41, + 0x26, 0xc4, 0x87, 0x3d, 0x4c, 0x2d, 0x5c, 0x5a, 0x10, 0xce, 0xe0, 0x6c, 0x50, 0x99, 0x6f, 0x47, + 0x74, 0x17, 0x1f, 0x1a, 0x09, 0x06, 0x3f, 0x06, 0x8b, 0x2e, 0xa1, 0xe6, 0xb8, 0xb6, 0xec, 0xc5, + 0x6b, 0xcb, 0xbb, 0x84, 0x26, 0x36, 0xf8, 0x2d, 0x58, 0x26, 0x36, 0xf5, 0x02, 0xdc, 0x35, 0x43, + 0x7c, 0x48, 0x7b, 0x6e, 0x58, 0x02, 0x55, 0x75, 0x25, 0xb7, 0xfa, 0xa1, 0x7e, 0xae, 0xc6, 0x7a, + 0xac, 0x97, 0xde, 0x92, 0x0c, 0xbb, 0xf8, 0x70, 0xbb, 0xe7, 0x1a, 0x88, 0xda, 0xb8, 0xf9, 0x3f, + 0x9e, 0x6b, 0x38, 0xa8, 0x2c, 0x4d, 0x61, 0xa1, 0xb1, 0x44, 0x92, 0x3d, 0x4f, 0x76, 0xe7, 0x2b, + 0x00, 0x5f, 0x8e, 0x86, 0x55, 0x90, 0x0e, 0x19, 0x0a, 0x98, 0xe8, 0x81, 0xe9, 0x4b, 0x90, 0x00, + 0x7c, 0x15, 0xa8, 0x98, 0x76, 0x4b, 0xa9, 0x97, 0x70, 0x6e, 0x5e, 0xcf, 0xfc, 0x74, 0x5c, 0x51, + 0x7e, 0x3b, 0xae, 0x28, 0xe3, 0xd5, 0x03, 0x2d, 0x93, 0x2a, 0xa8, 0x0f, 0xb4, 0x4c, 0xa6, 0x90, + 0xad, 0xfd, 0xa1, 0x82, 0xa5, 0xad, 0x47, 0x1b, 0x1b, 0xbb, 0x0c, 0xb1, 0x70, 0x13, 0x3b, 0x0c, + 0xc1, 0x37, 0x40, 0xd1, 0x41, 0x21, 0x33, 0x7b, 0x7e, 0x17, 0x31, 0x6c, 0x52, 0x44, 0xbd, 0x50, + 0xa4, 0x2f, 0x18, 0xcb, 0x1c, 0xd8, 0x13, 0xf6, 0x6d, 0x6e, 0x86, 0x77, 0x01, 0x20, 0x94, 0x61, + 0xca, 0x4c, 0x64, 0x63, 0x51, 0x43, 0xc1, 0xc8, 0x4a, 0x4b, 0xc3, 0xc6, 0xf0, 0x1d, 0x90, 0xb7, + 0x2d, 0xb3, 0xd3, 0x67, 0x38, 0x14, 0x0e, 0xbc, 0xe9, 0x0a, 0xcd, 0xa5, 0xe1, 0xa0, 0x02, 0x3e, + 0xda, 0x68, 0x72, 0x73, 0xc3, 0xc6, 0x06, 0xb0, 0xad, 0xd1, 0x9a, 0x13, 0x3a, 0xe4, 0x08, 0xcb, + 0x18, 0xd1, 0x90, 0xd0, 0xc8, 0x72, 0x8b, 0xf0, 0x48, 0x60, 0xcb, 0xeb, 0x51, 0x26, 0xfa, 0x30, + 0x86, 0x37, 0xb8, 0x01, 0xbe, 0x02, 0xb2, 0x07, 0xb8, 0x1f, 0x07, 0xcf, 0x0b, 0x34, 0x73, 0x80, + 0xfb, 0x32, 0x36, 0x06, 0x65, 0xe8, 0x42, 0x02, 0x26, 0x91, 0x47, 0xc8, 0x89, 0x23, 0x33, 0x12, + 0x3c, 0x42, 0x4e, 0x12, 0xc9, 0x41, 0x19, 0x99, 0x4d, 0x40, 0x19, 0x79, 0x0f, 0xe4, 0xe3, 0x2b, + 0x90, 0xc1, 0x40, 0xe0, 0x39, 0x69, 0x93, 0xf1, 0x63, 0x17, 0x49, 0x91, 0x9b, 0x74, 0x49, 0xf2, + 0x87, 0xfd, 0x30, 0xa6, 0xc8, 0xcb, 0x14, 0x61, 0x3f, 0x4c, 0xf2, 0x73, 0x50, 0x06, 0x2f, 0x26, + 0xa0, 0x8c, 0x7c, 0x1b, 0x40, 0xcb, 0xa3, 0x0c, 0x11, 0x1a, 0x9a, 0x38, 0x64, 0xc4, 0x45, 0x9c, + 0x62, 0xa9, 0xaa, 0xac, 0xa8, 0x46, 0x71, 0x84, 0xdc, 0x1f, 0x01, 0xeb, 0x1a, 0x6f, 0x81, 0xda, + 0x9f, 0x2a, 0xb8, 0xc5, 0x65, 0x7f, 0x88, 0x83, 0x90, 0x84, 0xbc, 0x0c, 0xd1, 0x00, 0xff, 0x35, + 0xed, 0xd5, 0xd9, 0xda, 0xab, 0x33, 0xb5, 0x57, 0x67, 0x69, 0xaf, 0xce, 0xd2, 0x5e, 0x9d, 0xa5, + 0xbd, 0x7a, 0x8e, 0xf6, 0xea, 0xf9, 0xda, 0xab, 0xe7, 0x68, 0xaf, 0xce, 0xd2, 0x5e, 0xbd, 0xba, + 0xf6, 0xc9, 0x13, 0x50, 0x7b, 0xae, 0x80, 0xa2, 0x78, 0x60, 0x1a, 0xbe, 0xef, 0x10, 0xdc, 0xe5, + 0xea, 0x63, 0xf8, 0x16, 0x80, 0x01, 0xfa, 0x9a, 0x99, 0x48, 0x1a, 0x4d, 0x42, 0xbb, 0x38, 0x12, + 0xf2, 0x6b, 0x46, 0x81, 0x23, 0xb1, 0x77, 0x8b, 0xdb, 0xa1, 0x0e, 0x6e, 0x39, 0x18, 0x85, 0xf8, + 0x05, 0xf7, 0x94, 0x70, 0x2f, 0x0a, 0x68, 0xca, 0xff, 0x4b, 0x90, 0x0b, 0x78, 0x4a, 0x33, 0xe4, + 0xad, 0x26, 0xfa, 0x21, 0xb7, 0xfa, 0xc1, 0x05, 0x1e, 0xd7, 0xbf, 0x69, 0xd4, 0xf8, 0x05, 0x07, + 0x82, 0x50, 0x58, 0x26, 0x0e, 0xf7, 0x0d, 0x28, 0xf0, 0x90, 0xc7, 0x01, 0x61, 0xf8, 0x11, 0x72, + 0x7a, 0x78, 0xc7, 0x1f, 0x4d, 0x3d, 0x65, 0x3c, 0xf5, 0xa6, 0x26, 0x5a, 0xea, 0x4a, 0x13, 0xed, + 0x36, 0x48, 0x1f, 0x71, 0xfe, 0x78, 0x98, 0xca, 0x4d, 0xed, 0xfb, 0x14, 0x28, 0x26, 0xf9, 0x5b, + 0x42, 0xe7, 0x1d, 0x1f, 0x7e, 0x01, 0xe6, 0x59, 0x44, 0xcd, 0x64, 0x9a, 0x6f, 0x5e, 0x6f, 0x9a, + 0xa7, 0xdb, 0x11, 0x6d, 0x6d, 0x1a, 0x69, 0x16, 0xd1, 0x56, 0x17, 0xfe, 0x1f, 0x2c, 0x70, 0x72, + 0x7e, 0xc2, 0x94, 0x28, 0x85, 0xe7, 0xfa, 0xe4, 0xc5, 0x43, 0xaa, 0x57, 0x3a, 0xe4, 0x0e, 0x28, + 0x72, 0xee, 0xe9, 0x29, 0xab, 0x5d, 0x9c, 0x6a, 0x99, 0x45, 0x74, 0x6b, 0x62, 0xd0, 0xd6, 0x7e, + 0x54, 0x00, 0xe4, 0xf7, 0x23, 0xdf, 0x92, 0x9b, 0xb9, 0xa0, 0xeb, 0x8b, 0x5d, 0xfb, 0x39, 0x2e, + 0x7b, 0xc3, 0x73, 0x5d, 0xc2, 0x6e, 0xa6, 0xec, 0xb8, 0x6b, 0x53, 0xff, 0xd0, 0xb5, 0xea, 0xf5, + 0xba, 0x56, 0x9b, 0xec, 0x5a, 0x5f, 0x36, 0x6d, 0xa3, 0xe3, 0x05, 0x37, 0x73, 0xb8, 0x9a, 0x2b, + 0xff, 0xf3, 0x10, 0x19, 0xdb, 0x11, 0xfd, 0xb7, 0xd3, 0xfd, 0xa0, 0x81, 0x45, 0x9e, 0xef, 0x53, + 0xcf, 0x26, 0x16, 0x72, 0x76, 0x7c, 0xd8, 0x06, 0xb9, 0x27, 0xfc, 0x1b, 0x35, 0xe5, 0x75, 0x28, + 0xe2, 0x36, 0xd7, 0x2e, 0xf8, 0x20, 0x4d, 0xbe, 0x2e, 0x06, 0x78, 0x92, 0xec, 0xe0, 0x63, 0x90, + 0x97, 0xac, 0xf2, 0x89, 0x8f, 0xbb, 0xed, 0xbd, 0xcb, 0xd0, 0x8e, 0xee, 0xdf, 0x90, 0xf5, 0xc9, + 0x2d, 0xfc, 0x1c, 0x2c, 0xc6, 0x63, 0x39, 0x66, 0x96, 0xf2, 0xbf, 0x7f, 0x41, 0xe6, 0xe9, 0xcf, + 0xcd, 0xc8, 0xf7, 0x26, 0xf6, 0x9c, 0xdb, 0x12, 0x7d, 0x3d, 0xe2, 0xd6, 0x2e, 0xc5, 0x3d, 0xfd, + 0x4d, 0x18, 0x79, 0x6b, 0x62, 0xcf, 0x2f, 0x04, 0x71, 0x8d, 0x47, 0xd4, 0xe9, 0x4b, 0x5d, 0xc8, + 0x54, 0x43, 0x1a, 0x39, 0x34, 0xde, 0xc2, 0x6d, 0x90, 0x95, 0xc4, 0x2c, 0xa2, 0x62, 0xe2, 0xe7, + 0x56, 0xdf, 0xbd, 0x0c, 0xab, 0x68, 0x3a, 0x23, 0x83, 0xe2, 0xf5, 0xba, 0x76, 0x72, 0x5c, 0x51, + 0x9a, 0xd5, 0x93, 0x5f, 0xcb, 0x73, 0x27, 0xc3, 0xb2, 0xf2, 0x74, 0x58, 0x56, 0x9e, 0x0d, 0xcb, + 0xca, 0x2f, 0xc3, 0xb2, 0xf2, 0xdd, 0x69, 0x79, 0xee, 0xe9, 0x69, 0x79, 0xee, 0xd9, 0x69, 0x79, + 0xae, 0x33, 0x2f, 0x7e, 0xa7, 0xad, 0xfd, 0x15, 0x00, 0x00, 0xff, 0xff, 0x87, 0x08, 0xb7, 0xe8, + 0x21, 0x0e, 0x00, 0x00, } diff --git a/pkg/storage/engine/enginepb/mvcc3.proto b/pkg/storage/engine/enginepb/mvcc3.proto index 4bb924e4e537..fd54e0643997 100644 --- a/pkg/storage/engine/enginepb/mvcc3.proto +++ b/pkg/storage/engine/enginepb/mvcc3.proto @@ -113,6 +113,20 @@ message TxnMeta { int32 sequence = 7 [(gogoproto.casttype) = "TxnSeq"]; reserved 8; + + message IgnoredSeqNumRange { + option (gogoproto.equal) = true; + option (gogoproto.populate) = true; + int32 start = 1 [(gogoproto.casttype) = "TxnSeq"]; + int32 end = 2 [(gogoproto.casttype) = "TxnSeq"]; + } + // A list of ignored seqnum ranges. + // + // The user code must guarantee this list to be non-overlapping, + // non-continuous (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 IgnoredSeqNumRange ignored_seqnums = 10 [(gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums"]; } // MVCCStatsDelta is convertible to MVCCStats, but uses signed variable width diff --git a/pkg/storage/engine/mvcc.go b/pkg/storage/engine/mvcc.go index b4c89f068813..04c3769ae40a 100644 --- a/pkg/storage/engine/mvcc.go +++ b/pkg/storage/engine/mvcc.go @@ -2597,7 +2597,24 @@ 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. + // + // 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 { + // 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 read 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). + collapsedIntent := false + latestKey := MVCCKey{Key: intent.Key, Timestamp: hlc.Timestamp(meta.Timestamp)} + if len(intent.Txn.IgnoredSeqNums) > 0 { + collapsedIntent, err = mvccRewriteIntentHistory(ctx, engine, &intent.Txn, meta, latestKey) + if err != nil { + return false, err + } + } + buf.newMeta = *meta // Set the timestamp for upcoming write (or at least the stats update). buf.newMeta.Timestamp = hlc.LegacyTimestamp(intent.Txn.Timestamp) @@ -2620,8 +2637,10 @@ func mvccResolveWriteIntent( } // 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 + // (no new value any more). var prevValSize int64 - if buf.newMeta.Timestamp != meta.Timestamp { + if buf.newMeta.Timestamp != meta.Timestamp && !collapsedIntent { // If there is a value under the intent as it moves timestamps, then // that value may need an adjustment of its GCBytesAge. This is // because it became non-live at orig.Timestamp originally, and now @@ -2630,7 +2649,6 @@ func mvccResolveWriteIntent( // // Look for the first real versioned key, i.e. the key just below // the (old) meta's timestamp. - latestKey := MVCCKey{Key: intent.Key, Timestamp: hlc.Timestamp(meta.Timestamp)} _, unsafeNextValue, haveNextVersion, err := unsafeNextVersion(iter, latestKey) if err != nil { return false, err @@ -2739,6 +2757,70 @@ func mvccResolveWriteIntent( return true, nil } +// mvccRewriteIntentHistory rolls back the write from the intent, ignoring +// all values from the history that have an ignored seqnum. +func mvccRewriteIntentHistory( + ctx context.Context, + engine ReadWriter, + txn *enginepb.TxnMeta, + meta *enginepb.MVCCMetadata, + latestKey MVCCKey, +) (clear bool, err error) { + if !seqNumIsIgnored(txn, meta.Txn.Sequence) { + // The latest write was not ignored. Nothing to do here. We'll + // proceed with the intent as usual. + return false, 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 !seqNumIsIgnored(txn, e.Sequence) { + 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, err + } + + // Otherwise, we place back the write at that history entry + // back into the intent. + restoredVal := meta.IntentHistory[i].Value + 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, err +} + +func seqNumIsIgnored(txn *enginepb.TxnMeta, sequence enginepb.TxnSeq) bool { + for i := len(txn.IgnoredSeqNums) - 1; i >= 0; i-- { + r := txn.IgnoredSeqNums[i] + if sequence > r.End { + // All ignored ranges lower than the current seqnum. Write is not ignored. + return false + } + if txn.Sequence < r.Start { + // Current ignored range higher than seqnum. Try next. + continue + } + // Splash: seqnum is inside ignored range. + return true + } + // Not in any ignored range: not ignored. + return false +} + // IterAndBuf used to pass iterators and buffers between MVCC* calls, allowing // reuse without the callers needing to know the particulars. type IterAndBuf struct { diff --git a/pkg/storage/engine/rocksdb.go b/pkg/storage/engine/rocksdb.go index 90ef16879726..c73f8a53ec89 100644 --- a/pkg/storage/engine/rocksdb.go +++ b/pkg/storage/engine/rocksdb.go @@ -2539,6 +2539,16 @@ func goToCSlice(b []byte) C.DBSlice { } } +func goToCIgnoredSeqNums(b []enginepb.TxnMeta_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), @@ -2625,6 +2635,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 }