mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #16029 from dusk125/revision-bump
Adding optional revision bump to snapshot restore
This commit is contained in:
commit
c6fd719ccf
@ -38,6 +38,8 @@ var (
|
||||
restorePeerURLs string
|
||||
restoreName string
|
||||
skipHashCheck bool
|
||||
markCompacted bool
|
||||
revisionBump uint64
|
||||
)
|
||||
|
||||
// NewSnapshotCommand returns the cobra command for "snapshot".
|
||||
@ -75,6 +77,8 @@ func NewSnapshotRestoreCommand() *cobra.Command {
|
||||
cmd.Flags().StringVar(&restorePeerURLs, "initial-advertise-peer-urls", defaultInitialAdvertisePeerURLs, "List of this member's peer URLs to advertise to the rest of the cluster")
|
||||
cmd.Flags().StringVar(&restoreName, "name", defaultName, "Human-readable name for this member")
|
||||
cmd.Flags().BoolVar(&skipHashCheck, "skip-hash-check", false, "Ignore snapshot integrity hash value (required if copied from data directory)")
|
||||
cmd.Flags().Uint64Var(&revisionBump, "bump-revision", 0, "How much to increase the latest revision after restore (required if --mark-compacted)")
|
||||
cmd.Flags().BoolVar(&markCompacted, "mark-compacted", false, "Mark the latest revision after restore as the point of scheduled compaction (required if --bump-revision > 0)")
|
||||
|
||||
cmd.MarkFlagDirname("data-dir")
|
||||
cmd.MarkFlagDirname("wal-dir")
|
||||
@ -100,7 +104,7 @@ func SnapshotStatusCommandFunc(cmd *cobra.Command, args []string) {
|
||||
|
||||
func snapshotRestoreCommandFunc(_ *cobra.Command, args []string) {
|
||||
SnapshotRestoreCommandFunc(restoreCluster, restoreClusterToken, restoreDataDir, restoreWalDir,
|
||||
restorePeerURLs, restoreName, skipHashCheck, args)
|
||||
restorePeerURLs, restoreName, skipHashCheck, revisionBump, markCompacted, args)
|
||||
}
|
||||
|
||||
func SnapshotRestoreCommandFunc(restoreCluster string,
|
||||
@ -110,12 +114,19 @@ func SnapshotRestoreCommandFunc(restoreCluster string,
|
||||
restorePeerURLs string,
|
||||
restoreName string,
|
||||
skipHashCheck bool,
|
||||
revisionBump uint64,
|
||||
markCompacted bool,
|
||||
args []string) {
|
||||
if len(args) != 1 {
|
||||
err := fmt.Errorf("snapshot restore requires exactly one argument")
|
||||
cobrautl.ExitWithError(cobrautl.ExitBadArgs, err)
|
||||
}
|
||||
|
||||
if (revisionBump == 0 && markCompacted) || (revisionBump > 0 && !markCompacted) {
|
||||
err := fmt.Errorf("--mark-compacted required if --revision-bump > 0")
|
||||
cobrautl.ExitWithError(cobrautl.ExitBadArgs, err)
|
||||
}
|
||||
|
||||
dataDir := restoreDataDir
|
||||
if dataDir == "" {
|
||||
dataDir = restoreName + ".etcd"
|
||||
@ -138,6 +149,8 @@ func SnapshotRestoreCommandFunc(restoreCluster string,
|
||||
InitialCluster: restoreCluster,
|
||||
InitialClusterToken: restoreClusterToken,
|
||||
SkipHashCheck: skipHashCheck,
|
||||
RevisionBump: revisionBump,
|
||||
MarkCompacted: markCompacted,
|
||||
}); err != nil {
|
||||
cobrautl.ExitWithError(cobrautl.ExitError, err)
|
||||
}
|
||||
|
@ -23,9 +23,31 @@ type revision struct {
|
||||
sub int64
|
||||
}
|
||||
|
||||
// GreaterThan should be synced with function in server
|
||||
// https://github.com/etcd-io/etcd/blob/main/server/storage/mvcc/revision.go
|
||||
func (a revision) GreaterThan(b revision) bool {
|
||||
if a.main > b.main {
|
||||
return true
|
||||
}
|
||||
if a.main < b.main {
|
||||
return false
|
||||
}
|
||||
return a.sub > b.sub
|
||||
}
|
||||
|
||||
// bytesToRev should be synced with function in server
|
||||
// https://github.com/etcd-io/etcd/blob/main/server/storage/mvcc/revision.go
|
||||
func bytesToRev(bytes []byte) revision {
|
||||
return revision{
|
||||
main: int64(binary.BigEndian.Uint64(bytes[0:8])),
|
||||
sub: int64(binary.BigEndian.Uint64(bytes[9:])),
|
||||
}
|
||||
}
|
||||
|
||||
// revToBytes should be synced with function in server
|
||||
// https://github.com/etcd-io/etcd/blob/main/server/storage/mvcc/revision.go
|
||||
func revToBytes(bytes []byte, rev revision) {
|
||||
binary.BigEndian.PutUint64(bytes[0:8], uint64(rev.main))
|
||||
bytes[8] = '_'
|
||||
binary.BigEndian.PutUint64(bytes[9:], uint64(rev.sub))
|
||||
}
|
||||
|
@ -41,6 +41,7 @@ import (
|
||||
"go.etcd.io/etcd/server/v3/etcdserver/api/v2store"
|
||||
"go.etcd.io/etcd/server/v3/etcdserver/cindex"
|
||||
"go.etcd.io/etcd/server/v3/storage/backend"
|
||||
"go.etcd.io/etcd/server/v3/storage/mvcc"
|
||||
"go.etcd.io/etcd/server/v3/storage/schema"
|
||||
"go.etcd.io/etcd/server/v3/storage/wal"
|
||||
"go.etcd.io/etcd/server/v3/storage/wal/walpb"
|
||||
@ -203,6 +204,16 @@ type RestoreConfig struct {
|
||||
// SkipHashCheck is "true" to ignore snapshot integrity hash value
|
||||
// (required if copied from data directory).
|
||||
SkipHashCheck bool
|
||||
|
||||
// RevisionBump is the amount to increase the latest revision after restore,
|
||||
// to allow administrators to trick clients into thinking that revision never decreased.
|
||||
// If 0, revision bumping is skipped.
|
||||
// (required if MarkCompacted == true)
|
||||
RevisionBump uint64
|
||||
|
||||
// MarkCompacted is "true" to mark the latest revision as compacted.
|
||||
// (required if RevisionBump > 0)
|
||||
MarkCompacted bool
|
||||
}
|
||||
|
||||
// Restore restores a new etcd data directory from given snapshot file.
|
||||
@ -265,6 +276,13 @@ func (s *v3Manager) Restore(cfg RestoreConfig) error {
|
||||
if err = s.saveDB(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if cfg.MarkCompacted && cfg.RevisionBump > 0 {
|
||||
if err = s.modifyLatestRevision(cfg.RevisionBump); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
hardstate, err := s.saveWALAndSnap()
|
||||
if err != nil {
|
||||
return err
|
||||
@ -311,6 +329,70 @@ func (s *v3Manager) saveDB() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// modifyLatestRevision can increase the latest revision by the given amount and sets the scheduled compaction
|
||||
// to that revision so that the server will consider this revision compacted.
|
||||
func (s *v3Manager) modifyLatestRevision(bumpAmount uint64) error {
|
||||
be := backend.NewDefaultBackend(s.lg, s.outDbPath())
|
||||
defer func() {
|
||||
be.ForceCommit()
|
||||
be.Close()
|
||||
}()
|
||||
|
||||
tx := be.BatchTx()
|
||||
tx.LockOutsideApply()
|
||||
defer tx.Unlock()
|
||||
|
||||
latest, err := s.unsafeGetLatestRevision(tx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
latest = s.unsafeBumpRevision(tx, latest, int64(bumpAmount))
|
||||
s.unsafeMarkRevisionCompacted(tx, latest)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *v3Manager) unsafeBumpRevision(tx backend.BatchTx, latest revision, amount int64) revision {
|
||||
s.lg.Info(
|
||||
"bumping latest revision",
|
||||
zap.Int64("latest-revision", latest.main),
|
||||
zap.Int64("bump-amount", amount),
|
||||
zap.Int64("new-latest-revision", latest.main+amount),
|
||||
)
|
||||
|
||||
latest.main += amount
|
||||
latest.sub = 0
|
||||
k := make([]byte, 17)
|
||||
revToBytes(k, latest)
|
||||
tx.UnsafePut(schema.Key, k, []byte{})
|
||||
|
||||
return latest
|
||||
}
|
||||
|
||||
func (s *v3Manager) unsafeMarkRevisionCompacted(tx backend.BatchTx, latest revision) {
|
||||
s.lg.Info(
|
||||
"marking revision compacted",
|
||||
zap.Int64("revision", latest.main),
|
||||
)
|
||||
|
||||
mvcc.UnsafeSetScheduledCompact(tx, latest.main)
|
||||
}
|
||||
|
||||
func (s *v3Manager) unsafeGetLatestRevision(tx backend.BatchTx) (revision, error) {
|
||||
var latest revision
|
||||
err := tx.UnsafeForEach(schema.Key, func(k, _ []byte) (err error) {
|
||||
rev := bytesToRev(k)
|
||||
|
||||
if rev.GreaterThan(latest) {
|
||||
latest = rev
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
return latest, err
|
||||
}
|
||||
|
||||
func (s *v3Manager) copyAndVerifyDB() error {
|
||||
srcf, ferr := os.Open(s.srcDbPath)
|
||||
if ferr != nil {
|
||||
|
@ -27,9 +27,12 @@ import (
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
"go.etcd.io/etcd/etcdutl/v3/snapshot"
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.etcd.io/etcd/tests/v3/framework/config"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
"go.etcd.io/etcd/tests/v3/framework/testutils"
|
||||
)
|
||||
|
||||
func TestCtlV3Snapshot(t *testing.T) { testCtl(t, snapshotTest) }
|
||||
@ -305,3 +308,130 @@ func snapshotVersionTest(cx ctlCtx) {
|
||||
cx.t.Fatalf("expected %q, got %q", "3.6.0", st.Version)
|
||||
}
|
||||
}
|
||||
|
||||
func TestRestoreCompactionRevBump(t *testing.T) {
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
epc, err := e2e.NewEtcdProcessCluster(context.TODO(), t,
|
||||
e2e.WithClusterSize(1),
|
||||
e2e.WithKeepDataDir(true),
|
||||
)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
defer func() {
|
||||
if errC := epc.Close(); errC != nil {
|
||||
t.Fatalf("error closing etcd processes (%v)", errC)
|
||||
}
|
||||
}()
|
||||
|
||||
ctl := epc.Etcdctl()
|
||||
|
||||
watchCh := ctl.Watch(context.Background(), "foo", config.WatchOptions{Prefix: true})
|
||||
// flake-fix: the watch can sometimes miss the first put below causing test failure
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
kvs := []testutils.KV{{Key: "foo1", Val: "val1"}, {Key: "foo2", Val: "val2"}, {Key: "foo3", Val: "val3"}}
|
||||
for i := range kvs {
|
||||
require.NoError(t, ctl.Put(context.Background(), kvs[i].Key, kvs[i].Val, config.PutOptions{}))
|
||||
}
|
||||
|
||||
watchTimeout := 1 * time.Second
|
||||
watchRes, err := testutils.KeyValuesFromWatchChan(watchCh, len(kvs), watchTimeout)
|
||||
require.NoErrorf(t, err, "failed to get key-values from watch channel %s", err)
|
||||
require.Equal(t, kvs, watchRes)
|
||||
|
||||
// ensure we get the right revision back for each of the keys
|
||||
currentRev := 4
|
||||
baseRev := 2
|
||||
hasKVs(t, ctl, kvs, currentRev, baseRev)
|
||||
|
||||
fpath := filepath.Join(t.TempDir(), "test.snapshot")
|
||||
|
||||
t.Log("etcdctl saving snapshot...")
|
||||
cmdPrefix := []string{e2e.BinPath.Etcdctl, "--endpoints", strings.Join(epc.EndpointsGRPC(), ",")}
|
||||
require.NoError(t, e2e.SpawnWithExpects(append(cmdPrefix, "snapshot", "save", fpath), nil, fmt.Sprintf("Snapshot saved at %s", fpath)))
|
||||
|
||||
// add some more kvs that are not in the snapshot that will be lost after restore
|
||||
unsnappedKVs := []testutils.KV{{Key: "unsnapped1", Val: "one"}, {Key: "unsnapped2", Val: "two"}, {Key: "unsnapped3", Val: "three"}}
|
||||
for i := range unsnappedKVs {
|
||||
require.NoError(t, ctl.Put(context.Background(), unsnappedKVs[i].Key, unsnappedKVs[i].Val, config.PutOptions{}))
|
||||
}
|
||||
|
||||
t.Log("Stopping the original server...")
|
||||
require.NoError(t, epc.Stop())
|
||||
|
||||
newDataDir := filepath.Join(t.TempDir(), "test.data")
|
||||
t.Log("etcdctl restoring the snapshot...")
|
||||
bumpAmount := 10000
|
||||
err = e2e.SpawnWithExpect([]string{
|
||||
e2e.BinPath.Etcdutl,
|
||||
"snapshot",
|
||||
"restore", fpath,
|
||||
"--name", epc.Procs[0].Config().Name,
|
||||
"--initial-cluster", epc.Procs[0].Config().InitialCluster,
|
||||
"--initial-cluster-token", epc.Procs[0].Config().InitialToken,
|
||||
"--initial-advertise-peer-urls", epc.Procs[0].Config().PeerURL.String(),
|
||||
"--bump-revision", fmt.Sprintf("%d", bumpAmount),
|
||||
"--mark-compacted",
|
||||
"--data-dir", newDataDir,
|
||||
}, "added member")
|
||||
require.NoError(t, err)
|
||||
|
||||
t.Log("(Re)starting the etcd member using the restored snapshot...")
|
||||
epc.Procs[0].Config().DataDirPath = newDataDir
|
||||
for i := range epc.Procs[0].Config().Args {
|
||||
if epc.Procs[0].Config().Args[i] == "--data-dir" {
|
||||
epc.Procs[0].Config().Args[i+1] = newDataDir
|
||||
}
|
||||
}
|
||||
|
||||
require.NoError(t, epc.Restart(context.Background()))
|
||||
|
||||
t.Log("Ensuring the restored member has the correct data...")
|
||||
hasKVs(t, ctl, kvs, currentRev, baseRev)
|
||||
for i := range unsnappedKVs {
|
||||
v, err := ctl.Get(context.Background(), unsnappedKVs[i].Key, config.GetOptions{})
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, int64(0), v.Count)
|
||||
}
|
||||
|
||||
cancelResult, ok := <-watchCh
|
||||
require.True(t, ok, "watchChannel should be open")
|
||||
require.Equal(t, v3rpc.ErrCompacted, cancelResult.Err())
|
||||
require.Truef(t, cancelResult.Canceled, "expected ongoing watch to be cancelled after restoring with --mark-compacted")
|
||||
require.Equal(t, int64(bumpAmount+currentRev), cancelResult.CompactRevision)
|
||||
_, ok = <-watchCh
|
||||
require.False(t, ok, "watchChannel should be closed after restoring with --mark-compacted")
|
||||
|
||||
// clients might restart the watch at the old base revision, that should not yield any new data
|
||||
// everything up until bumpAmount+currentRev should return "already compacted"
|
||||
for i := bumpAmount - 2; i < bumpAmount+currentRev; i++ {
|
||||
watchCh = ctl.Watch(context.Background(), "foo", config.WatchOptions{Prefix: true, Revision: int64(i)})
|
||||
cancelResult := <-watchCh
|
||||
require.Equal(t, v3rpc.ErrCompacted, cancelResult.Err())
|
||||
require.Truef(t, cancelResult.Canceled, "expected ongoing watch to be cancelled after restoring with --mark-compacted")
|
||||
require.Equal(t, int64(bumpAmount+currentRev), cancelResult.CompactRevision)
|
||||
}
|
||||
|
||||
// a watch after that revision should yield successful results when a new put arrives
|
||||
ctx, cancel := context.WithTimeout(context.Background(), watchTimeout*5)
|
||||
defer cancel()
|
||||
watchCh = ctl.Watch(ctx, "foo", config.WatchOptions{Prefix: true, Revision: int64(bumpAmount + currentRev + 1)})
|
||||
require.NoError(t, ctl.Put(context.Background(), "foo4", "val4", config.PutOptions{}))
|
||||
watchRes, err = testutils.KeyValuesFromWatchChan(watchCh, 1, watchTimeout)
|
||||
require.NoErrorf(t, err, "failed to get key-values from watch channel %s", err)
|
||||
require.Equal(t, []testutils.KV{{Key: "foo4", Val: "val4"}}, watchRes)
|
||||
}
|
||||
|
||||
func hasKVs(t *testing.T, ctl *e2e.EtcdctlV3, kvs []testutils.KV, currentRev int, baseRev int) {
|
||||
for i := range kvs {
|
||||
v, err := ctl.Get(context.Background(), kvs[i].Key, config.GetOptions{})
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, int64(1), v.Count)
|
||||
require.Equal(t, kvs[i].Val, string(v.Kvs[0].Value))
|
||||
require.Equal(t, int64(baseRev+i), v.Kvs[0].CreateRevision)
|
||||
require.Equal(t, int64(baseRev+i), v.Kvs[0].ModRevision)
|
||||
require.Equal(t, int64(1), v.Kvs[0].Version)
|
||||
}
|
||||
}
|
||||
|
@ -704,6 +704,7 @@ func (ctl *EtcdctlV3) Watch(ctx context.Context, key string, opts config.WatchOp
|
||||
var resp clientv3.WatchResponse
|
||||
json.Unmarshal([]byte(line), &resp)
|
||||
if resp.Canceled {
|
||||
ch <- resp
|
||||
close(ch)
|
||||
return
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user