Make NewTmpBackend use testing tmp location (so cleanup).

This commit is contained in:
Piotr Tabor 2021-03-26 13:19:49 +01:00
parent 8ee1dd9e23
commit 03f55eeb2c
18 changed files with 101 additions and 102 deletions

View File

@ -46,7 +46,7 @@ func dummyIndexWaiter(index uint64) <-chan struct{} {
// TestNewAuthStoreRevision ensures newly auth store
// keeps the old revision when there are no changes.
func TestNewAuthStoreRevision(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend()
b, tPath := backend.NewDefaultTmpBackend(t)
defer os.Remove(tPath)
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
@ -76,7 +76,7 @@ func TestNewAuthStoreRevision(t *testing.T) {
// TestNewAuthStoreBryptCost ensures that NewAuthStore uses default when given bcrypt-cost is invalid
func TestNewAuthStoreBcryptCost(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend()
b, tPath := backend.NewDefaultTmpBackend(t)
defer b.Close()
defer os.Remove(tPath)
@ -101,7 +101,7 @@ func encodePassword(s string) string {
}
func setupAuthStore(t *testing.T) (store *authStore, teardownfunc func(t *testing.T)) {
b, tPath := backend.NewDefaultTmpBackend()
b, tPath := backend.NewDefaultTmpBackend(t)
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
if err != nil {
@ -653,7 +653,7 @@ func TestIsAuthEnabled(t *testing.T) {
// TestAuthRevisionRace ensures that access to authStore.revision is thread-safe.
func TestAuthInfoFromCtxRace(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend()
b, tPath := backend.NewDefaultTmpBackend(t)
defer b.Close()
defer os.Remove(tPath)
@ -807,7 +807,7 @@ func TestHammerSimpleAuthenticate(t *testing.T) {
// TestRolesOrder tests authpb.User.Roles is sorted
func TestRolesOrder(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend()
b, tPath := backend.NewDefaultTmpBackend(t)
defer b.Close()
defer os.Remove(tPath)
@ -865,7 +865,7 @@ func TestAuthInfoFromCtxWithRootJWT(t *testing.T) {
// testAuthInfoFromCtxWithRoot ensures "WithRoot" properly embeds token in the context.
func testAuthInfoFromCtxWithRoot(t *testing.T, opts string) {
b, tPath := backend.NewDefaultTmpBackend()
b, tPath := backend.NewDefaultTmpBackend(t)
defer b.Close()
defer os.Remove(tPath)

View File

@ -26,7 +26,7 @@ import (
// TestConsistentIndex ensures that LoadConsistentIndex/Save/ConsistentIndex and backend.BatchTx can work well together.
func TestConsistentIndex(t *testing.T) {
be, tmpPath := backend.NewTmpBackend(time.Microsecond, 10)
be, tmpPath := backend.NewTmpBackend(t, time.Microsecond, 10)
defer os.Remove(tmpPath)
ci := NewConsistentIndex(be.BatchTx())

View File

@ -968,7 +968,7 @@ func TestSyncTrigger(t *testing.T) {
// snapshot should snapshot the store and cut the persistent
func TestSnapshot(t *testing.T) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(t)
defer func() {
os.RemoveAll(tmpPath)
}()
@ -1062,7 +1062,7 @@ func TestSnapshotOrdering(t *testing.T) {
storage: p,
raftStorage: rs,
})
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(t)
defer os.RemoveAll(tmpPath)
s := &EtcdServer{
lgMu: new(sync.RWMutex),
@ -1124,7 +1124,7 @@ func TestSnapshotOrdering(t *testing.T) {
// Applied > SnapshotCount should trigger a SaveSnap event
func TestTriggerSnap(t *testing.T) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(t)
defer func() {
os.RemoveAll(tmpPath)
}()
@ -1213,7 +1213,7 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
storage: mockstorage.NewStorageRecorder(testdir),
raftStorage: rs,
})
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(t)
defer func() {
os.RemoveAll(tmpPath)
}()

View File

@ -29,7 +29,7 @@ import (
func TestRenewHTTP(t *testing.T) {
lg := zap.NewNop()
be, tmpPath := backend.NewTmpBackend(time.Hour, 10000)
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath)
defer be.Close()
@ -54,7 +54,7 @@ func TestRenewHTTP(t *testing.T) {
func TestTimeToLiveHTTP(t *testing.T) {
lg := zap.NewNop()
be, tmpPath := backend.NewTmpBackend(time.Hour, 10000)
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath)
defer be.Close()
@ -96,7 +96,7 @@ func TestTimeToLiveHTTPTimeout(t *testing.T) {
func testApplyTimeout(t *testing.T, f func(*lease.Lease, string) error) {
lg := zap.NewNop()
be, tmpPath := backend.NewTmpBackend(time.Hour, 10000)
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath)
defer be.Close()

View File

@ -16,7 +16,6 @@ package lease
import (
"math/rand"
"os"
"testing"
"time"
@ -64,9 +63,9 @@ func demote(le *lessor) {
}
// return new lessor and tearDown to release resource
func setUp() (le *lessor, tearDown func()) {
func setUp(t testing.TB) (le *lessor, tearDown func()) {
lg := zap.NewNop()
be, tmpPath := backend.NewDefaultTmpBackend()
be, _ := backend.NewDefaultTmpBackend(t)
// MinLeaseTTL is negative, so we can grant expired lease in benchmark.
// ExpiredLeasesRetryInterval should small, so benchmark of findExpired will recheck expired lease.
le = newLessor(lg, be, LessorConfig{MinLeaseTTL: -1000, ExpiredLeasesRetryInterval: 10 * time.Microsecond}, nil)
@ -80,7 +79,6 @@ func setUp() (le *lessor, tearDown func()) {
return le, func() {
le.Stop()
be.Close()
os.Remove(tmpPath)
}
}
@ -97,7 +95,7 @@ func benchmarkLessorGrant(benchSize int, b *testing.B) {
tearDown()
tearDown = nil
}
le, tearDown = setUp()
le, tearDown = setUp(b)
b.StartTimer()
for j := 1; j <= benchSize; j++ {
@ -124,7 +122,7 @@ func benchmarkLessorRevoke(benchSize int, b *testing.B) {
tearDown()
tearDown = nil
}
le, tearDown = setUp()
le, tearDown = setUp(b)
for j := 1; j <= benchSize; j++ {
le.Grant(LeaseID(j), ttls[j-1])
}
@ -155,7 +153,7 @@ func benchmarkLessorRenew(benchSize int, b *testing.B) {
tearDown()
tearDown = nil
}
le, tearDown = setUp()
le, tearDown = setUp(b)
for j := 1; j <= benchSize; j++ {
le.Grant(LeaseID(j), ttls[j-1])
}
@ -188,7 +186,7 @@ func benchmarkLessorFindExpired(benchSize int, b *testing.B) {
tearDown()
tearDown = nil
}
le, tearDown = setUp()
le, tearDown = setUp(b)
for j := 1; j <= benchSize; j++ {
le.Grant(LeaseID(j), ttls[j-1])
}

View File

@ -23,6 +23,7 @@ import (
"path/filepath"
"sync"
"sync/atomic"
"testing"
"time"
humanize "github.com/dustin/go-humanize"
@ -544,8 +545,8 @@ func (b *backend) OpenReadTxN() int64 {
}
// NewTmpBackend creates a backend implementation for testing.
func NewTmpBackend(batchInterval time.Duration, batchLimit int) (*backend, string) {
dir, err := ioutil.TempDir(os.TempDir(), "etcd_backend_test")
func NewTmpBackend(t testing.TB, batchInterval time.Duration, batchLimit int) (*backend, string) {
dir, err := ioutil.TempDir(t.TempDir(), "etcd_backend_test")
if err != nil {
panic(err)
}
@ -555,8 +556,8 @@ func NewTmpBackend(batchInterval time.Duration, batchLimit int) (*backend, strin
return newBackend(bcfg), tmpPath
}
func NewDefaultTmpBackend() (*backend, string) {
return NewTmpBackend(defaultBatchInterval, defaultBatchLimit)
func NewDefaultTmpBackend(t testing.TB) (*backend, string) {
return NewTmpBackend(t, defaultBatchInterval, defaultBatchLimit)
}
type snapshot struct {

View File

@ -22,7 +22,7 @@ import (
)
func BenchmarkBackendPut(b *testing.B) {
backend, tmppath := NewTmpBackend(100*time.Millisecond, 10000)
backend, tmppath := NewTmpBackend(b, 100*time.Millisecond, 10000)
defer backend.Close()
defer os.Remove(tmppath)

View File

@ -26,7 +26,7 @@ import (
)
func TestBackendClose(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath)
// check close could work
@ -46,7 +46,7 @@ func TestBackendClose(t *testing.T) {
}
func TestBackendSnapshot(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath)
tx := b.BatchTx()
@ -86,7 +86,7 @@ func TestBackendSnapshot(t *testing.T) {
func TestBackendBatchIntervalCommit(t *testing.T) {
// start backend with super short batch interval so
// we do not need to wait long before commit to happen.
b, tmpPath := NewTmpBackend(time.Nanosecond, 10000)
b, tmpPath := NewTmpBackend(t, time.Nanosecond, 10000)
defer cleanup(b, tmpPath)
pc := b.Commits()
@ -120,7 +120,7 @@ func TestBackendBatchIntervalCommit(t *testing.T) {
}
func TestBackendDefrag(t *testing.T) {
b, tmpPath := NewDefaultTmpBackend()
b, tmpPath := NewDefaultTmpBackend(t)
defer cleanup(b, tmpPath)
tx := b.BatchTx()
@ -178,7 +178,7 @@ func TestBackendDefrag(t *testing.T) {
// TestBackendWriteback ensures writes are stored to the read txn on write txn unlock.
func TestBackendWriteback(t *testing.T) {
b, tmpPath := NewDefaultTmpBackend()
b, tmpPath := NewDefaultTmpBackend(t)
defer cleanup(b, tmpPath)
tx := b.BatchTx()
@ -252,7 +252,7 @@ func TestBackendWriteback(t *testing.T) {
// TestConcurrentReadTx ensures that current read transaction can see all prior writes stored in read buffer
func TestConcurrentReadTx(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath)
wtx1 := b.BatchTx()
@ -282,7 +282,7 @@ func TestConcurrentReadTx(t *testing.T) {
// TestBackendWritebackForEach checks that partially written / buffered
// data is visited in the same order as fully committed data.
func TestBackendWritebackForEach(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath)
tx := b.BatchTx()

View File

@ -23,7 +23,7 @@ import (
)
func TestBatchTxPut(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath)
tx := b.batchTx
@ -48,7 +48,7 @@ func TestBatchTxPut(t *testing.T) {
}
func TestBatchTxRange(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath)
tx := b.batchTx
@ -119,7 +119,7 @@ func TestBatchTxRange(t *testing.T) {
}
func TestBatchTxDelete(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath)
tx := b.batchTx
@ -142,7 +142,7 @@ func TestBatchTxDelete(t *testing.T) {
}
func TestBatchTxCommit(t *testing.T) {
b, tmpPath := NewTmpBackend(time.Hour, 10000)
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath)
tx := b.batchTx
@ -171,7 +171,7 @@ func TestBatchTxCommit(t *testing.T) {
func TestBatchTxBatchLimitCommit(t *testing.T) {
// start backend with batch limit 1 so one write can
// trigger a commit
b, tmpPath := NewTmpBackend(time.Hour, 1)
b, tmpPath := NewTmpBackend(t, time.Hour, 1)
defer cleanup(b, tmpPath)
tx := b.batchTx

View File

@ -77,7 +77,7 @@ func TestKVRange(t *testing.T) { testKVRange(t, normalRangeFunc) }
func TestKVTxnRange(t *testing.T) { testKVRange(t, txnRangeFunc) }
func testKVRange(t *testing.T, f rangeFunc) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -143,7 +143,7 @@ func TestKVRangeRev(t *testing.T) { testKVRangeRev(t, normalRangeFunc) }
func TestKVTxnRangeRev(t *testing.T) { testKVRangeRev(t, txnRangeFunc) }
func testKVRangeRev(t *testing.T, f rangeFunc) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -179,7 +179,7 @@ func TestKVRangeBadRev(t *testing.T) { testKVRangeBadRev(t, normalRangeFunc)
func TestKVTxnRangeBadRev(t *testing.T) { testKVRangeBadRev(t, txnRangeFunc) }
func testKVRangeBadRev(t *testing.T, f rangeFunc) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -212,7 +212,7 @@ func TestKVRangeLimit(t *testing.T) { testKVRangeLimit(t, normalRangeFunc) }
func TestKVTxnRangeLimit(t *testing.T) { testKVRangeLimit(t, txnRangeFunc) }
func testKVRangeLimit(t *testing.T, f rangeFunc) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -257,7 +257,7 @@ func TestKVPutMultipleTimes(t *testing.T) { testKVPutMultipleTimes(t, normalP
func TestKVTxnPutMultipleTimes(t *testing.T) { testKVPutMultipleTimes(t, txnPutFunc) }
func testKVPutMultipleTimes(t *testing.T, f putFunc) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -319,7 +319,7 @@ func testKVDeleteRange(t *testing.T, f deleteRangeFunc) {
}
for i, tt := range tests {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
@ -339,7 +339,7 @@ func TestKVDeleteMultipleTimes(t *testing.T) { testKVDeleteMultipleTimes(t, n
func TestKVTxnDeleteMultipleTimes(t *testing.T) { testKVDeleteMultipleTimes(t, txnDeleteRangeFunc) }
func testKVDeleteMultipleTimes(t *testing.T, f deleteRangeFunc) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -360,7 +360,7 @@ func testKVDeleteMultipleTimes(t *testing.T, f deleteRangeFunc) {
// test that range, put, delete on single key in sequence repeatedly works correctly.
func TestKVOperationInSequence(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -407,7 +407,7 @@ func TestKVOperationInSequence(t *testing.T) {
}
func TestKVTxnBlockWriteOperations(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
tests := []func(){
@ -441,7 +441,7 @@ func TestKVTxnBlockWriteOperations(t *testing.T) {
}
func TestKVTxnNonBlockRange(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -462,7 +462,7 @@ func TestKVTxnNonBlockRange(t *testing.T) {
// test that txn range, put, delete on single key in sequence repeatedly works correctly.
func TestKVTxnOperationInSequence(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -512,7 +512,7 @@ func TestKVTxnOperationInSequence(t *testing.T) {
}
func TestKVCompactReserveLastValue(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -566,7 +566,7 @@ func TestKVCompactReserveLastValue(t *testing.T) {
}
func TestKVCompactBad(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -599,7 +599,7 @@ func TestKVHash(t *testing.T) {
for i := 0; i < len(hashes); i++ {
var err error
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
kv := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
kv.Put([]byte("foo0"), []byte("bar0"), lease.NoLease)
kv.Put([]byte("foo1"), []byte("bar0"), lease.NoLease)
@ -637,7 +637,7 @@ func TestKVRestore(t *testing.T) {
},
}
for i, tt := range tests {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
tt(s)
var kvss [][]mvccpb.KeyValue
@ -681,7 +681,7 @@ func readGaugeInt(g prometheus.Gauge) int {
}
func TestKVSnapshot(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -717,7 +717,7 @@ func TestKVSnapshot(t *testing.T) {
}
func TestWatchableKVWatch(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath)

View File

@ -27,7 +27,7 @@ import (
)
func BenchmarkStorePut(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -46,7 +46,7 @@ func BenchmarkStoreRangeKey1(b *testing.B) { benchmarkStoreRange(b, 1) }
func BenchmarkStoreRangeKey100(b *testing.B) { benchmarkStoreRange(b, 100) }
func benchmarkStoreRange(b *testing.B, n int) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -73,7 +73,7 @@ func benchmarkStoreRange(b *testing.B, n int) {
}
func BenchmarkConsistentIndex(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -91,7 +91,7 @@ func BenchmarkConsistentIndex(b *testing.B) {
// BenchmarkStoreTxnPutUpdate is same as above, but instead updates single key
func BenchmarkStorePutUpdate(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -109,7 +109,7 @@ func BenchmarkStorePutUpdate(b *testing.B) {
// with transaction begin and end, where transaction involves
// some synchronization operations, such as mutex locking.
func BenchmarkStoreTxnPut(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -129,7 +129,7 @@ func BenchmarkStoreTxnPut(b *testing.B) {
// benchmarkStoreRestore benchmarks the restore operation
func benchmarkStoreRestore(revsPerKey int, b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
// use closure to capture 's' to pick up the reassignment
defer func() { cleanup(s, be, tmpPath) }()

View File

@ -66,7 +66,7 @@ func TestScheduleCompaction(t *testing.T) {
},
}
for i, tt := range tests {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
tx := s.b.BatchTx()
@ -100,7 +100,7 @@ func TestScheduleCompaction(t *testing.T) {
}
func TestCompactAllAndRestore(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)

View File

@ -41,7 +41,7 @@ import (
)
func TestStoreRev(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer s.Close()
defer os.Remove(tmpPath)
@ -425,7 +425,7 @@ func TestRestoreDelete(t *testing.T) {
restoreChunkKeys = mrand.Intn(3) + 2
defer func() { restoreChunkKeys = oldChunk }()
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)
@ -473,7 +473,7 @@ func TestRestoreDelete(t *testing.T) {
func TestRestoreContinueUnfinishedCompaction(t *testing.T) {
tests := []string{"recreate", "restore"}
for _, test := range tests {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)
@ -535,7 +535,7 @@ type hashKVResult struct {
// TestHashKVWhenCompacting ensures that HashKV returns correct hash when compacting.
func TestHashKVWhenCompacting(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)
@ -603,7 +603,7 @@ func TestHashKVWhenCompacting(t *testing.T) {
// TestHashKVZeroRevision ensures that "HashByRev(0)" computes
// correct hash value with latest revision.
func TestHashKVZeroRevision(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)
@ -636,7 +636,7 @@ func TestTxnPut(t *testing.T) {
keys := createBytesSlice(bytesN, sliceN)
vals := createBytesSlice(bytesN, sliceN)
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -652,7 +652,7 @@ func TestTxnPut(t *testing.T) {
// TestConcurrentReadNotBlockingWrite ensures Read does not blocking Write after its creation
func TestConcurrentReadNotBlockingWrite(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)
@ -721,7 +721,7 @@ func TestConcurrentReadTxAndWrite(t *testing.T) {
committedKVs kvs // committedKVs records the key-value pairs written by the finished Write Txns
mu sync.Mutex // mu protects committedKVs
)
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)

View File

@ -28,7 +28,7 @@ import (
)
func BenchmarkWatchableStorePut(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := New(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -48,7 +48,7 @@ func BenchmarkWatchableStorePut(b *testing.B) {
// with transaction begin and end, where transaction involves
// some synchronization operations, such as mutex locking.
func BenchmarkWatchableStoreTxnPut(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := New(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -79,7 +79,7 @@ func BenchmarkWatchableStoreWatchPutUnsync(b *testing.B) {
}
func benchmarkWatchableStoreWatchPut(b *testing.B, synced bool) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, be, tmpPath)
@ -122,7 +122,7 @@ func benchmarkWatchableStoreWatchPut(b *testing.B, synced bool) {
// TODO: k is an arbitrary constant. We need to figure out what factor
// we should put to simulate the real-world use cases.
func BenchmarkWatchableStoreUnsyncedCancel(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
// manually create watchableStore instead of newWatchableStore
@ -179,7 +179,7 @@ func BenchmarkWatchableStoreUnsyncedCancel(b *testing.B) {
}
func BenchmarkWatchableStoreSyncedCancel(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
s := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {

View File

@ -32,7 +32,7 @@ import (
)
func TestWatch(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {
@ -54,7 +54,7 @@ func TestWatch(t *testing.T) {
}
func TestNewWatcherCancel(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {
@ -80,7 +80,7 @@ func TestNewWatcherCancel(t *testing.T) {
// TestCancelUnsynced tests if running CancelFunc removes watchers from unsynced.
func TestCancelUnsynced(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
// manually create watchableStore instead of newWatchableStore
// because newWatchableStore automatically calls syncWatchers
@ -139,7 +139,7 @@ func TestCancelUnsynced(t *testing.T) {
// method to see if it correctly sends events to channel of unsynced watchers
// and moves these watchers to synced.
func TestSyncWatchers(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := &watchableStore{
store: NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}),
@ -224,7 +224,7 @@ func TestSyncWatchers(t *testing.T) {
// TestWatchCompacted tests a watcher that watches on a compacted revision.
func TestWatchCompacted(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {
@ -261,7 +261,7 @@ func TestWatchCompacted(t *testing.T) {
}
func TestWatchFutureRev(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {
@ -302,7 +302,7 @@ func TestWatchFutureRev(t *testing.T) {
func TestWatchRestore(t *testing.T) {
test := func(delay time.Duration) func(t *testing.T) {
return func(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, cindex.NewConsistentIndex(b.BatchTx()), StoreConfig{})
defer cleanup(s, b, tmpPath)
@ -310,7 +310,7 @@ func TestWatchRestore(t *testing.T) {
testValue := []byte("bar")
rev := s.Put(testKey, testValue, lease.NoLease)
newBackend, newPath := backend.NewDefaultTmpBackend()
newBackend, newPath := backend.NewDefaultTmpBackend(t)
newStore := newWatchableStore(zap.NewExample(), newBackend, &lease.FakeLessor{}, cindex.NewConsistentIndex(newBackend.BatchTx()), StoreConfig{})
defer cleanup(newStore, newBackend, newPath)
@ -348,11 +348,11 @@ func TestWatchRestore(t *testing.T) {
// 4. restore operation moves "synced" to "unsynced" watcher group
// 5. choose the watcher from step 1, without panic
func TestWatchRestoreSyncedWatcher(t *testing.T) {
b1, b1Path := backend.NewDefaultTmpBackend()
b1, b1Path := backend.NewDefaultTmpBackend(t)
s1 := newWatchableStore(zap.NewExample(), b1, &lease.FakeLessor{}, cindex.NewConsistentIndex(b1.BatchTx()), StoreConfig{})
defer cleanup(s1, b1, b1Path)
b2, b2Path := backend.NewDefaultTmpBackend()
b2, b2Path := backend.NewDefaultTmpBackend(t)
s2 := newWatchableStore(zap.NewExample(), b2, &lease.FakeLessor{}, cindex.NewConsistentIndex(b2.BatchTx()), StoreConfig{})
defer cleanup(s2, b2, b2Path)
@ -399,7 +399,7 @@ func TestWatchRestoreSyncedWatcher(t *testing.T) {
// TestWatchBatchUnsynced tests batching on unsynced watchers
func TestWatchBatchUnsynced(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
oldMaxRevs := watchBatchMaxRevs
@ -533,7 +533,7 @@ func TestNewMapwatcherToEventMap(t *testing.T) {
func TestWatchVictims(t *testing.T) {
oldChanBufLen, oldMaxWatchersPerSync := chanBufLen, maxWatchersPerSync
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {
@ -611,7 +611,7 @@ func TestWatchVictims(t *testing.T) {
// TestStressWatchCancelClose tests closing a watch stream while
// canceling its watches.
func TestStressWatchCancelClose(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {

View File

@ -25,7 +25,7 @@ import (
)
func BenchmarkKVWatcherMemoryUsage(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend()
be, tmpPath := backend.NewDefaultTmpBackend(b)
watchable := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(watchable, be, tmpPath)

View File

@ -31,7 +31,7 @@ import (
// TestWatcherWatchID tests that each watcher provides unique watchID,
// and the watched event attaches the correct watchID.
func TestWatcherWatchID(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath)
@ -81,7 +81,7 @@ func TestWatcherWatchID(t *testing.T) {
}
func TestWatcherRequestsCustomID(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath)
@ -118,7 +118,7 @@ func TestWatcherRequestsCustomID(t *testing.T) {
// TestWatcherWatchPrefix tests if Watch operation correctly watches
// and returns events with matching prefixes.
func TestWatcherWatchPrefix(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath)
@ -192,7 +192,7 @@ func TestWatcherWatchPrefix(t *testing.T) {
// TestWatcherWatchWrongRange ensures that watcher with wrong 'end' range
// does not create watcher, which panics when canceling in range tree.
func TestWatcherWatchWrongRange(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath)
@ -212,7 +212,7 @@ func TestWatcherWatchWrongRange(t *testing.T) {
}
func TestWatchDeleteRange(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer func() {
@ -251,7 +251,7 @@ func TestWatchDeleteRange(t *testing.T) {
// TestWatchStreamCancelWatcherByID ensures cancel calls the cancel func of the watcher
// with given id inside watchStream.
func TestWatchStreamCancelWatcherByID(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath)
@ -288,7 +288,7 @@ func TestWatchStreamCancelWatcherByID(t *testing.T) {
// TestWatcherRequestProgress ensures synced watcher can correctly
// report its correct progress.
func TestWatcherRequestProgress(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
// manually create watchableStore instead of newWatchableStore
// because newWatchableStore automatically calls syncWatchers
@ -343,7 +343,7 @@ func TestWatcherRequestProgress(t *testing.T) {
}
func TestWatcherWatchWithFilter(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend()
b, tmpPath := backend.NewDefaultTmpBackend(t)
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath)

View File

@ -181,18 +181,18 @@ func newEtcdProcessCluster(t testing.TB, cfg *etcdProcessClusterConfig) (*etcdPr
proc, err := newEtcdProcess(etcdCfgs[i])
if err != nil {
epc.Close()
return nil, err
return nil, fmt.Errorf("Cannot configure: %v", err)
}
epc.procs[i] = proc
}
if cfg.rollingStart {
if err := epc.RollingStart(); err != nil {
return nil, err
return nil, fmt.Errorf("Cannot rolling-start: %v", err)
}
} else {
if err := epc.Start(); err != nil {
return nil, err
return nil, fmt.Errorf("Cannot start: %v", err)
}
}
return epc, nil