mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
mvcc/backend tests: Refactor: Do not mix testing&prod code.
This commit is contained in:
parent
ea287dd9f8
commit
d7d110b5a8
@ -98,6 +98,15 @@
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"project": "github.com/davecgh/go-spew/spew",
|
||||
"licenses": [
|
||||
{
|
||||
"type": "ISC License",
|
||||
"confidence": 0.9850746268656716
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"project": "github.com/dustin/go-humanize",
|
||||
"licenses": [
|
||||
@ -386,6 +395,15 @@
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"project": "github.com/stretchr/testify/assert",
|
||||
"licenses": [
|
||||
{
|
||||
"type": "MIT License",
|
||||
"confidence": 1
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"project": "github.com/tmc/grpc-websocket-proxy/wsproxy",
|
||||
"licenses": [
|
||||
|
@ -18,7 +18,6 @@ import (
|
||||
"context"
|
||||
"encoding/base64"
|
||||
"fmt"
|
||||
"os"
|
||||
"reflect"
|
||||
"strings"
|
||||
"sync"
|
||||
@ -29,6 +28,7 @@ import (
|
||||
pb "go.etcd.io/etcd/api/v3/etcdserverpb"
|
||||
"go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/crypto/bcrypt"
|
||||
@ -46,8 +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(t)
|
||||
defer os.Remove(tPath)
|
||||
b, tPath := betesting.NewDefaultTmpBackend(t)
|
||||
|
||||
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
|
||||
if err != nil {
|
||||
@ -76,9 +75,8 @@ 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(t)
|
||||
defer b.Close()
|
||||
defer os.Remove(tPath)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
|
||||
if err != nil {
|
||||
@ -101,7 +99,7 @@ func encodePassword(s string) string {
|
||||
}
|
||||
|
||||
func setupAuthStore(t *testing.T) (store *authStore, teardownfunc func(t *testing.T)) {
|
||||
b, tPath := backend.NewDefaultTmpBackend(t)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
|
||||
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
|
||||
if err != nil {
|
||||
@ -127,7 +125,6 @@ func setupAuthStore(t *testing.T) (store *authStore, teardownfunc func(t *testin
|
||||
|
||||
tearDown := func(_ *testing.T) {
|
||||
b.Close()
|
||||
os.Remove(tPath)
|
||||
as.Close()
|
||||
}
|
||||
return as, tearDown
|
||||
@ -653,9 +650,8 @@ func TestIsAuthEnabled(t *testing.T) {
|
||||
|
||||
// TestAuthRevisionRace ensures that access to authStore.revision is thread-safe.
|
||||
func TestAuthInfoFromCtxRace(t *testing.T) {
|
||||
b, tPath := backend.NewDefaultTmpBackend(t)
|
||||
defer b.Close()
|
||||
defer os.Remove(tPath)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
|
||||
if err != nil {
|
||||
@ -807,9 +803,8 @@ func TestHammerSimpleAuthenticate(t *testing.T) {
|
||||
|
||||
// TestRolesOrder tests authpb.User.Roles is sorted
|
||||
func TestRolesOrder(t *testing.T) {
|
||||
b, tPath := backend.NewDefaultTmpBackend(t)
|
||||
defer b.Close()
|
||||
defer os.Remove(tPath)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
|
||||
defer tp.disable()
|
||||
@ -865,9 +860,8 @@ 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(t)
|
||||
defer b.Close()
|
||||
defer os.Remove(tPath)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tp, err := NewTokenProvider(zap.NewExample(), opts, dummyIndexWaiter, simpleTokenTTLDefault)
|
||||
if err != nil {
|
||||
|
@ -16,18 +16,17 @@ package cindex
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
)
|
||||
|
||||
// TestConsistentIndex ensures that LoadConsistentIndex/Save/ConsistentIndex and backend.BatchTx can work well together.
|
||||
func TestConsistentIndex(t *testing.T) {
|
||||
|
||||
be, tmpPath := backend.NewTmpBackend(t, time.Microsecond, 10)
|
||||
defer os.Remove(tmpPath)
|
||||
be, tmpPath := betesting.NewTmpBackend(t, time.Microsecond, 10)
|
||||
ci := NewConsistentIndex(be.BatchTx())
|
||||
|
||||
tx := be.BatchTx()
|
||||
|
@ -50,7 +50,7 @@ import (
|
||||
"go.etcd.io/etcd/server/v3/mock/mockstore"
|
||||
"go.etcd.io/etcd/server/v3/mock/mockwait"
|
||||
"go.etcd.io/etcd/server/v3/mvcc"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
"go.uber.org/zap"
|
||||
"go.uber.org/zap/zaptest"
|
||||
)
|
||||
@ -972,10 +972,7 @@ func TestSyncTrigger(t *testing.T) {
|
||||
|
||||
// snapshot should snapshot the store and cut the persistent
|
||||
func TestSnapshot(t *testing.T) {
|
||||
be, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
defer func() {
|
||||
os.RemoveAll(tmpPath)
|
||||
}()
|
||||
be, _ := betesting.NewDefaultTmpBackend(t)
|
||||
|
||||
s := raft.NewMemoryStorage()
|
||||
s.Append([]raftpb.Entry{{Index: 1}})
|
||||
@ -1066,7 +1063,7 @@ func TestSnapshotOrdering(t *testing.T) {
|
||||
storage: p,
|
||||
raftStorage: rs,
|
||||
})
|
||||
be, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
be, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
defer os.RemoveAll(tmpPath)
|
||||
s := &EtcdServer{
|
||||
lgMu: new(sync.RWMutex),
|
||||
@ -1128,7 +1125,7 @@ func TestSnapshotOrdering(t *testing.T) {
|
||||
|
||||
// Applied > SnapshotCount should trigger a SaveSnap event
|
||||
func TestTriggerSnap(t *testing.T) {
|
||||
be, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
be, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
defer func() {
|
||||
os.RemoveAll(tmpPath)
|
||||
}()
|
||||
@ -1217,7 +1214,7 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
|
||||
storage: mockstorage.NewStorageRecorder(testdir),
|
||||
raftStorage: rs,
|
||||
})
|
||||
be, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
be, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
defer func() {
|
||||
os.RemoveAll(tmpPath)
|
||||
}()
|
||||
@ -1552,7 +1549,7 @@ func TestPublishV3(t *testing.T) {
|
||||
w := wait.NewWithResponse(ch)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
lg := zaptest.NewLogger(t)
|
||||
be, _ := backend.NewDefaultTmpBackend(t)
|
||||
be, _ := betesting.NewDefaultTmpBackend(t)
|
||||
srv := &EtcdServer{
|
||||
lgMu: new(sync.RWMutex),
|
||||
lg: lg,
|
||||
@ -1622,7 +1619,7 @@ func TestPublishV3Retry(t *testing.T) {
|
||||
n := newNodeRecorderStream()
|
||||
|
||||
lg := zaptest.NewLogger(t)
|
||||
be, _ := backend.NewDefaultTmpBackend(t)
|
||||
be, _ := betesting.NewDefaultTmpBackend(t)
|
||||
srv := &EtcdServer{
|
||||
lgMu: new(sync.RWMutex),
|
||||
lg: lg,
|
||||
|
@ -18,20 +18,18 @@ import (
|
||||
"context"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
"go.uber.org/zap"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
"go.uber.org/zap/zaptest"
|
||||
)
|
||||
|
||||
func TestRenewHTTP(t *testing.T) {
|
||||
lg := zap.NewNop()
|
||||
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer os.Remove(tmpPath)
|
||||
defer be.Close()
|
||||
lg := zaptest.NewLogger(t)
|
||||
be, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, be)
|
||||
|
||||
le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil)
|
||||
le.Promote(time.Second)
|
||||
@ -53,10 +51,9 @@ func TestRenewHTTP(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestTimeToLiveHTTP(t *testing.T) {
|
||||
lg := zap.NewNop()
|
||||
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer os.Remove(tmpPath)
|
||||
defer be.Close()
|
||||
lg := zaptest.NewLogger(t)
|
||||
be, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, be)
|
||||
|
||||
le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil)
|
||||
le.Promote(time.Second)
|
||||
@ -95,10 +92,9 @@ func TestTimeToLiveHTTPTimeout(t *testing.T) {
|
||||
}
|
||||
|
||||
func testApplyTimeout(t *testing.T, f func(*lease.Lease, string) error) {
|
||||
lg := zap.NewNop()
|
||||
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer os.Remove(tmpPath)
|
||||
defer be.Close()
|
||||
lg := zaptest.NewLogger(t)
|
||||
be, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, be)
|
||||
|
||||
le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil)
|
||||
le.Promote(time.Second)
|
||||
|
@ -19,7 +19,7 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
@ -65,7 +65,7 @@ func demote(le *lessor) {
|
||||
// return new lessor and tearDown to release resource
|
||||
func setUp(t testing.TB) (le *lessor, tearDown func()) {
|
||||
lg := zap.NewNop()
|
||||
be, _ := backend.NewDefaultTmpBackend(t)
|
||||
be, _ := betesting.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)
|
||||
|
@ -23,7 +23,6 @@ import (
|
||||
"path/filepath"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
humanize "github.com/dustin/go-humanize"
|
||||
@ -544,22 +543,6 @@ func (b *backend) OpenReadTxN() int64 {
|
||||
return atomic.LoadInt64(&b.openReadTxN)
|
||||
}
|
||||
|
||||
// NewTmpBackend creates a backend implementation for testing.
|
||||
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)
|
||||
}
|
||||
tmpPath := filepath.Join(dir, "database")
|
||||
bcfg := DefaultBackendConfig()
|
||||
bcfg.Path, bcfg.BatchInterval, bcfg.BatchLimit = tmpPath, batchInterval, batchLimit
|
||||
return newBackend(bcfg), tmpPath
|
||||
}
|
||||
|
||||
func NewDefaultTmpBackend(t testing.TB) (*backend, string) {
|
||||
return NewTmpBackend(t, defaultBatchInterval, defaultBatchLimit)
|
||||
}
|
||||
|
||||
type snapshot struct {
|
||||
*bolt.Tx
|
||||
stopc chan struct{}
|
||||
|
@ -12,28 +12,31 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package backend
|
||||
package backend_test
|
||||
|
||||
import (
|
||||
"crypto/rand"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
)
|
||||
|
||||
func BenchmarkBackendPut(b *testing.B) {
|
||||
backend, tmppath := NewTmpBackend(b, 100*time.Millisecond, 10000)
|
||||
defer backend.Close()
|
||||
defer os.Remove(tmppath)
|
||||
backend, _ := betesting.NewTmpBackend(b, 100*time.Millisecond, 10000)
|
||||
defer betesting.Close(b, backend)
|
||||
|
||||
// prepare keys
|
||||
keys := make([][]byte, b.N)
|
||||
for i := 0; i < b.N; i++ {
|
||||
keys[i] = make([]byte, 64)
|
||||
rand.Read(keys[i])
|
||||
_, err := rand.Read(keys[i])
|
||||
assert.NoError(b, err)
|
||||
}
|
||||
value := make([]byte, 128)
|
||||
rand.Read(value)
|
||||
_, err := rand.Read(value)
|
||||
assert.NoError(b, err)
|
||||
|
||||
batchTx := backend.BatchTx()
|
||||
|
||||
|
@ -12,22 +12,23 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package backend
|
||||
package backend_test
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
bolt "go.etcd.io/bbolt"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
)
|
||||
|
||||
func TestBackendClose(t *testing.T) {
|
||||
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
|
||||
defer os.Remove(tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
|
||||
// check close could work
|
||||
done := make(chan struct{})
|
||||
@ -46,8 +47,8 @@ func TestBackendClose(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestBackendSnapshot(t *testing.T) {
|
||||
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
@ -57,22 +58,22 @@ func TestBackendSnapshot(t *testing.T) {
|
||||
b.ForceCommit()
|
||||
|
||||
// write snapshot to a new file
|
||||
f, err := ioutil.TempFile(os.TempDir(), "etcd_backend_test")
|
||||
f, err := ioutil.TempFile(t.TempDir(), "etcd_backend_test")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
snap := b.Snapshot()
|
||||
defer snap.Close()
|
||||
defer func() { assert.NoError(t, snap.Close()) }()
|
||||
if _, err := snap.WriteTo(f); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
f.Close()
|
||||
assert.NoError(t, f.Close())
|
||||
|
||||
// bootstrap new backend from the snapshot
|
||||
bcfg := DefaultBackendConfig()
|
||||
bcfg := backend.DefaultBackendConfig()
|
||||
bcfg.Path, bcfg.BatchInterval, bcfg.BatchLimit = f.Name(), time.Hour, 10000
|
||||
nb := New(bcfg)
|
||||
defer cleanup(nb, f.Name())
|
||||
nb := backend.New(bcfg)
|
||||
defer betesting.Close(t, nb)
|
||||
|
||||
newTx := nb.BatchTx()
|
||||
newTx.Lock()
|
||||
@ -86,10 +87,10 @@ 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(t, time.Nanosecond, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Nanosecond, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
pc := b.Commits()
|
||||
pc := backend.CommitsForTest(b)
|
||||
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
@ -98,14 +99,14 @@ func TestBackendBatchIntervalCommit(t *testing.T) {
|
||||
tx.Unlock()
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
if b.Commits() >= pc+1 {
|
||||
if backend.CommitsForTest(b) >= pc+1 {
|
||||
break
|
||||
}
|
||||
time.Sleep(time.Duration(i*100) * time.Millisecond)
|
||||
}
|
||||
|
||||
// check whether put happens via db view
|
||||
b.db.View(func(tx *bolt.Tx) error {
|
||||
assert.NoError(t, backend.DbFromBackendForTest(b).View(func(tx *bolt.Tx) error {
|
||||
bucket := tx.Bucket([]byte("test"))
|
||||
if bucket == nil {
|
||||
t.Errorf("bucket test does not exit")
|
||||
@ -116,17 +117,17 @@ func TestBackendBatchIntervalCommit(t *testing.T) {
|
||||
t.Errorf("foo key failed to written in backend")
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}))
|
||||
}
|
||||
|
||||
func TestBackendDefrag(t *testing.T) {
|
||||
b, tmpPath := NewDefaultTmpBackend(t)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
tx.UnsafeCreateBucket([]byte("test"))
|
||||
for i := 0; i < defragLimit+100; i++ {
|
||||
for i := 0; i < backend.DefragLimitForTest()+100; i++ {
|
||||
tx.UnsafePut([]byte("test"), []byte(fmt.Sprintf("foo_%d", i)), []byte("bar"))
|
||||
}
|
||||
tx.Unlock()
|
||||
@ -178,8 +179,8 @@ 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(t)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
@ -252,8 +253,8 @@ 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(t, time.Hour, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
wtx1 := b.BatchTx()
|
||||
wtx1.Lock()
|
||||
@ -282,8 +283,8 @@ 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(t, time.Hour, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
@ -312,7 +313,7 @@ func TestBackendWritebackForEach(t *testing.T) {
|
||||
}
|
||||
rtx := b.ReadTx()
|
||||
rtx.RLock()
|
||||
rtx.UnsafeForEach([]byte("key"), getSeq)
|
||||
assert.NoError(t, rtx.UnsafeForEach([]byte("key"), getSeq))
|
||||
rtx.RUnlock()
|
||||
|
||||
partialSeq := seq
|
||||
@ -321,15 +322,10 @@ func TestBackendWritebackForEach(t *testing.T) {
|
||||
b.ForceCommit()
|
||||
|
||||
tx.Lock()
|
||||
tx.UnsafeForEach([]byte("key"), getSeq)
|
||||
assert.NoError(t, tx.UnsafeForEach([]byte("key"), getSeq))
|
||||
tx.Unlock()
|
||||
|
||||
if seq != partialSeq {
|
||||
t.Fatalf("expected %q, got %q", seq, partialSeq)
|
||||
}
|
||||
}
|
||||
|
||||
func cleanup(b Backend, path string) {
|
||||
b.Close()
|
||||
os.Remove(path)
|
||||
}
|
||||
|
@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package backend
|
||||
package backend_test
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
@ -20,15 +20,17 @@ import (
|
||||
"time"
|
||||
|
||||
bolt "go.etcd.io/bbolt"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
)
|
||||
|
||||
func TestBatchTxPut(t *testing.T) {
|
||||
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.BatchTx()
|
||||
|
||||
tx := b.batchTx
|
||||
tx.Lock()
|
||||
defer tx.Unlock()
|
||||
|
||||
// create bucket
|
||||
tx.UnsafeCreateBucket([]byte("test"))
|
||||
@ -37,21 +39,25 @@ func TestBatchTxPut(t *testing.T) {
|
||||
v := []byte("bar")
|
||||
tx.UnsafePut([]byte("test"), []byte("foo"), v)
|
||||
|
||||
tx.Unlock()
|
||||
|
||||
// check put result before and after tx is committed
|
||||
for k := 0; k < 2; k++ {
|
||||
tx.Lock()
|
||||
_, gv := tx.UnsafeRange([]byte("test"), []byte("foo"), nil, 0)
|
||||
tx.Unlock()
|
||||
if !reflect.DeepEqual(gv[0], v) {
|
||||
t.Errorf("v = %s, want %s", string(gv[0]), string(v))
|
||||
}
|
||||
tx.commit(false)
|
||||
tx.Commit()
|
||||
}
|
||||
}
|
||||
|
||||
func TestBatchTxRange(t *testing.T) {
|
||||
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.batchTx
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
defer tx.Unlock()
|
||||
|
||||
@ -119,33 +125,36 @@ func TestBatchTxRange(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestBatchTxDelete(t *testing.T) {
|
||||
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.batchTx
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
defer tx.Unlock()
|
||||
|
||||
tx.UnsafeCreateBucket([]byte("test"))
|
||||
tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar"))
|
||||
|
||||
tx.UnsafeDelete([]byte("test"), []byte("foo"))
|
||||
|
||||
tx.Unlock()
|
||||
|
||||
// check put result before and after tx is committed
|
||||
for k := 0; k < 2; k++ {
|
||||
tx.Lock()
|
||||
ks, _ := tx.UnsafeRange([]byte("test"), []byte("foo"), nil, 0)
|
||||
tx.Unlock()
|
||||
if len(ks) != 0 {
|
||||
t.Errorf("keys on foo = %v, want nil", ks)
|
||||
}
|
||||
tx.commit(false)
|
||||
tx.Commit()
|
||||
}
|
||||
}
|
||||
|
||||
func TestBatchTxCommit(t *testing.T) {
|
||||
b, tmpPath := NewTmpBackend(t, time.Hour, 10000)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.batchTx
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
tx.UnsafeCreateBucket([]byte("test"))
|
||||
tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar"))
|
||||
@ -154,7 +163,7 @@ func TestBatchTxCommit(t *testing.T) {
|
||||
tx.Commit()
|
||||
|
||||
// check whether put happens via db view
|
||||
b.db.View(func(tx *bolt.Tx) error {
|
||||
backend.DbFromBackendForTest(b).View(func(tx *bolt.Tx) error {
|
||||
bucket := tx.Bucket([]byte("test"))
|
||||
if bucket == nil {
|
||||
t.Errorf("bucket test does not exit")
|
||||
@ -171,10 +180,10 @@ 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(t, time.Hour, 1)
|
||||
defer cleanup(b, tmpPath)
|
||||
b, _ := betesting.NewTmpBackend(t, time.Hour, 1)
|
||||
defer betesting.Close(t, b)
|
||||
|
||||
tx := b.batchTx
|
||||
tx := b.BatchTx()
|
||||
tx.Lock()
|
||||
tx.UnsafeCreateBucket([]byte("test"))
|
||||
tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar"))
|
||||
@ -182,7 +191,7 @@ func TestBatchTxBatchLimitCommit(t *testing.T) {
|
||||
|
||||
// batch limit commit should have been triggered
|
||||
// check whether put happens via db view
|
||||
b.db.View(func(tx *bolt.Tx) error {
|
||||
backend.DbFromBackendForTest(b).View(func(tx *bolt.Tx) error {
|
||||
bucket := tx.Bucket([]byte("test"))
|
||||
if bucket == nil {
|
||||
t.Errorf("bucket test does not exit")
|
||||
|
15
server/mvcc/backend/export_test.go
Normal file
15
server/mvcc/backend/export_test.go
Normal file
@ -0,0 +1,15 @@
|
||||
package backend
|
||||
|
||||
import bolt "go.etcd.io/bbolt"
|
||||
|
||||
func DbFromBackendForTest(b Backend) *bolt.DB {
|
||||
return b.(*backend).db
|
||||
}
|
||||
|
||||
func DefragLimitForTest() int {
|
||||
return defragLimit
|
||||
}
|
||||
|
||||
func CommitsForTest(b Backend) int64 {
|
||||
return b.(*backend).Commits()
|
||||
}
|
52
server/mvcc/backend/testing/betesting.go
Normal file
52
server/mvcc/backend/testing/betesting.go
Normal file
@ -0,0 +1,52 @@
|
||||
// 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 betesting
|
||||
|
||||
import (
|
||||
"io/ioutil"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
"go.uber.org/zap/zaptest"
|
||||
)
|
||||
|
||||
func NewTmpBackendFromCfg(t testing.TB, bcfg backend.BackendConfig) (backend.Backend, string) {
|
||||
dir, err := ioutil.TempDir(t.TempDir(), "etcd_backend_test")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
tmpPath := filepath.Join(dir, "database")
|
||||
bcfg.Path = tmpPath
|
||||
bcfg.Logger = zaptest.NewLogger(t)
|
||||
return backend.New(bcfg), tmpPath
|
||||
}
|
||||
|
||||
// NewTmpBackend creates a backend implementation for testing.
|
||||
func NewTmpBackend(t testing.TB, batchInterval time.Duration, batchLimit int) (backend.Backend, string) {
|
||||
bcfg := backend.DefaultBackendConfig()
|
||||
bcfg.BatchInterval, bcfg.BatchLimit = batchInterval, batchLimit
|
||||
return NewTmpBackendFromCfg(t, bcfg)
|
||||
}
|
||||
|
||||
func NewDefaultTmpBackend(t testing.TB) (backend.Backend, string) {
|
||||
return NewTmpBackendFromCfg(t, backend.DefaultBackendConfig())
|
||||
}
|
||||
|
||||
func Close(t testing.TB, b backend.Backend) {
|
||||
assert.NoError(t, b.Close())
|
||||
}
|
@ -27,6 +27,7 @@ import (
|
||||
"go.etcd.io/etcd/pkg/v3/traceutil"
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
dto "github.com/prometheus/client_model/go"
|
||||
@ -77,7 +78,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -143,7 +144,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -179,7 +180,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -212,7 +213,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -257,7 +258,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -319,7 +320,7 @@ func testKVDeleteRange(t *testing.T, f deleteRangeFunc) {
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
|
||||
s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
|
||||
@ -339,7 +340,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -360,7 +361,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -407,7 +408,7 @@ func TestKVOperationInSequence(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestKVTxnBlockWriteOperations(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
|
||||
tests := []func(){
|
||||
@ -441,7 +442,7 @@ func TestKVTxnBlockWriteOperations(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestKVTxnNonBlockRange(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -462,7 +463,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -512,7 +513,7 @@ func TestKVTxnOperationInSequence(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestKVCompactReserveLastValue(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -566,7 +567,7 @@ func TestKVCompactReserveLastValue(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestKVCompactBad(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -599,7 +600,7 @@ func TestKVHash(t *testing.T) {
|
||||
|
||||
for i := 0; i < len(hashes); i++ {
|
||||
var err error
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.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 +638,7 @@ func TestKVRestore(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
tt(s)
|
||||
var kvss [][]mvccpb.KeyValue
|
||||
@ -681,7 +682,7 @@ func readGaugeInt(g prometheus.Gauge) int {
|
||||
}
|
||||
|
||||
func TestKVSnapshot(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -717,7 +718,7 @@ func TestKVSnapshot(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestWatchableKVWatch(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
|
@ -21,13 +21,13 @@ import (
|
||||
"go.etcd.io/etcd/pkg/v3/traceutil"
|
||||
"go.etcd.io/etcd/server/v3/etcdserver/cindex"
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func BenchmarkStorePut(b *testing.B) {
|
||||
be, tmpPath := backend.NewDefaultTmpBackend(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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) }()
|
||||
|
@ -23,7 +23,7 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/pkg/v3/traceutil"
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
@ -66,7 +66,7 @@ func TestScheduleCompaction(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
|
@ -36,15 +36,15 @@ import (
|
||||
"go.etcd.io/etcd/pkg/v3/traceutil"
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func TestStoreRev(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer s.Close()
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
for i := 1; i <= 3; i++ {
|
||||
s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
|
||||
@ -425,9 +425,8 @@ func TestRestoreDelete(t *testing.T) {
|
||||
restoreChunkKeys = mrand.Intn(3) + 2
|
||||
defer func() { restoreChunkKeys = oldChunk }()
|
||||
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
keys := make(map[string]struct{})
|
||||
for i := 0; i < 20; i++ {
|
||||
@ -473,9 +472,8 @@ func TestRestoreDelete(t *testing.T) {
|
||||
func TestRestoreContinueUnfinishedCompaction(t *testing.T) {
|
||||
tests := []string{"recreate", "restore"}
|
||||
for _, test := range tests {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, _ := betesting.NewDefaultTmpBackend(t)
|
||||
s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
s0.Put([]byte("foo"), []byte("bar"), lease.NoLease)
|
||||
s0.Put([]byte("foo"), []byte("bar1"), lease.NoLease)
|
||||
@ -535,7 +533,7 @@ type hashKVResult struct {
|
||||
|
||||
// TestHashKVWhenCompacting ensures that HashKV returns correct hash when compacting.
|
||||
func TestHashKVWhenCompacting(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
@ -603,7 +601,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
@ -636,7 +634,7 @@ func TestTxnPut(t *testing.T) {
|
||||
keys := createBytesSlice(bytesN, sliceN)
|
||||
vals := createBytesSlice(bytesN, sliceN)
|
||||
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
@ -652,7 +650,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
@ -721,7 +719,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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
defer os.Remove(tmpPath)
|
||||
|
||||
|
@ -22,13 +22,13 @@ import (
|
||||
"go.etcd.io/etcd/pkg/v3/traceutil"
|
||||
"go.etcd.io/etcd/server/v3/etcdserver/cindex"
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func BenchmarkWatchableStorePut(b *testing.B) {
|
||||
be, tmpPath := backend.NewDefaultTmpBackend(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.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(b)
|
||||
be, tmpPath := betesting.NewDefaultTmpBackend(b)
|
||||
s := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
|
||||
defer func() {
|
||||
|
@ -27,12 +27,12 @@ import (
|
||||
"go.etcd.io/etcd/pkg/v3/traceutil"
|
||||
"go.etcd.io/etcd/server/v3/etcdserver/cindex"
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func TestWatch(t *testing.T) {
|
||||
b, tmpPath := backend.NewDefaultTmpBackend(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
newBackend, newPath := betesting.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(t)
|
||||
b1, b1Path := betesting.NewDefaultTmpBackend(t)
|
||||
s1 := newWatchableStore(zap.NewExample(), b1, &lease.FakeLessor{}, cindex.NewConsistentIndex(b1.BatchTx()), StoreConfig{})
|
||||
defer cleanup(s1, b1, b1Path)
|
||||
|
||||
b2, b2Path := backend.NewDefaultTmpBackend(t)
|
||||
b2, b2Path := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
|
||||
defer func() {
|
||||
|
@ -19,13 +19,13 @@ import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
func BenchmarkKVWatcherMemoryUsage(b *testing.B) {
|
||||
be, tmpPath := backend.NewDefaultTmpBackend(b)
|
||||
be, tmpPath := betesting.NewDefaultTmpBackend(b)
|
||||
watchable := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
|
||||
|
||||
defer cleanup(watchable, be, tmpPath)
|
||||
|
@ -24,14 +24,14 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
"go.etcd.io/etcd/server/v3/lease"
|
||||
"go.etcd.io/etcd/server/v3/mvcc/backend"
|
||||
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// 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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.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(t)
|
||||
b, tmpPath := betesting.NewDefaultTmpBackend(t)
|
||||
s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
|
||||
defer cleanup(s, b, tmpPath)
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user