From efb584cc9b4324f8b6df707c261d3b3cbb81a10a Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Fri, 5 Mar 2021 23:25:42 +0100 Subject: [PATCH 1/9] leak.go: Make the per-test AfterTest strictly wait for none of the unwanted rountines. --- pkg/testutil/leak.go | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/pkg/testutil/leak.go b/pkg/testutil/leak.go index 504ef962c..722645c3b 100644 --- a/pkg/testutil/leak.go +++ b/pkg/testutil/leak.go @@ -59,6 +59,7 @@ func CheckLeakedGoroutine() bool { func CheckAfterTest(d time.Duration) error { http.DefaultTransport.(*http.Transport).CloseIdleConnections() var bad string + // Presence of these goroutines causes immediate test failure. badSubstring := map[string]string{ ").writeLoop(": "a Transport", "created by net/http/httptest.(*Server).Start": "an httptest.Server", @@ -74,17 +75,20 @@ func CheckAfterTest(d time.Duration) error { begin := time.Now() for time.Since(begin) < d { bad = "" - stacks = strings.Join(interestingGoroutines(), "\n\n") + goroutines := interestingGoroutines() + if len(goroutines) == 0 { + return nil + } + stacks = strings.Join(goroutines, "\n\n") + for substr, what := range badSubstring { if strings.Contains(stacks, substr) { bad = what } } - if bad == "" { - return nil - } - // Bad stuff found, but goroutines might just still be + // Undesired goroutines found, but goroutines might just still be // shutting down, so give it some time. + runtime.Gosched() time.Sleep(50 * time.Millisecond) } return fmt.Errorf("appears to have leaked %s:\n%s", bad, stacks) @@ -94,7 +98,7 @@ func CheckAfterTest(d time.Duration) error { // It will detect common goroutine leaks, retrying in case there are goroutines // not synchronously torn down, and fail the test if any goroutines are stuck. func AfterTest(t *testing.T) { - if err := CheckAfterTest(300 * time.Millisecond); err != nil { + if err := CheckAfterTest(1 * time.Second); err != nil { t.Errorf("Test %v", err) } } @@ -126,7 +130,8 @@ func interestingGoroutines() (gs []string) { strings.Contains(stack, "created by text/template/parse.lex") || strings.Contains(stack, "runtime.MHeap_Scavenger") || strings.Contains(stack, "rcrypto/internal/boring.(*PublicKeyRSA).finalize") || - strings.Contains(stack, "net.(*netFD).Close(") { + strings.Contains(stack, "net.(*netFD).Close(") || + strings.Contains(stack, "testing.(*T).Run") { continue } gs = append(gs, stack) From 87258efd90224bc8b59e000f75fe07fdeab68e2d Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Fri, 5 Mar 2021 23:27:32 +0100 Subject: [PATCH 2/9] Integration tests: Use zaptest.Logger based testing.TB Thanks to this the logs: - are automatically printed if the test fails. - are in pretty consistent format. - are annotated by 'member' information of the cluster emitting them. Side changes: - Set propert default got DefaultWarningApplyDuration (used to be '0') - Name the members based on their 'place' on the list (as opposed to 'random') --- raft/logger.go | 10 +++++ raft/raft.go | 2 +- raft/status.go | 2 +- raft/storage.go | 8 ++-- test.sh | 2 +- tests/integration/clientv3/cluster_test.go | 2 +- tests/integration/cluster.go | 46 +++++++++++++++------- 7 files changed, 50 insertions(+), 22 deletions(-) diff --git a/raft/logger.go b/raft/logger.go index 6d8962965..dc73b1f21 100644 --- a/raft/logger.go +++ b/raft/logger.go @@ -48,6 +48,16 @@ func SetLogger(l Logger) { raftLoggerMu.Unlock() } +func ResetDefaultLogger() { + SetLogger(defaultLogger) +} + +func getLogger() Logger { + raftLoggerMu.Lock() + defer raftLoggerMu.Unlock() + return raftLogger +} + var ( defaultLogger = &DefaultLogger{Logger: log.New(os.Stderr, "raft", log.LstdFlags)} discardLogger = &DefaultLogger{Logger: log.New(ioutil.Discard, "", 0)} diff --git a/raft/raft.go b/raft/raft.go index f62e3cd75..73c3ca499 100644 --- a/raft/raft.go +++ b/raft/raft.go @@ -230,7 +230,7 @@ func (c *Config) validate() error { } if c.Logger == nil { - c.Logger = raftLogger + c.Logger = getLogger() } if c.ReadOnlyOption == ReadOnlyLeaseBased && !c.CheckQuorum { diff --git a/raft/status.go b/raft/status.go index 3098c2c73..d7c7a4fa7 100644 --- a/raft/status.go +++ b/raft/status.go @@ -100,7 +100,7 @@ func (s Status) MarshalJSON() ([]byte, error) { func (s Status) String() string { b, err := s.MarshalJSON() if err != nil { - raftLogger.Panicf("unexpected error: %v", err) + getLogger().Panicf("unexpected error: %v", err) } return string(b) } diff --git a/raft/storage.go b/raft/storage.go index 4a403d017..8b16d4fa2 100644 --- a/raft/storage.go +++ b/raft/storage.go @@ -115,7 +115,7 @@ func (ms *MemoryStorage) Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) { return nil, ErrCompacted } if hi > ms.lastIndex()+1 { - raftLogger.Panicf("entries' hi(%d) is out of bound lastindex(%d)", hi, ms.lastIndex()) + getLogger().Panicf("entries' hi(%d) is out of bound lastindex(%d)", hi, ms.lastIndex()) } // only contains dummy entries. if len(ms.ents) == 1 { @@ -200,7 +200,7 @@ func (ms *MemoryStorage) CreateSnapshot(i uint64, cs *pb.ConfState, data []byte) offset := ms.ents[0].Index if i > ms.lastIndex() { - raftLogger.Panicf("snapshot %d is out of bound lastindex(%d)", i, ms.lastIndex()) + getLogger().Panicf("snapshot %d is out of bound lastindex(%d)", i, ms.lastIndex()) } ms.snapshot.Metadata.Index = i @@ -223,7 +223,7 @@ func (ms *MemoryStorage) Compact(compactIndex uint64) error { return ErrCompacted } if compactIndex > ms.lastIndex() { - raftLogger.Panicf("compact %d is out of bound lastindex(%d)", compactIndex, ms.lastIndex()) + getLogger().Panicf("compact %d is out of bound lastindex(%d)", compactIndex, ms.lastIndex()) } i := compactIndex - offset @@ -266,7 +266,7 @@ func (ms *MemoryStorage) Append(entries []pb.Entry) error { case uint64(len(ms.ents)) == offset: ms.ents = append(ms.ents, entries...) default: - raftLogger.Panicf("missing log entry [last: %d, append at: %d]", + getLogger().Panicf("missing log entry [last: %d, append at: %d]", ms.lastIndex(), entries[0].Index) } return nil diff --git a/test.sh b/test.sh index 1a985653a..86a4bb3f8 100755 --- a/test.sh +++ b/test.sh @@ -110,7 +110,7 @@ function integration_extra { function integration_pass { local pkgs=${USERPKG:-"./integration/..."} - run_for_module "tests" go_test "${pkgs}" "parallel" : -timeout="${TIMEOUT:-15m}" "-v" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" "$@" || return $? + run_for_module "tests" go_test "${pkgs}" "parallel" : -timeout="${TIMEOUT:-15m}" "${COMMON_TEST_FLAGS[@]}" "${RUN_ARG[@]}" "$@" || return $? integration_extra "$@" } diff --git a/tests/integration/clientv3/cluster_test.go b/tests/integration/clientv3/cluster_test.go index fe6cb7c96..8753334f9 100644 --- a/tests/integration/clientv3/cluster_test.go +++ b/tests/integration/clientv3/cluster_test.go @@ -265,7 +265,7 @@ func TestMemberPromote(t *testing.T) { // create and launch learner member based on the response of V3 Member Add API. // (the response has information on peer urls of the existing members in cluster) learnerMember := clus.MustNewMember(t, memberAddResp) - clus.Members = append(clus.Members, learnerMember) + if err := learnerMember.Launch(); err != nil { t.Fatal(err) } diff --git a/tests/integration/cluster.go b/tests/integration/cluster.go index d7b1351ae..748f346bc 100644 --- a/tests/integration/cluster.go +++ b/tests/integration/cluster.go @@ -36,11 +36,11 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/client/v2" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/logutil" "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/tlsutil" "go.etcd.io/etcd/pkg/v3/transport" "go.etcd.io/etcd/pkg/v3/types" + "go.etcd.io/etcd/raft/v3" "go.etcd.io/etcd/server/v3/embed" "go.etcd.io/etcd/server/v3/etcdserver" "go.etcd.io/etcd/server/v3/etcdserver/api/etcdhttp" @@ -52,6 +52,8 @@ 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.uber.org/zap/zapcore" + "go.uber.org/zap/zaptest" "github.com/soheilhy/cmux" "go.uber.org/zap" @@ -167,8 +169,14 @@ type ClusterConfig struct { } type cluster struct { - cfg *ClusterConfig - Members []*member + cfg *ClusterConfig + Members []*member + lastMemberNum int +} + +func (c *cluster) generateMemberName() string { + c.lastMemberNum++ + return fmt.Sprintf("m%v", c.lastMemberNum) } func schemeFromTLSInfo(tls *transport.TLSInfo) string { @@ -294,7 +302,7 @@ func (c *cluster) HTTPMembers() []client.Member { func (c *cluster) mustNewMember(t testing.TB) *member { m := mustNewMember(t, memberConfig{ - name: c.name(rand.Int()), + name: c.generateMemberName(), authToken: c.cfg.AuthToken, peerTLS: c.cfg.PeerTLS, clientTLS: c.cfg.ClientTLS, @@ -707,18 +715,28 @@ func mustNewMember(t testing.TB, mcfg memberConfig) *member { m.WatchProgressNotifyInterval = mcfg.WatchProgressNotifyInterval m.InitialCorruptCheck = true + m.WarningApplyDuration = embed.DefaultWarningApplyDuration - lcfg := logutil.DefaultZapLoggerConfig - m.LoggerConfig = &lcfg - m.LoggerConfig.OutputPaths = []string{"/dev/null"} - m.LoggerConfig.ErrorOutputPaths = []string{"/dev/null"} + level := zapcore.InfoLevel if os.Getenv("CLUSTER_DEBUG") != "" { - m.LoggerConfig.OutputPaths = []string{"stderr"} - m.LoggerConfig.ErrorOutputPaths = []string{"stderr"} + level = zapcore.DebugLevel } - m.Logger, err = m.LoggerConfig.Build() - if err != nil { - t.Fatal(err) + + if t != nil { + options := zaptest.WrapOptions(zap.Fields(zap.String("member", mcfg.name))) + m.Logger = zaptest.NewLogger(t, zaptest.Level(level), options) + if t != nil { + t.Cleanup(func() { + // if we didn't cleanup the logger, the consecutive test + // might reuse this (t). + raft.ResetDefaultLogger() + }) + } + } else { + m.Logger, err = zap.NewDevelopment(zap.IncreaseLevel(level)) + if err != nil { + log.Panic(err) + } } return m } @@ -1518,6 +1536,6 @@ func (c *ClusterV3) MustNewMember(t testing.TB, resp *clientv3.MemberAddResponse m.InitialPeerURLsMap[mm.Name] = mm.PeerURLs } m.InitialPeerURLsMap[m.Name] = types.MustNewURLs(resp.Member.PeerURLs) - + c.Members = append(c.Members, m) return m } From fb1d48e98ea6867895708e1f14c0c1366d7219e4 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sat, 6 Mar 2021 15:08:16 +0100 Subject: [PATCH 3/9] Integration tests: Use BeforeTest(t) instead of defer AfterTest(). Thanks to this change, a single method BeforeTest(t) can handle before-test logic as well as registration of cleanup code (t.Cleanup(func)). --- client/v3/client_test.go | 2 +- client/v3/txn_test.go | 2 +- pkg/testutil/leak.go | 12 ++- server/etcdserver/api/rafthttp/stream_test.go | 2 +- tests/e2e/ctl_v2_test.go | 55 +++++-------- tests/e2e/ctl_v3_migrate_test.go | 6 +- tests/e2e/ctl_v3_move_leader_test.go | 3 +- tests/e2e/ctl_v3_snapshot_test.go | 3 +- tests/e2e/ctl_v3_test.go | 4 +- tests/e2e/etcd_release_upgrade_test.go | 5 +- tests/e2e/v2_curl_test.go | 9 +- tests/integration/client/client_test.go | 7 +- tests/integration/clientv3/cluster_test.go | 23 +++--- .../clientv3/connectivity/black_hole_test.go | 5 +- .../clientv3/connectivity/dial_test.go | 15 ++-- .../connectivity/network_partition_test.go | 9 +- .../connectivity/server_shutdown_test.go | 9 +- .../clientv3/examples/example_test.go | 3 +- .../experimental/recipes/v3_barrier_test.go | 5 +- .../experimental/recipes/v3_lock_test.go | 3 +- tests/integration/clientv3/kv_test.go | 43 +++++----- .../integration/clientv3/lease/lease_test.go | 45 +++++----- .../clientv3/lease/leasing_test.go | 82 +++++++++---------- .../integration/clientv3/maintenance_test.go | 11 ++- tests/integration/clientv3/metrics_test.go | 3 +- tests/integration/clientv3/mirror_test.go | 3 +- tests/integration/clientv3/namespace_test.go | 5 +- .../clientv3/naming/endpoints_test.go | 7 +- .../clientv3/naming/resolver_test.go | 3 +- .../integration/clientv3/ordering_kv_test.go | 5 +- .../clientv3/ordering_util_test.go | 5 +- tests/integration/clientv3/role_test.go | 3 +- tests/integration/clientv3/txn_test.go | 13 ++- tests/integration/clientv3/user_test.go | 7 +- tests/integration/clientv3/watch_test.go | 27 +++--- tests/integration/cluster_test.go | 35 ++++---- tests/integration/member_test.go | 7 +- tests/integration/metrics_test.go | 9 +- tests/integration/network_partition_test.go | 8 +- .../proxy/grpcproxy/cluster_test.go | 3 +- tests/integration/proxy/grpcproxy/kv_test.go | 3 +- .../proxy/grpcproxy/register_test.go | 3 +- tests/integration/testing.go | 25 ++++++ tests/integration/v2_http_kv_test.go | 25 +++--- tests/integration/v3_alarm_test.go | 2 +- tests/integration/v3_auth_test.go | 16 ++-- tests/integration/v3_grpc_inflight_test.go | 6 +- tests/integration/v3_grpc_test.go | 55 ++++++------- tests/integration/v3_health_test.go | 4 +- tests/integration/v3_kv_test.go | 12 +-- tests/integration/v3_leadership_test.go | 9 +- tests/integration/v3_lease_test.go | 28 +++---- tests/integration/v3_tls_test.go | 4 +- tests/integration/v3_watch_test.go | 35 ++++---- tests/integration/v3election_grpc_test.go | 5 +- tests/integration/v3lock_grpc_test.go | 3 +- 56 files changed, 357 insertions(+), 384 deletions(-) create mode 100644 tests/integration/testing.go diff --git a/client/v3/client_test.go b/client/v3/client_test.go index 50704e38c..5e0f5844c 100644 --- a/client/v3/client_test.go +++ b/client/v3/client_test.go @@ -28,7 +28,7 @@ import ( ) func TestDialCancel(t *testing.T) { - defer testutil.AfterTest(t) + testutil.BeforeTest(t) // accept first connection so client is created with dial timeout ln, err := net.Listen("unix", "dialcancel:12345") diff --git a/client/v3/txn_test.go b/client/v3/txn_test.go index da762fadf..9101b04fa 100644 --- a/client/v3/txn_test.go +++ b/client/v3/txn_test.go @@ -22,7 +22,7 @@ import ( ) func TestTxnPanics(t *testing.T) { - defer testutil.AfterTest(t) + testutil.BeforeTest(t) kv := &kv{} diff --git a/pkg/testutil/leak.go b/pkg/testutil/leak.go index 722645c3b..828717072 100644 --- a/pkg/testutil/leak.go +++ b/pkg/testutil/leak.go @@ -28,7 +28,7 @@ running(leaking) after all tests. } func TestSample(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) ... } @@ -94,10 +94,18 @@ func CheckAfterTest(d time.Duration) error { return fmt.Errorf("appears to have leaked %s:\n%s", bad, stacks) } +// BeforeTest is a convenient way to register before-and-after code to a test. +// If you execute BeforeTest, you don't need to explicitly register AfterTest. +func BeforeTest(t testing.TB) { + t.Cleanup(func() { + AfterTest(t) + }) +} + // AfterTest is meant to run in a defer that executes after a test completes. // It will detect common goroutine leaks, retrying in case there are goroutines // not synchronously torn down, and fail the test if any goroutines are stuck. -func AfterTest(t *testing.T) { +func AfterTest(t testing.TB) { if err := CheckAfterTest(1 * time.Second); err != nil { t.Errorf("Test %v", err) } diff --git a/server/etcdserver/api/rafthttp/stream_test.go b/server/etcdserver/api/rafthttp/stream_test.go index f63dddc17..db181b6ec 100644 --- a/server/etcdserver/api/rafthttp/stream_test.go +++ b/server/etcdserver/api/rafthttp/stream_test.go @@ -187,7 +187,7 @@ func TestStreamReaderDialResult(t *testing.T) { // TestStreamReaderStopOnDial tests a stream reader closes the connection on stop. func TestStreamReaderStopOnDial(t *testing.T) { - defer testutil.AfterTest(t) + testutil.BeforeTest(t) h := http.Header{} h.Add("X-Server-Version", version.Version) tr := &respWaitRoundTripper{rrt: &respRoundTripper{code: http.StatusOK, header: h}} diff --git a/tests/e2e/ctl_v2_test.go b/tests/e2e/ctl_v2_test.go index 928b1f26a..0cb562c54 100644 --- a/tests/e2e/ctl_v2_test.go +++ b/tests/e2e/ctl_v2_test.go @@ -24,15 +24,22 @@ import ( "go.etcd.io/etcd/pkg/v3/testutil" ) +func BeforeTestV2(t testing.TB) { + skipInShortMode(t) + os.Setenv("ETCDCTL_API", "2") + t.Cleanup(func() { + os.Unsetenv("ETCDCTL_API") + }) + testutil.BeforeTest(t) +} + func TestCtlV2Set(t *testing.T) { testCtlV2Set(t, newConfigNoTLS(), false) } func TestCtlV2SetQuorum(t *testing.T) { testCtlV2Set(t, newConfigNoTLS(), true) } func TestCtlV2SetClientTLS(t *testing.T) { testCtlV2Set(t, newConfigClientTLS(), false) } func TestCtlV2SetPeerTLS(t *testing.T) { testCtlV2Set(t, newConfigPeerTLS(), false) } func TestCtlV2SetTLS(t *testing.T) { testCtlV2Set(t, newConfigTLS(), false) } func testCtlV2Set(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) cfg.enableV2 = true epc := setupEtcdctlTest(t, cfg, quorum) @@ -53,9 +60,7 @@ func TestCtlV2Mk(t *testing.T) { testCtlV2Mk(t, newConfigNoTLS(), false) } func TestCtlV2MkQuorum(t *testing.T) { testCtlV2Mk(t, newConfigNoTLS(), true) } func TestCtlV2MkTLS(t *testing.T) { testCtlV2Mk(t, newConfigTLS(), false) } func testCtlV2Mk(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) cfg.enableV2 = true epc := setupEtcdctlTest(t, cfg, quorum) @@ -78,9 +83,7 @@ func testCtlV2Mk(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) { func TestCtlV2Rm(t *testing.T) { testCtlV2Rm(t, newConfigNoTLS()) } func TestCtlV2RmTLS(t *testing.T) { testCtlV2Rm(t, newConfigTLS()) } func testCtlV2Rm(t *testing.T, cfg *etcdProcessClusterConfig) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) cfg.enableV2 = true epc := setupEtcdctlTest(t, cfg, true) @@ -104,9 +107,7 @@ func TestCtlV2Ls(t *testing.T) { testCtlV2Ls(t, newConfigNoTLS(), false) } func TestCtlV2LsQuorum(t *testing.T) { testCtlV2Ls(t, newConfigNoTLS(), true) } func TestCtlV2LsTLS(t *testing.T) { testCtlV2Ls(t, newConfigTLS(), false) } func testCtlV2Ls(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) cfg.enableV2 = true epc := setupEtcdctlTest(t, cfg, quorum) @@ -127,9 +128,7 @@ func TestCtlV2Watch(t *testing.T) { testCtlV2Watch(t, newConfigNoTLS(), false func TestCtlV2WatchTLS(t *testing.T) { testCtlV2Watch(t, newConfigTLS(), false) } func testCtlV2Watch(t *testing.T, cfg *etcdProcessClusterConfig, noSync bool) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) cfg.enableV2 = true epc := setupEtcdctlTest(t, cfg, true) @@ -152,9 +151,7 @@ func testCtlV2Watch(t *testing.T, cfg *etcdProcessClusterConfig, noSync bool) { } func TestCtlV2GetRoleUser(t *testing.T) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) copied := newConfigNoTLS() copied.enableV2 = true @@ -185,9 +182,7 @@ func TestCtlV2GetRoleUser(t *testing.T) { func TestCtlV2UserListUsername(t *testing.T) { testCtlV2UserList(t, "username") } func TestCtlV2UserListRoot(t *testing.T) { testCtlV2UserList(t, "root") } func testCtlV2UserList(t *testing.T, username string) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) copied := newConfigNoTLS() copied.enableV2 = true @@ -203,9 +198,7 @@ func testCtlV2UserList(t *testing.T, username string) { } func TestCtlV2RoleList(t *testing.T) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) copied := newConfigNoTLS() copied.enableV2 = true @@ -227,9 +220,7 @@ func TestCtlV2BackupV3(t *testing.T) { testCtlV2Backup(t, 0, true) } func TestCtlV2BackupV3Snapshot(t *testing.T) { testCtlV2Backup(t, 1, true) } func testCtlV2Backup(t *testing.T, snapCount int, v3 bool) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) backupDir, err := ioutil.TempDir("", "testbackup0.etcd") if err != nil { @@ -309,9 +300,7 @@ func testCtlV2Backup(t *testing.T, snapCount int, v3 bool) { } func TestCtlV2AuthWithCommonName(t *testing.T) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) copiedCfg := newConfigClientTLS() copiedCfg.clientCertAuthEnabled = true @@ -343,9 +332,7 @@ func TestCtlV2AuthWithCommonName(t *testing.T) { } func TestCtlV2ClusterHealth(t *testing.T) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) copied := newConfigNoTLS() copied.enableV2 = true @@ -498,8 +485,6 @@ func etcdctlBackup(clus *etcdProcessCluster, dataDir, backupDir string, v3 bool) } func setupEtcdctlTest(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) *etcdProcessCluster { - skipInShortMode(t) - if !quorum { cfg = configStandalone(*cfg) } diff --git a/tests/e2e/ctl_v3_migrate_test.go b/tests/e2e/ctl_v3_migrate_test.go index fed955d1f..b97d0686b 100644 --- a/tests/e2e/ctl_v3_migrate_test.go +++ b/tests/e2e/ctl_v3_migrate_test.go @@ -25,8 +25,12 @@ import ( "go.etcd.io/etcd/pkg/v3/testutil" ) +func BeforeTest(t testing.TB) { + testutil.BeforeTest(t) +} + func TestCtlV3Migrate(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cfg := newConfigNoTLS() cfg.enableV2 = true diff --git a/tests/e2e/ctl_v3_move_leader_test.go b/tests/e2e/ctl_v3_move_leader_test.go index 507ca4c16..3259825dc 100644 --- a/tests/e2e/ctl_v3_move_leader_test.go +++ b/tests/e2e/ctl_v3_move_leader_test.go @@ -23,7 +23,6 @@ import ( "time" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" "go.etcd.io/etcd/pkg/v3/types" ) @@ -37,7 +36,7 @@ func TestCtlV3MoveLeaderInsecure(t *testing.T) { } func testCtlV3MoveLeader(t *testing.T, cfg etcdProcessClusterConfig) { - defer testutil.AfterTest(t) + BeforeTest(t) epc := setupEtcdctlTest(t, &cfg, true) defer func() { diff --git a/tests/e2e/ctl_v3_snapshot_test.go b/tests/e2e/ctl_v3_snapshot_test.go index c95a10bda..5ef41280b 100644 --- a/tests/e2e/ctl_v3_snapshot_test.go +++ b/tests/e2e/ctl_v3_snapshot_test.go @@ -26,7 +26,6 @@ import ( "go.etcd.io/etcd/etcdctl/v3/snapshot" "go.etcd.io/etcd/pkg/v3/expect" - "go.etcd.io/etcd/pkg/v3/testutil" ) func TestCtlV3Snapshot(t *testing.T) { testCtl(t, snapshotTest) } @@ -164,7 +163,7 @@ func TestIssue6361(t *testing.T) { os.Setenv("EXPECT_DEBUG", "1") } - defer testutil.AfterTest(t) + BeforeTest(t) os.Setenv("ETCDCTL_API", "3") defer os.Unsetenv("ETCDCTL_API") diff --git a/tests/e2e/ctl_v3_test.go b/tests/e2e/ctl_v3_test.go index 98b93783d..5240e351b 100644 --- a/tests/e2e/ctl_v3_test.go +++ b/tests/e2e/ctl_v3_test.go @@ -52,7 +52,7 @@ func TestClusterVersion(t *testing.T) { if !fileutil.Exist(binary) { t.Skipf("%q does not exist", binary) } - defer testutil.AfterTest(t) + BeforeTest(t) cfg := newConfigNoTLS() cfg.execPath = binary cfg.snapshotCount = 3 @@ -198,7 +198,7 @@ func withFlagByEnv() ctlOption { } func testCtl(t *testing.T, testFunc func(ctlCtx), opts ...ctlOption) { - defer testutil.AfterTest(t) + BeforeTest(t) ret := ctlCtx{ t: t, diff --git a/tests/e2e/etcd_release_upgrade_test.go b/tests/e2e/etcd_release_upgrade_test.go index cbd92ef0b..1269ff20e 100644 --- a/tests/e2e/etcd_release_upgrade_test.go +++ b/tests/e2e/etcd_release_upgrade_test.go @@ -23,7 +23,6 @@ import ( "go.etcd.io/etcd/api/v3/version" "go.etcd.io/etcd/pkg/v3/fileutil" - "go.etcd.io/etcd/pkg/v3/testutil" ) // TestReleaseUpgrade ensures that changes to master branch does not affect @@ -34,7 +33,7 @@ func TestReleaseUpgrade(t *testing.T) { t.Skipf("%q does not exist", lastReleaseBinary) } - defer testutil.AfterTest(t) + BeforeTest(t) copiedCfg := newConfigNoTLS() copiedCfg.execPath = lastReleaseBinary @@ -125,7 +124,7 @@ func TestReleaseUpgradeWithRestart(t *testing.T) { t.Skipf("%q does not exist", lastReleaseBinary) } - defer testutil.AfterTest(t) + BeforeTest(t) copiedCfg := newConfigNoTLS() copiedCfg.execPath = lastReleaseBinary diff --git a/tests/e2e/v2_curl_test.go b/tests/e2e/v2_curl_test.go index 47b4c5954..0285a7bef 100644 --- a/tests/e2e/v2_curl_test.go +++ b/tests/e2e/v2_curl_test.go @@ -17,11 +17,8 @@ package e2e import ( "fmt" "math/rand" - "os" "strings" "testing" - - "go.etcd.io/etcd/pkg/v3/testutil" ) func TestV2CurlNoTLS(t *testing.T) { testCurlPutGet(t, newConfigNoTLS()) } @@ -31,7 +28,7 @@ func TestV2CurlPeerTLS(t *testing.T) { testCurlPutGet(t, newConfigPeerTLS()) func TestV2CurlClientTLS(t *testing.T) { testCurlPutGet(t, newConfigClientTLS()) } func TestV2CurlClientBoth(t *testing.T) { testCurlPutGet(t, newConfigClientBoth()) } func testCurlPutGet(t *testing.T, cfg *etcdProcessClusterConfig) { - defer testutil.AfterTest(t) + BeforeTestV2(t) // test doesn't use quorum gets, so ensure there are no followers to avoid // stale reads that will break the test @@ -66,9 +63,7 @@ func testCurlPutGet(t *testing.T, cfg *etcdProcessClusterConfig) { } func TestV2CurlIssue5182(t *testing.T) { - os.Setenv("ETCDCTL_API", "2") - defer os.Unsetenv("ETCDCTL_API") - defer testutil.AfterTest(t) + BeforeTestV2(t) copied := newConfigNoTLS() copied.enableV2 = true diff --git a/tests/integration/client/client_test.go b/tests/integration/client/client_test.go index 4b85b03e2..630344cb8 100644 --- a/tests/integration/client/client_test.go +++ b/tests/integration/client/client_test.go @@ -25,13 +25,12 @@ import ( "testing" "go.etcd.io/etcd/client/v2" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) // TestV2NoRetryEOF tests destructive api calls won't retry on a disconnection. func TestV2NoRetryEOF(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) // generate an EOF response; specify address so appears first in sorted ep list lEOF := integration.NewListenerWithAddr(t, fmt.Sprintf("127.0.0.1:%05d", os.Getpid())) defer lEOF.Close() @@ -63,7 +62,7 @@ func TestV2NoRetryEOF(t *testing.T) { // TestV2NoRetryNoLeader tests destructive api calls won't retry if given an error code. func TestV2NoRetryNoLeader(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) lHTTP := integration.NewListenerWithAddr(t, fmt.Sprintf("127.0.0.1:%05d", os.Getpid())) eh := &errHandler{errCode: http.StatusServiceUnavailable} srv := httptest.NewUnstartedServer(eh) @@ -89,7 +88,7 @@ func TestV2NoRetryNoLeader(t *testing.T) { // TestV2RetryRefuse tests destructive api calls will retry if a connection is refused. func TestV2RetryRefuse(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) cl := integration.NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) diff --git a/tests/integration/clientv3/cluster_test.go b/tests/integration/clientv3/cluster_test.go index 8753334f9..8fdc45f89 100644 --- a/tests/integration/clientv3/cluster_test.go +++ b/tests/integration/clientv3/cluster_test.go @@ -22,13 +22,12 @@ import ( "testing" "time" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/types" "go.etcd.io/etcd/tests/v3/integration" ) func TestMemberList(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -46,7 +45,7 @@ func TestMemberList(t *testing.T) { } func TestMemberAdd(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -65,7 +64,7 @@ func TestMemberAdd(t *testing.T) { } func TestMemberAddWithExistingURLs(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -89,7 +88,7 @@ func TestMemberAddWithExistingURLs(t *testing.T) { } func TestMemberRemove(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -127,7 +126,7 @@ func TestMemberRemove(t *testing.T) { } func TestMemberUpdate(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -155,7 +154,7 @@ func TestMemberUpdate(t *testing.T) { } func TestMemberAddUpdateWrongURLs(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -188,7 +187,7 @@ func TestMemberAddUpdateWrongURLs(t *testing.T) { } func TestMemberAddForLearner(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -217,7 +216,7 @@ func TestMemberAddForLearner(t *testing.T) { } func TestMemberPromote(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -294,7 +293,7 @@ func TestMemberPromote(t *testing.T) { // TestMemberPromoteMemberNotLearner ensures that promoting a voting member fails. func TestMemberPromoteMemberNotLearner(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -330,7 +329,7 @@ func TestMemberPromoteMemberNotLearner(t *testing.T) { // TestMemberPromoteMemberNotExist ensures that promoting a member that does not exist in cluster fails. func TestMemberPromoteMemberNotExist(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -379,7 +378,7 @@ func TestMemberPromoteMemberNotExist(t *testing.T) { // TestMaxLearnerInCluster verifies that the maximum number of learners allowed in a cluster is 1 func TestMaxLearnerInCluster(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) // 1. start with a cluster with 3 voting member and 0 learner member clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) diff --git a/tests/integration/clientv3/connectivity/black_hole_test.go b/tests/integration/clientv3/connectivity/black_hole_test.go index fc8795564..5dff05d49 100644 --- a/tests/integration/clientv3/connectivity/black_hole_test.go +++ b/tests/integration/clientv3/connectivity/black_hole_test.go @@ -24,7 +24,6 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" "go.etcd.io/etcd/tests/v3/integration/clientv3" "google.golang.org/grpc" @@ -34,7 +33,7 @@ import ( // blackholed endpoint, client balancer switches to healthy one. // TODO: test server-to-client keepalive ping func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 2, @@ -166,7 +165,7 @@ func TestBalancerUnderBlackholeNoKeepAliveSerializableGet(t *testing.T) { // testBalancerUnderBlackholeNoKeepAlive ensures that first request to blackholed endpoint // fails due to context timeout, but succeeds on next try, with endpoint switch. func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Client, context.Context) error) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 2, diff --git a/tests/integration/clientv3/connectivity/dial_test.go b/tests/integration/clientv3/connectivity/dial_test.go index 5e4955969..483b97709 100644 --- a/tests/integration/clientv3/connectivity/dial_test.go +++ b/tests/integration/clientv3/connectivity/dial_test.go @@ -23,7 +23,6 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" "go.etcd.io/etcd/tests/v3/integration" clientv3test "go.etcd.io/etcd/tests/v3/integration/clientv3" @@ -48,7 +47,7 @@ var ( // TestDialTLSExpired tests client with expired certs fails to dial. func TestDialTLSExpired(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, PeerTLS: &testTLSInfo, ClientTLS: &testTLSInfo, SkipCreatingClient: true}) defer clus.Terminate(t) @@ -71,7 +70,7 @@ func TestDialTLSExpired(t *testing.T) { // TestDialTLSNoConfig ensures the client fails to dial / times out // when TLS endpoints (https, unixs) are given but no tls config. func TestDialTLSNoConfig(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, ClientTLS: &testTLSInfo, SkipCreatingClient: true}) defer clus.Terminate(t) // expect "signed by unknown authority" @@ -102,7 +101,7 @@ func TestDialSetEndpointsAfterFail(t *testing.T) { // testDialSetEndpoints ensures SetEndpoints can replace unavailable endpoints with available ones. func testDialSetEndpoints(t *testing.T, setBefore bool) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, SkipCreatingClient: true}) defer clus.Terminate(t) @@ -145,7 +144,7 @@ func testDialSetEndpoints(t *testing.T, setBefore bool) { // TestSwitchSetEndpoints ensures SetEndpoints can switch one endpoint // with a new one that doesn't include original endpoint. func TestSwitchSetEndpoints(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -165,7 +164,7 @@ func TestSwitchSetEndpoints(t *testing.T) { } func TestRejectOldCluster(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) // 2 endpoints to test multi-endpoint Status clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2, SkipCreatingClient: true}) defer clus.Terminate(t) @@ -186,7 +185,7 @@ func TestRejectOldCluster(t *testing.T) { // TestDialForeignEndpoint checks an endpoint that is not registered // with the balancer can be dialed. func TestDialForeignEndpoint(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2}) defer clus.Terminate(t) @@ -209,7 +208,7 @@ func TestDialForeignEndpoint(t *testing.T) { // TestSetEndpointAndPut checks that a Put following a SetEndpoints // to a working endpoint will always succeed. func TestSetEndpointAndPut(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/connectivity/network_partition_test.go b/tests/integration/clientv3/connectivity/network_partition_test.go index 6949403a8..8829a6cce 100644 --- a/tests/integration/clientv3/connectivity/network_partition_test.go +++ b/tests/integration/clientv3/connectivity/network_partition_test.go @@ -26,7 +26,6 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" "go.etcd.io/etcd/tests/v3/integration/clientv3" "google.golang.org/grpc" @@ -104,7 +103,7 @@ func TestBalancerUnderNetworkPartitionSerializableGet(t *testing.T) { } func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, context.Context) error, timeout time.Duration) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 3, @@ -159,7 +158,7 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c // switches endpoint when leader fails and linearizable get requests returns // "etcdserver: request timed out". func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 3, @@ -214,7 +213,7 @@ func TestBalancerUnderNetworkPartitionWatchFollower(t *testing.T) { // testBalancerUnderNetworkPartitionWatch ensures watch stream // to a partitioned node be closed when context requires leader. func testBalancerUnderNetworkPartitionWatch(t *testing.T, isolateLeader bool) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 3, @@ -274,7 +273,7 @@ func testBalancerUnderNetworkPartitionWatch(t *testing.T, isolateLeader bool) { } func TestDropReadUnderNetworkPartition(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 3, diff --git a/tests/integration/clientv3/connectivity/server_shutdown_test.go b/tests/integration/clientv3/connectivity/server_shutdown_test.go index 85aa08750..fe5486e85 100644 --- a/tests/integration/clientv3/connectivity/server_shutdown_test.go +++ b/tests/integration/clientv3/connectivity/server_shutdown_test.go @@ -22,7 +22,6 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" "go.etcd.io/etcd/tests/v3/integration/clientv3" ) @@ -30,7 +29,7 @@ import ( // TestBalancerUnderServerShutdownWatch expects that watch client // switch its endpoints when the member of the pinned endpoint fails. func TestBalancerUnderServerShutdownWatch(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 3, @@ -142,7 +141,7 @@ func TestBalancerUnderServerShutdownTxn(t *testing.T) { // the pinned endpoint is shut down, the balancer switches its endpoints // and all subsequent put/delete/txn requests succeed with new endpoints. func testBalancerUnderServerShutdownMutable(t *testing.T, op func(*clientv3.Client, context.Context) error) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 3, @@ -200,7 +199,7 @@ func TestBalancerUnderServerShutdownGetSerializable(t *testing.T) { // the pinned endpoint is shut down, the balancer switches its endpoints // and all subsequent range requests succeed with new endpoints. func testBalancerUnderServerShutdownImmutable(t *testing.T, op func(*clientv3.Client, context.Context) error, timeout time.Duration) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{ Size: 3, @@ -269,7 +268,7 @@ type pinTestOpt struct { // testBalancerUnderServerStopInflightRangeOnRestart expects // inflight range request reconnects on server restart. func testBalancerUnderServerStopInflightRangeOnRestart(t *testing.T, linearizable bool, opt pinTestOpt) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) cfg := &integration.ClusterConfig{ Size: 2, diff --git a/tests/integration/clientv3/examples/example_test.go b/tests/integration/clientv3/examples/example_test.go index f93308acc..63f794f04 100644 --- a/tests/integration/clientv3/examples/example_test.go +++ b/tests/integration/clientv3/examples/example_test.go @@ -16,9 +16,10 @@ package clientv3_test import ( "context" + "log" + "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/pkg/v3/transport" - "log" ) func mockConfig_insecure() {} diff --git a/tests/integration/clientv3/experimental/recipes/v3_barrier_test.go b/tests/integration/clientv3/experimental/recipes/v3_barrier_test.go index b9dafe3ee..52bde238d 100644 --- a/tests/integration/clientv3/experimental/recipes/v3_barrier_test.go +++ b/tests/integration/clientv3/experimental/recipes/v3_barrier_test.go @@ -20,19 +20,18 @@ import ( "go.etcd.io/etcd/client/v3" recipe "go.etcd.io/etcd/client/v3/experimental/recipes" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestBarrierSingleNode(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) testBarrier(t, 5, func() *clientv3.Client { return clus.Client(0) }) } func TestBarrierMultiNode(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) testBarrier(t, 5, func() *clientv3.Client { return clus.RandClient() }) diff --git a/tests/integration/clientv3/experimental/recipes/v3_lock_test.go b/tests/integration/clientv3/experimental/recipes/v3_lock_test.go index 1447e17cd..9befd5f1d 100644 --- a/tests/integration/clientv3/experimental/recipes/v3_lock_test.go +++ b/tests/integration/clientv3/experimental/recipes/v3_lock_test.go @@ -24,7 +24,6 @@ import ( "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/concurrency" recipe "go.etcd.io/etcd/client/v3/experimental/recipes" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) @@ -178,7 +177,7 @@ func TestMutexSessionRelock(t *testing.T) { // waiters older than the new owner are gone by testing the case where // the waiter prior to the acquirer expires before the current holder. func TestMutexWaitsOnCurrentHolder(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/kv_test.go b/tests/integration/clientv3/kv_test.go index d3ecb881e..efb55165e 100644 --- a/tests/integration/clientv3/kv_test.go +++ b/tests/integration/clientv3/kv_test.go @@ -29,14 +29,13 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/api/v3/version" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" "google.golang.org/grpc" "google.golang.org/grpc/codes" ) func TestKVPutError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) var ( maxReqBytes = 1.5 * 1024 * 1024 // hard coded max in v3_server.go @@ -72,7 +71,7 @@ func TestKVPutError(t *testing.T) { } func TestKVPut(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -117,7 +116,7 @@ func TestKVPut(t *testing.T) { // TestKVPutWithIgnoreValue ensures that Put with WithIgnoreValue does not clobber the old value. func TestKVPutWithIgnoreValue(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -150,7 +149,7 @@ func TestKVPutWithIgnoreValue(t *testing.T) { // TestKVPutWithIgnoreLease ensures that Put with WithIgnoreLease does not affect the existing lease for the key. func TestKVPutWithIgnoreLease(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -189,7 +188,7 @@ func TestKVPutWithIgnoreLease(t *testing.T) { } func TestKVPutWithRequireLeader(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -235,7 +234,7 @@ func TestKVPutWithRequireLeader(t *testing.T) { } func TestKVRange(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -464,7 +463,7 @@ func TestKVRange(t *testing.T) { } func TestKVGetErrConnClosed(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -493,7 +492,7 @@ func TestKVGetErrConnClosed(t *testing.T) { } func TestKVNewAfterClose(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -520,7 +519,7 @@ func TestKVNewAfterClose(t *testing.T) { } func TestKVDeleteRange(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -592,7 +591,7 @@ func TestKVDeleteRange(t *testing.T) { } func TestKVDelete(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -624,7 +623,7 @@ func TestKVDelete(t *testing.T) { } func TestKVCompactError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -654,7 +653,7 @@ func TestKVCompactError(t *testing.T) { } func TestKVCompact(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -709,7 +708,7 @@ func TestKVCompact(t *testing.T) { // TestKVGetRetry ensures get will retry on disconnect. func TestKVGetRetry(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clusterSize := 3 clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: clusterSize}) @@ -763,7 +762,7 @@ func TestKVGetRetry(t *testing.T) { // TestKVPutFailGetRetry ensures a get will retry following a failed put. func TestKVPutFailGetRetry(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -803,7 +802,7 @@ func TestKVPutFailGetRetry(t *testing.T) { // TestKVGetCancel tests that a context cancel on a Get terminates as expected. func TestKVGetCancel(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -826,7 +825,7 @@ func TestKVGetCancel(t *testing.T) { // TestKVGetStoppedServerAndClose ensures closing after a failed Get works. func TestKVGetStoppedServerAndClose(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -844,7 +843,7 @@ func TestKVGetStoppedServerAndClose(t *testing.T) { // TestKVPutStoppedServerAndClose ensures closing after a failed Put works. func TestKVPutStoppedServerAndClose(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -875,7 +874,7 @@ func TestKVPutStoppedServerAndClose(t *testing.T) { // TestKVPutAtMostOnce ensures that a Put will only occur at most once // in the presence of network errors. func TestKVPutAtMostOnce(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -911,7 +910,7 @@ func TestKVPutAtMostOnce(t *testing.T) { // TestKVLargeRequests tests various client/server side request limits. func TestKVLargeRequests(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) tests := []struct { // make sure that "MaxCallSendMsgSize" < server-side default send/recv limit maxRequestBytesServer uint @@ -1003,7 +1002,7 @@ func TestKVLargeRequests(t *testing.T) { // TestKVForLearner ensures learner member only accepts serializable read request. func TestKVForLearner(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1082,7 +1081,7 @@ func TestKVForLearner(t *testing.T) { // TestBalancerSupportLearner verifies that balancer's retry and failover mechanism supports cluster with learner member func TestBalancerSupportLearner(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/lease/lease_test.go b/tests/integration/clientv3/lease/lease_test.go index 78196dfc3..326289949 100644 --- a/tests/integration/clientv3/lease/lease_test.go +++ b/tests/integration/clientv3/lease/lease_test.go @@ -26,12 +26,11 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/concurrency" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestLeaseNotFoundError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -45,7 +44,7 @@ func TestLeaseNotFoundError(t *testing.T) { } func TestLeaseGrant(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -71,7 +70,7 @@ func TestLeaseGrant(t *testing.T) { } func TestLeaseRevoke(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -97,7 +96,7 @@ func TestLeaseRevoke(t *testing.T) { } func TestLeaseKeepAliveOnce(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -121,7 +120,7 @@ func TestLeaseKeepAliveOnce(t *testing.T) { } func TestLeaseKeepAlive(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -161,7 +160,7 @@ func TestLeaseKeepAlive(t *testing.T) { } func TestLeaseKeepAliveOneSecond(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -189,7 +188,7 @@ func TestLeaseKeepAliveOneSecond(t *testing.T) { func TestLeaseKeepAliveHandleFailure(t *testing.T) { t.Skip("test it when we have a cluster client") - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -244,7 +243,7 @@ type leaseCh struct { // TestLeaseKeepAliveNotFound ensures a revoked lease won't halt other leases. func TestLeaseKeepAliveNotFound(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -277,7 +276,7 @@ func TestLeaseKeepAliveNotFound(t *testing.T) { } func TestLeaseGrantErrConnClosed(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -309,7 +308,7 @@ func TestLeaseGrantErrConnClosed(t *testing.T) { // queue is full thus dropping keepalive response sends, // keepalive request is sent with the same rate of TTL / 3. func TestLeaseKeepAliveFullResponseQueue(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -349,7 +348,7 @@ func TestLeaseKeepAliveFullResponseQueue(t *testing.T) { } func TestLeaseGrantNewAfterClose(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -376,7 +375,7 @@ func TestLeaseGrantNewAfterClose(t *testing.T) { } func TestLeaseRevokeNewAfterClose(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -415,7 +414,7 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) { // TestLeaseKeepAliveCloseAfterDisconnectRevoke ensures the keep alive channel is closed // following a disconnection, lease revoke, then reconnect. func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -461,7 +460,7 @@ func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) { // TestLeaseKeepAliveInitTimeout ensures the keep alive channel closes if // the initial keep alive request never gets a response. func TestLeaseKeepAliveInitTimeout(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -494,7 +493,7 @@ func TestLeaseKeepAliveInitTimeout(t *testing.T) { // TestLeaseKeepAliveInitTimeout ensures the keep alive channel closes if // a keep alive request after the first never gets a response. func TestLeaseKeepAliveTTLTimeout(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -529,7 +528,7 @@ func TestLeaseKeepAliveTTLTimeout(t *testing.T) { } func TestLeaseTimeToLive(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -587,7 +586,7 @@ func TestLeaseTimeToLive(t *testing.T) { } func TestLeaseTimeToLiveLeaseNotFound(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -622,7 +621,7 @@ func TestLeaseTimeToLiveLeaseNotFound(t *testing.T) { } func TestLeaseLeases(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -655,7 +654,7 @@ func TestLeaseLeases(t *testing.T) { // TestLeaseRenewLostQuorum ensures keepalives work after losing quorum // for a while. func TestLeaseRenewLostQuorum(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -703,7 +702,7 @@ func TestLeaseRenewLostQuorum(t *testing.T) { } func TestLeaseKeepAliveLoopExit(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -728,7 +727,7 @@ func TestLeaseKeepAliveLoopExit(t *testing.T) { // before, during, and after quorum loss to confirm Grant/KeepAlive tolerates // transient cluster failure. func TestV3LeaseFailureOverlap(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2}) defer clus.Terminate(t) @@ -781,7 +780,7 @@ func TestV3LeaseFailureOverlap(t *testing.T) { // TestLeaseWithRequireLeader checks keep-alive channel close when no leader. func TestLeaseWithRequireLeader(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/lease/leasing_test.go b/tests/integration/clientv3/lease/leasing_test.go index 0091505e2..ec44a3829 100644 --- a/tests/integration/clientv3/lease/leasing_test.go +++ b/tests/integration/clientv3/lease/leasing_test.go @@ -31,7 +31,7 @@ import ( ) func TestLeasingPutGet(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -90,7 +90,7 @@ func TestLeasingPutGet(t *testing.T) { // TestLeasingInterval checks the leasing KV fetches key intervals. func TestLeasingInterval(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -129,7 +129,7 @@ func TestLeasingInterval(t *testing.T) { // TestLeasingPutInvalidateNew checks the leasing KV updates its cache on a Put to a new key. func TestLeasingPutInvalidateNew(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -159,7 +159,7 @@ func TestLeasingPutInvalidateNew(t *testing.T) { // TestLeasingPutInvalidateExisting checks the leasing KV updates its cache on a Put to an existing key. func TestLeasingPutInvalidateExisting(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -193,7 +193,7 @@ func TestLeasingPutInvalidateExisting(t *testing.T) { // TestLeasingGetNoLeaseTTL checks a key with a TTL is not leased. func TestLeasingGetNoLeaseTTL(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -222,7 +222,7 @@ func TestLeasingGetNoLeaseTTL(t *testing.T) { // TestLeasingGetSerializable checks the leasing KV can make serialized requests // when the etcd cluster is partitioned. func TestLeasingGetSerializable(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2}) defer clus.Terminate(t) @@ -262,7 +262,7 @@ func TestLeasingGetSerializable(t *testing.T) { // TestLeasingPrevKey checks the cache respects WithPrevKV on puts. func TestLeasingPrevKey(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2}) defer clus.Terminate(t) @@ -288,7 +288,7 @@ func TestLeasingPrevKey(t *testing.T) { // TestLeasingRevGet checks the cache respects Get by Revision. func TestLeasingRevGet(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -324,7 +324,7 @@ func TestLeasingRevGet(t *testing.T) { // TestLeasingGetWithOpts checks options that can be served through the cache do not depend on the server. func TestLeasingGetWithOpts(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -369,7 +369,7 @@ func TestLeasingGetWithOpts(t *testing.T) { // TestLeasingConcurrentPut ensures that a get after concurrent puts returns // the recently put data. func TestLeasingConcurrentPut(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -416,7 +416,7 @@ func TestLeasingConcurrentPut(t *testing.T) { } func TestLeasingDisconnectedGet(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -445,7 +445,7 @@ func TestLeasingDisconnectedGet(t *testing.T) { } func TestLeasingDeleteOwner(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -479,7 +479,7 @@ func TestLeasingDeleteOwner(t *testing.T) { } func TestLeasingDeleteNonOwner(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -514,7 +514,7 @@ func TestLeasingDeleteNonOwner(t *testing.T) { } func TestLeasingOverwriteResponse(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -548,7 +548,7 @@ func TestLeasingOverwriteResponse(t *testing.T) { } func TestLeasingOwnerPutResponse(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -586,7 +586,7 @@ func TestLeasingOwnerPutResponse(t *testing.T) { } func TestLeasingTxnOwnerGetRange(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -615,7 +615,7 @@ func TestLeasingTxnOwnerGetRange(t *testing.T) { } func TestLeasingTxnOwnerGet(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -701,7 +701,7 @@ func TestLeasingTxnOwnerGet(t *testing.T) { } func TestLeasingTxnOwnerDeleteRange(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -740,7 +740,7 @@ func TestLeasingTxnOwnerDeleteRange(t *testing.T) { } func TestLeasingTxnOwnerDelete(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -771,7 +771,7 @@ func TestLeasingTxnOwnerDelete(t *testing.T) { } func TestLeasingTxnOwnerIf(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -865,7 +865,7 @@ func TestLeasingTxnOwnerIf(t *testing.T) { } func TestLeasingTxnCancel(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -899,7 +899,7 @@ func TestLeasingTxnCancel(t *testing.T) { } func TestLeasingTxnNonOwnerPut(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -977,7 +977,7 @@ func TestLeasingTxnNonOwnerPut(t *testing.T) { // TestLeasingTxnRandIfThenOrElse randomly leases keys two separate clients, then // issues a random If/{Then,Else} transaction on those keys to one client. func TestLeasingTxnRandIfThenOrElse(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1083,7 +1083,7 @@ func TestLeasingTxnRandIfThenOrElse(t *testing.T) { } func TestLeasingOwnerPutError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1104,7 +1104,7 @@ func TestLeasingOwnerPutError(t *testing.T) { } func TestLeasingOwnerDeleteError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1125,7 +1125,7 @@ func TestLeasingOwnerDeleteError(t *testing.T) { } func TestLeasingNonOwnerPutError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1150,7 +1150,7 @@ func TestLeasingOwnerDeleteFrom(t *testing.T) { } func testLeasingOwnerDelete(t *testing.T, del clientv3.Op) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1199,7 +1199,7 @@ func testLeasingOwnerDelete(t *testing.T, del clientv3.Op) { } func TestLeasingDeleteRangeBounds(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1257,7 +1257,7 @@ func TestLeaseDeleteRangeContendDel(t *testing.T) { } func testLeasingDeleteRangeContend(t *testing.T, op clientv3.Op) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1315,7 +1315,7 @@ func testLeasingDeleteRangeContend(t *testing.T, op clientv3.Op) { } func TestLeasingPutGetDeleteConcurrent(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1374,7 +1374,7 @@ func TestLeasingPutGetDeleteConcurrent(t *testing.T) { // TestLeasingReconnectOwnerRevoke checks that revocation works if // disconnected when trying to submit revoke txn. func TestLeasingReconnectOwnerRevoke(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1435,7 +1435,7 @@ func TestLeasingReconnectOwnerRevoke(t *testing.T) { // TestLeasingReconnectOwnerRevokeCompact checks that revocation works if // disconnected and the watch is compacted. func TestLeasingReconnectOwnerRevokeCompact(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1488,7 +1488,7 @@ func TestLeasingReconnectOwnerRevokeCompact(t *testing.T) { // TestLeasingReconnectOwnerConsistency checks a write error on an owner will // not cause inconsistency between the server and the client. func TestLeasingReconnectOwnerConsistency(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1562,7 +1562,7 @@ func TestLeasingReconnectOwnerConsistency(t *testing.T) { } func TestLeasingTxnAtomicCache(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1648,7 +1648,7 @@ func TestLeasingTxnAtomicCache(t *testing.T) { // TestLeasingReconnectTxn checks that Txn is resilient to disconnects. func TestLeasingReconnectTxn(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1684,7 +1684,7 @@ func TestLeasingReconnectTxn(t *testing.T) { // TestLeasingReconnectNonOwnerGet checks a get error on an owner will // not cause inconsistency between the server and the client. func TestLeasingReconnectNonOwnerGet(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1735,7 +1735,7 @@ func TestLeasingReconnectNonOwnerGet(t *testing.T) { } func TestLeasingTxnRangeCmp(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1770,7 +1770,7 @@ func TestLeasingTxnRangeCmp(t *testing.T) { } func TestLeasingDo(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1812,7 +1812,7 @@ func TestLeasingDo(t *testing.T) { } func TestLeasingTxnOwnerPutBranch(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1906,7 +1906,7 @@ func randCmps(pfx string, dat []*clientv3.PutResponse) (cmps []clientv3.Cmp, the } func TestLeasingSessionExpire(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1982,7 +1982,7 @@ func TestLeasingSessionExpireCancel(t *testing.T) { } for i := range tests { t.Run(fmt.Sprintf("test %d", i), func(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/maintenance_test.go b/tests/integration/clientv3/maintenance_test.go index f2b35150a..7dd41ceff 100644 --- a/tests/integration/clientv3/maintenance_test.go +++ b/tests/integration/clientv3/maintenance_test.go @@ -30,7 +30,6 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/lease" "go.etcd.io/etcd/server/v3/mvcc" "go.etcd.io/etcd/server/v3/mvcc/backend" @@ -38,7 +37,7 @@ import ( ) func TestMaintenanceHashKV(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -71,7 +70,7 @@ func TestMaintenanceHashKV(t *testing.T) { } func TestMaintenanceMoveLeader(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -102,7 +101,7 @@ func TestMaintenanceMoveLeader(t *testing.T) { // TestMaintenanceSnapshotError ensures that context cancel/timeout // before snapshot reading returns corresponding context errors. func TestMaintenanceSnapshotError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -141,7 +140,7 @@ func TestMaintenanceSnapshotError(t *testing.T) { // TestMaintenanceSnapshotErrorInflight ensures that inflight context cancel/timeout // fails snapshot reading with corresponding context errors. func TestMaintenanceSnapshotErrorInflight(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -198,7 +197,7 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) { } func TestMaintenanceStatus(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/metrics_test.go b/tests/integration/clientv3/metrics_test.go index c051a75b3..d834e2e8e 100644 --- a/tests/integration/clientv3/metrics_test.go +++ b/tests/integration/clientv3/metrics_test.go @@ -26,7 +26,6 @@ import ( "time" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" "go.etcd.io/etcd/tests/v3/integration" @@ -36,7 +35,7 @@ import ( ) func TestV3ClientMetrics(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) var ( addr = "localhost:27989" diff --git a/tests/integration/clientv3/mirror_test.go b/tests/integration/clientv3/mirror_test.go index bce951d1f..0516c5bff 100644 --- a/tests/integration/clientv3/mirror_test.go +++ b/tests/integration/clientv3/mirror_test.go @@ -24,12 +24,11 @@ import ( "go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/client/v3/mirror" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestMirrorSync(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/namespace_test.go b/tests/integration/clientv3/namespace_test.go index 3e46ff411..b7e07f4f3 100644 --- a/tests/integration/clientv3/namespace_test.go +++ b/tests/integration/clientv3/namespace_test.go @@ -22,12 +22,11 @@ import ( "go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/namespace" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestNamespacePutGet(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -56,7 +55,7 @@ func TestNamespacePutGet(t *testing.T) { } func TestNamespaceWatch(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/naming/endpoints_test.go b/tests/integration/clientv3/naming/endpoints_test.go index 19a1a27c0..707616851 100644 --- a/tests/integration/clientv3/naming/endpoints_test.go +++ b/tests/integration/clientv3/naming/endpoints_test.go @@ -22,12 +22,11 @@ import ( etcd "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/naming/endpoints" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestEndpointManager(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -89,7 +88,7 @@ func TestEndpointManager(t *testing.T) { // correctly with multiple hosts and correctly receive multiple // updates in a single revision. func TestEndpointManagerAtomicity(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -131,7 +130,7 @@ func TestEndpointManagerAtomicity(t *testing.T) { } func TestEndpointManagerCRUD(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/naming/resolver_test.go b/tests/integration/clientv3/naming/resolver_test.go index 2adf935a7..b7cc91a18 100644 --- a/tests/integration/clientv3/naming/resolver_test.go +++ b/tests/integration/clientv3/naming/resolver_test.go @@ -23,7 +23,6 @@ import ( "go.etcd.io/etcd/client/v3/naming/endpoints" "go.etcd.io/etcd/client/v3/naming/resolver" grpctest "go.etcd.io/etcd/pkg/v3/grpc_testing" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" "google.golang.org/grpc" @@ -33,7 +32,7 @@ import ( // This test mimics scenario described in grpc_naming.md doc. func TestEtcdGrpcResolver(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) s1PayloadBody := []byte{'1'} s1 := newDummyStubServer(s1PayloadBody) diff --git a/tests/integration/clientv3/ordering_kv_test.go b/tests/integration/clientv3/ordering_kv_test.go index 5338313c9..af17051f9 100644 --- a/tests/integration/clientv3/ordering_kv_test.go +++ b/tests/integration/clientv3/ordering_kv_test.go @@ -22,14 +22,13 @@ import ( "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/ordering" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestDetectKvOrderViolation(t *testing.T) { var errOrderViolation = errors.New("Detected Order Violation") - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -92,7 +91,7 @@ func TestDetectKvOrderViolation(t *testing.T) { func TestDetectTxnOrderViolation(t *testing.T) { var errOrderViolation = errors.New("Detected Order Violation") - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/ordering_util_test.go b/tests/integration/clientv3/ordering_util_test.go index 5ba239e66..522305ca7 100644 --- a/tests/integration/clientv3/ordering_util_test.go +++ b/tests/integration/clientv3/ordering_util_test.go @@ -21,12 +21,11 @@ import ( "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/ordering" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestEndpointSwitchResolvesViolation(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) eps := []string{ @@ -80,7 +79,7 @@ func TestEndpointSwitchResolvesViolation(t *testing.T) { } func TestUnresolvableOrderViolation(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 5, SkipCreatingClient: true}) defer clus.Terminate(t) cfg := clientv3.Config{ diff --git a/tests/integration/clientv3/role_test.go b/tests/integration/clientv3/role_test.go index 779aef761..62ba49d3a 100644 --- a/tests/integration/clientv3/role_test.go +++ b/tests/integration/clientv3/role_test.go @@ -19,12 +19,11 @@ import ( "testing" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" ) func TestRoleError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/txn_test.go b/tests/integration/clientv3/txn_test.go index ae19b4f8c..ffe93e096 100644 --- a/tests/integration/clientv3/txn_test.go +++ b/tests/integration/clientv3/txn_test.go @@ -22,13 +22,12 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/embed" "go.etcd.io/etcd/tests/v3/integration" ) func TestTxnError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -52,7 +51,7 @@ func TestTxnError(t *testing.T) { } func TestTxnWriteFail(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -102,7 +101,7 @@ func TestTxnWriteFail(t *testing.T) { func TestTxnReadRetry(t *testing.T) { t.Skipf("skipping txn read retry test: re-enable after we do retry on txn read request") - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -141,7 +140,7 @@ func TestTxnReadRetry(t *testing.T) { } func TestTxnSuccess(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -164,7 +163,7 @@ func TestTxnSuccess(t *testing.T) { } func TestTxnCompareRange(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -191,7 +190,7 @@ func TestTxnCompareRange(t *testing.T) { } func TestTxnNested(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/user_test.go b/tests/integration/clientv3/user_test.go index b775b5959..b776f44ab 100644 --- a/tests/integration/clientv3/user_test.go +++ b/tests/integration/clientv3/user_test.go @@ -21,13 +21,12 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/tests/v3/integration" "google.golang.org/grpc" ) func TestUserError(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -56,7 +55,7 @@ func TestUserError(t *testing.T) { } func TestUserErrorAuth(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -113,7 +112,7 @@ func authSetupRoot(t *testing.T, auth clientv3.Auth) { // Client can connect to etcd even if they supply credentials and the server is in AuthDisable mode. func TestGetTokenWithoutAuth(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2}) defer clus.Terminate(t) diff --git a/tests/integration/clientv3/watch_test.go b/tests/integration/clientv3/watch_test.go index 87b5fe1d4..9f2b2627d 100644 --- a/tests/integration/clientv3/watch_test.go +++ b/tests/integration/clientv3/watch_test.go @@ -28,7 +28,6 @@ import ( "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/api/v3/version" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc" "go.etcd.io/etcd/tests/v3/integration" "google.golang.org/grpc/metadata" @@ -46,7 +45,7 @@ type watchctx struct { } func runWatchTest(t *testing.T, f watcherTest) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -346,7 +345,7 @@ func putAndWatch(t *testing.T, wctx *watchctx, key, val string) { } func TestWatchResumeInitRev(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -401,7 +400,7 @@ func TestWatchResumeInitRev(t *testing.T) { // either a compaction error or all keys by staying in sync before the compaction // is finally applied. func TestWatchResumeCompacted(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -488,7 +487,7 @@ func TestWatchResumeCompacted(t *testing.T) { // TestWatchCompactRevision ensures the CompactRevision error is given on a // compaction event ahead of a watcher. func TestWatchCompactRevision(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -530,7 +529,7 @@ func TestWatchWithProgressNotify(t *testing.T) { testWatchWithProgressNot func TestWatchWithProgressNotifyNoEvent(t *testing.T) { testWatchWithProgressNotify(t, false) } func testWatchWithProgressNotify(t *testing.T, watchOnPut bool) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) // accelerate report interval so test terminates quickly oldpi := v3rpc.GetProgressReportInterval() @@ -622,7 +621,7 @@ func TestWatchRequestProgress(t *testing.T) { for _, c := range testCases { t.Run(c.name, func(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) watchTimeout := 3 * time.Second @@ -755,7 +754,7 @@ func TestWatchEventType(t *testing.T) { } func TestWatchErrConnClosed(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -785,7 +784,7 @@ func TestWatchErrConnClosed(t *testing.T) { } func TestWatchAfterClose(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -813,7 +812,7 @@ func TestWatchAfterClose(t *testing.T) { // TestWatchWithRequireLeader checks the watch channel closes when no leader. func TestWatchWithRequireLeader(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1049,7 +1048,7 @@ func TestWatchOverlapDropConnContextCancel(t *testing.T) { } func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3)) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1110,7 +1109,7 @@ func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3)) // TestWatchCancelAndCloseClient ensures that canceling a watcher then immediately // closing the client does not return a client closing error. func TestWatchCancelAndCloseClient(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) cli := clus.Client(0) @@ -1140,7 +1139,7 @@ func TestWatchCancelAndCloseClient(t *testing.T) { // to put them in resuming mode, cancels them so some resumes by cancel fail, // then closes the watcher interface to ensure correct clean up. func TestWatchStressResumeClose(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) cli := clus.Client(0) @@ -1162,7 +1161,7 @@ func TestWatchStressResumeClose(t *testing.T) { // TestWatchCancelDisconnected ensures canceling a watcher works when // its grpc stream is disconnected / reconnecting. func TestWatchCancelDisconnected(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) cli := clus.Client(0) diff --git a/tests/integration/cluster_test.go b/tests/integration/cluster_test.go index 987b7a1b5..e25d77f21 100644 --- a/tests/integration/cluster_test.go +++ b/tests/integration/cluster_test.go @@ -26,7 +26,6 @@ import ( "time" "go.etcd.io/etcd/client/v2" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/etcdserver" ) @@ -44,7 +43,7 @@ func TestClusterOf1(t *testing.T) { testCluster(t, 1) } func TestClusterOf3(t *testing.T) { testCluster(t, 3) } func testCluster(t *testing.T, size int) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, size) c.Launch(t) defer c.Terminate(t) @@ -52,7 +51,7 @@ func testCluster(t *testing.T, size int) { } func TestTLSClusterOf3(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewClusterByConfig(t, &ClusterConfig{Size: 3, PeerTLS: &testTLSInfo}) c.Launch(t) defer c.Terminate(t) @@ -62,7 +61,7 @@ func TestTLSClusterOf3(t *testing.T) { // Test that a cluster can progress when using separate client and server certs when peering. This supports certificate // authorities that don't issue dual-usage certificates. func TestTLSClusterOf3WithSpecificUsage(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewClusterByConfig(t, &ClusterConfig{Size: 3, PeerTLS: &testTLSInfoWithSpecificUsage}) c.Launch(t) defer c.Terminate(t) @@ -73,7 +72,7 @@ func TestClusterOf1UsingDiscovery(t *testing.T) { testClusterUsingDiscovery(t, 1 func TestClusterOf3UsingDiscovery(t *testing.T) { testClusterUsingDiscovery(t, 3) } func testClusterUsingDiscovery(t *testing.T, size int) { - defer testutil.AfterTest(t) + BeforeTest(t) dc := NewCluster(t, 1) dc.Launch(t) defer dc.Terminate(t) @@ -96,7 +95,7 @@ func testClusterUsingDiscovery(t *testing.T, size int) { } func TestTLSClusterOf3UsingDiscovery(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) dc := NewCluster(t, 1) dc.Launch(t) defer dc.Terminate(t) @@ -124,7 +123,7 @@ func TestDoubleClusterSizeOf1(t *testing.T) { testDoubleClusterSize(t, 1) } func TestDoubleClusterSizeOf3(t *testing.T) { testDoubleClusterSize(t, 3) } func testDoubleClusterSize(t *testing.T, size int) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, size) c.Launch(t) defer c.Terminate(t) @@ -136,7 +135,7 @@ func testDoubleClusterSize(t *testing.T, size int) { } func TestDoubleTLSClusterSizeOf3(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewClusterByConfig(t, &ClusterConfig{Size: 3, PeerTLS: &testTLSInfo}) c.Launch(t) defer c.Terminate(t) @@ -151,7 +150,7 @@ func TestDecreaseClusterSizeOf3(t *testing.T) { testDecreaseClusterSize(t, 3) } func TestDecreaseClusterSizeOf5(t *testing.T) { testDecreaseClusterSize(t, 5) } func testDecreaseClusterSize(t *testing.T, size int) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, size) c.Launch(t) defer c.Terminate(t) @@ -215,7 +214,7 @@ func TestForceNewCluster(t *testing.T) { } func TestAddMemberAfterClusterFullRotation(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, 3) c.Launch(t) defer c.Terminate(t) @@ -237,7 +236,7 @@ func TestAddMemberAfterClusterFullRotation(t *testing.T) { // Ensure we can remove a member then add a new one back immediately. func TestIssue2681(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, 5) c.Launch(t) defer c.Terminate(t) @@ -257,7 +256,7 @@ func TestIssue2746(t *testing.T) { testIssue2746(t, 5) } func TestIssue2746WithThree(t *testing.T) { testIssue2746(t, 3) } func testIssue2746(t *testing.T, members int) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, members) for _, m := range c.Members { @@ -282,7 +281,7 @@ func testIssue2746(t *testing.T, members int) { // Ensure etcd will not panic when removing a just started member. func TestIssue2904(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) // start 1-member cluster to ensure member 0 is the leader of the cluster. c := NewCluster(t, 1) c.Launch(t) @@ -319,7 +318,7 @@ func TestIssue2904(t *testing.T) { // deadlocking. func TestIssue3699(t *testing.T) { // start a cluster of 3 nodes a, b, c - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, 3) c.Launch(t) defer c.Terminate(t) @@ -371,7 +370,7 @@ func TestIssue3699(t *testing.T) { // TestRejectUnhealthyAdd ensures an unhealthy cluster rejects adding members. func TestRejectUnhealthyAdd(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, 3) for _, m := range c.Members { m.ServerConfig.StrictReconfigCheck = true @@ -415,7 +414,7 @@ func TestRejectUnhealthyAdd(t *testing.T) { // TestRejectUnhealthyRemove ensures an unhealthy cluster rejects removing members // if quorum will be lost. func TestRejectUnhealthyRemove(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, 5) for _, m := range c.Members { m.ServerConfig.StrictReconfigCheck = true @@ -462,7 +461,7 @@ func TestRejectUnhealthyRemove(t *testing.T) { // if 'initial-cluster-state' is set 'new' and old data directory still exists // (see https://github.com/etcd-io/etcd/issues/7512 for more). func TestRestartRemoved(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) // 1. start single-member cluster c := NewCluster(t, 1) @@ -540,7 +539,7 @@ func clusterMustProgress(t *testing.T, membs []*member) { } func TestSpeedyTerminate(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) // Stop/Restart so requests will time out on lost leaders for i := 0; i < 3; i++ { diff --git a/tests/integration/member_test.go b/tests/integration/member_test.go index bc19deda6..54d3dbedc 100644 --- a/tests/integration/member_test.go +++ b/tests/integration/member_test.go @@ -23,11 +23,10 @@ import ( "testing" "go.etcd.io/etcd/client/v2" - "go.etcd.io/etcd/pkg/v3/testutil" ) func TestPauseMember(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, 5) c.Launch(t) @@ -46,7 +45,7 @@ func TestPauseMember(t *testing.T) { } func TestRestartMember(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) c := NewCluster(t, 3) c.Launch(t) defer c.Terminate(t) @@ -84,7 +83,7 @@ func TestLaunchDuplicateMemberShouldFail(t *testing.T) { } func TestSnapshotAndRestartMember(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) m := mustNewMember(t, memberConfig{name: "snapAndRestartTest"}) m.SnapshotCount = 100 m.Launch() diff --git a/tests/integration/metrics_test.go b/tests/integration/metrics_test.go index dfa3cc2e9..dc90f6198 100644 --- a/tests/integration/metrics_test.go +++ b/tests/integration/metrics_test.go @@ -23,14 +23,13 @@ import ( "time" pb "go.etcd.io/etcd/api/v3/etcdserverpb" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" "go.etcd.io/etcd/server/v3/etcdserver" ) // TestMetricDbSizeBoot checks that the db size metric is set on boot. func TestMetricDbSizeBoot(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -54,7 +53,7 @@ func TestMetricDbSizeDefragDebugging(t *testing.T) { // testMetricDbSizeDefrag checks that the db size metric is set after defrag. func testMetricDbSizeDefrag(t *testing.T, name string) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -168,7 +167,7 @@ func testMetricDbSizeDefrag(t *testing.T, name string) { } func TestMetricQuotaBackendBytes(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -186,7 +185,7 @@ func TestMetricQuotaBackendBytes(t *testing.T) { } func TestMetricsHealth(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/network_partition_test.go b/tests/integration/network_partition_test.go index 2210f9740..6abc36700 100644 --- a/tests/integration/network_partition_test.go +++ b/tests/integration/network_partition_test.go @@ -18,12 +18,10 @@ import ( "fmt" "testing" "time" - - "go.etcd.io/etcd/pkg/v3/testutil" ) func TestNetworkPartition5MembersLeaderInMinority(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 5}) defer clus.Terminate(t) @@ -71,7 +69,7 @@ func TestNetworkPartition5MembersLeaderInMajority(t *testing.T) { } func testNetworkPartition5MembersLeaderInMajority(t *testing.T) error { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 5}) defer clus.Terminate(t) @@ -110,7 +108,7 @@ func testNetworkPartition5MembersLeaderInMajority(t *testing.T) error { } func TestNetworkPartition4Members(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 4}) defer clus.Terminate(t) diff --git a/tests/integration/proxy/grpcproxy/cluster_test.go b/tests/integration/proxy/grpcproxy/cluster_test.go index ac5a1e01e..612a5d2a8 100644 --- a/tests/integration/proxy/grpcproxy/cluster_test.go +++ b/tests/integration/proxy/grpcproxy/cluster_test.go @@ -22,7 +22,6 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/proxy/grpcproxy" "go.etcd.io/etcd/tests/v3/integration" @@ -31,7 +30,7 @@ import ( ) func TestClusterProxyMemberList(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/proxy/grpcproxy/kv_test.go b/tests/integration/proxy/grpcproxy/kv_test.go index 8e9aa05a3..b73276001 100644 --- a/tests/integration/proxy/grpcproxy/kv_test.go +++ b/tests/integration/proxy/grpcproxy/kv_test.go @@ -22,7 +22,6 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/proxy/grpcproxy" "go.etcd.io/etcd/tests/v3/integration" @@ -30,7 +29,7 @@ import ( ) func TestKVProxyRange(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/proxy/grpcproxy/register_test.go b/tests/integration/proxy/grpcproxy/register_test.go index 914986c5f..01c2ed297 100644 --- a/tests/integration/proxy/grpcproxy/register_test.go +++ b/tests/integration/proxy/grpcproxy/register_test.go @@ -20,7 +20,6 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/naming/endpoints" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/proxy/grpcproxy" "go.etcd.io/etcd/tests/v3/integration" @@ -28,7 +27,7 @@ import ( ) func TestRegister(t *testing.T) { - defer testutil.AfterTest(t) + integration.BeforeTest(t) clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/testing.go b/tests/integration/testing.go new file mode 100644 index 000000000..a525bb9a4 --- /dev/null +++ b/tests/integration/testing.go @@ -0,0 +1,25 @@ +// 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 integration + +import ( + "testing" + + "go.etcd.io/etcd/pkg/v3/testutil" +) + +func BeforeTest(t testing.TB) { + testutil.BeforeTest(t) +} diff --git a/tests/integration/v2_http_kv_test.go b/tests/integration/v2_http_kv_test.go index daba250e8..46e7e445b 100644 --- a/tests/integration/v2_http_kv_test.go +++ b/tests/integration/v2_http_kv_test.go @@ -26,12 +26,11 @@ import ( "testing" "time" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" ) func TestV2Set(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -93,7 +92,7 @@ func TestV2Set(t *testing.T) { } func TestV2CreateUpdate(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -229,7 +228,7 @@ func TestV2CreateUpdate(t *testing.T) { } func TestV2CAS(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -377,7 +376,7 @@ func TestV2CAS(t *testing.T) { } func TestV2Delete(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -477,7 +476,7 @@ func TestV2Delete(t *testing.T) { } func TestV2CAD(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -577,7 +576,7 @@ func TestV2CAD(t *testing.T) { } func TestV2Unique(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -644,7 +643,7 @@ func TestV2Unique(t *testing.T) { } func TestV2Get(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -742,7 +741,7 @@ func TestV2Get(t *testing.T) { } func TestV2QuorumGet(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -840,7 +839,7 @@ func TestV2QuorumGet(t *testing.T) { } func TestV2Watch(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -878,7 +877,7 @@ func TestV2Watch(t *testing.T) { } func TestV2WatchWithIndex(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -945,7 +944,7 @@ func TestV2WatchWithIndex(t *testing.T) { } func TestV2WatchKeyInDir(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) @@ -1006,7 +1005,7 @@ func TestV2WatchKeyInDir(t *testing.T) { } func TestV2Head(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cl := NewCluster(t, 1) cl.Launch(t) defer cl.Terminate(t) diff --git a/tests/integration/v3_alarm_test.go b/tests/integration/v3_alarm_test.go index 2d6063562..7270f943d 100644 --- a/tests/integration/v3_alarm_test.go +++ b/tests/integration/v3_alarm_test.go @@ -147,7 +147,7 @@ func TestV3AlarmDeactivate(t *testing.T) { } func TestV3CorruptAlarm(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) diff --git a/tests/integration/v3_auth_test.go b/tests/integration/v3_auth_test.go index dc186824d..710c137e6 100644 --- a/tests/integration/v3_auth_test.go +++ b/tests/integration/v3_auth_test.go @@ -30,7 +30,7 @@ import ( // TestV3AuthEmptyUserGet ensures that a get with an empty user will return an empty user error. func TestV3AuthEmptyUserGet(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -49,7 +49,7 @@ func TestV3AuthEmptyUserGet(t *testing.T) { // TestV3AuthTokenWithDisable tests that auth won't crash if // given a valid token when authentication is disabled func TestV3AuthTokenWithDisable(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -81,7 +81,7 @@ func TestV3AuthTokenWithDisable(t *testing.T) { } func TestV3AuthRevision(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -119,7 +119,7 @@ func TestV3AuthWithLeaseRevokeWithRootJWT(t *testing.T) { } func testV3AuthWithLeaseRevokeWithRoot(t *testing.T, ccfg ClusterConfig) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ccfg) defer clus.Terminate(t) @@ -177,7 +177,7 @@ type user struct { } func TestV3AuthWithLeaseRevoke(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -223,7 +223,7 @@ func TestV3AuthWithLeaseRevoke(t *testing.T) { } func TestV3AuthWithLeaseAttach(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -335,7 +335,7 @@ func authSetupRoot(t *testing.T, auth pb.AuthClient) { } func TestV3AuthNonAuthorizedRPCs(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -358,7 +358,7 @@ func TestV3AuthNonAuthorizedRPCs(t *testing.T) { func TestV3AuthOldRevConcurrent(t *testing.T) { t.Skip() // TODO(jingyih): re-enable the test when #10408 is fixed. - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/v3_grpc_inflight_test.go b/tests/integration/v3_grpc_inflight_test.go index f24d77494..57eca5d1d 100644 --- a/tests/integration/v3_grpc_inflight_test.go +++ b/tests/integration/v3_grpc_inflight_test.go @@ -22,8 +22,6 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" - "go.etcd.io/etcd/pkg/v3/testutil" - "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -32,7 +30,7 @@ import ( // TestV3MaintenanceDefragmentInflightRange ensures inflight range requests // does not panic the mvcc backend while defragment is running. func TestV3MaintenanceDefragmentInflightRange(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -62,7 +60,7 @@ func TestV3MaintenanceDefragmentInflightRange(t *testing.T) { // They are either finished or canceled, but never crash the backend. // See https://github.com/etcd-io/etcd/issues/7322 for more detail. func TestV3KVInflightRangeRequests(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/v3_grpc_test.go b/tests/integration/v3_grpc_test.go index 3cd94c70e..5c29b6319 100644 --- a/tests/integration/v3_grpc_test.go +++ b/tests/integration/v3_grpc_test.go @@ -28,7 +28,6 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" "google.golang.org/grpc" @@ -40,7 +39,7 @@ import ( // TestV3PutOverwrite puts a key with the v3 api to a random cluster member, // overwrites it, then checks that the change was applied. func TestV3PutOverwrite(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -88,7 +87,7 @@ func TestV3PutOverwrite(t *testing.T) { // TestPutRestart checks if a put after an unrelated member restart succeeds func TestV3PutRestart(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -120,7 +119,7 @@ func TestV3PutRestart(t *testing.T) { // TestV3CompactCurrentRev ensures keys are present when compacting on current revision. func TestV3CompactCurrentRev(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -154,7 +153,7 @@ func TestV3CompactCurrentRev(t *testing.T) { // TestV3HashKV ensures that multiple calls of HashKV on same node return same hash and compact rev. func TestV3HashKV(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -202,7 +201,7 @@ func TestV3HashKV(t *testing.T) { } func TestV3TxnTooManyOps(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) maxTxnOps := uint(128) clus := NewClusterV3(t, &ClusterConfig{Size: 3, MaxTxnOps: maxTxnOps}) defer clus.Terminate(t) @@ -278,7 +277,7 @@ func TestV3TxnTooManyOps(t *testing.T) { } func TestV3TxnDuplicateKeys(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -396,7 +395,7 @@ func TestV3TxnDuplicateKeys(t *testing.T) { // Testv3TxnRevision tests that the transaction header revision is set as expected. func TestV3TxnRevision(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -447,7 +446,7 @@ func TestV3TxnRevision(t *testing.T) { // Testv3TxnCmpHeaderRev tests that the txn header revision is set as expected // when compared to the Succeeded field in the txn response. func TestV3TxnCmpHeaderRev(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -503,7 +502,7 @@ func TestV3TxnCmpHeaderRev(t *testing.T) { // TestV3TxnRangeCompare tests range comparisons in txns func TestV3TxnRangeCompare(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -614,7 +613,7 @@ func TestV3TxnRangeCompare(t *testing.T) { // TestV3TxnNested tests nested txns follow paths as expected. func TestV3TxnNestedPath(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -667,7 +666,7 @@ func TestV3TxnNestedPath(t *testing.T) { // TestV3PutIgnoreValue ensures that writes with ignore_value overwrites with previous key-value pair. func TestV3PutIgnoreValue(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -800,7 +799,7 @@ func TestV3PutIgnoreValue(t *testing.T) { // TestV3PutIgnoreLease ensures that writes with ignore_lease uses previous lease for the key overwrites. func TestV3PutIgnoreLease(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -940,7 +939,7 @@ func TestV3PutIgnoreLease(t *testing.T) { // TestV3PutMissingLease ensures that a Put on a key with a bogus lease fails. func TestV3PutMissingLease(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1011,7 +1010,7 @@ func TestV3PutMissingLease(t *testing.T) { // TestV3DeleteRange tests various edge cases in the DeleteRange API. func TestV3DeleteRange(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) tests := []struct { keySet []string begin string @@ -1120,7 +1119,7 @@ func TestV3DeleteRange(t *testing.T) { // TestV3TxnInvalidRange tests that invalid ranges are rejected in txns. func TestV3TxnInvalidRange(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1163,7 +1162,7 @@ func TestV3TxnInvalidRange(t *testing.T) { } func TestV3TooLargeRequest(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1182,7 +1181,7 @@ func TestV3TooLargeRequest(t *testing.T) { // TestV3Hash tests hash. func TestV3Hash(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1207,7 +1206,7 @@ func TestV3Hash(t *testing.T) { // TestV3HashRestart ensures that hash stays the same after restart. func TestV3HashRestart(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1238,7 +1237,7 @@ func TestV3HashRestart(t *testing.T) { // TestV3StorageQuotaAPI tests the V3 server respects quotas at the API layer func TestV3StorageQuotaAPI(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) quotasize := int64(16 * os.Getpagesize()) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) @@ -1285,7 +1284,7 @@ func TestV3StorageQuotaAPI(t *testing.T) { } func TestV3RangeRequest(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) tests := []struct { putKeys []string reqs []pb.RangeRequest @@ -1535,7 +1534,7 @@ func newClusterV3NoClients(t *testing.T, cfg *ClusterConfig) *ClusterV3 { // TestTLSGRPCRejectInsecureClient checks that connection is rejected if server is TLS but not client. func TestTLSGRPCRejectInsecureClient(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cfg := ClusterConfig{Size: 3, ClientTLS: &testTLSInfo} clus := newClusterV3NoClients(t, &cfg) @@ -1570,7 +1569,7 @@ func TestTLSGRPCRejectInsecureClient(t *testing.T) { // TestTLSGRPCRejectSecureClient checks that connection is rejected if client is TLS but not server. func TestTLSGRPCRejectSecureClient(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cfg := ClusterConfig{Size: 3} clus := newClusterV3NoClients(t, &cfg) @@ -1588,7 +1587,7 @@ func TestTLSGRPCRejectSecureClient(t *testing.T) { // TestTLSGRPCAcceptSecureAll checks that connection is accepted if both client and server are TLS func TestTLSGRPCAcceptSecureAll(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cfg := ClusterConfig{Size: 3, ClientTLS: &testTLSInfo} clus := newClusterV3NoClients(t, &cfg) @@ -1731,7 +1730,7 @@ func testTLSReload( replaceFunc func(), revertFunc func(), useIP bool) { - defer testutil.AfterTest(t) + BeforeTest(t) // 1. separate copies for TLS assets modification tlsInfo := cloneFunc() @@ -1806,7 +1805,7 @@ func testTLSReload( } func TestGRPCRequireLeader(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cfg := ClusterConfig{Size: 3} clus := newClusterV3NoClients(t, &cfg) @@ -1833,7 +1832,7 @@ func TestGRPCRequireLeader(t *testing.T) { } func TestGRPCStreamRequireLeader(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cfg := ClusterConfig{Size: 3} clus := newClusterV3NoClients(t, &cfg) @@ -1896,7 +1895,7 @@ func TestGRPCStreamRequireLeader(t *testing.T) { // TestV3LargeRequests ensures that configurable MaxRequestBytes works as intended. func TestV3LargeRequests(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) tests := []struct { maxRequestBytes uint valueSize int diff --git a/tests/integration/v3_health_test.go b/tests/integration/v3_health_test.go index 1f852d801..fd7257fb9 100644 --- a/tests/integration/v3_health_test.go +++ b/tests/integration/v3_health_test.go @@ -18,13 +18,11 @@ import ( "context" "testing" - "go.etcd.io/etcd/pkg/v3/testutil" - healthpb "google.golang.org/grpc/health/grpc_health_v1" ) func TestHealthCheck(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/v3_kv_test.go b/tests/integration/v3_kv_test.go index 9f3b4e7c6..db26c3a45 100644 --- a/tests/integration/v3_kv_test.go +++ b/tests/integration/v3_kv_test.go @@ -2,19 +2,19 @@ package integration import ( "context" - "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/client/v3/namespace" - "go.etcd.io/etcd/pkg/v3/testutil" - "go.etcd.io/etcd/server/v3/embed" - "go.etcd.io/etcd/server/v3/etcdserver/api/v3client" "io/ioutil" "os" "testing" + + "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/namespace" + "go.etcd.io/etcd/server/v3/embed" + "go.etcd.io/etcd/server/v3/etcdserver/api/v3client" ) // TestKVWithEmptyValue ensures that a get/delete with an empty value, and with WithFromKey/WithPrefix function will return an empty error. func TestKVWithEmptyValue(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) cfg := embed.NewConfig() diff --git a/tests/integration/v3_leadership_test.go b/tests/integration/v3_leadership_test.go index 8e8f2f137..7fd5ff1ca 100644 --- a/tests/integration/v3_leadership_test.go +++ b/tests/integration/v3_leadership_test.go @@ -22,14 +22,13 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" - "go.etcd.io/etcd/pkg/v3/testutil" ) func TestMoveLeader(t *testing.T) { testMoveLeader(t, true) } func TestMoveLeaderService(t *testing.T) { testMoveLeader(t, false) } func testMoveLeader(t *testing.T, auto bool) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -97,7 +96,7 @@ func testMoveLeader(t *testing.T, auto bool) { // TestMoveLeaderError ensures that request to non-leader fail. func TestMoveLeaderError(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -116,7 +115,7 @@ func TestMoveLeaderError(t *testing.T) { // TestMoveLeaderToLearnerError ensures that leader transfer to learner member will fail. func TestMoveLeaderToLearnerError(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -149,7 +148,7 @@ func TestMoveLeaderToLearnerError(t *testing.T) { // TestTransferLeadershipWithLearner ensures TransferLeadership does not timeout due to learner is // automatically picked by leader as transferee. func TestTransferLeadershipWithLearner(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/v3_lease_test.go b/tests/integration/v3_lease_test.go index 66f2a21e4..717400e49 100644 --- a/tests/integration/v3_lease_test.go +++ b/tests/integration/v3_lease_test.go @@ -33,7 +33,7 @@ import ( // TestV3LeasePrmote ensures the newly elected leader can promote itself // to the primary lessor, refresh the leases and start to manage leases. // TODO: use customized clock to make this test go faster? -func TestV3LeasePrmote(t *testing.T) { +func TestV3LeasePromote(t *testing.T) { clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -94,7 +94,7 @@ func TestV3LeasePrmote(t *testing.T) { // TestV3LeaseRevoke ensures a key is deleted once its lease is revoked. func TestV3LeaseRevoke(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testLeaseRemoveLeasedKey(t, func(clus *ClusterV3, leaseID int64) error { lc := toGRPC(clus.RandClient()).Lease _, err := lc.LeaseRevoke(context.TODO(), &pb.LeaseRevokeRequest{ID: leaseID}) @@ -104,7 +104,7 @@ func TestV3LeaseRevoke(t *testing.T) { // TestV3LeaseGrantById ensures leases may be created by a given id. func TestV3LeaseGrantByID(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -141,7 +141,7 @@ func TestV3LeaseGrantByID(t *testing.T) { // TestV3LeaseExpire ensures a key is deleted once a key expires. func TestV3LeaseExpire(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testLeaseRemoveLeasedKey(t, func(clus *ClusterV3, leaseID int64) error { // let lease lapse; wait for deleted key @@ -193,7 +193,7 @@ func TestV3LeaseExpire(t *testing.T) { // TestV3LeaseKeepAlive ensures keepalive keeps the lease alive. func TestV3LeaseKeepAlive(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testLeaseRemoveLeasedKey(t, func(clus *ClusterV3, leaseID int64) error { lc := toGRPC(clus.RandClient()).Lease lreq := &pb.LeaseKeepAliveRequest{ID: leaseID} @@ -229,7 +229,7 @@ func TestV3LeaseKeepAlive(t *testing.T) { func TestV3LeaseCheckpoint(t *testing.T) { var ttl int64 = 300 leaseInterval := 2 * time.Second - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{ Size: 3, EnableLeaseCheckpoint: true, @@ -280,7 +280,7 @@ func TestV3LeaseCheckpoint(t *testing.T) { // TestV3LeaseExists creates a lease on a random client and confirms it exists in the cluster. func TestV3LeaseExists(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -304,7 +304,7 @@ func TestV3LeaseExists(t *testing.T) { // TestV3LeaseLeases creates leases and confirms list RPC fetches created ones. func TestV3LeaseLeases(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -354,7 +354,7 @@ func TestV3LeaseTimeToLiveStress(t *testing.T) { } func testLeaseStress(t *testing.T, stresser func(context.Context, pb.LeaseClient) error) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -425,7 +425,7 @@ func stressLeaseTimeToLive(tctx context.Context, lc pb.LeaseClient) (reterr erro } func TestV3PutOnNonExistLease(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -443,7 +443,7 @@ func TestV3PutOnNonExistLease(t *testing.T) { // TestV3GetNonExistLease ensures client retrieving nonexistent lease on a follower doesn't result node panic // related issue https://github.com/etcd-io/etcd/issues/6537 func TestV3GetNonExistLease(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -481,7 +481,7 @@ func TestV3GetNonExistLease(t *testing.T) { // TestV3LeaseSwitch tests a key can be switched from one lease to another. func TestV3LeaseSwitch(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -543,7 +543,7 @@ func TestV3LeaseSwitch(t *testing.T) { // election timeout after it loses its quorum. And the new leader extends the TTL of // the lease to at least TTL + election timeout. func TestV3LeaseFailover(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -604,7 +604,7 @@ func TestV3LeaseFailover(t *testing.T) { // TestV3LeaseRequireLeader ensures that a Recv will get a leader // loss error if there is no leader. func TestV3LeaseRequireLeader(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) diff --git a/tests/integration/v3_tls_test.go b/tests/integration/v3_tls_test.go index 7a050964a..9cf21b01b 100644 --- a/tests/integration/v3_tls_test.go +++ b/tests/integration/v3_tls_test.go @@ -21,8 +21,6 @@ import ( "time" "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/pkg/v3/testutil" - "google.golang.org/grpc" ) @@ -32,7 +30,7 @@ func TestTLSClientCipherSuitesMismatch(t *testing.T) { testTLSCipherSuites(t, fa // testTLSCipherSuites ensures mismatching client-side cipher suite // fail TLS handshake with the server. func testTLSCipherSuites(t *testing.T, valid bool) { - defer testutil.AfterTest(t) + BeforeTest(t) cipherSuites := []uint16{ tls.TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, diff --git a/tests/integration/v3_watch_test.go b/tests/integration/v3_watch_test.go index ba09e3a6e..3688e5c90 100644 --- a/tests/integration/v3_watch_test.go +++ b/tests/integration/v3_watch_test.go @@ -26,13 +26,12 @@ import ( pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/mvccpb" - "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc" ) // TestV3WatchFromCurrentRevision tests Watch APIs from current revision. func TestV3WatchFromCurrentRevision(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) tests := []struct { putKeys []string watchRequest *pb.WatchRequest @@ -298,7 +297,7 @@ func TestV3WatchFromCurrentRevision(t *testing.T) { // TestV3WatchFutureRevision tests Watch APIs from a future revision. func TestV3WatchFutureRevision(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -359,7 +358,7 @@ func TestV3WatchFutureRevision(t *testing.T) { // TestV3WatchWrongRange tests wrong range does not create watchers. func TestV3WatchWrongRange(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -404,13 +403,13 @@ func TestV3WatchWrongRange(t *testing.T) { // TestV3WatchCancelSynced tests Watch APIs cancellation from synced map. func TestV3WatchCancelSynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchCancel(t, 0) } // TestV3WatchCancelUnsynced tests Watch APIs cancellation from unsynced map. func TestV3WatchCancelUnsynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchCancel(t, 1) } @@ -470,7 +469,7 @@ func testV3WatchCancel(t *testing.T, startRev int64) { // TestV3WatchCurrentPutOverlap ensures current watchers receive all events with // overlapping puts. func TestV3WatchCurrentPutOverlap(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -554,7 +553,7 @@ func TestV3WatchCurrentPutOverlap(t *testing.T) { // TestV3WatchEmptyKey ensures synced watchers see empty key PUTs as PUT events func TestV3WatchEmptyKey(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -600,12 +599,12 @@ func TestV3WatchEmptyKey(t *testing.T) { } func TestV3WatchMultipleWatchersSynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchMultipleWatchers(t, 0) } func TestV3WatchMultipleWatchersUnsynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchMultipleWatchers(t, 1) } @@ -705,12 +704,12 @@ func testV3WatchMultipleWatchers(t *testing.T, startRev int64) { } func TestV3WatchMultipleEventsTxnSynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchMultipleEventsTxn(t, 0) } func TestV3WatchMultipleEventsTxnUnsynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchMultipleEventsTxn(t, 1) } @@ -798,7 +797,7 @@ func (evs eventsSortByKey) Less(i, j int) bool { } func TestV3WatchMultipleEventsPutUnsynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -877,12 +876,12 @@ func TestV3WatchMultipleEventsPutUnsynced(t *testing.T) { } func TestV3WatchMultipleStreamsSynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchMultipleStreams(t, 0) } func TestV3WatchMultipleStreamsUnsynced(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) testV3WatchMultipleStreams(t, 1) } @@ -990,7 +989,7 @@ func TestWatchWithProgressNotify(t *testing.T) { testInterval := 3 * time.Second defer func() { v3rpc.SetProgressReportInterval(oldpi) }() - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -1040,7 +1039,7 @@ func TestWatchWithProgressNotify(t *testing.T) { // TestV3WatcMultiOpenhClose opens many watchers concurrently on multiple streams. func TestV3WatchClose(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -1142,7 +1141,7 @@ func TestV3WatchWithFilter(t *testing.T) { } func TestV3WatchWithPrevKV(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/v3election_grpc_test.go b/tests/integration/v3election_grpc_test.go index 4a529626c..d5f67dd9f 100644 --- a/tests/integration/v3election_grpc_test.go +++ b/tests/integration/v3election_grpc_test.go @@ -21,14 +21,13 @@ import ( "time" pb "go.etcd.io/etcd/api/v3/etcdserverpb" - "go.etcd.io/etcd/pkg/v3/testutil" epb "go.etcd.io/etcd/server/v3/etcdserver/api/v3election/v3electionpb" ) // TestV3ElectionCampaign checks that Campaign will not give // simultaneous leadership to multiple campaigners. func TestV3ElectionCampaign(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -90,7 +89,7 @@ func TestV3ElectionCampaign(t *testing.T) { // TestV3ElectionObserve checks that an Observe stream receives // proclamations from different leaders uninterrupted. func TestV3ElectionObserve(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/v3lock_grpc_test.go b/tests/integration/v3lock_grpc_test.go index cbff0465f..38a36369f 100644 --- a/tests/integration/v3lock_grpc_test.go +++ b/tests/integration/v3lock_grpc_test.go @@ -20,14 +20,13 @@ import ( "time" pb "go.etcd.io/etcd/api/v3/etcdserverpb" - "go.etcd.io/etcd/pkg/v3/testutil" lockpb "go.etcd.io/etcd/server/v3/etcdserver/api/v3lock/v3lockpb" ) // TestV3LockLockWaiter tests that a client will wait for a lock, then acquire it // once it is unlocked. func TestV3LockLockWaiter(t *testing.T) { - defer testutil.AfterTest(t) + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) From 41f6cc7234456e2090d471b2d80a1aa8293a2e1c Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 7 Mar 2021 01:25:37 +0100 Subject: [PATCH 4/9] Tests: Better isolation between store_v2v3 integration tests. --- tests/integration/v2store/main_test.go | 34 +++++++++----------- tests/integration/v2store/store_v2v3_test.go | 24 ++++++++++++-- 2 files changed, 36 insertions(+), 22 deletions(-) diff --git a/tests/integration/v2store/main_test.go b/tests/integration/v2store/main_test.go index a11f6c32b..eb1fc665a 100644 --- a/tests/integration/v2store/main_test.go +++ b/tests/integration/v2store/main_test.go @@ -15,30 +15,26 @@ package v2store_test import ( - "fmt" - "os" "testing" - "time" "go.etcd.io/etcd/pkg/v3/testutil" - "go.etcd.io/etcd/tests/v3/integration" ) -var endpoints []string +//var endpoints []string -// TestMain sets up an etcd cluster for running the examples. func TestMain(m *testing.M) { - cfg := integration.ClusterConfig{Size: 1} - clus := integration.NewClusterV3(nil, &cfg) - endpoints = []string{clus.Client(0).Endpoints()[0]} - v := m.Run() - clus.Terminate(nil) - if err := testutil.CheckAfterTest(time.Second); err != nil { - fmt.Fprintf(os.Stderr, "%v", err) - os.Exit(1) - } - if v == 0 && testutil.CheckLeakedGoroutine() { - os.Exit(1) - } - os.Exit(v) + //cfg := integration.ClusterConfig{Size: 1} + //clus := integration.NewClusterV3(nil, &cfg) + //endpoints = []string{clus.Client(0).Endpoints()[0]} + // v := m.Run() + //clus.Terminate(nil) + //if err := testutil.CheckAfterTest(time.Second); err != nil { + // fmt.Fprintf(os.Stderr, "%v", err) + // os.Exit(1) + //} + testutil.MustTestMainWithLeakDetection(m) + //if v == 0 && testutil.CheckLeakedGoroutine() { + // os.Exit(1) + //} + //os.Exit(v) } diff --git a/tests/integration/v2store/store_v2v3_test.go b/tests/integration/v2store/store_v2v3_test.go index 7249102b8..d734f284d 100644 --- a/tests/integration/v2store/store_v2v3_test.go +++ b/tests/integration/v2store/store_v2v3_test.go @@ -19,13 +19,27 @@ import ( "testing" "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/etcdserver/api/v2store" "go.etcd.io/etcd/server/v3/etcdserver/api/v2v3" + "go.etcd.io/etcd/tests/v3/integration" ) // TODO: fix tests -func TestCreateKV(t *testing.T) { +func runWithCluster(t testing.TB, runner func(testing.TB, []string)) { + testutil.BeforeTest(t) + cfg := integration.ClusterConfig{Size: 1} + clus := integration.NewClusterV3(nil, &cfg) + defer clus.Terminate(t) + endpoints := []string{clus.Client(0).Endpoints()[0]} + runner(t, endpoints) + +} + +func TestCreateKV(t *testing.T) { runWithCluster(t, testCreateKV) } + +func testCreateKV(t testing.TB, endpoints []string) { testCases := []struct { key string value string @@ -77,7 +91,9 @@ func TestCreateKV(t *testing.T) { } } -func TestSetKV(t *testing.T) { +func TestSetKV(t *testing.T) { runWithCluster(t, testSetKV) } + +func testSetKV(t testing.TB, endpoints []string) { testCases := []struct { key string value string @@ -113,7 +129,9 @@ func TestSetKV(t *testing.T) { } } -func TestCreateSetDir(t *testing.T) { +func TestCreateSetDir(t *testing.T) { runWithCluster(t, testCreateSetDir) } + +func testCreateSetDir(t testing.TB, endpoints []string) { testCases := []struct { dir string }{ From 9ba1287334a57076a5370685f193bfbef15e3a61 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 7 Mar 2021 11:59:46 +0100 Subject: [PATCH 5/9] travis script. Turning off verbose for grpcproxy. + Upgrade to '*.sh' variant of the scripts. --- .travis.yml | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/.travis.yml b/.travis.yml index 691422d2b..58508a956 100644 --- a/.travis.yml +++ b/.travis.yml @@ -72,25 +72,25 @@ script: linux-amd64-fmt) docker run --rm \ --volume=`pwd`:/go/src/go.etcd.io/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ - /bin/bash -c "GOARCH=amd64 PASSES='fmt bom dep' ./test" + /bin/bash -c "GOARCH=amd64 PASSES='fmt bom dep' ./test.sh" ;; linux-amd64-integration-1-cpu) # TODO: Reenable 'race' when https://github.com/etcd-io/etcd/issues/12336 fixed. docker run --rm \ --volume=`pwd`:/go/src/go.etcd.io/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ - /bin/bash -c "GOARCH=amd64 CPU=1 PASSES='integration' RACE='false' ./test" + /bin/bash -c "GOARCH=amd64 CPU=1 PASSES='integration' RACE='false' ./test.sh" ;; linux-amd64-integration-2-cpu) # TODO: Reenable 'race' when https://github.com/etcd-io/etcd/issues/12336 fixed. docker run --rm \ --volume=`pwd`:/go/src/go.etcd.io/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ - /bin/bash -c "GOARCH=amd64 CPU=2 PASSES='integration' RACE='false' ./test" + /bin/bash -c "GOARCH=amd64 CPU=2 PASSES='integration' RACE='false' ./test.sh" ;; linux-amd64-integration-4-cpu) # TODO: Reenable 'race' when https://github.com/etcd-io/etcd/issues/12336 fixed. docker run --rm \ --volume=`pwd`:/go/src/go.etcd.io/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ - /bin/bash -c "GOARCH=amd64 CPU=4 PASSES='integration' RACE='false' ./test" + /bin/bash -c "GOARCH=amd64 CPU=4 PASSES='integration' RACE='false' ./test.sh" ;; linux-amd64-functional) docker run --rm \ @@ -100,29 +100,29 @@ script: linux-amd64-unit-4-cpu-race) docker run --rm \ --volume=`pwd`:/go/src/go.etcd.io/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ - /bin/bash -c "GOARCH=amd64 PASSES='unit' RACE='true' CPU='4' ./test -p=2" + /bin/bash -c "GOARCH=amd64 PASSES='unit' RACE='true' CPU='4' ./test.sh -p=2" ;; all-build) docker run --rm \ --volume=`pwd`:/go/src/go.etcd.io/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ - /bin/bash -c "GOARCH=amd64 PASSES='build' ./test \ - && GOARCH=386 PASSES='build' ./test \ - && GO_BUILD_FLAGS='-v -mod=readonly' GOOS=darwin GOARCH=amd64 ./build \ - && GO_BUILD_FLAGS='-v -mod=readonly' GOOS=windows GOARCH=amd64 ./build \ - && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=arm ./build \ - && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=arm64 ./build \ - && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=ppc64le ./build \ - && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=s390x ./build" + /bin/bash -c "GOARCH=amd64 PASSES='build' ./test.sh \ + && GOARCH=386 PASSES='build' ./test.sh \ + && GO_BUILD_FLAGS='-v -mod=readonly' GOOS=darwin GOARCH=amd64 ./build.sh \ + && GO_BUILD_FLAGS='-v -mod=readonly' GOOS=windows GOARCH=amd64 ./build.sh \ + && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=arm ./build.sh \ + && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=arm64 ./build.sh \ + && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=ppc64le ./build.sh \ + && GO_BUILD_FLAGS='-v -mod=readonly' GOARCH=s390x ./build.sh" ;; linux-amd64-grpcproxy) # TODO: Reenable race when https://github.com/etcd-io/etcd/issues/12336 fixed. - sudo HOST_TMP_DIR=/tmp TEST_OPTS="PASSES='build grpcproxy' VERBOSE='1' CPU='4' COVER='false' RACE='false'" make docker-test + sudo HOST_TMP_DIR=/tmp TEST_OPTS="PASSES='build grpcproxy' CPU='4' COVER='false' RACE='false'" make docker-test ;; linux-amd64-coverage) sudo HOST_TMP_DIR=/tmp TEST_OPTS="VERBOSE='1'" make docker-test-coverage ;; linux-amd64-fmt-unit-go-tip-2-cpu) - GOARCH=amd64 PASSES='fmt unit' CPU='2' RACE='false' ./test -p=2 + GOARCH=amd64 PASSES='fmt unit' CPU='2' RACE='false' ./test.sh -p=2 ;; linux-386-unit-1-cpu) docker run --rm \ From bfe02c05264175e0f9f097c348c4f453af799c24 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 7 Mar 2021 13:37:53 +0100 Subject: [PATCH 6/9] tests: Cluster creation that failed shouldn't leak goroutines. --- tests/integration/cluster.go | 32 +++++++++++++------------------- 1 file changed, 13 insertions(+), 19 deletions(-) diff --git a/tests/integration/cluster.go b/tests/integration/cluster.go index 748f346bc..a4713da80 100644 --- a/tests/integration/cluster.go +++ b/tests/integration/cluster.go @@ -120,17 +120,8 @@ var ( } defaultTokenJWT = "jwt,pub-key=../fixtures/server.crt,priv-key=../fixtures/server.key.insecure,sign-method=RS256,ttl=1s" - - // Replace with just usage of testing.TB instead of zap - lg = zap.NewNop() ) -func init() { - if os.Getenv("CLUSTER_DEBUG") != "" { - lg, _ = zap.NewProduction() - } -} - type ClusterConfig struct { Size int PeerTLS *transport.TLSInfo @@ -249,6 +240,7 @@ func (c *cluster) Launch(t testing.TB) { } for range c.Members { if err := <-errc; err != nil { + c.Terminate(t) t.Fatalf("error setting up member: %v", err) } } @@ -648,7 +640,7 @@ func mustNewMember(t testing.TB, mcfg memberConfig) *member { m.Name = mcfg.name - m.DataDir, err = ioutil.TempDir(os.TempDir(), "etcd") + m.DataDir, err = ioutil.TempDir(t.TempDir(), "etcd") if err != nil { t.Fatal(err) } @@ -836,7 +828,7 @@ func (m *member) Clone(t testing.TB) *member { // Launch starts a member based on ServerConfig, PeerListeners // and ClientListeners. func (m *member) Launch() error { - lg.Info( + m.Logger.Info( "launching a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), @@ -993,7 +985,7 @@ func (m *member) Launch() error { m.serverClosers = append(m.serverClosers, closer) } - lg.Info( + m.Logger.Info( "launched a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), @@ -1086,7 +1078,9 @@ func (m *member) Close() { m.grpcServerPeer.Stop() m.grpcServerPeer = nil } - m.s.HardStop() + if m.s != nil { + m.s.HardStop() + } for _, f := range m.serverClosers { f() } @@ -1094,7 +1088,7 @@ func (m *member) Close() { // Stop stops the member, but the data dir of the member is preserved. func (m *member) Stop(t testing.TB) { - lg.Info( + m.Logger.Info( "stopping a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), @@ -1103,7 +1097,7 @@ func (m *member) Stop(t testing.TB) { ) m.Close() m.serverClosers = nil - lg.Info( + m.Logger.Info( "stopped a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), @@ -1128,7 +1122,7 @@ func (m *member) StopNotify() <-chan struct{} { // Restart starts the member using the preserved data dir. func (m *member) Restart(t testing.TB) error { - lg.Info( + m.Logger.Info( "restarting a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), @@ -1153,7 +1147,7 @@ func (m *member) Restart(t testing.TB) error { } err := m.Launch() - lg.Info( + m.Logger.Info( "restarted a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), @@ -1166,7 +1160,7 @@ func (m *member) Restart(t testing.TB) error { // Terminate stops the member and removes the data dir. func (m *member) Terminate(t testing.TB) { - lg.Info( + m.Logger.Info( "terminating a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), @@ -1179,7 +1173,7 @@ func (m *member) Terminate(t testing.TB) { t.Fatal(err) } } - lg.Info( + m.Logger.Info( "terminated a member", zap.String("name", m.Name), zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()), From 5ddabfdb2498af0c98bb5cdfbae696e93a77924d Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 7 Mar 2021 14:16:37 +0100 Subject: [PATCH 7/9] tests: Make tests operate in /tmp director instead of src. Thanks to this, unix sockets should be not longer created by integration tests in the the source code directory, so potentially trigger IDE reloads and unnecessery load (and mess). --- pkg/testutil/leak.go | 4 ++ .../clientv3/connectivity/dial_test.go | 12 +++--- .../clientv3/snapshot/v3_snapshot_test.go | 6 +-- tests/integration/cluster.go | 37 ++++++++++--------- tests/integration/embed/embed_test.go | 15 +++----- tests/integration/member_test.go | 3 +- tests/integration/snapshot/member_test.go | 2 +- .../integration/snapshot/v3_snapshot_test.go | 19 +++++----- tests/integration/testing.go | 20 ++++++++++ tests/integration/v2store/store_v2v3_test.go | 2 +- tests/integration/v3_election_test.go | 6 +++ tests/integration/v3_grpc_test.go | 10 ++--- tests/integration/v3_kv_test.go | 31 ++-------------- 13 files changed, 84 insertions(+), 83 deletions(-) diff --git a/pkg/testutil/leak.go b/pkg/testutil/leak.go index 828717072..524e8b25f 100644 --- a/pkg/testutil/leak.go +++ b/pkg/testutil/leak.go @@ -97,6 +97,10 @@ func CheckAfterTest(d time.Duration) error { // BeforeTest is a convenient way to register before-and-after code to a test. // If you execute BeforeTest, you don't need to explicitly register AfterTest. func BeforeTest(t testing.TB) { + if err := CheckAfterTest(10 * time.Millisecond); err != nil { + t.Skipf("Found leaked goroutined BEFORE test", err) + return + } t.Cleanup(func() { AfterTest(t) }) diff --git a/tests/integration/clientv3/connectivity/dial_test.go b/tests/integration/clientv3/connectivity/dial_test.go index 483b97709..6a645aba0 100644 --- a/tests/integration/clientv3/connectivity/dial_test.go +++ b/tests/integration/clientv3/connectivity/dial_test.go @@ -31,16 +31,16 @@ import ( var ( testTLSInfo = transport.TLSInfo{ - KeyFile: "../../../fixtures/server.key.insecure", - CertFile: "../../../fixtures/server.crt", - TrustedCAFile: "../../../fixtures/ca.crt", + KeyFile: integration.MustAbsPath("../../../fixtures/server.key.insecure"), + CertFile: integration.MustAbsPath("../../../fixtures/server.crt"), + TrustedCAFile: integration.MustAbsPath("../../../fixtures/ca.crt"), ClientCertAuth: true, } testTLSInfoExpired = transport.TLSInfo{ - KeyFile: "../../fixtures-expired/server.key.insecure", - CertFile: "../../fixtures-expired/server.crt", - TrustedCAFile: "../../fixtures-expired/ca.crt", + KeyFile: integration.MustAbsPath("../../fixtures-expired/server.key.insecure"), + CertFile: integration.MustAbsPath("../../fixtures-expired/server.crt"), + TrustedCAFile: integration.MustAbsPath("../../fixtures-expired/ca.crt"), ClientCertAuth: true, } ) diff --git a/tests/integration/clientv3/snapshot/v3_snapshot_test.go b/tests/integration/clientv3/snapshot/v3_snapshot_test.go index 1d141cab3..87da75683 100644 --- a/tests/integration/clientv3/snapshot/v3_snapshot_test.go +++ b/tests/integration/clientv3/snapshot/v3_snapshot_test.go @@ -72,13 +72,12 @@ func createSnapshotFile(t *testing.T, kvs []kv) string { cfg.LCUrls, cfg.ACUrls = cURLs, cURLs cfg.LPUrls, cfg.APUrls = pURLs, pURLs cfg.InitialCluster = fmt.Sprintf("%s=%s", cfg.Name, pURLs[0].String()) - cfg.Dir = filepath.Join(os.TempDir(), fmt.Sprint(time.Now().Nanosecond())) + cfg.Dir = filepath.Join(t.TempDir(), fmt.Sprint(time.Now().Nanosecond())) srv, err := embed.StartEtcd(cfg) if err != nil { t.Fatal(err) } defer func() { - os.RemoveAll(cfg.Dir) srv.Close() }() select { @@ -102,12 +101,11 @@ func createSnapshotFile(t *testing.T, kvs []kv) string { } } - dpPath := filepath.Join(os.TempDir(), fmt.Sprintf("snapshot%d.db", time.Now().Nanosecond())) + dpPath := filepath.Join(t.TempDir(), fmt.Sprintf("snapshot%d.db", time.Now().Nanosecond())) if err = snapshot.Save(context.Background(), zap.NewExample(), ccfg, dpPath); err != nil { t.Fatal(err) } - os.RemoveAll(cfg.Dir) srv.Close() return dpPath } diff --git a/tests/integration/cluster.go b/tests/integration/cluster.go index a4713da80..1222ee665 100644 --- a/tests/integration/cluster.go +++ b/tests/integration/cluster.go @@ -83,43 +83,44 @@ var ( localListenCount = int64(0) testTLSInfo = transport.TLSInfo{ - KeyFile: "../fixtures/server.key.insecure", - CertFile: "../fixtures/server.crt", - TrustedCAFile: "../fixtures/ca.crt", + KeyFile: MustAbsPath("../fixtures/server.key.insecure"), + CertFile: MustAbsPath("../fixtures/server.crt"), + TrustedCAFile: MustAbsPath("../fixtures/ca.crt"), ClientCertAuth: true, } testTLSInfoWithSpecificUsage = transport.TLSInfo{ - KeyFile: "../fixtures/server-serverusage.key.insecure", - CertFile: "../fixtures/server-serverusage.crt", - ClientKeyFile: "../fixtures/client-clientusage.key.insecure", - ClientCertFile: "../fixtures/client-clientusage.crt", - TrustedCAFile: "../fixtures/ca.crt", + KeyFile: MustAbsPath("../fixtures/server-serverusage.key.insecure"), + CertFile: MustAbsPath("../fixtures/server-serverusage.crt"), + ClientKeyFile: MustAbsPath("../fixtures/client-clientusage.key.insecure"), + ClientCertFile: MustAbsPath("../fixtures/client-clientusage.crt"), + TrustedCAFile: MustAbsPath("../fixtures/ca.crt"), ClientCertAuth: true, } testTLSInfoIP = transport.TLSInfo{ - KeyFile: "../fixtures/server-ip.key.insecure", - CertFile: "../fixtures/server-ip.crt", - TrustedCAFile: "../fixtures/ca.crt", + KeyFile: MustAbsPath("../fixtures/server-ip.key.insecure"), + CertFile: MustAbsPath("../fixtures/server-ip.crt"), + TrustedCAFile: MustAbsPath("../fixtures/ca.crt"), ClientCertAuth: true, } testTLSInfoExpired = transport.TLSInfo{ - KeyFile: "./fixtures-expired/server.key.insecure", - CertFile: "./fixtures-expired/server.crt", - TrustedCAFile: "./fixtures-expired/ca.crt", + KeyFile: MustAbsPath("./fixtures-expired/server.key.insecure"), + CertFile: MustAbsPath("./fixtures-expired/server.crt"), + TrustedCAFile: MustAbsPath("./fixtures-expired/ca.crt"), ClientCertAuth: true, } testTLSInfoExpiredIP = transport.TLSInfo{ - KeyFile: "./fixtures-expired/server-ip.key.insecure", - CertFile: "./fixtures-expired/server-ip.crt", - TrustedCAFile: "./fixtures-expired/ca.crt", + KeyFile: MustAbsPath("./fixtures-expired/server-ip.key.insecure"), + CertFile: MustAbsPath("./fixtures-expired/server-ip.crt"), + TrustedCAFile: MustAbsPath("./fixtures-expired/ca.crt"), ClientCertAuth: true, } - defaultTokenJWT = "jwt,pub-key=../fixtures/server.crt,priv-key=../fixtures/server.key.insecure,sign-method=RS256,ttl=1s" + defaultTokenJWT = fmt.Sprintf("jwt,pub-key=%s,priv-key=%s,sign-method=RS256,ttl=1s", + MustAbsPath("../fixtures/server.crt"), MustAbsPath("../fixtures/server.key.insecure")) ) type ClusterConfig struct { diff --git a/tests/integration/embed/embed_test.go b/tests/integration/embed/embed_test.go index 4459a1e02..ac8c9291b 100644 --- a/tests/integration/embed/embed_test.go +++ b/tests/integration/embed/embed_test.go @@ -34,13 +34,14 @@ import ( "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" "go.etcd.io/etcd/server/v3/embed" + "go.etcd.io/etcd/tests/v3/integration" ) var ( testTLSInfo = transport.TLSInfo{ - KeyFile: "../../fixtures/server.key.insecure", - CertFile: "../../fixtures/server.crt", - TrustedCAFile: "../../fixtures/ca.crt", + KeyFile: integration.MustAbsPath("../../fixtures/server.key.insecure"), + CertFile: integration.MustAbsPath("../../fixtures/server.crt"), + TrustedCAFile: integration.MustAbsPath("../../fixtures/ca.crt"), ClientCertAuth: true, } ) @@ -88,9 +89,7 @@ func TestEmbedEtcd(t *testing.T) { tests[7].cfg.LCUrls = []url.URL{*dnsURL} tests[8].cfg.LPUrls = []url.URL{*dnsURL} - dir := filepath.Join(os.TempDir(), fmt.Sprintf("embed-etcd")) - os.RemoveAll(dir) - defer os.RemoveAll(dir) + dir := filepath.Join(t.TempDir(), fmt.Sprintf("embed-etcd")) for i, tt := range tests { tests[i].cfg.Dir = dir @@ -143,9 +142,7 @@ func testEmbedEtcdGracefulStop(t *testing.T, secure bool) { urls := newEmbedURLs(secure, 2) setupEmbedCfg(cfg, []url.URL{urls[0]}, []url.URL{urls[1]}) - cfg.Dir = filepath.Join(os.TempDir(), fmt.Sprintf("embed-etcd")) - os.RemoveAll(cfg.Dir) - defer os.RemoveAll(cfg.Dir) + cfg.Dir = filepath.Join(t.TempDir(), fmt.Sprintf("embed-etcd")) e, err := embed.StartEtcd(cfg) if err != nil { diff --git a/tests/integration/member_test.go b/tests/integration/member_test.go index 54d3dbedc..62520bbe6 100644 --- a/tests/integration/member_test.go +++ b/tests/integration/member_test.go @@ -18,7 +18,6 @@ import ( "context" "fmt" "io/ioutil" - "os" "reflect" "testing" @@ -70,7 +69,7 @@ func TestLaunchDuplicateMemberShouldFail(t *testing.T) { c := NewCluster(t, size) m := c.Members[0].Clone(t) var err error - m.DataDir, err = ioutil.TempDir(os.TempDir(), "etcd") + m.DataDir, err = ioutil.TempDir(t.TempDir(), "etcd") if err != nil { t.Fatal(err) } diff --git a/tests/integration/snapshot/member_test.go b/tests/integration/snapshot/member_test.go index 1a6cecfa6..22997272d 100644 --- a/tests/integration/snapshot/member_test.go +++ b/tests/integration/snapshot/member_test.go @@ -76,7 +76,7 @@ func TestSnapshotV3RestoreMultiMemberAdd(t *testing.T) { } cfg.InitialCluster = cfg.InitialCluster[1:] cfg.InitialCluster += fmt.Sprintf(",%s=%s", cfg.Name, newPURLs[0].String()) - cfg.Dir = filepath.Join(os.TempDir(), fmt.Sprint(time.Now().Nanosecond())) + cfg.Dir = filepath.Join(t.TempDir(), fmt.Sprint(time.Now().Nanosecond())) srv, err := embed.StartEtcd(cfg) if err != nil { diff --git a/tests/integration/snapshot/v3_snapshot_test.go b/tests/integration/snapshot/v3_snapshot_test.go index 4c10f4292..0b7fcc52c 100644 --- a/tests/integration/snapshot/v3_snapshot_test.go +++ b/tests/integration/snapshot/v3_snapshot_test.go @@ -29,8 +29,7 @@ import ( "go.etcd.io/etcd/etcdctl/v3/snapshot" "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/server/v3/embed" - - "go.uber.org/zap" + "go.uber.org/zap/zaptest" ) // TestSnapshotV3RestoreSingle tests single node cluster restoring @@ -53,9 +52,9 @@ func TestSnapshotV3RestoreSingle(t *testing.T) { cfg.LCUrls, cfg.ACUrls = cURLs, cURLs cfg.LPUrls, cfg.APUrls = pURLs, pURLs cfg.InitialCluster = fmt.Sprintf("%s=%s", cfg.Name, pURLs[0].String()) - cfg.Dir = filepath.Join(os.TempDir(), fmt.Sprint(time.Now().Nanosecond())) + cfg.Dir = filepath.Join(t.TempDir(), fmt.Sprint(time.Now().Nanosecond())) - sp := snapshot.NewV3(zap.NewExample()) + sp := snapshot.NewV3(zaptest.NewLogger(t)) pss := make([]string, 0, len(pURLs)) for _, p := range pURLs { pss = append(pss, p.String()) @@ -149,7 +148,7 @@ func TestCorruptedBackupFileCheck(t *testing.T) { t.Fatalf("test file [%s] does not exist: %v", dbPath, err) } - sp := snapshot.NewV3(zap.NewExample()) + sp := snapshot.NewV3(zaptest.NewLogger(t)) _, err := sp.Status(dbPath) expectedErrKeywords := "snapshot file integrity check failed" /* example error message: @@ -187,7 +186,7 @@ func createSnapshotFile(t *testing.T, kvs []kv) string { cfg.LCUrls, cfg.ACUrls = cURLs, cURLs cfg.LPUrls, cfg.APUrls = pURLs, pURLs cfg.InitialCluster = fmt.Sprintf("%s=%s", cfg.Name, pURLs[0].String()) - cfg.Dir = filepath.Join(os.TempDir(), fmt.Sprint(time.Now().Nanosecond())) + cfg.Dir = filepath.Join(t.TempDir(), fmt.Sprint(time.Now().Nanosecond())) srv, err := embed.StartEtcd(cfg) if err != nil { t.Fatal(err) @@ -217,8 +216,8 @@ func createSnapshotFile(t *testing.T, kvs []kv) string { } } - sp := snapshot.NewV3(zap.NewExample()) - dpPath := filepath.Join(os.TempDir(), fmt.Sprintf("snapshot%d.db", time.Now().Nanosecond())) + sp := snapshot.NewV3(zaptest.NewLogger(t)) + dpPath := filepath.Join(t.TempDir(), fmt.Sprintf("snapshot%d.db", time.Now().Nanosecond())) if err = sp.Save(context.Background(), ccfg, dpPath); err != nil { t.Fatal(err) } @@ -254,9 +253,9 @@ func restoreCluster(t *testing.T, clusterN int, dbPath string) ( cfg.LCUrls, cfg.ACUrls = []url.URL{cURLs[i]}, []url.URL{cURLs[i]} cfg.LPUrls, cfg.APUrls = []url.URL{pURLs[i]}, []url.URL{pURLs[i]} cfg.InitialCluster = ics - cfg.Dir = filepath.Join(os.TempDir(), fmt.Sprint(time.Now().Nanosecond()+i)) + cfg.Dir = filepath.Join(t.TempDir(), fmt.Sprint(time.Now().Nanosecond()+i)) - sp := snapshot.NewV3(zap.NewExample()) + sp := snapshot.NewV3(zaptest.NewLogger(t)) if err := sp.Restore(snapshot.RestoreConfig{ SnapshotPath: dbPath, Name: cfg.Name, diff --git a/tests/integration/testing.go b/tests/integration/testing.go index a525bb9a4..e52c882ad 100644 --- a/tests/integration/testing.go +++ b/tests/integration/testing.go @@ -15,6 +15,8 @@ package integration import ( + "os" + "path/filepath" "testing" "go.etcd.io/etcd/pkg/v3/testutil" @@ -22,4 +24,22 @@ import ( func BeforeTest(t testing.TB) { testutil.BeforeTest(t) + + previousWD, err := os.Getwd() + if err != nil { + t.Fatal(err) + } + os.Chdir(t.TempDir()) + t.Cleanup(func() { + os.Chdir(previousWD) + }) + } + +func MustAbsPath(path string) string { + abs, err := filepath.Abs(path) + if err != nil { + panic(err) + } + return abs +} \ No newline at end of file diff --git a/tests/integration/v2store/store_v2v3_test.go b/tests/integration/v2store/store_v2v3_test.go index d734f284d..5a186389b 100644 --- a/tests/integration/v2store/store_v2v3_test.go +++ b/tests/integration/v2store/store_v2v3_test.go @@ -30,7 +30,7 @@ import ( func runWithCluster(t testing.TB, runner func(testing.TB, []string)) { testutil.BeforeTest(t) cfg := integration.ClusterConfig{Size: 1} - clus := integration.NewClusterV3(nil, &cfg) + clus := integration.NewClusterV3(t, &cfg) defer clus.Terminate(t) endpoints := []string{clus.Client(0).Endpoints()[0]} runner(t, endpoints) diff --git a/tests/integration/v3_election_test.go b/tests/integration/v3_election_test.go index 8ef54d23e..a3aed9ba6 100644 --- a/tests/integration/v3_election_test.go +++ b/tests/integration/v3_election_test.go @@ -26,6 +26,7 @@ import ( // TestElectionWait tests if followers can correctly wait for elections. func TestElectionWait(t *testing.T) { + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -107,6 +108,7 @@ func TestElectionWait(t *testing.T) { // TestElectionFailover tests that an election will func TestElectionFailover(t *testing.T) { + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 3}) defer clus.Terminate(t) @@ -174,6 +176,7 @@ func TestElectionFailover(t *testing.T) { // TestElectionSessionRelock ensures that campaigning twice on the same election // with the same lock will Proclaim instead of deadlocking. func TestElectionSessionRecampaign(t *testing.T) { + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) cli := clus.RandClient() @@ -206,6 +209,7 @@ func TestElectionSessionRecampaign(t *testing.T) { // of bug #6278. https://github.com/etcd-io/etcd/issues/6278 // func TestElectionOnPrefixOfExistingKey(t *testing.T) { + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) @@ -232,6 +236,7 @@ func TestElectionOnPrefixOfExistingKey(t *testing.T) { // in a new session with the same lease id) does not result in loss of // leadership. func TestElectionOnSessionRestart(t *testing.T) { + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) cli := clus.RandClient() @@ -278,6 +283,7 @@ func TestElectionOnSessionRestart(t *testing.T) { // TestElectionObserveCompacted checks that observe can tolerate // a leader key with a modrev less than the compaction revision. func TestElectionObserveCompacted(t *testing.T) { + BeforeTest(t) clus := NewClusterV3(t, &ClusterConfig{Size: 1}) defer clus.Terminate(t) diff --git a/tests/integration/v3_grpc_test.go b/tests/integration/v3_grpc_test.go index 5c29b6319..8354aa374 100644 --- a/tests/integration/v3_grpc_test.go +++ b/tests/integration/v3_grpc_test.go @@ -1609,20 +1609,20 @@ func TestTLSGRPCAcceptSecureAll(t *testing.T) { // when all certs are atomically replaced by directory renaming. // And expects server to reject client requests, and vice versa. func TestTLSReloadAtomicReplace(t *testing.T) { - tmpDir, err := ioutil.TempDir(os.TempDir(), "fixtures-tmp") + tmpDir, err := ioutil.TempDir(t.TempDir(), "fixtures-tmp") if err != nil { t.Fatal(err) } os.RemoveAll(tmpDir) defer os.RemoveAll(tmpDir) - certsDir, err := ioutil.TempDir(os.TempDir(), "fixtures-to-load") + certsDir, err := ioutil.TempDir(t.TempDir(), "fixtures-to-load") if err != nil { t.Fatal(err) } defer os.RemoveAll(certsDir) - certsDirExp, err := ioutil.TempDir(os.TempDir(), "fixtures-expired") + certsDirExp, err := ioutil.TempDir(t.TempDir(), "fixtures-expired") if err != nil { t.Fatal(err) } @@ -1668,7 +1668,7 @@ func TestTLSReloadAtomicReplace(t *testing.T) { // when new certs are copied over, one by one. And expects server // to reject client requests, and vice versa. func TestTLSReloadCopy(t *testing.T) { - certsDir, err := ioutil.TempDir(os.TempDir(), "fixtures-to-load") + certsDir, err := ioutil.TempDir(t.TempDir(), "fixtures-to-load") if err != nil { t.Fatal(err) } @@ -1698,7 +1698,7 @@ func TestTLSReloadCopy(t *testing.T) { // when new certs are copied over, one by one. And expects server // to reject client requests, and vice versa. func TestTLSReloadCopyIPOnly(t *testing.T) { - certsDir, err := ioutil.TempDir(os.TempDir(), "fixtures-to-load") + certsDir, err := ioutil.TempDir(t.TempDir(), "fixtures-to-load") if err != nil { t.Fatal(err) } diff --git a/tests/integration/v3_kv_test.go b/tests/integration/v3_kv_test.go index db26c3a45..cbd4e0acd 100644 --- a/tests/integration/v3_kv_test.go +++ b/tests/integration/v3_kv_test.go @@ -2,45 +2,22 @@ package integration import ( "context" - "io/ioutil" - "os" "testing" "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/namespace" - "go.etcd.io/etcd/server/v3/embed" - "go.etcd.io/etcd/server/v3/etcdserver/api/v3client" ) // TestKVWithEmptyValue ensures that a get/delete with an empty value, and with WithFromKey/WithPrefix function will return an empty error. func TestKVWithEmptyValue(t *testing.T) { BeforeTest(t) - cfg := embed.NewConfig() + clus := NewClusterV3(t, &ClusterConfig{Size: 1}) + defer clus.Terminate(t) - // Use temporary data directory. - dir, err := ioutil.TempDir("", "etcd-") - if err != nil { - panic(err) - } - defer os.RemoveAll(dir) - cfg.Dir = dir + client := clus.RandClient() - // Suppress server log to keep output clean. - //cfg.Logger = "zap" - //cfg.LogLevel = "error" - - etcd, err := embed.StartEtcd(cfg) - if err != nil { - panic(err) - } - defer etcd.Close() - <-etcd.Server.ReadyNotify() - - client := v3client.New(etcd.Server) - defer client.Close() - - _, err = client.Put(context.Background(), "my-namespace/foobar", "data") + _, err := client.Put(context.Background(), "my-namespace/foobar", "data") if err != nil { t.Fatal(err) } From b6c2e87a7467dad3628bc4153a1b0d54c30719b1 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 7 Mar 2021 15:05:57 +0100 Subject: [PATCH 8/9] Testing: Integration tests does not check whether t==nil --- pkg/testutil/leak.go | 6 +- pkg/testutil/testingtb.go | 130 ++++++++++++++++++++++++++++++ pkg/testutil/testutil.go | 2 +- tests/integration/cluster.go | 117 +++++++++++---------------- tests/integration/lazy_cluster.go | 19 ++++- tests/integration/testing.go | 5 +- 6 files changed, 201 insertions(+), 78 deletions(-) create mode 100644 pkg/testutil/testingtb.go diff --git a/pkg/testutil/leak.go b/pkg/testutil/leak.go index 524e8b25f..9f83d6eb5 100644 --- a/pkg/testutil/leak.go +++ b/pkg/testutil/leak.go @@ -96,9 +96,9 @@ func CheckAfterTest(d time.Duration) error { // BeforeTest is a convenient way to register before-and-after code to a test. // If you execute BeforeTest, you don't need to explicitly register AfterTest. -func BeforeTest(t testing.TB) { +func BeforeTest(t TB) { if err := CheckAfterTest(10 * time.Millisecond); err != nil { - t.Skipf("Found leaked goroutined BEFORE test", err) + t.Skip("Found leaked goroutined BEFORE test", err) return } t.Cleanup(func() { @@ -109,7 +109,7 @@ func BeforeTest(t testing.TB) { // AfterTest is meant to run in a defer that executes after a test completes. // It will detect common goroutine leaks, retrying in case there are goroutines // not synchronously torn down, and fail the test if any goroutines are stuck. -func AfterTest(t testing.TB) { +func AfterTest(t TB) { if err := CheckAfterTest(1 * time.Second); err != nil { t.Errorf("Test %v", err) } diff --git a/pkg/testutil/testingtb.go b/pkg/testutil/testingtb.go new file mode 100644 index 000000000..970542c04 --- /dev/null +++ b/pkg/testutil/testingtb.go @@ -0,0 +1,130 @@ +// 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 testutil + +import ( + "io/ioutil" + "log" + "os" +) + +// TB is a subset of methods of testing.TB interface. +// We cannot implement testing.TB due to protection, so we expose this simplified interface. +type TB interface { + Cleanup(func()) + Error(args ...interface{}) + Errorf(format string, args ...interface{}) + Fail() + FailNow() + Failed() bool + Fatal(args ...interface{}) + Fatalf(format string, args ...interface{}) + Logf(format string, args ...interface{}) + Name() string + TempDir() string + Helper() + Skip(args ...interface{}) +} + +// NewTestingTBProthesis creates a fake variant of testing.TB implementation. +// It's supposed to be used in contexts were real testing.T is not provided, +// e.g. in 'examples'. +// +// The `closef` goroutine should get executed when tb will not be needed any longer. +// +// The provided implementation is NOT thread safe (Cleanup() method). +func NewTestingTBProthesis(name string) (tb TB, closef func()) { + testtb := &testingTBProthesis{name: name} + return testtb, testtb.close +} + +type testingTBProthesis struct { + name string + failed bool + cleanups []func() +} + +func (t *testingTBProthesis) Helper() { + // Ignored +} + +func (t *testingTBProthesis) Skip(args ...interface{}) { + t.Log(append([]interface{}{"Skipping due to: "}, args...)) +} + +func (t *testingTBProthesis) Cleanup(f func()) { + t.cleanups = append(t.cleanups, f) +} + +func (t *testingTBProthesis) Error(args ...interface{}) { + log.Println(args...) + t.Fail() +} + +func (t *testingTBProthesis) Errorf(format string, args ...interface{}) { + log.Printf(format, args...) + t.Fail() +} + +func (t *testingTBProthesis) Fail() { + t.failed = true +} + +func (t *testingTBProthesis) FailNow() { + t.failed = true + panic("FailNow() called") +} + +func (t *testingTBProthesis) Failed() bool { + return t.failed +} + +func (t *testingTBProthesis) Fatal(args ...interface{}) { + log.Fatalln(args...) +} + +func (t *testingTBProthesis) Fatalf(format string, args ...interface{}) { + log.Fatalf(format, args...) +} + +func (t *testingTBProthesis) Logf(format string, args ...interface{}) { + log.Printf(format, args...) +} + +func (t *testingTBProthesis) Log(args ...interface{}) { + log.Println(args...) +} + +func (t *testingTBProthesis) Name() string { + return t.name +} + +func (t *testingTBProthesis) TempDir() string { + dir, err := ioutil.TempDir("", t.name) + if err != nil { + t.Fatal(err) + } + t.cleanups = append([]func(){func() { + t.Logf("Cleaning UP: %v", dir) + os.RemoveAll(dir) + }}, t.cleanups...) + return dir +} + +func (t *testingTBProthesis) close() { + for i := len(t.cleanups) - 1; i >= 0; i-- { + t.cleanups[i]() + } +} diff --git a/pkg/testutil/testutil.go b/pkg/testutil/testutil.go index f12566ab2..3eb94a328 100644 --- a/pkg/testutil/testutil.go +++ b/pkg/testutil/testutil.go @@ -84,7 +84,7 @@ func Poll(interval time.Duration, timeout time.Duration, condition ConditionFunc } } -func SkipTestIfShortMode(t testing.TB, reason string) { +func SkipTestIfShortMode(t TB, reason string) { if t != nil { t.Helper() if testing.Short() { diff --git a/tests/integration/cluster.go b/tests/integration/cluster.go index 1222ee665..b57312667 100644 --- a/tests/integration/cluster.go +++ b/tests/integration/cluster.go @@ -30,7 +30,6 @@ import ( "strings" "sync" "sync/atomic" - "testing" "time" pb "go.etcd.io/etcd/api/v3/etcdserverpb" @@ -202,7 +201,7 @@ func (c *cluster) fillClusterForMembers() error { return nil } -func newCluster(t testing.TB, cfg *ClusterConfig) *cluster { +func newCluster(t testutil.TB, cfg *ClusterConfig) *cluster { testutil.SkipTestIfShortMode(t, "Cannot start etcd cluster in --short tests") c := &cluster{cfg: cfg} @@ -220,17 +219,17 @@ func newCluster(t testing.TB, cfg *ClusterConfig) *cluster { // NewCluster returns an unlaunched cluster of the given size which has been // set to use static bootstrap. -func NewCluster(t testing.TB, size int) *cluster { +func NewCluster(t testutil.TB, size int) *cluster { t.Helper() return newCluster(t, &ClusterConfig{Size: size}) } // NewClusterByConfig returns an unlaunched cluster defined by a cluster configuration -func NewClusterByConfig(t testing.TB, cfg *ClusterConfig) *cluster { +func NewClusterByConfig(t testutil.TB, cfg *ClusterConfig) *cluster { return newCluster(t, cfg) } -func (c *cluster) Launch(t testing.TB) { +func (c *cluster) Launch(t testutil.TB) { errc := make(chan error) for _, m := range c.Members { // Members are launched in separate goroutines because if they boot @@ -292,7 +291,7 @@ func (c *cluster) HTTPMembers() []client.Member { return ms } -func (c *cluster) mustNewMember(t testing.TB) *member { +func (c *cluster) mustNewMember(t testutil.TB) *member { m := mustNewMember(t, memberConfig{ name: c.generateMemberName(), @@ -324,7 +323,7 @@ func (c *cluster) mustNewMember(t testing.TB) *member { } // addMember return PeerURLs of the added member. -func (c *cluster) addMember(t testing.TB) types.URLs { +func (c *cluster) addMember(t testutil.TB) types.URLs { m := c.mustNewMember(t) scheme := schemeFromTLSInfo(c.cfg.PeerTLS) @@ -339,11 +338,7 @@ func (c *cluster) addMember(t testing.TB) types.URLs { } } if err != nil { - if t != nil { - t.Fatalf("add member failed on all members error: %v", err) - } else { - log.Fatalf("add member failed on all members error: %v", err) - } + t.Fatalf("add member failed on all members error: %v", err) } m.InitialPeerURLsMap = types.URLsMap{} @@ -361,7 +356,7 @@ func (c *cluster) addMember(t testing.TB) types.URLs { return m.PeerURLs } -func (c *cluster) addMemberByURL(t testing.TB, clientURL, peerURL string) error { +func (c *cluster) addMemberByURL(t testutil.TB, clientURL, peerURL string) error { cc := MustNewHTTPClient(t, []string{clientURL}, c.cfg.ClientTLS) ma := client.NewMembersAPI(cc) ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) @@ -378,17 +373,17 @@ func (c *cluster) addMemberByURL(t testing.TB, clientURL, peerURL string) error } // AddMember return PeerURLs of the added member. -func (c *cluster) AddMember(t testing.TB) types.URLs { +func (c *cluster) AddMember(t testutil.TB) types.URLs { return c.addMember(t) } -func (c *cluster) RemoveMember(t testing.TB, id uint64) { +func (c *cluster) RemoveMember(t testutil.TB, id uint64) { if err := c.removeMember(t, id); err != nil { t.Fatal(err) } } -func (c *cluster) removeMember(t testing.TB, id uint64) error { +func (c *cluster) removeMember(t testutil.TB, id uint64) error { // send remove request to the cluster cc := MustNewHTTPClient(t, c.URLs(), c.cfg.ClientTLS) ma := client.NewMembersAPI(cc) @@ -419,7 +414,7 @@ func (c *cluster) removeMember(t testing.TB, id uint64) error { return nil } -func (c *cluster) Terminate(t testing.TB) { +func (c *cluster) Terminate(t testutil.TB) { var wg sync.WaitGroup wg.Add(len(c.Members)) for _, m := range c.Members { @@ -431,7 +426,7 @@ func (c *cluster) Terminate(t testing.TB) { wg.Wait() } -func (c *cluster) waitMembersMatch(t testing.TB, membs []client.Member) { +func (c *cluster) waitMembersMatch(t testutil.TB, membs []client.Member) { for _, u := range c.URLs() { cc := MustNewHTTPClient(t, []string{u}, c.cfg.ClientTLS) ma := client.NewMembersAPI(cc) @@ -448,11 +443,11 @@ func (c *cluster) waitMembersMatch(t testing.TB, membs []client.Member) { } // WaitLeader returns index of the member in c.Members that is leader (or -1). -func (c *cluster) WaitLeader(t testing.TB) int { return c.waitLeader(t, c.Members) } +func (c *cluster) WaitLeader(t testutil.TB) int { return c.waitLeader(t, c.Members) } // waitLeader waits until given members agree on the same leader, // and returns its 'index' in the 'membs' list (or -1). -func (c *cluster) waitLeader(t testing.TB, membs []*member) int { +func (c *cluster) waitLeader(t testutil.TB, membs []*member) int { possibleLead := make(map[uint64]bool) var lead uint64 for _, m := range membs { @@ -545,14 +540,14 @@ func isMembersEqual(membs []client.Member, wmembs []client.Member) bool { return reflect.DeepEqual(membs, wmembs) } -func newLocalListener(t testing.TB) net.Listener { +func newLocalListener(t testutil.TB) net.Listener { c := atomic.AddInt64(&localListenCount, 1) // Go 1.8+ allows only numbers in port addr := fmt.Sprintf("127.0.0.1:%05d%05d", c+basePort, os.Getpid()) return NewListenerWithAddr(t, addr) } -func NewListenerWithAddr(t testing.TB, addr string) net.Listener { +func NewListenerWithAddr(t testutil.TB, addr string) net.Listener { l, err := transport.NewUnixListener(addr) if err != nil { t.Fatal(err) @@ -616,7 +611,7 @@ type memberConfig struct { // mustNewMember return an inited member with the given name. If peerTLS is // set, it will use https scheme to communicate between peers. -func mustNewMember(t testing.TB, mcfg memberConfig) *member { +func mustNewMember(t testutil.TB, mcfg memberConfig) *member { var err error m := &member{} @@ -715,22 +710,13 @@ func mustNewMember(t testing.TB, mcfg memberConfig) *member { level = zapcore.DebugLevel } - if t != nil { - options := zaptest.WrapOptions(zap.Fields(zap.String("member", mcfg.name))) - m.Logger = zaptest.NewLogger(t, zaptest.Level(level), options) - if t != nil { - t.Cleanup(func() { - // if we didn't cleanup the logger, the consecutive test - // might reuse this (t). - raft.ResetDefaultLogger() - }) - } - } else { - m.Logger, err = zap.NewDevelopment(zap.IncreaseLevel(level)) - if err != nil { - log.Panic(err) - } - } + options := zaptest.WrapOptions(zap.Fields(zap.String("member", mcfg.name))) + m.Logger = zaptest.NewLogger(t, zaptest.Level(level), options) + t.Cleanup(func() { + // if we didn't cleanup the logger, the consecutive test + // might reuse this (t). + raft.ResetDefaultLogger() + }) return m } @@ -796,7 +782,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(t testing.TB) *member { +func (m *member) Clone(_ testutil.TB) *member { mm := &member{} mm.ServerConfig = m.ServerConfig @@ -996,14 +982,14 @@ func (m *member) Launch() error { return nil } -func (m *member) WaitOK(t testing.TB) { +func (m *member) WaitOK(t testutil.TB) { m.WaitStarted(t) for m.s.Leader() == 0 { time.Sleep(tickDuration) } } -func (m *member) WaitStarted(t testing.TB) { +func (m *member) WaitStarted(t testutil.TB) { cc := MustNewHTTPClient(t, []string{m.URL()}, m.ClientTLSInfo) kapi := client.NewKeysAPI(cc) for { @@ -1018,7 +1004,7 @@ func (m *member) WaitStarted(t testing.TB) { } } -func WaitClientV3(t testing.TB, kv clientv3.KV) { +func WaitClientV3(t testutil.TB, kv clientv3.KV) { timeout := time.Now().Add(requestTimeout) var err error for time.Now().Before(timeout) { @@ -1088,7 +1074,7 @@ func (m *member) Close() { } // Stop stops the member, but the data dir of the member is preserved. -func (m *member) Stop(t testing.TB) { +func (m *member) Stop(_ testutil.TB) { m.Logger.Info( "stopping a member", zap.String("name", m.Name), @@ -1122,7 +1108,7 @@ func (m *member) StopNotify() <-chan struct{} { } // Restart starts the member using the preserved data dir. -func (m *member) Restart(t testing.TB) error { +func (m *member) Restart(t testutil.TB) error { m.Logger.Info( "restarting a member", zap.String("name", m.Name), @@ -1160,7 +1146,7 @@ func (m *member) Restart(t testing.TB) error { } // Terminate stops the member and removes the data dir. -func (m *member) Terminate(t testing.TB) { +func (m *member) Terminate(t testutil.TB) { m.Logger.Info( "terminating a member", zap.String("name", m.Name), @@ -1221,7 +1207,7 @@ func (m *member) Metric(metricName string, expectLabels ...string) (string, erro } // InjectPartition drops connections from m to others, vice versa. -func (m *member) InjectPartition(t testing.TB, others ...*member) { +func (m *member) InjectPartition(t testutil.TB, others ...*member) { for _, other := range others { m.s.CutPeer(other.s.ID()) other.s.CutPeer(m.s.ID()) @@ -1230,7 +1216,7 @@ func (m *member) InjectPartition(t testing.TB, others ...*member) { } // RecoverPartition recovers connections from m to others, vice versa. -func (m *member) RecoverPartition(t testing.TB, others ...*member) { +func (m *member) RecoverPartition(t testutil.TB, others ...*member) { for _, other := range others { m.s.MendPeer(other.s.ID()) other.s.MendPeer(m.s.ID()) @@ -1242,7 +1228,7 @@ func (m *member) ReadyNotify() <-chan struct{} { return m.s.ReadyNotify() } -func MustNewHTTPClient(t testing.TB, eps []string, tls *transport.TLSInfo) client.Client { +func MustNewHTTPClient(t testutil.TB, eps []string, tls *transport.TLSInfo) client.Client { cfgtls := transport.TLSInfo{} if tls != nil { cfgtls = *tls @@ -1255,7 +1241,7 @@ func MustNewHTTPClient(t testing.TB, eps []string, tls *transport.TLSInfo) clien return c } -func mustNewTransport(t testing.TB, tlsInfo transport.TLSInfo) *http.Transport { +func mustNewTransport(t testutil.TB, tlsInfo transport.TLSInfo) *http.Transport { // tick in integration test is short, so 1s dial timeout could play well. tr, err := transport.NewTimeoutTransport(tlsInfo, time.Second, rafthttp.ConnReadTimeout, rafthttp.ConnWriteTimeout) if err != nil { @@ -1281,12 +1267,9 @@ type ClusterV3 struct { // NewClusterV3 returns a launched cluster with a grpc client connection // for each cluster member. -func NewClusterV3(t testing.TB, cfg *ClusterConfig) *ClusterV3 { - // t might be nil in case of Examples and clusters created per test-suite. - if t != nil { - t.Helper() - testutil.SkipTestIfShortMode(t, "Cannot create clusters in --short tests") - } +func NewClusterV3(t testutil.TB, cfg *ClusterConfig) *ClusterV3 { + t.Helper() + testutil.SkipTestIfShortMode(t, "Cannot create clusters in --short tests") cfg.UseGRPC = true if os.Getenv("CLIENT_DEBUG") != "" { @@ -1301,11 +1284,7 @@ func NewClusterV3(t testing.TB, cfg *ClusterConfig) *ClusterV3 { for _, m := range clus.Members { client, err := NewClientV3(m) if err != nil { - if t != nil { - t.Fatalf("cannot create client: %v", err) - } else { - log.Fatalf("cannot create client: %v", err) - } + t.Fatalf("cannot create client: %v", err) } clus.clients = append(clus.clients, client) } @@ -1320,7 +1299,7 @@ func (c *ClusterV3) TakeClient(idx int) { c.mu.Unlock() } -func (c *ClusterV3) Terminate(t testing.TB) { +func (c *ClusterV3) Terminate(t testutil.TB) { c.mu.Lock() for _, client := range c.clients { if client == nil { @@ -1347,7 +1326,7 @@ func (c *ClusterV3) NewClientV3(memberIndex int) (*clientv3.Client, error) { return NewClientV3(c.Members[memberIndex]) } -func makeClients(t *testing.T, clus *ClusterV3, clients *[]*clientv3.Client, chooseMemberIndex func() int) func() *clientv3.Client { +func makeClients(t testutil.TB, clus *ClusterV3, clients *[]*clientv3.Client, chooseMemberIndex func() int) func() *clientv3.Client { var mu sync.Mutex *clients = nil return func() *clientv3.Client { @@ -1364,18 +1343,18 @@ func makeClients(t *testing.T, clus *ClusterV3, clients *[]*clientv3.Client, cho // MakeSingleNodeClients creates factory of clients that all connect to member 0. // All the created clients are put on the 'clients' list. The factory is thread-safe. -func MakeSingleNodeClients(t *testing.T, clus *ClusterV3, clients *[]*clientv3.Client) func() *clientv3.Client { +func MakeSingleNodeClients(t testutil.TB, clus *ClusterV3, clients *[]*clientv3.Client) func() *clientv3.Client { return makeClients(t, clus, clients, func() int { return 0 }) } // MakeMultiNodeClients creates factory of clients that all connect to random members. // All the created clients are put on the 'clients' list. The factory is thread-safe. -func MakeMultiNodeClients(t *testing.T, clus *ClusterV3, clients *[]*clientv3.Client) func() *clientv3.Client { +func MakeMultiNodeClients(t testutil.TB, clus *ClusterV3, clients *[]*clientv3.Client) func() *clientv3.Client { return makeClients(t, clus, clients, func() int { return rand.Intn(len(clus.Members)) }) } // CloseClients closes all the clients from the 'clients' list. -func CloseClients(t *testing.T, clients []*clientv3.Client) { +func CloseClients(t testutil.TB, clients []*clientv3.Client) { for _, cli := range clients { if err := cli.Close(); err != nil { t.Fatal(err) @@ -1420,7 +1399,7 @@ func (c *ClusterV3) GetLearnerMembers() ([]*pb.Member, error) { // AddAndLaunchLearnerMember creates a leaner member, adds it to cluster // via v3 MemberAdd API, and then launches the new member. -func (c *ClusterV3) AddAndLaunchLearnerMember(t testing.TB) { +func (c *ClusterV3) AddAndLaunchLearnerMember(t testutil.TB) { m := c.mustNewMember(t) m.isLearner = true @@ -1487,7 +1466,7 @@ func (c *ClusterV3) getMembers() []*pb.Member { // indicate that the new learner member has applied the raftpb.ConfChangeAddLearnerNode entry // which was used to add the learner itself to the cluster, and therefore it has the correct info // on learner. -func (c *ClusterV3) waitMembersMatch(t testing.TB) { +func (c *ClusterV3) waitMembersMatch(t testutil.TB) { wMembers := c.getMembers() sort.Sort(SortableProtoMemberSliceByPeerURLs(wMembers)) cli := c.Client(0) @@ -1521,7 +1500,7 @@ func (p SortableProtoMemberSliceByPeerURLs) Less(i, j int) bool { func (p SortableProtoMemberSliceByPeerURLs) Swap(i, j int) { p[i], p[j] = p[j], p[i] } // MustNewMember creates a new member instance based on the response of V3 Member Add API. -func (c *ClusterV3) MustNewMember(t testing.TB, resp *clientv3.MemberAddResponse) *member { +func (c *ClusterV3) MustNewMember(t testutil.TB, resp *clientv3.MemberAddResponse) *member { m := c.mustNewMember(t) m.isLearner = resp.Member.IsLearner m.NewCluster = false diff --git a/tests/integration/lazy_cluster.go b/tests/integration/lazy_cluster.go index 0a92692e7..d16f8d91c 100644 --- a/tests/integration/lazy_cluster.go +++ b/tests/integration/lazy_cluster.go @@ -20,6 +20,7 @@ import ( "sync" "time" + "go.etcd.io/etcd/pkg/v3/testutil" "go.etcd.io/etcd/pkg/v3/transport" ) @@ -47,6 +48,8 @@ type LazyCluster interface { Transport() *http.Transport Terminate() + + TB() testutil.TB } type lazyCluster struct { @@ -54,6 +57,8 @@ type lazyCluster struct { cluster *ClusterV3 transport *http.Transport once sync.Once + tb testutil.TB + closer func() } // NewLazyCluster returns a new test cluster handler that gets created on the @@ -65,7 +70,8 @@ func NewLazyCluster() LazyCluster { // NewLazyClusterWithConfig returns a new test cluster handler that gets created // on the first call to GetEndpoints() or GetTransport() func NewLazyClusterWithConfig(cfg ClusterConfig) LazyCluster { - return &lazyCluster{cfg: cfg} + tb, closer := testutil.NewTestingTBProthesis("lazy_cluster") + return &lazyCluster{cfg: cfg, tb: tb, closer: closer} } func (lc *lazyCluster) mustLazyInit() { @@ -75,15 +81,20 @@ func (lc *lazyCluster) mustLazyInit() { if err != nil { log.Fatal(err) } - lc.cluster = NewClusterV3(nil, &lc.cfg) + lc.cluster = NewClusterV3(lc.tb, &lc.cfg) }) } func (lc *lazyCluster) Terminate() { + lc.tb.Logf("Terminating...") if lc != nil && lc.cluster != nil { lc.cluster.Terminate(nil) lc.cluster = nil } + if lc.closer != nil { + lc.tb.Logf("Closer...") + lc.closer() + } } func (lc *lazyCluster) EndpointsV2() []string { @@ -103,3 +114,7 @@ func (lc *lazyCluster) Transport() *http.Transport { lc.mustLazyInit() return lc.transport } + +func (lc *lazyCluster) TB() testutil.TB { + return lc.tb +} diff --git a/tests/integration/testing.go b/tests/integration/testing.go index e52c882ad..9479f7f39 100644 --- a/tests/integration/testing.go +++ b/tests/integration/testing.go @@ -17,12 +17,11 @@ package integration import ( "os" "path/filepath" - "testing" "go.etcd.io/etcd/pkg/v3/testutil" ) -func BeforeTest(t testing.TB) { +func BeforeTest(t testutil.TB) { testutil.BeforeTest(t) previousWD, err := os.Getwd() @@ -42,4 +41,4 @@ func MustAbsPath(path string) string { panic(err) } return abs -} \ No newline at end of file +} From c8243a9927b7335d3d13074d13e9a9fcc1541600 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 7 Mar 2021 22:26:02 +0100 Subject: [PATCH 9/9] Tests: Functional - in case of failure, log the exception. --- tests/functional/tester/cluster_run.go | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/functional/tester/cluster_run.go b/tests/functional/tester/cluster_run.go index 256540d55..2565ddfae 100644 --- a/tests/functional/tester/cluster_run.go +++ b/tests/functional/tester/cluster_run.go @@ -54,7 +54,7 @@ func (clus *Cluster) Run() { zap.Int("case-total", len(clus.cases)), zap.Error(err), ) - if clus.cleanup() != nil { + if clus.cleanup(err) != nil { return } // reset preModifiedKey after clean up @@ -86,7 +86,7 @@ func (clus *Cluster) Run() { zap.Int("case-total", len(clus.cases)), zap.Error(err), ) - if err = clus.cleanup(); err != nil { + if err = clus.cleanup(err); err != nil { clus.lg.Warn( "cleanup FAIL", zap.Int("round", clus.rd), @@ -101,7 +101,7 @@ func (clus *Cluster) Run() { } if round > 0 && round%500 == 0 { // every 500 rounds if err := clus.defrag(); err != nil { - clus.failed() + clus.failed(err) return } } @@ -315,21 +315,22 @@ func (clus *Cluster) compact(rev int64, timeout time.Duration) (err error) { return nil } -func (clus *Cluster) failed() { +func (clus *Cluster) failed(err error) { clus.lg.Error( "functional-tester FAIL", zap.Int("round", clus.rd), zap.Int("case", clus.cs), zap.Int("case-total", len(clus.cases)), + zap.Error(err), ) clus.Send_SIGQUIT_ETCD_AND_REMOVE_DATA_AND_STOP_AGENT() os.Exit(2) } -func (clus *Cluster) cleanup() error { +func (clus *Cluster) cleanup(err error) error { if clus.Tester.ExitOnCaseFail { - defer clus.failed() + defer clus.failed(err) } roundFailedTotalCounter.Inc()