mvcc/backend tests: Refactor: Do not mix testing&prod code.
This commit is contained in:
@ -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": [
|
||||||
|
@ -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 {
|
||||||
|
@ -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()
|
||||||
|
@ -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,
|
||||||
|
@ -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)
|
||||||
|
@ -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)
|
||||||
|
@ -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{}
|
||||||
|
@ -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()
|
||||||
|
|
||||||
|
@ -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)
|
|
||||||
}
|
|
||||||
|
@ -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")
|
||||||
|
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/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)
|
||||||
|
|
||||||
|
@ -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) }()
|
||||||
|
@ -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)
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
|
||||||
|
@ -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() {
|
||||||
|
@ -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() {
|
||||||
|
@ -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)
|
||||||
|
@ -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)
|
||||||
|
|
||||||
|
Reference in New Issue
Block a user