mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
test: add mix version e2e test.
Signed-off-by: Siyuan Zhang <sizhang@google.com>
This commit is contained in:
parent
38f3eb39a6
commit
c06ec72230
@ -50,13 +50,11 @@ func TestClusterVersion(t *testing.T) {
|
|||||||
|
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
binary := e2e.BinDir + "/etcd"
|
if !fileutil.Exist(e2e.BinPath) {
|
||||||
if !fileutil.Exist(binary) {
|
t.Skipf("%q does not exist", e2e.BinPath)
|
||||||
t.Skipf("%q does not exist", binary)
|
|
||||||
}
|
}
|
||||||
e2e.BeforeTest(t)
|
e2e.BeforeTest(t)
|
||||||
cfg := e2e.NewConfigNoTLS()
|
cfg := e2e.NewConfigNoTLS()
|
||||||
cfg.ExecPath = binary
|
|
||||||
cfg.SnapshotCount = 3
|
cfg.SnapshotCount = 3
|
||||||
cfg.BasePeerScheme = "unix" // to avoid port conflict
|
cfg.BasePeerScheme = "unix" // to avoid port conflict
|
||||||
cfg.RollingStart = tt.rollingStart
|
cfg.RollingStart = tt.rollingStart
|
||||||
|
194
tests/e2e/etcd_mix_versions_test.go
Normal file
194
tests/e2e/etcd_mix_versions_test.go
Normal file
@ -0,0 +1,194 @@
|
|||||||
|
// Copyright 2024 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"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
||||||
|
"go.etcd.io/etcd/tests/v3/framework/e2e"
|
||||||
|
)
|
||||||
|
|
||||||
|
type clusterTestCase struct {
|
||||||
|
name string
|
||||||
|
config e2e.EtcdProcessClusterConfig
|
||||||
|
}
|
||||||
|
|
||||||
|
func clusterTestCases(size int) []clusterTestCase {
|
||||||
|
tcs := []clusterTestCase{
|
||||||
|
{
|
||||||
|
name: "CurrentVersion",
|
||||||
|
config: e2e.EtcdProcessClusterConfig{ClusterSize: size},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
if !fileutil.Exist(e2e.BinPathLastRelease) {
|
||||||
|
return tcs
|
||||||
|
}
|
||||||
|
|
||||||
|
tcs = append(tcs,
|
||||||
|
clusterTestCase{
|
||||||
|
name: "LastVersion",
|
||||||
|
config: e2e.EtcdProcessClusterConfig{ClusterSize: size, Version: e2e.LastVersion},
|
||||||
|
},
|
||||||
|
)
|
||||||
|
if size > 2 {
|
||||||
|
tcs = append(tcs,
|
||||||
|
clusterTestCase{
|
||||||
|
name: "MinorityLastVersion",
|
||||||
|
config: e2e.EtcdProcessClusterConfig{ClusterSize: size, Version: e2e.MinorityLastVersion},
|
||||||
|
}, clusterTestCase{
|
||||||
|
name: "QuorumLastVersion",
|
||||||
|
config: e2e.EtcdProcessClusterConfig{ClusterSize: size, Version: e2e.QuorumLastVersion},
|
||||||
|
},
|
||||||
|
)
|
||||||
|
}
|
||||||
|
return tcs
|
||||||
|
}
|
||||||
|
|
||||||
|
// TestMixVersionsSnapshotByAddingMember tests the mix version send snapshots by adding member
|
||||||
|
func TestMixVersionsSnapshotByAddingMember(t *testing.T) {
|
||||||
|
for _, tc := range clusterTestCases(1) {
|
||||||
|
t.Run(tc.name+"-adding-new-member-of-current-version", func(t *testing.T) {
|
||||||
|
mixVersionsSnapshotTestByAddingMember(t, tc.config, e2e.CurrentVersion)
|
||||||
|
})
|
||||||
|
// TODO: uncomment after v3.4.32 release.
|
||||||
|
// After v3.4.32, etcd can support adding a new member of 3.4 into
|
||||||
|
// a cluster with 3.5 if the 3.4 member is started with '--next-cluster-version-compatible'.
|
||||||
|
|
||||||
|
// t.Run(tc.name+"-adding-new-member-of-last-version", func(t *testing.T) {
|
||||||
|
// mixVersionsSnapshotTestByAddingMember(t, tc.config, e2e.LastVersion)
|
||||||
|
// })
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func mixVersionsSnapshotTestByAddingMember(t *testing.T, cfg e2e.EtcdProcessClusterConfig, newInstanceVersion e2e.ClusterVersion) {
|
||||||
|
e2e.BeforeTest(t)
|
||||||
|
|
||||||
|
t.Logf("Create an etcd cluster with %d member\n", cfg.ClusterSize)
|
||||||
|
cfg.SnapshotCount = 10
|
||||||
|
cfg.BasePeerScheme = "unix" // to avoid port conflict
|
||||||
|
|
||||||
|
epc, err := e2e.NewEtcdProcessCluster(t, &cfg)
|
||||||
|
|
||||||
|
require.NoError(t, err, "failed to start etcd cluster: %v", err)
|
||||||
|
defer func() {
|
||||||
|
derr := epc.Close()
|
||||||
|
require.NoError(t, derr, "failed to close etcd cluster: %v", derr)
|
||||||
|
}()
|
||||||
|
|
||||||
|
t.Log("Writing 20 keys to the cluster (more than SnapshotCount entries to trigger at least a snapshot)")
|
||||||
|
|
||||||
|
etcdctl := epc.Procs[0].Etcdctl(e2e.ClientNonTLS, false, false)
|
||||||
|
writeKVs(t, etcdctl, 0, 20)
|
||||||
|
|
||||||
|
t.Log("Start a new etcd instance, which will receive a snapshot from the leader.")
|
||||||
|
newCfg := *epc.Cfg
|
||||||
|
newCfg.Version = newInstanceVersion
|
||||||
|
t.Log("Starting a new etcd instance")
|
||||||
|
_, err = epc.StartNewProc(&newCfg, t)
|
||||||
|
require.NoError(t, err, "failed to start the new etcd instance: %v", err)
|
||||||
|
defer epc.Close()
|
||||||
|
|
||||||
|
assertKVHash(t, epc)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestMixVersionsSnapshotByMockingPartition(t *testing.T) {
|
||||||
|
mockPartitionNodeIndex := 2
|
||||||
|
for _, tc := range clusterTestCases(3) {
|
||||||
|
t.Run(tc.name, func(t *testing.T) {
|
||||||
|
mixVersionsSnapshotTestByMockPartition(t, tc.config, mockPartitionNodeIndex)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func mixVersionsSnapshotTestByMockPartition(t *testing.T, cfg e2e.EtcdProcessClusterConfig, mockPartitionNodeIndex int) {
|
||||||
|
e2e.BeforeTest(t)
|
||||||
|
|
||||||
|
if !fileutil.Exist(e2e.BinPathLastRelease) {
|
||||||
|
t.Skipf("%q does not exist", e2e.BinPathLastRelease)
|
||||||
|
}
|
||||||
|
|
||||||
|
t.Logf("Create an etcd cluster with %d member\n", cfg.ClusterSize)
|
||||||
|
cfg.SnapshotCount = 10
|
||||||
|
cfg.BasePeerScheme = "unix" // to avoid port conflict
|
||||||
|
|
||||||
|
epc, err := e2e.NewEtcdProcessCluster(t, &cfg)
|
||||||
|
require.NoError(t, err, "failed to start etcd cluster: %v", err)
|
||||||
|
defer func() {
|
||||||
|
derr := epc.Close()
|
||||||
|
require.NoError(t, derr, "failed to close etcd cluster: %v", derr)
|
||||||
|
}()
|
||||||
|
toPartitionedMember := epc.Procs[mockPartitionNodeIndex]
|
||||||
|
|
||||||
|
t.Log("Stop and restart the partitioned member")
|
||||||
|
err = toPartitionedMember.Stop()
|
||||||
|
require.NoError(t, err)
|
||||||
|
time.Sleep(2 * time.Second)
|
||||||
|
|
||||||
|
t.Log("Writing 20 keys to the cluster (more than SnapshotCount entries to trigger at least a snapshot)")
|
||||||
|
etcdctl := epc.Procs[0].Etcdctl(e2e.ClientNonTLS, false, false)
|
||||||
|
writeKVs(t, etcdctl, 0, 20)
|
||||||
|
|
||||||
|
t.Log("Verify logs to check leader has saved snapshot")
|
||||||
|
leaderEPC := epc.Procs[epc.WaitLeader(t)]
|
||||||
|
e2e.AssertProcessLogs(t, leaderEPC, "saved snapshot")
|
||||||
|
|
||||||
|
t.Log("Restart the partitioned member")
|
||||||
|
err = toPartitionedMember.Restart()
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
assertKVHash(t, epc)
|
||||||
|
}
|
||||||
|
|
||||||
|
func writeKVs(t *testing.T, etcdctl *e2e.Etcdctl, startIdx, endIdx int) {
|
||||||
|
for i := startIdx; i < endIdx; i++ {
|
||||||
|
key := fmt.Sprintf("key-%d", i)
|
||||||
|
value := fmt.Sprintf("value-%d", i)
|
||||||
|
err := etcdctl.Put(key, value)
|
||||||
|
require.NoError(t, err, "failed to put %q, error: %v", key, err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func assertKVHash(t *testing.T, epc *e2e.EtcdProcessCluster) {
|
||||||
|
clusterSize := len(epc.Procs)
|
||||||
|
if clusterSize < 2 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
t.Log("Verify all nodes have exact same revision and hash")
|
||||||
|
assert.Eventually(t, func() bool {
|
||||||
|
hashKvs, err := epc.Etcdctl().HashKV(0)
|
||||||
|
if err != nil {
|
||||||
|
t.Logf("failed to get HashKV: %v", err)
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
if len(hashKvs) != clusterSize {
|
||||||
|
t.Logf("expected %d hashkv responses, but got: %d", clusterSize, len(hashKvs))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
for i := 1; i < clusterSize; i++ {
|
||||||
|
if hashKvs[0].Header.Revision != hashKvs[i].Header.Revision {
|
||||||
|
t.Logf("Got different revisions, [%d, %d]", hashKvs[0].Header.Revision, hashKvs[1].Header.Revision)
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
assert.Equal(t, hashKvs[0].Hash, hashKvs[i].Hash)
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}, 10*time.Second, 500*time.Millisecond)
|
||||||
|
}
|
@ -37,7 +37,7 @@ func TestReleaseUpgrade(t *testing.T) {
|
|||||||
e2e.BeforeTest(t)
|
e2e.BeforeTest(t)
|
||||||
|
|
||||||
copiedCfg := e2e.NewConfigNoTLS()
|
copiedCfg := e2e.NewConfigNoTLS()
|
||||||
copiedCfg.ExecPath = lastReleaseBinary
|
copiedCfg.Version = e2e.LastVersion
|
||||||
copiedCfg.SnapshotCount = 3
|
copiedCfg.SnapshotCount = 3
|
||||||
copiedCfg.BasePeerScheme = "unix" // to avoid port conflict
|
copiedCfg.BasePeerScheme = "unix" // to avoid port conflict
|
||||||
|
|
||||||
@ -78,7 +78,7 @@ func TestReleaseUpgrade(t *testing.T) {
|
|||||||
t.Fatalf("#%d: error closing etcd process (%v)", i, err)
|
t.Fatalf("#%d: error closing etcd process (%v)", i, err)
|
||||||
}
|
}
|
||||||
t.Logf("Stopped node: %v", i)
|
t.Logf("Stopped node: %v", i)
|
||||||
epc.Procs[i].Config().ExecPath = e2e.BinDir + "/etcd"
|
epc.Procs[i].Config().ExecPath = e2e.BinPath
|
||||||
epc.Procs[i].Config().KeepDataDir = true
|
epc.Procs[i].Config().KeepDataDir = true
|
||||||
|
|
||||||
t.Logf("Restarting node in the new version: %v", i)
|
t.Logf("Restarting node in the new version: %v", i)
|
||||||
@ -123,7 +123,7 @@ func TestReleaseUpgradeWithRestart(t *testing.T) {
|
|||||||
e2e.BeforeTest(t)
|
e2e.BeforeTest(t)
|
||||||
|
|
||||||
copiedCfg := e2e.NewConfigNoTLS()
|
copiedCfg := e2e.NewConfigNoTLS()
|
||||||
copiedCfg.ExecPath = lastReleaseBinary
|
copiedCfg.Version = e2e.LastVersion
|
||||||
copiedCfg.SnapshotCount = 10
|
copiedCfg.SnapshotCount = 10
|
||||||
copiedCfg.BasePeerScheme = "unix"
|
copiedCfg.BasePeerScheme = "unix"
|
||||||
|
|
||||||
@ -166,7 +166,7 @@ func TestReleaseUpgradeWithRestart(t *testing.T) {
|
|||||||
wg.Add(len(epc.Procs))
|
wg.Add(len(epc.Procs))
|
||||||
for i := range epc.Procs {
|
for i := range epc.Procs {
|
||||||
go func(i int) {
|
go func(i int) {
|
||||||
epc.Procs[i].Config().ExecPath = e2e.BinDir + "/etcd"
|
epc.Procs[i].Config().ExecPath = e2e.BinPath
|
||||||
epc.Procs[i].Config().KeepDataDir = true
|
epc.Procs[i].Config().KeepDataDir = true
|
||||||
if err := epc.Procs[i].Restart(); err != nil {
|
if err := epc.Procs[i].Restart(); err != nil {
|
||||||
t.Errorf("error restarting etcd process (%v)", err)
|
t.Errorf("error restarting etcd process (%v)", err)
|
||||||
|
@ -135,16 +135,18 @@ type EtcdProcessCluster struct {
|
|||||||
lg *zap.Logger
|
lg *zap.Logger
|
||||||
Cfg *EtcdProcessClusterConfig
|
Cfg *EtcdProcessClusterConfig
|
||||||
Procs []EtcdProcess
|
Procs []EtcdProcess
|
||||||
|
nextSeq int // sequence number of the next etcd process (if it will be required)
|
||||||
}
|
}
|
||||||
|
|
||||||
type EtcdProcessClusterConfig struct {
|
type EtcdProcessClusterConfig struct {
|
||||||
ExecPath string
|
|
||||||
DataDirPath string
|
DataDirPath string
|
||||||
KeepDataDir bool
|
KeepDataDir bool
|
||||||
|
Logger *zap.Logger
|
||||||
GoFailEnabled bool
|
GoFailEnabled bool
|
||||||
GoFailClientTimeout time.Duration
|
GoFailClientTimeout time.Duration
|
||||||
PeerProxy bool
|
PeerProxy bool
|
||||||
EnvVars map[string]string
|
EnvVars map[string]string
|
||||||
|
Version ClusterVersion
|
||||||
|
|
||||||
ClusterSize int
|
ClusterSize int
|
||||||
|
|
||||||
@ -210,6 +212,7 @@ func InitEtcdProcessCluster(t testing.TB, cfg *EtcdProcessClusterConfig) (*EtcdP
|
|||||||
Cfg: cfg,
|
Cfg: cfg,
|
||||||
lg: zaptest.NewLogger(t),
|
lg: zaptest.NewLogger(t),
|
||||||
Procs: make([]EtcdProcess, cfg.ClusterSize),
|
Procs: make([]EtcdProcess, cfg.ClusterSize),
|
||||||
|
nextSeq: cfg.ClusterSize,
|
||||||
}
|
}
|
||||||
|
|
||||||
// launch etcd processes
|
// launch etcd processes
|
||||||
@ -253,22 +256,7 @@ func (cfg *EtcdProcessClusterConfig) PeerScheme() string {
|
|||||||
return setupScheme(cfg.BasePeerScheme, cfg.IsPeerTLS)
|
return setupScheme(cfg.BasePeerScheme, cfg.IsPeerTLS)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*EtcdServerProcessConfig {
|
func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfig(tb testing.TB, i int) *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 curls []string
|
||||||
var curl string
|
var curl string
|
||||||
port := cfg.BasePort + 5*i
|
port := cfg.BasePort + 5*i
|
||||||
@ -305,7 +293,6 @@ func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*
|
|||||||
if cfg.DataDirPath == "" {
|
if cfg.DataDirPath == "" {
|
||||||
dataDirPath = tb.TempDir()
|
dataDirPath = tb.TempDir()
|
||||||
}
|
}
|
||||||
initialCluster[i] = fmt.Sprintf("%s=%s", name, peerAdvertiseUrl.String())
|
|
||||||
|
|
||||||
args := []string{
|
args := []string{
|
||||||
"--name", name,
|
"--name", name,
|
||||||
@ -393,9 +380,31 @@ func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*
|
|||||||
envVars["GOFAIL_HTTP"] = fmt.Sprintf("127.0.0.1:%d", gofailPort)
|
envVars["GOFAIL_HTTP"] = fmt.Sprintf("127.0.0.1:%d", gofailPort)
|
||||||
}
|
}
|
||||||
|
|
||||||
etcdCfgs[i] = &EtcdServerProcessConfig{
|
var execPath string
|
||||||
lg: lg,
|
switch cfg.Version {
|
||||||
ExecPath: cfg.ExecPath,
|
case CurrentVersion:
|
||||||
|
execPath = BinPath
|
||||||
|
case MinorityLastVersion:
|
||||||
|
if i <= cfg.ClusterSize/2 {
|
||||||
|
execPath = BinPath
|
||||||
|
} else {
|
||||||
|
execPath = BinPathLastRelease
|
||||||
|
}
|
||||||
|
case QuorumLastVersion:
|
||||||
|
if i <= cfg.ClusterSize/2 {
|
||||||
|
execPath = BinPathLastRelease
|
||||||
|
} else {
|
||||||
|
execPath = BinPath
|
||||||
|
}
|
||||||
|
case LastVersion:
|
||||||
|
execPath = BinPathLastRelease
|
||||||
|
default:
|
||||||
|
panic(fmt.Sprintf("Unknown cluster version %v", cfg.Version))
|
||||||
|
}
|
||||||
|
|
||||||
|
return &EtcdServerProcessConfig{
|
||||||
|
lg: cfg.Logger,
|
||||||
|
ExecPath: execPath,
|
||||||
Args: args,
|
Args: args,
|
||||||
EnvVars: envVars,
|
EnvVars: envVars,
|
||||||
TlsArgs: cfg.TlsArgs(),
|
TlsArgs: cfg.TlsArgs(),
|
||||||
@ -413,10 +422,24 @@ func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
initialClusterArgs := []string{"--initial-cluster", strings.Join(initialCluster, ",")}
|
func (cfg *EtcdProcessClusterConfig) EtcdServerProcessConfigs(tb testing.TB) []*EtcdServerProcessConfig {
|
||||||
|
cfg.Logger = zaptest.NewLogger(tb)
|
||||||
|
if cfg.BasePort == 0 {
|
||||||
|
cfg.BasePort = EtcdProcessBasePort
|
||||||
|
}
|
||||||
|
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++ {
|
||||||
|
etcdCfgs[i] = cfg.EtcdServerProcessConfig(tb, i)
|
||||||
|
initialCluster[i] = fmt.Sprintf("%s=%s", etcdCfgs[i].Name, etcdCfgs[i].Purl.String())
|
||||||
|
}
|
||||||
|
|
||||||
for i := range etcdCfgs {
|
for i := range etcdCfgs {
|
||||||
etcdCfgs[i].InitialCluster = strings.Join(initialCluster, ",")
|
cfg.SetInitialCluster(etcdCfgs[i], initialCluster, "")
|
||||||
etcdCfgs[i].Args = append(etcdCfgs[i].Args, initialClusterArgs...)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return etcdCfgs
|
return etcdCfgs
|
||||||
@ -557,6 +580,10 @@ func (epc *EtcdProcessCluster) Stop() (err error) {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (epc *EtcdProcessCluster) Etcdctl() *Etcdctl {
|
||||||
|
return NewEtcdctl(epc.EndpointsV3(), epc.Cfg.ClientTLS, epc.Cfg.IsClientAutoTLS, epc.Cfg.EnableV2)
|
||||||
|
}
|
||||||
|
|
||||||
func (epc *EtcdProcessCluster) Close() error {
|
func (epc *EtcdProcessCluster) Close() error {
|
||||||
epc.lg.Info("closing test cluster...")
|
epc.lg.Info("closing test cluster...")
|
||||||
err := epc.Stop()
|
err := epc.Stop()
|
||||||
@ -581,6 +608,75 @@ func (epc *EtcdProcessCluster) WithStopSignal(sig os.Signal) (ret os.Signal) {
|
|||||||
return ret
|
return ret
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// StartNewProc grows cluster size by one with two phases
|
||||||
|
// Phase 1 - Inform cluster of new configuration
|
||||||
|
// Phase 2 - Start new member
|
||||||
|
func (epc *EtcdProcessCluster) StartNewProc(cfg *EtcdProcessClusterConfig, tb testing.TB) (memberID uint64, err error) {
|
||||||
|
memberID, serverCfg, err := epc.AddMember(cfg, tb)
|
||||||
|
if err != nil {
|
||||||
|
return 0, err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Then start process
|
||||||
|
if err = epc.StartNewProcFromConfig(tb, serverCfg); err != nil {
|
||||||
|
return 0, err
|
||||||
|
}
|
||||||
|
|
||||||
|
return memberID, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// AddMember adds a new member to the cluster without starting it.
|
||||||
|
func (epc *EtcdProcessCluster) AddMember(cfg *EtcdProcessClusterConfig, tb testing.TB) (memberID uint64, serverCfg *EtcdServerProcessConfig, err error) {
|
||||||
|
if cfg != nil {
|
||||||
|
serverCfg = cfg.EtcdServerProcessConfig(tb, epc.nextSeq)
|
||||||
|
} else {
|
||||||
|
serverCfg = epc.Cfg.EtcdServerProcessConfig(tb, epc.nextSeq)
|
||||||
|
}
|
||||||
|
|
||||||
|
epc.nextSeq++
|
||||||
|
|
||||||
|
initialCluster := []string{
|
||||||
|
fmt.Sprintf("%s=%s", serverCfg.Name, serverCfg.Purl.String()),
|
||||||
|
}
|
||||||
|
for _, p := range epc.Procs {
|
||||||
|
initialCluster = append(initialCluster, fmt.Sprintf("%s=%s", p.Config().Name, p.Config().Purl.String()))
|
||||||
|
}
|
||||||
|
|
||||||
|
epc.Cfg.SetInitialCluster(serverCfg, initialCluster, "existing")
|
||||||
|
|
||||||
|
// First add new member to cluster
|
||||||
|
tb.Logf("add new member to cluster; member-name %s, member-peer-url %s", serverCfg.Name, serverCfg.Purl.String())
|
||||||
|
memberCtl := NewEtcdctl(epc.Procs[0].EndpointsV3(), cfg.ClientTLS, cfg.IsClientAutoTLS, false)
|
||||||
|
resp, err := memberCtl.MemberAdd(serverCfg.Name, []string{serverCfg.Purl.String()})
|
||||||
|
if err != nil {
|
||||||
|
return 0, nil, fmt.Errorf("failed to add new member: %w", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
return resp.Member.ID, serverCfg, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (cfg *EtcdProcessClusterConfig) SetInitialCluster(serverCfg *EtcdServerProcessConfig, initialCluster []string, initialClusterState string) {
|
||||||
|
serverCfg.InitialCluster = strings.Join(initialCluster, ",")
|
||||||
|
serverCfg.Args = append(serverCfg.Args, "--initial-cluster", serverCfg.InitialCluster)
|
||||||
|
if len(initialClusterState) > 0 {
|
||||||
|
serverCfg.Args = append(serverCfg.Args, "--initial-cluster-state", initialClusterState)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// StartNewProcFromConfig starts a new member process from the given config.
|
||||||
|
func (epc *EtcdProcessCluster) StartNewProcFromConfig(tb testing.TB, serverCfg *EtcdServerProcessConfig) error {
|
||||||
|
tb.Log("start new member")
|
||||||
|
proc, err := NewEtcdProcess(serverCfg)
|
||||||
|
if err != nil {
|
||||||
|
epc.Close()
|
||||||
|
return fmt.Errorf("cannot configure: %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
|
epc.Procs = append(epc.Procs, proc)
|
||||||
|
|
||||||
|
return proc.Start()
|
||||||
|
}
|
||||||
|
|
||||||
// WaitLeader returns index of the member in c.Members() that is leader
|
// WaitLeader returns index of the member in c.Members() that is leader
|
||||||
// or fails the test (if not established in 30s).
|
// or fails the test (if not established in 30s).
|
||||||
func (epc *EtcdProcessCluster) WaitLeader(t testing.TB) int {
|
func (epc *EtcdProcessCluster) WaitLeader(t testing.TB) int {
|
||||||
|
28
tests/framework/e2e/config.go
Normal file
28
tests/framework/e2e/config.go
Normal file
@ -0,0 +1,28 @@
|
|||||||
|
// Copyright 2024 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
|
||||||
|
|
||||||
|
type ClusterVersion string
|
||||||
|
|
||||||
|
const (
|
||||||
|
CurrentVersion ClusterVersion = ""
|
||||||
|
MinorityLastVersion ClusterVersion = "minority-last-version"
|
||||||
|
QuorumLastVersion ClusterVersion = "quorum-last-version"
|
||||||
|
LastVersion ClusterVersion = "last-version"
|
||||||
|
)
|
||||||
|
|
||||||
|
type ClusterContext struct {
|
||||||
|
Version ClusterVersion
|
||||||
|
}
|
@ -24,6 +24,7 @@ import (
|
|||||||
"net/url"
|
"net/url"
|
||||||
"os"
|
"os"
|
||||||
"strings"
|
"strings"
|
||||||
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
"go.etcd.io/etcd/client/pkg/v3/fileutil"
|
||||||
@ -35,6 +36,7 @@ import (
|
|||||||
var (
|
var (
|
||||||
EtcdServerReadyLines = []string{"ready to serve client requests"}
|
EtcdServerReadyLines = []string{"ready to serve client requests"}
|
||||||
BinPath string
|
BinPath string
|
||||||
|
BinPathLastRelease string
|
||||||
CtlBinPath string
|
CtlBinPath string
|
||||||
UtlBinPath string
|
UtlBinPath string
|
||||||
)
|
)
|
||||||
@ -232,6 +234,14 @@ func (ep *EtcdServerProcess) Logs() LogsExpect {
|
|||||||
return ep.proc
|
return ep.proc
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func AssertProcessLogs(t *testing.T, ep EtcdProcess, expectLog string) {
|
||||||
|
t.Helper()
|
||||||
|
_, err := ep.Logs().Expect(expectLog)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func (ep *EtcdServerProcess) PeerProxy() proxy.Server {
|
func (ep *EtcdServerProcess) PeerProxy() proxy.Server {
|
||||||
return ep.proxy
|
return ep.proxy
|
||||||
}
|
}
|
||||||
|
@ -39,6 +39,22 @@ func NewEtcdctl(endpoints []string, connType ClientConnType, isAutoTLS bool, v2
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (ctl *Etcdctl) HashKV(rev int64) ([]*clientv3.HashKVResponse, error) {
|
||||||
|
var epHashKVs []*struct {
|
||||||
|
Endpoint string
|
||||||
|
HashKV *clientv3.HashKVResponse
|
||||||
|
}
|
||||||
|
err := ctl.spawnJsonCmd(&epHashKVs, "endpoint", "hashkv", "--rev", fmt.Sprint(rev))
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
resp := make([]*clientv3.HashKVResponse, len(epHashKVs))
|
||||||
|
for i, e := range epHashKVs {
|
||||||
|
resp[i] = e.HashKV
|
||||||
|
}
|
||||||
|
return resp, err
|
||||||
|
}
|
||||||
|
|
||||||
func (ctl *Etcdctl) Get(key string) (*clientv3.GetResponse, error) {
|
func (ctl *Etcdctl) Get(key string) (*clientv3.GetResponse, error) {
|
||||||
var resp clientv3.GetResponse
|
var resp clientv3.GetResponse
|
||||||
err := ctl.spawnJsonCmd(&resp, "get", key)
|
err := ctl.spawnJsonCmd(&resp, "get", key)
|
||||||
|
@ -55,6 +55,7 @@ func InitFlags() {
|
|||||||
flag.Parse()
|
flag.Parse()
|
||||||
|
|
||||||
BinPath = BinDir + "/etcd"
|
BinPath = BinDir + "/etcd"
|
||||||
|
BinPathLastRelease = BinDir + "/etcd-last-release"
|
||||||
CtlBinPath = BinDir + "/etcdctl"
|
CtlBinPath = BinDir + "/etcdctl"
|
||||||
UtlBinPath = BinDir + "/etcdutl"
|
UtlBinPath = BinDir + "/etcdutl"
|
||||||
CertPath = CertDir + "/server.crt"
|
CertPath = CertDir + "/server.crt"
|
||||||
|
Loading…
x
Reference in New Issue
Block a user