mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
tests: Extract e2e cluster setup to separate package
Signed-off-by: Siyuan Zhang <sizhang@google.com>
This commit is contained in:
parent
7df2e908a1
commit
795dcedd1d
@ -1,537 +0,0 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/url"
|
||||
"os"
|
||||
"path"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/server/v3/etcdserver"
|
||||
"go.etcd.io/etcd/tests/v3/integration"
|
||||
"go.uber.org/zap"
|
||||
"go.uber.org/zap/zaptest"
|
||||
)
|
||||
|
||||
const etcdProcessBasePort = 20000
|
||||
|
||||
var (
|
||||
fixturesDir = integration.MustAbsPath("../fixtures")
|
||||
)
|
||||
|
||||
func newConfigNoTLS() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{clusterSize: 3,
|
||||
initialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigAutoTLS() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
isPeerTLS: true,
|
||||
isPeerAutoTLS: true,
|
||||
initialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigTLS() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
clientTLS: clientTLS,
|
||||
isPeerTLS: true,
|
||||
initialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigClientTLS() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
clientTLS: clientTLS,
|
||||
initialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigClientBoth() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
clientTLS: clientTLSAndNonTLS,
|
||||
initialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigClientAutoTLS() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
isClientAutoTLS: true,
|
||||
clientTLS: clientTLS,
|
||||
initialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigPeerTLS() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
isPeerTLS: true,
|
||||
initialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigClientTLSCertAuth() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
clientTLS: clientTLS,
|
||||
initialToken: "new",
|
||||
clientCertAuthEnabled: true,
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigClientTLSCertAuthWithNoCN() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
clientTLS: clientTLS,
|
||||
initialToken: "new",
|
||||
clientCertAuthEnabled: true,
|
||||
noCN: true,
|
||||
}
|
||||
}
|
||||
|
||||
func newConfigJWT() *etcdProcessClusterConfig {
|
||||
return &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
initialToken: "new",
|
||||
authTokenOpts: "jwt,pub-key=" + path.Join(fixturesDir, "server.crt") +
|
||||
",priv-key=" + path.Join(fixturesDir, "server.key.insecure") + ",sign-method=RS256,ttl=1s",
|
||||
}
|
||||
}
|
||||
|
||||
func configStandalone(cfg etcdProcessClusterConfig) *etcdProcessClusterConfig {
|
||||
ret := cfg
|
||||
ret.clusterSize = 1
|
||||
return &ret
|
||||
}
|
||||
|
||||
type etcdProcessCluster struct {
|
||||
lg *zap.Logger
|
||||
cfg *etcdProcessClusterConfig
|
||||
procs []etcdProcess
|
||||
}
|
||||
|
||||
type etcdProcessClusterConfig struct {
|
||||
execPath string
|
||||
dataDirPath string
|
||||
keepDataDir bool
|
||||
envVars map[string]string
|
||||
|
||||
clusterSize int
|
||||
|
||||
baseScheme string
|
||||
basePort int
|
||||
|
||||
metricsURLScheme string
|
||||
|
||||
snapshotCount int // default is 10000
|
||||
|
||||
clientTLS clientConnType
|
||||
clientCertAuthEnabled bool
|
||||
clientHttpSeparate bool
|
||||
isPeerTLS bool
|
||||
isPeerAutoTLS bool
|
||||
isClientAutoTLS bool
|
||||
isClientCRL bool
|
||||
noCN bool
|
||||
|
||||
cipherSuites []string
|
||||
|
||||
forceNewCluster bool
|
||||
initialToken string
|
||||
quotaBackendBytes int64
|
||||
noStrictReconfig bool
|
||||
enableV2 bool
|
||||
initialCorruptCheck bool
|
||||
authTokenOpts string
|
||||
v2deprecation string
|
||||
|
||||
rollingStart bool
|
||||
logLevel string
|
||||
|
||||
MaxConcurrentStreams uint32 // default is math.MaxUint32
|
||||
CorruptCheckTime time.Duration
|
||||
CompactHashCheckEnabled bool
|
||||
CompactHashCheckTime time.Duration
|
||||
WatchProcessNotifyInterval time.Duration
|
||||
CompactionBatchLimit int
|
||||
}
|
||||
|
||||
// newEtcdProcessCluster launches a new cluster from etcd processes, returning
|
||||
// a new etcdProcessCluster once all nodes are ready to accept client requests.
|
||||
func newEtcdProcessCluster(t testing.TB, cfg *etcdProcessClusterConfig) (*etcdProcessCluster, error) {
|
||||
epc, err := initEtcdProcessCluster(t, cfg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return startEtcdProcessCluster(epc, cfg)
|
||||
}
|
||||
|
||||
// initEtcdProcessCluster initializes a new cluster based on the given config.
|
||||
// It doesn't start the cluster.
|
||||
func initEtcdProcessCluster(t testing.TB, cfg *etcdProcessClusterConfig) (*etcdProcessCluster, error) {
|
||||
skipInShortMode(t)
|
||||
|
||||
etcdCfgs := cfg.etcdServerProcessConfigs(t)
|
||||
epc := &etcdProcessCluster{
|
||||
cfg: cfg,
|
||||
lg: zaptest.NewLogger(t),
|
||||
procs: make([]etcdProcess, cfg.clusterSize),
|
||||
}
|
||||
|
||||
// launch etcd processes
|
||||
for i := range etcdCfgs {
|
||||
proc, err := newEtcdProcess(etcdCfgs[i])
|
||||
if err != nil {
|
||||
epc.Close()
|
||||
return nil, fmt.Errorf("Cannot configure: %v", err)
|
||||
}
|
||||
epc.procs[i] = proc
|
||||
}
|
||||
return epc, nil
|
||||
}
|
||||
|
||||
// startEtcdProcessCluster launches a new cluster from etcd processes.
|
||||
func startEtcdProcessCluster(epc *etcdProcessCluster, cfg *etcdProcessClusterConfig) (*etcdProcessCluster, error) {
|
||||
if cfg.rollingStart {
|
||||
if err := epc.RollingStart(); err != nil {
|
||||
return nil, fmt.Errorf("Cannot rolling-start: %v", err)
|
||||
}
|
||||
} else {
|
||||
if err := epc.Start(); err != nil {
|
||||
return nil, fmt.Errorf("Cannot start: %v", err)
|
||||
}
|
||||
}
|
||||
return epc, nil
|
||||
}
|
||||
|
||||
func (cfg *etcdProcessClusterConfig) clientScheme() string {
|
||||
if cfg.clientTLS == clientTLS {
|
||||
return "https"
|
||||
}
|
||||
return "http"
|
||||
}
|
||||
|
||||
func (cfg *etcdProcessClusterConfig) peerScheme() string {
|
||||
peerScheme := cfg.baseScheme
|
||||
if peerScheme == "" {
|
||||
peerScheme = "http"
|
||||
}
|
||||
if cfg.isPeerTLS {
|
||||
peerScheme += "s"
|
||||
}
|
||||
return peerScheme
|
||||
}
|
||||
|
||||
func (cfg *etcdProcessClusterConfig) etcdServerProcessConfigs(tb testing.TB) []*etcdServerProcessConfig {
|
||||
lg := zaptest.NewLogger(tb)
|
||||
|
||||
if cfg.basePort == 0 {
|
||||
cfg.basePort = etcdProcessBasePort
|
||||
}
|
||||
if cfg.execPath == "" {
|
||||
cfg.execPath = binPath
|
||||
}
|
||||
if cfg.snapshotCount == 0 {
|
||||
cfg.snapshotCount = etcdserver.DefaultSnapshotCount
|
||||
}
|
||||
|
||||
etcdCfgs := make([]*etcdServerProcessConfig, cfg.clusterSize)
|
||||
initialCluster := make([]string, cfg.clusterSize)
|
||||
for i := 0; i < cfg.clusterSize; i++ {
|
||||
var curls []string
|
||||
var curl string
|
||||
port := cfg.basePort + 5*i
|
||||
clientPort := port
|
||||
clientHttpPort := port + 4
|
||||
|
||||
if cfg.clientTLS == clientTLSAndNonTLS {
|
||||
curl = clientURL(clientPort, clientNonTLS)
|
||||
curls = []string{curl, clientURL(clientPort, clientTLS)}
|
||||
} else {
|
||||
curl = clientURL(clientPort, cfg.clientTLS)
|
||||
curls = []string{curl}
|
||||
}
|
||||
|
||||
purl := url.URL{Scheme: cfg.peerScheme(), Host: fmt.Sprintf("localhost:%d", port+1)}
|
||||
name := fmt.Sprintf("test-%d", i)
|
||||
dataDirPath := cfg.dataDirPath
|
||||
if cfg.dataDirPath == "" {
|
||||
dataDirPath = tb.TempDir()
|
||||
}
|
||||
initialCluster[i] = fmt.Sprintf("%s=%s", name, purl.String())
|
||||
|
||||
args := []string{
|
||||
"--name", name,
|
||||
"--listen-client-urls", strings.Join(curls, ","),
|
||||
"--advertise-client-urls", strings.Join(curls, ","),
|
||||
"--listen-peer-urls", purl.String(),
|
||||
"--initial-advertise-peer-urls", purl.String(),
|
||||
"--initial-cluster-token", cfg.initialToken,
|
||||
"--data-dir", dataDirPath,
|
||||
"--snapshot-count", fmt.Sprintf("%d", cfg.snapshotCount),
|
||||
}
|
||||
var clientHttpUrl string
|
||||
if cfg.clientHttpSeparate {
|
||||
clientHttpUrl = clientURL(clientHttpPort, cfg.clientTLS)
|
||||
args = append(args, "--listen-client-http-urls", clientHttpUrl)
|
||||
}
|
||||
args = addV2Args(args)
|
||||
if cfg.forceNewCluster {
|
||||
args = append(args, "--force-new-cluster")
|
||||
}
|
||||
if cfg.quotaBackendBytes > 0 {
|
||||
args = append(args,
|
||||
"--quota-backend-bytes", fmt.Sprintf("%d", cfg.quotaBackendBytes),
|
||||
)
|
||||
}
|
||||
if cfg.noStrictReconfig {
|
||||
args = append(args, "--strict-reconfig-check=false")
|
||||
}
|
||||
if cfg.enableV2 {
|
||||
args = append(args, "--enable-v2")
|
||||
}
|
||||
if cfg.initialCorruptCheck {
|
||||
args = append(args, "--experimental-initial-corrupt-check")
|
||||
}
|
||||
var murl string
|
||||
if cfg.metricsURLScheme != "" {
|
||||
murl = (&url.URL{
|
||||
Scheme: cfg.metricsURLScheme,
|
||||
Host: fmt.Sprintf("localhost:%d", port+2),
|
||||
}).String()
|
||||
args = append(args, "--listen-metrics-urls", murl)
|
||||
}
|
||||
|
||||
args = append(args, cfg.tlsArgs()...)
|
||||
|
||||
if cfg.authTokenOpts != "" {
|
||||
args = append(args, "--auth-token", cfg.authTokenOpts)
|
||||
}
|
||||
|
||||
if cfg.v2deprecation != "" {
|
||||
args = append(args, "--v2-deprecation", cfg.v2deprecation)
|
||||
}
|
||||
|
||||
if cfg.logLevel != "" {
|
||||
args = append(args, "--log-level", cfg.logLevel)
|
||||
}
|
||||
|
||||
if cfg.MaxConcurrentStreams != 0 {
|
||||
args = append(args, "--max-concurrent-streams", fmt.Sprintf("%d", cfg.MaxConcurrentStreams))
|
||||
}
|
||||
|
||||
if cfg.CorruptCheckTime != 0 {
|
||||
args = append(args, "--experimental-corrupt-check-time", fmt.Sprintf("%s", cfg.CorruptCheckTime))
|
||||
}
|
||||
if cfg.CompactHashCheckEnabled {
|
||||
args = append(args, "--experimental-compact-hash-check-enabled")
|
||||
}
|
||||
if cfg.CompactHashCheckTime != 0 {
|
||||
args = append(args, "--experimental-compact-hash-check-time", cfg.CompactHashCheckTime.String())
|
||||
}
|
||||
if cfg.WatchProcessNotifyInterval != 0 {
|
||||
args = append(args, "--experimental-watch-progress-notify-interval", cfg.WatchProcessNotifyInterval.String())
|
||||
}
|
||||
if cfg.CompactionBatchLimit != 0 {
|
||||
args = append(args, "--experimental-compaction-batch-limit", fmt.Sprintf("%d", cfg.CompactionBatchLimit))
|
||||
}
|
||||
|
||||
etcdCfgs[i] = &etcdServerProcessConfig{
|
||||
lg: lg,
|
||||
execPath: cfg.execPath,
|
||||
args: args,
|
||||
envVars: cfg.envVars,
|
||||
tlsArgs: cfg.tlsArgs(),
|
||||
dataDirPath: dataDirPath,
|
||||
keepDataDir: cfg.keepDataDir,
|
||||
name: name,
|
||||
purl: purl,
|
||||
acurl: curl,
|
||||
murl: murl,
|
||||
initialToken: cfg.initialToken,
|
||||
clientHttpUrl: clientHttpUrl,
|
||||
}
|
||||
}
|
||||
|
||||
initialClusterArgs := []string{"--initial-cluster", strings.Join(initialCluster, ",")}
|
||||
for i := range etcdCfgs {
|
||||
etcdCfgs[i].initialCluster = strings.Join(initialCluster, ",")
|
||||
etcdCfgs[i].args = append(etcdCfgs[i].args, initialClusterArgs...)
|
||||
}
|
||||
|
||||
return etcdCfgs
|
||||
}
|
||||
|
||||
func clientURL(port int, connType clientConnType) string {
|
||||
curlHost := fmt.Sprintf("localhost:%d", port)
|
||||
switch connType {
|
||||
case clientNonTLS:
|
||||
return (&url.URL{Scheme: "http", Host: curlHost}).String()
|
||||
case clientTLS:
|
||||
return (&url.URL{Scheme: "https", Host: curlHost}).String()
|
||||
default:
|
||||
panic(fmt.Sprintf("Unsupported connection type %v", connType))
|
||||
}
|
||||
}
|
||||
|
||||
func (cfg *etcdProcessClusterConfig) tlsArgs() (args []string) {
|
||||
if cfg.clientTLS != clientNonTLS {
|
||||
if cfg.isClientAutoTLS {
|
||||
args = append(args, "--auto-tls")
|
||||
} else {
|
||||
tlsClientArgs := []string{
|
||||
"--cert-file", certPath,
|
||||
"--key-file", privateKeyPath,
|
||||
"--trusted-ca-file", caPath,
|
||||
}
|
||||
args = append(args, tlsClientArgs...)
|
||||
|
||||
if cfg.clientCertAuthEnabled {
|
||||
args = append(args, "--client-cert-auth")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if cfg.isPeerTLS {
|
||||
if cfg.isPeerAutoTLS {
|
||||
args = append(args, "--peer-auto-tls")
|
||||
} else {
|
||||
tlsPeerArgs := []string{
|
||||
"--peer-cert-file", certPath,
|
||||
"--peer-key-file", privateKeyPath,
|
||||
"--peer-trusted-ca-file", caPath,
|
||||
}
|
||||
args = append(args, tlsPeerArgs...)
|
||||
}
|
||||
}
|
||||
|
||||
if cfg.isClientCRL {
|
||||
args = append(args, "--client-crl-file", crlPath, "--client-cert-auth")
|
||||
}
|
||||
|
||||
if len(cfg.cipherSuites) > 0 {
|
||||
args = append(args, "--cipher-suites", strings.Join(cfg.cipherSuites, ","))
|
||||
}
|
||||
|
||||
return args
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) EndpointsV2() []string {
|
||||
return epc.endpoints(func(ep etcdProcess) []string { return ep.EndpointsV2() })
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) EndpointsV3() []string {
|
||||
return epc.endpoints(func(ep etcdProcess) []string { return ep.EndpointsV3() })
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) endpoints(f func(ep etcdProcess) []string) (ret []string) {
|
||||
for _, p := range epc.procs {
|
||||
ret = append(ret, f(p)...)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) Start() error {
|
||||
return epc.start(func(ep etcdProcess) error { return ep.Start() })
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) RollingStart() error {
|
||||
return epc.rollingStart(func(ep etcdProcess) error { return ep.Start() })
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) Restart() error {
|
||||
return epc.start(func(ep etcdProcess) error { return ep.Restart() })
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) start(f func(ep etcdProcess) error) error {
|
||||
readyC := make(chan error, len(epc.procs))
|
||||
for i := range epc.procs {
|
||||
go func(n int) { readyC <- f(epc.procs[n]) }(i)
|
||||
}
|
||||
for range epc.procs {
|
||||
if err := <-readyC; err != nil {
|
||||
epc.Close()
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) rollingStart(f func(ep etcdProcess) error) error {
|
||||
readyC := make(chan error, len(epc.procs))
|
||||
for i := range epc.procs {
|
||||
go func(n int) { readyC <- f(epc.procs[n]) }(i)
|
||||
// make sure the servers do not start at the same time
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
for range epc.procs {
|
||||
if err := <-readyC; err != nil {
|
||||
epc.Close()
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) Stop() (err error) {
|
||||
for _, p := range epc.procs {
|
||||
if p == nil {
|
||||
continue
|
||||
}
|
||||
if curErr := p.Stop(); curErr != nil {
|
||||
if err != nil {
|
||||
err = fmt.Errorf("%v; %v", err, curErr)
|
||||
} else {
|
||||
err = curErr
|
||||
}
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) Close() error {
|
||||
epc.lg.Info("closing test cluster...")
|
||||
err := epc.Stop()
|
||||
for _, p := range epc.procs {
|
||||
// p is nil when newEtcdProcess fails in the middle
|
||||
// Close still gets called to clean up test data
|
||||
if p == nil {
|
||||
continue
|
||||
}
|
||||
if cerr := p.Close(); cerr != nil {
|
||||
err = cerr
|
||||
}
|
||||
}
|
||||
epc.lg.Info("closed test cluster.")
|
||||
return err
|
||||
}
|
||||
|
||||
func (epc *etcdProcessCluster) WithStopSignal(sig os.Signal) (ret os.Signal) {
|
||||
for _, p := range epc.procs {
|
||||
ret = p.WithStopSignal(sig)
|
||||
}
|
||||
return ret
|
||||
}
|
@ -34,76 +34,77 @@ import (
|
||||
"go.etcd.io/etcd/api/v3/version"
|
||||
clientv2 "go.etcd.io/etcd/client/v2"
|
||||
"go.etcd.io/etcd/server/v3/etcdserver/api/etcdhttp"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestConnectionMultiplexing(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
for _, tc := range []struct {
|
||||
name string
|
||||
serverTLS clientConnType
|
||||
serverTLS e2e.ClientConnType
|
||||
separateHttpPort bool
|
||||
}{
|
||||
{
|
||||
name: "ServerTLS",
|
||||
serverTLS: clientTLS,
|
||||
serverTLS: e2e.ClientTLS,
|
||||
},
|
||||
{
|
||||
name: "ServerNonTLS",
|
||||
serverTLS: clientNonTLS,
|
||||
serverTLS: e2e.ClientNonTLS,
|
||||
},
|
||||
{
|
||||
name: "ServerTLSAndNonTLS",
|
||||
serverTLS: clientTLSAndNonTLS,
|
||||
serverTLS: e2e.ClientTLSAndNonTLS,
|
||||
},
|
||||
{
|
||||
name: "SeparateHTTP/ServerTLS",
|
||||
serverTLS: clientTLS,
|
||||
serverTLS: e2e.ClientTLS,
|
||||
separateHttpPort: true,
|
||||
},
|
||||
{
|
||||
name: "SeparateHTTP/ServerNonTLS",
|
||||
serverTLS: clientNonTLS,
|
||||
serverTLS: e2e.ClientNonTLS,
|
||||
separateHttpPort: true,
|
||||
},
|
||||
} {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
cfg := etcdProcessClusterConfig{clusterSize: 1, clientTLS: tc.serverTLS, enableV2: true, clientHttpSeparate: tc.separateHttpPort}
|
||||
clus, err := newEtcdProcessCluster(t, &cfg)
|
||||
cfg := e2e.EtcdProcessClusterConfig{ClusterSize: 1, ClientTLS: tc.serverTLS, EnableV2: true, ClientHttpSeparate: tc.separateHttpPort}
|
||||
clus, err := e2e.NewEtcdProcessCluster(t, &cfg)
|
||||
require.NoError(t, err)
|
||||
defer clus.Close()
|
||||
|
||||
var clientScenarios []clientConnType
|
||||
var clientScenarios []e2e.ClientConnType
|
||||
switch tc.serverTLS {
|
||||
case clientTLS:
|
||||
clientScenarios = []clientConnType{clientTLS}
|
||||
case clientNonTLS:
|
||||
clientScenarios = []clientConnType{clientNonTLS}
|
||||
case clientTLSAndNonTLS:
|
||||
clientScenarios = []clientConnType{clientTLS, clientNonTLS}
|
||||
case e2e.ClientTLS:
|
||||
clientScenarios = []e2e.ClientConnType{e2e.ClientTLS}
|
||||
case e2e.ClientNonTLS:
|
||||
clientScenarios = []e2e.ClientConnType{e2e.ClientNonTLS}
|
||||
case e2e.ClientTLSAndNonTLS:
|
||||
clientScenarios = []e2e.ClientConnType{e2e.ClientTLS, e2e.ClientNonTLS}
|
||||
}
|
||||
|
||||
for _, connType := range clientScenarios {
|
||||
name := "ClientNonTLS"
|
||||
if connType == clientTLS {
|
||||
if connType == e2e.ClientTLS {
|
||||
name = "ClientTLS"
|
||||
}
|
||||
t.Run(name, func(t *testing.T) {
|
||||
testConnectionMultiplexing(ctx, t, clus.procs[0], connType)
|
||||
testConnectionMultiplexing(ctx, t, clus.Procs[0], connType)
|
||||
})
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func testConnectionMultiplexing(ctx context.Context, t *testing.T, member etcdProcess, connType clientConnType) {
|
||||
func testConnectionMultiplexing(ctx context.Context, t *testing.T, member e2e.EtcdProcess, connType e2e.ClientConnType) {
|
||||
httpEndpoint := member.EndpointsHTTP()[0]
|
||||
grpcEndpoint := member.EndpointsGRPC()[0]
|
||||
switch connType {
|
||||
case clientTLS:
|
||||
httpEndpoint = toTLS(httpEndpoint)
|
||||
grpcEndpoint = toTLS(grpcEndpoint)
|
||||
case clientNonTLS:
|
||||
case e2e.ClientTLS:
|
||||
httpEndpoint = e2e.ToTLS(httpEndpoint)
|
||||
grpcEndpoint = e2e.ToTLS(grpcEndpoint)
|
||||
case e2e.ClientNonTLS:
|
||||
default:
|
||||
panic(fmt.Sprintf("Unsupported conn type %v", connType))
|
||||
}
|
||||
@ -148,14 +149,14 @@ func testConnectionMultiplexing(ctx context.Context, t *testing.T, member etcdPr
|
||||
})
|
||||
}
|
||||
|
||||
func fetchGrpcGateway(endpoint string, httpVersion string, connType clientConnType) error {
|
||||
func fetchGrpcGateway(endpoint string, httpVersion string, connType e2e.ClientConnType) error {
|
||||
rangeData, err := json.Marshal(&pb.RangeRequest{
|
||||
Key: []byte("a"),
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
req := cURLReq{endpoint: "/v3/kv/range", value: string(rangeData), timeout: 5, httpVersion: httpVersion}
|
||||
req := e2e.CURLReq{Endpoint: "/v3/kv/range", Value: string(rangeData), Timeout: 5, HttpVersion: httpVersion}
|
||||
respData, err := curl(endpoint, "POST", req, connType)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -189,11 +190,11 @@ func validateGrpcgatewayRangeReponse(respData []byte) error {
|
||||
return json.Unmarshal(respData, &resp)
|
||||
}
|
||||
|
||||
func fetchMetrics(t *testing.T, endpoint string, httpVersion string, connType clientConnType) error {
|
||||
func fetchMetrics(t *testing.T, endpoint string, httpVersion string, connType e2e.ClientConnType) error {
|
||||
tmpDir := t.TempDir()
|
||||
metricFile := filepath.Join(tmpDir, "metrics")
|
||||
|
||||
req := cURLReq{endpoint: "/metrics", timeout: 5, httpVersion: httpVersion, OutputFile: metricFile}
|
||||
req := e2e.CURLReq{Endpoint: "/metrics", Timeout: 5, HttpVersion: httpVersion, OutputFile: metricFile}
|
||||
if _, err := curl(endpoint, "GET", req, connType); err != nil {
|
||||
return err
|
||||
}
|
||||
@ -209,8 +210,8 @@ func fetchMetrics(t *testing.T, endpoint string, httpVersion string, connType cl
|
||||
return err
|
||||
}
|
||||
|
||||
func fetchVersion(endpoint string, httpVersion string, connType clientConnType) error {
|
||||
req := cURLReq{endpoint: "/version", timeout: 5, httpVersion: httpVersion}
|
||||
func fetchVersion(endpoint string, httpVersion string, connType e2e.ClientConnType) error {
|
||||
req := e2e.CURLReq{Endpoint: "/version", Timeout: 5, HttpVersion: httpVersion}
|
||||
respData, err := curl(endpoint, "GET", req, connType)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -219,8 +220,8 @@ func fetchVersion(endpoint string, httpVersion string, connType clientConnType)
|
||||
return json.Unmarshal([]byte(respData), &resp)
|
||||
}
|
||||
|
||||
func fetchHealth(endpoint string, httpVersion string, connType clientConnType) error {
|
||||
req := cURLReq{endpoint: "/health", timeout: 5, httpVersion: httpVersion}
|
||||
func fetchHealth(endpoint string, httpVersion string, connType e2e.ClientConnType) error {
|
||||
req := e2e.CURLReq{Endpoint: "/health", Timeout: 5, HttpVersion: httpVersion}
|
||||
respData, err := curl(endpoint, "GET", req, connType)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -229,8 +230,8 @@ func fetchHealth(endpoint string, httpVersion string, connType clientConnType) e
|
||||
return json.Unmarshal([]byte(respData), &resp)
|
||||
}
|
||||
|
||||
func fetchDebugVars(endpoint string, httpVersion string, connType clientConnType) error {
|
||||
req := cURLReq{endpoint: "/debug/vars", timeout: 5, httpVersion: httpVersion}
|
||||
func fetchDebugVars(endpoint string, httpVersion string, connType e2e.ClientConnType) error {
|
||||
req := e2e.CURLReq{Endpoint: "/debug/vars", Timeout: 5, HttpVersion: httpVersion}
|
||||
respData, err := curl(endpoint, "GET", req, connType)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -239,9 +240,9 @@ func fetchDebugVars(endpoint string, httpVersion string, connType clientConnType
|
||||
return json.Unmarshal([]byte(respData), &resp)
|
||||
}
|
||||
|
||||
func curl(endpoint string, method string, curlReq cURLReq, connType clientConnType) (string, error) {
|
||||
args := cURLPrefixArgs(endpoint, connType, false, method, curlReq)
|
||||
lines, err := runUtilCompletion(args, nil)
|
||||
func curl(endpoint string, method string, curlReq e2e.CURLReq, connType e2e.ClientConnType) (string, error) {
|
||||
args := e2e.CURLPrefixArgs(endpoint, connType, false, method, curlReq)
|
||||
lines, err := e2e.RunUtilCompletion(args, nil)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
@ -27,9 +27,10 @@ import (
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.etcd.io/etcd/api/v3/etcdserverpb"
|
||||
"go.etcd.io/etcd/client/v3"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/server/v3/datadir"
|
||||
"go.etcd.io/etcd/server/v3/storage/mvcc/testutil"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestEtcdCorruptHash(t *testing.T) {
|
||||
@ -37,10 +38,10 @@ func TestEtcdCorruptHash(t *testing.T) {
|
||||
// defer os.Setenv("EXPECT_DEBUG", oldenv)
|
||||
// os.Setenv("EXPECT_DEBUG", "1")
|
||||
|
||||
cfg := newConfigNoTLS()
|
||||
cfg := e2e.NewConfigNoTLS()
|
||||
|
||||
// trigger snapshot so that restart member can load peers from disk
|
||||
cfg.snapshotCount = 3
|
||||
cfg.SnapshotCount = 3
|
||||
|
||||
testCtl(t, corruptTest, withQuorum(),
|
||||
withCfg(*cfg),
|
||||
@ -78,18 +79,18 @@ func corruptTest(cx ctlCtx) {
|
||||
id0 := sresp.Header.GetMemberId()
|
||||
|
||||
cx.t.Log("stopping etcd[0]...")
|
||||
cx.epc.procs[0].Stop()
|
||||
cx.epc.Procs[0].Stop()
|
||||
|
||||
// corrupting first member by modifying backend offline.
|
||||
fp := datadir.ToBackendFileName(cx.epc.procs[0].Config().dataDirPath)
|
||||
fp := datadir.ToBackendFileName(cx.epc.Procs[0].Config().DataDirPath)
|
||||
cx.t.Logf("corrupting backend: %v", fp)
|
||||
if err = cx.corruptFunc(fp); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
cx.t.Log("restarting etcd[0]")
|
||||
ep := cx.epc.procs[0]
|
||||
proc, err := spawnCmd(append([]string{ep.Config().execPath}, ep.Config().args...), cx.envMap)
|
||||
ep := cx.epc.Procs[0]
|
||||
proc, err := e2e.SpawnCmd(append([]string{ep.Config().ExecPath}, ep.Config().Args...), cx.envMap)
|
||||
if err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
@ -97,21 +98,21 @@ func corruptTest(cx ctlCtx) {
|
||||
|
||||
cx.t.Log("waiting for etcd[0] failure...")
|
||||
// restarting corrupted member should fail
|
||||
waitReadyExpectProc(proc, []string{fmt.Sprintf("etcdmain: %016x found data inconsistency with peers", id0)})
|
||||
e2e.WaitReadyExpectProc(proc, []string{fmt.Sprintf("etcdmain: %016x found data inconsistency with peers", id0)})
|
||||
}
|
||||
|
||||
func TestInPlaceRecovery(t *testing.T) {
|
||||
basePort := 20000
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
// Initialize the cluster.
|
||||
epcOld, err := newEtcdProcessCluster(t,
|
||||
&etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
initialToken: "old",
|
||||
keepDataDir: false,
|
||||
epcOld, err := e2e.NewEtcdProcessCluster(t,
|
||||
&e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
InitialToken: "old",
|
||||
KeepDataDir: false,
|
||||
CorruptCheckTime: time.Second,
|
||||
basePort: basePort,
|
||||
BasePort: basePort,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
@ -127,7 +128,7 @@ func TestInPlaceRecovery(t *testing.T) {
|
||||
//Put some data into the old cluster, so that after recovering from a blank db, the hash diverges.
|
||||
t.Log("putting 10 keys...")
|
||||
|
||||
oldCc := NewEtcdctl(epcOld.EndpointsV3(), clientNonTLS, false, false)
|
||||
oldCc := NewEtcdctl(epcOld.EndpointsV3(), e2e.ClientNonTLS, false, false)
|
||||
for i := 0; i < 10; i++ {
|
||||
err := oldCc.Put(testutil.PickKey(int64(i)), fmt.Sprint(i))
|
||||
assert.NoError(t, err, "error on put")
|
||||
@ -135,15 +136,15 @@ func TestInPlaceRecovery(t *testing.T) {
|
||||
|
||||
// Create a new cluster config, but with the same port numbers. In this way the new servers can stay in
|
||||
// contact with the old ones.
|
||||
epcNewConfig := &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
initialToken: "new",
|
||||
keepDataDir: false,
|
||||
epcNewConfig := &e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
InitialToken: "new",
|
||||
KeepDataDir: false,
|
||||
CorruptCheckTime: time.Second,
|
||||
basePort: basePort,
|
||||
initialCorruptCheck: true,
|
||||
BasePort: basePort,
|
||||
InitialCorruptCheck: true,
|
||||
}
|
||||
epcNew, err := initEtcdProcessCluster(t, epcNewConfig)
|
||||
epcNew, err := e2e.InitEtcdProcessCluster(t, epcNewConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("could not init etcd process cluster (%v)", err)
|
||||
}
|
||||
@ -153,30 +154,30 @@ func TestInPlaceRecovery(t *testing.T) {
|
||||
}
|
||||
})
|
||||
|
||||
newCc := NewEtcdctl(epcNew.EndpointsV3(), clientNonTLS, false, false)
|
||||
newCc := NewEtcdctl(epcNew.EndpointsV3(), e2e.ClientNonTLS, false, false)
|
||||
assert.NoError(t, err)
|
||||
|
||||
wg := sync.WaitGroup{}
|
||||
// Rolling recovery of the servers.
|
||||
t.Log("rolling updating servers in place...")
|
||||
for i := range epcNew.procs {
|
||||
oldProc := epcOld.procs[i]
|
||||
for i := range epcNew.Procs {
|
||||
oldProc := epcOld.Procs[i]
|
||||
err = oldProc.Close()
|
||||
if err != nil {
|
||||
t.Fatalf("could not stop etcd process (%v)", err)
|
||||
}
|
||||
t.Logf("old cluster server %d: %s stopped.", i, oldProc.Config().name)
|
||||
t.Logf("old cluster server %d: %s stopped.", i, oldProc.Config().Name)
|
||||
wg.Add(1)
|
||||
// Start servers in background to avoid blocking on server start.
|
||||
// EtcdProcess.Start waits until etcd becomes healthy, which will not happen here until we restart at least 2 members.
|
||||
go func(proc etcdProcess) {
|
||||
go func(proc e2e.EtcdProcess) {
|
||||
defer wg.Done()
|
||||
err = proc.Start()
|
||||
if err != nil {
|
||||
t.Errorf("could not start etcd process (%v)", err)
|
||||
}
|
||||
t.Logf("new cluster server: %s started in-place with blank db.", proc.Config().name)
|
||||
}(epcNew.procs[i])
|
||||
t.Logf("new cluster server: %s started in-place with blank db.", proc.Config().Name)
|
||||
}(epcNew.Procs[i])
|
||||
t.Log("sleeping 5 sec to let nodes do periodical check...")
|
||||
time.Sleep(5 * time.Second)
|
||||
}
|
||||
@ -195,10 +196,10 @@ func TestInPlaceRecovery(t *testing.T) {
|
||||
|
||||
func TestPeriodicCheckDetectsCorruption(t *testing.T) {
|
||||
checkTime := time.Second
|
||||
BeforeTest(t)
|
||||
epc, err := newEtcdProcessCluster(t, &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
keepDataDir: true,
|
||||
e2e.BeforeTest(t)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
KeepDataDir: true,
|
||||
CorruptCheckTime: time.Second,
|
||||
})
|
||||
if err != nil {
|
||||
@ -210,7 +211,7 @@ func TestPeriodicCheckDetectsCorruption(t *testing.T) {
|
||||
}
|
||||
})
|
||||
|
||||
cc := NewEtcdctl(epc.EndpointsV3(), clientNonTLS, false, false)
|
||||
cc := NewEtcdctl(epc.EndpointsV3(), e2e.ClientNonTLS, false, false)
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
err := cc.Put(testutil.PickKey(int64(i)), fmt.Sprint(i))
|
||||
@ -221,16 +222,16 @@ func TestPeriodicCheckDetectsCorruption(t *testing.T) {
|
||||
assert.NoError(t, err, "error on member list")
|
||||
var memberID uint64
|
||||
for _, m := range members.Members {
|
||||
if m.Name == epc.procs[0].Config().name {
|
||||
if m.Name == epc.Procs[0].Config().Name {
|
||||
memberID = m.ID
|
||||
}
|
||||
}
|
||||
assert.NotZero(t, memberID, "member not found")
|
||||
epc.procs[0].Stop()
|
||||
err = testutil.CorruptBBolt(datadir.ToBackendFileName(epc.procs[0].Config().dataDirPath))
|
||||
epc.Procs[0].Stop()
|
||||
err = testutil.CorruptBBolt(datadir.ToBackendFileName(epc.Procs[0].Config().DataDirPath))
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = epc.procs[0].Restart()
|
||||
err = epc.Procs[0].Restart()
|
||||
assert.NoError(t, err)
|
||||
time.Sleep(checkTime * 11 / 10)
|
||||
alarmResponse, err := cc.AlarmList()
|
||||
@ -240,10 +241,10 @@ func TestPeriodicCheckDetectsCorruption(t *testing.T) {
|
||||
|
||||
func TestCompactHashCheckDetectCorruption(t *testing.T) {
|
||||
checkTime := time.Second
|
||||
BeforeTest(t)
|
||||
epc, err := newEtcdProcessCluster(t, &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
keepDataDir: true,
|
||||
e2e.BeforeTest(t)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
KeepDataDir: true,
|
||||
CompactHashCheckEnabled: true,
|
||||
CompactHashCheckTime: checkTime,
|
||||
})
|
||||
@ -256,7 +257,7 @@ func TestCompactHashCheckDetectCorruption(t *testing.T) {
|
||||
}
|
||||
})
|
||||
|
||||
cc := NewEtcdctl(epc.EndpointsV3(), clientNonTLS, false, false)
|
||||
cc := NewEtcdctl(epc.EndpointsV3(), e2e.ClientNonTLS, false, false)
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
err := cc.Put(testutil.PickKey(int64(i)), fmt.Sprint(i))
|
||||
@ -265,11 +266,11 @@ func TestCompactHashCheckDetectCorruption(t *testing.T) {
|
||||
_, err = cc.MemberList()
|
||||
assert.NoError(t, err, "error on member list")
|
||||
|
||||
epc.procs[0].Stop()
|
||||
err = testutil.CorruptBBolt(datadir.ToBackendFileName(epc.procs[0].Config().dataDirPath))
|
||||
epc.Procs[0].Stop()
|
||||
err = testutil.CorruptBBolt(datadir.ToBackendFileName(epc.Procs[0].Config().DataDirPath))
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = epc.procs[0].Restart()
|
||||
err = epc.Procs[0].Restart()
|
||||
assert.NoError(t, err)
|
||||
_, err = cc.Compact(5)
|
||||
assert.NoError(t, err)
|
||||
@ -281,19 +282,19 @@ func TestCompactHashCheckDetectCorruption(t *testing.T) {
|
||||
|
||||
func TestCompactHashCheckDetectCorruptionInterrupt(t *testing.T) {
|
||||
checkTime := time.Second
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
slowCompactionNodeIndex := 1
|
||||
|
||||
// Start a new cluster, with compact hash check enabled.
|
||||
t.Log("creating a new cluster with 3 nodes...")
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, &etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
keepDataDir: true,
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
KeepDataDir: true,
|
||||
CompactHashCheckEnabled: true,
|
||||
CompactHashCheckTime: checkTime,
|
||||
logLevel: "info",
|
||||
LogLevel: "info",
|
||||
CompactionBatchLimit: 1,
|
||||
})
|
||||
require.NoError(t, err)
|
||||
@ -306,7 +307,7 @@ func TestCompactHashCheckDetectCorruptionInterrupt(t *testing.T) {
|
||||
// Put 200 identical keys to the cluster, so that the compaction will drop some stale values.
|
||||
// We need a relatively big number here to make the compaction takes a non-trivial time, and we can interrupt it.
|
||||
t.Log("putting 200 values to the identical key...")
|
||||
cc := NewEtcdctl(epc.EndpointsV3(), clientNonTLS, false, false)
|
||||
cc := NewEtcdctl(epc.EndpointsV3(), e2e.ClientNonTLS, false, false)
|
||||
|
||||
for i := 0; i < 200; i++ {
|
||||
err = cc.Put("key", fmt.Sprint(i))
|
||||
@ -317,11 +318,11 @@ func TestCompactHashCheckDetectCorruptionInterrupt(t *testing.T) {
|
||||
_, err = cc.Compact(200)
|
||||
|
||||
t.Logf("restart proc %d to interrupt its compaction...", slowCompactionNodeIndex)
|
||||
err = epc.procs[slowCompactionNodeIndex].Restart()
|
||||
err = epc.Procs[slowCompactionNodeIndex].Restart()
|
||||
require.NoError(t, err)
|
||||
|
||||
// Wait until the node finished compaction.
|
||||
_, err = epc.procs[slowCompactionNodeIndex].Logs().Expect("finished scheduled compaction")
|
||||
_, err = epc.Procs[slowCompactionNodeIndex].Logs().Expect("finished scheduled compaction")
|
||||
require.NoError(t, err, "can't get log indicating finished scheduled compaction")
|
||||
|
||||
// Wait for compaction hash check
|
||||
|
@ -21,25 +21,27 @@ import (
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func BeforeTestV2(t testing.TB) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
os.Setenv("ETCDCTL_API", "2")
|
||||
t.Cleanup(func() {
|
||||
os.Unsetenv("ETCDCTL_API")
|
||||
})
|
||||
}
|
||||
|
||||
func TestCtlV2Set(t *testing.T) { testCtlV2Set(t, newConfigNoTLS(), false) }
|
||||
func TestCtlV2SetQuorum(t *testing.T) { testCtlV2Set(t, newConfigNoTLS(), true) }
|
||||
func TestCtlV2SetClientTLS(t *testing.T) { testCtlV2Set(t, newConfigClientTLS(), false) }
|
||||
func TestCtlV2SetPeerTLS(t *testing.T) { testCtlV2Set(t, newConfigPeerTLS(), false) }
|
||||
func TestCtlV2SetTLS(t *testing.T) { testCtlV2Set(t, newConfigTLS(), false) }
|
||||
func testCtlV2Set(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) {
|
||||
func TestCtlV2Set(t *testing.T) { testCtlV2Set(t, e2e.NewConfigNoTLS(), false) }
|
||||
func TestCtlV2SetQuorum(t *testing.T) { testCtlV2Set(t, e2e.NewConfigNoTLS(), true) }
|
||||
func TestCtlV2SetClientTLS(t *testing.T) { testCtlV2Set(t, e2e.NewConfigClientTLS(), false) }
|
||||
func TestCtlV2SetPeerTLS(t *testing.T) { testCtlV2Set(t, e2e.NewConfigPeerTLS(), false) }
|
||||
func TestCtlV2SetTLS(t *testing.T) { testCtlV2Set(t, e2e.NewConfigTLS(), false) }
|
||||
func testCtlV2Set(t *testing.T, cfg *e2e.EtcdProcessClusterConfig, quorum bool) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
cfg.enableV2 = true
|
||||
cfg.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, cfg, quorum)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -54,13 +56,13 @@ func testCtlV2Set(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCtlV2Mk(t *testing.T) { testCtlV2Mk(t, newConfigNoTLS(), false) }
|
||||
func TestCtlV2MkQuorum(t *testing.T) { testCtlV2Mk(t, newConfigNoTLS(), true) }
|
||||
func TestCtlV2MkTLS(t *testing.T) { testCtlV2Mk(t, newConfigTLS(), false) }
|
||||
func testCtlV2Mk(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) {
|
||||
func TestCtlV2Mk(t *testing.T) { testCtlV2Mk(t, e2e.NewConfigNoTLS(), false) }
|
||||
func TestCtlV2MkQuorum(t *testing.T) { testCtlV2Mk(t, e2e.NewConfigNoTLS(), true) }
|
||||
func TestCtlV2MkTLS(t *testing.T) { testCtlV2Mk(t, e2e.NewConfigTLS(), false) }
|
||||
func testCtlV2Mk(t *testing.T, cfg *e2e.EtcdProcessClusterConfig, quorum bool) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
cfg.enableV2 = true
|
||||
cfg.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, cfg, quorum)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -78,12 +80,12 @@ func testCtlV2Mk(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCtlV2Rm(t *testing.T) { testCtlV2Rm(t, newConfigNoTLS()) }
|
||||
func TestCtlV2RmTLS(t *testing.T) { testCtlV2Rm(t, newConfigTLS()) }
|
||||
func testCtlV2Rm(t *testing.T, cfg *etcdProcessClusterConfig) {
|
||||
func TestCtlV2Rm(t *testing.T) { testCtlV2Rm(t, e2e.NewConfigNoTLS()) }
|
||||
func TestCtlV2RmTLS(t *testing.T) { testCtlV2Rm(t, e2e.NewConfigTLS()) }
|
||||
func testCtlV2Rm(t *testing.T, cfg *e2e.EtcdProcessClusterConfig) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
cfg.enableV2 = true
|
||||
cfg.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, cfg, true)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -101,13 +103,13 @@ func testCtlV2Rm(t *testing.T, cfg *etcdProcessClusterConfig) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCtlV2Ls(t *testing.T) { testCtlV2Ls(t, newConfigNoTLS(), false) }
|
||||
func TestCtlV2LsQuorum(t *testing.T) { testCtlV2Ls(t, newConfigNoTLS(), true) }
|
||||
func TestCtlV2LsTLS(t *testing.T) { testCtlV2Ls(t, newConfigTLS(), false) }
|
||||
func testCtlV2Ls(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) {
|
||||
func TestCtlV2Ls(t *testing.T) { testCtlV2Ls(t, e2e.NewConfigNoTLS(), false) }
|
||||
func TestCtlV2LsQuorum(t *testing.T) { testCtlV2Ls(t, e2e.NewConfigNoTLS(), true) }
|
||||
func TestCtlV2LsTLS(t *testing.T) { testCtlV2Ls(t, e2e.NewConfigTLS(), false) }
|
||||
func testCtlV2Ls(t *testing.T, cfg *e2e.EtcdProcessClusterConfig, quorum bool) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
cfg.enableV2 = true
|
||||
cfg.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, cfg, quorum)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -122,13 +124,13 @@ func testCtlV2Ls(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCtlV2Watch(t *testing.T) { testCtlV2Watch(t, newConfigNoTLS(), false) }
|
||||
func TestCtlV2WatchTLS(t *testing.T) { testCtlV2Watch(t, newConfigTLS(), false) }
|
||||
func TestCtlV2Watch(t *testing.T) { testCtlV2Watch(t, e2e.NewConfigNoTLS(), false) }
|
||||
func TestCtlV2WatchTLS(t *testing.T) { testCtlV2Watch(t, e2e.NewConfigTLS(), false) }
|
||||
|
||||
func testCtlV2Watch(t *testing.T, cfg *etcdProcessClusterConfig, noSync bool) {
|
||||
func testCtlV2Watch(t *testing.T, cfg *e2e.EtcdProcessClusterConfig, noSync bool) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
cfg.enableV2 = true
|
||||
cfg.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, cfg, true)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -151,8 +153,8 @@ func testCtlV2Watch(t *testing.T, cfg *etcdProcessClusterConfig, noSync bool) {
|
||||
func TestCtlV2GetRoleUser(t *testing.T) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
copied := newConfigNoTLS()
|
||||
copied.enableV2 = true
|
||||
copied := e2e.NewConfigNoTLS()
|
||||
copied.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, copied, false)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -172,7 +174,7 @@ func TestCtlV2GetRoleUser(t *testing.T) {
|
||||
// ensure double grant gives an error; was crashing in 2.3.1
|
||||
regrantArgs := etcdctlPrefixArgs(epc)
|
||||
regrantArgs = append(regrantArgs, "user", "grant", "--roles", "foo", "username")
|
||||
if err := spawnWithExpect(regrantArgs, "duplicate"); err != nil {
|
||||
if err := e2e.SpawnWithExpect(regrantArgs, "duplicate"); err != nil {
|
||||
t.Fatalf("missing duplicate error on double grant role (%v)", err)
|
||||
}
|
||||
}
|
||||
@ -182,8 +184,8 @@ func TestCtlV2UserListRoot(t *testing.T) { testCtlV2UserList(t, "root") }
|
||||
func testCtlV2UserList(t *testing.T, username string) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
copied := newConfigNoTLS()
|
||||
copied.enableV2 = true
|
||||
copied := e2e.NewConfigNoTLS()
|
||||
copied.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, copied, false)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -198,8 +200,8 @@ func testCtlV2UserList(t *testing.T, username string) {
|
||||
func TestCtlV2RoleList(t *testing.T) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
copied := newConfigNoTLS()
|
||||
copied.enableV2 = true
|
||||
copied := e2e.NewConfigNoTLS()
|
||||
copied.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, copied, false)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -232,9 +234,9 @@ func testUtlCtlV2Backup(t *testing.T, snapCount int, v3 bool, utl bool) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
etcdCfg := newConfigNoTLS()
|
||||
etcdCfg.snapshotCount = snapCount
|
||||
etcdCfg.enableV2 = true
|
||||
etcdCfg := e2e.NewConfigNoTLS()
|
||||
etcdCfg.SnapshotCount = snapCount
|
||||
etcdCfg.EnableV2 = true
|
||||
t.Log("Starting etcd-1")
|
||||
epc1 := setupEtcdctlTest(t, etcdCfg, false)
|
||||
|
||||
@ -259,7 +261,7 @@ func testUtlCtlV2Backup(t *testing.T, snapCount int, v3 bool, utl bool) {
|
||||
}
|
||||
}
|
||||
t.Log("Triggering etcd backup")
|
||||
if err := etcdctlBackup(t, epc1, epc1.procs[0].Config().dataDirPath, backupDir, v3, utl); err != nil {
|
||||
if err := etcdctlBackup(t, epc1, epc1.Procs[0].Config().DataDirPath, backupDir, v3, utl); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
t.Log("Closing etcd-1 backup")
|
||||
@ -271,11 +273,11 @@ func testUtlCtlV2Backup(t *testing.T, snapCount int, v3 bool, utl bool) {
|
||||
|
||||
t.Log("Starting etcd-2 (post backup)")
|
||||
// restart from the backup directory
|
||||
cfg2 := newConfigNoTLS()
|
||||
cfg2.dataDirPath = backupDir
|
||||
cfg2.keepDataDir = true
|
||||
cfg2.forceNewCluster = true
|
||||
cfg2.enableV2 = true
|
||||
cfg2 := e2e.NewConfigNoTLS()
|
||||
cfg2.DataDirPath = backupDir
|
||||
cfg2.KeepDataDir = true
|
||||
cfg2.ForceNewCluster = true
|
||||
cfg2.EnableV2 = true
|
||||
epc2 := setupEtcdctlTest(t, cfg2, false)
|
||||
// Make sure a failing test is not leaking resources (running server).
|
||||
defer epc2.Close()
|
||||
@ -318,9 +320,9 @@ func testUtlCtlV2Backup(t *testing.T, snapCount int, v3 bool, utl bool) {
|
||||
func TestCtlV2AuthWithCommonName(t *testing.T) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
copiedCfg := newConfigClientTLS()
|
||||
copiedCfg.clientCertAuthEnabled = true
|
||||
copiedCfg.enableV2 = true
|
||||
copiedCfg := e2e.NewConfigClientTLS()
|
||||
copiedCfg.ClientCertAuthEnabled = true
|
||||
copiedCfg.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, copiedCfg, false)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -350,8 +352,8 @@ func TestCtlV2AuthWithCommonName(t *testing.T) {
|
||||
func TestCtlV2ClusterHealth(t *testing.T) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
copied := newConfigNoTLS()
|
||||
copied.enableV2 = true
|
||||
copied := e2e.NewConfigNoTLS()
|
||||
copied.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, copied, true)
|
||||
defer cleanupEtcdProcessCluster(epc, t)
|
||||
|
||||
@ -361,7 +363,7 @@ func TestCtlV2ClusterHealth(t *testing.T) {
|
||||
}
|
||||
|
||||
// missing members, has quorum
|
||||
epc.procs[0].Stop()
|
||||
epc.Procs[0].Stop()
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
err := etcdctlClusterHealth(epc, "cluster is degraded")
|
||||
@ -375,129 +377,129 @@ func TestCtlV2ClusterHealth(t *testing.T) {
|
||||
}
|
||||
|
||||
// no quorum
|
||||
epc.procs[1].Stop()
|
||||
epc.Procs[1].Stop()
|
||||
if err := etcdctlClusterHealth(epc, "cluster is unavailable"); err != nil {
|
||||
t.Fatalf("cluster-health expected to be unavailable (%v)", err)
|
||||
}
|
||||
|
||||
epc.procs[0], epc.procs[1] = nil, nil
|
||||
epc.Procs[0], epc.Procs[1] = nil, nil
|
||||
}
|
||||
|
||||
func etcdctlPrefixArgs(clus *etcdProcessCluster) []string {
|
||||
func etcdctlPrefixArgs(clus *e2e.EtcdProcessCluster) []string {
|
||||
endpoints := strings.Join(clus.EndpointsV2(), ",")
|
||||
cmdArgs := []string{ctlBinPath}
|
||||
cmdArgs := []string{e2e.CtlBinPath}
|
||||
|
||||
cmdArgs = append(cmdArgs, "--endpoints", endpoints)
|
||||
if clus.cfg.clientTLS == clientTLS {
|
||||
cmdArgs = append(cmdArgs, "--ca-file", caPath, "--cert-file", certPath, "--key-file", privateKeyPath)
|
||||
if clus.Cfg.ClientTLS == e2e.ClientTLS {
|
||||
cmdArgs = append(cmdArgs, "--ca-file", e2e.CaPath, "--cert-file", e2e.CertPath, "--key-file", e2e.PrivateKeyPath)
|
||||
}
|
||||
return cmdArgs
|
||||
}
|
||||
|
||||
func etcductlPrefixArgs(utl bool) []string {
|
||||
if utl {
|
||||
return []string{utlBinPath}
|
||||
return []string{e2e.UtlBinPath}
|
||||
}
|
||||
return []string{ctlBinPath}
|
||||
return []string{e2e.CtlBinPath}
|
||||
}
|
||||
|
||||
func etcdctlClusterHealth(clus *etcdProcessCluster, val string) error {
|
||||
func etcdctlClusterHealth(clus *e2e.EtcdProcessCluster, val string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "cluster-health")
|
||||
return spawnWithExpect(cmdArgs, val)
|
||||
return e2e.SpawnWithExpect(cmdArgs, val)
|
||||
}
|
||||
|
||||
func etcdctlSet(clus *etcdProcessCluster, key, value string) error {
|
||||
func etcdctlSet(clus *e2e.EtcdProcessCluster, key, value string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "set", key, value)
|
||||
return spawnWithExpect(cmdArgs, value)
|
||||
return e2e.SpawnWithExpect(cmdArgs, value)
|
||||
}
|
||||
|
||||
func etcdctlMk(clus *etcdProcessCluster, key, value string, first bool) error {
|
||||
func etcdctlMk(clus *e2e.EtcdProcessCluster, key, value string, first bool) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "mk", key, value)
|
||||
if first {
|
||||
return spawnWithExpect(cmdArgs, value)
|
||||
return e2e.SpawnWithExpect(cmdArgs, value)
|
||||
}
|
||||
return spawnWithExpect(cmdArgs, "Error: 105: Key already exists")
|
||||
return e2e.SpawnWithExpect(cmdArgs, "Error: 105: Key already exists")
|
||||
}
|
||||
|
||||
func etcdctlGet(clus *etcdProcessCluster, key, value string, quorum bool) error {
|
||||
func etcdctlGet(clus *e2e.EtcdProcessCluster, key, value string, quorum bool) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "get", key)
|
||||
if quorum {
|
||||
cmdArgs = append(cmdArgs, "--quorum")
|
||||
}
|
||||
return spawnWithExpect(cmdArgs, value)
|
||||
return e2e.SpawnWithExpect(cmdArgs, value)
|
||||
}
|
||||
|
||||
func etcdctlRm(clus *etcdProcessCluster, key, value string, first bool) error {
|
||||
func etcdctlRm(clus *e2e.EtcdProcessCluster, key, value string, first bool) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "rm", key)
|
||||
if first {
|
||||
return spawnWithExpect(cmdArgs, "PrevNode.Value: "+value)
|
||||
return e2e.SpawnWithExpect(cmdArgs, "PrevNode.Value: "+value)
|
||||
}
|
||||
return spawnWithExpect(cmdArgs, "Error: 100: Key not found")
|
||||
return e2e.SpawnWithExpect(cmdArgs, "Error: 100: Key not found")
|
||||
}
|
||||
|
||||
func etcdctlLs(clus *etcdProcessCluster, key string, quorum bool) error {
|
||||
func etcdctlLs(clus *e2e.EtcdProcessCluster, key string, quorum bool) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "ls")
|
||||
if quorum {
|
||||
cmdArgs = append(cmdArgs, "--quorum")
|
||||
}
|
||||
return spawnWithExpect(cmdArgs, key)
|
||||
return e2e.SpawnWithExpect(cmdArgs, key)
|
||||
}
|
||||
|
||||
func etcdctlWatch(clus *etcdProcessCluster, key, value string, noSync bool) <-chan error {
|
||||
func etcdctlWatch(clus *e2e.EtcdProcessCluster, key, value string, noSync bool) <-chan error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "watch", "--after-index=1", key)
|
||||
if noSync {
|
||||
cmdArgs = append(cmdArgs, "--no-sync")
|
||||
}
|
||||
errc := make(chan error, 1)
|
||||
go func() {
|
||||
errc <- spawnWithExpect(cmdArgs, value)
|
||||
errc <- e2e.SpawnWithExpect(cmdArgs, value)
|
||||
}()
|
||||
return errc
|
||||
}
|
||||
|
||||
func etcdctlRoleAdd(clus *etcdProcessCluster, role string) error {
|
||||
func etcdctlRoleAdd(clus *e2e.EtcdProcessCluster, role string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "role", "add", role)
|
||||
return spawnWithExpect(cmdArgs, role)
|
||||
return e2e.SpawnWithExpect(cmdArgs, role)
|
||||
}
|
||||
|
||||
func etcdctlRoleGrant(clus *etcdProcessCluster, role string, perms ...string) error {
|
||||
func etcdctlRoleGrant(clus *e2e.EtcdProcessCluster, role string, perms ...string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "role", "grant")
|
||||
cmdArgs = append(cmdArgs, perms...)
|
||||
cmdArgs = append(cmdArgs, role)
|
||||
return spawnWithExpect(cmdArgs, role)
|
||||
return e2e.SpawnWithExpect(cmdArgs, role)
|
||||
}
|
||||
|
||||
func etcdctlRoleList(clus *etcdProcessCluster, expectedRole string) error {
|
||||
func etcdctlRoleList(clus *e2e.EtcdProcessCluster, expectedRole string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "role", "list")
|
||||
return spawnWithExpect(cmdArgs, expectedRole)
|
||||
return e2e.SpawnWithExpect(cmdArgs, expectedRole)
|
||||
}
|
||||
|
||||
func etcdctlUserAdd(clus *etcdProcessCluster, user, pass string) error {
|
||||
func etcdctlUserAdd(clus *e2e.EtcdProcessCluster, user, pass string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "user", "add", user+":"+pass)
|
||||
return spawnWithExpect(cmdArgs, "User "+user+" created")
|
||||
return e2e.SpawnWithExpect(cmdArgs, "User "+user+" created")
|
||||
}
|
||||
|
||||
func etcdctlUserGrant(clus *etcdProcessCluster, user, role string) error {
|
||||
func etcdctlUserGrant(clus *e2e.EtcdProcessCluster, user, role string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "user", "grant", "--roles", role, user)
|
||||
return spawnWithExpect(cmdArgs, "User "+user+" updated")
|
||||
return e2e.SpawnWithExpect(cmdArgs, "User "+user+" updated")
|
||||
}
|
||||
|
||||
func etcdctlUserGet(clus *etcdProcessCluster, user string) error {
|
||||
func etcdctlUserGet(clus *e2e.EtcdProcessCluster, user string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "user", "get", user)
|
||||
return spawnWithExpect(cmdArgs, "User: "+user)
|
||||
return e2e.SpawnWithExpect(cmdArgs, "User: "+user)
|
||||
}
|
||||
|
||||
func etcdctlUserList(clus *etcdProcessCluster, expectedUser string) error {
|
||||
func etcdctlUserList(clus *e2e.EtcdProcessCluster, expectedUser string) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "user", "list")
|
||||
return spawnWithExpect(cmdArgs, expectedUser)
|
||||
return e2e.SpawnWithExpect(cmdArgs, expectedUser)
|
||||
}
|
||||
|
||||
func etcdctlAuthEnable(clus *etcdProcessCluster) error {
|
||||
func etcdctlAuthEnable(clus *e2e.EtcdProcessCluster) error {
|
||||
cmdArgs := append(etcdctlPrefixArgs(clus), "auth", "enable")
|
||||
return spawnWithExpect(cmdArgs, "Authentication Enabled")
|
||||
return e2e.SpawnWithExpect(cmdArgs, "Authentication Enabled")
|
||||
}
|
||||
|
||||
func etcdctlBackup(t testing.TB, clus *etcdProcessCluster, dataDir, backupDir string, v3 bool, utl bool) error {
|
||||
func etcdctlBackup(t testing.TB, clus *e2e.EtcdProcessCluster, dataDir, backupDir string, v3 bool, utl bool) error {
|
||||
cmdArgs := append(etcductlPrefixArgs(utl), "backup", "--data-dir", dataDir, "--backup-dir", backupDir)
|
||||
if v3 {
|
||||
cmdArgs = append(cmdArgs, "--with-v3")
|
||||
@ -505,7 +507,7 @@ func etcdctlBackup(t testing.TB, clus *etcdProcessCluster, dataDir, backupDir st
|
||||
cmdArgs = append(cmdArgs, "--with-v3=false")
|
||||
}
|
||||
t.Logf("Running: %v", cmdArgs)
|
||||
proc, err := spawnCmd(cmdArgs, nil)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -516,18 +518,18 @@ func etcdctlBackup(t testing.TB, clus *etcdProcessCluster, dataDir, backupDir st
|
||||
return proc.ProcessError()
|
||||
}
|
||||
|
||||
func setupEtcdctlTest(t *testing.T, cfg *etcdProcessClusterConfig, quorum bool) *etcdProcessCluster {
|
||||
func setupEtcdctlTest(t *testing.T, cfg *e2e.EtcdProcessClusterConfig, quorum bool) *e2e.EtcdProcessCluster {
|
||||
if !quorum {
|
||||
cfg = configStandalone(*cfg)
|
||||
cfg = e2e.ConfigStandalone(*cfg)
|
||||
}
|
||||
epc, err := newEtcdProcessCluster(t, cfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
return epc
|
||||
}
|
||||
|
||||
func cleanupEtcdProcessCluster(epc *etcdProcessCluster, t *testing.T) {
|
||||
func cleanupEtcdProcessCluster(epc *e2e.EtcdProcessCluster, t *testing.T) {
|
||||
if errC := epc.Close(); errC != nil {
|
||||
t.Fatalf("error closing etcd processes (%v)", errC)
|
||||
}
|
||||
|
@ -22,6 +22,7 @@ import (
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Alarm(t *testing.T) {
|
||||
@ -53,7 +54,7 @@ func alarmTest(cx ctlCtx) {
|
||||
}
|
||||
|
||||
// '/health' handler should return 'false'
|
||||
if err := cURLGet(cx.epc, cURLReq{endpoint: "/health", expected: `{"health":"false","reason":"ALARM NOSPACE"}`}); err != nil {
|
||||
if err := e2e.CURLGet(cx.epc, e2e.CURLReq{Endpoint: "/health", Expected: `{"health":"false","reason":"ALARM NOSPACE"}`}); err != nil {
|
||||
cx.t.Fatalf("failed get with curl (%v)", err)
|
||||
}
|
||||
|
||||
@ -101,5 +102,5 @@ func alarmTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3Alarm(cx ctlCtx, cmd string, as ...string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "alarm", cmd)
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, as...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, as...)
|
||||
}
|
||||
|
@ -24,20 +24,22 @@ import (
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3AuthCertCN(t *testing.T) {
|
||||
testCtl(t, authTestCertCN, withCfg(*newConfigClientTLSCertAuth()))
|
||||
testCtl(t, authTestCertCN, withCfg(*e2e.NewConfigClientTLSCertAuth()))
|
||||
}
|
||||
func TestCtlV3AuthCertCNAndUsername(t *testing.T) {
|
||||
testCtl(t, authTestCertCNAndUsername, withCfg(*newConfigClientTLSCertAuth()))
|
||||
testCtl(t, authTestCertCNAndUsername, withCfg(*e2e.NewConfigClientTLSCertAuth()))
|
||||
}
|
||||
func TestCtlV3AuthCertCNAndUsernameNoPassword(t *testing.T) {
|
||||
testCtl(t, authTestCertCNAndUsernameNoPassword, withCfg(*newConfigClientTLSCertAuth()))
|
||||
testCtl(t, authTestCertCNAndUsernameNoPassword, withCfg(*e2e.NewConfigClientTLSCertAuth()))
|
||||
}
|
||||
|
||||
func TestCtlV3AuthCertCNWithWithConcurrentOperation(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
// apply the certificate which has `root` CommonName,
|
||||
// and reset the setting when the test case finishes.
|
||||
@ -49,19 +51,19 @@ func TestCtlV3AuthCertCNWithWithConcurrentOperation(t *testing.T) {
|
||||
|
||||
t.Log("Create an etcd cluster")
|
||||
cx := getDefaultCtlCtx(t)
|
||||
cx.cfg = etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
clientTLS: clientTLS,
|
||||
clientCertAuthEnabled: true,
|
||||
initialToken: "new",
|
||||
cx.cfg = e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
ClientTLS: e2e.ClientTLS,
|
||||
ClientCertAuthEnabled: true,
|
||||
InitialToken: "new",
|
||||
}
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, &cx.cfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &cx.cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start etcd cluster: %v", err)
|
||||
}
|
||||
cx.epc = epc
|
||||
cx.dataDir = epc.procs[0].Config().dataDirPath
|
||||
cx.dataDir = epc.Procs[0].Config().DataDirPath
|
||||
|
||||
defer func() {
|
||||
if err := epc.Close(); err != nil {
|
||||
|
@ -21,11 +21,12 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
// TestAuth_CVE_2021_28235 verifies https://nvd.nist.gov/vuln/detail/CVE-2021-28235
|
||||
func TestAuth_CVE_2021_28235(t *testing.T) {
|
||||
testCtl(t, authTest_CVE_2021_28235, withCfg(*newConfigNoTLS()), withLogLevel("debug"))
|
||||
testCtl(t, authTest_CVE_2021_28235, withCfg(*e2e.NewConfigNoTLS()), withLogLevel("debug"))
|
||||
}
|
||||
|
||||
func authTest_CVE_2021_28235(cx ctlCtx) {
|
||||
@ -44,9 +45,9 @@ func authTest_CVE_2021_28235(cx ctlCtx) {
|
||||
require.NoError(cx.t, err)
|
||||
|
||||
// GET /debug/requests
|
||||
httpEndpoint := cx.epc.procs[0].EndpointsHTTP()[0]
|
||||
req := cURLReq{endpoint: "/debug/requests?fam=grpc.Recv.etcdserverpb.Auth&b=0&exp=1", timeout: 5}
|
||||
respData, err := curl(httpEndpoint, "GET", req, clientNonTLS)
|
||||
httpEndpoint := cx.epc.Procs[0].EndpointsHTTP()[0]
|
||||
req := e2e.CURLReq{Endpoint: "/debug/requests?fam=grpc.Recv.etcdserverpb.Auth&b=0&exp=1", Timeout: 5}
|
||||
respData, err := curl(httpEndpoint, "GET", req, e2e.ClientNonTLS)
|
||||
require.NoError(cx.t, err)
|
||||
|
||||
if strings.Contains(respData, rootPass) {
|
||||
|
@ -26,8 +26,8 @@ import (
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3AuthEnable(t *testing.T) {
|
||||
@ -41,7 +41,7 @@ func TestCtlV3AuthRoleUpdate(t *testing.T) { testCtl(t, authRoleUpdateT
|
||||
func TestCtlV3AuthUserDeleteDuringOps(t *testing.T) { testCtl(t, authUserDeleteDuringOpsTest) }
|
||||
func TestCtlV3AuthRoleRevokeDuringOps(t *testing.T) { testCtl(t, authRoleRevokeDuringOpsTest) }
|
||||
func TestCtlV3AuthTxn(t *testing.T) { testCtl(t, authTestTxn) }
|
||||
func TestCtlV3AuthTxnJWT(t *testing.T) { testCtl(t, authTestTxn, withCfg(*newConfigJWT())) }
|
||||
func TestCtlV3AuthTxnJWT(t *testing.T) { testCtl(t, authTestTxn, withCfg(*e2e.NewConfigJWT())) }
|
||||
func TestCtlV3AuthPrefixPerm(t *testing.T) { testCtl(t, authTestPrefixPerm) }
|
||||
func TestCtlV3AuthMemberAdd(t *testing.T) { testCtl(t, authTestMemberAdd) }
|
||||
func TestCtlV3AuthMemberRemove(t *testing.T) {
|
||||
@ -52,7 +52,7 @@ func TestCtlV3AuthRevokeWithDelete(t *testing.T) { testCtl(t, authTestRevokeWith
|
||||
func TestCtlV3AuthInvalidMgmt(t *testing.T) { testCtl(t, authTestInvalidMgmt) }
|
||||
func TestCtlV3AuthFromKeyPerm(t *testing.T) { testCtl(t, authTestFromKeyPerm) }
|
||||
func TestCtlV3AuthAndWatch(t *testing.T) { testCtl(t, authTestWatch) }
|
||||
func TestCtlV3AuthAndWatchJWT(t *testing.T) { testCtl(t, authTestWatch, withCfg(*newConfigJWT())) }
|
||||
func TestCtlV3AuthAndWatchJWT(t *testing.T) { testCtl(t, authTestWatch, withCfg(*e2e.NewConfigJWT())) }
|
||||
|
||||
func TestCtlV3AuthLeaseTestKeepAlive(t *testing.T) { testCtl(t, authLeaseTestKeepAlive) }
|
||||
func TestCtlV3AuthLeaseTestTimeToLiveExpired(t *testing.T) {
|
||||
@ -60,7 +60,7 @@ func TestCtlV3AuthLeaseTestTimeToLiveExpired(t *testing.T) {
|
||||
}
|
||||
func TestCtlV3AuthLeaseGrantLeases(t *testing.T) { testCtl(t, authLeaseTestLeaseGrantLeases) }
|
||||
func TestCtlV3AuthLeaseGrantLeasesJWT(t *testing.T) {
|
||||
testCtl(t, authLeaseTestLeaseGrantLeases, withCfg(*newConfigJWT()))
|
||||
testCtl(t, authLeaseTestLeaseGrantLeases, withCfg(*e2e.NewConfigJWT()))
|
||||
}
|
||||
func TestCtlV3AuthLeaseRevoke(t *testing.T) { testCtl(t, authLeaseTestLeaseRevoke) }
|
||||
|
||||
@ -72,15 +72,19 @@ func TestCtlV3AuthDefrag(t *testing.T) { testCtl(t, authTestDefrag) }
|
||||
func TestCtlV3AuthEndpointHealth(t *testing.T) {
|
||||
testCtl(t, authTestEndpointHealth, withQuorum())
|
||||
}
|
||||
func TestCtlV3AuthSnapshot(t *testing.T) { testCtl(t, authTestSnapshot) }
|
||||
func TestCtlV3AuthSnapshotJWT(t *testing.T) { testCtl(t, authTestSnapshot, withCfg(*newConfigJWT())) }
|
||||
func TestCtlV3AuthJWTExpire(t *testing.T) { testCtl(t, authTestJWTExpire, withCfg(*newConfigJWT())) }
|
||||
func TestCtlV3AuthSnapshot(t *testing.T) { testCtl(t, authTestSnapshot) }
|
||||
func TestCtlV3AuthSnapshotJWT(t *testing.T) {
|
||||
testCtl(t, authTestSnapshot, withCfg(*e2e.NewConfigJWT()))
|
||||
}
|
||||
func TestCtlV3AuthJWTExpire(t *testing.T) {
|
||||
testCtl(t, authTestJWTExpire, withCfg(*e2e.NewConfigJWT()))
|
||||
}
|
||||
func TestCtlV3AuthRevisionConsistency(t *testing.T) { testCtl(t, authTestRevisionConsistency) }
|
||||
func TestCtlV3AuthTestCacheReload(t *testing.T) { testCtl(t, authTestCacheReload) }
|
||||
func TestCtlV3AuthLeaseTimeToLive(t *testing.T) { testCtl(t, authTestLeaseTimeToLive) }
|
||||
|
||||
func TestCtlV3AuthRecoverFromSnapshot(t *testing.T) {
|
||||
testCtl(t, authTestRecoverSnapshot, withCfg(*newConfigNoTLS()), withQuorum(), withSnapshotCount(5))
|
||||
testCtl(t, authTestRecoverSnapshot, withCfg(*e2e.NewConfigNoTLS()), withQuorum(), withSnapshotCount(5))
|
||||
}
|
||||
|
||||
func authEnableTest(cx ctlCtx) {
|
||||
@ -105,29 +109,29 @@ func authEnable(cx ctlCtx) error {
|
||||
|
||||
func applyTLSWithRootCommonName() func() {
|
||||
var (
|
||||
oldCertPath = certPath
|
||||
oldPrivateKeyPath = privateKeyPath
|
||||
oldCaPath = caPath
|
||||
oldCertPath = e2e.CertPath
|
||||
oldPrivateKeyPath = e2e.PrivateKeyPath
|
||||
oldCaPath = e2e.CaPath
|
||||
|
||||
newCertPath = filepath.Join(fixturesDir, "CommonName-root.crt")
|
||||
newPrivateKeyPath = filepath.Join(fixturesDir, "CommonName-root.key")
|
||||
newCaPath = filepath.Join(fixturesDir, "CommonName-root.crt")
|
||||
newCertPath = filepath.Join(e2e.FixturesDir, "CommonName-root.crt")
|
||||
newPrivateKeyPath = filepath.Join(e2e.FixturesDir, "CommonName-root.key")
|
||||
newCaPath = filepath.Join(e2e.FixturesDir, "CommonName-root.crt")
|
||||
)
|
||||
|
||||
certPath = newCertPath
|
||||
privateKeyPath = newPrivateKeyPath
|
||||
caPath = newCaPath
|
||||
e2e.CertPath = newCertPath
|
||||
e2e.PrivateKeyPath = newPrivateKeyPath
|
||||
e2e.CaPath = newCaPath
|
||||
|
||||
return func() {
|
||||
certPath = oldCertPath
|
||||
privateKeyPath = oldPrivateKeyPath
|
||||
caPath = oldCaPath
|
||||
e2e.CertPath = oldCertPath
|
||||
e2e.PrivateKeyPath = oldPrivateKeyPath
|
||||
e2e.CaPath = oldCaPath
|
||||
}
|
||||
}
|
||||
|
||||
func ctlV3AuthEnable(cx ctlCtx) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "auth", "enable")
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, "Authentication Enabled")
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "Authentication Enabled")
|
||||
}
|
||||
|
||||
func authDisableTest(cx ctlCtx) {
|
||||
@ -192,7 +196,7 @@ func authGracefulDisableTest(cx ctlCtx) {
|
||||
}
|
||||
|
||||
// ...and restart the node
|
||||
node0 := cx.epc.procs[0]
|
||||
node0 := cx.epc.Procs[0]
|
||||
node0.WithStopSignal(syscall.SIGINT)
|
||||
if rerr := node0.Restart(); rerr != nil {
|
||||
cx.t.Fatal(rerr)
|
||||
@ -217,12 +221,12 @@ func authGracefulDisableTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3AuthDisable(cx ctlCtx) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "auth", "disable")
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, "Authentication Disabled")
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "Authentication Disabled")
|
||||
}
|
||||
|
||||
func authStatusTest(cx ctlCtx) {
|
||||
cmdArgs := append(cx.PrefixArgs(), "auth", "status")
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, "Authentication Status: false", "AuthRevision:"); err != nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "Authentication Status: false", "AuthRevision:"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -233,15 +237,15 @@ func authStatusTest(cx ctlCtx) {
|
||||
cx.user, cx.pass = "root", "root"
|
||||
cmdArgs = append(cx.PrefixArgs(), "auth", "status")
|
||||
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, "Authentication Status: true", "AuthRevision:"); err != nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "Authentication Status: true", "AuthRevision:"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
cmdArgs = append(cx.PrefixArgs(), "auth", "status", "--write-out", "json")
|
||||
if err := spawnWithExpectWithEnv(cmdArgs, cx.envMap, "enabled"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "enabled"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := spawnWithExpectWithEnv(cmdArgs, cx.envMap, "authRevision"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "authRevision"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -459,25 +463,25 @@ func authRoleRevokeDuringOpsTest(cx ctlCtx) {
|
||||
}
|
||||
|
||||
func ctlV3PutFailAuth(cx ctlCtx, key, val string) error {
|
||||
return spawnWithExpectWithEnv(append(cx.PrefixArgs(), "put", key, val), cx.envMap, "authentication failed")
|
||||
return e2e.SpawnWithExpectWithEnv(append(cx.PrefixArgs(), "put", key, val), cx.envMap, "authentication failed")
|
||||
}
|
||||
|
||||
func ctlV3PutFailPerm(cx ctlCtx, key, val string) error {
|
||||
return spawnWithExpectWithEnv(append(cx.PrefixArgs(), "put", key, val), cx.envMap, "permission denied")
|
||||
return e2e.SpawnWithExpectWithEnv(append(cx.PrefixArgs(), "put", key, val), cx.envMap, "permission denied")
|
||||
}
|
||||
|
||||
func authSetupTestUser(cx ctlCtx) {
|
||||
if err := ctlV3User(cx, []string{"add", "test-user", "--interactive=false"}, "User test-user created", []string{"pass"}); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := spawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "add", "test-role"), cx.envMap, "Role test-role created"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "add", "test-role"), cx.envMap, "Role test-role created"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := ctlV3User(cx, []string{"grant-role", "test-user", "test-role"}, "Role test-role is granted to user test-user", nil); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
cmd := append(cx.PrefixArgs(), "role", "grant-permission", "test-role", "readwrite", "foo")
|
||||
if err := spawnWithExpectWithEnv(cmd, cx.envMap, "Role test-role updated"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(cmd, cx.envMap, "Role test-role updated"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -615,7 +619,7 @@ func authTestMemberAdd(cx ctlCtx) {
|
||||
cx.user, cx.pass = "root", "root"
|
||||
authSetupTestUser(cx)
|
||||
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", etcdProcessBasePort+11)
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", e2e.EtcdProcessBasePort+11)
|
||||
// ordinary user cannot add a new member
|
||||
cx.user, cx.pass = "test-user", "pass"
|
||||
if err := ctlV3MemberAdd(cx, peerURL, false); err == nil {
|
||||
@ -667,7 +671,7 @@ func authTestMemberUpdate(cx ctlCtx) {
|
||||
|
||||
// ordinary user cannot update a member
|
||||
cx.user, cx.pass = "test-user", "pass"
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", etcdProcessBasePort+11)
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", e2e.EtcdProcessBasePort+11)
|
||||
memberID := fmt.Sprintf("%x", mr.Members[0].ID)
|
||||
if err = ctlV3MemberUpdate(cx, memberID, peerURL); err == nil {
|
||||
cx.t.Fatalf("ordinary user must not be allowed to update a member")
|
||||
@ -689,7 +693,7 @@ func authTestCertCN(cx ctlCtx) {
|
||||
if err := ctlV3User(cx, []string{"add", "example.com", "--interactive=false"}, "User example.com created", []string{""}); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := spawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "add", "test-role"), cx.envMap, "Role test-role created"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "add", "test-role"), cx.envMap, "Role test-role created"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := ctlV3User(cx, []string{"grant-role", "example.com", "test-role"}, "Role test-role is granted to user example.com", nil); err != nil {
|
||||
@ -999,13 +1003,13 @@ func authTestRoleGet(cx ctlCtx) {
|
||||
"KV Read:", "foo",
|
||||
"KV Write:", "foo",
|
||||
}
|
||||
if err := spawnWithExpects(append(cx.PrefixArgs(), "role", "get", "test-role"), cx.envMap, expected...); err != nil {
|
||||
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "role", "get", "test-role"), cx.envMap, expected...); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
// test-user can get the information of test-role because it belongs to the role
|
||||
cx.user, cx.pass = "test-user", "pass"
|
||||
if err := spawnWithExpects(append(cx.PrefixArgs(), "role", "get", "test-role"), cx.envMap, expected...); err != nil {
|
||||
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "role", "get", "test-role"), cx.envMap, expected...); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -1013,7 +1017,7 @@ func authTestRoleGet(cx ctlCtx) {
|
||||
expected = []string{
|
||||
"Error: etcdserver: permission denied",
|
||||
}
|
||||
if err := spawnWithExpects(append(cx.PrefixArgs(), "role", "get", "root"), cx.envMap, expected...); err != nil {
|
||||
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "role", "get", "root"), cx.envMap, expected...); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -1030,13 +1034,13 @@ func authTestUserGet(cx ctlCtx) {
|
||||
"Roles: test-role",
|
||||
}
|
||||
|
||||
if err := spawnWithExpects(append(cx.PrefixArgs(), "user", "get", "test-user"), cx.envMap, expected...); err != nil {
|
||||
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "user", "get", "test-user"), cx.envMap, expected...); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
// test-user can get the information of test-user itself
|
||||
cx.user, cx.pass = "test-user", "pass"
|
||||
if err := spawnWithExpects(append(cx.PrefixArgs(), "user", "get", "test-user"), cx.envMap, expected...); err != nil {
|
||||
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "user", "get", "test-user"), cx.envMap, expected...); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -1044,7 +1048,7 @@ func authTestUserGet(cx ctlCtx) {
|
||||
expected = []string{
|
||||
"Error: etcdserver: permission denied",
|
||||
}
|
||||
if err := spawnWithExpects(append(cx.PrefixArgs(), "user", "get", "root"), cx.envMap, expected...); err != nil {
|
||||
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "user", "get", "root"), cx.envMap, expected...); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -1055,7 +1059,7 @@ func authTestRoleList(cx ctlCtx) {
|
||||
}
|
||||
cx.user, cx.pass = "root", "root"
|
||||
authSetupTestUser(cx)
|
||||
if err := spawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "list"), cx.envMap, "test-role"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "list"), cx.envMap, "test-role"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -1166,7 +1170,7 @@ func certCNAndUsername(cx ctlCtx, noPassword bool) {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
if err := spawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "add", "test-role-cn"), cx.envMap, "Role test-role-cn created"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(append(cx.PrefixArgs(), "role", "add", "test-role-cn"), cx.envMap, "Role test-role-cn created"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := ctlV3User(cx, []string{"grant-role", "example.com", "test-role-cn"}, "Role test-role-cn is granted to user example.com", nil); err != nil {
|
||||
@ -1252,7 +1256,7 @@ func authTestRevisionConsistency(cx ctlCtx) {
|
||||
}
|
||||
|
||||
// get node0 auth revision
|
||||
node0 := cx.epc.procs[0]
|
||||
node0 := cx.epc.Procs[0]
|
||||
endpoint := node0.EndpointsV3()[0]
|
||||
cli, err := clientv3.New(clientv3.Config{Endpoints: []string{endpoint}, Username: cx.user, Password: cx.pass, DialTimeout: 3 * time.Second})
|
||||
if err != nil {
|
||||
@ -1305,7 +1309,7 @@ func authTestCacheReload(cx ctlCtx) {
|
||||
},
|
||||
}
|
||||
|
||||
node0 := cx.epc.procs[0]
|
||||
node0 := cx.epc.Procs[0]
|
||||
endpoint := node0.EndpointsV3()[0]
|
||||
|
||||
// create a client
|
||||
@ -1415,11 +1419,11 @@ func authTestRecoverSnapshot(cx ctlCtx) {
|
||||
var (
|
||||
idx = 3
|
||||
name = fmt.Sprintf("test-%d", idx)
|
||||
port = cx.cfg.basePort + 5*idx
|
||||
port = cx.cfg.BasePort + 5*idx
|
||||
curlHost = fmt.Sprintf("localhost:%d", port)
|
||||
nodeClientURL = url.URL{Scheme: cx.cfg.clientScheme(), Host: curlHost}
|
||||
nodePeerURL = url.URL{Scheme: cx.cfg.peerScheme(), Host: fmt.Sprintf("localhost:%d", port+1)}
|
||||
initialCluster = cx.epc.procs[0].Config().initialCluster + "," + fmt.Sprintf("%s=%s", name, nodePeerURL.String())
|
||||
nodeClientURL = url.URL{Scheme: cx.cfg.ClientScheme(), Host: curlHost}
|
||||
nodePeerURL = url.URL{Scheme: cx.cfg.PeerScheme(), Host: fmt.Sprintf("localhost:%d", port+1)}
|
||||
initialCluster = cx.epc.Procs[0].Config().InitialCluster + "," + fmt.Sprintf("%s=%s", name, nodePeerURL.String())
|
||||
)
|
||||
cx.t.Logf("Adding a new member: %s", nodePeerURL.String())
|
||||
// Must wait at least 5 seconds, otherwise it will always get an
|
||||
@ -1453,8 +1457,8 @@ func authTestRecoverSnapshot(cx ctlCtx) {
|
||||
|
||||
//verify all nodes have the same revision and hash
|
||||
var endpoints []string
|
||||
for _, proc := range cx.epc.procs {
|
||||
endpoints = append(endpoints, proc.Config().acurl)
|
||||
for _, proc := range cx.epc.Procs {
|
||||
endpoints = append(endpoints, proc.Config().Acurl)
|
||||
}
|
||||
endpoints = append(endpoints, nodeClientURL.String())
|
||||
cx.t.Log("Verify all members have the same revision and hash")
|
||||
@ -1503,7 +1507,7 @@ type authUser struct {
|
||||
}
|
||||
|
||||
func setupAuth(cx ctlCtx, roles []authRole, users []authUser) {
|
||||
endpoint := cx.epc.procs[0].EndpointsV3()[0]
|
||||
endpoint := cx.epc.Procs[0].EndpointsV3()[0]
|
||||
|
||||
// create a client
|
||||
c, err := clientv3.New(clientv3.Config{Endpoints: []string{endpoint}, DialTimeout: 3 * time.Second})
|
||||
|
@ -18,6 +18,8 @@ import (
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Compact(t *testing.T) { testCtl(t, compactTest) }
|
||||
@ -71,5 +73,5 @@ func ctlV3Compact(cx ctlCtx, rev int64, physical bool) error {
|
||||
if physical {
|
||||
cmdArgs = append(cmdArgs, "--physical")
|
||||
}
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, "compacted revision "+rs)
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "compacted revision "+rs)
|
||||
}
|
||||
|
@ -14,7 +14,11 @@
|
||||
|
||||
package e2e
|
||||
|
||||
import "testing"
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3DefragOnline(t *testing.T) { testCtl(t, defragOnlineTest) }
|
||||
|
||||
@ -48,17 +52,17 @@ func defragOnlineTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3OnlineDefrag(cx ctlCtx) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "defrag")
|
||||
lines := make([]string, cx.epc.cfg.clusterSize)
|
||||
lines := make([]string, cx.epc.Cfg.ClusterSize)
|
||||
for i := range lines {
|
||||
lines[i] = "Finished defragmenting etcd member"
|
||||
}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
}
|
||||
|
||||
func ctlV3OfflineDefrag(cx ctlCtx) error {
|
||||
cmdArgs := append(cx.PrefixArgsUtl(), "defrag", "--data-dir", cx.dataDir)
|
||||
lines := []string{"finished defragmenting directory"}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
}
|
||||
|
||||
func defragOfflineTest(cx ctlCtx) {
|
||||
|
@ -21,6 +21,7 @@ import (
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Elect(t *testing.T) {
|
||||
@ -72,7 +73,7 @@ func testElect(cx ctlCtx) {
|
||||
if err = blocked.Signal(os.Interrupt); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err = closeWithTimeout(blocked, time.Second); err != nil {
|
||||
if err = e2e.CloseWithTimeout(blocked, time.Second); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -80,7 +81,7 @@ func testElect(cx ctlCtx) {
|
||||
if err = holder.Signal(os.Interrupt); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err = closeWithTimeout(holder, time.Second); err != nil {
|
||||
if err = e2e.CloseWithTimeout(holder, time.Second); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -98,7 +99,7 @@ func testElect(cx ctlCtx) {
|
||||
// ctlV3Elect creates a elect process with a channel listening for when it wins the election.
|
||||
func ctlV3Elect(cx ctlCtx, name, proposal string) (*expect.ExpectProcess, <-chan string, error) {
|
||||
cmdArgs := append(cx.PrefixArgs(), "elect", name, proposal)
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
outc := make(chan string, 1)
|
||||
if err != nil {
|
||||
close(outc)
|
||||
|
@ -22,6 +22,7 @@ import (
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3EndpointHealth(t *testing.T) { testCtl(t, endpointHealthTest, withQuorum()) }
|
||||
@ -36,11 +37,11 @@ func endpointHealthTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3EndpointHealth(cx ctlCtx) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "endpoint", "health")
|
||||
lines := make([]string, cx.epc.cfg.clusterSize)
|
||||
lines := make([]string, cx.epc.Cfg.ClusterSize)
|
||||
for i := range lines {
|
||||
lines[i] = "is healthy"
|
||||
}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
}
|
||||
|
||||
func endpointStatusTest(cx ctlCtx) {
|
||||
@ -56,7 +57,7 @@ func ctlV3EndpointStatus(cx ctlCtx) error {
|
||||
u, _ := url.Parse(ep)
|
||||
eps = append(eps, u.Host)
|
||||
}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, eps...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, eps...)
|
||||
}
|
||||
|
||||
func endpointHashKVTest(cx ctlCtx) {
|
||||
@ -88,5 +89,5 @@ func ctlV3EndpointHashKV(cx ctlCtx) error {
|
||||
u, _ := url.Parse(ep)
|
||||
ss = append(ss, fmt.Sprintf("%s, %d", u.Host, hresp.Hash))
|
||||
}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, ss...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, ss...)
|
||||
}
|
||||
|
@ -27,6 +27,7 @@ import (
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/testutil"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestAuthority(t *testing.T) {
|
||||
@ -82,18 +83,18 @@ func TestAuthority(t *testing.T) {
|
||||
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)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
cfg := newConfigNoTLS()
|
||||
cfg.clusterSize = clusterSize
|
||||
cfg := e2e.NewConfigNoTLS()
|
||||
cfg.ClusterSize = clusterSize
|
||||
if tc.useTLS {
|
||||
cfg.clientTLS = clientTLS
|
||||
cfg.ClientTLS = e2e.ClientTLS
|
||||
}
|
||||
cfg.isClientAutoTLS = tc.useInsecureTLS
|
||||
cfg.IsClientAutoTLS = tc.useInsecureTLS
|
||||
// Enable debug mode to get logs with http2 headers (including authority)
|
||||
cfg.envVars = map[string]string{"GODEBUG": "http2debug=2"}
|
||||
cfg.EnvVars = map[string]string{"GODEBUG": "http2debug=2"}
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, cfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
@ -116,13 +117,13 @@ func TestAuthority(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func templateEndpoints(t *testing.T, pattern string, clus *etcdProcessCluster) []string {
|
||||
func templateEndpoints(t *testing.T, pattern string, clus *e2e.EtcdProcessCluster) []string {
|
||||
t.Helper()
|
||||
endpoints := []string{}
|
||||
for i := 0; i < clus.cfg.clusterSize; i++ {
|
||||
for i := 0; i < clus.Cfg.ClusterSize; i++ {
|
||||
ent := pattern
|
||||
if strings.Contains(ent, "%d") {
|
||||
ent = fmt.Sprintf(ent, etcdProcessBasePort+i*5)
|
||||
ent = fmt.Sprintf(ent, e2e.EtcdProcessBasePort+i*5)
|
||||
}
|
||||
if strings.Contains(ent, "%") {
|
||||
t.Fatalf("Failed to template pattern, %% symbol left %q", ent)
|
||||
@ -132,12 +133,12 @@ func templateEndpoints(t *testing.T, pattern string, clus *etcdProcessCluster) [
|
||||
return endpoints
|
||||
}
|
||||
|
||||
func assertAuthority(t *testing.T, expectAuthorityPattern string, clus *etcdProcessCluster) {
|
||||
for i := range clus.procs {
|
||||
line, _ := clus.procs[i].Logs().Expect(`http2: decoded hpack field header field ":authority"`)
|
||||
func assertAuthority(t *testing.T, expectAuthorityPattern string, clus *e2e.EtcdProcessCluster) {
|
||||
for i := range clus.Procs {
|
||||
line, _ := clus.Procs[i].Logs().Expect(`http2: decoded hpack field header field ":authority"`)
|
||||
line = strings.TrimSuffix(line, "\n")
|
||||
line = strings.TrimSuffix(line, "\r")
|
||||
u, err := url.Parse(clus.procs[i].EndpointsGRPC()[0])
|
||||
u, err := url.Parse(clus.Procs[i].EndpointsGRPC()[0])
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -162,11 +163,11 @@ func executeWithTimeout(t *testing.T, timeout time.Duration, f func()) {
|
||||
}
|
||||
|
||||
type etcdctlV3 struct {
|
||||
cfg *etcdProcessClusterConfig
|
||||
cfg *e2e.EtcdProcessClusterConfig
|
||||
endpoints []string
|
||||
}
|
||||
|
||||
func clusterEtcdctlV3(cfg *etcdProcessClusterConfig, endpoints []string) *etcdctlV3 {
|
||||
func clusterEtcdctlV3(cfg *e2e.EtcdProcessClusterConfig, endpoints []string) *etcdctlV3 {
|
||||
return &etcdctlV3{
|
||||
cfg: cfg,
|
||||
endpoints: endpoints,
|
||||
@ -178,28 +179,28 @@ func (ctl *etcdctlV3) Put(key, value string) error {
|
||||
}
|
||||
|
||||
func (ctl *etcdctlV3) runCmd(args ...string) error {
|
||||
cmdArgs := []string{ctlBinPath + "3"}
|
||||
cmdArgs := []string{e2e.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")
|
||||
return e2e.SpawnWithExpect(cmdArgs, "OK")
|
||||
}
|
||||
|
||||
func (ctl *etcdctlV3) flags() map[string]string {
|
||||
fmap := make(map[string]string)
|
||||
if ctl.cfg.clientTLS == clientTLS {
|
||||
if ctl.cfg.isClientAutoTLS {
|
||||
if ctl.cfg.ClientTLS == e2e.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 if ctl.cfg.IsClientCRL {
|
||||
fmap["cacert"] = e2e.CaPath
|
||||
fmap["cert"] = e2e.RevokedCertPath
|
||||
fmap["key"] = e2e.RevokedPrivateKeyPath
|
||||
} else {
|
||||
fmap["cacert"] = caPath
|
||||
fmap["cert"] = certPath
|
||||
fmap["key"] = privateKeyPath
|
||||
fmap["cacert"] = e2e.CaPath
|
||||
fmap["cert"] = e2e.CertPath
|
||||
fmap["key"] = e2e.PrivateKeyPath
|
||||
}
|
||||
}
|
||||
fmap["endpoints"] = strings.Join(ctl.endpoints, ",")
|
||||
|
@ -19,27 +19,33 @@ import (
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Put(t *testing.T) { testCtl(t, putTest, withDialTimeout(7*time.Second)) }
|
||||
func TestCtlV3PutNoTLS(t *testing.T) { testCtl(t, putTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3PutClientTLS(t *testing.T) { testCtl(t, putTest, withCfg(*newConfigClientTLS())) }
|
||||
func TestCtlV3PutClientAutoTLS(t *testing.T) { testCtl(t, putTest, withCfg(*newConfigClientAutoTLS())) }
|
||||
func TestCtlV3PutPeerTLS(t *testing.T) { testCtl(t, putTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3PutTimeout(t *testing.T) { testCtl(t, putTest, withDialTimeout(0)) }
|
||||
func TestCtlV3Put(t *testing.T) { testCtl(t, putTest, withDialTimeout(7*time.Second)) }
|
||||
func TestCtlV3PutNoTLS(t *testing.T) { testCtl(t, putTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3PutClientTLS(t *testing.T) { testCtl(t, putTest, withCfg(*e2e.NewConfigClientTLS())) }
|
||||
func TestCtlV3PutClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, putTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3PutPeerTLS(t *testing.T) { testCtl(t, putTest, withCfg(*e2e.NewConfigPeerTLS())) }
|
||||
func TestCtlV3PutTimeout(t *testing.T) { testCtl(t, putTest, withDialTimeout(0)) }
|
||||
func TestCtlV3PutClientTLSFlagByEnv(t *testing.T) {
|
||||
testCtl(t, putTest, withCfg(*newConfigClientTLS()), withFlagByEnv())
|
||||
testCtl(t, putTest, withCfg(*e2e.NewConfigClientTLS()), withFlagByEnv())
|
||||
}
|
||||
func TestCtlV3PutIgnoreValue(t *testing.T) { testCtl(t, putTestIgnoreValue) }
|
||||
func TestCtlV3PutIgnoreLease(t *testing.T) { testCtl(t, putTestIgnoreLease) }
|
||||
|
||||
func TestCtlV3Get(t *testing.T) { testCtl(t, getTest) }
|
||||
func TestCtlV3GetNoTLS(t *testing.T) { testCtl(t, getTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3GetClientTLS(t *testing.T) { testCtl(t, getTest, withCfg(*newConfigClientTLS())) }
|
||||
func TestCtlV3GetClientAutoTLS(t *testing.T) { testCtl(t, getTest, withCfg(*newConfigClientAutoTLS())) }
|
||||
func TestCtlV3GetPeerTLS(t *testing.T) { testCtl(t, getTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3GetTimeout(t *testing.T) { testCtl(t, getTest, withDialTimeout(0)) }
|
||||
func TestCtlV3GetQuorum(t *testing.T) { testCtl(t, getTest, withQuorum()) }
|
||||
func TestCtlV3Get(t *testing.T) { testCtl(t, getTest) }
|
||||
func TestCtlV3GetNoTLS(t *testing.T) { testCtl(t, getTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3GetClientTLS(t *testing.T) { testCtl(t, getTest, withCfg(*e2e.NewConfigClientTLS())) }
|
||||
func TestCtlV3GetClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, getTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3GetPeerTLS(t *testing.T) { testCtl(t, getTest, withCfg(*e2e.NewConfigPeerTLS())) }
|
||||
func TestCtlV3GetTimeout(t *testing.T) { testCtl(t, getTest, withDialTimeout(0)) }
|
||||
func TestCtlV3GetQuorum(t *testing.T) { testCtl(t, getTest, withQuorum()) }
|
||||
|
||||
func TestCtlV3GetFormat(t *testing.T) { testCtl(t, getFormatTest) }
|
||||
func TestCtlV3GetRev(t *testing.T) { testCtl(t, getRevTest) }
|
||||
@ -47,18 +53,18 @@ func TestCtlV3GetKeysOnly(t *testing.T) { testCtl(t, getKeysOnlyTest) }
|
||||
func TestCtlV3GetCountOnly(t *testing.T) { testCtl(t, getCountOnlyTest) }
|
||||
|
||||
func TestCtlV3Del(t *testing.T) { testCtl(t, delTest) }
|
||||
func TestCtlV3DelNoTLS(t *testing.T) { testCtl(t, delTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3DelClientTLS(t *testing.T) { testCtl(t, delTest, withCfg(*newConfigClientTLS())) }
|
||||
func TestCtlV3DelPeerTLS(t *testing.T) { testCtl(t, delTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3DelNoTLS(t *testing.T) { testCtl(t, delTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3DelClientTLS(t *testing.T) { testCtl(t, delTest, withCfg(*e2e.NewConfigClientTLS())) }
|
||||
func TestCtlV3DelPeerTLS(t *testing.T) { testCtl(t, delTest, withCfg(*e2e.NewConfigPeerTLS())) }
|
||||
func TestCtlV3DelTimeout(t *testing.T) { testCtl(t, delTest, withDialTimeout(0)) }
|
||||
|
||||
func TestCtlV3GetRevokedCRL(t *testing.T) {
|
||||
cfg := etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
initialToken: "new",
|
||||
clientTLS: clientTLS,
|
||||
isClientCRL: true,
|
||||
clientCertAuthEnabled: true,
|
||||
cfg := e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
InitialToken: "new",
|
||||
ClientTLS: e2e.ClientTLS,
|
||||
IsClientCRL: true,
|
||||
ClientCertAuthEnabled: true,
|
||||
}
|
||||
testCtl(t, testGetRevokedCRL, withCfg(cfg))
|
||||
}
|
||||
@ -69,7 +75,7 @@ func testGetRevokedCRL(cx ctlCtx) {
|
||||
cx.t.Fatalf("expected reset connection on put, got %v", err)
|
||||
}
|
||||
// test accept
|
||||
cx.epc.cfg.isClientCRL = false
|
||||
cx.epc.Cfg.IsClientCRL = false
|
||||
if err := ctlV3Put(cx, "k", "v", ""); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
@ -190,7 +196,7 @@ func getFormatTest(cx ctlCtx) {
|
||||
cmdArgs = append(cmdArgs, "--print-value-only")
|
||||
}
|
||||
cmdArgs = append(cmdArgs, "abc")
|
||||
if err := spawnWithExpectWithEnv(cmdArgs, cx.envMap, tt.wstr); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, tt.wstr); err != nil {
|
||||
cx.t.Errorf("#%d: error (%v), wanted %v", i, err, tt.wstr)
|
||||
}
|
||||
}
|
||||
@ -228,24 +234,24 @@ func getKeysOnlyTest(cx ctlCtx) {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
cmdArgs := append(cx.PrefixArgs(), []string{"get", "--keys-only", "key"}...)
|
||||
if err := spawnWithExpectWithEnv(cmdArgs, cx.envMap, "key"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "key"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, "val"); err == nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "val"); err == nil {
|
||||
cx.t.Fatalf("got value but passed --keys-only")
|
||||
}
|
||||
}
|
||||
|
||||
func getCountOnlyTest(cx ctlCtx) {
|
||||
cmdArgs := append(cx.PrefixArgs(), []string{"get", "--count-only", "key", "--prefix", "--write-out=fields"}...)
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 0"); err != nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 0"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := ctlV3Put(cx, "key", "val", ""); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
cmdArgs = append(cx.PrefixArgs(), []string{"get", "--count-only", "key", "--prefix", "--write-out=fields"}...)
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 1"); err != nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 1"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := ctlV3Put(cx, "key1", "val", ""); err != nil {
|
||||
@ -255,21 +261,21 @@ func getCountOnlyTest(cx ctlCtx) {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
cmdArgs = append(cx.PrefixArgs(), []string{"get", "--count-only", "key", "--prefix", "--write-out=fields"}...)
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 2"); err != nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 2"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := ctlV3Put(cx, "key2", "val", ""); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
cmdArgs = append(cx.PrefixArgs(), []string{"get", "--count-only", "key", "--prefix", "--write-out=fields"}...)
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 3"); err != nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 3"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
expected := []string{
|
||||
"\"Count\" : 3",
|
||||
}
|
||||
cmdArgs = append(cx.PrefixArgs(), []string{"get", "--count-only", "key3", "--prefix", "--write-out=fields"}...)
|
||||
if err := spawnWithExpects(cmdArgs, cx.envMap, expected...); err == nil {
|
||||
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, expected...); err == nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -348,7 +354,7 @@ func ctlV3Put(cx ctlCtx, key, value, leaseID string, flags ...string) error {
|
||||
if len(flags) != 0 {
|
||||
cmdArgs = append(cmdArgs, flags...)
|
||||
}
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, "OK")
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "OK")
|
||||
}
|
||||
|
||||
type kv struct {
|
||||
@ -365,7 +371,7 @@ func ctlV3Get(cx ctlCtx, args []string, kvs ...kv) error {
|
||||
for _, elem := range kvs {
|
||||
lines = append(lines, elem.key, elem.val)
|
||||
}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
}
|
||||
|
||||
// ctlV3GetWithErr runs "get" command expecting no output but error
|
||||
@ -375,11 +381,11 @@ func ctlV3GetWithErr(cx ctlCtx, args []string, errs []string) error {
|
||||
if !cx.quorum {
|
||||
cmdArgs = append(cmdArgs, "--consistency", "s")
|
||||
}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, errs...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, errs...)
|
||||
}
|
||||
|
||||
func ctlV3Del(cx ctlCtx, args []string, num int) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "del")
|
||||
cmdArgs = append(cmdArgs, args...)
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, fmt.Sprintf("%d", num))
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, fmt.Sprintf("%d", num))
|
||||
}
|
||||
|
@ -20,90 +20,92 @@ import (
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3LeaseGrantTimeToLive(t *testing.T) { testCtl(t, leaseTestGrantTimeToLive) }
|
||||
func TestCtlV3LeaseGrantTimeToLiveNoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseGrantTimeToLiveClientTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseGrantTimeToLiveClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseGrantTimeToLivePeerTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, leaseTestGrantTimeToLive, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func TestCtlV3LeaseGrantLeases(t *testing.T) { testCtl(t, leaseTestGrantLeaseListed) }
|
||||
func TestCtlV3LeaseGrantLeasesNoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseGrantLeasesClientTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseGrantLeasesClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseGrantLeasesPeerTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, leaseTestGrantLeaseListed, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func TestCtlV3LeaseTestTimeToLiveExpired(t *testing.T) { testCtl(t, leaseTestTimeToLiveExpired) }
|
||||
func TestCtlV3LeaseTestTimeToLiveExpiredNoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseTestTimeToLiveExpiredClientTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseTestTimeToLiveExpiredClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseTestTimeToLiveExpiredPeerTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, leaseTestTimeToLiveExpired, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func TestCtlV3LeaseKeepAlive(t *testing.T) { testCtl(t, leaseTestKeepAlive) }
|
||||
func TestCtlV3LeaseKeepAliveNoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseKeepAliveClientTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseKeepAliveClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseKeepAlivePeerTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, leaseTestKeepAlive, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func TestCtlV3LeaseKeepAliveOnce(t *testing.T) { testCtl(t, leaseTestKeepAliveOnce) }
|
||||
func TestCtlV3LeaseKeepAliveOnceNoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseKeepAliveOnceClientTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseKeepAliveOnceClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseKeepAliveOncePeerTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, leaseTestKeepAliveOnce, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func TestCtlV3LeaseRevoke(t *testing.T) { testCtl(t, leaseTestRevoked) }
|
||||
func TestCtlV3LeaseRevokeNoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestRevoked, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, leaseTestRevoked, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseRevokeClientTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestRevoked, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, leaseTestRevoked, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseRevokeClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestRevoked, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, leaseTestRevoked, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3LeaseRevokePeerTLS(t *testing.T) {
|
||||
testCtl(t, leaseTestRevoked, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, leaseTestRevoked, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func leaseTestGrantTimeToLive(cx ctlCtx) {
|
||||
@ -113,7 +115,7 @@ func leaseTestGrantTimeToLive(cx ctlCtx) {
|
||||
}
|
||||
|
||||
cmdArgs := append(cx.PrefixArgs(), "lease", "timetolive", id, "--keys")
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("leaseTestGrantTimeToLive: error (%v)", err)
|
||||
}
|
||||
@ -146,7 +148,7 @@ func leaseTestGrantLeasesList(cx ctlCtx) error {
|
||||
}
|
||||
|
||||
cmdArgs := append(cx.PrefixArgs(), "lease", "list")
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
return fmt.Errorf("lease list failed (%v)", err)
|
||||
}
|
||||
@ -177,7 +179,7 @@ func leaseTestTimeToLiveExpire(cx ctlCtx, ttl int) error {
|
||||
time.Sleep(time.Duration(ttl+1) * time.Second)
|
||||
cmdArgs := append(cx.PrefixArgs(), "lease", "timetolive", leaseID)
|
||||
exp := fmt.Sprintf("lease %s already expired", leaseID)
|
||||
if err = spawnWithExpectWithEnv(cmdArgs, cx.envMap, exp); err != nil {
|
||||
if err = e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, exp); err != nil {
|
||||
return fmt.Errorf("lease not properly expired: (%v)", err)
|
||||
}
|
||||
if err := ctlV3Get(cx, []string{"key"}); err != nil {
|
||||
@ -247,7 +249,7 @@ func leaseTestRevoke(cx ctlCtx) error {
|
||||
|
||||
func ctlV3LeaseGrant(cx ctlCtx, ttl int) (string, error) {
|
||||
cmdArgs := append(cx.PrefixArgs(), "lease", "grant", strconv.Itoa(ttl))
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
@ -271,7 +273,7 @@ func ctlV3LeaseGrant(cx ctlCtx, ttl int) (string, error) {
|
||||
func ctlV3LeaseKeepAlive(cx ctlCtx, leaseID string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "lease", "keep-alive", leaseID)
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, nil)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -285,7 +287,7 @@ func ctlV3LeaseKeepAlive(cx ctlCtx, leaseID string) error {
|
||||
func ctlV3LeaseKeepAliveOnce(cx ctlCtx, leaseID string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "lease", "keep-alive", "--once", leaseID)
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, nil)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -298,7 +300,7 @@ func ctlV3LeaseKeepAliveOnce(cx ctlCtx, leaseID string) error {
|
||||
|
||||
func ctlV3LeaseRevoke(cx ctlCtx, leaseID string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "lease", "revoke", leaseID)
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("lease %s revoked", leaseID))
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("lease %s revoked", leaseID))
|
||||
}
|
||||
|
||||
func ctlV3LeaseTimeToLive(cx ctlCtx, leaseID string, withKeys bool) error {
|
||||
@ -306,5 +308,5 @@ func ctlV3LeaseTimeToLive(cx ctlCtx, leaseID string, withKeys bool) error {
|
||||
if withKeys {
|
||||
cmdArgs = append(cmdArgs, "--keys")
|
||||
}
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("lease %s granted with", leaseID))
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("lease %s granted with", leaseID))
|
||||
}
|
||||
|
@ -22,6 +22,7 @@ import (
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Lock(t *testing.T) {
|
||||
@ -77,7 +78,7 @@ func testLock(cx ctlCtx) {
|
||||
if err = blocked.Signal(os.Interrupt); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err = closeWithTimeout(blocked, time.Second); err != nil {
|
||||
if err = e2e.CloseWithTimeout(blocked, time.Second); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -85,7 +86,7 @@ func testLock(cx ctlCtx) {
|
||||
if err = holder.Signal(os.Interrupt); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err = closeWithTimeout(holder, 200*time.Millisecond+time.Second); err != nil {
|
||||
if err = e2e.CloseWithTimeout(holder, 200*time.Millisecond+time.Second); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -119,7 +120,7 @@ func testLockWithCmd(cx ctlCtx) {
|
||||
// ctlV3Lock creates a lock process with a channel listening for when it acquires the lock.
|
||||
func ctlV3Lock(cx ctlCtx, name string) (*expect.ExpectProcess, <-chan string, error) {
|
||||
cmdArgs := append(cx.PrefixArgs(), "lock", name)
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
outc := make(chan string, 1)
|
||||
if err != nil {
|
||||
close(outc)
|
||||
@ -140,5 +141,5 @@ func ctlV3LockWithCmd(cx ctlCtx, execCmd []string, as ...string) error {
|
||||
// use command as lock name
|
||||
cmdArgs := append(cx.PrefixArgs(), "lock", execCmd[0])
|
||||
cmdArgs = append(cmdArgs, execCmd...)
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, as...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, as...)
|
||||
}
|
||||
|
@ -18,6 +18,8 @@ import (
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3MakeMirror(t *testing.T) { testCtl(t, makeMirrorTest) }
|
||||
@ -59,16 +61,16 @@ func makeMirrorNoDestPrefixTest(cx ctlCtx) {
|
||||
|
||||
func testMirrorCommand(cx ctlCtx, flags []string, sourcekvs []kv, destkvs []kvExec, srcprefix, destprefix string) {
|
||||
// set up another cluster to mirror with
|
||||
mirrorcfg := newConfigAutoTLS()
|
||||
mirrorcfg.clusterSize = 1
|
||||
mirrorcfg.basePort = 10000
|
||||
mirrorcfg := e2e.NewConfigAutoTLS()
|
||||
mirrorcfg.ClusterSize = 1
|
||||
mirrorcfg.BasePort = 10000
|
||||
mirrorctx := ctlCtx{
|
||||
t: cx.t,
|
||||
cfg: *mirrorcfg,
|
||||
dialTimeout: 7 * time.Second,
|
||||
}
|
||||
|
||||
mirrorepc, err := newEtcdProcessCluster(cx.t, &mirrorctx.cfg)
|
||||
mirrorepc, err := e2e.NewEtcdProcessCluster(cx.t, &mirrorctx.cfg)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
@ -82,8 +84,8 @@ func testMirrorCommand(cx ctlCtx, flags []string, sourcekvs []kv, destkvs []kvEx
|
||||
|
||||
cmdArgs := append(cx.PrefixArgs(), "make-mirror")
|
||||
cmdArgs = append(cmdArgs, flags...)
|
||||
cmdArgs = append(cmdArgs, fmt.Sprintf("localhost:%d", mirrorcfg.basePort))
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
cmdArgs = append(cmdArgs, fmt.Sprintf("localhost:%d", mirrorcfg.BasePort))
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
@ -23,64 +23,69 @@ import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/etcdserverpb"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3MemberList(t *testing.T) { testCtl(t, memberListTest) }
|
||||
func TestCtlV3MemberListWithHex(t *testing.T) { testCtl(t, memberListWithHexTest) }
|
||||
func TestCtlV3MemberListNoTLS(t *testing.T) { testCtl(t, memberListTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3MemberListNoTLS(t *testing.T) {
|
||||
testCtl(t, memberListTest, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3MemberListClientTLS(t *testing.T) {
|
||||
testCtl(t, memberListTest, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, memberListTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3MemberListClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, memberListTest, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, memberListTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3MemberListPeerTLS(t *testing.T) {
|
||||
testCtl(t, memberListTest, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, memberListTest, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
func TestCtlV3MemberRemove(t *testing.T) {
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig())
|
||||
}
|
||||
func TestCtlV3MemberRemoveNoTLS(t *testing.T) {
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig(), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig(), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3MemberRemoveClientTLS(t *testing.T) {
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig(), withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig(), withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3MemberRemoveClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig(), withCfg(
|
||||
// default clusterSize is 1
|
||||
etcdProcessClusterConfig{
|
||||
clusterSize: 3,
|
||||
isClientAutoTLS: true,
|
||||
clientTLS: clientTLS,
|
||||
initialToken: "new",
|
||||
// default ClusterSize is 1
|
||||
e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
IsClientAutoTLS: true,
|
||||
ClientTLS: e2e.ClientTLS,
|
||||
InitialToken: "new",
|
||||
}))
|
||||
}
|
||||
func TestCtlV3MemberRemovePeerTLS(t *testing.T) {
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig(), withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, memberRemoveTest, withQuorum(), withNoStrictReconfig(), withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
func TestCtlV3MemberAdd(t *testing.T) { testCtl(t, memberAddTest) }
|
||||
func TestCtlV3MemberAddNoTLS(t *testing.T) { testCtl(t, memberAddTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3MemberAddNoTLS(t *testing.T) { testCtl(t, memberAddTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3MemberAddClientTLS(t *testing.T) {
|
||||
testCtl(t, memberAddTest, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, memberAddTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3MemberAddClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, memberAddTest, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, memberAddTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3MemberAddPeerTLS(t *testing.T) {
|
||||
testCtl(t, memberAddTest, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
func TestCtlV3MemberAddPeerTLS(t *testing.T) { testCtl(t, memberAddTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3MemberAddForLearner(t *testing.T) { testCtl(t, memberAddForLearnerTest) }
|
||||
func TestCtlV3MemberUpdate(t *testing.T) { testCtl(t, memberUpdateTest) }
|
||||
func TestCtlV3MemberUpdateNoTLS(t *testing.T) {
|
||||
testCtl(t, memberUpdateTest, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, memberUpdateTest, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3MemberUpdateClientTLS(t *testing.T) {
|
||||
testCtl(t, memberUpdateTest, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, memberUpdateTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3MemberUpdateClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, memberUpdateTest, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, memberUpdateTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3MemberUpdatePeerTLS(t *testing.T) {
|
||||
testCtl(t, memberUpdateTest, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, memberUpdateTest, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func memberListTest(cx ctlCtx) {
|
||||
@ -91,17 +96,17 @@ func memberListTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3MemberList(cx ctlCtx) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "member", "list")
|
||||
lines := make([]string, cx.cfg.clusterSize)
|
||||
lines := make([]string, cx.cfg.ClusterSize)
|
||||
for i := range lines {
|
||||
lines[i] = "started"
|
||||
}
|
||||
return spawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, lines...)
|
||||
}
|
||||
|
||||
func getMemberList(cx ctlCtx) (etcdserverpb.MemberListResponse, error) {
|
||||
cmdArgs := append(cx.PrefixArgs(), "--write-out", "json", "member", "list")
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
return etcdserverpb.MemberListResponse{}, err
|
||||
}
|
||||
@ -130,7 +135,7 @@ func memberListWithHexTest(cx ctlCtx) {
|
||||
|
||||
cmdArgs := append(cx.PrefixArgs(), "--write-out", "json", "--hex", "member", "list")
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("memberListWithHexTest error (%v)", err)
|
||||
}
|
||||
@ -182,17 +187,17 @@ func memberRemoveTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3MemberRemove(cx ctlCtx, ep, memberID, clusterID string) error {
|
||||
cmdArgs := append(cx.prefixArgs([]string{ep}), "member", "remove", memberID)
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("%s removed from cluster %s", memberID, clusterID))
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("%s removed from cluster %s", memberID, clusterID))
|
||||
}
|
||||
|
||||
func memberAddTest(cx ctlCtx) {
|
||||
if err := ctlV3MemberAdd(cx, fmt.Sprintf("http://localhost:%d", etcdProcessBasePort+11), false); err != nil {
|
||||
if err := ctlV3MemberAdd(cx, fmt.Sprintf("http://localhost:%d", e2e.EtcdProcessBasePort+11), false); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func memberAddForLearnerTest(cx ctlCtx) {
|
||||
if err := ctlV3MemberAdd(cx, fmt.Sprintf("http://localhost:%d", etcdProcessBasePort+11), true); err != nil {
|
||||
if err := ctlV3MemberAdd(cx, fmt.Sprintf("http://localhost:%d", e2e.EtcdProcessBasePort+11), true); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -202,7 +207,7 @@ func ctlV3MemberAdd(cx ctlCtx, peerURL string, isLearner bool) error {
|
||||
if isLearner {
|
||||
cmdArgs = append(cmdArgs, "--learner")
|
||||
}
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, " added to cluster ")
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, " added to cluster ")
|
||||
}
|
||||
|
||||
func memberUpdateTest(cx ctlCtx) {
|
||||
@ -211,7 +216,7 @@ func memberUpdateTest(cx ctlCtx) {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", etcdProcessBasePort+11)
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", e2e.EtcdProcessBasePort+11)
|
||||
memberID := fmt.Sprintf("%x", mr.Members[0].ID)
|
||||
if err = ctlV3MemberUpdate(cx, memberID, peerURL); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
@ -220,5 +225,5 @@ func memberUpdateTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3MemberUpdate(cx ctlCtx, memberID, peerURL string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "member", "update", memberID, fmt.Sprintf("--peer-urls=%s", peerURL))
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, " updated in cluster ")
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, " updated in cluster ")
|
||||
}
|
||||
|
@ -24,15 +24,16 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/transport"
|
||||
"go.etcd.io/etcd/client/pkg/v3/types"
|
||||
"go.etcd.io/etcd/client/v3"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3MoveLeaderScenarios(t *testing.T) {
|
||||
securityParent := map[string]struct {
|
||||
cfg etcdProcessClusterConfig
|
||||
cfg e2e.EtcdProcessClusterConfig
|
||||
}{
|
||||
"Secure": {cfg: *newConfigTLS()},
|
||||
"Insecure": {cfg: *newConfigNoTLS()},
|
||||
"Secure": {cfg: *e2e.NewConfigTLS()},
|
||||
"Insecure": {cfg: *e2e.NewConfigNoTLS()},
|
||||
}
|
||||
|
||||
tests := map[string]struct {
|
||||
@ -51,8 +52,8 @@ func TestCtlV3MoveLeaderScenarios(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func testCtlV3MoveLeader(t *testing.T, cfg etcdProcessClusterConfig, envVars map[string]string) {
|
||||
BeforeTest(t)
|
||||
func testCtlV3MoveLeader(t *testing.T, cfg e2e.EtcdProcessClusterConfig, envVars map[string]string) {
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
epc := setupEtcdctlTest(t, &cfg, true)
|
||||
defer func() {
|
||||
@ -62,11 +63,11 @@ func testCtlV3MoveLeader(t *testing.T, cfg etcdProcessClusterConfig, envVars map
|
||||
}()
|
||||
|
||||
var tcfg *tls.Config
|
||||
if cfg.clientTLS == clientTLS {
|
||||
if cfg.ClientTLS == e2e.ClientTLS {
|
||||
tinfo := transport.TLSInfo{
|
||||
CertFile: certPath,
|
||||
KeyFile: privateKeyPath,
|
||||
TrustedCAFile: caPath,
|
||||
CertFile: e2e.CertPath,
|
||||
KeyFile: e2e.PrivateKeyPath,
|
||||
TrustedCAFile: e2e.CaPath,
|
||||
}
|
||||
var err error
|
||||
tcfg, err = tinfo.ClientConfig()
|
||||
@ -107,7 +108,7 @@ func testCtlV3MoveLeader(t *testing.T, cfg etcdProcessClusterConfig, envVars map
|
||||
defer os.Unsetenv("ETCDCTL_API")
|
||||
cx := ctlCtx{
|
||||
t: t,
|
||||
cfg: *newConfigNoTLS(),
|
||||
cfg: *e2e.NewConfigNoTLS(),
|
||||
dialTimeout: 7 * time.Second,
|
||||
epc: epc,
|
||||
envMap: envVars,
|
||||
@ -133,7 +134,7 @@ func testCtlV3MoveLeader(t *testing.T, cfg etcdProcessClusterConfig, envVars map
|
||||
for i, tc := range tests {
|
||||
prefix := cx.prefixArgs(tc.eps)
|
||||
cmdArgs := append(prefix, "move-leader", types.ID(transferee).String())
|
||||
if err := spawnWithExpectWithEnv(cmdArgs, cx.envMap, tc.expect); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, tc.expect); err != nil {
|
||||
t.Fatalf("#%d: %v", i, err)
|
||||
}
|
||||
}
|
||||
|
@ -17,13 +17,17 @@ package e2e
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3RoleAdd(t *testing.T) { testCtl(t, roleAddTest) }
|
||||
func TestCtlV3RoleAddNoTLS(t *testing.T) { testCtl(t, roleAddTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3RoleAddClientTLS(t *testing.T) { testCtl(t, roleAddTest, withCfg(*newConfigClientTLS())) }
|
||||
func TestCtlV3RoleAddPeerTLS(t *testing.T) { testCtl(t, roleAddTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3RoleAddTimeout(t *testing.T) { testCtl(t, roleAddTest, withDialTimeout(0)) }
|
||||
func TestCtlV3RoleAdd(t *testing.T) { testCtl(t, roleAddTest) }
|
||||
func TestCtlV3RoleAddNoTLS(t *testing.T) { testCtl(t, roleAddTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3RoleAddClientTLS(t *testing.T) {
|
||||
testCtl(t, roleAddTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3RoleAddPeerTLS(t *testing.T) { testCtl(t, roleAddTest, withCfg(*e2e.NewConfigPeerTLS())) }
|
||||
func TestCtlV3RoleAddTimeout(t *testing.T) { testCtl(t, roleAddTest, withDialTimeout(0)) }
|
||||
|
||||
func TestCtlV3RoleGrant(t *testing.T) { testCtl(t, roleGrantTest) }
|
||||
|
||||
@ -96,7 +100,7 @@ func ctlV3Role(cx ctlCtx, args []string, expStr string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "role")
|
||||
cmdArgs = append(cmdArgs, args...)
|
||||
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, expStr)
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, expStr)
|
||||
}
|
||||
|
||||
func ctlV3RoleGrantPermission(cx ctlCtx, rolename string, perm grantingPerm) error {
|
||||
@ -110,7 +114,7 @@ func ctlV3RoleGrantPermission(cx ctlCtx, rolename string, perm grantingPerm) err
|
||||
cmdArgs = append(cmdArgs, rolename)
|
||||
cmdArgs = append(cmdArgs, grantingPermToArgs(perm)...)
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -136,7 +140,7 @@ func ctlV3RoleRevokePermission(cx ctlCtx, rolename string, key, rangeEnd string,
|
||||
expStr = fmt.Sprintf("Permission of key %s is revoked from role %s", key, rolename)
|
||||
}
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -31,6 +31,7 @@ import (
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/etcdutl/v3/snapshot"
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Snapshot(t *testing.T) { testCtl(t, snapshotTest) }
|
||||
@ -89,7 +90,7 @@ func snapshotCorruptTest(cx ctlCtx) {
|
||||
|
||||
datadir := cx.t.TempDir()
|
||||
|
||||
serr := spawnWithExpectWithEnv(
|
||||
serr := e2e.SpawnWithExpectWithEnv(
|
||||
append(cx.PrefixArgsUtl(), "snapshot", "restore",
|
||||
"--data-dir", datadir,
|
||||
fpath),
|
||||
@ -123,7 +124,7 @@ func snapshotStatusBeforeRestoreTest(cx ctlCtx) {
|
||||
|
||||
dataDir := cx.t.TempDir()
|
||||
defer os.RemoveAll(dataDir)
|
||||
serr := spawnWithExpectWithEnv(
|
||||
serr := e2e.SpawnWithExpectWithEnv(
|
||||
append(cx.PrefixArgsUtl(), "snapshot", "restore",
|
||||
"--data-dir", dataDir,
|
||||
fpath),
|
||||
@ -136,13 +137,13 @@ func snapshotStatusBeforeRestoreTest(cx ctlCtx) {
|
||||
|
||||
func ctlV3SnapshotSave(cx ctlCtx, fpath string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "snapshot", "save", fpath)
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("Snapshot saved at %s", fpath))
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, fmt.Sprintf("Snapshot saved at %s", fpath))
|
||||
}
|
||||
|
||||
func getSnapshotStatus(cx ctlCtx, fpath string) (snapshot.Status, error) {
|
||||
cmdArgs := append(cx.PrefixArgsUtl(), "--write-out", "json", "snapshot", "status", fpath)
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, nil)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, nil)
|
||||
if err != nil {
|
||||
return snapshot.Status{}, err
|
||||
}
|
||||
@ -177,14 +178,14 @@ func testIssue6361(t *testing.T, etcdutl bool) {
|
||||
os.Setenv("EXPECT_DEBUG", "1")
|
||||
}
|
||||
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
os.Setenv("ETCDCTL_API", "3")
|
||||
defer os.Unsetenv("ETCDCTL_API")
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
initialToken: "new",
|
||||
keepDataDir: true,
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
InitialToken: "new",
|
||||
KeepDataDir: true,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
@ -196,12 +197,12 @@ func testIssue6361(t *testing.T, etcdutl bool) {
|
||||
}()
|
||||
|
||||
dialTimeout := 10 * time.Second
|
||||
prefixArgs := []string{ctlBinPath, "--endpoints", strings.Join(epc.EndpointsV3(), ","), "--dial-timeout", dialTimeout.String()}
|
||||
prefixArgs := []string{e2e.CtlBinPath, "--endpoints", strings.Join(epc.EndpointsV3(), ","), "--dial-timeout", dialTimeout.String()}
|
||||
|
||||
t.Log("Writing some keys...")
|
||||
kvs := []kv{{"foo1", "val1"}, {"foo2", "val2"}, {"foo3", "val3"}}
|
||||
for i := range kvs {
|
||||
if err = spawnWithExpect(append(prefixArgs, "put", kvs[i].key, kvs[i].val), "OK"); err != nil {
|
||||
if err = e2e.SpawnWithExpect(append(prefixArgs, "put", kvs[i].key, kvs[i].val), "OK"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -209,7 +210,7 @@ func testIssue6361(t *testing.T, etcdutl bool) {
|
||||
fpath := filepath.Join(t.TempDir(), "test.snapshot")
|
||||
|
||||
t.Log("etcdctl saving snapshot...")
|
||||
if err = spawnWithExpects(append(prefixArgs, "snapshot", "save", fpath),
|
||||
if err = e2e.SpawnWithExpects(append(prefixArgs, "snapshot", "save", fpath),
|
||||
nil,
|
||||
fmt.Sprintf("Snapshot saved at %s", fpath),
|
||||
); err != nil {
|
||||
@ -217,45 +218,45 @@ func testIssue6361(t *testing.T, etcdutl bool) {
|
||||
}
|
||||
|
||||
t.Log("Stopping the original server...")
|
||||
if err = epc.procs[0].Stop(); err != nil {
|
||||
if err = epc.Procs[0].Stop(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
newDataDir := filepath.Join(t.TempDir(), "test.data")
|
||||
|
||||
uctlBinPath := ctlBinPath
|
||||
uctlBinPath := e2e.CtlBinPath
|
||||
if etcdutl {
|
||||
uctlBinPath = utlBinPath
|
||||
uctlBinPath = e2e.UtlBinPath
|
||||
}
|
||||
|
||||
t.Log("etcdctl restoring the snapshot...")
|
||||
err = spawnWithExpect([]string{uctlBinPath, "snapshot", "restore", fpath, "--name", epc.procs[0].Config().name, "--initial-cluster", epc.procs[0].Config().initialCluster, "--initial-cluster-token", epc.procs[0].Config().initialToken, "--initial-advertise-peer-urls", epc.procs[0].Config().purl.String(), "--data-dir", newDataDir}, "added member")
|
||||
err = e2e.SpawnWithExpect([]string{uctlBinPath, "snapshot", "restore", fpath, "--name", epc.Procs[0].Config().Name, "--initial-cluster", epc.Procs[0].Config().InitialCluster, "--initial-cluster-token", epc.Procs[0].Config().InitialToken, "--initial-advertise-peer-urls", epc.Procs[0].Config().Purl.String(), "--data-dir", newDataDir}, "added member")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
t.Log("(Re)starting the etcd member using the restored snapshot...")
|
||||
epc.procs[0].Config().dataDirPath = newDataDir
|
||||
for i := range epc.procs[0].Config().args {
|
||||
if epc.procs[0].Config().args[i] == "--data-dir" {
|
||||
epc.procs[0].Config().args[i+1] = newDataDir
|
||||
epc.Procs[0].Config().DataDirPath = newDataDir
|
||||
for i := range epc.Procs[0].Config().Args {
|
||||
if epc.Procs[0].Config().Args[i] == "--data-dir" {
|
||||
epc.Procs[0].Config().Args[i+1] = newDataDir
|
||||
}
|
||||
}
|
||||
if err = epc.procs[0].Restart(); err != nil {
|
||||
if err = epc.Procs[0].Restart(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
t.Log("Ensuring the restored member has the correct data...")
|
||||
for i := range kvs {
|
||||
if err = spawnWithExpect(append(prefixArgs, "get", kvs[i].key), kvs[i].val); err != nil {
|
||||
if err = e2e.SpawnWithExpect(append(prefixArgs, "get", kvs[i].key), kvs[i].val); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
t.Log("Adding new member into the cluster")
|
||||
clientURL := fmt.Sprintf("http://localhost:%d", etcdProcessBasePort+30)
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", etcdProcessBasePort+31)
|
||||
err = spawnWithExpect(append(prefixArgs, "member", "add", "newmember", fmt.Sprintf("--peer-urls=%s", peerURL)), " added to cluster ")
|
||||
clientURL := fmt.Sprintf("http://localhost:%d", e2e.EtcdProcessBasePort+30)
|
||||
peerURL := fmt.Sprintf("http://localhost:%d", e2e.EtcdProcessBasePort+31)
|
||||
err = e2e.SpawnWithExpect(append(prefixArgs, "member", "add", "newmember", fmt.Sprintf("--peer-urls=%s", peerURL)), " added to cluster ")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -264,12 +265,12 @@ func testIssue6361(t *testing.T, etcdutl bool) {
|
||||
defer os.RemoveAll(newDataDir2)
|
||||
|
||||
name2 := "infra2"
|
||||
initialCluster2 := epc.procs[0].Config().initialCluster + fmt.Sprintf(",%s=%s", name2, peerURL)
|
||||
initialCluster2 := epc.Procs[0].Config().InitialCluster + fmt.Sprintf(",%s=%s", name2, peerURL)
|
||||
|
||||
t.Log("Starting the new member")
|
||||
// start the new member
|
||||
var nepc *expect.ExpectProcess
|
||||
nepc, err = spawnCmd([]string{epc.procs[0].Config().execPath, "--name", name2,
|
||||
nepc, err = e2e.SpawnCmd([]string{epc.Procs[0].Config().ExecPath, "--name", name2,
|
||||
"--listen-client-urls", clientURL, "--advertise-client-urls", clientURL,
|
||||
"--listen-peer-urls", peerURL, "--initial-advertise-peer-urls", peerURL,
|
||||
"--initial-cluster", initialCluster2, "--initial-cluster-state", "existing", "--data-dir", newDataDir2}, nil)
|
||||
@ -280,11 +281,11 @@ func testIssue6361(t *testing.T, etcdutl bool) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
prefixArgs = []string{ctlBinPath, "--endpoints", clientURL, "--dial-timeout", dialTimeout.String()}
|
||||
prefixArgs = []string{e2e.CtlBinPath, "--endpoints", clientURL, "--dial-timeout", dialTimeout.String()}
|
||||
|
||||
t.Log("Ensuring added member has data from incoming snapshot...")
|
||||
for i := range kvs {
|
||||
if err = spawnWithExpect(append(prefixArgs, "get", kvs[i].key), kvs[i].val); err != nil {
|
||||
if err = e2e.SpawnWithExpect(append(prefixArgs, "get", kvs[i].key), kvs[i].val); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -297,12 +298,12 @@ func testIssue6361(t *testing.T, etcdutl bool) {
|
||||
}
|
||||
|
||||
func TestRestoreCompactionRevBump(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
initialToken: "new",
|
||||
keepDataDir: true,
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
InitialToken: "new",
|
||||
KeepDataDir: true,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
@ -314,16 +315,16 @@ func TestRestoreCompactionRevBump(t *testing.T) {
|
||||
}()
|
||||
|
||||
dialTimeout := 10 * time.Second
|
||||
prefixArgs := []string{ctlBinPath, "--endpoints", strings.Join(epc.EndpointsV3(), ","), "--dial-timeout", dialTimeout.String()}
|
||||
prefixArgs := []string{e2e.CtlBinPath, "--endpoints", strings.Join(epc.EndpointsV3(), ","), "--dial-timeout", dialTimeout.String()}
|
||||
|
||||
ctl := newClient(t, epc.EndpointsV3(), epc.cfg.clientTLS, epc.cfg.isClientAutoTLS)
|
||||
ctl := newClient(t, epc.EndpointsV3(), epc.Cfg.ClientTLS, epc.Cfg.IsClientAutoTLS)
|
||||
watchCh := ctl.Watch(context.Background(), "foo", clientv3.WithPrefix())
|
||||
// flake-fix: the watch can sometimes miss the first put below causing test failure
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
kvs := []kv{{"foo1", "val1"}, {"foo2", "val2"}, {"foo3", "val3"}}
|
||||
for i := range kvs {
|
||||
if err = spawnWithExpect(append(prefixArgs, "put", kvs[i].key, kvs[i].val), "OK"); err != nil {
|
||||
if err = e2e.SpawnWithExpect(append(prefixArgs, "put", kvs[i].key, kvs[i].val), "OK"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -341,12 +342,12 @@ func TestRestoreCompactionRevBump(t *testing.T) {
|
||||
fpath := filepath.Join(t.TempDir(), "test.snapshot")
|
||||
|
||||
t.Log("etcdctl saving snapshot...")
|
||||
require.NoError(t, spawnWithExpect(append(prefixArgs, "snapshot", "save", fpath), fmt.Sprintf("Snapshot saved at %s", fpath)))
|
||||
require.NoError(t, e2e.SpawnWithExpect(append(prefixArgs, "snapshot", "save", fpath), fmt.Sprintf("Snapshot saved at %s", fpath)))
|
||||
|
||||
// add some more kvs that are not in the snapshot that will be lost after restore
|
||||
unsnappedKVs := []kv{{"unsnapped1", "one"}, {"unsnapped2", "two"}, {"unsnapped3", "three"}}
|
||||
for i := range unsnappedKVs {
|
||||
if err = spawnWithExpect(append(prefixArgs, "put", unsnappedKVs[i].key, unsnappedKVs[i].val), "OK"); err != nil {
|
||||
if err = e2e.SpawnWithExpect(append(prefixArgs, "put", unsnappedKVs[i].key, unsnappedKVs[i].val), "OK"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -357,14 +358,14 @@ func TestRestoreCompactionRevBump(t *testing.T) {
|
||||
newDataDir := filepath.Join(t.TempDir(), "test.data")
|
||||
t.Log("etcdctl restoring the snapshot...")
|
||||
bumpAmount := 10000
|
||||
err = spawnWithExpect([]string{
|
||||
utlBinPath,
|
||||
err = e2e.SpawnWithExpect([]string{
|
||||
e2e.UtlBinPath,
|
||||
"snapshot",
|
||||
"restore", fpath,
|
||||
"--name", epc.procs[0].Config().name,
|
||||
"--initial-cluster", epc.procs[0].Config().initialCluster,
|
||||
"--initial-cluster-token", epc.procs[0].Config().initialToken,
|
||||
"--initial-advertise-peer-urls", epc.procs[0].Config().purl.String(),
|
||||
"--name", epc.Procs[0].Config().Name,
|
||||
"--initial-cluster", epc.Procs[0].Config().InitialCluster,
|
||||
"--initial-cluster-token", epc.Procs[0].Config().InitialToken,
|
||||
"--initial-advertise-peer-urls", epc.Procs[0].Config().Purl.String(),
|
||||
"--bump-revision", fmt.Sprintf("%d", bumpAmount),
|
||||
"--mark-compacted",
|
||||
"--data-dir", newDataDir,
|
||||
@ -372,10 +373,10 @@ func TestRestoreCompactionRevBump(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
t.Log("(Re)starting the etcd member using the restored snapshot...")
|
||||
epc.procs[0].Config().dataDirPath = newDataDir
|
||||
for i := range epc.procs[0].Config().args {
|
||||
if epc.procs[0].Config().args[i] == "--data-dir" {
|
||||
epc.procs[0].Config().args[i+1] = newDataDir
|
||||
epc.Procs[0].Config().DataDirPath = newDataDir
|
||||
for i := range epc.Procs[0].Config().Args {
|
||||
if epc.Procs[0].Config().Args[i] == "--data-dir" {
|
||||
epc.Procs[0].Config().Args[i+1] = newDataDir
|
||||
}
|
||||
}
|
||||
|
||||
@ -412,7 +413,7 @@ func TestRestoreCompactionRevBump(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), watchTimeout*5)
|
||||
defer cancel()
|
||||
watchCh = ctl.Watch(ctx, "foo", clientv3.WithPrefix(), clientv3.WithRev(int64(bumpAmount+currentRev+1)))
|
||||
if err = spawnWithExpect(append(prefixArgs, "put", "foo4", "val4"), "OK"); err != nil {
|
||||
if err = e2e.SpawnWithExpect(append(prefixArgs, "put", "foo4", "val4"), "OK"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
watchRes, err = keyValuesFromWatchChan(watchCh, 1, watchTimeout)
|
||||
|
@ -26,12 +26,13 @@ import (
|
||||
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
||||
"go.etcd.io/etcd/client/pkg/v3/testutil"
|
||||
"go.etcd.io/etcd/pkg/v3/flags"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Version(t *testing.T) { testCtl(t, versionTest) }
|
||||
|
||||
func TestClusterVersion(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
@ -49,18 +50,18 @@ func TestClusterVersion(t *testing.T) {
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
binary := binDir + "/etcd"
|
||||
binary := e2e.BinDir + "/etcd"
|
||||
if !fileutil.Exist(binary) {
|
||||
t.Skipf("%q does not exist", binary)
|
||||
}
|
||||
BeforeTest(t)
|
||||
cfg := newConfigNoTLS()
|
||||
cfg.execPath = binary
|
||||
cfg.snapshotCount = 3
|
||||
cfg.baseScheme = "unix" // to avoid port conflict
|
||||
cfg.rollingStart = tt.rollingStart
|
||||
e2e.BeforeTest(t)
|
||||
cfg := e2e.NewConfigNoTLS()
|
||||
cfg.ExecPath = binary
|
||||
cfg.SnapshotCount = 3
|
||||
cfg.BaseScheme = "unix" // to avoid port conflict
|
||||
cfg.RollingStart = tt.rollingStart
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, cfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
@ -90,7 +91,7 @@ func versionTest(cx ctlCtx) {
|
||||
func clusterVersionTest(cx ctlCtx, expected string) {
|
||||
var err error
|
||||
for i := 0; i < 35; i++ {
|
||||
if err = cURLGet(cx.epc, cURLReq{endpoint: "/version", expected: expected}); err != nil {
|
||||
if err = e2e.CURLGet(cx.epc, e2e.CURLReq{Endpoint: "/version", Expected: expected}); err != nil {
|
||||
cx.t.Logf("#%d: v3 is not ready yet (%v)", i, err)
|
||||
time.Sleep(200 * time.Millisecond)
|
||||
continue
|
||||
@ -104,17 +105,17 @@ func clusterVersionTest(cx ctlCtx, expected string) {
|
||||
|
||||
func ctlV3Version(cx ctlCtx) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "version")
|
||||
return spawnWithExpectWithEnv(cmdArgs, cx.envMap, version.Version)
|
||||
return e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, version.Version)
|
||||
}
|
||||
|
||||
// TestCtlV3DialWithHTTPScheme ensures that client handles endpoints with HTTPS scheme.
|
||||
// TestCtlV3DialWithHTTPScheme ensures that client handles Endpoints with HTTPS scheme.
|
||||
func TestCtlV3DialWithHTTPScheme(t *testing.T) {
|
||||
testCtl(t, dialWithSchemeTest, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, dialWithSchemeTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
|
||||
func dialWithSchemeTest(cx ctlCtx) {
|
||||
cmdArgs := append(cx.prefixArgs(cx.epc.EndpointsV3()), "put", "foo", "bar")
|
||||
if err := spawnWithExpectWithEnv(cmdArgs, cx.envMap, "OK"); err != nil {
|
||||
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "OK"); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -122,12 +123,12 @@ func dialWithSchemeTest(cx ctlCtx) {
|
||||
type ctlCtx struct {
|
||||
t *testing.T
|
||||
apiPrefix string
|
||||
cfg etcdProcessClusterConfig
|
||||
cfg e2e.EtcdProcessClusterConfig
|
||||
quotaBackendBytes int64
|
||||
corruptFunc func(string) error
|
||||
noStrictReconfig bool
|
||||
|
||||
epc *etcdProcessCluster
|
||||
epc *e2e.EtcdProcessCluster
|
||||
|
||||
envMap map[string]string
|
||||
|
||||
@ -161,7 +162,7 @@ func (cx *ctlCtx) applyOpts(opts []ctlOption) {
|
||||
cx.initialCorruptCheck = true
|
||||
}
|
||||
|
||||
func withCfg(cfg etcdProcessClusterConfig) ctlOption {
|
||||
func withCfg(cfg e2e.EtcdProcessClusterConfig) ctlOption {
|
||||
return func(cx *ctlCtx) { cx.cfg = cfg }
|
||||
}
|
||||
|
||||
@ -225,13 +226,13 @@ func withMaxConcurrentStreams(streams uint32) ctlOption {
|
||||
// may be overwritten by `withCfg`.
|
||||
func withSnapshotCount(snapshotCount int) ctlOption {
|
||||
return func(cx *ctlCtx) {
|
||||
cx.cfg.snapshotCount = snapshotCount
|
||||
cx.cfg.SnapshotCount = snapshotCount
|
||||
}
|
||||
}
|
||||
|
||||
func withLogLevel(logLevel string) ctlOption {
|
||||
return func(cx *ctlCtx) {
|
||||
cx.cfg.logLevel = logLevel
|
||||
cx.cfg.LogLevel = logLevel
|
||||
}
|
||||
}
|
||||
|
||||
@ -242,37 +243,37 @@ func testCtl(t *testing.T, testFunc func(ctlCtx), opts ...ctlOption) {
|
||||
func getDefaultCtlCtx(t *testing.T) ctlCtx {
|
||||
return ctlCtx{
|
||||
t: t,
|
||||
cfg: *newConfigAutoTLS(),
|
||||
cfg: *e2e.NewConfigAutoTLS(),
|
||||
dialTimeout: 7 * time.Second,
|
||||
}
|
||||
}
|
||||
|
||||
func testCtlWithOffline(t *testing.T, testFunc func(ctlCtx), testOfflineFunc func(ctlCtx), opts ...ctlOption) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
ret := getDefaultCtlCtx(t)
|
||||
ret.applyOpts(opts)
|
||||
|
||||
if !ret.quorum {
|
||||
ret.cfg = *configStandalone(ret.cfg)
|
||||
ret.cfg = *e2e.ConfigStandalone(ret.cfg)
|
||||
}
|
||||
if ret.quotaBackendBytes > 0 {
|
||||
ret.cfg.quotaBackendBytes = ret.quotaBackendBytes
|
||||
ret.cfg.QuotaBackendBytes = ret.quotaBackendBytes
|
||||
}
|
||||
ret.cfg.noStrictReconfig = ret.noStrictReconfig
|
||||
ret.cfg.NoStrictReconfig = ret.noStrictReconfig
|
||||
if ret.initialCorruptCheck {
|
||||
ret.cfg.initialCorruptCheck = ret.initialCorruptCheck
|
||||
ret.cfg.InitialCorruptCheck = ret.initialCorruptCheck
|
||||
}
|
||||
if testOfflineFunc != nil {
|
||||
ret.cfg.keepDataDir = true
|
||||
ret.cfg.KeepDataDir = true
|
||||
}
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, &ret.cfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &ret.cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
ret.epc = epc
|
||||
ret.dataDir = epc.procs[0].Config().dataDirPath
|
||||
ret.dataDir = epc.Procs[0].Config().DataDirPath
|
||||
|
||||
defer func() {
|
||||
if ret.envMap != nil {
|
||||
@ -328,18 +329,18 @@ func (cx *ctlCtx) prefixArgs(eps []string) []string {
|
||||
fmap := make(map[string]string)
|
||||
fmap["endpoints"] = strings.Join(eps, ",")
|
||||
fmap["dial-timeout"] = cx.dialTimeout.String()
|
||||
if cx.epc.cfg.clientTLS == clientTLS {
|
||||
if cx.epc.cfg.isClientAutoTLS {
|
||||
if cx.epc.Cfg.ClientTLS == e2e.ClientTLS {
|
||||
if cx.epc.Cfg.IsClientAutoTLS {
|
||||
fmap["insecure-transport"] = "false"
|
||||
fmap["insecure-skip-tls-verify"] = "true"
|
||||
} else if cx.epc.cfg.isClientCRL {
|
||||
fmap["cacert"] = caPath
|
||||
fmap["cert"] = revokedCertPath
|
||||
fmap["key"] = revokedPrivateKeyPath
|
||||
} else if cx.epc.Cfg.IsClientCRL {
|
||||
fmap["cacert"] = e2e.CaPath
|
||||
fmap["cert"] = e2e.RevokedCertPath
|
||||
fmap["key"] = e2e.RevokedPrivateKeyPath
|
||||
} else {
|
||||
fmap["cacert"] = caPath
|
||||
fmap["cert"] = certPath
|
||||
fmap["key"] = privateKeyPath
|
||||
fmap["cacert"] = e2e.CaPath
|
||||
fmap["cert"] = e2e.CertPath
|
||||
fmap["key"] = e2e.PrivateKeyPath
|
||||
}
|
||||
}
|
||||
if cx.user != "" {
|
||||
@ -348,7 +349,7 @@ func (cx *ctlCtx) prefixArgs(eps []string) []string {
|
||||
|
||||
useEnv := cx.envMap != nil
|
||||
|
||||
cmdArgs := []string{ctlBinPath + "3"}
|
||||
cmdArgs := []string{e2e.CtlBinPath + "3"}
|
||||
for k, v := range fmap {
|
||||
if useEnv {
|
||||
ek := flags.FlagToEnv("ETCDCTL", k)
|
||||
@ -371,9 +372,9 @@ func (cx *ctlCtx) PrefixArgs() []string {
|
||||
// Please not thet 'utl' compatible commands does not consume --endpoints flag.
|
||||
func (cx *ctlCtx) PrefixArgsUtl() []string {
|
||||
if cx.etcdutl {
|
||||
return []string{utlBinPath}
|
||||
return []string{e2e.UtlBinPath}
|
||||
}
|
||||
return []string{ctlBinPath}
|
||||
return []string{e2e.CtlBinPath}
|
||||
}
|
||||
|
||||
func isGRPCTimedout(err error) bool {
|
||||
@ -381,7 +382,7 @@ func isGRPCTimedout(err error) bool {
|
||||
}
|
||||
|
||||
func (cx *ctlCtx) memberToRemove() (ep string, memberID string, clusterID string) {
|
||||
n1 := cx.cfg.clusterSize
|
||||
n1 := cx.cfg.ClusterSize
|
||||
if n1 < 2 {
|
||||
cx.t.Fatalf("%d-node is too small to test 'member remove'", n1)
|
||||
}
|
||||
|
@ -14,19 +14,23 @@
|
||||
|
||||
package e2e
|
||||
|
||||
import "testing"
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3TxnInteractiveSuccess(t *testing.T) {
|
||||
testCtl(t, txnTestSuccess, withInteractive())
|
||||
}
|
||||
func TestCtlV3TxnInteractiveSuccessNoTLS(t *testing.T) {
|
||||
testCtl(t, txnTestSuccess, withInteractive(), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, txnTestSuccess, withInteractive(), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3TxnInteractiveSuccessClientTLS(t *testing.T) {
|
||||
testCtl(t, txnTestSuccess, withInteractive(), withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, txnTestSuccess, withInteractive(), withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3TxnInteractiveSuccessPeerTLS(t *testing.T) {
|
||||
testCtl(t, txnTestSuccess, withInteractive(), withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, txnTestSuccess, withInteractive(), withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
func TestCtlV3TxnInteractiveFail(t *testing.T) {
|
||||
testCtl(t, txnTestFail, withInteractive())
|
||||
@ -102,7 +106,7 @@ func ctlV3Txn(cx ctlCtx, rqs txnRequests) error {
|
||||
if cx.interactive {
|
||||
cmdArgs = append(cmdArgs, "--interactive")
|
||||
}
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -14,44 +14,56 @@
|
||||
|
||||
package e2e
|
||||
|
||||
import "testing"
|
||||
import (
|
||||
"testing"
|
||||
|
||||
func TestCtlV3UserAdd(t *testing.T) { testCtl(t, userAddTest) }
|
||||
func TestCtlV3UserAddNoTLS(t *testing.T) { testCtl(t, userAddTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3UserAddClientTLS(t *testing.T) { testCtl(t, userAddTest, withCfg(*newConfigClientTLS())) }
|
||||
func TestCtlV3UserAddPeerTLS(t *testing.T) { testCtl(t, userAddTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3UserAddTimeout(t *testing.T) { testCtl(t, userAddTest, withDialTimeout(0)) }
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3UserAdd(t *testing.T) { testCtl(t, userAddTest) }
|
||||
func TestCtlV3UserAddNoTLS(t *testing.T) { testCtl(t, userAddTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3UserAddClientTLS(t *testing.T) {
|
||||
testCtl(t, userAddTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3UserAddPeerTLS(t *testing.T) { testCtl(t, userAddTest, withCfg(*e2e.NewConfigPeerTLS())) }
|
||||
func TestCtlV3UserAddTimeout(t *testing.T) { testCtl(t, userAddTest, withDialTimeout(0)) }
|
||||
func TestCtlV3UserAddClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, userAddTest, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, userAddTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3UserList(t *testing.T) { testCtl(t, userListTest) }
|
||||
func TestCtlV3UserListNoTLS(t *testing.T) { testCtl(t, userListTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3UserListNoTLS(t *testing.T) { testCtl(t, userListTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3UserListClientTLS(t *testing.T) {
|
||||
testCtl(t, userListTest, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, userListTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3UserListPeerTLS(t *testing.T) {
|
||||
testCtl(t, userListTest, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
func TestCtlV3UserListPeerTLS(t *testing.T) { testCtl(t, userListTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3UserListClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, userListTest, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, userListTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3UserDelete(t *testing.T) { testCtl(t, userDelTest) }
|
||||
func TestCtlV3UserDeleteNoTLS(t *testing.T) { testCtl(t, userDelTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3UserDeleteNoTLS(t *testing.T) { testCtl(t, userDelTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3UserDeleteClientTLS(t *testing.T) {
|
||||
testCtl(t, userDelTest, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, userDelTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3UserDeletePeerTLS(t *testing.T) {
|
||||
testCtl(t, userDelTest, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
func TestCtlV3UserDeletePeerTLS(t *testing.T) { testCtl(t, userDelTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3UserDeleteClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, userDelTest, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, userDelTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
func TestCtlV3UserPasswd(t *testing.T) { testCtl(t, userPasswdTest) }
|
||||
func TestCtlV3UserPasswdNoTLS(t *testing.T) {
|
||||
testCtl(t, userPasswdTest, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3UserPasswd(t *testing.T) { testCtl(t, userPasswdTest) }
|
||||
func TestCtlV3UserPasswdNoTLS(t *testing.T) { testCtl(t, userPasswdTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3UserPasswdClientTLS(t *testing.T) {
|
||||
testCtl(t, userPasswdTest, withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, userPasswdTest, withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3UserPasswdPeerTLS(t *testing.T) {
|
||||
testCtl(t, userPasswdTest, withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, userPasswdTest, withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
func TestCtlV3UserPasswdClientAutoTLS(t *testing.T) {
|
||||
testCtl(t, userPasswdTest, withCfg(*newConfigClientAutoTLS()))
|
||||
testCtl(t, userPasswdTest, withCfg(*e2e.NewConfigClientAutoTLS()))
|
||||
}
|
||||
|
||||
type userCmdDesc struct {
|
||||
@ -179,7 +191,7 @@ func ctlV3User(cx ctlCtx, args []string, expStr string, stdIn []string) error {
|
||||
cmdArgs := append(cx.PrefixArgs(), "user")
|
||||
cmdArgs = append(cmdArgs, args...)
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -20,25 +20,27 @@ package e2e
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Watch(t *testing.T) { testCtl(t, watchTest) }
|
||||
func TestCtlV3WatchNoTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3WatchClientTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*newConfigClientTLS())) }
|
||||
func TestCtlV3WatchPeerTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3WatchNoTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3WatchClientTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*e2e.NewConfigClientTLS())) }
|
||||
func TestCtlV3WatchPeerTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*e2e.NewConfigPeerTLS())) }
|
||||
func TestCtlV3WatchTimeout(t *testing.T) { testCtl(t, watchTest, withDialTimeout(0)) }
|
||||
|
||||
func TestCtlV3WatchInteractive(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive())
|
||||
}
|
||||
func TestCtlV3WatchInteractiveNoTLS(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3WatchInteractiveClientTLS(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3WatchInteractivePeerTLS(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func watchTest(cx ctlCtx) {
|
||||
|
@ -20,25 +20,27 @@ package e2e
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestCtlV3Watch(t *testing.T) { testCtl(t, watchTest) }
|
||||
func TestCtlV3WatchNoTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*newConfigNoTLS())) }
|
||||
func TestCtlV3WatchClientTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*newConfigClientTLS())) }
|
||||
func TestCtlV3WatchPeerTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*newConfigPeerTLS())) }
|
||||
func TestCtlV3WatchNoTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*e2e.NewConfigNoTLS())) }
|
||||
func TestCtlV3WatchClientTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*e2e.NewConfigClientTLS())) }
|
||||
func TestCtlV3WatchPeerTLS(t *testing.T) { testCtl(t, watchTest, withCfg(*e2e.NewConfigPeerTLS())) }
|
||||
func TestCtlV3WatchTimeout(t *testing.T) { testCtl(t, watchTest, withDialTimeout(0)) }
|
||||
|
||||
func TestCtlV3WatchInteractive(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive())
|
||||
}
|
||||
func TestCtlV3WatchInteractiveNoTLS(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
func TestCtlV3WatchInteractiveClientTLS(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
func TestCtlV3WatchInteractivePeerTLS(t *testing.T) {
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, watchTest, withInteractive(), withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
|
||||
func watchTest(cx ctlCtx) {
|
||||
|
@ -14,7 +14,11 @@
|
||||
|
||||
package e2e
|
||||
|
||||
import "strings"
|
||||
import (
|
||||
"strings"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
type kvExec struct {
|
||||
key, val string
|
||||
@ -35,7 +39,7 @@ func setupWatchArgs(cx ctlCtx, args []string) []string {
|
||||
func ctlV3Watch(cx ctlCtx, args []string, kvs ...kvExec) error {
|
||||
cmdArgs := setupWatchArgs(cx, args)
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, nil)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -66,7 +70,7 @@ func ctlV3Watch(cx ctlCtx, args []string, kvs ...kvExec) error {
|
||||
func ctlV3WatchFailPerm(cx ctlCtx, args []string) error {
|
||||
cmdArgs := setupWatchArgs(cx, args)
|
||||
|
||||
proc, err := spawnCmd(cmdArgs, nil)
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -23,18 +23,19 @@ import (
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
const exampleConfigFile = "../../etcd.conf.yml.sample"
|
||||
|
||||
func TestEtcdExampleConfig(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
proc, err := spawnCmd([]string{binDir + "/etcd", "--config-file", exampleConfigFile}, nil)
|
||||
proc, err := e2e.SpawnCmd([]string{e2e.BinDir + "/etcd", "--config-file", exampleConfigFile}, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = waitReadyExpectProc(proc, etcdServerReadyLines); err != nil {
|
||||
if err = e2e.WaitReadyExpectProc(proc, e2e.EtcdServerReadyLines); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = proc.Stop(); err != nil {
|
||||
@ -43,11 +44,11 @@ func TestEtcdExampleConfig(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestEtcdMultiPeer(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
peers, tmpdirs := make([]string, 3), make([]string, 3)
|
||||
for i := range peers {
|
||||
peers[i] = fmt.Sprintf("e%d=http://127.0.0.1:%d", i, etcdProcessBasePort+i)
|
||||
peers[i] = fmt.Sprintf("e%d=http://127.0.0.1:%d", i, e2e.EtcdProcessBasePort+i)
|
||||
d, err := ioutil.TempDir("", fmt.Sprintf("e%d.etcd", i))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -67,16 +68,16 @@ func TestEtcdMultiPeer(t *testing.T) {
|
||||
}()
|
||||
for i := range procs {
|
||||
args := []string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"--name", fmt.Sprintf("e%d", i),
|
||||
"--listen-client-urls", "http://0.0.0.0:0",
|
||||
"--data-dir", tmpdirs[i],
|
||||
"--advertise-client-urls", "http://0.0.0.0:0",
|
||||
"--listen-peer-urls", fmt.Sprintf("http://127.0.0.1:%d,http://127.0.0.1:%d", etcdProcessBasePort+i, etcdProcessBasePort+len(peers)+i),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("http://127.0.0.1:%d", etcdProcessBasePort+i),
|
||||
"--listen-peer-urls", fmt.Sprintf("http://127.0.0.1:%d,http://127.0.0.1:%d", e2e.EtcdProcessBasePort+i, e2e.EtcdProcessBasePort+len(peers)+i),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("http://127.0.0.1:%d", e2e.EtcdProcessBasePort+i),
|
||||
"--initial-cluster", ic,
|
||||
}
|
||||
p, err := spawnCmd(args, nil)
|
||||
p, err := e2e.SpawnCmd(args, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -84,7 +85,7 @@ func TestEtcdMultiPeer(t *testing.T) {
|
||||
}
|
||||
|
||||
for _, p := range procs {
|
||||
if err := waitReadyExpectProc(p, etcdServerReadyLines); err != nil {
|
||||
if err := e2e.WaitReadyExpectProc(p, e2e.EtcdServerReadyLines); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -92,16 +93,16 @@ func TestEtcdMultiPeer(t *testing.T) {
|
||||
|
||||
// TestEtcdUnixPeers checks that etcd will boot with unix socket peers.
|
||||
func TestEtcdUnixPeers(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
d, err := ioutil.TempDir("", "e1.etcd")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer os.RemoveAll(d)
|
||||
proc, err := spawnCmd(
|
||||
proc, err := e2e.SpawnCmd(
|
||||
[]string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"--data-dir", d,
|
||||
"--name", "e1",
|
||||
"--listen-peer-urls", "unix://etcd.unix:1",
|
||||
@ -113,7 +114,7 @@ func TestEtcdUnixPeers(t *testing.T) {
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = waitReadyExpectProc(proc, etcdServerReadyLines); err != nil {
|
||||
if err = e2e.WaitReadyExpectProc(proc, e2e.EtcdServerReadyLines); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = proc.Stop(); err != nil {
|
||||
@ -123,11 +124,11 @@ func TestEtcdUnixPeers(t *testing.T) {
|
||||
|
||||
// TestEtcdPeerCNAuth checks that the inter peer auth based on CN of cert is working correctly.
|
||||
func TestEtcdPeerCNAuth(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
peers, tmpdirs := make([]string, 3), make([]string, 3)
|
||||
for i := range peers {
|
||||
peers[i] = fmt.Sprintf("e%d=https://127.0.0.1:%d", i, etcdProcessBasePort+i)
|
||||
peers[i] = fmt.Sprintf("e%d=https://127.0.0.1:%d", i, e2e.EtcdProcessBasePort+i)
|
||||
d, err := ioutil.TempDir("", fmt.Sprintf("e%d.etcd", i))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -149,34 +150,34 @@ func TestEtcdPeerCNAuth(t *testing.T) {
|
||||
// node 0 and 1 have a cert with the correct CN, node 2 doesn't
|
||||
for i := range procs {
|
||||
commonArgs := []string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"--name", fmt.Sprintf("e%d", i),
|
||||
"--listen-client-urls", "http://0.0.0.0:0",
|
||||
"--data-dir", tmpdirs[i],
|
||||
"--advertise-client-urls", "http://0.0.0.0:0",
|
||||
"--listen-peer-urls", fmt.Sprintf("https://127.0.0.1:%d,https://127.0.0.1:%d", etcdProcessBasePort+i, etcdProcessBasePort+len(peers)+i),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", etcdProcessBasePort+i),
|
||||
"--listen-peer-urls", fmt.Sprintf("https://127.0.0.1:%d,https://127.0.0.1:%d", e2e.EtcdProcessBasePort+i, e2e.EtcdProcessBasePort+len(peers)+i),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", e2e.EtcdProcessBasePort+i),
|
||||
"--initial-cluster", ic,
|
||||
}
|
||||
|
||||
var args []string
|
||||
if i <= 1 {
|
||||
args = []string{
|
||||
"--peer-cert-file", certPath,
|
||||
"--peer-key-file", privateKeyPath,
|
||||
"--peer-client-cert-file", certPath,
|
||||
"--peer-client-key-file", privateKeyPath,
|
||||
"--peer-trusted-ca-file", caPath,
|
||||
"--peer-cert-file", e2e.CertPath,
|
||||
"--peer-key-file", e2e.PrivateKeyPath,
|
||||
"--peer-client-cert-file", e2e.CertPath,
|
||||
"--peer-client-key-file", e2e.PrivateKeyPath,
|
||||
"--peer-trusted-ca-file", e2e.CaPath,
|
||||
"--peer-client-cert-auth",
|
||||
"--peer-cert-allowed-cn", "example.com",
|
||||
}
|
||||
} else {
|
||||
args = []string{
|
||||
"--peer-cert-file", certPath2,
|
||||
"--peer-key-file", privateKeyPath2,
|
||||
"--peer-client-cert-file", certPath2,
|
||||
"--peer-client-key-file", privateKeyPath2,
|
||||
"--peer-trusted-ca-file", caPath,
|
||||
"--peer-cert-file", e2e.CertPath2,
|
||||
"--peer-key-file", e2e.PrivateKeyPath2,
|
||||
"--peer-client-cert-file", e2e.CertPath2,
|
||||
"--peer-client-key-file", e2e.PrivateKeyPath2,
|
||||
"--peer-trusted-ca-file", e2e.CaPath,
|
||||
"--peer-client-cert-auth",
|
||||
"--peer-cert-allowed-cn", "example2.com",
|
||||
}
|
||||
@ -184,7 +185,7 @@ func TestEtcdPeerCNAuth(t *testing.T) {
|
||||
|
||||
commonArgs = append(commonArgs, args...)
|
||||
|
||||
p, err := spawnCmd(commonArgs, nil)
|
||||
p, err := e2e.SpawnCmd(commonArgs, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -194,11 +195,11 @@ func TestEtcdPeerCNAuth(t *testing.T) {
|
||||
for i, p := range procs {
|
||||
var expect []string
|
||||
if i <= 1 {
|
||||
expect = etcdServerReadyLines
|
||||
expect = e2e.EtcdServerReadyLines
|
||||
} else {
|
||||
expect = []string{"remote error: tls: bad certificate"}
|
||||
}
|
||||
if err := waitReadyExpectProc(p, expect); err != nil {
|
||||
if err := e2e.WaitReadyExpectProc(p, expect); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -206,11 +207,11 @@ func TestEtcdPeerCNAuth(t *testing.T) {
|
||||
|
||||
// TestEtcdPeerNameAuth checks that the inter peer auth based on cert name validation is working correctly.
|
||||
func TestEtcdPeerNameAuth(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
peers, tmpdirs := make([]string, 3), make([]string, 3)
|
||||
for i := range peers {
|
||||
peers[i] = fmt.Sprintf("e%d=https://127.0.0.1:%d", i, etcdProcessBasePort+i)
|
||||
peers[i] = fmt.Sprintf("e%d=https://127.0.0.1:%d", i, e2e.EtcdProcessBasePort+i)
|
||||
d, err := ioutil.TempDir("", fmt.Sprintf("e%d.etcd", i))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -232,30 +233,30 @@ func TestEtcdPeerNameAuth(t *testing.T) {
|
||||
// node 0 and 1 have a cert with the correct certificate name, node 2 doesn't
|
||||
for i := range procs {
|
||||
commonArgs := []string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"--name", fmt.Sprintf("e%d", i),
|
||||
"--listen-client-urls", "http://0.0.0.0:0",
|
||||
"--data-dir", tmpdirs[i],
|
||||
"--advertise-client-urls", "http://0.0.0.0:0",
|
||||
"--listen-peer-urls", fmt.Sprintf("https://127.0.0.1:%d,https://127.0.0.1:%d", etcdProcessBasePort+i, etcdProcessBasePort+len(peers)+i),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", etcdProcessBasePort+i),
|
||||
"--listen-peer-urls", fmt.Sprintf("https://127.0.0.1:%d,https://127.0.0.1:%d", e2e.EtcdProcessBasePort+i, e2e.EtcdProcessBasePort+len(peers)+i),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", e2e.EtcdProcessBasePort+i),
|
||||
"--initial-cluster", ic,
|
||||
}
|
||||
|
||||
var args []string
|
||||
if i <= 1 {
|
||||
args = []string{
|
||||
"--peer-cert-file", certPath,
|
||||
"--peer-key-file", privateKeyPath,
|
||||
"--peer-trusted-ca-file", caPath,
|
||||
"--peer-cert-file", e2e.CertPath,
|
||||
"--peer-key-file", e2e.PrivateKeyPath,
|
||||
"--peer-trusted-ca-file", e2e.CaPath,
|
||||
"--peer-client-cert-auth",
|
||||
"--peer-cert-allowed-hostname", "localhost",
|
||||
}
|
||||
} else {
|
||||
args = []string{
|
||||
"--peer-cert-file", certPath2,
|
||||
"--peer-key-file", privateKeyPath2,
|
||||
"--peer-trusted-ca-file", caPath,
|
||||
"--peer-cert-file", e2e.CertPath2,
|
||||
"--peer-key-file", e2e.PrivateKeyPath2,
|
||||
"--peer-trusted-ca-file", e2e.CaPath,
|
||||
"--peer-client-cert-auth",
|
||||
"--peer-cert-allowed-hostname", "example2.com",
|
||||
}
|
||||
@ -263,7 +264,7 @@ func TestEtcdPeerNameAuth(t *testing.T) {
|
||||
|
||||
commonArgs = append(commonArgs, args...)
|
||||
|
||||
p, err := spawnCmd(commonArgs, nil)
|
||||
p, err := e2e.SpawnCmd(commonArgs, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -273,43 +274,43 @@ func TestEtcdPeerNameAuth(t *testing.T) {
|
||||
for i, p := range procs {
|
||||
var expect []string
|
||||
if i <= 1 {
|
||||
expect = etcdServerReadyLines
|
||||
expect = e2e.EtcdServerReadyLines
|
||||
} else {
|
||||
expect = []string{"client certificate authentication failed"}
|
||||
}
|
||||
if err := waitReadyExpectProc(p, expect); err != nil {
|
||||
if err := e2e.WaitReadyExpectProc(p, expect); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestGrpcproxyAndCommonName(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
argsWithNonEmptyCN := []string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"grpc-proxy",
|
||||
"start",
|
||||
"--cert", certPath2,
|
||||
"--key", privateKeyPath2,
|
||||
"--cacert", caPath,
|
||||
"--cert", e2e.CertPath2,
|
||||
"--key", e2e.PrivateKeyPath2,
|
||||
"--cacert", e2e.CaPath,
|
||||
}
|
||||
|
||||
argsWithEmptyCN := []string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"grpc-proxy",
|
||||
"start",
|
||||
"--cert", certPath3,
|
||||
"--key", privateKeyPath3,
|
||||
"--cacert", caPath,
|
||||
"--cert", e2e.CertPath3,
|
||||
"--key", e2e.PrivateKeyPath3,
|
||||
"--cacert", e2e.CaPath,
|
||||
}
|
||||
|
||||
err := spawnWithExpect(argsWithNonEmptyCN, "cert has non empty Common Name")
|
||||
err := e2e.SpawnWithExpect(argsWithNonEmptyCN, "cert has non empty Common Name")
|
||||
if err != nil {
|
||||
t.Errorf("Unexpected error: %s", err)
|
||||
}
|
||||
|
||||
p, err := spawnCmd(argsWithEmptyCN, nil)
|
||||
p, err := e2e.SpawnCmd(argsWithEmptyCN, nil)
|
||||
defer func() {
|
||||
if p != nil {
|
||||
p.Stop()
|
||||
@ -322,7 +323,7 @@ func TestGrpcproxyAndCommonName(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestGrpcproxyAndListenCipherSuite(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
cases := []struct {
|
||||
name string
|
||||
@ -331,7 +332,7 @@ func TestGrpcproxyAndListenCipherSuite(t *testing.T) {
|
||||
{
|
||||
name: "ArgsWithCipherSuites",
|
||||
args: []string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"grpc-proxy",
|
||||
"start",
|
||||
"--listen-cipher-suites", "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256,TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
|
||||
@ -340,7 +341,7 @@ func TestGrpcproxyAndListenCipherSuite(t *testing.T) {
|
||||
{
|
||||
name: "ArgsWithoutCipherSuites",
|
||||
args: []string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"grpc-proxy",
|
||||
"start",
|
||||
"--listen-cipher-suites", "",
|
||||
@ -350,7 +351,7 @@ func TestGrpcproxyAndListenCipherSuite(t *testing.T) {
|
||||
|
||||
for _, test := range cases {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
pw, err := spawnCmd(test.args, nil)
|
||||
pw, err := e2e.SpawnCmd(test.args, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -362,13 +363,13 @@ func TestGrpcproxyAndListenCipherSuite(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestBootstrapDefragFlag(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
proc, err := spawnCmd([]string{binDir + "/etcd", "--experimental-bootstrap-defrag-threshold-megabytes", "1000"}, nil)
|
||||
proc, err := e2e.SpawnCmd([]string{e2e.BinDir + "/etcd", "--experimental-bootstrap-defrag-threshold-megabytes", "1000"}, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = waitReadyExpectProc(proc, []string{"Skipping defragmentation"}); err != nil {
|
||||
if err = e2e.WaitReadyExpectProc(proc, []string{"Skipping defragmentation"}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err = proc.Stop(); err != nil {
|
||||
@ -377,30 +378,30 @@ func TestBootstrapDefragFlag(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestEtcdTLSVersion(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
d := t.TempDir()
|
||||
proc, err := spawnCmd(
|
||||
proc, err := e2e.SpawnCmd(
|
||||
[]string{
|
||||
binDir + "/etcd",
|
||||
e2e.BinDir + "/etcd",
|
||||
"--data-dir", d,
|
||||
"--name", "e1",
|
||||
"--listen-client-urls", "https://0.0.0.0:0",
|
||||
"--advertise-client-urls", "https://0.0.0.0:0",
|
||||
"--listen-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", etcdProcessBasePort),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", etcdProcessBasePort),
|
||||
"--initial-cluster", fmt.Sprintf("e1=https://127.0.0.1:%d", etcdProcessBasePort),
|
||||
"--peer-cert-file", certPath,
|
||||
"--peer-key-file", privateKeyPath,
|
||||
"--cert-file", certPath2,
|
||||
"--key-file", privateKeyPath2,
|
||||
"--listen-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", e2e.EtcdProcessBasePort),
|
||||
"--initial-advertise-peer-urls", fmt.Sprintf("https://127.0.0.1:%d", e2e.EtcdProcessBasePort),
|
||||
"--initial-cluster", fmt.Sprintf("e1=https://127.0.0.1:%d", e2e.EtcdProcessBasePort),
|
||||
"--peer-cert-file", e2e.CertPath,
|
||||
"--peer-key-file", e2e.PrivateKeyPath,
|
||||
"--cert-file", e2e.CertPath2,
|
||||
"--key-file", e2e.PrivateKeyPath2,
|
||||
|
||||
"--tls-min-version", "TLS1.2",
|
||||
"--tls-max-version", "TLS1.3",
|
||||
}, nil,
|
||||
)
|
||||
assert.NoError(t, err)
|
||||
assert.NoError(t, waitReadyExpectProc(proc, etcdServerReadyLines), "did not receive expected output from etcd process")
|
||||
assert.NoError(t, e2e.WaitReadyExpectProc(proc, e2e.EtcdServerReadyLines), "did not receive expected output from etcd process")
|
||||
assert.NoError(t, proc.Stop())
|
||||
|
||||
}
|
||||
|
@ -27,10 +27,11 @@ import (
|
||||
"go.etcd.io/etcd/api/v3/etcdserverpb"
|
||||
"go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestGrpcProxyAutoSync(t *testing.T) {
|
||||
skipInShortMode(t)
|
||||
e2e.SkipInShortMode(t)
|
||||
|
||||
var (
|
||||
node1Name = "node1"
|
||||
@ -55,17 +56,17 @@ func TestGrpcProxyAutoSync(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
// Run grpc-proxy instance
|
||||
proxyProc, err := spawnCmd([]string{binDir + "/etcd", "grpc-proxy", "start",
|
||||
proxyProc, err := e2e.SpawnCmd([]string{e2e.BinDir + "/etcd", "grpc-proxy", "start",
|
||||
"--advertise-client-url", proxyClientURL, "--listen-addr", proxyClientURL,
|
||||
"--endpoints", node1ClientURL,
|
||||
"--endpoints-auto-sync-interval", autoSyncInterval.String(),
|
||||
}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
err = spawnWithExpect([]string{ctlBinPath, "--endpoints", proxyClientURL, "put", "k1", "v1"}, "OK")
|
||||
err = e2e.SpawnWithExpect([]string{e2e.CtlBinPath, "--endpoints", proxyClientURL, "put", "k1", "v1"}, "OK")
|
||||
require.NoError(t, err)
|
||||
|
||||
err = spawnWithExpect([]string{ctlBinPath, "--endpoints", node1ClientURL, "member", "add", node2Name, "--peer-urls", node2PeerURL}, "added")
|
||||
err = e2e.SpawnWithExpect([]string{e2e.CtlBinPath, "--endpoints", node1ClientURL, "member", "add", node2Name, "--peer-urls", node2PeerURL}, "added")
|
||||
require.NoError(t, err)
|
||||
|
||||
// Run new member
|
||||
@ -93,7 +94,7 @@ func TestGrpcProxyAutoSync(t *testing.T) {
|
||||
|
||||
// Second node could be not ready yet
|
||||
for i := 0; i < 10; i++ {
|
||||
err = spawnWithExpect([]string{ctlBinPath, "--endpoints", node2ClientURL, "member", "remove", fmt.Sprintf("%x", node1MemberID)}, "removed")
|
||||
err = e2e.SpawnWithExpect([]string{e2e.CtlBinPath, "--endpoints", node2ClientURL, "member", "remove", fmt.Sprintf("%x", node1MemberID)}, "removed")
|
||||
if err != nil && strings.Contains(err.Error(), rpctypes.ErrGRPCUnhealthy.Error()) {
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
continue
|
||||
@ -111,7 +112,7 @@ func TestGrpcProxyAutoSync(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
err = spawnWithExpect([]string{ctlBinPath, "--endpoints", proxyClientURL, "get", "k1"}, "v1")
|
||||
err = e2e.SpawnWithExpect([]string{e2e.CtlBinPath, "--endpoints", proxyClientURL, "get", "k1"}, "v1")
|
||||
if err != nil && (strings.Contains(err.Error(), rpctypes.ErrGRPCLeaderChanged.Error()) ||
|
||||
strings.Contains(err.Error(), context.DeadlineExceeded.Error())) {
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
@ -126,7 +127,7 @@ func TestGrpcProxyAutoSync(t *testing.T) {
|
||||
}
|
||||
|
||||
func runEtcdNode(name, dataDir, clientURL, peerURL, clusterState, initialCluster string) (*expect.ExpectProcess, error) {
|
||||
proc, err := spawnCmd([]string{binPath,
|
||||
proc, err := e2e.SpawnCmd([]string{e2e.BinPath,
|
||||
"--name", name,
|
||||
"--data-dir", dataDir,
|
||||
"--listen-client-urls", clientURL, "--advertise-client-urls", clientURL,
|
||||
@ -155,7 +156,7 @@ func findMemberIDByEndpoint(members []*etcdserverpb.Member, endpoint string) (ui
|
||||
}
|
||||
|
||||
func getMemberListFromEndpoint(endpoint string) (etcdserverpb.MemberListResponse, error) {
|
||||
proc, err := spawnCmd([]string{ctlBinPath, "--endpoints", endpoint, "member", "list", "--write-out", "json"}, nil)
|
||||
proc, err := e2e.SpawnCmd([]string{e2e.CtlBinPath, "--endpoints", endpoint, "member", "list", "--write-out", "json"}, nil)
|
||||
if err != nil {
|
||||
return etcdserverpb.MemberListResponse{}, err
|
||||
}
|
||||
|
@ -1,190 +0,0 @@
|
||||
// Copyright 2017 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/url"
|
||||
"os"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
var (
|
||||
etcdServerReadyLines = []string{"ready to serve client requests"}
|
||||
binPath string
|
||||
ctlBinPath string
|
||||
utlBinPath string
|
||||
)
|
||||
|
||||
// etcdProcess is a process that serves etcd requests.
|
||||
type etcdProcess interface {
|
||||
EndpointsV2() []string
|
||||
EndpointsV3() []string
|
||||
EndpointsGRPC() []string
|
||||
EndpointsHTTP() []string
|
||||
EndpointsMetrics() []string
|
||||
|
||||
Start() error
|
||||
Restart() error
|
||||
Stop() error
|
||||
Close() error
|
||||
WithStopSignal(sig os.Signal) os.Signal
|
||||
Config() *etcdServerProcessConfig
|
||||
Logs() logsExpect
|
||||
}
|
||||
|
||||
type logsExpect interface {
|
||||
Expect(string) (string, error)
|
||||
Lines() []string
|
||||
LineCount() int
|
||||
}
|
||||
|
||||
type etcdServerProcess struct {
|
||||
cfg *etcdServerProcessConfig
|
||||
proc *expect.ExpectProcess
|
||||
donec chan struct{} // closed when Interact() terminates
|
||||
}
|
||||
|
||||
type etcdServerProcessConfig struct {
|
||||
lg *zap.Logger
|
||||
execPath string
|
||||
args []string
|
||||
tlsArgs []string
|
||||
envVars map[string]string
|
||||
|
||||
dataDirPath string
|
||||
keepDataDir bool
|
||||
|
||||
name string
|
||||
|
||||
purl url.URL
|
||||
|
||||
acurl string
|
||||
murl string
|
||||
clientHttpUrl string
|
||||
|
||||
initialToken string
|
||||
initialCluster string
|
||||
}
|
||||
|
||||
func newEtcdServerProcess(cfg *etcdServerProcessConfig) (*etcdServerProcess, error) {
|
||||
if !fileutil.Exist(cfg.execPath) {
|
||||
return nil, fmt.Errorf("could not find etcd binary: %s", cfg.execPath)
|
||||
}
|
||||
if !cfg.keepDataDir {
|
||||
if err := os.RemoveAll(cfg.dataDirPath); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return &etcdServerProcess{cfg: cfg, donec: make(chan struct{})}, nil
|
||||
}
|
||||
|
||||
func (ep *etcdServerProcess) EndpointsV2() []string { return ep.EndpointsHTTP() }
|
||||
func (ep *etcdServerProcess) EndpointsV3() []string { return ep.EndpointsGRPC() }
|
||||
func (ep *etcdServerProcess) EndpointsGRPC() []string { return []string{ep.cfg.acurl} }
|
||||
func (ep *etcdServerProcess) EndpointsHTTP() []string {
|
||||
if ep.cfg.clientHttpUrl == "" {
|
||||
return []string{ep.cfg.acurl}
|
||||
}
|
||||
return []string{ep.cfg.clientHttpUrl}
|
||||
}
|
||||
func (ep *etcdServerProcess) EndpointsMetrics() []string { return []string{ep.cfg.murl} }
|
||||
|
||||
func (ep *etcdServerProcess) Start() error {
|
||||
if ep.proc != nil {
|
||||
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...), ep.cfg.envVars)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
ep.proc = proc
|
||||
err = ep.waitReady()
|
||||
if err == nil {
|
||||
ep.cfg.lg.Info("started server.", zap.String("name", ep.cfg.name))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (ep *etcdServerProcess) Restart() error {
|
||||
ep.cfg.lg.Info("restaring server...", zap.String("name", ep.cfg.name))
|
||||
if err := ep.Stop(); err != nil {
|
||||
return err
|
||||
}
|
||||
ep.donec = make(chan struct{})
|
||||
err := ep.Start()
|
||||
if err == nil {
|
||||
ep.cfg.lg.Info("restared server", zap.String("name", ep.cfg.name))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (ep *etcdServerProcess) Stop() (err error) {
|
||||
ep.cfg.lg.Info("stoping server...", zap.String("name", ep.cfg.name))
|
||||
if ep == nil || ep.proc == nil {
|
||||
return nil
|
||||
}
|
||||
err = ep.proc.Stop()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
ep.proc = nil
|
||||
<-ep.donec
|
||||
ep.donec = make(chan struct{})
|
||||
if ep.cfg.purl.Scheme == "unix" || ep.cfg.purl.Scheme == "unixs" {
|
||||
err = os.Remove(ep.cfg.purl.Host + ep.cfg.purl.Path)
|
||||
if err != nil && !os.IsNotExist(err) {
|
||||
return err
|
||||
}
|
||||
}
|
||||
ep.cfg.lg.Info("stopped server.", zap.String("name", ep.cfg.name))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ep *etcdServerProcess) Close() error {
|
||||
ep.cfg.lg.Info("closing server...", zap.String("name", ep.cfg.name))
|
||||
if err := ep.Stop(); err != nil {
|
||||
return err
|
||||
}
|
||||
if !ep.cfg.keepDataDir {
|
||||
ep.cfg.lg.Info("removing directory", zap.String("data-dir", ep.cfg.dataDirPath))
|
||||
return os.RemoveAll(ep.cfg.dataDirPath)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ep *etcdServerProcess) WithStopSignal(sig os.Signal) os.Signal {
|
||||
ret := ep.proc.StopSignal
|
||||
ep.proc.StopSignal = sig
|
||||
return ret
|
||||
}
|
||||
|
||||
func (ep *etcdServerProcess) waitReady() error {
|
||||
defer close(ep.donec)
|
||||
return waitReadyExpectProc(ep.proc, etcdServerReadyLines)
|
||||
}
|
||||
|
||||
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
|
||||
}
|
@ -23,24 +23,25 @@ import (
|
||||
|
||||
"go.etcd.io/etcd/api/v3/version"
|
||||
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
// TestReleaseUpgrade ensures that changes to master branch does not affect
|
||||
// upgrade from latest etcd releases.
|
||||
func TestReleaseUpgrade(t *testing.T) {
|
||||
lastReleaseBinary := binDir + "/etcd-last-release"
|
||||
lastReleaseBinary := e2e.BinDir + "/etcd-last-release"
|
||||
if !fileutil.Exist(lastReleaseBinary) {
|
||||
t.Skipf("%q does not exist", lastReleaseBinary)
|
||||
}
|
||||
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
copiedCfg := newConfigNoTLS()
|
||||
copiedCfg.execPath = lastReleaseBinary
|
||||
copiedCfg.snapshotCount = 3
|
||||
copiedCfg.baseScheme = "unix" // to avoid port conflict
|
||||
copiedCfg := e2e.NewConfigNoTLS()
|
||||
copiedCfg.ExecPath = lastReleaseBinary
|
||||
copiedCfg.SnapshotCount = 3
|
||||
copiedCfg.BaseScheme = "unix" // to avoid port conflict
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, copiedCfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, copiedCfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
@ -54,7 +55,7 @@ func TestReleaseUpgrade(t *testing.T) {
|
||||
defer os.Unsetenv("ETCDCTL_API")
|
||||
cx := ctlCtx{
|
||||
t: t,
|
||||
cfg: *newConfigNoTLS(),
|
||||
cfg: *e2e.NewConfigNoTLS(),
|
||||
dialTimeout: 7 * time.Second,
|
||||
quorum: true,
|
||||
epc: epc,
|
||||
@ -71,17 +72,17 @@ func TestReleaseUpgrade(t *testing.T) {
|
||||
|
||||
t.Log("Cluster of etcd in old version running")
|
||||
|
||||
for i := range epc.procs {
|
||||
for i := range epc.Procs {
|
||||
t.Logf("Stopping node: %v", i)
|
||||
if err := epc.procs[i].Stop(); err != nil {
|
||||
if err := epc.Procs[i].Stop(); err != nil {
|
||||
t.Fatalf("#%d: error closing etcd process (%v)", i, err)
|
||||
}
|
||||
t.Logf("Stopped node: %v", i)
|
||||
epc.procs[i].Config().execPath = binDir + "/etcd"
|
||||
epc.procs[i].Config().keepDataDir = true
|
||||
epc.Procs[i].Config().ExecPath = e2e.BinDir + "/etcd"
|
||||
epc.Procs[i].Config().KeepDataDir = true
|
||||
|
||||
t.Logf("Restarting node in the new version: %v", i)
|
||||
if err := epc.procs[i].Restart(); err != nil {
|
||||
if err := epc.Procs[i].Restart(); err != nil {
|
||||
t.Fatalf("error restarting etcd process (%v)", err)
|
||||
}
|
||||
|
||||
@ -100,7 +101,7 @@ func TestReleaseUpgrade(t *testing.T) {
|
||||
// new cluster version needs more time to upgrade
|
||||
ver := version.Cluster(version.Version)
|
||||
for i := 0; i < 7; i++ {
|
||||
if err = cURLGet(epc, cURLReq{endpoint: "/version", expected: `"etcdcluster":"` + ver}); err != nil {
|
||||
if err = e2e.CURLGet(epc, e2e.CURLReq{Endpoint: "/version", Expected: `"etcdcluster":"` + ver}); err != nil {
|
||||
t.Logf("#%d: %v is not ready yet (%v)", i, ver, err)
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
@ -114,19 +115,19 @@ func TestReleaseUpgrade(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestReleaseUpgradeWithRestart(t *testing.T) {
|
||||
lastReleaseBinary := binDir + "/etcd-last-release"
|
||||
lastReleaseBinary := e2e.BinDir + "/etcd-last-release"
|
||||
if !fileutil.Exist(lastReleaseBinary) {
|
||||
t.Skipf("%q does not exist", lastReleaseBinary)
|
||||
}
|
||||
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
copiedCfg := newConfigNoTLS()
|
||||
copiedCfg.execPath = lastReleaseBinary
|
||||
copiedCfg.snapshotCount = 10
|
||||
copiedCfg.baseScheme = "unix"
|
||||
copiedCfg := e2e.NewConfigNoTLS()
|
||||
copiedCfg.ExecPath = lastReleaseBinary
|
||||
copiedCfg.SnapshotCount = 10
|
||||
copiedCfg.BaseScheme = "unix"
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, copiedCfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, copiedCfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
@ -140,7 +141,7 @@ func TestReleaseUpgradeWithRestart(t *testing.T) {
|
||||
defer os.Unsetenv("ETCDCTL_API")
|
||||
cx := ctlCtx{
|
||||
t: t,
|
||||
cfg: *newConfigNoTLS(),
|
||||
cfg: *e2e.NewConfigNoTLS(),
|
||||
dialTimeout: 7 * time.Second,
|
||||
quorum: true,
|
||||
epc: epc,
|
||||
@ -155,19 +156,19 @@ func TestReleaseUpgradeWithRestart(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
for i := range epc.procs {
|
||||
if err := epc.procs[i].Stop(); err != nil {
|
||||
for i := range epc.Procs {
|
||||
if err := epc.Procs[i].Stop(); err != nil {
|
||||
t.Fatalf("#%d: error closing etcd process (%v)", i, err)
|
||||
}
|
||||
}
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(len(epc.procs))
|
||||
for i := range epc.procs {
|
||||
wg.Add(len(epc.Procs))
|
||||
for i := range epc.Procs {
|
||||
go func(i int) {
|
||||
epc.procs[i].Config().execPath = binDir + "/etcd"
|
||||
epc.procs[i].Config().keepDataDir = true
|
||||
if err := epc.procs[i].Restart(); err != nil {
|
||||
epc.Procs[i].Config().ExecPath = e2e.BinDir + "/etcd"
|
||||
epc.Procs[i].Config().KeepDataDir = true
|
||||
if err := epc.Procs[i].Restart(); err != nil {
|
||||
t.Errorf("error restarting etcd process (%v)", err)
|
||||
}
|
||||
wg.Done()
|
||||
|
@ -20,17 +20,18 @@ import (
|
||||
"strings"
|
||||
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
"go.etcd.io/etcd/tests/v3/integration"
|
||||
)
|
||||
|
||||
type Etcdctl struct {
|
||||
connType clientConnType
|
||||
connType e2e.ClientConnType
|
||||
isAutoTLS bool
|
||||
endpoints []string
|
||||
v2 bool
|
||||
}
|
||||
|
||||
func NewEtcdctl(endpoints []string, connType clientConnType, isAutoTLS bool, v2 bool) *Etcdctl {
|
||||
func NewEtcdctl(endpoints []string, connType e2e.ClientConnType, isAutoTLS bool, v2 bool) *Etcdctl {
|
||||
return &Etcdctl{
|
||||
endpoints: endpoints,
|
||||
connType: connType,
|
||||
@ -51,7 +52,7 @@ func (ctl *Etcdctl) Put(key, value string) error {
|
||||
}
|
||||
args := ctl.cmdArgs()
|
||||
args = append(args, "put", key, value)
|
||||
return spawnWithExpectWithEnv(args, ctl.env(), "OK")
|
||||
return e2e.SpawnWithExpectWithEnv(args, ctl.env(), "OK")
|
||||
}
|
||||
|
||||
func (ctl *Etcdctl) Set(key, value string) error {
|
||||
@ -60,7 +61,7 @@ func (ctl *Etcdctl) Set(key, value string) error {
|
||||
}
|
||||
args := ctl.cmdArgs()
|
||||
args = append(args, "set", key, value)
|
||||
lines, err := runUtilCompletion(args, ctl.env())
|
||||
lines, err := e2e.RunUtilCompletion(args, ctl.env())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -94,12 +95,12 @@ func (ctl *Etcdctl) Compact(rev int64) (*clientv3.CompactResponse, error) {
|
||||
panic("Unsupported method for v2")
|
||||
}
|
||||
args := ctl.cmdArgs("compact", fmt.Sprint(rev))
|
||||
return nil, spawnWithExpectWithEnv(args, ctl.env(), fmt.Sprintf("compacted revision %v", rev))
|
||||
return nil, e2e.SpawnWithExpectWithEnv(args, ctl.env(), fmt.Sprintf("compacted revision %v", rev))
|
||||
}
|
||||
|
||||
func (ctl *Etcdctl) spawnJsonCmd(output interface{}, args ...string) error {
|
||||
args = append(args, "-w", "json")
|
||||
cmd, err := spawnCmd(append(ctl.cmdArgs(), args...), ctl.env())
|
||||
cmd, err := e2e.SpawnCmd(append(ctl.cmdArgs(), args...), ctl.env())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -111,7 +112,7 @@ func (ctl *Etcdctl) spawnJsonCmd(output interface{}, args ...string) error {
|
||||
}
|
||||
|
||||
func (ctl *Etcdctl) cmdArgs(args ...string) []string {
|
||||
cmdArgs := []string{ctlBinPath}
|
||||
cmdArgs := []string{e2e.CtlBinPath}
|
||||
for k, v := range ctl.flags() {
|
||||
cmdArgs = append(cmdArgs, fmt.Sprintf("--%s=%s", k, v))
|
||||
}
|
||||
@ -122,13 +123,13 @@ func (ctl *Etcdctl) flags() map[string]string {
|
||||
fmap := make(map[string]string)
|
||||
if ctl.v2 {
|
||||
fmap["no-sync"] = "true"
|
||||
if ctl.connType == clientTLS {
|
||||
if ctl.connType == e2e.ClientTLS {
|
||||
fmap["ca-file"] = integration.TestTLSInfo.TrustedCAFile
|
||||
fmap["cert-file"] = integration.TestTLSInfo.CertFile
|
||||
fmap["key-file"] = integration.TestTLSInfo.KeyFile
|
||||
}
|
||||
} else {
|
||||
if ctl.connType == clientTLS {
|
||||
if ctl.connType == e2e.ClientTLS {
|
||||
if ctl.isAutoTLS {
|
||||
fmap["insecure-transport"] = "false"
|
||||
fmap["insecure-skip-tls-verify"] = "true"
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -27,7 +28,7 @@ var (
|
||||
)
|
||||
|
||||
func TestGateway(t *testing.T) {
|
||||
ec, err := newEtcdProcessCluster(t, newConfigNoTLS())
|
||||
ec, err := e2e.NewEtcdProcessCluster(t, e2e.NewConfigNoTLS())
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -41,14 +42,14 @@ func TestGateway(t *testing.T) {
|
||||
os.Setenv("ETCDCTL_API", "3")
|
||||
defer os.Unsetenv("ETCDCTL_API")
|
||||
|
||||
err = spawnWithExpect([]string{ctlBinPath, "--endpoints=" + defaultGatewayEndpoint, "put", "foo", "bar"}, "OK\r\n")
|
||||
err = e2e.SpawnWithExpect([]string{e2e.CtlBinPath, "--endpoints=" + defaultGatewayEndpoint, "put", "foo", "bar"}, "OK\r\n")
|
||||
if err != nil {
|
||||
t.Errorf("failed to finish put request through gateway: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func startGateway(t *testing.T, endpoints string) *expect.ExpectProcess {
|
||||
p, err := expect.NewExpect(binPath, "gateway", "--endpoints="+endpoints, "start")
|
||||
p, err := expect.NewExpect(e2e.BinPath, "gateway", "--endpoints="+endpoints, "start")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -5,61 +5,15 @@
|
||||
package e2e
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"os"
|
||||
"runtime"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/testutil"
|
||||
"go.etcd.io/etcd/tests/v3/integration"
|
||||
)
|
||||
|
||||
var (
|
||||
binDir string
|
||||
certDir string
|
||||
|
||||
certPath string
|
||||
privateKeyPath string
|
||||
caPath string
|
||||
|
||||
certPath2 string
|
||||
privateKeyPath2 string
|
||||
|
||||
certPath3 string
|
||||
privateKeyPath3 string
|
||||
|
||||
crlPath string
|
||||
revokedCertPath string
|
||||
revokedPrivateKeyPath string
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
os.Setenv("ETCD_UNSUPPORTED_ARCH", runtime.GOARCH)
|
||||
os.Unsetenv("ETCDCTL_API")
|
||||
|
||||
binDirDef := integration.MustAbsPath("../../bin")
|
||||
certDirDef := fixturesDir
|
||||
|
||||
flag.StringVar(&binDir, "bin-dir", binDirDef, "The directory for store etcd and etcdctl binaries.")
|
||||
flag.StringVar(&certDir, "cert-dir", certDirDef, "The directory for store certificate files.")
|
||||
flag.Parse()
|
||||
|
||||
binPath = binDir + "/etcd"
|
||||
ctlBinPath = binDir + "/etcdctl"
|
||||
utlBinPath = binDir + "/etcdutl"
|
||||
certPath = certDir + "/server.crt"
|
||||
privateKeyPath = certDir + "/server.key.insecure"
|
||||
caPath = certDir + "/ca.crt"
|
||||
revokedCertPath = certDir + "/server-revoked.crt"
|
||||
revokedPrivateKeyPath = certDir + "/server-revoked.key.insecure"
|
||||
crlPath = certDir + "/revoke.crl"
|
||||
|
||||
certPath2 = certDir + "/server2.crt"
|
||||
privateKeyPath2 = certDir + "/server2.key.insecure"
|
||||
|
||||
certPath3 = certDir + "/server3.crt"
|
||||
privateKeyPath3 = certDir + "/server3.key.insecure"
|
||||
|
||||
e2e.InitFlags()
|
||||
v := m.Run()
|
||||
if v == 0 && testutil.CheckLeakedGoroutine() {
|
||||
os.Exit(1)
|
||||
|
@ -19,19 +19,20 @@ import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/version"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestV3MetricsSecure(t *testing.T) {
|
||||
cfg := newConfigTLS()
|
||||
cfg.clusterSize = 1
|
||||
cfg.metricsURLScheme = "https"
|
||||
cfg := e2e.NewConfigTLS()
|
||||
cfg.ClusterSize = 1
|
||||
cfg.MetricsURLScheme = "https"
|
||||
testCtl(t, metricsTest)
|
||||
}
|
||||
|
||||
func TestV3MetricsInsecure(t *testing.T) {
|
||||
cfg := newConfigTLS()
|
||||
cfg.clusterSize = 1
|
||||
cfg.metricsURLScheme = "http"
|
||||
cfg := e2e.NewConfigTLS()
|
||||
cfg.ClusterSize = 1
|
||||
cfg.MetricsURLScheme = "http"
|
||||
testCtl(t, metricsTest)
|
||||
}
|
||||
|
||||
@ -62,7 +63,7 @@ func metricsTest(cx ctlCtx) {
|
||||
if err := ctlV3Watch(cx, []string{"k", "--rev", "1"}, []kvExec{{key: "k", val: "v"}}...); err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err := cURLGet(cx.epc, cURLReq{endpoint: test.endpoint, expected: test.expected, metricsURLScheme: cx.cfg.metricsURLScheme}); err != nil {
|
||||
if err := e2e.CURLGet(cx.epc, e2e.CURLReq{Endpoint: test.endpoint, Expected: test.expected, MetricsURLScheme: cx.cfg.MetricsURLScheme}); err != nil {
|
||||
cx.t.Fatalf("failed get with curl (%v)", err)
|
||||
}
|
||||
}
|
||||
|
@ -21,6 +21,7 @@ import (
|
||||
"time"
|
||||
|
||||
clientv2 "go.etcd.io/etcd/client/v2"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
"go.etcd.io/etcd/tests/v3/integration"
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/sync/errgroup"
|
||||
@ -31,15 +32,7 @@ import (
|
||||
"go.etcd.io/etcd/pkg/v3/stringutil"
|
||||
)
|
||||
|
||||
type clientConnType int
|
||||
|
||||
const (
|
||||
clientNonTLS clientConnType = iota
|
||||
clientTLS
|
||||
clientTLSAndNonTLS
|
||||
)
|
||||
|
||||
func newClient(t *testing.T, entpoints []string, connType clientConnType, isAutoTLS bool) *clientv3.Client {
|
||||
func newClient(t *testing.T, entpoints []string, connType e2e.ClientConnType, isAutoTLS bool) *clientv3.Client {
|
||||
tlscfg, err := tlsInfo(t, connType, isAutoTLS)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -66,7 +59,7 @@ func newClient(t *testing.T, entpoints []string, connType clientConnType, isAuto
|
||||
return c
|
||||
}
|
||||
|
||||
func newClientV2(t *testing.T, endpoints []string, connType clientConnType, isAutoTLS bool) (clientv2.Client, error) {
|
||||
func newClientV2(t *testing.T, endpoints []string, connType e2e.ClientConnType, isAutoTLS bool) (clientv2.Client, error) {
|
||||
tls, err := tlsInfo(t, connType, isAutoTLS)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -83,11 +76,11 @@ func newClientV2(t *testing.T, endpoints []string, connType clientConnType, isAu
|
||||
return clientv2.New(cfg)
|
||||
}
|
||||
|
||||
func tlsInfo(t testing.TB, connType clientConnType, isAutoTLS bool) (*transport.TLSInfo, error) {
|
||||
func tlsInfo(t testing.TB, connType e2e.ClientConnType, isAutoTLS bool) (*transport.TLSInfo, error) {
|
||||
switch connType {
|
||||
case clientNonTLS, clientTLSAndNonTLS:
|
||||
case e2e.ClientNonTLS, e2e.ClientTLSAndNonTLS:
|
||||
return nil, nil
|
||||
case clientTLS:
|
||||
case e2e.ClientTLS:
|
||||
if isAutoTLS {
|
||||
tls, err := transport.SelfCert(zap.NewNop(), t.TempDir(), []string{"localhost"}, 1)
|
||||
if err != nil {
|
||||
|
@ -15,27 +15,27 @@
|
||||
package e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestV2CurlNoTLS(t *testing.T) { testCurlPutGet(t, newConfigNoTLS()) }
|
||||
func TestV2CurlAutoTLS(t *testing.T) { testCurlPutGet(t, newConfigAutoTLS()) }
|
||||
func TestV2CurlAllTLS(t *testing.T) { testCurlPutGet(t, newConfigTLS()) }
|
||||
func TestV2CurlPeerTLS(t *testing.T) { testCurlPutGet(t, newConfigPeerTLS()) }
|
||||
func TestV2CurlClientTLS(t *testing.T) { testCurlPutGet(t, newConfigClientTLS()) }
|
||||
func TestV2CurlClientBoth(t *testing.T) { testCurlPutGet(t, newConfigClientBoth()) }
|
||||
func testCurlPutGet(t *testing.T, cfg *etcdProcessClusterConfig) {
|
||||
func TestV2CurlNoTLS(t *testing.T) { testCurlPutGet(t, e2e.NewConfigNoTLS()) }
|
||||
func TestV2CurlAutoTLS(t *testing.T) { testCurlPutGet(t, e2e.NewConfigAutoTLS()) }
|
||||
func TestV2CurlAllTLS(t *testing.T) { testCurlPutGet(t, e2e.NewConfigTLS()) }
|
||||
func TestV2CurlPeerTLS(t *testing.T) { testCurlPutGet(t, e2e.NewConfigPeerTLS()) }
|
||||
func TestV2CurlClientTLS(t *testing.T) { testCurlPutGet(t, e2e.NewConfigClientTLS()) }
|
||||
func TestV2CurlClientBoth(t *testing.T) { testCurlPutGet(t, e2e.NewConfigClientBoth()) }
|
||||
func testCurlPutGet(t *testing.T, cfg *e2e.EtcdProcessClusterConfig) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
// test doesn't use quorum gets, so ensure there are no followers to avoid
|
||||
// stale reads that will break the test
|
||||
cfg = configStandalone(*cfg)
|
||||
cfg = e2e.ConfigStandalone(*cfg)
|
||||
|
||||
cfg.enableV2 = true
|
||||
epc, err := newEtcdProcessCluster(t, cfg)
|
||||
cfg.EnableV2 = true
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
@ -49,14 +49,14 @@ func testCurlPutGet(t *testing.T, cfg *etcdProcessClusterConfig) {
|
||||
expectPut = `{"action":"set","node":{"key":"/foo","value":"bar","`
|
||||
expectGet = `{"action":"get","node":{"key":"/foo","value":"bar","`
|
||||
)
|
||||
if err := cURLPut(epc, cURLReq{endpoint: "/v2/keys/foo", value: "bar", expected: expectPut}); err != nil {
|
||||
if err := e2e.CURLPut(epc, e2e.CURLReq{Endpoint: "/v2/keys/foo", Value: "bar", Expected: expectPut}); err != nil {
|
||||
t.Fatalf("failed put with curl (%v)", err)
|
||||
}
|
||||
if err := cURLGet(epc, cURLReq{endpoint: "/v2/keys/foo", expected: expectGet}); err != nil {
|
||||
if err := e2e.CURLGet(epc, e2e.CURLReq{Endpoint: "/v2/keys/foo", Expected: expectGet}); err != nil {
|
||||
t.Fatalf("failed get with curl (%v)", err)
|
||||
}
|
||||
if cfg.clientTLS == clientTLSAndNonTLS {
|
||||
if err := cURLGet(epc, cURLReq{endpoint: "/v2/keys/foo", expected: expectGet, isTLS: true}); err != nil {
|
||||
if cfg.ClientTLS == e2e.ClientTLSAndNonTLS {
|
||||
if err := e2e.CURLGet(epc, e2e.CURLReq{Endpoint: "/v2/keys/foo", Expected: expectGet, IsTLS: true}); err != nil {
|
||||
t.Fatalf("failed get with curl (%v)", err)
|
||||
}
|
||||
}
|
||||
@ -65,8 +65,8 @@ func testCurlPutGet(t *testing.T, cfg *etcdProcessClusterConfig) {
|
||||
func TestV2CurlIssue5182(t *testing.T) {
|
||||
BeforeTestV2(t)
|
||||
|
||||
copied := newConfigNoTLS()
|
||||
copied.enableV2 = true
|
||||
copied := e2e.NewConfigNoTLS()
|
||||
copied.EnableV2 = true
|
||||
epc := setupEtcdctlTest(t, copied, false)
|
||||
defer func() {
|
||||
if err := epc.Close(); err != nil {
|
||||
@ -75,20 +75,20 @@ func TestV2CurlIssue5182(t *testing.T) {
|
||||
}()
|
||||
|
||||
expectPut := `{"action":"set","node":{"key":"/foo","value":"bar","`
|
||||
if err := cURLPut(epc, cURLReq{endpoint: "/v2/keys/foo", value: "bar", expected: expectPut}); err != nil {
|
||||
if err := e2e.CURLPut(epc, e2e.CURLReq{Endpoint: "/v2/keys/foo", Value: "bar", Expected: expectPut}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
expectUserAdd := `{"user":"foo","roles":null}`
|
||||
if err := cURLPut(epc, cURLReq{endpoint: "/v2/auth/users/foo", value: `{"user":"foo", "password":"pass"}`, expected: expectUserAdd}); err != nil {
|
||||
if err := e2e.CURLPut(epc, e2e.CURLReq{Endpoint: "/v2/auth/users/foo", Value: `{"user":"foo", "password":"pass"}`, Expected: expectUserAdd}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
expectRoleAdd := `{"role":"foo","permissions":{"kv":{"read":["/foo/*"],"write":null}}`
|
||||
if err := cURLPut(epc, cURLReq{endpoint: "/v2/auth/roles/foo", value: `{"role":"foo", "permissions": {"kv": {"read": ["/foo/*"]}}}`, expected: expectRoleAdd}); err != nil {
|
||||
if err := e2e.CURLPut(epc, e2e.CURLReq{Endpoint: "/v2/auth/roles/foo", Value: `{"role":"foo", "permissions": {"kv": {"read": ["/foo/*"]}}}`, Expected: expectRoleAdd}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
expectUserUpdate := `{"user":"foo","roles":["foo"]}`
|
||||
if err := cURLPut(epc, cURLReq{endpoint: "/v2/auth/users/foo", value: `{"user": "foo", "grant": ["foo"]}`, expected: expectUserUpdate}); err != nil {
|
||||
if err := e2e.CURLPut(epc, e2e.CURLReq{Endpoint: "/v2/auth/users/foo", Value: `{"user": "foo", "grant": ["foo"]}`, Expected: expectUserUpdate}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
@ -99,13 +99,13 @@ func TestV2CurlIssue5182(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if err := cURLGet(epc, cURLReq{endpoint: "/v2/keys/foo/", username: "root", password: "a", expected: "bar"}); err != nil {
|
||||
if err := e2e.CURLGet(epc, e2e.CURLReq{Endpoint: "/v2/keys/foo/", Username: "root", Password: "a", Expected: "bar"}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := cURLGet(epc, cURLReq{endpoint: "/v2/keys/foo/", username: "foo", password: "pass", expected: "bar"}); err != nil {
|
||||
if err := e2e.CURLGet(epc, e2e.CURLReq{Endpoint: "/v2/keys/foo/", Username: "foo", Password: "pass", Expected: "bar"}); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if err := cURLGet(epc, cURLReq{endpoint: "/v2/keys/foo/", username: "foo", password: "", expected: "bar"}); err != nil {
|
||||
if err := e2e.CURLGet(epc, e2e.CURLReq{Endpoint: "/v2/keys/foo/", Username: "foo", Password: "", Expected: "bar"}); err != nil {
|
||||
if !strings.Contains(err.Error(), `The request requires user authentication`) {
|
||||
t.Fatalf("expected 'The request requires user authentication' error, got %v", err)
|
||||
}
|
||||
@ -113,103 +113,3 @@ func TestV2CurlIssue5182(t *testing.T) {
|
||||
t.Fatalf("expected 'The request requires user authentication' error")
|
||||
}
|
||||
}
|
||||
|
||||
type cURLReq struct {
|
||||
username string
|
||||
password string
|
||||
|
||||
isTLS bool
|
||||
timeout int
|
||||
|
||||
endpoint string
|
||||
|
||||
value string
|
||||
expected string
|
||||
header string
|
||||
|
||||
metricsURLScheme string
|
||||
|
||||
ciphers string
|
||||
httpVersion string
|
||||
|
||||
OutputFile string
|
||||
}
|
||||
|
||||
// cURLPrefixArgsCluster builds the beginning of a curl command for a given key
|
||||
// addressed to a random URL in the given cluster.
|
||||
func cURLPrefixArgsCluster(clus *etcdProcessCluster, method string, req cURLReq) []string {
|
||||
member := clus.procs[rand.Intn(clus.cfg.clusterSize)]
|
||||
clientURL := member.Config().acurl
|
||||
if req.metricsURLScheme != "" {
|
||||
clientURL = member.EndpointsMetrics()[0]
|
||||
}
|
||||
return cURLPrefixArgs(clientURL, clus.cfg.clientTLS, !clus.cfg.noCN, method, req)
|
||||
}
|
||||
|
||||
func cURLPrefixArgs(clientURL string, connType clientConnType, CN bool, method string, req cURLReq) []string {
|
||||
var (
|
||||
cmdArgs = []string{"curl"}
|
||||
)
|
||||
if req.httpVersion != "" {
|
||||
cmdArgs = append(cmdArgs, "--http"+req.httpVersion)
|
||||
}
|
||||
if req.metricsURLScheme != "https" {
|
||||
if req.isTLS {
|
||||
if connType != clientTLSAndNonTLS {
|
||||
panic("should not use cURLPrefixArgsUseTLS when serving only TLS or non-TLS")
|
||||
}
|
||||
cmdArgs = append(cmdArgs, "--cacert", caPath, "--cert", certPath, "--key", privateKeyPath)
|
||||
clientURL = toTLS(clientURL)
|
||||
} else if connType == clientTLS {
|
||||
if CN {
|
||||
cmdArgs = append(cmdArgs, "--cacert", caPath, "--cert", certPath, "--key", privateKeyPath)
|
||||
} else {
|
||||
cmdArgs = append(cmdArgs, "--cacert", caPath, "--cert", certPath3, "--key", privateKeyPath3)
|
||||
}
|
||||
}
|
||||
}
|
||||
ep := clientURL + req.endpoint
|
||||
|
||||
if req.username != "" || req.password != "" {
|
||||
cmdArgs = append(cmdArgs, "-L", "-u", fmt.Sprintf("%s:%s", req.username, req.password), ep)
|
||||
} else {
|
||||
cmdArgs = append(cmdArgs, "-L", ep)
|
||||
}
|
||||
if req.timeout != 0 {
|
||||
cmdArgs = append(cmdArgs, "-m", fmt.Sprintf("%d", req.timeout))
|
||||
}
|
||||
|
||||
if req.header != "" {
|
||||
cmdArgs = append(cmdArgs, "-H", req.header)
|
||||
}
|
||||
|
||||
if req.ciphers != "" {
|
||||
cmdArgs = append(cmdArgs, "--ciphers", req.ciphers)
|
||||
}
|
||||
|
||||
if req.OutputFile != "" {
|
||||
cmdArgs = append(cmdArgs, "--output", req.OutputFile)
|
||||
}
|
||||
|
||||
switch method {
|
||||
case "POST", "PUT":
|
||||
dt := req.value
|
||||
if !strings.HasPrefix(dt, "{") { // for non-JSON value
|
||||
dt = "value=" + dt
|
||||
}
|
||||
cmdArgs = append(cmdArgs, "-X", method, "-d", dt)
|
||||
}
|
||||
return cmdArgs
|
||||
}
|
||||
|
||||
func cURLPost(clus *etcdProcessCluster, req cURLReq) error {
|
||||
return spawnWithExpect(cURLPrefixArgsCluster(clus, "POST", req), req.expected)
|
||||
}
|
||||
|
||||
func cURLPut(clus *etcdProcessCluster, req cURLReq) error {
|
||||
return spawnWithExpect(cURLPrefixArgsCluster(clus, "PUT", req), req.expected)
|
||||
}
|
||||
|
||||
func cURLGet(clus *etcdProcessCluster, req cURLReq) error {
|
||||
return spawnWithExpect(cURLPrefixArgsCluster(clus, "GET", req), req.expected)
|
||||
}
|
||||
|
@ -19,24 +19,25 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func createV2store(t testing.TB, dataDirPath string) {
|
||||
t.Log("Creating not-yet v2-deprecated etcd")
|
||||
|
||||
cfg := configStandalone(etcdProcessClusterConfig{enableV2: true, dataDirPath: dataDirPath, snapshotCount: 5})
|
||||
epc, err := newEtcdProcessCluster(t, cfg)
|
||||
cfg := e2e.ConfigStandalone(e2e.EtcdProcessClusterConfig{EnableV2: true, DataDirPath: dataDirPath, SnapshotCount: 5})
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, cfg)
|
||||
assert.NoError(t, err)
|
||||
|
||||
defer func() {
|
||||
assert.NoError(t, epc.Stop())
|
||||
}()
|
||||
|
||||
// We need to exceed 'snapshotCount' such that v2 snapshot is dumped.
|
||||
// We need to exceed 'SnapshotCount' such that v2 snapshot is dumped.
|
||||
for i := 0; i < 10; i++ {
|
||||
if err := cURLPut(epc, cURLReq{
|
||||
endpoint: "/v2/keys/foo", value: "bar" + fmt.Sprint(i),
|
||||
expected: `{"action":"set","node":{"key":"/foo","value":"bar` + fmt.Sprint(i)}); err != nil {
|
||||
if err := e2e.CURLPut(epc, e2e.CURLReq{
|
||||
Endpoint: "/v2/keys/foo", Value: "bar" + fmt.Sprint(i),
|
||||
Expected: `{"action":"set","node":{"key":"/foo","value":"bar` + fmt.Sprint(i)}); err != nil {
|
||||
t.Fatalf("failed put with curl (%v)", err)
|
||||
}
|
||||
}
|
||||
@ -45,17 +46,17 @@ func createV2store(t testing.TB, dataDirPath string) {
|
||||
func assertVerifyCanStartV2deprecationNotYet(t testing.TB, dataDirPath string) {
|
||||
t.Log("verify: possible to start etcd with --v2-deprecation=not-yet mode")
|
||||
|
||||
cfg := configStandalone(etcdProcessClusterConfig{enableV2: true, dataDirPath: dataDirPath, v2deprecation: "not-yet", keepDataDir: true})
|
||||
epc, err := newEtcdProcessCluster(t, cfg)
|
||||
cfg := e2e.ConfigStandalone(e2e.EtcdProcessClusterConfig{EnableV2: true, DataDirPath: dataDirPath, V2deprecation: "not-yet", KeepDataDir: true})
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, cfg)
|
||||
assert.NoError(t, err)
|
||||
|
||||
defer func() {
|
||||
assert.NoError(t, epc.Stop())
|
||||
}()
|
||||
|
||||
if err := cURLGet(epc, cURLReq{
|
||||
endpoint: "/v2/keys/foo",
|
||||
expected: `{"action":"get","node":{"key":"/foo","value":"bar9","modifiedIndex":13,"createdIndex":13}}`}); err != nil {
|
||||
if err := e2e.CURLGet(epc, e2e.CURLReq{
|
||||
Endpoint: "/v2/keys/foo",
|
||||
Expected: `{"action":"get","node":{"key":"/foo","value":"bar9","modifiedIndex":13,"createdIndex":13}}`}); err != nil {
|
||||
t.Fatalf("failed get with curl (%v)", err)
|
||||
}
|
||||
|
||||
@ -63,7 +64,7 @@ func assertVerifyCanStartV2deprecationNotYet(t testing.TB, dataDirPath string) {
|
||||
|
||||
func assertVerifyCannotStartV2deprecationWriteOnly(t testing.TB, dataDirPath string) {
|
||||
t.Log("Verify its infeasible to start etcd with --v2-deprecation=write-only mode")
|
||||
proc, err := spawnCmd([]string{binDir + "/etcd", "--v2-deprecation=write-only", "--data-dir=" + dataDirPath}, nil)
|
||||
proc, err := e2e.SpawnCmd([]string{e2e.BinDir + "/etcd", "--v2-deprecation=write-only", "--data-dir=" + dataDirPath}, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
_, err = proc.Expect("detected disallowed custom content in v2store for stage --v2-deprecation=write-only")
|
||||
@ -71,7 +72,7 @@ func assertVerifyCannotStartV2deprecationWriteOnly(t testing.TB, dataDirPath str
|
||||
}
|
||||
|
||||
func TestV2Deprecation(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
dataDirPath := t.TempDir()
|
||||
|
||||
t.Run("create-storev2-data", func(t *testing.T) {
|
||||
@ -89,8 +90,8 @@ func TestV2Deprecation(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestV2DeprecationWriteOnlyNoV2Api(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
proc, err := spawnCmd([]string{binDir + "/etcd", "--v2-deprecation=write-only", "--enable-v2"}, nil)
|
||||
e2e.BeforeTest(t)
|
||||
proc, err := e2e.SpawnCmd([]string{e2e.BinDir + "/etcd", "--v2-deprecation=write-only", "--enable-v2"}, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
_, err = proc.Expect("--enable-v2 and --v2-deprecation=write-only are mutually exclusive")
|
||||
|
@ -22,14 +22,15 @@ import (
|
||||
"testing"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/version"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestV3CurlCipherSuitesValid(t *testing.T) { testV3CurlCipherSuites(t, true) }
|
||||
func TestV3CurlCipherSuitesMismatch(t *testing.T) { testV3CurlCipherSuites(t, false) }
|
||||
func testV3CurlCipherSuites(t *testing.T, valid bool) {
|
||||
cc := newConfigClientTLS()
|
||||
cc.clusterSize = 1
|
||||
cc.cipherSuites = []string{
|
||||
cc := e2e.NewConfigClientTLS()
|
||||
cc.ClusterSize = 1
|
||||
cc.CipherSuites = []string{
|
||||
"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
|
||||
"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
|
||||
"TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
|
||||
@ -45,11 +46,11 @@ func testV3CurlCipherSuites(t *testing.T, valid bool) {
|
||||
}
|
||||
|
||||
func cipherSuiteTestValid(cx ctlCtx) {
|
||||
if err := cURLGet(cx.epc, cURLReq{
|
||||
endpoint: "/metrics",
|
||||
expected: fmt.Sprintf(`etcd_server_version{server_version="%s"} 1`, version.Version),
|
||||
metricsURLScheme: cx.cfg.metricsURLScheme,
|
||||
ciphers: "ECDHE-RSA-AES128-GCM-SHA256", // TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256
|
||||
if err := e2e.CURLGet(cx.epc, e2e.CURLReq{
|
||||
Endpoint: "/metrics",
|
||||
Expected: fmt.Sprintf(`etcd_server_version{server_version="%s"} 1`, version.Version),
|
||||
MetricsURLScheme: cx.cfg.MetricsURLScheme,
|
||||
Ciphers: "ECDHE-RSA-AES128-GCM-SHA256", // TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256
|
||||
}); err != nil {
|
||||
cx.t.Fatalf("failed get with curl (%v)", err)
|
||||
}
|
||||
@ -58,11 +59,11 @@ func cipherSuiteTestValid(cx ctlCtx) {
|
||||
func cipherSuiteTestMismatch(cx ctlCtx) {
|
||||
var err error
|
||||
for _, exp := range []string{"alert handshake failure", "failed setting cipher list"} {
|
||||
err = cURLGet(cx.epc, cURLReq{
|
||||
endpoint: "/metrics",
|
||||
expected: exp,
|
||||
metricsURLScheme: cx.cfg.metricsURLScheme,
|
||||
ciphers: "ECDHE-RSA-DES-CBC3-SHA", // TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
|
||||
err = e2e.CURLGet(cx.epc, e2e.CURLReq{
|
||||
Endpoint: "/metrics",
|
||||
Expected: exp,
|
||||
MetricsURLScheme: cx.cfg.MetricsURLScheme,
|
||||
Ciphers: "ECDHE-RSA-DES-CBC3-SHA", // TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
|
||||
})
|
||||
if err == nil {
|
||||
break
|
||||
|
@ -19,26 +19,27 @@ import (
|
||||
"testing"
|
||||
|
||||
pb "go.etcd.io/etcd/api/v3/etcdserverpb"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestV3CurlLeaseGrantNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlLeaseGrant, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlLeaseGrant, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlLeaseRevokeNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlLeaseRevoke, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlLeaseRevoke, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlLeaseLeasesNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlLeaseLeases, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlLeaseLeases, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlLeaseKeepAliveNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlLeaseKeepAlive, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlLeaseKeepAlive, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
|
||||
@ -51,7 +52,7 @@ type v3cURLTest struct {
|
||||
// TODO remove /kv/lease/timetolive, /kv/lease/revoke, /kv/lease/leases tests in 3.5 release
|
||||
|
||||
func testV3CurlLeaseGrant(cx ctlCtx) {
|
||||
leaseID := randomLeaseID()
|
||||
leaseID := e2e.RandomLeaseID()
|
||||
|
||||
tests := []v3cURLTest{
|
||||
{
|
||||
@ -80,13 +81,13 @@ func testV3CurlLeaseGrant(cx ctlCtx) {
|
||||
expected: `"grantedTTL"`,
|
||||
},
|
||||
}
|
||||
if err := cURLWithExpected(cx, tests); err != nil {
|
||||
if err := CURLWithExpected(cx, tests); err != nil {
|
||||
cx.t.Fatalf("testV3CurlLeaseGrant: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func testV3CurlLeaseRevoke(cx ctlCtx) {
|
||||
leaseID := randomLeaseID()
|
||||
leaseID := e2e.RandomLeaseID()
|
||||
|
||||
tests := []v3cURLTest{
|
||||
{
|
||||
@ -105,13 +106,13 @@ func testV3CurlLeaseRevoke(cx ctlCtx) {
|
||||
expected: `etcdserver: requested lease not found`,
|
||||
},
|
||||
}
|
||||
if err := cURLWithExpected(cx, tests); err != nil {
|
||||
if err := CURLWithExpected(cx, tests); err != nil {
|
||||
cx.t.Fatalf("testV3CurlLeaseRevoke: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func testV3CurlLeaseLeases(cx ctlCtx) {
|
||||
leaseID := randomLeaseID()
|
||||
leaseID := e2e.RandomLeaseID()
|
||||
|
||||
tests := []v3cURLTest{
|
||||
{
|
||||
@ -130,13 +131,13 @@ func testV3CurlLeaseLeases(cx ctlCtx) {
|
||||
expected: gwLeaseIDExpected(leaseID),
|
||||
},
|
||||
}
|
||||
if err := cURLWithExpected(cx, tests); err != nil {
|
||||
if err := CURLWithExpected(cx, tests); err != nil {
|
||||
cx.t.Fatalf("testV3CurlLeaseGrant: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func testV3CurlLeaseKeepAlive(cx ctlCtx) {
|
||||
leaseID := randomLeaseID()
|
||||
leaseID := e2e.RandomLeaseID()
|
||||
|
||||
tests := []v3cURLTest{
|
||||
{
|
||||
@ -150,7 +151,7 @@ func testV3CurlLeaseKeepAlive(cx ctlCtx) {
|
||||
expected: gwLeaseIDExpected(leaseID),
|
||||
},
|
||||
}
|
||||
if err := cURLWithExpected(cx, tests); err != nil {
|
||||
if err := CURLWithExpected(cx, tests); err != nil {
|
||||
cx.t.Fatalf("testV3CurlLeaseGrant: %v", err)
|
||||
}
|
||||
}
|
||||
@ -161,7 +162,7 @@ func gwLeaseIDExpected(leaseID int64) string {
|
||||
|
||||
func gwLeaseTTLWithKeys(cx ctlCtx, leaseID int64) string {
|
||||
d := &pb.LeaseTimeToLiveRequest{ID: leaseID, Keys: true}
|
||||
s, err := dataMarshal(d)
|
||||
s, err := e2e.DataMarshal(d)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("gwLeaseTTLWithKeys: error (%v)", err)
|
||||
}
|
||||
@ -170,7 +171,7 @@ func gwLeaseTTLWithKeys(cx ctlCtx, leaseID int64) string {
|
||||
|
||||
func gwLeaseKeepAlive(cx ctlCtx, leaseID int64) string {
|
||||
d := &pb.LeaseKeepAliveRequest{ID: leaseID}
|
||||
s, err := dataMarshal(d)
|
||||
s, err := e2e.DataMarshal(d)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("gwLeaseKeepAlive: Marshal error (%v)", err)
|
||||
}
|
||||
@ -179,7 +180,7 @@ func gwLeaseKeepAlive(cx ctlCtx, leaseID int64) string {
|
||||
|
||||
func gwLeaseGrant(cx ctlCtx, leaseID int64, ttl int64) string {
|
||||
d := &pb.LeaseGrantRequest{ID: leaseID, TTL: ttl}
|
||||
s, err := dataMarshal(d)
|
||||
s, err := e2e.DataMarshal(d)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("gwLeaseGrant: Marshal error (%v)", err)
|
||||
}
|
||||
@ -188,7 +189,7 @@ func gwLeaseGrant(cx ctlCtx, leaseID int64, ttl int64) string {
|
||||
|
||||
func gwLeaseRevoke(cx ctlCtx, leaseID int64) string {
|
||||
d := &pb.LeaseRevokeRequest{ID: leaseID}
|
||||
s, err := dataMarshal(d)
|
||||
s, err := e2e.DataMarshal(d)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("gwLeaseRevoke: Marshal error (%v)", err)
|
||||
}
|
||||
@ -197,7 +198,7 @@ func gwLeaseRevoke(cx ctlCtx, leaseID int64) string {
|
||||
|
||||
func gwKVPutLease(cx ctlCtx, k string, v string, leaseID int64) string {
|
||||
d := pb.PutRequest{Key: []byte(k), Value: []byte(v), Lease: leaseID}
|
||||
s, err := dataMarshal(d)
|
||||
s, err := e2e.DataMarshal(d)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("gwKVPutLease: Marshal error (%v)", err)
|
||||
}
|
||||
|
@ -25,15 +25,16 @@ import (
|
||||
"github.com/stretchr/testify/assert"
|
||||
pb "go.etcd.io/etcd/api/v3/etcdserverpb"
|
||||
"go.etcd.io/etcd/client/pkg/v3/testutil"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
// TestV3Curl_MaxStreams_BelowLimit_NoTLS_Small tests no TLS
|
||||
func TestV3Curl_MaxStreams_BelowLimit_NoTLS_Small(t *testing.T) {
|
||||
testV3CurlMaxStream(t, false, withCfg(*newConfigNoTLS()), withMaxConcurrentStreams(3))
|
||||
testV3CurlMaxStream(t, false, withCfg(*e2e.NewConfigNoTLS()), withMaxConcurrentStreams(3))
|
||||
}
|
||||
|
||||
func TestV3Curl_MaxStreams_BelowLimit_NoTLS_Medium(t *testing.T) {
|
||||
testV3CurlMaxStream(t, false, withCfg(*newConfigNoTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
testV3CurlMaxStream(t, false, withCfg(*e2e.NewConfigNoTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
}
|
||||
|
||||
/*
|
||||
@ -51,32 +52,32 @@ func TestV3Curl_MaxStreamsNoTLS_BelowLimit_Large(t *testing.T) {
|
||||
} */
|
||||
|
||||
func TestV3Curl_MaxStreams_ReachLimit_NoTLS_Small(t *testing.T) {
|
||||
testV3CurlMaxStream(t, true, withCfg(*newConfigNoTLS()), withMaxConcurrentStreams(3))
|
||||
testV3CurlMaxStream(t, true, withCfg(*e2e.NewConfigNoTLS()), withMaxConcurrentStreams(3))
|
||||
}
|
||||
|
||||
func TestV3Curl_MaxStreams_ReachLimit_NoTLS_Medium(t *testing.T) {
|
||||
testV3CurlMaxStream(t, true, withCfg(*newConfigNoTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
testV3CurlMaxStream(t, true, withCfg(*e2e.NewConfigNoTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
}
|
||||
|
||||
// TestV3Curl_MaxStreams_BelowLimit_TLS_Small tests with TLS
|
||||
func TestV3Curl_MaxStreams_BelowLimit_TLS_Small(t *testing.T) {
|
||||
testV3CurlMaxStream(t, false, withCfg(*newConfigTLS()), withMaxConcurrentStreams(3))
|
||||
testV3CurlMaxStream(t, false, withCfg(*e2e.NewConfigTLS()), withMaxConcurrentStreams(3))
|
||||
}
|
||||
|
||||
func TestV3Curl_MaxStreams_BelowLimit_TLS_Medium(t *testing.T) {
|
||||
testV3CurlMaxStream(t, false, withCfg(*newConfigTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
testV3CurlMaxStream(t, false, withCfg(*e2e.NewConfigTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
}
|
||||
|
||||
func TestV3Curl_MaxStreams_ReachLimit_TLS_Small(t *testing.T) {
|
||||
testV3CurlMaxStream(t, true, withCfg(*newConfigTLS()), withMaxConcurrentStreams(3))
|
||||
testV3CurlMaxStream(t, true, withCfg(*e2e.NewConfigTLS()), withMaxConcurrentStreams(3))
|
||||
}
|
||||
|
||||
func TestV3Curl_MaxStreams_ReachLimit_TLS_Medium(t *testing.T) {
|
||||
testV3CurlMaxStream(t, true, withCfg(*newConfigTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
testV3CurlMaxStream(t, true, withCfg(*e2e.NewConfigTLS()), withMaxConcurrentStreams(100), withTestTimeout(20*time.Second))
|
||||
}
|
||||
|
||||
func testV3CurlMaxStream(t *testing.T, reachLimit bool, opts ...ctlOption) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
// Step 1: generate configuration for creating cluster
|
||||
t.Log("Generating configuration for creating cluster.")
|
||||
@ -85,16 +86,16 @@ func testV3CurlMaxStream(t *testing.T, reachLimit bool, opts ...ctlOption) {
|
||||
// We must set the `ClusterSize` to 1, otherwise different streams may
|
||||
// connect to different members, accordingly it's difficult to test the
|
||||
// behavior.
|
||||
cx.cfg.clusterSize = 1
|
||||
cx.cfg.ClusterSize = 1
|
||||
|
||||
// Step 2: create the cluster
|
||||
t.Log("Creating an etcd cluster")
|
||||
epc, err := newEtcdProcessCluster(t, &cx.cfg)
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &cx.cfg)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start etcd cluster: %v", err)
|
||||
}
|
||||
cx.epc = epc
|
||||
cx.dataDir = epc.procs[0].Config().dataDirPath
|
||||
cx.dataDir = epc.Procs[0].Config().DataDirPath
|
||||
|
||||
// Step 3: run test
|
||||
// (a) generate ${concurrentNumber} concurrent watch streams;
|
||||
@ -163,7 +164,7 @@ func submitConcurrentWatch(cx ctlCtx, number int, wgDone *sync.WaitGroup, errCh
|
||||
go func(i int) {
|
||||
wgSchedule.Done()
|
||||
defer wgDone.Done()
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: "/v3/watch", value: string(watchData), expected: `"revision":"`}); err != nil {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: "/v3/watch", Value: string(watchData), Expected: `"revision":"`}); err != nil {
|
||||
werr := fmt.Errorf("testV3CurlMaxStream watch failed: %d, error: %v", i, err)
|
||||
cx.t.Error(werr)
|
||||
errCh <- werr
|
||||
@ -187,7 +188,7 @@ func submitRangeAfterConcurrentWatch(cx ctlCtx, expectedValue string) {
|
||||
}
|
||||
|
||||
cx.t.Log("Submitting range request...")
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: "/v3/kv/range", value: string(rangeData), expected: expectedValue, timeout: 5}); err != nil {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: "/v3/kv/range", Value: string(rangeData), Expected: expectedValue, Timeout: 5}); err != nil {
|
||||
cx.t.Fatalf("testV3CurlMaxStream get failed, error: %v", err)
|
||||
}
|
||||
cx.t.Log("range request done")
|
||||
|
@ -27,6 +27,7 @@ import (
|
||||
"go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
"go.etcd.io/etcd/client/pkg/v3/testutil"
|
||||
epb "go.etcd.io/etcd/server/v3/etcdserver/api/v3election/v3electionpb"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
|
||||
"github.com/grpc-ecosystem/grpc-gateway/runtime"
|
||||
)
|
||||
@ -36,27 +37,27 @@ var apiPrefix = []string{"/v3", "/v3beta"}
|
||||
|
||||
func TestV3CurlPutGetNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlPutGetAutoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*newConfigAutoTLS()))
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*e2e.NewConfigAutoTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlPutGetAllTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*newConfigTLS()))
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*e2e.NewConfigTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlPutGetPeerTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*newConfigPeerTLS()))
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*e2e.NewConfigPeerTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlPutGetClientTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*newConfigClientTLS()))
|
||||
testCtl(t, testV3CurlPutGet, withApiPrefix(p), withCfg(*e2e.NewConfigClientTLS()))
|
||||
}
|
||||
}
|
||||
func TestV3CurlWatch(t *testing.T) {
|
||||
@ -76,7 +77,7 @@ func TestV3CurlAuth(t *testing.T) {
|
||||
}
|
||||
func TestV3CurlAuthClientTLSCertAuth(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlAuth, withApiPrefix(p), withCfg(*newConfigClientTLSCertAuthWithNoCN()))
|
||||
testCtl(t, testV3CurlAuth, withApiPrefix(p), withCfg(*e2e.NewConfigClientTLSCertAuthWithNoCN()))
|
||||
}
|
||||
}
|
||||
|
||||
@ -104,14 +105,14 @@ func testV3CurlPutGet(cx ctlCtx) {
|
||||
|
||||
p := cx.apiPrefix
|
||||
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putData), expected: expectPut}); err != nil {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/put"), Value: string(putData), Expected: expectPut}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlPutGet put with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/range"), value: string(rangeData), expected: expectGet}); err != nil {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/range"), Value: string(rangeData), Expected: expectGet}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlPutGet get with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
if cx.cfg.clientTLS == clientTLSAndNonTLS {
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/range"), value: string(rangeData), expected: expectGet, isTLS: true}); err != nil {
|
||||
if cx.cfg.ClientTLS == e2e.ClientTLSAndNonTLS {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/range"), Value: string(rangeData), Expected: expectGet, IsTLS: true}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlPutGet get with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
}
|
||||
@ -135,11 +136,11 @@ func testV3CurlWatch(cx ctlCtx) {
|
||||
wstr := `{"create_request" : ` + string(wreq) + "}"
|
||||
p := cx.apiPrefix
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putreq), expected: "revision"}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/put"), Value: string(putreq), Expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlWatch put with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
// expects "bar", timeout after 2 seconds since stream waits forever
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/watch"), value: wstr, expected: `"YmFy"`, timeout: 2}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/watch"), Value: wstr, Expected: `"YmFy"`, Timeout: 2}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlWatch watch with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
}
|
||||
@ -172,13 +173,13 @@ func testV3CurlTxn(cx ctlCtx) {
|
||||
}
|
||||
expected := `"succeeded":true,"responses":[{"response_put":{"header":{"revision":"2"}}}]`
|
||||
p := cx.apiPrefix
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/txn"), value: string(jsonDat), expected: expected}); err != nil {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/txn"), Value: string(jsonDat), Expected: expected}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlTxn txn with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
// was crashing etcd server
|
||||
malformed := `{"compare":[{"result":0,"target":1,"key":"Zm9v","TargetUnion":null}],"success":[{"Request":{"RequestPut":{"key":"Zm9v","value":"YmFy"}}}]}`
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/txn"), value: malformed, expected: "error"}); err != nil {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/txn"), Value: malformed, Expected: "error"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlTxn put with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
@ -195,7 +196,7 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
user, err := json.Marshal(&pb.AuthUserAddRequest{Name: usernames[i], Password: pwds[i], Options: options[i]})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/add"), value: string(user), expected: "revision"}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/auth/user/add"), Value: string(user), Expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth add user %v with curl (%v)", usernames[i], err)
|
||||
}
|
||||
}
|
||||
@ -204,7 +205,7 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
rolereq, err := json.Marshal(&pb.AuthRoleAddRequest{Name: string("root")})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/role/add"), value: string(rolereq), expected: "revision"}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/auth/role/add"), Value: string(rolereq), Expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth create role with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
@ -213,13 +214,13 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
grantroleroot, err := json.Marshal(&pb.AuthUserGrantRoleRequest{User: usernames[i], Role: "root"})
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/user/grant"), value: string(grantroleroot), expected: "revision"}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/auth/user/grant"), Value: string(grantroleroot), Expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth grant role with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
}
|
||||
|
||||
// enable auth
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/auth/enable"), value: string("{}"), expected: "revision"}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/auth/enable"), Value: string("{}"), Expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth enable auth with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
@ -229,7 +230,7 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
testutil.AssertNil(cx.t, err)
|
||||
|
||||
// fail put no auth
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putreq), expected: "error"}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/put"), Value: string(putreq), Expected: "error"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth no auth put with curl using prefix (%s) (%v)", p, err)
|
||||
}
|
||||
|
||||
@ -243,8 +244,8 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
lineFunc = func(txt string) bool { return true }
|
||||
)
|
||||
|
||||
cmdArgs = cURLPrefixArgsCluster(cx.epc, "POST", cURLReq{endpoint: path.Join(p, "/auth/authenticate"), value: string(authreq)})
|
||||
proc, err := spawnCmd(cmdArgs, cx.envMap)
|
||||
cmdArgs = e2e.CURLPrefixArgsCluster(cx.epc, "POST", e2e.CURLReq{Endpoint: path.Join(p, "/auth/authenticate"), Value: string(authreq)})
|
||||
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
|
||||
testutil.AssertNil(cx.t, err)
|
||||
defer proc.Close()
|
||||
|
||||
@ -262,7 +263,7 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
authHeader = "Authorization: " + token
|
||||
|
||||
// put with auth
|
||||
if err = cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, "/kv/put"), value: string(putreq), header: authHeader, expected: "revision"}); err != nil {
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/kv/put"), Value: string(putreq), Header: authHeader, Expected: "revision"}); err != nil {
|
||||
cx.t.Fatalf("failed testV3CurlAuth auth put with curl using prefix (%s) and user (%v) (%v)", p, usernames[i], err)
|
||||
}
|
||||
}
|
||||
@ -270,7 +271,7 @@ func testV3CurlAuth(cx ctlCtx) {
|
||||
|
||||
func TestV3CurlCampaignNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlCampaign, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlCampaign, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
|
||||
@ -282,11 +283,11 @@ func testV3CurlCampaign(cx ctlCtx) {
|
||||
if err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
cargs := cURLPrefixArgsCluster(cx.epc, "POST", cURLReq{
|
||||
endpoint: path.Join(cx.apiPrefix, "/election/campaign"),
|
||||
value: string(cdata),
|
||||
cargs := e2e.CURLPrefixArgsCluster(cx.epc, "POST", e2e.CURLReq{
|
||||
Endpoint: path.Join(cx.apiPrefix, "/election/campaign"),
|
||||
Value: string(cdata),
|
||||
})
|
||||
lines, err := spawnWithExpectLines(cargs, cx.envMap, `"leader":{"name":"`)
|
||||
lines, err := e2e.SpawnWithExpectLines(cargs, cx.envMap, `"leader":{"name":"`)
|
||||
if err != nil {
|
||||
cx.t.Fatalf("failed post campaign request (%s) (%v)", cx.apiPrefix, err)
|
||||
}
|
||||
@ -321,10 +322,10 @@ func testV3CurlCampaign(cx ctlCtx) {
|
||||
if err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err = cURLPost(cx.epc, cURLReq{
|
||||
endpoint: path.Join(cx.apiPrefix, "/election/proclaim"),
|
||||
value: string(pdata),
|
||||
expected: `"revision":`,
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{
|
||||
Endpoint: path.Join(cx.apiPrefix, "/election/proclaim"),
|
||||
Value: string(pdata),
|
||||
Expected: `"revision":`,
|
||||
}); err != nil {
|
||||
cx.t.Fatalf("failed post proclaim request (%s) (%v)", cx.apiPrefix, err)
|
||||
}
|
||||
@ -332,7 +333,7 @@ func testV3CurlCampaign(cx ctlCtx) {
|
||||
|
||||
func TestV3CurlProclaimMissiongLeaderKeyNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlProclaimMissiongLeaderKey, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlProclaimMissiongLeaderKey, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
|
||||
@ -341,10 +342,10 @@ func testV3CurlProclaimMissiongLeaderKey(cx ctlCtx) {
|
||||
if err != nil {
|
||||
cx.t.Fatal(err)
|
||||
}
|
||||
if err = cURLPost(cx.epc, cURLReq{
|
||||
endpoint: path.Join(cx.apiPrefix, "/election/proclaim"),
|
||||
value: string(pdata),
|
||||
expected: `{"error":"\"leader\" field must be provided","code":2,"message":"\"leader\" field must be provided"}`,
|
||||
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{
|
||||
Endpoint: path.Join(cx.apiPrefix, "/election/proclaim"),
|
||||
Value: string(pdata),
|
||||
Expected: `{"error":"\"leader\" field must be provided","code":2,"message":"\"leader\" field must be provided"}`,
|
||||
}); err != nil {
|
||||
cx.t.Fatalf("failed post proclaim request (%s) (%v)", cx.apiPrefix, err)
|
||||
}
|
||||
@ -352,15 +353,15 @@ func testV3CurlProclaimMissiongLeaderKey(cx ctlCtx) {
|
||||
|
||||
func TestV3CurlResignMissiongLeaderKeyNoTLS(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlResignMissiongLeaderKey, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlResignMissiongLeaderKey, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
|
||||
func testV3CurlResignMissiongLeaderKey(cx ctlCtx) {
|
||||
if err := cURLPost(cx.epc, cURLReq{
|
||||
endpoint: path.Join(cx.apiPrefix, "/election/resign"),
|
||||
value: `{}`,
|
||||
expected: `{"error":"\"leader\" field must be provided","code":2,"message":"\"leader\" field must be provided"}`,
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{
|
||||
Endpoint: path.Join(cx.apiPrefix, "/election/resign"),
|
||||
Value: `{}`,
|
||||
Expected: `{"error":"\"leader\" field must be provided","code":2,"message":"\"leader\" field must be provided"}`,
|
||||
}); err != nil {
|
||||
cx.t.Fatalf("failed post resign request (%s) (%v)", cx.apiPrefix, err)
|
||||
}
|
||||
@ -368,42 +369,42 @@ func testV3CurlResignMissiongLeaderKey(cx ctlCtx) {
|
||||
|
||||
func TestV3CurlMaintenanceAlarmMissiongAlarm(t *testing.T) {
|
||||
for _, p := range apiPrefix {
|
||||
testCtl(t, testV3CurlMaintenanceAlarmMissiongAlarm, withApiPrefix(p), withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlMaintenanceAlarmMissiongAlarm, withApiPrefix(p), withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
}
|
||||
|
||||
func testV3CurlMaintenanceAlarmMissiongAlarm(cx ctlCtx) {
|
||||
if err := cURLPost(cx.epc, cURLReq{
|
||||
endpoint: path.Join(cx.apiPrefix, "/maintenance/alarm"),
|
||||
value: `{"action": "ACTIVATE"}`,
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{
|
||||
Endpoint: path.Join(cx.apiPrefix, "/maintenance/alarm"),
|
||||
Value: `{"action": "ACTIVATE"}`,
|
||||
}); err != nil {
|
||||
cx.t.Fatalf("failed post maintenance alarm (%s) (%v)", cx.apiPrefix, err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestV3CurlMaintenanceHash(t *testing.T) {
|
||||
testCtl(t, testV3CurlMaintenanceHash, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlMaintenanceHash, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
|
||||
func testV3CurlMaintenanceHash(cx ctlCtx) {
|
||||
if err := cURLPost(cx.epc, cURLReq{
|
||||
endpoint: "/v3/maintenance/hash",
|
||||
value: "{}",
|
||||
expected: `,"revision":"1","raft_term":"2"},"hash":`,
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{
|
||||
Endpoint: "/v3/maintenance/hash",
|
||||
Value: "{}",
|
||||
Expected: `,"revision":"1","raft_term":"2"},"hash":`,
|
||||
}); err != nil {
|
||||
cx.t.Fatalf("failed post maintenance hash request (%s) (%v)", cx.apiPrefix, err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestV3CurlMaintenanceHashKV(t *testing.T) {
|
||||
testCtl(t, testV3CurlMaintenanceHashKV, withCfg(*newConfigNoTLS()))
|
||||
testCtl(t, testV3CurlMaintenanceHashKV, withCfg(*e2e.NewConfigNoTLS()))
|
||||
}
|
||||
|
||||
func testV3CurlMaintenanceHashKV(cx ctlCtx) {
|
||||
if err := cURLPost(cx.epc, cURLReq{
|
||||
endpoint: "/v3/maintenance/hashkv",
|
||||
value: `{"revision": 1}`,
|
||||
expected: `,"compact_revision":`,
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{
|
||||
Endpoint: "/v3/maintenance/hashkv",
|
||||
Value: `{"revision": 1}`,
|
||||
Expected: `,"compact_revision":`,
|
||||
}); err != nil {
|
||||
cx.t.Fatalf("failed post maintenance hashKV request (%s) (%v)", cx.apiPrefix, err)
|
||||
}
|
||||
@ -420,11 +421,11 @@ type campaignResponse struct {
|
||||
} `json:"leader,omitempty"`
|
||||
}
|
||||
|
||||
func cURLWithExpected(cx ctlCtx, tests []v3cURLTest) error {
|
||||
func CURLWithExpected(cx ctlCtx, tests []v3cURLTest) error {
|
||||
p := cx.apiPrefix
|
||||
for _, t := range tests {
|
||||
value := fmt.Sprintf("%v", t.value)
|
||||
if err := cURLPost(cx.epc, cURLReq{endpoint: path.Join(p, t.endpoint), value: value, expected: t.expected}); err != nil {
|
||||
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, t.endpoint), Value: value, Expected: t.expected}); err != nil {
|
||||
return fmt.Errorf("prefix (%s) endpoint (%s): error (%v), wanted %v", p, t.endpoint, err, t.expected)
|
||||
}
|
||||
}
|
||||
|
@ -27,6 +27,7 @@ import (
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
"golang.org/x/sync/errgroup"
|
||||
)
|
||||
|
||||
@ -38,7 +39,7 @@ const (
|
||||
|
||||
type testCase struct {
|
||||
name string
|
||||
config etcdProcessClusterConfig
|
||||
config e2e.EtcdProcessClusterConfig
|
||||
maxWatchDelay time.Duration
|
||||
dbSizeBytes int
|
||||
}
|
||||
@ -54,40 +55,40 @@ const (
|
||||
var tcs = []testCase{
|
||||
{
|
||||
name: "NoTLS",
|
||||
config: etcdProcessClusterConfig{clusterSize: 1},
|
||||
config: e2e.EtcdProcessClusterConfig{ClusterSize: 1},
|
||||
maxWatchDelay: 150 * time.Millisecond,
|
||||
dbSizeBytes: 5 * Mega,
|
||||
},
|
||||
{
|
||||
name: "TLS",
|
||||
config: etcdProcessClusterConfig{clusterSize: 1, isClientAutoTLS: true, clientTLS: clientTLS},
|
||||
config: e2e.EtcdProcessClusterConfig{ClusterSize: 1, IsClientAutoTLS: true, ClientTLS: e2e.ClientTLS},
|
||||
maxWatchDelay: 150 * time.Millisecond,
|
||||
dbSizeBytes: 5 * Mega,
|
||||
},
|
||||
{
|
||||
name: "SeparateHttpNoTLS",
|
||||
config: etcdProcessClusterConfig{clusterSize: 1, clientHttpSeparate: true},
|
||||
config: e2e.EtcdProcessClusterConfig{ClusterSize: 1, ClientHttpSeparate: true},
|
||||
maxWatchDelay: 150 * time.Millisecond,
|
||||
dbSizeBytes: 5 * Mega,
|
||||
},
|
||||
{
|
||||
name: "SeparateHttpTLS",
|
||||
config: etcdProcessClusterConfig{clusterSize: 1, isClientAutoTLS: true, clientTLS: clientTLS, clientHttpSeparate: true},
|
||||
config: e2e.EtcdProcessClusterConfig{ClusterSize: 1, IsClientAutoTLS: true, ClientTLS: e2e.ClientTLS, ClientHttpSeparate: true},
|
||||
maxWatchDelay: 150 * time.Millisecond,
|
||||
dbSizeBytes: 5 * Mega,
|
||||
},
|
||||
}
|
||||
|
||||
func TestWatchDelayForPeriodicProgressNotification(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
for _, tc := range tcs {
|
||||
tc := tc
|
||||
tc.config.WatchProcessNotifyInterval = watchResponsePeriod
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
clus, err := newEtcdProcessCluster(t, &tc.config)
|
||||
clus, err := e2e.NewEtcdProcessCluster(t, &tc.config)
|
||||
require.NoError(t, err)
|
||||
defer clus.Close()
|
||||
c := newClient(t, clus.EndpointsV3(), tc.config.clientTLS, tc.config.isClientAutoTLS)
|
||||
c := newClient(t, clus.EndpointsV3(), tc.config.ClientTLS, tc.config.IsClientAutoTLS)
|
||||
require.NoError(t, fillEtcdWithData(context.Background(), c, tc.dbSizeBytes))
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), watchTestDuration)
|
||||
@ -101,13 +102,13 @@ func TestWatchDelayForPeriodicProgressNotification(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestWatchDelayForManualProgressNotification(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
for _, tc := range tcs {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
clus, err := newEtcdProcessCluster(t, &tc.config)
|
||||
clus, err := e2e.NewEtcdProcessCluster(t, &tc.config)
|
||||
require.NoError(t, err)
|
||||
defer clus.Close()
|
||||
c := newClient(t, clus.EndpointsV3(), tc.config.clientTLS, tc.config.isClientAutoTLS)
|
||||
c := newClient(t, clus.EndpointsV3(), tc.config.ClientTLS, tc.config.IsClientAutoTLS)
|
||||
require.NoError(t, fillEtcdWithData(context.Background(), c, tc.dbSizeBytes))
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), watchTestDuration)
|
||||
@ -133,13 +134,13 @@ func TestWatchDelayForManualProgressNotification(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestWatchDelayForEvent(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
for _, tc := range tcs {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
clus, err := newEtcdProcessCluster(t, &tc.config)
|
||||
clus, err := e2e.NewEtcdProcessCluster(t, &tc.config)
|
||||
require.NoError(t, err)
|
||||
defer clus.Close()
|
||||
c := newClient(t, clus.EndpointsV3(), tc.config.clientTLS, tc.config.isClientAutoTLS)
|
||||
c := newClient(t, clus.EndpointsV3(), tc.config.ClientTLS, tc.config.IsClientAutoTLS)
|
||||
require.NoError(t, fillEtcdWithData(context.Background(), c, tc.dbSizeBytes))
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), watchTestDuration)
|
||||
|
@ -21,20 +21,22 @@ import (
|
||||
"encoding/json"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||
)
|
||||
|
||||
func TestServerJsonLogging(t *testing.T) {
|
||||
BeforeTest(t)
|
||||
e2e.BeforeTest(t)
|
||||
|
||||
epc, err := newEtcdProcessCluster(t, &etcdProcessClusterConfig{
|
||||
clusterSize: 1,
|
||||
initialToken: "new",
|
||||
logLevel: "debug",
|
||||
epc, err := e2e.NewEtcdProcessCluster(t, &e2e.EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
InitialToken: "new",
|
||||
LogLevel: "debug",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("could not start etcd process cluster (%v)", err)
|
||||
}
|
||||
logs := epc.procs[0].Logs()
|
||||
logs := epc.Procs[0].Logs()
|
||||
time.Sleep(time.Second)
|
||||
if err = epc.Close(); err != nil {
|
||||
t.Fatalf("error closing etcd processes (%v)", err)
|
||||
|
540
tests/framework/e2e/cluster.go
Normal file
540
tests/framework/e2e/cluster.go
Normal file
@ -0,0 +1,540 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/url"
|
||||
"os"
|
||||
"path"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/server/v3/etcdserver"
|
||||
"go.uber.org/zap"
|
||||
"go.uber.org/zap/zaptest"
|
||||
)
|
||||
|
||||
const EtcdProcessBasePort = 20000
|
||||
|
||||
type ClientConnType int
|
||||
|
||||
const (
|
||||
ClientNonTLS ClientConnType = iota
|
||||
ClientTLS
|
||||
ClientTLSAndNonTLS
|
||||
)
|
||||
|
||||
func NewConfigNoTLS() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{ClusterSize: 3,
|
||||
InitialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigAutoTLS() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
IsPeerTLS: true,
|
||||
IsPeerAutoTLS: true,
|
||||
InitialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigTLS() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
ClientTLS: ClientTLS,
|
||||
IsPeerTLS: true,
|
||||
InitialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigClientTLS() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
ClientTLS: ClientTLS,
|
||||
InitialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigClientBoth() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
ClientTLS: ClientTLSAndNonTLS,
|
||||
InitialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigClientAutoTLS() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
IsClientAutoTLS: true,
|
||||
ClientTLS: ClientTLS,
|
||||
InitialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigPeerTLS() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 3,
|
||||
IsPeerTLS: true,
|
||||
InitialToken: "new",
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigClientTLSCertAuth() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
ClientTLS: ClientTLS,
|
||||
InitialToken: "new",
|
||||
ClientCertAuthEnabled: true,
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigClientTLSCertAuthWithNoCN() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
ClientTLS: ClientTLS,
|
||||
InitialToken: "new",
|
||||
ClientCertAuthEnabled: true,
|
||||
NoCN: true,
|
||||
}
|
||||
}
|
||||
|
||||
func NewConfigJWT() *EtcdProcessClusterConfig {
|
||||
return &EtcdProcessClusterConfig{
|
||||
ClusterSize: 1,
|
||||
InitialToken: "new",
|
||||
AuthTokenOpts: "jwt,pub-key=" + path.Join(FixturesDir, "server.crt") +
|
||||
",priv-key=" + path.Join(FixturesDir, "server.key.insecure") + ",sign-method=RS256,ttl=1s",
|
||||
}
|
||||
}
|
||||
|
||||
func ConfigStandalone(cfg EtcdProcessClusterConfig) *EtcdProcessClusterConfig {
|
||||
ret := cfg
|
||||
ret.ClusterSize = 1
|
||||
return &ret
|
||||
}
|
||||
|
||||
type EtcdProcessCluster struct {
|
||||
lg *zap.Logger
|
||||
Cfg *EtcdProcessClusterConfig
|
||||
Procs []EtcdProcess
|
||||
}
|
||||
|
||||
type EtcdProcessClusterConfig struct {
|
||||
ExecPath string
|
||||
DataDirPath string
|
||||
KeepDataDir bool
|
||||
EnvVars map[string]string
|
||||
|
||||
ClusterSize int
|
||||
|
||||
BaseScheme string
|
||||
BasePort int
|
||||
|
||||
MetricsURLScheme string
|
||||
|
||||
SnapshotCount int // default is 10000
|
||||
|
||||
ClientTLS ClientConnType
|
||||
ClientCertAuthEnabled bool
|
||||
ClientHttpSeparate bool
|
||||
IsPeerTLS bool
|
||||
IsPeerAutoTLS bool
|
||||
IsClientAutoTLS bool
|
||||
IsClientCRL bool
|
||||
NoCN bool
|
||||
|
||||
CipherSuites []string
|
||||
|
||||
ForceNewCluster bool
|
||||
InitialToken string
|
||||
QuotaBackendBytes int64
|
||||
NoStrictReconfig bool
|
||||
EnableV2 bool
|
||||
InitialCorruptCheck bool
|
||||
AuthTokenOpts string
|
||||
V2deprecation string
|
||||
|
||||
RollingStart bool
|
||||
LogLevel string
|
||||
|
||||
MaxConcurrentStreams uint32 // default is math.MaxUint32
|
||||
CorruptCheckTime time.Duration
|
||||
CompactHashCheckEnabled bool
|
||||
CompactHashCheckTime time.Duration
|
||||
WatchProcessNotifyInterval time.Duration
|
||||
CompactionBatchLimit int
|
||||
}
|
||||
|
||||
// NewEtcdProcessCluster launches a new cluster from etcd processes, returning
|
||||
// a new NewEtcdProcessCluster once all nodes are ready to accept client requests.
|
||||
func NewEtcdProcessCluster(t testing.TB, cfg *EtcdProcessClusterConfig) (*EtcdProcessCluster, error) {
|
||||
epc, err := InitEtcdProcessCluster(t, cfg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return StartEtcdProcessCluster(epc, cfg)
|
||||
}
|
||||
|
||||
// InitEtcdProcessCluster initializes a new cluster based on the given config.
|
||||
// It doesn't start the cluster.
|
||||
func InitEtcdProcessCluster(t testing.TB, cfg *EtcdProcessClusterConfig) (*EtcdProcessCluster, error) {
|
||||
SkipInShortMode(t)
|
||||
|
||||
etcdCfgs := cfg.EtcdServerProcessConfigs(t)
|
||||
epc := &EtcdProcessCluster{
|
||||
Cfg: cfg,
|
||||
lg: zaptest.NewLogger(t),
|
||||
Procs: make([]EtcdProcess, cfg.ClusterSize),
|
||||
}
|
||||
|
||||
// launch etcd processes
|
||||
for i := range etcdCfgs {
|
||||
proc, err := NewEtcdProcess(etcdCfgs[i])
|
||||
if err != nil {
|
||||
epc.Close()
|
||||
return nil, fmt.Errorf("Cannot configure: %v", err)
|
||||
}
|
||||
epc.Procs[i] = proc
|
||||
}
|
||||
return epc, nil
|
||||
}
|
||||
|
||||
// StartEtcdProcessCluster launches a new cluster from etcd processes.
|
||||
func StartEtcdProcessCluster(epc *EtcdProcessCluster, cfg *EtcdProcessClusterConfig) (*EtcdProcessCluster, error) {
|
||||
if cfg.RollingStart {
|
||||
if err := epc.RollingStart(); err != nil {
|
||||
return nil, fmt.Errorf("Cannot rolling-start: %v", err)
|
||||
}
|
||||
} else {
|
||||
if err := epc.Start(); err != nil {
|
||||
return nil, fmt.Errorf("Cannot start: %v", err)
|
||||
}
|
||||
}
|
||||
return epc, nil
|
||||
}
|
||||
|
||||
func (cfg *EtcdProcessClusterConfig) ClientScheme() string {
|
||||
if cfg.ClientTLS == ClientTLS {
|
||||
return "https"
|
||||
}
|
||||
return "http"
|
||||
}
|
||||
|
||||
func (cfg *EtcdProcessClusterConfig) PeerScheme() string {
|
||||
peerScheme := cfg.BaseScheme
|
||||
if peerScheme == "" {
|
||||
peerScheme = "http"
|
||||
}
|
||||
if cfg.IsPeerTLS {
|
||||
peerScheme += "s"
|
||||
}
|
||||
return peerScheme
|
||||
}
|
||||
|
||||
func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*EtcdServerProcessConfig {
|
||||
lg := zaptest.NewLogger(tb)
|
||||
|
||||
if cfg.BasePort == 0 {
|
||||
cfg.BasePort = EtcdProcessBasePort
|
||||
}
|
||||
if cfg.ExecPath == "" {
|
||||
cfg.ExecPath = BinPath
|
||||
}
|
||||
if cfg.SnapshotCount == 0 {
|
||||
cfg.SnapshotCount = etcdserver.DefaultSnapshotCount
|
||||
}
|
||||
|
||||
etcdCfgs := make([]*EtcdServerProcessConfig, cfg.ClusterSize)
|
||||
initialCluster := make([]string, cfg.ClusterSize)
|
||||
for i := 0; i < cfg.ClusterSize; i++ {
|
||||
var curls []string
|
||||
var curl string
|
||||
port := cfg.BasePort + 5*i
|
||||
clientPort := port
|
||||
clientHttpPort := port + 4
|
||||
|
||||
if cfg.ClientTLS == ClientTLSAndNonTLS {
|
||||
curl = clientURL(clientPort, ClientNonTLS)
|
||||
curls = []string{curl, clientURL(clientPort, ClientTLS)}
|
||||
} else {
|
||||
curl = clientURL(clientPort, cfg.ClientTLS)
|
||||
curls = []string{curl}
|
||||
}
|
||||
|
||||
purl := url.URL{Scheme: cfg.PeerScheme(), Host: fmt.Sprintf("localhost:%d", port+1)}
|
||||
name := fmt.Sprintf("test-%d", i)
|
||||
dataDirPath := cfg.DataDirPath
|
||||
if cfg.DataDirPath == "" {
|
||||
dataDirPath = tb.TempDir()
|
||||
}
|
||||
initialCluster[i] = fmt.Sprintf("%s=%s", name, purl.String())
|
||||
|
||||
args := []string{
|
||||
"--name", name,
|
||||
"--listen-client-urls", strings.Join(curls, ","),
|
||||
"--advertise-client-urls", strings.Join(curls, ","),
|
||||
"--listen-peer-urls", purl.String(),
|
||||
"--initial-advertise-peer-urls", purl.String(),
|
||||
"--initial-cluster-token", cfg.InitialToken,
|
||||
"--data-dir", dataDirPath,
|
||||
"--snapshot-count", fmt.Sprintf("%d", cfg.SnapshotCount),
|
||||
}
|
||||
var clientHttpUrl string
|
||||
if cfg.ClientHttpSeparate {
|
||||
clientHttpUrl = clientURL(clientHttpPort, cfg.ClientTLS)
|
||||
args = append(args, "--listen-client-http-urls", clientHttpUrl)
|
||||
}
|
||||
args = AddV2Args(args)
|
||||
if cfg.ForceNewCluster {
|
||||
args = append(args, "--force-new-cluster")
|
||||
}
|
||||
if cfg.QuotaBackendBytes > 0 {
|
||||
args = append(args,
|
||||
"--quota-backend-bytes", fmt.Sprintf("%d", cfg.QuotaBackendBytes),
|
||||
)
|
||||
}
|
||||
if cfg.NoStrictReconfig {
|
||||
args = append(args, "--strict-reconfig-check=false")
|
||||
}
|
||||
if cfg.EnableV2 {
|
||||
args = append(args, "--enable-v2")
|
||||
}
|
||||
if cfg.InitialCorruptCheck {
|
||||
args = append(args, "--experimental-initial-corrupt-check")
|
||||
}
|
||||
var murl string
|
||||
if cfg.MetricsURLScheme != "" {
|
||||
murl = (&url.URL{
|
||||
Scheme: cfg.MetricsURLScheme,
|
||||
Host: fmt.Sprintf("localhost:%d", port+2),
|
||||
}).String()
|
||||
args = append(args, "--listen-metrics-urls", murl)
|
||||
}
|
||||
|
||||
args = append(args, cfg.TlsArgs()...)
|
||||
|
||||
if cfg.AuthTokenOpts != "" {
|
||||
args = append(args, "--auth-token", cfg.AuthTokenOpts)
|
||||
}
|
||||
|
||||
if cfg.V2deprecation != "" {
|
||||
args = append(args, "--v2-deprecation", cfg.V2deprecation)
|
||||
}
|
||||
|
||||
if cfg.LogLevel != "" {
|
||||
args = append(args, "--log-level", cfg.LogLevel)
|
||||
}
|
||||
|
||||
if cfg.MaxConcurrentStreams != 0 {
|
||||
args = append(args, "--max-concurrent-streams", fmt.Sprintf("%d", cfg.MaxConcurrentStreams))
|
||||
}
|
||||
|
||||
if cfg.CorruptCheckTime != 0 {
|
||||
args = append(args, "--experimental-corrupt-check-time", fmt.Sprintf("%s", cfg.CorruptCheckTime))
|
||||
}
|
||||
if cfg.CompactHashCheckEnabled {
|
||||
args = append(args, "--experimental-compact-hash-check-enabled")
|
||||
}
|
||||
if cfg.CompactHashCheckTime != 0 {
|
||||
args = append(args, "--experimental-compact-hash-check-time", cfg.CompactHashCheckTime.String())
|
||||
}
|
||||
if cfg.WatchProcessNotifyInterval != 0 {
|
||||
args = append(args, "--experimental-watch-progress-notify-interval", cfg.WatchProcessNotifyInterval.String())
|
||||
}
|
||||
if cfg.CompactionBatchLimit != 0 {
|
||||
args = append(args, "--experimental-compaction-batch-limit", fmt.Sprintf("%d", cfg.CompactionBatchLimit))
|
||||
}
|
||||
|
||||
etcdCfgs[i] = &EtcdServerProcessConfig{
|
||||
lg: lg,
|
||||
ExecPath: cfg.ExecPath,
|
||||
Args: args,
|
||||
EnvVars: cfg.EnvVars,
|
||||
TlsArgs: cfg.TlsArgs(),
|
||||
DataDirPath: dataDirPath,
|
||||
KeepDataDir: cfg.KeepDataDir,
|
||||
Name: name,
|
||||
Purl: purl,
|
||||
Acurl: curl,
|
||||
Murl: murl,
|
||||
InitialToken: cfg.InitialToken,
|
||||
ClientHttpUrl: clientHttpUrl,
|
||||
}
|
||||
}
|
||||
|
||||
initialClusterArgs := []string{"--initial-cluster", strings.Join(initialCluster, ",")}
|
||||
for i := range etcdCfgs {
|
||||
etcdCfgs[i].InitialCluster = strings.Join(initialCluster, ",")
|
||||
etcdCfgs[i].Args = append(etcdCfgs[i].Args, initialClusterArgs...)
|
||||
}
|
||||
|
||||
return etcdCfgs
|
||||
}
|
||||
|
||||
func clientURL(port int, connType ClientConnType) string {
|
||||
curlHost := fmt.Sprintf("localhost:%d", port)
|
||||
switch connType {
|
||||
case ClientNonTLS:
|
||||
return (&url.URL{Scheme: "http", Host: curlHost}).String()
|
||||
case ClientTLS:
|
||||
return (&url.URL{Scheme: "https", Host: curlHost}).String()
|
||||
default:
|
||||
panic(fmt.Sprintf("Unsupported connection type %v", connType))
|
||||
}
|
||||
}
|
||||
|
||||
func (cfg *EtcdProcessClusterConfig) TlsArgs() (args []string) {
|
||||
if cfg.ClientTLS != ClientNonTLS {
|
||||
if cfg.IsClientAutoTLS {
|
||||
args = append(args, "--auto-tls")
|
||||
} else {
|
||||
tlsClientArgs := []string{
|
||||
"--cert-file", CertPath,
|
||||
"--key-file", PrivateKeyPath,
|
||||
"--trusted-ca-file", CaPath,
|
||||
}
|
||||
args = append(args, tlsClientArgs...)
|
||||
|
||||
if cfg.ClientCertAuthEnabled {
|
||||
args = append(args, "--client-cert-auth")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if cfg.IsPeerTLS {
|
||||
if cfg.IsPeerAutoTLS {
|
||||
args = append(args, "--peer-auto-tls")
|
||||
} else {
|
||||
tlsPeerArgs := []string{
|
||||
"--peer-cert-file", CertPath,
|
||||
"--peer-key-file", PrivateKeyPath,
|
||||
"--peer-trusted-ca-file", CaPath,
|
||||
}
|
||||
args = append(args, tlsPeerArgs...)
|
||||
}
|
||||
}
|
||||
|
||||
if cfg.IsClientCRL {
|
||||
args = append(args, "--client-crl-file", CrlPath, "--client-cert-auth")
|
||||
}
|
||||
|
||||
if len(cfg.CipherSuites) > 0 {
|
||||
args = append(args, "--cipher-suites", strings.Join(cfg.CipherSuites, ","))
|
||||
}
|
||||
|
||||
return args
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) EndpointsV2() []string {
|
||||
return epc.Endpoints(func(ep EtcdProcess) []string { return ep.EndpointsV2() })
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) EndpointsV3() []string {
|
||||
return epc.Endpoints(func(ep EtcdProcess) []string { return ep.EndpointsV3() })
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) Endpoints(f func(ep EtcdProcess) []string) (ret []string) {
|
||||
for _, p := range epc.Procs {
|
||||
ret = append(ret, f(p)...)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) Start() error {
|
||||
return epc.start(func(ep EtcdProcess) error { return ep.Start() })
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) RollingStart() error {
|
||||
return epc.rollingStart(func(ep EtcdProcess) error { return ep.Start() })
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) Restart() error {
|
||||
return epc.start(func(ep EtcdProcess) error { return ep.Restart() })
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) start(f func(ep EtcdProcess) error) error {
|
||||
readyC := make(chan error, len(epc.Procs))
|
||||
for i := range epc.Procs {
|
||||
go func(n int) { readyC <- f(epc.Procs[n]) }(i)
|
||||
}
|
||||
for range epc.Procs {
|
||||
if err := <-readyC; err != nil {
|
||||
epc.Close()
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) rollingStart(f func(ep EtcdProcess) error) error {
|
||||
readyC := make(chan error, len(epc.Procs))
|
||||
for i := range epc.Procs {
|
||||
go func(n int) { readyC <- f(epc.Procs[n]) }(i)
|
||||
// make sure the servers do not start at the same time
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
for range epc.Procs {
|
||||
if err := <-readyC; err != nil {
|
||||
epc.Close()
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) Stop() (err error) {
|
||||
for _, p := range epc.Procs {
|
||||
if p == nil {
|
||||
continue
|
||||
}
|
||||
if curErr := p.Stop(); curErr != nil {
|
||||
if err != nil {
|
||||
err = fmt.Errorf("%v; %v", err, curErr)
|
||||
} else {
|
||||
err = curErr
|
||||
}
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) Close() error {
|
||||
epc.lg.Info("closing test cluster...")
|
||||
err := epc.Stop()
|
||||
for _, p := range epc.Procs {
|
||||
// p is nil when NewEtcdProcess fails in the middle
|
||||
// Close still gets called to clean up test data
|
||||
if p == nil {
|
||||
continue
|
||||
}
|
||||
if cerr := p.Close(); cerr != nil {
|
||||
err = cerr
|
||||
}
|
||||
}
|
||||
epc.lg.Info("closed test cluster.")
|
||||
return err
|
||||
}
|
||||
|
||||
func (epc *EtcdProcessCluster) WithStopSignal(sig os.Signal) (ret os.Signal) {
|
||||
for _, p := range epc.Procs {
|
||||
ret = p.WithStopSignal(sig)
|
||||
}
|
||||
return ret
|
||||
}
|
@ -17,6 +17,6 @@
|
||||
|
||||
package e2e
|
||||
|
||||
func newEtcdProcess(cfg *etcdServerProcessConfig) (etcdProcess, error) {
|
||||
return newEtcdServerProcess(cfg)
|
||||
func NewEtcdProcess(cfg *EtcdServerProcessConfig) (EtcdProcess, error) {
|
||||
return NewEtcdServerProcess(cfg)
|
||||
}
|
@ -31,17 +31,17 @@ import (
|
||||
)
|
||||
|
||||
type proxyEtcdProcess struct {
|
||||
etcdProc etcdProcess
|
||||
etcdProc EtcdProcess
|
||||
proxyV2 *proxyV2Proc
|
||||
proxyV3 *proxyV3Proc
|
||||
}
|
||||
|
||||
func newEtcdProcess(cfg *etcdServerProcessConfig) (etcdProcess, error) {
|
||||
return newProxyEtcdProcess(cfg)
|
||||
func NewEtcdProcess(cfg *EtcdServerProcessConfig) (EtcdProcess, error) {
|
||||
return NewProxyEtcdProcess(cfg)
|
||||
}
|
||||
|
||||
func newProxyEtcdProcess(cfg *etcdServerProcessConfig) (*proxyEtcdProcess, error) {
|
||||
ep, err := newEtcdServerProcess(cfg)
|
||||
func NewProxyEtcdProcess(cfg *EtcdServerProcessConfig) (*proxyEtcdProcess, error) {
|
||||
ep, err := NewEtcdServerProcess(cfg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -53,7 +53,7 @@ func newProxyEtcdProcess(cfg *etcdServerProcessConfig) (*proxyEtcdProcess, error
|
||||
return pep, nil
|
||||
}
|
||||
|
||||
func (p *proxyEtcdProcess) Config() *etcdServerProcessConfig { return p.etcdProc.Config() }
|
||||
func (p *proxyEtcdProcess) Config() *EtcdServerProcessConfig { return p.etcdProc.Config() }
|
||||
|
||||
func (p *proxyEtcdProcess) EndpointsV2() []string { return p.EndpointsHTTP() }
|
||||
func (p *proxyEtcdProcess) EndpointsV3() []string { return p.EndpointsGRPC() }
|
||||
@ -117,7 +117,7 @@ func (p *proxyEtcdProcess) WithStopSignal(sig os.Signal) os.Signal {
|
||||
return p.etcdProc.WithStopSignal(sig)
|
||||
}
|
||||
|
||||
func (p *proxyEtcdProcess) Logs() logsExpect {
|
||||
func (p *proxyEtcdProcess) Logs() LogsExpect {
|
||||
return p.etcdProc.Logs()
|
||||
}
|
||||
|
||||
@ -138,7 +138,7 @@ func (pp *proxyProc) start() error {
|
||||
if pp.proc != nil {
|
||||
panic("already started")
|
||||
}
|
||||
proc, err := spawnCmdWithLogger(pp.lg, append([]string{pp.execPath}, pp.args...), nil)
|
||||
proc, err := SpawnCmdWithLogger(pp.lg, append([]string{pp.execPath}, pp.args...), nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -148,7 +148,7 @@ func (pp *proxyProc) start() error {
|
||||
|
||||
func (pp *proxyProc) waitReady(readyStr string) error {
|
||||
defer close(pp.donec)
|
||||
return waitReadyExpectProc(pp.proc, []string{readyStr})
|
||||
return WaitReadyExpectProc(pp.proc, []string{readyStr})
|
||||
}
|
||||
|
||||
func (pp *proxyProc) Stop() error {
|
||||
@ -178,8 +178,8 @@ type proxyV2Proc struct {
|
||||
dataDir string
|
||||
}
|
||||
|
||||
func proxyListenURL(cfg *etcdServerProcessConfig, portOffset int) string {
|
||||
u, err := url.Parse(cfg.acurl)
|
||||
func proxyListenURL(cfg *EtcdServerProcessConfig, portOffset int) string {
|
||||
u, err := url.Parse(cfg.Acurl)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
@ -189,22 +189,22 @@ func proxyListenURL(cfg *etcdServerProcessConfig, portOffset int) string {
|
||||
return u.String()
|
||||
}
|
||||
|
||||
func newProxyV2Proc(cfg *etcdServerProcessConfig) *proxyV2Proc {
|
||||
func newProxyV2Proc(cfg *EtcdServerProcessConfig) *proxyV2Proc {
|
||||
listenAddr := proxyListenURL(cfg, 2)
|
||||
name := fmt.Sprintf("testname-proxy-%p", cfg)
|
||||
dataDir := path.Join(cfg.dataDirPath, name+".etcd")
|
||||
dataDir := path.Join(cfg.DataDirPath, name+".etcd")
|
||||
args := []string{
|
||||
"--name", name,
|
||||
"--proxy", "on",
|
||||
"--listen-client-urls", listenAddr,
|
||||
"--initial-cluster", cfg.name + "=" + cfg.purl.String(),
|
||||
"--initial-cluster", cfg.Name + "=" + cfg.Purl.String(),
|
||||
"--data-dir", dataDir,
|
||||
}
|
||||
return &proxyV2Proc{
|
||||
proxyProc: proxyProc{
|
||||
lg: cfg.lg,
|
||||
execPath: cfg.execPath,
|
||||
args: append(args, cfg.tlsArgs...),
|
||||
execPath: cfg.ExecPath,
|
||||
args: append(args, cfg.TlsArgs...),
|
||||
ep: listenAddr,
|
||||
donec: make(chan struct{}),
|
||||
},
|
||||
@ -241,33 +241,33 @@ type proxyV3Proc struct {
|
||||
proxyProc
|
||||
}
|
||||
|
||||
func newProxyV3Proc(cfg *etcdServerProcessConfig) *proxyV3Proc {
|
||||
func newProxyV3Proc(cfg *EtcdServerProcessConfig) *proxyV3Proc {
|
||||
listenAddr := proxyListenURL(cfg, 3)
|
||||
args := []string{
|
||||
"grpc-proxy",
|
||||
"start",
|
||||
"--listen-addr", strings.Split(listenAddr, "/")[2],
|
||||
"--endpoints", cfg.acurl,
|
||||
"--endpoints", cfg.Acurl,
|
||||
// pass-through member RPCs
|
||||
"--advertise-client-url", "",
|
||||
"--data-dir", cfg.dataDirPath,
|
||||
"--data-dir", cfg.DataDirPath,
|
||||
}
|
||||
murl := ""
|
||||
if cfg.murl != "" {
|
||||
if cfg.Murl != "" {
|
||||
murl = proxyListenURL(cfg, 4)
|
||||
args = append(args, "--metrics-addr", murl)
|
||||
}
|
||||
tlsArgs := []string{}
|
||||
for i := 0; i < len(cfg.tlsArgs); i++ {
|
||||
switch cfg.tlsArgs[i] {
|
||||
for i := 0; i < len(cfg.TlsArgs); i++ {
|
||||
switch cfg.TlsArgs[i] {
|
||||
case "--cert-file":
|
||||
tlsArgs = append(tlsArgs, "--cert-file", cfg.tlsArgs[i+1])
|
||||
tlsArgs = append(tlsArgs, "--cert-file", cfg.TlsArgs[i+1])
|
||||
i++
|
||||
case "--key-file":
|
||||
tlsArgs = append(tlsArgs, "--key-file", cfg.tlsArgs[i+1])
|
||||
tlsArgs = append(tlsArgs, "--key-file", cfg.TlsArgs[i+1])
|
||||
i++
|
||||
case "--trusted-ca-file":
|
||||
tlsArgs = append(tlsArgs, "--trusted-ca-file", cfg.tlsArgs[i+1])
|
||||
tlsArgs = append(tlsArgs, "--trusted-ca-file", cfg.TlsArgs[i+1])
|
||||
i++
|
||||
case "--auto-tls":
|
||||
tlsArgs = append(tlsArgs, "--auto-tls", "--insecure-skip-tls-verify")
|
||||
@ -275,21 +275,21 @@ func newProxyV3Proc(cfg *etcdServerProcessConfig) *proxyV3Proc {
|
||||
i++ // skip arg
|
||||
case "--client-cert-auth", "--peer-auto-tls":
|
||||
default:
|
||||
tlsArgs = append(tlsArgs, cfg.tlsArgs[i])
|
||||
tlsArgs = append(tlsArgs, cfg.TlsArgs[i])
|
||||
}
|
||||
|
||||
// Configure certificates for connection proxy ---> server.
|
||||
// This certificate must NOT have CN set.
|
||||
tlsArgs = append(tlsArgs,
|
||||
"--cert", path.Join(fixturesDir, "client-nocn.crt"),
|
||||
"--key", path.Join(fixturesDir, "client-nocn.key.insecure"),
|
||||
"--cacert", path.Join(fixturesDir, "ca.crt"),
|
||||
"--client-crl-file", path.Join(fixturesDir, "revoke.crl"))
|
||||
"--cert", path.Join(FixturesDir, "client-nocn.crt"),
|
||||
"--key", path.Join(FixturesDir, "client-nocn.key.insecure"),
|
||||
"--cacert", path.Join(FixturesDir, "ca.crt"),
|
||||
"--client-crl-file", path.Join(FixturesDir, "revoke.crl"))
|
||||
}
|
||||
return &proxyV3Proc{
|
||||
proxyProc{
|
||||
lg: cfg.lg,
|
||||
execPath: cfg.execPath,
|
||||
execPath: cfg.ExecPath,
|
||||
args: append(args, tlsArgs...),
|
||||
ep: listenAddr,
|
||||
murl: murl,
|
123
tests/framework/e2e/curl.go
Normal file
123
tests/framework/e2e/curl.go
Normal file
@ -0,0 +1,123 @@
|
||||
// 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 e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"strings"
|
||||
)
|
||||
|
||||
type CURLReq struct {
|
||||
Username string
|
||||
Password string
|
||||
|
||||
IsTLS bool
|
||||
Timeout int
|
||||
|
||||
Endpoint string
|
||||
|
||||
Value string
|
||||
Expected string
|
||||
Header string
|
||||
|
||||
MetricsURLScheme string
|
||||
|
||||
Ciphers string
|
||||
HttpVersion string
|
||||
|
||||
OutputFile string
|
||||
}
|
||||
|
||||
// CURLPrefixArgsCluster builds the beginning of a curl command for a given key
|
||||
// addressed to a random URL in the given cluster.
|
||||
func CURLPrefixArgsCluster(clus *EtcdProcessCluster, method string, req CURLReq) []string {
|
||||
member := clus.Procs[rand.Intn(clus.Cfg.ClusterSize)]
|
||||
clientURL := member.Config().Acurl
|
||||
if req.MetricsURLScheme != "" {
|
||||
clientURL = member.EndpointsMetrics()[0]
|
||||
}
|
||||
return CURLPrefixArgs(clientURL, clus.Cfg.ClientTLS, !clus.Cfg.NoCN, method, req)
|
||||
}
|
||||
|
||||
// CURLPrefixArgs builds the beginning of a curl command for a given key
|
||||
// addressed to a random URL in the given cluster.
|
||||
func CURLPrefixArgs(clientURL string, connType ClientConnType, CN bool, method string, req CURLReq) []string {
|
||||
var (
|
||||
cmdArgs = []string{"curl"}
|
||||
)
|
||||
if req.HttpVersion != "" {
|
||||
cmdArgs = append(cmdArgs, "--http"+req.HttpVersion)
|
||||
}
|
||||
if req.MetricsURLScheme != "https" {
|
||||
if req.IsTLS {
|
||||
if connType != ClientTLSAndNonTLS {
|
||||
panic("should not use cURLPrefixArgsUseTLS when serving only TLS or non-TLS")
|
||||
}
|
||||
cmdArgs = append(cmdArgs, "--cacert", CaPath, "--cert", CertPath, "--key", PrivateKeyPath)
|
||||
clientURL = ToTLS(clientURL)
|
||||
} else if connType == ClientTLS {
|
||||
if CN {
|
||||
cmdArgs = append(cmdArgs, "--cacert", CaPath, "--cert", CertPath, "--key", PrivateKeyPath)
|
||||
} else {
|
||||
cmdArgs = append(cmdArgs, "--cacert", CaPath, "--cert", CertPath3, "--key", PrivateKeyPath3)
|
||||
}
|
||||
}
|
||||
}
|
||||
ep := clientURL + req.Endpoint
|
||||
|
||||
if req.Username != "" || req.Password != "" {
|
||||
cmdArgs = append(cmdArgs, "-L", "-u", fmt.Sprintf("%s:%s", req.Username, req.Password), ep)
|
||||
} else {
|
||||
cmdArgs = append(cmdArgs, "-L", ep)
|
||||
}
|
||||
if req.Timeout != 0 {
|
||||
cmdArgs = append(cmdArgs, "-m", fmt.Sprintf("%d", req.Timeout))
|
||||
}
|
||||
|
||||
if req.Header != "" {
|
||||
cmdArgs = append(cmdArgs, "-H", req.Header)
|
||||
}
|
||||
|
||||
if req.Ciphers != "" {
|
||||
cmdArgs = append(cmdArgs, "--ciphers", req.Ciphers)
|
||||
}
|
||||
|
||||
if req.OutputFile != "" {
|
||||
cmdArgs = append(cmdArgs, "--output", req.OutputFile)
|
||||
}
|
||||
|
||||
switch method {
|
||||
case "POST", "PUT":
|
||||
dt := req.Value
|
||||
if !strings.HasPrefix(dt, "{") { // for non-JSON value
|
||||
dt = "value=" + dt
|
||||
}
|
||||
cmdArgs = append(cmdArgs, "-X", method, "-d", dt)
|
||||
}
|
||||
return cmdArgs
|
||||
}
|
||||
|
||||
func CURLPost(clus *EtcdProcessCluster, req CURLReq) error {
|
||||
return SpawnWithExpect(CURLPrefixArgsCluster(clus, "POST", req), req.Expected)
|
||||
}
|
||||
|
||||
func CURLPut(clus *EtcdProcessCluster, req CURLReq) error {
|
||||
return SpawnWithExpect(CURLPrefixArgsCluster(clus, "PUT", req), req.Expected)
|
||||
}
|
||||
|
||||
func CURLGet(clus *EtcdProcessCluster, req CURLReq) error {
|
||||
return SpawnWithExpect(CURLPrefixArgsCluster(clus, "GET", req), req.Expected)
|
||||
}
|
190
tests/framework/e2e/etcd_process.go
Normal file
190
tests/framework/e2e/etcd_process.go
Normal file
@ -0,0 +1,190 @@
|
||||
// Copyright 2017 The etcd Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package e2e
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/url"
|
||||
"os"
|
||||
|
||||
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
var (
|
||||
EtcdServerReadyLines = []string{"ready to serve client requests"}
|
||||
BinPath string
|
||||
CtlBinPath string
|
||||
UtlBinPath string
|
||||
)
|
||||
|
||||
// EtcdProcess is a process that serves etcd requests.
|
||||
type EtcdProcess interface {
|
||||
EndpointsV2() []string
|
||||
EndpointsV3() []string
|
||||
EndpointsGRPC() []string
|
||||
EndpointsHTTP() []string
|
||||
EndpointsMetrics() []string
|
||||
|
||||
Start() error
|
||||
Restart() error
|
||||
Stop() error
|
||||
Close() error
|
||||
WithStopSignal(sig os.Signal) os.Signal
|
||||
Config() *EtcdServerProcessConfig
|
||||
Logs() LogsExpect
|
||||
}
|
||||
|
||||
type LogsExpect interface {
|
||||
Expect(string) (string, error)
|
||||
Lines() []string
|
||||
LineCount() int
|
||||
}
|
||||
|
||||
type EtcdServerProcess struct {
|
||||
cfg *EtcdServerProcessConfig
|
||||
proc *expect.ExpectProcess
|
||||
donec chan struct{} // closed when Interact() terminates
|
||||
}
|
||||
|
||||
type EtcdServerProcessConfig struct {
|
||||
lg *zap.Logger
|
||||
ExecPath string
|
||||
Args []string
|
||||
TlsArgs []string
|
||||
EnvVars map[string]string
|
||||
|
||||
DataDirPath string
|
||||
KeepDataDir bool
|
||||
|
||||
Name string
|
||||
|
||||
Purl url.URL
|
||||
|
||||
Acurl string
|
||||
Murl string
|
||||
ClientHttpUrl string
|
||||
|
||||
InitialToken string
|
||||
InitialCluster string
|
||||
}
|
||||
|
||||
func NewEtcdServerProcess(cfg *EtcdServerProcessConfig) (*EtcdServerProcess, error) {
|
||||
if !fileutil.Exist(cfg.ExecPath) {
|
||||
return nil, fmt.Errorf("could not find etcd binary: %s", cfg.ExecPath)
|
||||
}
|
||||
if !cfg.KeepDataDir {
|
||||
if err := os.RemoveAll(cfg.DataDirPath); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return &EtcdServerProcess{cfg: cfg, donec: make(chan struct{})}, nil
|
||||
}
|
||||
|
||||
func (ep *EtcdServerProcess) EndpointsV2() []string { return ep.EndpointsHTTP() }
|
||||
func (ep *EtcdServerProcess) EndpointsV3() []string { return ep.EndpointsGRPC() }
|
||||
func (ep *EtcdServerProcess) EndpointsGRPC() []string { return []string{ep.cfg.Acurl} }
|
||||
func (ep *EtcdServerProcess) EndpointsHTTP() []string {
|
||||
if ep.cfg.ClientHttpUrl == "" {
|
||||
return []string{ep.cfg.Acurl}
|
||||
}
|
||||
return []string{ep.cfg.ClientHttpUrl}
|
||||
}
|
||||
func (ep *EtcdServerProcess) EndpointsMetrics() []string { return []string{ep.cfg.Murl} }
|
||||
|
||||
func (ep *EtcdServerProcess) Start() error {
|
||||
if ep.proc != nil {
|
||||
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...), ep.cfg.EnvVars)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
ep.proc = proc
|
||||
err = ep.waitReady()
|
||||
if err == nil {
|
||||
ep.cfg.lg.Info("started server.", zap.String("name", ep.cfg.Name))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (ep *EtcdServerProcess) Restart() error {
|
||||
ep.cfg.lg.Info("restaring server...", zap.String("name", ep.cfg.Name))
|
||||
if err := ep.Stop(); err != nil {
|
||||
return err
|
||||
}
|
||||
ep.donec = make(chan struct{})
|
||||
err := ep.Start()
|
||||
if err == nil {
|
||||
ep.cfg.lg.Info("restared server", zap.String("name", ep.cfg.Name))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (ep *EtcdServerProcess) Stop() (err error) {
|
||||
ep.cfg.lg.Info("stoping server...", zap.String("name", ep.cfg.Name))
|
||||
if ep == nil || ep.proc == nil {
|
||||
return nil
|
||||
}
|
||||
err = ep.proc.Stop()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
ep.proc = nil
|
||||
<-ep.donec
|
||||
ep.donec = make(chan struct{})
|
||||
if ep.cfg.Purl.Scheme == "unix" || ep.cfg.Purl.Scheme == "unixs" {
|
||||
err = os.Remove(ep.cfg.Purl.Host + ep.cfg.Purl.Path)
|
||||
if err != nil && !os.IsNotExist(err) {
|
||||
return err
|
||||
}
|
||||
}
|
||||
ep.cfg.lg.Info("stopped server.", zap.String("name", ep.cfg.Name))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ep *EtcdServerProcess) Close() error {
|
||||
ep.cfg.lg.Info("closing server...", zap.String("name", ep.cfg.Name))
|
||||
if err := ep.Stop(); err != nil {
|
||||
return err
|
||||
}
|
||||
if !ep.cfg.KeepDataDir {
|
||||
ep.cfg.lg.Info("removing directory", zap.String("data-dir", ep.cfg.DataDirPath))
|
||||
return os.RemoveAll(ep.cfg.DataDirPath)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ep *EtcdServerProcess) WithStopSignal(sig os.Signal) os.Signal {
|
||||
ret := ep.proc.StopSignal
|
||||
ep.proc.StopSignal = sig
|
||||
return ret
|
||||
}
|
||||
|
||||
func (ep *EtcdServerProcess) waitReady() error {
|
||||
defer close(ep.donec)
|
||||
return WaitReadyExpectProc(ep.proc, EtcdServerReadyLines)
|
||||
}
|
||||
|
||||
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
|
||||
}
|
@ -36,11 +36,11 @@ var (
|
||||
coverDir = integration.MustAbsPath(os.Getenv("COVERDIR"))
|
||||
)
|
||||
|
||||
func spawnCmd(args []string) (*expect.ExpectProcess, error) {
|
||||
return spawnCmdWithLogger(zap.NewNop(), args)
|
||||
func SpawnCmd(args []string) (*expect.ExpectProcess, error) {
|
||||
return SpawnCmdWithLogger(zap.NewNop(), args)
|
||||
}
|
||||
|
||||
func spawnCmdWithLogger(lg *zap.Logger, args []string) (*expect.ExpectProcess, error) {
|
||||
func SpawnCmdWithLogger(lg *zap.Logger, args []string) (*expect.ExpectProcess, error) {
|
||||
cmd := args[0]
|
||||
env := make([]string, 0)
|
||||
switch {
|
||||
@ -51,7 +51,7 @@ func spawnCmdWithLogger(lg *zap.Logger, args []string) (*expect.ExpectProcess, e
|
||||
case strings.HasSuffix(cmd, "/etcdutl"):
|
||||
cmd = cmd + "_test"
|
||||
case strings.HasSuffix(cmd, "/etcdctl3"):
|
||||
cmd = ctlBinPath + "_test"
|
||||
cmd = CtlBinPath + "_test"
|
||||
env = append(env, "ETCDCTL_API=3")
|
||||
}
|
||||
|
@ -28,11 +28,11 @@ import (
|
||||
|
||||
const noOutputLineCount = 0 // regular binaries emit no extra lines
|
||||
|
||||
func spawnCmd(args []string, envVars map[string]string) (*expect.ExpectProcess, error) {
|
||||
return spawnCmdWithLogger(zap.NewNop(), args, envVars)
|
||||
func SpawnCmd(args []string, envVars map[string]string) (*expect.ExpectProcess, error) {
|
||||
return SpawnCmdWithLogger(zap.NewNop(), args, envVars)
|
||||
}
|
||||
|
||||
func spawnCmdWithLogger(lg *zap.Logger, args []string, envVars map[string]string) (*expect.ExpectProcess, error) {
|
||||
func SpawnCmdWithLogger(lg *zap.Logger, args []string, envVars map[string]string) (*expect.ExpectProcess, error) {
|
||||
wd, err := os.Getwd()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -41,7 +41,7 @@ func spawnCmdWithLogger(lg *zap.Logger, args []string, envVars map[string]string
|
||||
if strings.HasSuffix(args[0], "/etcdctl3") {
|
||||
env = append(env, "ETCDCTL_API=3")
|
||||
lg.Info("spawning process with ETCDCTL_API=3", zap.Strings("args", args), zap.String("working-dir", wd), zap.Strings("environment-variables", env))
|
||||
return expect.NewExpectWithEnv(ctlBinPath, args[1:], env)
|
||||
return expect.NewExpectWithEnv(CtlBinPath, args[1:], env)
|
||||
}
|
||||
lg.Info("spawning process", zap.Strings("args", args), zap.String("working-dir", wd), zap.Strings("environment-variables", env))
|
||||
return expect.NewExpectWithEnv(args[0], args[1:], env)
|
72
tests/framework/e2e/flags.go
Normal file
72
tests/framework/e2e/flags.go
Normal file
@ -0,0 +1,72 @@
|
||||
// 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 e2e
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"os"
|
||||
"runtime"
|
||||
|
||||
"go.etcd.io/etcd/tests/v3/integration"
|
||||
)
|
||||
|
||||
var (
|
||||
BinDir string
|
||||
CertDir string
|
||||
|
||||
CertPath string
|
||||
PrivateKeyPath string
|
||||
CaPath string
|
||||
|
||||
CertPath2 string
|
||||
PrivateKeyPath2 string
|
||||
|
||||
CertPath3 string
|
||||
PrivateKeyPath3 string
|
||||
|
||||
CrlPath string
|
||||
RevokedCertPath string
|
||||
RevokedPrivateKeyPath string
|
||||
|
||||
FixturesDir = integration.MustAbsPath("../fixtures")
|
||||
)
|
||||
|
||||
func InitFlags() {
|
||||
os.Setenv("ETCD_UNSUPPORTED_ARCH", runtime.GOARCH)
|
||||
os.Unsetenv("ETCDCTL_API")
|
||||
|
||||
binDirDef := integration.MustAbsPath("../../bin")
|
||||
certDirDef := FixturesDir
|
||||
|
||||
flag.StringVar(&BinDir, "bin-dir", binDirDef, "The directory for store etcd and etcdctl binaries.")
|
||||
flag.StringVar(&CertDir, "cert-dir", certDirDef, "The directory for store certificate files.")
|
||||
flag.Parse()
|
||||
|
||||
BinPath = BinDir + "/etcd"
|
||||
CtlBinPath = BinDir + "/etcdctl"
|
||||
UtlBinPath = BinDir + "/etcdutl"
|
||||
CertPath = CertDir + "/server.crt"
|
||||
PrivateKeyPath = CertDir + "/server.key.insecure"
|
||||
CaPath = CertDir + "/ca.crt"
|
||||
RevokedCertPath = CertDir + "/server-revoked.crt"
|
||||
RevokedPrivateKeyPath = CertDir + "/server-revoked.key.insecure"
|
||||
CrlPath = CertDir + "/revoke.crl"
|
||||
|
||||
CertPath2 = CertDir + "/server2.crt"
|
||||
PrivateKeyPath2 = CertDir + "/server2.key.insecure"
|
||||
|
||||
CertPath3 = CertDir + "/server3.crt"
|
||||
PrivateKeyPath3 = CertDir + "/server3.key.insecure"
|
||||
}
|
@ -24,7 +24,7 @@ import (
|
||||
)
|
||||
|
||||
func BeforeTest(t testing.TB) {
|
||||
skipInShortMode(t)
|
||||
SkipInShortMode(t)
|
||||
testutil.RegisterLeakDetection(t)
|
||||
os.Setenv(verify.ENV_VERIFY, verify.ENV_VERIFY_ALL_VALUE)
|
||||
|
@ -26,7 +26,7 @@ import (
|
||||
"go.etcd.io/etcd/pkg/v3/expect"
|
||||
)
|
||||
|
||||
func waitReadyExpectProc(exproc *expect.ExpectProcess, readyStrs []string) error {
|
||||
func WaitReadyExpectProc(exproc *expect.ExpectProcess, readyStrs []string) error {
|
||||
matchSet := func(l string) bool {
|
||||
for _, s := range readyStrs {
|
||||
if strings.Contains(l, s) {
|
||||
@ -39,21 +39,21 @@ func waitReadyExpectProc(exproc *expect.ExpectProcess, readyStrs []string) error
|
||||
return err
|
||||
}
|
||||
|
||||
func spawnWithExpect(args []string, expected string) error {
|
||||
return spawnWithExpects(args, nil, []string{expected}...)
|
||||
func SpawnWithExpect(args []string, expected string) error {
|
||||
return SpawnWithExpects(args, nil, []string{expected}...)
|
||||
}
|
||||
|
||||
func spawnWithExpectWithEnv(args []string, envVars map[string]string, expected string) error {
|
||||
return spawnWithExpects(args, envVars, []string{expected}...)
|
||||
func SpawnWithExpectWithEnv(args []string, envVars map[string]string, expected string) error {
|
||||
return SpawnWithExpects(args, envVars, []string{expected}...)
|
||||
}
|
||||
|
||||
func spawnWithExpects(args []string, envVars map[string]string, xs ...string) error {
|
||||
_, err := spawnWithExpectLines(args, envVars, xs...)
|
||||
func SpawnWithExpects(args []string, envVars map[string]string, xs ...string) error {
|
||||
_, err := SpawnWithExpectLines(args, envVars, xs...)
|
||||
return err
|
||||
}
|
||||
|
||||
func spawnWithExpectLines(args []string, envVars map[string]string, xs ...string) ([]string, error) {
|
||||
proc, err := spawnCmd(args, envVars)
|
||||
func SpawnWithExpectLines(args []string, envVars map[string]string, xs ...string) ([]string, error) {
|
||||
proc, err := SpawnCmd(args, envVars)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -78,8 +78,8 @@ func spawnWithExpectLines(args []string, envVars map[string]string, xs ...string
|
||||
return lines, perr
|
||||
}
|
||||
|
||||
func runUtilCompletion(args []string, envVars map[string]string) ([]string, error) {
|
||||
proc, err := spawnCmd(args, envVars)
|
||||
func RunUtilCompletion(args []string, envVars map[string]string) ([]string, error) {
|
||||
proc, err := SpawnCmd(args, envVars)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to spawn command %v with error: %w", args, err)
|
||||
}
|
||||
@ -93,11 +93,11 @@ func runUtilCompletion(args []string, envVars map[string]string) ([]string, erro
|
||||
return proc.Lines(), nil
|
||||
}
|
||||
|
||||
func randomLeaseID() int64 {
|
||||
func RandomLeaseID() int64 {
|
||||
return rand.New(rand.NewSource(time.Now().UnixNano())).Int63()
|
||||
}
|
||||
|
||||
func dataMarshal(data interface{}) (d string, e error) {
|
||||
func DataMarshal(data interface{}) (d string, e error) {
|
||||
m, err := json.Marshal(data)
|
||||
if err != nil {
|
||||
return "", err
|
||||
@ -105,7 +105,7 @@ func dataMarshal(data interface{}) (d string, e error) {
|
||||
return string(m), nil
|
||||
}
|
||||
|
||||
func closeWithTimeout(p *expect.ExpectProcess, d time.Duration) error {
|
||||
func CloseWithTimeout(p *expect.ExpectProcess, d time.Duration) error {
|
||||
errc := make(chan error, 1)
|
||||
go func() { errc <- p.Close() }()
|
||||
select {
|
||||
@ -114,15 +114,15 @@ func closeWithTimeout(p *expect.ExpectProcess, d time.Duration) error {
|
||||
case <-time.After(d):
|
||||
p.Stop()
|
||||
// retry close after stopping to collect SIGQUIT data, if any
|
||||
closeWithTimeout(p, time.Second)
|
||||
CloseWithTimeout(p, time.Second)
|
||||
}
|
||||
return fmt.Errorf("took longer than %v to Close process %+v", d, p)
|
||||
}
|
||||
|
||||
func toTLS(s string) string {
|
||||
func ToTLS(s string) string {
|
||||
return strings.Replace(s, "http://", "https://", 1)
|
||||
}
|
||||
|
||||
func skipInShortMode(t testing.TB) {
|
||||
func SkipInShortMode(t testing.TB) {
|
||||
testutil.SkipTestIfShortMode(t, "e2e tests are not running in --short mode")
|
||||
}
|
@ -17,4 +17,4 @@
|
||||
|
||||
package e2e
|
||||
|
||||
func addV2Args(args []string) []string { return args }
|
||||
func AddV2Args(args []string) []string { return args }
|
@ -17,6 +17,6 @@
|
||||
|
||||
package e2e
|
||||
|
||||
func addV2Args(args []string) []string {
|
||||
func AddV2Args(args []string) []string {
|
||||
return append(args, "--experimental-enable-v2v3", "v2/")
|
||||
}
|
Loading…
x
Reference in New Issue
Block a user