diff --git a/tools/functional-tester/rpcpb/etcd_config.go b/tools/functional-tester/rpcpb/etcd_config.go new file mode 100644 index 000000000..452b4dc71 --- /dev/null +++ b/tools/functional-tester/rpcpb/etcd_config.go @@ -0,0 +1,76 @@ +// 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 rpcpb + +import ( + "fmt" + "reflect" + "strings" +) + +var etcdFields = []string{ + "Name", + "DataDir", + "WALDir", + "ListenClientURLs", + "AdvertiseClientURLs", + "ListenPeerURLs", + "InitialAdvertisePeerURLs", + "InitialCluster", + "InitialClusterState", + "InitialClusterToken", + "SnapshotCount", + "QuotaBackendBytes", + "PreVote", + "InitialCorruptCheck", +} + +// Flags returns etcd flags in string slice. +func (cfg *Etcd) Flags() (fs []string) { + tp := reflect.TypeOf(*cfg) + vo := reflect.ValueOf(*cfg) + for _, name := range etcdFields { + field, ok := tp.FieldByName(name) + if !ok { + panic(fmt.Errorf("field %q not found", name)) + } + fv := reflect.Indirect(vo).FieldByName(name) + var sv string + switch fv.Type().Kind() { + case reflect.String: + sv = fv.String() + case reflect.Slice: + n := fv.Len() + sl := make([]string, n) + for i := 0; i < n; i++ { + sl[i] = fv.Index(i).String() + } + sv = strings.Join(sl, ",") + case reflect.Int64: + sv = fmt.Sprintf("%d", fv.Int()) + case reflect.Bool: + sv = fmt.Sprintf("%v", fv.Bool()) + default: + panic(fmt.Errorf("field %q (%v) cannot be parsed", name, fv.Type().Kind())) + } + fname := field.Tag.Get("yaml") + // TODO: remove this + if fname == "initial-corrupt-check" { + fname = "experimental-" + fname + } + fs = append(fs, fmt.Sprintf("--%s=%s", fname, sv)) + } + return fs +} diff --git a/tools/functional-tester/rpcpb/etcd_config_test.go b/tools/functional-tester/rpcpb/etcd_config_test.go new file mode 100644 index 000000000..c60716691 --- /dev/null +++ b/tools/functional-tester/rpcpb/etcd_config_test.go @@ -0,0 +1,59 @@ +// 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 rpcpb + +import ( + "reflect" + "testing" +) + +func TestEtcdFlags(t *testing.T) { + cfg := &Etcd{ + Name: "s1", + DataDir: "/tmp/etcd-agent-data-1/etcd.data", + WALDir: "/tmp/etcd-agent-data-1/etcd.data/member/wal", + ListenClientURLs: []string{"127.0.0.1:1379"}, + AdvertiseClientURLs: []string{"127.0.0.1:13790"}, + ListenPeerURLs: []string{"127.0.0.1:1380"}, + InitialAdvertisePeerURLs: []string{"127.0.0.1:13800"}, + InitialCluster: "s1=127.0.0.1:13800,s2=127.0.0.1:23800,s3=127.0.0.1:33800", + InitialClusterState: "new", + InitialClusterToken: "tkn", + SnapshotCount: 10000, + QuotaBackendBytes: 10740000000, + PreVote: true, + InitialCorruptCheck: true, + } + exp := []string{ + "--name=s1", + "--data-dir=/tmp/etcd-agent-data-1/etcd.data", + "--wal-dir=/tmp/etcd-agent-data-1/etcd.data/member/wal", + "--listen-client-urls=127.0.0.1:1379", + "--advertise-client-urls=127.0.0.1:13790", + "--listen-peer-urls=127.0.0.1:1380", + "--initial-advertise-peer-urls=127.0.0.1:13800", + "--initial-cluster=s1=127.0.0.1:13800,s2=127.0.0.1:23800,s3=127.0.0.1:33800", + "--initial-cluster-state=new", + "--initial-cluster-token=tkn", + "--snapshot-count=10000", + "--quota-backend-bytes=10740000000", + "--pre-vote=true", + "--experimental-initial-corrupt-check=true", + } + fs := cfg.Flags() + if !reflect.DeepEqual(exp, fs) { + t.Fatalf("expected %q, got %q", exp, fs) + } +} diff --git a/tools/functional-tester/rpcpb/member.go b/tools/functional-tester/rpcpb/member.go new file mode 100644 index 000000000..3e617d3c6 --- /dev/null +++ b/tools/functional-tester/rpcpb/member.go @@ -0,0 +1,161 @@ +// 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 rpcpb + +import ( + "context" + "fmt" + "time" + + "github.com/coreos/etcd/clientv3" + pb "github.com/coreos/etcd/etcdserver/etcdserverpb" + + grpc "google.golang.org/grpc" +) + +var dialOpts = []grpc.DialOption{ + grpc.WithInsecure(), + grpc.WithTimeout(5 * time.Second), + grpc.WithBlock(), +} + +// DialEtcdGRPCServer creates a raw gRPC connection to an etcd member. +func (m *Member) DialEtcdGRPCServer() (*grpc.ClientConn, error) { + if m.EtcdClientTLS { + // TODO: support TLS + panic("client TLS not supported yet") + } + return grpc.Dial(m.EtcdClientEndpoint, dialOpts...) +} + +// CreateEtcdClient creates a client from member. +func (m *Member) CreateEtcdClient() (*clientv3.Client, error) { + if m.EtcdClientTLS { + // TODO: support TLS + panic("client TLS not supported yet") + } + return clientv3.New(clientv3.Config{ + Endpoints: []string{m.EtcdClientEndpoint}, + DialTimeout: 5 * time.Second, + }) +} + +// CheckCompact ensures that historical data before given revision has been compacted. +func (m *Member) CheckCompact(rev int64) error { + cli, err := m.CreateEtcdClient() + if err != nil { + return fmt.Errorf("%v (%q)", err, m.EtcdClientEndpoint) + } + defer cli.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + wch := cli.Watch(ctx, "\x00", clientv3.WithFromKey(), clientv3.WithRev(rev-1)) + wr, ok := <-wch + cancel() + + if !ok { + return fmt.Errorf("watch channel terminated (endpoint %q)", m.EtcdClientEndpoint) + } + if wr.CompactRevision != rev { + return fmt.Errorf("got compact revision %v, wanted %v (endpoint %q)", wr.CompactRevision, rev, m.EtcdClientEndpoint) + } + + return nil +} + +// Defrag runs defragmentation on this member. +func (m *Member) Defrag() error { + cli, err := m.CreateEtcdClient() + if err != nil { + return fmt.Errorf("%v (%q)", err, m.EtcdClientEndpoint) + } + defer cli.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + _, err = cli.Defragment(ctx, m.EtcdClientEndpoint) + cancel() + if err != nil { + return err + } + return nil +} + +// RevHash fetches current revision and hash on this member. +func (m *Member) RevHash() (int64, int64, error) { + conn, err := m.DialEtcdGRPCServer() + if err != nil { + return 0, 0, err + } + defer conn.Close() + + mt := pb.NewMaintenanceClient(conn) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + resp, err := mt.Hash(ctx, &pb.HashRequest{}, grpc.FailFast(false)) + cancel() + + if err != nil { + return 0, 0, err + } + + return resp.Header.Revision, int64(resp.Hash), nil +} + +// Rev fetches current revision on this member. +func (m *Member) Rev(ctx context.Context) (int64, error) { + cli, err := m.CreateEtcdClient() + if err != nil { + return 0, fmt.Errorf("%v (%q)", err, m.EtcdClientEndpoint) + } + defer cli.Close() + + resp, err := cli.Status(ctx, m.EtcdClientEndpoint) + if err != nil { + return 0, err + } + return resp.Header.Revision, nil +} + +// IsLeader returns true if this member is the current cluster leader. +func (m *Member) IsLeader() (bool, error) { + cli, err := m.CreateEtcdClient() + if err != nil { + return false, fmt.Errorf("%v (%q)", err, m.EtcdClientEndpoint) + } + defer cli.Close() + + resp, err := cli.Status(context.Background(), m.EtcdClientEndpoint) + if err != nil { + return false, err + } + return resp.Header.MemberId == resp.Leader, nil +} + +// WriteHealthKey writes a health key to this member. +func (m *Member) WriteHealthKey() error { + cli, err := m.CreateEtcdClient() + if err != nil { + return fmt.Errorf("%v (%q)", err, m.EtcdClientEndpoint) + } + defer cli.Close() + + // give enough time-out in case expensive requests (range/delete) are pending + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + _, err = cli.Put(ctx, "health", "good") + cancel() + if err != nil { + return fmt.Errorf("%v (%q)", err, m.EtcdClientEndpoint) + } + return nil +} diff --git a/tools/functional-tester/rpcpb/rpc.pb.go b/tools/functional-tester/rpcpb/rpc.pb.go new file mode 100644 index 000000000..d75518506 --- /dev/null +++ b/tools/functional-tester/rpcpb/rpc.pb.go @@ -0,0 +1,2862 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: rpcpb/rpc.proto + +/* + Package rpcpb is a generated protocol buffer package. + + It is generated from these files: + rpcpb/rpc.proto + + It has these top-level messages: + Etcd + Member + Tester + Request + Response +*/ +package rpcpb + +import proto "github.com/golang/protobuf/proto" +import fmt "fmt" +import math "math" +import _ "github.com/gogo/protobuf/gogoproto" + +import context "golang.org/x/net/context" +import grpc "google.golang.org/grpc" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package + +type Operation int32 + +const ( + Operation_NotStarted Operation = 0 + // InitialStartEtcd is only called to start etcd very first time. + Operation_InitialStartEtcd Operation = 1 + // RestartEtcd is sent to restart killed etcd. + Operation_RestartEtcd Operation = 2 + // KillEtcd pauses etcd process while keeping data directories + // and previous etcd configurations. + Operation_KillEtcd Operation = 3 + // FailArchive is sent when consistency check failed, + // thus need to archive etcd data directories. + Operation_FailArchive Operation = 4 + // DestroyEtcdAgent destroys etcd process, etcd data, and agent server. + Operation_DestroyEtcdAgent Operation = 5 + Operation_BlackholePeerPortTxRx Operation = 100 + Operation_UnblackholePeerPortTxRx Operation = 101 + Operation_DelayPeerPortTxRx Operation = 102 + Operation_UndelayPeerPortTxRx Operation = 103 +) + +var Operation_name = map[int32]string{ + 0: "NotStarted", + 1: "InitialStartEtcd", + 2: "RestartEtcd", + 3: "KillEtcd", + 4: "FailArchive", + 5: "DestroyEtcdAgent", + 100: "BlackholePeerPortTxRx", + 101: "UnblackholePeerPortTxRx", + 102: "DelayPeerPortTxRx", + 103: "UndelayPeerPortTxRx", +} +var Operation_value = map[string]int32{ + "NotStarted": 0, + "InitialStartEtcd": 1, + "RestartEtcd": 2, + "KillEtcd": 3, + "FailArchive": 4, + "DestroyEtcdAgent": 5, + "BlackholePeerPortTxRx": 100, + "UnblackholePeerPortTxRx": 101, + "DelayPeerPortTxRx": 102, + "UndelayPeerPortTxRx": 103, +} + +func (x Operation) String() string { + return proto.EnumName(Operation_name, int32(x)) +} +func (Operation) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{0} } + +type FailureCase int32 + +const ( + FailureCase_KILL_ONE_FOLLOWER FailureCase = 0 + FailureCase_KILL_LEADER FailureCase = 1 + FailureCase_KILL_ONE_FOLLOWER_FOR_LONG FailureCase = 2 + FailureCase_KILL_LEADER_FOR_LONG FailureCase = 3 + FailureCase_KILL_QUORUM FailureCase = 4 + FailureCase_KILL_ALL FailureCase = 5 + FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER FailureCase = 6 + FailureCase_BLACKHOLE_PEER_PORT_TX_RX_LEADER_ONE FailureCase = 7 + FailureCase_BLACKHOLE_PEER_PORT_TX_RX_ALL FailureCase = 8 + FailureCase_DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER FailureCase = 9 + FailureCase_DELAY_PEER_PORT_TX_RX_LEADER FailureCase = 10 + FailureCase_DELAY_PEER_PORT_TX_RX_ALL FailureCase = 11 + FailureCase_FAILPOINTS FailureCase = 100 + FailureCase_NO_FAIL FailureCase = 200 + FailureCase_EXTERNAL FailureCase = 300 +) + +var FailureCase_name = map[int32]string{ + 0: "KILL_ONE_FOLLOWER", + 1: "KILL_LEADER", + 2: "KILL_ONE_FOLLOWER_FOR_LONG", + 3: "KILL_LEADER_FOR_LONG", + 4: "KILL_QUORUM", + 5: "KILL_ALL", + 6: "BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER", + 7: "BLACKHOLE_PEER_PORT_TX_RX_LEADER_ONE", + 8: "BLACKHOLE_PEER_PORT_TX_RX_ALL", + 9: "DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER", + 10: "DELAY_PEER_PORT_TX_RX_LEADER", + 11: "DELAY_PEER_PORT_TX_RX_ALL", + 100: "FAILPOINTS", + 200: "NO_FAIL", + 300: "EXTERNAL", +} +var FailureCase_value = map[string]int32{ + "KILL_ONE_FOLLOWER": 0, + "KILL_LEADER": 1, + "KILL_ONE_FOLLOWER_FOR_LONG": 2, + "KILL_LEADER_FOR_LONG": 3, + "KILL_QUORUM": 4, + "KILL_ALL": 5, + "BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER": 6, + "BLACKHOLE_PEER_PORT_TX_RX_LEADER_ONE": 7, + "BLACKHOLE_PEER_PORT_TX_RX_ALL": 8, + "DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER": 9, + "DELAY_PEER_PORT_TX_RX_LEADER": 10, + "DELAY_PEER_PORT_TX_RX_ALL": 11, + "FAILPOINTS": 100, + "NO_FAIL": 200, + "EXTERNAL": 300, +} + +func (x FailureCase) String() string { + return proto.EnumName(FailureCase_name, int32(x)) +} +func (FailureCase) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{1} } + +type StressType int32 + +const ( + StressType_KV StressType = 0 + StressType_LEASE StressType = 1 + StressType_NO_STRESS StressType = 2 + StressType_ELECTION_RUNNER StressType = 3 + StressType_WATCH_RUNNER StressType = 4 + StressType_LOCK_RACER_RUNNER StressType = 5 + StressType_LEASE_RUNNER StressType = 6 +) + +var StressType_name = map[int32]string{ + 0: "KV", + 1: "LEASE", + 2: "NO_STRESS", + 3: "ELECTION_RUNNER", + 4: "WATCH_RUNNER", + 5: "LOCK_RACER_RUNNER", + 6: "LEASE_RUNNER", +} +var StressType_value = map[string]int32{ + "KV": 0, + "LEASE": 1, + "NO_STRESS": 2, + "ELECTION_RUNNER": 3, + "WATCH_RUNNER": 4, + "LOCK_RACER_RUNNER": 5, + "LEASE_RUNNER": 6, +} + +func (x StressType) String() string { + return proto.EnumName(StressType_name, int32(x)) +} +func (StressType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRpc, []int{2} } + +type Etcd struct { + Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"Name,omitempty" yaml:"name"` + DataDir string `protobuf:"bytes,2,opt,name=DataDir,proto3" json:"DataDir,omitempty" yaml:"data-dir"` + WALDir string `protobuf:"bytes,3,opt,name=WALDir,proto3" json:"WALDir,omitempty" yaml:"wal-dir"` + ListenClientURLs []string `protobuf:"bytes,4,rep,name=ListenClientURLs" json:"ListenClientURLs,omitempty" yaml:"listen-client-urls"` + AdvertiseClientURLs []string `protobuf:"bytes,5,rep,name=AdvertiseClientURLs" json:"AdvertiseClientURLs,omitempty" yaml:"advertise-client-urls"` + ListenPeerURLs []string `protobuf:"bytes,6,rep,name=ListenPeerURLs" json:"ListenPeerURLs,omitempty" yaml:"listen-peer-urls"` + InitialAdvertisePeerURLs []string `protobuf:"bytes,7,rep,name=InitialAdvertisePeerURLs" json:"InitialAdvertisePeerURLs,omitempty" yaml:"initial-advertise-peer-urls"` + InitialCluster string `protobuf:"bytes,8,opt,name=InitialCluster,proto3" json:"InitialCluster,omitempty" yaml:"initial-cluster"` + InitialClusterState string `protobuf:"bytes,9,opt,name=InitialClusterState,proto3" json:"InitialClusterState,omitempty" yaml:"initial-cluster-state"` + InitialClusterToken string `protobuf:"bytes,10,opt,name=InitialClusterToken,proto3" json:"InitialClusterToken,omitempty" yaml:"initial-cluster-token"` + SnapshotCount int64 `protobuf:"varint,11,opt,name=SnapshotCount,proto3" json:"SnapshotCount,omitempty" yaml:"snapshot-count"` + QuotaBackendBytes int64 `protobuf:"varint,12,opt,name=QuotaBackendBytes,proto3" json:"QuotaBackendBytes,omitempty" yaml:"quota-backend-bytes"` + PreVote bool `protobuf:"varint,13,opt,name=PreVote,proto3" json:"PreVote,omitempty" yaml:"pre-vote"` + InitialCorruptCheck bool `protobuf:"varint,14,opt,name=InitialCorruptCheck,proto3" json:"InitialCorruptCheck,omitempty" yaml:"initial-corrupt-check"` +} + +func (m *Etcd) Reset() { *m = Etcd{} } +func (m *Etcd) String() string { return proto.CompactTextString(m) } +func (*Etcd) ProtoMessage() {} +func (*Etcd) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{0} } + +type Member struct { + // EtcdExecPath is the executable etcd binary path in agent server. + EtcdExecPath string `protobuf:"bytes,1,opt,name=EtcdExecPath,proto3" json:"EtcdExecPath,omitempty" yaml:"etcd-exec-path"` + // AgentAddr is the agent HTTP server address. + AgentAddr string `protobuf:"bytes,11,opt,name=AgentAddr,proto3" json:"AgentAddr,omitempty" yaml:"agent-addr"` + // FailpointHTTPAddr is the agent's failpoints HTTP server address. + FailpointHTTPAddr string `protobuf:"bytes,12,opt,name=FailpointHTTPAddr,proto3" json:"FailpointHTTPAddr,omitempty" yaml:"failpoint-http-addr"` + // BaseDir is the base directory where all logs and etcd data are stored. + BaseDir string `protobuf:"bytes,101,opt,name=BaseDir,proto3" json:"BaseDir,omitempty" yaml:"base-dir"` + // EtcdLogPath is the log file to store current etcd server logs. + EtcdLogPath string `protobuf:"bytes,102,opt,name=EtcdLogPath,proto3" json:"EtcdLogPath,omitempty" yaml:"etcd-log-path"` + // EtcdClientTLS is true when client traffic needs to be encrypted. + EtcdClientTLS bool `protobuf:"varint,201,opt,name=EtcdClientTLS,proto3" json:"EtcdClientTLS,omitempty" yaml:"etcd-client-tls"` + // EtcdClientProxy is true when client traffic needs to be proxied. + // If true, listen client URL port must be different than advertise client URL port. + EtcdClientProxy bool `protobuf:"varint,202,opt,name=EtcdClientProxy,proto3" json:"EtcdClientProxy,omitempty" yaml:"etcd-client-proxy"` + // EtcdPeerProxy is true when peer traffic needs to be proxied. + // If true, listen peer URL port must be different than advertise peer URL port. + EtcdPeerProxy bool `protobuf:"varint,203,opt,name=EtcdPeerProxy,proto3" json:"EtcdPeerProxy,omitempty" yaml:"etcd-peer-proxy"` + // EtcdClientEndpoint is the etcd client endpoint. + EtcdClientEndpoint string `protobuf:"bytes,204,opt,name=EtcdClientEndpoint,proto3" json:"EtcdClientEndpoint,omitempty" yaml:"etcd-client-endpoint"` + // Etcd defines etcd binary configuration flags. + Etcd *Etcd `protobuf:"bytes,301,opt,name=Etcd" json:"Etcd,omitempty" yaml:"etcd-config"` +} + +func (m *Member) Reset() { *m = Member{} } +func (m *Member) String() string { return proto.CompactTextString(m) } +func (*Member) ProtoMessage() {} +func (*Member) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{1} } + +type Tester struct { + TesterNetwork string `protobuf:"bytes,1,opt,name=TesterNetwork,proto3" json:"TesterNetwork,omitempty" yaml:"tester-network"` + TesterAddr string `protobuf:"bytes,2,opt,name=TesterAddr,proto3" json:"TesterAddr,omitempty" yaml:"tester-addr"` + // DelayLatencyMsRv is the delay latency in milliseconds, + // to inject to simulated slow network. + DelayLatencyMs uint32 `protobuf:"varint,11,opt,name=DelayLatencyMs,proto3" json:"DelayLatencyMs,omitempty" yaml:"delay-latency-ms"` + // DelayLatencyMsRv is the delay latency random variable in milliseconds. + DelayLatencyMsRv uint32 `protobuf:"varint,12,opt,name=DelayLatencyMsRv,proto3" json:"DelayLatencyMsRv,omitempty" yaml:"delay-latency-ms-rv"` + // RoundLimit is the limit of rounds to run failure set (-1 to run without limits). + RoundLimit int32 `protobuf:"varint,21,opt,name=RoundLimit,proto3" json:"RoundLimit,omitempty" yaml:"round-limit"` + // ExitOnFailure is true, then exit tester on first failure. + ExitOnFailure bool `protobuf:"varint,22,opt,name=ExitOnFailure,proto3" json:"ExitOnFailure,omitempty" yaml:"exit-on-failure"` + // ConsistencyCheck is true to check consistency (revision, hash). + ConsistencyCheck bool `protobuf:"varint,23,opt,name=ConsistencyCheck,proto3" json:"ConsistencyCheck,omitempty" yaml:"consistency-check"` + // EnablePprof is true to enable profiler. + EnablePprof bool `protobuf:"varint,24,opt,name=EnablePprof,proto3" json:"EnablePprof,omitempty" yaml:"enable-pprof"` + // FailureCases is the selected test cases to schedule. + // If empty, run all failure cases. + // TODO: support no-op + FailureCases []string `protobuf:"bytes,31,rep,name=FailureCases" json:"FailureCases,omitempty" yaml:"failure-cases"` + // FailureShuffle is true to randomize failure injecting order. + // TODO: support shuffle + // bool FailureShuffle = 32 [(gogoproto.moretags) = "yaml:\"failure-shuffle\""]; + // FailpointCommands is the list of "gofail" commands (e.g. panic("etcd-tester"),1*sleep(1000)). + FailpointCommands []string `protobuf:"bytes,33,rep,name=FailpointCommands" json:"FailpointCommands,omitempty" yaml:"failpoint-commands"` + // RunnerExecPath is a path of etcd-runner binary. + RunnerExecPath string `protobuf:"bytes,41,opt,name=RunnerExecPath,proto3" json:"RunnerExecPath,omitempty" yaml:"runner-exec-path"` + // ExternalExecPath is a path of script for enabling/disabling an external fault injector. + ExternalExecPath string `protobuf:"bytes,42,opt,name=ExternalExecPath,proto3" json:"ExternalExecPath,omitempty" yaml:"external-exec-path"` + // StressTypes is the list of stresser names: + // keys, lease, nop, election-runner, watch-runner, lock-racer-runner, lease-runner. + StressTypes []string `protobuf:"bytes,101,rep,name=StressTypes" json:"StressTypes,omitempty" yaml:"stress-types"` + // StressKeySize is the size of each small key written into etcd. + StressKeySize int32 `protobuf:"varint,102,opt,name=StressKeySize,proto3" json:"StressKeySize,omitempty" yaml:"stress-key-size"` + // StressKeySizeLarge is the size of each large key written into etcd. + StressKeySizeLarge int32 `protobuf:"varint,103,opt,name=StressKeySizeLarge,proto3" json:"StressKeySizeLarge,omitempty" 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)". + StressKeySuffixRange int32 `protobuf:"varint,104,opt,name=StressKeySuffixRange,proto3" json:"StressKeySuffixRange,omitempty" 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)". + StressKeySuffixRangeTxn int32 `protobuf:"varint,105,opt,name=StressKeySuffixRangeTxn,proto3" json:"StressKeySuffixRangeTxn,omitempty" yaml:"stress-key-suffix-range-txn"` + // StressKeyTxnOps is the number of operations per a transaction (max 64). + StressKeyTxnOps int32 `protobuf:"varint,106,opt,name=StressKeyTxnOps,proto3" json:"StressKeyTxnOps,omitempty" yaml:"stress-key-txn-ops"` + // StressQPS is the maximum number of stresser requests per second. + StressQPS int32 `protobuf:"varint,107,opt,name=StressQPS,proto3" json:"StressQPS,omitempty" yaml:"stress-qps"` +} + +func (m *Tester) Reset() { *m = Tester{} } +func (m *Tester) String() string { return proto.CompactTextString(m) } +func (*Tester) ProtoMessage() {} +func (*Tester) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{2} } + +type Request struct { + Operation Operation `protobuf:"varint,1,opt,name=Operation,proto3,enum=rpcpb.Operation" json:"Operation,omitempty"` + Member *Member `protobuf:"bytes,2,opt,name=Member" json:"Member,omitempty"` + Tester *Tester `protobuf:"bytes,3,opt,name=Tester" json:"Tester,omitempty"` +} + +func (m *Request) Reset() { *m = Request{} } +func (m *Request) String() string { return proto.CompactTextString(m) } +func (*Request) ProtoMessage() {} +func (*Request) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{3} } + +type Response struct { + Success bool `protobuf:"varint,1,opt,name=Success,proto3" json:"Success,omitempty"` + Status string `protobuf:"bytes,2,opt,name=Status,proto3" json:"Status,omitempty"` +} + +func (m *Response) Reset() { *m = Response{} } +func (m *Response) String() string { return proto.CompactTextString(m) } +func (*Response) ProtoMessage() {} +func (*Response) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{4} } + +func init() { + proto.RegisterType((*Etcd)(nil), "rpcpb.Etcd") + proto.RegisterType((*Member)(nil), "rpcpb.Member") + proto.RegisterType((*Tester)(nil), "rpcpb.Tester") + proto.RegisterType((*Request)(nil), "rpcpb.Request") + proto.RegisterType((*Response)(nil), "rpcpb.Response") + proto.RegisterEnum("rpcpb.Operation", Operation_name, Operation_value) + proto.RegisterEnum("rpcpb.FailureCase", FailureCase_name, FailureCase_value) + proto.RegisterEnum("rpcpb.StressType", StressType_name, StressType_value) +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// Client API for Transport service + +type TransportClient interface { + Transport(ctx context.Context, opts ...grpc.CallOption) (Transport_TransportClient, error) +} + +type transportClient struct { + cc *grpc.ClientConn +} + +func NewTransportClient(cc *grpc.ClientConn) TransportClient { + return &transportClient{cc} +} + +func (c *transportClient) Transport(ctx context.Context, opts ...grpc.CallOption) (Transport_TransportClient, error) { + stream, err := grpc.NewClientStream(ctx, &_Transport_serviceDesc.Streams[0], c.cc, "/rpcpb.Transport/Transport", opts...) + if err != nil { + return nil, err + } + x := &transportTransportClient{stream} + return x, nil +} + +type Transport_TransportClient interface { + Send(*Request) error + Recv() (*Response, error) + grpc.ClientStream +} + +type transportTransportClient struct { + grpc.ClientStream +} + +func (x *transportTransportClient) Send(m *Request) error { + return x.ClientStream.SendMsg(m) +} + +func (x *transportTransportClient) Recv() (*Response, error) { + m := new(Response) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// Server API for Transport service + +type TransportServer interface { + Transport(Transport_TransportServer) error +} + +func RegisterTransportServer(s *grpc.Server, srv TransportServer) { + s.RegisterService(&_Transport_serviceDesc, srv) +} + +func _Transport_Transport_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(TransportServer).Transport(&transportTransportServer{stream}) +} + +type Transport_TransportServer interface { + Send(*Response) error + Recv() (*Request, error) + grpc.ServerStream +} + +type transportTransportServer struct { + grpc.ServerStream +} + +func (x *transportTransportServer) Send(m *Response) error { + return x.ServerStream.SendMsg(m) +} + +func (x *transportTransportServer) Recv() (*Request, error) { + m := new(Request) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _Transport_serviceDesc = grpc.ServiceDesc{ + ServiceName: "rpcpb.Transport", + HandlerType: (*TransportServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Transport", + Handler: _Transport_Transport_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "rpcpb/rpc.proto", +} + +func (m *Etcd) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Etcd) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Name) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.Name))) + i += copy(dAtA[i:], m.Name) + } + if len(m.DataDir) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.DataDir))) + i += copy(dAtA[i:], m.DataDir) + } + if len(m.WALDir) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.WALDir))) + i += copy(dAtA[i:], m.WALDir) + } + if len(m.ListenClientURLs) > 0 { + for _, s := range m.ListenClientURLs { + dAtA[i] = 0x22 + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if len(m.AdvertiseClientURLs) > 0 { + for _, s := range m.AdvertiseClientURLs { + dAtA[i] = 0x2a + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if len(m.ListenPeerURLs) > 0 { + for _, s := range m.ListenPeerURLs { + dAtA[i] = 0x32 + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if len(m.InitialAdvertisePeerURLs) > 0 { + for _, s := range m.InitialAdvertisePeerURLs { + dAtA[i] = 0x3a + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if len(m.InitialCluster) > 0 { + dAtA[i] = 0x42 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.InitialCluster))) + i += copy(dAtA[i:], m.InitialCluster) + } + if len(m.InitialClusterState) > 0 { + dAtA[i] = 0x4a + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.InitialClusterState))) + i += copy(dAtA[i:], m.InitialClusterState) + } + if len(m.InitialClusterToken) > 0 { + dAtA[i] = 0x52 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.InitialClusterToken))) + i += copy(dAtA[i:], m.InitialClusterToken) + } + if m.SnapshotCount != 0 { + dAtA[i] = 0x58 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.SnapshotCount)) + } + if m.QuotaBackendBytes != 0 { + dAtA[i] = 0x60 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.QuotaBackendBytes)) + } + if m.PreVote { + dAtA[i] = 0x68 + i++ + if m.PreVote { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.InitialCorruptCheck { + dAtA[i] = 0x70 + i++ + if m.InitialCorruptCheck { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + return i, nil +} + +func (m *Member) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Member) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.EtcdExecPath) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.EtcdExecPath))) + i += copy(dAtA[i:], m.EtcdExecPath) + } + if len(m.AgentAddr) > 0 { + dAtA[i] = 0x5a + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.AgentAddr))) + i += copy(dAtA[i:], m.AgentAddr) + } + if len(m.FailpointHTTPAddr) > 0 { + dAtA[i] = 0x62 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.FailpointHTTPAddr))) + i += copy(dAtA[i:], m.FailpointHTTPAddr) + } + if len(m.BaseDir) > 0 { + dAtA[i] = 0xaa + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.BaseDir))) + i += copy(dAtA[i:], m.BaseDir) + } + if len(m.EtcdLogPath) > 0 { + dAtA[i] = 0xb2 + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.EtcdLogPath))) + i += copy(dAtA[i:], m.EtcdLogPath) + } + if m.EtcdClientTLS { + dAtA[i] = 0xc8 + i++ + dAtA[i] = 0xc + i++ + if m.EtcdClientTLS { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.EtcdClientProxy { + dAtA[i] = 0xd0 + i++ + dAtA[i] = 0xc + i++ + if m.EtcdClientProxy { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.EtcdPeerProxy { + dAtA[i] = 0xd8 + i++ + dAtA[i] = 0xc + i++ + if m.EtcdPeerProxy { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if len(m.EtcdClientEndpoint) > 0 { + dAtA[i] = 0xe2 + i++ + dAtA[i] = 0xc + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.EtcdClientEndpoint))) + i += copy(dAtA[i:], m.EtcdClientEndpoint) + } + if m.Etcd != nil { + dAtA[i] = 0xea + i++ + dAtA[i] = 0x12 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.Etcd.Size())) + n1, err := m.Etcd.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + } + return i, nil +} + +func (m *Tester) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Tester) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.TesterNetwork) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.TesterNetwork))) + i += copy(dAtA[i:], m.TesterNetwork) + } + if len(m.TesterAddr) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.TesterAddr))) + i += copy(dAtA[i:], m.TesterAddr) + } + if m.DelayLatencyMs != 0 { + dAtA[i] = 0x58 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.DelayLatencyMs)) + } + if m.DelayLatencyMsRv != 0 { + dAtA[i] = 0x60 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.DelayLatencyMsRv)) + } + if m.RoundLimit != 0 { + dAtA[i] = 0xa8 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.RoundLimit)) + } + if m.ExitOnFailure { + dAtA[i] = 0xb0 + i++ + dAtA[i] = 0x1 + i++ + if m.ExitOnFailure { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.ConsistencyCheck { + dAtA[i] = 0xb8 + i++ + dAtA[i] = 0x1 + i++ + if m.ConsistencyCheck { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.EnablePprof { + dAtA[i] = 0xc0 + i++ + dAtA[i] = 0x1 + i++ + if m.EnablePprof { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if len(m.FailureCases) > 0 { + for _, s := range m.FailureCases { + dAtA[i] = 0xfa + i++ + dAtA[i] = 0x1 + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if len(m.FailpointCommands) > 0 { + for _, s := range m.FailpointCommands { + dAtA[i] = 0x8a + i++ + dAtA[i] = 0x2 + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if len(m.RunnerExecPath) > 0 { + dAtA[i] = 0xca + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.RunnerExecPath))) + i += copy(dAtA[i:], m.RunnerExecPath) + } + if len(m.ExternalExecPath) > 0 { + dAtA[i] = 0xd2 + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.ExternalExecPath))) + i += copy(dAtA[i:], m.ExternalExecPath) + } + if len(m.StressTypes) > 0 { + for _, s := range m.StressTypes { + dAtA[i] = 0xaa + i++ + dAtA[i] = 0x6 + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + if m.StressKeySize != 0 { + dAtA[i] = 0xb0 + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.StressKeySize)) + } + if m.StressKeySizeLarge != 0 { + dAtA[i] = 0xb8 + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.StressKeySizeLarge)) + } + if m.StressKeySuffixRange != 0 { + dAtA[i] = 0xc0 + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.StressKeySuffixRange)) + } + if m.StressKeySuffixRangeTxn != 0 { + dAtA[i] = 0xc8 + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.StressKeySuffixRangeTxn)) + } + if m.StressKeyTxnOps != 0 { + dAtA[i] = 0xd0 + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.StressKeyTxnOps)) + } + if m.StressQPS != 0 { + dAtA[i] = 0xd8 + i++ + dAtA[i] = 0x6 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.StressQPS)) + } + return i, nil +} + +func (m *Request) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Request) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Operation != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.Operation)) + } + if m.Member != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.Member.Size())) + n2, err := m.Member.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + } + if m.Tester != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintRpc(dAtA, i, uint64(m.Tester.Size())) + n3, err := m.Tester.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + } + return i, nil +} + +func (m *Response) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Success { + dAtA[i] = 0x8 + i++ + if m.Success { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if len(m.Status) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintRpc(dAtA, i, uint64(len(m.Status))) + i += copy(dAtA[i:], m.Status) + } + return i, nil +} + +func encodeVarintRpc(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *Etcd) Size() (n int) { + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.DataDir) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.WALDir) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + if len(m.ListenClientURLs) > 0 { + for _, s := range m.ListenClientURLs { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if len(m.AdvertiseClientURLs) > 0 { + for _, s := range m.AdvertiseClientURLs { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if len(m.ListenPeerURLs) > 0 { + for _, s := range m.ListenPeerURLs { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + if len(m.InitialAdvertisePeerURLs) > 0 { + for _, s := range m.InitialAdvertisePeerURLs { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } + l = len(m.InitialCluster) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.InitialClusterState) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.InitialClusterToken) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + if m.SnapshotCount != 0 { + n += 1 + sovRpc(uint64(m.SnapshotCount)) + } + if m.QuotaBackendBytes != 0 { + n += 1 + sovRpc(uint64(m.QuotaBackendBytes)) + } + if m.PreVote { + n += 2 + } + if m.InitialCorruptCheck { + n += 2 + } + return n +} + +func (m *Member) Size() (n int) { + var l int + _ = l + l = len(m.EtcdExecPath) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.AgentAddr) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.FailpointHTTPAddr) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.BaseDir) + if l > 0 { + n += 2 + l + sovRpc(uint64(l)) + } + l = len(m.EtcdLogPath) + if l > 0 { + n += 2 + l + sovRpc(uint64(l)) + } + if m.EtcdClientTLS { + n += 3 + } + if m.EtcdClientProxy { + n += 3 + } + if m.EtcdPeerProxy { + n += 3 + } + l = len(m.EtcdClientEndpoint) + if l > 0 { + n += 2 + l + sovRpc(uint64(l)) + } + if m.Etcd != nil { + l = m.Etcd.Size() + n += 2 + l + sovRpc(uint64(l)) + } + return n +} + +func (m *Tester) Size() (n int) { + var l int + _ = l + l = len(m.TesterNetwork) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + l = len(m.TesterAddr) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + if m.DelayLatencyMs != 0 { + n += 1 + sovRpc(uint64(m.DelayLatencyMs)) + } + if m.DelayLatencyMsRv != 0 { + n += 1 + sovRpc(uint64(m.DelayLatencyMsRv)) + } + if m.RoundLimit != 0 { + n += 2 + sovRpc(uint64(m.RoundLimit)) + } + if m.ExitOnFailure { + n += 3 + } + if m.ConsistencyCheck { + n += 3 + } + if m.EnablePprof { + n += 3 + } + if len(m.FailureCases) > 0 { + for _, s := range m.FailureCases { + l = len(s) + n += 2 + l + sovRpc(uint64(l)) + } + } + if len(m.FailpointCommands) > 0 { + for _, s := range m.FailpointCommands { + l = len(s) + n += 2 + l + sovRpc(uint64(l)) + } + } + l = len(m.RunnerExecPath) + if l > 0 { + n += 2 + l + sovRpc(uint64(l)) + } + l = len(m.ExternalExecPath) + if l > 0 { + n += 2 + l + sovRpc(uint64(l)) + } + if len(m.StressTypes) > 0 { + for _, s := range m.StressTypes { + l = len(s) + n += 2 + l + sovRpc(uint64(l)) + } + } + if m.StressKeySize != 0 { + n += 2 + sovRpc(uint64(m.StressKeySize)) + } + if m.StressKeySizeLarge != 0 { + n += 2 + sovRpc(uint64(m.StressKeySizeLarge)) + } + if m.StressKeySuffixRange != 0 { + n += 2 + sovRpc(uint64(m.StressKeySuffixRange)) + } + if m.StressKeySuffixRangeTxn != 0 { + n += 2 + sovRpc(uint64(m.StressKeySuffixRangeTxn)) + } + if m.StressKeyTxnOps != 0 { + n += 2 + sovRpc(uint64(m.StressKeyTxnOps)) + } + if m.StressQPS != 0 { + n += 2 + sovRpc(uint64(m.StressQPS)) + } + return n +} + +func (m *Request) Size() (n int) { + var l int + _ = l + if m.Operation != 0 { + n += 1 + sovRpc(uint64(m.Operation)) + } + if m.Member != nil { + l = m.Member.Size() + n += 1 + l + sovRpc(uint64(l)) + } + if m.Tester != nil { + l = m.Tester.Size() + n += 1 + l + sovRpc(uint64(l)) + } + return n +} + +func (m *Response) Size() (n int) { + var l int + _ = l + if m.Success { + n += 2 + } + l = len(m.Status) + if l > 0 { + n += 1 + l + sovRpc(uint64(l)) + } + return n +} + +func sovRpc(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozRpc(x uint64) (n int) { + return sovRpc(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Etcd) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Etcd: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Etcd: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DataDir", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DataDir = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WALDir", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WALDir = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ListenClientURLs", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ListenClientURLs = append(m.ListenClientURLs, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AdvertiseClientURLs", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AdvertiseClientURLs = append(m.AdvertiseClientURLs, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ListenPeerURLs", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ListenPeerURLs = append(m.ListenPeerURLs, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InitialAdvertisePeerURLs", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InitialAdvertisePeerURLs = append(m.InitialAdvertisePeerURLs, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InitialCluster", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InitialCluster = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InitialClusterState", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InitialClusterState = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InitialClusterToken", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InitialClusterToken = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SnapshotCount", wireType) + } + m.SnapshotCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SnapshotCount |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field QuotaBackendBytes", wireType) + } + m.QuotaBackendBytes = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.QuotaBackendBytes |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PreVote", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.PreVote = bool(v != 0) + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field InitialCorruptCheck", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.InitialCorruptCheck = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Member) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Member: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Member: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdExecPath", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.EtcdExecPath = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AgentAddr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AgentAddr = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FailpointHTTPAddr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FailpointHTTPAddr = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 101: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseDir", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseDir = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 102: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdLogPath", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.EtcdLogPath = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 201: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdClientTLS", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.EtcdClientTLS = bool(v != 0) + case 202: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdClientProxy", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.EtcdClientProxy = bool(v != 0) + case 203: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdPeerProxy", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.EtcdPeerProxy = bool(v != 0) + case 204: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdClientEndpoint", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.EtcdClientEndpoint = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 301: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Etcd", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Etcd == nil { + m.Etcd = &Etcd{} + } + if err := m.Etcd.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Tester) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Tester: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Tester: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TesterNetwork", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TesterNetwork = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TesterAddr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TesterAddr = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DelayLatencyMs", wireType) + } + m.DelayLatencyMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DelayLatencyMs |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DelayLatencyMsRv", wireType) + } + m.DelayLatencyMsRv = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DelayLatencyMsRv |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 21: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RoundLimit", wireType) + } + m.RoundLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RoundLimit |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 22: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExitOnFailure", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ExitOnFailure = bool(v != 0) + case 23: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsistencyCheck", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ConsistencyCheck = bool(v != 0) + case 24: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EnablePprof", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.EnablePprof = bool(v != 0) + case 31: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FailureCases", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FailureCases = append(m.FailureCases, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 33: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FailpointCommands", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.FailpointCommands = append(m.FailpointCommands, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 41: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RunnerExecPath", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RunnerExecPath = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 42: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExternalExecPath", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExternalExecPath = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 101: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StressTypes", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StressTypes = append(m.StressTypes, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 102: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StressKeySize", wireType) + } + m.StressKeySize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StressKeySize |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 103: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StressKeySizeLarge", wireType) + } + m.StressKeySizeLarge = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StressKeySizeLarge |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 104: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StressKeySuffixRange", wireType) + } + m.StressKeySuffixRange = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StressKeySuffixRange |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 105: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StressKeySuffixRangeTxn", wireType) + } + m.StressKeySuffixRangeTxn = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StressKeySuffixRangeTxn |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 106: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StressKeyTxnOps", wireType) + } + m.StressKeyTxnOps = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StressKeyTxnOps |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 107: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StressQPS", wireType) + } + m.StressQPS = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StressQPS |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Request) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Request: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Request: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Operation", wireType) + } + m.Operation = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Operation |= (Operation(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Member", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Member == nil { + m.Member = &Member{} + } + if err := m.Member.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tester", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Tester == nil { + m.Tester = &Tester{} + } + if err := m.Tester.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Response: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Response: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Success", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Success = bool(v != 0) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipRpc(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthRpc + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRpc + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipRpc(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthRpc = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowRpc = fmt.Errorf("proto: integer overflow") +) + +func init() { proto.RegisterFile("rpcpb/rpc.proto", fileDescriptorRpc) } + +var fileDescriptorRpc = []byte{ + // 1800 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x57, 0xcd, 0x72, 0xdb, 0xba, + 0x15, 0xb6, 0x6c, 0x4b, 0xb6, 0x60, 0xcb, 0x66, 0x60, 0x3b, 0x66, 0x9c, 0xc4, 0xf4, 0x65, 0x7b, + 0x33, 0xae, 0x67, 0xe8, 0x74, 0x72, 0x67, 0x3a, 0xed, 0x9d, 0xdb, 0x49, 0x25, 0x99, 0xb9, 0x76, + 0xcd, 0x48, 0x0a, 0x24, 0x27, 0xe9, 0x4a, 0xa5, 0x48, 0x48, 0x62, 0x4d, 0x91, 0x0c, 0x08, 0xf9, + 0x4a, 0x77, 0xd1, 0x6d, 0xb7, 0x5d, 0x76, 0xd3, 0x37, 0x68, 0x57, 0x7d, 0x89, 0xb4, 0xcd, 0xa2, + 0x4f, 0xa0, 0xb6, 0xe9, 0x1b, 0xf0, 0x09, 0x3a, 0x00, 0x28, 0x09, 0xfa, 0x71, 0xbb, 0x33, 0xce, + 0xf9, 0xbe, 0x8f, 0xc0, 0xd1, 0x39, 0x1f, 0x60, 0xb0, 0x4b, 0x22, 0x27, 0x6a, 0x3d, 0x27, 0x91, + 0x73, 0x1e, 0x91, 0x90, 0x86, 0x30, 0xcb, 0x03, 0x47, 0x46, 0xc7, 0xa3, 0xdd, 0x7e, 0xeb, 0xdc, + 0x09, 0x7b, 0xcf, 0x3b, 0x61, 0x27, 0x7c, 0xce, 0xb3, 0xad, 0x7e, 0x9b, 0xaf, 0xf8, 0x82, 0xff, + 0x25, 0x58, 0xfa, 0xa7, 0x0d, 0xb0, 0x6e, 0x52, 0xc7, 0x85, 0x3f, 0x00, 0xeb, 0x15, 0xbb, 0x87, + 0xd5, 0xcc, 0x49, 0xe6, 0x34, 0x5f, 0xda, 0x4d, 0x46, 0xda, 0xd6, 0xd0, 0xee, 0xf9, 0x5f, 0xeb, + 0x81, 0xdd, 0xc3, 0x3a, 0xe2, 0x49, 0x68, 0x80, 0x8d, 0x0b, 0x9b, 0xda, 0x17, 0x1e, 0x51, 0x57, + 0x39, 0x6e, 0x2f, 0x19, 0x69, 0xbb, 0x02, 0xe7, 0xda, 0xd4, 0x36, 0x5c, 0x8f, 0xe8, 0x68, 0x8c, + 0x81, 0x67, 0x20, 0xf7, 0xae, 0x68, 0x31, 0xf4, 0x1a, 0x47, 0xc3, 0x64, 0xa4, 0xed, 0x08, 0xf4, + 0x77, 0xb6, 0x2f, 0xc0, 0x29, 0x02, 0x5e, 0x01, 0xc5, 0xf2, 0x62, 0x8a, 0x83, 0xb2, 0xef, 0xe1, + 0x80, 0xde, 0x20, 0x2b, 0x56, 0xd7, 0x4f, 0xd6, 0x4e, 0xf3, 0xa5, 0xa7, 0xc9, 0x48, 0x7b, 0x24, + 0x58, 0x3e, 0x47, 0x18, 0x0e, 0x87, 0x18, 0x7d, 0xe2, 0xc7, 0x3a, 0x5a, 0xa0, 0x41, 0x04, 0xf6, + 0x8a, 0xee, 0x1d, 0x26, 0xd4, 0x8b, 0xb1, 0xa4, 0x96, 0xe5, 0x6a, 0x27, 0xc9, 0x48, 0x7b, 0x22, + 0xd4, 0xec, 0x31, 0x68, 0x56, 0x70, 0x19, 0x19, 0x96, 0xc1, 0x8e, 0xf8, 0x4e, 0x0d, 0x63, 0xc2, + 0xe5, 0x72, 0x5c, 0xee, 0x71, 0x32, 0xd2, 0x0e, 0x67, 0x36, 0x17, 0x61, 0x4c, 0x52, 0xa5, 0x39, + 0x0a, 0x6c, 0x01, 0xf5, 0x2a, 0xf0, 0xa8, 0x67, 0xfb, 0x93, 0x4f, 0x4c, 0xe4, 0x36, 0xb8, 0xdc, + 0xb3, 0x64, 0xa4, 0xe9, 0x42, 0xce, 0x13, 0x48, 0x63, 0xba, 0x4b, 0x49, 0xf9, 0x5e, 0x1d, 0x58, + 0x02, 0x3b, 0x69, 0xae, 0xec, 0xf7, 0x63, 0x8a, 0x89, 0xba, 0xc9, 0x6b, 0x7f, 0x94, 0x8c, 0xb4, + 0x87, 0xb3, 0xca, 0x8e, 0x00, 0xe8, 0x68, 0x8e, 0xc1, 0x0a, 0x38, 0x1b, 0xa9, 0x53, 0x9b, 0x62, + 0x35, 0xcf, 0x85, 0xa4, 0x02, 0xce, 0x09, 0x19, 0x31, 0x83, 0xe9, 0x68, 0x19, 0x79, 0x51, 0xb3, + 0x11, 0xde, 0xe2, 0x40, 0x05, 0xff, 0x4f, 0x93, 0x32, 0xd8, 0x82, 0x26, 0x27, 0xc3, 0x97, 0xa0, + 0x50, 0x0f, 0xec, 0x28, 0xee, 0x86, 0xb4, 0x1c, 0xf6, 0x03, 0xaa, 0x6e, 0x9d, 0x64, 0x4e, 0xd7, + 0x4a, 0x8f, 0x92, 0x91, 0x76, 0x20, 0xd4, 0xe2, 0x34, 0x6d, 0x38, 0x2c, 0xaf, 0xa3, 0x59, 0x3c, + 0xb4, 0xc0, 0x83, 0x37, 0xfd, 0x90, 0xda, 0x25, 0xdb, 0xb9, 0xc5, 0x81, 0x5b, 0x1a, 0x52, 0x1c, + 0xab, 0xdb, 0x5c, 0xe4, 0x38, 0x19, 0x69, 0x47, 0x42, 0xe4, 0x03, 0x83, 0x18, 0x2d, 0x81, 0x31, + 0x5a, 0x0c, 0xa4, 0xa3, 0x45, 0x22, 0x9b, 0x8e, 0x1a, 0xc1, 0x6f, 0x43, 0x8a, 0xd5, 0xc2, 0x49, + 0xe6, 0x74, 0x53, 0x9e, 0x8e, 0x88, 0x60, 0xe3, 0x2e, 0x64, 0xd5, 0x19, 0x63, 0xe4, 0x8a, 0x84, + 0x84, 0xf4, 0x23, 0x5a, 0xee, 0x62, 0xe7, 0x56, 0xdd, 0xe1, 0xd4, 0x65, 0x15, 0x11, 0x28, 0xc3, + 0x61, 0x30, 0xa9, 0x22, 0x12, 0x59, 0xff, 0x7d, 0x16, 0xe4, 0x5e, 0xe3, 0x5e, 0x0b, 0x13, 0xf8, + 0x73, 0xb0, 0xcd, 0x06, 0xdb, 0x1c, 0x60, 0xa7, 0x66, 0xd3, 0x6e, 0x3a, 0xd8, 0x52, 0x6d, 0x30, + 0x75, 0x5c, 0x03, 0x0f, 0xb0, 0x63, 0x44, 0x36, 0xed, 0xea, 0x68, 0x06, 0x0e, 0xbf, 0x02, 0xf9, + 0x62, 0x07, 0x07, 0xb4, 0xe8, 0xba, 0x84, 0xd7, 0x35, 0x5f, 0x3a, 0x48, 0x46, 0xda, 0x83, 0x74, + 0x74, 0x58, 0xca, 0xb0, 0x5d, 0x97, 0xe8, 0x68, 0x8a, 0x63, 0xf5, 0x7c, 0x65, 0x7b, 0x7e, 0x14, + 0x7a, 0x01, 0xbd, 0x6c, 0x34, 0x6a, 0x9c, 0xbc, 0xcd, 0xc9, 0x52, 0x3d, 0xdb, 0x63, 0x88, 0xd1, + 0xa5, 0x34, 0x4a, 0x55, 0x16, 0x89, 0xac, 0x9e, 0x25, 0x3b, 0xc6, 0xcc, 0x3f, 0xf0, 0xbc, 0xdb, + 0xb4, 0xec, 0x18, 0xa7, 0x6e, 0x93, 0x62, 0xe0, 0xd7, 0x60, 0x8b, 0x9d, 0xc0, 0x0a, 0x3b, 0xfc, + 0xbc, 0x6d, 0x4e, 0x51, 0x93, 0x91, 0xb6, 0x2f, 0x9d, 0xd7, 0x0f, 0x3b, 0xe9, 0x71, 0x65, 0x30, + 0x2c, 0x82, 0x02, 0x5b, 0x8a, 0x81, 0x6f, 0x58, 0x75, 0xf5, 0xaf, 0x19, 0xfe, 0x33, 0x48, 0x53, + 0xc3, 0xe9, 0xa9, 0x51, 0x50, 0x36, 0x83, 0xb3, 0x0c, 0xf8, 0x2d, 0xd8, 0x9d, 0x06, 0x6a, 0x24, + 0x1c, 0x0c, 0xd5, 0xbf, 0x09, 0x91, 0x27, 0xc9, 0x48, 0x53, 0x17, 0x45, 0x22, 0x86, 0xd1, 0xd1, + 0x3c, 0x6b, 0xbc, 0x17, 0x36, 0xd1, 0x42, 0xe6, 0xef, 0xcb, 0xf7, 0xc2, 0xed, 0x20, 0x15, 0x99, + 0x65, 0xc0, 0x1a, 0x80, 0x53, 0x55, 0x33, 0x70, 0x79, 0x5d, 0xd5, 0x4f, 0xa2, 0x05, 0xb4, 0x64, + 0xa4, 0x3d, 0x5e, 0xdc, 0x0e, 0x4e, 0x61, 0x3a, 0x5a, 0xc2, 0x85, 0x3f, 0x15, 0xd7, 0x84, 0xfa, + 0x67, 0xe6, 0xfb, 0x5b, 0x2f, 0xb6, 0xce, 0xf9, 0x6d, 0x73, 0xce, 0x62, 0xa5, 0x87, 0xc9, 0x48, + 0x83, 0xb2, 0x60, 0x18, 0xb4, 0xbd, 0x8e, 0x8e, 0x38, 0x43, 0xff, 0x0b, 0x00, 0xb9, 0x06, 0xe6, + 0xbe, 0xf2, 0x12, 0x14, 0xc4, 0x5f, 0x15, 0x4c, 0xbf, 0x0b, 0xc9, 0xed, 0x62, 0x4f, 0x52, 0x9e, + 0x36, 0x02, 0x91, 0xd7, 0xd1, 0x2c, 0x1e, 0xfe, 0x04, 0x00, 0x11, 0xe0, 0x8d, 0x25, 0xae, 0x20, + 0xe9, 0xeb, 0x29, 0x5b, 0x34, 0x94, 0x84, 0x64, 0xee, 0x7d, 0x81, 0x7d, 0x7b, 0x68, 0xd9, 0x14, + 0x07, 0xce, 0xf0, 0x75, 0xcc, 0x3b, 0xba, 0x20, 0xbb, 0xb7, 0xcb, 0xf2, 0x86, 0x2f, 0x00, 0x46, + 0x8f, 0xb9, 0xf7, 0x2c, 0x05, 0xfe, 0x12, 0x28, 0xb3, 0x11, 0x74, 0xc7, 0x7b, 0xbb, 0x20, 0xf7, + 0xf6, 0xbc, 0x8c, 0x41, 0xee, 0x74, 0xb4, 0xc0, 0x63, 0x07, 0x41, 0x61, 0x3f, 0x70, 0x2d, 0xaf, + 0xe7, 0x51, 0xf5, 0xe0, 0x24, 0x73, 0x9a, 0x95, 0x0f, 0x42, 0x58, 0xce, 0xf0, 0x59, 0x52, 0x47, + 0x12, 0x12, 0xfe, 0x02, 0x14, 0xcc, 0x81, 0x47, 0xab, 0x01, 0x9b, 0x96, 0x3e, 0xc1, 0xea, 0xc3, + 0x85, 0xd6, 0x18, 0x78, 0xd4, 0x08, 0x03, 0xa3, 0x2d, 0x00, 0xac, 0x35, 0x64, 0x02, 0xbc, 0x04, + 0x4a, 0x39, 0x0c, 0x62, 0x7e, 0x31, 0x39, 0x43, 0x61, 0x39, 0x87, 0xf3, 0x6d, 0xea, 0x4c, 0x11, + 0x63, 0xbb, 0x59, 0x60, 0xc1, 0x9f, 0x81, 0x2d, 0x33, 0xb0, 0x5b, 0x3e, 0xae, 0x45, 0x24, 0x6c, + 0xab, 0x2a, 0x17, 0x39, 0x4c, 0x46, 0xda, 0x5e, 0xba, 0x13, 0x9e, 0x34, 0x22, 0x96, 0x65, 0xe3, + 0x36, 0xc5, 0xc2, 0x6f, 0xc0, 0x76, 0xba, 0x9f, 0xb2, 0x1d, 0xe3, 0x58, 0xd5, 0xf8, 0xe5, 0x27, + 0xcd, 0x6a, 0xba, 0x7b, 0xc3, 0x61, 0x69, 0x1d, 0xcd, 0xa0, 0xe1, 0xb5, 0xe4, 0x32, 0xe5, 0xb0, + 0xd7, 0xb3, 0x03, 0x37, 0x56, 0xbf, 0x98, 0x7f, 0x2b, 0x4c, 0x5d, 0xc6, 0x49, 0x31, 0xb2, 0xc9, + 0x8c, 0x79, 0xac, 0x35, 0x50, 0x3f, 0x08, 0x30, 0x99, 0x18, 0xe5, 0x8f, 0x78, 0x5b, 0x49, 0xad, + 0x41, 0x78, 0x5e, 0xb6, 0xca, 0x39, 0x0a, 0x7b, 0xbc, 0x98, 0x03, 0x8a, 0x49, 0x60, 0xfb, 0x13, + 0x99, 0x33, 0x2e, 0x23, 0x6d, 0x08, 0xa7, 0x08, 0x59, 0x68, 0x81, 0xc6, 0xaa, 0x5a, 0xa7, 0x04, + 0xc7, 0x71, 0x63, 0x18, 0xe1, 0x58, 0xc5, 0xfc, 0x58, 0x52, 0x55, 0x63, 0x9e, 0x34, 0x28, 0xcb, + 0xea, 0x48, 0xc6, 0xb2, 0xe6, 0x10, 0xcb, 0x6b, 0x3c, 0xac, 0x7b, 0xdf, 0x63, 0x6e, 0x81, 0x59, + 0xb9, 0x39, 0x52, 0xf2, 0x2d, 0x1e, 0x1a, 0xb1, 0xf7, 0x3d, 0x6b, 0x8e, 0x19, 0x02, 0xf3, 0x8d, + 0x99, 0x80, 0x65, 0x93, 0x0e, 0x56, 0x3b, 0x5c, 0x46, 0xba, 0x91, 0xe6, 0x64, 0x0c, 0x9f, 0xc1, + 0x74, 0xb4, 0x84, 0x0b, 0xdf, 0x82, 0xfd, 0x69, 0xb4, 0xdf, 0x6e, 0x7b, 0x03, 0x64, 0x07, 0x1d, + 0xac, 0x76, 0xb9, 0xa6, 0x9e, 0x8c, 0xb4, 0xe3, 0x45, 0x4d, 0x8e, 0x33, 0x08, 0x03, 0xea, 0x68, + 0x29, 0x1f, 0xfe, 0x1a, 0x1c, 0x2e, 0x8b, 0x37, 0x06, 0x81, 0xea, 0x71, 0x69, 0xe9, 0x25, 0x75, + 0x8f, 0xb4, 0x41, 0x07, 0x81, 0x8e, 0xee, 0x93, 0x61, 0x7e, 0x3e, 0x49, 0x35, 0x06, 0x41, 0x35, + 0x8a, 0xd5, 0xdf, 0x70, 0x65, 0xe9, 0x27, 0x95, 0x94, 0xe9, 0x20, 0x30, 0xc2, 0x28, 0xd6, 0xd1, + 0x3c, 0x8b, 0xdd, 0xa4, 0x22, 0xf4, 0xa6, 0x56, 0x57, 0x6f, 0xb9, 0x84, 0x74, 0x93, 0xa6, 0x12, + 0x1f, 0x18, 0x75, 0x8a, 0xd3, 0x7f, 0x97, 0x01, 0x1b, 0x08, 0x7f, 0xe8, 0xe3, 0x98, 0xc2, 0x73, + 0x90, 0xaf, 0x46, 0x98, 0xd8, 0xd4, 0x0b, 0x03, 0x6e, 0x99, 0x3b, 0x2f, 0x94, 0xd4, 0x7f, 0x27, + 0x71, 0x34, 0x85, 0xc0, 0x2f, 0xc7, 0x6f, 0x00, 0x55, 0x98, 0x75, 0x21, 0x05, 0x8b, 0x20, 0x1a, + 0x3f, 0x10, 0xbe, 0x1c, 0xfb, 0x32, 0x7f, 0x9d, 0x4f, 0x61, 0x22, 0x88, 0xd2, 0xa4, 0xfe, 0x0d, + 0xd8, 0x44, 0x38, 0x8e, 0xc2, 0x20, 0xc6, 0x50, 0x05, 0x1b, 0xf5, 0xbe, 0xe3, 0xe0, 0x38, 0xe6, + 0xfb, 0xd8, 0x44, 0xe3, 0x25, 0x7c, 0x08, 0x72, 0xec, 0x9d, 0xd7, 0x8f, 0x85, 0x2b, 0xa3, 0x74, + 0x75, 0xf6, 0xcf, 0x8c, 0xb4, 0x79, 0xb8, 0x03, 0x40, 0x25, 0xa4, 0x75, 0x6a, 0x13, 0x8a, 0x5d, + 0x65, 0x05, 0xee, 0x03, 0x25, 0x7d, 0xc5, 0xf0, 0x18, 0xbb, 0x2f, 0x94, 0x0c, 0xdc, 0x05, 0x5b, + 0x08, 0xc7, 0x93, 0xc0, 0x2a, 0xdc, 0x06, 0x9b, 0xd7, 0x9e, 0xef, 0xf3, 0xd5, 0x1a, 0x4b, 0xb3, + 0x31, 0x2e, 0x12, 0xa7, 0xeb, 0xdd, 0x61, 0x65, 0x9d, 0xa9, 0x5c, 0xe0, 0x98, 0x92, 0x70, 0xc8, + 0x10, 0xfc, 0x35, 0xa2, 0x64, 0xe1, 0x23, 0x70, 0x50, 0xf2, 0x6d, 0xe7, 0xb6, 0x1b, 0xfa, 0xfc, + 0x75, 0x5c, 0x0b, 0x09, 0x6d, 0x0c, 0xd0, 0x40, 0x71, 0xe1, 0x63, 0x70, 0x78, 0x13, 0xb4, 0x96, + 0x26, 0x31, 0x3c, 0x00, 0x0f, 0xb8, 0x5d, 0xcf, 0x84, 0xdb, 0xf0, 0x10, 0xec, 0xdd, 0x04, 0xee, + 0x42, 0xa2, 0x73, 0xf6, 0xc7, 0x35, 0xb1, 0x9f, 0xd4, 0x9e, 0x18, 0xff, 0xfa, 0xca, 0xb2, 0x9a, + 0xd5, 0x8a, 0xd9, 0x7c, 0x55, 0xb5, 0xac, 0xea, 0x3b, 0x13, 0x29, 0x2b, 0x6c, 0xd7, 0x3c, 0x6c, + 0x99, 0xc5, 0x0b, 0x13, 0x29, 0x19, 0x78, 0x0c, 0x8e, 0x16, 0x70, 0xcd, 0x57, 0x55, 0xd4, 0xb4, + 0xaa, 0x95, 0x6f, 0x95, 0x55, 0xa8, 0x82, 0x7d, 0x89, 0x30, 0xcd, 0xac, 0x4d, 0xa4, 0xde, 0xdc, + 0x54, 0xd1, 0xcd, 0x6b, 0x65, 0x9d, 0xd7, 0x87, 0x05, 0x8a, 0x96, 0xa5, 0x64, 0xe1, 0x19, 0x78, + 0x56, 0xb2, 0x8a, 0xe5, 0xeb, 0xcb, 0xaa, 0x65, 0x36, 0x6b, 0xa6, 0x89, 0x9a, 0xb5, 0x2a, 0x6a, + 0x34, 0x1b, 0xef, 0x9b, 0xe8, 0xfd, 0xec, 0xae, 0x72, 0xf0, 0x14, 0xfc, 0xf0, 0x7e, 0x6c, 0xfa, + 0xe5, 0x6a, 0xc5, 0x54, 0x36, 0xe0, 0x17, 0xe0, 0xe9, 0xfd, 0x48, 0xf6, 0xe1, 0x4d, 0xf8, 0x0c, + 0xe8, 0x17, 0xa6, 0x55, 0xfc, 0xd5, 0xff, 0xfe, 0x68, 0x1e, 0x9e, 0x80, 0x27, 0xcb, 0x71, 0x69, + 0x6d, 0x00, 0x7c, 0x0a, 0x1e, 0x2d, 0x47, 0xb0, 0x0f, 0x6d, 0xb1, 0x36, 0x7a, 0x55, 0xbc, 0xb2, + 0x6a, 0xd5, 0xab, 0x4a, 0xa3, 0xae, 0xb8, 0x70, 0x1b, 0x6c, 0x54, 0xaa, 0x4d, 0x16, 0x52, 0x3e, + 0x66, 0x60, 0x01, 0x6c, 0x9a, 0xef, 0x1b, 0x26, 0xaa, 0x14, 0x2d, 0xe5, 0x4f, 0xab, 0x67, 0xbf, + 0x05, 0x60, 0x6a, 0x92, 0x30, 0x07, 0x56, 0xaf, 0xdf, 0x2a, 0x2b, 0x30, 0x0f, 0xb2, 0x96, 0x59, + 0xac, 0x9b, 0x0a, 0xc3, 0xe7, 0x2b, 0xd5, 0x66, 0xbd, 0x81, 0xcc, 0x7a, 0x5d, 0x59, 0x85, 0x7b, + 0x60, 0xd7, 0xb4, 0xcc, 0x72, 0xe3, 0xaa, 0x5a, 0x69, 0xa2, 0x9b, 0x4a, 0xc5, 0x44, 0xca, 0x1a, + 0x54, 0xc0, 0xf6, 0xbb, 0x62, 0xa3, 0x7c, 0x39, 0x8e, 0xac, 0xb3, 0x9f, 0xd9, 0xaa, 0x96, 0xaf, + 0x9b, 0xa8, 0x58, 0x36, 0xd1, 0x38, 0x9c, 0x65, 0x40, 0xae, 0x3b, 0x8e, 0xe4, 0x5e, 0xbc, 0x04, + 0xf9, 0x06, 0xb1, 0x83, 0x38, 0x0a, 0x09, 0x85, 0x2f, 0xe4, 0xc5, 0x4e, 0x3a, 0x70, 0xe9, 0x9c, + 0x1f, 0xed, 0x4e, 0xd6, 0x62, 0xdc, 0xf4, 0x95, 0xd3, 0xcc, 0x8f, 0x33, 0xa5, 0xfd, 0x8f, 0xff, + 0x3e, 0x5e, 0xf9, 0xf8, 0xf9, 0x38, 0xf3, 0x8f, 0xcf, 0xc7, 0x99, 0x7f, 0x7d, 0x3e, 0xce, 0xfc, + 0xe1, 0x3f, 0xc7, 0x2b, 0xad, 0x1c, 0xff, 0xff, 0xfd, 0xab, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, + 0x53, 0xa4, 0xc8, 0x19, 0x08, 0x10, 0x00, 0x00, +} diff --git a/tools/functional-tester/rpcpb/rpc.proto b/tools/functional-tester/rpcpb/rpc.proto new file mode 100644 index 000000000..3fa4a8688 --- /dev/null +++ b/tools/functional-tester/rpcpb/rpc.proto @@ -0,0 +1,187 @@ +syntax = "proto3"; +package rpcpb; + +import "github.com/gogo/protobuf/gogoproto/gogo.proto"; + +option (gogoproto.marshaler_all) = true; +option (gogoproto.sizer_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.goproto_getters_all) = false; + +service Transport { + rpc Transport(stream Request) returns (stream Response) {} +} + +enum Operation { + NotStarted = 0; + + // InitialStartEtcd is only called to start etcd very first time. + InitialStartEtcd = 1; + // RestartEtcd is sent to restart killed etcd. + RestartEtcd = 2; + // KillEtcd pauses etcd process while keeping data directories + // and previous etcd configurations. + KillEtcd = 3; + // FailArchive is sent when consistency check failed, + // thus need to archive etcd data directories. + FailArchive = 4; + // DestroyEtcdAgent destroys etcd process, etcd data, and agent server. + DestroyEtcdAgent = 5; + + BlackholePeerPortTxRx = 100; + UnblackholePeerPortTxRx = 101; + DelayPeerPortTxRx = 102; + UndelayPeerPortTxRx = 103; +} + +message Etcd { + string Name = 1 [(gogoproto.moretags) = "yaml:\"name\""]; + string DataDir = 2 [(gogoproto.moretags) = "yaml:\"data-dir\""]; + string WALDir = 3 [(gogoproto.moretags) = "yaml:\"wal-dir\""]; + repeated string ListenClientURLs = 4 [(gogoproto.moretags) = "yaml:\"listen-client-urls\""]; + repeated string AdvertiseClientURLs = 5 [(gogoproto.moretags) = "yaml:\"advertise-client-urls\""]; + repeated string ListenPeerURLs = 6 [(gogoproto.moretags) = "yaml:\"listen-peer-urls\""]; + repeated string InitialAdvertisePeerURLs = 7 [(gogoproto.moretags) = "yaml:\"initial-advertise-peer-urls\""]; + string InitialCluster = 8 [(gogoproto.moretags) = "yaml:\"initial-cluster\""]; + string InitialClusterState = 9 [(gogoproto.moretags) = "yaml:\"initial-cluster-state\""]; + string InitialClusterToken = 10 [(gogoproto.moretags) = "yaml:\"initial-cluster-token\""]; + int64 SnapshotCount = 11 [(gogoproto.moretags) = "yaml:\"snapshot-count\""]; + int64 QuotaBackendBytes = 12 [(gogoproto.moretags) = "yaml:\"quota-backend-bytes\""]; + bool PreVote = 13 [(gogoproto.moretags) = "yaml:\"pre-vote\""]; + bool InitialCorruptCheck = 14 [(gogoproto.moretags) = "yaml:\"initial-corrupt-check\""]; + + // TODO: support TLS +} + +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. + string AgentAddr = 11 [(gogoproto.moretags) = "yaml:\"agent-addr\""]; + // FailpointHTTPAddr is the agent's failpoints HTTP server address. + string FailpointHTTPAddr = 12 [(gogoproto.moretags) = "yaml:\"failpoint-http-addr\""]; + + // BaseDir is the base directory where all logs and etcd data are stored. + string BaseDir = 101 [(gogoproto.moretags) = "yaml:\"base-dir\""]; + // EtcdLogPath is the log file to store current etcd server logs. + string EtcdLogPath = 102 [(gogoproto.moretags) = "yaml:\"etcd-log-path\""]; + + // EtcdClientTLS is true when client traffic needs to be encrypted. + bool EtcdClientTLS = 201 [(gogoproto.moretags) = "yaml:\"etcd-client-tls\""]; + // EtcdClientProxy is true when client traffic needs to be proxied. + // If true, listen client URL port must be different than advertise client URL port. + bool EtcdClientProxy = 202 [(gogoproto.moretags) = "yaml:\"etcd-client-proxy\""]; + // EtcdPeerProxy is true when peer traffic needs to be proxied. + // If true, listen peer URL port must be different than advertise peer URL port. + bool EtcdPeerProxy = 203 [(gogoproto.moretags) = "yaml:\"etcd-peer-proxy\""]; + // EtcdClientEndpoint is the etcd client endpoint. + string EtcdClientEndpoint = 204 [(gogoproto.moretags) = "yaml:\"etcd-client-endpoint\""]; + + // Etcd defines etcd binary configuration flags. + Etcd Etcd = 301 [(gogoproto.moretags) = "yaml:\"etcd-config\""]; +} + +enum FailureCase { + KILL_ONE_FOLLOWER = 0; + KILL_LEADER = 1; + KILL_ONE_FOLLOWER_FOR_LONG = 2; + KILL_LEADER_FOR_LONG = 3; + KILL_QUORUM = 4; + KILL_ALL = 5; + + BLACKHOLE_PEER_PORT_TX_RX_ONE_FOLLOWER = 6; + BLACKHOLE_PEER_PORT_TX_RX_LEADER_ONE = 7; + BLACKHOLE_PEER_PORT_TX_RX_ALL = 8; + + DELAY_PEER_PORT_TX_RX_ONE_FOLLOWER = 9; + DELAY_PEER_PORT_TX_RX_LEADER = 10; + DELAY_PEER_PORT_TX_RX_ALL = 11; + + FAILPOINTS = 100; + + NO_FAIL = 200; + // TODO: support no-op of liveness duration + // NO_FAIL_LIVENESS = 201; + + EXTERNAL = 300; +} + +enum StressType { + KV = 0; + LEASE = 1; + NO_STRESS = 2; + ELECTION_RUNNER = 3; + WATCH_RUNNER = 4; + LOCK_RACER_RUNNER = 5; + LEASE_RUNNER = 6; +} + +message Tester { + string TesterNetwork = 1 [(gogoproto.moretags) = "yaml:\"tester-network\""]; + string TesterAddr = 2 [(gogoproto.moretags) = "yaml:\"tester-addr\""]; + + // DelayLatencyMsRv is the delay latency in milliseconds, + // to inject to simulated slow network. + uint32 DelayLatencyMs = 11 [(gogoproto.moretags) = "yaml:\"delay-latency-ms\""]; + // DelayLatencyMsRv is the delay latency random variable in milliseconds. + uint32 DelayLatencyMsRv = 12 [(gogoproto.moretags) = "yaml:\"delay-latency-ms-rv\""]; + + // 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\""]; + // EnablePprof is true to enable profiler. + bool EnablePprof = 24 [(gogoproto.moretags) = "yaml:\"enable-pprof\""]; + + // FailureCases is the selected test cases to schedule. + // If empty, run all failure cases. + // TODO: support no-op + repeated string FailureCases = 31 [(gogoproto.moretags) = "yaml:\"failure-cases\""]; + // FailureShuffle is true to randomize failure injecting order. + // TODO: support shuffle + // bool FailureShuffle = 32 [(gogoproto.moretags) = "yaml:\"failure-shuffle\""]; + // FailpointCommands is the list of "gofail" commands (e.g. panic("etcd-tester"),1*sleep(1000)). + repeated string FailpointCommands = 33 [(gogoproto.moretags) = "yaml:\"failpoint-commands\""]; + + // RunnerExecPath is a path of etcd-runner binary. + string RunnerExecPath = 41 [(gogoproto.moretags) = "yaml:\"runner-exec-path\""]; + // 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\""]; + // StressKeySize is the size of each small key written into etcd. + int32 StressKeySize = 102 [(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\""]; + // 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\""]; + // 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\""]; + // StressKeyTxnOps is the number of operations per a transaction (max 64). + int32 StressKeyTxnOps = 106 [(gogoproto.moretags) = "yaml:\"stress-key-txn-ops\""]; + // StressQPS is the maximum number of stresser requests per second. + int32 StressQPS = 107 [(gogoproto.moretags) = "yaml:\"stress-qps\""]; +} + +message Request { + Operation Operation = 1; + + Member Member = 2; + Tester Tester = 3; +} + +message Response { + bool Success = 1; + string Status = 2; + + // TODO: support TLS +}