Testing: Integration tests does not check whether t==nil

This commit is contained in:
Piotr Tabor 2021-03-07 15:05:57 +01:00
parent 5ddabfdb24
commit b6c2e87a74
6 changed files with 201 additions and 78 deletions

View File

@ -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)
}

130
pkg/testutil/testingtb.go Normal file
View File

@ -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]()
}
}

View File

@ -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() {

View File

@ -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

View File

@ -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
}

View File

@ -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
}
}