diff --git a/c-deps/libroach/protos/roachpb/metadata.pb.cc b/c-deps/libroach/protos/roachpb/metadata.pb.cc index 71a60b9f48cd..0dd2b4ff83ce 100644 --- a/c-deps/libroach/protos/roachpb/metadata.pb.cc +++ b/c-deps/libroach/protos/roachpb/metadata.pb.cc @@ -1296,7 +1296,6 @@ const int RangeDescriptor::kEndKeyFieldNumber; const int RangeDescriptor::kInternalReplicasFieldNumber; const int RangeDescriptor::kNextReplicaIdFieldNumber; const int RangeDescriptor::kGenerationFieldNumber; -const int RangeDescriptor::kGenerationComparableFieldNumber; const int RangeDescriptor::kStickyBitFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 @@ -1327,8 +1326,8 @@ RangeDescriptor::RangeDescriptor(const RangeDescriptor& from) sticky_bit_ = NULL; } ::memcpy(&range_id_, &from.range_id_, - static_cast(reinterpret_cast(&generation_comparable_) - - reinterpret_cast(&range_id_)) + sizeof(generation_comparable_)); + static_cast(reinterpret_cast(&next_replica_id_) - + reinterpret_cast(&range_id_)) + sizeof(next_replica_id_)); // @@protoc_insertion_point(copy_constructor:cockroach.roachpb.RangeDescriptor) } @@ -1336,8 +1335,8 @@ void RangeDescriptor::SharedCtor() { start_key_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); end_key_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); ::memset(&sticky_bit_, 0, static_cast( - reinterpret_cast(&generation_comparable_) - - reinterpret_cast(&sticky_bit_)) + sizeof(generation_comparable_)); + reinterpret_cast(&next_replica_id_) - + reinterpret_cast(&sticky_bit_)) + sizeof(next_replica_id_)); } RangeDescriptor::~RangeDescriptor() { @@ -1380,10 +1379,10 @@ void RangeDescriptor::Clear() { sticky_bit_->Clear(); } } - if (cached_has_bits & 120u) { + if (cached_has_bits & 56u) { ::memset(&range_id_, 0, static_cast( - reinterpret_cast(&generation_comparable_) - - reinterpret_cast(&range_id_)) + sizeof(generation_comparable_)); + reinterpret_cast(&next_replica_id_) - + reinterpret_cast(&range_id_)) + sizeof(next_replica_id_)); } _has_bits_.Clear(); _internal_metadata_.Clear(); @@ -1489,20 +1488,6 @@ bool RangeDescriptor::MergePartialFromCodedStream( break; } - // optional bool generation_comparable = 8; - case 8: { - if (static_cast< ::google::protobuf::uint8>(tag) == - static_cast< ::google::protobuf::uint8>(64u /* 64 & 0xFF */)) { - set_has_generation_comparable(); - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( - input, &generation_comparable_))); - } else { - goto handle_unusual; - } - break; - } - default: { handle_unusual: if (tag == 0) { @@ -1566,11 +1551,6 @@ void RangeDescriptor::SerializeWithCachedSizes( 7, this->_internal_sticky_bit(), output); } - // optional bool generation_comparable = 8; - if (cached_has_bits & 0x00000040u) { - ::google::protobuf::internal::WireFormatLite::WriteBool(8, this->generation_comparable(), output); - } - output->WriteRaw(_internal_metadata_.unknown_fields().data(), static_cast(_internal_metadata_.unknown_fields().size())); // @@protoc_insertion_point(serialize_end:cockroach.roachpb.RangeDescriptor) @@ -1592,7 +1572,7 @@ size_t RangeDescriptor::ByteSizeLong() const { } } - if (_has_bits_[0 / 32] & 127u) { + if (_has_bits_[0 / 32] & 63u) { if (has_start_key()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::BytesSize( @@ -1630,11 +1610,6 @@ size_t RangeDescriptor::ByteSizeLong() const { this->next_replica_id()); } - // optional bool generation_comparable = 8; - if (has_generation_comparable()) { - total_size += 1 + 1; - } - } int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); SetCachedSize(cached_size); @@ -1655,7 +1630,7 @@ void RangeDescriptor::MergeFrom(const RangeDescriptor& from) { internal_replicas_.MergeFrom(from.internal_replicas_); cached_has_bits = from._has_bits_[0]; - if (cached_has_bits & 127u) { + if (cached_has_bits & 63u) { if (cached_has_bits & 0x00000001u) { set_has_start_key(); start_key_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.start_key_); @@ -1676,9 +1651,6 @@ void RangeDescriptor::MergeFrom(const RangeDescriptor& from) { if (cached_has_bits & 0x00000020u) { next_replica_id_ = from.next_replica_id_; } - if (cached_has_bits & 0x00000040u) { - generation_comparable_ = from.generation_comparable_; - } _has_bits_[0] |= cached_has_bits; } } @@ -1709,7 +1681,6 @@ void RangeDescriptor::InternalSwap(RangeDescriptor* other) { swap(range_id_, other->range_id_); swap(generation_, other->generation_); swap(next_replica_id_, other->next_replica_id_); - swap(generation_comparable_, other->generation_comparable_); swap(_has_bits_[0], other->_has_bits_[0]); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/c-deps/libroach/protos/roachpb/metadata.pb.h b/c-deps/libroach/protos/roachpb/metadata.pb.h index e9cfd8aa3b6f..0304ab3e3d1d 100644 --- a/c-deps/libroach/protos/roachpb/metadata.pb.h +++ b/c-deps/libroach/protos/roachpb/metadata.pb.h @@ -796,13 +796,6 @@ class RangeDescriptor : public ::google::protobuf::MessageLite /* @@protoc_inser ::google::protobuf::int32 next_replica_id() const; void set_next_replica_id(::google::protobuf::int32 value); - // optional bool generation_comparable = 8; - bool has_generation_comparable() const; - void clear_generation_comparable(); - static const int kGenerationComparableFieldNumber = 8; - bool generation_comparable() const; - void set_generation_comparable(bool value); - // @@protoc_insertion_point(class_scope:cockroach.roachpb.RangeDescriptor) private: void set_has_range_id(); @@ -815,8 +808,6 @@ class RangeDescriptor : public ::google::protobuf::MessageLite /* @@protoc_inser void clear_has_next_replica_id(); void set_has_generation(); void clear_has_generation(); - void set_has_generation_comparable(); - void clear_has_generation_comparable(); void set_has_sticky_bit(); void clear_has_sticky_bit(); @@ -830,7 +821,6 @@ class RangeDescriptor : public ::google::protobuf::MessageLite /* @@protoc_inser ::google::protobuf::int64 range_id_; ::google::protobuf::int64 generation_; ::google::protobuf::int32 next_replica_id_; - bool generation_comparable_; friend struct ::protobuf_roachpb_2fmetadata_2eproto::TableStruct; }; // ------------------------------------------------------------------- @@ -2764,30 +2754,6 @@ inline void RangeDescriptor::set_generation(::google::protobuf::int64 value) { // @@protoc_insertion_point(field_set:cockroach.roachpb.RangeDescriptor.generation) } -// optional bool generation_comparable = 8; -inline bool RangeDescriptor::has_generation_comparable() const { - return (_has_bits_[0] & 0x00000040u) != 0; -} -inline void RangeDescriptor::set_has_generation_comparable() { - _has_bits_[0] |= 0x00000040u; -} -inline void RangeDescriptor::clear_has_generation_comparable() { - _has_bits_[0] &= ~0x00000040u; -} -inline void RangeDescriptor::clear_generation_comparable() { - generation_comparable_ = false; - clear_has_generation_comparable(); -} -inline bool RangeDescriptor::generation_comparable() const { - // @@protoc_insertion_point(field_get:cockroach.roachpb.RangeDescriptor.generation_comparable) - return generation_comparable_; -} -inline void RangeDescriptor::set_generation_comparable(bool value) { - set_has_generation_comparable(); - generation_comparable_ = value; - // @@protoc_insertion_point(field_set:cockroach.roachpb.RangeDescriptor.generation_comparable) -} - // optional .cockroach.util.hlc.Timestamp sticky_bit = 7; inline bool RangeDescriptor::has_sticky_bit() const { return (_has_bits_[0] & 0x00000004u) != 0; diff --git a/pkg/cli/cli_debug_test.go b/pkg/cli/cli_debug_test.go index aaf03bcd928b..85ef42c04f71 100644 --- a/pkg/cli/cli_debug_test.go +++ b/pkg/cli/cli_debug_test.go @@ -28,7 +28,7 @@ func Example_debug_decode_key_value() { // if the result below looks like garbage, then it likely is: // // 0.987654321,0 /Local/Range/Table/53/1/-4560243296450227838/RangeDescriptor (0x016b12bd8980c0b6c2e211ba518200017264736300000000003ade68b109): [/Table/53/1/-4560243296450227838, /Table/53/1/-4559358311118345834) - // Raw:r1179:/Table/53/1/-45{60243296450227838-59358311118345834} [(n1,s1):1, (n4,s4):2, (n2,s2):4, next=5, gen=4?] + // Raw:r1179:/Table/53/1/-45{60243296450227838-59358311118345834} [(n1,s1):1, (n4,s4):2, (n2,s2):4, next=5, gen=4] } func TestDebugKeysHex(t *testing.T) { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index 0498afd16334..3e16002ab10a 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -3416,6 +3416,7 @@ func TestEvictionTokenCoalesce(t *testing.T) { stopper := stop.NewStopper() defer stopper.Stop(context.TODO()) + initGen := int64(1) testUserRangeDescriptor := roachpb.RangeDescriptor{ RangeID: 2, StartKey: roachpb.RKey("a"), @@ -3426,6 +3427,7 @@ func TestEvictionTokenCoalesce(t *testing.T) { StoreID: 1, }, }, + Generation: initGen, } clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) @@ -3491,7 +3493,7 @@ func TestEvictionTokenCoalesce(t *testing.T) { if err := testutils.SucceedsSoonError(func() error { // Since the previously fetched RangeDescriptor was ["a", "d"), the request keys // would be coalesced to "a". - numCalls := ds.rangeCache.lookupRequests.NumCalls(string(roachpb.RKey("a")) + ":false") + numCalls := ds.rangeCache.lookupRequests.NumCalls(fmt.Sprintf("a:false:%d", initGen)) if numCalls != 2 { return errors.Errorf("expected %d in-flight requests, got %d", 2, numCalls) } @@ -3543,7 +3545,7 @@ func TestDistSenderSlowLogMessage(t *testing.T) { desc := &roachpb.RangeDescriptor{RangeID: 9, StartKey: roachpb.RKey("x")} { exp := `have been waiting 8.16s (120 attempts) for RPC to` + - ` r9:{-} [, next=0, gen=0?]: boom` + ` r9:{-} [, next=0, gen=0]: boom` act := slowRangeRPCWarningStr( dur, 120, diff --git a/pkg/kv/kvclient/kvcoord/range_cache.go b/pkg/kv/kvclient/kvcoord/range_cache.go index 26558950684e..495c0c37379d 100644 --- a/pkg/kv/kvclient/kvcoord/range_cache.go +++ b/pkg/kv/kvclient/kvcoord/range_cache.go @@ -164,7 +164,7 @@ func makeLookupRequestKey( // split again into [a, b) and [b, c), we don't want to the requests on [a, // b) to be coalesced with the retried requests on [c, e). To distinguish the // two cases, we can use the generation of the previous descriptor. - if prevDesc != nil && prevDesc.GetGenerationComparable() { + if prevDesc != nil { ret.WriteString(":") ret.WriteString(strconv.FormatInt(prevDesc.Generation, 10)) } @@ -540,23 +540,12 @@ func (rdc *RangeDescriptorCache) evictCachedRangeDescriptorLocked( // Note that we're doing a "compare-and-erase": If seenDesc is not nil, we // want to clean the cache only if it equals the cached range descriptor. We - // try to use Generation and GenerationComparable to determine if the range - // descriptors are equal, but if we cannot, we fallback to - // pointer-comparison. If the range descriptors are not equal, then likely - // some other caller already evicted previously, and we can save work by not - // doing it again (which would prompt another expensive lookup). + // use Generation to determine if the range descriptors are equal. If the + // range descriptors are not equal, then likely some other caller already + // evicted previously, and we can save work by not doing it again (which would + // prompt another expensive lookup). if seenDesc != nil { - if seenDesc.GetGenerationComparable() && cachedDesc.GetGenerationComparable() { - if seenDesc.Generation != cachedDesc.Generation { - return nil - } - } else if !seenDesc.GetGenerationComparable() && !cachedDesc.GetGenerationComparable() { - if seenDesc != cachedDesc { - return nil - } - } else { - // One descriptor's generation is comparable, while the other is - // incomparable, so the descriptors are guaranteed to be different. + if seenDesc.Generation != cachedDesc.Generation { return nil } } @@ -683,15 +672,8 @@ func (rdc *RangeDescriptorCache) clearOverlappingCachedRangeDescriptors( // check ["c", "d"). We do, however, want to check ["b", "c"), which is why // the end key is inclusive. if cached.StartKey.Less(desc.EndKey) && !cached.EndKey.Less(desc.EndKey) { - if desc.GetGenerationComparable() && cached.GetGenerationComparable() { - if desc.Generation <= cached.Generation { - // Generations are comparable and a newer descriptor already exists in - // cache. - continueWithInsert = false - } - } else if desc.Equal(*cached) { - // Generations are incomparable so we don't continue with insertion - // only if the descriptor already exists. + if desc.Generation <= cached.Generation { + // A newer descriptor already exists in cache. continueWithInsert = false } if continueWithInsert { @@ -715,16 +697,10 @@ func (rdc *RangeDescriptorCache) clearOverlappingCachedRangeDescriptors( // cache unconditionally. rdc.rangeCache.cache.DoRangeEntry(func(e *cache.Entry) bool { descriptor := e.Value.(*roachpb.RangeDescriptor) - if desc.GetGenerationComparable() && descriptor.GetGenerationComparable() { - // If generations are comparable, then check generations to see if we - // evict. - if desc.Generation <= descriptor.Generation { - continueWithInsert = false - } else { - entriesToEvict = append(entriesToEvict, e) - } + // Check generations to see if we evict. + if desc.Generation <= descriptor.Generation { + continueWithInsert = false } else { - // If generations are not comparable, evict. entriesToEvict = append(entriesToEvict, e) } return false diff --git a/pkg/kv/kvclient/kvcoord/range_cache_test.go b/pkg/kv/kvclient/kvcoord/range_cache_test.go index 50127b69a6e0..a9c1fdc89e89 100644 --- a/pkg/kv/kvclient/kvcoord/range_cache_test.go +++ b/pkg/kv/kvclient/kvcoord/range_cache_test.go @@ -27,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" - "github.com/gogo/protobuf/proto" "github.com/stretchr/testify/require" ) @@ -90,6 +89,7 @@ func (db *testDescriptorDB) getDescriptors( rs = append(rs, desc) // Fake an intent. desc.RangeID++ + desc.Generation = desc.Generation + 1 rs = append(rs, desc) } else if db.disablePrefetch { break @@ -180,16 +180,19 @@ func (db *testDescriptorDB) splitRange(t *testing.T, key roachpb.RKey) { if bytes.Equal(val.EndKey, key) { t.Fatalf("Attempt to split existing range at Endkey: %s", string(key)) } + newGen := val.Generation + 1 db.data.Insert(testDescriptorNode{ &roachpb.RangeDescriptor{ - StartKey: val.StartKey, - EndKey: key, + StartKey: val.StartKey, + EndKey: key, + Generation: newGen, }, }) db.data.Insert(testDescriptorNode{ &roachpb.RangeDescriptor{ - StartKey: key, - EndKey: val.EndKey, + StartKey: key, + EndKey: val.EndKey, + Generation: newGen, }, }) } @@ -206,6 +209,10 @@ func newTestDescriptorDB() *testDescriptorDB { db := &testDescriptorDB{ pauseChan: make(chan struct{}), } + // NOTE: The range descriptors created below are not initialized with a + // generation. The ones created by splitting them will have generations, + // though. Not putting generations in these initial ones is done for diversity + // in the tests. td1 := &roachpb.RangeDescriptor{ StartKey: roachpb.RKeyMin, EndKey: roachpb.RKey(keys.Meta2Prefix), @@ -957,11 +964,12 @@ func TestRangeCacheUseIntents(t *testing.T) { // cached entries are cleared when adding a new entry. func TestRangeCacheClearOverlapping(t *testing.T) { defer leaktest.AfterTest(t)() - ctx := context.TODO() + ctx := context.Background() defDesc := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKeyMin, - EndKey: roachpb.RKeyMax, + StartKey: roachpb.RKeyMin, + EndKey: roachpb.RKeyMax, + Generation: 0, } st := cluster.MakeTestingClusterSettings() @@ -970,25 +978,28 @@ func TestRangeCacheClearOverlapping(t *testing.T) { // Now, add a new, overlapping set of descriptors. minToBDesc := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKeyMin, - EndKey: roachpb.RKey("b"), + StartKey: roachpb.RKeyMin, + EndKey: roachpb.RKey("b"), + Generation: 1, } bToMaxDesc := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("b"), - EndKey: roachpb.RKeyMax, - } - if _, err := cache.clearOverlappingCachedRangeDescriptors(ctx, minToBDesc); err != nil { - t.Fatal(err) - } + StartKey: roachpb.RKey("b"), + EndKey: roachpb.RKeyMax, + Generation: 1, + } + curGeneration := int64(1) + ok, err := cache.clearOverlappingCachedRangeDescriptors(ctx, minToBDesc) + require.NoError(t, err) + require.True(t, ok) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("b"))), minToBDesc) if desc, err := cache.GetCachedRangeDescriptor(roachpb.RKey("b"), false); err != nil { t.Fatal(err) } else if desc != nil { t.Errorf("descriptor unexpectedly non-nil: %s", desc) } - if _, err := cache.clearOverlappingCachedRangeDescriptors(ctx, bToMaxDesc); err != nil { - t.Fatal(err) - } + ok, err = cache.clearOverlappingCachedRangeDescriptors(ctx, bToMaxDesc) + require.NoError(t, err) + require.True(t, ok) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKeyMax)), bToMaxDesc) if desc, err := cache.GetCachedRangeDescriptor(roachpb.RKey("b"), false); err != nil { t.Fatal(err) @@ -997,9 +1008,12 @@ func TestRangeCacheClearOverlapping(t *testing.T) { } // Add default descriptor back which should remove two split descriptors. - if _, err := cache.clearOverlappingCachedRangeDescriptors(ctx, defDesc); err != nil { - t.Fatal(err) - } + defDescCpy := *defDesc + curGeneration++ + defDescCpy.Generation = curGeneration + ok, err = cache.clearOverlappingCachedRangeDescriptors(ctx, &defDescCpy) + require.NoError(t, err) + require.True(t, ok) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKeyMax)), defDesc) for _, key := range []roachpb.RKey{roachpb.RKey("a"), roachpb.RKey("b")} { if desc, err := cache.GetCachedRangeDescriptor(key, false); err != nil { @@ -1010,13 +1024,15 @@ func TestRangeCacheClearOverlapping(t *testing.T) { } // Insert ["b", "c") and then insert ["a", b"). Verify that the former is not evicted by the latter. + curGeneration++ bToCDesc := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("b"), - EndKey: roachpb.RKey("c"), - } - if _, err := cache.clearOverlappingCachedRangeDescriptors(ctx, bToCDesc); err != nil { - t.Fatal(err) + StartKey: roachpb.RKey("b"), + EndKey: roachpb.RKey("c"), + Generation: curGeneration, } + ok, err = cache.clearOverlappingCachedRangeDescriptors(ctx, bToCDesc) + require.NoError(t, err) + require.True(t, ok) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("c"))), bToCDesc) if desc, err := cache.GetCachedRangeDescriptor(roachpb.RKey("c"), true); err != nil { t.Fatal(err) @@ -1024,13 +1040,15 @@ func TestRangeCacheClearOverlapping(t *testing.T) { t.Errorf("expected descriptor %s; got %s", bToCDesc, desc) } + curGeneration++ aToBDesc := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("a"), - EndKey: roachpb.RKey("b"), - } - if _, err := cache.clearOverlappingCachedRangeDescriptors(ctx, aToBDesc); err != nil { - t.Fatal(err) + StartKey: roachpb.RKey("a"), + EndKey: roachpb.RKey("b"), + Generation: curGeneration, } + ok, err = cache.clearOverlappingCachedRangeDescriptors(ctx, aToBDesc) + require.NoError(t, err) + require.True(t, ok) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("b"))), aToBDesc) if desc, err := cache.GetCachedRangeDescriptor(roachpb.RKey("c"), true); err != nil { t.Fatal(err) @@ -1164,39 +1182,30 @@ func TestRangeCacheGeneration(t *testing.T) { ctx := context.TODO() descAM1 := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("a"), - EndKey: roachpb.RKey("m"), - Generation: 1, - GenerationComparable: proto.Bool(true), + StartKey: roachpb.RKey("a"), + EndKey: roachpb.RKey("m"), + Generation: 1, } descMZ3 := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("m"), - EndKey: roachpb.RKey("z"), - Generation: 3, - GenerationComparable: proto.Bool(true), + StartKey: roachpb.RKey("m"), + EndKey: roachpb.RKey("z"), + Generation: 3, } descBY0 := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("b"), - EndKey: roachpb.RKey("y"), - Generation: 0, - GenerationComparable: proto.Bool(true), + StartKey: roachpb.RKey("b"), + EndKey: roachpb.RKey("y"), + Generation: 0, } descBY2 := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("b"), - EndKey: roachpb.RKey("y"), - Generation: 2, - GenerationComparable: proto.Bool(true), + StartKey: roachpb.RKey("b"), + EndKey: roachpb.RKey("y"), + Generation: 2, } descBY4 := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("b"), - EndKey: roachpb.RKey("y"), - Generation: 4, - GenerationComparable: proto.Bool(true), - } - descBYIncomparable := &roachpb.RangeDescriptor{ - StartKey: roachpb.RKey("b"), - EndKey: roachpb.RKey("y"), + StartKey: roachpb.RKey("b"), + EndKey: roachpb.RKey("y"), + Generation: 4, } testCases := []struct { @@ -1208,7 +1217,7 @@ func TestRangeCacheGeneration(t *testing.T) { { // descBY0 is ignored since the existing keyspace is covered by // descriptors of generations 1 and 3, respectively. - name: "generation comparable evict 0", + name: "evict 0", insertDesc: descBY0, queryKeys: []roachpb.RKey{roachpb.RKey("b"), roachpb.RKey("y")}, expectedDesc: []*roachpb.RangeDescriptor{descAM1, descMZ3}, @@ -1217,27 +1226,18 @@ func TestRangeCacheGeneration(t *testing.T) { // descBY2 evicts descAM1, but not descMZ3 based on Generation. Since // there is an overlapping descriptor with higher Generation (descMZ3), // it is not inserted. - name: "generation comparable evict 1", + name: "evict 1", insertDesc: descBY2, queryKeys: []roachpb.RKey{roachpb.RKey("b"), roachpb.RKey("y")}, expectedDesc: []*roachpb.RangeDescriptor{nil, descMZ3}, }, { // descBY4 replaces both existing descriptors and it is inserted. - name: "generation comparable evict 2", + name: "evict 2", insertDesc: descBY4, queryKeys: []roachpb.RKey{roachpb.RKey("b"), roachpb.RKey("y")}, expectedDesc: []*roachpb.RangeDescriptor{descBY4, nil}, }, - { - // descBYIncomparable has an incomparable Generation, so it evicts all - // overlapping descriptors. This behavior is clearly less desirable in - // general, but there's no better option in this case. - name: "generation incomparable evict 2", - insertDesc: descBYIncomparable, - queryKeys: []roachpb.RKey{roachpb.RKey("b"), roachpb.RKey("y")}, - expectedDesc: []*roachpb.RangeDescriptor{descBYIncomparable, nil}, - }, } for _, tc := range testCases { diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_test.go b/pkg/kv/kvserver/batcheval/cmd_lease_test.go index b0ef793c5338..db9716a4000c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_test.go @@ -140,7 +140,7 @@ func TestLeaseCommandLearnerReplica(t *testing.T) { const expForUnknown = `cannot replace lease repl=(n0,s0):? seq=0 start=0,0 exp= ` + `with repl=(n0,s0):? seq=0 start=0,0 exp=: ` + - `replica (n0,s0):? not found in r0:{-} [(n1,s1):1, (n2,s2):2LEARNER, next=0, gen=0?]` + `replica (n0,s0):? not found in r0:{-} [(n1,s1):1, (n2,s2):2LEARNER, next=0, gen=0]` require.EqualError(t, err, expForUnknown) cArgs.Args = &roachpb.RequestLeaseRequest{ diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 9c809b5f3466..5a4da66803ed 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -40,7 +40,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" - "github.com/gogo/protobuf/proto" "go.etcd.io/etcd/raft" "go.etcd.io/etcd/raft/raftpb" "go.etcd.io/etcd/raft/tracker" @@ -132,14 +131,12 @@ func prepareSplitDescs( } leftDesc.IncrementGeneration() - leftDesc.GenerationComparable = proto.Bool(true) leftDesc.EndKey = splitKey // Set the generation of the right hand side descriptor to match that of the // (updated) left hand side. See the comment on the field for an explanation // of why generations are useful. rightDesc.Generation = leftDesc.Generation - rightDesc.GenerationComparable = proto.Bool(true) setStickyBit(rightDesc, expiration) return leftDesc, rightDesc @@ -648,7 +645,6 @@ func (r *Replica) AdminMerge( updatedLeftDesc.Generation = rightDesc.Generation } updatedLeftDesc.IncrementGeneration() - updatedLeftDesc.GenerationComparable = proto.Bool(true) updatedLeftDesc.EndKey = rightDesc.EndKey log.Infof(ctx, "initiating a merge of %s into this range (%s)", &rightDesc, reason) @@ -1405,7 +1401,6 @@ func prepareChangeReplicasTrigger( updatedDesc := *desc updatedDesc.SetReplicas(desc.Replicas().DeepCopy()) updatedDesc.IncrementGeneration() - updatedDesc.GenerationComparable = proto.Bool(true) var added, removed []roachpb.ReplicaDescriptor if !chgs.leaveJoint() { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 2191baeda760..3b775ec313cc 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -12625,7 +12625,7 @@ func TestRangeUnavailableMessage(t *testing.T) { This range is likely unavailable. Please submit this message to Cockroach Labs support along with the following information: -Descriptor: r10:{-} [(n1,s10):1, (n2,s20):2, next=3, gen=0?] +Descriptor: r10:{-} [(n1,s10):1, (n2,s20):2, next=3, gen=0] Live: (n1,s10):1 Non-live: (n2,s20):2 Raft Status: {"id":"0","term":0,"vote":"0","commit":0,"lead":"0","raftState":"StateFollower","applied":0,"progress":{},"leadtransferee":"0"} diff --git a/pkg/kv/kvserver/store_bootstrap.go b/pkg/kv/kvserver/store_bootstrap.go index bd54a3c94421..2df94330ed7f 100644 --- a/pkg/kv/kvserver/store_bootstrap.go +++ b/pkg/kv/kvserver/store_bootstrap.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" - "github.com/gogo/protobuf/proto" ) // InitEngine writes a new store ident to the underlying engine. To @@ -139,11 +138,10 @@ func WriteInitialClusterData( } desc := &roachpb.RangeDescriptor{ - RangeID: rangeID, - StartKey: startKey, - EndKey: endKey, - NextReplicaID: 2, - GenerationComparable: proto.Bool(true), + RangeID: rangeID, + StartKey: startKey, + EndKey: endKey, + NextReplicaID: 2, } replicas := []roachpb.ReplicaDescriptor{ { diff --git a/pkg/kv/range_lookup.go b/pkg/kv/range_lookup.go index b8d695f69fb0..6facaebdc972 100644 --- a/pkg/kv/range_lookup.go +++ b/pkg/kv/range_lookup.go @@ -12,7 +12,6 @@ package kv import ( "context" - "math/rand" "time" "github.com/cockroachdb/cockroach/pkg/keys" @@ -218,24 +217,9 @@ func RangeLookup( } else { // Since we support scanning non-transactionally, it's possible that // we pick up both the pre- and post-split descriptor for a range. - if desc.GetGenerationComparable() && matchingRanges[0].GetGenerationComparable() { - if desc.Generation > matchingRanges[0].Generation { - // If both generations are comparable, we take the range - // descriptor with the newer generation. - matchingRanges[0] = *desc - } - } else { - if rand.Intn(index+1) == 0 { - // Generations are not comparable, so we randomly choose using - // reservoir sampling. Note that we cannot determine the newer - // version of the descriptor by looking at the size of the range - // because both splits and merges can happen. Using randomness to - // determine which range to return is okay, because if we guess - // wrong we will try the lookup again. Randomness is used to - // ensure we probabilistically converge to the correct - // descriptor. - matchingRanges[0] = *desc - } + if desc.Generation > matchingRanges[0].Generation { + // Take the range descriptor with the newer generation. + matchingRanges[0] = *desc } } } else { diff --git a/pkg/kv/range_lookup_test.go b/pkg/kv/range_lookup_test.go index 0cef879f076c..089c3a7e7763 100644 --- a/pkg/kv/range_lookup_test.go +++ b/pkg/kv/range_lookup_test.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/gogo/protobuf/proto" ) // TestRangeLookupRace tests that a RangeLookup will retry its scanning process @@ -30,25 +29,22 @@ func TestRangeLookupRaceSplits(t *testing.T) { defer leaktest.AfterTest(t)() desc1BeforeSplit := roachpb.RangeDescriptor{ - RangeID: 1, - StartKey: roachpb.RKey("j"), - EndKey: roachpb.RKey("p"), - Generation: 0, - GenerationComparable: proto.Bool(true), + RangeID: 1, + StartKey: roachpb.RKey("j"), + EndKey: roachpb.RKey("p"), + Generation: 0, } desc1AfterSplit := roachpb.RangeDescriptor{ - RangeID: 1, - StartKey: roachpb.RKey("j"), - EndKey: roachpb.RKey("m"), - Generation: 1, - GenerationComparable: proto.Bool(true), + RangeID: 1, + StartKey: roachpb.RKey("j"), + EndKey: roachpb.RKey("m"), + Generation: 1, } desc2AfterSplit := roachpb.RangeDescriptor{ - RangeID: 2, - StartKey: roachpb.RKey("m"), - EndKey: roachpb.RKey("p"), - Generation: 1, - GenerationComparable: proto.Bool(true), + RangeID: 2, + StartKey: roachpb.RKey("m"), + EndKey: roachpb.RKey("p"), + Generation: 1, } lookupKey := roachpb.Key("k") diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index a4cccfc96f1c..d2eddb9b52a6 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -33,7 +33,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/gogo/protobuf/proto" "github.com/kr/pretty" "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft/raftpb" @@ -1673,9 +1672,8 @@ func TestChangeReplicasTrigger_String(t *testing.T) { learner, repl3, }, - NextReplicaID: 10, - Generation: 5, - GenerationComparable: proto.Bool(true), + NextReplicaID: 10, + Generation: 5, }, } act := crt.String() diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index a395f8d7e68d..ff9abbe8a90c 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -243,14 +243,6 @@ func (r *RangeDescriptor) IncrementGeneration() { r.Generation++ } -// GetGenerationComparable returns if the generation of this RangeDescriptor is comparable. -func (r *RangeDescriptor) GetGenerationComparable() bool { - if r.GenerationComparable == nil { - return false - } - return *r.GenerationComparable -} - // GetStickyBit returns the sticky bit of this RangeDescriptor. func (r *RangeDescriptor) GetStickyBit() hlc.Timestamp { if r.StickyBit == nil { @@ -310,9 +302,6 @@ func (r RangeDescriptor) String() string { buf.WriteString("") } fmt.Fprintf(&buf, ", next=%d, gen=%d", r.NextReplicaID, r.Generation) - if !r.GetGenerationComparable() { - buf.WriteString("?") - } if s := r.GetStickyBit(); !s.IsEmpty() { fmt.Fprintf(&buf, ", sticky=%s", s) } diff --git a/pkg/roachpb/metadata.pb.go b/pkg/roachpb/metadata.pb.go index d492009a5e75..123baca0f1e5 100644 --- a/pkg/roachpb/metadata.pb.go +++ b/pkg/roachpb/metadata.pb.go @@ -112,7 +112,7 @@ func (x *ReplicaType) UnmarshalJSON(data []byte) error { return nil } func (ReplicaType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{0} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{0} } // Attributes specifies a list of arbitrary strings describing @@ -124,7 +124,7 @@ type Attributes struct { func (m *Attributes) Reset() { *m = Attributes{} } func (*Attributes) ProtoMessage() {} func (*Attributes) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{0} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{0} } func (m *Attributes) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -158,7 +158,7 @@ type ReplicationTarget struct { func (m *ReplicationTarget) Reset() { *m = ReplicationTarget{} } func (*ReplicationTarget) ProtoMessage() {} func (*ReplicationTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{1} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{1} } func (m *ReplicationTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -203,7 +203,7 @@ type ReplicaDescriptor struct { func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } func (*ReplicaDescriptor) ProtoMessage() {} func (*ReplicaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{2} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{2} } func (m *ReplicaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -238,7 +238,7 @@ func (m *ReplicaIdent) Reset() { *m = ReplicaIdent{} } func (m *ReplicaIdent) String() string { return proto.CompactTextString(m) } func (*ReplicaIdent) ProtoMessage() {} func (*ReplicaIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{3} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{3} } func (m *ReplicaIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -342,16 +342,6 @@ type RangeDescriptor struct { // the replica it's supposed to update, and no splits and merges have taken // place at all). // - // Note also that when the generation counter was first introduced, it only - // ever incremented (by one) the generation of the left hand side on merges - // and splits, so the above overlap arguments only hold if we know that the - // descriptors involved never used that code. Generations were first introduced - // in the 19.1 release, though, the behavior described here was only introduced - // in a late release candidate. If we allow such a release candidate cluster - // to transition into the final 19.1 release, we will need to introduce - // additional state to mark descriptors as obeying the new rules. If we don't, - // then we are free to assume that the semantics always hold. - // // For a third note, observe that the generational semantics above may // possibly allow range merges without colocation, at least in the sense that // the counter examples in #28071 are defused. This is because the @@ -359,16 +349,6 @@ type RangeDescriptor struct { // replica is gc'able or not. If it is not gc'able, then by definition the // replica applying the merge is. Generation int64 `protobuf:"varint,6,opt,name=generation" json:"generation"` - // generation_comparable is additional state to mark descriptors as obeying - // the new generation rules. We can determine the causality of two range - // descriptors if they overlap and both their generation_comparable are set. - // If one or both of their generation_comparable is not set, we cannot - // determine which range descriptor is newer. This is needed because older - // versions of Cockroach did not set the generation in some cases where it is - // currently set. Code that relies on a total order of range descriptor - // generations can determine when this order is ensured from this state. See - // #36654 for more details. - GenerationComparable *bool `protobuf:"varint,8,opt,name=generation_comparable,json=generationComparable" json:"generation_comparable,omitempty"` // The presence of the sticky_bit indicates that the range should not be // automatically merged by the merge queue with the range to its left. It is // set during a split operation and unset during an unsplit operation. Note @@ -390,7 +370,7 @@ type RangeDescriptor struct { func (m *RangeDescriptor) Reset() { *m = RangeDescriptor{} } func (*RangeDescriptor) ProtoMessage() {} func (*RangeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{4} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{4} } func (m *RangeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -429,7 +409,7 @@ type Percentiles struct { func (m *Percentiles) Reset() { *m = Percentiles{} } func (*Percentiles) ProtoMessage() {} func (*Percentiles) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{5} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{5} } func (m *Percentiles) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -491,7 +471,7 @@ type StoreCapacity struct { func (m *StoreCapacity) Reset() { *m = StoreCapacity{} } func (*StoreCapacity) ProtoMessage() {} func (*StoreCapacity) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{6} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{6} } func (m *StoreCapacity) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -536,7 +516,7 @@ func (m *NodeDescriptor) Reset() { *m = NodeDescriptor{} } func (m *NodeDescriptor) String() string { return proto.CompactTextString(m) } func (*NodeDescriptor) ProtoMessage() {} func (*NodeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{7} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{7} } func (m *NodeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -572,7 +552,7 @@ func (m *LocalityAddress) Reset() { *m = LocalityAddress{} } func (m *LocalityAddress) String() string { return proto.CompactTextString(m) } func (*LocalityAddress) ProtoMessage() {} func (*LocalityAddress) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{8} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{8} } func (m *LocalityAddress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -610,7 +590,7 @@ func (m *StoreDescriptor) Reset() { *m = StoreDescriptor{} } func (m *StoreDescriptor) String() string { return proto.CompactTextString(m) } func (*StoreDescriptor) ProtoMessage() {} func (*StoreDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{9} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{9} } func (m *StoreDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -647,7 +627,7 @@ func (m *StoreDeadReplicas) Reset() { *m = StoreDeadReplicas{} } func (m *StoreDeadReplicas) String() string { return proto.CompactTextString(m) } func (*StoreDeadReplicas) ProtoMessage() {} func (*StoreDeadReplicas) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{10} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{10} } func (m *StoreDeadReplicas) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -681,7 +661,7 @@ type Locality struct { func (m *Locality) Reset() { *m = Locality{} } func (*Locality) ProtoMessage() {} func (*Locality) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{11} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{11} } func (m *Locality) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -717,7 +697,7 @@ type Tier struct { func (m *Tier) Reset() { *m = Tier{} } func (*Tier) ProtoMessage() {} func (*Tier) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{12} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{12} } func (m *Tier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -756,7 +736,7 @@ type Version struct { func (m *Version) Reset() { *m = Version{} } func (*Version) ProtoMessage() {} func (*Version) Descriptor() ([]byte, []int) { - return fileDescriptor_metadata_436899bb717723ad, []int{13} + return fileDescriptor_metadata_4d08fb4df4010e8b, []int{13} } func (m *Version) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -906,15 +886,6 @@ func (this *RangeDescriptor) Equal(that interface{}) bool { if this.Generation != that1.Generation { return false } - if this.GenerationComparable != nil && that1.GenerationComparable != nil { - if *this.GenerationComparable != *that1.GenerationComparable { - return false - } - } else if this.GenerationComparable != nil { - return false - } else if that1.GenerationComparable != nil { - return false - } if !this.StickyBit.Equal(that1.StickyBit) { return false } @@ -1096,16 +1067,6 @@ func (m *RangeDescriptor) MarshalTo(dAtA []byte) (int, error) { } i += n2 } - if m.GenerationComparable != nil { - dAtA[i] = 0x40 - i++ - if *m.GenerationComparable { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } return i, nil } @@ -1564,10 +1525,6 @@ func NewPopulatedRangeDescriptor(r randyMetadata, easy bool) *RangeDescriptor { if r.Intn(10) != 0 { this.StickyBit = hlc.NewPopulatedTimestamp(r, easy) } - if r.Intn(10) != 0 { - v6 := bool(bool(r.Intn(2) == 0)) - this.GenerationComparable = &v6 - } if !easy && r.Intn(10) != 0 { } return this @@ -1592,9 +1549,9 @@ func randUTF8RuneMetadata(r randyMetadata) rune { return rune(ru + 61) } func randStringMetadata(r randyMetadata) string { - v7 := r.Intn(100) - tmps := make([]rune, v7) - for i := 0; i < v7; i++ { + v6 := r.Intn(100) + tmps := make([]rune, v6) + for i := 0; i < v6; i++ { tmps[i] = randUTF8RuneMetadata(r) } return string(tmps) @@ -1616,11 +1573,11 @@ func randFieldMetadata(dAtA []byte, r randyMetadata, fieldNumber int, wire int) switch wire { case 0: dAtA = encodeVarintPopulateMetadata(dAtA, uint64(key)) - v8 := r.Int63() + v7 := r.Int63() if r.Intn(2) == 0 { - v8 *= -1 + v7 *= -1 } - dAtA = encodeVarintPopulateMetadata(dAtA, uint64(v8)) + dAtA = encodeVarintPopulateMetadata(dAtA, uint64(v7)) case 1: dAtA = encodeVarintPopulateMetadata(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))) @@ -1725,9 +1682,6 @@ func (m *RangeDescriptor) Size() (n int) { l = m.StickyBit.Size() n += 1 + l + sovMetadata(uint64(l)) } - if m.GenerationComparable != nil { - n += 2 - } return n } @@ -2502,27 +2456,6 @@ func (m *RangeDescriptor) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 8: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field GenerationComparable", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowMetadata - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - b := bool(v != 0) - m.GenerationComparable = &b default: iNdEx = preIndex skippy, err := skipMetadata(dAtA[iNdEx:]) @@ -4022,96 +3955,95 @@ var ( ErrIntOverflowMetadata = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_436899bb717723ad) } - -var fileDescriptor_metadata_436899bb717723ad = []byte{ - // 1404 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcf, 0x6f, 0x1b, 0xc5, - 0x17, 0xf7, 0xda, 0xeb, 0xd8, 0x7e, 0xce, 0x0f, 0x7b, 0xd4, 0xf6, 0x6b, 0xf9, 0x2b, 0x6c, 0xd7, - 0xb4, 0x22, 0x2d, 0x28, 0x49, 0x53, 0x45, 0x55, 0x03, 0x05, 0xe2, 0x24, 0xad, 0x4c, 0xf3, 0x8b, - 0x8d, 0x5b, 0x24, 0x2e, 0xab, 0xc9, 0xee, 0xe0, 0x2c, 0x5d, 0xef, 0x6e, 0x67, 0xc7, 0x69, 0x7d, - 0x47, 0x02, 0x09, 0x21, 0x71, 0x41, 0xe2, 0x58, 0x09, 0xf5, 0x7f, 0xe0, 0x4f, 0xe8, 0xb1, 0xc7, - 0x9e, 0xa2, 0x92, 0x5e, 0x38, 0x73, 0xec, 0x01, 0xa1, 0x99, 0x9d, 0xd9, 0x5d, 0xa7, 0x29, 0xb4, - 0x41, 0xe2, 0x36, 0xfb, 0x79, 0x9f, 0xcf, 0xdb, 0x37, 0x6f, 0xde, 0xbc, 0x37, 0x70, 0x8e, 0xfa, - 0xd8, 0xda, 0x0f, 0xf6, 0xe6, 0x07, 0x84, 0x61, 0x1b, 0x33, 0x3c, 0x17, 0x50, 0x9f, 0xf9, 0xa8, - 0x6a, 0xf9, 0xd6, 0x3d, 0x61, 0x9b, 0x93, 0x8c, 0x7a, 0x7d, 0xc8, 0x1c, 0x77, 0x7e, 0xe8, 0x51, - 0x12, 0xfa, 0xee, 0x01, 0xb1, 0x4d, 0x6c, 0xdb, 0x34, 0xa2, 0xd7, 0x6b, 0xc2, 0xb6, 0xef, 0x5a, - 0xf3, 0xcc, 0x19, 0x90, 0x90, 0xe1, 0x41, 0x20, 0x2d, 0x67, 0xfa, 0x7e, 0xdf, 0x17, 0xcb, 0x79, - 0xbe, 0x8a, 0xd0, 0xf6, 0x27, 0x00, 0x2b, 0x8c, 0x51, 0x67, 0x6f, 0xc8, 0x48, 0x88, 0xde, 0x87, - 0x3c, 0x66, 0x8c, 0x86, 0x35, 0xad, 0x95, 0x9b, 0x2d, 0x75, 0xce, 0xfe, 0x71, 0xd8, 0xac, 0x8e, - 0xf0, 0xc0, 0x5d, 0x6e, 0x0b, 0xf8, 0x83, 0xaf, 0x5c, 0xff, 0x41, 0xdb, 0x88, 0x38, 0xcb, 0xfa, - 0xcf, 0x8f, 0x9a, 0x99, 0xf6, 0xb7, 0x1a, 0x54, 0x0d, 0x12, 0xb8, 0x8e, 0x85, 0x99, 0xe3, 0x7b, - 0x3d, 0x4c, 0xfb, 0x84, 0xa1, 0x2b, 0x50, 0xf0, 0x7c, 0x9b, 0x98, 0x8e, 0x5d, 0xd3, 0x5a, 0xda, - 0x6c, 0xbe, 0x53, 0x7b, 0x72, 0xd8, 0xcc, 0x1c, 0x1d, 0x36, 0x27, 0xb6, 0x7c, 0x9b, 0x74, 0xd7, - 0x5e, 0xc6, 0x2b, 0x63, 0x82, 0x13, 0xbb, 0x36, 0x5a, 0x82, 0x62, 0xc8, 0x7c, 0x2a, 0x34, 0x59, - 0xa1, 0xa9, 0x4b, 0x4d, 0x61, 0x97, 0xe3, 0x42, 0xa4, 0x96, 0x46, 0x41, 0x70, 0xbb, 0xf6, 0x72, - 0x91, 0x47, 0xf1, 0xfb, 0xa3, 0xa6, 0xd6, 0xfe, 0x33, 0x89, 0x64, 0x8d, 0x84, 0x16, 0x75, 0x02, - 0xe6, 0xd3, 0xff, 0x2e, 0x12, 0x74, 0x03, 0x80, 0x46, 0xbf, 0xe7, 0xc2, 0x9c, 0x10, 0x36, 0xa4, - 0xb0, 0x24, 0x03, 0x13, 0xd2, 0xe4, 0xc3, 0x28, 0x49, 0x45, 0xd7, 0x46, 0x8b, 0xa0, 0xb3, 0x51, - 0x40, 0x6a, 0x7a, 0x4b, 0x9b, 0x9d, 0x5e, 0x6c, 0xcc, 0xbd, 0x72, 0xee, 0x73, 0x52, 0xd6, 0x1b, - 0x05, 0xc4, 0x10, 0xdc, 0xe5, 0x49, 0xbe, 0xf9, 0x5f, 0x1f, 0x35, 0x35, 0x91, 0x80, 0xef, 0x35, - 0x98, 0x54, 0xae, 0x6d, 0xe2, 0x31, 0xbe, 0x11, 0x8a, 0xbd, 0x7e, 0xbc, 0xf9, 0x5c, 0xb2, 0x11, - 0x83, 0xe3, 0xd1, 0x46, 0xe4, 0xd2, 0x28, 0x08, 0x6e, 0xd7, 0x46, 0x6b, 0x50, 0x90, 0x61, 0x89, - 0xed, 0x97, 0x17, 0x2f, 0xbc, 0x3e, 0x98, 0x24, 0xd3, 0x1d, 0x9d, 0xfb, 0x36, 0x94, 0xb4, 0x7d, - 0x98, 0x83, 0x19, 0xe1, 0x3a, 0x75, 0x18, 0xa7, 0x0c, 0xe8, 0x22, 0x94, 0x42, 0x86, 0x29, 0x33, - 0xef, 0x91, 0x91, 0x08, 0x69, 0xb2, 0x53, 0x7c, 0x79, 0xd8, 0xd4, 0x8d, 0xdb, 0x64, 0x64, 0x14, - 0x85, 0xe9, 0x36, 0x19, 0xa1, 0xf3, 0x50, 0x20, 0x9e, 0x2d, 0x48, 0xb9, 0x63, 0xa4, 0x09, 0xe2, - 0xd9, 0x9c, 0xf2, 0x05, 0x54, 0x1d, 0x8f, 0x11, 0xea, 0x61, 0xd7, 0x94, 0x81, 0x86, 0x35, 0xbd, - 0x95, 0x7b, 0xcb, 0x4d, 0x56, 0x94, 0x13, 0x49, 0x08, 0xd1, 0x67, 0x30, 0xe3, 0x91, 0x87, 0xcc, - 0x4c, 0x55, 0x40, 0x5e, 0x54, 0x40, 0x5b, 0x6e, 0x70, 0x6a, 0x8b, 0x3c, 0x64, 0xaf, 0xa9, 0x82, - 0x29, 0x2f, 0x65, 0xb3, 0xd1, 0x05, 0x80, 0x3e, 0xf1, 0x08, 0x15, 0x17, 0xaa, 0x36, 0x21, 0xf2, - 0x14, 0xfd, 0x37, 0x85, 0xa3, 0x8f, 0x00, 0x42, 0xe6, 0x58, 0xf7, 0x46, 0xe6, 0x9e, 0xc3, 0x6a, - 0x05, 0x71, 0x50, 0xef, 0xa4, 0xf6, 0xc0, 0x1b, 0xc1, 0xdc, 0xbe, 0x6b, 0xcd, 0xf5, 0x54, 0x23, - 0x30, 0x4a, 0x91, 0xa0, 0xe3, 0x30, 0x74, 0x15, 0xce, 0x26, 0xbe, 0x4c, 0xcb, 0x1f, 0x04, 0x98, - 0xe2, 0x3d, 0x97, 0xd4, 0x8a, 0x2d, 0x6d, 0xb6, 0x68, 0x9c, 0x49, 0x8c, 0xab, 0xb1, 0xed, 0x58, - 0xb9, 0x3d, 0xd6, 0xa0, 0xbc, 0x43, 0xa8, 0x45, 0x3c, 0xe6, 0xb8, 0x24, 0x44, 0xe7, 0x20, 0x17, - 0x5c, 0x59, 0x10, 0xe7, 0xaa, 0xc9, 0x78, 0x39, 0x20, 0xf0, 0xc5, 0x25, 0x71, 0x6e, 0x09, 0xbe, - 0xb8, 0x24, 0xf0, 0xa5, 0x05, 0x71, 0x54, 0x09, 0xbe, 0x14, 0xf1, 0xaf, 0x2d, 0x89, 0x7b, 0x90, - 0xe0, 0xd7, 0x22, 0xfe, 0xf5, 0x05, 0x91, 0xd6, 0x04, 0xbf, 0xbe, 0x80, 0x6a, 0xa0, 0x07, 0x9b, - 0xf8, 0xa1, 0x48, 0x94, 0x32, 0x08, 0x44, 0x76, 0xa8, 0x97, 0x39, 0x98, 0x12, 0x97, 0x75, 0x15, - 0x07, 0xd8, 0x72, 0xd8, 0x08, 0xb5, 0xa0, 0x68, 0xc9, 0xb5, 0x2c, 0xc3, 0x48, 0x15, 0xa3, 0xa8, - 0x0d, 0x25, 0x7c, 0x80, 0x1d, 0x57, 0xa4, 0x24, 0x9b, 0xa2, 0x24, 0x30, 0xba, 0x08, 0xe5, 0xa8, - 0x98, 0x2d, 0x7f, 0xe8, 0x31, 0x79, 0xe1, 0xe5, 0x39, 0x09, 0xc3, 0x2a, 0xc7, 0x39, 0xcd, 0x25, - 0x38, 0x54, 0x34, 0x3d, 0x4d, 0x13, 0x86, 0x88, 0xb6, 0x00, 0xd5, 0x07, 0xd4, 0x61, 0x24, 0x34, - 0x03, 0x42, 0xcd, 0x90, 0x58, 0xbe, 0x67, 0x8f, 0xed, 0x75, 0x26, 0x32, 0xef, 0x10, 0xba, 0x2b, - 0x8c, 0x68, 0x07, 0xaa, 0x7b, 0x23, 0x25, 0x50, 0x17, 0x76, 0x42, 0xd4, 0xc1, 0x49, 0xdd, 0x23, - 0x75, 0x54, 0xca, 0xa3, 0x90, 0xef, 0x10, 0x2a, 0x4b, 0x0f, 0x19, 0x80, 0x52, 0x31, 0x28, 0x97, - 0x85, 0xb7, 0x70, 0x59, 0x89, 0x83, 0x54, 0x3e, 0x6b, 0xa0, 0x0f, 0x43, 0x62, 0x8b, 0xba, 0x52, - 0x49, 0x14, 0x08, 0xba, 0x04, 0x53, 0xae, 0xdf, 0x77, 0x2c, 0xec, 0x9a, 0x22, 0x90, 0x5a, 0x29, - 0x45, 0x99, 0x94, 0xa6, 0x0e, 0xb7, 0xa0, 0x45, 0x40, 0xf7, 0x87, 0x84, 0x3a, 0xe3, 0xd9, 0x81, - 0x54, 0x76, 0x2a, 0xd2, 0x1e, 0xa7, 0x47, 0x1e, 0xfe, 0x73, 0x1d, 0xa6, 0x79, 0x7b, 0xff, 0x77, - 0x13, 0xe1, 0x63, 0x28, 0xf0, 0x19, 0x4b, 0xc2, 0x50, 0x76, 0xc4, 0xc6, 0xf1, 0x8b, 0x76, 0x27, - 0x9e, 0xc6, 0x2b, 0xb6, 0x1d, 0xf7, 0x42, 0x29, 0x42, 0xd7, 0xd5, 0x5c, 0xcd, 0xbd, 0x72, 0x4d, - 0x55, 0x2e, 0x93, 0x29, 0x2c, 0xc5, 0x91, 0x02, 0xdd, 0x80, 0xa2, 0xeb, 0x5b, 0xd8, 0xe5, 0xb5, - 0xaa, 0x0b, 0xf5, 0xff, 0x4f, 0x50, 0x6f, 0x48, 0x8a, 0x2a, 0x64, 0x25, 0x41, 0x37, 0x61, 0x6a, - 0x97, 0xd0, 0x03, 0x42, 0xef, 0x12, 0x1a, 0xf2, 0x76, 0x92, 0x17, 0x3e, 0xea, 0x27, 0xf8, 0x90, - 0x0c, 0xe9, 0x62, 0x5c, 0x86, 0xce, 0x43, 0x69, 0x6f, 0xe8, 0xb8, 0xb6, 0xc9, 0x70, 0x5f, 0x14, - 0x59, 0x49, 0xfd, 0x4a, 0xc0, 0x3d, 0xdc, 0x47, 0xef, 0xf2, 0x86, 0x84, 0x29, 0xe3, 0x0f, 0x92, - 0xa8, 0x21, 0xc5, 0x97, 0x46, 0xe2, 0x2b, 0x0c, 0xed, 0x42, 0x45, 0xc5, 0x66, 0xaa, 0x94, 0x16, - 0x45, 0xff, 0x6d, 0xff, 0xcd, 0xb6, 0x56, 0x22, 0xa6, 0xaa, 0x5b, 0x77, 0x1c, 0x46, 0xef, 0xc1, - 0xa4, 0xe5, 0x0e, 0x43, 0x46, 0xa8, 0xe9, 0xe1, 0x01, 0x11, 0x85, 0xa4, 0xe2, 0x2b, 0x4b, 0xcb, - 0x16, 0x1e, 0x10, 0xb4, 0x0b, 0xe5, 0xf0, 0xbe, 0x1b, 0xff, 0x18, 0xde, 0xe8, 0x2c, 0x91, 0x2c, - 0x0f, 0xd8, 0xfd, 0x7c, 0x43, 0xfe, 0xd1, 0x80, 0xf0, 0xbe, 0x2b, 0xd7, 0xed, 0x9f, 0x34, 0x98, - 0x39, 0x16, 0x68, 0xba, 0x60, 0xb4, 0xd3, 0x14, 0x4c, 0x87, 0xdf, 0x0d, 0x99, 0x26, 0xe6, 0x10, - 0x2a, 0xcb, 0xee, 0x7f, 0x27, 0xe4, 0xa8, 0xe7, 0x10, 0x9a, 0x5c, 0x9a, 0x48, 0xc3, 0xb1, 0xf6, - 0x37, 0x59, 0x98, 0x11, 0x7d, 0x6f, 0x7c, 0x00, 0xc7, 0x4f, 0x1b, 0xed, 0xcd, 0x9f, 0x36, 0x71, - 0xfd, 0x66, 0xdf, 0xba, 0x7e, 0x3f, 0x04, 0x9d, 0x5f, 0x22, 0x59, 0xf9, 0xe7, 0x4f, 0x50, 0x8e, - 0x5f, 0x4f, 0xd5, 0x22, 0xb8, 0x08, 0x75, 0x52, 0x8d, 0x3a, 0x2a, 0xfe, 0xd6, 0x09, 0x0e, 0xc6, - 0x9a, 0xfb, 0xf1, 0x56, 0xde, 0xfe, 0x41, 0x83, 0xaa, 0x4c, 0x03, 0xb6, 0xe3, 0x79, 0x7d, 0xca, - 0x44, 0xac, 0x40, 0x31, 0x7e, 0x36, 0x64, 0x45, 0xd9, 0x36, 0x5f, 0xff, 0x6c, 0x10, 0x8f, 0x30, - 0x15, 0x8f, 0x92, 0xb5, 0xd7, 0xa1, 0xa8, 0xaa, 0x05, 0x5d, 0x85, 0x3c, 0x3f, 0xdd, 0xe8, 0xbd, - 0xfd, 0x8f, 0xc7, 0x1b, 0x71, 0x65, 0x63, 0xfb, 0x14, 0x74, 0x6e, 0xe2, 0x53, 0x91, 0x3f, 0x78, - 0xb4, 0x54, 0xc9, 0x73, 0x00, 0xd5, 0x21, 0x7f, 0x80, 0xdd, 0x61, 0x34, 0xbd, 0x94, 0x25, 0x82, - 0xa4, 0x87, 0xc7, 0x1a, 0x14, 0xd4, 0xf5, 0xbe, 0x0c, 0xa5, 0x01, 0xfe, 0xda, 0xa7, 0xe6, 0x01, - 0x76, 0x65, 0x3e, 0xa6, 0x64, 0x3e, 0xf2, 0x9b, 0xdc, 0x60, 0x14, 0x85, 0xfd, 0x2e, 0x76, 0x05, - 0xd7, 0xf1, 0x24, 0x37, 0x7b, 0x8c, 0xcb, 0x0d, 0x46, 0x51, 0xd8, 0x39, 0xb7, 0x0e, 0xf9, 0x00, - 0x33, 0x6b, 0x7f, 0x6c, 0x3a, 0x46, 0x10, 0x9f, 0xc2, 0x43, 0x2f, 0x64, 0x62, 0xc4, 0xa6, 0xa7, - 0x62, 0x8c, 0x46, 0x71, 0x5e, 0x76, 0xa1, 0x9c, 0x7a, 0xf9, 0xa2, 0x69, 0x80, 0xbb, 0xdb, 0xbd, - 0x75, 0xc3, 0xbc, 0x79, 0x67, 0x63, 0xa3, 0x92, 0x41, 0x08, 0xa6, 0xa3, 0xef, 0xee, 0xd6, 0xea, - 0xf6, 0x66, 0x77, 0xeb, 0x56, 0x25, 0x9b, 0x60, 0xdb, 0x77, 0x7a, 0xb7, 0xb6, 0x39, 0x96, 0x4b, - 0xb0, 0xb5, 0xf5, 0xcd, 0xed, 0x1e, 0xc7, 0x74, 0x54, 0x86, 0xc2, 0xc6, 0xfa, 0x8a, 0xb1, 0xb5, - 0x6e, 0x54, 0xb4, 0xba, 0xfe, 0xdd, 0x2f, 0x8d, 0x4c, 0xe7, 0xd2, 0x93, 0xdf, 0x1a, 0x99, 0x27, - 0x47, 0x0d, 0xed, 0xe9, 0x51, 0x43, 0x7b, 0x76, 0xd4, 0xd0, 0x9e, 0x1f, 0x35, 0xb4, 0x1f, 0x5f, - 0x34, 0x32, 0x4f, 0x5f, 0x34, 0x32, 0xcf, 0x5e, 0x34, 0x32, 0x5f, 0x16, 0xe4, 0xd1, 0xfc, 0x15, - 0x00, 0x00, 0xff, 0xff, 0x28, 0x2a, 0x5e, 0xcf, 0xb3, 0x0d, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_4d08fb4df4010e8b) } + +var fileDescriptor_metadata_4d08fb4df4010e8b = []byte{ + // 1385 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcf, 0x6f, 0x1b, 0x45, + 0x14, 0xf6, 0xc6, 0xeb, 0x78, 0xfd, 0x9c, 0x1f, 0xf6, 0x08, 0x8a, 0x65, 0x84, 0x9d, 0x2e, 0xad, + 0x68, 0x0b, 0x4a, 0xd2, 0xa0, 0xa8, 0x6a, 0xa0, 0x40, 0xdc, 0xa4, 0x95, 0xdb, 0xfc, 0x62, 0xe3, + 0x16, 0x89, 0xcb, 0x6a, 0xb2, 0x3b, 0x38, 0x4b, 0xd7, 0xbb, 0xee, 0xec, 0x38, 0xad, 0xef, 0x48, + 0x20, 0x21, 0x24, 0x2e, 0x48, 0x1c, 0x2b, 0xa1, 0x9e, 0xb9, 0xf2, 0x27, 0xf4, 0xd8, 0x63, 0x4f, + 0x51, 0x49, 0x2f, 0x9c, 0x39, 0xf6, 0x80, 0xd0, 0xcc, 0xce, 0xec, 0xae, 0x53, 0x17, 0xda, 0x20, + 0x71, 0x9b, 0xfd, 0xde, 0xf7, 0xbd, 0x7d, 0xf3, 0xe6, 0xcd, 0x7b, 0x03, 0xa7, 0x68, 0x88, 0x9d, + 0xfd, 0xfe, 0xde, 0x42, 0x8f, 0x30, 0xec, 0x62, 0x86, 0xe7, 0xfb, 0x34, 0x64, 0x21, 0xaa, 0x3a, + 0xa1, 0x73, 0x47, 0xd8, 0xe6, 0x25, 0xa3, 0x5e, 0x1f, 0x30, 0xcf, 0x5f, 0x18, 0x04, 0x94, 0x44, + 0xa1, 0x7f, 0x40, 0x5c, 0x1b, 0xbb, 0x2e, 0x8d, 0xe9, 0xf5, 0x9a, 0xb0, 0xed, 0xfb, 0xce, 0x02, + 0xf3, 0x7a, 0x24, 0x62, 0xb8, 0xd7, 0x97, 0x96, 0x37, 0xba, 0x61, 0x37, 0x14, 0xcb, 0x05, 0xbe, + 0x8a, 0x51, 0xf3, 0x53, 0x80, 0x55, 0xc6, 0xa8, 0xb7, 0x37, 0x60, 0x24, 0x42, 0xef, 0x43, 0x01, + 0x33, 0x46, 0xa3, 0x9a, 0x36, 0x97, 0x3f, 0x57, 0x6a, 0xbd, 0xf9, 0xe7, 0x61, 0xb3, 0x3a, 0xc4, + 0x3d, 0x7f, 0xc5, 0x14, 0xf0, 0x07, 0x5f, 0xf9, 0xe1, 0x3d, 0xd3, 0x8a, 0x39, 0x2b, 0xfa, 0xcf, + 0x0f, 0x9a, 0x39, 0xf3, 0x5b, 0x0d, 0xaa, 0x16, 0xe9, 0xfb, 0x9e, 0x83, 0x99, 0x17, 0x06, 0x1d, + 0x4c, 0xbb, 0x84, 0xa1, 0x8b, 0x50, 0x0c, 0x42, 0x97, 0xd8, 0x9e, 0x5b, 0xd3, 0xe6, 0xb4, 0x73, + 0x85, 0x56, 0xed, 0xd1, 0x61, 0x33, 0x77, 0x74, 0xd8, 0x9c, 0xdc, 0x0a, 0x5d, 0xd2, 0x5e, 0x7b, + 0x9e, 0xac, 0xac, 0x49, 0x4e, 0x6c, 0xbb, 0x68, 0x19, 0x8c, 0x88, 0x85, 0x54, 0x68, 0x26, 0x84, + 0xa6, 0x2e, 0x35, 0xc5, 0x5d, 0x8e, 0x0b, 0x91, 0x5a, 0x5a, 0x45, 0xc1, 0x6d, 0xbb, 0x2b, 0x06, + 0x8f, 0xe2, 0x8f, 0x07, 0x4d, 0xcd, 0xfc, 0x2b, 0x8d, 0x64, 0x8d, 0x44, 0x0e, 0xf5, 0xfa, 0x2c, + 0xa4, 0xff, 0x5f, 0x24, 0xe8, 0x0a, 0x00, 0x8d, 0x7f, 0xcf, 0x85, 0x79, 0x21, 0x6c, 0x48, 0x61, + 0x49, 0x06, 0x26, 0xa4, 0xe9, 0x87, 0x55, 0x92, 0x8a, 0xb6, 0x8b, 0x96, 0x40, 0x67, 0xc3, 0x3e, + 0xa9, 0xe9, 0x73, 0xda, 0xb9, 0x99, 0xa5, 0xc6, 0xfc, 0x0b, 0xe7, 0x3e, 0x2f, 0x65, 0x9d, 0x61, + 0x9f, 0x58, 0x82, 0xbb, 0x32, 0xc5, 0x37, 0xff, 0xdb, 0x83, 0xa6, 0x26, 0x12, 0xf0, 0xbd, 0x06, + 0x53, 0xca, 0xb5, 0x4b, 0x02, 0xc6, 0x37, 0x42, 0x71, 0xd0, 0x4d, 0x36, 0x9f, 0x4f, 0x37, 0x62, + 0x71, 0x3c, 0xde, 0x88, 0x5c, 0x5a, 0x45, 0xc1, 0x6d, 0xbb, 0x68, 0x0d, 0x8a, 0x32, 0x2c, 0xb1, + 0xfd, 0xf2, 0xd2, 0x99, 0x97, 0x07, 0x93, 0x66, 0xba, 0xa5, 0x73, 0xdf, 0x96, 0x92, 0x9a, 0xbf, + 0xe6, 0x61, 0x56, 0xb8, 0xce, 0x1c, 0xc6, 0x09, 0x03, 0x3a, 0x0b, 0xa5, 0x88, 0x61, 0xca, 0xec, + 0x3b, 0x64, 0x28, 0x42, 0x9a, 0x6a, 0x19, 0xcf, 0x0f, 0x9b, 0xba, 0x75, 0x93, 0x0c, 0x2d, 0x43, + 0x98, 0x6e, 0x92, 0x21, 0x3a, 0x0d, 0x45, 0x12, 0xb8, 0x82, 0x94, 0x3f, 0x46, 0x9a, 0x24, 0x81, + 0xcb, 0x29, 0x5f, 0x40, 0xd5, 0x0b, 0x18, 0xa1, 0x01, 0xf6, 0x6d, 0x19, 0x68, 0x54, 0xd3, 0xe7, + 0xf2, 0xaf, 0xb9, 0xc9, 0x8a, 0x72, 0x22, 0x09, 0x11, 0xba, 0x01, 0xb3, 0x01, 0xb9, 0xcf, 0xec, + 0x4c, 0x05, 0x14, 0x44, 0x05, 0x98, 0x72, 0x83, 0xd3, 0x5b, 0xe4, 0x3e, 0x7b, 0x49, 0x15, 0x4c, + 0x07, 0x19, 0x9b, 0x8b, 0xce, 0x00, 0x74, 0x49, 0x40, 0xa8, 0xb8, 0x50, 0xb5, 0x49, 0x91, 0xa7, + 0xf8, 0xbf, 0x19, 0x1c, 0x7d, 0x0c, 0x10, 0x31, 0xcf, 0xb9, 0x33, 0xb4, 0xf7, 0x3c, 0x56, 0x2b, + 0x8a, 0x83, 0x7a, 0x27, 0xb3, 0x07, 0xde, 0x08, 0xe6, 0xf7, 0x7d, 0x67, 0xbe, 0xa3, 0x1a, 0x81, + 0x55, 0x8a, 0x05, 0x2d, 0x8f, 0x8d, 0x56, 0xce, 0x0d, 0xdd, 0x30, 0x2a, 0x25, 0xf3, 0xa1, 0x06, + 0xe5, 0x1d, 0x42, 0x1d, 0x12, 0x30, 0xcf, 0x27, 0x11, 0x3a, 0x05, 0xf9, 0xfe, 0xc5, 0x45, 0x71, + 0x50, 0x9a, 0x0c, 0x80, 0x03, 0x02, 0x5f, 0x5a, 0x16, 0x07, 0x91, 0xe2, 0x4b, 0xcb, 0x02, 0x5f, + 0x5e, 0x14, 0xb9, 0x4f, 0xf1, 0xe5, 0x98, 0x7f, 0x69, 0x59, 0x14, 0x76, 0x8a, 0x5f, 0x8a, 0xf9, + 0x97, 0x17, 0x45, 0x9e, 0x52, 0xfc, 0xf2, 0x22, 0xaa, 0x81, 0xde, 0xdf, 0xc4, 0xf7, 0xc5, 0xce, + 0x95, 0x41, 0x20, 0xb2, 0xe5, 0x3c, 0xcf, 0xc3, 0xb4, 0xb8, 0x7d, 0x57, 0x71, 0x1f, 0x3b, 0x1e, + 0x1b, 0xa2, 0x39, 0x30, 0x1c, 0xb9, 0x96, 0x75, 0x15, 0xab, 0x12, 0x14, 0x99, 0x50, 0xc2, 0x07, + 0xd8, 0xf3, 0xf1, 0x9e, 0x4f, 0x44, 0xe4, 0x8a, 0x92, 0xc2, 0xe8, 0x2c, 0x94, 0xe3, 0xea, 0x74, + 0xc2, 0x41, 0xc0, 0xe4, 0x0d, 0x96, 0x89, 0x17, 0x86, 0xab, 0x1c, 0xe7, 0x34, 0x9f, 0xe0, 0x48, + 0xd1, 0xf4, 0x2c, 0x4d, 0x18, 0x62, 0xda, 0x22, 0x54, 0xef, 0x51, 0x8f, 0x91, 0xc8, 0xee, 0x13, + 0x6a, 0x47, 0xc4, 0x09, 0x03, 0x77, 0x64, 0xaf, 0xb3, 0xb1, 0x79, 0x87, 0xd0, 0x5d, 0x61, 0x44, + 0x3b, 0x50, 0xdd, 0x1b, 0x2a, 0x81, 0xba, 0x81, 0x93, 0xe2, 0x60, 0xc7, 0xb5, 0x83, 0xcc, 0x51, + 0x29, 0x8f, 0x42, 0xbe, 0x43, 0xa8, 0xac, 0x25, 0x64, 0x01, 0xca, 0xc4, 0xa0, 0x5c, 0x16, 0x5f, + 0xc3, 0x65, 0x25, 0x09, 0x52, 0xf9, 0xac, 0x81, 0x3e, 0x88, 0x88, 0x5b, 0x33, 0x32, 0x49, 0x14, + 0x08, 0x3a, 0x0f, 0xd3, 0x7e, 0xd8, 0xf5, 0x1c, 0xec, 0xdb, 0x22, 0x90, 0x5a, 0x29, 0x43, 0x99, + 0x92, 0xa6, 0x16, 0xb7, 0xa0, 0x25, 0x40, 0x77, 0x07, 0x84, 0x7a, 0xa3, 0xd9, 0x81, 0x4c, 0x76, + 0x2a, 0xd2, 0x9e, 0xa4, 0x47, 0x1e, 0xfe, 0x53, 0x1d, 0x66, 0x78, 0xbf, 0xfe, 0x6f, 0x2d, 0xfe, + 0x13, 0x28, 0xf2, 0xa1, 0x49, 0xa2, 0x48, 0xb6, 0xb8, 0xc6, 0xf1, 0x9b, 0x73, 0x2b, 0x19, 0xaf, + 0xab, 0xae, 0x9b, 0x34, 0x37, 0x29, 0x42, 0x97, 0xd5, 0xa0, 0xcc, 0xbf, 0x70, 0xef, 0x54, 0x2e, + 0xd3, 0xb1, 0x2a, 0xc5, 0xb1, 0x02, 0x5d, 0x01, 0xc3, 0x0f, 0x1d, 0xec, 0xf3, 0x5a, 0xd5, 0x85, + 0xfa, 0xed, 0x31, 0xea, 0x0d, 0x49, 0x51, 0x85, 0xac, 0x24, 0xe8, 0x1a, 0x4c, 0xef, 0x12, 0x7a, + 0x40, 0xe8, 0x6d, 0x42, 0x23, 0xde, 0x1f, 0x0a, 0xc2, 0x47, 0x7d, 0x8c, 0x0f, 0xc9, 0x90, 0x2e, + 0x46, 0x65, 0xe8, 0x34, 0x94, 0xf6, 0x06, 0x9e, 0xef, 0xda, 0x0c, 0x77, 0x45, 0x91, 0x95, 0xd4, + 0xaf, 0x04, 0xdc, 0xc1, 0x5d, 0xf4, 0x2e, 0xef, 0x30, 0x98, 0x32, 0xfe, 0xc2, 0x88, 0x3b, 0x4c, + 0x72, 0x69, 0x24, 0xbe, 0xca, 0xd0, 0x2e, 0x54, 0x54, 0x6c, 0xb6, 0x4a, 0xa9, 0x21, 0x1a, 0xaa, + 0xf9, 0x0f, 0xdb, 0x5a, 0x8d, 0x99, 0xaa, 0x6e, 0xfd, 0x51, 0x18, 0xbd, 0x07, 0x53, 0x8e, 0x3f, + 0x88, 0x18, 0xa1, 0x76, 0x80, 0x7b, 0x44, 0x14, 0x92, 0x8a, 0xaf, 0x2c, 0x2d, 0x5b, 0xb8, 0x47, + 0xd0, 0x2e, 0x94, 0xa3, 0xbb, 0x7e, 0xf2, 0x63, 0x78, 0xa5, 0xb3, 0x44, 0xb2, 0x3c, 0x60, 0xf7, + 0xf3, 0x0d, 0xf9, 0x47, 0x0b, 0xa2, 0xbb, 0xbe, 0x5c, 0x9b, 0x3f, 0x69, 0x30, 0x7b, 0x2c, 0xd0, + 0x6c, 0xc1, 0x68, 0x27, 0x29, 0x98, 0x16, 0xbf, 0x1b, 0x32, 0x4d, 0xcc, 0x23, 0x54, 0x96, 0xdd, + 0x5b, 0x63, 0x72, 0xd4, 0xf1, 0x08, 0x4d, 0x2f, 0x4d, 0xac, 0xe1, 0x98, 0xf9, 0xcd, 0x04, 0xcc, + 0x8a, 0xbe, 0x37, 0x3a, 0x51, 0x93, 0xb7, 0x8a, 0xf6, 0xea, 0x6f, 0x95, 0xa4, 0x7e, 0x27, 0x5e, + 0xbb, 0x7e, 0x3f, 0x02, 0x9d, 0x5f, 0x22, 0x59, 0xf9, 0xa7, 0xc7, 0x28, 0x47, 0xaf, 0xa7, 0x6a, + 0x11, 0x5c, 0x84, 0x5a, 0x99, 0x46, 0x1d, 0x17, 0xff, 0xdc, 0x18, 0x07, 0x23, 0xcd, 0xfd, 0x78, + 0x2b, 0x37, 0x7f, 0xd0, 0xa0, 0x2a, 0xd3, 0x80, 0xdd, 0x64, 0x00, 0x9f, 0x30, 0x11, 0xab, 0x60, + 0x24, 0xef, 0x80, 0x09, 0x51, 0xb6, 0xcd, 0x97, 0xbf, 0x03, 0xc4, 0xab, 0x4a, 0xc5, 0xa3, 0x64, + 0xe6, 0x3a, 0x18, 0xaa, 0x5a, 0xd0, 0x87, 0x50, 0xe0, 0xa7, 0x1b, 0x3f, 0xa0, 0xff, 0xf5, 0x78, + 0x63, 0xae, 0x6c, 0x6c, 0x9f, 0x81, 0xce, 0x4d, 0x7c, 0x2a, 0xf2, 0x17, 0x8c, 0x96, 0x29, 0x79, + 0x0e, 0xa0, 0x3a, 0x14, 0x0e, 0xb0, 0x3f, 0x88, 0xa7, 0x97, 0xb2, 0xc4, 0x90, 0xf4, 0xf0, 0x50, + 0x83, 0xa2, 0xba, 0xde, 0x17, 0xa0, 0xd4, 0xc3, 0x5f, 0x87, 0xd4, 0x3e, 0xc0, 0xbe, 0xcc, 0xc7, + 0xb4, 0xcc, 0x47, 0x61, 0x93, 0x1b, 0x2c, 0x43, 0xd8, 0x6f, 0x63, 0x5f, 0x70, 0xbd, 0x40, 0x72, + 0x27, 0x8e, 0x71, 0xb9, 0xc1, 0x32, 0x84, 0x9d, 0x73, 0xeb, 0x50, 0xe8, 0x63, 0xe6, 0xec, 0x8f, + 0x4c, 0xc7, 0x18, 0xe2, 0x53, 0x78, 0x10, 0x44, 0x4c, 0x8c, 0xd8, 0xec, 0x54, 0x4c, 0xd0, 0x38, + 0xce, 0x0b, 0x3e, 0x94, 0x33, 0x4f, 0x59, 0x34, 0x03, 0x70, 0x7b, 0xbb, 0xb3, 0x6e, 0xd9, 0xd7, + 0x6e, 0x6d, 0x6c, 0x54, 0x72, 0x08, 0xc1, 0x4c, 0xfc, 0xdd, 0xde, 0xba, 0xba, 0xbd, 0xd9, 0xde, + 0xba, 0x5e, 0x99, 0x48, 0xb1, 0xed, 0x5b, 0x9d, 0xeb, 0xdb, 0x1c, 0xcb, 0xa7, 0xd8, 0xda, 0xfa, + 0xe6, 0x76, 0x87, 0x63, 0x3a, 0x2a, 0x43, 0x71, 0x63, 0x7d, 0xd5, 0xda, 0x5a, 0xb7, 0x2a, 0x5a, + 0x5d, 0xff, 0xee, 0x97, 0x46, 0xae, 0x75, 0xfe, 0xd1, 0xef, 0x8d, 0xdc, 0xa3, 0xa3, 0x86, 0xf6, + 0xf8, 0xa8, 0xa1, 0x3d, 0x39, 0x6a, 0x68, 0x4f, 0x8f, 0x1a, 0xda, 0x8f, 0xcf, 0x1a, 0xb9, 0xc7, + 0xcf, 0x1a, 0xb9, 0x27, 0xcf, 0x1a, 0xb9, 0x2f, 0x8b, 0xf2, 0x68, 0xfe, 0x0e, 0x00, 0x00, 0xff, + 0xff, 0x6e, 0xce, 0xc8, 0xc3, 0x84, 0x0d, 0x00, 0x00, } diff --git a/pkg/roachpb/metadata.proto b/pkg/roachpb/metadata.proto index 37d38389a731..5dc565f98598 100644 --- a/pkg/roachpb/metadata.proto +++ b/pkg/roachpb/metadata.proto @@ -210,16 +210,6 @@ message RangeDescriptor { // the replica it's supposed to update, and no splits and merges have taken // place at all). // - // Note also that when the generation counter was first introduced, it only - // ever incremented (by one) the generation of the left hand side on merges - // and splits, so the above overlap arguments only hold if we know that the - // descriptors involved never used that code. Generations were first introduced - // in the 19.1 release, though, the behavior described here was only introduced - // in a late release candidate. If we allow such a release candidate cluster - // to transition into the final 19.1 release, we will need to introduce - // additional state to mark descriptors as obeying the new rules. If we don't, - // then we are free to assume that the semantics always hold. - // // For a third note, observe that the generational semantics above may // possibly allow range merges without colocation, at least in the sense that // the counter examples in #28071 are defused. This is because the @@ -227,16 +217,6 @@ message RangeDescriptor { // replica is gc'able or not. If it is not gc'able, then by definition the // replica applying the merge is. optional int64 generation = 6 [(gogoproto.nullable) = false]; - // generation_comparable is additional state to mark descriptors as obeying - // the new generation rules. We can determine the causality of two range - // descriptors if they overlap and both their generation_comparable are set. - // If one or both of their generation_comparable is not set, we cannot - // determine which range descriptor is newer. This is needed because older - // versions of Cockroach did not set the generation in some cases where it is - // currently set. Code that relies on a total order of range descriptor - // generations can determine when this order is ensured from this state. See - // #36654 for more details. - optional bool generation_comparable = 8; // The presence of the sticky_bit indicates that the range should not be // automatically merged by the merge queue with the range to its left. It is // set during a split operation and unset during an unsplit operation. Note @@ -253,6 +233,8 @@ message RangeDescriptor { // queue is enabled. With sticky_bit, users can manually split ranges without // diabling the merge queue. optional util.hlc.Timestamp sticky_bit = 7; + + reserved 8; } // Percentiles contains a handful of hard-coded percentiles meant to summarize