From b6c2e87a7467dad3628bc4153a1b0d54c30719b1 Mon Sep 17 00:00:00 2001 From: Piotr Tabor Date: Sun, 7 Mar 2021 15:05:57 +0100 Subject: [PATCH] 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 +}