-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
38302: storage: CPut bytes instead of a proto when updating RangeDescriptors r=tbg a=danhhz In #38147, a new non-nullable field was added to the ReplicaDescriptor proto that is serialized inside RangeDescriptors. RangeDescriptors are updated using CPut to detect races. This means that if a RangeDescriptor had been written by an old version of cockroach and we then attempt to update it, the CPut will fail because the encoded version of it is different (non-nullable proto2 fields are always included). A similar issue was introduced in #38004 which made the StickyBit field on RangeDescriptor non-nullable. We could keep the fields as nullable, but this has allocation costs (and is also more annoying to work with). Worse, the proto spec is pretty explicit about not relying on serialization of a given message to always produce exactly the same bytes: From https://developers.google.com/protocol-buffers/docs/encoding#implications Do not assume the byte output of a serialized message is stable. So instead, we've decided to stop CPut-ing protos and to change the interface of CPut to take the expected value as bytes. To CPut a proto, the encoded value will be read from kv, passed around with the decoded struct, and used in the eventual CPut. This work is upcoming, but the above two PRs are real breakages, so this commit fixes those first. Neither of these PRs made the last alpha so no release note. Touches #38308 Closes #38183 Release note: None Co-authored-by: Daniel Harrison <[email protected]>
- Loading branch information
Showing
3 changed files
with
186 additions
and
39 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,93 @@ | ||
// Copyright 2019 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License included | ||
// in the file licenses/BSL.txt and at www.mariadb.com/bsl11. | ||
// | ||
// Change Date: 2022-10-01 | ||
// | ||
// On the date above, in accordance with the Business Source License, use | ||
// of this software will be governed by the Apache License, Version 2.0, | ||
// included in the file licenses/APL.txt and at | ||
// https://www.apache.org/licenses/LICENSE-2.0 | ||
|
||
package storage | ||
|
||
import ( | ||
"context" | ||
"encoding/binary" | ||
"testing" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/base" | ||
"github.com/cockroachdb/cockroach/pkg/keys" | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils" | ||
"github.com/cockroachdb/cockroach/pkg/util/hlc" | ||
"github.com/cockroachdb/cockroach/pkg/util/leaktest" | ||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
// Regression test for #38308. Summary: a non-nullable field was added to | ||
// RangeDescriptor which broke splits, merges, and replica changes if the | ||
// cluster had been upgraded from a previous version of cockroach. | ||
func TestRangeDescriptorUpdateProtoChangedAcrossVersions(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
|
||
// Control our own split destiny. | ||
args := base.TestServerArgs{Knobs: base.TestingKnobs{Store: &StoreTestingKnobs{ | ||
DisableSplitQueue: true, | ||
DisableMergeQueue: true, | ||
}}} | ||
ctx := context.Background() | ||
s, _, kvDB := serverutils.StartServer(t, args) | ||
defer s.Stopper().Stop(ctx) | ||
|
||
bKey := roachpb.Key("b") | ||
if err := kvDB.AdminSplit(ctx, bKey, bKey, hlc.MaxTimestamp /* expirationTime */); err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
// protoVarintField returns an encoded proto field of type varint with the | ||
// given id. | ||
protoVarintField := func(fieldID int) []byte { | ||
var scratch [binary.MaxVarintLen64]byte | ||
const typ = 0 // varint type field | ||
tag := uint64(fieldID<<3) | typ | ||
tagLen := binary.PutUvarint(scratch[:], tag) | ||
// A proto message is a series of <tag><data> where <tag> is a varint | ||
// including the field id and the data type and <data> depends on the type. | ||
buf := append([]byte(nil), scratch[:tagLen]...) | ||
// The test doesn't care what we use for the field data, so use the tag | ||
// since the data is a varint and it's already an encoded varint. | ||
buf = append(buf, scratch[:tagLen]...) | ||
return buf | ||
} | ||
|
||
// Update the serialized RangeDescriptor proto for the b to max range to have | ||
// an unknown proto field. Previously, this would break splits, merges, | ||
// replica changes. The real regression was a missing field, but an extra | ||
// unknown field tests the same thing. | ||
{ | ||
bDescKey := keys.RangeDescriptorKey(roachpb.RKey(bKey)) | ||
bDescKV, err := kvDB.Get(ctx, bDescKey) | ||
require.NoError(t, err) | ||
require.NotNil(t, bDescKV.Value, `could not find "b" descriptor`) | ||
|
||
// Update the serialized proto with a new field we don't know about. The | ||
// proto encoding is just a series of these, so we can do this simply by | ||
// appending it. | ||
newBDescBytes, err := bDescKV.Value.GetBytes() | ||
require.NoError(t, err) | ||
newBDescBytes = append(newBDescBytes, protoVarintField(9999)...) | ||
|
||
newBDescValue := roachpb.MakeValueFromBytes(newBDescBytes) | ||
require.NoError(t, kvDB.Put(ctx, bDescKey, &newBDescValue)) | ||
} | ||
|
||
// Verify that splits still work. We could also do a similar thing to test | ||
// merges and replica changes, but they all go through updateRangeDescriptor | ||
// so it's unnecessary. | ||
cKey := roachpb.Key("c") | ||
if err := kvDB.AdminSplit(ctx, cKey, cKey, hlc.MaxTimestamp /* expirationTime */); err != nil { | ||
t.Fatal(err) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters