Skip to content

Commit

Permalink
Merge pull request #15228 from bdarnell/below-raft-protos
Browse files Browse the repository at this point in the history
storage: Remove the last vestiges of non-proposer-evaluated-KV
  • Loading branch information
bdarnell authored Apr 23, 2017
2 parents f957403 + 8871d35 commit a9aebe9
Show file tree
Hide file tree
Showing 9 changed files with 50 additions and 261 deletions.
51 changes: 18 additions & 33 deletions pkg/acceptance/reference_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,11 +62,7 @@ export COCKROACH_SKIP_UPDATE_CHECK=1
export COCKROACH_CERTS_DIR=/certs/
bin=/%s/cockroach
# TODO(bdarnell): when --background is in referenceBinPath, use it here and below.
# The until loop will also be unnecessary at that point.
$bin start --logtostderr & &> oldout
# Wait until cockroach has started up successfully.
until $bin sql -e "SELECT 1"; do sleep 1; done
$bin start --background --logtostderr &> oldout
echo "Use the reference binary to write a couple rows, then render its output to a file and shut down."
$bin sql -e "CREATE DATABASE old"
Expand Down Expand Up @@ -102,6 +98,10 @@ bin=/%s/cockroach
runReferenceTestWithScript(ctx, t, referenceTestScript)
}

// TestDockerReadWriteBidirectionalReferenceVersion verifies that we can
// upgrade from the bidirectional reference version (i.e. the oldest
// version that we support downgrading to, specified in the
// postgres-test dockerfile), then downgrade to it again.
func TestDockerReadWriteBidirectionalReferenceVersion(t *testing.T) {
s := log.Scope(t)
defer s.Close(t)
Expand All @@ -125,6 +125,14 @@ $bin quit && wait
runReadWriteReferenceTest(ctx, t, `bidirectional-reference-version`, backwardReferenceTest)
}

// TestDockerReadWriteForwardReferenceVersion verifies that we can
// upgrade from the forward reference version (i.e. the oldest version
// that we support upgrading from, specified in the postgres-test
// dockerfile), and that downgrading to it fails in the expected ways.
//
// When the forward and bidirectional reference versions are the same,
// this test is a duplicate of the bidirectional test above and exists
// solely to preserve the scaffolding for when they differ again.
func TestDockerReadWriteForwardReferenceVersion(t *testing.T) {
s := log.Scope(t)
defer s.Close(t)
Expand All @@ -138,35 +146,12 @@ function finish() {
trap finish EXIT
export COCKROACH_CERTS_DIR=/certs/
$bin start & &> out
until $bin sql -e "SELECT 1"; do sleep 1; done
# grep returns non-zero if it didn't match anything. With set -e above, that would exit here.
$bin sql -d old -e "SELECT i, b, s, d, f, extract(epoch from (timestamp '1970-01-01 00:00:00' + v)) as v, extract(epoch FROM t) as e FROM testing_new" 2>&1 | grep "is encoded using using version 3, but this client only supports version 1"
$bin start --background --logtostderr &> out
$bin sql -d old -e "SELECT i, b, s, d, f, extract(epoch from (timestamp '1970-01-01 00:00:00' + v)) as v, extract(epoch FROM t) as e FROM testing_old" > old.everything
$bin sql -d old -e "SELECT i, b, s, d, f, extract(epoch from (timestamp '1970-01-01 00:00:00' + v)) as v, extract(epoch FROM t) as e FROM testing_new" >> old.everything
# diff returns non-zero if different. With set -e above, that would exit here.
diff new.everything old.everything
$bin quit && wait
`
runReadWriteReferenceTest(ctx, t, `forward-reference-version`, backwardReferenceTest)
}

func TestDockerMigration_7429(t *testing.T) {
s := log.Scope(t)
defer s.Close(t)

ctx := context.Background()
script := `
set -eux
bin=/cockroach/cockroach
touch out
function finish() {
cat out
}
trap finish EXIT
export COCKROACH_SKIP_UPDATE_CHECK=1
export COCKROACH_CERTS_DIR=/certs/
$bin start --logtostderr=INFO --background --store=/cockroach-data-reference-7429 &> out
$bin debug range ls
$bin quit
`
runReferenceTestWithScript(ctx, t, script)
}
5 changes: 4 additions & 1 deletion pkg/acceptance/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,10 @@

set -euxo pipefail

"$(dirname "${0}")"/../../build/builder.sh make install TAGS=clockoffset
# We must make a release build here because the binary needs to work
# in both the builder image and the postgres-test image, which have
# different libstc++ versions.
"$(dirname "${0}")"/../../build/builder.sh make install TAGS=clockoffset TYPE=release-linux-gnu

# The log files that should be created by -l below can only
# be created if the parent directory already exists. Ensure
Expand Down
2 changes: 1 addition & 1 deletion pkg/acceptance/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -550,7 +550,7 @@ func testDockerSuccess(ctx context.Context, t *testing.T, name string, cmd []str
const (
// Iterating against a locally built version of the docker image can be done
// by changing postgresTestImage to the hash of the container.
postgresTestImage = "docker.io/cockroachdb/postgres-test:20170308-1644"
postgresTestImage = "docker.io/cockroachdb/postgres-test:20170423-1100"
)

func testDocker(
Expand Down
33 changes: 0 additions & 33 deletions pkg/storage/below_raft_protos_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,9 @@ import (
"reflect"
"testing"

"github.com/coreos/etcd/raft/raftpb"
"github.com/gogo/protobuf/proto"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
)
Expand Down Expand Up @@ -66,11 +63,6 @@ type fixture struct {
}

var belowRaftGoldenProtos = map[reflect.Type]fixture{
reflect.TypeOf(&raftpb.HardState{}): {
populatedConstructor: func(r *rand.Rand) proto.Message { return &raftpb.HardState{Term: 1, Vote: 2, Commit: 3} },
emptySum: 13621293256077144893,
populatedSum: 11100902660574274053,
},
reflect.TypeOf(&enginepb.MVCCMetadata{}): {
populatedConstructor: func(r *rand.Rand) proto.Message { return enginepb.NewPopulatedMVCCMetadata(r, false) },
emptySum: 7551962144604783939,
Expand All @@ -81,31 +73,6 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{
emptySum: 18064891702890239528,
populatedSum: 4287370248246326846,
},
reflect.TypeOf(&roachpb.AbortCacheEntry{}): {
populatedConstructor: func(r *rand.Rand) proto.Message { return roachpb.NewPopulatedAbortCacheEntry(r, false) },
emptySum: 11932598136014321867,
populatedSum: 5118321872981034391,
},
reflect.TypeOf(&roachpb.Lease{}): {
populatedConstructor: func(r *rand.Rand) proto.Message { return roachpb.NewPopulatedLease(r, false) },
emptySum: 10006158318270644799,
populatedSum: 1304511461063751549,
},
reflect.TypeOf(&roachpb.RaftTruncatedState{}): {
populatedConstructor: func(r *rand.Rand) proto.Message { return roachpb.NewPopulatedRaftTruncatedState(r, false) },
emptySum: 5531676819244041709,
populatedSum: 14781226418259198098,
},
reflect.TypeOf(&hlc.Timestamp{}): {
populatedConstructor: func(r *rand.Rand) proto.Message { return hlc.NewPopulatedTimestamp(r, false) },
emptySum: 5531676819244041709,
populatedSum: 10735653246768912584,
},
reflect.TypeOf(&roachpb.Transaction{}): {
populatedConstructor: func(r *rand.Rand) proto.Message { return roachpb.NewPopulatedTransaction(r, false) },
emptySum: 8650182997796107667,
populatedSum: 15911378024840759412,
},
}

func TestBelowRaftProtos(t *testing.T) {
Expand Down
74 changes: 0 additions & 74 deletions pkg/storage/migration.go

This file was deleted.

76 changes: 0 additions & 76 deletions pkg/storage/migration_test.go

This file was deleted.

26 changes: 1 addition & 25 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -3659,31 +3659,7 @@ func (r *Replica) processRaftCommand(
var writeBatch *storagebase.WriteBatch
{
if raftCmd.ReplicatedEvalResult == nil && forcedErr == nil {
// If not proposer-evaluating, then our raftCmd consists only of
// the BatchRequest and some metadata.
innerResult, pErr := r.evaluateProposal(
ctx,
idKey,
*raftCmd.BatchRequest,
nil,
)
// Then, change the raftCmd to reflect the result of the
// evaluation, filling in the EvalResult (which is now properly
// populated, including a WriteBatch, and does not contain the
// BatchRequest any more).
//
// Note that this (intentionally) overwrites the LocalEvalResult,
// so we must salvage the done channel if we have a client waiting
// on it.
raftCmd.ReplicatedEvalResult = &innerResult.Replicated
writeBatch = innerResult.WriteBatch
if proposedLocally {
proposal.Local = &innerResult.Local
}
// Proposals which would failfast with proposer-evaluated KV now
// go this route, writing an empty entry and returning this error
// to the client.
forcedErr = pErr
panic("non-proposer-evaluated command found in raft log. Cannot upgrade directly from versions older than beta-20170413 to beta-20170420 or newer.")
}

if filter := r.store.cfg.TestingKnobs.TestingApplyFilter; forcedErr == nil && filter != nil && raftCmd.ReplicatedEvalResult != nil {
Expand Down
12 changes: 0 additions & 12 deletions pkg/storage/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -1091,17 +1091,6 @@ func IterateRangeDescriptors(
return err
}

func (s *Store) migrate(ctx context.Context, desc roachpb.RangeDescriptor) {
batch := s.engine.NewBatch()
defer batch.Close()
if err := migrate7310And6991(ctx, batch, desc); err != nil {
log.Fatal(ctx, errors.Wrap(err, "during migration"))
}
if err := batch.Commit(false /* !sync */); err != nil {
log.Fatal(ctx, errors.Wrap(err, "could not migrate Raft state"))
}
}

// ReadStoreIdent reads the StoreIdent from the store.
// It returns *NotBootstrappedError if the ident is missing (meaning that the
// store needs to be bootstrapped).
Expand Down Expand Up @@ -1173,7 +1162,6 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error {
if !desc.IsInitialized() {
return false, errors.Errorf("found uninitialized RangeDescriptor: %+v", desc)
}
s.migrate(ctx, desc)

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

0 comments on commit a9aebe9

Please sign in to comment.