Merge pull request #9565 from gyuho/quorum-disaster

functional: simulate quorum disaster
This commit is contained in:
Gyuho Lee 2018-04-11 21:47:37 -07:00 committed by GitHub
commit 70341b1614
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
37 changed files with 3099 additions and 1558 deletions

View File

@ -32,7 +32,7 @@ See [code changes](https://github.com/coreos/etcd/compare/v3.3.0...v3.4.0) and [
- Make [Lease `Lookup` non-blocking with concurrent `Grant`/`Revoke`](https://github.com/coreos/etcd/pull/9229).
- Make etcd server return `raft.ErrProposalDropped` on internal Raft proposal drop in [v3 applier](https://github.com/coreos/etcd/pull/9549) and [v2 applier](https://github.com/coreos/etcd/pull/9558).
- e.g. a node is removed from cluster, or [`raftpb.MsgProp` arrives at current leader while there is an ongoing leadership transfer](https://github.com/coreos/etcd/issues/8975).
- Improve [functional tester](https://github.com/coreos/etcd/tree/master/functional) coverage: [proxy layer to run network fault tests in CI](https://github.com/coreos/etcd/pull/9081), [TLS is enabled both for server and client](https://github.com/coreos/etcd/pull/9534), [liveness mode](https://github.com/coreos/etcd/issues/9230), [shuffle test sequence](https://github.com/coreos/etcd/issues/9381), [membership reconfiguration failure cases](https://github.com/coreos/etcd/pull/9564), [disastrous quorum loss and snapshot recovery](TODO).
- Improve [functional tester](https://github.com/coreos/etcd/tree/master/functional) coverage: [proxy layer to run network fault tests in CI](https://github.com/coreos/etcd/pull/9081), [TLS is enabled both for server and client](https://github.com/coreos/etcd/pull/9534), [liveness mode](https://github.com/coreos/etcd/issues/9230), [shuffle test sequence](https://github.com/coreos/etcd/issues/9381), [membership reconfiguration failure cases](https://github.com/coreos/etcd/pull/9564), [disastrous quorum loss and snapshot recover from a seed member](https://github.com/coreos/etcd/pull/9565).
### Breaking Changes

View File

@ -2,15 +2,15 @@ agent-configs:
- etcd-exec-path: ./bin/etcd
agent-addr: 127.0.0.1:19027
failpoint-http-addr: http://127.0.0.1:7381
base-dir: /tmp/etcd-agent-data-1
etcd-log-path: /tmp/etcd-agent-data-1/current-etcd.log
base-dir: /tmp/etcd-functional-1
etcd-log-path: /tmp/etcd-functional-1/etcd.log
etcd-client-proxy: false
etcd-peer-proxy: true
etcd-client-endpoint: 127.0.0.1:1379
etcd:
name: s1
data-dir: /tmp/etcd-agent-data-1/etcd.data
wal-dir: /tmp/etcd-agent-data-1/etcd.data/member/wal
data-dir: /tmp/etcd-functional-1/etcd.data
wal-dir: /tmp/etcd-functional-1/etcd.data/member/wal
heartbeat-interval: 100
election-timeout: 1000
listen-client-urls: ["https://127.0.0.1:1379"]
@ -34,18 +34,32 @@ agent-configs:
quota-backend-bytes: 10740000000 # 10 GiB
pre-vote: true
initial-corrupt-check: true
client-cert-data: ""
client-cert-path: ""
client-key-data: ""
client-key-path: ""
client-trusted-ca-data: ""
client-trusted-ca-path: ""
peer-cert-data: ""
peer-cert-path: ""
peer-key-data: ""
peer-key-path: ""
peer-trusted-ca-data: ""
peer-trusted-ca-path: ""
snapshot-path: /tmp/etcd-functional-1.snapshot.db
- etcd-exec-path: ./bin/etcd
agent-addr: 127.0.0.1:29027
failpoint-http-addr: http://127.0.0.1:7382
base-dir: /tmp/etcd-agent-data-2
etcd-log-path: /tmp/etcd-agent-data-2/current-etcd.log
base-dir: /tmp/etcd-functional-2
etcd-log-path: /tmp/etcd-functional-2/etcd.log
etcd-client-proxy: false
etcd-peer-proxy: true
etcd-client-endpoint: 127.0.0.1:2379
etcd:
name: s2
data-dir: /tmp/etcd-agent-data-2/etcd.data
wal-dir: /tmp/etcd-agent-data-2/etcd.data/member/wal
data-dir: /tmp/etcd-functional-2/etcd.data
wal-dir: /tmp/etcd-functional-2/etcd.data/member/wal
heartbeat-interval: 100
election-timeout: 1000
listen-client-urls: ["https://127.0.0.1:2379"]
@ -69,18 +83,32 @@ agent-configs:
quota-backend-bytes: 10740000000 # 10 GiB
pre-vote: true
initial-corrupt-check: true
client-cert-data: ""
client-cert-path: ""
client-key-data: ""
client-key-path: ""
client-trusted-ca-data: ""
client-trusted-ca-path: ""
peer-cert-data: ""
peer-cert-path: ""
peer-key-data: ""
peer-key-path: ""
peer-trusted-ca-data: ""
peer-trusted-ca-path: ""
snapshot-path: /tmp/etcd-functional-2.snapshot.db
- etcd-exec-path: ./bin/etcd
agent-addr: 127.0.0.1:39027
failpoint-http-addr: http://127.0.0.1:7383
base-dir: /tmp/etcd-agent-data-3
etcd-log-path: /tmp/etcd-agent-data-3/current-etcd.log
base-dir: /tmp/etcd-functional-3
etcd-log-path: /tmp/etcd-functional-3/etcd.log
etcd-client-proxy: false
etcd-peer-proxy: true
etcd-client-endpoint: 127.0.0.1:3379
etcd:
name: s3
data-dir: /tmp/etcd-agent-data-3/etcd.data
wal-dir: /tmp/etcd-agent-data-3/etcd.data/member/wal
data-dir: /tmp/etcd-functional-3/etcd.data
wal-dir: /tmp/etcd-functional-3/etcd.data/member/wal
heartbeat-interval: 100
election-timeout: 1000
listen-client-urls: ["https://127.0.0.1:3379"]
@ -104,6 +132,19 @@ agent-configs:
quota-backend-bytes: 10740000000 # 10 GiB
pre-vote: true
initial-corrupt-check: true
client-cert-data: ""
client-cert-path: ""
client-key-data: ""
client-key-path: ""
client-trusted-ca-data: ""
client-trusted-ca-path: ""
peer-cert-data: ""
peer-cert-path: ""
peer-key-data: ""
peer-key-path: ""
peer-trusted-ca-data: ""
peer-trusted-ca-path: ""
snapshot-path: /tmp/etcd-functional-3.snapshot.db
tester-config:
data-dir: /tmp/etcd-tester-data
@ -116,15 +157,14 @@ tester-config:
round-limit: 1
exit-on-failure: true
consistency-check: true
enable-pprof: true
failure-delay-ms: 7000
failure-shuffle: true
case-delay-ms: 7000
case-shuffle: true
# For full descriptions,
# https://godoc.org/github.com/coreos/etcd/functional/rpcpb#FailureCase
failure-cases:
# https://godoc.org/github.com/coreos/etcd/functional/rpcpb#Case
cases:
- SIGTERM_ONE_FOLLOWER
- SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT
- SIGTERM_LEADER
@ -153,9 +193,9 @@ tester-config:
- RANDOM_DELAY_PEER_PORT_TX_RX_ALL
- NO_FAIL_WITH_STRESS
- NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS
# - SIGQUIT_AND_REMOVE_LEADER
# - SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT
# - SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH
failpoint-commands:
- panic("etcd-tester")
@ -164,7 +204,7 @@ tester-config:
runner-exec-path: ./bin/etcd-runner
external-exec-path: ""
stress-types:
stressers:
- KV
- LEASE
# - ELECTION_RUNNER
@ -172,6 +212,10 @@ tester-config:
# - LOCK_RACER_RUNNER
# - LEASE_RUNNER
checkers:
- KV_HASH
- LEASE_EXPIRE
stress-key-size: 100
stress-key-size-large: 32769
stress-key-suffix-range: 250000

View File

@ -2,7 +2,7 @@
[`functional`](https://godoc.org/github.com/coreos/etcd/functional) verifies the correct behavior of etcd under various system and network malfunctions. It sets up an etcd cluster under high pressure loads and continuously injects failures into the cluster. Then it expects the etcd cluster to recover within a few seconds. This has been extremely helpful to find critical bugs.
See [`rpcpb.FailureCase`](https://godoc.org/github.com/coreos/etcd/functional/rpcpb#FailureCase) for all failure cases.
See [`rpcpb.Case`](https://godoc.org/github.com/coreos/etcd/functional/rpcpb#Case) for all failure cases.
See [functional.yaml](https://github.com/coreos/etcd/blob/master/functional.yaml) for an example configuration.

View File

@ -57,6 +57,13 @@ func (srv *Server) handleTesterRequest(req *rpcpb.Request) (resp *rpcpb.Response
case rpcpb.Operation_SIGQUIT_ETCD_AND_REMOVE_DATA:
return srv.handle_SIGQUIT_ETCD_AND_REMOVE_DATA()
case rpcpb.Operation_SAVE_SNAPSHOT:
return srv.handle_SAVE_SNAPSHOT()
case rpcpb.Operation_RESTORE_RESTART_FROM_SNAPSHOT:
return srv.handle_RESTORE_RESTART_FROM_SNAPSHOT()
case rpcpb.Operation_RESTART_FROM_SNAPSHOT:
return srv.handle_RESTART_FROM_SNAPSHOT()
case rpcpb.Operation_SIGQUIT_ETCD_AND_ARCHIVE_DATA:
return srv.handle_SIGQUIT_ETCD_AND_ARCHIVE_DATA()
case rpcpb.Operation_SIGQUIT_ETCD_AND_REMOVE_DATA_AND_STOP_AGENT:
@ -96,7 +103,7 @@ func (srv *Server) handle_INITIAL_START_ETCD(req *rpcpb.Request) (*rpcpb.Respons
return nil, err
}
srv.creatEtcdCmd()
srv.creatEtcdCmd(false)
if err = srv.saveTLSAssets(); err != nil {
return nil, err
@ -225,8 +232,11 @@ func (srv *Server) createEtcdLogFile() error {
return nil
}
func (srv *Server) creatEtcdCmd() {
func (srv *Server) creatEtcdCmd(fromSnapshot bool) {
etcdPath, etcdFlags := srv.Member.EtcdExecPath, srv.Member.Etcd.Flags()
if fromSnapshot {
etcdFlags = srv.Member.EtcdOnSnapshotRestore.Flags()
}
u, _ := url.Parse(srv.Member.FailpointHTTPAddr)
srv.lg.Info("creating etcd command",
zap.String("etcd-exec-path", etcdPath),
@ -416,7 +426,7 @@ func (srv *Server) handle_RESTART_ETCD() (*rpcpb.Response, error) {
}
}
srv.creatEtcdCmd()
srv.creatEtcdCmd(false)
if err = srv.saveTLSAssets(); err != nil {
return nil, err
@ -502,6 +512,60 @@ func (srv *Server) handle_SIGQUIT_ETCD_AND_REMOVE_DATA() (*rpcpb.Response, error
}, nil
}
func (srv *Server) handle_SAVE_SNAPSHOT() (*rpcpb.Response, error) {
err := srv.Member.SaveSnapshot(srv.lg)
if err != nil {
return nil, err
}
return &rpcpb.Response{
Success: true,
Status: "saved snapshot",
SnapshotInfo: srv.Member.SnapshotInfo,
}, nil
}
func (srv *Server) handle_RESTORE_RESTART_FROM_SNAPSHOT() (resp *rpcpb.Response, err error) {
err = srv.Member.RestoreSnapshot(srv.lg)
if err != nil {
return nil, err
}
resp, err = srv.handle_RESTART_FROM_SNAPSHOT()
if resp != nil && err == nil {
resp.Status = "restored snapshot and " + resp.Status
}
return resp, err
}
func (srv *Server) handle_RESTART_FROM_SNAPSHOT() (resp *rpcpb.Response, err error) {
srv.creatEtcdCmd(true)
if err = srv.saveTLSAssets(); err != nil {
return nil, err
}
if err = srv.startEtcdCmd(); err != nil {
return nil, err
}
srv.lg.Info("restarted etcd", zap.String("command-path", srv.etcdCmd.Path))
if err = srv.loadAutoTLSAssets(); err != nil {
return nil, err
}
// wait some time for etcd listener start
// before setting up proxy
// TODO: local tests should handle port conflicts
// with clients on restart
time.Sleep(time.Second)
if err = srv.startProxy(); err != nil {
return nil, err
}
return &rpcpb.Response{
Success: true,
Status: "restarted etcd from snapshot",
SnapshotInfo: srv.Member.SnapshotInfo,
}, nil
}
func (srv *Server) handle_SIGQUIT_ETCD_AND_ARCHIVE_DATA() (*rpcpb.Response, error) {
srv.stopProxy()

View File

@ -18,16 +18,26 @@ import (
"context"
"fmt"
"net/url"
"os"
"time"
"github.com/coreos/etcd/clientv3"
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
"github.com/coreos/etcd/pkg/transport"
"github.com/coreos/etcd/pkg/types"
"github.com/coreos/etcd/snapshot"
"github.com/dustin/go-humanize"
"go.uber.org/zap"
grpc "google.golang.org/grpc"
"google.golang.org/grpc/credentials"
)
// ElectionTimeout returns an election timeout duration.
func (m *Member) ElectionTimeout() time.Duration {
return time.Duration(m.Etcd.ElectionTimeoutMs) * time.Millisecond
}
// DialEtcdGRPCServer creates a raw gRPC connection to an etcd member.
func (m *Member) DialEtcdGRPCServer(opts ...grpc.DialOption) (*grpc.ClientConn, error) {
dialOpts := []grpc.DialOption{
@ -85,7 +95,7 @@ func (m *Member) CreateEtcdClient(opts ...grpc.DialOption) (*clientv3.Client, er
cfg := clientv3.Config{
Endpoints: []string{m.EtcdClientEndpoint},
DialTimeout: 5 * time.Second,
DialTimeout: 10 * time.Second,
DialOptions: opts,
}
if secure {
@ -227,3 +237,126 @@ func (m *Member) WriteHealthKey() error {
}
return nil
}
// SaveSnapshot downloads a snapshot file from this member, locally.
// It's meant to requested remotely, so that local member can store
// snapshot file on local disk.
func (m *Member) SaveSnapshot(lg *zap.Logger) (err error) {
// remove existing snapshot first
if err = os.RemoveAll(m.SnapshotPath); err != nil {
return err
}
var cli *clientv3.Client
cli, err = m.CreateEtcdClient()
if err != nil {
return fmt.Errorf("%v (%q)", err, m.EtcdClientEndpoint)
}
defer cli.Close()
lg.Info(
"snapshot save START",
zap.String("member-name", m.Etcd.Name),
zap.Strings("member-client-urls", m.Etcd.AdvertiseClientURLs),
zap.String("snapshot-path", m.SnapshotPath),
)
now := time.Now()
mgr := snapshot.NewV3(cli, lg)
if err = mgr.Save(context.Background(), m.SnapshotPath); err != nil {
return err
}
took := time.Since(now)
var fi os.FileInfo
fi, err = os.Stat(m.SnapshotPath)
if err != nil {
return err
}
var st snapshot.Status
st, err = mgr.Status(m.SnapshotPath)
if err != nil {
return err
}
m.SnapshotInfo = &SnapshotInfo{
MemberName: m.Etcd.Name,
MemberClientURLs: m.Etcd.AdvertiseClientURLs,
SnapshotPath: m.SnapshotPath,
SnapshotFileSize: humanize.Bytes(uint64(fi.Size())),
SnapshotTotalSize: humanize.Bytes(uint64(st.TotalSize)),
SnapshotTotalKey: int64(st.TotalKey),
SnapshotHash: int64(st.Hash),
SnapshotRevision: st.Revision,
Took: fmt.Sprintf("%v", took),
}
lg.Info(
"snapshot save END",
zap.String("member-name", m.SnapshotInfo.MemberName),
zap.Strings("member-client-urls", m.SnapshotInfo.MemberClientURLs),
zap.String("snapshot-path", m.SnapshotPath),
zap.String("snapshot-file-size", m.SnapshotInfo.SnapshotFileSize),
zap.String("snapshot-total-size", m.SnapshotInfo.SnapshotTotalSize),
zap.Int64("snapshot-total-key", m.SnapshotInfo.SnapshotTotalKey),
zap.Int64("snapshot-hash", m.SnapshotInfo.SnapshotHash),
zap.Int64("snapshot-revision", m.SnapshotInfo.SnapshotRevision),
zap.String("took", m.SnapshotInfo.Took),
)
return nil
}
// RestoreSnapshot restores a cluster from a given snapshot file on disk.
// It's meant to requested remotely, so that local member can load the
// snapshot file from local disk.
func (m *Member) RestoreSnapshot(lg *zap.Logger) (err error) {
if err = os.RemoveAll(m.EtcdOnSnapshotRestore.DataDir); err != nil {
return err
}
if err = os.RemoveAll(m.EtcdOnSnapshotRestore.WALDir); err != nil {
return err
}
var initialCluster types.URLsMap
initialCluster, err = types.NewURLsMap(m.EtcdOnSnapshotRestore.InitialCluster)
if err != nil {
return err
}
var peerURLs types.URLs
peerURLs, err = types.NewURLs(m.EtcdOnSnapshotRestore.AdvertisePeerURLs)
if err != nil {
return err
}
lg.Info(
"snapshot restore START",
zap.String("member-name", m.Etcd.Name),
zap.Strings("member-client-urls", m.Etcd.AdvertiseClientURLs),
zap.String("snapshot-path", m.SnapshotPath),
)
now := time.Now()
mgr := snapshot.NewV3(nil, lg)
err = mgr.Restore(m.SnapshotInfo.SnapshotPath, snapshot.RestoreConfig{
Name: m.EtcdOnSnapshotRestore.Name,
OutputDataDir: m.EtcdOnSnapshotRestore.DataDir,
OutputWALDir: m.EtcdOnSnapshotRestore.WALDir,
InitialCluster: initialCluster,
InitialClusterToken: m.EtcdOnSnapshotRestore.InitialClusterToken,
PeerURLs: peerURLs,
SkipHashCheck: false,
// TODO: SkipHashCheck == true, for recover from existing db file
})
took := time.Since(now)
lg.Info(
"snapshot restore END",
zap.String("member-name", m.SnapshotInfo.MemberName),
zap.Strings("member-client-urls", m.SnapshotInfo.MemberClientURLs),
zap.String("snapshot-path", m.SnapshotPath),
zap.String("snapshot-file-size", m.SnapshotInfo.SnapshotFileSize),
zap.String("snapshot-total-size", m.SnapshotInfo.SnapshotTotalSize),
zap.Int64("snapshot-total-key", m.SnapshotInfo.SnapshotTotalKey),
zap.Int64("snapshot-hash", m.SnapshotInfo.SnapshotHash),
zap.Int64("snapshot-revision", m.SnapshotInfo.SnapshotRevision),
zap.String("took", took.String()),
zap.Error(err),
)
return err
}

File diff suppressed because it is too large Load Diff

View File

@ -16,11 +16,28 @@ message Request {
Tester Tester = 3;
}
// SnapshotInfo contains SAVE_SNAPSHOT request results.
message SnapshotInfo {
string MemberName = 1;
repeated string MemberClientURLs = 2;
string SnapshotPath = 3;
string SnapshotFileSize = 4;
string SnapshotTotalSize = 5;
int64 SnapshotTotalKey = 6;
int64 SnapshotHash = 7;
int64 SnapshotRevision = 8;
string Took = 9;
}
message Response {
bool Success = 1;
string Status = 2;
// Member contains the same Member object from tester request.
Member Member = 3;
// SnapshotInfo contains SAVE_SNAPSHOT request results.
SnapshotInfo SnapshotInfo = 4;
}
service Transport {
@ -30,7 +47,6 @@ service Transport {
message Member {
// EtcdExecPath is the executable etcd binary path in agent server.
string EtcdExecPath = 1 [(gogoproto.moretags) = "yaml:\"etcd-exec-path\""];
// TODO: support embedded etcd
// AgentAddr is the agent HTTP server address.
@ -54,6 +70,9 @@ message Member {
string EtcdClientEndpoint = 301 [(gogoproto.moretags) = "yaml:\"etcd-client-endpoint\""];
// Etcd defines etcd binary configuration flags.
Etcd Etcd = 302 [(gogoproto.moretags) = "yaml:\"etcd\""];
// EtcdOnSnapshotRestore defines one-time use configuration during etcd
// snapshot recovery process.
Etcd EtcdOnSnapshotRestore = 303;
// ClientCertData contains cert file contents from this member's etcd server.
string ClientCertData = 401 [(gogoproto.moretags) = "yaml:\"client-cert-data\""];
@ -74,6 +93,11 @@ message Member {
// PeerTrustedCAData contains trusted CA file contents from this member's etcd server.
string PeerTrustedCAData = 505 [(gogoproto.moretags) = "yaml:\"peer-trusted-ca-data\""];
string PeerTrustedCAPath = 506 [(gogoproto.moretags) = "yaml:\"peer-trusted-ca-path\""];
// SnapshotPath is the snapshot file path to store or restore from.
string SnapshotPath = 601 [(gogoproto.moretags) = "yaml:\"snapshot-path\""];
// SnapshotInfo contains last SAVE_SNAPSHOT request results.
SnapshotInfo SnapshotInfo = 602;
}
message Tester {
@ -93,22 +117,21 @@ message Tester {
// RoundLimit is the limit of rounds to run failure set (-1 to run without limits).
int32 RoundLimit = 21 [(gogoproto.moretags) = "yaml:\"round-limit\""];
// ExitOnFailure is true, then exit tester on first failure.
bool ExitOnFailure = 22 [(gogoproto.moretags) = "yaml:\"exit-on-failure\""];
// ConsistencyCheck is true to check consistency (revision, hash).
bool ConsistencyCheck = 23 [(gogoproto.moretags) = "yaml:\"consistency-check\""];
// ExitOnCaseFail is true, then exit tester on first failure.
bool ExitOnCaseFail = 22 [(gogoproto.moretags) = "yaml:\"exit-on-failure\""];
// EnablePprof is true to enable profiler.
bool EnablePprof = 24 [(gogoproto.moretags) = "yaml:\"enable-pprof\""];
bool EnablePprof = 23 [(gogoproto.moretags) = "yaml:\"enable-pprof\""];
// FailureDelayMs is the delay duration after failure is injected.
// CaseDelayMs is the delay duration after failure is injected.
// Useful when triggering snapshot or no-op failure cases.
uint32 FailureDelayMs = 31 [(gogoproto.moretags) = "yaml:\"failure-delay-ms\""];
// FailureShuffle is true to randomize failure injecting order.
bool FailureShuffle = 32 [(gogoproto.moretags) = "yaml:\"failure-shuffle\""];
// FailureCases is the selected test cases to schedule.
uint32 CaseDelayMs = 31 [(gogoproto.moretags) = "yaml:\"case-delay-ms\""];
// CaseShuffle is true to randomize failure injecting order.
bool CaseShuffle = 32 [(gogoproto.moretags) = "yaml:\"case-shuffle\""];
// Cases is the selected test cases to schedule.
// If empty, run all failure cases.
repeated string FailureCases = 33 [(gogoproto.moretags) = "yaml:\"failure-cases\""];
// Failpoinommands is the list of "gofail" commands (e.g. panic("etcd-tester"),1*sleep(1000)
repeated string Cases = 33 [(gogoproto.moretags) = "yaml:\"cases\""];
// FailpointCommands is the list of "gofail" commands
// (e.g. panic("etcd-tester"),1*sleep(1000).
repeated string FailpointCommands = 34 [(gogoproto.moretags) = "yaml:\"failpoint-commands\""];
// RunnerExecPath is a path of etcd-runner binary.
@ -116,27 +139,32 @@ message Tester {
// ExternalExecPath is a path of script for enabling/disabling an external fault injector.
string ExternalExecPath = 42 [(gogoproto.moretags) = "yaml:\"external-exec-path\""];
// StressTypes is the list of stresser names:
// keys, lease, nop, election-runner, watch-runner, lock-racer-runner, lease-runner.
repeated string StressTypes = 101 [(gogoproto.moretags) = "yaml:\"stress-types\""];
// Stressers is the list of stresser types:
// KV, LEASE, ELECTION_RUNNER, WATCH_RUNNER, LOCK_RACER_RUNNER, LEASE_RUNNER.
repeated string Stressers = 101 [(gogoproto.moretags) = "yaml:\"stressers\""];
// Checkers is the list of consistency checker types:
// KV_HASH, LEASE_EXPIRE, NO_CHECK, RUNNER.
// Leave empty to skip consistency checks.
repeated string Checkers = 102 [(gogoproto.moretags) = "yaml:\"checkers\""];
// StressKeySize is the size of each small key written into etcd.
int32 StressKeySize = 102 [(gogoproto.moretags) = "yaml:\"stress-key-size\""];
int32 StressKeySize = 201 [(gogoproto.moretags) = "yaml:\"stress-key-size\""];
// StressKeySizeLarge is the size of each large key written into etcd.
int32 StressKeySizeLarge = 103 [(gogoproto.moretags) = "yaml:\"stress-key-size-large\""];
int32 StressKeySizeLarge = 202 [(gogoproto.moretags) = "yaml:\"stress-key-size-large\""];
// StressKeySuffixRange is the count of key range written into etcd.
// Stress keys are created with "fmt.Sprintf("foo%016x", rand.Intn(keySuffixRange)".
int32 StressKeySuffixRange = 104 [(gogoproto.moretags) = "yaml:\"stress-key-suffix-range\""];
int32 StressKeySuffixRange = 203 [(gogoproto.moretags) = "yaml:\"stress-key-suffix-range\""];
// StressKeySuffixRangeTxn is the count of key range written into etcd txn (max 100).
// Stress keys are created with "fmt.Sprintf("/k%03d", i)".
int32 StressKeySuffixRangeTxn = 105 [(gogoproto.moretags) = "yaml:\"stress-key-suffix-range-txn\""];
int32 StressKeySuffixRangeTxn = 204 [(gogoproto.moretags) = "yaml:\"stress-key-suffix-range-txn\""];
// StressKeyTxnOps is the number of operations per a transaction (max 64).
int32 StressKeyTxnOps = 106 [(gogoproto.moretags) = "yaml:\"stress-key-txn-ops\""];
int32 StressKeyTxnOps = 205 [(gogoproto.moretags) = "yaml:\"stress-key-txn-ops\""];
// StressClients is the number of concurrent stressing clients
// with "one" shared TCP connection.
int32 StressClients = 201 [(gogoproto.moretags) = "yaml:\"stress-clients\""];
int32 StressClients = 301 [(gogoproto.moretags) = "yaml:\"stress-clients\""];
// StressQPS is the maximum number of stresser requests per second.
int32 StressQPS = 202 [(gogoproto.moretags) = "yaml:\"stress-qps\""];
int32 StressQPS = 302 [(gogoproto.moretags) = "yaml:\"stress-qps\""];
}
message Etcd {
@ -194,12 +222,24 @@ enum Operation {
// directories to simulate destroying the whole machine.
SIGQUIT_ETCD_AND_REMOVE_DATA = 21;
// SAVE_SNAPSHOT is sent to trigger local member to download its snapshot
// onto its local disk with the specified path from tester.
SAVE_SNAPSHOT = 30;
// RESTORE_RESTART_FROM_SNAPSHOT is sent to trigger local member to
// restore a cluster from existing snapshot from disk, and restart
// an etcd instance from recovered data.
RESTORE_RESTART_FROM_SNAPSHOT = 31;
// RESTART_FROM_SNAPSHOT is sent to trigger local member to restart
// and join an existing cluster that has been recovered from a snapshot.
// Local member joins this cluster with fresh data.
RESTART_FROM_SNAPSHOT = 32;
// SIGQUIT_ETCD_AND_ARCHIVE_DATA is sent when consistency check failed,
// thus need to archive etcd data directories.
SIGQUIT_ETCD_AND_ARCHIVE_DATA = 30;
SIGQUIT_ETCD_AND_ARCHIVE_DATA = 40;
// SIGQUIT_ETCD_AND_REMOVE_DATA_AND_STOP_AGENT destroys etcd process,
// etcd data, and agent server.
SIGQUIT_ETCD_AND_REMOVE_DATA_AND_STOP_AGENT = 31;
SIGQUIT_ETCD_AND_REMOVE_DATA_AND_STOP_AGENT = 41;
// BLACKHOLE_PEER_PORT_TX_RX drops all outgoing/incoming packets from/to
// the peer port on target member's peer port.
@ -214,12 +254,12 @@ enum Operation {
UNDELAY_PEER_PORT_TX_RX = 201;
}
// FailureCase defines various system faults in distributed systems,
// Case defines various system faults or test case in distributed systems,
// in order to verify correct behavior of etcd servers and clients.
enum FailureCase {
enum Case {
// SIGTERM_ONE_FOLLOWER stops a randomly chosen follower (non-leader)
// but does not delete its data directories on disk for next restart.
// It waits "failure-delay-ms" before recovering this failure.
// It waits "delay-ms" before recovering this failure.
// The expected behavior is that the follower comes back online
// and rejoins the cluster, and then each member continues to process
// client requests ('Put' request that requires Raft consensus).
@ -237,9 +277,8 @@ enum FailureCase {
SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT = 1;
// SIGTERM_LEADER stops the active leader node but does not delete its
// data directories on disk for next restart. Then it waits
// "failure-delay-ms" before recovering this failure, in order to
// trigger election timeouts.
// data directories on disk for next restart. Then it waits "delay-ms"
// before recovering this failure, in order to trigger election timeouts.
// The expected behavior is that a new leader gets elected, and the
// old leader comes back online and rejoins the cluster as a follower.
// As always, after recovery, each member must be able to process
@ -259,16 +298,15 @@ enum FailureCase {
// SIGTERM_QUORUM stops majority number of nodes to make the whole cluster
// inoperable but does not delete data directories on stopped nodes
// for next restart. And it waits "failure-delay-ms" before recovering
// this failure.
// for next restart. And it waits "delay-ms" before recovering failure.
// The expected behavior is that nodes come back online, thus cluster
// comes back operative as well. As always, after recovery, each member
// must be able to process client requests.
SIGTERM_QUORUM = 4;
// SIGTERM_ALL stops the whole cluster but does not delete data directories
// on disk for next restart. And it waits "failure-delay-ms" before
// recovering this failure.
// on disk for next restart. And it waits "delay-ms" before recovering
// this failure.
// The expected behavior is that nodes come back online, thus cluster
// comes back operative as well. As always, after recovery, each member
// must be able to process client requests.
@ -278,7 +316,7 @@ enum FailureCase {
// (non-leader), deletes its data directories on disk, and removes
// this member from cluster (membership reconfiguration). On recovery,
// tester adds a new member, and this member joins the existing cluster
// with fresh data. It waits "failure-delay-ms" before recovering this
// with fresh data. It waits "delay-ms" before recovering this
// failure. This simulates destroying one follower machine, where operator
// needs to add a new member from a fresh machine.
// The expected behavior is that a new member joins the existing cluster,
@ -301,7 +339,7 @@ enum FailureCase {
// SIGQUIT_AND_REMOVE_LEADER stops the active leader node, deletes its
// data directories on disk, and removes this member from cluster.
// On recovery, tester adds a new member, and this member joins the
// existing cluster with fresh data. It waits "failure-delay-ms" before
// existing cluster with fresh data. It waits "delay-ms" before
// recovering this failure. This simulates destroying a leader machine,
// where operator needs to add a new member from a fresh machine.
// The expected behavior is that a new member joins the existing cluster,
@ -322,9 +360,36 @@ enum FailureCase {
// member must be able to process client requests.
SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT = 13;
// SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH first
// stops majority number of nodes, deletes data directories on those quorum
// nodes, to make the whole cluster inoperable. Now that quorum and their
// data are totally destroyed, cluster cannot even remove unavailable nodes
// (e.g. 2 out of 3 are lost, so no leader can be elected).
// Let's assume 3-node cluster of node A, B, and C. One day, node A and B
// are destroyed and all their data are gone. The only viable solution is
// to recover from C's latest snapshot.
//
// To simulate:
// 1. Assume node C is the current leader with most up-to-date data.
// 2. Download snapshot from node C, before destroying node A and B.
// 3. Destroy node A and B, and make the whole cluster inoperable.
// 4. Now node C cannot operate either.
// 5. SIGTERM node C and remove its data directories.
// 6. Restore a new seed member from node C's latest snapshot file.
// 7. Add another member to establish 2-node cluster.
// 8. Add another member to establish 3-node cluster.
// 9. Add more if any.
//
// The expected behavior is that etcd successfully recovers from such
// disastrous situation as only 1-node survives out of 3-node cluster,
// new members joins the existing cluster, and previous data from snapshot
// are still preserved after recovery process. As always, after recovery,
// each member must be able to process client requests.
SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH = 14;
// BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER drops all outgoing/incoming
// packets from/to the peer port on a randomly chosen follower
// (non-leader), and waits for "failure-delay-ms" until recovery.
// (non-leader), and waits for "delay-ms" until recovery.
// The expected behavior is that once dropping operation is undone,
// each member must be able to process client requests.
BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER = 100;
@ -342,7 +407,7 @@ enum FailureCase {
// BLACKHOLE_PEER_PORT_TX_RX_LEADER drops all outgoing/incoming packets
// from/to the peer port on the active leader (isolated), and waits for
// "failure-delay-ms" until recovery, in order to trigger election timeout.
// "delay-ms" until recovery, in order to trigger election timeout.
// The expected behavior is that after election timeout, a new leader gets
// elected, and once dropping operation is undone, the old leader comes
// back and rejoins the cluster as a follower. As always, after recovery,
@ -362,7 +427,7 @@ enum FailureCase {
// BLACKHOLE_PEER_PORT_TX_RX_QUORUM drops all outgoing/incoming packets
// from/to the peer ports on majority nodes of cluster, thus losing its
// leader and cluster being inoperable. And it waits for "failure-delay-ms"
// leader and cluster being inoperable. And it waits for "delay-ms"
// until recovery.
// The expected behavior is that once packet drop operation is undone,
// nodes come back online, thus cluster comes back operative. As always,
@ -371,7 +436,7 @@ enum FailureCase {
// BLACKHOLE_PEER_PORT_TX_RX_ALL drops all outgoing/incoming packets
// from/to the peer ports on all nodes, thus making cluster totally
// inoperable. It waits for "failure-delay-ms" until recovery.
// inoperable. It waits for "delay-ms" until recovery.
// The expected behavior is that once packet drop operation is undone,
// nodes come back online, thus cluster comes back operative. As always,
// after recovery, each member must be able to process client requests.
@ -379,7 +444,7 @@ enum FailureCase {
// DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER delays outgoing/incoming packets
// from/to the peer port on a randomly chosen follower (non-leader).
// It waits for "failure-delay-ms" until recovery.
// It waits for "delay-ms" until recovery.
// The expected behavior is that once packet delay operation is undone,
// the follower comes back and tries to catch up with latest changes from
// cluster. And as always, after recovery, each member must be able to
@ -388,8 +453,8 @@ enum FailureCase {
// RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER delays outgoing/incoming
// packets from/to the peer port on a randomly chosen follower
// (non-leader) with a randomized time duration (thus isolated). It waits
// for "failure-delay-ms" until recovery.
// (non-leader) with a randomized time duration (thus isolated). It
// waits for "delay-ms" until recovery.
// The expected behavior is that once packet delay operation is undone,
// each member must be able to process client requests.
RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER = 201;
@ -418,8 +483,8 @@ enum FailureCase {
RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT = 203;
// DELAY_PEER_PORT_TX_RX_LEADER delays outgoing/incoming packets from/to
// the peer port on the active leader. And waits for "failure-delay-ms"
// until recovery.
// the peer port on the active leader. And waits for "delay-ms" until
// recovery.
// The expected behavior is that cluster may elect a new leader, and
// once packet delay operation is undone, the (old) leader comes back
// and tries to catch up with latest changes from cluster. As always,
@ -428,7 +493,7 @@ enum FailureCase {
// RANDOM_DELAY_PEER_PORT_TX_RX_LEADER delays outgoing/incoming packets
// from/to the peer port on the active leader with a randomized time
// duration. And waits for "failure-delay-ms" until recovery.
// duration. And waits for "delay-ms" until recovery.
// The expected behavior is that cluster may elect a new leader, and
// once packet delay operation is undone, the (old) leader comes back
// and tries to catch up with latest changes from cluster. As always,
@ -462,7 +527,7 @@ enum FailureCase {
// DELAY_PEER_PORT_TX_RX_QUORUM delays outgoing/incoming packets from/to
// the peer ports on majority nodes of cluster. And it waits for
// "failure-delay-ms" until recovery, likely to trigger election timeouts.
// "delay-ms" until recovery, likely to trigger election timeouts.
// The expected behavior is that cluster may elect a new leader, while
// quorum of nodes struggle with slow networks, and once delay operation
// is undone, nodes come back and cluster comes back operative. As always,
@ -471,8 +536,8 @@ enum FailureCase {
// RANDOM_DELAY_PEER_PORT_TX_RX_QUORUM delays outgoing/incoming packets
// from/to the peer ports on majority nodes of cluster, with randomized
// time durations. And it waits for "failure-delay-ms" until recovery,
// likely to trigger election timeouts.
// time durations. And it waits for "delay-ms" until recovery, likely
// to trigger election timeouts.
// The expected behavior is that cluster may elect a new leader, while
// quorum of nodes struggle with slow networks, and once delay operation
// is undone, nodes come back and cluster comes back operative. As always,
@ -480,8 +545,8 @@ enum FailureCase {
RANDOM_DELAY_PEER_PORT_TX_RX_QUORUM = 209;
// DELAY_PEER_PORT_TX_RX_ALL delays outgoing/incoming packets from/to the
// peer ports on all nodes. And it waits for "failure-delay-ms" until
// recovery, likely to trigger election timeouts.
// peer ports on all nodes. And it waits for "delay-ms" until recovery,
// likely to trigger election timeouts.
// The expected behavior is that cluster may become totally inoperable,
// struggling with slow networks across the whole cluster. Once delay
// operation is undone, nodes come back and cluster comes back operative.
@ -491,7 +556,7 @@ enum FailureCase {
// RANDOM_DELAY_PEER_PORT_TX_RX_ALL delays outgoing/incoming packets
// from/to the peer ports on all nodes, with randomized time durations.
// And it waits for "failure-delay-ms" until recovery, likely to trigger
// And it waits for "delay-ms" until recovery, likely to trigger
// election timeouts.
// The expected behavior is that cluster may become totally inoperable,
// struggling with slow networks across the whole cluster. Once delay
@ -500,14 +565,26 @@ enum FailureCase {
// requests.
RANDOM_DELAY_PEER_PORT_TX_RX_ALL = 211;
// NO_FAIL_WITH_STRESS runs no-op failure injection that does not do
// anything against cluster for "failure-delay-ms" duration, while
// stressers are still sending requests.
// NO_FAIL_WITH_STRESS stops injecting failures while testing the
// consistency and correctness under pressure loads, for the duration of
// "delay-ms". Goal is to ensure cluster be still making progress
// on recovery, and verify system does not deadlock following a sequence
// of failure injections.
// The expected behavior is that cluster remains fully operative in healthy
// condition. As always, after recovery, each member must be able to process
// client requests.
NO_FAIL_WITH_STRESS = 300;
// NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS runs no-op failure injection
// that does not do anything against cluster for "failure-delay-ms"
// duration, while all stressers are stopped.
// NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS neither injects failures nor
// sends stressig client requests to the cluster, for the duration of
// "delay-ms". Goal is to ensure cluster be still making progress
// on recovery, and verify system does not deadlock following a sequence
// of failure injections.
// The expected behavior is that cluster remains fully operative in healthy
// condition, and clients requests during liveness period succeed without
// errors.
// Note: this is how Google Chubby does failure injection testing
// https://static.googleusercontent.com/media/research.google.com/en//archive/paxos_made_live.pdf.
NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS = 301;
// FAILPOINTS injects failpoints to etcd server runtime, triggering panics
@ -518,7 +595,7 @@ enum FailureCase {
EXTERNAL = 500;
}
enum StressType {
enum Stresser {
KV = 0;
LEASE = 1;
ELECTION_RUNNER = 2;
@ -526,3 +603,10 @@ enum StressType {
LOCK_RACER_RUNNER = 4;
LEASE_RUNNER = 5;
}
enum Checker {
KV_HASH = 0;
LEASE_EXPIRE = 1;
RUNNER = 2;
NO_CHECK = 3;
}

320
functional/tester/case.go Normal file
View File

@ -0,0 +1,320 @@
// Copyright 2018 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 tester
import (
"fmt"
"math/rand"
"time"
"github.com/coreos/etcd/functional/rpcpb"
"go.uber.org/zap"
)
// Case defines failure/test injection interface.
// To add a test case:
// 1. implement "Case" interface
// 2. define fail case name in "rpcpb.Case"
type Case interface {
// Inject injeccts the failure into the testing cluster at the given
// round. When calling the function, the cluster should be in health.
Inject(clus *Cluster) error
// Recover recovers the injected failure caused by the injection of the
// given round and wait for the recovery of the testing cluster.
Recover(clus *Cluster) error
// Desc returns a description of the failure
Desc() string
// TestCase returns "rpcpb.Case" enum type.
TestCase() rpcpb.Case
}
type injectMemberFunc func(*Cluster, int) error
type recoverMemberFunc func(*Cluster, int) error
type caseByFunc struct {
desc string
rpcpbCase rpcpb.Case
injectMember injectMemberFunc
recoverMember recoverMemberFunc
}
func (c *caseByFunc) Desc() string {
if c.desc != "" {
return c.desc
}
return c.rpcpbCase.String()
}
func (c *caseByFunc) TestCase() rpcpb.Case {
return c.rpcpbCase
}
type caseFollower struct {
caseByFunc
last int
lead int
}
func (c *caseFollower) updateIndex(clus *Cluster) error {
lead, err := clus.GetLeader()
if err != nil {
return err
}
c.lead = lead
n := len(clus.Members)
if c.last == -1 { // first run
c.last = clus.rd % n
if c.last == c.lead {
c.last = (c.last + 1) % n
}
} else {
c.last = (c.last + 1) % n
if c.last == c.lead {
c.last = (c.last + 1) % n
}
}
return nil
}
func (c *caseFollower) Inject(clus *Cluster) error {
if err := c.updateIndex(clus); err != nil {
return err
}
return c.injectMember(clus, c.last)
}
func (c *caseFollower) Recover(clus *Cluster) error {
return c.recoverMember(clus, c.last)
}
func (c *caseFollower) Desc() string {
if c.desc != "" {
return c.desc
}
return c.rpcpbCase.String()
}
func (c *caseFollower) TestCase() rpcpb.Case {
return c.rpcpbCase
}
type caseLeader struct {
caseByFunc
last int
lead int
}
func (c *caseLeader) updateIndex(clus *Cluster) error {
lead, err := clus.GetLeader()
if err != nil {
return err
}
c.lead = lead
c.last = lead
return nil
}
func (c *caseLeader) Inject(clus *Cluster) error {
if err := c.updateIndex(clus); err != nil {
return err
}
return c.injectMember(clus, c.last)
}
func (c *caseLeader) Recover(clus *Cluster) error {
return c.recoverMember(clus, c.last)
}
func (c *caseLeader) TestCase() rpcpb.Case {
return c.rpcpbCase
}
type caseQuorum struct {
caseByFunc
injected map[int]struct{}
}
func (c *caseQuorum) Inject(clus *Cluster) error {
c.injected = pickQuorum(len(clus.Members))
for idx := range c.injected {
if err := c.injectMember(clus, idx); err != nil {
return err
}
}
return nil
}
func (c *caseQuorum) Recover(clus *Cluster) error {
for idx := range c.injected {
if err := c.recoverMember(clus, idx); err != nil {
return err
}
}
return nil
}
func (c *caseQuorum) Desc() string {
if c.desc != "" {
return c.desc
}
return c.rpcpbCase.String()
}
func (c *caseQuorum) TestCase() rpcpb.Case {
return c.rpcpbCase
}
func pickQuorum(size int) (picked map[int]struct{}) {
picked = make(map[int]struct{})
r := rand.New(rand.NewSource(time.Now().UnixNano()))
quorum := size/2 + 1
for len(picked) < quorum {
idx := r.Intn(size)
picked[idx] = struct{}{}
}
return picked
}
type caseAll caseByFunc
func (c *caseAll) Inject(clus *Cluster) error {
for i := range clus.Members {
if err := c.injectMember(clus, i); err != nil {
return err
}
}
return nil
}
func (c *caseAll) Recover(clus *Cluster) error {
for i := range clus.Members {
if err := c.recoverMember(clus, i); err != nil {
return err
}
}
return nil
}
func (c *caseAll) Desc() string {
if c.desc != "" {
return c.desc
}
return c.rpcpbCase.String()
}
func (c *caseAll) TestCase() rpcpb.Case {
return c.rpcpbCase
}
// caseUntilSnapshot injects a failure/test and waits for a snapshot event
type caseUntilSnapshot struct {
desc string
rpcpbCase rpcpb.Case
Case
}
// all delay failure cases except the ones failing with latency
// greater than election timeout (trigger leader election and
// cluster keeps operating anyways)
var slowCases = map[rpcpb.Case]bool{
rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER: true,
rpcpb.Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER: true,
rpcpb.Case_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_QUORUM: true,
rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_ALL: true,
}
func (c *caseUntilSnapshot) Inject(clus *Cluster) error {
if err := c.Case.Inject(clus); err != nil {
return err
}
snapshotCount := clus.Members[0].Etcd.SnapshotCount
now := time.Now()
clus.lg.Info(
"trigger snapshot START",
zap.String("desc", c.Desc()),
zap.Int64("etcd-snapshot-count", snapshotCount),
)
// maxRev may fail since failure just injected, retry if failed.
startRev, err := clus.maxRev()
for i := 0; i < 10 && startRev == 0; i++ {
startRev, err = clus.maxRev()
}
if startRev == 0 {
return err
}
lastRev := startRev
// healthy cluster could accept 1000 req/sec at least.
// 3x time to trigger snapshot.
retries := int(snapshotCount) / 1000 * 3
if v, ok := slowCases[c.TestCase()]; v && ok {
// slow network takes more retries
retries *= 5
}
for i := 0; i < retries; i++ {
lastRev, _ = clus.maxRev()
// If the number of proposals committed is bigger than snapshot count,
// a new snapshot should have been created.
dicc := lastRev - startRev
if dicc > snapshotCount {
clus.lg.Info(
"trigger snapshot PASS",
zap.Int("retries", i),
zap.String("desc", c.Desc()),
zap.Int64("committed-entries", dicc),
zap.Int64("etcd-snapshot-count", snapshotCount),
zap.Int64("last-revision", lastRev),
zap.Duration("took", time.Since(now)),
)
return nil
}
clus.lg.Info(
"trigger snapshot PROGRESS",
zap.Int("retries", i),
zap.Int64("committed-entries", dicc),
zap.Int64("etcd-snapshot-count", snapshotCount),
zap.Int64("last-revision", lastRev),
zap.Duration("took", time.Since(now)),
)
time.Sleep(time.Second)
}
return fmt.Errorf("cluster too slow: only %d commits in %d retries", lastRev-startRev, retries)
}
func (c *caseUntilSnapshot) Desc() string {
if c.desc != "" {
return c.desc
}
if c.rpcpbCase.String() != "" {
return c.rpcpbCase.String()
}
return c.Case.Desc()
}
func (c *caseUntilSnapshot) TestCase() rpcpb.Case {
return c.rpcpbCase
}

View File

@ -20,22 +20,22 @@ import (
"go.uber.org/zap"
)
type failureDelay struct {
Failure
type caseDelay struct {
Case
delayDuration time.Duration
}
func (f *failureDelay) Inject(clus *Cluster) error {
if err := f.Failure.Inject(clus); err != nil {
func (c *caseDelay) Inject(clus *Cluster) error {
if err := c.Case.Inject(clus); err != nil {
return err
}
if f.delayDuration > 0 {
if c.delayDuration > 0 {
clus.lg.Info(
"wait after inject",
zap.Duration("delay", f.delayDuration),
zap.String("desc", f.Failure.Desc()),
zap.Duration("delay", c.delayDuration),
zap.String("desc", c.Case.Desc()),
)
time.Sleep(f.delayDuration)
time.Sleep(c.delayDuration)
}
return nil
}

View File

@ -21,35 +21,35 @@ import (
"github.com/coreos/etcd/functional/rpcpb"
)
type failureExternal struct {
Failure
type caseExternal struct {
Case
desc string
failureCase rpcpb.FailureCase
desc string
rpcpbCase rpcpb.Case
scriptPath string
}
func (f *failureExternal) Inject(clus *Cluster) error {
return exec.Command(f.scriptPath, "enable", fmt.Sprintf("%d", clus.rd)).Run()
func (c *caseExternal) Inject(clus *Cluster) error {
return exec.Command(c.scriptPath, "enable", fmt.Sprintf("%d", clus.rd)).Run()
}
func (f *failureExternal) Recover(clus *Cluster) error {
return exec.Command(f.scriptPath, "disable", fmt.Sprintf("%d", clus.rd)).Run()
func (c *caseExternal) Recover(clus *Cluster) error {
return exec.Command(c.scriptPath, "disable", fmt.Sprintf("%d", clus.rd)).Run()
}
func (f *failureExternal) Desc() string {
return f.desc
func (c *caseExternal) Desc() string {
return c.desc
}
func (f *failureExternal) FailureCase() rpcpb.FailureCase {
return f.failureCase
func (c *caseExternal) TestCase() rpcpb.Case {
return c.rpcpbCase
}
func new_FailureCase_EXTERNAL(scriptPath string) Failure {
return &failureExternal{
desc: fmt.Sprintf("external fault injector (script: %q)", scriptPath),
failureCase: rpcpb.FailureCase_EXTERNAL,
scriptPath: scriptPath,
func new_Case_EXTERNAL(scriptPath string) Case {
return &caseExternal{
desc: fmt.Sprintf("external fault injector (script: %q)", scriptPath),
rpcpbCase: rpcpb.Case_EXTERNAL,
scriptPath: scriptPath,
}
}

View File

@ -32,7 +32,7 @@ type failpointStats struct {
var fpStats failpointStats
func failpointFailures(clus *Cluster) (ret []Failure, err error) {
func failpointFailures(clus *Cluster) (ret []Case, err error) {
var fps []string
fps, err = failpointPaths(clus.Members[0].FailpointHTTPAddr)
if err != nil {
@ -44,21 +44,21 @@ func failpointFailures(clus *Cluster) (ret []Failure, err error) {
continue
}
fpFails := failuresFromFailpoint(fp, clus.Tester.FailpointCommands)
fpFails := casesFromFailpoint(fp, clus.Tester.FailpointCommands)
// wrap in delays so failpoint has time to trigger
for i, fpf := range fpFails {
if strings.Contains(fp, "Snap") {
// hack to trigger snapshot failpoints
fpFails[i] = &failureUntilSnapshot{
desc: fpf.Desc(),
failureCase: rpcpb.FailureCase_FAILPOINTS,
Failure: fpf,
fpFails[i] = &caseUntilSnapshot{
desc: fpf.Desc(),
rpcpbCase: rpcpb.Case_FAILPOINTS,
Case: fpf,
}
} else {
fpFails[i] = &failureDelay{
Failure: fpf,
delayDuration: clus.GetFailureDelayDuration(),
fpFails[i] = &caseDelay{
Case: fpf,
delayDuration: clus.GetCaseDelayDuration(),
}
}
}
@ -86,42 +86,45 @@ func failpointPaths(endpoint string) ([]string, error) {
return fps, nil
}
// failpoints follows FreeBSD KFAIL_POINT syntax.
// failpoints follows FreeBSD FAIL_POINT syntax.
// e.g. panic("etcd-tester"),1*sleep(1000)->panic("etcd-tester")
func failuresFromFailpoint(fp string, failpointCommands []string) (fs []Failure) {
func casesFromFailpoint(fp string, failpointCommands []string) (fs []Case) {
recov := makeRecoverFailpoint(fp)
for _, fcmd := range failpointCommands {
inject := makeInjectFailpoint(fp, fcmd)
fs = append(fs, []Failure{
&failureFollower{
failureByFunc: failureByFunc{
fs = append(fs, []Case{
&caseFollower{
caseByFunc: caseByFunc{
desc: fmt.Sprintf("failpoint %q (one: %q)", fp, fcmd),
failureCase: rpcpb.FailureCase_FAILPOINTS,
rpcpbCase: rpcpb.Case_FAILPOINTS,
injectMember: inject,
recoverMember: recov,
},
last: -1,
lead: -1,
},
&failureLeader{
failureByFunc: failureByFunc{
&caseLeader{
caseByFunc: caseByFunc{
desc: fmt.Sprintf("failpoint %q (leader: %q)", fp, fcmd),
failureCase: rpcpb.FailureCase_FAILPOINTS,
rpcpbCase: rpcpb.Case_FAILPOINTS,
injectMember: inject,
recoverMember: recov,
},
last: -1,
lead: -1,
},
&failureQuorum{
desc: fmt.Sprintf("failpoint %q (quorum: %q)", fp, fcmd),
failureCase: rpcpb.FailureCase_FAILPOINTS,
injectMember: inject,
recoverMember: recov,
&caseQuorum{
caseByFunc: caseByFunc{
desc: fmt.Sprintf("failpoint %q (quorum: %q)", fp, fcmd),
rpcpbCase: rpcpb.Case_FAILPOINTS,
injectMember: inject,
recoverMember: recov,
},
injected: make(map[int]struct{}),
},
&failureAll{
&caseAll{
desc: fmt.Sprintf("failpoint %q (all: %q)", fp, fcmd),
failureCase: rpcpb.FailureCase_FAILPOINTS,
rpcpbCase: rpcpb.Case_FAILPOINTS,
injectMember: inject,
recoverMember: recov,
},

View File

@ -0,0 +1,104 @@
// Copyright 2018 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 tester
import "github.com/coreos/etcd/functional/rpcpb"
func inject_BLACKHOLE_PEER_PORT_TX_RX(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_BLACKHOLE_PEER_PORT_TX_RX)
}
func recover_BLACKHOLE_PEER_PORT_TX_RX(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_UNBLACKHOLE_PEER_PORT_TX_RX)
}
func new_Case_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER(clus *Cluster) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
c := &caseFollower{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_Case_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT() Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
c := &caseFollower{cc, -1, -1}
return &caseUntilSnapshot{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
Case: c,
}
}
func new_Case_BLACKHOLE_PEER_PORT_TX_RX_LEADER(clus *Cluster) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_LEADER,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
c := &caseLeader{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_Case_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT() Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
c := &caseLeader{cc, -1, -1}
return &caseUntilSnapshot{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT,
Case: c,
}
}
func new_Case_BLACKHOLE_PEER_PORT_TX_RX_QUORUM(clus *Cluster) Case {
c := &caseQuorum{
caseByFunc: caseByFunc{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_QUORUM,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
},
injected: make(map[int]struct{}),
}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_Case_BLACKHOLE_PEER_PORT_TX_RX_ALL(clus *Cluster) Case {
c := &caseAll{
rpcpbCase: rpcpb.Case_BLACKHOLE_PEER_PORT_TX_RX_ALL,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}

View File

@ -46,108 +46,111 @@ func recover_DELAY_PEER_PORT_TX_RX(clus *Cluster, idx int) error {
return err
}
func new_FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER(clus *Cluster, random bool) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER,
func new_Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER(clus *Cluster, random bool) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER,
injectMember: inject_DELAY_PEER_PORT_TX_RX,
recoverMember: recover_DELAY_PEER_PORT_TX_RX,
}
clus.Tester.UpdatedDelayLatencyMs = clus.Tester.DelayLatencyMs
if random {
clus.UpdateDelayLatencyMs()
ff.failureCase = rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER
cc.rpcpbCase = rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER
}
f := &failureFollower{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
c := &caseFollower{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster, random bool) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
func new_Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster, random bool) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
injectMember: inject_DELAY_PEER_PORT_TX_RX,
recoverMember: recover_DELAY_PEER_PORT_TX_RX,
}
clus.Tester.UpdatedDelayLatencyMs = clus.Tester.DelayLatencyMs
if random {
clus.UpdateDelayLatencyMs()
ff.failureCase = rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT
cc.rpcpbCase = rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT
}
f := &failureFollower{ff, -1, -1}
return &failureUntilSnapshot{
failureCase: ff.failureCase,
Failure: f,
c := &caseFollower{cc, -1, -1}
return &caseUntilSnapshot{
rpcpbCase: cc.rpcpbCase,
Case: c,
}
}
func new_FailureCase_DELAY_PEER_PORT_TX_RX_LEADER(clus *Cluster, random bool) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_LEADER,
func new_Case_DELAY_PEER_PORT_TX_RX_LEADER(clus *Cluster, random bool) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_DELAY_PEER_PORT_TX_RX_LEADER,
injectMember: inject_DELAY_PEER_PORT_TX_RX,
recoverMember: recover_DELAY_PEER_PORT_TX_RX,
}
clus.Tester.UpdatedDelayLatencyMs = clus.Tester.DelayLatencyMs
if random {
clus.UpdateDelayLatencyMs()
ff.failureCase = rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER
cc.rpcpbCase = rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER
}
f := &failureLeader{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
c := &caseLeader{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_FailureCase_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster, random bool) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT,
func new_Case_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster, random bool) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT,
injectMember: inject_DELAY_PEER_PORT_TX_RX,
recoverMember: recover_DELAY_PEER_PORT_TX_RX,
}
clus.Tester.UpdatedDelayLatencyMs = clus.Tester.DelayLatencyMs
if random {
clus.UpdateDelayLatencyMs()
ff.failureCase = rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT
cc.rpcpbCase = rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT
}
f := &failureLeader{ff, -1, -1}
return &failureUntilSnapshot{
failureCase: ff.failureCase,
Failure: f,
c := &caseLeader{cc, -1, -1}
return &caseUntilSnapshot{
rpcpbCase: cc.rpcpbCase,
Case: c,
}
}
func new_FailureCase_DELAY_PEER_PORT_TX_RX_QUORUM(clus *Cluster, random bool) Failure {
f := &failureQuorum{
failureCase: rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_QUORUM,
injectMember: inject_DELAY_PEER_PORT_TX_RX,
recoverMember: recover_DELAY_PEER_PORT_TX_RX,
func new_Case_DELAY_PEER_PORT_TX_RX_QUORUM(clus *Cluster, random bool) Case {
c := &caseQuorum{
caseByFunc: caseByFunc{
rpcpbCase: rpcpb.Case_DELAY_PEER_PORT_TX_RX_QUORUM,
injectMember: inject_DELAY_PEER_PORT_TX_RX,
recoverMember: recover_DELAY_PEER_PORT_TX_RX,
},
injected: make(map[int]struct{}),
}
clus.Tester.UpdatedDelayLatencyMs = clus.Tester.DelayLatencyMs
if random {
clus.UpdateDelayLatencyMs()
f.failureCase = rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_QUORUM
c.rpcpbCase = rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_QUORUM
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_FailureCase_DELAY_PEER_PORT_TX_RX_ALL(clus *Cluster, random bool) Failure {
f := &failureAll{
failureCase: rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_ALL,
func new_Case_DELAY_PEER_PORT_TX_RX_ALL(clus *Cluster, random bool) Case {
c := &caseAll{
rpcpbCase: rpcpb.Case_DELAY_PEER_PORT_TX_RX_ALL,
injectMember: inject_DELAY_PEER_PORT_TX_RX,
recoverMember: recover_DELAY_PEER_PORT_TX_RX,
}
clus.Tester.UpdatedDelayLatencyMs = clus.Tester.DelayLatencyMs
if random {
clus.UpdateDelayLatencyMs()
f.failureCase = rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_ALL
c.rpcpbCase = rpcpb.Case_RANDOM_DELAY_PEER_PORT_TX_RX_ALL
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}

View File

@ -0,0 +1,99 @@
// Copyright 2018 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 tester
import (
"time"
"github.com/coreos/etcd/functional/rpcpb"
"go.uber.org/zap"
)
type caseNoFailWithStress caseByFunc
func (c *caseNoFailWithStress) Inject(clus *Cluster) error {
return nil
}
func (c *caseNoFailWithStress) Recover(clus *Cluster) error {
return nil
}
func (c *caseNoFailWithStress) Desc() string {
if c.desc != "" {
return c.desc
}
return c.rpcpbCase.String()
}
func (c *caseNoFailWithStress) TestCase() rpcpb.Case {
return c.rpcpbCase
}
func new_Case_NO_FAIL_WITH_STRESS(clus *Cluster) Case {
c := &caseNoFailWithStress{
rpcpbCase: rpcpb.Case_NO_FAIL_WITH_STRESS,
}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
type caseNoFailWithNoStressForLiveness caseByFunc
func (c *caseNoFailWithNoStressForLiveness) Inject(clus *Cluster) error {
clus.lg.Info(
"extra delay for liveness mode with no stresser",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.String("desc", c.Desc()),
)
time.Sleep(clus.GetCaseDelayDuration())
clus.lg.Info(
"wait health in liveness mode",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.String("desc", c.Desc()),
)
return clus.WaitHealth()
}
func (c *caseNoFailWithNoStressForLiveness) Recover(clus *Cluster) error {
return nil
}
func (c *caseNoFailWithNoStressForLiveness) Desc() string {
if c.desc != "" {
return c.desc
}
return c.rpcpbCase.String()
}
func (c *caseNoFailWithNoStressForLiveness) TestCase() rpcpb.Case {
return c.rpcpbCase
}
func new_Case_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS(clus *Cluster) Case {
c := &caseNoFailWithNoStressForLiveness{
rpcpbCase: rpcpb.Case_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS,
}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}

View File

@ -174,43 +174,43 @@ func recover_SIGQUIT_ETCD_AND_REMOVE_DATA(clus *Cluster, idx1 int) error {
return err
}
func new_FailureCase_SIGQUIT_AND_REMOVE_ONE_FOLLOWER(clus *Cluster) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_SIGQUIT_AND_REMOVE_ONE_FOLLOWER,
func new_Case_SIGQUIT_AND_REMOVE_ONE_FOLLOWER(clus *Cluster) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_SIGQUIT_AND_REMOVE_ONE_FOLLOWER,
injectMember: inject_SIGQUIT_ETCD_AND_REMOVE_DATA,
recoverMember: recover_SIGQUIT_ETCD_AND_REMOVE_DATA,
}
f := &failureFollower{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
c := &caseFollower{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_FailureCase_SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Failure {
return &failureUntilSnapshot{
failureCase: rpcpb.FailureCase_SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
Failure: new_FailureCase_SIGQUIT_AND_REMOVE_ONE_FOLLOWER(clus),
func new_Case_SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Case {
return &caseUntilSnapshot{
rpcpbCase: rpcpb.Case_SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
Case: new_Case_SIGQUIT_AND_REMOVE_ONE_FOLLOWER(clus),
}
}
func new_FailureCase_SIGQUIT_AND_REMOVE_LEADER(clus *Cluster) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_SIGQUIT_AND_REMOVE_LEADER,
func new_Case_SIGQUIT_AND_REMOVE_LEADER(clus *Cluster) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_SIGQUIT_AND_REMOVE_LEADER,
injectMember: inject_SIGQUIT_ETCD_AND_REMOVE_DATA,
recoverMember: recover_SIGQUIT_ETCD_AND_REMOVE_DATA,
}
f := &failureLeader{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
c := &caseLeader{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_FailureCase_SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Failure {
return &failureUntilSnapshot{
failureCase: rpcpb.FailureCase_SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT,
Failure: new_FailureCase_SIGQUIT_AND_REMOVE_LEADER(clus),
func new_Case_SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Case {
return &caseUntilSnapshot{
rpcpbCase: rpcpb.Case_SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT,
Case: new_Case_SIGQUIT_AND_REMOVE_LEADER(clus),
}
}

View File

@ -0,0 +1,275 @@
// Copyright 2018 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 tester
import (
"context"
"fmt"
"strings"
"time"
"github.com/coreos/etcd/clientv3"
"github.com/coreos/etcd/functional/rpcpb"
"go.uber.org/zap"
)
type fetchSnapshotCaseQuorum struct {
desc string
rpcpbCase rpcpb.Case
injected map[int]struct{}
snapshotted int
}
func (c *fetchSnapshotCaseQuorum) Inject(clus *Cluster) error {
// 1. Assume node C is the current leader with most up-to-date data.
lead, err := clus.GetLeader()
if err != nil {
return err
}
c.snapshotted = lead
// 2. Download snapshot from node C, before destroying node A and B.
clus.lg.Info(
"save snapshot on leader node START",
zap.String("target-endpoint", clus.Members[lead].EtcdClientEndpoint),
)
var resp *rpcpb.Response
resp, err = clus.sendOpWithResp(lead, rpcpb.Operation_SAVE_SNAPSHOT)
if resp == nil || (resp != nil && !resp.Success) || err != nil {
clus.lg.Info(
"save snapshot on leader node FAIL",
zap.String("target-endpoint", clus.Members[lead].EtcdClientEndpoint),
zap.Error(err),
)
return err
}
clus.lg.Info(
"save snapshot on leader node SUCCESS",
zap.String("target-endpoint", clus.Members[lead].EtcdClientEndpoint),
zap.String("member-name", resp.SnapshotInfo.MemberName),
zap.Strings("member-client-urls", resp.SnapshotInfo.MemberClientURLs),
zap.String("snapshot-path", resp.SnapshotInfo.SnapshotPath),
zap.String("snapshot-file-size", resp.SnapshotInfo.SnapshotFileSize),
zap.String("snapshot-total-size", resp.SnapshotInfo.SnapshotTotalSize),
zap.Int64("snapshot-total-key", resp.SnapshotInfo.SnapshotTotalKey),
zap.Int64("snapshot-hash", resp.SnapshotInfo.SnapshotHash),
zap.Int64("snapshot-revision", resp.SnapshotInfo.SnapshotRevision),
zap.String("took", resp.SnapshotInfo.Took),
zap.Error(err),
)
if err != nil {
return err
}
clus.Members[lead].SnapshotInfo = resp.SnapshotInfo
leaderc, err := clus.Members[lead].CreateEtcdClient()
if err != nil {
return err
}
defer leaderc.Close()
var mresp *clientv3.MemberListResponse
mresp, err = leaderc.MemberList(context.Background())
mss := []string{}
if err == nil && mresp != nil {
mss = describeMembers(mresp)
}
clus.lg.Info(
"member list before disastrous machine failure",
zap.String("request-to", clus.Members[lead].EtcdClientEndpoint),
zap.Strings("members", mss),
zap.Error(err),
)
if err != nil {
return err
}
// simulate real life; machine failures may happen
// after some time since last snapshot save
time.Sleep(time.Second)
// 3. Destroy node A and B, and make the whole cluster inoperable.
for {
c.injected = pickQuorum(len(clus.Members))
if _, ok := c.injected[lead]; !ok {
break
}
}
for idx := range c.injected {
clus.lg.Info(
"disastrous machine failure to quorum START",
zap.String("target-endpoint", clus.Members[idx].EtcdClientEndpoint),
)
err = clus.sendOp(idx, rpcpb.Operation_SIGQUIT_ETCD_AND_REMOVE_DATA)
clus.lg.Info(
"disastrous machine failure to quorum END",
zap.String("target-endpoint", clus.Members[idx].EtcdClientEndpoint),
zap.Error(err),
)
if err != nil {
return err
}
}
// 4. Now node C cannot operate either.
// 5. SIGTERM node C and remove its data directories.
clus.lg.Info(
"disastrous machine failure to old leader START",
zap.String("target-endpoint", clus.Members[lead].EtcdClientEndpoint),
)
err = clus.sendOp(lead, rpcpb.Operation_SIGQUIT_ETCD_AND_REMOVE_DATA)
clus.lg.Info(
"disastrous machine failure to old leader END",
zap.String("target-endpoint", clus.Members[lead].EtcdClientEndpoint),
zap.Error(err),
)
return err
}
func (c *fetchSnapshotCaseQuorum) Recover(clus *Cluster) error {
// 6. Restore a new seed member from node C's latest snapshot file.
oldlead := c.snapshotted
// configuration on restart from recovered snapshot
// seed member's configuration is all the same as previous one
// except initial cluster string is now a single-node cluster
clus.Members[oldlead].EtcdOnSnapshotRestore = clus.Members[oldlead].Etcd
clus.Members[oldlead].EtcdOnSnapshotRestore.InitialClusterState = "existing"
name := clus.Members[oldlead].Etcd.Name
initClus := []string{}
for _, u := range clus.Members[oldlead].Etcd.AdvertisePeerURLs {
initClus = append(initClus, fmt.Sprintf("%s=%s", name, u))
}
clus.Members[oldlead].EtcdOnSnapshotRestore.InitialCluster = strings.Join(initClus, ",")
clus.lg.Info(
"restore snapshot and restart from snapshot request START",
zap.String("target-endpoint", clus.Members[oldlead].EtcdClientEndpoint),
zap.Strings("initial-cluster", initClus),
)
err := clus.sendOp(oldlead, rpcpb.Operation_RESTORE_RESTART_FROM_SNAPSHOT)
clus.lg.Info(
"restore snapshot and restart from snapshot request END",
zap.String("target-endpoint", clus.Members[oldlead].EtcdClientEndpoint),
zap.Strings("initial-cluster", initClus),
zap.Error(err),
)
if err != nil {
return err
}
leaderc, err := clus.Members[oldlead].CreateEtcdClient()
if err != nil {
return err
}
defer leaderc.Close()
// 7. Add another member to establish 2-node cluster.
// 8. Add another member to establish 3-node cluster.
// 9. Add more if any.
idxs := make([]int, 0, len(c.injected))
for idx := range c.injected {
idxs = append(idxs, idx)
}
clus.lg.Info("member add START", zap.Int("members-to-add", len(idxs)))
for i, idx := range idxs {
clus.lg.Info(
"member add request SENT",
zap.String("target-endpoint", clus.Members[idx].EtcdClientEndpoint),
zap.Strings("peer-urls", clus.Members[idx].Etcd.AdvertisePeerURLs),
)
ctx, cancel := context.WithTimeout(context.Background(), time.Minute)
_, err := leaderc.MemberAdd(ctx, clus.Members[idx].Etcd.AdvertisePeerURLs)
cancel()
clus.lg.Info(
"member add request DONE",
zap.String("target-endpoint", clus.Members[idx].EtcdClientEndpoint),
zap.Strings("peer-urls", clus.Members[idx].Etcd.AdvertisePeerURLs),
zap.Error(err),
)
if err != nil {
return err
}
// start the added(new) member with fresh data
clus.Members[idx].EtcdOnSnapshotRestore = clus.Members[idx].Etcd
clus.Members[idx].EtcdOnSnapshotRestore.InitialClusterState = "existing"
name := clus.Members[idx].Etcd.Name
for _, u := range clus.Members[idx].Etcd.AdvertisePeerURLs {
initClus = append(initClus, fmt.Sprintf("%s=%s", name, u))
}
clus.Members[idx].EtcdOnSnapshotRestore.InitialCluster = strings.Join(initClus, ",")
clus.lg.Info(
"restart from snapshot request SENT",
zap.String("target-endpoint", clus.Members[idx].EtcdClientEndpoint),
zap.Strings("initial-cluster", initClus),
)
err = clus.sendOp(idx, rpcpb.Operation_RESTART_FROM_SNAPSHOT)
clus.lg.Info(
"restart from snapshot request DONE",
zap.String("target-endpoint", clus.Members[idx].EtcdClientEndpoint),
zap.Strings("initial-cluster", initClus),
zap.Error(err),
)
if err != nil {
return err
}
if i != len(c.injected)-1 {
// wait until membership reconfiguration entry gets applied
// TODO: test concurrent member add
dur := 5 * clus.Members[idx].ElectionTimeout()
clus.lg.Info(
"waiting after restart from snapshot request",
zap.Int("i", i),
zap.Int("idx", idx),
zap.Duration("sleep", dur),
)
time.Sleep(dur)
} else {
clus.lg.Info(
"restart from snapshot request ALL END",
zap.Int("i", i),
zap.Int("idx", idx),
)
}
}
return nil
}
func (c *fetchSnapshotCaseQuorum) Desc() string {
if c.desc != "" {
return c.desc
}
return c.rpcpbCase.String()
}
func (c *fetchSnapshotCaseQuorum) TestCase() rpcpb.Case {
return c.rpcpbCase
}
func new_Case_SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH(clus *Cluster) Case {
c := &fetchSnapshotCaseQuorum{
rpcpbCase: rpcpb.Case_SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH,
injected: make(map[int]struct{}),
snapshotted: -1,
}
// simulate real life; machine replacements may happen
// after some time since disaster
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}

View File

@ -0,0 +1,92 @@
// Copyright 2018 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 tester
import "github.com/coreos/etcd/functional/rpcpb"
func inject_SIGTERM_ETCD(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_SIGTERM_ETCD)
}
func recover_SIGTERM_ETCD(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_RESTART_ETCD)
}
func new_Case_SIGTERM_ONE_FOLLOWER(clus *Cluster) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_SIGTERM_ONE_FOLLOWER,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
}
c := &caseFollower{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_Case_SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Case {
return &caseUntilSnapshot{
rpcpbCase: rpcpb.Case_SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
Case: new_Case_SIGTERM_ONE_FOLLOWER(clus),
}
}
func new_Case_SIGTERM_LEADER(clus *Cluster) Case {
cc := caseByFunc{
rpcpbCase: rpcpb.Case_SIGTERM_LEADER,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
}
c := &caseLeader{cc, -1, -1}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_Case_SIGTERM_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Case {
return &caseUntilSnapshot{
rpcpbCase: rpcpb.Case_SIGTERM_LEADER_UNTIL_TRIGGER_SNAPSHOT,
Case: new_Case_SIGTERM_LEADER(clus),
}
}
func new_Case_SIGTERM_QUORUM(clus *Cluster) Case {
c := &caseQuorum{
caseByFunc: caseByFunc{
rpcpbCase: rpcpb.Case_SIGTERM_QUORUM,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
},
injected: make(map[int]struct{}),
}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}
func new_Case_SIGTERM_ALL(clus *Cluster) Case {
c := &caseAll{
rpcpbCase: rpcpb.Case_SIGTERM_ALL,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
}
return &caseDelay{
Case: c,
delayDuration: clus.GetCaseDelayDuration(),
}
}

View File

@ -0,0 +1,28 @@
// Copyright 2018 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 tester
import "github.com/coreos/etcd/functional/rpcpb"
// Checker checks cluster consistency.
type Checker interface {
// Type returns the checker type.
Type() rpcpb.Checker
// EtcdClientEndpoints returns the client endpoints of
// all checker target nodes..
EtcdClientEndpoints() []string
// Check returns an error if the system fails a consistency check.
Check() error
}

View File

@ -0,0 +1,89 @@
// Copyright 2018 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 tester
import (
"fmt"
"time"
"github.com/coreos/etcd/functional/rpcpb"
"go.uber.org/zap"
)
const retries = 7
type kvHashChecker struct {
ctype rpcpb.Checker
clus *Cluster
}
func newKVHashChecker(clus *Cluster) Checker {
return &kvHashChecker{
ctype: rpcpb.Checker_KV_HASH,
clus: clus,
}
}
func (hc *kvHashChecker) checkRevAndHashes() (err error) {
var (
revs map[string]int64
hashes map[string]int64
)
// retries in case of transient failure or etcd cluster has not stablized yet.
for i := 0; i < retries; i++ {
revs, hashes, err = hc.clus.getRevisionHash()
if err != nil {
hc.clus.lg.Warn(
"failed to get revision and hash",
zap.Int("retries", i),
zap.Error(err),
)
} else {
sameRev := getSameValue(revs)
sameHashes := getSameValue(hashes)
if sameRev && sameHashes {
return nil
}
hc.clus.lg.Warn(
"retrying; etcd cluster is not stable",
zap.Int("retries", i),
zap.Bool("same-revisions", sameRev),
zap.Bool("same-hashes", sameHashes),
zap.String("revisions", fmt.Sprintf("%+v", revs)),
zap.String("hashes", fmt.Sprintf("%+v", hashes)),
)
}
time.Sleep(time.Second)
}
if err != nil {
return fmt.Errorf("failed revision and hash check (%v)", err)
}
return fmt.Errorf("etcd cluster is not stable: [revisions: %v] and [hashes: %v]", revs, hashes)
}
func (hc *kvHashChecker) Type() rpcpb.Checker {
return hc.ctype
}
func (hc *kvHashChecker) EtcdClientEndpoints() []string {
return hc.clus.EtcdClientEndpoints()
}
func (hc *kvHashChecker) Check() error {
return hc.checkRevAndHashes()
}

View File

@ -27,83 +27,32 @@ import (
"google.golang.org/grpc"
)
const retries = 7
// Checker checks cluster consistency.
type Checker interface {
// Check returns an error if the system fails a consistency check.
Check() error
type leaseExpireChecker struct {
ctype rpcpb.Checker
lg *zap.Logger
m *rpcpb.Member
ls *leaseStresser
cli *clientv3.Client
}
type hashAndRevGetter interface {
getRevisionHash() (revs map[string]int64, hashes map[string]int64, err error)
}
type hashChecker struct {
lg *zap.Logger
hrg hashAndRevGetter
}
func newHashChecker(lg *zap.Logger, hrg hashAndRevGetter) Checker {
return &hashChecker{
lg: lg,
hrg: hrg,
func newLeaseExpireChecker(ls *leaseStresser) Checker {
return &leaseExpireChecker{
ctype: rpcpb.Checker_LEASE_EXPIRE,
lg: ls.lg,
m: ls.m,
ls: ls,
}
}
const leaseCheckerTimeout = 10 * time.Second
func (hc *hashChecker) checkRevAndHashes() (err error) {
var (
revs map[string]int64
hashes map[string]int64
)
// retries in case of transient failure or etcd cluster has not stablized yet.
for i := 0; i < retries; i++ {
revs, hashes, err = hc.hrg.getRevisionHash()
if err != nil {
hc.lg.Warn(
"failed to get revision and hash",
zap.Int("retries", i),
zap.Error(err),
)
} else {
sameRev := getSameValue(revs)
sameHashes := getSameValue(hashes)
if sameRev && sameHashes {
return nil
}
hc.lg.Warn(
"retrying; etcd cluster is not stable",
zap.Int("retries", i),
zap.Bool("same-revisions", sameRev),
zap.Bool("same-hashes", sameHashes),
zap.String("revisions", fmt.Sprintf("%+v", revs)),
zap.String("hashes", fmt.Sprintf("%+v", hashes)),
)
}
time.Sleep(time.Second)
}
if err != nil {
return fmt.Errorf("failed revision and hash check (%v)", err)
}
return fmt.Errorf("etcd cluster is not stable: [revisions: %v] and [hashes: %v]", revs, hashes)
func (lc *leaseExpireChecker) Type() rpcpb.Checker {
return lc.ctype
}
func (hc *hashChecker) Check() error {
return hc.checkRevAndHashes()
func (lc *leaseExpireChecker) EtcdClientEndpoints() []string {
return []string{lc.m.EtcdClientEndpoint}
}
type leaseChecker struct {
lg *zap.Logger
m *rpcpb.Member
ls *leaseStresser
cli *clientv3.Client
}
func (lc *leaseChecker) Check() error {
func (lc *leaseExpireChecker) Check() error {
if lc.ls == nil {
return nil
}
@ -134,9 +83,11 @@ func (lc *leaseChecker) Check() error {
return lc.checkShortLivedLeases()
}
const leaseExpireCheckerTimeout = 10 * time.Second
// checkShortLivedLeases ensures leases expire.
func (lc *leaseChecker) checkShortLivedLeases() error {
ctx, cancel := context.WithTimeout(context.Background(), leaseCheckerTimeout)
func (lc *leaseExpireChecker) checkShortLivedLeases() error {
ctx, cancel := context.WithTimeout(context.Background(), leaseExpireCheckerTimeout)
errc := make(chan error)
defer cancel()
for leaseID := range lc.ls.shortLivedLeases.leases {
@ -154,7 +105,7 @@ func (lc *leaseChecker) checkShortLivedLeases() error {
return errsToError(errs)
}
func (lc *leaseChecker) checkShortLivedLease(ctx context.Context, leaseID int64) (err error) {
func (lc *leaseExpireChecker) checkShortLivedLease(ctx context.Context, leaseID int64) (err error) {
// retry in case of transient failure or lease is expired but not yet revoked due to the fact that etcd cluster didn't have enought time to delete it.
var resp *clientv3.LeaseTimeToLiveResponse
for i := 0; i < retries; i++ {
@ -199,7 +150,7 @@ func (lc *leaseChecker) checkShortLivedLease(ctx context.Context, leaseID int64)
return err
}
func (lc *leaseChecker) checkLease(ctx context.Context, expired bool, leaseID int64) error {
func (lc *leaseExpireChecker) checkLease(ctx context.Context, expired bool, leaseID int64) error {
keysExpired, err := lc.hasKeysAttachedToLeaseExpired(ctx, leaseID)
if err != nil {
lc.lg.Warn(
@ -227,8 +178,8 @@ func (lc *leaseChecker) checkLease(ctx context.Context, expired bool, leaseID in
return nil
}
func (lc *leaseChecker) check(expired bool, leases map[int64]time.Time) error {
ctx, cancel := context.WithTimeout(context.Background(), leaseCheckerTimeout)
func (lc *leaseExpireChecker) check(expired bool, leases map[int64]time.Time) error {
ctx, cancel := context.WithTimeout(context.Background(), leaseExpireCheckerTimeout)
defer cancel()
for leaseID := range leases {
if err := lc.checkLease(ctx, expired, leaseID); err != nil {
@ -239,7 +190,7 @@ func (lc *leaseChecker) check(expired bool, leases map[int64]time.Time) error {
}
// TODO: handle failures from "grpc.FailFast(false)"
func (lc *leaseChecker) getLeaseByID(ctx context.Context, leaseID int64) (*clientv3.LeaseTimeToLiveResponse, error) {
func (lc *leaseExpireChecker) getLeaseByID(ctx context.Context, leaseID int64) (*clientv3.LeaseTimeToLiveResponse, error) {
return lc.cli.TimeToLive(
ctx,
clientv3.LeaseID(leaseID),
@ -247,7 +198,7 @@ func (lc *leaseChecker) getLeaseByID(ctx context.Context, leaseID int64) (*clien
)
}
func (lc *leaseChecker) hasLeaseExpired(ctx context.Context, leaseID int64) (bool, error) {
func (lc *leaseExpireChecker) hasLeaseExpired(ctx context.Context, leaseID int64) (bool, error) {
// keep retrying until lease's state is known or ctx is being canceled
for ctx.Err() == nil {
resp, err := lc.getLeaseByID(ctx, leaseID)
@ -272,7 +223,7 @@ func (lc *leaseChecker) hasLeaseExpired(ctx context.Context, leaseID int64) (boo
// The keys attached to the lease has the format of "<leaseID>_<idx>" where idx is the ordering key creation
// Since the format of keys contains about leaseID, finding keys base on "<leaseID>" prefix
// determines whether the attached keys for a given leaseID has been deleted or not
func (lc *leaseChecker) hasKeysAttachedToLeaseExpired(ctx context.Context, leaseID int64) (bool, error) {
func (lc *leaseExpireChecker) hasKeysAttachedToLeaseExpired(ctx context.Context, leaseID int64) (bool, error) {
resp, err := lc.cli.Get(ctx, fmt.Sprintf("%d", leaseID), clientv3.WithPrefix())
if err != nil {
lc.lg.Warn(
@ -285,42 +236,3 @@ func (lc *leaseChecker) hasKeysAttachedToLeaseExpired(ctx context.Context, lease
}
return len(resp.Kvs) == 0, nil
}
// compositeChecker implements a checker that runs a slice of Checkers concurrently.
type compositeChecker struct{ checkers []Checker }
func newCompositeChecker(checkers []Checker) Checker {
return &compositeChecker{checkers}
}
func (cchecker *compositeChecker) Check() error {
errc := make(chan error)
for _, c := range cchecker.checkers {
go func(chk Checker) { errc <- chk.Check() }(c)
}
var errs []error
for range cchecker.checkers {
if err := <-errc; err != nil {
errs = append(errs, err)
}
}
return errsToError(errs)
}
type runnerChecker struct {
errc chan error
}
func (rc *runnerChecker) Check() error {
select {
case err := <-rc.errc:
return err
default:
return nil
}
}
type noChecker struct{}
func newNoChecker() Checker { return &noChecker{} }
func (nc *noChecker) Check() error { return nil }

View File

@ -0,0 +1,24 @@
// Copyright 2018 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 tester
import "github.com/coreos/etcd/functional/rpcpb"
type noCheck struct{}
func newNoChecker() Checker { return &noCheck{} }
func (nc *noCheck) Type() rpcpb.Checker { return rpcpb.Checker_NO_CHECK }
func (nc *noCheck) EtcdClientEndpoints() []string { return nil }
func (nc *noCheck) Check() error { return nil }

View File

@ -0,0 +1,48 @@
// Copyright 2018 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 tester
import "github.com/coreos/etcd/functional/rpcpb"
type runnerChecker struct {
ctype rpcpb.Checker
etcdClientEndpoint string
errc chan error
}
func newRunnerChecker(ep string, errc chan error) Checker {
return &runnerChecker{
ctype: rpcpb.Checker_RUNNER,
etcdClientEndpoint: ep,
errc: errc,
}
}
func (rc *runnerChecker) Type() rpcpb.Checker {
return rc.ctype
}
func (rc *runnerChecker) EtcdClientEndpoints() []string {
return []string{rc.etcdClientEndpoint}
}
func (rc *runnerChecker) Check() error {
select {
case err := <-rc.errc:
return err
default:
return nil
}
}

View File

@ -52,11 +52,11 @@ type Cluster struct {
Members []*rpcpb.Member `yaml:"agent-configs"`
Tester *rpcpb.Tester `yaml:"tester-config"`
failures []Failure
cases []Case
rateLimiter *rate.Limiter
stresser Stresser
checker Checker
checkers []Checker
currentRevision int64
rd int
@ -80,7 +80,7 @@ func NewCluster(lg *zap.Logger, fpath string) (*Cluster, error) {
clus.agentClients = make([]rpcpb.TransportClient, len(clus.Members))
clus.agentStreams = make([]rpcpb.Transport_TransportClient, len(clus.Members))
clus.agentRequests = make([]*rpcpb.Request, len(clus.Members))
clus.failures = make([]Failure, 0)
clus.cases = make([]Case, 0)
for i, ap := range clus.Members {
var err error
@ -111,18 +111,27 @@ func NewCluster(lg *zap.Logger, fpath string) (*Cluster, error) {
}
go clus.serveTesterServer()
clus.updateFailures()
clus.updateCases()
clus.rateLimiter = rate.NewLimiter(
rate.Limit(int(clus.Tester.StressQPS)),
int(clus.Tester.StressQPS),
)
clus.updateStresserChecker()
clus.setStresserChecker()
return clus, nil
}
// EtcdClientEndpoints returns all etcd client endpoints.
func (clus *Cluster) EtcdClientEndpoints() (css []string) {
css = make([]string, len(clus.Members))
for i := range clus.Members {
css[i] = clus.Members[i].EtcdClientEndpoint
}
return css
}
func (clus *Cluster) serveTesterServer() {
clus.lg.Info(
"started tester HTTP server",
@ -139,124 +148,127 @@ func (clus *Cluster) serveTesterServer() {
}
}
func (clus *Cluster) updateFailures() {
for _, cs := range clus.Tester.FailureCases {
func (clus *Cluster) updateCases() {
for _, cs := range clus.Tester.Cases {
switch cs {
case "SIGTERM_ONE_FOLLOWER":
clus.failures = append(clus.failures,
new_FailureCase_SIGTERM_ONE_FOLLOWER(clus))
clus.cases = append(clus.cases,
new_Case_SIGTERM_ONE_FOLLOWER(clus))
case "SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus))
clus.cases = append(clus.cases,
new_Case_SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus))
case "SIGTERM_LEADER":
clus.failures = append(clus.failures,
new_FailureCase_SIGTERM_LEADER(clus))
clus.cases = append(clus.cases,
new_Case_SIGTERM_LEADER(clus))
case "SIGTERM_LEADER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_SIGTERM_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus))
clus.cases = append(clus.cases,
new_Case_SIGTERM_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus))
case "SIGTERM_QUORUM":
clus.failures = append(clus.failures,
new_FailureCase_SIGTERM_QUORUM(clus))
clus.cases = append(clus.cases,
new_Case_SIGTERM_QUORUM(clus))
case "SIGTERM_ALL":
clus.failures = append(clus.failures,
new_FailureCase_SIGTERM_ALL(clus))
clus.cases = append(clus.cases,
new_Case_SIGTERM_ALL(clus))
case "SIGQUIT_AND_REMOVE_ONE_FOLLOWER":
clus.failures = append(clus.failures,
new_FailureCase_SIGQUIT_AND_REMOVE_ONE_FOLLOWER(clus))
clus.cases = append(clus.cases,
new_Case_SIGQUIT_AND_REMOVE_ONE_FOLLOWER(clus))
case "SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus))
clus.cases = append(clus.cases,
new_Case_SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus))
case "SIGQUIT_AND_REMOVE_LEADER":
clus.failures = append(clus.failures,
new_FailureCase_SIGQUIT_AND_REMOVE_LEADER(clus))
clus.cases = append(clus.cases,
new_Case_SIGQUIT_AND_REMOVE_LEADER(clus))
case "SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus))
clus.cases = append(clus.cases,
new_Case_SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus))
case "SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH":
clus.cases = append(clus.cases,
new_Case_SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH(clus))
case "BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER":
clus.failures = append(clus.failures,
new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER(clus))
clus.cases = append(clus.cases,
new_Case_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER(clus))
case "BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT())
clus.cases = append(clus.cases,
new_Case_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT())
case "BLACKHOLE_PEER_PORT_TX_RX_LEADER":
clus.failures = append(clus.failures,
new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER(clus))
clus.cases = append(clus.cases,
new_Case_BLACKHOLE_PEER_PORT_TX_RX_LEADER(clus))
case "BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT())
clus.cases = append(clus.cases,
new_Case_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT())
case "BLACKHOLE_PEER_PORT_TX_RX_QUORUM":
clus.failures = append(clus.failures,
new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_QUORUM(clus))
clus.cases = append(clus.cases,
new_Case_BLACKHOLE_PEER_PORT_TX_RX_QUORUM(clus))
case "BLACKHOLE_PEER_PORT_TX_RX_ALL":
clus.failures = append(clus.failures,
new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ALL(clus))
clus.cases = append(clus.cases,
new_Case_BLACKHOLE_PEER_PORT_TX_RX_ALL(clus))
case "DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER(clus, false))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER(clus, false))
case "RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER(clus, true))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER(clus, true))
case "DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus, false))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus, false))
case "RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus, true))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus, true))
case "DELAY_PEER_PORT_TX_RX_LEADER":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_LEADER(clus, false))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_LEADER(clus, false))
case "RANDOM_DELAY_PEER_PORT_TX_RX_LEADER":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_LEADER(clus, true))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_LEADER(clus, true))
case "DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus, false))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus, false))
case "RANDOM_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus, true))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus, true))
case "DELAY_PEER_PORT_TX_RX_QUORUM":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_QUORUM(clus, false))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_QUORUM(clus, false))
case "RANDOM_DELAY_PEER_PORT_TX_RX_QUORUM":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_QUORUM(clus, true))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_QUORUM(clus, true))
case "DELAY_PEER_PORT_TX_RX_ALL":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_ALL(clus, false))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_ALL(clus, false))
case "RANDOM_DELAY_PEER_PORT_TX_RX_ALL":
clus.failures = append(clus.failures,
new_FailureCase_DELAY_PEER_PORT_TX_RX_ALL(clus, true))
clus.cases = append(clus.cases,
new_Case_DELAY_PEER_PORT_TX_RX_ALL(clus, true))
case "NO_FAIL_WITH_STRESS":
clus.failures = append(clus.failures,
new_FailureCase_NO_FAIL_WITH_STRESS(clus))
clus.cases = append(clus.cases,
new_Case_NO_FAIL_WITH_STRESS(clus))
case "NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS":
clus.failures = append(clus.failures,
new_FailureCase_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS(clus))
clus.cases = append(clus.cases,
new_Case_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS(clus))
case "EXTERNAL":
clus.failures = append(clus.failures,
new_FailureCase_EXTERNAL(clus.Tester.ExternalExecPath))
clus.cases = append(clus.cases,
new_Case_EXTERNAL(clus.Tester.ExternalExecPath))
case "FAILPOINTS":
fpFailures, fperr := failpointFailures(clus)
if len(fpFailures) == 0 {
clus.lg.Info("no failpoints found!", zap.Error(fperr))
}
clus.failures = append(clus.failures,
clus.cases = append(clus.cases,
fpFailures...)
}
}
}
func (clus *Cluster) failureStrings() (fs []string) {
fs = make([]string, len(clus.failures))
for i := range clus.failures {
fs[i] = clus.failures[i].Desc()
func (clus *Cluster) listCases() (css []string) {
css = make([]string, len(clus.cases))
for i := range clus.cases {
css[i] = clus.cases[i].Desc()
}
return fs
return css
}
// UpdateDelayLatencyMs updates delay latency with random value
@ -271,26 +283,49 @@ func (clus *Cluster) UpdateDelayLatencyMs() {
}
}
func (clus *Cluster) updateStresserChecker() {
cs := &compositeStresser{}
func (clus *Cluster) setStresserChecker() {
css := &compositeStresser{}
lss := []*leaseStresser{}
rss := []*runnerStresser{}
for _, m := range clus.Members {
cs.stressers = append(cs.stressers, newStresser(clus, m))
}
clus.stresser = cs
if clus.Tester.ConsistencyCheck {
clus.checker = newHashChecker(clus.lg, hashAndRevGetter(clus))
if schk := cs.Checker(); schk != nil {
clus.checker = newCompositeChecker([]Checker{clus.checker, schk})
sss := newStresser(clus, m)
css.stressers = append(css.stressers, &compositeStresser{sss})
for _, s := range sss {
if v, ok := s.(*leaseStresser); ok {
lss = append(lss, v)
clus.lg.Info("added lease stresser", zap.String("endpoint", m.EtcdClientEndpoint))
}
if v, ok := s.(*runnerStresser); ok {
rss = append(rss, v)
clus.lg.Info("added lease stresser", zap.String("endpoint", m.EtcdClientEndpoint))
}
}
} else {
clus.checker = newNoChecker()
}
clus.stresser = css
for _, cs := range clus.Tester.Checkers {
switch cs {
case "KV_HASH":
clus.checkers = append(clus.checkers, newKVHashChecker(clus))
case "LEASE_EXPIRE":
for _, ls := range lss {
clus.checkers = append(clus.checkers, newLeaseExpireChecker(ls))
}
case "RUNNER":
for _, rs := range rss {
clus.checkers = append(clus.checkers, newRunnerChecker(rs.etcdClientEndpoint, rs.errc))
}
case "NO_CHECK":
clus.checkers = append(clus.checkers, newNoChecker())
}
}
clus.lg.Info("updated stressers")
}
func (clus *Cluster) checkConsistency() (err error) {
func (clus *Cluster) runCheckers(exceptions ...rpcpb.Checker) (err error) {
defer func() {
if err != nil {
return
@ -304,23 +339,37 @@ func (clus *Cluster) checkConsistency() (err error) {
}
}()
if err = clus.checker.Check(); err != nil {
exs := make(map[rpcpb.Checker]struct{})
for _, e := range exceptions {
exs[e] = struct{}{}
}
for _, chk := range clus.checkers {
clus.lg.Warn(
"consistency check FAIL",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
"consistency check START",
zap.String("checker", chk.Type().String()),
zap.Strings("client-endpoints", chk.EtcdClientEndpoints()),
)
err = chk.Check()
clus.lg.Warn(
"consistency check END",
zap.String("checker", chk.Type().String()),
zap.Strings("client-endpoints", chk.EtcdClientEndpoints()),
zap.Error(err),
)
return err
if err != nil {
_, ok := exs[chk.Type()]
if !ok {
return err
}
clus.lg.Warn(
"consistency check SKIP FAIL",
zap.String("checker", chk.Type().String()),
zap.Strings("client-endpoints", chk.EtcdClientEndpoints()),
zap.Error(err),
)
}
}
clus.lg.Info(
"consistency check ALL PASS",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.String("desc", clus.failures[clus.cs].Desc()),
)
return err
return nil
}
// Send_INITIAL_START_ETCD bootstraps etcd cluster the very first time.
@ -390,6 +439,11 @@ func (clus *Cluster) broadcast(op rpcpb.Operation) error {
}
func (clus *Cluster) sendOp(idx int, op rpcpb.Operation) error {
_, err := clus.sendOpWithResp(idx, op)
return err
}
func (clus *Cluster) sendOpWithResp(idx int, op rpcpb.Operation) (*rpcpb.Response, error) {
// maintain the initial member object
// throughout the test time
clus.agentRequests[idx] = &rpcpb.Request{
@ -406,7 +460,7 @@ func (clus *Cluster) sendOp(idx int, op rpcpb.Operation) error {
zap.Error(err),
)
if err != nil {
return err
return nil, err
}
resp, err := clus.agentStreams[idx].Recv()
@ -428,18 +482,18 @@ func (clus *Cluster) sendOp(idx int, op rpcpb.Operation) error {
)
}
if err != nil {
return err
return nil, err
}
if !resp.Success {
return errors.New(resp.Status)
return nil, errors.New(resp.Status)
}
m, secure := clus.Members[idx], false
for _, cu := range m.Etcd.AdvertiseClientURLs {
u, err := url.Parse(cu)
if err != nil {
return err
return nil, err
}
if u.Scheme == "https" { // TODO: handle unix
secure = true
@ -455,16 +509,16 @@ func (clus *Cluster) sendOp(idx int, op rpcpb.Operation) error {
"client",
)
if err = fileutil.TouchDirAll(dirClient); err != nil {
return err
return nil, err
}
clientCertData := []byte(resp.Member.ClientCertData)
if len(clientCertData) == 0 {
return fmt.Errorf("got empty client cert from %q", m.EtcdClientEndpoint)
return nil, fmt.Errorf("got empty client cert from %q", m.EtcdClientEndpoint)
}
clientCertPath := filepath.Join(dirClient, "cert.pem")
if err = ioutil.WriteFile(clientCertPath, clientCertData, 0644); err != nil { // overwrite if exists
return err
return nil, err
}
resp.Member.ClientCertPath = clientCertPath
clus.lg.Info(
@ -474,11 +528,11 @@ func (clus *Cluster) sendOp(idx int, op rpcpb.Operation) error {
clientKeyData := []byte(resp.Member.ClientKeyData)
if len(clientKeyData) == 0 {
return fmt.Errorf("got empty client key from %q", m.EtcdClientEndpoint)
return nil, fmt.Errorf("got empty client key from %q", m.EtcdClientEndpoint)
}
clientKeyPath := filepath.Join(dirClient, "key.pem")
if err = ioutil.WriteFile(clientKeyPath, clientKeyData, 0644); err != nil { // overwrite if exists
return err
return nil, err
}
resp.Member.ClientKeyPath = clientKeyPath
clus.lg.Info(
@ -491,7 +545,7 @@ func (clus *Cluster) sendOp(idx int, op rpcpb.Operation) error {
// TODO: disable this when auto TLS is deprecated
clientTrustedCAPath := filepath.Join(dirClient, "ca.pem")
if err = ioutil.WriteFile(clientTrustedCAPath, clientTrustedCAData, 0644); err != nil { // overwrite if exists
return err
return nil, err
}
resp.Member.ClientTrustedCAPath = clientTrustedCAPath
clus.lg.Info(
@ -504,7 +558,8 @@ func (clus *Cluster) sendOp(idx int, op rpcpb.Operation) error {
clus.Members[idx] = resp.Member
}
return nil
return resp, nil
}
// Send_SIGQUIT_ETCD_AND_REMOVE_DATA_AND_STOP_AGENT terminates all tester connections to agents and etcd servers.
@ -690,14 +745,14 @@ func (clus *Cluster) defrag() error {
"defrag ALL PASS",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
)
return nil
}
// GetFailureDelayDuration computes failure delay duration.
func (clus *Cluster) GetFailureDelayDuration() time.Duration {
return time.Duration(clus.Tester.FailureDelayMs) * time.Millisecond
// GetCaseDelayDuration computes failure delay duration.
func (clus *Cluster) GetCaseDelayDuration() time.Duration {
return time.Duration(clus.Tester.CaseDelayMs) * time.Millisecond
}
// Report reports the number of modified keys.

View File

@ -320,8 +320,8 @@ func read(lg *zap.Logger, fpath string) (*Cluster, error) {
}
}
if len(clus.Tester.FailureCases) == 0 {
return nil, errors.New("FailureCases not found")
if len(clus.Tester.Cases) == 0 {
return nil, errors.New("Cases not found")
}
if clus.Tester.DelayLatencyMs <= clus.Tester.DelayLatencyMsRv*5 {
return nil, fmt.Errorf("delay latency %d ms must be greater than 5x of delay latency random variable %d ms", clus.Tester.DelayLatencyMs, clus.Tester.DelayLatencyMsRv)
@ -330,15 +330,20 @@ func read(lg *zap.Logger, fpath string) (*Cluster, error) {
clus.Tester.UpdatedDelayLatencyMs = clus.Tester.DelayLatencyMs
}
for _, v := range clus.Tester.FailureCases {
if _, ok := rpcpb.FailureCase_value[v]; !ok {
return nil, fmt.Errorf("%q is not defined in 'rpcpb.FailureCase_value'", v)
for _, v := range clus.Tester.Cases {
if _, ok := rpcpb.Case_value[v]; !ok {
return nil, fmt.Errorf("%q is not defined in 'rpcpb.Case_value'", v)
}
}
for _, v := range clus.Tester.StressTypes {
if _, ok := rpcpb.StressType_value[v]; !ok {
return nil, fmt.Errorf("StressType is unknown; got %q", v)
for _, v := range clus.Tester.Stressers {
if _, ok := rpcpb.Stresser_value[v]; !ok {
return nil, fmt.Errorf("Stresser is unknown; got %q", v)
}
}
for _, v := range clus.Tester.Checkers {
if _, ok := rpcpb.Checker_value[v]; !ok {
return nil, fmt.Errorf("Checker is unknown; got %q", v)
}
}

View File

@ -51,7 +51,7 @@ func (clus *Cluster) Run() {
"round FAIL",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.Error(err),
)
if clus.cleanup() != nil {
@ -75,7 +75,7 @@ func (clus *Cluster) Run() {
"compact START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.Duration("timeout", timeout),
)
if err := clus.compact(revToCompact, timeout); err != nil {
@ -83,7 +83,7 @@ func (clus *Cluster) Run() {
"compact FAIL",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.Error(err),
)
if err = clus.cleanup(); err != nil {
@ -91,7 +91,7 @@ func (clus *Cluster) Run() {
"cleanup FAIL",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.Error(err),
)
return
@ -111,13 +111,13 @@ func (clus *Cluster) Run() {
"functional-tester PASS",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
)
}
func (clus *Cluster) doRound() error {
if clus.Tester.FailureShuffle {
clus.shuffleFailures()
if clus.Tester.CaseShuffle {
clus.shuffleCases()
}
roundNow := time.Now()
@ -125,10 +125,10 @@ func (clus *Cluster) doRound() error {
"round START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Strings("failures", clus.failureStrings()),
zap.Int("case-total", len(clus.cases)),
zap.Strings("cases", clus.listCases()),
)
for i, fa := range clus.failures {
for i, fa := range clus.cases {
clus.cs = i
caseTotal[fa.Desc()]++
@ -139,7 +139,7 @@ func (clus *Cluster) doRound() error {
"case START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
)
@ -149,13 +149,13 @@ func (clus *Cluster) doRound() error {
}
stressStarted := false
fcase := fa.FailureCase()
if fcase != rpcpb.FailureCase_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS {
fcase := fa.TestCase()
if fcase != rpcpb.Case_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS {
clus.lg.Info(
"stress START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
)
if err := clus.stresser.Stress(); err != nil {
@ -168,7 +168,7 @@ func (clus *Cluster) doRound() error {
"inject START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
)
if err := fa.Inject(clus); err != nil {
@ -182,7 +182,7 @@ func (clus *Cluster) doRound() error {
"recover START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
)
if err := fa.Recover(clus); err != nil {
@ -194,11 +194,11 @@ func (clus *Cluster) doRound() error {
"stress PAUSE",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
)
ems := clus.stresser.Pause()
if fcase == rpcpb.FailureCase_NO_FAIL_WITH_STRESS && len(ems) > 0 {
if fcase == rpcpb.Case_NO_FAIL_WITH_STRESS && len(ems) > 0 {
ess := make([]string, 0, len(ems))
cnt := 0
for k, v := range ems {
@ -223,29 +223,35 @@ func (clus *Cluster) doRound() error {
"health check START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
)
if err := clus.WaitHealth(); err != nil {
return fmt.Errorf("wait full health error: %v", err)
}
checkerFailExceptions := []rpcpb.Checker{}
switch fcase {
case rpcpb.Case_SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH:
// TODO: restore from snapshot
checkerFailExceptions = append(checkerFailExceptions, rpcpb.Checker_LEASE_EXPIRE)
}
clus.lg.Info(
"consistency check START",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
)
if err := clus.checkConsistency(); err != nil {
if err := clus.runCheckers(checkerFailExceptions...); err != nil {
return fmt.Errorf("consistency check error (%v)", err)
}
clus.lg.Info(
"case PASS",
"consistency check PASS",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.String("desc", fa.Desc()),
zap.Duration("took", time.Since(caseNow)),
)
@ -254,8 +260,8 @@ func (clus *Cluster) doRound() error {
clus.lg.Info(
"round ALL PASS",
zap.Int("round", clus.rd),
zap.Strings("failures", clus.failureStrings()),
zap.Int("case-total", len(clus.failures)),
zap.Strings("cases", clus.listCases()),
zap.Int("case-total", len(clus.cases)),
zap.Duration("took", time.Since(roundNow)),
)
return nil
@ -314,7 +320,7 @@ func (clus *Cluster) failed() {
"functional-tester FAIL",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
)
clus.Send_SIGQUIT_ETCD_AND_REMOVE_DATA_AND_STOP_AGENT()
@ -322,14 +328,14 @@ func (clus *Cluster) failed() {
}
func (clus *Cluster) cleanup() error {
if clus.Tester.ExitOnFailure {
if clus.Tester.ExitOnCaseFail {
defer clus.failed()
}
roundFailedTotalCounter.Inc()
desc := "compact/defrag"
if clus.cs != -1 {
desc = clus.failures[clus.cs].Desc()
desc = clus.cases[clus.cs].Desc()
}
caseFailedTotalCounter.WithLabelValues(desc).Inc()
@ -337,7 +343,7 @@ func (clus *Cluster) cleanup() error {
"closing stressers before archiving failure data",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
)
clus.stresser.Close()
@ -346,7 +352,7 @@ func (clus *Cluster) cleanup() error {
"cleanup FAIL",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.Error(err),
)
return err
@ -356,12 +362,12 @@ func (clus *Cluster) cleanup() error {
"restart FAIL",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.Int("case-total", len(clus.failures)),
zap.Int("case-total", len(clus.cases)),
zap.Error(err),
)
return err
}
clus.updateStresserChecker()
clus.setStresserChecker()
return nil
}

View File

@ -21,17 +21,17 @@ import (
"go.uber.org/zap"
)
func (clus *Cluster) shuffleFailures() {
func (clus *Cluster) shuffleCases() {
rand.Seed(time.Now().UnixNano())
offset := rand.Intn(1000)
n := len(clus.failures)
n := len(clus.cases)
cp := coprime(n)
fs := make([]Failure, n)
css := make([]Case, n)
for i := 0; i < n; i++ {
fs[i] = clus.failures[(cp*i+offset)%n]
css[i] = clus.cases[(cp*i+offset)%n]
}
clus.failures = fs
clus.cases = css
clus.lg.Info("shuffled test failure cases", zap.Int("total", n))
}

View File

@ -31,15 +31,15 @@ func Test_read(t *testing.T) {
EtcdExecPath: "./bin/etcd",
AgentAddr: "127.0.0.1:19027",
FailpointHTTPAddr: "http://127.0.0.1:7381",
BaseDir: "/tmp/etcd-agent-data-1",
EtcdLogPath: "/tmp/etcd-agent-data-1/current-etcd.log",
BaseDir: "/tmp/etcd-functional-1",
EtcdLogPath: "/tmp/etcd-functional-1/etcd.log",
EtcdClientProxy: false,
EtcdPeerProxy: true,
EtcdClientEndpoint: "127.0.0.1:1379",
Etcd: &rpcpb.Etcd{
Name: "s1",
DataDir: "/tmp/etcd-agent-data-1/etcd.data",
WALDir: "/tmp/etcd-agent-data-1/etcd.data/member/wal",
DataDir: "/tmp/etcd-functional-1/etcd.data",
WALDir: "/tmp/etcd-functional-1/etcd.data/member/wal",
HeartbeatIntervalMs: 100,
ElectionTimeoutMs: 1000,
ListenClientURLs: []string{"https://127.0.0.1:1379"},
@ -64,20 +64,33 @@ func Test_read(t *testing.T) {
PreVote: true,
InitialCorruptCheck: true,
},
ClientCertData: "",
ClientCertPath: "",
ClientKeyData: "",
ClientKeyPath: "",
ClientTrustedCAData: "",
ClientTrustedCAPath: "",
PeerCertData: "",
PeerCertPath: "",
PeerKeyData: "",
PeerKeyPath: "",
PeerTrustedCAData: "",
PeerTrustedCAPath: "",
SnapshotPath: "/tmp/etcd-functional-1.snapshot.db",
},
{
EtcdExecPath: "./bin/etcd",
AgentAddr: "127.0.0.1:29027",
FailpointHTTPAddr: "http://127.0.0.1:7382",
BaseDir: "/tmp/etcd-agent-data-2",
EtcdLogPath: "/tmp/etcd-agent-data-2/current-etcd.log",
BaseDir: "/tmp/etcd-functional-2",
EtcdLogPath: "/tmp/etcd-functional-2/etcd.log",
EtcdClientProxy: false,
EtcdPeerProxy: true,
EtcdClientEndpoint: "127.0.0.1:2379",
Etcd: &rpcpb.Etcd{
Name: "s2",
DataDir: "/tmp/etcd-agent-data-2/etcd.data",
WALDir: "/tmp/etcd-agent-data-2/etcd.data/member/wal",
DataDir: "/tmp/etcd-functional-2/etcd.data",
WALDir: "/tmp/etcd-functional-2/etcd.data/member/wal",
HeartbeatIntervalMs: 100,
ElectionTimeoutMs: 1000,
ListenClientURLs: []string{"https://127.0.0.1:2379"},
@ -102,20 +115,33 @@ func Test_read(t *testing.T) {
PreVote: true,
InitialCorruptCheck: true,
},
ClientCertData: "",
ClientCertPath: "",
ClientKeyData: "",
ClientKeyPath: "",
ClientTrustedCAData: "",
ClientTrustedCAPath: "",
PeerCertData: "",
PeerCertPath: "",
PeerKeyData: "",
PeerKeyPath: "",
PeerTrustedCAData: "",
PeerTrustedCAPath: "",
SnapshotPath: "/tmp/etcd-functional-2.snapshot.db",
},
{
EtcdExecPath: "./bin/etcd",
AgentAddr: "127.0.0.1:39027",
FailpointHTTPAddr: "http://127.0.0.1:7383",
BaseDir: "/tmp/etcd-agent-data-3",
EtcdLogPath: "/tmp/etcd-agent-data-3/current-etcd.log",
BaseDir: "/tmp/etcd-functional-3",
EtcdLogPath: "/tmp/etcd-functional-3/etcd.log",
EtcdClientProxy: false,
EtcdPeerProxy: true,
EtcdClientEndpoint: "127.0.0.1:3379",
Etcd: &rpcpb.Etcd{
Name: "s3",
DataDir: "/tmp/etcd-agent-data-3/etcd.data",
WALDir: "/tmp/etcd-agent-data-3/etcd.data/member/wal",
DataDir: "/tmp/etcd-functional-3/etcd.data",
WALDir: "/tmp/etcd-functional-3/etcd.data/member/wal",
HeartbeatIntervalMs: 100,
ElectionTimeoutMs: 1000,
ListenClientURLs: []string{"https://127.0.0.1:3379"},
@ -140,6 +166,19 @@ func Test_read(t *testing.T) {
PreVote: true,
InitialCorruptCheck: true,
},
ClientCertData: "",
ClientCertPath: "",
ClientKeyData: "",
ClientKeyPath: "",
ClientTrustedCAData: "",
ClientTrustedCAPath: "",
PeerCertData: "",
PeerCertPath: "",
PeerKeyData: "",
PeerKeyPath: "",
PeerTrustedCAData: "",
PeerTrustedCAPath: "",
SnapshotPath: "/tmp/etcd-functional-3.snapshot.db",
},
},
Tester: &rpcpb.Tester{
@ -150,12 +189,11 @@ func Test_read(t *testing.T) {
DelayLatencyMsRv: 500,
UpdatedDelayLatencyMs: 5000,
RoundLimit: 1,
ExitOnFailure: true,
ConsistencyCheck: true,
ExitOnCaseFail: true,
EnablePprof: true,
FailureDelayMs: 7000,
FailureShuffle: true,
FailureCases: []string{
CaseDelayMs: 7000,
CaseShuffle: true,
Cases: []string{
"SIGTERM_ONE_FOLLOWER",
"SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT",
"SIGTERM_LEADER",
@ -166,6 +204,7 @@ func Test_read(t *testing.T) {
"SIGQUIT_AND_REMOVE_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT",
// "SIGQUIT_AND_REMOVE_LEADER",
// "SIGQUIT_AND_REMOVE_LEADER_UNTIL_TRIGGER_SNAPSHOT",
"SIGQUIT_AND_REMOVE_QUORUM_AND_RESTORE_LEADER_SNAPSHOT_FROM_SCRATCH",
"BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER",
"BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT",
"BLACKHOLE_PEER_PORT_TX_RX_LEADER",
@ -190,7 +229,8 @@ func Test_read(t *testing.T) {
FailpointCommands: []string{`panic("etcd-tester")`},
RunnerExecPath: "./bin/etcd-runner",
ExternalExecPath: "",
StressTypes: []string{"KV", "LEASE"},
Stressers: []string{"KV", "LEASE"},
Checkers: []string{"KV_HASH", "LEASE_EXPIRE"},
StressKeySize: 100,
StressKeySizeLarge: 32769,
StressKeySuffixRange: 250000,
@ -219,17 +259,17 @@ func Test_read(t *testing.T) {
cfg.lg = logger
cfg.updateFailures()
fs1 := cfg.failureStrings()
cfg.updateCases()
fs1 := cfg.listCases()
cfg.shuffleFailures()
fs2 := cfg.failureStrings()
cfg.shuffleCases()
fs2 := cfg.listCases()
if reflect.DeepEqual(fs1, fs2) {
t.Fatalf("expected shuffled failure cases, got %q", fs2)
}
cfg.shuffleFailures()
fs3 := cfg.failureStrings()
cfg.shuffleCases()
fs3 := cfg.listCases()
if reflect.DeepEqual(fs2, fs3) {
t.Fatalf("expected reshuffled failure cases from %q, got %q", fs2, fs3)
}

View File

@ -1,317 +0,0 @@
// Copyright 2018 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 tester
import (
"fmt"
"math/rand"
"time"
"github.com/coreos/etcd/functional/rpcpb"
"go.uber.org/zap"
)
// Failure defines failure injection interface.
// To add a fail case:
// 1. implement "Failure" interface
// 2. define fail case name in "rpcpb.FailureCase"
type Failure interface {
// Inject injeccts the failure into the testing cluster at the given
// round. When calling the function, the cluster should be in health.
Inject(clus *Cluster) error
// Recover recovers the injected failure caused by the injection of the
// given round and wait for the recovery of the testing cluster.
Recover(clus *Cluster) error
// Desc returns a description of the failure
Desc() string
// FailureCase returns "rpcpb.FailureCase" enum type.
FailureCase() rpcpb.FailureCase
}
type injectMemberFunc func(*Cluster, int) error
type recoverMemberFunc func(*Cluster, int) error
type failureByFunc struct {
desc string
failureCase rpcpb.FailureCase
injectMember injectMemberFunc
recoverMember recoverMemberFunc
}
func (f *failureByFunc) Desc() string {
if f.desc != "" {
return f.desc
}
return f.failureCase.String()
}
func (f *failureByFunc) FailureCase() rpcpb.FailureCase {
return f.failureCase
}
type failureFollower struct {
failureByFunc
last int
lead int
}
func (f *failureFollower) updateIndex(clus *Cluster) error {
idx, err := clus.GetLeader()
if err != nil {
return err
}
f.lead = idx
n := len(clus.Members)
if f.last == -1 { // first run
f.last = clus.rd % n
if f.last == f.lead {
f.last = (f.last + 1) % n
}
} else {
f.last = (f.last + 1) % n
if f.last == f.lead {
f.last = (f.last + 1) % n
}
}
return nil
}
func (f *failureFollower) Inject(clus *Cluster) error {
if err := f.updateIndex(clus); err != nil {
return err
}
return f.injectMember(clus, f.last)
}
func (f *failureFollower) Recover(clus *Cluster) error {
return f.recoverMember(clus, f.last)
}
func (f *failureFollower) Desc() string {
if f.desc != "" {
return f.desc
}
return f.failureCase.String()
}
func (f *failureFollower) FailureCase() rpcpb.FailureCase {
return f.failureCase
}
type failureLeader struct {
failureByFunc
last int
lead int
}
func (f *failureLeader) updateIndex(clus *Cluster) error {
idx, err := clus.GetLeader()
if err != nil {
return err
}
f.lead = idx
f.last = idx
return nil
}
func (f *failureLeader) Inject(clus *Cluster) error {
if err := f.updateIndex(clus); err != nil {
return err
}
return f.injectMember(clus, f.last)
}
func (f *failureLeader) Recover(clus *Cluster) error {
return f.recoverMember(clus, f.last)
}
func (f *failureLeader) FailureCase() rpcpb.FailureCase {
return f.failureCase
}
type failureQuorum failureByFunc
func (f *failureQuorum) Inject(clus *Cluster) error {
for i := range killMap(len(clus.Members), clus.rd) {
if err := f.injectMember(clus, i); err != nil {
return err
}
}
return nil
}
func (f *failureQuorum) Recover(clus *Cluster) error {
for i := range killMap(len(clus.Members), clus.rd) {
if err := f.recoverMember(clus, i); err != nil {
return err
}
}
return nil
}
func (f *failureQuorum) Desc() string {
if f.desc != "" {
return f.desc
}
return f.failureCase.String()
}
func (f *failureQuorum) FailureCase() rpcpb.FailureCase {
return f.failureCase
}
func killMap(size int, seed int) map[int]bool {
m := make(map[int]bool)
r := rand.New(rand.NewSource(int64(seed)))
majority := size/2 + 1
for {
m[r.Intn(size)] = true
if len(m) >= majority {
return m
}
}
}
type failureAll failureByFunc
func (f *failureAll) Inject(clus *Cluster) error {
for i := range clus.Members {
if err := f.injectMember(clus, i); err != nil {
return err
}
}
return nil
}
func (f *failureAll) Recover(clus *Cluster) error {
for i := range clus.Members {
if err := f.recoverMember(clus, i); err != nil {
return err
}
}
return nil
}
func (f *failureAll) Desc() string {
if f.desc != "" {
return f.desc
}
return f.failureCase.String()
}
func (f *failureAll) FailureCase() rpcpb.FailureCase {
return f.failureCase
}
// failureUntilSnapshot injects a failure and waits for a snapshot event
type failureUntilSnapshot struct {
desc string
failureCase rpcpb.FailureCase
Failure
}
// all delay failure cases except the ones failing with latency
// greater than election timeout (trigger leader election and
// cluster keeps operating anyways)
var slowCases = map[rpcpb.FailureCase]bool{
rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER: true,
rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER: true,
rpcpb.FailureCase_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT: true,
rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_QUORUM: true,
rpcpb.FailureCase_RANDOM_DELAY_PEER_PORT_TX_RX_ALL: true,
}
func (f *failureUntilSnapshot) Inject(clus *Cluster) error {
if err := f.Failure.Inject(clus); err != nil {
return err
}
snapshotCount := clus.Members[0].Etcd.SnapshotCount
now := time.Now()
clus.lg.Info(
"trigger snapshot START",
zap.String("desc", f.Desc()),
zap.Int64("etcd-snapshot-count", snapshotCount),
)
// maxRev may fail since failure just injected, retry if failed.
startRev, err := clus.maxRev()
for i := 0; i < 10 && startRev == 0; i++ {
startRev, err = clus.maxRev()
}
if startRev == 0 {
return err
}
lastRev := startRev
// healthy cluster could accept 1000 req/sec at least.
// 3x time to trigger snapshot.
retries := int(snapshotCount) / 1000 * 3
if v, ok := slowCases[f.FailureCase()]; v && ok {
// slow network takes more retries
retries *= 5
}
for i := 0; i < retries; i++ {
lastRev, _ = clus.maxRev()
// If the number of proposals committed is bigger than snapshot count,
// a new snapshot should have been created.
diff := lastRev - startRev
if diff > snapshotCount {
clus.lg.Info(
"trigger snapshot PASS",
zap.Int("retries", i),
zap.String("desc", f.Desc()),
zap.Int64("committed-entries", diff),
zap.Int64("etcd-snapshot-count", snapshotCount),
zap.Int64("last-revision", lastRev),
zap.Duration("took", time.Since(now)),
)
return nil
}
clus.lg.Info(
"trigger snapshot PROGRESS",
zap.Int("retries", i),
zap.Int64("committed-entries", diff),
zap.Int64("etcd-snapshot-count", snapshotCount),
zap.Int64("last-revision", lastRev),
zap.Duration("took", time.Since(now)),
)
time.Sleep(time.Second)
}
return fmt.Errorf("cluster too slow: only %d commits in %d retries", lastRev-startRev, retries)
}
func (f *failureUntilSnapshot) Desc() string {
if f.desc != "" {
return f.desc
}
if f.failureCase.String() != "" {
return f.failureCase.String()
}
return f.Failure.Desc()
}
func (f *failureUntilSnapshot) FailureCase() rpcpb.FailureCase {
return f.failureCase
}

View File

@ -1,101 +0,0 @@
// Copyright 2018 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 tester
import "github.com/coreos/etcd/functional/rpcpb"
func inject_BLACKHOLE_PEER_PORT_TX_RX(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_BLACKHOLE_PEER_PORT_TX_RX)
}
func recover_BLACKHOLE_PEER_PORT_TX_RX(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_UNBLACKHOLE_PEER_PORT_TX_RX)
}
func new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER(clus *Cluster) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
f := &failureFollower{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}
func new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT() Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
f := &failureFollower{ff, -1, -1}
return &failureUntilSnapshot{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
Failure: f,
}
}
func new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER(clus *Cluster) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
f := &failureLeader{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}
func new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT() Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
f := &failureLeader{ff, -1, -1}
return &failureUntilSnapshot{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER_UNTIL_TRIGGER_SNAPSHOT,
Failure: f,
}
}
func new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_QUORUM(clus *Cluster) Failure {
f := &failureQuorum{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_QUORUM,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}
func new_FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ALL(clus *Cluster) Failure {
f := &failureAll{
failureCase: rpcpb.FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ALL,
injectMember: inject_BLACKHOLE_PEER_PORT_TX_RX,
recoverMember: recover_BLACKHOLE_PEER_PORT_TX_RX,
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}

View File

@ -1,99 +0,0 @@
// Copyright 2018 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 tester
import (
"time"
"github.com/coreos/etcd/functional/rpcpb"
"go.uber.org/zap"
)
type failureNoFailWithStress failureByFunc
func (f *failureNoFailWithStress) Inject(clus *Cluster) error {
return nil
}
func (f *failureNoFailWithStress) Recover(clus *Cluster) error {
return nil
}
func (f *failureNoFailWithStress) Desc() string {
if f.desc != "" {
return f.desc
}
return f.failureCase.String()
}
func (f *failureNoFailWithStress) FailureCase() rpcpb.FailureCase {
return f.failureCase
}
func new_FailureCase_NO_FAIL_WITH_STRESS(clus *Cluster) Failure {
f := &failureNoFailWithStress{
failureCase: rpcpb.FailureCase_NO_FAIL_WITH_STRESS,
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}
type failureNoFailWithNoStressForLiveness failureByFunc
func (f *failureNoFailWithNoStressForLiveness) Inject(clus *Cluster) error {
clus.lg.Info(
"extra delay for liveness mode with no stresser",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.String("desc", f.Desc()),
)
time.Sleep(clus.GetFailureDelayDuration())
clus.lg.Info(
"wait health in liveness mode",
zap.Int("round", clus.rd),
zap.Int("case", clus.cs),
zap.String("desc", f.Desc()),
)
return clus.WaitHealth()
}
func (f *failureNoFailWithNoStressForLiveness) Recover(clus *Cluster) error {
return nil
}
func (f *failureNoFailWithNoStressForLiveness) Desc() string {
if f.desc != "" {
return f.desc
}
return f.failureCase.String()
}
func (f *failureNoFailWithNoStressForLiveness) FailureCase() rpcpb.FailureCase {
return f.failureCase
}
func new_FailureCase_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS(clus *Cluster) Failure {
f := &failureNoFailWithNoStressForLiveness{
failureCase: rpcpb.FailureCase_NO_FAIL_WITH_NO_STRESS_FOR_LIVENESS,
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}

View File

@ -1,89 +0,0 @@
// Copyright 2018 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 tester
import "github.com/coreos/etcd/functional/rpcpb"
func inject_SIGTERM_ETCD(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_SIGTERM_ETCD)
}
func recover_SIGTERM_ETCD(clus *Cluster, idx int) error {
return clus.sendOp(idx, rpcpb.Operation_RESTART_ETCD)
}
func new_FailureCase_SIGTERM_ONE_FOLLOWER(clus *Cluster) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_SIGTERM_ONE_FOLLOWER,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
}
f := &failureFollower{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}
func new_FailureCase_SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Failure {
return &failureUntilSnapshot{
failureCase: rpcpb.FailureCase_SIGTERM_ONE_FOLLOWER_UNTIL_TRIGGER_SNAPSHOT,
Failure: new_FailureCase_SIGTERM_ONE_FOLLOWER(clus),
}
}
func new_FailureCase_SIGTERM_LEADER(clus *Cluster) Failure {
ff := failureByFunc{
failureCase: rpcpb.FailureCase_SIGTERM_LEADER,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
}
f := &failureLeader{ff, -1, -1}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}
func new_FailureCase_SIGTERM_LEADER_UNTIL_TRIGGER_SNAPSHOT(clus *Cluster) Failure {
return &failureUntilSnapshot{
failureCase: rpcpb.FailureCase_SIGTERM_LEADER_UNTIL_TRIGGER_SNAPSHOT,
Failure: new_FailureCase_SIGTERM_LEADER(clus),
}
}
func new_FailureCase_SIGTERM_QUORUM(clus *Cluster) Failure {
f := &failureQuorum{
failureCase: rpcpb.FailureCase_SIGTERM_QUORUM,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}
func new_FailureCase_SIGTERM_ALL(clus *Cluster) Failure {
f := &failureAll{
failureCase: rpcpb.FailureCase_SIGTERM_ALL,
injectMember: inject_SIGTERM_ETCD,
recoverMember: recover_SIGTERM_ETCD,
}
return &failureDelay{
Failure: f,
delayDuration: clus.GetFailureDelayDuration(),
}
}

View File

@ -33,14 +33,12 @@ type Stresser interface {
Close() map[string]int
// ModifiedKeys reports the number of keys created and deleted by stresser
ModifiedKeys() int64
// Checker returns an invariant checker for after the stresser is canceled.
Checker() Checker
}
// newStresser creates stresser from a comma separated list of stresser types.
func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
stressers := make([]Stresser, len(clus.Tester.StressTypes))
for i, stype := range clus.Tester.StressTypes {
func newStresser(clus *Cluster, m *rpcpb.Member) (stressers []Stresser) {
stressers = make([]Stresser, len(clus.Tester.Stressers))
for i, stype := range clus.Tester.Stressers {
clus.lg.Info(
"creating stresser",
zap.String("type", stype),
@ -52,7 +50,7 @@ func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
// TODO: Too intensive stressing clients can panic etcd member with
// 'out of memory' error. Put rate limits in server side.
stressers[i] = &keyStresser{
stype: rpcpb.StressType_KV,
stype: rpcpb.Stresser_KV,
lg: clus.lg,
m: m,
keySize: int(clus.Tester.StressKeySize),
@ -66,7 +64,7 @@ func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
case "LEASE":
stressers[i] = &leaseStresser{
stype: rpcpb.StressType_LEASE,
stype: rpcpb.Stresser_LEASE,
lg: clus.lg,
m: m,
numLeases: 10, // TODO: configurable
@ -86,7 +84,8 @@ func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
"--req-rate", fmt.Sprintf("%v", reqRate),
}
stressers[i] = newRunnerStresser(
rpcpb.StressType_ELECTION_RUNNER,
rpcpb.Stresser_ELECTION_RUNNER,
m.EtcdClientEndpoint,
clus.lg,
clus.Tester.RunnerExecPath,
args,
@ -107,7 +106,8 @@ func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
"--req-rate", fmt.Sprintf("%v", reqRate),
}
stressers[i] = newRunnerStresser(
rpcpb.StressType_WATCH_RUNNER,
rpcpb.Stresser_WATCH_RUNNER,
m.EtcdClientEndpoint,
clus.lg,
clus.Tester.RunnerExecPath,
args,
@ -126,7 +126,8 @@ func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
"--req-rate", fmt.Sprintf("%v", reqRate),
}
stressers[i] = newRunnerStresser(
rpcpb.StressType_LOCK_RACER_RUNNER,
rpcpb.Stresser_LOCK_RACER_RUNNER,
m.EtcdClientEndpoint,
clus.lg,
clus.Tester.RunnerExecPath,
args,
@ -141,7 +142,8 @@ func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
"--endpoints", m.EtcdClientEndpoint,
}
stressers[i] = newRunnerStresser(
rpcpb.StressType_LEASE_RUNNER,
rpcpb.Stresser_LEASE_RUNNER,
m.EtcdClientEndpoint,
clus.lg,
clus.Tester.RunnerExecPath,
args,
@ -150,5 +152,5 @@ func newStresser(clus *Cluster, m *rpcpb.Member) Stresser {
)
}
}
return &compositeStresser{stressers}
return stressers
}

View File

@ -74,16 +74,3 @@ func (cs *compositeStresser) ModifiedKeys() (modifiedKey int64) {
}
return modifiedKey
}
func (cs *compositeStresser) Checker() Checker {
var chks []Checker
for _, s := range cs.stressers {
if chk := s.Checker(); chk != nil {
chks = append(chks, chk)
}
}
if len(chks) == 0 {
return nil
}
return newCompositeChecker(chks)
}

View File

@ -35,7 +35,7 @@ import (
)
type keyStresser struct {
stype rpcpb.StressType
stype rpcpb.Stresser
lg *zap.Logger
m *rpcpb.Member
@ -204,8 +204,6 @@ func (s *keyStresser) ModifiedKeys() int64 {
return atomic.LoadInt64(&s.atomicModifiedKeys)
}
func (s *keyStresser) Checker() Checker { return nil }
type stressFunc func(ctx context.Context) (err error, modifiedKeys int64)
type stressEntry struct {

View File

@ -38,7 +38,7 @@ const (
)
type leaseStresser struct {
stype rpcpb.StressType
stype rpcpb.Stresser
lg *zap.Logger
m *rpcpb.Member
@ -485,7 +485,3 @@ func (ls *leaseStresser) Close() map[string]int {
func (ls *leaseStresser) ModifiedKeys() int64 {
return atomic.LoadInt64(&ls.atomicModifiedKey)
}
func (ls *leaseStresser) Checker() Checker {
return &leaseChecker{lg: ls.lg, m: ls.m, ls: ls}
}

View File

@ -27,8 +27,9 @@ import (
)
type runnerStresser struct {
stype rpcpb.StressType
lg *zap.Logger
stype rpcpb.Stresser
etcdClientEndpoint string
lg *zap.Logger
cmd *exec.Cmd
cmdStr string
@ -41,7 +42,8 @@ type runnerStresser struct {
}
func newRunnerStresser(
stype rpcpb.StressType,
stype rpcpb.Stresser,
ep string,
lg *zap.Logger,
cmdStr string,
args []string,
@ -50,13 +52,14 @@ func newRunnerStresser(
) *runnerStresser {
rl.SetLimit(rl.Limit() - rate.Limit(reqRate))
return &runnerStresser{
stype: stype,
cmdStr: cmdStr,
args: args,
rl: rl,
reqRate: reqRate,
errc: make(chan error, 1),
donec: make(chan struct{}),
stype: stype,
etcdClientEndpoint: ep,
cmdStr: cmdStr,
args: args,
rl: rl,
reqRate: reqRate,
errc: make(chan error, 1),
donec: make(chan struct{}),
}
}
@ -115,7 +118,3 @@ func (rs *runnerStresser) Close() map[string]int {
func (rs *runnerStresser) ModifiedKeys() int64 {
return 1
}
func (rs *runnerStresser) Checker() Checker {
return &runnerChecker{rs.errc}
}

2
test
View File

@ -180,7 +180,7 @@ function integration_extra {
function functional_pass {
# Clean up any data and logs from previous runs
rm -rf /tmp/etcd-agent-data-* /tmp/etcd-agent-data-*.backup
rm -rf /tmp/etcd-functional-* /tmp/etcd-functional-*.backup
for a in 1 2 3; do
./bin/etcd-agent --network tcp --address 127.0.0.1:${a}9027 &