Skip to content

Commit

Permalink
storage: unit test migrations
Browse files Browse the repository at this point in the history
  • Loading branch information
tbg committed Jul 8, 2016
1 parent b1a0005 commit 5995ab4
Show file tree
Hide file tree
Showing 2 changed files with 78 additions and 18 deletions.
72 changes: 72 additions & 0 deletions storage/migration_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
// 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 (
"reflect"
"testing"

"github.com/cockroachdb/cockroach/roachpb"
"github.com/cockroachdb/cockroach/storage/engine"
"github.com/cockroachdb/cockroach/util/leaktest"
"github.com/cockroachdb/cockroach/util/stop"
"github.com/coreos/etcd/raft/raftpb"
)

func TestMigrate7310And6991(t *testing.T) {
defer leaktest.AfterTest(t)()
stopper := stop.NewStopper()
defer stopper.Stop()
eng := engine.NewInMem(roachpb.Attributes{}, 1<<10, stopper)

desc := *testRangeDescriptor()

if err := migrate7310And6991(eng, desc); err != nil {
t.Fatal(err)
}

ts, err := loadTruncatedState(eng, desc.RangeID)
if err != nil {
t.Fatal(err)
}

hs, err := loadHardState(eng, desc.RangeID)
if err != nil {
t.Fatal(err)
}

rApplied, lApplied, err := loadAppliedIndex(eng, desc.RangeID)
if err != nil {
t.Fatal(err)
}

expTS := roachpb.RaftTruncatedState{Term: raftInitialLogTerm, Index: raftInitialLogIndex}
if !reflect.DeepEqual(expTS, ts) {
t.Errorf("expected %+v, got %+v", &expTS, &ts)
}

expHS := raftpb.HardState{Term: raftInitialLogTerm, Commit: raftInitialLogIndex}
if !reflect.DeepEqual(expHS, hs) {
t.Errorf("expected %+v, got %+v", &expHS, &hs)
}

expRApplied, expLApplied := uint64(raftInitialLogIndex), uint64(0)
if expRApplied != rApplied || expLApplied != lApplied {
t.Errorf("expected (raftApplied,leaseApplied)=(%d,%d), got (%d,%d)",
expRApplied, expLApplied, rApplied, lApplied)
}
}
24 changes: 6 additions & 18 deletions storage/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -866,22 +866,7 @@ func IterateRangeDescriptors(
return err
}

// MIGRATION(tschottdorf): As of #7310, we make sure that a Replica always has
// a complete Raft state on disk. Prior versions may not have that, which
// causes issues due to the fact that we used to synthesize a TruncatedState
// and do so no more. To make up for that, write a missing TruncatedState here.
// That key is in the replicated state, but since during a cluster upgrade, all
// nodes do it, it's fine (and we never CPut on that key, so anything in the
// Raft pipeline will simply overwrite it).
//
// Migration(tschottdorf): See #6991. It's possible that the HardState is
// missing after a snapshot was applied (so there is a TruncatedState). In this
// case, synthesize a HardState (simply setting everything that was in the
// snapshot to committed). Having lost the original HardState can theoretically
// mean that the replica was further ahead or had voted, and so there's no
// guarantee that this will be correct. But it will be correct in the majority
// of cases, and some state *has* to be recovered.
func (s *Store) migrate7310And6991(desc roachpb.RangeDescriptor) {
func (s *Store) migrate(desc roachpb.RangeDescriptor) {
batch := s.engine.NewBatch()
if err := migrate7310And6991(batch, desc); err != nil {
log.Fatal(errors.Wrap(err, "during migration"))
Expand Down Expand Up @@ -914,6 +899,7 @@ func migrate7310And6991(batch engine.ReadWriter, desc roachpb.RangeDescriptor) e
if (*state.TruncatedState == roachpb.RaftTruncatedState{}) {
state.TruncatedState.Term = raftInitialLogTerm
state.TruncatedState.Index = raftInitialLogIndex
state.RaftAppliedIndex = raftInitialLogIndex
if _, err := saveState(batch, state); err != nil {
return errors.Wrapf(err, "could not migrate TruncatedState to %+v", &state.TruncatedState)
}
Expand Down Expand Up @@ -1021,8 +1007,10 @@ func (s *Store) Start(stopper *stop.Stopper) error {
// (which is necessary to have a non-nil raft group)
return false, s.destroyReplicaData(&desc)
}

s.migrate7310And6991(desc)
if !desc.IsInitialized() {
return false, errors.Errorf("found uninitialized RangeDescriptor: %+v", desc)
}
s.migrate(desc)

rng, err := NewReplica(&desc, s, 0)
if err != nil {
Expand Down

0 comments on commit 5995ab4

Please sign in to comment.