Skip to content

Commit

Permalink
storage: test preemptive snapshot refusal
Browse files Browse the repository at this point in the history
  • Loading branch information
tbg committed Jul 12, 2016
1 parent 09da3fb commit 7f2218c
Show file tree
Hide file tree
Showing 2 changed files with 91 additions and 7 deletions.
15 changes: 8 additions & 7 deletions storage/replica_raftstorage.go
Original file line number Diff line number Diff line change
Expand Up @@ -583,6 +583,11 @@ func (r *Replica) applySnapshot(
if err != nil {
return 0, errors.Wrap(err, "error loading last index")
}
// Similar strategy for the HardState.
oldHardState, err := loadHardState(batch, desc.RangeID)
if err != nil {
return 0, errors.Wrap(err, "unable to load HardState")
}

// Delete everything in the range and recreate it from the snapshot.
// We need to delete any old Raft log entries here because any log entries
Expand Down Expand Up @@ -655,15 +660,11 @@ func (r *Replica) applySnapshot(
// check.
return 0, errors.Errorf("preemptive snapshot would erase acknowledged log entries")
}
oldHS, err := loadHardState(batch, s.Desc.RangeID)
if err != nil {
return 0, errors.Wrap(err, "unable to load HardState")
}
if snap.Metadata.Term < oldHS.Term {
if snap.Metadata.Term < oldHardState.Term {
return 0, errors.Errorf("cannot apply preemptive snapshot from past term %d at term %d",
snap.Metadata.Term, oldHS.Term)
snap.Metadata.Term, oldHardState.Term)
}
if err := synthesizeHardState(batch, s, oldHS); err != nil {
if err := synthesizeHardState(batch, s, oldHardState); err != nil {
return 0, errors.Wrap(err, "unable to write synthesized HardState")
}
} else {
Expand Down
83 changes: 83 additions & 0 deletions storage/replica_raftstorage_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// Copyright 2016 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
//
// Author: Tobias Schottdorf ([email protected])

package storage

import (
"testing"

"golang.org/x/net/context"

"github.com/cockroachdb/cockroach/roachpb"
"github.com/cockroachdb/cockroach/testutils"
"github.com/cockroachdb/cockroach/util/leaktest"
"github.com/coreos/etcd/raft/raftpb"
)

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

var tc testContext
tc.Start(t)
defer tc.Stop()

key := roachpb.RKey("a")
realRng := tc.store.LookupReplica(key, nil)

// Use Raft to get a nontrivial term for our snapshot.
if pErr := realRng.redirectOnOrAcquireLease(context.Background()); pErr != nil {
t.Fatal(pErr)
}

snap, err := realRng.GetSnapshot()
if err != nil {
t.Fatal(err)
}

// Make sure that the Term is behind our first range term (raftInitialLogTerm)
snap.Metadata.Term--

// Create an uninitialized version of the first range. This is only ok
// because in the case we test, there's an error (and so we don't clobber
// our actual first range in the Store). If we want snapshots to apply
// successfully during tests, we need to adapt the snapshots to a new
// RangeID first and generally do a lot more work.
rng, err := NewReplica(&roachpb.RangeDescriptor{RangeID: 1}, tc.store, 0)
if err != nil {
t.Fatal(err)
}

if _, err := rng.applySnapshot(snap, raftpb.HardState{}); !testutils.IsError(
err, "cannot apply preemptive snapshot from past term",
) {
t.Fatal(err)
}

// Do something that extends the Raft log past what we have in the
// snapshot.
put := putArgs(roachpb.Key("a"), []byte("foo"))
if _, pErr := tc.SendWrapped(&put); pErr != nil {
t.Fatal(pErr)
}
snap.Metadata.Term++ // restore the "real" term of the snapshot

if _, err := rng.applySnapshot(snap, raftpb.HardState{}); !testutils.IsError(
err, "would erase acknowledged log entries",
) {
t.Fatal(err)
}

}

0 comments on commit 7f2218c

Please sign in to comment.