Skip to content

Commit

Permalink
Merge #95513
Browse files Browse the repository at this point in the history
95513: kvstorage: complete RaftReplicaID migration r=pavelkalinnikov a=tbg

As of v22.1[^1], we always write the RaftReplicaID when creating a
Replica or updating a snapshot. However, since this is
persisted state that could've originated in older versions and not
updated yet, we couldn't rely on a persisted ReplicaID yet.

This commit adds code to the `(*Store).Start` boot sequence that

- persists a RaftReplicaID for all initialized replicas (using the
  ReplicaID from the descriptor)
- deletes all uninitialized replicas lacking RaftReplicaID (since we don't know their
  ReplicaID at this point).

The second item in theory violates Raft invariants, as uninitialized
Replicas are allowed to vote (though they then cannot accept log
entries). So in theory:

- an uninitialized replica casts a decisive vote for a leader
- it restarts
- code in this commit removes the uninited replica (and its vote)
- delayed MsgVote from another leader arrives
- it casts another vote for the same term for a dueling leader
- now there are two leaders in the same term.

The above in addition presupposes that the two leaders cannot
communicate with each other. Also, even if that is the case, since the
two leaders cannot append to the uninitialized replica (it doesn't
accept entries), we also need additional voters to return at the exact
right time.

Since an uninitialized replica without RaftReplicaID in is necessarily
at least one release old, this is exceedingly unlikely and we will
live with this theoretical risk.

This PR also adds a first stab at a datadriven test harness for
`kvstorage` which is likely to be of use for #93247.

[^1]: #75761

Epic: [CRDB-220](https://cockroachlabs.atlassian.net/browse/CRDB-220)
Release note: None

Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
  • Loading branch information
craig[bot] and tbg committed Feb 6, 2023
2 parents 8e24570 + 672e8b1 commit 622956b
Show file tree
Hide file tree
Showing 9 changed files with 468 additions and 106 deletions.
18 changes: 16 additions & 2 deletions pkg/kv/kvserver/kvstorage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,27 +16,41 @@ go_library(
"//pkg/kv/kvserver/logstore",
"//pkg/roachpb",
"//pkg/storage",
"//pkg/util/buildutil",
"//pkg/util/hlc",
"//pkg/util/iterutil",
"//pkg/util/log",
"//pkg/util/protoutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@io_etcd_go_raft_v3//raftpb",
],
)

go_test(
name = "kvstorage_test",
srcs = ["cluster_version_test.go"],
srcs = [
"cluster_version_test.go",
"datadriven_test.go",
],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
embed = [":kvstorage"],
deps = [
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/kv/kvserver/logstore",
"//pkg/roachpb",
"//pkg/storage",
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/tracing",
"//pkg/util/uuid",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_stretchr_testify//require",
"@io_etcd_go_raft_v3//raftpb",
],
)

Expand Down
184 changes: 184 additions & 0 deletions pkg/kv/kvserver/kvstorage/datadriven_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,184 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, 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.

package kvstorage

import (
"context"
"fmt"
"regexp"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/datadriven"
"github.com/stretchr/testify/require"
"go.etcd.io/raft/v3/raftpb"
)

type env struct {
eng storage.Engine
tr *tracing.Tracer
}

func newEnv(t *testing.T) *env {
ctx := context.Background()
eng := storage.NewDefaultInMemForTesting()
// TODO(tbg): ideally this would do full bootstrap, which requires
// moving a lot more code from kvserver. But then we could unit test
// all of it with the datadriven harness!
require.NoError(t, WriteClusterVersion(ctx, eng, clusterversion.TestingClusterVersion))
require.NoError(t, InitEngine(ctx, eng, roachpb.StoreIdent{
ClusterID: uuid.FastMakeV4(),
NodeID: 1,
StoreID: 1,
}))
tr := tracing.NewTracer()
tr.SetRedactable(true)
return &env{
eng: eng,
tr: tr,
}
}

func (e *env) close() {
e.eng.Close()
e.tr.Close()
}

func (e *env) handleNewReplica(
t *testing.T,
ctx context.Context,
id storage.FullReplicaID,
skipRaftReplicaID bool,
k, ek roachpb.RKey,
) *roachpb.RangeDescriptor {
sl := logstore.NewStateLoader(id.RangeID)
require.NoError(t, sl.SetHardState(ctx, e.eng, raftpb.HardState{}))
if !skipRaftReplicaID && id.ReplicaID != 0 {
require.NoError(t, sl.SetRaftReplicaID(ctx, e.eng, id.ReplicaID))
}
if len(ek) == 0 {
return nil
}
desc := &roachpb.RangeDescriptor{
RangeID: id.RangeID,
StartKey: keys.MustAddr(roachpb.Key(k)),
EndKey: keys.MustAddr(roachpb.Key(ek)),
InternalReplicas: []roachpb.ReplicaDescriptor{{
NodeID: 1,
StoreID: 1,
ReplicaID: id.ReplicaID,
}},
NextReplicaID: id.ReplicaID + 1,
}
var v roachpb.Value
require.NoError(t, v.SetProto(desc))
ts := hlc.Timestamp{WallTime: 123}
require.NoError(t, e.eng.PutMVCC(storage.MVCCKey{
Key: keys.RangeDescriptorKey(desc.StartKey),
Timestamp: ts,
}, storage.MVCCValue{Value: v}))
return desc
}

func TestDataDriven(t *testing.T) {
defer leaktest.AfterTest(t)()

reStripFileLinePrefix := regexp.MustCompile(`^[^ ]+ `)

datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) {
e := newEnv(t)
defer e.close()
datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) (output string) {
ctx, finishAndGet := tracing.ContextWithRecordingSpan(context.Background(), e.tr, path)
// This method prints all output to `buf`.
var buf strings.Builder
var printTrace bool // if true, trace printed to buf on return
if d.HasArg("trace") {
d.ScanArgs(t, "trace", &printTrace)
}

defer func() {
if r := recover(); r != nil {
fmt.Fprintln(&buf, r)
}
rec := finishAndGet()[0]
for _, l := range rec.Logs {
if !printTrace || !strings.Contains(string(l.Message), "kvstorage") {
continue
}

fmt.Fprintln(&buf, reStripFileLinePrefix.ReplaceAllString(string(l.Message), ``))
}
if buf.Len() == 0 {
fmt.Fprintln(&buf, "ok")
}
output = buf.String()
}()

switch d.Cmd {
case "new-replica":
var rangeID int
d.ScanArgs(t, "range-id", &rangeID)
var replicaID int
if d.HasArg("replica-id") { // optional to allow making incomplete state
d.ScanArgs(t, "replica-id", &replicaID)
}
var k string
if d.HasArg("k") {
d.ScanArgs(t, "k", &k)
}
var ek string
if d.HasArg("ek") {
d.ScanArgs(t, "ek", &ek)
}
var skipRaftReplicaID bool
if d.HasArg("skip-raft-replica-id") {
d.ScanArgs(t, "skip-raft-replica-id", &skipRaftReplicaID)
}
if desc := e.handleNewReplica(t, ctx,
storage.FullReplicaID{RangeID: roachpb.RangeID(rangeID), ReplicaID: roachpb.ReplicaID(replicaID)},
skipRaftReplicaID, keys.MustAddr(roachpb.Key(k)), keys.MustAddr(roachpb.Key(ek)),
); desc != nil {
fmt.Fprintln(&buf, desc)
}
case "load-and-reconcile":
replicas, err := LoadAndReconcileReplicas(ctx, e.eng)
if err != nil {
fmt.Fprintln(&buf, err)
break
}
for _, repl := range replicas {
fmt.Fprintf(&buf, "%s: ", repl.ID())
if desc := repl.Desc; desc != nil {
fmt.Fprint(&buf, desc)
} else {
fmt.Fprintf(&buf, "uninitialized")
}
fmt.Fprintln(&buf)
}
default:
t.Fatalf("unknown command %s", d.Cmd)
}
return "" // defer will do it
})
})

}
Loading

0 comments on commit 622956b

Please sign in to comment.