diff --git a/CHANGELOG-3.5.md b/CHANGELOG-3.5.md
index 45db3a171e6..1c12d02993d 100644
--- a/CHANGELOG-3.5.md
+++ b/CHANGELOG-3.5.md
@@ -9,7 +9,7 @@ The minimum recommended etcd versions to run in **production** are 3.2.28+, 3.3.
-## v3.5.0 (2021 TBD)
+## v3.5.0 (2021-06)
See [code changes](https://github.com/etcd-io/etcd/compare/v3.4.0...v3.5.0) and [v3.5 upgrade guide](https://etcd.io/docs/latest/upgrades/upgrade_3_5/) for any breaking changes.
@@ -160,6 +160,7 @@ Note that any `etcd_debugging_*` metrics are experimental and subject to change.
- Add [`--socket-reuse-address`](https://github.com/etcd-io/etcd/pull/12702) flag
- Setting this flag enables `SO_REUSEADDR` which allows binding to an address in `TIME_WAIT` state, improving etcd restart time.
- Reduce [around 30% memory allocation by logging range response size without marshal](https://github.com/etcd-io/etcd/pull/12871).
+- `ETCD_VERIFY="all"` enviroment triggers [additional verification of consistency](https://github.com/etcd-io/etcd/pull/) of etcd data-dir files.
### Package `runtime`
- Optimize [`runtime.FDUsage` by removing unnecessary sorting](https://github.com/etcd-io/etcd/pull/12214).
diff --git a/etcdctl/ctlv2/command/backup_command.go b/etcdctl/ctlv2/command/backup_command.go
index 6004ee8c721..d022ad6a1fa 100644
--- a/etcdctl/ctlv2/command/backup_command.go
+++ b/etcdctl/ctlv2/command/backup_command.go
@@ -18,7 +18,6 @@ import (
"log"
"os"
"path"
- "path/filepath"
"regexp"
"time"
@@ -28,6 +27,7 @@ import (
"go.etcd.io/etcd/pkg/v3/idutil"
"go.etcd.io/etcd/pkg/v3/pbutil"
"go.etcd.io/etcd/raft/v3/raftpb"
+ "go.etcd.io/etcd/server/v3/datadir"
"go.etcd.io/etcd/server/v3/etcdserver/api/membership"
"go.etcd.io/etcd/server/v3/etcdserver/api/snap"
"go.etcd.io/etcd/server/v3/etcdserver/api/v2store"
@@ -93,19 +93,22 @@ func handleBackup(c *cli.Context) error {
lg := zap.NewExample()
withV3 := c.Bool("with-v3")
- srcSnap := filepath.Join(c.String("data-dir"), "member", "snap")
- destSnap := filepath.Join(c.String("backup-dir"), "member", "snap")
+ srcDir := c.String("data-dir")
+ destDir := c.String("backup-dir")
+
+ srcSnap := datadir.ToSnapDir(srcDir)
+ destSnap := datadir.ToSnapDir(destDir)
if c.String("wal-dir") != "" {
srcWAL = c.String("wal-dir")
} else {
- srcWAL = filepath.Join(c.String("data-dir"), "member", "wal")
+ srcWAL = datadir.ToWalDir(srcDir)
}
if c.String("backup-wal-dir") != "" {
destWAL = c.String("backup-wal-dir")
} else {
- destWAL = filepath.Join(c.String("backup-dir"), "member", "wal")
+ destWAL = datadir.ToWalDir(destDir)
}
if err := fileutil.CreateDirAll(destSnap); err != nil {
@@ -116,8 +119,8 @@ func handleBackup(c *cli.Context) error {
walsnap := saveSnap(lg, destSnap, srcSnap, &desired)
metadata, state, ents := loadWAL(srcWAL, walsnap, withV3)
- destDbPath := filepath.Join(destSnap, "db")
- saveDB(lg, destDbPath, filepath.Join(srcSnap, "db"), state.Commit, &desired, withV3)
+ destDbPath := datadir.ToBackendFileName(destDir)
+ saveDB(lg, destDbPath, datadir.ToBackendFileName(srcDir), state.Commit, &desired, withV3)
neww, err := wal.Create(zap.NewExample(), destWAL, pbutil.MustMarshal(&metadata))
if err != nil {
diff --git a/etcdctl/snapshot/v3_snapshot.go b/etcdctl/snapshot/v3_snapshot.go
index 98046954334..0fc0e1497e9 100644
--- a/etcdctl/snapshot/v3_snapshot.go
+++ b/etcdctl/snapshot/v3_snapshot.go
@@ -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/mvcc/backend"
+ "go.etcd.io/etcd/server/v3/verify"
"go.etcd.io/etcd/server/v3/wal"
"go.etcd.io/etcd/server/v3/wal/walpb"
"go.uber.org/zap"
@@ -276,7 +277,11 @@ func (s *v3Manager) Restore(cfg RestoreConfig) error {
zap.String("snap-dir", s.snapDir),
)
- return nil
+ return verify.VerifyIfEnabled(verify.Config{
+ ExactIndex: true,
+ Logger: s.lg,
+ DataDir: dataDir,
+ })
}
func (s *v3Manager) outDbPath() string {
diff --git a/server/config/config.go b/server/config/config.go
index 4c82cfae2e6..337d23c4624 100644
--- a/server/config/config.go
+++ b/server/config/config.go
@@ -25,6 +25,7 @@ import (
"go.etcd.io/etcd/client/pkg/v3/transport"
"go.etcd.io/etcd/client/pkg/v3/types"
"go.etcd.io/etcd/pkg/v3/netutil"
+ "go.etcd.io/etcd/server/v3/datadir"
bolt "go.etcd.io/bbolt"
"go.uber.org/zap"
@@ -274,13 +275,13 @@ func (c *ServerConfig) advertiseMatchesCluster() error {
return fmt.Errorf("failed to resolve %s to match --initial-cluster=%s (%v)", apStr, umap.String(), err)
}
-func (c *ServerConfig) MemberDir() string { return filepath.Join(c.DataDir, "member") }
+func (c *ServerConfig) MemberDir() string { return datadir.ToMemberDir(c.DataDir) }
func (c *ServerConfig) WALDir() string {
if c.DedicatedWALDir != "" {
return c.DedicatedWALDir
}
- return filepath.Join(c.MemberDir(), "wal")
+ return datadir.ToWalDir(c.DataDir)
}
func (c *ServerConfig) SnapDir() string { return filepath.Join(c.MemberDir(), "snap") }
@@ -324,4 +325,4 @@ func (c *ServerConfig) BootstrapTimeoutEffective() time.Duration {
return time.Second
}
-func (c *ServerConfig) BackendPath() string { return filepath.Join(c.SnapDir(), "db") }
+func (c *ServerConfig) BackendPath() string { return datadir.ToBackendFileName(c.DataDir) }
diff --git a/server/datadir/datadir.go b/server/datadir/datadir.go
new file mode 100644
index 00000000000..fa4c51ad1a2
--- /dev/null
+++ b/server/datadir/datadir.go
@@ -0,0 +1,40 @@
+// Copyright 2021 The etcd 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.
+
+package datadir
+
+import "path/filepath"
+
+const (
+ memberDirSegment = "member"
+ snapDirSegment = "snap"
+ walDirSegment = "wal"
+ backendFileSegment = "db"
+)
+
+func ToBackendFileName(dataDir string) string {
+ return filepath.Join(ToSnapDir(dataDir), backendFileSegment)
+}
+
+func ToSnapDir(dataDir string) string {
+ return filepath.Join(ToMemberDir(dataDir), snapDirSegment)
+}
+
+func ToWalDir(dataDir string) string {
+ return filepath.Join(ToMemberDir(dataDir), walDirSegment)
+}
+
+func ToMemberDir(dataDir string) string {
+ return filepath.Join(dataDir, memberDirSegment)
+}
diff --git a/server/datadir/datadir_test.go b/server/datadir/datadir_test.go
new file mode 100644
index 00000000000..f6fe19b1c01
--- /dev/null
+++ b/server/datadir/datadir_test.go
@@ -0,0 +1,33 @@
+package datadir_test
+
+import (
+ "testing"
+
+ "github.com/stretchr/testify/assert"
+ "go.etcd.io/etcd/server/v3/datadir"
+)
+
+func TestToBackendFileName(t *testing.T) {
+ result := datadir.ToBackendFileName("/dir/data-dir")
+ assert.Equal(t, "/dir/data-dir/member/snap/db", result)
+}
+
+func TestToMemberDir(t *testing.T) {
+ result := datadir.ToMemberDir("/dir/data-dir")
+ assert.Equal(t, "/dir/data-dir/member", result)
+}
+
+func TestToSnapDir(t *testing.T) {
+ result := datadir.ToSnapDir("/dir/data-dir")
+ assert.Equal(t, "/dir/data-dir/member/snap", result)
+}
+
+func TestToWalDir(t *testing.T) {
+ result := datadir.ToWalDir("/dir/data-dir")
+ assert.Equal(t, "/dir/data-dir/member/wal", result)
+}
+
+func TestToWalDirSlash(t *testing.T) {
+ result := datadir.ToWalDir("/dir/data-dir/")
+ assert.Equal(t, "/dir/data-dir/member/wal", result)
+}
diff --git a/server/datadir/doc.go b/server/datadir/doc.go
new file mode 100644
index 00000000000..92ca4b253dd
--- /dev/null
+++ b/server/datadir/doc.go
@@ -0,0 +1,17 @@
+// Copyright 2021 The etcd 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.
+
+package datadir
+
+// datadir contains functions to navigate file-layout of etcd data-directory.
diff --git a/server/embed/etcd.go b/server/embed/etcd.go
index c53450cd8b1..7dcdbc56df4 100644
--- a/server/embed/etcd.go
+++ b/server/embed/etcd.go
@@ -42,6 +42,7 @@ import (
"go.etcd.io/etcd/server/v3/etcdserver/api/v2v3"
"go.etcd.io/etcd/server/v3/etcdserver/api/v3client"
"go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc"
+ "go.etcd.io/etcd/server/v3/verify"
grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
"github.com/soheilhy/cmux"
@@ -338,6 +339,11 @@ func (e *Etcd) Close() {
lg.Info("closing etcd server", fields...)
defer func() {
lg.Info("closed etcd server", fields...)
+ verify.MustVerifyIfEnabled(verify.Config{
+ Logger: lg,
+ DataDir: e.cfg.Dir,
+ ExactIndex: false,
+ })
lg.Sync()
}()
@@ -513,7 +519,6 @@ func (e *Etcd) servePeers() (err error) {
e.cfg.logger.Info(
"cmux::serve",
zap.String("address", u),
- zap.String("cmuxp", fmt.Sprintf("%p", m)),
)
return m.Serve()
}
@@ -524,16 +529,13 @@ func (e *Etcd) servePeers() (err error) {
e.cfg.logger.Info(
"stopping serving peer traffic",
zap.String("address", u),
- zap.String("cmuxp", fmt.Sprintf("%p", m)),
)
stopServers(ctx, &servers{secure: peerTLScfg != nil, grpc: gs, http: srv})
e.cfg.logger.Info(
"stopped serving peer traffic",
zap.String("address", u),
- zap.String("cmuxp", fmt.Sprintf("%p", m)),
)
m.Close()
- e.cfg.logger.Info("Closed", zap.String("cmuxp", fmt.Sprintf("%p", m)))
return nil
}
}
diff --git a/server/verify/doc.go b/server/verify/doc.go
new file mode 100644
index 00000000000..2c42bf6f198
--- /dev/null
+++ b/server/verify/doc.go
@@ -0,0 +1,20 @@
+// Copyright 2021 The etcd 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.
+
+package verify
+
+// verify package is analyzing persistent state of etcd to find potential
+// inconsistencies.
+// In particular it covers cross-checking between different aspacts of etcd
+// storage like WAL & Backend.
diff --git a/server/verify/verify.go b/server/verify/verify.go
new file mode 100644
index 00000000000..b2483cb1cf6
--- /dev/null
+++ b/server/verify/verify.go
@@ -0,0 +1,138 @@
+// Copyright 2021 The etcd 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.
+
+package verify
+
+import (
+ "fmt"
+ "os"
+
+ "go.etcd.io/etcd/raft/v3/raftpb"
+ "go.etcd.io/etcd/server/v3/datadir"
+ "go.etcd.io/etcd/server/v3/etcdserver/cindex"
+ "go.etcd.io/etcd/server/v3/mvcc/backend"
+ wal2 "go.etcd.io/etcd/server/v3/wal"
+ "go.etcd.io/etcd/server/v3/wal/walpb"
+ "go.uber.org/zap"
+)
+
+const ENV_VERIFY = "ETCD_VERIFY"
+const ENV_VERIFY_ALL_VALUE = "all"
+
+type Config struct {
+ // DataDir is a root directory where the data being verified are stored.
+ DataDir string
+
+ // ExactIndex requires consistent_index in backend exactly match the last committed WAL entry.
+ // Usually backend's consistent_index needs to be <= WAL.commit, but for backups the match
+ // is expected to be exact.
+ ExactIndex bool
+
+ Logger *zap.Logger
+}
+
+// Verify performs consistency checks of given etcd data-directory.
+// The errors are reported as the returned error, but for some situations
+// the function can also panic.
+// The function is expected to work on not-in-use data model, i.e.
+// no file-locks should be taken. Verify does not modified the data.
+func Verify(cfg Config) error {
+ lg := cfg.Logger
+ if lg == nil {
+ lg = zap.NewNop()
+ }
+
+ var err error
+ lg.Info("verification of persisted state", zap.String("data-dir", cfg.DataDir))
+ defer func() {
+ if err != nil {
+ lg.Error("verification of persisted state failed",
+ zap.String("data-dir", cfg.DataDir),
+ zap.Error(err))
+ } else if r := recover(); r != nil {
+ lg.Error("verification of persisted state failed",
+ zap.String("data-dir", cfg.DataDir))
+ panic(r)
+ } else {
+ lg.Info("verification of persisted state successful", zap.String("data-dir", cfg.DataDir))
+ }
+ }()
+
+ beConfig := backend.DefaultBackendConfig()
+ beConfig.Path = datadir.ToBackendFileName(cfg.DataDir)
+ beConfig.Logger = cfg.Logger
+
+ be := backend.New(beConfig)
+ defer be.Close()
+
+ _, hardstate, err := validateWal(cfg)
+ if err != nil {
+ return err
+ }
+
+ // TODO: Perform validation of consistency of membership between
+ // backend/members & WAL confstate (and maybe storev2 if still exists).
+
+ return validateConsistentIndex(cfg, hardstate, be)
+}
+
+// VerifyIfEnabled performs verification according to ETCD_VERIFY env settings.
+// See Verify for more information.
+func VerifyIfEnabled(cfg Config) error {
+ if os.Getenv(ENV_VERIFY) == ENV_VERIFY_ALL_VALUE {
+ return Verify(cfg)
+ }
+ return nil
+}
+
+// MustVerifyIfEnabled performs verification according to ETCD_VERIFY env settings
+// and exits in case of found problems.
+// See Verify for more information.
+func MustVerifyIfEnabled(cfg Config) {
+ if err := VerifyIfEnabled(cfg); err != nil {
+ cfg.Logger.Panic("Verification failed",
+ zap.String("data-dir", cfg.DataDir),
+ zap.Error(err))
+ }
+}
+
+func validateConsistentIndex(cfg Config, hardstate *raftpb.HardState, be backend.Backend) error {
+ tx := be.BatchTx()
+ ci := cindex.NewConsistentIndex(tx)
+ index := ci.ConsistentIndex()
+ if cfg.ExactIndex && index != hardstate.Commit {
+ return fmt.Errorf("backend.ConsistentIndex (%v) expected == WAL.HardState.commit (%v)", index, hardstate.Commit)
+ }
+ if index > hardstate.Commit {
+ return fmt.Errorf("backend.ConsistentIndex (%v) must be <= WAL.HardState.commit (%v)", index, hardstate.Commit)
+ }
+ cfg.Logger.Info("verification: consistentIndex OK", zap.Uint64("backend-consistent-index", index), zap.Uint64("hardstate-commit", hardstate.Commit))
+ return nil
+}
+
+func validateWal(cfg Config) (*walpb.Snapshot, *raftpb.HardState, error) {
+ walDir := datadir.ToWalDir(cfg.DataDir)
+
+ walSnaps, err := wal2.ValidSnapshotEntries(cfg.Logger, walDir)
+ if err != nil {
+ return nil, nil, err
+ }
+
+ snapshot := walSnaps[len(walSnaps)-1]
+ hardstate, err := wal2.Verify(cfg.Logger, walDir, snapshot)
+ if err != nil {
+ return nil, nil, err
+ }
+ return &snapshot, hardstate, nil
+}
diff --git a/server/wal/wal.go b/server/wal/wal.go
index b4d072d4c41..3c940e0cdeb 100644
--- a/server/wal/wal.go
+++ b/server/wal/wal.go
@@ -618,10 +618,11 @@ func ValidSnapshotEntries(lg *zap.Logger, walDir string) ([]walpb.Snapshot, erro
// If it cannot read out the expected snap, it will return ErrSnapshotNotFound.
// If the loaded snap doesn't match with the expected one, it will
// return error ErrSnapshotMismatch.
-func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) error {
+func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) (*raftpb.HardState, error) {
var metadata []byte
var err error
var match bool
+ var state raftpb.HardState
rec := &walpb.Record{}
@@ -630,14 +631,14 @@ func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) error {
}
names, nameIndex, err := selectWALFiles(lg, walDir, snap)
if err != nil {
- return err
+ return nil, err
}
// open wal files in read mode, so that there is no conflict
// when the same WAL is opened elsewhere in write mode
rs, _, closer, err := openWALFiles(lg, walDir, names, nameIndex, false)
if err != nil {
- return err
+ return nil, err
}
defer func() {
if closer != nil {
@@ -652,7 +653,7 @@ func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) error {
switch rec.Type {
case metadataType:
if metadata != nil && !bytes.Equal(metadata, rec.Data) {
- return ErrMetadataConflict
+ return nil, ErrMetadataConflict
}
metadata = rec.Data
case crcType:
@@ -660,7 +661,7 @@ func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) error {
// Current crc of decoder must match the crc of the record.
// We need not match 0 crc, since the decoder is a new one at this point.
if crc != 0 && rec.Validate(crc) != nil {
- return ErrCRCMismatch
+ return nil, ErrCRCMismatch
}
decoder.updateCRC(rec.Crc)
case snapshotType:
@@ -668,7 +669,7 @@ func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) error {
pbutil.MustUnmarshal(&loadedSnap, rec.Data)
if loadedSnap.Index == snap.Index {
if loadedSnap.Term != snap.Term {
- return ErrSnapshotMismatch
+ return nil, ErrSnapshotMismatch
}
match = true
}
@@ -676,22 +677,23 @@ func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) error {
// are not necessary for validating the WAL contents
case entryType:
case stateType:
+ pbutil.MustUnmarshal(&state, rec.Data)
default:
- return fmt.Errorf("unexpected block type %d", rec.Type)
+ return nil, fmt.Errorf("unexpected block type %d", rec.Type)
}
}
// We do not have to read out all the WAL entries
// as the decoder is opened in read mode.
if err != io.EOF && err != io.ErrUnexpectedEOF {
- return err
+ return nil, err
}
if !match {
- return ErrSnapshotNotFound
+ return nil, ErrSnapshotNotFound
}
- return nil
+ return &state, nil
}
// cut closes current file written and creates a new one ready to append.
diff --git a/server/wal/wal_test.go b/server/wal/wal_test.go
index 86e188daacb..05014086c26 100644
--- a/server/wal/wal_test.go
+++ b/server/wal/wal_test.go
@@ -27,10 +27,12 @@ import (
"regexp"
"testing"
+ "github.com/stretchr/testify/assert"
"go.etcd.io/etcd/client/pkg/v3/fileutil"
"go.etcd.io/etcd/pkg/v3/pbutil"
"go.etcd.io/etcd/raft/v3/raftpb"
"go.etcd.io/etcd/server/v3/wal/walpb"
+ "go.uber.org/zap/zaptest"
"go.uber.org/zap"
)
@@ -231,14 +233,14 @@ func TestOpenAtIndex(t *testing.T) {
// The test creates a WAL directory and cuts out multiple WAL files. Then
// it corrupts one of the files by completely truncating it.
func TestVerify(t *testing.T) {
+ lg := zaptest.NewLogger(t)
walDir, err := ioutil.TempDir(t.TempDir(), "waltest")
if err != nil {
t.Fatal(err)
}
- defer os.RemoveAll(walDir)
// create WAL
- w, err := Create(zap.NewExample(), walDir, nil)
+ w, err := Create(lg, walDir, nil)
if err != nil {
t.Fatal(err)
}
@@ -255,11 +257,15 @@ func TestVerify(t *testing.T) {
}
}
+ hs := raftpb.HardState{Term: 1, Vote: 3, Commit: 5}
+ assert.NoError(t, w.Save(hs, nil))
+
// to verify the WAL is not corrupted at this point
- err = Verify(zap.NewExample(), walDir, walpb.Snapshot{})
+ hardstate, err := Verify(lg, walDir, walpb.Snapshot{})
if err != nil {
t.Errorf("expected a nil error, got %v", err)
}
+ assert.Equal(t, hs, *hardstate)
walFiles, err := ioutil.ReadDir(walDir)
if err != nil {
@@ -272,7 +278,7 @@ func TestVerify(t *testing.T) {
t.Fatal(err)
}
- err = Verify(zap.NewExample(), walDir, walpb.Snapshot{})
+ _, err = Verify(lg, walDir, walpb.Snapshot{})
if err == nil {
t.Error("expected a non-nil error, got nil")
}
diff --git a/tests/integration/cluster.go b/tests/integration/cluster.go
index 1291b8b724a..755d8520a66 100644
--- a/tests/integration/cluster.go
+++ b/tests/integration/cluster.go
@@ -52,6 +52,7 @@ import (
"go.etcd.io/etcd/server/v3/etcdserver/api/v3lock"
lockpb "go.etcd.io/etcd/server/v3/etcdserver/api/v3lock/v3lockpb"
"go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc"
+ "go.etcd.io/etcd/server/v3/verify"
"go.uber.org/zap/zapcore"
"go.uber.org/zap/zaptest"
@@ -583,6 +584,7 @@ type member struct {
useIP bool
isLearner bool
+ closed bool
}
func (m *member) GRPCAddr() string { return m.grpcAddr }
@@ -704,13 +706,7 @@ func mustNewMember(t testutil.TB, mcfg memberConfig) *member {
m.InitialCorruptCheck = true
m.WarningApplyDuration = embed.DefaultWarningApplyDuration
- level := zapcore.InfoLevel
- if os.Getenv("CLUSTER_DEBUG") != "" {
- level = zapcore.DebugLevel
- }
-
- options := zaptest.WrapOptions(zap.Fields(zap.String("member", mcfg.name)))
- m.Logger = zaptest.NewLogger(t, zaptest.Level(level), options).Named(mcfg.name)
+ m.Logger = memberLogger(t, mcfg.name)
t.Cleanup(func() {
// if we didn't cleanup the logger, the consecutive test
// might reuse this (t).
@@ -719,6 +715,16 @@ func mustNewMember(t testutil.TB, mcfg memberConfig) *member {
return m
}
+func memberLogger(t testutil.TB, name string) *zap.Logger {
+ level := zapcore.InfoLevel
+ if os.Getenv("CLUSTER_DEBUG") != "" {
+ level = zapcore.DebugLevel
+ }
+
+ options := zaptest.WrapOptions(zap.Fields(zap.String("member", name)))
+ return zaptest.NewLogger(t, zaptest.Level(level), options).Named(name)
+}
+
// listenGRPC starts a grpc server over a unix domain socket on the member
func (m *member) listenGRPC() error {
// prefix with localhost so cert has right domain
@@ -782,7 +788,7 @@ func NewClientV3(m *member) (*clientv3.Client, error) {
// Clone returns a member with the same server configuration. The returned
// member will not set PeerListeners and ClientListeners.
-func (m *member) Clone(_ testutil.TB) *member {
+func (m *member) Clone(t testutil.TB) *member {
mm := &member{}
mm.ServerConfig = m.ServerConfig
@@ -809,6 +815,7 @@ func (m *member) Clone(_ testutil.TB) *member {
mm.ElectionTicks = m.ElectionTicks
mm.PeerTLSInfo = m.PeerTLSInfo
mm.ClientTLSInfo = m.ClientTLSInfo
+ mm.Logger = memberLogger(t, mm.Name+"c")
return mm
}
@@ -1071,6 +1078,16 @@ func (m *member) Close() {
for _, f := range m.serverClosers {
f()
}
+ if !m.closed {
+ // Avoid verification of the same file multiple times
+ // (that might not exist any longer)
+ verify.MustVerifyIfEnabled(verify.Config{
+ Logger: m.Logger,
+ DataDir: m.DataDir,
+ ExactIndex: false,
+ })
+ }
+ m.closed = true
}
// Stop stops the member, but the data dir of the member is preserved.
diff --git a/tests/integration/member_test.go b/tests/integration/member_test.go
index 62520bbe662..5493924c9d2 100644
--- a/tests/integration/member_test.go
+++ b/tests/integration/member_test.go
@@ -21,6 +21,7 @@ import (
"reflect"
"testing"
+ "github.com/stretchr/testify/assert"
"go.etcd.io/etcd/client/v2"
)
@@ -65,6 +66,7 @@ func TestRestartMember(t *testing.T) {
}
func TestLaunchDuplicateMemberShouldFail(t *testing.T) {
+ BeforeTest(t)
size := 3
c := NewCluster(t, size)
m := c.Members[0].Clone(t)
@@ -78,6 +80,9 @@ func TestLaunchDuplicateMemberShouldFail(t *testing.T) {
if err := m.Launch(); err == nil {
t.Errorf("unexpect successful launch")
+ } else {
+ t.Logf("launch failed as expected: %v", err)
+ assert.Contains(t, err.Error(), "has already been bootstrapped")
}
}
diff --git a/tests/integration/snapshot/member_test.go b/tests/integration/snapshot/member_test.go
index 320c06c8dfc..4ae13f395b6 100644
--- a/tests/integration/snapshot/member_test.go
+++ b/tests/integration/snapshot/member_test.go
@@ -17,7 +17,6 @@ package snapshot_test
import (
"context"
"fmt"
- "os"
"testing"
"time"
@@ -42,7 +41,6 @@ func TestSnapshotV3RestoreMultiMemberAdd(t *testing.T) {
defer func() {
for i := 0; i < clusterN; i++ {
- os.RemoveAll(srvs[i].Config().Dir)
srvs[i].Close()
}
}()
@@ -82,7 +80,6 @@ func TestSnapshotV3RestoreMultiMemberAdd(t *testing.T) {
t.Fatal(err)
}
defer func() {
- os.RemoveAll(cfg.Dir)
srv.Close()
}()
select {
diff --git a/tests/integration/snapshot/v3_snapshot_test.go b/tests/integration/snapshot/v3_snapshot_test.go
index c0df2e48c13..4f6e557858b 100644
--- a/tests/integration/snapshot/v3_snapshot_test.go
+++ b/tests/integration/snapshot/v3_snapshot_test.go
@@ -30,6 +30,7 @@ import (
"go.etcd.io/etcd/etcdctl/v3/snapshot"
"go.etcd.io/etcd/server/v3/embed"
"go.etcd.io/etcd/tests/v3/integration"
+ "go.uber.org/zap/zapcore"
"go.uber.org/zap/zaptest"
)
@@ -39,7 +40,6 @@ func TestSnapshotV3RestoreSingle(t *testing.T) {
integration.BeforeTest(t)
kvs := []kv{{"foo1", "bar1"}, {"foo2", "bar2"}, {"foo3", "bar3"}}
dbPath := createSnapshotFile(t, kvs)
- defer os.RemoveAll(dbPath)
clusterN := 1
urls := newEmbedURLs(clusterN * 2)
@@ -73,7 +73,6 @@ func TestSnapshotV3RestoreSingle(t *testing.T) {
t.Fatal(err)
}
defer func() {
- os.RemoveAll(cfg.Dir)
srv.Close()
}()
select {
@@ -215,7 +214,6 @@ func createSnapshotFile(t *testing.T, kvs []kv) string {
t.Fatal(err)
}
- os.RemoveAll(cfg.Dir)
return dpPath
}
@@ -243,7 +241,8 @@ func restoreCluster(t *testing.T, clusterN int, dbPath string) (
cfg.LPUrls, cfg.APUrls = []url.URL{pURLs[i]}, []url.URL{pURLs[i]}
cfg.InitialCluster = ics
- sp := snapshot.NewV3(zaptest.NewLogger(t))
+ sp := snapshot.NewV3(
+ zaptest.NewLogger(t, zaptest.Level(zapcore.InfoLevel)).Named(cfg.Name).Named("sm"))
if err := sp.Restore(snapshot.RestoreConfig{
SnapshotPath: dbPath,
diff --git a/tests/integration/testing.go b/tests/integration/testing.go
index df7b3dd1fe0..27d5730f33f 100644
--- a/tests/integration/testing.go
+++ b/tests/integration/testing.go
@@ -22,6 +22,7 @@ import (
grpc_logsettable "github.com/grpc-ecosystem/go-grpc-middleware/logging/settable"
"go.etcd.io/etcd/client/pkg/v3/testutil"
"go.etcd.io/etcd/server/v3/embed"
+ "go.etcd.io/etcd/server/v3/verify"
"go.uber.org/zap/zapcore"
"go.uber.org/zap/zapgrpc"
"go.uber.org/zap/zaptest"
@@ -38,6 +39,9 @@ func BeforeTest(t testutil.TB) {
grpc_logger.Set(zapgrpc.NewLogger(zaptest.NewLogger(t).Named("grpc")))
+ // Integration tests should verify written state as much as possible.
+ os.Setenv(verify.ENV_VERIFY, verify.ENV_VERIFY_ALL_VALUE)
+
previousWD, err := os.Getwd()
if err != nil {
t.Fatal(err)