diff --git a/storage/migration_test.go b/storage/migration_test.go new file mode 100644 index 000000000000..83982d479141 --- /dev/null +++ b/storage/migration_test.go @@ -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 (tobias.schottdorf@gmail.com) + +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) + } +} diff --git a/storage/store.go b/storage/store.go index d0d3dd96e235..b058e8f54039 100644 --- a/storage/store.go +++ b/storage/store.go @@ -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")) @@ -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) } @@ -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 {