Skip to content

Commit

Permalink
storage: apply preemptive snaps under a real replicaID
Browse files Browse the repository at this point in the history
As of recenly, Raft [checks] that a peer receiving a snapshot is
actually contained in the configuration of a snapshot.  Preemptive
snapshots were failing that check because we applied them via a
temporary Raft group that had ID MaxUint64.

Preemptive snapshots are already a lie, so until we get rid of them
completely we'll have to lie a little more and use an ID that is
actually in the config. Any ID from the config will do; we take that of
the sending replica. Additionally, we're a little more careful than
previously to not leak anything about this temporary Raft group to the
outside world - previously we allowed it to send messages. This was
always a bad idea, but under a real ID this would be outright byzantine.

[checks]: https://github.com/etcd-io/etcd/blob/aa158f36b9832038a38f611d075264acb3874c8c/raft/raft.go#L1357-L1362

Release note: None
  • Loading branch information
tbg authored and danhhz committed Jul 18, 2019
1 parent 9575ac0 commit 509dbd2
Show file tree
Hide file tree
Showing 2 changed files with 13 additions and 5 deletions.
5 changes: 0 additions & 5 deletions pkg/storage/store_snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"fmt"
"io"
"math"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
Expand All @@ -35,10 +34,6 @@ import (
)

const (
// preemptiveSnapshotRaftGroupID is a bogus ID for which a Raft group is
// temporarily created during the application of a preemptive snapshot.
preemptiveSnapshotRaftGroupID = math.MaxUint64

// Messages that provide detail about why a preemptive snapshot was rejected.
snapshotStoreTooFullMsg = "store almost out of disk space"
snapshotApplySemBusyMsg = "store busy applying snapshots"
Expand Down
13 changes: 13 additions & 0 deletions pkg/storage/store_snapshot_preemptive.go
Original file line number Diff line number Diff line change
Expand Up @@ -282,6 +282,16 @@ func (s *Store) processPreemptiveSnapshotRequest(
if err != nil {
return roachpb.NewError(err)
}
// We need to create a temporary RawNode to process the snapshot. Raft
// internally runs safety checks on the snapshot, among them one that
// verifies that the peer is actually part of the configuration encoded
// in the snapshot. Awkwardly, it isn't actually a peer (preemptive
// snapshot...). To get around this, pretend the RawNode has the ID of a
// peer we know exists, namely the one that sent us the snap. This won't
// be persisted anywhere, and since we're only using the RawNode for
// this one snapshot, everything is ok. However, we'll make sure that
// no messages are sent in the resulting Ready.
preemptiveSnapshotRaftGroupID := uint64(snapHeader.RaftMessageRequest.FromReplica.ReplicaID)
raftGroup, err := raft.NewRawNode(
newRaftConfig(
raft.Storage((*replicaRaftStorage)(r)),
Expand All @@ -308,6 +318,9 @@ func (s *Store) processPreemptiveSnapshotRequest(
var ready raft.Ready
if raftGroup.HasReady() {
ready = raftGroup.Ready()
// See the comment above - we don't want this temporary Raft group
// to contact the outside world. Apply the snapshot and that's it.
ready.Messages = nil
}

if needTombstone {
Expand Down

0 comments on commit 509dbd2

Please sign in to comment.