mvcc/backend tests: Refactor: Do not mix testing&prod code.

This commit is contained in:
Piotr Tabor
2021-04-09 15:49:50 +02:00
parent ea287dd9f8
commit d7d110b5a8
20 changed files with 257 additions and 196 deletions

View File

@ -98,6 +98,15 @@
} }
] ]
}, },
{
"project": "github.com/davecgh/go-spew/spew",
"licenses": [
{
"type": "ISC License",
"confidence": 0.9850746268656716
}
]
},
{ {
"project": "github.com/dustin/go-humanize", "project": "github.com/dustin/go-humanize",
"licenses": [ "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", "project": "github.com/tmc/grpc-websocket-proxy/wsproxy",
"licenses": [ "licenses": [

View File

@ -18,7 +18,6 @@ import (
"context" "context"
"encoding/base64" "encoding/base64"
"fmt" "fmt"
"os"
"reflect" "reflect"
"strings" "strings"
"sync" "sync"
@ -29,6 +28,7 @@ import (
pb "go.etcd.io/etcd/api/v3/etcdserverpb" pb "go.etcd.io/etcd/api/v3/etcdserverpb"
"go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
"go.etcd.io/etcd/server/v3/mvcc/backend" "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"
"golang.org/x/crypto/bcrypt" "golang.org/x/crypto/bcrypt"
@ -46,8 +46,7 @@ func dummyIndexWaiter(index uint64) <-chan struct{} {
// TestNewAuthStoreRevision ensures newly auth store // TestNewAuthStoreRevision ensures newly auth store
// keeps the old revision when there are no changes. // keeps the old revision when there are no changes.
func TestNewAuthStoreRevision(t *testing.T) { func TestNewAuthStoreRevision(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend(t) b, tPath := betesting.NewDefaultTmpBackend(t)
defer os.Remove(tPath)
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault) tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
if err != nil { if err != nil {
@ -76,9 +75,8 @@ func TestNewAuthStoreRevision(t *testing.T) {
// TestNewAuthStoreBryptCost ensures that NewAuthStore uses default when given bcrypt-cost is invalid // TestNewAuthStoreBryptCost ensures that NewAuthStore uses default when given bcrypt-cost is invalid
func TestNewAuthStoreBcryptCost(t *testing.T) { func TestNewAuthStoreBcryptCost(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
defer b.Close() defer betesting.Close(t, b)
defer os.Remove(tPath)
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault) tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
if err != nil { if err != nil {
@ -101,7 +99,7 @@ func encodePassword(s string) string {
} }
func setupAuthStore(t *testing.T) (store *authStore, teardownfunc func(t *testing.T)) { 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) tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
if err != nil { if err != nil {
@ -127,7 +125,6 @@ func setupAuthStore(t *testing.T) (store *authStore, teardownfunc func(t *testin
tearDown := func(_ *testing.T) { tearDown := func(_ *testing.T) {
b.Close() b.Close()
os.Remove(tPath)
as.Close() as.Close()
} }
return as, tearDown return as, tearDown
@ -653,9 +650,8 @@ func TestIsAuthEnabled(t *testing.T) {
// TestAuthRevisionRace ensures that access to authStore.revision is thread-safe. // TestAuthRevisionRace ensures that access to authStore.revision is thread-safe.
func TestAuthInfoFromCtxRace(t *testing.T) { func TestAuthInfoFromCtxRace(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
defer b.Close() defer betesting.Close(t, b)
defer os.Remove(tPath)
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault) tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
if err != nil { if err != nil {
@ -807,9 +803,8 @@ func TestHammerSimpleAuthenticate(t *testing.T) {
// TestRolesOrder tests authpb.User.Roles is sorted // TestRolesOrder tests authpb.User.Roles is sorted
func TestRolesOrder(t *testing.T) { func TestRolesOrder(t *testing.T) {
b, tPath := backend.NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
defer b.Close() defer betesting.Close(t, b)
defer os.Remove(tPath)
tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault) tp, err := NewTokenProvider(zap.NewExample(), tokenTypeSimple, dummyIndexWaiter, simpleTokenTTLDefault)
defer tp.disable() defer tp.disable()
@ -865,9 +860,8 @@ func TestAuthInfoFromCtxWithRootJWT(t *testing.T) {
// testAuthInfoFromCtxWithRoot ensures "WithRoot" properly embeds token in the context. // testAuthInfoFromCtxWithRoot ensures "WithRoot" properly embeds token in the context.
func testAuthInfoFromCtxWithRoot(t *testing.T, opts string) { func testAuthInfoFromCtxWithRoot(t *testing.T, opts string) {
b, tPath := backend.NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
defer b.Close() defer betesting.Close(t, b)
defer os.Remove(tPath)
tp, err := NewTokenProvider(zap.NewExample(), opts, dummyIndexWaiter, simpleTokenTTLDefault) tp, err := NewTokenProvider(zap.NewExample(), opts, dummyIndexWaiter, simpleTokenTTLDefault)
if err != nil { if err != nil {

View File

@ -16,18 +16,17 @@ package cindex
import ( import (
"math/rand" "math/rand"
"os"
"testing" "testing"
"time" "time"
"go.etcd.io/etcd/server/v3/mvcc/backend" "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. // TestConsistentIndex ensures that LoadConsistentIndex/Save/ConsistentIndex and backend.BatchTx can work well together.
func TestConsistentIndex(t *testing.T) { func TestConsistentIndex(t *testing.T) {
be, tmpPath := backend.NewTmpBackend(t, time.Microsecond, 10) be, tmpPath := betesting.NewTmpBackend(t, time.Microsecond, 10)
defer os.Remove(tmpPath)
ci := NewConsistentIndex(be.BatchTx()) ci := NewConsistentIndex(be.BatchTx())
tx := be.BatchTx() tx := be.BatchTx()

View File

@ -50,7 +50,7 @@ import (
"go.etcd.io/etcd/server/v3/mock/mockstore" "go.etcd.io/etcd/server/v3/mock/mockstore"
"go.etcd.io/etcd/server/v3/mock/mockwait" "go.etcd.io/etcd/server/v3/mock/mockwait"
"go.etcd.io/etcd/server/v3/mvcc" "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"
"go.uber.org/zap/zaptest" "go.uber.org/zap/zaptest"
) )
@ -972,10 +972,7 @@ func TestSyncTrigger(t *testing.T) {
// snapshot should snapshot the store and cut the persistent // snapshot should snapshot the store and cut the persistent
func TestSnapshot(t *testing.T) { func TestSnapshot(t *testing.T) {
be, tmpPath := backend.NewDefaultTmpBackend(t) be, _ := betesting.NewDefaultTmpBackend(t)
defer func() {
os.RemoveAll(tmpPath)
}()
s := raft.NewMemoryStorage() s := raft.NewMemoryStorage()
s.Append([]raftpb.Entry{{Index: 1}}) s.Append([]raftpb.Entry{{Index: 1}})
@ -1066,7 +1063,7 @@ func TestSnapshotOrdering(t *testing.T) {
storage: p, storage: p,
raftStorage: rs, raftStorage: rs,
}) })
be, tmpPath := backend.NewDefaultTmpBackend(t) be, tmpPath := betesting.NewDefaultTmpBackend(t)
defer os.RemoveAll(tmpPath) defer os.RemoveAll(tmpPath)
s := &EtcdServer{ s := &EtcdServer{
lgMu: new(sync.RWMutex), lgMu: new(sync.RWMutex),
@ -1128,7 +1125,7 @@ func TestSnapshotOrdering(t *testing.T) {
// Applied > SnapshotCount should trigger a SaveSnap event // Applied > SnapshotCount should trigger a SaveSnap event
func TestTriggerSnap(t *testing.T) { func TestTriggerSnap(t *testing.T) {
be, tmpPath := backend.NewDefaultTmpBackend(t) be, tmpPath := betesting.NewDefaultTmpBackend(t)
defer func() { defer func() {
os.RemoveAll(tmpPath) os.RemoveAll(tmpPath)
}() }()
@ -1217,7 +1214,7 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
storage: mockstorage.NewStorageRecorder(testdir), storage: mockstorage.NewStorageRecorder(testdir),
raftStorage: rs, raftStorage: rs,
}) })
be, tmpPath := backend.NewDefaultTmpBackend(t) be, tmpPath := betesting.NewDefaultTmpBackend(t)
defer func() { defer func() {
os.RemoveAll(tmpPath) os.RemoveAll(tmpPath)
}() }()
@ -1552,7 +1549,7 @@ func TestPublishV3(t *testing.T) {
w := wait.NewWithResponse(ch) w := wait.NewWithResponse(ch)
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
lg := zaptest.NewLogger(t) lg := zaptest.NewLogger(t)
be, _ := backend.NewDefaultTmpBackend(t) be, _ := betesting.NewDefaultTmpBackend(t)
srv := &EtcdServer{ srv := &EtcdServer{
lgMu: new(sync.RWMutex), lgMu: new(sync.RWMutex),
lg: lg, lg: lg,
@ -1622,7 +1619,7 @@ func TestPublishV3Retry(t *testing.T) {
n := newNodeRecorderStream() n := newNodeRecorderStream()
lg := zaptest.NewLogger(t) lg := zaptest.NewLogger(t)
be, _ := backend.NewDefaultTmpBackend(t) be, _ := betesting.NewDefaultTmpBackend(t)
srv := &EtcdServer{ srv := &EtcdServer{
lgMu: new(sync.RWMutex), lgMu: new(sync.RWMutex),
lg: lg, lg: lg,

View File

@ -18,20 +18,18 @@ import (
"context" "context"
"net/http" "net/http"
"net/http/httptest" "net/http/httptest"
"os"
"testing" "testing"
"time" "time"
"go.etcd.io/etcd/server/v3/lease" "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" "go.uber.org/zap/zaptest"
) )
func TestRenewHTTP(t *testing.T) { func TestRenewHTTP(t *testing.T) {
lg := zap.NewNop() lg := zaptest.NewLogger(t)
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000) be, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath) defer betesting.Close(t, be)
defer be.Close()
le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil) le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil)
le.Promote(time.Second) le.Promote(time.Second)
@ -53,10 +51,9 @@ func TestRenewHTTP(t *testing.T) {
} }
func TestTimeToLiveHTTP(t *testing.T) { func TestTimeToLiveHTTP(t *testing.T) {
lg := zap.NewNop() lg := zaptest.NewLogger(t)
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000) be, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath) defer betesting.Close(t, be)
defer be.Close()
le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil) le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil)
le.Promote(time.Second) le.Promote(time.Second)
@ -95,10 +92,9 @@ func TestTimeToLiveHTTPTimeout(t *testing.T) {
} }
func testApplyTimeout(t *testing.T, f func(*lease.Lease, string) error) { func testApplyTimeout(t *testing.T, f func(*lease.Lease, string) error) {
lg := zap.NewNop() lg := zaptest.NewLogger(t)
be, tmpPath := backend.NewTmpBackend(t, time.Hour, 10000) be, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath) defer betesting.Close(t, be)
defer be.Close()
le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil) le := lease.NewLessor(lg, be, lease.LessorConfig{MinLeaseTTL: int64(5)}, nil)
le.Promote(time.Second) le.Promote(time.Second)

View File

@ -19,7 +19,7 @@ import (
"testing" "testing"
"time" "time"
"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"
) )
@ -65,7 +65,7 @@ func demote(le *lessor) {
// return new lessor and tearDown to release resource // return new lessor and tearDown to release resource
func setUp(t testing.TB) (le *lessor, tearDown func()) { func setUp(t testing.TB) (le *lessor, tearDown func()) {
lg := zap.NewNop() lg := zap.NewNop()
be, _ := backend.NewDefaultTmpBackend(t) be, _ := betesting.NewDefaultTmpBackend(t)
// MinLeaseTTL is negative, so we can grant expired lease in benchmark. // MinLeaseTTL is negative, so we can grant expired lease in benchmark.
// ExpiredLeasesRetryInterval should small, so benchmark of findExpired will recheck expired lease. // ExpiredLeasesRetryInterval should small, so benchmark of findExpired will recheck expired lease.
le = newLessor(lg, be, LessorConfig{MinLeaseTTL: -1000, ExpiredLeasesRetryInterval: 10 * time.Microsecond}, nil) le = newLessor(lg, be, LessorConfig{MinLeaseTTL: -1000, ExpiredLeasesRetryInterval: 10 * time.Microsecond}, nil)

View File

@ -23,7 +23,6 @@ import (
"path/filepath" "path/filepath"
"sync" "sync"
"sync/atomic" "sync/atomic"
"testing"
"time" "time"
humanize "github.com/dustin/go-humanize" humanize "github.com/dustin/go-humanize"
@ -544,22 +543,6 @@ func (b *backend) OpenReadTxN() int64 {
return atomic.LoadInt64(&b.openReadTxN) 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 { type snapshot struct {
*bolt.Tx *bolt.Tx
stopc chan struct{} stopc chan struct{}

View File

@ -12,28 +12,31 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package backend package backend_test
import ( import (
"crypto/rand" "crypto/rand"
"os"
"testing" "testing"
"time" "time"
"github.com/stretchr/testify/assert"
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
) )
func BenchmarkBackendPut(b *testing.B) { func BenchmarkBackendPut(b *testing.B) {
backend, tmppath := NewTmpBackend(b, 100*time.Millisecond, 10000) backend, _ := betesting.NewTmpBackend(b, 100*time.Millisecond, 10000)
defer backend.Close() defer betesting.Close(b, backend)
defer os.Remove(tmppath)
// prepare keys // prepare keys
keys := make([][]byte, b.N) keys := make([][]byte, b.N)
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
keys[i] = make([]byte, 64) keys[i] = make([]byte, 64)
rand.Read(keys[i]) _, err := rand.Read(keys[i])
assert.NoError(b, err)
} }
value := make([]byte, 128) value := make([]byte, 128)
rand.Read(value) _, err := rand.Read(value)
assert.NoError(b, err)
batchTx := backend.BatchTx() batchTx := backend.BatchTx()

View File

@ -12,22 +12,23 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package backend package backend_test
import ( import (
"fmt" "fmt"
"io/ioutil" "io/ioutil"
"os"
"reflect" "reflect"
"testing" "testing"
"time" "time"
"github.com/stretchr/testify/assert"
bolt "go.etcd.io/bbolt" 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) { func TestBackendClose(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer os.Remove(tmpPath)
// check close could work // check close could work
done := make(chan struct{}) done := make(chan struct{})
@ -46,8 +47,8 @@ func TestBackendClose(t *testing.T) {
} }
func TestBackendSnapshot(t *testing.T) { func TestBackendSnapshot(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.BatchTx() tx := b.BatchTx()
tx.Lock() tx.Lock()
@ -57,22 +58,22 @@ func TestBackendSnapshot(t *testing.T) {
b.ForceCommit() b.ForceCommit()
// write snapshot to a new file // 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 { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
snap := b.Snapshot() snap := b.Snapshot()
defer snap.Close() defer func() { assert.NoError(t, snap.Close()) }()
if _, err := snap.WriteTo(f); err != nil { if _, err := snap.WriteTo(f); err != nil {
t.Fatal(err) t.Fatal(err)
} }
f.Close() assert.NoError(t, f.Close())
// bootstrap new backend from the snapshot // bootstrap new backend from the snapshot
bcfg := DefaultBackendConfig() bcfg := backend.DefaultBackendConfig()
bcfg.Path, bcfg.BatchInterval, bcfg.BatchLimit = f.Name(), time.Hour, 10000 bcfg.Path, bcfg.BatchInterval, bcfg.BatchLimit = f.Name(), time.Hour, 10000
nb := New(bcfg) nb := backend.New(bcfg)
defer cleanup(nb, f.Name()) defer betesting.Close(t, nb)
newTx := nb.BatchTx() newTx := nb.BatchTx()
newTx.Lock() newTx.Lock()
@ -86,10 +87,10 @@ func TestBackendSnapshot(t *testing.T) {
func TestBackendBatchIntervalCommit(t *testing.T) { func TestBackendBatchIntervalCommit(t *testing.T) {
// start backend with super short batch interval so // start backend with super short batch interval so
// we do not need to wait long before commit to happen. // we do not need to wait long before commit to happen.
b, tmpPath := NewTmpBackend(t, time.Nanosecond, 10000) b, _ := betesting.NewTmpBackend(t, time.Nanosecond, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
pc := b.Commits() pc := backend.CommitsForTest(b)
tx := b.BatchTx() tx := b.BatchTx()
tx.Lock() tx.Lock()
@ -98,14 +99,14 @@ func TestBackendBatchIntervalCommit(t *testing.T) {
tx.Unlock() tx.Unlock()
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
if b.Commits() >= pc+1 { if backend.CommitsForTest(b) >= pc+1 {
break break
} }
time.Sleep(time.Duration(i*100) * time.Millisecond) time.Sleep(time.Duration(i*100) * time.Millisecond)
} }
// check whether put happens via db view // 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")) bucket := tx.Bucket([]byte("test"))
if bucket == nil { if bucket == nil {
t.Errorf("bucket test does not exit") 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") t.Errorf("foo key failed to written in backend")
} }
return nil return nil
}) }))
} }
func TestBackendDefrag(t *testing.T) { func TestBackendDefrag(t *testing.T) {
b, tmpPath := NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.BatchTx() tx := b.BatchTx()
tx.Lock() tx.Lock()
tx.UnsafeCreateBucket([]byte("test")) 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.UnsafePut([]byte("test"), []byte(fmt.Sprintf("foo_%d", i)), []byte("bar"))
} }
tx.Unlock() tx.Unlock()
@ -178,8 +179,8 @@ func TestBackendDefrag(t *testing.T) {
// TestBackendWriteback ensures writes are stored to the read txn on write txn unlock. // TestBackendWriteback ensures writes are stored to the read txn on write txn unlock.
func TestBackendWriteback(t *testing.T) { func TestBackendWriteback(t *testing.T) {
b, tmpPath := NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.BatchTx() tx := b.BatchTx()
tx.Lock() 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 // TestConcurrentReadTx ensures that current read transaction can see all prior writes stored in read buffer
func TestConcurrentReadTx(t *testing.T) { func TestConcurrentReadTx(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
wtx1 := b.BatchTx() wtx1 := b.BatchTx()
wtx1.Lock() wtx1.Lock()
@ -282,8 +283,8 @@ func TestConcurrentReadTx(t *testing.T) {
// TestBackendWritebackForEach checks that partially written / buffered // TestBackendWritebackForEach checks that partially written / buffered
// data is visited in the same order as fully committed data. // data is visited in the same order as fully committed data.
func TestBackendWritebackForEach(t *testing.T) { func TestBackendWritebackForEach(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.BatchTx() tx := b.BatchTx()
tx.Lock() tx.Lock()
@ -312,7 +313,7 @@ func TestBackendWritebackForEach(t *testing.T) {
} }
rtx := b.ReadTx() rtx := b.ReadTx()
rtx.RLock() rtx.RLock()
rtx.UnsafeForEach([]byte("key"), getSeq) assert.NoError(t, rtx.UnsafeForEach([]byte("key"), getSeq))
rtx.RUnlock() rtx.RUnlock()
partialSeq := seq partialSeq := seq
@ -321,15 +322,10 @@ func TestBackendWritebackForEach(t *testing.T) {
b.ForceCommit() b.ForceCommit()
tx.Lock() tx.Lock()
tx.UnsafeForEach([]byte("key"), getSeq) assert.NoError(t, tx.UnsafeForEach([]byte("key"), getSeq))
tx.Unlock() tx.Unlock()
if seq != partialSeq { if seq != partialSeq {
t.Fatalf("expected %q, got %q", seq, partialSeq) t.Fatalf("expected %q, got %q", seq, partialSeq)
} }
} }
func cleanup(b Backend, path string) {
b.Close()
os.Remove(path)
}

View File

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package backend package backend_test
import ( import (
"reflect" "reflect"
@ -20,15 +20,17 @@ import (
"time" "time"
bolt "go.etcd.io/bbolt" 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) { func TestBatchTxPut(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.BatchTx()
tx := b.batchTx
tx.Lock() tx.Lock()
defer tx.Unlock()
// create bucket // create bucket
tx.UnsafeCreateBucket([]byte("test")) tx.UnsafeCreateBucket([]byte("test"))
@ -37,21 +39,25 @@ func TestBatchTxPut(t *testing.T) {
v := []byte("bar") v := []byte("bar")
tx.UnsafePut([]byte("test"), []byte("foo"), v) tx.UnsafePut([]byte("test"), []byte("foo"), v)
tx.Unlock()
// check put result before and after tx is committed // check put result before and after tx is committed
for k := 0; k < 2; k++ { for k := 0; k < 2; k++ {
tx.Lock()
_, gv := tx.UnsafeRange([]byte("test"), []byte("foo"), nil, 0) _, gv := tx.UnsafeRange([]byte("test"), []byte("foo"), nil, 0)
tx.Unlock()
if !reflect.DeepEqual(gv[0], v) { if !reflect.DeepEqual(gv[0], v) {
t.Errorf("v = %s, want %s", string(gv[0]), string(v)) t.Errorf("v = %s, want %s", string(gv[0]), string(v))
} }
tx.commit(false) tx.Commit()
} }
} }
func TestBatchTxRange(t *testing.T) { func TestBatchTxRange(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.batchTx tx := b.BatchTx()
tx.Lock() tx.Lock()
defer tx.Unlock() defer tx.Unlock()
@ -119,33 +125,36 @@ func TestBatchTxRange(t *testing.T) {
} }
func TestBatchTxDelete(t *testing.T) { func TestBatchTxDelete(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.batchTx tx := b.BatchTx()
tx.Lock() tx.Lock()
defer tx.Unlock()
tx.UnsafeCreateBucket([]byte("test")) tx.UnsafeCreateBucket([]byte("test"))
tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar")) tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar"))
tx.UnsafeDelete([]byte("test"), []byte("foo")) tx.UnsafeDelete([]byte("test"), []byte("foo"))
tx.Unlock()
// check put result before and after tx is committed // check put result before and after tx is committed
for k := 0; k < 2; k++ { for k := 0; k < 2; k++ {
tx.Lock()
ks, _ := tx.UnsafeRange([]byte("test"), []byte("foo"), nil, 0) ks, _ := tx.UnsafeRange([]byte("test"), []byte("foo"), nil, 0)
tx.Unlock()
if len(ks) != 0 { if len(ks) != 0 {
t.Errorf("keys on foo = %v, want nil", ks) t.Errorf("keys on foo = %v, want nil", ks)
} }
tx.commit(false) tx.Commit()
} }
} }
func TestBatchTxCommit(t *testing.T) { func TestBatchTxCommit(t *testing.T) {
b, tmpPath := NewTmpBackend(t, time.Hour, 10000) b, _ := betesting.NewTmpBackend(t, time.Hour, 10000)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.batchTx tx := b.BatchTx()
tx.Lock() tx.Lock()
tx.UnsafeCreateBucket([]byte("test")) tx.UnsafeCreateBucket([]byte("test"))
tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar")) tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar"))
@ -154,7 +163,7 @@ func TestBatchTxCommit(t *testing.T) {
tx.Commit() tx.Commit()
// check whether put happens via db view // 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")) bucket := tx.Bucket([]byte("test"))
if bucket == nil { if bucket == nil {
t.Errorf("bucket test does not exit") t.Errorf("bucket test does not exit")
@ -171,10 +180,10 @@ func TestBatchTxCommit(t *testing.T) {
func TestBatchTxBatchLimitCommit(t *testing.T) { func TestBatchTxBatchLimitCommit(t *testing.T) {
// start backend with batch limit 1 so one write can // start backend with batch limit 1 so one write can
// trigger a commit // trigger a commit
b, tmpPath := NewTmpBackend(t, time.Hour, 1) b, _ := betesting.NewTmpBackend(t, time.Hour, 1)
defer cleanup(b, tmpPath) defer betesting.Close(t, b)
tx := b.batchTx tx := b.BatchTx()
tx.Lock() tx.Lock()
tx.UnsafeCreateBucket([]byte("test")) tx.UnsafeCreateBucket([]byte("test"))
tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar")) tx.UnsafePut([]byte("test"), []byte("foo"), []byte("bar"))
@ -182,7 +191,7 @@ func TestBatchTxBatchLimitCommit(t *testing.T) {
// batch limit commit should have been triggered // batch limit commit should have been triggered
// check whether put happens via db view // 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")) bucket := tx.Bucket([]byte("test"))
if bucket == nil { if bucket == nil {
t.Errorf("bucket test does not exit") t.Errorf("bucket test does not exit")

View 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()
}

View 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())
}

View File

@ -27,6 +27,7 @@ import (
"go.etcd.io/etcd/pkg/v3/traceutil" "go.etcd.io/etcd/pkg/v3/traceutil"
"go.etcd.io/etcd/server/v3/lease" "go.etcd.io/etcd/server/v3/lease"
"go.etcd.io/etcd/server/v3/mvcc/backend" "go.etcd.io/etcd/server/v3/mvcc/backend"
betesting "go.etcd.io/etcd/server/v3/mvcc/backend/testing"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
dto "github.com/prometheus/client_model/go" 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 TestKVTxnRange(t *testing.T) { testKVRange(t, txnRangeFunc) }
func testKVRange(t *testing.T, f rangeFunc) { 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) 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 TestKVTxnRangeRev(t *testing.T) { testKVRangeRev(t, txnRangeFunc) }
func testKVRangeRev(t *testing.T, f rangeFunc) { 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) 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 TestKVTxnRangeBadRev(t *testing.T) { testKVRangeBadRev(t, txnRangeFunc) }
func testKVRangeBadRev(t *testing.T, f rangeFunc) { 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) 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 TestKVTxnRangeLimit(t *testing.T) { testKVRangeLimit(t, txnRangeFunc) }
func testKVRangeLimit(t *testing.T, f rangeFunc) { 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) 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 TestKVTxnPutMultipleTimes(t *testing.T) { testKVPutMultipleTimes(t, txnPutFunc) }
func testKVPutMultipleTimes(t *testing.T, f putFunc) { 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)
@ -319,7 +320,7 @@ func testKVDeleteRange(t *testing.T, f deleteRangeFunc) {
} }
for i, tt := range tests { 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 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
s.Put([]byte("foo"), []byte("bar"), lease.NoLease) 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 TestKVTxnDeleteMultipleTimes(t *testing.T) { testKVDeleteMultipleTimes(t, txnDeleteRangeFunc) }
func testKVDeleteMultipleTimes(t *testing.T, f deleteRangeFunc) { 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) 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. // test that range, put, delete on single key in sequence repeatedly works correctly.
func TestKVOperationInSequence(t *testing.T) { func TestKVOperationInSequence(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, tmpPath := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)
@ -407,7 +408,7 @@ func TestKVOperationInSequence(t *testing.T) {
} }
func TestKVTxnBlockWriteOperations(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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
tests := []func(){ tests := []func(){
@ -441,7 +442,7 @@ func TestKVTxnBlockWriteOperations(t *testing.T) {
} }
func TestKVTxnNonBlockRange(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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) 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. // test that txn range, put, delete on single key in sequence repeatedly works correctly.
func TestKVTxnOperationInSequence(t *testing.T) { func TestKVTxnOperationInSequence(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, tmpPath := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)
@ -512,7 +513,7 @@ func TestKVTxnOperationInSequence(t *testing.T) {
} }
func TestKVCompactReserveLastValue(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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)
@ -566,7 +567,7 @@ func TestKVCompactReserveLastValue(t *testing.T) {
} }
func TestKVCompactBad(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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)
@ -599,7 +600,7 @@ func TestKVHash(t *testing.T) {
for i := 0; i < len(hashes); i++ { for i := 0; i < len(hashes); i++ {
var err error var err error
b, tmpPath := backend.NewDefaultTmpBackend(t) b, tmpPath := betesting.NewDefaultTmpBackend(t)
kv := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) kv := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
kv.Put([]byte("foo0"), []byte("bar0"), lease.NoLease) kv.Put([]byte("foo0"), []byte("bar0"), lease.NoLease)
kv.Put([]byte("foo1"), []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 { 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 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
tt(s) tt(s)
var kvss [][]mvccpb.KeyValue var kvss [][]mvccpb.KeyValue
@ -681,7 +682,7 @@ func readGaugeInt(g prometheus.Gauge) int {
} }
func TestKVSnapshot(t *testing.T) { func TestKVSnapshot(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, tmpPath := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)
@ -717,7 +718,7 @@ func TestKVSnapshot(t *testing.T) {
} }
func TestWatchableKVWatch(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{})) s := WatchableKV(newWatchableStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}))
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)

View File

@ -21,13 +21,13 @@ import (
"go.etcd.io/etcd/pkg/v3/traceutil" "go.etcd.io/etcd/pkg/v3/traceutil"
"go.etcd.io/etcd/server/v3/etcdserver/cindex" "go.etcd.io/etcd/server/v3/etcdserver/cindex"
"go.etcd.io/etcd/server/v3/lease" "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" "go.uber.org/zap"
) )
func BenchmarkStorePut(b *testing.B) { 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{}) s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath) 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 BenchmarkStoreRangeKey100(b *testing.B) { benchmarkStoreRange(b, 100) }
func benchmarkStoreRange(b *testing.B, n int) { 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{}) s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath) defer cleanup(s, be, tmpPath)
@ -73,7 +73,7 @@ func benchmarkStoreRange(b *testing.B, n int) {
} }
func BenchmarkConsistentIndex(b *testing.B) { 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{}) s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath) defer cleanup(s, be, tmpPath)
@ -91,7 +91,7 @@ func BenchmarkConsistentIndex(b *testing.B) {
// BenchmarkStoreTxnPutUpdate is same as above, but instead updates single key // BenchmarkStoreTxnPutUpdate is same as above, but instead updates single key
func BenchmarkStorePutUpdate(b *testing.B) { 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{}) s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath) defer cleanup(s, be, tmpPath)
@ -109,7 +109,7 @@ func BenchmarkStorePutUpdate(b *testing.B) {
// with transaction begin and end, where transaction involves // with transaction begin and end, where transaction involves
// some synchronization operations, such as mutex locking. // some synchronization operations, such as mutex locking.
func BenchmarkStoreTxnPut(b *testing.B) { 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{}) s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
defer cleanup(s, be, tmpPath) defer cleanup(s, be, tmpPath)
@ -129,7 +129,7 @@ func BenchmarkStoreTxnPut(b *testing.B) {
// benchmarkStoreRestore benchmarks the restore operation // benchmarkStoreRestore benchmarks the restore operation
func benchmarkStoreRestore(revsPerKey int, b *testing.B) { 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{}) s := NewStore(zap.NewExample(), be, &lease.FakeLessor{}, cindex.NewConsistentIndex(be.BatchTx()), StoreConfig{})
// use closure to capture 's' to pick up the reassignment // use closure to capture 's' to pick up the reassignment
defer func() { cleanup(s, be, tmpPath) }() defer func() { cleanup(s, be, tmpPath) }()

View File

@ -23,7 +23,7 @@ import (
"go.etcd.io/etcd/pkg/v3/traceutil" "go.etcd.io/etcd/pkg/v3/traceutil"
"go.etcd.io/etcd/server/v3/lease" "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" "go.uber.org/zap"
) )
@ -66,7 +66,7 @@ func TestScheduleCompaction(t *testing.T) {
}, },
} }
for i, tt := range tests { 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 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
tx := s.b.BatchTx() tx := s.b.BatchTx()
@ -100,7 +100,7 @@ func TestScheduleCompaction(t *testing.T) {
} }
func TestCompactAllAndRestore(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{}) s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath) defer os.Remove(tmpPath)

View File

@ -36,15 +36,15 @@ import (
"go.etcd.io/etcd/pkg/v3/traceutil" "go.etcd.io/etcd/pkg/v3/traceutil"
"go.etcd.io/etcd/server/v3/lease" "go.etcd.io/etcd/server/v3/lease"
"go.etcd.io/etcd/server/v3/mvcc/backend" "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"
) )
func TestStoreRev(t *testing.T) { func TestStoreRev(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer s.Close() defer s.Close()
defer os.Remove(tmpPath)
for i := 1; i <= 3; i++ { for i := 1; i <= 3; i++ {
s.Put([]byte("foo"), []byte("bar"), lease.NoLease) s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
@ -425,9 +425,8 @@ func TestRestoreDelete(t *testing.T) {
restoreChunkKeys = mrand.Intn(3) + 2 restoreChunkKeys = mrand.Intn(3) + 2
defer func() { restoreChunkKeys = oldChunk }() defer func() { restoreChunkKeys = oldChunk }()
b, tmpPath := backend.NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath)
keys := make(map[string]struct{}) keys := make(map[string]struct{})
for i := 0; i < 20; i++ { for i := 0; i < 20; i++ {
@ -473,9 +472,8 @@ func TestRestoreDelete(t *testing.T) {
func TestRestoreContinueUnfinishedCompaction(t *testing.T) { func TestRestoreContinueUnfinishedCompaction(t *testing.T) {
tests := []string{"recreate", "restore"} tests := []string{"recreate", "restore"}
for _, test := range tests { for _, test := range tests {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, _ := betesting.NewDefaultTmpBackend(t)
s0 := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) 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("bar"), lease.NoLease)
s0.Put([]byte("foo"), []byte("bar1"), 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. // TestHashKVWhenCompacting ensures that HashKV returns correct hash when compacting.
func TestHashKVWhenCompacting(t *testing.T) { func TestHashKVWhenCompacting(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, tmpPath := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath) defer os.Remove(tmpPath)
@ -603,7 +601,7 @@ func TestHashKVWhenCompacting(t *testing.T) {
// TestHashKVZeroRevision ensures that "HashByRev(0)" computes // TestHashKVZeroRevision ensures that "HashByRev(0)" computes
// correct hash value with latest revision. // correct hash value with latest revision.
func TestHashKVZeroRevision(t *testing.T) { func TestHashKVZeroRevision(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, tmpPath := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath) defer os.Remove(tmpPath)
@ -636,7 +634,7 @@ func TestTxnPut(t *testing.T) {
keys := createBytesSlice(bytesN, sliceN) keys := createBytesSlice(bytesN, sliceN)
vals := 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(s, b, tmpPath) defer cleanup(s, b, tmpPath)
@ -652,7 +650,7 @@ func TestTxnPut(t *testing.T) {
// TestConcurrentReadNotBlockingWrite ensures Read does not blocking Write after its creation // TestConcurrentReadNotBlockingWrite ensures Read does not blocking Write after its creation
func TestConcurrentReadNotBlockingWrite(t *testing.T) { func TestConcurrentReadNotBlockingWrite(t *testing.T) {
b, tmpPath := backend.NewDefaultTmpBackend(t) b, tmpPath := betesting.NewDefaultTmpBackend(t)
s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath) 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 committedKVs kvs // committedKVs records the key-value pairs written by the finished Write Txns
mu sync.Mutex // mu protects committedKVs 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{}) s := NewStore(zap.NewExample(), b, &lease.FakeLessor{}, nil, StoreConfig{})
defer os.Remove(tmpPath) defer os.Remove(tmpPath)

View File

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

View File

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

View File

@ -19,13 +19,13 @@ import (
"testing" "testing"
"go.etcd.io/etcd/server/v3/lease" "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" "go.uber.org/zap"
) )
func BenchmarkKVWatcherMemoryUsage(b *testing.B) { func BenchmarkKVWatcherMemoryUsage(b *testing.B) {
be, tmpPath := backend.NewDefaultTmpBackend(b) be, tmpPath := betesting.NewDefaultTmpBackend(b)
watchable := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{}) watchable := newWatchableStore(zap.NewExample(), be, &lease.FakeLessor{}, nil, StoreConfig{})
defer cleanup(watchable, be, tmpPath) defer cleanup(watchable, be, tmpPath)

View File

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