mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #13621 from serathius/integration-v2-api
Remove V2 API usage from Integration tests
This commit is contained in:
commit
a1fb9ff1e4
@ -17,9 +17,10 @@ package membership
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"go.etcd.io/etcd/client/pkg/v3/types"
|
||||
"path"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/types"
|
||||
|
||||
"go.etcd.io/etcd/server/v3/etcdserver/api/v2store"
|
||||
|
||||
"github.com/coreos/go-semver/semver"
|
||||
|
78
tests/e2e/discovery_test.go
Normal file
78
tests/e2e/discovery_test.go
Normal file
@ -0,0 +1,78 @@
|
||||
// Copyright 2022 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 e2e
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
||||
"go.etcd.io/etcd/client/v2"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
"go.etcd.io/etcd/tests/v3/framework/integration"
|
||||
)
|
||||
|
||||
func TestClusterOf1UsingDiscovery(t *testing.T) { testClusterUsingDiscovery(t, 1, false) }
|
||||
func TestClusterOf3UsingDiscovery(t *testing.T) { testClusterUsingDiscovery(t, 3, false) }
|
||||
func TestTLSClusterOf3UsingDiscovery(t *testing.T) { testClusterUsingDiscovery(t, 3, true) }
|
||||
|
||||
func testClusterUsingDiscovery(t *testing.T, size int, peerTLS bool) {
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
lastReleaseBinary := e2e.BinDir + "/etcd-last-release"
|
||||
if !fileutil.Exist(lastReleaseBinary) {
|
||||
t.Skipf("%q does not exist", lastReleaseBinary)
|
||||
}
|
||||
|
||||
dc, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
BasePort: 2000,
|
||||
ExecPath: lastReleaseBinary,
|
||||
ClusterSize: 1,
|
||||
EnableV2: true,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
defer dc.Close()
|
||||
|
||||
dcc := integration.MustNewHTTPClient(t, dc.EndpointsV2(), nil)
|
||||
dkapi := client.NewKeysAPI(dcc)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
if _, err := dkapi.Create(ctx, "/_config/size", fmt.Sprintf("%d", size)); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
cancel()
|
||||
|
||||
c, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
BasePort: 3000,
|
||||
ClusterSize: size,
|
||||
IsPeerTLS: peerTLS,
|
||||
Discovery: dc.EndpointsV2()[0] + "/v2/keys",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
kubectl := []string{e2e.CtlBinPath, "--endpoints", strings.Join(c.EndpointsV3(), ",")}
|
||||
if err := e2e.SpawnWithExpect(append(kubectl, "put", "key", "value"), "OK"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := e2e.SpawnWithExpect(append(kubectl, "get", "key"), "value"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
@ -170,6 +170,7 @@ type EtcdProcessClusterConfig struct {
|
||||
V2deprecation string
|
||||
|
||||
RollingStart bool
|
||||
Discovery string
|
||||
}
|
||||
|
||||
// NewEtcdProcessCluster launches a new cluster from etcd processes, returning
|
||||
@ -273,6 +274,7 @@ func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*
|
||||
"--data-dir", dataDirPath,
|
||||
"--snapshot-count", fmt.Sprintf("%d", cfg.SnapshotCount),
|
||||
}
|
||||
|
||||
if cfg.ForceNewCluster {
|
||||
args = append(args, "--force-new-cluster")
|
||||
}
|
||||
@ -309,6 +311,10 @@ func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*
|
||||
args = append(args, "--v2-deprecation", cfg.V2deprecation)
|
||||
}
|
||||
|
||||
if cfg.Discovery != "" {
|
||||
args = append(args, "--discovery", cfg.Discovery)
|
||||
}
|
||||
|
||||
etcdCfgs[i] = &EtcdServerProcessConfig{
|
||||
lg: lg,
|
||||
ExecPath: cfg.ExecPath,
|
||||
@ -325,10 +331,12 @@ func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*
|
||||
}
|
||||
}
|
||||
|
||||
initialClusterArgs := []string{"--initial-cluster", strings.Join(initialCluster, ",")}
|
||||
for i := range etcdCfgs {
|
||||
etcdCfgs[i].InitialCluster = strings.Join(initialCluster, ",")
|
||||
etcdCfgs[i].Args = append(etcdCfgs[i].Args, initialClusterArgs...)
|
||||
if cfg.Discovery == "" {
|
||||
for i := range etcdCfgs {
|
||||
initialClusterArgs := []string{"--initial-cluster", strings.Join(initialCluster, ",")}
|
||||
etcdCfgs[i].InitialCluster = strings.Join(initialCluster, ",")
|
||||
etcdCfgs[i].Args = append(etcdCfgs[i].Args, initialClusterArgs...)
|
||||
}
|
||||
}
|
||||
|
||||
return etcdCfgs
|
||||
|
@ -83,7 +83,7 @@ var (
|
||||
|
||||
// LocalListenCount integration test uses unique ports, counting up, to listen for each
|
||||
// member, ensuring restarted members can listen on the same port again.
|
||||
LocalListenCount = int64(0)
|
||||
LocalListenCount = int32(0)
|
||||
|
||||
TestTLSInfo = transport.TLSInfo{
|
||||
KeyFile: MustAbsPath("../fixtures/server.key.insecure"),
|
||||
@ -139,8 +139,6 @@ type ClusterConfig struct {
|
||||
|
||||
AuthToken string
|
||||
|
||||
UseGRPC bool
|
||||
|
||||
QuotaBackendBytes int64
|
||||
|
||||
MaxTxnOps uint
|
||||
@ -152,9 +150,6 @@ type ClusterConfig struct {
|
||||
GRPCKeepAliveInterval time.Duration
|
||||
GRPCKeepAliveTimeout time.Duration
|
||||
|
||||
// SkipCreatingClient to skip creating clients for each member.
|
||||
SkipCreatingClient bool
|
||||
|
||||
ClientMaxCallSendMsgSize int
|
||||
ClientMaxCallRecvMsgSize int
|
||||
|
||||
@ -172,17 +167,16 @@ type ClusterConfig struct {
|
||||
|
||||
WatchProgressNotifyInterval time.Duration
|
||||
ExperimentalMaxLearners int
|
||||
StrictReconfigCheck bool
|
||||
}
|
||||
|
||||
type Cluster struct {
|
||||
Cfg *ClusterConfig
|
||||
Members []*Member
|
||||
LastMemberNum int
|
||||
}
|
||||
|
||||
func (c *Cluster) generateMemberName() string {
|
||||
c.LastMemberNum++
|
||||
return fmt.Sprintf("m%v", c.LastMemberNum-1)
|
||||
mu sync.Mutex
|
||||
clusterClient *clientv3.Client
|
||||
}
|
||||
|
||||
func SchemeFromTLSInfo(tls *transport.TLSInfo) string {
|
||||
@ -216,34 +210,6 @@ func (c *Cluster) fillClusterForMembers() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewClusterFromConfig(t testutil.TB, cfg *ClusterConfig) *Cluster {
|
||||
testutil.SkipTestIfShortMode(t, "Cannot start etcd Cluster in --short tests")
|
||||
|
||||
c := &Cluster{Cfg: cfg}
|
||||
ms := make([]*Member, cfg.Size)
|
||||
for i := 0; i < cfg.Size; i++ {
|
||||
ms[i] = c.mustNewMember(t, int64(i))
|
||||
}
|
||||
c.Members = ms
|
||||
if err := c.fillClusterForMembers(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
return c
|
||||
}
|
||||
|
||||
// NewCluster returns an unlaunched Cluster of the given size which has been
|
||||
// set to use static bootstrap.
|
||||
func NewCluster(t testutil.TB, size int) *Cluster {
|
||||
t.Helper()
|
||||
return NewClusterFromConfig(t, &ClusterConfig{Size: size})
|
||||
}
|
||||
|
||||
// NewClusterByConfig returns an unlaunched Cluster defined by a Cluster configuration
|
||||
func NewClusterByConfig(t testutil.TB, cfg *ClusterConfig) *Cluster {
|
||||
return NewClusterFromConfig(t, cfg)
|
||||
}
|
||||
|
||||
func (c *Cluster) Launch(t testutil.TB) {
|
||||
errc := make(chan error)
|
||||
for _, m := range c.Members {
|
||||
@ -309,10 +275,12 @@ func (c *Cluster) HTTPMembers() []client.Member {
|
||||
return ms
|
||||
}
|
||||
|
||||
func (c *Cluster) mustNewMember(t testutil.TB, memberNumber int64) *Member {
|
||||
func (c *Cluster) mustNewMember(t testutil.TB) *Member {
|
||||
memberNumber := c.LastMemberNum
|
||||
c.LastMemberNum++
|
||||
m := MustNewMember(t,
|
||||
MemberConfig{
|
||||
Name: c.generateMemberName(),
|
||||
Name: fmt.Sprintf("m%v", memberNumber-1),
|
||||
MemberNumber: memberNumber,
|
||||
AuthToken: c.Cfg.AuthToken,
|
||||
PeerTLS: c.Cfg.PeerTLS,
|
||||
@ -335,19 +303,15 @@ func (c *Cluster) mustNewMember(t testutil.TB, memberNumber int64) *Member {
|
||||
LeaseCheckpointPersist: c.Cfg.LeaseCheckpointPersist,
|
||||
WatchProgressNotifyInterval: c.Cfg.WatchProgressNotifyInterval,
|
||||
ExperimentalMaxLearners: c.Cfg.ExperimentalMaxLearners,
|
||||
StrictReconfigCheck: c.Cfg.StrictReconfigCheck,
|
||||
})
|
||||
m.DiscoveryURL = c.Cfg.DiscoveryURL
|
||||
if c.Cfg.UseGRPC {
|
||||
if err := m.listenGRPC(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
return m
|
||||
}
|
||||
|
||||
// addMember return PeerURLs of the added member.
|
||||
func (c *Cluster) addMember(t testutil.TB) types.URLs {
|
||||
m := c.mustNewMember(t, 0)
|
||||
m := c.mustNewMember(t)
|
||||
|
||||
scheme := SchemeFromTLSInfo(c.Cfg.PeerTLS)
|
||||
|
||||
@ -421,6 +385,7 @@ func (c *Cluster) RemoveMember(t testutil.TB, id uint64) error {
|
||||
if uint64(m.Server.ID()) != id {
|
||||
newMembers = append(newMembers, m)
|
||||
} else {
|
||||
m.Client.Close()
|
||||
select {
|
||||
case <-m.Server.StopNotify():
|
||||
m.Terminate(t)
|
||||
@ -437,18 +402,6 @@ func (c *Cluster) RemoveMember(t testutil.TB, id uint64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Cluster) Terminate(t testutil.TB) {
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(len(c.Members))
|
||||
for _, m := range c.Members {
|
||||
go func(mm *Member) {
|
||||
defer wg.Done()
|
||||
mm.Terminate(t)
|
||||
}(m)
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
func (c *Cluster) WaitMembersMatch(t testutil.TB, membs []client.Member) {
|
||||
for _, u := range c.URLs() {
|
||||
cc := MustNewHTTPClient(t, []string{u}, c.Cfg.ClientTLS)
|
||||
@ -560,7 +513,7 @@ func isMembersEqual(membs []client.Member, wmembs []client.Member) bool {
|
||||
}
|
||||
|
||||
func newLocalListener(t testutil.TB) net.Listener {
|
||||
c := atomic.AddInt64(&LocalListenCount, 1)
|
||||
c := atomic.AddInt32(&LocalListenCount, 1)
|
||||
// Go 1.8+ allows only numbers in port
|
||||
addr := fmt.Sprintf("127.0.0.1:%05d%05d", c+BasePort, os.Getpid())
|
||||
return NewListenerWithAddr(t, addr)
|
||||
@ -576,8 +529,8 @@ func NewListenerWithAddr(t testutil.TB, addr string) net.Listener {
|
||||
|
||||
type Member struct {
|
||||
config.ServerConfig
|
||||
UniqNumber int64
|
||||
MemberNumber int64
|
||||
UniqNumber int
|
||||
MemberNumber int
|
||||
PeerListeners, ClientListeners []net.Listener
|
||||
GrpcListener net.Listener
|
||||
// PeerTLSInfo enables peer TLS when set
|
||||
@ -598,6 +551,8 @@ type Member struct {
|
||||
|
||||
// ServerClient is a clientv3 that directly calls the etcdserver.
|
||||
ServerClient *clientv3.Client
|
||||
// Client is a clientv3 that communicates via socket, either UNIX or TCP.
|
||||
Client *clientv3.Client
|
||||
|
||||
KeepDataDirTerminate bool
|
||||
ClientMaxCallSendMsgSize int
|
||||
@ -617,7 +572,7 @@ func (m *Member) GRPCURL() string { return m.GrpcURL }
|
||||
type MemberConfig struct {
|
||||
Name string
|
||||
UniqNumber int64
|
||||
MemberNumber int64
|
||||
MemberNumber int
|
||||
PeerTLS *transport.TLSInfo
|
||||
ClientTLS *transport.TLSInfo
|
||||
AuthToken string
|
||||
@ -639,6 +594,7 @@ type MemberConfig struct {
|
||||
LeaseCheckpointPersist bool
|
||||
WatchProgressNotifyInterval time.Duration
|
||||
ExperimentalMaxLearners int
|
||||
StrictReconfigCheck bool
|
||||
}
|
||||
|
||||
// MustNewMember return an inited member with the given name. If peerTLS is
|
||||
@ -647,7 +603,7 @@ func MustNewMember(t testutil.TB, mcfg MemberConfig) *Member {
|
||||
var err error
|
||||
m := &Member{
|
||||
MemberNumber: mcfg.MemberNumber,
|
||||
UniqNumber: atomic.AddInt64(&LocalListenCount, 1),
|
||||
UniqNumber: int(atomic.AddInt32(&LocalListenCount, 1)),
|
||||
}
|
||||
|
||||
peerScheme := SchemeFromTLSInfo(mcfg.PeerTLS)
|
||||
@ -750,6 +706,10 @@ func MustNewMember(t testutil.TB, mcfg MemberConfig) *Member {
|
||||
m.V2Deprecation = config.V2_DEPR_DEFAULT
|
||||
m.GrpcServerRecorder = &grpc_testing.GrpcRecorder{}
|
||||
m.Logger = memberLogger(t, mcfg.Name)
|
||||
m.StrictReconfigCheck = mcfg.StrictReconfigCheck
|
||||
if err := m.listenGRPC(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
t.Cleanup(func() {
|
||||
// if we didn't cleanup the logger, the consecutive test
|
||||
// might reuse this (t).
|
||||
@ -847,7 +807,7 @@ func (m *Member) grpcAddr() (network, host, port string) {
|
||||
return network, host, port
|
||||
}
|
||||
|
||||
func GrpcPortNumber(uniqNumber, memberNumber int64) int64 {
|
||||
func GrpcPortNumber(uniqNumber, memberNumber int) int {
|
||||
return BaseGRPCPort + uniqNumber*10 + memberNumber
|
||||
}
|
||||
|
||||
@ -1086,6 +1046,12 @@ func (m *Member) Launch() error {
|
||||
}
|
||||
m.ServerClosers = append(m.ServerClosers, closer)
|
||||
}
|
||||
if m.GrpcURL != "" && m.Client == nil {
|
||||
m.Client, err = NewClientV3(m)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
m.Logger.Info(
|
||||
"launched a member",
|
||||
@ -1387,75 +1353,68 @@ func (p SortableMemberSliceByPeerURLs) Less(i, j int) bool {
|
||||
}
|
||||
func (p SortableMemberSliceByPeerURLs) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
|
||||
|
||||
type ClusterV3 struct {
|
||||
*Cluster
|
||||
|
||||
mu sync.Mutex
|
||||
Clients []*clientv3.Client
|
||||
clusterClient *clientv3.Client
|
||||
}
|
||||
|
||||
// NewClusterV3 returns a launched Cluster with a grpc client connection
|
||||
// NewCluster returns a launched Cluster with a grpc client connection
|
||||
// for each Cluster member.
|
||||
func NewClusterV3(t testutil.TB, cfg *ClusterConfig) *ClusterV3 {
|
||||
func NewCluster(t testutil.TB, cfg *ClusterConfig) *Cluster {
|
||||
t.Helper()
|
||||
|
||||
assertInTestContext(t)
|
||||
|
||||
cfg.UseGRPC = true
|
||||
testutil.SkipTestIfShortMode(t, "Cannot start etcd Cluster in --short tests")
|
||||
|
||||
clus := &ClusterV3{
|
||||
Cluster: NewClusterByConfig(t, cfg),
|
||||
c := &Cluster{Cfg: cfg}
|
||||
ms := make([]*Member, cfg.Size)
|
||||
for i := 0; i < cfg.Size; i++ {
|
||||
ms[i] = c.mustNewMember(t)
|
||||
}
|
||||
clus.Launch(t)
|
||||
|
||||
if !cfg.SkipCreatingClient {
|
||||
for _, m := range clus.Members {
|
||||
client, err := NewClientV3(m)
|
||||
if err != nil {
|
||||
t.Fatalf("cannot create client: %v", err)
|
||||
}
|
||||
clus.Clients = append(clus.Clients, client)
|
||||
}
|
||||
c.Members = ms
|
||||
if err := c.fillClusterForMembers(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
c.Launch(t)
|
||||
|
||||
return clus
|
||||
return c
|
||||
}
|
||||
|
||||
func (c *ClusterV3) TakeClient(idx int) {
|
||||
func (c *Cluster) TakeClient(idx int) {
|
||||
c.mu.Lock()
|
||||
c.Clients[idx] = nil
|
||||
c.Members[idx].Client = nil
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
func (c *ClusterV3) Terminate(t testutil.TB) {
|
||||
func (c *Cluster) Terminate(t testutil.TB) {
|
||||
c.mu.Lock()
|
||||
for _, client := range c.Clients {
|
||||
if client == nil {
|
||||
continue
|
||||
}
|
||||
if err := client.Close(); err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
}
|
||||
if c.clusterClient != nil {
|
||||
if err := c.clusterClient.Close(); err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
c.Cluster.Terminate(t)
|
||||
for _, m := range c.Members {
|
||||
if m.Client != nil {
|
||||
m.Client.Close()
|
||||
}
|
||||
}
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(len(c.Members))
|
||||
for _, m := range c.Members {
|
||||
go func(mm *Member) {
|
||||
defer wg.Done()
|
||||
mm.Terminate(t)
|
||||
}(m)
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
func (c *ClusterV3) RandClient() *clientv3.Client {
|
||||
return c.Clients[rand.Intn(len(c.Clients))]
|
||||
func (c *Cluster) RandClient() *clientv3.Client {
|
||||
return c.Members[rand.Intn(len(c.Members))].Client
|
||||
}
|
||||
|
||||
func (c *ClusterV3) Client(i int) *clientv3.Client {
|
||||
return c.Clients[i]
|
||||
func (c *Cluster) Client(i int) *clientv3.Client {
|
||||
return c.Members[i].Client
|
||||
}
|
||||
|
||||
func (c *ClusterV3) ClusterClient() (client *clientv3.Client, err error) {
|
||||
func (c *Cluster) ClusterClient() (client *clientv3.Client, err error) {
|
||||
if c.clusterClient == nil {
|
||||
endpoints := []string{}
|
||||
for _, m := range c.Members {
|
||||
@ -1475,11 +1434,11 @@ func (c *ClusterV3) ClusterClient() (client *clientv3.Client, err error) {
|
||||
}
|
||||
|
||||
// NewClientV3 creates a new grpc client connection to the member
|
||||
func (c *ClusterV3) NewClientV3(memberIndex int) (*clientv3.Client, error) {
|
||||
func (c *Cluster) NewClientV3(memberIndex int) (*clientv3.Client, error) {
|
||||
return NewClientV3(c.Members[memberIndex])
|
||||
}
|
||||
|
||||
func makeClients(t testutil.TB, clus *ClusterV3, clients *[]*clientv3.Client, chooseMemberIndex func() int) func() *clientv3.Client {
|
||||
func makeClients(t testutil.TB, clus *Cluster, clients *[]*clientv3.Client, chooseMemberIndex func() int) func() *clientv3.Client {
|
||||
var mu sync.Mutex
|
||||
*clients = nil
|
||||
return func() *clientv3.Client {
|
||||
@ -1496,13 +1455,13 @@ func makeClients(t testutil.TB, clus *ClusterV3, clients *[]*clientv3.Client, ch
|
||||
|
||||
// MakeSingleNodeClients creates factory of clients that all connect to member 0.
|
||||
// All the created clients are put on the 'clients' list. The factory is thread-safe.
|
||||
func MakeSingleNodeClients(t testutil.TB, clus *ClusterV3, clients *[]*clientv3.Client) func() *clientv3.Client {
|
||||
func MakeSingleNodeClients(t testutil.TB, clus *Cluster, clients *[]*clientv3.Client) func() *clientv3.Client {
|
||||
return makeClients(t, clus, clients, func() int { return 0 })
|
||||
}
|
||||
|
||||
// MakeMultiNodeClients creates factory of clients that all connect to random members.
|
||||
// All the created clients are put on the 'clients' list. The factory is thread-safe.
|
||||
func MakeMultiNodeClients(t testutil.TB, clus *ClusterV3, clients *[]*clientv3.Client) func() *clientv3.Client {
|
||||
func MakeMultiNodeClients(t testutil.TB, clus *Cluster, clients *[]*clientv3.Client) func() *clientv3.Client {
|
||||
return makeClients(t, clus, clients, func() int { return rand.Intn(len(clus.Members)) })
|
||||
}
|
||||
|
||||
@ -1535,7 +1494,7 @@ type GrpcAPI struct {
|
||||
}
|
||||
|
||||
// GetLearnerMembers returns the list of learner members in Cluster using MemberList API.
|
||||
func (c *ClusterV3) GetLearnerMembers() ([]*pb.Member, error) {
|
||||
func (c *Cluster) GetLearnerMembers() ([]*pb.Member, error) {
|
||||
cli := c.Client(0)
|
||||
resp, err := cli.MemberList(context.Background())
|
||||
if err != nil {
|
||||
@ -1552,8 +1511,8 @@ func (c *ClusterV3) GetLearnerMembers() ([]*pb.Member, error) {
|
||||
|
||||
// AddAndLaunchLearnerMember creates a leaner member, adds it to Cluster
|
||||
// via v3 MemberAdd API, and then launches the new member.
|
||||
func (c *ClusterV3) AddAndLaunchLearnerMember(t testutil.TB) {
|
||||
m := c.mustNewMember(t, 0)
|
||||
func (c *Cluster) AddAndLaunchLearnerMember(t testutil.TB) {
|
||||
m := c.mustNewMember(t)
|
||||
m.IsLearner = true
|
||||
|
||||
scheme := SchemeFromTLSInfo(c.Cfg.PeerTLS)
|
||||
@ -1582,7 +1541,7 @@ func (c *ClusterV3) AddAndLaunchLearnerMember(t testutil.TB) {
|
||||
}
|
||||
|
||||
// getMembers returns a list of members in Cluster, in format of etcdserverpb.Member
|
||||
func (c *ClusterV3) getMembers() []*pb.Member {
|
||||
func (c *Cluster) getMembers() []*pb.Member {
|
||||
var mems []*pb.Member
|
||||
for _, m := range c.Members {
|
||||
mem := &pb.Member{
|
||||
@ -1619,7 +1578,7 @@ func (c *ClusterV3) getMembers() []*pb.Member {
|
||||
// indicate that the new learner member has applied the raftpb.ConfChangeAddLearnerNode entry
|
||||
// which was used to add the learner itself to the Cluster, and therefore it has the correct info
|
||||
// on learner.
|
||||
func (c *ClusterV3) waitMembersMatch(t testutil.TB) {
|
||||
func (c *Cluster) waitMembersMatch(t testutil.TB) {
|
||||
wMembers := c.getMembers()
|
||||
sort.Sort(SortableProtoMemberSliceByPeerURLs(wMembers))
|
||||
cli := c.Client(0)
|
||||
@ -1653,8 +1612,8 @@ func (p SortableProtoMemberSliceByPeerURLs) Less(i, j int) bool {
|
||||
func (p SortableProtoMemberSliceByPeerURLs) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
|
||||
|
||||
// MustNewMember creates a new member instance based on the response of V3 Member Add API.
|
||||
func (c *ClusterV3) MustNewMember(t testutil.TB, resp *clientv3.MemberAddResponse) *Member {
|
||||
m := c.mustNewMember(t, 0)
|
||||
func (c *Cluster) MustNewMember(t testutil.TB, resp *clientv3.MemberAddResponse) *Member {
|
||||
m := c.mustNewMember(t)
|
||||
m.IsLearner = resp.Member.IsLearner
|
||||
m.NewCluster = false
|
||||
|
||||
|
@ -1,131 +0,0 @@
|
||||
// Copyright 2016 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 integration
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"os"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/client/v2"
|
||||
integration2 "go.etcd.io/etcd/tests/v3/framework/integration"
|
||||
)
|
||||
|
||||
// TestV2NoRetryEOF tests destructive api calls won't retry on a disconnection.
|
||||
func TestV2NoRetryEOF(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
// generate an EOF response; specify address so appears first in sorted ep list
|
||||
lEOF := integration2.NewListenerWithAddr(t, fmt.Sprintf("127.0.0.1:%05d", os.Getpid()))
|
||||
defer lEOF.Close()
|
||||
tries := uint32(0)
|
||||
go func() {
|
||||
for {
|
||||
conn, err := lEOF.Accept()
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
atomic.AddUint32(&tries, 1)
|
||||
conn.Close()
|
||||
}
|
||||
}()
|
||||
eofURL := integration2.URLScheme + "://" + lEOF.Addr().String()
|
||||
cli := integration2.MustNewHTTPClient(t, []string{eofURL, eofURL}, nil)
|
||||
kapi := client.NewKeysAPI(cli)
|
||||
for i, f := range noRetryList(kapi) {
|
||||
startTries := atomic.LoadUint32(&tries)
|
||||
if err := f(); err == nil {
|
||||
t.Errorf("#%d: expected EOF error, got nil", i)
|
||||
}
|
||||
endTries := atomic.LoadUint32(&tries)
|
||||
if startTries+1 != endTries {
|
||||
t.Errorf("#%d: expected 1 try, got %d", i, endTries-startTries)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestV2NoRetryNoLeader tests destructive api calls won't retry if given an error code.
|
||||
func TestV2NoRetryNoLeader(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
lHTTP := integration2.NewListenerWithAddr(t, fmt.Sprintf("127.0.0.1:%05d", os.Getpid()))
|
||||
eh := &errHandler{errCode: http.StatusServiceUnavailable}
|
||||
srv := httptest.NewUnstartedServer(eh)
|
||||
defer lHTTP.Close()
|
||||
defer srv.Close()
|
||||
srv.Listener = lHTTP
|
||||
go srv.Start()
|
||||
lHTTPURL := integration2.URLScheme + "://" + lHTTP.Addr().String()
|
||||
|
||||
cli := integration2.MustNewHTTPClient(t, []string{lHTTPURL, lHTTPURL}, nil)
|
||||
kapi := client.NewKeysAPI(cli)
|
||||
// test error code
|
||||
for i, f := range noRetryList(kapi) {
|
||||
reqs := eh.reqs
|
||||
if err := f(); err == nil || !strings.Contains(err.Error(), "no leader") {
|
||||
t.Errorf("#%d: expected \"no leader\", got %v", i, err)
|
||||
}
|
||||
if eh.reqs != reqs+1 {
|
||||
t.Errorf("#%d: expected 1 request, got %d", i, eh.reqs-reqs)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestV2RetryRefuse tests destructive api calls will retry if a connection is refused.
|
||||
func TestV2RetryRefuse(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
cl := integration2.NewCluster(t, 1)
|
||||
cl.Launch(t)
|
||||
defer cl.Terminate(t)
|
||||
// test connection refused; expect no error failover
|
||||
cli := integration2.MustNewHTTPClient(t, []string{integration2.URLScheme + "://refuseconn:123", cl.URL(0)}, nil)
|
||||
kapi := client.NewKeysAPI(cli)
|
||||
if _, err := kapi.Set(context.Background(), "/delkey", "def", nil); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for i, f := range noRetryList(kapi) {
|
||||
if err := f(); err != nil {
|
||||
t.Errorf("#%d: unexpected retry failure (%v)", i, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type errHandler struct {
|
||||
errCode int
|
||||
reqs int
|
||||
}
|
||||
|
||||
func (eh *errHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
|
||||
req.Body.Close()
|
||||
eh.reqs++
|
||||
w.WriteHeader(eh.errCode)
|
||||
}
|
||||
|
||||
func noRetryList(kapi client.KeysAPI) []func() error {
|
||||
return []func() error{
|
||||
func() error {
|
||||
opts := &client.SetOptions{PrevExist: client.PrevNoExist}
|
||||
_, err := kapi.Set(context.Background(), "/setkey", "bar", opts)
|
||||
return err
|
||||
},
|
||||
func() error {
|
||||
_, err := kapi.Delete(context.Background(), "/delkey", nil)
|
||||
return err
|
||||
},
|
||||
}
|
||||
}
|
@ -1,17 +0,0 @@
|
||||
// Copyright 2016 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 integration implements tests built upon embedded etcd, focusing on
|
||||
// the correctness of the etcd v2 client.
|
||||
package integration
|
@ -1,15 +0,0 @@
|
||||
// Copyright 2013 The Go Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package integration
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/testutil"
|
||||
)
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
testutil.MustTestMainWithLeakDetection(m)
|
||||
}
|
@ -30,7 +30,7 @@ import (
|
||||
func TestMemberList(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
capi := clus.RandClient()
|
||||
@ -48,7 +48,7 @@ func TestMemberList(t *testing.T) {
|
||||
func TestMemberAdd(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
capi := clus.RandClient()
|
||||
@ -67,7 +67,7 @@ func TestMemberAdd(t *testing.T) {
|
||||
func TestMemberAddWithExistingURLs(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
capi := clus.RandClient()
|
||||
@ -91,7 +91,7 @@ func TestMemberAddWithExistingURLs(t *testing.T) {
|
||||
func TestMemberRemove(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
capi := clus.Client(1)
|
||||
@ -129,7 +129,7 @@ func TestMemberRemove(t *testing.T) {
|
||||
func TestMemberUpdate(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
capi := clus.RandClient()
|
||||
@ -157,7 +157,7 @@ func TestMemberUpdate(t *testing.T) {
|
||||
func TestMemberAddUpdateWrongURLs(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
capi := clus.RandClient()
|
||||
@ -190,7 +190,7 @@ func TestMemberAddUpdateWrongURLs(t *testing.T) {
|
||||
func TestMemberAddForLearner(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
capi := clus.RandClient()
|
||||
@ -219,7 +219,7 @@ func TestMemberAddForLearner(t *testing.T) {
|
||||
func TestMemberPromote(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// member promote request can be sent to any server in cluster,
|
||||
@ -296,7 +296,7 @@ func TestMemberPromote(t *testing.T) {
|
||||
func TestMemberPromoteMemberNotLearner(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// member promote request can be sent to any server in cluster,
|
||||
@ -332,7 +332,7 @@ func TestMemberPromoteMemberNotLearner(t *testing.T) {
|
||||
func TestMemberPromoteMemberNotExist(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// member promote request can be sent to any server in cluster,
|
||||
@ -382,7 +382,7 @@ func TestMaxLearnerInCluster(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
// 1. start with a cluster with 3 voting member and max learner 2
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, ExperimentalMaxLearners: 2})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, ExperimentalMaxLearners: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// 2. adding 2 learner members should succeed
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
)
|
||||
|
||||
func TestRevisionMonotonicWithLeaderPartitions(t *testing.T) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.ClusterV3) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.Cluster) {
|
||||
for i := 0; i < 5; i++ {
|
||||
leader := clus.WaitLeader(t)
|
||||
time.Sleep(time.Second)
|
||||
@ -40,7 +40,7 @@ func TestRevisionMonotonicWithLeaderPartitions(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestRevisionMonotonicWithPartitions(t *testing.T) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.ClusterV3) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.Cluster) {
|
||||
for i := 0; i < 5; i++ {
|
||||
time.Sleep(time.Second)
|
||||
clus.Members[i%3].InjectPartition(t, clus.Members[(i+1)%3], clus.Members[(i+2)%3])
|
||||
@ -51,7 +51,7 @@ func TestRevisionMonotonicWithPartitions(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestRevisionMonotonicWithLeaderRestarts(t *testing.T) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.ClusterV3) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.Cluster) {
|
||||
for i := 0; i < 5; i++ {
|
||||
leader := clus.WaitLeader(t)
|
||||
time.Sleep(time.Second)
|
||||
@ -63,7 +63,7 @@ func TestRevisionMonotonicWithLeaderRestarts(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestRevisionMonotonicWithRestarts(t *testing.T) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.ClusterV3) {
|
||||
testRevisionMonotonicWithFailures(t, 11*time.Second, func(clus *integration.Cluster) {
|
||||
for i := 0; i < 5; i++ {
|
||||
time.Sleep(time.Second)
|
||||
clus.Members[i%3].Stop(t)
|
||||
@ -73,9 +73,9 @@ func TestRevisionMonotonicWithRestarts(t *testing.T) {
|
||||
})
|
||||
}
|
||||
|
||||
func testRevisionMonotonicWithFailures(t *testing.T, testDuration time.Duration, injectFailures func(clus *integration.ClusterV3)) {
|
||||
func testRevisionMonotonicWithFailures(t *testing.T, testDuration time.Duration, injectFailures func(clus *integration.Cluster)) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), testDuration)
|
||||
@ -108,7 +108,7 @@ func testRevisionMonotonicWithFailures(t *testing.T, testDuration time.Duration,
|
||||
t.Logf("Revision %d", resp.Header.Revision)
|
||||
}
|
||||
|
||||
func putWorker(t *testing.T, ctx context.Context, clus *integration.ClusterV3) {
|
||||
func putWorker(t *testing.T, ctx context.Context, clus *integration.Cluster) {
|
||||
for i := 0; ; i++ {
|
||||
kv := clus.Client(i % 3)
|
||||
_, err := kv.Put(ctx, "foo", fmt.Sprintf("%d", i))
|
||||
@ -121,7 +121,7 @@ func putWorker(t *testing.T, ctx context.Context, clus *integration.ClusterV3) {
|
||||
}
|
||||
}
|
||||
|
||||
func getWorker(t *testing.T, ctx context.Context, clus *integration.ClusterV3) {
|
||||
func getWorker(t *testing.T, ctx context.Context, clus *integration.Cluster) {
|
||||
var prevRev int64
|
||||
for i := 0; ; i++ {
|
||||
kv := clus.Client(i % 3)
|
||||
|
@ -35,7 +35,7 @@ import (
|
||||
func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 2,
|
||||
GRPCKeepAliveMinTime: time.Millisecond, // avoid too_many_pings
|
||||
UseBridge: true,
|
||||
@ -168,10 +168,9 @@ func TestBalancerUnderBlackholeNoKeepAliveSerializableGet(t *testing.T) {
|
||||
func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Client, context.Context) error) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 2,
|
||||
SkipCreatingClient: true,
|
||||
UseBridge: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 2,
|
||||
UseBridge: true,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
|
@ -48,7 +48,7 @@ var (
|
||||
// TestDialTLSExpired tests client with expired certs fails to dial.
|
||||
func TestDialTLSExpired(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, PeerTLS: &testTLSInfo, ClientTLS: &testTLSInfo, SkipCreatingClient: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, PeerTLS: &testTLSInfo, ClientTLS: &testTLSInfo})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
tls, err := testTLSInfoExpired.ClientConfig()
|
||||
@ -71,7 +71,7 @@ func TestDialTLSExpired(t *testing.T) {
|
||||
// when TLS endpoints (https, unixs) are given but no tls config.
|
||||
func TestDialTLSNoConfig(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, ClientTLS: &testTLSInfo, SkipCreatingClient: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, ClientTLS: &testTLSInfo})
|
||||
defer clus.Terminate(t)
|
||||
// expect "signed by unknown authority"
|
||||
c, err := integration2.NewClient(t, clientv3.Config{
|
||||
@ -102,7 +102,7 @@ func TestDialSetEndpointsAfterFail(t *testing.T) {
|
||||
// testDialSetEndpoints ensures SetEndpoints can replace unavailable endpoints with available ones.
|
||||
func testDialSetEndpoints(t *testing.T, setBefore bool) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, SkipCreatingClient: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// get endpoint list
|
||||
@ -145,7 +145,7 @@ func testDialSetEndpoints(t *testing.T, setBefore bool) {
|
||||
// with a new one that doesn't include original endpoint.
|
||||
func TestSwitchSetEndpoints(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// get non partitioned members endpoints
|
||||
@ -166,7 +166,7 @@ func TestSwitchSetEndpoints(t *testing.T) {
|
||||
func TestRejectOldCluster(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
// 2 endpoints to test multi-endpoint Status
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2, SkipCreatingClient: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
@ -186,7 +186,7 @@ func TestRejectOldCluster(t *testing.T) {
|
||||
// with the balancer can be dialed.
|
||||
func TestDialForeignEndpoint(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
conn, err := clus.Client(0).Dial(clus.Client(1).Endpoints()[0])
|
||||
@ -209,7 +209,7 @@ func TestDialForeignEndpoint(t *testing.T) {
|
||||
// to a working endpoint will always succeed.
|
||||
func TestSetEndpointAndPut(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.Client(1).SetEndpoints(clus.Members[0].GRPCURL())
|
||||
|
@ -105,9 +105,8 @@ func TestBalancerUnderNetworkPartitionSerializableGet(t *testing.T) {
|
||||
func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, context.Context) error, timeout time.Duration) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
@ -161,9 +160,8 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c
|
||||
func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
@ -216,9 +214,8 @@ func TestBalancerUnderNetworkPartitionWatchFollower(t *testing.T) {
|
||||
func testBalancerUnderNetworkPartitionWatch(t *testing.T, isolateLeader bool) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
@ -276,9 +273,8 @@ func testBalancerUnderNetworkPartitionWatch(t *testing.T, isolateLeader bool) {
|
||||
func TestDropReadUnderNetworkPartition(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
leaderIndex := clus.WaitLeader(t)
|
||||
|
@ -32,10 +32,9 @@ import (
|
||||
func TestBalancerUnderServerShutdownWatch(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
UseBridge: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
UseBridge: true,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
@ -145,9 +144,8 @@ func TestBalancerUnderServerShutdownTxn(t *testing.T) {
|
||||
func testBalancerUnderServerShutdownMutable(t *testing.T, op func(*clientv3.Client, context.Context) error) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
@ -203,9 +201,8 @@ func TestBalancerUnderServerShutdownGetSerializable(t *testing.T) {
|
||||
func testBalancerUnderServerShutdownImmutable(t *testing.T, op func(*clientv3.Client, context.Context) error, timeout time.Duration) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
@ -277,15 +274,14 @@ func testBalancerUnderServerStopInflightRangeOnRestart(t *testing.T, linearizabl
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
cfg := &integration2.ClusterConfig{
|
||||
Size: 2,
|
||||
SkipCreatingClient: true,
|
||||
UseBridge: true,
|
||||
Size: 2,
|
||||
UseBridge: true,
|
||||
}
|
||||
if linearizable {
|
||||
cfg.Size = 3
|
||||
}
|
||||
|
||||
clus := integration2.NewClusterV3(t, cfg)
|
||||
clus := integration2.NewCluster(t, cfg)
|
||||
defer clus.Terminate(t)
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL()}
|
||||
if linearizable {
|
||||
|
@ -25,14 +25,14 @@ import (
|
||||
|
||||
func TestBarrierSingleNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
testBarrier(t, 5, func() *clientv3.Client { return clus.Client(0) })
|
||||
}
|
||||
|
||||
func TestBarrierMultiNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
testBarrier(t, 5, func() *clientv3.Client { return clus.RandClient() })
|
||||
}
|
||||
|
@ -26,7 +26,7 @@ import (
|
||||
func TestDoubleBarrier(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
waiters := 10
|
||||
@ -100,7 +100,7 @@ func TestDoubleBarrier(t *testing.T) {
|
||||
func TestDoubleBarrierFailover(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
waiters := 10
|
||||
|
@ -30,7 +30,7 @@ import (
|
||||
func TestMutexLockSingleNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
var clients []*clientv3.Client
|
||||
@ -41,7 +41,7 @@ func TestMutexLockSingleNode(t *testing.T) {
|
||||
func TestMutexLockMultiNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
var clients []*clientv3.Client
|
||||
@ -94,7 +94,7 @@ func testMutexLock(t *testing.T, waiters int, chooseClient func() *clientv3.Clie
|
||||
|
||||
func TestMutexTryLockSingleNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
var clients []*clientv3.Client
|
||||
@ -104,7 +104,7 @@ func TestMutexTryLockSingleNode(t *testing.T) {
|
||||
|
||||
func TestMutexTryLockMultiNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
var clients []*clientv3.Client
|
||||
@ -165,7 +165,7 @@ func testMutexTryLock(t *testing.T, lockers int, chooseClient func() *clientv3.C
|
||||
func TestMutexSessionRelock(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
session, err := concurrency.NewSession(clus.RandClient())
|
||||
if err != nil {
|
||||
@ -189,7 +189,7 @@ func TestMutexSessionRelock(t *testing.T) {
|
||||
func TestMutexWaitsOnCurrentHolder(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cctx := context.Background()
|
||||
@ -297,7 +297,7 @@ func TestMutexWaitsOnCurrentHolder(t *testing.T) {
|
||||
func BenchmarkMutex4Waiters(b *testing.B) {
|
||||
integration2.BeforeTest(b)
|
||||
// XXX switch tests to use TB interface
|
||||
clus := integration2.NewClusterV3(nil, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(nil, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(nil)
|
||||
for i := 0; i < b.N; i++ {
|
||||
testMutexLock(nil, 4, func() *clientv3.Client { return clus.RandClient() })
|
||||
@ -306,14 +306,14 @@ func BenchmarkMutex4Waiters(b *testing.B) {
|
||||
|
||||
func TestRWMutexSingleNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
testRWMutex(t, 5, func() *clientv3.Client { return clus.Client(0) })
|
||||
}
|
||||
|
||||
func TestRWMutexMultiNode(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
testRWMutex(t, 5, func() *clientv3.Client { return clus.RandClient() })
|
||||
}
|
||||
|
@ -33,7 +33,7 @@ const (
|
||||
func TestQueueOneReaderOneWriter(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
done := make(chan struct{})
|
||||
@ -81,7 +81,7 @@ func BenchmarkQueue(b *testing.B) {
|
||||
integration2.BeforeTest(b)
|
||||
|
||||
// XXX switch tests to use TB interface
|
||||
clus := integration2.NewClusterV3(nil, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(nil, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(nil)
|
||||
for i := 0; i < b.N; i++ {
|
||||
testQueueNReaderMWriter(nil, manyQueueClients, manyQueueClients)
|
||||
@ -92,7 +92,7 @@ func BenchmarkQueue(b *testing.B) {
|
||||
func TestPrQueueOneReaderOneWriter(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// write out five items with random priority
|
||||
@ -126,7 +126,7 @@ func TestPrQueueOneReaderOneWriter(t *testing.T) {
|
||||
func TestPrQueueManyReaderManyWriter(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
rqs := newPriorityQueues(clus, manyQueueClients)
|
||||
wqs := newPriorityQueues(clus, manyQueueClients)
|
||||
@ -138,7 +138,7 @@ func BenchmarkPrQueueOneReaderOneWriter(b *testing.B) {
|
||||
integration2.BeforeTest(b)
|
||||
|
||||
// XXX switch tests to use TB interface
|
||||
clus := integration2.NewClusterV3(nil, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(nil, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(nil)
|
||||
rqs := newPriorityQueues(clus, 1)
|
||||
wqs := newPriorityQueues(clus, 1)
|
||||
@ -149,12 +149,12 @@ func BenchmarkPrQueueOneReaderOneWriter(b *testing.B) {
|
||||
|
||||
func testQueueNReaderMWriter(t *testing.T, n int, m int) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
testReadersWriters(t, newQueues(clus, n), newQueues(clus, m))
|
||||
}
|
||||
|
||||
func newQueues(clus *integration2.ClusterV3, n int) (qs []testQueue) {
|
||||
func newQueues(clus *integration2.Cluster, n int) (qs []testQueue) {
|
||||
for i := 0; i < n; i++ {
|
||||
etcdc := clus.RandClient()
|
||||
qs = append(qs, recipe.NewQueue(etcdc, "q"))
|
||||
@ -162,7 +162,7 @@ func newQueues(clus *integration2.ClusterV3, n int) (qs []testQueue) {
|
||||
return qs
|
||||
}
|
||||
|
||||
func newPriorityQueues(clus *integration2.ClusterV3, n int) (qs []testQueue) {
|
||||
func newPriorityQueues(clus *integration2.Cluster, n int) (qs []testQueue) {
|
||||
for i := 0; i < n; i++ {
|
||||
etcdc := clus.RandClient()
|
||||
q := &flatPriorityQueue{recipe.NewPriorityQueue(etcdc, "prq")}
|
||||
|
@ -42,7 +42,7 @@ func TestKVPutError(t *testing.T) {
|
||||
maxReqBytes = 1.5 * 1024 * 1024 // hard coded max in v3_server.go
|
||||
quota = int64(int(maxReqBytes*1.2) + 8*os.Getpagesize()) // make sure we have enough overhead in backend quota. See discussion in #6486.
|
||||
)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, QuotaBackendBytes: quota, ClientMaxCallSendMsgSize: 100 * 1024 * 1024})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, QuotaBackendBytes: quota, ClientMaxCallSendMsgSize: 100 * 1024 * 1024})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -74,7 +74,7 @@ func TestKVPutError(t *testing.T) {
|
||||
func TestKVPut(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lapi := clus.RandClient()
|
||||
@ -119,7 +119,7 @@ func TestKVPut(t *testing.T) {
|
||||
func TestKVPutWithIgnoreValue(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -152,7 +152,7 @@ func TestKVPutWithIgnoreValue(t *testing.T) {
|
||||
func TestKVPutWithIgnoreLease(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -191,7 +191,7 @@ func TestKVPutWithIgnoreLease(t *testing.T) {
|
||||
func TestKVPutWithRequireLeader(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.Members[1].Stop(t)
|
||||
@ -237,7 +237,7 @@ func TestKVPutWithRequireLeader(t *testing.T) {
|
||||
func TestKVRange(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -466,7 +466,7 @@ func TestKVRange(t *testing.T) {
|
||||
func TestKVGetErrConnClosed(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -495,7 +495,7 @@ func TestKVGetErrConnClosed(t *testing.T) {
|
||||
func TestKVNewAfterClose(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -522,7 +522,7 @@ func TestKVNewAfterClose(t *testing.T) {
|
||||
func TestKVDeleteRange(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -594,7 +594,7 @@ func TestKVDeleteRange(t *testing.T) {
|
||||
func TestKVDelete(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -626,7 +626,7 @@ func TestKVDelete(t *testing.T) {
|
||||
func TestKVCompactError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -656,7 +656,7 @@ func TestKVCompactError(t *testing.T) {
|
||||
func TestKVCompact(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -712,7 +712,7 @@ func TestKVGetRetry(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clusterSize := 3
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: clusterSize, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: clusterSize, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// because killing leader and following election
|
||||
@ -765,7 +765,7 @@ func TestKVGetRetry(t *testing.T) {
|
||||
func TestKVPutFailGetRetry(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
@ -805,7 +805,7 @@ func TestKVPutFailGetRetry(t *testing.T) {
|
||||
func TestKVGetCancel(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
oldconn := clus.Client(0).ActiveConnection()
|
||||
@ -828,7 +828,7 @@ func TestKVGetCancel(t *testing.T) {
|
||||
func TestKVGetStoppedServerAndClose(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -846,7 +846,7 @@ func TestKVGetStoppedServerAndClose(t *testing.T) {
|
||||
func TestKVPutStoppedServerAndClose(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -876,7 +876,7 @@ func TestKVPutStoppedServerAndClose(t *testing.T) {
|
||||
// in the presence of network errors.
|
||||
func TestKVPutAtMostOnce(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
if _, err := clus.Client(0).Put(context.TODO(), "k", "1"); err != nil {
|
||||
@ -970,7 +970,7 @@ func TestKVLargeRequests(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i, test := range tests {
|
||||
clus := integration2.NewClusterV3(t,
|
||||
clus := integration2.NewCluster(t,
|
||||
&integration2.ClusterConfig{
|
||||
Size: 1,
|
||||
MaxRequestBytes: test.maxRequestBytesServer,
|
||||
@ -1005,7 +1005,7 @@ func TestKVLargeRequests(t *testing.T) {
|
||||
func TestKVForLearner(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// we have to add and launch learner member after initial cluster was created, because
|
||||
@ -1084,7 +1084,7 @@ func TestKVForLearner(t *testing.T) {
|
||||
func TestBalancerSupportLearner(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// we have to add and launch learner member after initial cluster was created, because
|
||||
|
@ -32,7 +32,7 @@ import (
|
||||
func TestLeaseNotFoundError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -46,7 +46,7 @@ func TestLeaseNotFoundError(t *testing.T) {
|
||||
func TestLeaseGrant(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lapi := clus.RandClient()
|
||||
@ -72,7 +72,7 @@ func TestLeaseGrant(t *testing.T) {
|
||||
func TestLeaseRevoke(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lapi := clus.RandClient()
|
||||
@ -98,7 +98,7 @@ func TestLeaseRevoke(t *testing.T) {
|
||||
func TestLeaseKeepAliveOnce(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lapi := clus.RandClient()
|
||||
@ -122,7 +122,7 @@ func TestLeaseKeepAliveOnce(t *testing.T) {
|
||||
func TestLeaseKeepAlive(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lapi := clus.Client(0)
|
||||
@ -162,7 +162,7 @@ func TestLeaseKeepAlive(t *testing.T) {
|
||||
func TestLeaseKeepAliveOneSecond(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -190,7 +190,7 @@ func TestLeaseKeepAliveHandleFailure(t *testing.T) {
|
||||
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// TODO: change this line to get a cluster client
|
||||
@ -245,7 +245,7 @@ type leaseCh struct {
|
||||
func TestLeaseKeepAliveNotFound(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -278,7 +278,7 @@ func TestLeaseKeepAliveNotFound(t *testing.T) {
|
||||
func TestLeaseGrantErrConnClosed(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -310,7 +310,7 @@ func TestLeaseGrantErrConnClosed(t *testing.T) {
|
||||
func TestLeaseKeepAliveFullResponseQueue(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lapi := clus.Client(0)
|
||||
@ -350,7 +350,7 @@ func TestLeaseKeepAliveFullResponseQueue(t *testing.T) {
|
||||
func TestLeaseGrantNewAfterClose(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -377,7 +377,7 @@ func TestLeaseGrantNewAfterClose(t *testing.T) {
|
||||
func TestLeaseRevokeNewAfterClose(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -416,7 +416,7 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) {
|
||||
func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -462,7 +462,7 @@ func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) {
|
||||
func TestLeaseKeepAliveInitTimeout(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -495,7 +495,7 @@ func TestLeaseKeepAliveInitTimeout(t *testing.T) {
|
||||
func TestLeaseKeepAliveTTLTimeout(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -530,7 +530,7 @@ func TestLeaseKeepAliveTTLTimeout(t *testing.T) {
|
||||
func TestLeaseTimeToLive(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.RandClient()
|
||||
@ -588,7 +588,7 @@ func TestLeaseTimeToLive(t *testing.T) {
|
||||
func TestLeaseTimeToLiveLeaseNotFound(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -623,7 +623,7 @@ func TestLeaseTimeToLiveLeaseNotFound(t *testing.T) {
|
||||
func TestLeaseLeases(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -656,7 +656,7 @@ func TestLeaseLeases(t *testing.T) {
|
||||
func TestLeaseRenewLostQuorum(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -704,7 +704,7 @@ func TestLeaseRenewLostQuorum(t *testing.T) {
|
||||
func TestLeaseKeepAliveLoopExit(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx := context.Background()
|
||||
@ -728,7 +728,7 @@ func TestLeaseKeepAliveLoopExit(t *testing.T) {
|
||||
// transient cluster failure.
|
||||
func TestV3LeaseFailureOverlap(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
numReqs := 5
|
||||
@ -782,7 +782,7 @@ func TestV3LeaseFailureOverlap(t *testing.T) {
|
||||
func TestLeaseWithRequireLeader(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.Client(0)
|
||||
|
@ -34,7 +34,7 @@ import (
|
||||
func TestLeasingPutGet(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lKV1, closeLKV1, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -92,7 +92,7 @@ func TestLeasingPutGet(t *testing.T) {
|
||||
// TestLeasingInterval checks the leasing KV fetches key intervals.
|
||||
func TestLeasingInterval(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -131,7 +131,7 @@ func TestLeasingInterval(t *testing.T) {
|
||||
// TestLeasingPutInvalidateNew checks the leasing KV updates its cache on a Put to a new key.
|
||||
func TestLeasingPutInvalidateNew(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -161,7 +161,7 @@ func TestLeasingPutInvalidateNew(t *testing.T) {
|
||||
// TestLeasingPutInvalidateExisting checks the leasing KV updates its cache on a Put to an existing key.
|
||||
func TestLeasingPutInvalidateExisting(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
if _, err := clus.Client(0).Put(context.TODO(), "k", "abc"); err != nil {
|
||||
@ -195,7 +195,7 @@ func TestLeasingPutInvalidateExisting(t *testing.T) {
|
||||
// TestLeasingGetNoLeaseTTL checks a key with a TTL is not leased.
|
||||
func TestLeasingGetNoLeaseTTL(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -224,7 +224,7 @@ func TestLeasingGetNoLeaseTTL(t *testing.T) {
|
||||
// when the etcd cluster is partitioned.
|
||||
func TestLeasingGetSerializable(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -264,7 +264,7 @@ func TestLeasingGetSerializable(t *testing.T) {
|
||||
// TestLeasingPrevKey checks the cache respects WithPrevKV on puts.
|
||||
func TestLeasingPrevKey(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -290,7 +290,7 @@ func TestLeasingPrevKey(t *testing.T) {
|
||||
// TestLeasingRevGet checks the cache respects Get by Revision.
|
||||
func TestLeasingRevGet(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -326,7 +326,7 @@ func TestLeasingRevGet(t *testing.T) {
|
||||
// TestLeasingGetWithOpts checks options that can be served through the cache do not depend on the server.
|
||||
func TestLeasingGetWithOpts(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -371,7 +371,7 @@ func TestLeasingGetWithOpts(t *testing.T) {
|
||||
// the recently put data.
|
||||
func TestLeasingConcurrentPut(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -418,7 +418,7 @@ func TestLeasingConcurrentPut(t *testing.T) {
|
||||
|
||||
func TestLeasingDisconnectedGet(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -447,7 +447,7 @@ func TestLeasingDisconnectedGet(t *testing.T) {
|
||||
|
||||
func TestLeasingDeleteOwner(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -481,7 +481,7 @@ func TestLeasingDeleteOwner(t *testing.T) {
|
||||
|
||||
func TestLeasingDeleteNonOwner(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv1, closeLKV1, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -516,7 +516,7 @@ func TestLeasingDeleteNonOwner(t *testing.T) {
|
||||
|
||||
func TestLeasingOverwriteResponse(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -550,7 +550,7 @@ func TestLeasingOverwriteResponse(t *testing.T) {
|
||||
|
||||
func TestLeasingOwnerPutResponse(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -588,7 +588,7 @@ func TestLeasingOwnerPutResponse(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerGetRange(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -617,7 +617,7 @@ func TestLeasingTxnOwnerGetRange(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerGet(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
client := clus.Client(0)
|
||||
@ -703,7 +703,7 @@ func TestLeasingTxnOwnerGet(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerDeleteRange(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -742,7 +742,7 @@ func TestLeasingTxnOwnerDeleteRange(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerDelete(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -773,7 +773,7 @@ func TestLeasingTxnOwnerDelete(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerIf(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -867,7 +867,7 @@ func TestLeasingTxnOwnerIf(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnCancel(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv1, closeLKV1, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -901,7 +901,7 @@ func TestLeasingTxnCancel(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnNonOwnerPut(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -979,7 +979,7 @@ func TestLeasingTxnNonOwnerPut(t *testing.T) {
|
||||
// issues a random If/{Then,Else} transaction on those keys to one client.
|
||||
func TestLeasingTxnRandIfThenOrElse(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv1, closeLKV1, err1 := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -1085,7 +1085,7 @@ func TestLeasingTxnRandIfThenOrElse(t *testing.T) {
|
||||
|
||||
func TestLeasingOwnerPutError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -1106,7 +1106,7 @@ func TestLeasingOwnerPutError(t *testing.T) {
|
||||
|
||||
func TestLeasingOwnerDeleteError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -1127,7 +1127,7 @@ func TestLeasingOwnerDeleteError(t *testing.T) {
|
||||
|
||||
func TestLeasingNonOwnerPutError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -1152,7 +1152,7 @@ func TestLeasingOwnerDeleteFrom(t *testing.T) {
|
||||
|
||||
func testLeasingOwnerDelete(t *testing.T, del clientv3.Op) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "0/")
|
||||
@ -1201,7 +1201,7 @@ func testLeasingOwnerDelete(t *testing.T, del clientv3.Op) {
|
||||
|
||||
func TestLeasingDeleteRangeBounds(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
delkv, closeDelKV, err := leasing.NewKV(clus.Client(0), "0/")
|
||||
@ -1259,7 +1259,7 @@ func TestLeaseDeleteRangeContendDel(t *testing.T) {
|
||||
|
||||
func testLeasingDeleteRangeContend(t *testing.T, op clientv3.Op) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
delkv, closeDelKV, err := leasing.NewKV(clus.Client(0), "0/")
|
||||
@ -1317,7 +1317,7 @@ func testLeasingDeleteRangeContend(t *testing.T, op clientv3.Op) {
|
||||
|
||||
func TestLeasingPutGetDeleteConcurrent(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkvs := make([]clientv3.KV, 16)
|
||||
@ -1376,7 +1376,7 @@ func TestLeasingPutGetDeleteConcurrent(t *testing.T) {
|
||||
// disconnected when trying to submit revoke txn.
|
||||
func TestLeasingReconnectOwnerRevoke(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv1, closeLKV1, err1 := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1437,7 +1437,7 @@ func TestLeasingReconnectOwnerRevoke(t *testing.T) {
|
||||
// disconnected and the watch is compacted.
|
||||
func TestLeasingReconnectOwnerRevokeCompact(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv1, closeLKV1, err1 := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1490,7 +1490,7 @@ func TestLeasingReconnectOwnerRevokeCompact(t *testing.T) {
|
||||
// not cause inconsistency between the server and the client.
|
||||
func TestLeasingReconnectOwnerConsistency(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1564,7 +1564,7 @@ func TestLeasingReconnectOwnerConsistency(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnAtomicCache(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1650,7 +1650,7 @@ func TestLeasingTxnAtomicCache(t *testing.T) {
|
||||
// TestLeasingReconnectTxn checks that Txn is resilient to disconnects.
|
||||
func TestLeasingReconnectTxn(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1686,7 +1686,7 @@ func TestLeasingReconnectTxn(t *testing.T) {
|
||||
// not cause inconsistency between the server and the client.
|
||||
func TestLeasingReconnectNonOwnerGet(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1737,7 +1737,7 @@ func TestLeasingReconnectNonOwnerGet(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnRangeCmp(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1772,7 +1772,7 @@ func TestLeasingTxnRangeCmp(t *testing.T) {
|
||||
|
||||
func TestLeasingDo(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1814,7 +1814,7 @@ func TestLeasingDo(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerPutBranch(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1908,7 +1908,7 @@ func randCmps(pfx string, dat []*clientv3.PutResponse) (cmps []clientv3.Cmp, the
|
||||
|
||||
func TestLeasingSessionExpire(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/", concurrency.WithTTL(1))
|
||||
@ -1984,7 +1984,7 @@ func TestLeasingSessionExpireCancel(t *testing.T) {
|
||||
for i := range tests {
|
||||
t.Run(fmt.Sprintf("test %d", i), func(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/", concurrency.WithTTL(1))
|
||||
|
@ -39,7 +39,7 @@ import (
|
||||
func TestMaintenanceHashKV(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
@ -72,7 +72,7 @@ func TestMaintenanceHashKV(t *testing.T) {
|
||||
func TestMaintenanceMoveLeader(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
oldLeadIdx := clus.WaitLeader(t)
|
||||
@ -103,7 +103,7 @@ func TestMaintenanceMoveLeader(t *testing.T) {
|
||||
func TestMaintenanceSnapshotCancel(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// reading snapshot with canceled context should error out
|
||||
@ -146,7 +146,7 @@ func TestMaintenanceSnapshotTimeout(t *testing.T) {
|
||||
func testMaintenanceSnapshotTimeout(t *testing.T, snapshot func(context.Context, *clientv3.Client) (io.ReadCloser, error)) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// reading snapshot with deadline exceeded should error out
|
||||
@ -191,7 +191,7 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) {
|
||||
func testMaintenanceSnapshotErrorInflight(t *testing.T, snapshot func(context.Context, *clientv3.Client) (io.ReadCloser, error)) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// take about 1-second to read snapshot
|
||||
@ -249,7 +249,7 @@ func TestMaintenanceSnapshotWithVersionVersion(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
// Set SnapshotCount to 1 to force raft snapshot to ensure that storage version is set
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, SnapshotCount: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, SnapshotCount: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// Put some keys to ensure that wal snapshot is triggered
|
||||
@ -271,7 +271,7 @@ func TestMaintenanceSnapshotWithVersionVersion(t *testing.T) {
|
||||
func TestMaintenanceStatus(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.WaitLeader(t)
|
||||
|
@ -70,7 +70,7 @@ func TestV3ClientMetrics(t *testing.T) {
|
||||
|
||||
url := "unix://" + addr + "/metrics"
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, SkipCreatingClient: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
|
@ -30,7 +30,7 @@ import (
|
||||
func TestMirrorSync(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.Client(0)
|
||||
@ -74,7 +74,7 @@ func TestMirrorSync(t *testing.T) {
|
||||
func TestMirrorSyncBase(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
cluster := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
cluster := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
cli := cluster.Client(0)
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
func TestNamespacePutGet(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.Client(0)
|
||||
@ -57,7 +57,7 @@ func TestNamespacePutGet(t *testing.T) {
|
||||
func TestNamespaceWatch(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.Client(0)
|
||||
|
@ -27,7 +27,7 @@ import (
|
||||
func TestEndpointManager(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
em, err := endpoints.NewManager(clus.RandClient(), "foo")
|
||||
@ -89,7 +89,7 @@ func TestEndpointManager(t *testing.T) {
|
||||
func TestEndpointManagerAtomicity(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.RandClient()
|
||||
@ -131,7 +131,7 @@ func TestEndpointManagerAtomicity(t *testing.T) {
|
||||
func TestEndpointManagerCRUD(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
em, err := endpoints.NewManager(clus.RandClient(), "foo")
|
||||
|
@ -47,7 +47,7 @@ func TestEtcdGrpcResolver(t *testing.T) {
|
||||
}
|
||||
defer s2.Stop()
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
em, err := endpoints.NewManager(clus.Client(0), "foo")
|
||||
|
@ -30,7 +30,7 @@ func TestDetectKvOrderViolation(t *testing.T) {
|
||||
var errOrderViolation = errors.New("DetectedOrderViolation")
|
||||
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
@ -97,7 +97,7 @@ func TestDetectTxnOrderViolation(t *testing.T) {
|
||||
var errOrderViolation = errors.New("DetectedOrderViolation")
|
||||
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
|
@ -26,7 +26,7 @@ import (
|
||||
|
||||
func TestEndpointSwitchResolvesViolation(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
eps := []string{
|
||||
clus.Members[0].GRPCURL(),
|
||||
@ -82,7 +82,7 @@ func TestEndpointSwitchResolvesViolation(t *testing.T) {
|
||||
|
||||
func TestUnresolvableOrderViolation(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 5, SkipCreatingClient: true, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 5, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{
|
||||
|
@ -25,7 +25,7 @@ import (
|
||||
func TestRoleError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
authapi := clus.RandClient()
|
||||
|
@ -29,7 +29,7 @@ import (
|
||||
func TestTxnError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.RandClient()
|
||||
@ -53,7 +53,7 @@ func TestTxnError(t *testing.T) {
|
||||
func TestTxnWriteFail(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
@ -103,7 +103,7 @@ func TestTxnReadRetry(t *testing.T) {
|
||||
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
@ -142,7 +142,7 @@ func TestTxnReadRetry(t *testing.T) {
|
||||
func TestTxnSuccess(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
@ -165,7 +165,7 @@ func TestTxnSuccess(t *testing.T) {
|
||||
func TestTxnCompareRange(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
@ -192,7 +192,7 @@ func TestTxnCompareRange(t *testing.T) {
|
||||
func TestTxnNested(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
func TestUserError(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
authapi := clus.RandClient()
|
||||
@ -57,7 +57,7 @@ func TestUserError(t *testing.T) {
|
||||
func TestUserErrorAuth(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
authapi := clus.RandClient()
|
||||
@ -114,7 +114,7 @@ func authSetupRoot(t *testing.T, auth clientv3.Auth) {
|
||||
func TestGetTokenWithoutAuth(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 2})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
authapi := clus.RandClient()
|
||||
|
@ -73,7 +73,7 @@ func testWatchFragment(t *testing.T, fragment, exceedRecvLimit bool) {
|
||||
if exceedRecvLimit {
|
||||
cfg.ClientMaxCallRecvMsgSize = 1.5 * 1024 * 1024
|
||||
}
|
||||
clus := integration2.NewClusterV3(t, cfg)
|
||||
clus := integration2.NewCluster(t, cfg)
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
|
@ -36,7 +36,7 @@ import (
|
||||
type watcherTest func(*testing.T, *watchctx)
|
||||
|
||||
type watchctx struct {
|
||||
clus *integration2.ClusterV3
|
||||
clus *integration2.Cluster
|
||||
w clientv3.Watcher
|
||||
kv clientv3.KV
|
||||
wclientMember int
|
||||
@ -47,7 +47,7 @@ type watchctx struct {
|
||||
func runWatchTest(t *testing.T, f watcherTest) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wclientMember := rand.Intn(3)
|
||||
@ -348,7 +348,7 @@ func putAndWatch(t *testing.T, wctx *watchctx, key, val string) {
|
||||
|
||||
func TestWatchResumeInitRev(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -404,7 +404,7 @@ func TestWatchResumeInitRev(t *testing.T) {
|
||||
func TestWatchResumeCompacted(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// create a waiting watcher at rev 1
|
||||
@ -491,7 +491,7 @@ func TestWatchResumeCompacted(t *testing.T) {
|
||||
func TestWatchCompactRevision(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// set some keys
|
||||
@ -540,7 +540,7 @@ func testWatchWithProgressNotify(t *testing.T, watchOnPut bool) {
|
||||
pi := 3 * time.Second
|
||||
defer func() { v3rpc.SetProgressReportInterval(oldpi) }()
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wc := clus.RandClient()
|
||||
@ -588,7 +588,7 @@ func TestConfigurableWatchProgressNotifyInterval(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
progressInterval := 200 * time.Millisecond
|
||||
clus := integration2.NewClusterV3(t,
|
||||
clus := integration2.NewCluster(t,
|
||||
&integration2.ClusterConfig{
|
||||
Size: 3,
|
||||
WatchProgressNotifyInterval: progressInterval,
|
||||
@ -629,7 +629,7 @@ func TestWatchRequestProgress(t *testing.T) {
|
||||
|
||||
watchTimeout := 3 * time.Second
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wc := clus.RandClient()
|
||||
@ -688,7 +688,7 @@ func TestWatchRequestProgress(t *testing.T) {
|
||||
func TestWatchEventType(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
cluster := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
cluster := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
client := cluster.RandClient()
|
||||
@ -762,7 +762,7 @@ func TestWatchEventType(t *testing.T) {
|
||||
func TestWatchErrConnClosed(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -792,7 +792,7 @@ func TestWatchErrConnClosed(t *testing.T) {
|
||||
func TestWatchAfterClose(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -820,7 +820,7 @@ func TestWatchAfterClose(t *testing.T) {
|
||||
func TestWatchWithRequireLeader(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 3})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// Put a key for the non-require leader watch to read as an event.
|
||||
@ -894,7 +894,7 @@ func TestWatchWithRequireLeader(t *testing.T) {
|
||||
func TestWatchWithFilter(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
cluster := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
cluster := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
client := cluster.RandClient()
|
||||
@ -933,7 +933,7 @@ func TestWatchWithFilter(t *testing.T) {
|
||||
func TestWatchWithCreatedNotification(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
cluster := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
cluster := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
client := cluster.RandClient()
|
||||
@ -955,7 +955,7 @@ func TestWatchWithCreatedNotification(t *testing.T) {
|
||||
func TestWatchWithCreatedNotificationDropConn(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
cluster := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
cluster := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
client := cluster.RandClient()
|
||||
@ -984,7 +984,7 @@ func TestWatchWithCreatedNotificationDropConn(t *testing.T) {
|
||||
func TestWatchCancelOnServer(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
cluster := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
cluster := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
client := cluster.RandClient()
|
||||
@ -1050,20 +1050,20 @@ func TestWatchCancelOnServer(t *testing.T) {
|
||||
// 4. watcher client finishes tearing down stream on "ctx"
|
||||
// 5. w2 comes back canceled
|
||||
func TestWatchOverlapContextCancel(t *testing.T) {
|
||||
f := func(clus *integration2.ClusterV3) {}
|
||||
f := func(clus *integration2.Cluster) {}
|
||||
testWatchOverlapContextCancel(t, f)
|
||||
}
|
||||
|
||||
func TestWatchOverlapDropConnContextCancel(t *testing.T) {
|
||||
f := func(clus *integration2.ClusterV3) {
|
||||
f := func(clus *integration2.Cluster) {
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
}
|
||||
testWatchOverlapContextCancel(t, f)
|
||||
}
|
||||
|
||||
func testWatchOverlapContextCancel(t *testing.T, f func(*integration2.ClusterV3)) {
|
||||
func testWatchOverlapContextCancel(t *testing.T, f func(*integration2.Cluster)) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
n := 100
|
||||
@ -1124,7 +1124,7 @@ func testWatchOverlapContextCancel(t *testing.T, f func(*integration2.ClusterV3)
|
||||
// closing the client does not return a client closing error.
|
||||
func TestWatchCancelAndCloseClient(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.Client(0)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
@ -1154,7 +1154,7 @@ func TestWatchCancelAndCloseClient(t *testing.T) {
|
||||
// then closes the watcher interface to ensure correct clean up.
|
||||
func TestWatchStressResumeClose(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.Client(0)
|
||||
|
||||
@ -1176,7 +1176,7 @@ func TestWatchStressResumeClose(t *testing.T) {
|
||||
// its grpc stream is disconnected / reconnecting.
|
||||
func TestWatchCancelDisconnected(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.Client(0)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
@ -26,6 +26,7 @@ import (
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/client/v2"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/server/v3/etcdserver"
|
||||
"go.etcd.io/etcd/tests/v3/framework/integration"
|
||||
)
|
||||
@ -45,16 +46,14 @@ func TestClusterOf3(t *testing.T) { testCluster(t, 3) }
|
||||
|
||||
func testCluster(t *testing.T, size int) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewCluster(t, size)
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: size})
|
||||
defer c.Terminate(t)
|
||||
clusterMustProgress(t, c.Members)
|
||||
}
|
||||
|
||||
func TestTLSClusterOf3(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewClusterByConfig(t, &integration.ClusterConfig{Size: 3, PeerTLS: &integration.TestTLSInfo})
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, PeerTLS: &integration.TestTLSInfo})
|
||||
defer c.Terminate(t)
|
||||
clusterMustProgress(t, c.Members)
|
||||
}
|
||||
@ -63,59 +62,7 @@ func TestTLSClusterOf3(t *testing.T) {
|
||||
// authorities that don't issue dual-usage certificates.
|
||||
func TestTLSClusterOf3WithSpecificUsage(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewClusterByConfig(t, &integration.ClusterConfig{Size: 3, PeerTLS: &integration.TestTLSInfoWithSpecificUsage})
|
||||
c.Launch(t)
|
||||
defer c.Terminate(t)
|
||||
clusterMustProgress(t, c.Members)
|
||||
}
|
||||
|
||||
func TestClusterOf1UsingDiscovery(t *testing.T) { testClusterUsingDiscovery(t, 1) }
|
||||
func TestClusterOf3UsingDiscovery(t *testing.T) { testClusterUsingDiscovery(t, 3) }
|
||||
|
||||
func testClusterUsingDiscovery(t *testing.T, size int) {
|
||||
integration.BeforeTest(t)
|
||||
dc := integration.NewCluster(t, 1)
|
||||
dc.Launch(t)
|
||||
defer dc.Terminate(t)
|
||||
// init discovery token space
|
||||
dcc := integration.MustNewHTTPClient(t, dc.URLs(), nil)
|
||||
dkapi := client.NewKeysAPI(dcc)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
if _, err := dkapi.Create(ctx, "/_config/size", fmt.Sprintf("%d", size)); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
cancel()
|
||||
|
||||
c := integration.NewClusterByConfig(
|
||||
t,
|
||||
&integration.ClusterConfig{Size: size, DiscoveryURL: dc.URL(0) + "/v2/keys"},
|
||||
)
|
||||
c.Launch(t)
|
||||
defer c.Terminate(t)
|
||||
clusterMustProgress(t, c.Members)
|
||||
}
|
||||
|
||||
func TestTLSClusterOf3UsingDiscovery(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
dc := integration.NewCluster(t, 1)
|
||||
dc.Launch(t)
|
||||
defer dc.Terminate(t)
|
||||
// init discovery token space
|
||||
dcc := integration.MustNewHTTPClient(t, dc.URLs(), nil)
|
||||
dkapi := client.NewKeysAPI(dcc)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
if _, err := dkapi.Create(ctx, "/_config/size", fmt.Sprintf("%d", 3)); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
cancel()
|
||||
|
||||
c := integration.NewClusterByConfig(t,
|
||||
&integration.ClusterConfig{
|
||||
Size: 3,
|
||||
PeerTLS: &integration.TestTLSInfo,
|
||||
DiscoveryURL: dc.URL(0) + "/v2/keys"},
|
||||
)
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, PeerTLS: &integration.TestTLSInfoWithSpecificUsage})
|
||||
defer c.Terminate(t)
|
||||
clusterMustProgress(t, c.Members)
|
||||
}
|
||||
@ -125,8 +72,7 @@ func TestDoubleClusterSizeOf3(t *testing.T) { testDoubleClusterSize(t, 3) }
|
||||
|
||||
func testDoubleClusterSize(t *testing.T, size int) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewCluster(t, size)
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: size})
|
||||
defer c.Terminate(t)
|
||||
|
||||
for i := 0; i < size; i++ {
|
||||
@ -137,8 +83,7 @@ func testDoubleClusterSize(t *testing.T, size int) {
|
||||
|
||||
func TestDoubleTLSClusterSizeOf3(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewClusterByConfig(t, &integration.ClusterConfig{Size: 3, PeerTLS: &integration.TestTLSInfo})
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, PeerTLS: &integration.TestTLSInfo})
|
||||
defer c.Terminate(t)
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
@ -152,8 +97,7 @@ func TestDecreaseClusterSizeOf5(t *testing.T) { testDecreaseClusterSize(t, 5) }
|
||||
|
||||
func testDecreaseClusterSize(t *testing.T, size int) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewCluster(t, size)
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: size})
|
||||
defer c.Terminate(t)
|
||||
|
||||
// TODO: remove the last but one member
|
||||
@ -174,20 +118,29 @@ func testDecreaseClusterSize(t *testing.T, size int) {
|
||||
}
|
||||
|
||||
func TestForceNewCluster(t *testing.T) {
|
||||
c := integration.NewClusterFromConfig(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
c.Launch(t)
|
||||
cc := integration.MustNewHTTPClient(t, []string{c.Members[0].URL()}, nil)
|
||||
kapi := client.NewKeysAPI(cc)
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer c.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
resp, err := kapi.Create(ctx, "/foo", "bar")
|
||||
resp, err := c.Members[0].Client.Put(ctx, "/foo", "bar")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected create error: %v", err)
|
||||
}
|
||||
cancel()
|
||||
// ensure create has been applied in this machine
|
||||
ctx, cancel = context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
if _, err = kapi.Watcher("/foo", &client.WatcherOptions{AfterIndex: resp.Node.ModifiedIndex - 1}).Next(ctx); err != nil {
|
||||
t.Fatalf("unexpected watch error: %v", err)
|
||||
watch := c.Members[0].Client.Watcher.Watch(ctx, "/foo", clientv3.WithRev(resp.Header.Revision-1))
|
||||
for resp := range watch {
|
||||
if len(resp.Events) != 0 {
|
||||
break
|
||||
}
|
||||
if resp.Err() != nil {
|
||||
t.Fatalf("unexpected watch error: %q", resp.Err())
|
||||
}
|
||||
if resp.Canceled {
|
||||
t.Fatalf("watch cancelled")
|
||||
}
|
||||
}
|
||||
cancel()
|
||||
|
||||
@ -199,16 +152,22 @@ func TestForceNewCluster(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected ForceRestart error: %v", err)
|
||||
}
|
||||
defer c.Members[0].Terminate(t)
|
||||
c.WaitMembersForLeader(t, c.Members[:1])
|
||||
|
||||
// use new http client to init new connection
|
||||
cc = integration.MustNewHTTPClient(t, []string{c.Members[0].URL()}, nil)
|
||||
kapi = client.NewKeysAPI(cc)
|
||||
// ensure force restart keep the old data, and new Cluster can make progress
|
||||
ctx, cancel = context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
if _, err := kapi.Watcher("/foo", &client.WatcherOptions{AfterIndex: resp.Node.ModifiedIndex - 1}).Next(ctx); err != nil {
|
||||
t.Fatalf("unexpected watch error: %v", err)
|
||||
watch = c.Members[0].Client.Watcher.Watch(ctx, "/foo", clientv3.WithRev(resp.Header.Revision-1))
|
||||
for resp := range watch {
|
||||
if len(resp.Events) != 0 {
|
||||
break
|
||||
}
|
||||
if resp.Err() != nil {
|
||||
t.Fatalf("unexpected watch error: %q", resp.Err())
|
||||
}
|
||||
if resp.Canceled {
|
||||
t.Fatalf("watch cancelled")
|
||||
}
|
||||
}
|
||||
cancel()
|
||||
clusterMustProgress(t, c.Members[:1])
|
||||
@ -216,8 +175,7 @@ func TestForceNewCluster(t *testing.T) {
|
||||
|
||||
func TestAddMemberAfterClusterFullRotation(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewCluster(t, 3)
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer c.Terminate(t)
|
||||
|
||||
// remove all the previous three members and add in three new members.
|
||||
@ -238,8 +196,7 @@ func TestAddMemberAfterClusterFullRotation(t *testing.T) {
|
||||
// Ensure we can remove a member then add a new one back immediately.
|
||||
func TestIssue2681(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewCluster(t, 5)
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 5})
|
||||
defer c.Terminate(t)
|
||||
|
||||
c.MustRemoveMember(t, uint64(c.Members[4].Server.ID()))
|
||||
@ -258,13 +215,7 @@ func TestIssue2746WithThree(t *testing.T) { testIssue2746(t, 3) }
|
||||
|
||||
func testIssue2746(t *testing.T, members int) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewCluster(t, members)
|
||||
|
||||
for _, m := range c.Members {
|
||||
m.SnapshotCount = 10
|
||||
}
|
||||
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: members, SnapshotCount: 10})
|
||||
defer c.Terminate(t)
|
||||
|
||||
// force a snapshot
|
||||
@ -284,8 +235,7 @@ func testIssue2746(t *testing.T, members int) {
|
||||
func TestIssue2904(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
// start 1-member Cluster to ensure member 0 is the leader of the Cluster.
|
||||
c := integration.NewClusterFromConfig(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer c.Terminate(t)
|
||||
|
||||
c.AddMember(t)
|
||||
@ -309,6 +259,7 @@ func TestIssue2904(t *testing.T) {
|
||||
<-c.Members[1].Server.StopNotify()
|
||||
|
||||
// terminate removed member
|
||||
c.Members[1].Client.Close()
|
||||
c.Members[1].Terminate(t)
|
||||
c.Members = c.Members[:1]
|
||||
// wait member to be removed.
|
||||
@ -320,8 +271,7 @@ func TestIssue2904(t *testing.T) {
|
||||
func TestIssue3699(t *testing.T) {
|
||||
// start a Cluster of 3 nodes a, b, c
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewClusterFromConfig(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer c.Terminate(t)
|
||||
|
||||
// make node a unavailable
|
||||
@ -360,10 +310,8 @@ func TestIssue3699(t *testing.T) {
|
||||
c.WaitMembersForLeader(t, c.Members)
|
||||
|
||||
// try to participate in Cluster
|
||||
cc := integration.MustNewHTTPClient(t, []string{c.URL(0)}, c.Cfg.ClientTLS)
|
||||
kapi := client.NewKeysAPI(cc)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
if _, err := kapi.Set(ctx, "/foo", "bar", nil); err != nil {
|
||||
if _, err := c.Members[0].Client.Put(ctx, "/foo", "bar"); err != nil {
|
||||
t.Fatalf("unexpected error on Set (%v)", err)
|
||||
}
|
||||
cancel()
|
||||
@ -372,11 +320,7 @@ func TestIssue3699(t *testing.T) {
|
||||
// TestRejectUnhealthyAdd ensures an unhealthy cluster rejects adding members.
|
||||
func TestRejectUnhealthyAdd(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewClusterFromConfig(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
for _, m := range c.Members {
|
||||
m.ServerConfig.StrictReconfigCheck = true
|
||||
}
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true, StrictReconfigCheck: true})
|
||||
defer c.Terminate(t)
|
||||
|
||||
// make Cluster unhealthy and wait for downed peer
|
||||
@ -416,11 +360,7 @@ func TestRejectUnhealthyAdd(t *testing.T) {
|
||||
// if quorum will be lost.
|
||||
func TestRejectUnhealthyRemove(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewClusterFromConfig(t, &integration.ClusterConfig{Size: 5, UseBridge: true})
|
||||
for _, m := range c.Members {
|
||||
m.ServerConfig.StrictReconfigCheck = true
|
||||
}
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 5, UseBridge: true, StrictReconfigCheck: true})
|
||||
defer c.Terminate(t)
|
||||
|
||||
// make cluster unhealthy and wait for downed peer; (3 up, 2 down)
|
||||
@ -465,38 +405,35 @@ func TestRestartRemoved(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
// 1. start single-member Cluster
|
||||
c := integration.NewClusterFromConfig(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
for _, m := range c.Members {
|
||||
m.ServerConfig.StrictReconfigCheck = true
|
||||
}
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, StrictReconfigCheck: true, UseBridge: true})
|
||||
defer c.Terminate(t)
|
||||
|
||||
// 2. add a new member
|
||||
c.Cfg.StrictReconfigCheck = false
|
||||
c.AddMember(t)
|
||||
c.WaitLeader(t)
|
||||
|
||||
oldm := c.Members[0]
|
||||
oldm.KeepDataDirTerminate = true
|
||||
firstMember := c.Members[0]
|
||||
firstMember.KeepDataDirTerminate = true
|
||||
|
||||
// 3. remove first member, shut down without deleting data
|
||||
if err := c.RemoveMember(t, uint64(c.Members[0].Server.ID())); err != nil {
|
||||
if err := c.RemoveMember(t, uint64(firstMember.Server.ID())); err != nil {
|
||||
t.Fatalf("expected to remove member, got error %v", err)
|
||||
}
|
||||
c.WaitLeader(t)
|
||||
|
||||
// 4. restart first member with 'initial-cluster-state=new'
|
||||
// wrong config, expects exit within ReqTimeout
|
||||
oldm.ServerConfig.NewCluster = false
|
||||
if err := oldm.Restart(t); err != nil {
|
||||
firstMember.ServerConfig.NewCluster = false
|
||||
if err := firstMember.Restart(t); err != nil {
|
||||
t.Fatalf("unexpected ForceRestart error: %v", err)
|
||||
}
|
||||
defer func() {
|
||||
oldm.Close()
|
||||
os.RemoveAll(oldm.ServerConfig.DataDir)
|
||||
firstMember.Close()
|
||||
os.RemoveAll(firstMember.ServerConfig.DataDir)
|
||||
}()
|
||||
select {
|
||||
case <-oldm.Server.StopNotify():
|
||||
case <-firstMember.Server.StopNotify():
|
||||
case <-time.After(time.Minute):
|
||||
t.Fatalf("removed member didn't exit within %v", time.Minute)
|
||||
}
|
||||
@ -505,35 +442,39 @@ func TestRestartRemoved(t *testing.T) {
|
||||
// clusterMustProgress ensures that cluster can make progress. It creates
|
||||
// a random key first, and check the new key could be got from all client urls
|
||||
// of the cluster.
|
||||
func clusterMustProgress(t *testing.T, membs []*integration.Member) {
|
||||
cc := integration.MustNewHTTPClient(t, []string{membs[0].URL()}, nil)
|
||||
kapi := client.NewKeysAPI(cc)
|
||||
func clusterMustProgress(t *testing.T, members []*integration.Member) {
|
||||
key := fmt.Sprintf("foo%d", rand.Int())
|
||||
var (
|
||||
err error
|
||||
resp *client.Response
|
||||
resp *clientv3.PutResponse
|
||||
)
|
||||
// retry in case of leader loss induced by slow CI
|
||||
for i := 0; i < 3; i++ {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
resp, err = kapi.Create(ctx, "/"+key, "bar")
|
||||
resp, err = members[0].Client.Put(ctx, key, "bar")
|
||||
cancel()
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
t.Logf("failed to create key on %q (%v)", membs[0].URL(), err)
|
||||
t.Logf("failed to create key on #0 (%v)", err)
|
||||
}
|
||||
if err != nil {
|
||||
t.Fatalf("create on %s error: %v", membs[0].URL(), err)
|
||||
t.Fatalf("create on #0 error: %v", err)
|
||||
}
|
||||
|
||||
for i, m := range membs {
|
||||
u := m.URL()
|
||||
mcc := integration.MustNewHTTPClient(t, []string{u}, nil)
|
||||
mkapi := client.NewKeysAPI(mcc)
|
||||
for i, m := range members {
|
||||
mctx, mcancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
if _, err := mkapi.Watcher(key, &client.WatcherOptions{AfterIndex: resp.Node.ModifiedIndex - 1}).Next(mctx); err != nil {
|
||||
t.Fatalf("#%d: watch on %s error: %v", i, u, err)
|
||||
watch := m.Client.Watcher.Watch(mctx, key, clientv3.WithRev(resp.Header.Revision-1))
|
||||
for resp := range watch {
|
||||
if len(resp.Events) != 0 {
|
||||
break
|
||||
}
|
||||
if resp.Err() != nil {
|
||||
t.Fatalf("#%d: watch error: %q", i, resp.Err())
|
||||
}
|
||||
if resp.Canceled {
|
||||
t.Fatalf("#%d: watch: cancelled", i)
|
||||
}
|
||||
}
|
||||
mcancel()
|
||||
}
|
||||
@ -541,7 +482,7 @@ func clusterMustProgress(t *testing.T, membs []*integration.Member) {
|
||||
|
||||
func TestSpeedyTerminate(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
// Stop/Restart so requests will time out on lost leaders
|
||||
for i := 0; i < 3; i++ {
|
||||
clus.Members[i].Stop(t)
|
||||
|
@ -102,7 +102,7 @@ func TestAuthority(t *testing.T) {
|
||||
UseIP: tc.useTCP,
|
||||
}
|
||||
cfg, tlsConfig := setupTLS(t, tc.useTLS, cfg)
|
||||
clus := integration.NewClusterV3(t, &cfg)
|
||||
clus := integration.NewCluster(t, &cfg)
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := setupClient(t, tc.clientURLPattern, clus, tlsConfig)
|
||||
@ -132,7 +132,7 @@ func setupTLS(t *testing.T, useTLS bool, cfg integration.ClusterConfig) (integra
|
||||
return cfg, nil
|
||||
}
|
||||
|
||||
func setupClient(t *testing.T, endpointPattern string, clus *integration.ClusterV3, tlsConfig *tls.Config) *clientv3.Client {
|
||||
func setupClient(t *testing.T, endpointPattern string, clus *integration.Cluster, tlsConfig *tls.Config) *clientv3.Client {
|
||||
t.Helper()
|
||||
endpoints := templateEndpoints(t, endpointPattern, clus)
|
||||
kv, err := clientv3.New(clientv3.Config{
|
||||
@ -147,7 +147,7 @@ func setupClient(t *testing.T, endpointPattern string, clus *integration.Cluster
|
||||
return kv
|
||||
}
|
||||
|
||||
func templateEndpoints(t *testing.T, pattern string, clus *integration.ClusterV3) []string {
|
||||
func templateEndpoints(t *testing.T, pattern string, clus *integration.Cluster) []string {
|
||||
t.Helper()
|
||||
endpoints := []string{}
|
||||
for _, m := range clus.Members {
|
||||
@ -181,7 +181,7 @@ func templateAuthority(t *testing.T, pattern string, m *integration.Member) stri
|
||||
return authority
|
||||
}
|
||||
|
||||
func assertAuthority(t *testing.T, expectedAuthority string, clus *integration.ClusterV3) {
|
||||
func assertAuthority(t *testing.T, expectedAuthority string, clus *integration.Cluster) {
|
||||
t.Helper()
|
||||
requestsFound := 0
|
||||
for _, m := range clus.Members {
|
||||
|
@ -43,7 +43,7 @@ type LazyCluster interface {
|
||||
EndpointsV3() []string
|
||||
|
||||
// Cluster - calls to this method might initialize the cluster.
|
||||
Cluster() *integration.ClusterV3
|
||||
Cluster() *integration.Cluster
|
||||
|
||||
// Transport - call to this method might initialize the cluster.
|
||||
Transport() *http.Transport
|
||||
@ -55,7 +55,7 @@ type LazyCluster interface {
|
||||
|
||||
type lazyCluster struct {
|
||||
cfg integration.ClusterConfig
|
||||
cluster *integration.ClusterV3
|
||||
cluster *integration.Cluster
|
||||
transport *http.Transport
|
||||
once sync.Once
|
||||
tb testutil.TB
|
||||
@ -82,7 +82,7 @@ func (lc *lazyCluster) mustLazyInit() {
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
lc.cluster = integration.NewClusterV3(lc.tb, &lc.cfg)
|
||||
lc.cluster = integration.NewCluster(lc.tb, &lc.cfg)
|
||||
})
|
||||
}
|
||||
|
||||
@ -106,7 +106,7 @@ func (lc *lazyCluster) EndpointsV3() []string {
|
||||
return lc.Cluster().Client(0).Endpoints()
|
||||
}
|
||||
|
||||
func (lc *lazyCluster) Cluster() *integration.ClusterV3 {
|
||||
func (lc *lazyCluster) Cluster() *integration.Cluster {
|
||||
lc.mustLazyInit()
|
||||
return lc.cluster
|
||||
}
|
||||
|
@ -18,19 +18,16 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.etcd.io/etcd/client/v2"
|
||||
"go.etcd.io/etcd/tests/v3/framework/integration"
|
||||
)
|
||||
|
||||
func TestPauseMember(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
c := integration.NewCluster(t, 5)
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 5})
|
||||
defer c.Terminate(t)
|
||||
|
||||
for i := 0; i < 5; i++ {
|
||||
@ -47,8 +44,7 @@ func TestPauseMember(t *testing.T) {
|
||||
|
||||
func TestRestartMember(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
c := integration.NewClusterFromConfig(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
c.Launch(t)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer c.Terminate(t)
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
@ -69,14 +65,13 @@ func TestRestartMember(t *testing.T) {
|
||||
func TestLaunchDuplicateMemberShouldFail(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
size := 3
|
||||
c := integration.NewCluster(t, size)
|
||||
c := integration.NewCluster(t, &integration.ClusterConfig{Size: size})
|
||||
m := c.Members[0].Clone(t)
|
||||
var err error
|
||||
m.DataDir, err = os.MkdirTemp(t.TempDir(), "etcd")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
c.Launch(t)
|
||||
defer c.Terminate(t)
|
||||
|
||||
if err := m.Launch(); err == nil {
|
||||
@ -93,16 +88,14 @@ func TestSnapshotAndRestartMember(t *testing.T) {
|
||||
m.SnapshotCount = 100
|
||||
m.Launch()
|
||||
defer m.Terminate(t)
|
||||
defer m.Client.Close()
|
||||
m.WaitOK(t)
|
||||
|
||||
resps := make([]*client.Response, 120)
|
||||
var err error
|
||||
for i := 0; i < 120; i++ {
|
||||
cc := integration.MustNewHTTPClient(t, []string{m.URL()}, nil)
|
||||
kapi := client.NewKeysAPI(cc)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
key := fmt.Sprintf("foo%d", i)
|
||||
resps[i], err = kapi.Create(ctx, "/"+key, "bar")
|
||||
_, err = m.Client.Put(ctx, "/"+key, "bar")
|
||||
if err != nil {
|
||||
t.Fatalf("#%d: create on %s error: %v", i, m.URL(), err)
|
||||
}
|
||||
@ -113,18 +106,16 @@ func TestSnapshotAndRestartMember(t *testing.T) {
|
||||
|
||||
m.WaitOK(t)
|
||||
for i := 0; i < 120; i++ {
|
||||
cc := integration.MustNewHTTPClient(t, []string{m.URL()}, nil)
|
||||
kapi := client.NewKeysAPI(cc)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestTimeout)
|
||||
key := fmt.Sprintf("foo%d", i)
|
||||
resp, err := kapi.Get(ctx, "/"+key, nil)
|
||||
resp, err := m.Client.Get(ctx, "/"+key)
|
||||
if err != nil {
|
||||
t.Fatalf("#%d: get on %s error: %v", i, m.URL(), err)
|
||||
}
|
||||
cancel()
|
||||
|
||||
if !reflect.DeepEqual(resp.Node, resps[i].Node) {
|
||||
t.Errorf("#%d: node = %v, want %v", i, resp.Node, resps[i].Node)
|
||||
if len(resp.Kvs) != 1 || string(resp.Kvs[0].Value) != "bar" {
|
||||
t.Errorf("#%d: got = %v, want %v", i, resp.Kvs[0], "bar")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ import (
|
||||
// TestMetricDbSizeBoot checks that the db size metric is set on boot.
|
||||
func TestMetricDbSizeBoot(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
v, err := clus.Members[0].Metric("etcd_debugging_mvcc_db_total_size_in_bytes")
|
||||
@ -51,7 +51,7 @@ func TestMetricDbSizeDefrag(t *testing.T) {
|
||||
// testMetricDbSizeDefrag checks that the db size metric is set after defrag.
|
||||
func testMetricDbSizeDefrag(t *testing.T, name string) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.Client(0)).KV
|
||||
@ -165,7 +165,7 @@ func testMetricDbSizeDefrag(t *testing.T, name string) {
|
||||
|
||||
func TestMetricQuotaBackendBytes(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
qs, err := clus.Members[0].Metric("etcd_server_quota_backend_bytes")
|
||||
@ -183,7 +183,7 @@ func TestMetricQuotaBackendBytes(t *testing.T) {
|
||||
|
||||
func TestMetricsHealth(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
tr, err := transport.NewTransport(transport.TLSInfo{}, 5*time.Second)
|
||||
|
@ -25,7 +25,7 @@ import (
|
||||
func TestNetworkPartition5MembersLeaderInMinority(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 5})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 5})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
leadIndex := clus.WaitLeader(t)
|
||||
@ -34,8 +34,8 @@ func TestNetworkPartition5MembersLeaderInMinority(t *testing.T) {
|
||||
minority := []int{leadIndex, (leadIndex + 1) % 5}
|
||||
majority := []int{(leadIndex + 2) % 5, (leadIndex + 3) % 5, (leadIndex + 4) % 5}
|
||||
|
||||
minorityMembers := getMembersByIndexSlice(clus.Cluster, minority)
|
||||
majorityMembers := getMembersByIndexSlice(clus.Cluster, majority)
|
||||
minorityMembers := getMembersByIndexSlice(clus, minority)
|
||||
majorityMembers := getMembersByIndexSlice(clus, majority)
|
||||
|
||||
// network partition (bi-directional)
|
||||
injectPartition(t, minorityMembers, majorityMembers)
|
||||
@ -73,7 +73,7 @@ func TestNetworkPartition5MembersLeaderInMajority(t *testing.T) {
|
||||
func testNetworkPartition5MembersLeaderInMajority(t *testing.T) error {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 5})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 5})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
leadIndex := clus.WaitLeader(t)
|
||||
@ -82,8 +82,8 @@ func testNetworkPartition5MembersLeaderInMajority(t *testing.T) error {
|
||||
majority := []int{leadIndex, (leadIndex + 1) % 5, (leadIndex + 2) % 5}
|
||||
minority := []int{(leadIndex + 3) % 5, (leadIndex + 4) % 5}
|
||||
|
||||
majorityMembers := getMembersByIndexSlice(clus.Cluster, majority)
|
||||
minorityMembers := getMembersByIndexSlice(clus.Cluster, minority)
|
||||
majorityMembers := getMembersByIndexSlice(clus, majority)
|
||||
minorityMembers := getMembersByIndexSlice(clus, minority)
|
||||
|
||||
// network partition (bi-directional)
|
||||
injectPartition(t, majorityMembers, minorityMembers)
|
||||
@ -112,7 +112,7 @@ func testNetworkPartition5MembersLeaderInMajority(t *testing.T) error {
|
||||
func TestNetworkPartition4Members(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 4})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 4})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
leadIndex := clus.WaitLeader(t)
|
||||
@ -121,8 +121,8 @@ func TestNetworkPartition4Members(t *testing.T) {
|
||||
groupA := []int{leadIndex, (leadIndex + 1) % 4}
|
||||
groupB := []int{(leadIndex + 2) % 4, (leadIndex + 3) % 4}
|
||||
|
||||
leaderPartition := getMembersByIndexSlice(clus.Cluster, groupA)
|
||||
followerPartition := getMembersByIndexSlice(clus.Cluster, groupB)
|
||||
leaderPartition := getMembersByIndexSlice(clus, groupA)
|
||||
followerPartition := getMembersByIndexSlice(clus, groupB)
|
||||
|
||||
// network partition (bi-directional)
|
||||
injectPartition(t, leaderPartition, followerPartition)
|
||||
|
@ -33,7 +33,7 @@ import (
|
||||
func TestClusterProxyMemberList(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cts := newClusterProxyServer(zaptest.NewLogger(t), []string{clus.Members[0].GRPCURL()}, t)
|
||||
|
@ -30,7 +30,7 @@ import (
|
||||
func TestKVProxyRange(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvts := newKVProxyServer([]string{clus.Members[0].GRPCURL()}, t)
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
func TestRegister(t *testing.T) {
|
||||
integration2.BeforeTest(t)
|
||||
|
||||
clus := integration2.NewClusterV3(t, &integration2.ClusterConfig{Size: 1})
|
||||
clus := integration2.NewCluster(t, &integration2.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.Client(0)
|
||||
paddr := clus.Members[0].GRPCURL()
|
||||
|
@ -36,9 +36,8 @@ func TestV3StorageQuotaApply(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
quotasize := int64(16 * os.Getpagesize())
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
kvc0 := integration.ToGRPC(clus.Client(0)).KV
|
||||
kvc1 := integration.ToGRPC(clus.Client(1)).KV
|
||||
|
||||
// Set a quota on one node
|
||||
@ -46,6 +45,7 @@ func TestV3StorageQuotaApply(t *testing.T) {
|
||||
clus.Members[0].Stop(t)
|
||||
clus.Members[0].Restart(t)
|
||||
clus.WaitMembersForLeader(t, clus.Members)
|
||||
kvc0 := integration.ToGRPC(clus.Client(0)).KV
|
||||
waitForRestart(t, kvc0)
|
||||
|
||||
key := []byte("abc")
|
||||
@ -139,7 +139,7 @@ func TestV3StorageQuotaApply(t *testing.T) {
|
||||
func TestV3AlarmDeactivate(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
mt := integration.ToGRPC(clus.RandClient()).Maintenance
|
||||
@ -172,7 +172,7 @@ func TestV3AlarmDeactivate(t *testing.T) {
|
||||
|
||||
func TestV3CorruptAlarm(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
var wg sync.WaitGroup
|
||||
|
@ -32,7 +32,7 @@ import (
|
||||
// TestV3AuthEmptyUserGet ensures that a get with an empty user will return an empty user error.
|
||||
func TestV3AuthEmptyUserGet(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), 30*time.Second)
|
||||
@ -51,7 +51,7 @@ func TestV3AuthEmptyUserGet(t *testing.T) {
|
||||
// given a valid token when authentication is disabled
|
||||
func TestV3AuthTokenWithDisable(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
authSetupRoot(t, integration.ToGRPC(clus.Client(0)).Auth)
|
||||
@ -83,7 +83,7 @@ func TestV3AuthTokenWithDisable(t *testing.T) {
|
||||
|
||||
func TestV3AuthRevision(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
api := integration.ToGRPC(clus.Client(0))
|
||||
@ -122,7 +122,7 @@ func TestV3AuthWithLeaseRevokeWithRootJWT(t *testing.T) {
|
||||
func testV3AuthWithLeaseRevokeWithRoot(t *testing.T, ccfg integration.ClusterConfig) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &ccfg)
|
||||
clus := integration.NewCluster(t, &ccfg)
|
||||
defer clus.Terminate(t)
|
||||
|
||||
api := integration.ToGRPC(clus.Client(0))
|
||||
@ -179,7 +179,7 @@ type user struct {
|
||||
|
||||
func TestV3AuthWithLeaseRevoke(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
users := []user{
|
||||
@ -225,7 +225,7 @@ func TestV3AuthWithLeaseRevoke(t *testing.T) {
|
||||
|
||||
func TestV3AuthWithLeaseAttach(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
users := []user{
|
||||
@ -337,7 +337,7 @@ func authSetupRoot(t *testing.T, auth pb.AuthClient) {
|
||||
|
||||
func TestV3AuthNonAuthorizedRPCs(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
nonAuthedKV := clus.Client(0).KV
|
||||
@ -360,7 +360,7 @@ func TestV3AuthNonAuthorizedRPCs(t *testing.T) {
|
||||
func TestV3AuthOldRevConcurrent(t *testing.T) {
|
||||
t.Skip() // TODO(jingyih): re-enable the test when #10408 is fixed.
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
authSetupRoot(t, integration.ToGRPC(clus.Client(0)).Auth)
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
// TestElectionWait tests if followers can correctly wait for elections.
|
||||
func TestElectionWait(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
leaders := 3
|
||||
@ -110,7 +110,7 @@ func TestElectionWait(t *testing.T) {
|
||||
// TestElectionFailover tests that an election will
|
||||
func TestElectionFailover(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cctx, cancel := context.WithCancel(context.TODO())
|
||||
@ -120,7 +120,7 @@ func TestElectionFailover(t *testing.T) {
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
var err error
|
||||
ss[i], err = concurrency.NewSession(clus.Clients[i])
|
||||
ss[i], err = concurrency.NewSession(clus.Client(i))
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
@ -178,7 +178,7 @@ func TestElectionFailover(t *testing.T) {
|
||||
// with the same lock will Proclaim instead of deadlocking.
|
||||
func TestElectionSessionRecampaign(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.RandClient()
|
||||
|
||||
@ -211,7 +211,7 @@ func TestElectionSessionRecampaign(t *testing.T) {
|
||||
//
|
||||
func TestElectionOnPrefixOfExistingKey(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -238,7 +238,7 @@ func TestElectionOnPrefixOfExistingKey(t *testing.T) {
|
||||
// leadership.
|
||||
func TestElectionOnSessionRestart(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.RandClient()
|
||||
|
||||
@ -285,7 +285,7 @@ func TestElectionOnSessionRestart(t *testing.T) {
|
||||
// a leader key with a modrev less than the compaction revision.
|
||||
func TestElectionObserveCompacted(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
|
@ -32,7 +32,7 @@ import (
|
||||
// does not panic the mvcc backend while defragment is running.
|
||||
func TestV3MaintenanceDefragmentInflightRange(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -62,7 +62,7 @@ func TestV3MaintenanceDefragmentInflightRange(t *testing.T) {
|
||||
// See https://github.com/etcd-io/etcd/issues/7322 for more detail.
|
||||
func TestV3KVInflightRangeRequests(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
|
@ -41,7 +41,7 @@ import (
|
||||
// overwrites it, then checks that the change was applied.
|
||||
func TestV3PutOverwrite(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -89,7 +89,7 @@ func TestV3PutOverwrite(t *testing.T) {
|
||||
// TestPutRestart checks if a put after an unrelated member restart succeeds
|
||||
func TestV3PutRestart(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvIdx := rand.Intn(3)
|
||||
@ -100,14 +100,14 @@ func TestV3PutRestart(t *testing.T) {
|
||||
stopIdx = rand.Intn(3)
|
||||
}
|
||||
|
||||
clus.Clients[stopIdx].Close()
|
||||
clus.Client(stopIdx).Close()
|
||||
clus.Members[stopIdx].Stop(t)
|
||||
clus.Members[stopIdx].Restart(t)
|
||||
c, cerr := integration.NewClientV3(clus.Members[stopIdx])
|
||||
if cerr != nil {
|
||||
t.Fatalf("cannot create client: %v", cerr)
|
||||
}
|
||||
clus.Clients[stopIdx] = c
|
||||
clus.Members[stopIdx].ServerClient = c
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), 10*time.Second)
|
||||
defer cancel()
|
||||
@ -121,7 +121,7 @@ func TestV3PutRestart(t *testing.T) {
|
||||
// TestV3CompactCurrentRev ensures keys are present when compacting on current revision.
|
||||
func TestV3CompactCurrentRev(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -155,7 +155,7 @@ func TestV3CompactCurrentRev(t *testing.T) {
|
||||
// TestV3HashKV ensures that multiple calls of HashKV on same node return same hash and compact rev.
|
||||
func TestV3HashKV(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -204,7 +204,7 @@ func TestV3HashKV(t *testing.T) {
|
||||
func TestV3TxnTooManyOps(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
maxTxnOps := uint(128)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, MaxTxnOps: maxTxnOps})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, MaxTxnOps: maxTxnOps})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -279,7 +279,7 @@ func TestV3TxnTooManyOps(t *testing.T) {
|
||||
|
||||
func TestV3TxnDuplicateKeys(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
putreq := &pb.RequestOp{Request: &pb.RequestOp_RequestPut{RequestPut: &pb.PutRequest{Key: []byte("abc"), Value: []byte("def")}}}
|
||||
@ -397,7 +397,7 @@ func TestV3TxnDuplicateKeys(t *testing.T) {
|
||||
// Testv3TxnRevision tests that the transaction header revision is set as expected.
|
||||
func TestV3TxnRevision(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -448,7 +448,7 @@ func TestV3TxnRevision(t *testing.T) {
|
||||
// when compared to the Succeeded field in the txn response.
|
||||
func TestV3TxnCmpHeaderRev(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -504,7 +504,7 @@ func TestV3TxnCmpHeaderRev(t *testing.T) {
|
||||
// TestV3TxnRangeCompare tests range comparisons in txns
|
||||
func TestV3TxnRangeCompare(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// put keys, named by expected revision
|
||||
@ -615,7 +615,7 @@ func TestV3TxnRangeCompare(t *testing.T) {
|
||||
// TestV3TxnNested tests nested txns follow paths as expected.
|
||||
func TestV3TxnNestedPath(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -669,7 +669,7 @@ func TestV3TxnNestedPath(t *testing.T) {
|
||||
func TestV3PutIgnoreValue(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -802,7 +802,7 @@ func TestV3PutIgnoreValue(t *testing.T) {
|
||||
func TestV3PutIgnoreLease(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -941,7 +941,7 @@ func TestV3PutIgnoreLease(t *testing.T) {
|
||||
// TestV3PutMissingLease ensures that a Put on a key with a bogus lease fails.
|
||||
func TestV3PutMissingLease(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -1069,7 +1069,7 @@ func TestV3DeleteRange(t *testing.T) {
|
||||
|
||||
for i, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
defer clus.Terminate(t)
|
||||
|
||||
@ -1124,7 +1124,7 @@ func TestV3DeleteRange(t *testing.T) {
|
||||
// TestV3TxnInvalidRange tests that invalid ranges are rejected in txns.
|
||||
func TestV3TxnInvalidRange(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -1168,7 +1168,7 @@ func TestV3TxnInvalidRange(t *testing.T) {
|
||||
func TestV3TooLargeRequest(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -1186,7 +1186,7 @@ func TestV3TooLargeRequest(t *testing.T) {
|
||||
// TestV3Hash tests hash.
|
||||
func TestV3Hash(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -1211,7 +1211,7 @@ func TestV3Hash(t *testing.T) {
|
||||
// TestV3HashRestart ensures that hash stays the same after restart.
|
||||
func TestV3HashRestart(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -1244,7 +1244,7 @@ func TestV3StorageQuotaAPI(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
quotasize := int64(16 * os.Getpagesize())
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
|
||||
// Set a quota on one node
|
||||
clus.Members[0].QuotaBackendBytes = quotasize
|
||||
@ -1508,7 +1508,7 @@ func TestV3RangeRequest(t *testing.T) {
|
||||
|
||||
for i, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
for _, k := range tt.putKeys {
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -1550,19 +1550,11 @@ func TestV3RangeRequest(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func newClusterV3NoClients(t *testing.T, cfg *integration.ClusterConfig) *integration.ClusterV3 {
|
||||
cfg.UseGRPC = true
|
||||
clus := &integration.ClusterV3{Cluster: integration.NewClusterByConfig(t, cfg)}
|
||||
clus.Launch(t)
|
||||
return clus
|
||||
}
|
||||
|
||||
// TestTLSGRPCRejectInsecureClient checks that connection is rejected if server is TLS but not client.
|
||||
func TestTLSGRPCRejectInsecureClient(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
cfg := integration.ClusterConfig{Size: 3, ClientTLS: &integration.TestTLSInfo}
|
||||
clus := newClusterV3NoClients(t, &cfg)
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, ClientTLS: &integration.TestTLSInfo})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// nil out TLS field so client will use an insecure connection
|
||||
@ -1596,8 +1588,7 @@ func TestTLSGRPCRejectInsecureClient(t *testing.T) {
|
||||
func TestTLSGRPCRejectSecureClient(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
cfg := integration.ClusterConfig{Size: 3}
|
||||
clus := newClusterV3NoClients(t, &cfg)
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.Members[0].ClientTLSInfo = &integration.TestTLSInfo
|
||||
@ -1616,8 +1607,7 @@ func TestTLSGRPCRejectSecureClient(t *testing.T) {
|
||||
func TestTLSGRPCAcceptSecureAll(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
cfg := integration.ClusterConfig{Size: 3, ClientTLS: &integration.TestTLSInfo}
|
||||
clus := newClusterV3NoClients(t, &cfg)
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, ClientTLS: &integration.TestTLSInfo})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
client, err := integration.NewClientV3(clus.Members[0])
|
||||
@ -1763,7 +1753,7 @@ func testTLSReload(
|
||||
tlsInfo := cloneFunc()
|
||||
|
||||
// 2. start cluster with valid certs
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{
|
||||
Size: 1,
|
||||
PeerTLS: &tlsInfo,
|
||||
ClientTLS: &tlsInfo,
|
||||
@ -1834,8 +1824,7 @@ func testTLSReload(
|
||||
func TestGRPCRequireLeader(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
cfg := integration.ClusterConfig{Size: 3}
|
||||
clus := newClusterV3NoClients(t, &cfg)
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.Members[1].Stop(t)
|
||||
@ -1861,8 +1850,7 @@ func TestGRPCRequireLeader(t *testing.T) {
|
||||
func TestGRPCStreamRequireLeader(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
cfg := integration.ClusterConfig{Size: 3, UseBridge: true}
|
||||
clus := newClusterV3NoClients(t, &cfg)
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
client, err := integration.NewClientV3(clus.Members[0])
|
||||
@ -1936,7 +1924,7 @@ func TestV3LargeRequests(t *testing.T) {
|
||||
}
|
||||
for i, test := range tests {
|
||||
t.Run(fmt.Sprintf("#%d", i), func(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, MaxRequestBytes: test.maxRequestBytes})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, MaxRequestBytes: test.maxRequestBytes})
|
||||
defer clus.Terminate(t)
|
||||
kvcli := integration.ToGRPC(clus.Client(0)).KV
|
||||
reqput := &pb.PutRequest{Key: []byte("foo"), Value: make([]byte, test.valueSize)}
|
||||
|
@ -25,7 +25,7 @@ import (
|
||||
func TestHealthCheck(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := healthpb.NewHealthClient(clus.RandClient().ActiveConnection())
|
||||
|
@ -13,7 +13,7 @@ import (
|
||||
func TestKVWithEmptyValue(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
client := clus.RandClient()
|
||||
|
@ -33,7 +33,7 @@ func TestMoveLeaderService(t *testing.T) { testMoveLeader(t, false) }
|
||||
func testMoveLeader(t *testing.T, auto bool) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
oldLeadIdx := clus.WaitLeader(t)
|
||||
@ -101,7 +101,7 @@ func testMoveLeader(t *testing.T, auto bool) {
|
||||
func TestMoveLeaderError(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
oldLeadIdx := clus.WaitLeader(t)
|
||||
@ -120,7 +120,7 @@ func TestMoveLeaderError(t *testing.T) {
|
||||
func TestMoveLeaderToLearnerError(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// we have to add and launch learner member after initial cluster was created, because
|
||||
@ -153,7 +153,7 @@ func TestMoveLeaderToLearnerError(t *testing.T) {
|
||||
func TestTransferLeadershipWithLearner(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.AddAndLaunchLearnerMember(t)
|
||||
@ -187,7 +187,7 @@ func TestFirstCommitNotification(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
ctx := context.Background()
|
||||
clusterSize := 3
|
||||
cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: clusterSize})
|
||||
cluster := integration.NewCluster(t, &integration.ClusterConfig{Size: clusterSize})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
oldLeaderIdx := cluster.WaitLeader(t)
|
||||
|
@ -37,7 +37,7 @@ import (
|
||||
func TestV3LeasePromote(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// create lease
|
||||
@ -98,7 +98,7 @@ func TestV3LeasePromote(t *testing.T) {
|
||||
// TestV3LeaseRevoke ensures a key is deleted once its lease is revoked.
|
||||
func TestV3LeaseRevoke(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
testLeaseRemoveLeasedKey(t, func(clus *integration.ClusterV3, leaseID int64) error {
|
||||
testLeaseRemoveLeasedKey(t, func(clus *integration.Cluster, leaseID int64) error {
|
||||
lc := integration.ToGRPC(clus.RandClient()).Lease
|
||||
_, err := lc.LeaseRevoke(context.TODO(), &pb.LeaseRevokeRequest{ID: leaseID})
|
||||
return err
|
||||
@ -108,7 +108,7 @@ func TestV3LeaseRevoke(t *testing.T) {
|
||||
// TestV3LeaseGrantById ensures leases may be created by a given id.
|
||||
func TestV3LeaseGrantByID(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// create fixed lease
|
||||
@ -145,7 +145,7 @@ func TestV3LeaseGrantByID(t *testing.T) {
|
||||
// TestV3LeaseExpire ensures a key is deleted once a key expires.
|
||||
func TestV3LeaseExpire(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
testLeaseRemoveLeasedKey(t, func(clus *integration.ClusterV3, leaseID int64) error {
|
||||
testLeaseRemoveLeasedKey(t, func(clus *integration.Cluster, leaseID int64) error {
|
||||
// let lease lapse; wait for deleted key
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
@ -197,7 +197,7 @@ func TestV3LeaseExpire(t *testing.T) {
|
||||
// TestV3LeaseKeepAlive ensures keepalive keeps the lease alive.
|
||||
func TestV3LeaseKeepAlive(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
testLeaseRemoveLeasedKey(t, func(clus *integration.ClusterV3, leaseID int64) error {
|
||||
testLeaseRemoveLeasedKey(t, func(clus *integration.Cluster, leaseID int64) error {
|
||||
lc := integration.ToGRPC(clus.RandClient()).Lease
|
||||
lreq := &pb.LeaseKeepAliveRequest{ID: leaseID}
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
@ -284,7 +284,7 @@ func TestV3LeaseCheckpoint(t *testing.T) {
|
||||
EnableLeaseCheckpoint: tc.checkpointingEnabled,
|
||||
LeaseCheckpointInterval: tc.checkpointingInterval,
|
||||
}
|
||||
clus := integration.NewClusterV3(t, config)
|
||||
clus := integration.NewCluster(t, config)
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// create lease
|
||||
@ -339,7 +339,7 @@ func TestV3LeaseCheckpoint(t *testing.T) {
|
||||
// TestV3LeaseExists creates a lease on a random client and confirms it exists in the cluster.
|
||||
func TestV3LeaseExists(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// create lease
|
||||
@ -363,7 +363,7 @@ func TestV3LeaseExists(t *testing.T) {
|
||||
// TestV3LeaseLeases creates leases and confirms list RPC fetches created ones.
|
||||
func TestV3LeaseLeases(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx0, cancel0 := context.WithCancel(context.Background())
|
||||
@ -413,7 +413,7 @@ func TestV3LeaseTimeToLiveStress(t *testing.T) {
|
||||
|
||||
func testLeaseStress(t *testing.T, stresser func(context.Context, pb.LeaseClient) error) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||
@ -484,7 +484,7 @@ func stressLeaseTimeToLive(tctx context.Context, lc pb.LeaseClient) (reterr erro
|
||||
|
||||
func TestV3PutOnNonExistLease(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
@ -502,7 +502,7 @@ func TestV3PutOnNonExistLease(t *testing.T) {
|
||||
// related issue https://github.com/etcd-io/etcd/issues/6537
|
||||
func TestV3GetNonExistLease(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
@ -522,12 +522,12 @@ func TestV3GetNonExistLease(t *testing.T) {
|
||||
Keys: true,
|
||||
}
|
||||
|
||||
for _, client := range clus.Clients {
|
||||
for _, m := range clus.Members {
|
||||
// quorum-read to ensure revoke completes before TimeToLive
|
||||
if _, err := integration.ToGRPC(client).KV.Range(ctx, &pb.RangeRequest{Key: []byte("_")}); err != nil {
|
||||
if _, err := integration.ToGRPC(m.Client).KV.Range(ctx, &pb.RangeRequest{Key: []byte("_")}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
resp, err := integration.ToGRPC(client).Lease.LeaseTimeToLive(ctx, leaseTTLr)
|
||||
resp, err := integration.ToGRPC(m.Client).Lease.LeaseTimeToLive(ctx, leaseTTLr)
|
||||
if err != nil {
|
||||
t.Fatalf("expected non nil error, but go %v", err)
|
||||
}
|
||||
@ -540,7 +540,7 @@ func TestV3GetNonExistLease(t *testing.T) {
|
||||
// TestV3LeaseSwitch tests a key can be switched from one lease to another.
|
||||
func TestV3LeaseSwitch(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
key := "foo"
|
||||
@ -603,7 +603,7 @@ func TestV3LeaseSwitch(t *testing.T) {
|
||||
func TestV3LeaseFailover(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
toIsolate := clus.WaitMembersForLeader(t, clus.Members)
|
||||
@ -664,7 +664,7 @@ func TestV3LeaseFailover(t *testing.T) {
|
||||
func TestV3LeaseRequireLeader(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lc := integration.ToGRPC(clus.Client(0)).Lease
|
||||
@ -704,7 +704,7 @@ const fiveMinTTL int64 = 300
|
||||
func TestV3LeaseRecoverAndRevoke(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.Client(0)).KV
|
||||
@ -755,7 +755,7 @@ func TestV3LeaseRecoverAndRevoke(t *testing.T) {
|
||||
func TestV3LeaseRevokeAndRecover(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.Client(0)).KV
|
||||
@ -807,7 +807,7 @@ func TestV3LeaseRevokeAndRecover(t *testing.T) {
|
||||
func TestV3LeaseRecoverKeyWithDetachedLease(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.Client(0)).KV
|
||||
@ -863,7 +863,7 @@ func TestV3LeaseRecoverKeyWithDetachedLease(t *testing.T) {
|
||||
func TestV3LeaseRecoverKeyWithMutipleLease(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.Client(0)).KV
|
||||
@ -935,7 +935,7 @@ func TestV3LeaseRecoverKeyWithMutipleLease(t *testing.T) {
|
||||
}
|
||||
|
||||
// acquireLeaseAndKey creates a new lease and creates an attached key.
|
||||
func acquireLeaseAndKey(clus *integration.ClusterV3, key string) (int64, error) {
|
||||
func acquireLeaseAndKey(clus *integration.Cluster, key string) (int64, error) {
|
||||
// create lease
|
||||
lresp, err := integration.ToGRPC(clus.RandClient()).Lease.LeaseGrant(
|
||||
context.TODO(),
|
||||
@ -956,8 +956,8 @@ func acquireLeaseAndKey(clus *integration.ClusterV3, key string) (int64, error)
|
||||
|
||||
// testLeaseRemoveLeasedKey performs some action while holding a lease with an
|
||||
// attached key "foo", then confirms the key is gone.
|
||||
func testLeaseRemoveLeasedKey(t *testing.T, act func(*integration.ClusterV3, int64) error) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
func testLeaseRemoveLeasedKey(t *testing.T, act func(*integration.Cluster, int64) error) {
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
leaseID, err := acquireLeaseAndKey(clus, "foo")
|
||||
@ -980,7 +980,7 @@ func testLeaseRemoveLeasedKey(t *testing.T, act func(*integration.ClusterV3, int
|
||||
}
|
||||
}
|
||||
|
||||
func leaseExist(t *testing.T, clus *integration.ClusterV3, leaseID int64) bool {
|
||||
func leaseExist(t *testing.T, clus *integration.Cluster, leaseID int64) bool {
|
||||
l := integration.ToGRPC(clus.RandClient()).Lease
|
||||
|
||||
_, err := l.LeaseGrant(context.Background(), &pb.LeaseGrantRequest{ID: leaseID, TTL: 5})
|
||||
|
@ -31,7 +31,7 @@ import (
|
||||
func TestSTMConflict(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
etcdc := clus.RandClient()
|
||||
@ -99,7 +99,7 @@ func TestSTMConflict(t *testing.T) {
|
||||
func TestSTMPutNewKey(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
etcdc := clus.RandClient()
|
||||
@ -126,7 +126,7 @@ func TestSTMPutNewKey(t *testing.T) {
|
||||
func TestSTMAbort(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
etcdc := clus.RandClient()
|
||||
@ -157,7 +157,7 @@ func TestSTMAbort(t *testing.T) {
|
||||
func TestSTMSerialize(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
etcdc := clus.RandClient()
|
||||
@ -220,7 +220,7 @@ func TestSTMSerialize(t *testing.T) {
|
||||
func TestSTMApplyOnConcurrentDeletion(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
etcdc := clus.RandClient()
|
||||
@ -269,7 +269,7 @@ func TestSTMApplyOnConcurrentDeletion(t *testing.T) {
|
||||
func TestSTMSerializableSnapshotPut(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
|
@ -48,7 +48,7 @@ func testTLSCipherSuites(t *testing.T, valid bool) {
|
||||
srvTLS.CipherSuites, cliTLS.CipherSuites = cipherSuites[:2], cipherSuites[2:]
|
||||
}
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, ClientTLS: &srvTLS})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, ClientTLS: &srvTLS})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cc, err := cliTLS.ClientConfig()
|
||||
|
@ -53,7 +53,7 @@ func MustFetchNotEmptyMetric(tb testing.TB, member *integration.Member, metric s
|
||||
func TestV3WatchRestoreSnapshotUnsync(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{
|
||||
Size: 3,
|
||||
SnapshotCount: 10,
|
||||
SnapshotCatchUpEntries: 5,
|
||||
|
@ -207,7 +207,7 @@ func TestV3WatchFromCurrentRevision(t *testing.T) {
|
||||
|
||||
for i, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wAPI := integration.ToGRPC(clus.RandClient()).Watch
|
||||
@ -294,7 +294,7 @@ func TestV3WatchFromCurrentRevision(t *testing.T) {
|
||||
func TestV3WatchFutureRevision(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wAPI := integration.ToGRPC(clus.RandClient()).Watch
|
||||
@ -355,7 +355,7 @@ func TestV3WatchFutureRevision(t *testing.T) {
|
||||
func TestV3WatchWrongRange(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wAPI := integration.ToGRPC(clus.RandClient()).Watch
|
||||
@ -409,7 +409,7 @@ func TestV3WatchCancelUnsynced(t *testing.T) {
|
||||
}
|
||||
|
||||
func testV3WatchCancel(t *testing.T, startRev int64) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
@ -465,7 +465,7 @@ func testV3WatchCancel(t *testing.T, startRev int64) {
|
||||
// overlapping puts.
|
||||
func TestV3WatchCurrentPutOverlap(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
@ -550,7 +550,7 @@ func TestV3WatchCurrentPutOverlap(t *testing.T) {
|
||||
func TestV3WatchEmptyKey(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
@ -608,7 +608,7 @@ func TestV3WatchMultipleWatchersUnsynced(t *testing.T) {
|
||||
// that matches all watchers, and another key that matches only
|
||||
// one watcher to test if it receives expected events.
|
||||
func testV3WatchMultipleWatchers(t *testing.T, startRev int64) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -710,7 +710,7 @@ func TestV3WatchMultipleEventsTxnUnsynced(t *testing.T) {
|
||||
|
||||
// testV3WatchMultipleEventsTxn tests Watch APIs when it receives multiple events.
|
||||
func testV3WatchMultipleEventsTxn(t *testing.T, startRev int64) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
@ -793,7 +793,7 @@ func (evs eventsSortByKey) Less(i, j int) bool {
|
||||
|
||||
func TestV3WatchMultipleEventsPutUnsynced(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := integration.ToGRPC(clus.RandClient()).KV
|
||||
@ -882,7 +882,7 @@ func TestV3WatchMultipleStreamsUnsynced(t *testing.T) {
|
||||
|
||||
// testV3WatchMultipleStreams tests multiple watchers on the same key on multiple streams.
|
||||
func testV3WatchMultipleStreams(t *testing.T, startRev int64) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wAPI := integration.ToGRPC(clus.RandClient()).Watch
|
||||
@ -985,7 +985,7 @@ func TestWatchWithProgressNotify(t *testing.T) {
|
||||
defer func() { v3rpc.SetProgressReportInterval(oldpi) }()
|
||||
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
@ -1035,7 +1035,7 @@ func TestWatchWithProgressNotify(t *testing.T) {
|
||||
// TestV3WatcMultiOpenhClose opens many watchers concurrently on multiple streams.
|
||||
func TestV3WatchClose(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.Client(0)
|
||||
@ -1071,7 +1071,7 @@ func TestV3WatchClose(t *testing.T) {
|
||||
func TestV3WatchWithFilter(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
@ -1139,7 +1139,7 @@ func TestV3WatchWithFilter(t *testing.T) {
|
||||
|
||||
func TestV3WatchWithPrevKV(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wctx, wcancel := context.WithCancel(context.Background())
|
||||
@ -1214,7 +1214,7 @@ func TestV3WatchWithPrevKV(t *testing.T) {
|
||||
func TestV3WatchCancellation(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
@ -1256,7 +1256,7 @@ func TestV3WatchCancellation(t *testing.T) {
|
||||
func TestV3WatchCloseCancelRace(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
|
@ -29,7 +29,7 @@ import (
|
||||
// simultaneous leadership to multiple campaigners.
|
||||
func TestV3ElectionCampaign(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lease1, err1 := integration.ToGRPC(clus.RandClient()).Lease.LeaseGrant(context.TODO(), &pb.LeaseGrantRequest{TTL: 30})
|
||||
@ -91,7 +91,7 @@ func TestV3ElectionCampaign(t *testing.T) {
|
||||
// proclamations from different leaders uninterrupted.
|
||||
func TestV3ElectionObserve(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lc := integration.ToGRPC(clus.Client(0)).Election
|
||||
|
@ -28,7 +28,7 @@ import (
|
||||
// once it is unlocked.
|
||||
func TestV3LockLockWaiter(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewCluster(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lease1, err1 := integration.ToGRPC(clus.RandClient()).Lease.LeaseGrant(context.TODO(), &pb.LeaseGrantRequest{TTL: 30})
|
||||
|
Loading…
x
Reference in New Issue
Block a user