mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #13359 from serathius/authority
Fix http2 authority header in single endpoint scenario
This commit is contained in:
commit
f3cfe0f1a5
@ -16,6 +16,13 @@ See [code changes](https://github.com/etcd-io/etcd/compare/v3.5.0...v3.5.1) and
|
||||
|
||||
- Fix [self-signed-cert-validity parameter cannot be specified in the config file](https://github.com/etcd-io/etcd/pull/13237).
|
||||
|
||||
### etcd client
|
||||
|
||||
- [Fix etcd client sends invalid :authority header](https://github.com/etcd-io/etcd/issues/13192)
|
||||
|
||||
### package clientv3
|
||||
|
||||
- Endpoints self identify now as `etcd-endpoints://{id}/{authority}` where authority is based on first endpoint passed, for example `etcd-endpoints://0xc0009d8540/localhost:2079`
|
||||
|
||||
<hr>
|
||||
|
||||
|
@ -297,9 +297,7 @@ func (c *Client) dial(creds grpccredentials.TransportCredentials, dopts ...grpc.
|
||||
dctx, cancel = context.WithTimeout(c.ctx, c.cfg.DialTimeout)
|
||||
defer cancel() // TODO: Is this right for cases where grpc.WithBlock() is not set on the dial options?
|
||||
}
|
||||
|
||||
initialEndpoints := strings.Join(c.Endpoints(), ";")
|
||||
target := fmt.Sprintf("%s://%p/#initially=[%s]", resolver.Schema, c, initialEndpoints)
|
||||
target := fmt.Sprintf("%s://%p/%s", resolver.Schema, c, authority(c.endpoints[0]))
|
||||
conn, err := grpc.DialContext(dctx, target, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -307,6 +305,20 @@ func (c *Client) dial(creds grpccredentials.TransportCredentials, dopts ...grpc.
|
||||
return conn, nil
|
||||
}
|
||||
|
||||
func authority(endpoint string) string {
|
||||
spl := strings.SplitN(endpoint, "://", 2)
|
||||
if len(spl) < 2 {
|
||||
if strings.HasPrefix(endpoint, "unix:") {
|
||||
return endpoint[len("unix:"):]
|
||||
}
|
||||
if strings.HasPrefix(endpoint, "unixs:") {
|
||||
return endpoint[len("unixs:"):]
|
||||
}
|
||||
return endpoint
|
||||
}
|
||||
return spl[1]
|
||||
}
|
||||
|
||||
func (c *Client) credentialsForEndpoint(ep string) grpccredentials.TransportCredentials {
|
||||
r := endpoint.RequiresCredentials(ep)
|
||||
switch r {
|
||||
|
69
pkg/grpc_testing/recorder.go
Normal file
69
pkg/grpc_testing/recorder.go
Normal file
@ -0,0 +1,69 @@
|
||||
// Copyright 2021 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package grpc_testing
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/metadata"
|
||||
)
|
||||
|
||||
type GrpcRecorder struct {
|
||||
mux sync.RWMutex
|
||||
requests []RequestInfo
|
||||
}
|
||||
|
||||
type RequestInfo struct {
|
||||
FullMethod string
|
||||
Authority string
|
||||
}
|
||||
|
||||
func (ri *GrpcRecorder) UnaryInterceptor() grpc.UnaryServerInterceptor {
|
||||
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
|
||||
ri.record(toRequestInfo(ctx, info))
|
||||
resp, err := handler(ctx, req)
|
||||
return resp, err
|
||||
}
|
||||
}
|
||||
|
||||
func (ri *GrpcRecorder) RecordedRequests() []RequestInfo {
|
||||
ri.mux.RLock()
|
||||
defer ri.mux.RUnlock()
|
||||
reqs := make([]RequestInfo, len(ri.requests))
|
||||
copy(reqs, ri.requests)
|
||||
return reqs
|
||||
}
|
||||
|
||||
func toRequestInfo(ctx context.Context, info *grpc.UnaryServerInfo) RequestInfo {
|
||||
req := RequestInfo{
|
||||
FullMethod: info.FullMethod,
|
||||
}
|
||||
md, ok := metadata.FromIncomingContext(ctx)
|
||||
if ok {
|
||||
as := md.Get(":authority")
|
||||
if len(as) != 0 {
|
||||
req.Authority = as[0]
|
||||
}
|
||||
}
|
||||
return req
|
||||
}
|
||||
|
||||
func (ri *GrpcRecorder) record(r RequestInfo) {
|
||||
ri.mux.Lock()
|
||||
defer ri.mux.Unlock()
|
||||
ri.requests = append(ri.requests, r)
|
||||
}
|
@ -539,7 +539,7 @@ func (e *Etcd) servePeers() (err error) {
|
||||
|
||||
for _, p := range e.Peers {
|
||||
u := p.Listener.Addr().String()
|
||||
gs := v3rpc.Server(e.Server, peerTLScfg)
|
||||
gs := v3rpc.Server(e.Server, peerTLScfg, nil)
|
||||
m := cmux.New(p.Listener)
|
||||
go gs.Serve(m.Match(cmux.HTTP2()))
|
||||
srv := &http.Server{
|
||||
|
@ -110,7 +110,7 @@ func (sctx *serveCtx) serve(
|
||||
}()
|
||||
|
||||
if sctx.insecure {
|
||||
gs = v3rpc.Server(s, nil, gopts...)
|
||||
gs = v3rpc.Server(s, nil, nil, gopts...)
|
||||
v3electionpb.RegisterElectionServer(gs, servElection)
|
||||
v3lockpb.RegisterLockServer(gs, servLock)
|
||||
if sctx.serviceRegister != nil {
|
||||
@ -148,7 +148,7 @@ func (sctx *serveCtx) serve(
|
||||
if tlsErr != nil {
|
||||
return tlsErr
|
||||
}
|
||||
gs = v3rpc.Server(s, tlscfg, gopts...)
|
||||
gs = v3rpc.Server(s, tlscfg, nil, gopts...)
|
||||
v3electionpb.RegisterElectionServer(gs, servElection)
|
||||
v3lockpb.RegisterLockServer(gs, servLock)
|
||||
if sctx.serviceRegister != nil {
|
||||
|
@ -36,19 +36,21 @@ const (
|
||||
maxSendBytes = math.MaxInt32
|
||||
)
|
||||
|
||||
func Server(s *etcdserver.EtcdServer, tls *tls.Config, gopts ...grpc.ServerOption) *grpc.Server {
|
||||
func Server(s *etcdserver.EtcdServer, tls *tls.Config, interceptor grpc.UnaryServerInterceptor, gopts ...grpc.ServerOption) *grpc.Server {
|
||||
var opts []grpc.ServerOption
|
||||
opts = append(opts, grpc.CustomCodec(&codec{}))
|
||||
if tls != nil {
|
||||
bundle := credentials.NewBundle(credentials.Config{TLSConfig: tls})
|
||||
opts = append(opts, grpc.Creds(bundle.TransportCredentials()))
|
||||
}
|
||||
|
||||
chainUnaryInterceptors := []grpc.UnaryServerInterceptor{
|
||||
newLogUnaryInterceptor(s),
|
||||
newUnaryInterceptor(s),
|
||||
grpc_prometheus.UnaryServerInterceptor,
|
||||
}
|
||||
if interceptor != nil {
|
||||
chainUnaryInterceptors = append(chainUnaryInterceptors, interceptor)
|
||||
}
|
||||
|
||||
chainStreamInterceptors := []grpc.StreamServerInterceptor{
|
||||
newStreamInterceptor(s),
|
||||
|
@ -76,7 +76,7 @@ func newLogUnaryInterceptor(s *etcdserver.EtcdServer) grpc.UnaryServerIntercepto
|
||||
startTime := time.Now()
|
||||
resp, err := handler(ctx, req)
|
||||
lg := s.Logger()
|
||||
if lg != nil { // acquire stats if debug level is enabled or request is expensive
|
||||
if lg != nil { // acquire stats if debug level is enabled or RequestInfo is expensive
|
||||
defer logUnaryRequestStats(ctx, lg, s.Cfg.WarningUnaryRequestDuration, info, startTime, req, resp)
|
||||
}
|
||||
return resp, err
|
||||
|
@ -115,6 +115,10 @@ func (p *proxyEtcdProcess) WithStopSignal(sig os.Signal) os.Signal {
|
||||
return p.etcdProc.WithStopSignal(sig)
|
||||
}
|
||||
|
||||
func (p *proxyEtcdProcess) Logs() logsExpect {
|
||||
return p.etcdProc.Logs()
|
||||
}
|
||||
|
||||
type proxyProc struct {
|
||||
lg *zap.Logger
|
||||
execPath string
|
||||
|
@ -144,6 +144,7 @@ type etcdProcessClusterConfig struct {
|
||||
execPath string
|
||||
dataDirPath string
|
||||
keepDataDir bool
|
||||
envVars map[string]string
|
||||
|
||||
clusterSize int
|
||||
|
||||
@ -318,6 +319,7 @@ func (cfg *etcdProcessClusterConfig) etcdServerProcessConfigs(tb testing.TB) []*
|
||||
lg: lg,
|
||||
execPath: cfg.execPath,
|
||||
args: args,
|
||||
envVars: cfg.envVars,
|
||||
tlsArgs: cfg.tlsArgs(),
|
||||
dataDirPath: dataDirPath,
|
||||
keepDataDir: cfg.keepDataDir,
|
||||
|
213
tests/e2e/ctl_v3_grpc_test.go
Normal file
213
tests/e2e/ctl_v3_grpc_test.go
Normal file
@ -0,0 +1,213 @@
|
||||
// Copyright 2021 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
//go:build !cluster_proxy
|
||||
// +build !cluster_proxy
|
||||
|
||||
package e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.etcd.io/etcd/client/pkg/v3/testutil"
|
||||
)
|
||||
|
||||
func TestAuthority(t *testing.T) {
|
||||
tcs := []struct {
|
||||
name string
|
||||
useTLS bool
|
||||
useInsecureTLS bool
|
||||
// Pattern used to generate endpoints for client. Fields filled
|
||||
// %d - will be filled with member grpc port
|
||||
clientURLPattern string
|
||||
|
||||
// Pattern used to validate authority received by server. Fields filled:
|
||||
// %d - will be filled with first member grpc port
|
||||
expectAuthorityPattern string
|
||||
}{
|
||||
{
|
||||
name: "http://domain[:port]",
|
||||
clientURLPattern: "http://localhost:%d",
|
||||
expectAuthorityPattern: "localhost:%d",
|
||||
},
|
||||
{
|
||||
name: "http://address[:port]",
|
||||
clientURLPattern: "http://127.0.0.1:%d",
|
||||
expectAuthorityPattern: "127.0.0.1:%d",
|
||||
},
|
||||
{
|
||||
name: "https://domain[:port] insecure",
|
||||
useTLS: true,
|
||||
useInsecureTLS: true,
|
||||
clientURLPattern: "https://localhost:%d",
|
||||
expectAuthorityPattern: "localhost:%d",
|
||||
},
|
||||
{
|
||||
name: "https://address[:port] insecure",
|
||||
useTLS: true,
|
||||
useInsecureTLS: true,
|
||||
clientURLPattern: "https://127.0.0.1:%d",
|
||||
expectAuthorityPattern: "127.0.0.1:%d",
|
||||
},
|
||||
{
|
||||
name: "https://domain[:port]",
|
||||
useTLS: true,
|
||||
clientURLPattern: "https://localhost:%d",
|
||||
expectAuthorityPattern: "localhost:%d",
|
||||
},
|
||||
{
|
||||
name: "https://address[:port]",
|
||||
useTLS: true,
|
||||
clientURLPattern: "https://127.0.0.1:%d",
|
||||
expectAuthorityPattern: "127.0.0.1:%d",
|
||||
},
|
||||
}
|
||||
for _, tc := range tcs {
|
||||
for _, clusterSize := range []int{1, 3} {
|
||||
t.Run(fmt.Sprintf("Size: %d, Scenario: %q", clusterSize, tc.name), func(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
cfg := newConfigNoTLS()
|
||||
cfg.clusterSize = clusterSize
|
||||
if tc.useTLS {
|
||||
cfg.clientTLS = clientTLS
|
||||
}
|
||||
cfg.isClientAutoTLS = tc.useInsecureTLS
|
||||
// Enable debug mode to get logs with http2 headers (including authority)
|
||||
cfg.envVars = map[string]string{"GODEBUG": "http2debug=2"}
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
defer epc.Close()
|
||||
endpoints := templateEndpoints(t, tc.clientURLPattern, epc)
|
||||
|
||||
client := clusterEtcdctlV3(cfg, endpoints)
|
||||
err = client.Put("foo", "bar")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
executeWithTimeout(t, 5*time.Second, func() {
|
||||
assertAuthority(t, fmt.Sprintf(tc.expectAuthorityPattern, 20000), epc)
|
||||
})
|
||||
})
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func templateEndpoints(t *testing.T, pattern string, clus *etcdProcessCluster) []string {
|
||||
t.Helper()
|
||||
endpoints := []string{}
|
||||
for i := 0; i < clus.cfg.clusterSize; i++ {
|
||||
ent := pattern
|
||||
if strings.Contains(ent, "%d") {
|
||||
ent = fmt.Sprintf(ent, etcdProcessBasePort+i*5)
|
||||
}
|
||||
if strings.Contains(ent, "%") {
|
||||
t.Fatalf("Failed to template pattern, %% symbol left %q", ent)
|
||||
}
|
||||
endpoints = append(endpoints, ent)
|
||||
}
|
||||
return endpoints
|
||||
}
|
||||
|
||||
func assertAuthority(t *testing.T, expectAurhority string, clus *etcdProcessCluster) {
|
||||
logs := []logsExpect{}
|
||||
for _, proc := range clus.procs {
|
||||
logs = append(logs, proc.Logs())
|
||||
}
|
||||
line := firstMatch(t, `http2: decoded hpack field header field ":authority"`, logs...)
|
||||
line = strings.TrimSuffix(line, "\n")
|
||||
line = strings.TrimSuffix(line, "\r")
|
||||
expectLine := fmt.Sprintf(`http2: decoded hpack field header field ":authority" = %q`, expectAurhority)
|
||||
assert.True(t, strings.HasSuffix(line, expectLine), fmt.Sprintf("Got %q expected suffix %q", line, expectLine))
|
||||
}
|
||||
|
||||
func firstMatch(t *testing.T, expectLine string, logs ...logsExpect) string {
|
||||
t.Helper()
|
||||
match := make(chan string, len(logs))
|
||||
for i := range logs {
|
||||
go func(l logsExpect) {
|
||||
line, _ := l.Expect(expectLine)
|
||||
match <- line
|
||||
}(logs[i])
|
||||
}
|
||||
return <-match
|
||||
}
|
||||
|
||||
func executeWithTimeout(t *testing.T, timeout time.Duration, f func()) {
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
defer close(donec)
|
||||
f()
|
||||
}()
|
||||
|
||||
select {
|
||||
case <-time.After(timeout):
|
||||
testutil.FatalStack(t, fmt.Sprintf("test timed out after %v", timeout))
|
||||
case <-donec:
|
||||
}
|
||||
}
|
||||
|
||||
type etcdctlV3 struct {
|
||||
cfg *etcdProcessClusterConfig
|
||||
endpoints []string
|
||||
}
|
||||
|
||||
func clusterEtcdctlV3(cfg *etcdProcessClusterConfig, endpoints []string) *etcdctlV3 {
|
||||
return &etcdctlV3{
|
||||
cfg: cfg,
|
||||
endpoints: endpoints,
|
||||
}
|
||||
}
|
||||
|
||||
func (ctl *etcdctlV3) Put(key, value string) error {
|
||||
return ctl.runCmd("put", key, value)
|
||||
}
|
||||
|
||||
func (ctl *etcdctlV3) runCmd(args ...string) error {
|
||||
cmdArgs := []string{ctlBinPath + "3"}
|
||||
for k, v := range ctl.flags() {
|
||||
cmdArgs = append(cmdArgs, fmt.Sprintf("--%s=%s", k, v))
|
||||
}
|
||||
cmdArgs = append(cmdArgs, args...)
|
||||
return spawnWithExpect(cmdArgs, "OK")
|
||||
}
|
||||
|
||||
func (ctl *etcdctlV3) flags() map[string]string {
|
||||
fmap := make(map[string]string)
|
||||
if ctl.cfg.clientTLS == clientTLS {
|
||||
if ctl.cfg.isClientAutoTLS {
|
||||
fmap["insecure-transport"] = "false"
|
||||
fmap["insecure-skip-tls-verify"] = "true"
|
||||
} else if ctl.cfg.isClientCRL {
|
||||
fmap["cacert"] = caPath
|
||||
fmap["cert"] = revokedCertPath
|
||||
fmap["key"] = revokedPrivateKeyPath
|
||||
} else {
|
||||
fmap["cacert"] = caPath
|
||||
fmap["cert"] = certPath
|
||||
fmap["key"] = privateKeyPath
|
||||
}
|
||||
}
|
||||
fmap["endpoints"] = strings.Join(ctl.endpoints, ",")
|
||||
return fmap
|
||||
}
|
@ -43,6 +43,11 @@ type etcdProcess interface {
|
||||
Close() error
|
||||
WithStopSignal(sig os.Signal) os.Signal
|
||||
Config() *etcdServerProcessConfig
|
||||
Logs() logsExpect
|
||||
}
|
||||
|
||||
type logsExpect interface {
|
||||
Expect(string) (string, error)
|
||||
}
|
||||
|
||||
type etcdServerProcess struct {
|
||||
@ -56,6 +61,7 @@ type etcdServerProcessConfig struct {
|
||||
execPath string
|
||||
args []string
|
||||
tlsArgs []string
|
||||
envVars map[string]string
|
||||
|
||||
dataDirPath string
|
||||
keepDataDir bool
|
||||
@ -92,7 +98,7 @@ func (ep *etcdServerProcess) Start() error {
|
||||
panic("already started")
|
||||
}
|
||||
ep.cfg.lg.Info("starting server...", zap.String("name", ep.cfg.name))
|
||||
proc, err := spawnCmdWithLogger(ep.cfg.lg, append([]string{ep.cfg.execPath}, ep.cfg.args...), nil)
|
||||
proc, err := spawnCmdWithLogger(ep.cfg.lg, append([]string{ep.cfg.execPath}, ep.cfg.args...), ep.cfg.envVars)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -163,3 +169,10 @@ func (ep *etcdServerProcess) waitReady() error {
|
||||
}
|
||||
|
||||
func (ep *etcdServerProcess) Config() *etcdServerProcessConfig { return ep.cfg }
|
||||
|
||||
func (ep *etcdServerProcess) Logs() logsExpect {
|
||||
if ep.proc == nil {
|
||||
ep.cfg.lg.Panic("Please grap logs before process is stopped")
|
||||
}
|
||||
return ep.proc
|
||||
}
|
||||
|
@ -15,22 +15,22 @@
|
||||
package integration
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"sync"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/transport"
|
||||
)
|
||||
|
||||
// bridge creates a unix socket bridge to another unix socket, making it possible
|
||||
type Dialer interface {
|
||||
Dial() (net.Conn, error)
|
||||
}
|
||||
|
||||
// bridge proxies connections between listener and dialer, making it possible
|
||||
// to disconnect grpc network connections without closing the logical grpc connection.
|
||||
type bridge struct {
|
||||
inaddr string
|
||||
outaddr string
|
||||
l net.Listener
|
||||
conns map[*bridgeConn]struct{}
|
||||
dialer Dialer
|
||||
l net.Listener
|
||||
conns map[*bridgeConn]struct{}
|
||||
|
||||
stopc chan struct{}
|
||||
pausec chan struct{}
|
||||
@ -40,30 +40,22 @@ type bridge struct {
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
func newBridge(addr string) (*bridge, error) {
|
||||
func newBridge(dialer Dialer, listener net.Listener) (*bridge, error) {
|
||||
b := &bridge{
|
||||
// bridge "port" is ("%05d%05d0", port, pid) since go1.8 expects the port to be a number
|
||||
inaddr: addr + "0",
|
||||
outaddr: addr,
|
||||
dialer: dialer,
|
||||
l: listener,
|
||||
conns: make(map[*bridgeConn]struct{}),
|
||||
stopc: make(chan struct{}),
|
||||
pausec: make(chan struct{}),
|
||||
blackholec: make(chan struct{}),
|
||||
}
|
||||
close(b.pausec)
|
||||
|
||||
l, err := transport.NewUnixListener(b.inaddr)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("listen failed on socket %s (%v)", addr, err)
|
||||
}
|
||||
b.l = l
|
||||
b.wg.Add(1)
|
||||
go b.serveListen()
|
||||
return b, nil
|
||||
}
|
||||
|
||||
func (b *bridge) URL() string { return "unix://" + b.inaddr }
|
||||
|
||||
func (b *bridge) Close() {
|
||||
b.l.Close()
|
||||
b.mu.Lock()
|
||||
@ -76,7 +68,7 @@ func (b *bridge) Close() {
|
||||
b.wg.Wait()
|
||||
}
|
||||
|
||||
func (b *bridge) Reset() {
|
||||
func (b *bridge) DropConnections() {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
for bc := range b.conns {
|
||||
@ -85,13 +77,13 @@ func (b *bridge) Reset() {
|
||||
b.conns = make(map[*bridgeConn]struct{})
|
||||
}
|
||||
|
||||
func (b *bridge) Pause() {
|
||||
func (b *bridge) PauseConnections() {
|
||||
b.mu.Lock()
|
||||
b.pausec = make(chan struct{})
|
||||
b.mu.Unlock()
|
||||
}
|
||||
|
||||
func (b *bridge) Unpause() {
|
||||
func (b *bridge) UnpauseConnections() {
|
||||
b.mu.Lock()
|
||||
select {
|
||||
case <-b.pausec:
|
||||
@ -127,7 +119,7 @@ func (b *bridge) serveListen() {
|
||||
case <-pausec:
|
||||
}
|
||||
|
||||
outc, oerr := net.Dial("unix", b.outaddr)
|
||||
outc, oerr := b.dialer.Dial()
|
||||
if oerr != nil {
|
||||
inc.Close()
|
||||
return
|
||||
|
@ -38,10 +38,11 @@ func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{
|
||||
Size: 2,
|
||||
GRPCKeepAliveMinTime: time.Millisecond, // avoid too_many_pings
|
||||
UseBridge: true,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL()}
|
||||
|
||||
ccfg := clientv3.Config{
|
||||
Endpoints: []string{eps[0]},
|
||||
@ -76,7 +77,7 @@ func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) {
|
||||
// give enough time for balancer resolution
|
||||
time.Sleep(5 * time.Second)
|
||||
|
||||
clus.Members[0].Blackhole()
|
||||
clus.Members[0].Bridge().Blackhole()
|
||||
|
||||
if _, err = clus.Client(1).Put(context.TODO(), "foo", "bar"); err != nil {
|
||||
t.Fatal(err)
|
||||
@ -87,12 +88,12 @@ func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) {
|
||||
t.Error("took too long to receive watch events")
|
||||
}
|
||||
|
||||
clus.Members[0].Unblackhole()
|
||||
clus.Members[0].Bridge().Unblackhole()
|
||||
|
||||
// waiting for moving eps[0] out of unhealthy, so that it can be re-pined.
|
||||
time.Sleep(ccfg.DialTimeout)
|
||||
|
||||
clus.Members[1].Blackhole()
|
||||
clus.Members[1].Bridge().Blackhole()
|
||||
|
||||
// make sure client[0] can connect to eps[0] after remove the blackhole.
|
||||
if _, err = clus.Client(0).Get(context.TODO(), "foo"); err != nil {
|
||||
@ -170,10 +171,11 @@ func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Clien
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{
|
||||
Size: 2,
|
||||
SkipCreatingClient: true,
|
||||
UseBridge: true,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL()}
|
||||
|
||||
ccfg := clientv3.Config{
|
||||
Endpoints: []string{eps[0]},
|
||||
@ -194,7 +196,7 @@ func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Clien
|
||||
cli.SetEndpoints(eps...)
|
||||
|
||||
// blackhole eps[0]
|
||||
clus.Members[0].Blackhole()
|
||||
clus.Members[0].Bridge().Blackhole()
|
||||
|
||||
// With round robin balancer, client will make a request to a healthy endpoint
|
||||
// within a few requests.
|
||||
|
@ -57,7 +57,7 @@ func TestDialTLSExpired(t *testing.T) {
|
||||
}
|
||||
// expect remote errors "tls: bad certificate"
|
||||
_, err = integration.NewClient(t, clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
Endpoints: []string{clus.Members[0].GRPCURL()},
|
||||
DialTimeout: 3 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
TLS: tls,
|
||||
@ -75,7 +75,7 @@ func TestDialTLSNoConfig(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
// expect "signed by unknown authority"
|
||||
c, err := integration.NewClient(t, clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
Endpoints: []string{clus.Members[0].GRPCURL()},
|
||||
DialTimeout: time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
})
|
||||
@ -108,7 +108,7 @@ func testDialSetEndpoints(t *testing.T, setBefore bool) {
|
||||
// get endpoint list
|
||||
eps := make([]string, 3)
|
||||
for i := range eps {
|
||||
eps[i] = clus.Members[i].GRPCAddr()
|
||||
eps[i] = clus.Members[i].GRPCURL()
|
||||
}
|
||||
toKill := rand.Intn(len(eps))
|
||||
|
||||
@ -149,7 +149,7 @@ func TestSwitchSetEndpoints(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// get non partitioned members endpoints
|
||||
eps := []string{clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
|
||||
eps := []string{clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
|
||||
cli := clus.Client(0)
|
||||
clus.Members[0].InjectPartition(t, clus.Members[1:]...)
|
||||
@ -170,7 +170,7 @@ func TestRejectOldCluster(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()},
|
||||
Endpoints: []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL()},
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
RejectOldCluster: true,
|
||||
@ -212,7 +212,7 @@ func TestSetEndpointAndPut(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.Client(1).SetEndpoints(clus.Members[0].GRPCAddr())
|
||||
clus.Client(1).SetEndpoints(clus.Members[0].GRPCURL())
|
||||
_, err := clus.Client(1).Put(context.TODO(), "foo", "bar")
|
||||
if err != nil && !strings.Contains(err.Error(), "closing") {
|
||||
t.Fatal(err)
|
||||
|
@ -111,7 +111,7 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
|
||||
// expect pin eps[0]
|
||||
ccfg := clientv3.Config{
|
||||
@ -166,7 +166,7 @@ func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T
|
||||
SkipCreatingClient: true,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
|
||||
lead := clus.WaitLeader(t)
|
||||
|
||||
@ -222,7 +222,7 @@ func testBalancerUnderNetworkPartitionWatch(t *testing.T, isolateLeader bool) {
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
|
||||
target := clus.WaitLeader(t)
|
||||
if !isolateLeader {
|
||||
@ -283,7 +283,7 @@ func TestDropReadUnderNetworkPartition(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
leaderIndex := clus.WaitLeader(t)
|
||||
// get a follower endpoint
|
||||
eps := []string{clus.Members[(leaderIndex+1)%3].GRPCAddr()}
|
||||
eps := []string{clus.Members[(leaderIndex+1)%3].GRPCURL()}
|
||||
ccfg := clientv3.Config{
|
||||
Endpoints: eps,
|
||||
DialTimeout: 10 * time.Second,
|
||||
@ -301,7 +301,7 @@ func TestDropReadUnderNetworkPartition(t *testing.T) {
|
||||
// add other endpoints for later endpoint switch
|
||||
cli.SetEndpoints(eps...)
|
||||
time.Sleep(time.Second * 2)
|
||||
conn, err := cli.Dial(clus.Members[(leaderIndex+1)%3].GRPCAddr())
|
||||
conn, err := cli.Dial(clus.Members[(leaderIndex+1)%3].GRPCURL())
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -35,10 +35,11 @@ func TestBalancerUnderServerShutdownWatch(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{
|
||||
Size: 3,
|
||||
SkipCreatingClient: true,
|
||||
UseBridge: true,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
|
||||
lead := clus.WaitLeader(t)
|
||||
|
||||
@ -150,7 +151,7 @@ func testBalancerUnderServerShutdownMutable(t *testing.T, op func(*clientv3.Clie
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
|
||||
// pin eps[0]
|
||||
cli, err := integration.NewClient(t, clientv3.Config{Endpoints: []string{eps[0]}})
|
||||
@ -208,7 +209,7 @@ func testBalancerUnderServerShutdownImmutable(t *testing.T, op func(*clientv3.Cl
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr(), clus.Members[2].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL(), clus.Members[2].GRPCURL()}
|
||||
|
||||
// pin eps[0]
|
||||
cli, err := integration.NewClient(t, clientv3.Config{Endpoints: []string{eps[0]}})
|
||||
@ -278,6 +279,7 @@ func testBalancerUnderServerStopInflightRangeOnRestart(t *testing.T, linearizabl
|
||||
cfg := &integration.ClusterConfig{
|
||||
Size: 2,
|
||||
SkipCreatingClient: true,
|
||||
UseBridge: true,
|
||||
}
|
||||
if linearizable {
|
||||
cfg.Size = 3
|
||||
@ -285,9 +287,9 @@ func testBalancerUnderServerStopInflightRangeOnRestart(t *testing.T, linearizabl
|
||||
|
||||
clus := integration.NewClusterV3(t, cfg)
|
||||
defer clus.Terminate(t)
|
||||
eps := []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()}
|
||||
eps := []string{clus.Members[0].GRPCURL(), clus.Members[1].GRPCURL()}
|
||||
if linearizable {
|
||||
eps = append(eps, clus.Members[2].GRPCAddr())
|
||||
eps = append(eps, clus.Members[2].GRPCURL())
|
||||
}
|
||||
|
||||
lead := clus.WaitLeader(t)
|
||||
|
@ -712,7 +712,7 @@ func TestKVGetRetry(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clusterSize := 3
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: clusterSize})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
@ -876,7 +876,7 @@ func TestKVPutStoppedServerAndClose(t *testing.T) {
|
||||
// in the presence of network errors.
|
||||
func TestKVPutAtMostOnce(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
if _, err := clus.Client(0).Put(context.TODO(), "k", "1"); err != nil {
|
||||
@ -884,12 +884,12 @@ func TestKVPutAtMostOnce(t *testing.T) {
|
||||
}
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
defer close(donec)
|
||||
for i := 0; i < 10; i++ {
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
time.Sleep(5 * time.Millisecond)
|
||||
}
|
||||
}()
|
||||
@ -1027,7 +1027,7 @@ func TestKVForLearner(t *testing.T) {
|
||||
// 1. clus.Members[3] is the newly added learner member, which was appended to clus.Members
|
||||
// 2. we are using member's grpcAddr instead of clientURLs as the endpoint for clientv3.Config,
|
||||
// because the implementation of integration test has diverged from embed/etcd.go.
|
||||
learnerEp := clus.Members[3].GRPCAddr()
|
||||
learnerEp := clus.Members[3].GRPCURL()
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{learnerEp},
|
||||
DialTimeout: 5 * time.Second,
|
||||
@ -1100,7 +1100,7 @@ func TestBalancerSupportLearner(t *testing.T) {
|
||||
}
|
||||
|
||||
// clus.Members[3] is the newly added learner member, which was appended to clus.Members
|
||||
learnerEp := clus.Members[3].GRPCAddr()
|
||||
learnerEp := clus.Members[3].GRPCURL()
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{learnerEp},
|
||||
DialTimeout: 5 * time.Second,
|
||||
@ -1120,7 +1120,7 @@ func TestBalancerSupportLearner(t *testing.T) {
|
||||
}
|
||||
t.Logf("Expected: Read from learner error: %v", err)
|
||||
|
||||
eps := []string{learnerEp, clus.Members[0].GRPCAddr()}
|
||||
eps := []string{learnerEp, clus.Members[0].GRPCURL()}
|
||||
cli.SetEndpoints(eps...)
|
||||
if _, err := cli.Get(context.Background(), "foo"); err != nil {
|
||||
t.Errorf("expect no error (balancer should retry when request to learner fails), got error: %v", err)
|
||||
|
@ -190,7 +190,7 @@ func TestLeaseKeepAliveHandleFailure(t *testing.T) {
|
||||
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// TODO: change this line to get a cluster client
|
||||
@ -416,7 +416,7 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) {
|
||||
func TestLeaseKeepAliveCloseAfterDisconnectRevoke(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.RandClient()
|
||||
@ -656,7 +656,7 @@ func TestLeaseLeases(t *testing.T) {
|
||||
func TestLeaseRenewLostQuorum(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -728,7 +728,7 @@ func TestLeaseKeepAliveLoopExit(t *testing.T) {
|
||||
// transient cluster failure.
|
||||
func TestV3LeaseFailureOverlap(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.Client(0)
|
||||
|
@ -195,7 +195,7 @@ func TestLeasingPutInvalidateExisting(t *testing.T) {
|
||||
// TestLeasingGetNoLeaseTTL checks a key with a TTL is not leased.
|
||||
func TestLeasingGetNoLeaseTTL(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 2, UseBridge: true})
|
||||
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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
client := clus.Client(0)
|
||||
@ -773,7 +773,7 @@ func TestLeasingTxnOwnerDelete(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerIf(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv1, closeLKV1, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -1085,7 +1085,7 @@ func TestLeasingTxnRandIfThenOrElse(t *testing.T) {
|
||||
|
||||
func TestLeasingOwnerPutError(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "pfx/")
|
||||
@ -1201,7 +1201,7 @@ func testLeasingOwnerDelete(t *testing.T, del clientv3.Op) {
|
||||
|
||||
func TestLeasingDeleteRangeBounds(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
delkv, closeDelKV, err := leasing.NewKV(clus.Client(0), "0/")
|
||||
@ -1376,7 +1376,7 @@ func TestLeasingPutGetDeleteConcurrent(t *testing.T) {
|
||||
// disconnected when trying to submit revoke txn.
|
||||
func TestLeasingReconnectOwnerRevoke(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1510,11 +1510,11 @@ func TestLeasingReconnectOwnerConsistency(t *testing.T) {
|
||||
for i := 0; i < 10; i++ {
|
||||
v := fmt.Sprintf("%d", i)
|
||||
donec := make(chan struct{})
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
go func() {
|
||||
defer close(donec)
|
||||
for i := 0; i < 20; i++ {
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
time.Sleep(time.Millisecond)
|
||||
}
|
||||
}()
|
||||
@ -1650,7 +1650,7 @@ func TestLeasingTxnAtomicCache(t *testing.T) {
|
||||
// TestLeasingReconnectTxn checks that Txn is resilient to disconnects.
|
||||
func TestLeasingReconnectTxn(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1664,9 +1664,9 @@ func TestLeasingReconnectTxn(t *testing.T) {
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
defer close(donec)
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
for i := 0; i < 10; i++ {
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
time.Sleep(time.Millisecond)
|
||||
}
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
@ -1686,7 +1686,7 @@ func TestLeasingReconnectTxn(t *testing.T) {
|
||||
// not cause inconsistency between the server and the client.
|
||||
func TestLeasingReconnectNonOwnerGet(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/")
|
||||
@ -1704,11 +1704,11 @@ func TestLeasingReconnectNonOwnerGet(t *testing.T) {
|
||||
n := 0
|
||||
for i := 0; i < 10; i++ {
|
||||
donec := make(chan struct{})
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
go func() {
|
||||
defer close(donec)
|
||||
for j := 0; j < 10; j++ {
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
time.Sleep(time.Millisecond)
|
||||
}
|
||||
}()
|
||||
@ -1814,7 +1814,7 @@ func TestLeasingDo(t *testing.T) {
|
||||
|
||||
func TestLeasingTxnOwnerPutBranch(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.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) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/", concurrency.WithTTL(1))
|
||||
|
@ -56,7 +56,7 @@ func TestMaintenanceHashKV(t *testing.T) {
|
||||
if _, err := cli.Get(context.TODO(), "foo"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
hresp, err := cli.HashKV(context.Background(), clus.Members[i].GRPCAddr(), 0)
|
||||
hresp, err := cli.HashKV(context.Background(), clus.Members[i].GRPCURL(), 0)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -192,7 +192,7 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) {
|
||||
func testMaintenanceSnapshotErrorInflight(t *testing.T, snapshot func(context.Context, *clientv3.Client) (io.ReadCloser, error)) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// take about 1-second to read snapshot
|
||||
@ -279,7 +279,7 @@ func TestMaintenanceStatus(t *testing.T) {
|
||||
|
||||
eps := make([]string, 3)
|
||||
for i := 0; i < 3; i++ {
|
||||
eps[i] = clus.Members[i].GRPCAddr()
|
||||
eps[i] = clus.Members[i].GRPCURL()
|
||||
}
|
||||
|
||||
cli, err := integration.NewClient(t, clientv3.Config{Endpoints: eps, DialOptions: []grpc.DialOption{grpc.WithBlock()}})
|
||||
|
@ -75,7 +75,7 @@ func TestV3ClientMetrics(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
Endpoints: []string{clus.Members[0].GRPCURL()},
|
||||
DialOptions: []grpc.DialOption{
|
||||
grpc.WithUnaryInterceptor(grpcprom.UnaryClientInterceptor),
|
||||
grpc.WithStreamInterceptor(grpcprom.StreamClientInterceptor),
|
||||
|
@ -30,14 +30,14 @@ func TestDetectKvOrderViolation(t *testing.T) {
|
||||
var errOrderViolation = errors.New("DetectedOrderViolation")
|
||||
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{
|
||||
clus.Members[0].GRPCAddr(),
|
||||
clus.Members[1].GRPCAddr(),
|
||||
clus.Members[2].GRPCAddr(),
|
||||
clus.Members[0].GRPCURL(),
|
||||
clus.Members[1].GRPCURL(),
|
||||
clus.Members[2].GRPCURL(),
|
||||
},
|
||||
}
|
||||
cli, err := integration.NewClient(t, cfg)
|
||||
@ -82,7 +82,7 @@ func TestDetectKvOrderViolation(t *testing.T) {
|
||||
clus.Members[1].Stop(t)
|
||||
assert.NoError(t, clus.Members[2].Restart(t))
|
||||
// force OrderingKv to query the third member
|
||||
cli.SetEndpoints(clus.Members[2].GRPCAddr())
|
||||
cli.SetEndpoints(clus.Members[2].GRPCURL())
|
||||
time.Sleep(2 * time.Second) // FIXME: Figure out how pause SetEndpoints sufficiently that this is not needed
|
||||
|
||||
t.Logf("Quering m2 after restart")
|
||||
@ -97,14 +97,14 @@ func TestDetectTxnOrderViolation(t *testing.T) {
|
||||
var errOrderViolation = errors.New("DetectedOrderViolation")
|
||||
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{
|
||||
clus.Members[0].GRPCAddr(),
|
||||
clus.Members[1].GRPCAddr(),
|
||||
clus.Members[2].GRPCAddr(),
|
||||
clus.Members[0].GRPCURL(),
|
||||
clus.Members[1].GRPCURL(),
|
||||
clus.Members[2].GRPCURL(),
|
||||
},
|
||||
}
|
||||
cli, err := integration.NewClient(t, cfg)
|
||||
@ -151,7 +151,7 @@ func TestDetectTxnOrderViolation(t *testing.T) {
|
||||
clus.Members[1].Stop(t)
|
||||
assert.NoError(t, clus.Members[2].Restart(t))
|
||||
// force OrderingKv to query the third member
|
||||
cli.SetEndpoints(clus.Members[2].GRPCAddr())
|
||||
cli.SetEndpoints(clus.Members[2].GRPCURL())
|
||||
time.Sleep(2 * time.Second) // FIXME: Figure out how pause SetEndpoints sufficiently that this is not needed
|
||||
_, err = orderingKv.Get(ctx, "foo", clientv3.WithSerializable())
|
||||
if err != errOrderViolation {
|
||||
|
@ -29,11 +29,11 @@ func TestEndpointSwitchResolvesViolation(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
eps := []string{
|
||||
clus.Members[0].GRPCAddr(),
|
||||
clus.Members[1].GRPCAddr(),
|
||||
clus.Members[2].GRPCAddr(),
|
||||
clus.Members[0].GRPCURL(),
|
||||
clus.Members[1].GRPCURL(),
|
||||
clus.Members[2].GRPCURL(),
|
||||
}
|
||||
cfg := clientv3.Config{Endpoints: []string{clus.Members[0].GRPCAddr()}}
|
||||
cfg := clientv3.Config{Endpoints: []string{clus.Members[0].GRPCURL()}}
|
||||
cli, err := integration.NewClient(t, cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -71,7 +71,7 @@ func TestEndpointSwitchResolvesViolation(t *testing.T) {
|
||||
}
|
||||
|
||||
t.Logf("Reconfigure client to speak only to the 'partitioned' member")
|
||||
cli.SetEndpoints(clus.Members[2].GRPCAddr())
|
||||
cli.SetEndpoints(clus.Members[2].GRPCURL())
|
||||
_, err = orderingKv.Get(ctx, "foo", clientv3.WithSerializable())
|
||||
if err != ordering.ErrNoGreaterRev {
|
||||
t.Fatal("While speaking to partitioned leader, we should get ErrNoGreaterRev error")
|
||||
@ -80,15 +80,15 @@ func TestEndpointSwitchResolvesViolation(t *testing.T) {
|
||||
|
||||
func TestUnresolvableOrderViolation(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 5, SkipCreatingClient: true})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 5, SkipCreatingClient: true, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{
|
||||
clus.Members[0].GRPCAddr(),
|
||||
clus.Members[1].GRPCAddr(),
|
||||
clus.Members[2].GRPCAddr(),
|
||||
clus.Members[3].GRPCAddr(),
|
||||
clus.Members[4].GRPCAddr(),
|
||||
clus.Members[0].GRPCURL(),
|
||||
clus.Members[1].GRPCURL(),
|
||||
clus.Members[2].GRPCURL(),
|
||||
clus.Members[3].GRPCURL(),
|
||||
clus.Members[4].GRPCURL(),
|
||||
},
|
||||
}
|
||||
cli, err := integration.NewClient(t, cfg)
|
||||
@ -99,7 +99,7 @@ func TestUnresolvableOrderViolation(t *testing.T) {
|
||||
eps := cli.Endpoints()
|
||||
ctx := context.TODO()
|
||||
|
||||
cli.SetEndpoints(clus.Members[0].GRPCAddr())
|
||||
cli.SetEndpoints(clus.Members[0].GRPCURL())
|
||||
time.Sleep(1 * time.Second)
|
||||
_, err = cli.Put(ctx, "foo", "bar")
|
||||
if err != nil {
|
||||
@ -139,7 +139,7 @@ func TestUnresolvableOrderViolation(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
clus.Members[3].WaitStarted(t)
|
||||
cli.SetEndpoints(clus.Members[3].GRPCAddr())
|
||||
cli.SetEndpoints(clus.Members[3].GRPCURL())
|
||||
|
||||
_, err = OrderingKv.Get(ctx, "foo", clientv3.WithSerializable())
|
||||
if err != ordering.ErrNoGreaterRev {
|
||||
|
@ -53,7 +53,7 @@ func TestTxnError(t *testing.T) {
|
||||
func TestTxnWriteFail(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
@ -103,7 +103,7 @@ func TestTxnReadRetry(t *testing.T) {
|
||||
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := clus.Client(0)
|
||||
|
@ -47,7 +47,7 @@ type watchctx struct {
|
||||
func runWatchTest(t *testing.T, f watcherTest) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
wclientMember := rand.Intn(3)
|
||||
@ -188,7 +188,7 @@ func testWatchReconnRequest(t *testing.T, wctx *watchctx) {
|
||||
defer close(donec)
|
||||
// take down watcher connection
|
||||
for {
|
||||
wctx.clus.Members[wctx.wclientMember].DropConnections()
|
||||
wctx.clus.Members[wctx.wclientMember].Bridge().DropConnections()
|
||||
select {
|
||||
case <-timer:
|
||||
// spinning on close may live lock reconnection
|
||||
@ -230,7 +230,7 @@ func testWatchReconnInit(t *testing.T, wctx *watchctx) {
|
||||
if wctx.ch = wctx.w.Watch(context.TODO(), "a"); wctx.ch == nil {
|
||||
t.Fatalf("expected non-nil channel")
|
||||
}
|
||||
wctx.clus.Members[wctx.wclientMember].DropConnections()
|
||||
wctx.clus.Members[wctx.wclientMember].Bridge().DropConnections()
|
||||
// watcher should recover
|
||||
putAndWatch(t, wctx, "a", "a")
|
||||
}
|
||||
@ -247,7 +247,7 @@ func testWatchReconnRunning(t *testing.T, wctx *watchctx) {
|
||||
}
|
||||
putAndWatch(t, wctx, "a", "a")
|
||||
// take down watcher connection
|
||||
wctx.clus.Members[wctx.wclientMember].DropConnections()
|
||||
wctx.clus.Members[wctx.wclientMember].Bridge().DropConnections()
|
||||
// watcher should recover
|
||||
putAndWatch(t, wctx, "a", "b")
|
||||
}
|
||||
@ -348,7 +348,7 @@ func putAndWatch(t *testing.T, wctx *watchctx, key, val string) {
|
||||
|
||||
func TestWatchResumeInitRev(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
@ -368,8 +368,8 @@ func TestWatchResumeInitRev(t *testing.T) {
|
||||
t.Fatalf("got (%v, %v), expected create notification rev=4", resp, ok)
|
||||
}
|
||||
// pause wch
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].PauseConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
clus.Members[0].Bridge().PauseConnections()
|
||||
|
||||
select {
|
||||
case resp, ok := <-wch:
|
||||
@ -378,7 +378,7 @@ func TestWatchResumeInitRev(t *testing.T) {
|
||||
}
|
||||
|
||||
// resume wch
|
||||
clus.Members[0].UnpauseConnections()
|
||||
clus.Members[0].Bridge().UnpauseConnections()
|
||||
|
||||
select {
|
||||
case resp, ok := <-wch:
|
||||
@ -404,7 +404,7 @@ func TestWatchResumeInitRev(t *testing.T) {
|
||||
func TestWatchResumeCompacted(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// create a waiting watcher at rev 1
|
||||
@ -955,7 +955,7 @@ func TestWatchWithCreatedNotification(t *testing.T) {
|
||||
func TestWatchWithCreatedNotificationDropConn(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
|
||||
cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer cluster.Terminate(t)
|
||||
|
||||
client := cluster.RandClient()
|
||||
@ -968,7 +968,7 @@ func TestWatchWithCreatedNotificationDropConn(t *testing.T) {
|
||||
t.Fatalf("expected created event, got %v", resp)
|
||||
}
|
||||
|
||||
cluster.Members[0].DropConnections()
|
||||
cluster.Members[0].Bridge().DropConnections()
|
||||
|
||||
// check watch channel doesn't post another watch response.
|
||||
select {
|
||||
@ -1056,14 +1056,14 @@ func TestWatchOverlapContextCancel(t *testing.T) {
|
||||
|
||||
func TestWatchOverlapDropConnContextCancel(t *testing.T) {
|
||||
f := func(clus *integration.ClusterV3) {
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
}
|
||||
testWatchOverlapContextCancel(t, f)
|
||||
}
|
||||
|
||||
func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3)) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
n := 100
|
||||
@ -1154,7 +1154,7 @@ func TestWatchCancelAndCloseClient(t *testing.T) {
|
||||
// then closes the watcher interface to ensure correct clean up.
|
||||
func TestWatchStressResumeClose(t *testing.T) {
|
||||
integration.BeforeTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.Client(0)
|
||||
|
||||
@ -1164,7 +1164,7 @@ func TestWatchStressResumeClose(t *testing.T) {
|
||||
for i := range wchs {
|
||||
wchs[i] = cli.Watch(ctx, "abc")
|
||||
}
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
cancel()
|
||||
if err := cli.Close(); err != nil {
|
||||
t.Fatal(err)
|
||||
|
@ -39,6 +39,7 @@ import (
|
||||
"go.etcd.io/etcd/client/pkg/v3/types"
|
||||
"go.etcd.io/etcd/client/v2"
|
||||
"go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/pkg/v3/grpc_testing"
|
||||
"go.etcd.io/etcd/raft/v3"
|
||||
"go.etcd.io/etcd/server/v3/config"
|
||||
"go.etcd.io/etcd/server/v3/embed"
|
||||
@ -73,6 +74,7 @@ const (
|
||||
basePort = 21000
|
||||
URLScheme = "unix"
|
||||
URLSchemeTLS = "unixs"
|
||||
baseGRPCPort = 30000
|
||||
)
|
||||
|
||||
var (
|
||||
@ -121,6 +123,10 @@ var (
|
||||
|
||||
defaultTokenJWT = fmt.Sprintf("jwt,pub-key=%s,priv-key=%s,sign-method=RS256,ttl=1s",
|
||||
MustAbsPath("../fixtures/server.crt"), MustAbsPath("../fixtures/server.key.insecure"))
|
||||
|
||||
// uniqueNumber is used to generate unique port numbers
|
||||
// Should only be accessed via atomic package methods.
|
||||
uniqueNumber int32
|
||||
)
|
||||
|
||||
type ClusterConfig struct {
|
||||
@ -153,6 +159,11 @@ type ClusterConfig struct {
|
||||
|
||||
// UseIP is true to use only IP for gRPC requests.
|
||||
UseIP bool
|
||||
// UseBridge adds bridge between client and grpc server. Should be used in tests that
|
||||
// want to manipulate connection or require connection not breaking despite server stop/restart.
|
||||
UseBridge bool
|
||||
// UseTCP configures server listen on tcp socket. If disabled unix socket is used.
|
||||
UseTCP bool
|
||||
|
||||
EnableLeaseCheckpoint bool
|
||||
LeaseCheckpointInterval time.Duration
|
||||
@ -208,7 +219,7 @@ func newCluster(t testutil.TB, cfg *ClusterConfig) *cluster {
|
||||
c := &cluster{cfg: cfg}
|
||||
ms := make([]*member, cfg.Size)
|
||||
for i := 0; i < cfg.Size; i++ {
|
||||
ms[i] = c.mustNewMember(t)
|
||||
ms[i] = c.mustNewMember(t, int64(i))
|
||||
}
|
||||
c.Members = ms
|
||||
if err := c.fillClusterForMembers(); err != nil {
|
||||
@ -249,7 +260,7 @@ func (c *cluster) Launch(t testutil.TB) {
|
||||
c.waitMembersMatch(t, c.HTTPMembers())
|
||||
c.waitVersion()
|
||||
for _, m := range c.Members {
|
||||
t.Logf(" - %v -> %v (%v)", m.Name, m.ID(), m.GRPCAddr())
|
||||
t.Logf(" - %v -> %v (%v)", m.Name, m.ID(), m.GRPCURL())
|
||||
}
|
||||
}
|
||||
|
||||
@ -295,10 +306,11 @@ func (c *cluster) HTTPMembers() []client.Member {
|
||||
return ms
|
||||
}
|
||||
|
||||
func (c *cluster) mustNewMember(t testutil.TB) *member {
|
||||
func (c *cluster) mustNewMember(t testutil.TB, memberNumber int64) *member {
|
||||
m := mustNewMember(t,
|
||||
memberConfig{
|
||||
name: c.generateMemberName(),
|
||||
memberNumber: memberNumber,
|
||||
authToken: c.cfg.AuthToken,
|
||||
peerTLS: c.cfg.PeerTLS,
|
||||
clientTLS: c.cfg.ClientTLS,
|
||||
@ -313,6 +325,8 @@ func (c *cluster) mustNewMember(t testutil.TB) *member {
|
||||
clientMaxCallSendMsgSize: c.cfg.ClientMaxCallSendMsgSize,
|
||||
clientMaxCallRecvMsgSize: c.cfg.ClientMaxCallRecvMsgSize,
|
||||
useIP: c.cfg.UseIP,
|
||||
useBridge: c.cfg.UseBridge,
|
||||
useTCP: c.cfg.UseTCP,
|
||||
enableLeaseCheckpoint: c.cfg.EnableLeaseCheckpoint,
|
||||
leaseCheckpointInterval: c.cfg.LeaseCheckpointInterval,
|
||||
WatchProgressNotifyInterval: c.cfg.WatchProgressNotifyInterval,
|
||||
@ -328,7 +342,7 @@ func (c *cluster) mustNewMember(t testutil.TB) *member {
|
||||
|
||||
// addMember return PeerURLs of the added member.
|
||||
func (c *cluster) addMember(t testutil.TB) types.URLs {
|
||||
m := c.mustNewMember(t)
|
||||
m := c.mustNewMember(t, 0)
|
||||
|
||||
scheme := schemeFromTLSInfo(c.cfg.PeerTLS)
|
||||
|
||||
@ -557,6 +571,8 @@ func NewListenerWithAddr(t testutil.TB, addr string) net.Listener {
|
||||
|
||||
type member struct {
|
||||
config.ServerConfig
|
||||
UniqNumber int64
|
||||
MemberNumber int64
|
||||
PeerListeners, ClientListeners []net.Listener
|
||||
grpcListener net.Listener
|
||||
// PeerTLSInfo enables peer TLS when set
|
||||
@ -572,7 +588,7 @@ type member struct {
|
||||
grpcServerOpts []grpc.ServerOption
|
||||
grpcServer *grpc.Server
|
||||
grpcServerPeer *grpc.Server
|
||||
grpcAddr string
|
||||
grpcURL string
|
||||
grpcBridge *bridge
|
||||
|
||||
// serverClient is a clientv3 that directly calls the etcdserver.
|
||||
@ -582,15 +598,21 @@ type member struct {
|
||||
clientMaxCallSendMsgSize int
|
||||
clientMaxCallRecvMsgSize int
|
||||
useIP bool
|
||||
useBridge bool
|
||||
useTCP bool
|
||||
|
||||
isLearner bool
|
||||
closed bool
|
||||
|
||||
grpcServerRecorder *grpc_testing.GrpcRecorder
|
||||
}
|
||||
|
||||
func (m *member) GRPCAddr() string { return m.grpcAddr }
|
||||
func (m *member) GRPCURL() string { return m.grpcURL }
|
||||
|
||||
type memberConfig struct {
|
||||
name string
|
||||
uniqNumber int64
|
||||
memberNumber int64
|
||||
peerTLS *transport.TLSInfo
|
||||
clientTLS *transport.TLSInfo
|
||||
authToken string
|
||||
@ -605,6 +627,8 @@ type memberConfig struct {
|
||||
clientMaxCallSendMsgSize int
|
||||
clientMaxCallRecvMsgSize int
|
||||
useIP bool
|
||||
useBridge bool
|
||||
useTCP bool
|
||||
enableLeaseCheckpoint bool
|
||||
leaseCheckpointInterval time.Duration
|
||||
WatchProgressNotifyInterval time.Duration
|
||||
@ -614,7 +638,10 @@ type memberConfig struct {
|
||||
// set, it will use https scheme to communicate between peers.
|
||||
func mustNewMember(t testutil.TB, mcfg memberConfig) *member {
|
||||
var err error
|
||||
m := &member{}
|
||||
m := &member{
|
||||
MemberNumber: mcfg.memberNumber,
|
||||
UniqNumber: atomic.AddInt64(&localListenCount, 1),
|
||||
}
|
||||
|
||||
peerScheme := schemeFromTLSInfo(mcfg.peerTLS)
|
||||
clientScheme := schemeFromTLSInfo(mcfg.clientTLS)
|
||||
@ -698,6 +725,8 @@ func mustNewMember(t testutil.TB, mcfg memberConfig) *member {
|
||||
m.clientMaxCallSendMsgSize = mcfg.clientMaxCallSendMsgSize
|
||||
m.clientMaxCallRecvMsgSize = mcfg.clientMaxCallRecvMsgSize
|
||||
m.useIP = mcfg.useIP
|
||||
m.useBridge = mcfg.useBridge
|
||||
m.useTCP = mcfg.useTCP
|
||||
m.EnableLeaseCheckpoint = mcfg.enableLeaseCheckpoint
|
||||
m.LeaseCheckpointInterval = mcfg.leaseCheckpointInterval
|
||||
|
||||
@ -708,7 +737,7 @@ func mustNewMember(t testutil.TB, mcfg memberConfig) *member {
|
||||
m.WarningUnaryRequestDuration = embed.DefaultWarningUnaryRequestDuration
|
||||
|
||||
m.V2Deprecation = config.V2_DEPR_DEFAULT
|
||||
|
||||
m.grpcServerRecorder = &grpc_testing.GrpcRecorder{}
|
||||
m.Logger = memberLogger(t, mcfg.name)
|
||||
t.Cleanup(func() {
|
||||
// if we didn't cleanup the logger, the consecutive test
|
||||
@ -731,45 +760,109 @@ func memberLogger(t testutil.TB, name string) *zap.Logger {
|
||||
// listenGRPC starts a grpc server over a unix domain socket on the member
|
||||
func (m *member) listenGRPC() error {
|
||||
// prefix with localhost so cert has right domain
|
||||
m.grpcAddr = "localhost:" + m.Name
|
||||
m.Logger.Info("LISTEN GRPC", zap.String("m.grpcAddr", m.grpcAddr), zap.String("m.Name", m.Name))
|
||||
if m.useIP { // for IP-only TLS certs
|
||||
m.grpcAddr = "127.0.0.1:" + m.Name
|
||||
}
|
||||
l, err := transport.NewUnixListener(m.grpcAddr)
|
||||
network, host, port := m.grpcAddr()
|
||||
grpcAddr := host + ":" + port
|
||||
m.Logger.Info("LISTEN GRPC", zap.String("grpcAddr", grpcAddr), zap.String("m.Name", m.Name))
|
||||
grpcListener, err := net.Listen(network, grpcAddr)
|
||||
if err != nil {
|
||||
return fmt.Errorf("listen failed on grpc socket %s (%v)", m.grpcAddr, err)
|
||||
return fmt.Errorf("listen failed on grpc socket %s (%v)", grpcAddr, err)
|
||||
}
|
||||
m.grpcBridge, err = newBridge(m.grpcAddr)
|
||||
if err != nil {
|
||||
l.Close()
|
||||
return err
|
||||
m.grpcURL = fmt.Sprintf("%s://%s", m.clientScheme(), grpcAddr)
|
||||
if m.useBridge {
|
||||
_, err = m.addBridge()
|
||||
if err != nil {
|
||||
grpcListener.Close()
|
||||
return err
|
||||
}
|
||||
}
|
||||
m.grpcAddr = schemeFromTLSInfo(m.ClientTLSInfo) + "://" + m.grpcBridge.inaddr
|
||||
m.grpcListener = l
|
||||
m.grpcListener = grpcListener
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *member) clientScheme() string {
|
||||
switch {
|
||||
case m.useTCP && m.ClientTLSInfo != nil:
|
||||
return "https"
|
||||
case m.useTCP && m.ClientTLSInfo == nil:
|
||||
return "http"
|
||||
case !m.useTCP && m.ClientTLSInfo != nil:
|
||||
return "unixs"
|
||||
case !m.useTCP && m.ClientTLSInfo == nil:
|
||||
return "unix"
|
||||
}
|
||||
m.Logger.Panic("Failed to determine client schema")
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *member) addBridge() (*bridge, error) {
|
||||
network, host, port := m.grpcAddr()
|
||||
grpcAddr := host + ":" + port
|
||||
bridgeAddr := grpcAddr + "0"
|
||||
m.Logger.Info("LISTEN BRIDGE", zap.String("grpc-address", bridgeAddr), zap.String("member", m.Name))
|
||||
bridgeListener, err := transport.NewUnixListener(bridgeAddr)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("listen failed on bridge socket %s (%v)", bridgeAddr, err)
|
||||
}
|
||||
m.grpcBridge, err = newBridge(dialer{network: network, addr: grpcAddr}, bridgeListener)
|
||||
if err != nil {
|
||||
bridgeListener.Close()
|
||||
return nil, err
|
||||
}
|
||||
m.grpcURL = m.clientScheme() + "://" + bridgeAddr
|
||||
return m.grpcBridge, nil
|
||||
}
|
||||
|
||||
func (m *member) Bridge() *bridge {
|
||||
if !m.useBridge {
|
||||
m.Logger.Panic("Bridge not available. Please configure using bridge before creating cluster.")
|
||||
}
|
||||
return m.grpcBridge
|
||||
}
|
||||
|
||||
func (m *member) grpcAddr() (network, host, port string) {
|
||||
// prefix with localhost so cert has right domain
|
||||
host = "localhost"
|
||||
if m.useIP { // for IP-only TLS certs
|
||||
host = "127.0.0.1"
|
||||
}
|
||||
network = "unix"
|
||||
if m.useTCP {
|
||||
network = "tcp"
|
||||
}
|
||||
port = m.Name
|
||||
if m.useTCP {
|
||||
port = fmt.Sprintf("%d", GrpcPortNumber(m.UniqNumber, m.MemberNumber))
|
||||
}
|
||||
return network, host, port
|
||||
}
|
||||
|
||||
func GrpcPortNumber(uniqNumber, memberNumber int64) int64 {
|
||||
return baseGRPCPort + uniqNumber*10 + memberNumber
|
||||
}
|
||||
|
||||
type dialer struct {
|
||||
network string
|
||||
addr string
|
||||
}
|
||||
|
||||
func (d dialer) Dial() (net.Conn, error) {
|
||||
return net.Dial(d.network, d.addr)
|
||||
}
|
||||
|
||||
func (m *member) ElectionTimeout() time.Duration {
|
||||
return time.Duration(m.s.Cfg.ElectionTicks*int(m.s.Cfg.TickMs)) * time.Millisecond
|
||||
}
|
||||
|
||||
func (m *member) ID() types.ID { return m.s.ID() }
|
||||
|
||||
func (m *member) DropConnections() { m.grpcBridge.Reset() }
|
||||
func (m *member) PauseConnections() { m.grpcBridge.Pause() }
|
||||
func (m *member) UnpauseConnections() { m.grpcBridge.Unpause() }
|
||||
func (m *member) Blackhole() { m.grpcBridge.Blackhole() }
|
||||
func (m *member) Unblackhole() { m.grpcBridge.Unblackhole() }
|
||||
|
||||
// NewClientV3 creates a new grpc client connection to the member
|
||||
func NewClientV3(m *member) (*clientv3.Client, error) {
|
||||
if m.grpcAddr == "" {
|
||||
if m.grpcURL == "" {
|
||||
return nil, fmt.Errorf("member not configured for grpc")
|
||||
}
|
||||
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{m.grpcAddr},
|
||||
Endpoints: []string{m.grpcURL},
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
MaxCallSendMsgSize: m.clientMaxCallSendMsgSize,
|
||||
@ -831,7 +924,7 @@ func (m *member) Launch() error {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
)
|
||||
var err error
|
||||
if m.s, err = etcdserver.NewServer(m.ServerConfig); err != nil {
|
||||
@ -857,8 +950,8 @@ func (m *member) Launch() error {
|
||||
return err
|
||||
}
|
||||
}
|
||||
m.grpcServer = v3rpc.Server(m.s, tlscfg, m.grpcServerOpts...)
|
||||
m.grpcServerPeer = v3rpc.Server(m.s, peerTLScfg)
|
||||
m.grpcServer = v3rpc.Server(m.s, tlscfg, m.grpcServerRecorder.UnaryInterceptor(), m.grpcServerOpts...)
|
||||
m.grpcServerPeer = v3rpc.Server(m.s, peerTLScfg, m.grpcServerRecorder.UnaryInterceptor())
|
||||
m.serverClient = v3client.New(m.s)
|
||||
lockpb.RegisterLockServer(m.grpcServer, v3lock.NewLockServer(m.serverClient))
|
||||
epb.RegisterElectionServer(m.grpcServer, v3election.NewElectionServer(m.serverClient))
|
||||
@ -988,11 +1081,15 @@ func (m *member) Launch() error {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *member) RecordedRequests() []grpc_testing.RequestInfo {
|
||||
return m.grpcServerRecorder.RecordedRequests()
|
||||
}
|
||||
|
||||
func (m *member) WaitOK(t testutil.TB) {
|
||||
m.WaitStarted(t)
|
||||
for m.s.Leader() == 0 {
|
||||
@ -1101,7 +1198,7 @@ func (m *member) Stop(_ testutil.TB) {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
)
|
||||
m.Close()
|
||||
m.serverClosers = nil
|
||||
@ -1110,7 +1207,7 @@ func (m *member) Stop(_ testutil.TB) {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
)
|
||||
}
|
||||
|
||||
@ -1135,7 +1232,7 @@ func (m *member) Restart(t testutil.TB) error {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
)
|
||||
newPeerListeners := make([]net.Listener, 0)
|
||||
for _, ln := range m.PeerListeners {
|
||||
@ -1160,7 +1257,7 @@ func (m *member) Restart(t testutil.TB) error {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
zap.Error(err),
|
||||
)
|
||||
return err
|
||||
@ -1173,7 +1270,7 @@ func (m *member) Terminate(t testutil.TB) {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
)
|
||||
m.Close()
|
||||
if !m.keepDataDirTerminate {
|
||||
@ -1186,7 +1283,7 @@ func (m *member) Terminate(t testutil.TB) {
|
||||
zap.String("name", m.Name),
|
||||
zap.Strings("advertise-peer-urls", m.PeerURLs.StringSlice()),
|
||||
zap.Strings("listen-client-urls", m.ClientURLs.StringSlice()),
|
||||
zap.String("grpc-address", m.grpcAddr),
|
||||
zap.String("grpc-url", m.grpcURL),
|
||||
)
|
||||
}
|
||||
|
||||
@ -1282,8 +1379,9 @@ 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
|
||||
mu sync.Mutex
|
||||
clients []*clientv3.Client
|
||||
clusterClient *clientv3.Client
|
||||
}
|
||||
|
||||
// NewClusterV3 returns a launched cluster with a grpc client connection
|
||||
@ -1329,6 +1427,11 @@ func (c *ClusterV3) Terminate(t testutil.TB) {
|
||||
t.Error(err)
|
||||
}
|
||||
}
|
||||
if c.clusterClient != nil {
|
||||
if err := c.clusterClient.Close(); err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
c.cluster.Terminate(t)
|
||||
}
|
||||
@ -1341,6 +1444,25 @@ func (c *ClusterV3) Client(i int) *clientv3.Client {
|
||||
return c.clients[i]
|
||||
}
|
||||
|
||||
func (c *ClusterV3) ClusterClient() (client *clientv3.Client, err error) {
|
||||
if c.clusterClient == nil {
|
||||
endpoints := []string{}
|
||||
for _, m := range c.Members {
|
||||
endpoints = append(endpoints, m.grpcURL)
|
||||
}
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: endpoints,
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
}
|
||||
c.clusterClient, err = newClientV3(cfg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return c.clusterClient, nil
|
||||
}
|
||||
|
||||
// NewClientV3 creates a new grpc client connection to the member
|
||||
func (c *ClusterV3) NewClientV3(memberIndex int) (*clientv3.Client, error) {
|
||||
return NewClientV3(c.Members[memberIndex])
|
||||
@ -1420,7 +1542,7 @@ 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)
|
||||
m := c.mustNewMember(t, 0)
|
||||
m.isLearner = true
|
||||
|
||||
scheme := schemeFromTLSInfo(c.cfg.PeerTLS)
|
||||
@ -1521,7 +1643,7 @@ func (p SortableProtoMemberSliceByPeerURLs) Swap(i, j int) { p[i], p[j] = p[j],
|
||||
|
||||
// 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)
|
||||
m := c.mustNewMember(t, 0)
|
||||
m.isLearner = resp.Member.IsLearner
|
||||
m.NewCluster = false
|
||||
|
||||
|
@ -173,7 +173,7 @@ func testDecreaseClusterSize(t *testing.T, size int) {
|
||||
}
|
||||
|
||||
func TestForceNewCluster(t *testing.T) {
|
||||
c := NewCluster(t, 3)
|
||||
c := newCluster(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
c.Launch(t)
|
||||
cc := MustNewHTTPClient(t, []string{c.Members[0].URL()}, nil)
|
||||
kapi := client.NewKeysAPI(cc)
|
||||
@ -283,7 +283,7 @@ func testIssue2746(t *testing.T, members int) {
|
||||
func TestIssue2904(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
// start 1-member cluster to ensure member 0 is the leader of the cluster.
|
||||
c := NewCluster(t, 1)
|
||||
c := newCluster(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
c.Launch(t)
|
||||
defer c.Terminate(t)
|
||||
|
||||
@ -319,7 +319,7 @@ func TestIssue2904(t *testing.T) {
|
||||
func TestIssue3699(t *testing.T) {
|
||||
// start a cluster of 3 nodes a, b, c
|
||||
BeforeTest(t)
|
||||
c := NewCluster(t, 3)
|
||||
c := newCluster(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
c.Launch(t)
|
||||
defer c.Terminate(t)
|
||||
|
||||
@ -371,7 +371,7 @@ func TestIssue3699(t *testing.T) {
|
||||
// TestRejectUnhealthyAdd ensures an unhealthy cluster rejects adding members.
|
||||
func TestRejectUnhealthyAdd(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
c := NewCluster(t, 3)
|
||||
c := newCluster(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
for _, m := range c.Members {
|
||||
m.ServerConfig.StrictReconfigCheck = true
|
||||
}
|
||||
@ -415,7 +415,7 @@ func TestRejectUnhealthyAdd(t *testing.T) {
|
||||
// if quorum will be lost.
|
||||
func TestRejectUnhealthyRemove(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
c := NewCluster(t, 5)
|
||||
c := newCluster(t, &ClusterConfig{Size: 5, UseBridge: true})
|
||||
for _, m := range c.Members {
|
||||
m.ServerConfig.StrictReconfigCheck = true
|
||||
}
|
||||
@ -464,7 +464,7 @@ func TestRestartRemoved(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
// 1. start single-member cluster
|
||||
c := NewCluster(t, 1)
|
||||
c := newCluster(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
for _, m := range c.Members {
|
||||
m.ServerConfig.StrictReconfigCheck = true
|
||||
}
|
||||
@ -540,7 +540,7 @@ func clusterMustProgress(t *testing.T, membs []*member) {
|
||||
|
||||
func TestSpeedyTerminate(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3})
|
||||
clus := NewClusterV3(t, &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)
|
||||
|
197
tests/integration/grpc_test.go
Normal file
197
tests/integration/grpc_test.go
Normal file
@ -0,0 +1,197 @@
|
||||
// Copyright 2021 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package integration
|
||||
|
||||
import (
|
||||
"context"
|
||||
tls "crypto/tls"
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
func TestAuthority(t *testing.T) {
|
||||
tcs := []struct {
|
||||
name string
|
||||
useTCP bool
|
||||
useTLS bool
|
||||
// Pattern used to generate endpoints for client. Fields filled
|
||||
// %d - will be filled with member grpc port
|
||||
// %s - will be filled with member name
|
||||
clientURLPattern string
|
||||
|
||||
// Pattern used to validate authority received by server. Fields filled:
|
||||
// %d - will be filled with first member grpc port
|
||||
// %s - will be filled with first member name
|
||||
expectAuthorityPattern string
|
||||
}{
|
||||
{
|
||||
name: "unix:path",
|
||||
clientURLPattern: "unix:localhost:%s",
|
||||
expectAuthorityPattern: "localhost:%s",
|
||||
},
|
||||
{
|
||||
name: "unix://absolute_path",
|
||||
clientURLPattern: "unix://localhost:%s",
|
||||
expectAuthorityPattern: "localhost:%s",
|
||||
},
|
||||
// "unixs" is not standard schema supported by etcd
|
||||
{
|
||||
name: "unixs:absolute_path",
|
||||
useTLS: true,
|
||||
clientURLPattern: "unixs:localhost:%s",
|
||||
expectAuthorityPattern: "localhost:%s",
|
||||
},
|
||||
{
|
||||
name: "unixs://absolute_path",
|
||||
useTLS: true,
|
||||
clientURLPattern: "unixs://localhost:%s",
|
||||
expectAuthorityPattern: "localhost:%s",
|
||||
},
|
||||
{
|
||||
name: "http://domain[:port]",
|
||||
useTCP: true,
|
||||
clientURLPattern: "http://localhost:%d",
|
||||
expectAuthorityPattern: "localhost:%d",
|
||||
},
|
||||
{
|
||||
name: "https://domain[:port]",
|
||||
useTLS: true,
|
||||
useTCP: true,
|
||||
clientURLPattern: "https://localhost:%d",
|
||||
expectAuthorityPattern: "localhost:%d",
|
||||
},
|
||||
{
|
||||
name: "http://address[:port]",
|
||||
useTCP: true,
|
||||
clientURLPattern: "http://127.0.0.1:%d",
|
||||
expectAuthorityPattern: "127.0.0.1:%d",
|
||||
},
|
||||
{
|
||||
name: "https://address[:port]",
|
||||
useTCP: true,
|
||||
useTLS: true,
|
||||
clientURLPattern: "https://127.0.0.1:%d",
|
||||
expectAuthorityPattern: "127.0.0.1:%d",
|
||||
},
|
||||
}
|
||||
for _, tc := range tcs {
|
||||
for _, clusterSize := range []int{1, 3} {
|
||||
t.Run(fmt.Sprintf("Size: %d, Scenario: %q", clusterSize, tc.name), func(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
cfg := ClusterConfig{
|
||||
Size: clusterSize,
|
||||
UseTCP: tc.useTCP,
|
||||
UseIP: tc.useTCP,
|
||||
}
|
||||
cfg, tlsConfig := setupTLS(t, tc.useTLS, cfg)
|
||||
clus := NewClusterV3(t, &cfg)
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kv := setupClient(t, tc.clientURLPattern, clus, tlsConfig)
|
||||
defer kv.Close()
|
||||
|
||||
_, err := kv.Put(context.TODO(), "foo", "bar")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
assertAuthority(t, templateAuthority(t, tc.expectAuthorityPattern, clus.Members[0]), clus)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func setupTLS(t *testing.T, useTLS bool, cfg ClusterConfig) (ClusterConfig, *tls.Config) {
|
||||
t.Helper()
|
||||
if useTLS {
|
||||
cfg.ClientTLS = &testTLSInfo
|
||||
tlsConfig, err := testTLSInfo.ClientConfig()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return cfg, tlsConfig
|
||||
}
|
||||
return cfg, nil
|
||||
}
|
||||
|
||||
func setupClient(t *testing.T, endpointPattern string, clus *ClusterV3, tlsConfig *tls.Config) *clientv3.Client {
|
||||
t.Helper()
|
||||
endpoints := templateEndpoints(t, endpointPattern, clus)
|
||||
kv, err := clientv3.New(clientv3.Config{
|
||||
Endpoints: endpoints,
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
TLS: tlsConfig,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return kv
|
||||
}
|
||||
|
||||
func templateEndpoints(t *testing.T, pattern string, clus *ClusterV3) []string {
|
||||
t.Helper()
|
||||
endpoints := []string{}
|
||||
for _, m := range clus.Members {
|
||||
ent := pattern
|
||||
if strings.Contains(ent, "%d") {
|
||||
ent = fmt.Sprintf(ent, GrpcPortNumber(m.UniqNumber, m.MemberNumber))
|
||||
}
|
||||
if strings.Contains(ent, "%s") {
|
||||
ent = fmt.Sprintf(ent, m.Name)
|
||||
}
|
||||
if strings.Contains(ent, "%") {
|
||||
t.Fatalf("Failed to template pattern, %% symbol left %q", ent)
|
||||
}
|
||||
endpoints = append(endpoints, ent)
|
||||
}
|
||||
return endpoints
|
||||
}
|
||||
|
||||
func templateAuthority(t *testing.T, pattern string, m *member) string {
|
||||
t.Helper()
|
||||
authority := pattern
|
||||
if strings.Contains(authority, "%d") {
|
||||
authority = fmt.Sprintf(authority, GrpcPortNumber(m.UniqNumber, m.MemberNumber))
|
||||
}
|
||||
if strings.Contains(authority, "%s") {
|
||||
authority = fmt.Sprintf(authority, m.Name)
|
||||
}
|
||||
if strings.Contains(authority, "%") {
|
||||
t.Fatalf("Failed to template pattern, %% symbol left %q", authority)
|
||||
}
|
||||
return authority
|
||||
}
|
||||
|
||||
func assertAuthority(t *testing.T, expectedAuthority string, clus *ClusterV3) {
|
||||
t.Helper()
|
||||
requestsFound := 0
|
||||
for _, m := range clus.Members {
|
||||
for _, r := range m.RecordedRequests() {
|
||||
requestsFound++
|
||||
if r.Authority != expectedAuthority {
|
||||
t.Errorf("Got unexpected authority header, member: %q, request: %q, got authority: %q, expected %q", m.Name, r.FullMethod, r.Authority, expectedAuthority)
|
||||
}
|
||||
}
|
||||
}
|
||||
if requestsFound == 0 {
|
||||
t.Errorf("Expected at least one request")
|
||||
}
|
||||
}
|
@ -46,7 +46,7 @@ func TestPauseMember(t *testing.T) {
|
||||
|
||||
func TestRestartMember(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
c := NewCluster(t, 3)
|
||||
c := newCluster(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
c.Launch(t)
|
||||
defer c.Terminate(t)
|
||||
|
||||
@ -88,7 +88,7 @@ func TestLaunchDuplicateMemberShouldFail(t *testing.T) {
|
||||
|
||||
func TestSnapshotAndRestartMember(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
m := mustNewMember(t, memberConfig{name: "snapAndRestartTest"})
|
||||
m := mustNewMember(t, memberConfig{name: "snapAndRestartTest", useBridge: true})
|
||||
m.SnapshotCount = 100
|
||||
m.Launch()
|
||||
defer m.Terminate(t)
|
||||
|
@ -36,7 +36,7 @@ func TestClusterProxyMemberList(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cts := newClusterProxyServer(zaptest.NewLogger(t), []string{clus.Members[0].GRPCAddr()}, t)
|
||||
cts := newClusterProxyServer(zaptest.NewLogger(t), []string{clus.Members[0].GRPCURL()}, t)
|
||||
defer cts.close(t)
|
||||
|
||||
cfg := clientv3.Config{
|
||||
|
@ -34,7 +34,7 @@ func TestKVProxyRange(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvts := newKVProxyServer([]string{clus.Members[0].GRPCAddr()}, t)
|
||||
kvts := newKVProxyServer([]string{clus.Members[0].GRPCURL()}, t)
|
||||
defer kvts.close()
|
||||
|
||||
// create a client and try to get key from proxy.
|
||||
|
@ -31,7 +31,7 @@ func TestRegister(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.Client(0)
|
||||
paddr := clus.Members[0].GRPCAddr()
|
||||
paddr := clus.Members[0].GRPCURL()
|
||||
|
||||
testPrefix := "test-name"
|
||||
wa := mustCreateWatcher(t, cli, testPrefix)
|
||||
|
@ -35,7 +35,7 @@ func TestV3StorageQuotaApply(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
quotasize := int64(16 * os.Getpagesize())
|
||||
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 2})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 2, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
kvc0 := toGRPC(clus.Client(0)).KV
|
||||
kvc1 := toGRPC(clus.Client(1)).KV
|
||||
@ -147,7 +147,7 @@ func TestV3AlarmDeactivate(t *testing.T) {
|
||||
|
||||
func TestV3CorruptAlarm(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
var wg sync.WaitGroup
|
||||
|
@ -61,7 +61,7 @@ func TestV3MaintenanceDefragmentInflightRange(t *testing.T) {
|
||||
// See https://github.com/etcd-io/etcd/issues/7322 for more detail.
|
||||
func TestV3KVInflightRangeRequests(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
|
@ -22,6 +22,7 @@ import (
|
||||
"math/rand"
|
||||
"os"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@ -88,7 +89,7 @@ func TestV3PutOverwrite(t *testing.T) {
|
||||
// TestPutRestart checks if a put after an unrelated member restart succeeds
|
||||
func TestV3PutRestart(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvIdx := rand.Intn(3)
|
||||
@ -1210,7 +1211,7 @@ func TestV3Hash(t *testing.T) {
|
||||
// TestV3HashRestart ensures that hash stays the same after restart.
|
||||
func TestV3HashRestart(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
@ -1243,7 +1244,7 @@ func TestV3StorageQuotaAPI(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
quotasize := int64(16 * os.Getpagesize())
|
||||
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
|
||||
// Set a quota on one node
|
||||
clus.Members[0].QuotaBackendBytes = quotasize
|
||||
@ -1601,8 +1602,10 @@ func TestTLSGRPCRejectSecureClient(t *testing.T) {
|
||||
|
||||
clus.Members[0].ClientTLSInfo = &testTLSInfo
|
||||
clus.Members[0].DialOptions = []grpc.DialOption{grpc.WithBlock()}
|
||||
clus.Members[0].grpcURL = strings.Replace(clus.Members[0].grpcURL, "http://", "https://", 1)
|
||||
client, err := NewClientV3(clus.Members[0])
|
||||
if client != nil || err == nil {
|
||||
client.Close()
|
||||
t.Fatalf("expected no client")
|
||||
} else if err != context.DeadlineExceeded {
|
||||
t.Fatalf("unexpected error (%v)", err)
|
||||
@ -1784,7 +1787,7 @@ func testTLSReload(
|
||||
}
|
||||
cli, cerr := NewClient(t, clientv3.Config{
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
Endpoints: []string{clus.Members[0].GRPCURL()},
|
||||
DialTimeout: time.Second,
|
||||
TLS: cc,
|
||||
})
|
||||
@ -1818,7 +1821,7 @@ func testTLSReload(
|
||||
t.Fatal(terr)
|
||||
}
|
||||
cl, cerr := NewClient(t, clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
Endpoints: []string{clus.Members[0].GRPCURL()},
|
||||
DialTimeout: 5 * time.Second,
|
||||
TLS: tls,
|
||||
})
|
||||
@ -1858,7 +1861,7 @@ func TestGRPCRequireLeader(t *testing.T) {
|
||||
func TestGRPCStreamRequireLeader(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
cfg := ClusterConfig{Size: 3}
|
||||
cfg := ClusterConfig{Size: 3, UseBridge: true}
|
||||
clus := newClusterV3NoClients(t, &cfg)
|
||||
defer clus.Terminate(t)
|
||||
|
||||
|
@ -36,7 +36,7 @@ import (
|
||||
func TestV3LeasePromote(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 3, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
// create lease
|
||||
@ -237,6 +237,7 @@ func TestV3LeaseCheckpoint(t *testing.T) {
|
||||
Size: 3,
|
||||
EnableLeaseCheckpoint: true,
|
||||
LeaseCheckpointInterval: leaseInterval,
|
||||
UseBridge: true,
|
||||
})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
@ -649,7 +650,7 @@ const fiveMinTTL int64 = 300
|
||||
func TestV3LeaseRecoverAndRevoke(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := toGRPC(clus.Client(0)).KV
|
||||
@ -700,7 +701,7 @@ func TestV3LeaseRecoverAndRevoke(t *testing.T) {
|
||||
func TestV3LeaseRevokeAndRecover(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := toGRPC(clus.Client(0)).KV
|
||||
@ -752,7 +753,7 @@ func TestV3LeaseRevokeAndRecover(t *testing.T) {
|
||||
func TestV3LeaseRecoverKeyWithDetachedLease(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := toGRPC(clus.Client(0)).KV
|
||||
@ -808,7 +809,7 @@ func TestV3LeaseRecoverKeyWithDetachedLease(t *testing.T) {
|
||||
func TestV3LeaseRecoverKeyWithMutipleLease(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
kvc := toGRPC(clus.Client(0)).KV
|
||||
|
@ -55,7 +55,7 @@ func testTLSCipherSuites(t *testing.T, valid bool) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
cli, cerr := NewClient(t, clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
Endpoints: []string{clus.Members[0].GRPCURL()},
|
||||
DialTimeout: time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
TLS: cc,
|
||||
|
@ -1034,7 +1034,7 @@ func TestWatchWithProgressNotify(t *testing.T) {
|
||||
// TestV3WatcMultiOpenhClose opens many watchers concurrently on multiple streams.
|
||||
func TestV3WatchClose(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1})
|
||||
clus := NewClusterV3(t, &ClusterConfig{Size: 1, UseBridge: true})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
c := clus.Client(0)
|
||||
@ -1062,7 +1062,7 @@ func TestV3WatchClose(t *testing.T) {
|
||||
}()
|
||||
}
|
||||
|
||||
clus.Members[0].DropConnections()
|
||||
clus.Members[0].Bridge().DropConnections()
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user