diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index f2d6b7c86a6c..deec8244a7bf 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -108,12 +108,12 @@ var ( // 2. Unreplicated Range-ID keys // - // localRangeIDUnreplicatedInfix is the post-Range ID specifier for all + // LocalRangeIDUnreplicatedInfix is the post-Range ID specifier for all // per-range data that is not fully Raft replicated. By appending this // after the Range ID, these keys will be sorted directly after the local // replicated keys for the same Range ID, so they can be manipulated either // together or individually in a single scan. - localRangeIDUnreplicatedInfix = []byte("u") + LocalRangeIDUnreplicatedInfix = []byte("u") // LocalRangeTombstoneSuffix is the suffix for the range tombstone. // // NB: This suffix was originally named LocalRaftTombstoneSuffix, which is diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 221c59e68040..de9b897ff777 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -299,7 +299,7 @@ func RangeVersionKey(rangeID roachpb.RangeID) roachpb.Key { // MakeRangeIDUnreplicatedPrefix creates a range-local key prefix from // rangeID for all unreplicated data. func MakeRangeIDUnreplicatedPrefix(rangeID roachpb.RangeID) roachpb.Key { - return makePrefixWithRangeID(LocalRangeIDPrefix, rangeID, localRangeIDUnreplicatedInfix) + return makePrefixWithRangeID(LocalRangeIDPrefix, rangeID, LocalRangeIDUnreplicatedInfix) } // makeRangeIDUnreplicatedKey creates a range-local unreplicated key based @@ -994,7 +994,7 @@ func (b RangeIDPrefixBuf) replicatedPrefix() roachpb.Key { } func (b RangeIDPrefixBuf) unreplicatedPrefix() roachpb.Key { - return append(roachpb.Key(b), localRangeIDUnreplicatedInfix...) + return append(roachpb.Key(b), LocalRangeIDUnreplicatedInfix...) } // AbortSpanKey returns a range-local key by Range ID for an AbortSpan diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index e113979a5a97..728e4808135d 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -351,7 +351,7 @@ func localRangeIDKeyParse(input string) (remainder string, key roachpb.Key) { infix, input = mustShift(input) var replicated bool switch { - case bytes.Equal(localRangeIDUnreplicatedInfix, []byte(infix)): + case bytes.Equal(LocalRangeIDUnreplicatedInfix, []byte(infix)): case bytes.Equal(LocalRangeIDReplicatedInfix, []byte(infix)): replicated = true default: diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 9f129b8189ce..5757f16143b1 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -71,6 +71,7 @@ go_library( "replica_sst_snapshot_storage.go", "replica_tscache.go", "replica_write.go", + "replicas_storage.go", "replicate_queue.go", "scanner.go", "scheduler.go", @@ -294,6 +295,7 @@ go_test( "replica_sst_snapshot_storage_test.go", "replica_test.go", "replica_tscache_test.go", + "replicas_storage_test.go", "replicate_queue_test.go", "replicate_test.go", "reset_quorum_test.go", @@ -436,6 +438,7 @@ go_test( "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_logtags//:logtags", "@com_github_cockroachdb_pebble//:pebble", + "@com_github_cockroachdb_pebble//vfs", "@com_github_cockroachdb_redact//:redact", "@com_github_gogo_protobuf//proto", "@com_github_google_btree//:btree", diff --git a/pkg/kv/kvserver/replica_sst_snapshot_storage.go b/pkg/kv/kvserver/replica_sst_snapshot_storage.go index 709439009452..0314ec8a6b6d 100644 --- a/pkg/kv/kvserver/replica_sst_snapshot_storage.go +++ b/pkg/kv/kvserver/replica_sst_snapshot_storage.go @@ -143,15 +143,24 @@ func (s *SSTSnapshotStorageScratch) NewFile( // the provided SST when it is finished using it. If the provided SST is empty, // then no file will be created and nothing will be written. func (s *SSTSnapshotStorageScratch) WriteSST(ctx context.Context, data []byte) error { + _, err := s.WriteSSTAndReturnPath(ctx, data) + return err +} + +// WriteSSTAndReturnPath is similar to WriteSST, but returns the path of the +// written file. +func (s *SSTSnapshotStorageScratch) WriteSSTAndReturnPath( + ctx context.Context, data []byte, +) (path string, err error) { if s.closed { - return errors.AssertionFailedf("SSTSnapshotStorageScratch closed") + return "", errors.AssertionFailedf("SSTSnapshotStorageScratch closed") } if len(data) == 0 { - return nil + return "", nil } f, err := s.NewFile(ctx, 512<<10 /* 512 KB */) if err != nil { - return err + return "", err } defer func() { // Closing an SSTSnapshotStorageFile multiple times is idempotent. Nothing @@ -159,12 +168,15 @@ func (s *SSTSnapshotStorageScratch) WriteSST(ctx context.Context, data []byte) e _ = f.Close() }() if _, err := f.Write(data); err != nil { - return err + return "", err } if err := f.Sync(); err != nil { - return err + return "", err + } + if err := f.Close(); err != nil { + return "", err } - return f.Close() + return s.ssts[len(s.ssts)-1], nil } // SSTs returns the names of the files created. diff --git a/pkg/kv/kvserver/replicas_storage.go b/pkg/kv/kvserver/replicas_storage.go new file mode 100644 index 000000000000..6fae5c5b7de8 --- /dev/null +++ b/pkg/kv/kvserver/replicas_storage.go @@ -0,0 +1,2540 @@ +// Copyright 2021 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 kvserver + +import ( + "bytes" + "context" + "math" + "sort" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "go.etcd.io/etcd/raft/v3" + "go.etcd.io/etcd/raft/v3/raftpb" +) + +// TODO(sumeer): +// Interface and design: +// - Pick names for C1-C3 and D1-D2, to make it easier to remember what we are +// referring to in various places. +// - Consider separating RecoveryInconsistentReplica into two different states +// RecoveryRaftAndStateInconsistent, and RecoveryStateInconsistent. +// - Proof sketch. +// - Read cockroach debug check-store to see additional assertion ideas/code +// we can lift. + +// TODO(sumeer): +// Implementation: +// x Initial interface implementation without applying to state machine in +// Init. +// - Unit tests and randomized tests, including engine restarts that lose +// some state (using vfs.NewStrictMem). +// - Improved implementation: concurrency control; complete Init impl etc. +// - Ensure range keys are being properly handled, i.e., they need to be +// cleared. +// - Optimize implementation: there are places the implementation simply used +// range tombstones instead of the heuristic of writing point deletes if the +// number of points is small. +// - Benchmarks comparing single and two engine implementations: this is +// already done in +// https://github.com/cockroachdb/cockroach/issues/16624#issuecomment-1137394935 +// - Race-build dynamically asserts that SSTs or MutationBatches that are +// passed through this interface only touch the keys they are allowed to +// touch. +// - Integration (can be done incrementally). +// - Misc cleanup: +// - Merges should cleanup QueueLastProcessedKey, + +// High-level overview: +// +// ReplicasStorage provides an interface to manage the persistent state that +// includes the lifecycle of a range replica, its raft log, and the state +// machine state. The implementation(s) are expected to be a stateless wrapper +// around persistent state in the underlying engine(s) (any state they +// maintain in-memory is simply a performance optimization and always +// in-sync with the persistent state). Since this abstraction is mutating the +// same underlying engine state that was previously mutated via lower-level +// interfaces, and is not a data-structure in the usual sense, we can migrate +// callers incrementally to use this interface. That is, callers that use this +// interface, and those that use the lower-level engine interfaces can +// co-exist correctly. +// +// TODO(sumeer): this co-existence is not completely true since the following +// attempts to define an ideal interface where no sst or MutationBatch touches +// both raft engine state or state machine engine state. Which means transient +// inconsistencies can develop. We will either +// - alter this interface to be more pragmatic, i.e., be a step towards the +// ideal interface, but not the end product, once we have settled on the +// ideal interface. +// - ensure that the minimal integration step includes ReplicasStorage.Init, +// which can eliminate any inconsistencies caused by an inopportune crash. +// Hopefully, the latter is sufficient. +// +// We consider the following distinct kinds of persistent state: +// - State machine state: It contains all replicated keys: replicated range-id +// local keys, range local keys, range lock keys, global keys. NB: this +// includes the RangeAppliedState and the RangeDescriptor. +// +// - Raft state: This includes all the unreplicated range-ID local key names +// prefixed by Raft. We will loosely refer to all of these as "raft state". +// RangeLastReplicaGCTimestamp changes are ignored below, since it is +// best-effort persistent state used to pace queues, and the caller is +// allowed to mutate it out-of-band. However when deleting a replica, +// ReplicasStorage will clear that key too. RangeLastReplicaGCTimestamp is +// placed in this state category because it is not replicated state machine +// state. +// +// The interface requires that any mutation (batch or sst) only touch one of +// these kinds of state. This discipline will allow us to eventually separate +// the engines containing these two kinds of state. This interface is not +// relevant for store local keys though they will be in the latter engine. The +// interface does not allow the caller to specify whether to sync a mutation +// to the raft log or state machine state -- that decision is left to the +// implementation of ReplicasStorage (with a couple of small exceptions where +// a sync is explicitly requested, which are explained later). So even when we +// don't separate the state machine and raft engines, this abstraction forces +// us to reason more carefully about effects of crashes, and when to sync, and +// allow us to test more thoroughly. +// +// RangeTombstoneKey: This is an unreplicated key that is critical to the +// replica lifecycle. Since it is unreplicated, it is not part of the state +// machine. However, placing it in the category of "raft state" with the other +// unreplicated keys turns out to be complicated: +// (a) in various range merge situations (including replicas being subsumed +// during snapshot application) we need to atomically move the state machine +// forward for the surviving range, delete the state machine state for the +// subsumed range(s) and set the RangeTombstone. +// (b) when removing a replica due to rebalancing we need to atomically remove +// the state machine and set the RangeTombstone. +// For these reasons, we require that the RangeTombstone be in the same engine +// as the state machine state. However, it can only be mutated by +// ReplicasStorage. +// +// Note about terminology as pertaining to range-id keys: "range-id local +// keys" and "range-id keys" are the same thing, since all range-id keys are +// local. As documented in the keys package, range-id keys can be replicated +// or unreplicated. All the replicated range-id keys plus the +// RangeTombstoneKey (which is unreplicated) are referred to as "range-id +// state machine" keys. All the remaining unreplicated range-id keys belong to +// raft state and are referred to as "range-id raft" keys or simply "raft" +// keys (since all raft keys are also unreplicated range-id keys). +// +// Note that the interface is not currently designed such that raft log writes +// avoid syncing to disk as discussed in +// https://github.com/cockroachdb/cockroach/issues/17500#issuecomment-727094672 +// and followup comments on that issue. However, having a clean storage +// abstraction should be a reasonable step in that direction. +// +// ReplicasStorage does not interpret most of the data in the state machine. +// It expects mutations to that state to be provided as an opaque batch, or a +// set of files to be ingested. There are a few exceptions where it can read +// state machine state, mainly when recovering from a crash, so as to make +// changes to get to a consistent state. +// - RangeAppliedStateKey: needs to read this in order to truncate the log, +// both as part of regular log truncation (see the comment section on +// "Normal Replica Operation") and on crash recovery (see the comment +// section on "Replica Initialization" and "Crash Recovery" for details). +// - RangeDescriptorKey: needs to read this to discover the spans of +// initialized replicas (see the comment sections on "Invariants" and "Crash +// Recovery"). +// +// A corollary to this lack of interpretation is that reads of the state +// machine are not handled by this interface, though it does expose some +// metadata in case the reader want to be sure that the replica it is trying +// to read actually exists in storage. ReplicasStorage also does not offer an +// interface to construct changes to the state machine state. It applies +// changes, and requires the caller to obey some simple invariants to not +// cause inconsistencies. It is aware of the keyspace occupied by a replica +// and the difference between range-ID keys and range keys -- it needs this +// awareness to discard (parts of) replica state when replicas are moved or +// merged away. +// +// ReplicasStorage does interpret the raft state (all the unreplicated +// range-ID local key names prefixed by Raft), and the RangeTombstoneKey. This +// is necessary for it to be able to maintain invariants spanning the raft log +// and the state machine (related to raft log truncation, replica lifetime +// etc.), including reapplying raft log entries on restart, to the state +// machine. All accesses (read or write) to the raft log and RangeTombstoneKey +// must happen via ReplicasStorage. ReplicasStorage does not by itself apply +// committed raft log entries to the state machine in a running system -- this +// is because state machine application in a running system has complex +// data-structure side-effects that are outside the scope of ReplicasStorage. +// It is told by the caller to apply a committed entry, which also requires +// the caller to provide the state machine changes. ReplicasStorage does apply +// "simple" entries directly to the state machine during Init to fix any +// inconsistency of the state machine caused by durable sst ingestion and +// non-durable batch application (see ApplyCommitted* methods in the +// interface). Since these could be preceded by non-durable configuration +// changes, the notion of "simple" entries includes configuration changes, +// except for splits and merges (which we sync to ensure durability -- this is +// justified in the section below on "Implementation constraints on +// ReplicasStorage"). +// +// TODO(sumeer): +// https://github.com/etcd-io/etcd/issues/7625#issuecomment-489232411 relies +// on a correctness argument based on bounded regression of conf changes. +// Consider strengthening that correctness argument by making the committed +// index durable for a conf change before applying it. We could introduce a +// `ApplyConfChange(MutationBatch, highestRaftIndex uint64)` method, like we +// have for ingestion, and first sync the Commit state if needed. That way we +// will not have any conf change regression. +// +// ============================================================================ +// Invariants: +// +// INVARIANT (RaftAndStateConsistency): when there is any data in the state +// machine associated with a given RangeID, there is a corresponding +// internally consistent Raft state (according to etcd/raft) that is also +// consistent with the applied state on the state machine (i.e. the latter +// references a valid log position). Specifically, +// - HardState.Commit >= RangeAppliedState.RaftAppliedIndex +// - if HardState.Commit > RangeAppliedState.RaftAppliedIndex, it points to an +// entry in the raft log. +// - RaftTruncatedState.{Index,Term} must be a valid value corresponding to +// what was truncated. If there are no raft log entries, +// RaftTruncatedState.{Index,Term} must equal +// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm}. +// +// INVARIANT (StateConsistency): when there is any data in the state machine +// associated with a given RangeID, it will reflect the replicated state at +// the corresponding applied index (i.e., it materializes the replicated log +// at this index). +// Additionally, a range is first created with a RangeDescriptor present +// (external invariant) and neither the callers nor ReplicasStorage will ever +// selectively delete it. NOTE: subsumption as part of a range merge does +// delete the RangeDescriptor, but the Replica atomically ceases to exist in +// the process. +// Specifically, +// - The state machine state must be consistent with the value of +// RaftAppliedIndex, i.e., it equals a state generated from the full history +// of this range (for a range that has never been the LHS of a merge, this +// is the initial snapshot when the range came into being, followed by all +// subsequent raft log entries). +// - RaftAppliedIndex >= raftInitialLogIndex +// - RaftAppliedIndexTerm >= raftInitialLogTerm +// - Has at least 1 non-provisional RangeDescriptor. +// - Regression of the HardState.Commit and RaftAppliedIndex is permitted due +// to a crash except for the following: +// - Split that has progressed to applying a state machine change that +// results in a non-provisional RangeDescriptor for the RHS must not +// regress after the crash (i.e., must sync application of the split +// trigger). +// - Merge that has progressed to applying a state machine change that +// deletes the RangeDescriptor for the RHS must not regress after the +// crash (i.e., must sync application of the merge trigger). +// One could possibly relax these split/merge invariants but the corner +// cases are very subtle and make it hard to reason about correctness. +// As an example, see the discussion about "not syncing for splits" in +// https://github.com/cockroachdb/cockroach/pull/72745#pullrequestreview-807989047 +// +// INVARIANT (InterReplicaStateConsistency): The latest non-provisional +// RangeDescriptors of replicas with state machine state have spans that do +// not overlap. We use the term replica-descriptor to refer to this latest +// non-provisional RangeDescriptor, in the text below. +// +// DEFINITION (InitializedStateMachine): a Replica with state +// InitializedStateMachine, has state machine state and obeys the invariants +// RaftAndStateConsistency, StateConsistency, InterReplicaStateConsistency. +// +// DEFINITION (DeletedReplica): it can be convenient to reference Replicas +// that once existed but no longer do, as evidenced by the presence of a +// RangeTombstone for a RangeID, but no state machine or raft state. +// RangeTombstone.NextReplicaID is populated with a value > the last ReplicaID +// seen by ReplicasStorage for this range. Note that RangeTombstone is +// populated even for ranges that no longer exist (RHS of a merge) -- in this +// case it is set to a constant (mergedTombstoneReplicaID, equal to MaxInt32). +// +// DEFINITION (UninitializedStateMachine): this is a Replica with no state +// machine, i.e., there is Raft state and possibly a RangeTombstone. In +// particular, there is no RangeDescriptor and so it has no key span +// associated with it yet. The RangeTombstone.NextReplicaID is <= +// RaftReplicaID.ReplicaID. +// The HardState{Term,Vote} can have arbitrary values since this replica can +// vote. However, it has a zero HardState.Commit and no log entries -- this +// Raft invariant is upheld externally by a combination of mostly external +// invariants: +// A new Range is initialized with all Replicas at truncated index equal to +// raftInitialLogIndex (10) (so they are in InitializedStateMachine state), +// and any future Replicas will be initialized via a snapshot reflecting a +// nonzero applied index >= 10. In particular, prior to receiving the +// snapshot, no log entries can be sent to the Replica. And etcd/raft only +// communicates Commit entries for which the recipient has the log entry. +// +// Some of the above invariants may be violated when non-durable state is lost +// due to a crash, but ReplicasStorage.Init is required to fix the persistent +// state such that the above invariants are true. These are not exposed to the +// user of the interface. We list these below. +// +// DEFINITION (RecoveryDeletingReplica): a Replica whose Raft state requires a +// nonzero applied index in the state machine, but there is no state machine +// state. This is an intermediate state entered when transitioning from +// InitializedStateMachine to DeletedReplica, after the state machine state +// has been deleted and RangeTombstoneKey updated and before the raft state +// has been deleted. This is distinguishable from UninitializedStateMachine +// since RaftTruncatedState.{Index,Term} are guaranteed to exist and have +// values >= raftInitialLogIndex, raftInitialLogTerm. ReplicasStorage.Init +// will transition out of this state into DeletedReplica state. +// The RecoveryDeletingReplica can also occur when removing a replica in state +// UninitializedStateMachine. This is because the RangeTombstone is written +// first to the state machine, and subsequently the raft state is removed. +// This corresponds to the condition RangeTombstone.NextReplicaID > +// RaftReplicaID.ReplicaID. +// +// DEFINITION (RecoveryInconsistentReplica): This is a Replica that mostly +// looks like to be in state InitializedStateMachine, but has suffered +// regression in durable state such that the state machine has advanced past +// HardState.Commit, or a snapshot has been applied and all raft log entries +// are < RaftAppliedIndex, i.e., it violates RaftAndStateConsistency +// invariants. More severely, it can also violate StateConsistency invariants +// by having durably ingested SSTables but not yet updated the +// RaftAppliedIndex to reflect that state machine change. ReplicasStorage.Init +// restores all the invariants needed by an InitializedStateMachine replica, +// by fixing the raft log to be consistent with the state machine, and +// re-applying log entries up to HardState.Commit (except for log entries that +// indicate a split or merge -- see below). +// +// Replica state transitions: +// - Initial state: UninitializedStateMachine +// - Final state: DeletedReplica +// - UninitializedStateMachine => RecoveryDeletingReplica, InitializedStateMachine +// - InitializedStateMachine => RecoveryDeletingReplica, RecoveryInconsistentReplica +// - RecoveryDeletingReplica => DeletedReplica +// - RecoveryInconsistentReplica => InitializedStateMachine +// +// ============================================================================ +// Implementation constraints on ReplicasStorage: +// - Splits and Merges typically happen by applying an entry in the raft log. +// It is feasible for ReplicasStorage.Init to apply such committed entries. +// However, the logic in such cases can add additional mutations to the +// batch in the raft log, that have nothing to do with the normal scope of +// what ReplicasStorage is concerned with. For example, splitPreApply has +// logic to set RangeAppliedState.RaftClosedTimestamp. For this reason +// ReplicasStorage ensures durability of split/merge application and does +// not apply any Split/Merge log entries in ReplicasStorage.Init. +// +// ============================================================================ +// Replica Initialization: +// +// Since ReplicasStorage does not permit atomic updates spanning the state +// machine and raft state (even if they are a single engine), replica creation +// is sequenced as (* indicates durable writes): +// +// - [C1*] creation of RaftHardStateKey in raft state with +// {Term:0,Vote:0,Commit:0}. This is a replica in UninitializedStateMachine +// state. +// - [C2*] creation of state machine state (via snapshot or some synthesized +// state for range-ID and range local keys in the case of split). +// - [C3] set RaftTruncatedStateKey with RaftTruncatedState.{Index,Term} equal +// to RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm} and adjust +// value of RaftHardStateKey (specifically HardState.Commit needs to be set +// to RangeAppliedState.RaftAppliedIndex -- see below for details). Also +// discard all raft log entries if any (see below). At this point the +// replica is in InitializedStateMachine state. +// +// Every step above needs to be atomic. Note that we are doing 2 syncs, in +// steps C1 and C2, for the split case, where we currently do 1 sync -- splits +// are not common enough for this to matter. If we did not sync C2 we could +// start adding log entries after C3 but lose the state machine state in the +// case of a crash, which would violate the replica state invariants. +// +// An initialized replica that receives a snapshot because it has lagged +// behind will execute C2 and C3. The C3 step throws away all the existing +// raft log entries. So a precondition for applying such a snapshot is: +// - The raft log does not have entries beyond the snapshot's +// RangeAppliedState.RaftAppliedIndex. If it did, there would be no benefit +// in applying this snapshot. +// The following etcd/raft code +// https://github.com/etcd-io/etcd/blob/7572a61a39d4eaad596ab8d9364f7df9a84ff4a3/raft/raft.go#L1584-L1589 +// ensures this behavior -- if the raft log entry corresponding to the +// snapshot is already present locally, it only advances the commit index to +// the snapshot index, and does not actually apply the snapshot. +// - Corollary: since HardState.Commit cannot refer to log entries beyond the +// locally persisted ones, the existing HardState.Commit <= +// RangeAppliedState.RaftAppliedIndex, so the HardState manipulation done in +// step C3 will only need to increase the value of HardState.Commit. +// +// Why C2 before C3?: +// If we performed step C3 before C2, there is a possibility that a crash +// prevents C2. Now we would need to rollback the change made in C3 to reach a +// fully consistent state on crash recovery. Rolling back HardState.Commit is +// easy, since there is no raft log, we can set it to +// RangeAppliedState.RaftAppliedIndex if it exists, else 0. Similarly, we can +// rollback RaftTruncatedState by either: +// - deleting it if the RangeAppliedState does not exist, which implies C3 did +// not happen. +// - if RangeAppliedState exists, roll back RaftTruncatedState.{Index,Term} to +// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm}. Note that this +// is a case where an already initialized lagging replica has a snapshot +// being applied. +// The correctness problem with doing C3 before C2 is that the store violates +// raft promises it has made earlier. For example, say the state machine had +// applied index 20 and the raft log contained [15, 25), then this store is +// part of the quorum that causes [21, 25) to commit. We receive a snapshot +// for 30, and crash after C3, and since C3 is before C2 in this scenario, we +// rollback to 20 and have no raft state. Therefore, this is in effect an +// unavailable replica, since it no longer has [21, 25). +// +// Rolling forward if crash after C2 and before C3: +// ReplicasStorage.Init will roll forward to C3 when initializing itself. +// - If HardState.Commit < RangeAppliedState.RaftAppliedIndex, update +// HardState.Commit +// - If RaftTruncatedState does not exist, or RaftTruncatedState.Index < +// RangeAppliedState.RaftAppliedIndex and all log entries are <= +// RangeAppliedState.RaftAppliedIndex +// - Discard all raft log entries. +// - Set RaftTruncatedState.{Index,Term} using +// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm} +// +// Aside: +// Since we now have RangeAppliesState.RaftAppliedIndexTerm, constructing an +// outgoing snapshot only involves reading state machine state (this is a tiny +// bit related to #72222, in that we are also assuming here that the outgoing +// snapshot is constructed purely by reading state machine engine state). +// TODO(sumeer): create issue for this, since we can do this for 23.1 since +// RaftAppliedIndexTerm was introduced in 22.1. +// +// ============================================================================ +// Replica Deletion: +// +// Replica deletion is sequenced as the following steps (* indicates durable +// writes): +// +// - [D1*] deletion of state machine state (iff the replica is in state +// InitializedStateMachine) and write to the RangeTombstoneKey. If prior to +// this step the range was in state InitializedStateMachine, it is now in +// state RecoveryDeletingReplica. If it was in state UninitializedStateMachine, +// again it is now in the state RecoveryDeletingReplica. +// This latter case can occur for various reasons: one cause is this range +// is the RHS of a split where the split has not yet happened, but we've +// created an uninitialized RHS. So we can't delete the state machine state +// for the RHS since it doesn't exist yet (there is some replicated state in +// the state machine that could in the future belong to the RHS, but not +// yet, and we don't know the span of that future RHS either). By updating +// the RangeTombstone, when the split occurs, D1 will be repeated. +// - [D2] deletion of all Raft state for this RangeID, i.e., RaftHardStateKey, +// RaftTruncatedStateKey, log entries, RangeLastReplicaGCTimestampKey. +// +// Every step above needs to be atomic. One of the reasons to sync after D1 it +// that we could later execute C1 when adding the range back to this store, and +// then crash. On crash recovery we'd find the raft HardState and old state +// machine state and incorrectly think this is an initialized replica. +// +// Note that we don't delete the RangeTombstoneKey even when the range itself +// is being deleted (due to a merge). The replay protection offered by it is +// more important than the minuscule cost of leaking a RangeTombstoneKey per +// range. It is possible to have some cleanup of RangeTombstoneKeys for long +// dead ranges, but it is outside of the scope here. +// +// A crash after D1 will result in a replica in state RecoveryDeletingReplica. +// ReplicasStorage.Init will execute D2. See also +// https://github.com/cockroachdb/cockroach/issues/73424 which presumably +// deals with cleaning up UninitializedStateMachine replicas in the absence of +// a crash. +// +// ============================================================================ +// Normal Replica Operation: +// +// - ReplicasStorage is used to append/replace log entries and update +// HardState. This is done via a RaftMutationBatch. There is a +// RaftMutationBatch.MustSync that the caller uses to specify the minimal +// sync requirements imposed by Raft (ReplicasStorage is not in the business +// of understanding Raft correctness requirements). Typically MustSync will +// be true only if entries are appended, or a vote and/or term change has to +// be recorded. In particular, a change solely to HardState.Commit would +// have MustSync=false. See +// https://github.com/etcd-io/etcd/blob/7572a61a39d4eaad596ab8d9364f7df9a84ff4a3/raft/node.go#L584-L593. +// Note that this means that HardState.Commit can regress and become less +// than RangeAppliedState.RaftAppliedIndex in case of a crash. We will fix +// this in ReplicasStorage.Init, as discussed later. +// +// - The caller keeps track of HardState.Commit, since it constructed +// HardState for the RaftMutationBatch. It applies committed entries to the +// state machine using ApplyCommittedUsingIngest and ApplyCommittedBatch. +// The ApplyCommitted* methods should not be used for log entries that are +// performing splits or merges -- the caller should do those by calling +// SplitReplica and MergeReplicas. ReplicasStorage decides when it is +// necessary to sync -- ApplyCommitted* will not sync the state machine, and +// SplitReplica/MergeReplicas will sync the state machine. Note that the +// caller may not need to read a raft entry from ReplicasStorage in order to +// apply it, if it happens to have stashed it somewhere in its in-memory +// data-structures. +// +// For log entries that are ingesting side-loaded files, the application of +// a single entry is split into a pair, ApplyCommittedUsingIngest, that +// usually does not update the RaftAppliedIndex and then ApplyCommittedBatch +// which updates the RaftAppliedIndex. A crash after the first and before +// the second leaves the state machine in an inconsistent state +// (RecoveryInconsistentReplica) which needs to be fixed by +// ReplicasStorage.Init. For this reason, ReplicasStorage keeps track of the +// highest HardState.Commit known to be durable, and requires +// ApplyCommittedUsingIngest to provide the highestRaftIndex of the changes +// included in the files. ReplicasStorage will sync the raft state if +// needed, to make the highestRaftIndex durable, before ingesting these +// files. This prevents regression of HardState.Commit past an index that +// contains side-loaded files. Note that this assumes that +// ReplicasStorage.Init has the capability of applying all raft log entries +// except for splits and merges (we've already mentioned that splits/merges +// are made durable at application time). +// +// - Log truncation is done by the caller, based on various signals relevant +// to the proper functioning of the distributed raft group. The truncation +// is notified via RangeStorage.TruncatedRaftLog. This is breaking the +// abstraction, in that the caller has changed the raft log and removed +// sideloaded entries without going through RangeStorage, so we should see +// if we can do better here. The current log truncation methods are: +// - For strongly-coupled truncation (with a single engine): the truncation +// happens when applying to the state machine (ApplyCommittedBatch) and we +// don't want to leak this information via the MutationBatch that is +// supposed to only touch the state machine. See more details below. +// - For loosely-coupled truncation (single engine or separate engines): the +// truncation happens in raftLogTruncator.tryEnactTruncations which is +// mutating only the raft log. For this case we could keep all the +// business logic related to deciding what to truncate (which requires +// interaction with the Replica object) in raftLogTruncator, while giving +// RangeStorage the actual batch (with additional information on what is +// being truncated) to commit. +// As of https://github.com/cockroachdb/cockroach/pull/80193 the +// loosely-coupled raft log truncation is disabled due to a performance +// regression in write-heavy workloads (see comment +// https://github.com/cockroachdb/cockroach/issues/78412#issuecomment-1119922463 +// for conclusion of investigation). The revised plan is to +// - Do strongly-coupled truncation in +// CanTruncateRaftIfStateMachineIsDurable for a ReplicasStorage +// implementation that shares the same engine for the state machine and +// raft state. This relies on external code structure for correctness: the +// truncation proposal flows through raft, so we have already applied the +// state machine changes for the preceding entries. A crash will cause a +// suffix of the unsynced changes to be lost, so we cannot lose the state +// machine changes while not losing the truncation. +// +// This is the similar to the correctness argument that the code preceding +// ReplicasStorage relies on. The separation of the RaftMutationBatch +// provided to DoRaftMutation and the MutationBatch provided to +// ApplyCommittedBatch is only more formalization of the separation that +// already exists: handleRaftReadyRaftMuLocked makes raft changes with one +// batch, and replicaAppBatch.ApplyToStateMachine is used to make changes +// to the state machine with another batch. +// replicaAppBatch.ApplyToStateMachine also does the raft log truncation, +// and raft changes for splits and merges, which ReplicasStorage is doing +// in a different way, but it does not change the correctness claim. Note +// that #38566, a flaw in this correctness argument, is also unaffected. +// +// - Do loosely-coupled truncation for a ReplicasStorage implementation that +// has different engines for the state machine and raft state. The +// experiments in +// https://github.com/cockroachdb/cockroach/issues/16624#issuecomment-1137394935 +// have demonstrated that we do not have a performance regression. We +// speculate that the absence of performance regression is due to: +// - With multiple key-value pairs in a batch, the memtable for the raft +// engine will be able to store more than the corresponding state +// machine memtable where the key-value pairs get individual entries in +// the memtable. This is because of the per-entry overhead. This means +// there is a decent probability that the state machine memtable will +// start getting flushed before the corresponding raft engine memtable +// is flushed. If the flush is fast enough, we would be able to truncate +// the raft log before the raft log entries are flushed. +// - The smaller raft engine will have a higher likelihood that deletes +// due to truncation get flushed to L0 while the log entry being deleted +// is also in L0. This should reduce the likelihood of wasteful +// compaction of raft log entries to lower levels. +// +// - Range merges impose an additional requirement: the merge protocol (at a +// higher layer) needs the RHS replica of a merge to have applied all raft +// entries up to a specified index and that this application is durable. To +// ensure the durability we expose a SyncStateMachine method for the higher +// layer. +// +// ============================================================================ +// Crash Recovery: +// ReplicasStorage needs to be self contained in the sense that it must be +// able to execute state changes to reach a fully consistent state without +// needing any external input, as part of its initialization. Init will block +// until all the raft and state machine states have been made mutually +// consistent. +// - Iterate over RaftHardStateKeys and identify a set of replicas R_r. This +// is efficiently done by seeking to the current RangeID+1. +// - Iterate over RangeDescriptorKeys and identify a set of replicas R_s. This +// is efficiently done by using the latest non-provisional RangeDescriptor +// (replica-descriptor) of the current range and then seeking to the end key +// of the range's span. +// - Note that this way of skipping spans will ensure that we will not find +// RangeDescriptors that have overlapping spans, which is ideally an +// invariant we should verify. Instead of verifying that invariant, which +// is expensive, we additionally iterate over all the +// RangeAppliedStateKeys, which are Range-ID local keys -- this iteration +// can be accomplished by seeking using current RangeID+1. If we find +// RangeAppliedStateKeys whose RangeID is not mentioned in a corresponding +// RangeDescriptor we have an invariant violation. +// - The set R_s - R_r must be empty, i.e., R_s is a subset of R_r. +// - The set R_r - R_s are replicas are either in state +// UninitializedStateMachine or RecoveryDeletingReplica. +// - Remove RecoveryDeletingReplica replicas by transitioning them to DeletedReplica +// by executing D2. +// - The set R_s are replicas that ought to be in state +// InitializedStateMachine, though may have been in the middle of that state +// transition, or become inconsistent for other reasons mentioned earlier. +// That is, they are potentially in RecoveryInconsistentReplica state. +// - If RangeAppliedState.RaftAppliedIndex > HardState.Commit (NB: HardState +// must exist), execute the following atomically: +// - If there are no log entries or all log entries are < +// RaftAppliedIndex: remove all log entries and set +// RaftTruncatedState.{Index,Term} equal to +// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm}. +// - Set HardState.Commit to RaftAppliedIndex. +// These steps handle (a) crash in the middle of replica creation (doing +// step C3), and (b) regression of HardState.Commit under normal replica +// operation. The RaftAndStateConsistency invariant now holds. +// - The StateConsistency invariant may not hold. To ensure that it holds: +// for ranges whose RangeAppliedState.RaftAppliedIndex < HardState.Commit, +// apply log entries, including those that remove this replica, until one +// encounters a log entry that is performing a split or merge. +// - InitializedStateMachine replicas: +// - using the replica-descriptors, check that the spans do not overlap. +// - This InterReplicaStateConsistency invariant must also hold before we +// fixed the RecoveryInconsistentReplicas, so we could additionally check +// it then. +// ============================================================================ + +// ReplicaState represents the current state of a range replica in this store. +type ReplicaState int + +const ( + // UninitializedStateMachine is a replica with raft state but no state + // machine. + UninitializedStateMachine ReplicaState = iota + // InitializedStateMachine is a replica with both raft state and state + // machine. + InitializedStateMachine + // DeletedReplica is a replica with neither raft state or state machine. + DeletedReplica +) + +// FullReplicaID is a fully-qualified replica ID. +type FullReplicaID struct { + // RangeID is the id of the range. + RangeID roachpb.RangeID + // ReplicaID is the id of the replica. + ReplicaID roachpb.ReplicaID +} + +// ReplicaInfo provides the replica ID and state pair. +type ReplicaInfo struct { + FullReplicaID + // State of the replica. + State ReplicaState +} + +// MutationBatch can be committed to the underlying engine. Additionally, it +// provides access to the underlying Batch. In some cases the implementation +// of ReplicasStorage will add additional mutations before committing. We +// expect the caller to know which engine to construct a batch from, in order +// to update the state machine or the raft state. ReplicasStorage does not +// hide such details since we expect the caller to mostly do reads using the +// engine Reader interface. +// +// TODO(sumeer): there are some operations that need to use the storage.Batch +// as a Reader. Be clear on when a storage.Batch can be read from, and whether +// it includes the changes in the batch or is reading from the underlying DB +// (see the code in pebbleBatch that selects between the two). Ideally, the +// supported semantics should be specified via an interface method on +// storage.Batch, and ReplicasStorage can check that the semantics are what it +// expects. +type MutationBatch interface { + // Commit writes the mutation to the engine. + Commit(sync bool) error + // Batch returns the underlying storage.Batch. + Batch() storage.Batch +} + +// RaftMutationBatch specifies mutations to the raft log entries and/or +// HardState. +type RaftMutationBatch struct { + // MutationBatch should be created using NewUnindexedBatch(false) so it + // can be read from, but the reads will read the underlying engine. It is + // important for the reads to not see the state in the batch, since reading + // the underlying engine is used to clear stale raft log entries that are + // getting overwritten by this batch. + MutationBatch + // [Lo, Hi) represents the raft log entries, if any in the MutationBatch. + // This is appending/overwriting entries in the raft log. That is, if the + // log is [a,b,c,d], with a at index 12 and one appends e at index 13, the + // result will be [a,e]. Note that the MutationBatch only contains the write + // at index 13, and the removal of indices 14, 15 is done by the callee. The + // callee assumes that the entries to remove are from Term-1. We assume the + // caller is upholding Raft semantics (such as not overwriting raft log + // entries that have been committed) -- ReplicasStorage is not in the + // business of validating that such semantics are being upheld. + Lo, Hi uint64 + // Term represents the term of those entries. + Term uint64 + // HardState, if non-nil, specifies the HardState value being set by + // MutationBatch. + HardState *raftpb.HardState + // MustSync is set to true if the mutation must be synced. + MustSync bool +} + +// RangeStorage is a handle for a FullReplicaID that provides the ability to +// write to the raft state and state machine state. This could have been named +// ReplicaStorage, but that sounds too similar to ReplicasStorage. Note that, +// even though a caller can have two different RangeStorage handles for the +// same range, if the range has been added and removed and so has different +// ReplicaIDs, at most one of them will be in state != DeletedReplica. +// +// Other than the FullReplicaID() method, the methods access mutable state, +// and so may not execute concurrently. +type RangeStorage interface { + // FullReplicaID returns the FullReplicaID of this replica. + FullReplicaID() FullReplicaID + // State returns the ReplicaState of this replica. + State() ReplicaState + + // CurrentRaftEntriesRange returns [lo, hi) representing the locally stored + // raft entries. These are guaranteed to be locally durable. + CurrentRaftEntriesRange(ctx context.Context) (lo uint64, hi uint64, err error) + + // GetHardState returns the current HardState. HardState.Commit is not + // guaranteed to be durable. + GetHardState(ctx context.Context) (raftpb.HardState, error) + + // TruncatedRaftLog provides the inclusive index up to which truncation has + // been done. + TruncatedRaftLog(index uint64) + + // DoRaftMutation changes the raft state. This will also purge sideloaded + // files if any entries are being removed. + // REQUIRES: if rBatch.Lo < rBatch.Hi, the range is in state + // InitializedStateMachine. + DoRaftMutation(ctx context.Context, rBatch RaftMutationBatch) error + + // TODO(sumeer): + // - add raft log read methods. + // - what raft log stats do we need to maintain and expose (raftLogSize?)? + // We could accept a callback with a truncated index parameter that + // RangeStorage invokes whenever it truncates entries, and let the caller + // maintain the size. + + // State machine commands. + + // IngestRangeSnapshot ingests a snapshot for the range. + // - The replica-descriptor in the snapshot describes the range as equal to + // span. + // - The snapshot corresponds to application of the log up to + // raftAppliedIndex. + // - sstPaths represent the ssts for this snapshot, and do not include + // anything other than state machine state and do not contain any keys + // outside span (after accounting for range and replicated range-ID local + // keys), or Range-ID keys for other ranges. + // - sstPaths include a RANGEDEL that will clear all the existing state + // machine state in the store for span (including range-id and range local + // keys) "before" adding the snapshot state (see below for additional + // RANGEDELs that may be added by ReplicasStorage if the previous span for + // this replica was wider). + // NB: the ssts contain RangeAppliedState, RangeDescriptor (including + // possibly a provisional RangeDescriptor). Ingestion is the only way to + // initialize a range except for the RHS of a split. + // + // Snapshot ingestion will not be accepted in the following cases: + // - span overlaps with the (span in the) replica-descriptor of another + // range, unless the range is listed in subsumedRangeIDs. The ranges + // listed in subsumedRangeIDs must have spans that at least partially + // overlap with span. + // TODO(sumeer): copy the partial overlap example documented in + // replica_raftstorage.go clearSubsumedReplicaDiskData. + // The span of a range can change only via IngestRangeSnapshot, + // SplitReplica, MergeRange, so ReplicasStorage can keep track of all + // spans without resorting to repeated reading from the engine. + // - the raft log already has entries beyond the snapshot (this is an + // invariant that Raft is already expected to maintain, so it is not + // an expected error). + // + // For reference, ReplicasStorage will do: + // - If this replica is already initialized compute + // rhsSpan = current span - span provided in this call. + // rhsSpan is non-empty if we are moving the LHS past a split using a + // snapshot. In this case any replica(s) corresponding to rhsSpan cannot + // possibly be in InitializedStateMachine state (since it would be a + // violation of spans being non-overlapping). That is, they may be + // - participating in the raft group(s) for the RHS, but will not have any + // log entries. + // - rebalanced away. + // In either case, it is safe to clear all range local and global keys for + // the rhsSpan. ssts will be added to clear this state. Note, that those + // uninitialized ranges cannot have any replicated range-ID local keys. + // They may have a RangeTombstoneKey, but it is not something this method + // needs to touch. + // - Add additional ssts that clear the replicated Range-ID keys for the + // subsumed ranges, set the RangeTombstone to mergedTombstoneReplicaID, + // and the non-overlapping replicated range key spans for the subsumed + // range. + // - Atomically ingest the ssts. This does step C2 for this range and D1 for + // all subsumed ranges. This is durable. A crash after this step and + // before the next step is rolled forward in ReplicasStorage.Init. + // - Do steps C3 for this range and steps D2 for the subsumed ranges. + // + // In handleRaftReadyRaftMuLocked, if there is a snapshot, it will first + // call IngestRangeSnapshot, and then DoRaftMutation to change the + // HardState.{Term,Vote,Commit}. Note that the etcd/raft logic fast forwards + // the HardState.Commit to the snapshot index, so the DoRaftMutation will + // not actually change the stored value of HardState.Commit from what was + // already set in IngestRangeSnapshot. + IngestRangeSnapshot( + ctx context.Context, span roachpb.RSpan, raftAppliedIndex uint64, + sstPaths []string, subsumedRangeIDs []roachpb.RangeID, + sstScratch *SSTSnapshotStorageScratch) error + + // ApplyCommittedUsingIngest applies committed changes to the state machine + // state by ingesting sstPaths. highestRaftIndex is the highest index whose + // changes are included in the sstPaths. This is due to "sideloaded sst" + // raft log entries. These ssts do not contain an update to + // RangeAppliedState, so this call must be immediately followed by a call to + // ApplyCommittedBatch that does update the RangeAppliedState. + // It is possible for the node containing this store to crash prior to that + // call to ApplyCommittedBatch -- this is ok since ReplicasStorage.Init will + // replay this idempotent ingest and the following ApplyCommittedBatch. + // REQUIRES: replica is in state InitializedStateMachine. + ApplyCommittedUsingIngest( + ctx context.Context, sstPaths []string, highestRaftIndex uint64) error + + // ApplyCommittedBatch applies committed changes to the state machine state. + // Does not sync. Do not use this for applying raft log entries that perform + // split, merge, or remove this replica (due to rebalancing) -- see the + // methods in ReplicasStorage that accomplish that. + // REQUIRES: replica is in state InitializedStateMachine (this is because we + // create a new range with the first log entry at raftInitialLogIndex (10), + // so a range always requires an initial state "snapshot" before it can + // apply raft entries). + ApplyCommittedBatch(smBatch MutationBatch) error + + // SyncStateMachine is for use by higher-level code that needs to ensure + // durability of the RHS of a merge. It simply syncs the state machine state + // to ensure all previous mutations are durable. + // REQUIRES: replica is in state InitializedStateMachine. + SyncStateMachine(ctx context.Context) error +} + +// ReplicasStorage provides an interface to manage the persistent state of a +// store that includes the lifecycle of a range replica, its raft log, and the +// state machine state. See the comment at the top of the file. +type ReplicasStorage interface { + // Init will block until all the raft and state machine states have been + // made consistent. + Init(ctx context.Context) error + + // CurrentRanges returns the replicas in the store. It does not return any + // ranges with state DeletedReplica, since it has no knowledge of them. + CurrentRanges() []ReplicaInfo + + // GetRangeTombstone returns the nextReplicaID in the range tombstone for + // the range, if any. + GetRangeTombstone( + ctx context.Context, rangeID roachpb.RangeID) (nextReplicaID roachpb.ReplicaID, err error) + + // GetHandle returns a handle for a range listed in CurrentRanges(). + // ReplicasStorage will return the same handle object for a FullReplicaID + // during its lifetime. Once the FullReplicaID transitions to DeletedReplica + // state, ReplicasStorage will forget the RangeStorage handle and it is up + // to the caller to decide when to throw away a handle it may be holding + // (the handle is not really usable for doing anything once the range is + // deleted). + GetHandle(rr FullReplicaID) (RangeStorage, error) + + // CreateUninitializedRange is used when rebalancing is used to add a range + // to this store, or a peer informs this store that it has a replica of a + // range. This is the first step in creating a raft group for this + // FullReplicaID. It will return an error if: + // - This ReplicaID is too old based on the RangeTombstone.NextReplicaID + // - There already exists some state under any raft key for this range. + // + // The call will cause HardState to be initialized to + // {Term:0,Vote:0,Commit:0}. + // + // Typically there will be no state machine state for this range. However it + // is possible that a split is delayed and some other store has informed this + // store about the RHS of the split, in which case part of the state machine + // (except for the Range-ID keys, RangeDescriptor) already exist. Note that + // this locally lagging split case is one where the RHS does not transition + // to initialized via anything other than a call to SplitReplica (i.e., does + // not apply a snapshot), except when the LHS moves past the split using a + // snapshot, in which case the RHS can also then apply a snapshot. + CreateUninitializedRange(ctx context.Context, rr FullReplicaID) (RangeStorage, error) + + // SplitReplica is called to split range r into a LHS and RHS, where the RHS + // is represented by rhsRR. The smBatch specifies the state machine state to + // modify for the LHS and RHS. For the RHS, the smBatch must be constructing + // the appropriate range-ID local state and range local state that doesn't + // already exist in the store (including the RangeDescriptor). rhsSpan is + // the span in the RangeDescriptor for the RHS. The following cases can + // occur: + // + // - [A1] RangeTombstone for the RHS indicates rhsRR.ReplicaID has already + // been removed. Two subcases: + // - [A11] There exists a HardState for rhsRR.RangeID: the range has been + // added back with a new ReplicaID. + // - [A12] There exists no HardState, so rhsRR.RangeID should not exist on + // this store. + // - [A2] RangeTombstone for the RHS indicates that rhsRR.ReplicaID has not + // been removed. + // + // For A11 and A12, the smBatch will be altered to clear all state in the + // state machine for the RHS. The final state RHS will be in for A11 is + // UninitializedStateMachine, for A12 is DeletedReplica. For A2, the smBatch + // is not altered and the final RHS state is InitializedStateMachine. If the + // final RHS state is DeletedReplica, a nil RangeStorage will be returned. + // The application of smBatch is synced. + // + // From our earlier discussion of replica creation and deletion. + // - For case A2, the callee will perform step C1 if needed, then commit + // smBatch (step C2), and then perform step C3. + // - For case A11 there is no need to do step C1. Steps C2 and C3 cannot be + // performed since the RHS ReplicaID has changed and the state here is + // stale. All we are doing is cleaning up the state machine state for the + // RHS when committing smBatch. The callee is doing step D1 of deletion, + // of the RHS for the old replicaID. + // - For case A12, the callee is doing step D1 of deletion, by altering and + // committing smBatch. Since the RHS range never transitioned to + // initialized (it never had a RangeDescriptor), the deletion was unable + // to execute D1 when the HardState etc. was being deleted (it only + // executed D2). The RHS will continue to be in DeletedReplica state when + // the method returns. + // + // REQUIRES: The range being split is in state InitializedStateMachine, and + // RHS either does not exist or is in state UninitializedStateMachine. + // + // Called below Raft -- this is being called when the split transaction commits. + SplitReplica( + ctx context.Context, r RangeStorage, rhsRR FullReplicaID, rhsSpan roachpb.RSpan, + smBatch MutationBatch, + ) (RangeStorage, error) + + // MergeReplicas is called to merge two range replicas. smBatch contains + // changes to the LHS state machine to incorporate the state of the RHS, and + // the intent resolution of the RHS RangeDescriptor. + // + // It will perform the following steps: + // - Alter smBatch to remove all Range-ID local keys in the RHS and write the + // RangeTombstone to the RHS with value mergedTombstoneReplicaID. + // + // - Apply and sync smBatch, which transforms the LHS into the merged range, + // and performs step D1 for the RHS. The sync ensures that a crash after + // this step and before the next step will be rolled forward in Init. + // + // - Do step D2 for the RHS. + // + // REQUIRES: LHS and RHS are in state InitializedStateMachine, and RHS has + // durably applied all commands up to the merge. + // + // Code above this layer ensures the above durability of application of all + // commands in the RHS and additionally ensures that the RHS of a merge is + // immovable once in the critical phase (i.e. past the SubsumeRequest is + // handled), until the merge txn aborts (if it ever does). On the + // leaseholder handling Subsume, this is done by the Subsume. But we also + // prevent all future leaseholders from doing anything that would violate + // the critical phase by observing the deletion intent on the range + // descriptor. If a merge transaction commits, regardless of which replicas + // know about this yet, the LHS and RHS will be fully colocated. + // + // Called below Raft -- this is being called when the merge transaction commits. + MergeReplicas( + ctx context.Context, lhsRS RangeStorage, rhsRS RangeStorage, smBatch MutationBatch) error + + // DiscardReplica is called to discard a replica that has been rebalanced + // away. The replica is either in UninitializedStateMachine or + // InitializedStateMachine state. There are multiple reasons for this to be + // called, such as the raft log entry that removes the replica is being + // applied, or ReplicaGCQueue notices that the replica is too old. Due to + // these multiple callers, ReplicasStorage is not in a position to compute + // what the nextReplicaID for the RangeTombstone should be. Therefore, it + // expects the caller to provide that value as a parameter. + DiscardReplica( + ctx context.Context, r RangeStorage, nextReplicaID roachpb.ReplicaID) error +} + +type sideloadedStorageConstructor func( + rangeID roachpb.RangeID, replicaID roachpb.ReplicaID) SideloadStorage + +// MakeSingleEngineReplicasStorage constructs a ReplicasStorage where the same +// Engine contains the raft log and the state machine. +func MakeSingleEngineReplicasStorage( + nodeID roachpb.NodeID, + storeID roachpb.StoreID, + eng storage.Engine, + ssConstructor sideloadedStorageConstructor, + st *cluster.Settings, +) ReplicasStorage { + reps := &replicasStorageImpl{ + nodeID: nodeID, + storeID: storeID, + eng: eng, + ssConstructor: ssConstructor, + st: st, + } + reps.mu.replicasMap = make(map[roachpb.RangeID]*rangeStorageImpl) + return reps +} + +// rangeStorageImpl is an implementation of RangeStorage. +type rangeStorageImpl struct { + id FullReplicaID + reps *replicasStorageImpl + mu struct { + syncutil.RWMutex + state ReplicaState + // span is only populated for InitializedStateMachine. span.EndKey changes + // when the replica is split or merged. span.Key never changes. + span roachpb.RSpan + lastSyncedCommit uint64 + lastCommit uint64 + } + rsl stateloader.StateLoader + ss SideloadStorage +} + +var _ RangeStorage = &rangeStorageImpl{} + +func makeRangeStorageImpl( + id FullReplicaID, reps *replicasStorageImpl, ssConstructor sideloadedStorageConstructor, +) *rangeStorageImpl { + return &rangeStorageImpl{ + id: id, + reps: reps, + rsl: stateloader.Make(id.RangeID), + ss: ssConstructor(id.RangeID, id.ReplicaID), + } +} + +func (r *rangeStorageImpl) FullReplicaID() FullReplicaID { + return r.id +} + +func (r *rangeStorageImpl) State() ReplicaState { + r.mu.RLock() + defer r.mu.RUnlock() + return r.mu.state +} + +func (r *rangeStorageImpl) CurrentRaftEntriesRange( + ctx context.Context, +) (lo uint64, hi uint64, err error) { + r.mu.RLock() + defer r.mu.RUnlock() + + reader := r.reps.eng.NewReadOnly(storage.StandardDurability) + defer reader.Close() + truncState, err := r.rsl.LoadRaftTruncatedState(ctx, reader) + if err != nil { + return 0, 0, err + } + lastRaftIndex, hasEntry, err := r.rsl.LoadLastIndexWithLogEntry(ctx, reader) + if err != nil { + return 0, 0, err + } + if !hasEntry { + return 0, 0, nil + } + if truncState.Index >= lastRaftIndex { + // Found entry that is expected to be truncated. + return 0, 0, errors.Errorf( + "truncState.Index %d >= lastRaftIndex %d", truncState.Index, lastRaftIndex) + } + return truncState.Index + 1, lastRaftIndex + 1, nil +} + +func (r *rangeStorageImpl) GetHardState(ctx context.Context) (raftpb.HardState, error) { + r.mu.RLock() + defer r.mu.RUnlock() + + reader := r.reps.eng.NewReadOnly(storage.GuaranteedDurability) + defer reader.Close() + return r.rsl.LoadHardState(ctx, reader) +} + +func (r *rangeStorageImpl) TruncatedRaftLog(index uint64) { + // Noop since rangeStorageImpl does not have in-memory truncation state. +} + +func (r *rangeStorageImpl) DoRaftMutation(ctx context.Context, rBatch RaftMutationBatch) error { + r.mu.Lock() + defer r.mu.Unlock() + + var sideloadedDeleteFunc func() error + if rBatch.Lo < rBatch.Hi { + if r.mu.state != InitializedStateMachine { + return errors.Errorf("cannot add raft log entries to a !InitializedStateMachine replica ") + } + if !rBatch.MustSync { + return errors.Errorf("must not append raft log entries without syncing") + } + + var err error + sideloadedDeleteFunc, err = func() (func() error, error) { + batch := rBatch.Batch() + lastRaftIndex, hasEntry, err := r.rsl.LoadLastIndexWithLogEntry(ctx, batch) + if err != nil { + return nil, err + } + // TODO(sumeer): + // - stats delta needs to be plumbed back. + // - Replica.append in replica_raftstorage.go has slightly different logic + // where instead of doing MVCCDelete for parts that it is over-writing, + // and then doing a write, it does a MVCCPut. And does a MVCCBlindPut for + // parts that it is not over-writing. If we had log entries [5,11) and we + // are adding [5,8) we can do such logic for [5,8), but we also need to + // delete [9,11). + var ms enginepb.MVCCStats + var sdFunc func() error + if hasEntry { + // Clear [hi, lastRaftIndex]. If there are existing entries in [lo, + // hi) they are already being overwritten in the batch. Though we will + // need to compute stats corresponding to their deletion. + for i := rBatch.Hi; i <= lastRaftIndex; i++ { + if _, err := storage.MVCCDelete( + ctx, batch, &ms, r.rsl.RaftLogKey(i), hlc.Timestamp{}, hlc.ClockTimestamp{}, + nil /* txn */); err != nil { + return nil, err + } + } + sdFunc = func() error { + for i := rBatch.Lo; i <= lastRaftIndex; i++ { + if _, err := r.ss.Purge(ctx, i, rBatch.Term-1); err != nil && !errors.Is(err, errSideloadedFileNotFound) { + return err + } + } + return nil + } + } + return sdFunc, nil + }() + if err != nil { + return err + } + } + if rBatch.HardState != nil { + if rBatch.HardState.Commit < r.mu.lastCommit { + return errors.Errorf("Commit value must not regress") + } + } + // Commit the batch. + if err := rBatch.Commit(rBatch.MustSync); err != nil { + return err + } + if rBatch.HardState != nil { + r.mu.lastCommit = rBatch.HardState.Commit + if rBatch.MustSync { + r.mu.lastSyncedCommit = rBatch.HardState.Commit + } + } + if sideloadedDeleteFunc != nil { + return sideloadedDeleteFunc() + } + return nil +} + +func unionSortedSpans(a []roachpb.RSpan, b []roachpb.RSpan) []roachpb.RSpan { + var result []roachpb.RSpan + addToResult := func(x roachpb.RSpan) { + n := len(result) + if n > 0 && result[n-1].EndKey.Compare(x.Key) >= 0 { + // Overlap or adjacent, so merge. + if result[n-1].EndKey.Compare(x.EndKey) >= 0 { + // Nothing to merge. + return + } + result[n-1].EndKey = x.EndKey + return + } + result = append(result, x) + } + for i, j, n, m := 0, 0, len(a), len(b); i < n || j < m; { + if i < n && (j == m || a[i].Key.Compare(b[j].Key) <= 0) { + addToResult(a[i]) + i++ + } else { + addToResult(b[j]) + j++ + } + } + return result +} + +func (r *rangeStorageImpl) IngestRangeSnapshot( + ctx context.Context, + span roachpb.RSpan, + raftAppliedIndex uint64, + sstPaths []string, + subsumedRangeIDs []roachpb.RangeID, + sstScratch *SSTSnapshotStorageScratch, +) error { + r.reps.mu.Lock() + defer r.reps.mu.Unlock() + + reader := r.reps.eng.NewReadOnly(storage.StandardDurability) + defer reader.Close() + lastRaftIndex, _, err := r.rsl.LoadLastIndexWithLogEntry(ctx, reader) + if err != nil { + return err + } + if lastRaftIndex > raftAppliedIndex { + // etcd/raft is expected to maintain this invariant, so this error should + // not occur in practice. + return errors.Errorf("snapshot with applied index %d < index of latest raft entry %d", + raftAppliedIndex, lastRaftIndex) + } + subsumedRanges, extraRangeSpansToDelete, err := + r.reps.checkNoAdditionalOverlap(span, r.id.RangeID, subsumedRangeIDs) + if err != nil { + return err + } + if r.mu.state == InitializedStateMachine { + // We may be moving past splits with this ingest, so there can be + // additional spans to clear (fixes #73462). + additionalRangeSpansToDelete := diffSpans(r.mu.span, span) + extraRangeSpansToDelete = unionSortedSpans(extraRangeSpansToDelete, additionalRangeSpansToDelete) + } + + // Add ssts to clear range local and global keys for these spans. + for i := range extraRangeSpansToDelete { + ranges := keyRangesForStateMachineExceptRangeIDKeys(extraRangeSpansToDelete[i]) + for j := range ranges { + path, err := makeSSTToDeleteRange(ctx, ranges[j], sstScratch, r.reps.st) + if err != nil { + return err + } + if len(path) > 0 { + sstPaths = append(sstPaths, path) + } + } + } + + // Add ssts that clear replicated range-ID keys for the subsumed ranges and write + // RangeTombstone. + for i := range subsumedRangeIDs { + keyRange := rditer.MakeRangeIDLocalKeySpan(subsumedRangeIDs[i], true) + path, err := makeSSTToDeleteRange(ctx, keyRange, sstScratch, r.reps.st) + if err != nil { + return err + } + if len(path) > 0 { + sstPaths = append(sstPaths, path) + } + path, err = makeSSTForRangeTombstoneForMergedRange(ctx, subsumedRangeIDs[i], sstScratch, r.reps.st) + if err != nil { + return err + } + sstPaths = append(sstPaths, path) + } + // Ingest the ssts. + if err := r.reps.eng.IngestExternalFiles(ctx, sstPaths); err != nil { + return err + } + batch := r.reps.eng.NewUnindexedBatch(false) + defer batch.Close() + var sideloadedDeleteFuncs []func() error + // Step C3 for this range. + sideloadedDeleteFunc, err := makeRaftConsistentWithStateMachineHelper( + ctx, batch, r.rsl, r.ss) + if err != nil { + return err + } + if sideloadedDeleteFunc != nil { + sideloadedDeleteFuncs = append(sideloadedDeleteFuncs, sideloadedDeleteFunc) + } + // Step D2 for all subsumed ranges. + for i := range subsumedRanges { + sideloadedDeleteFunc, err := clearRaftStateHelper( + ctx, batch, subsumedRanges[i].rsl, subsumedRanges[i].ss) + if err != nil { + return err + } + if sideloadedDeleteFunc != nil { + sideloadedDeleteFuncs = append(sideloadedDeleteFuncs, sideloadedDeleteFunc) + } + } + // Commit the batch containing raft changes for steps C3 and D2. Neither + // step needs syncing, so the batch is not synced. + if err := batch.Commit(false); err != nil { + err = errors.Wrapf(err, "failed to do steps C3 and D2") + log.Fatalf(ctx, "%v", err) + return err + } + // Update the in-memory data-structures. + for i := range subsumedRanges { + if err := r.reps.removeFromMaps(subsumedRanges[i]); err != nil { + log.Fatalf(ctx, "%v", err) + return err + } + subsumedRanges[i].mu.Lock() + subsumedRanges[i].mu.state = DeletedReplica + subsumedRanges[i].mu.Unlock() + } + // Remove the sideloaded files. + for i := range sideloadedDeleteFuncs { + if err := sideloadedDeleteFuncs[i](); err != nil { + return err + } + } + r.mu.Lock() + r.mu.span = span + if r.mu.state == UninitializedStateMachine { + r.mu.state = InitializedStateMachine + if err := r.reps.addInitializedRangeToMap(r); err != nil { + log.Fatalf(ctx, "%v", err) + } + } + r.mu.Unlock() + return nil +} + +func diffSpans(x roachpb.RSpan, y roachpb.RSpan) []roachpb.RSpan { + var spans []roachpb.RSpan + if x.Key.Compare(y.Key) < 0 { + endKey := y.Key + noOverlap := false + if endKey.Compare(x.EndKey) > 0 { + noOverlap = true + endKey = x.EndKey + } + spans = append(spans, roachpb.RSpan{ + Key: x.Key, + EndKey: endKey, + }) + if noOverlap { + return spans + } + } + if x.EndKey.Compare(y.EndKey) > 0 { + startKey := y.EndKey + if startKey.Compare(x.Key) < 0 { + startKey = x.Key + } + spans = append(spans, roachpb.RSpan{ + Key: startKey, + EndKey: x.EndKey, + }) + } + return spans +} +func (r *rangeStorageImpl) ApplyCommittedUsingIngest( + ctx context.Context, sstPaths []string, highestRaftIndex uint64, +) error { + r.mu.Lock() + defer r.mu.Unlock() + + if r.mu.state != InitializedStateMachine { + return errors.Errorf("cannot apply to state machine for a !InitializedStateMachine replica ") + } + if highestRaftIndex > r.mu.lastCommit { + return errors.Errorf( + "trying to apply at raft index %d while highest committed index is %d", + highestRaftIndex, r.mu.lastCommit) + } + if highestRaftIndex > r.mu.lastSyncedCommit { + if err := storage.WriteSyncNoop(r.reps.eng); err != nil { + return err + } + r.mu.lastSyncedCommit = highestRaftIndex + } + // TODO(sumeer): caller probably wants IngestOperationStats, so change + // interface and call IngestExternalFilesWithStats. + return r.reps.eng.IngestExternalFiles(ctx, sstPaths) +} + +func (r *rangeStorageImpl) ApplyCommittedBatch(smBatch MutationBatch) error { + r.mu.Lock() + defer r.mu.Unlock() + + if r.mu.state != InitializedStateMachine { + return errors.Errorf("cannot apply to state machine for a !InitializedStateMachine replica ") + } + return smBatch.Commit(false) +} + +func (r *rangeStorageImpl) SyncStateMachine(ctx context.Context) error { + return storage.WriteSyncNoop(r.reps.eng) +} + +// replicasStorageImpl is an implementation of ReplicasStorage. +// +// It assumes that the caller of methods on this implementation is not trying +// to concurrently write (or concurrently read-write) to the same Range-ID or +// roachpb.RSpans (including the case where spans that are being extended or +// shortened could partially overlap). +// TODO(sumeer): +// - revisit this concurrency control assumption, since this is just a +// temporary assumption to get all the logic done without worrying about +// fine-grained locking. +// - the code below reads rangeStorageImpl.mu.{span,state} without any locking +// of rangeStorageImpl.mu, since those mutations also hold +// replicasStorageImpl.mu. Revisit this scheme. +type replicasStorageImpl struct { + nodeID roachpb.NodeID + storeID roachpb.StoreID + eng storage.Engine + ssConstructor sideloadedStorageConstructor + st *cluster.Settings + mu struct { + syncutil.RWMutex + replicasMap map[roachpb.RangeID]*rangeStorageImpl + // The subset of replicas in replicasMap that are in state + // InitializedStateMachine are in replicasSpans in span order. The spans + // do not overlap. + // TODO(sumeer): switch to using a btree. + replicasSpans []*rangeStorageImpl + } +} + +var _ ReplicasStorage = &replicasStorageImpl{} + +type rangeInfo struct { + id FullReplicaID + span roachpb.RSpan +} + +func decodeProtoFromValue(ts hlc.Timestamp, rawValue []byte, msg protoutil.Message) error { + if ts.IsEmpty() { + var meta enginepb.MVCCMetadata + err := protoutil.Unmarshal(rawValue, &meta) + if err != nil { + return err + } + rawValue = meta.RawBytes + } else { + mvccValue, err := storage.DecodeMVCCValue(rawValue) + if err != nil { + return err + } + rawValue = mvccValue.Value.RawBytes + } + value := roachpb.Value{ + RawBytes: rawValue, + Timestamp: ts, + } + return value.GetProto(msg) +} + +func (reps *replicasStorageImpl) Init(ctx context.Context) error { + reader := reps.eng.NewReadOnly(storage.StandardDurability) + defer reader.Close() + + // raftRangeIDs represents the set of ranges R_r, that have raft state. + // replicatedRangeIDs represents the set of ranges that have a state + // machine. + var raftRangeIDs, replicatedRangeIDs []roachpb.RangeID + err := func() error { + nextRangeID := roachpb.RangeID(0) + // Since the raft state and state machine state are in the same engine, + // and the unreplicated and replicated range-ID keys for the same range are + // adjacent to each other, we do a single coordinated iteration. + // + // Unreplicated range-ID local keys come after the replicated ones, so we + // set the upper bound after the unreplicated HardState. + iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + UpperBound: keys.RaftTruncatedStateKey(math.MaxInt64)}) + defer iter.Close() + done := false + for !done { + seekKey := keys.RangeAppliedStateKey(nextRangeID) + iter.SeekGE(storage.MVCCKey{Key: seekKey}) + for { + valid, err := iter.Valid() + if err != nil { + return err + } + if !valid { + done = true + break + } + rangeID, infix, suffix, _, err := keys.DecodeRangeIDKey(iter.UnsafeKey().Key) + if err != nil { + return err + } + if bytes.Equal(infix, keys.LocalRangeIDReplicatedInfix) { + replicatedRangeIDs = append(replicatedRangeIDs, rangeID) + nextRangeID = rangeID + seekKey = keys.RaftHardStateKey(nextRangeID) + iter.SeekGE(storage.MVCCKey{Key: seekKey}) + continue + } else { + // Unreplicated key + if bytes.Equal(keys.LocalRangeTombstoneSuffix, suffix) { + nextRangeID = rangeID + seekKey = keys.RaftHardStateKey(nextRangeID) + iter.SeekGE(storage.MVCCKey{Key: seekKey}) + continue + } + raftRangeIDs = append(raftRangeIDs, rangeID) + nextRangeID = rangeID + 1 + break + } + } + } + return nil + }() + if err != nil { + return err + } + + // Iterate over RangeDescriptorKeys and identify a set of replicas R_s. This + // is efficiently done by using the latest non-provisional RangeDescriptor + // of the current range and then seeking to the end key of the range's span. + var rangeInfos []rangeInfo + err = func() error { + lower := keys.RangeDescriptorKey(roachpb.RKeyMin) + upper := keys.RangeDescriptorKey(roachpb.RKeyMax) + iter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + UpperBound: upper}) + defer iter.Close() + iter.SeekGE(storage.MVCCKey{Key: lower}) + valid, err := iter.Valid() + for valid && err == nil { + var suffix roachpb.Key + key := iter.UnsafeKey() + _, suffix, _, err := keys.DecodeRangeKey(key.Key) + if err != nil { + return err + } + if bytes.Equal(suffix, keys.LocalRangeDescriptorSuffix) { + if !key.IsValue() { + savedKey := iter.Key() + // Next one should be provisional value. + iter.Next() + if valid, err = iter.Valid(); err != nil { + return err + } + if !valid || iter.UnsafeKey().Key.Equal(savedKey.Key) { + return errors.Errorf("RangeDescriptor intent with no provisional value") + } + // Next one should be a committed range descriptor + iter.Next() + if valid, err = iter.Valid(); err != nil { + return err + } + if !valid || iter.UnsafeKey().Key.Equal(savedKey.Key) { + return errors.Errorf("no committed RangeDescriptor") + } + } + // Committed RangeDescriptor. + var desc roachpb.RangeDescriptor + if err := decodeProtoFromValue(iter.UnsafeKey().Timestamp, iter.UnsafeValue(), &desc); err != nil { + return err + } + rInfo := rangeInfo{ + id: FullReplicaID{ + RangeID: desc.GetRangeID(), + }, + span: desc.RSpan(), + } + found := false + for i := range desc.InternalReplicas { + if desc.InternalReplicas[i].NodeID == reps.nodeID && + desc.InternalReplicas[i].StoreID == reps.storeID { + found = true + rInfo.id.ReplicaID = desc.InternalReplicas[i].ReplicaID + break + } + } + if !found { + return errors.Errorf("stored RangeDescriptor does not mention replica for this store") + } + rangeInfos = append(rangeInfos, rInfo) + iter.SeekGE(storage.MVCCKey{Key: keys.RangeDescriptorKey(rInfo.span.EndKey)}) + } else { + // Step until come to a RangeDescriptor. + iter.Next() + } + valid, err = iter.Valid() //lint:ignore SA4006 this value of err is never used + } + if err != nil { + return err + } + return nil + }() + if err != nil { + return err + } + // Validate that rangeInfos and replicatedRangeIDs contain the same + // RangeIDs. These are the initialized ranges. + sort.Slice(rangeInfos, func(i, j int) bool { + return rangeInfos[i].id.RangeID < rangeInfos[j].id.RangeID + }) + { + i := 0 + for n, m := len(rangeInfos), len(replicatedRangeIDs); i < n || i < m; i++ { + if i < n && (i >= m || rangeInfos[i].id.RangeID < replicatedRangeIDs[i]) { + return errors.Errorf("rangeID %d has RangeDescriptor but no replicated range-id keys", + rangeInfos[i].id.RangeID) + } else if i < m && (i >= n || rangeInfos[i].id.RangeID > replicatedRangeIDs[i]) { + return errors.Errorf("rangeID %d has no RangeDescriptor but has replicated range-id key", + rangeInfos[i].id.RangeID) + } + } + } + + // Validate that the set R_s - R_r is empty, i.e., R_s is a subset of R_r. + // And compute R_r - R_s. These replicas are either in state + // UninitializedStateMachine or RecoveryDeletingReplica. + var rangeIDsWithNoStateMachine []roachpb.RangeID + { + i := 0 + j := 0 + for n, m := len(rangeInfos), len(raftRangeIDs); i < n || j < m; { + if i < n && (j >= m || rangeInfos[i].id.RangeID < raftRangeIDs[j]) { + return errors.Errorf("rangeID %d has RangeDescriptor but no raft state", + rangeInfos[i].id.RangeID) + } else if j < m && (i >= n || rangeInfos[i].id.RangeID > raftRangeIDs[j]) { + rangeIDsWithNoStateMachine = append(rangeIDsWithNoStateMachine, raftRangeIDs[j]) + j++ + } else { + i++ + j++ + } + } + } + + // Remove RecoveryDeletingReplica replicas by transitioning them to + // DeletedReplica by executing D2. These are distinguishable from + // UninitializedStateMachine since RaftTruncatedState.{Index,Term} are + // guaranteed to exist and have values > 0. + if n := len(rangeIDsWithNoStateMachine); n > 0 { + err := func() error { + iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + UpperBound: encoding.BytesNext(keys.RaftTruncatedStateKey( + rangeIDsWithNoStateMachine[n-1])), + }) + defer iter.Close() + for i := 0; i < n; i++ { + rangeID := rangeIDsWithNoStateMachine[i] + rsl := stateloader.Make(rangeID) + foundTombstone, rangeTombstone, err := getRangeTombstone(ctx, reader, rsl) + if err != nil { + return err + } + replicaID, foundReplicaID, err := rsl.LoadRaftReplicaID(ctx, reader) + if err != nil { + return err + } + if !foundReplicaID { + return errors.Errorf("did not find RaftReplicaID for range %v with raft state", + rangeID) + } + var truncState roachpb.RaftTruncatedState + foundTruncState, err := storage.MVCCGetProto( + ctx, reader, rsl.RaftTruncatedStateKey(), hlc.Timestamp{}, &truncState, storage.MVCCGetOptions{}) + if err != nil { + return err + } + // The second condition is sufficient. + if (foundTruncState && truncState.Index > 0 && truncState.Term > 0) || + (foundTombstone && replicaID.ReplicaID < rangeTombstone.NextReplicaID) { + if err := clearRaftState(ctx, reps.eng, stateloader.Make(rangeID), + reps.ssConstructor(rangeID, replicaID.ReplicaID)); err != nil { + return err + } + continue + } + // Not deleted. + rsi := makeRangeStorageImpl(FullReplicaID{ + RangeID: rangeID, + ReplicaID: replicaID.ReplicaID, + }, reps, reps.ssConstructor) + rsi.mu.state = UninitializedStateMachine + reps.mu.replicasMap[rangeID] = rsi + } + return nil + }() + if err != nil { + return err + } + } + + // InitializedStateMachine replicas: check that the spans do not overlap. + sort.Slice(rangeInfos, func(i, j int) bool { + return rangeInfos[i].span.Key.Less(rangeInfos[j].span.Key) + }) + for i := 1; i < len(rangeInfos); i++ { + if rangeInfos[i-1].span.EndKey.Compare(rangeInfos[i].span.Key) > 0 { + // Overlapping spans. + return errors.Errorf("ranges %v and %v have overlapping spans %v, %v", + rangeInfos[i-1].id.RangeID, rangeInfos[i].id.RangeID, rangeInfos[i-1].span, + rangeInfos[i].span) + } + } + // Handle RecoveryInconsistentReplicas: This is a Replica that mostly looks + // like to be in state InitializedStateMachine, but has suffered regression + // in durable state such that the state machine has advanced past + // HardState.Commit, or a snapshot has been applied and all raft log entries + // are < RaftAppliedIndex, i.e., it violates RaftAndStateConsistency + // invariants. More severely, it can also violate StateConsistency + // invariants by having durably ingested SSTables but not yet updated the + // RaftAppliedIndex to reflect that state machine change. Restores all the + // invariants needed by an InitializedStateMachine replica, by fixing the + // raft log to be consistent with the state machine, and re-applying log + // entries up to HardState.Commit (except for log entries that indicate a + // split or merge -- see below). + for i := 0; i < len(rangeInfos); i++ { + rsi := makeRangeStorageImpl(rangeInfos[i].id, reps, reps.ssConstructor) + rsi.mu.state = InitializedStateMachine + rsi.mu.span = rangeInfos[i].span + reps.mu.replicasMap[rsi.id.RangeID] = rsi + reps.mu.replicasSpans = append(reps.mu.replicasSpans, rsi) + err := func(rsi *rangeStorageImpl) error { + batch := reps.eng.NewUnindexedBatch(false) + defer batch.Close() + hs, err := rsi.rsl.LoadHardState(ctx, batch) + if err != nil { + return err + } + as, err := rsi.rsl.LoadRangeAppliedState(ctx, batch) + if err != nil { + return err + } + + // If RangeAppliedState.RaftAppliedIndex > HardState.Commit (NB: + // HardState must exist), execute the following atomically: + // - If there are no log entries or all log entries are < + // RaftAppliedIndex: remove all log entries and set + // RaftTruncatedState.{Index,Term} equal to + // RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm}. + // - Set HardState.Commit to RaftAppliedIndex. + // These steps handle (a) crash in the middle of replica creation + // (doing step C3), and (b) regression of HardState.Commit under normal + // replica operation. The RaftAndStateConsistency invariant now holds. + if hs.Commit >= as.RaftAppliedIndex { + rsi.mu.lastCommit = hs.Commit + return nil + } + // INVARIANT: hs.Commit < as.RaftAppliedIndex + hs.Commit = as.RaftAppliedIndex + rsi.mu.lastCommit = hs.Commit + if err := rsi.rsl.SetHardState(ctx, batch, hs); err != nil { + return err + } + truncState, err := rsi.rsl.LoadRaftTruncatedState(ctx, batch) + if err != nil { + return err + } + lastRaftIndex, hasEntry, err := rsi.rsl.LoadLastIndexWithLogEntry(ctx, batch) + if err != nil { + return err + } + if !hasEntry || lastRaftIndex < as.RaftAppliedIndex { + if hasEntry { + // Delete all raft log entries. Callee is going to again load + // lastRaftIndex, which is wasteful. + sideloadedDeleteFunc, err := clearRaftLogEntries(ctx, batch, rsi.rsl, truncState, rsi.ss) + if err != nil { + return err + } + // The log truncation is not durable, but a crash now will ensure that Init + // will truncate the log. So we don't need the sideloaded entries even if + // the log entries come back. + if err := sideloadedDeleteFunc(); err != nil { + return err + } + } + truncState.Index = as.RaftAppliedIndex + truncState.Term = as.RaftAppliedIndexTerm + if err := rsi.rsl.SetRaftTruncatedState(ctx, batch, &truncState); err != nil { + return err + } + } + if err := batch.Commit(false); err != nil { + return err + } + return nil + }(rsi) + if err != nil { + return err + } + + // The StateConsistency invariant may not hold. To ensure that it holds: + // for ranges whose RangeAppliedState.RaftAppliedIndex < HardState.Commit, + // apply log entries, including those that remove this replica, until one + // encounters a log entry that is performing a split or merge. + // TODO(sumeer): Following is just rough sketching of the code based on + // copy-pasting etc., which I did for my understanding. It doesn't really + // represent what we really want, which is the refactor outlined in + // https://github.com/cockroachdb/cockroach/issues/75729 + + // var sideloaded SideloadStorage + // sideloaded, err = + // newDiskSideloadStorage(nil /*TODO*/, rsi.id.RangeID, rsi.id.ReplicaID, + // "" /*TODO*/, nil, reps.eng) + err = func() error { + batch := reps.eng.NewUnindexedBatch(false) + defer batch.Close() + hs, err := rsi.rsl.LoadHardState(ctx, batch) + if err != nil { + return err + } + as, err := rsi.rsl.LoadRangeAppliedState(ctx, batch) + if err != nil { + return err + } + if hs.Commit == as.RaftAppliedIndex { + return nil + } + if hs.Commit < as.RaftAppliedIndex { + // Should have fixed this earlier in Init. + panic("") + } + /* + // hs.Commit > as.RaftAppliedIndex. Need to apply entries. + for i := as.RaftAppliedIndex + 1; i <= hs.Commit; i++ { + // Load log entry i + committedEntries[] + raftpb.Entry + } + key := keys.RaftLogKey(rangeID, lo) + endKey := keys.RaftLogKey(rangeID, hi) + iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + UpperBound: endKey, + }) + defer iter.Close() + + var meta enginepb.MVCCMetadata + var ent raftpb.Entry + + iter.SeekGE(storage.MakeMVCCMetadataKey(key)) + for ; ; iter.Next() { + if ok, err := iter.Valid(); err != nil || !ok { + return err + } + + if err := protoutil.Unmarshal(iter.UnsafeValue(), &meta); err != nil { + return errors.Wrap(err, "unable to decode MVCCMetadata") + } + if err := storage.MakeValue(meta).GetProto(&ent); err != nil { + return errors.Wrap(err, "unable to unmarshal raft Entry") + } + if sniffSideloadedRaftCommand(ent.Data) { + newEnt, err := maybeInlineSideloadedRaftCommand( + ctx, rsi.id.RangeID, ent, sideloaded, nil) + if err != nil { + return err + } + if newEnt != nil { + ent = *newEnt + } + } + // If we want to do size accounting, which we don't + size += uint64(ent.Size()) + + // Now we need to use replicaDecoder.DecodeAndBind. No decode is sufficient since + // don't need to retrieve local proposals + cmd := &replicatedCmd{} + if err := cmd.decode(ctx, &ent); err != nil { + return err + } + // See Task.applyOneBatch + cmd.IsTrivial() + // See replicaStateMachine. It has a Replica which we don't. + // Directly use a replicaAppBatch? That also has a Replica and a replicaStateMachine. + batch = reps.eng.NewBatch() + defer batch.Close() + // Look at replicaAppBatch.Stage that converts to a CheckedCommand. + if cmd.ent.Index == 0 { + return makeNonDeterministicFailure("processRaftCommand requires a non-zero index") + } + if idx, applied := cmd.ent.Index, b.state.RaftAppliedIndex; idx != applied+1 { + // If we have an out of order index, there's corruption. No sense in + // trying to update anything or running the command. Simply return. + return makeNonDeterministicFailure("applied index jumped from %d to %d", applied, idx) + } + // Replica.shouldApplyCommand and checkForcedErr. Where do we get a *kvserverpb.ReplicaState + // checkForcedErr is stateless -- it does not need a Replica. + // We can use StateLoader.Load to load a ReplicaState. How will we keep it up to date when + // applying?** + + if cmd.raftCmd.ReplicatedEvalResult.Split != nil || cmd.raftCmd.ReplicatedEvalResult.Merge != nil { + // Stop here. + } + // replicaAppBatch.Stage should not be copied. We need to refactor it. + // replicaAppBatch.ApplyToStateMachine + + // Also see replicaStateMachine.handleNonTrivialReplicatedEvalResult + // for code that updates the lease etc. that is needed to have the latest + // ReplicaState. It is called by replicaStateMachine.ApplySideEffects. + */ + return nil + }() + if err != nil { + return err + } + + } + return nil +} + +func (reps *replicasStorageImpl) CurrentRanges() []ReplicaInfo { + reps.mu.RLock() + defer reps.mu.RUnlock() + replicas := make([]ReplicaInfo, len(reps.mu.replicasMap)) + i := 0 + for _, r := range reps.mu.replicasMap { + replicas[i].FullReplicaID = r.id + replicas[i].State = r.mu.state + i++ + } + return replicas +} + +func (reps *replicasStorageImpl) GetRangeTombstone( + ctx context.Context, rangeID roachpb.RangeID, +) (nextReplicaID roachpb.ReplicaID, err error) { + reader := reps.eng.NewReadOnly(storage.StandardDurability) + defer reader.Close() + _, tombstone, err := getRangeTombstone(ctx, reader, stateloader.Make(rangeID)) + if err != nil { + return roachpb.ReplicaID(0), err + } + return tombstone.NextReplicaID, nil +} + +func (reps *replicasStorageImpl) GetHandle(rr FullReplicaID) (RangeStorage, error) { + reps.mu.RLock() + defer reps.mu.RUnlock() + v := reps.mu.replicasMap[rr.RangeID] + if v.id.ReplicaID != rr.ReplicaID { + v = nil + } + return v, nil +} + +func (reps *replicasStorageImpl) CreateUninitializedRange( + ctx context.Context, rr FullReplicaID, +) (RangeStorage, error) { + // Initialize rangeStorageImpl and install if it does not already exist. + r := makeRangeStorageImpl(rr, reps, reps.ssConstructor) + rsl := r.rsl + r.mu.state = UninitializedStateMachine + reps.mu.Lock() + v := reps.mu.replicasMap[rr.RangeID] + var err error + if v != nil { + r = nil + err = errors.Errorf("range %v already exists", rr.RangeID) + } else { + reps.mu.replicasMap[rr.RangeID] = r + } + reps.mu.Unlock() + if err != nil { + return nil, err + } + removeFromMapDueToErr := func() { + reps.mu.Lock() + defer reps.mu.Unlock() + v := reps.mu.replicasMap[rr.RangeID] + if v == r { + delete(reps.mu.replicasMap, rr.RangeID) + } + r = nil + } + reader := reps.eng.NewReadOnly(storage.StandardDurability) + defer reader.Close() + // Ensure no HardState. + hs, err := rsl.LoadHardState(ctx, reader) + if err != nil { + removeFromMapDueToErr() + return nil, err + } + if !raft.IsEmptyHardState(hs) { + removeFromMapDueToErr() + return nil, errors.Errorf("HardState for range %v already exists", rr.RangeID) + } + // Ensure RangeTombstone does not block this replica. + found, tombstone, err := getRangeTombstone(ctx, reader, rsl) + if err != nil { + removeFromMapDueToErr() + return nil, err + } + if found && rr.ReplicaID < tombstone.NextReplicaID { + removeFromMapDueToErr() + return nil, &roachpb.RaftGroupDeletedError{} + } + // Write HardState. + if err := createHardStateAndReplicaID(ctx, reps.eng, rr, rsl); err != nil { + removeFromMapDueToErr() + return nil, err + } + return r, err +} + +func (reps *replicasStorageImpl) SplitReplica( + ctx context.Context, + r RangeStorage, + rhsRR FullReplicaID, + rhsSpan roachpb.RSpan, + smBatch MutationBatch, +) (RangeStorage, error) { + reps.mu.Lock() + defer reps.mu.Unlock() + rs := r.(*rangeStorageImpl) + rsInMap := reps.mu.replicasMap[rs.id.RangeID] + if rs != rsInMap { + return nil, errors.Errorf( + "provided RangeStorage is not current for this range %v", rs.id.RangeID) + } + if rs.mu.state != InitializedStateMachine { + return nil, errors.Errorf( + "range %v with replica %v being split is not initialized", rs.id.RangeID, rs.id.ReplicaID) + } + reader := reps.eng.NewReadOnly(storage.StandardDurability) + defer reader.Close() + rhsRSL := stateloader.Make(rhsRR.RangeID) + _, tombstone, err := getRangeTombstone(ctx, reader, rhsRSL) + if err != nil { + return nil, err + } + var hs raftpb.HardState + foundHS, err := storage.MVCCGetProto(ctx, reader, rhsRSL.RaftHardStateKey(), + hlc.Timestamp{}, &hs, storage.MVCCGetOptions{}) + if err != nil { + return nil, err + } + var rhsRS *rangeStorageImpl + doStepC3 := false + if tombstone.NextReplicaID > rhsRR.ReplicaID { + // The replica has been removed. + var ok bool + rhsRS, ok = reps.mu.replicasMap[rhsRR.RangeID] + if foundHS { + // Case A11: RHS has been added back with a later ReplicaID + // We must have a rangeStorageImpl with state UninitializedStateMachine, + // that we need to return. + if !ok { + return nil, errors.Errorf( + "range %v with replica with HardState is unknown to ReplicasStorage", + rhsRR.RangeID) + } + } else { + // Case A12: The RHS range ID must not exist on this store. + // We must not have any rangeStorageImpl and we will return nil. + if ok { + return nil, errors.Errorf( + "range %v with replica %v without HardState is known to ReplicasStorage", + rhsRS.id.RangeID, rhsRS.id.ReplicaID) + } + } + if rhsRS != nil { + if rhsRS.mu.state != UninitializedStateMachine { + return nil, errors.Errorf("RHS range %v with replica %v is already initialized", + rhsRS.id.RangeID, rhsRS.id.ReplicaID) + } + } + // NB: The engine does not contain the RSH range-id keys, but the smBatch + // does, because it was setting up the RHS of the split. So by removing + // both the range keys and the range-id keys for the RHS, we are + // effectively removing the former from the engine and the latter from the + // smBatch. + if err := clearRanges( + smBatch.Batch(), replicatedKeySpansForStateMachine(rhsRR.RangeID, rhsSpan)); err != nil { + return nil, err + } + } else { + // The replica has not been removed. + // Do step C1 if needed. + if !foundHS { + // Durably create the HardState. + if err = createHardStateAndReplicaID(ctx, reps.eng, rhsRR, rhsRSL); err != nil { + // This is a retryable error (i.e., we don't need to crash). + return nil, err + } + } + doStepC3 = true + } + // Commit smBatch. It needs to sync in both the case when it is C2 or D1. + if err := smBatch.Commit(true); err != nil { + // This is a retryable error (i.e., we don't need to crash), since the + // only mutation we may have already committed is creating the empty + // HardState. + return nil, err + } + rs.mu.span.EndKey = rhsSpan.Key + if doStepC3 { + rhsRS = makeRangeStorageImpl(rhsRR, reps, reps.ssConstructor) + if err := makeRaftConsistentWithStateMachine(ctx, reps.eng, rhsRSL, rhsRS.ss); err != nil { + err := errors.Wrapf(err, "failing because step C3 of creation has failed") + log.Fatalf(ctx, "%v", err) + return nil, err + } + rhsRS.mu.state = InitializedStateMachine + rhsRS.mu.span = rhsSpan + reps.mu.replicasMap[rhsRS.id.RangeID] = rhsRS + if err := reps.addInitializedRangeToMap(rhsRS); err != nil { + log.Fatalf(ctx, "%v", err) + } + } + return rhsRS, nil +} + +func (reps *replicasStorageImpl) MergeReplicas( + ctx context.Context, lhsRS RangeStorage, rhsRS RangeStorage, smBatch MutationBatch, +) error { + reps.mu.Lock() + defer reps.mu.Unlock() + + lRS := lhsRS.(*rangeStorageImpl) + rRS := rhsRS.(*rangeStorageImpl) + if lRS.mu.state != InitializedStateMachine || rRS.mu.state != InitializedStateMachine { + return errors.Errorf("ranges %v and %v being merged are not initialized", + lRS.id.RangeID, rRS.id.RangeID) + } + batch := smBatch.Batch() + if err := clearRanges(batch, []roachpb.Span{ + rditer.MakeRangeIDLocalKeySpan(rRS.id.RangeID, true)}); err != nil { + return err + } + // The RangeTombstoneKey, which is an unreplicated range-id local key that + // is in the state machine, needs to be adjusted. + err := storage.MVCCBlindPutProto(ctx, batch, nil, rRS.rsl.RangeTombstoneKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, + &roachpb.RangeTombstone{NextReplicaID: mergedTombstoneReplicaID}, nil) + if err != nil { + return err + } + // Sync is set to true since this is step D1 for the RHS. + if err := smBatch.Commit(true); err != nil { + return err + } + // Do step D2. + if err := clearRaftState(ctx, reps.eng, rRS.rsl, rRS.ss); err != nil { + err = errors.Wrap(err, "failing because D2 for RHS of a merge failed") + log.Fatalf(ctx, "%v", err) + return err + } + if err := reps.removeFromMaps(rRS); err != nil { + log.Fatalf(ctx, "%v", err) + return err + } + rRS.mu.Lock() + rRS.mu.state = DeletedReplica + rRS.mu.Unlock() + lRS.mu.Lock() + lRS.mu.span.EndKey = rRS.mu.span.EndKey + lRS.mu.Unlock() + return nil +} + +func (reps *replicasStorageImpl) DiscardReplica( + ctx context.Context, r RangeStorage, nextReplicaID roachpb.ReplicaID, +) error { + reps.mu.Lock() + defer reps.mu.Unlock() + + // The replica is either in UninitializedStateMachine or + // InitializedStateMachine state. + rs := r.(*rangeStorageImpl) + // Step D1 + { + batch := reps.eng.NewUnindexedBatch(true) + defer batch.Close() + if rs.mu.state == InitializedStateMachine { + if err := clearRanges(batch, replicatedKeySpansForStateMachine(rs.id.RangeID, rs.mu.span)); err != nil { + return err + } + } + err := storage.MVCCBlindPutProto(ctx, batch, nil, rs.rsl.RangeTombstoneKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, + &roachpb.RangeTombstone{NextReplicaID: nextReplicaID}, nil) + if err != nil { + return err + } + if err := batch.Commit(true); err != nil { + return err + } + } + // Step D2 + if err := clearRaftState(ctx, rs.reps.eng, rs.rsl, rs.ss); err != nil { + err = errors.Wrap(err, "failing because D2 for RHS of a merge failed") + log.Fatalf(ctx, "%v", err) + return err + } + if err := reps.removeFromMaps(rs); err != nil { + log.Fatalf(ctx, "%v", err) + return err + } + rs.mu.Lock() + rs.mu.state = DeletedReplica + rs.mu.Unlock() + return nil +} + +func (reps *replicasStorageImpl) addInitializedRangeToMap(rs *rangeStorageImpl) error { + n := len(reps.mu.replicasSpans) + i := sort.Search(n, func(i int) bool { + return reps.mu.replicasSpans[i].mu.span.Key.Compare(rs.mu.span.Key) > 0 + }) + if i > 0 { + if reps.mu.replicasSpans[i-1].mu.span.EndKey.Compare(rs.mu.span.Key) > 0 { + return errors.Errorf("overlapping spans %s, %s", + reps.mu.replicasSpans[i-1].mu.span.String(), rs.mu.span.String()) + } + } + if i < n { + if rs.mu.span.EndKey.Compare(reps.mu.replicasSpans[i].mu.span.Key) > 0 { + return errors.Errorf("overlapping spans %s, %s", + rs.mu.span.String(), reps.mu.replicasSpans[i].mu.span.String()) + } + } + if i == n { // nil or empty slice or after last element + reps.mu.replicasSpans = append(reps.mu.replicasSpans, rs) + return nil + } + reps.mu.replicasSpans = append(reps.mu.replicasSpans[:i+1], reps.mu.replicasSpans[i:]...) + reps.mu.replicasSpans[i] = rs + return nil +} + +// Removes uninitialized or initialized range from the maps. +func (reps *replicasStorageImpl) removeFromMaps(rs *rangeStorageImpl) error { + delete(reps.mu.replicasMap, rs.id.RangeID) + if rs.mu.state == InitializedStateMachine { + n := len(reps.mu.replicasSpans) + i := sort.Search(n, func(i int) bool { + return reps.mu.replicasSpans[i].mu.span.Key.Compare(rs.mu.span.Key) >= 0 + }) + if i == n || reps.mu.replicasSpans[i].mu.span.Key.Compare(rs.mu.span.Key) != 0 { + return errors.Errorf("initialized range not found in map %s", rs.mu.span.String()) + } + if i == n-1 { + reps.mu.replicasSpans = reps.mu.replicasSpans[:n-1] + } else if i == 0 { + reps.mu.replicasSpans = reps.mu.replicasSpans[1:] + } else { + reps.mu.replicasSpans = append(reps.mu.replicasSpans[:i], reps.mu.replicasSpans[i+1:]...) + } + } + return nil +} + +func getRangeTombstone( + ctx context.Context, reader storage.Reader, rsl stateloader.StateLoader, +) (bool, roachpb.RangeTombstone, error) { + rsl.RangeTombstoneKey() + tombstoneKey := rsl.RangeTombstoneKey() + var tombstone roachpb.RangeTombstone + if ok, err := storage.MVCCGetProto( + ctx, reader, tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, + ); !ok || err != nil { + return false, roachpb.RangeTombstone{}, err + } + return true, tombstone, nil +} + +func replicatedKeySpansForStateMachine(rangeID roachpb.RangeID, span roachpb.RSpan) []roachpb.Span { + ranges := keyRangesForStateMachineExceptRangeIDKeys(span) + ranges = append(ranges, rditer.MakeRangeIDLocalKeySpan(rangeID, true)) + return ranges +} + +func keyRangesForStateMachineExceptRangeIDKeys(span roachpb.RSpan) []roachpb.Span { + // TODO(sumeer): remove this hack that relies on knowing that the rditer + // function only looks at StartKey and EndKey. We should refactor the rditer + // functions. + descriptor := &roachpb.RangeDescriptor{StartKey: span.Key, EndKey: span.EndKey} + return rditer.MakeReplicatedKeySpansExceptRangeID(descriptor) +} + +func clearRanges(writer storage.Writer, keyRanges []roachpb.Span) error { + // TODO(sumeer): add option to use ClearRangeWithHeuristic. We will need a + // ReadWriter, so do this after resolving consistency questions and whether + // Batch can be read from. + // TODO(sumeer): some callers need this to also clear range keys. + for _, keyRange := range keyRanges { + if err := writer.ClearRawRange( + keyRange.Key, keyRange.EndKey, true, false); err != nil { + return err + } + } + return nil +} + +// Performs step D2 when D2 does not need to be atomic with some other change, +// so this function can create and commit the batch. +func clearRaftState( + ctx context.Context, eng storage.Engine, rsl stateloader.StateLoader, ss SideloadStorage, +) error { + batch := eng.NewUnindexedBatch(false /* writeOnly */) + defer batch.Close() + sideloadedDeleteFunc, err := clearRaftStateHelper(ctx, batch, rsl, ss) + if err != nil { + return err + } + if err := batch.Commit(false); err != nil { + return err + } + // The log truncation is not durable, but a crash now will ensure that Init + // will truncate the log. So we don't need the sideloaded entries even if + // the log entries come back. + if sideloadedDeleteFunc != nil { + return sideloadedDeleteFunc() + } + return nil +} + +// Performs step D2 with the given batch. +func clearRaftStateHelper( + ctx context.Context, batch storage.ReadWriter, rsl stateloader.StateLoader, ss SideloadStorage, +) (sideloadedDeleteFunc func() error, err error) { + if err := batch.ClearUnversioned(rsl.RaftHardStateKey()); err != nil { + return nil, err + } + truncState, err := rsl.LoadRaftTruncatedState(ctx, batch) + if err != nil { + return nil, err + } + sideloadedDeleteFunc, err = clearRaftLogEntries(ctx, batch, rsl, truncState, ss) + if err != nil { + return nil, err + } + if err := batch.ClearUnversioned(rsl.RaftReplicaIDKey()); err != nil { + return nil, err + } + if err := batch.ClearUnversioned(rsl.RaftTruncatedStateKey()); err != nil { + return nil, err + } + if err := batch.ClearUnversioned(rsl.RangeLastReplicaGCTimestampKey()); err != nil { + return nil, err + } + return sideloadedDeleteFunc, nil +} + +func createHardStateAndReplicaID( + ctx context.Context, eng storage.Engine, rid FullReplicaID, rsl stateloader.StateLoader, +) error { + batch := eng.NewUnindexedBatch(true /* writeOnly */) + defer batch.Close() + if err := rsl.SetHardState(ctx, batch, raftpb.HardState{}); err != nil { + return err + } + if err := rsl.SetRaftReplicaID(ctx, batch, rid.ReplicaID); err != nil { + return err + } + if err := batch.Commit(true); err != nil { + return err + } + return nil +} + +// Used for step C3, where we don't need to perform it atomically with +// something else, so the construction of a batch, and its commit can be done +// by this function. +func makeRaftConsistentWithStateMachine( + ctx context.Context, eng storage.Engine, rsl stateloader.StateLoader, ss SideloadStorage, +) error { + batch := eng.NewUnindexedBatch(false) + defer batch.Close() + sideloadedDeleteFunc, err := makeRaftConsistentWithStateMachineHelper(ctx, batch, rsl, ss) + if err != nil { + return err + } + if err := batch.Commit(false); err != nil { + return err + } + // The log truncation is not durable, but a crash now will ensure that Init + // will truncate the log. So we don't need the sideloaded entries even if + // the log entries come back. + if sideloadedDeleteFunc != nil { + return sideloadedDeleteFunc() + } + return nil +} + +// This helper is used when we want to discard all raft log entries because we +// are applying a snapshot or creating the RHS during a split. And we need to +// move HardState.Commit forward. That is, this is step C3. It is not what we +// want to do in ReplicasStorage.Init. +func makeRaftConsistentWithStateMachineHelper( + ctx context.Context, batch storage.ReadWriter, rsl stateloader.StateLoader, ss SideloadStorage, +) (sideloadedDeleteFunc func() error, err error) { + hs, err := rsl.LoadHardState(ctx, batch) + if err != nil { + return nil, err + } + truncState, err := rsl.LoadRaftTruncatedState(ctx, batch) + if err != nil { + return nil, err + } + as, err := rsl.LoadRangeAppliedState(ctx, batch) + if err != nil { + return nil, err + } + sideloadedDeleteFunc, err = clearRaftLogEntries(ctx, batch, rsl, truncState, ss) + if err != nil { + return nil, err + } + truncState.Index = as.RaftAppliedIndex + truncState.Term = as.RaftAppliedIndexTerm + if hs.Commit > as.RaftAppliedIndex { + return nil, errors.Errorf("cannot decrease HardState.Commit from %d to %d", + hs.Commit, as.RaftAppliedIndex) + } + hs.Commit = as.RaftAppliedIndex + if err := rsl.SetHardState(ctx, batch, hs); err != nil { + return nil, err + } + if err := rsl.SetRaftTruncatedState(ctx, batch, &truncState); err != nil { + return nil, err + } + return sideloadedDeleteFunc, err +} + +// clearRaftLogEntries clears entries in (truncState.Index, lastRaftIndex], +// where lastRaftIndex will be computed by the callee. +func clearRaftLogEntries( + ctx context.Context, + readWriter storage.ReadWriter, + rsl stateloader.StateLoader, + truncState roachpb.RaftTruncatedState, + ss SideloadStorage, +) (sideloadedDeleteFunc func() error, err error) { + lo := truncState.Index + 1 + lastRaftIndex, hasEntry, err := rsl.LoadLastIndexWithLogEntry(ctx, readWriter) + if err != nil { + return nil, err + } + // TODO(sumeer): + // - stats delta needs to be plumbed back. + // - Replica.append in replica_raftstorage.go has slightly different logic + // where instead of doing MVCCDelete for parts that it is over-writing, + // and then doing a write, it does a MVCCPut. And does a MVCCBlindPut for + // parts that it is not over-writing. If we had log entries [5,11) and we + // are adding [5,8) we can do such logic for [5,8), but we also need to + // delete [9,11). + var ms enginepb.MVCCStats + if hasEntry { + // Clear [lo, lastRaftIndex]. + for i := lo; i <= lastRaftIndex; i++ { + if _, err := storage.MVCCDelete( + ctx, readWriter, &ms, rsl.RaftLogKey(i), hlc.Timestamp{}, hlc.ClockTimestamp{}, + nil /* txn */); err != nil { + return nil, err + } + } + sideloadedDeleteFunc = func() error { + _, _, err := ss.TruncateTo(ctx, lastRaftIndex+1) + return err + } + } + return sideloadedDeleteFunc, nil +} + +// Checks that span only overlaps with ranges mentioned in subsumedRangeIDs. +// Returns the rangeStorageImpls corresponding to those subsumed ranges, and +// the sorted spans in those ranges that need to be deleted. +func (reps *replicasStorageImpl) checkNoAdditionalOverlap( + span roachpb.RSpan, rangeID roachpb.RangeID, subsumedRangeIDs []roachpb.RangeID, +) (ranges []*rangeStorageImpl, extraRangeSpansToDelete []roachpb.RSpan, err error) { + // Find index of first span whose EndKey overlaps. + n := len(reps.mu.replicasSpans) + i := sort.Search(n, func(i int) bool { + return reps.mu.replicasSpans[i].mu.span.EndKey.Compare(span.Key) > 0 + }) + for ; i < n; i++ { + subsumedSpan := reps.mu.replicasSpans[i].mu.span + if subsumedSpan.Key.Compare(span.EndKey) >= 0 { + break + } + if reps.mu.replicasSpans[i].id.RangeID == rangeID { + continue + } + found := false + for _, id := range subsumedRangeIDs { + if id == reps.mu.replicasSpans[i].id.RangeID { + found = true + ranges = append(ranges, reps.mu.replicasSpans[i]) + break + } + } + if subsumedSpan.Key.Compare(span.Key) < 0 { + extraRangeSpansToDelete = append(extraRangeSpansToDelete, roachpb.RSpan{ + Key: subsumedSpan.Key, EndKey: span.Key}) + } + if span.EndKey.Compare(subsumedSpan.EndKey) < 0 { + extraRangeSpansToDelete = append(extraRangeSpansToDelete, roachpb.RSpan{ + Key: span.EndKey, EndKey: subsumedSpan.EndKey}) + } + if !found { + return nil, nil, errors.Errorf("additional range with rangeID %d is being subsumed", + reps.mu.replicasSpans[i].id.RangeID) + } + } + if len(subsumedRangeIDs) != len(ranges) { + return nil, nil, errors.Errorf("did not find all subsumed replicas") + } + return ranges, extraRangeSpansToDelete, nil +} + +func makeSSTToDeleteRange( + ctx context.Context, + keyRange roachpb.Span, + sstScratch *SSTSnapshotStorageScratch, + st *cluster.Settings, +) (path string, err error) { + sstFile := &storage.MemFile{} + sstWriter := storage.MakeIngestionSSTWriter(ctx, st, sstFile) + defer sstWriter.Close() + if err = sstWriter.ClearRawRange( + keyRange.Key, keyRange.EndKey, true, false); err != nil { + return "", err + } + return sstWriterToFile(ctx, sstWriter, sstFile, sstScratch) +} + +func makeSSTForRangeTombstoneForMergedRange( + ctx context.Context, + rangeID roachpb.RangeID, + sstScratch *SSTSnapshotStorageScratch, + st *cluster.Settings, +) (path string, err error) { + sstFile := &storage.MemFile{} + sstWriter := storage.MakeIngestionSSTWriter(ctx, st, sstFile) + defer sstWriter.Close() + err = storage.MVCCBlindPutProto(ctx, &sstWriter, nil, keys.RangeTombstoneKey(rangeID), + hlc.Timestamp{}, hlc.ClockTimestamp{}, + &roachpb.RangeTombstone{NextReplicaID: mergedTombstoneReplicaID}, nil) + if err != nil { + return "", err + } + return sstWriterToFile(ctx, sstWriter, sstFile, sstScratch) +} + +func sstWriterToFile( + ctx context.Context, + sstWriter storage.SSTWriter, + sstFile *storage.MemFile, + sstScratch *SSTSnapshotStorageScratch, +) (path string, err error) { + if err := sstWriter.Finish(); err != nil { + return "", err + } + if sstWriter.DataSize > 0 { + var path string + if path, err = sstScratch.WriteSSTAndReturnPath(ctx, sstFile.Data()); err != nil { + return "", err + } + return path, nil + } + return "", nil +} diff --git a/pkg/kv/kvserver/replicas_storage_test.go b/pkg/kv/kvserver/replicas_storage_test.go new file mode 100644 index 000000000000..d7e52b5fa459 --- /dev/null +++ b/pkg/kv/kvserver/replicas_storage_test.go @@ -0,0 +1,703 @@ +// Copyright 2022 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 kvserver + +import ( + "context" + "fmt" + "math" + "sort" + "strconv" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/pebble/vfs" + "github.com/stretchr/testify/require" + "go.etcd.io/etcd/raft/v3/raftpb" + "golang.org/x/time/rate" +) + +const testNodeID = roachpb.NodeID(1) +const testStoredID = roachpb.StoreID(2) + +// TODO(sumeer): remove this duplication of +// stateloader.raftInitialLog{Index,Term}. +const testRaftInitialLogIndex = 10 +const testRaftInitialLogTerm = 5 + +// TODO(sumeer): additional cases to test +// - All the sideloaded cases: purging, truncation, application. +// - Skipping over provisional RangeDescriptor in ReplicasStorage.Init. +// - Failure after split creates HardState for RHS (requires adding test +// callback in ReplicasStorage). +// - RecoveryInconsistentReplica in ReplicasStorage.Init. + +func TestReplicasStorageBasic(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + datadriven.Walk(t, testutils.TestDataPath(t, "replicas_storage"), func(t *testing.T, path string) { + ctx := context.Background() + var reps *replicasStorageImpl + fs := vfs.NewStrictMem() + var eng storage.Engine + defer func() { + if eng != nil { + eng.Close() + } + }() + st := cluster.MakeTestingClusterSettings() + limiter := rate.NewLimiter(1<<30, 1<<30) + ssConstructor := func(rangeID roachpb.RangeID, replicaID roachpb.ReplicaID) SideloadStorage { + ss, err := newDiskSideloadStorage(st, rangeID, replicaID, eng.GetAuxiliaryDir(), limiter, eng) + require.NoError(t, err) + return ss + } + var sstSnapshotStorage SSTSnapshotStorage + makeEngEtc := func() { + var err error + eng, err = storage.Open(ctx, storage.MakeLocation(fs), storage.ForTesting, storage.MaxSize(1<<20)) + require.NoError(t, err) + sstSnapshotStorage = NewSSTSnapshotStorage(eng, limiter) + } + // Construct new replicasStorageImpl. + newReps := func(discardUnsyncedState bool) error { + if eng != nil { + fs.SetIgnoreSyncs(true) + require.NoError(t, sstSnapshotStorage.Clear()) + eng.Close() + if discardUnsyncedState { + fs.ResetToSyncedState() + } + fs.SetIgnoreSyncs(false) + } + makeEngEtc() + reps = MakeSingleEngineReplicasStorage( + testNodeID, testStoredID, eng, ssConstructor, st).(*replicasStorageImpl) + return reps.Init(ctx) + } + // Check some correctness of replicasStorageImpl and print contents + // (including the underlying engine). + checkAndPrintReps := func(discardUnsyncedState bool) string { + beforeStr := printReps(t, reps) + // Create a new replicasStorageImpl and check that its printed state is + // the same. + require.NoError(t, newReps(discardUnsyncedState)) + afterStr := printReps(t, reps) + require.Equal(t, beforeStr, afterStr) + return afterStr + } + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "new": + require.NoError(t, newReps(false)) + return printReps(t, reps) + + case "create-uninit": + rangeID := scanRangeIDForTest(t, d, "") + replicaID := scanReplicaID(t, d, "") + r, err := reps.CreateUninitializedRange( + ctx, FullReplicaID{RangeID: rangeID, ReplicaID: replicaID}) + if err != nil { + return err.Error() + } + require.NotNil(t, r) + return checkAndPrintReps(true) + + case "discard-replica": + r := scanAndGetRangeStorage(t, d, "", reps) + nextReplicaID := scanReplicaID(t, d, "next-") + if err := reps.DiscardReplica(ctx, r, nextReplicaID); err != nil { + return err.Error() + } + return checkAndPrintReps(true) + + case "ingest-snapshot": + err := func() error { + r := scanAndGetRangeStorage(t, d, "", reps) + rangeID, replicaID := r.FullReplicaID().RangeID, r.FullReplicaID().ReplicaID + + raftAppliedIndex, raftAppliedIndexTerm := scanRaftIndexAndTerm(t, d) + // The global keys to write in this snapshot. They correspond to the + // state machine state at (raftAppliedIndex,raftAppliedIndexTerm). + keySlice, ts := scanGlobalKeys(t, d) + // The claimed span of the replica, in the snapshot. + span := scanSpan(t, d, "") + var subsumedRangeIDs []roachpb.RangeID + if d.HasArg("subsumed-range-ids") { + var idsStr string + d.ScanArgs(t, "subsumed-range-ids", &idsStr) + idSlice := strings.Split(idsStr, ",") + for i := range idSlice { + id, err := strconv.Atoi(idSlice[i]) + require.NoError(t, err) + subsumedRangeIDs = append(subsumedRangeIDs, roachpb.RangeID(id)) + } + } + + sstScratch := sstSnapshotStorage.NewScratchSpace(rangeID, uuid.UUID{234}) + defer func() { + require.NoError(t, sstScratch.Close()) + }() + paths, err := writeRangeSnapshot(ctx, rangeID, replicaID, span, keySlice, ts, + raftAppliedIndex, raftAppliedIndexTerm, st, sstScratch) + if err != nil { + return err + } + if err := r.IngestRangeSnapshot(ctx, span, raftAppliedIndex, paths, subsumedRangeIDs, sstScratch); err != nil { + return err + } + return nil + }() + if err != nil { + return err.Error() + } + return checkAndPrintReps(true) + + case "mutation": + err := func() error { + // apply specifies whether the mutation should be applied to the state + // machine. + apply := true + if d.HasArg("apply") { + d.ScanArgs(t, "apply", &apply) + } + r := scanAndGetRangeStorage(t, d, "", reps) + var raftBatch, smBatch storage.Batch + var raftAppliedIndex, raftAppliedIndexTerm uint64 + // INVARIANT: apply => must have some global keys to write. + if d.HasArg("keys") { + rangeID := r.FullReplicaID().RangeID + raftAppliedIndex, raftAppliedIndexTerm = scanRaftIndexAndTerm(t, d) + require.Less(t, uint64(testRaftInitialLogIndex), raftAppliedIndex) + keySlice, ts := scanGlobalKeys(t, d) + var err error + raftBatch, smBatch, err = writeGlobalKeysViaRaft(ctx, rangeID, keySlice, ts, raftAppliedIndex, + raftAppliedIndexTerm, eng) + require.NoError(t, err) + } else { + require.False(t, apply) + } + // Load the HardState since we need to modify it. + hs, err := r.rsl.LoadHardState(ctx, eng) + require.NoError(t, err) + var hsPtr *raftpb.HardState + var hi uint64 + if raftAppliedIndex > 0 { + // There were some global keys to write. Reminder: the test is + // writing a single raft log entry at a time, and can only apply to + // the state machine the entry that it is writing at this momemnt. + hi = raftAppliedIndex + 1 + if apply { + // Need to advance the HardState.Commit. + hs.Commit = raftAppliedIndex + err := r.rsl.SetHardState(ctx, raftBatch, hs) + require.NoError(t, err) + hsPtr = &hs + } + // Else, not applying, so HardState.Commit is not updated. + } else { + // No raft log entries, so must be only updating HardState. The test + // only updates Term and Vote, though it could be extended to update + // Commit if it gained the ability to apply raft log entries that + // were written previously. + var raftTerm, raftVote uint64 + d.ScanArgs(t, "raft-term", &raftTerm) + d.ScanArgs(t, "raft-vote", &raftVote) + hs.Term = raftTerm + hs.Vote = raftVote + raftBatch = eng.NewUnindexedBatch(false) + err := r.rsl.SetHardState(ctx, raftBatch, hs) + require.NoError(t, err) + hsPtr = &hs + } + raftMutBatch := RaftMutationBatch{ + MutationBatch: testMutationBatch{b: raftBatch}, + Lo: raftAppliedIndex, + Hi: hi, + Term: raftAppliedIndexTerm, + HardState: hsPtr, + MustSync: true, + } + defer raftBatch.Close() + if smBatch != nil { + defer smBatch.Close() + } + if err := r.DoRaftMutation(ctx, raftMutBatch); err != nil { + return err + } + if apply { + if err := r.ApplyCommittedBatch(testMutationBatch{b: smBatch}); err != nil { + return err + } + } + return nil + }() + if err != nil { + return err.Error() + } + // discardUnsyncedState=false since replicasStorageImpl.Init() cannot + // yet roll forward the state machine by applying committed log + // entries. + return checkAndPrintReps(false) + + case "split-replica": + err := func() error { + r := scanAndGetRangeStorage(t, d, "", reps) + rhsFullID := FullReplicaID{ + RangeID: scanRangeIDForTest(t, d, "rhs-"), + ReplicaID: scanReplicaID(t, d, "rhs-"), + } + rhsSpan := scanSpan(t, d, "rhs-") + // Timestamp of the split. Used for RangeDescriptor updates. + ts := scanTimestamp(t, d) + var lhsDesc roachpb.RangeDescriptor + found, err := storage.MVCCGetProto(ctx, eng, keys.RangeDescriptorKey(r.mu.span.Key), hlc.MaxTimestamp, &lhsDesc, + storage.MVCCGetOptions{}) + require.True(t, found) + require.NoError(t, err) + lhsDesc.EndKey = rhsSpan.Key + smBatch := eng.NewBatch() + defer smBatch.Close() + // RHS RangeDescriptor. This test does not bother with a distributed + // transaction where the provisional LHS and RHS RangeDescriptors are + // written and then resolved when the transaction commits. + if err := writeDescriptor(ctx, rhsFullID.RangeID, rhsFullID.ReplicaID, rhsSpan, ts, smBatch); err != nil { + return err + } + // LHS RangeDescriptor. + if err := storage.MVCCBlindPutProto(ctx, smBatch, nil, keys.RangeDescriptorKey(lhsDesc.StartKey), + hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, &lhsDesc, nil); err != nil { + return err + } + // RHS RangeAppliedState. + appliedState := enginepb.RangeAppliedState{ + RaftAppliedIndex: testRaftInitialLogIndex, RaftAppliedIndexTerm: testRaftInitialLogTerm} + if err = storage.MVCCBlindPutProto(ctx, smBatch, nil, keys.RangeAppliedStateKey(rhsFullID.RangeID), + hlc.Timestamp{}, hlc.ClockTimestamp{}, &appliedState, nil); err != nil { + return err + } + _, err = reps.SplitReplica(ctx, r, rhsFullID, rhsSpan, testMutationBatch{b: smBatch}) + if err != nil { + return err + } + return nil + }() + if err != nil { + return err.Error() + } + return checkAndPrintReps(true) + + case "merge-replicas": + err := func() error { + lhsR := scanAndGetRangeStorage(t, d, "lhs-", reps) + rhsR := scanAndGetRangeStorage(t, d, "rhs-", reps) + // Timestamp of the merge. Used for RangeDescriptor updates. + ts := scanTimestamp(t, d) + var lhsDesc roachpb.RangeDescriptor + found, err := storage.MVCCGetProto(ctx, eng, keys.RangeDescriptorKey(lhsR.mu.span.Key), hlc.MaxTimestamp, + &lhsDesc, storage.MVCCGetOptions{}) + require.True(t, found) + require.NoError(t, err) + lhsDesc.EndKey = rhsR.mu.span.EndKey + smBatch := eng.NewBatch() + defer smBatch.Close() + // This test does not bother with a distributed transaction where the + // provisional LHS and RHS RangeDescriptors are written and then + // resolved when the transaction commits. + + // LHS RangeDescriptor update. + if err := storage.MVCCBlindPutProto(ctx, smBatch, nil, keys.RangeDescriptorKey(lhsDesc.StartKey), + hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, &lhsDesc, nil); err != nil { + return err + } + // RHS RangeDescriptor MVCC deletion. + if err := storage.MVCCBlindPut(ctx, smBatch, nil, keys.RangeDescriptorKey(rhsR.mu.span.Key), + hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, roachpb.Value{}, nil); err != nil { + return err + } + if err := reps.MergeReplicas(ctx, lhsR, rhsR, testMutationBatch{b: smBatch}); err != nil { + return err + } + return nil + }() + if err != nil { + return err.Error() + } + return checkAndPrintReps(true) + + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) + }) +} + +type testMutationBatch struct { + b storage.Batch +} + +func (tmb testMutationBatch) Commit(sync bool) error { + return tmb.b.Commit(sync) +} +func (tmb testMutationBatch) Batch() storage.Batch { + return tmb.b +} + +func scanRangeIDForTest(t *testing.T, d *datadriven.TestData, prefix string) roachpb.RangeID { + var rangeID int + d.ScanArgs(t, prefix+"range-id", &rangeID) + return roachpb.RangeID(rangeID) +} + +func scanReplicaID(t *testing.T, d *datadriven.TestData, prefix string) roachpb.ReplicaID { + var replicaID int + d.ScanArgs(t, prefix+"replica-id", &replicaID) + return roachpb.ReplicaID(replicaID) +} + +func scanAndGetRangeStorage( + t *testing.T, d *datadriven.TestData, prefix string, reps *replicasStorageImpl, +) *rangeStorageImpl { + rangeID := scanRangeIDForTest(t, d, prefix) + replicaID := scanReplicaID(t, d, prefix) + r, err := reps.GetHandle(FullReplicaID{RangeID: rangeID, ReplicaID: replicaID}) + require.NoError(t, err) + require.NotNil(t, r) + return r.(*rangeStorageImpl) +} + +func scanGlobalKeys(t *testing.T, d *datadriven.TestData) (keys []string, ts int64) { + var keysStr string + d.ScanArgs(t, "keys", &keysStr) + keys = strings.Split(keysStr, ",") + ts = scanTimestamp(t, d) + return keys, ts +} + +func scanTimestamp(t *testing.T, d *datadriven.TestData) int64 { + var tsint int + d.ScanArgs(t, "ts", &tsint) + return int64(tsint) +} + +func scanSpan(t *testing.T, d *datadriven.TestData, prefix string) roachpb.RSpan { + var spanStr string + d.ScanArgs(t, prefix+"span", &spanStr) + parts := strings.Split(spanStr, ",") + require.Equal(t, 2, len(parts)) + return roachpb.RSpan{Key: roachpb.RKey(parts[0]), EndKey: roachpb.RKey(parts[1])} +} + +func scanRaftIndexAndTerm( + t *testing.T, d *datadriven.TestData, +) (raftIndex uint64, raftTerm uint64) { + d.ScanArgs(t, "raft-index", &raftIndex) + d.ScanArgs(t, "raft-term", &raftTerm) + require.LessOrEqual(t, uint64(testRaftInitialLogIndex), raftIndex) + require.LessOrEqual(t, uint64(testRaftInitialLogTerm), raftTerm) + return raftIndex, raftTerm +} + +func printReps(t *testing.T, reps *replicasStorageImpl) string { + // TODO(sumeer): Print any unexpected stuff in engine, and not just the + // state based on the ranges known to reps. + var b strings.Builder + for i, r := range reps.mu.replicasSpans { + if i > 0 { + rPrev := reps.mu.replicasSpans[i-1] + require.True(t, rPrev.mu.span.EndKey.Compare(r.mu.span.Key) <= 0) + } + require.Equal(t, InitializedStateMachine, r.mu.state) + require.LessOrEqual(t, r.mu.lastSyncedCommit, r.mu.lastCommit) + rangeID := r.id.RangeID + r2, ok := reps.mu.replicasMap[rangeID] + require.True(t, ok) + require.Equal(t, r2, r) + printRange(t, &b, r, reps.eng) + } + var uninitializedRanges []*rangeStorageImpl + for _, r := range reps.mu.replicasMap { + require.NotEqual(t, DeletedReplica, r.mu.state) + if r.mu.state == UninitializedStateMachine { + uninitializedRanges = append(uninitializedRanges, r) + } + } + sort.Slice(uninitializedRanges, func(i, j int) bool { + return uninitializedRanges[i].id.RangeID < uninitializedRanges[j].id.RangeID + }) + for _, r := range uninitializedRanges { + printRange(t, &b, r, reps.eng) + } + return b.String() +} + +func replicaStateStr(state ReplicaState) string { + switch state { + case UninitializedStateMachine: + return "uninited" + case InitializedStateMachine: + return "inited" + case DeletedReplica: + return "deleted" + default: + return "unknown" + } +} + +func printRange(t *testing.T, b *strings.Builder, r *rangeStorageImpl, eng storage.Engine) { + ctx := context.Background() + r.mu.RLock() + defer r.mu.RUnlock() + fmt.Fprintf(b, "== r%s/%s %s ==\n", r.id.RangeID.String(), r.id.ReplicaID.String(), replicaStateStr(r.mu.state)) + hs, err := r.rsl.LoadHardState(ctx, eng) + require.NoError(t, err) + // TODO: CI fails race tests when hs is zero, by printing "hs: " instead of + // "hs: term:0 vote:0 commit:0". + fmt.Fprintf(b, " hs: %s", hs.String()) + replicaID, found, err := r.rsl.LoadRaftReplicaID(ctx, eng) + require.True(t, found) + require.NoError(t, err) + require.Equal(t, r.id.ReplicaID, replicaID.ReplicaID) + tombstoneKey := keys.RangeTombstoneKey(r.id.RangeID) + var tombstone roachpb.RangeTombstone + ok, err := storage.MVCCGetProto(ctx, eng, tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}) + require.NoError(t, err) + if ok { + require.LessOrEqual(t, tombstone.NextReplicaID, r.id.ReplicaID) + fmt.Fprintf(b, " next-replica-id: %d", tombstone.NextReplicaID) + } + truncState, err := r.rsl.LoadRaftTruncatedState(ctx, eng) + require.NoError(t, err) + fmt.Fprintf(b, " trunc-state: %s\n", truncState.String()) + func() { + prefix := r.rsl.RaftLogPrefix() + iter := eng.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + LowerBound: prefix, UpperBound: keys.RaftLogKeyFromPrefix(prefix, math.MaxUint64)}) + defer iter.Close() + iter.SeekGE(storage.MVCCKey{Key: prefix}) + first := true + for { + valid, err := iter.Valid() + require.NoError(t, err) + if !valid { + break + } + key := iter.Key().Key + suffix := key[len(prefix):] + index, err := keys.DecodeRaftLogKeyFromSuffix(suffix) + require.NoError(t, err) + if first { + fmt.Fprintf(b, " raft-log: %d", index) + first = false + } else { + fmt.Fprintf(b, ", %d", index) + } + iter.Next() + } + if !first { + fmt.Fprintf(b, "\n") + } + }() + if r.mu.state != InitializedStateMachine { + return + } + fmt.Fprintf(b, " span: %s", r.mu.span.String()) + var desc roachpb.RangeDescriptor + value, _, err := storage.MVCCGet(ctx, eng, keys.RangeDescriptorKey(r.mu.span.Key), hlc.MaxTimestamp, + storage.MVCCGetOptions{}) + require.NoError(t, err) + require.NotNil(t, value) + require.NoError(t, value.GetProto(&desc)) + require.True(t, r.mu.span.Equal(desc.RSpan())) + require.Equal(t, r.id.RangeID, desc.RangeID) + fmt.Fprintf(b, " desc-ts: %d", value.Timestamp.WallTime) + + appliedState, err := r.rsl.LoadRangeAppliedState(ctx, eng) + require.NoError(t, err) + fmt.Fprintf(b, " appliedState: index: %d, term: %d\n", appliedState.RaftAppliedIndex, + appliedState.RaftAppliedIndexTerm) + func() { + iter := eng.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ + LowerBound: roachpb.Key(desc.StartKey), UpperBound: roachpb.Key(desc.EndKey)}) + defer iter.Close() + iter.SeekGE(storage.MVCCKey{Key: roachpb.Key(desc.StartKey)}) + fmt.Fprintf(b, " global:") + for { + valid, err := iter.Valid() + require.NoError(t, err) + if !valid { + break + } + value, err := storage.DecodeMVCCValue(iter.UnsafeValue()) + require.NoError(t, err) + val, err := value.Value.GetBytes() + require.NoError(t, err) + key := iter.UnsafeKey() + fmt.Fprintf(b, " %s=%s", key.String(), string(val)) + iter.Next() + } + fmt.Fprintf(b, "\n") + }() +} + +func writeRangeSnapshot( + ctx context.Context, + rangeID roachpb.RangeID, + replicaID roachpb.ReplicaID, + span roachpb.RSpan, + keySlice []string, + ts int64, + raftIndex uint64, + raftTerm uint64, + st *cluster.Settings, + sstScratch *SSTSnapshotStorageScratch, +) (paths []string, err error) { + rangeIDLocalSpan := rditer.MakeRangeIDLocalKeySpan(rangeID, true) + sstFile := &storage.MemFile{} + sstWriter := storage.MakeIngestionSSTWriter(ctx, st, sstFile) + defer sstWriter.Close() + if err := sstWriter.ClearRawRange( + rangeIDLocalSpan.Key, rangeIDLocalSpan.EndKey, true, false); err != nil { + return nil, err + } + appliedState := enginepb.RangeAppliedState{RaftAppliedIndex: raftIndex, RaftAppliedIndexTerm: raftTerm} + if err = storage.MVCCBlindPutProto(ctx, &sstWriter, nil, keys.RangeAppliedStateKey(rangeID), + hlc.Timestamp{}, hlc.ClockTimestamp{}, &appliedState, nil); err != nil { + return nil, err + } + path, err := sstWriterToFile(ctx, sstWriter, sstFile, sstScratch) + if err != nil { + return nil, err + } + paths = append(paths, path) + rangeSpans := keyRangesForStateMachineExceptRangeIDKeys(span) + for i := 0; i < len(rangeSpans); i++ { + sstFile = &storage.MemFile{} + sstWriter = storage.MakeIngestionSSTWriter(ctx, st, sstFile) + defer sstWriter.Close() + if err := sstWriter.ClearRawRange( + rangeSpans[i].Key, rangeSpans[i].EndKey, true, false); err != nil { + return nil, err + } + if i == 0 { + // Range local span. + if err := writeDescriptor(ctx, rangeID, replicaID, span, ts, &sstWriter); err != nil { + return nil, err + } + } + if i == len(rangeSpans)-1 { + // Global span. + if err := writeGlobalKeys(ctx, keySlice, ts, &sstWriter); err != nil { + return nil, err + } + } + path, err = sstWriterToFile(ctx, sstWriter, sstFile, sstScratch) + if err != nil { + return nil, err + } + paths = append(paths, path) + } + return paths, nil +} + +func writeDescriptor( + ctx context.Context, + rangeID roachpb.RangeID, + replicaID roachpb.ReplicaID, + span roachpb.RSpan, + ts int64, + w storage.Writer, +) error { + descriptorState := roachpb.RangeDescriptor{RangeID: rangeID, StartKey: span.Key, EndKey: span.EndKey, + InternalReplicas: []roachpb.ReplicaDescriptor{ + { + // Have an additional replica unrelated to the one we are adding. + NodeID: testNodeID + 50, + StoreID: testStoredID + 50, + ReplicaID: replicaID + 50, + }, + { + NodeID: testNodeID, + StoreID: testStoredID, + ReplicaID: replicaID, + }, + }} + if err := storage.MVCCBlindPutProto(ctx, w, nil, keys.RangeDescriptorKey(span.Key), + hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, &descriptorState, nil); err != nil { + return err + } + return nil +} + +// Writes the given global keys to a raftBatch and smBatch (state machine). +// This function is limited in that it puts all the key-value pairs in a +// single raft log entry. +func writeGlobalKeysViaRaft( + ctx context.Context, + rangeID roachpb.RangeID, + keySlice []string, + ts int64, + raftIndex uint64, + raftTerm uint64, + eng storage.Engine, +) (raftBatch storage.Batch, smBatch storage.Batch, err error) { + tmpBatch := eng.NewBatch() + defer tmpBatch.Close() + if err = writeGlobalKeys(ctx, keySlice, ts, tmpBatch); err != nil { + return nil, nil, err + } + batchBytes := tmpBatch.Repr() + raftBatch = eng.NewUnindexedBatch(false) + // TODO(sumeer): this is not the right way to construct a raft entry in the + // engine. It is a raftpb.Entry. Fix. This works for now since neither the + // test code, nor replicasStorageImpl is reading the raft log entries. + if err := storage.MVCCBlindPut(ctx, raftBatch, nil, keys.RaftLogKey(rangeID, raftIndex), hlc.Timestamp{}, + hlc.ClockTimestamp{}, roachpb.Value{RawBytes: batchBytes}, nil); err != nil { + return nil, nil, err + } + smBatch = eng.NewBatch() + if err := smBatch.ApplyBatchRepr(batchBytes, false); err != nil { + return nil, nil, err + } + appliedState := enginepb.RangeAppliedState{RaftAppliedIndex: raftIndex, RaftAppliedIndexTerm: raftTerm} + if err = storage.MVCCBlindPutProto(ctx, smBatch, nil, keys.RangeAppliedStateKey(rangeID), + hlc.Timestamp{}, hlc.ClockTimestamp{}, &appliedState, nil); err != nil { + return nil, nil, err + } + return raftBatch, smBatch, nil +} + +func writeGlobalKeys(ctx context.Context, keys []string, ts int64, w storage.Writer) error { + for i := range keys { + var ms enginepb.MVCCStats + var v roachpb.Value + v.SetBytes([]byte(fmt.Sprintf("%s.%d", keys[i], ts))) + err := storage.MVCCBlindPut(ctx, w, &ms, roachpb.Key(keys[i]), + hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, nil) + if err != nil { + return err + } + } + return nil +} diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 5dab4a573310..2c94a2673ed9 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -344,6 +344,27 @@ func (rsl StateLoader) SetVersion( // LoadLastIndex loads the last index. func (rsl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) (uint64, error) { + lastIndex, found, err := rsl.LoadLastIndexWithLogEntry(ctx, reader) + if err != nil { + return 0, err + } + if found { + return lastIndex, nil + } + // The log is empty, which means we are either starting from scratch + // or the entire log has been truncated away. + lastEnt, err := rsl.LoadRaftTruncatedState(ctx, reader) + if err != nil { + return 0, err + } + return lastEnt.Index, nil +} + +// LoadLastIndexWithLogEntry loads the last index containing an actual log +// entry. +func (rsl StateLoader) LoadLastIndexWithLogEntry( + ctx context.Context, reader storage.Reader, +) (uint64, bool, error) { prefix := rsl.RaftLogPrefix() // NB: raft log has no intents. iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{LowerBound: prefix}) @@ -362,18 +383,9 @@ func (rsl StateLoader) LoadLastIndex(ctx context.Context, reader storage.Reader) if err != nil { log.Fatalf(ctx, "unable to decode Raft log index key: %s; %v", key.String(), err) } + return lastIndex, true, nil } - - if lastIndex == 0 { - // The log is empty, which means we are either starting from scratch - // or the entire log has been truncated away. - lastEnt, err := rsl.LoadRaftTruncatedState(ctx, reader) - if err != nil { - return 0, err - } - lastIndex = lastEnt.Index - } - return lastIndex, nil + return 0, false, nil } // LoadRaftTruncatedState loads the truncated state. diff --git a/pkg/kv/kvserver/testdata/replicas_storage/basic b/pkg/kv/kvserver/testdata/replicas_storage/basic new file mode 100644 index 000000000000..1cb78b9cca2f --- /dev/null +++ b/pkg/kv/kvserver/testdata/replicas_storage/basic @@ -0,0 +1,198 @@ +new +---- + +# Create and discard an uninitialized replica. +create-uninit range-id=1 replica-id=2 +---- +== r1/2 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +discard-replica range-id=1 replica-id=2 next-replica-id=4 +---- + +# Try to recreate the discarded replica, which will fail due to the +# RangeTombstone. +create-uninit range-id=1 replica-id=3 +---- +raft group deleted + +# Create it again with a higher replicaID. +create-uninit range-id=1 replica-id=4 +---- +== r1/4 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 4 trunc-state: index:0 term:0 + +# Create another range. +create-uninit range-id=13 replica-id=15 +---- +== r1/4 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 4 trunc-state: index:0 term:0 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Ingest snapshot for rangeID 1. +ingest-snapshot range-id=1 replica-id=4 raft-index=11 raft-term=5 span=a,c keys=a,b ts=12 +---- +== r1/4 inited == + hs: term:0 vote:0 commit:11 next-replica-id: 4 trunc-state: index:11 term:5 + span: {a-c} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Create a mutation with a raft log entry and apply it to the state machine. +# It appears in the raft log, state machine, and the HardState (hs) commit +# index moves forward. +mutation range-id=1 replica-id=4 raft-index=12 raft-term=5 keys=a,b ts=14 +---- +== r1/4 inited == + hs: term:0 vote:0 commit:12 next-replica-id: 4 trunc-state: index:11 term:5 + raft-log: 12 + span: {a-c} desc-ts: 12 appliedState: index: 12, term: 5 + global: "a"/0.000000014,0=a.14 "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Create a mutation with a raft log entry for the uninitialized replica. +mutation range-id=13 replica-id=15 raft-index=12 raft-term=5 keys=a,b ts=14 +---- +cannot add raft log entries to a !InitializedStateMachine replica + +# Create a mutation for rangeID 1 but don't apply it. Appears in the raft log. +mutation range-id=1 replica-id=4 raft-index=13 raft-term=5 keys=a ts=15 apply=false +---- +== r1/4 inited == + hs: term:0 vote:0 commit:12 next-replica-id: 4 trunc-state: index:11 term:5 + raft-log: 12, 13 + span: {a-c} desc-ts: 12 appliedState: index: 12, term: 5 + global: "a"/0.000000014,0=a.14 "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Create another mutation with a raft log entry that is not applied. +mutation range-id=1 replica-id=4 raft-index=14 raft-term=5 keys=a ts=16 apply=false +---- +== r1/4 inited == + hs: term:0 vote:0 commit:12 next-replica-id: 4 trunc-state: index:11 term:5 + raft-log: 12, 13, 14 + span: {a-c} desc-ts: 12 appliedState: index: 12, term: 5 + global: "a"/0.000000014,0=a.14 "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Increase the raft term and wipe out the old entries at index 13, 14 by +# adding an entry at index 13, and also apply it. +# TODO(sumeer): add test case that also wipes out sideloaded storage. +mutation range-id=1 replica-id=4 raft-index=13 raft-term=6 keys=a ts=18 +---- +== r1/4 inited == + hs: term:0 vote:0 commit:13 next-replica-id: 4 trunc-state: index:11 term:5 + raft-log: 12, 13 + span: {a-c} desc-ts: 12 appliedState: index: 13, term: 6 + global: "a"/0.000000018,0=a.18 "a"/0.000000014,0=a.14 "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Split rangeID 1. The latest descriptor(s) for LHS and RHS have the expected +# timestamps and the RHS has the expected state. +split-replica range-id=1 replica-id=4 rhs-range-id=21 rhs-replica-id=24 rhs-span=b,c ts=20 +---- +== r1/4 inited == + hs: term:0 vote:0 commit:13 next-replica-id: 4 trunc-state: index:11 term:5 + raft-log: 12, 13 + span: {a-b} desc-ts: 20 appliedState: index: 13, term: 6 + global: "a"/0.000000018,0=a.18 "a"/0.000000014,0=a.14 "a"/0.000000012,0=a.12 +== r21/24 inited == + hs: term:0 vote:0 commit:10 trunc-state: index:10 term:5 + span: {b-c} desc-ts: 20 appliedState: index: 10, term: 5 + global: "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Merge back into rangeID 1. +merge-replicas lhs-range-id=1 lhs-replica-id=4 rhs-range-id=21 rhs-replica-id=24 ts=22 +---- +== r1/4 inited == + hs: term:0 vote:0 commit:13 next-replica-id: 4 trunc-state: index:11 term:5 + raft-log: 12, 13 + span: {a-c} desc-ts: 22 appliedState: index: 13, term: 6 + global: "a"/0.000000018,0=a.18 "a"/0.000000014,0=a.14 "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Try to recreate the RHS rangeID with a high replicaID. Fails due to RangeTombstone. +create-uninit range-id=21 replica-id=3000 +---- +raft group deleted + +# Split rangeID 1 again, at a different key. +split-replica range-id=1 replica-id=4 rhs-range-id=22 rhs-replica-id=25 rhs-span=aa,c ts=23 +---- +== r1/4 inited == + hs: term:0 vote:0 commit:13 next-replica-id: 4 trunc-state: index:11 term:5 + raft-log: 12, 13 + span: a{-a} desc-ts: 23 appliedState: index: 13, term: 6 + global: "a"/0.000000018,0=a.18 "a"/0.000000014,0=a.14 "a"/0.000000012,0=a.12 +== r22/25 inited == + hs: term:0 vote:0 commit:10 trunc-state: index:10 term:5 + span: {aa-c} desc-ts: 23 appliedState: index: 10, term: 5 + global: "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Discard rangeID 1. +discard-replica range-id=1 replica-id=4 next-replica-id=5 +---- +== r22/25 inited == + hs: term:0 vote:0 commit:10 trunc-state: index:10 term:5 + span: {aa-c} desc-ts: 23 appliedState: index: 10, term: 5 + global: "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Add it back at a higher replicaID than the RangeTombstone. +create-uninit range-id=1 replica-id=5 +---- +== r22/25 inited == + hs: term:0 vote:0 commit:10 trunc-state: index:10 term:5 + span: {aa-c} desc-ts: 23 appliedState: index: 10, term: 5 + global: "b"/0.000000014,0=b.14 "b"/0.000000012,0=b.12 +== r1/5 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 5 trunc-state: index:0 term:0 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Remove rangeID 22 and add it back, so we can look at the RangeTombstone. +discard-replica range-id=22 replica-id=25 next-replica-id=26 +---- +== r1/5 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 5 trunc-state: index:0 term:0 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +create-uninit range-id=22 replica-id=26 +---- +== r1/5 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 5 trunc-state: index:0 term:0 +== r13/15 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 +== r22/26 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 26 trunc-state: index:0 term:0 + +# Do the same for the uninitialized rangeID 13, to check the RangeTombstone +# behavior when discarding an uninitialized replica. +discard-replica range-id=13 replica-id=15 next-replica-id=16 +---- +== r1/5 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 5 trunc-state: index:0 term:0 +== r22/26 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 26 trunc-state: index:0 term:0 + +create-uninit range-id=13 replica-id=16 +---- +== r1/5 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 5 trunc-state: index:0 term:0 +== r13/16 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 16 trunc-state: index:0 term:0 +== r22/26 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 26 trunc-state: index:0 term:0 diff --git a/pkg/kv/kvserver/testdata/replicas_storage/ingest_recovery b/pkg/kv/kvserver/testdata/replicas_storage/ingest_recovery new file mode 100644 index 000000000000..873aa5973bdc --- /dev/null +++ b/pkg/kv/kvserver/testdata/replicas_storage/ingest_recovery @@ -0,0 +1,66 @@ +# Test that an ingest that subsumes existing raft log entries is recovered +# from correctly in ReplicasStorage.Init: note that the removal of the raft +# log entry will be lost by the test and so the replica should appear in state +# RecoveryInconsistentReplica. +new +---- + +create-uninit range-id=1 replica-id=2 +---- +== r1/2 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Initialize the replica. +ingest-snapshot range-id=1 replica-id=2 raft-index=11 raft-term=5 span=a,f keys=a ts=12 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-f} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 + +# Apply mutations that build up the raft log. +mutation range-id=1 replica-id=2 raft-index=12 raft-term=5 keys=b ts=14 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:12 trunc-state: index:11 term:5 + raft-log: 12 + span: {a-f} desc-ts: 12 appliedState: index: 12, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 + +mutation range-id=1 replica-id=2 raft-index=13 raft-term=5 keys=e ts=15 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:13 trunc-state: index:11 term:5 + raft-log: 12, 13 + span: {a-f} desc-ts: 12 appliedState: index: 13, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 "e"/0.000000015,0=e.15 + +# Ensure we have synced everything in the raft log by doing a split. This +# isn't actually necessary since the test sets RaftMutationBatch.MustSync=true +# in the previous steps, but just in case. +split-replica range-id=1 replica-id=2 rhs-range-id=21 rhs-replica-id=22 rhs-span=e,f ts=16 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:13 trunc-state: index:11 term:5 + raft-log: 12, 13 + span: {a-e} desc-ts: 16 appliedState: index: 13, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000014,0=b.14 +== r21/22 inited == + hs: term:0 vote:0 commit:10 trunc-state: index:10 term:5 + span: {e-f} desc-ts: 16 appliedState: index: 10, term: 5 + global: "e"/0.000000015,0=e.15 + +# Ingest a snapshot ahead of all these log entries. The log entries will be +# cleared without syncing, so when the test resets the strict-mem and creates +# a new ReplicasStorage, the latter will need to again remove the raft log +# entries. +ingest-snapshot range-id=1 replica-id=2 raft-index=17 raft-term=5 span=a,e keys=c ts=17 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:17 trunc-state: index:17 term:5 + span: {a-e} desc-ts: 17 appliedState: index: 17, term: 5 + global: "c"/0.000000017,0=c.17 +== r21/22 inited == + hs: term:0 vote:0 commit:10 trunc-state: index:10 term:5 + span: {e-f} desc-ts: 16 appliedState: index: 10, term: 5 + global: "e"/0.000000015,0=e.15 diff --git a/pkg/kv/kvserver/testdata/replicas_storage/ingest_subsumed b/pkg/kv/kvserver/testdata/replicas_storage/ingest_subsumed new file mode 100644 index 000000000000..fd60d3f6b5f6 --- /dev/null +++ b/pkg/kv/kvserver/testdata/replicas_storage/ingest_subsumed @@ -0,0 +1,129 @@ +new +---- + +create-uninit range-id=1 replica-id=2 +---- +== r1/2 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +ingest-snapshot range-id=1 replica-id=2 raft-index=11 raft-term=5 span=a,c keys=a,b ts=12 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-c} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 + +create-uninit range-id=2 replica-id=3 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-c} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 +== r2/3 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +ingest-snapshot range-id=2 replica-id=3 raft-index=12 raft-term=6 span=c,e keys=c,d ts=12 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-c} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 +== r2/3 inited == + hs: term:0 vote:0 commit:12 trunc-state: index:12 term:6 + span: {c-e} desc-ts: 12 appliedState: index: 12, term: 6 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 + + +# RangeID 1 [a,c), RangeID 2 [c,e). Try to destroy 1 by ingesting a snapshot +# for 2 with span [b,e). Fails because did not list rangeID 1 as subsumed in +# the parameter. +ingest-snapshot range-id=2 replica-id=3 raft-index=13 raft-term=6 span=b,e keys=b,c,d ts=13 +---- +additional range with rangeID 1 is being subsumed + +# Subsumption succeeds. +ingest-snapshot range-id=2 replica-id=3 raft-index=13 raft-term=6 span=b,e keys=b,c,d ts=13 subsumed-range-ids=1 +---- +== r2/3 inited == + hs: term:0 vote:0 commit:13 trunc-state: index:13 term:6 + span: {b-e} desc-ts: 13 appliedState: index: 13, term: 6 + global: "b"/0.000000013,0=b.13 "c"/0.000000013,0=c.13 "d"/0.000000013,0=d.13 + +# Cannot recreate rangeID 1 because we know it has been merged in history and +# there is a RangeTombstone preventing it from ever being recreated. +create-uninit range-id=1 replica-id=20000 +---- +raft group deleted + +# Create ranges on either side of rangeID 2 that it will subsume. +create-uninit range-id=3 replica-id=4 +---- +== r2/3 inited == + hs: term:0 vote:0 commit:13 trunc-state: index:13 term:6 + span: {b-e} desc-ts: 13 appliedState: index: 13, term: 6 + global: "b"/0.000000013,0=b.13 "c"/0.000000013,0=c.13 "d"/0.000000013,0=d.13 +== r3/4 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +ingest-snapshot range-id=3 replica-id=4 raft-index=12 raft-term=6 span=a,b keys=a,ab ts=12 +---- +== r3/4 inited == + hs: term:0 vote:0 commit:12 trunc-state: index:12 term:6 + span: {a-b} desc-ts: 12 appliedState: index: 12, term: 6 + global: "a"/0.000000012,0=a.12 "ab"/0.000000012,0=ab.12 +== r2/3 inited == + hs: term:0 vote:0 commit:13 trunc-state: index:13 term:6 + span: {b-e} desc-ts: 13 appliedState: index: 13, term: 6 + global: "b"/0.000000013,0=b.13 "c"/0.000000013,0=c.13 "d"/0.000000013,0=d.13 + +create-uninit range-id=4 replica-id=5 +---- +== r3/4 inited == + hs: term:0 vote:0 commit:12 trunc-state: index:12 term:6 + span: {a-b} desc-ts: 12 appliedState: index: 12, term: 6 + global: "a"/0.000000012,0=a.12 "ab"/0.000000012,0=ab.12 +== r2/3 inited == + hs: term:0 vote:0 commit:13 trunc-state: index:13 term:6 + span: {b-e} desc-ts: 13 appliedState: index: 13, term: 6 + global: "b"/0.000000013,0=b.13 "c"/0.000000013,0=c.13 "d"/0.000000013,0=d.13 +== r4/5 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +ingest-snapshot range-id=4 replica-id=5 raft-index=12 raft-term=6 span=e,g keys=e,f ts=12 +---- +== r3/4 inited == + hs: term:0 vote:0 commit:12 trunc-state: index:12 term:6 + span: {a-b} desc-ts: 12 appliedState: index: 12, term: 6 + global: "a"/0.000000012,0=a.12 "ab"/0.000000012,0=ab.12 +== r2/3 inited == + hs: term:0 vote:0 commit:13 trunc-state: index:13 term:6 + span: {b-e} desc-ts: 13 appliedState: index: 13, term: 6 + global: "b"/0.000000013,0=b.13 "c"/0.000000013,0=c.13 "d"/0.000000013,0=d.13 +== r4/5 inited == + hs: term:0 vote:0 commit:12 trunc-state: index:12 term:6 + span: {e-g} desc-ts: 12 appliedState: index: 12, term: 6 + global: "e"/0.000000012,0=e.12 "f"/0.000000012,0=f.12 + +# RangeID 3 [a,b), RangeID 2 [b,e), RangeID 4 [e,g). Try to subsume 3 and 4 by +# expanding span of 2 to [ab,f). +ingest-snapshot range-id=2 replica-id=3 raft-index=14 raft-term=6 span=ab,f keys=ab,c,e ts=14 +---- +additional range with rangeID 3 is being subsumed + +# Did not list all the rangeIDs being subsumed. +ingest-snapshot range-id=2 replica-id=3 raft-index=14 raft-term=6 span=ab,f keys=ab,c,e ts=14 subsumed-range-ids=3 +---- +additional range with rangeID 4 is being subsumed + +# Did not list all the rangeIDs being subsumed. +ingest-snapshot range-id=2 replica-id=3 raft-index=14 raft-term=6 span=ab,f keys=ab,c,e ts=14 subsumed-range-ids=4 +---- +additional range with rangeID 3 is being subsumed + +# Subsumption succeeds. +ingest-snapshot range-id=2 replica-id=3 raft-index=14 raft-term=6 span=ab,f keys=ab,c,e ts=14 subsumed-range-ids=3,4 +---- +== r2/3 inited == + hs: term:0 vote:0 commit:14 trunc-state: index:14 term:6 + span: {ab-f} desc-ts: 14 appliedState: index: 14, term: 6 + global: "ab"/0.000000014,0=ab.14 "c"/0.000000014,0=c.14 "e"/0.000000014,0=e.14 diff --git a/pkg/kv/kvserver/testdata/replicas_storage/split b/pkg/kv/kvserver/testdata/replicas_storage/split new file mode 100644 index 000000000000..e10f05e08580 --- /dev/null +++ b/pkg/kv/kvserver/testdata/replicas_storage/split @@ -0,0 +1,202 @@ +new +---- + +create-uninit range-id=1 replica-id=2 +---- +== r1/2 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +ingest-snapshot range-id=1 replica-id=2 raft-index=11 raft-term=5 span=a,e keys=a,b,c,d ts=12 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-e} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 + +# Create rangeID 2 which will be the RHS of a split. +create-uninit range-id=2 replica-id=3 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-e} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 +== r2/3 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +# Update the HardState for rangeID 2 since an uninitialized replica can vote. +mutation range-id=2 replica-id=3 apply=false raft-term=6 raft-vote=7 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-e} desc-ts: 12 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 +== r2/3 uninited == + hs: term:6 vote:7 commit:0 trunc-state: index:0 term:0 + +# Split rangeID 1 where the RHS has existing raft state. +split-replica range-id=1 replica-id=2 rhs-range-id=2 rhs-replica-id=3 rhs-span=c,e ts=13 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-c} desc-ts: 13 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 + +# Create rangeID 3 that will be removed, creating a RangeTombstone preventing +# replicaID's < 5. +create-uninit range-id=3 replica-id=4 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-c} desc-ts: 13 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 +== r3/4 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +discard-replica range-id=3 replica-id=4 next-replica-id=5 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-c} desc-ts: 13 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 "b"/0.000000012,0=b.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 + +# Split rangeID 1 where the RHS is rangeID 3 and replicaID 4, which has +# already been removed. The global state at key b is also deleted. +split-replica range-id=1 replica-id=2 rhs-range-id=3 rhs-replica-id=4 rhs-span=b,c ts=14 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 + + +# Create, remove and create again rangeID 4. +create-uninit range-id=4 replica-id=5 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 +== r4/5 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +discard-replica range-id=4 replica-id=5 next-replica-id=6 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 + +create-uninit range-id=4 replica-id=6 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 +== r4/6 uninited == + hs: term:0 vote:0 commit:0 next-replica-id: 6 trunc-state: index:0 term:0 + +mutation range-id=4 replica-id=6 apply=false raft-term=6 raft-vote=7 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-e} desc-ts: 13 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 "d"/0.000000012,0=d.12 +== r4/6 uninited == + hs: term:6 vote:7 commit:0 next-replica-id: 6 trunc-state: index:0 term:0 + +# Split rangeID 2 where the RHS is rangeID 4 and replicaID 5, which has +# already been removed, though replicaID 6 is present. +split-replica range-id=2 replica-id=3 rhs-range-id=4 rhs-replica-id=5 rhs-span=d,e ts=15 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-d} desc-ts: 15 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 +== r4/6 uninited == + hs: term:6 vote:7 commit:0 next-replica-id: 6 trunc-state: index:0 term:0 + +# Test split via snapshot ingestion by first initializing a new range. +create-uninit range-id=5 replica-id=6 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-d} desc-ts: 15 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 +== r4/6 uninited == + hs: term:6 vote:7 commit:0 next-replica-id: 6 trunc-state: index:0 term:0 +== r5/6 uninited == + hs: term:0 vote:0 commit:0 trunc-state: index:0 term:0 + +ingest-snapshot range-id=5 replica-id=6 raft-index=11 raft-term=5 span=e,h keys=e,f,g ts=12 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-d} desc-ts: 15 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 +== r5/6 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {e-h} desc-ts: 12 appliedState: index: 11, term: 5 + global: "e"/0.000000012,0=e.12 "f"/0.000000012,0=f.12 "g"/0.000000012,0=g.12 +== r4/6 uninited == + hs: term:6 vote:7 commit:0 next-replica-id: 6 trunc-state: index:0 term:0 + +ingest-snapshot range-id=5 replica-id=6 raft-index=12 raft-term=6 span=f,g keys=f ts=14 +---- +== r1/2 inited == + hs: term:0 vote:0 commit:11 trunc-state: index:11 term:5 + span: {a-b} desc-ts: 14 appliedState: index: 11, term: 5 + global: "a"/0.000000012,0=a.12 +== r2/3 inited == + hs: term:6 vote:7 commit:10 trunc-state: index:10 term:5 + span: {c-d} desc-ts: 15 appliedState: index: 10, term: 5 + global: "c"/0.000000012,0=c.12 +== r5/6 inited == + hs: term:0 vote:0 commit:12 trunc-state: index:12 term:6 + span: {f-g} desc-ts: 14 appliedState: index: 12, term: 6 + global: "f"/0.000000014,0=f.14 +== r4/6 uninited == + hs: term:6 vote:7 commit:0 next-replica-id: 6 trunc-state: index:0 term:0 diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index ffa749daf3d1..dec93405fff6 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -31,7 +31,6 @@ go_library( "pebble_mvcc_scanner.go", "point_synthesizing_iter.go", "read_as_of_iterator.go", - "replicas_storage.go", "resource_limiter.go", "row_counter.go", "slice.go", @@ -94,7 +93,6 @@ go_library( "@com_github_elastic_gosigar//:gosigar", "@com_github_gogo_protobuf//proto", "@com_github_prometheus_client_model//go", - "@io_etcd_go_etcd_raft_v3//raftpb", ], ) diff --git a/pkg/storage/open.go b/pkg/storage/open.go index 6cd9bd82f199..66cf26955826 100644 --- a/pkg/storage/open.go +++ b/pkg/storage/open.go @@ -222,6 +222,15 @@ func InMemory() Location { } } +// MakeLocation constructs a Location that instructs the storage engine to +// store data in fs. +func MakeLocation(fs vfs.FS) Location { + return Location{ + dir: "", + fs: fs, + } +} + type engineConfig struct { PebbleConfig // cacheSize is stored separately so that we can avoid constructing the diff --git a/pkg/storage/replicas_storage.go b/pkg/storage/replicas_storage.go deleted file mode 100644 index 0c574e2a2b0b..000000000000 --- a/pkg/storage/replicas_storage.go +++ /dev/null @@ -1,845 +0,0 @@ -// Copyright 2021 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 storage - -import ( - "github.com/cockroachdb/cockroach/pkg/roachpb" - "go.etcd.io/etcd/raft/v3/raftpb" -) - -// TODO(sumeer): -// Steps: -// - Pick names for C1-C3 and D1-D2, to make it easier to remember what we are -// referring to in various places. -// - Consider separating RecoveryInconsistentReplica into two different states -// RecoveryRaftAndStateInconsistent, and RecoveryStateInconsistent. -// - Proof sketch. -// - Read cockroach debug check-store to see additional assertion ideas/code -// we can lift. -// - Implement interface. -// - Unit tests and randomized tests, including engine restarts that lose -// state (using vfs.NewStrictMem). -// - Benchmarks comparing single and two engine implementations. -// - Race-build dynamically asserts that SSTs or MutationBatches that are -// passed through this interface only touch the keys they are allowed to -// touch. -// - Integration (can be done incrementally). -// - Misc cleanup: -// - Merges should cleanup QueueLastProcessedKey, - -// High-level overview: -// -// ReplicasStorage provides an interface to manage the persistent state that -// includes the lifecycle of a range replica, its raft log, and the state -// machine state. The implementation(s) are expected to be a stateless wrapper -// around persistent state in the underlying engine(s) (any state they -// maintain in-memory is simply a performance optimization and always -// in-sync with the persistent state). Since this abstraction is mutating the -// same underlying engine state that was previously mutated via lower-level -// interfaces, and is not a data-structure in the usual sense, we can migrate -// callers incrementally to use this interface. That is, callers that use this -// interface, and those that use the lower-level engine interfaces can -// co-exist correctly. -// -// TODO(sumeer): this co-existence is not completely true since the following -// attempts to define an ideal interface where no sst or MutationBatch touches -// both raft engine state or state machine engine state. Which means transient -// inconsistencies can develop. We will either -// - alter this interface to be more pragmatic, i.e., be a step towards the -// ideal interface, but not the end product, once we have settled on the -// ideal interface. -// - ensure that the minimal integration step includes ReplicasStorage.Init, -// which can eliminate any inconsistencies caused by an inopportune crash. -// Hopefully, the latter is sufficient. -// -// We consider the following distinct kinds of persistent state: -// - State machine state: It contains all replicated keys: replicated range-id -// local keys, range local keys, range lock keys, global keys. NB: this -// includes the RangeAppliedState and the RangeDescriptor. -// -// - Raft state: This includes all the unreplicated range-ID local key names -// prefixed by Raft. We will loosely refer to all of these as "raft state". -// RangeLastReplicaGCTimestamp changes are ignored below, since it is -// best-effort persistent state used to pace queues, and the caller is -// allowed to mutate it out-of-band. However when deleting a replica, -// ReplicasStorage will clear that key too. RangeLastReplicaGCTimestamp is -// placed in this state category because it is not replicated state machine -// state. -// -// The interface requires that any mutation (batch or sst) only touch one of -// these kinds of state. This discipline will allow us to eventually separate -// the engines containing these two kinds of state. This interface is not -// relevant for store local keys though they will be in the latter engine. The -// interface does not allow the caller to specify whether to sync a mutation -// to the raft log or state machine state -- that decision is left to the -// implementation of ReplicasStorage (with a couple of small exceptions where -// a sync is explicitly requested, which are explained later). So even when we -// don't separate the state machine and raft engines, this abstraction forces -// us to reason more carefully about effects of crashes, and when to sync, and -// allow us to test more thoroughly. -// -// RangeTombstoneKey: This is an unreplicated key that is critical to the -// replica lifecycle. Since it is unreplicated, it is not part of the state -// machine. However, placing it in the category of "raft state" with the other -// unreplicated keys turns out to be complicated: -// (a) in various range merge situations (including replicas being subsumed -// during snapshot application) we need to atomically move the state machine -// forward for the surviving range, delete the state machine state for the -// subsumed range(s) and set the RangeTombstone. -// (b) when removing a replica due to rebalancing we need to atomically remove -// the state machine and set the RangeTombstone. -// For these reasons, we require that the RangeTombstone be in the same engine -// as the state machine state. However, it can only be mutated by -// ReplicasStorage. -// -// Note that the interface is not currently designed such that raft log writes -// avoid syncing to disk as discussed in -// https://github.com/cockroachdb/cockroach/issues/17500#issuecomment-727094672 -// and followup comments on that issue. However, having a clean storage -// abstraction should be a reasonable step in that direction. -// -// ReplicasStorage does not interpret most of the data in the state machine. -// It expects mutations to that state to be provided as an opaque batch, or a -// set of files to be ingested. There are a few exceptions where it can read -// state machine state, mainly when recovering from a crash, so as to make -// changes to get to a consistent state. -// - RangeAppliedStateKey: needs to read this in order to truncate the log, -// both as part of regular log truncation (see the comment section on -// "Normal Replica Operation") and on crash recovery (see the comment -// section on "Replica Initialization" and "Crash Recovery" for details). -// - RangeDescriptorKey: needs to read this to discover the spans of -// initialized replicas (see the comment sections on "Invariants" and "Crash -// Recovery"). -// -// A corollary to this lack of interpretation is that reads of the state -// machine are not handled by this interface, though it does expose some -// metadata in case the reader want to be sure that the replica it is trying -// to read actually exists in storage. ReplicasStorage also does not offer an -// interface to construct changes to the state machine state. It applies -// changes, and requires the caller to obey some simple invariants to not -// cause inconsistencies. It is aware of the keyspace occupied by a replica -// and the difference between rangeID keys and range keys -- it needs this -// awareness to discard (parts of) replica state when replicas are moved or -// merged away. -// -// ReplicasStorage does interpret the raft state (all the unreplicated -// range-ID local key names prefixed by Raft), and the RangeTombstoneKey. This -// is necessary for it to be able to maintain invariants spanning the raft log -// and the state machine (related to raft log truncation, replica lifetime -// etc.), including reapplying raft log entries on restart, to the state -// machine. All accesses (read or write) to the raft log and RangeTombstoneKey -// must happen via ReplicasStorage. ReplicasStorage does not by itself apply -// committed raft log entries to the state machine in a running system -- this -// is because state machine application in a running system has complex -// data-structure side-effects that are outside the scope of ReplicasStorage. - -// It is told by the caller to apply a committed entry, which also requires -// the caller to provide the state machine changes. ReplicasStorage does apply -// "simple" entries directly to the state machine during Init to fix any -// inconsistency of the state machine caused by durable sst ingestion and -// non-durable batch application (see ApplyCommitted* methods in the -// interface). Since these could be preceded by non-durable configuration -// changes, the notion of "simple" entries includes configuration changes, -// except for splits and merges (which we sync to ensure durability -- this is -// justified in the section below on "Implementation constraints on -// ReplicasStorage"). -// -// TODO(sumeer): -// https://github.com/etcd-io/etcd/issues/7625#issuecomment-489232411 relies -// on a correctness argument based on bounded regression of conf changes. -// Consider strengthening that correctness argument by making the committed -// index durable for a conf change before applying it. We could introduce a -// `ApplyConfChange(MutationBatch, highestRaftIndex uint64)` method, like we -// have for ingestion, and first sync the Commit state if needed. That way we -// will not have any conf change regression. -// -// ============================================================================ -// Invariants: -// -// INVARIANT (RaftAndStateConsistency): when there is any data in the state -// machine associated with a given RangeID, there is a corresponding -// internally consistent Raft state (according to etcd/raft) that is also -// consistent with the applied state on the state machine (i.e. the latter -// references a valid log position). Specifically, -// - HardState.Commit >= RangeAppliedState.RaftAppliedIndex -// - if HardState.Commit > RangeAppliedState.RaftAppliedIndex, it points to an -// entry in the raft log. -// - RaftTruncatedState.{Index,Term} must be a valid value corresponding to -// what was truncated. If there are no raft log entries, -// RaftTruncatedState.{Index,Term} must equal -// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm}. -// -// INVARIANT (StateConsistency): when there is any data in the state machine -// associated with a given RangeID, it will reflect the replicated state at -// the corresponding applied index (i.e., it materializes the replicated log -// at this index). -// Additionally, a range is first created with a RangeDescriptor present -// (external invariant) and neither the callers nor ReplicasStorage will ever -// selectively delete it. NOTE: subsumption as part of a range merge does -// delete the RangeDescriptor, but the Replica atomically ceases to exist in -// the process. -// Specifically, -// - The state machine state must be consistent with the value of -// RaftAppliedIndex, i.e., it equals a state generated from the full history -// of this range (for a range that has never been the LHS of a merge, this -// is the initial snapshot when the range came into being, followed by all -// subsequent raft log entries). -// - RaftAppliedIndex >= raftInitialLogIndex -// - RaftAppliedIndexTerm >= raftInitialLogTerm -// - Has at least 1 non-provisional RangeDescriptor. -// - Regression of the HardState.Commit and RaftAppliedIndex is permitted due -// to a crash except for the following: -// - Split that has progressed to applying a state machine change that -// results in a non-provisional RangeDescriptor for the RHS must not -// regress after the crash (i.e., must sync application of the split -// trigger). -// - Merge that has progressed to applying a state machine change that -// deletes the RangeDescriptor for the RHS must not regress after the -// crash (i.e., must sync application of the merge trigger). -// One could possibly relax these split/merge invariants but the corner -// cases are very subtle and make it hard to reason about correctness. -// As an example, see the discussion about "not syncing for splits" in -// https://github.com/cockroachdb/cockroach/pull/72745#pullrequestreview-807989047 -// -// INVARIANT (InterReplicaStateConsistency): The latest non-provisional -// RangeDescriptors of replicas with state machine state have spans that do -// not overlap. We use the term replica-descriptor to refer to this latest -// non-provisional RangeDescriptor, in the text below. -// -// DEFINITION (InitializedStateMachine): a Replica with state -// InitializedStateMachine, has state machine state and obeys the invariants -// RaftAndStateConsistency, StateConsistency, InterReplicaStateConsistency. -// -// DEFINITION (DeletedReplica): it can be convenient to reference Replicas -// that once existed but no longer do, as evidenced by the presence of a -// RangeTombstone for a RangeID, but no state machine or raft state. -// RangeTombstone.NextReplicaID is populated with a value > the last ReplicaID -// seen by ReplicasStorage for this range. Note that RangeTombstone is -// populated even for ranges that no longer exist (RHS of a merge) -- in this -// case it is set to a constant (mergedTombstoneReplicaID, equal to MaxInt32). -// -// DEFINITION (UninitializedStateMachine): this is a Replica with no state -// machine, i.e., there is Raft state and possibly a RangeTombstone. In -// particular, there is no RangeDescriptor and so it has no key span -// associated with it yet. -// The HardState{Term,Vote} can have arbitrary values since this replica can -// vote. However, it has a zero HardState.Commit and no log entries -- this -// Raft invariant is upheld externally by a combination of mostly external -// invariants: -// A new Range is initialized with all Replicas at truncated index equal to -// raftInitialLogIndex (10) (so they are in InitializedStateMachine state), -// and any future Replicas will be initialized via a snapshot reflecting a -// nonzero applied index >= 10. In particular, prior to receiving the -// snapshot, no log entries can be sent to the Replica. And etcd/raft only -// communicates Commit entries for which the recipient has the log entry. -// -// -// Some of the above invariants may be violated when non-durable state is lost -// due to a crash, but ReplicasStorage.Init is required to fix the persistent -// state such that the above invariants are true. These are not exposed to the -// user of the interface. We list these below. -// -// DEFINITION (RecoveryDeletingReplica): a Replica whose Raft state requires a -// nonzero applied index in the state machine, but there is no state machine -// state. This is an intermediate state entered when transitioning from -// InitializedStateMachine to DeletedReplica, after the state machine state -// has been deleted and RangeTombstoneKey updated and before the raft state -// has been deleted. This is distinguishable from UninitializedStateMachine -// since RaftTruncatedState.{Index,Term} are guaranteed to exist and have -// values >= raftInitialLogIndex, raftInitialLogTerm. ReplicasStorage.Init -// will transition out of this state into DeletedReplica state. -// -// DEFINITION (RecoveryInconsistentReplica): This is a Replica that mostly -// looks like to be in state InitializedStateMachine, but has suffered -// regression in durable state such that the state machine has advanced past -// HardState.Commit, or a snapshot has been applied and all raft log entries -// are < RaftAppliedIndex, i.e., it violates RaftAndStateConsistency -// invariants. More severely, it can also violate StateConsistency invariants -// by having durably ingested SSTables but not yet updated the -// RaftAppliedIndex to reflect that state machine change. ReplicasStorage.Init -// restores all the invariants needed by an InitializedStateMachine replica, -// by fixing the raft log to be consistent with the state machine, and -// re-applying log entries up to HardState.Commit (except for log entries that -// indicate a split or merge -- see below). -// -// Replica state transitions: -// - Initial state: UninitializedStateMachine -// - Final state: DeletedReplica -// - UninitializedStateMachine => DeletedReplica, InitializedStateMachine -// - InitializedStateMachine => RecoveryDeletingReplica, RecoveryInconsistentReplica -// - RecoveryDeletingReplica => DeletedReplica -// - RecoveryInconsistentReplica => InitializedStateMachine -// -// ============================================================================ -// Implementation constraints on ReplicasStorage: -// - Splits and Merges typically happen by applying an entry in the raft log. -// It is feasible for ReplicasStorage.Init to apply such committed entries. -// However, the logic in such cases can add additional mutations to the -// batch in the raft log, that have nothing to do with the normal scope of -// what ReplicasStorage is concerned with. For example, splitPreApply has -// logic to set RangeAppliedState.RaftClosedTimestamp. For this reason -// ReplicasStorage ensures durability of split/merge application and does -// not apply any Split/Merge log entries in ReplicasStorage.Init. -// -// ============================================================================ -// Replica Initialization: -// -// Since ReplicasStorage does not permit atomic updates spanning the state -// machine and raft state (even if they are a single engine), replica creation -// is sequenced as (* indicates durable writes): -// -// - [C1*] creation of RaftHardStateKey in raft state with -// {Term:0,Vote:0,Commit:0}. This is a replica in UninitializedStateMachine -// state. -// - [C2*] creation of state machine state (via snapshot or some synthesized -// state for rangeID and range local keys in the case of split). -// - [C3] set RaftTruncatedStateKey with RaftTruncatedState.{Index,Term} equal -// to RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm} and adjust -// of RaftHardStateKey (specifically HardState.Commit needs to be set to -// RangeAppliedState.RaftAppliedIndex -- see below for details). Also -// discard all raft log entries if any (see below). At this point the -// replica is in InitializedStateMachine state. -// -// Every step above needs to be atomic. Note that we are doing 2 syncs, in -// steps C1 and C2, for the split case, where we currently do 1 sync -- splits -// are not common enough for this to matter. If we did not sync C2 we could -// start adding log entries after C3 but lose the state machine state in the -// case of a crash, which would violate the replica state invariants. -// -// An initialized replica that receives a snapshot because it has lagged -// behind will execute C2 and C3. The C3 step throws away all the existing -// raft log entries. So a precondition for applying such a snapshot is: -// - The raft log does not have entries beyond the snapshot's -// RangeAppliedState.RaftAppliedIndex. If it did, there would be no benefit -// in applying this snapshot. -// The following etcd/raft code -// https://github.com/etcd-io/etcd/blob/7572a61a39d4eaad596ab8d9364f7df9a84ff4a3/raft/raft.go#L1584-L1589 -// ensures this behavior -- if the raft log entry corresponding to the -// snapshot is already present locally, it only advances the commit index to -// the snapshot index, and does not actually apply the snapshot. -// - Corollary: since HardState.Commit cannot refer to log entries beyond the -// locally persisted ones, the existing HardState.Commit <= -// RangeAppliedState.RaftAppliedIndex, so the HardState manipulation done in -// step C3 will only need to increase the value of HardState.Commit. -// -// Why C2 before C3?: -// If we performed step C3 before C2, there is a possibility that a crash -// prevents C2. Now we would need to rollback the change made in C3 to reach a -// fully consistent state on crash recovery. Rolling back HardState.Commit is -// easy, since there is no raft log, we can set it to -// RangeAppliedState.RaftAppliedIndex if it exists, else 0. Similarly, we can -// rollback RaftTruncatedState by either: -// - deleting it if the RangeAppliedState does not exist, which implies C3 did -// not happen. -// - if RangeAppliedState exists, roll back RaftTruncatedState.{Index,Term} to -// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm}. Note that this -// is a case where an already initialized lagging replica has a snapshot -// being applied. -// The correctness problem with doing C3 before C2 is that the store violates -// raft promises it has made earlier. For example, if the state machine had -// applied index 20 and the raft log contained [15, 25), then this store is -// part of the quorum that causes [21, 25) to commit. If after the crash this -// store has applied index 20 and has no raft state, it is in effect an -// unavailable replica, since it no longer has [21, 25). -// -// Rolling forward if crash after C2 and before C3: -// ReplicasStorage.Init will roll forward to C3 when initializing itself. -// - If HardState.Commit < RangeAppliedState.RaftAppliedIndex, update -// HardState.Commit -// - If RaftTruncatedState does not exist, or RaftTruncatedState.Index < -// RangeAppliedState.RaftAppliedIndex and all log entries are <= -// RangeAppliedState.RaftAppliedIndex -// - Discard all raft log entries. -// - Set RaftTruncatedState.{Index,Term} using -// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm} -// -// Aside: -// Since we now have RangeAppliesState.RaftAppliedIndexTerm, constructing an -// outgoing snapshot only involves reading state machine state (this is a tiny -// bit related to #72222, in that we are also assuming here that the outgoing -// snapshot is constructed purely by reading state machine engine state). -// -// ============================================================================ -// Replica Deletion: -// -// Replica deletion is sequenced as the following steps (* indicates durable -// writes): - -// - [D1*] deletion of state machine state (iff the replica is in state -// InitializedStateMachine) and write to the RangeTombstoneKey. If prior to -// this step the range was in state InitializedStateMachine, it is now in -// state RecoveryDeletingReplica. If it was in state UninitializedStateMachine, -// it continues to be in the same state (but with a RangeTombstone that will -// prevent this replica from ever transitioning to InitializedStateMachine). -// This latter case can occur for various reasons: one cause is this range -// is the RHS of a split where the split has not yet happened, but we've -// created an uninitialized RHS. So we can't delete the state machine state -// for the RHS since it doesn't exist yet (there is some replicated state in -// the state machine that could in the future belong to the RHS, but not -// yet, and we don't know the span of that future RHS either). By updating -// the RangeTombstone, when the split occurs, D1 will be repeated. -// - This step will only delete the RangeID local keys, when this replica -// deletion is due to a merge (the range itself is being deleted). -// - [D2] deletion of all Raft state for this RangeID, i.e., RaftHardStateKey, -// RaftTruncatedStateKey, log entries, RangeLastReplicaGCTimestampKey. -// -// Every step above needs to be atomic. One of the reasons to sync after D1 it -// that we could later execute C1 when adding the range back to this store, and -// then crash. On crash recovery we'd find the raft HardState and old state -// machine state and incorrectly think this is an initialized replica. -// -// Note that we don't delete the RangeTombstoneKey even when the range itself -// is being deleted (due to a merge). The replay protection offered by it is -// more important than the minuscule cost of leaking a RangeTombstoneKey per -// range. It is possible to have some cleanup of RangeTombstoneKeys for long -// dead ranges, but it is outside of the scope here. -// -// A crash after D1 will result in a replica either in state -// RecoveryDeletingReplica or UninitializedStateMachine. For the latter, some -// code above ReplicasStorage will eventually ask for the replica to be -// cleaned up (see https://github.com/cockroachdb/cockroach/issues/73424 for -// cleanup improvement). For the former, ReplicasStorage.Init will execute D2. -// -// ============================================================================ -// Normal Replica Operation: -// -// - ReplicasStorage is used to append/replace log entries and update -// HardState. This is done via a RaftMutationBatch. There is a -// RaftMutationBatch.MustSync that the caller uses to specify the minimal -// sync requirements imposed by Raft (ReplicasStorage is not in the business -// of understanding Raft correctness requirements). Typically MustSync will -// be true only if entries are appended, or a vote and/or term change has to -// be recorded. In particular, a change solely to HardState.Commit would -// have MustSync=false. See -// https://github.com/etcd-io/etcd/blob/7572a61a39d4eaad596ab8d9364f7df9a84ff4a3/raft/node.go#L584-L593. -// Note that this means that HardState.Commit can regress and become less -// than RangeAppliedState.RaftAppliedIndex in case of a crash. We will fix -// this in ReplicasStorage.Init, as discussed later. -// -// - The caller keeps track of HardState.Commit, since it constructed -// HardState for the RaftMutationBatch. It applies committed entries to the -// state machine using ApplyCommittedUsingIngest and ApplyCommittedBatch. -// The ApplyCommitted* methods should not be used for log entries that are -// performing splits or merges -- the caller should do those by calling -// SplitReplica and MergeReplicas. ReplicasStorage decides when it is -// necessary to sync -- ApplyCommitted* will not sync the state machine, and -// SplitReplica/MergeReplicas will sync the state machine. Note that the -// caller may not need to read a raft entry from ReplicasStorage in order to -// apply it, if it happens to have stashed it somewhere in its in-memory -// data-structures. -// -// For log entries that are ingesting side-loaded files, the application of -// a single entry is split into a pair, ApplyCommittedUsingIngest, that -// usually does not update the RaftAppliedIndex and then ApplyCommittedBatch -// which updates the RaftAppliedIndex. A crash after the first and before -// the second leaves the state machine in an inconsistent state -// (RecoveryInconsistentReplica) which needs to be fixed by -// ReplicasStorage.Init. For this reason, ReplicasStorage keeps track of the -// highest HardState.Commit known to be durable, and requires -// ApplyCommittedUsingIngest to provide the highestRaftIndex of the changes -// included in the files. ReplicasStorage will sync the raft state if -// needed, to make the highestRaftIndex durable, before ingesting these -// files. This prevents regression of HardState.Commit past an index that -// contains side-loaded files. Note that this assumes that -// ReplicasStorage.Init has the capability of applying all raft log entries -// except for splits and merges (we've already mentioned that splits/merges -// are made durable at application time). -// -// - Log truncation is advised by the caller, based on various signals -// relevant to the proper functioning of the distributed raft group, except -// that the caller is unaware of what is durable in the state machine. Hence -// the advise provided by the caller serves as an upper bound of what can be -// truncated. Log truncation does not need to be synced. The implementation -// of ReplicasStorage reads the durable state of -// RangeAppliedState.RaftAppliedIndex to use as an additional upper bound of -// what can be truncated. -// -// - Range merges impose an additional requirement: the merge protocol (at a -// higher layer) needs the RHS replica of a merge to have applied all raft -// entries up to a specified index and that this application is durable. To -// ensure the durability we expose a SyncStateMachine method for the higher -// layer. -// -// ============================================================================ -// Crash Recovery: -// ReplicasStorage needs to be self contained in the sense that it must be -// able to execute state changes to reach a fully consistent state without -// needing any external input, as part of its initialization. Init will block -// until all the raft and state machine states have been made mutually -// consistent. -// - Iterate over RaftHardStateKeys and identify a set of replicas R_r. This -// is efficiently done by seeking to the current RangeID+1. -// - Iterate over RangeDescriptorKeys and identify a set of replicas R_s. This -// is efficiently done by using the latest non-provisional RangeDescriptor -// (replica-descriptor) of the current range and then seeking to the end key -// of the range's span. -// - Note that this way of skipping spans will ensure that we will not find -// RangeDescriptors that have overlapping spans, which is ideally an invariant -// we should verify. Instead of verifying that invariant, which is expensive, -// we can additionally iterate over all the RangeAppliedStateKeys, which are -// RangeID local keys -- this iteration can be accomplished by seeking using -// current RangeID+1. If we find RangeAppliedStateKeys whose RangeID is not -// mentioned in a corresponding RangeDescriptor we have an invariant violation. -// - The set R_s - R_r must be empty, i.e., R_s is a subset of R_r. -// - The set R_r - R_s are replicas are either in state -// UninitializedStateMachine or RecoveryDeletingReplica. -// - Remove RecoveryDeletingReplica replicas by transitioning them to DeletedReplica -// by executing D2. -// - The set R_s are replicas that ought to be in state -// InitializedStateMachine, though may have been in the middle of that state -// transition, or become inconsistent for other reasons mentioned earlier. -// That is, they are potentially in RecoveryInconsistentReplica state. -// - If RangeAppliedState.RaftAppliedIndex > HardState.Commit (or there is -// no HardState), execute the following atomically: -// - If there are no log entries or all log entries are < -// RaftAppliedIndex: remove all log entries and set -// RaftTruncatedState.{Index,Term} equal to -// RangeAppliedState.{RaftAppliedIndex,RaftAppliedIndexTerm}. -// - Set HardState.Commit to RaftAppliedIndex. -// These steps handle (a) crash in the middle of replica creation (doing -// step C3), and (b) regression of HardState.Commit under normal replica -// operation. The RaftAndStateConsistency invariant now holds. -// - The StateConsistency invariant may not hold. To ensure that it holds: -// for ranges whose RangeAppliedState.RaftAppliedIndex < HardState.Commit, -// apply log entries, including those that remove this replica, until one -// encounters a log entry that is performing a split or merge. -// - InitializedStateMachine replicas: -// - using the replica-descriptors, check that the spans do not overlap. -// - This InterReplicaStateConsistency invariant must also hold before we -// fixed the RecoveryInconsistentReplicas, so we could additionally check -// it then. -// ============================================================================ - -// ReplicaState represents the current state of a range replica in this store. -type ReplicaState int - -const ( - // UninitializedStateMachine is a replica with raft state but no state - // machine. - UninitializedStateMachine ReplicaState = 0 - // InitializedStateMachine is a replica with both raft state and state - // machine. - InitializedStateMachine - // DeletedReplica is a replica with neither raft state or state machine. - DeletedReplica -) - -// FullReplicaID is a fully-qualified replica ID. -type FullReplicaID struct { - // RangeID is the id of the range. - RangeID roachpb.RangeID - // ReplicaID is the id of the replica. - ReplicaID roachpb.ReplicaID -} - -// ReplicaInfo provides the replica ID and state pair. -type ReplicaInfo struct { - FullReplicaID - // State of the replica. - State ReplicaState -} - -// MutationBatch can be committed to the underlying engine. Additionally it -// provides access to the underlying Batch. In some cases the implementation -// of ReplicasStorage will add additional mutations before committing. We -// expect the caller to know which engine to construct a batch from, in order -// to update the state machine or the raft state. ReplicasStorage does not -// hide such details since we expect the caller to mostly do reads using the -// engine Reader interface. -type MutationBatch interface { - // Commit writes the mutation to the engine. - Commit(sync bool) error - // Batch returns the underlying storage.Batch. - Batch() *Batch -} - -// RaftMutationBatch specifies mutations to the raft log entries and/or -// HardState. -type RaftMutationBatch struct { - MutationBatch - // [Lo, Hi) represents the raft log entries, if any in the MutationBatch. - // This is appending/overwriting entries in the raft log. That is, if the - // log is [a,b,c,d], with a at index 12 and one appends e at index 13, the - // result will be [a,e]. We assume the caller is upholding Raft semantics - // (such as not overwriting raft log entries that have been committed) -- - // ReplicasStorage is not in the business of validating that such semantics - // are being upheld. - Lo, Hi uint64 - // HardState, if non-nil, specifies the HardState value being set by - // MutationBatch. - HardState *raftpb.HardState - // MustSync is set to true if the mutation must be synced. - MustSync bool -} - -// RangeStorage is a handle for a FullReplicaID that provides the ability to -// write to the raft state and state machine state. This could have been named -// ReplicaStorage, but that sounds too similar to ReplicasStorage. Note that, -// even though a caller can have two different RangeStorage handles for the -// same range, if it has been added and removed and so has different -// ReplicaIDs, at most one of them will be in state != DeletedReplica. -type RangeStorage interface { - // FullReplicaID returns the FullReplicaID of this replica. - FullReplicaID() FullReplicaID - // State returns the ReplicaState of this replica. - State() ReplicaState - - // CurrentRaftEntriesRange returns [lo, hi) representing the locally stored - // raft entries. These are guaranteed to be locally durable. - CurrentRaftEntriesRange() (lo uint64, hi uint64, err error) - - // GetHardState returns the current HardState. HardState.Commit is not - // guaranteed to be durable. - GetHardState() (raftpb.HardState, error) - - // CanTruncateRaftIfStateMachineIsDurable provides a new upper bound on what - // can be truncated. - CanTruncateRaftIfStateMachineIsDurable(index uint64) - - // DoRaftMutation changes the raft state. This will also purge sideloaded - // files if any entries are being removed. - // REQUIRES: if rBatch.Lo < rBatch.Hi, the range is in state - // InitializedStateMachine. - DoRaftMutation(rBatch RaftMutationBatch) error - - // TODO(sumeer): - // - add raft log read methods. - // - what raft log stats do we need to maintain and expose (raftLogSize?)? - // We could accept a callback with a truncated index parameter that - // RangeStorage invokes whenever it truncates entries, and let the caller - // maintain the size. - - // State machine commands. - - // IngestRangeSnapshot ingests a snapshot for the range. - // - The replica-descriptor in the snapshot describes the range as equal to - // span. - // - The snapshot corresponds to application of the log up to - // raftAppliedIndex. raftAppliedIndexTerm is the term of that log entry. - // - sstPaths represent the ssts for this snapshot, and do not include - // anything other than state machine state and do not contain any keys - // outside span (after accounting for range local keys) and RangeID keys. - // - sstPaths include a RANGEDEL that will clear all the existing state - // machine state in the store for span (including local keys) "before" - // adding the snapshot state (see below for additional RANGEDELs that may - // be added by ReplicasStorage if the previous span for this replica was - // wider). - // NB: the ssts contain RangeAppliedState, RangeDescriptor (including - // possibly a provisional RangeDescriptor). Ingestion is the only way to - // initialize a range except for the RHS of a split. - // - // Snapshot ingestion will not be accepted in the following cases: - // - span overlaps with the (span in the) replica-descriptor of another - // range, unless the range is listed in subsumedRangeIDs. The ranges - // listed in subsumedRangeIDs must have spans that lie wholly within span. - // The span of a range can change only via IngestRangeSnapshot, - // SplitReplica, MergeRange, so ReplicasStorage can keep track of all - // spans without resorting to repeated reading from the engine. - // - the raft log already has entries beyond the snapshot (this is an - // invariant that Raft is already expected to maintain, so it is not - // an expected error). - // - // For reference, ReplicasStorage will do: - // - If this replica is already initialized compute - // rhsSpan = current span - span provided in this call - // rhsSpan is non-empty if we are moving the LHS past a split using a - // snapshot. In this case any replica(s) corresponding to rhsSpan cannot - // possibly be in InitializedStateMachine state (since it would be a - // violation of spans being non-overlapping). That is, they may be - // - participating in the raft group(s) for the RHS, but will not have any - // log entries. - // - rebalanced away. - // In either case, it is safe to clear all range local and global keys for - // the rhsSpan. ssts will be added to clear this state. - // - Add additional ssts that clear the RangeID keys for the subsumed - // ranges. - // - Atomically ingest the ssts. This does step C2 for this range and D1 for - // all subsumed ranges. This is durable. A crash after this step and - // before the next step is rolled forward in ReplicasStorage.Init. - // - Do steps C3 for this range and steps D2 for the subsumed ranges. - // - // In handleRaftReadyRaftMuLocked, if there is a snapshot, it will first - // call IngestRangeSnapshot, and then DoRaftMutation to change the - // HardState.{Term,Vote,Commit}. Note that the etcd/raft logic fast forwards - // the HardState.Commit to the snapshot index, so the DoRaftMutation will - // not actually change the stored value of HardState.Commit from what was - // already set in IngestRangeSnapshot. - IngestRangeSnapshot( - span roachpb.Span, raftAppliedIndex uint64, raftAppliedIndexTerm uint64, - sstPaths []string, subsumedRangeIDs []roachpb.RangeID) error - - // ApplyCommittedUsingIngest applies committed changes to the state machine - // state by ingesting sstPaths. highestRaftIndex is the highest index whose - // changes are included in the sstPaths. This is due to "sideloaded sst" - // raft log entries. These ssts do not contain an update to - // RangeAppliedState, so this call must be immediately followed by a call to - // ApplyCommittedBatch that does update the RangeAppliedState. - // It is possible for the node containing this store to crash prior to that - // call to ApplyCommittedBatch -- this is ok since ReplicasStorage.Init will - // replay this idempotent ingest and the following ApplyCommittedBatch. - // REQUIRES: replica is in state InitializedStateMachine. - ApplyCommittedUsingIngest(sstPaths []string, highestRaftIndex uint64) error - - // ApplyCommittedBatch applies committed changes to the state machine state. - // Does not sync. Do not use this for applying raft log entries that perform - // split, merge, or remove this replica (due to rebalancing) -- see the - // methods in ReplicasStorage that accomplish that. - // REQUIRES: replica is in state InitializedStateMachine (this is because we - // create a new range with the first log entry at raftInitialLogIndex (10), - // so a range always requires an initial state "snapshot" before it can - // apply raft entries). - ApplyCommittedBatch(smBatch MutationBatch) error - - // SyncStateMachine is for use by higher-level code that needs to ensure - // durability of the RHS of a merge. It simply sync the state machine state - // to ensure all previous mutations are durable. - // REQUIRES: replica is in state InitializedStateMachine. - SyncStateMachine() error -} - -// ReplicasStorage provides an interface to manage the persistent state of a -// store that includes the lifecycle of a range replica, its raft log, and the -// state machine state. See the comment at the top of the file. -type ReplicasStorage interface { - // Init will block until all the raft and state machine states have been - // made consistent. - Init() - - // CurrentRanges returns the replicas in the store. It does not return any - // ranges with state DeletedReplica, since it has no knowledge of them. - CurrentRanges() []ReplicaInfo - - // GetRangeTombstone returns the nextReplicaID in the range tombstone for - // the range, if any. - GetRangeTombstone(rangeID roachpb.RangeID) (nextReplicaID roachpb.ReplicaID, err error) - - // GetHandle returns a handle for a range listed in CurrentRanges(). - // ReplicasStorage will return the same handle object for a FullReplicaID - // during its lifetime. Once the FullReplicaID transitions to DeletedReplica - // state, ReplicasStorage will forget the RangeStorage handle and it is up - // to the caller to decide when to throw away a handle it may be holding - // (the handle is not really usable for doing anything once the range is - // deleted). - GetHandle(rr FullReplicaID) (RangeStorage, error) - - // CreateUninitializedRange is used when rebalancing is used to add a range - // to this store, or a peer informs this store that it has a replica of a - // range. This is the first step in creating a raft group for this - // FullReplicaID. It will return an error if: - // - This ReplicaID is too old based on the RangeTombstone.NextReplicaID - // - There already exists some state under any raft key for this range. - // - // The call will cause HardState to be initialized to - // {Term:0,Vote:0,Commit:0}. - // - // Typically there will be no state machine state for this range. However it - // is possible that a split is delayed and some other store has informed this - // store about the RHS of the split, in which case part of the state machine - // (except for the RangeID keys, RangeDescriptor) already exist. Note that - // this locally lagging split case is one where the RHS does not transition - // to initialized via anything other than a call to SplitReplica (i.e., does - // not apply a snapshot), except when the LHS moves past the split using a - // snapshot, in which case the RHS can also then apply a snapshot. - CreateUninitializedRange(rr FullReplicaID) (RangeStorage, error) - - // SplitReplica is called to split range r into a LHS and RHS, where the RHS - // is represented by rhsRR. The smBatch specifies the state machine state to - // modify for the LHS and RHS. For the RHS, the smBatch must be constructing - // the appropriate rangeID local state and range local state that doesn't - // already exist in the store (including the RangeDescriptor). rhsSpan is - // the span in the RangeDescriptor for the RHS. The following cases can - // occur: - // - // - [A1] RangeTombstone for the RHS indicates rhsRR.ReplicaID has already - // been removed. Two subcases: - // - [A11] There exists a HardState for rhsRR.RangeID: the range has been - // added back with a new ReplicaID. - // - [A12] There exists no HardState, so rhsRR.RangeID should not exist on - // this store. - // - [A2] RangeTombstone for the RHS indicates that rhsRR.ReplicaID has not - // been removed. - // - // For A11 and A12, the smBatch will be altered to clear all state in the - // state machine for the RHS. The final state RHS will be in for A11 is - // UninitializedStateMachine, for A12 is DeletedReplica. For A2, the smBatch - // is not altered and the final RHS state is InitializedStateMachine. If the - // final RHS state is not InitializedStateMachine, a nil RangeStorage will - // be returned. The application of smBatch is synced. - // - // From our earlier discussion of replica creation and deletion. - // - For case A2, the callee will perform step C1 if needed, then commit - // smBatch (step C2), and then perform step C3. - // - For case A11 there is no need to do step C1. Steps C2 and C3 cannot be - // performed since the RHS ReplicaID has changed and the state here is - // stale. All we are doing is cleaning up the state machine state for the - // RHS when committing smBatch. - // - For case A12, the callee is doing step D1 of deletion, by altering and - // committing smBatch. Since the RHS range never transitioned to - // initialized (it never had a RangeDescriptor), the deletion was unable - // to execute D1 when the HardState etc. was being deleted (it only - // executed D2). The RHS will continue to be in DeletedReplica state when - // the method returns. - // - // REQUIRES: The range being split is in state InitializedStateMachine, and - // RHS either does not exist or is in state UninitializedStateMachine. - // - // Called below Raft -- this is being called when the split transaction commits. - SplitReplica( - r RangeStorage, rhsRR FullReplicaID, rhsSpan roachpb.Span, smBatch MutationBatch, - ) (RangeStorage, error) - - // MergeReplicas is called to merge two range replicas. smBatch contains - // changes to the LHS state machine to incorporate the state of the RHS, and - // the intent resolution of the RHS RangeDescriptor. - // - // It will perform the following steps: - // - Alter smBatch to remove all RangeID local keys in the RHS and write the - // RangeTombstone to the RHS with value mergeTombstoneReplicaID. - // - // - Apply and sync smBatch, which transforms the LHS into the merged range, - // and performs step D1 for the RHS. The sync ensures that a crash after - // this step and before the next step will be rolled forward in Init. - // - // - Do step D2 for the RHS. - // - // REQUIRES: LHS and RHS are in state InitializedStateMachine, and RHS has - // durably applied all commands up to the merge. - // - // Code above this layer ensures the above durability of application of all - // commands in the RHS and additionally ensures that the RHS of a merge is - // immovable once in the critical phase (i.e. past the SubsumeRequest is - // handled), until the merge txn aborts (if it ever does). On the - // leaseholder handling Subsume, this is done by the Subsume. But we also - // prevent all future leaseholders from doing anything that would violate - // the critical phase by observing the deletion intent on the range - // descriptor. If a merge transaction commits, regardless of which replicas - // know about this yet, the LHS and RHS will be fully colocated. - // - // Called below Raft -- this is being called when the merge transaction commits. - MergeReplicas(lhsRS RangeStorage, rhsRS RangeStorage, smBatch MutationBatch) error - - // DiscardReplica is called to discard a replica that has been rebalanced - // away. The replica is either in UninitializedStateMachine or - // InitializedStateMachine state. There are multiple reasons for this to be - // called, such as the raft log entry that removes the replica is being - // applied, or ReplicaGCQueue notices that the replica is too old. Due to - // these multiple callers, ReplicasStorage is not in a position to compute - // what the nextReplicaID for the RangeTombstone should be. Therefore, it - // expects the caller to provide that value as a parameter. - DiscardReplica(r RangeStorage, nextReplicaID roachpb.ReplicaID) error -} - -// MakeSingleEngineReplicasStorage constructs a ReplicasStorage where the same -// Engine contains the the raft log and the state machine. -func MakeSingleEngineReplicasStorage(storeID roachpb.StoreID, eng Engine) ReplicasStorage { - // TODO(sumeer): implement - return nil -}