stress_key.go 9.1 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344
  1. // Copyright 2018 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package tester
  15. import (
  16. "context"
  17. "fmt"
  18. "math/rand"
  19. "sync"
  20. "sync/atomic"
  21. "time"
  22. "github.com/coreos/etcd/etcdserver"
  23. "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
  24. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  25. "go.uber.org/zap"
  26. "golang.org/x/time/rate"
  27. "google.golang.org/grpc"
  28. "google.golang.org/grpc/transport"
  29. )
  30. type keyStresser struct {
  31. lg *zap.Logger
  32. Endpoint string // TODO: use Member
  33. keySize int
  34. keyLargeSize int
  35. keySuffixRange int
  36. keyTxnSuffixRange int
  37. keyTxnOps int
  38. N int
  39. rateLimiter *rate.Limiter
  40. wg sync.WaitGroup
  41. cancel func()
  42. conn *grpc.ClientConn
  43. // atomicModifiedKeys records the number of keys created and deleted by the stresser.
  44. atomicModifiedKeys int64
  45. stressTable *stressTable
  46. }
  47. func (s *keyStresser) Stress() error {
  48. // TODO: add backoff option
  49. conn, err := grpc.Dial(s.Endpoint, grpc.WithInsecure())
  50. if err != nil {
  51. return fmt.Errorf("%v (%s)", err, s.Endpoint)
  52. }
  53. ctx, cancel := context.WithCancel(context.Background())
  54. s.wg.Add(s.N)
  55. s.conn = conn
  56. s.cancel = cancel
  57. kvc := pb.NewKVClient(conn)
  58. var stressEntries = []stressEntry{
  59. {weight: 0.7, f: newStressPut(kvc, s.keySuffixRange, s.keySize)},
  60. {
  61. weight: 0.7 * float32(s.keySize) / float32(s.keyLargeSize),
  62. f: newStressPut(kvc, s.keySuffixRange, s.keyLargeSize),
  63. },
  64. {weight: 0.07, f: newStressRange(kvc, s.keySuffixRange)},
  65. {weight: 0.07, f: newStressRangeInterval(kvc, s.keySuffixRange)},
  66. {weight: 0.07, f: newStressDelete(kvc, s.keySuffixRange)},
  67. {weight: 0.07, f: newStressDeleteInterval(kvc, s.keySuffixRange)},
  68. }
  69. if s.keyTxnSuffixRange > 0 {
  70. // adjust to make up ±70% of workloads with writes
  71. stressEntries[0].weight = 0.35
  72. stressEntries = append(stressEntries, stressEntry{
  73. weight: 0.35,
  74. f: newStressTxn(kvc, s.keyTxnSuffixRange, s.keyTxnOps),
  75. })
  76. }
  77. s.stressTable = createStressTable(stressEntries)
  78. for i := 0; i < s.N; i++ {
  79. go s.run(ctx)
  80. }
  81. s.lg.Info(
  82. "key stresser started in background",
  83. zap.String("endpoint", s.Endpoint),
  84. )
  85. return nil
  86. }
  87. func (s *keyStresser) run(ctx context.Context) {
  88. defer s.wg.Done()
  89. for {
  90. if err := s.rateLimiter.Wait(ctx); err == context.Canceled {
  91. return
  92. }
  93. // TODO: 10-second is enough timeout to cover leader failure
  94. // and immediate leader election. Find out what other cases this
  95. // could be timed out.
  96. sctx, scancel := context.WithTimeout(ctx, 10*time.Second)
  97. err, modifiedKeys := s.stressTable.choose()(sctx)
  98. scancel()
  99. if err == nil {
  100. atomic.AddInt64(&s.atomicModifiedKeys, modifiedKeys)
  101. continue
  102. }
  103. switch rpctypes.ErrorDesc(err) {
  104. case context.DeadlineExceeded.Error():
  105. // This retries when request is triggered at the same time as
  106. // leader failure. When we terminate the leader, the request to
  107. // that leader cannot be processed, and times out. Also requests
  108. // to followers cannot be forwarded to the old leader, so timing out
  109. // as well. We want to keep stressing until the cluster elects a
  110. // new leader and start processing requests again.
  111. case etcdserver.ErrTimeoutDueToLeaderFail.Error(), etcdserver.ErrTimeout.Error():
  112. // This retries when request is triggered at the same time as
  113. // leader failure and follower nodes receive time out errors
  114. // from losing their leader. Followers should retry to connect
  115. // to the new leader.
  116. case etcdserver.ErrStopped.Error():
  117. // one of the etcd nodes stopped from failure injection
  118. case transport.ErrConnClosing.Desc:
  119. // server closed the transport (failure injected node)
  120. case rpctypes.ErrNotCapable.Error():
  121. // capability check has not been done (in the beginning)
  122. case rpctypes.ErrTooManyRequests.Error():
  123. // hitting the recovering member.
  124. case context.Canceled.Error():
  125. // from stresser.Cancel method:
  126. return
  127. case grpc.ErrClientConnClosing.Error():
  128. // from stresser.Cancel method:
  129. return
  130. default:
  131. s.lg.Warn(
  132. "key stresser exited with error",
  133. zap.String("endpoint", s.Endpoint),
  134. zap.Error(err),
  135. )
  136. return
  137. }
  138. }
  139. }
  140. func (s *keyStresser) Pause() {
  141. s.Close()
  142. }
  143. func (s *keyStresser) Close() {
  144. s.cancel()
  145. s.conn.Close()
  146. s.wg.Wait()
  147. s.lg.Info(
  148. "key stresser is closed",
  149. zap.String("endpoint", s.Endpoint),
  150. )
  151. }
  152. func (s *keyStresser) ModifiedKeys() int64 {
  153. return atomic.LoadInt64(&s.atomicModifiedKeys)
  154. }
  155. func (s *keyStresser) Checker() Checker { return nil }
  156. type stressFunc func(ctx context.Context) (err error, modifiedKeys int64)
  157. type stressEntry struct {
  158. weight float32
  159. f stressFunc
  160. }
  161. type stressTable struct {
  162. entries []stressEntry
  163. sumWeights float32
  164. }
  165. func createStressTable(entries []stressEntry) *stressTable {
  166. st := stressTable{entries: entries}
  167. for _, entry := range st.entries {
  168. st.sumWeights += entry.weight
  169. }
  170. return &st
  171. }
  172. func (st *stressTable) choose() stressFunc {
  173. v := rand.Float32() * st.sumWeights
  174. var sum float32
  175. var idx int
  176. for i := range st.entries {
  177. sum += st.entries[i].weight
  178. if sum >= v {
  179. idx = i
  180. break
  181. }
  182. }
  183. return st.entries[idx].f
  184. }
  185. func newStressPut(kvc pb.KVClient, keySuffixRange, keySize int) stressFunc {
  186. return func(ctx context.Context) (error, int64) {
  187. _, err := kvc.Put(ctx, &pb.PutRequest{
  188. Key: []byte(fmt.Sprintf("foo%016x", rand.Intn(keySuffixRange))),
  189. Value: randBytes(keySize),
  190. }, grpc.FailFast(false))
  191. return err, 1
  192. }
  193. }
  194. func newStressTxn(kvc pb.KVClient, keyTxnSuffixRange, txnOps int) stressFunc {
  195. keys := make([]string, keyTxnSuffixRange)
  196. for i := range keys {
  197. keys[i] = fmt.Sprintf("/k%03d", i)
  198. }
  199. return writeTxn(kvc, keys, txnOps)
  200. }
  201. func writeTxn(kvc pb.KVClient, keys []string, txnOps int) stressFunc {
  202. return func(ctx context.Context) (error, int64) {
  203. ks := make(map[string]struct{}, txnOps)
  204. for len(ks) != txnOps {
  205. ks[keys[rand.Intn(len(keys))]] = struct{}{}
  206. }
  207. selected := make([]string, 0, txnOps)
  208. for k := range ks {
  209. selected = append(selected, k)
  210. }
  211. com, delOp, putOp := getTxnReqs(selected[0], "bar00")
  212. txnReq := &pb.TxnRequest{
  213. Compare: []*pb.Compare{com},
  214. Success: []*pb.RequestOp{delOp},
  215. Failure: []*pb.RequestOp{putOp},
  216. }
  217. // add nested txns if any
  218. for i := 1; i < txnOps; i++ {
  219. k, v := selected[i], fmt.Sprintf("bar%02d", i)
  220. com, delOp, putOp = getTxnReqs(k, v)
  221. nested := &pb.RequestOp{
  222. Request: &pb.RequestOp_RequestTxn{
  223. RequestTxn: &pb.TxnRequest{
  224. Compare: []*pb.Compare{com},
  225. Success: []*pb.RequestOp{delOp},
  226. Failure: []*pb.RequestOp{putOp},
  227. },
  228. },
  229. }
  230. txnReq.Success = append(txnReq.Success, nested)
  231. txnReq.Failure = append(txnReq.Failure, nested)
  232. }
  233. _, err := kvc.Txn(ctx, txnReq, grpc.FailFast(false))
  234. return err, int64(txnOps)
  235. }
  236. }
  237. func getTxnReqs(key, val string) (com *pb.Compare, delOp *pb.RequestOp, putOp *pb.RequestOp) {
  238. // if key exists (version > 0)
  239. com = &pb.Compare{
  240. Key: []byte(key),
  241. Target: pb.Compare_VERSION,
  242. Result: pb.Compare_GREATER,
  243. TargetUnion: &pb.Compare_Version{Version: 0},
  244. }
  245. delOp = &pb.RequestOp{
  246. Request: &pb.RequestOp_RequestDeleteRange{
  247. RequestDeleteRange: &pb.DeleteRangeRequest{
  248. Key: []byte(key),
  249. },
  250. },
  251. }
  252. putOp = &pb.RequestOp{
  253. Request: &pb.RequestOp_RequestPut{
  254. RequestPut: &pb.PutRequest{
  255. Key: []byte(key),
  256. Value: []byte(val),
  257. },
  258. },
  259. }
  260. return com, delOp, putOp
  261. }
  262. func newStressRange(kvc pb.KVClient, keySuffixRange int) stressFunc {
  263. return func(ctx context.Context) (error, int64) {
  264. _, err := kvc.Range(ctx, &pb.RangeRequest{
  265. Key: []byte(fmt.Sprintf("foo%016x", rand.Intn(keySuffixRange))),
  266. }, grpc.FailFast(false))
  267. return err, 0
  268. }
  269. }
  270. func newStressRangeInterval(kvc pb.KVClient, keySuffixRange int) stressFunc {
  271. return func(ctx context.Context) (error, int64) {
  272. start := rand.Intn(keySuffixRange)
  273. end := start + 500
  274. _, err := kvc.Range(ctx, &pb.RangeRequest{
  275. Key: []byte(fmt.Sprintf("foo%016x", start)),
  276. RangeEnd: []byte(fmt.Sprintf("foo%016x", end)),
  277. }, grpc.FailFast(false))
  278. return err, 0
  279. }
  280. }
  281. func newStressDelete(kvc pb.KVClient, keySuffixRange int) stressFunc {
  282. return func(ctx context.Context) (error, int64) {
  283. _, err := kvc.DeleteRange(ctx, &pb.DeleteRangeRequest{
  284. Key: []byte(fmt.Sprintf("foo%016x", rand.Intn(keySuffixRange))),
  285. }, grpc.FailFast(false))
  286. return err, 1
  287. }
  288. }
  289. func newStressDeleteInterval(kvc pb.KVClient, keySuffixRange int) stressFunc {
  290. return func(ctx context.Context) (error, int64) {
  291. start := rand.Intn(keySuffixRange)
  292. end := start + 500
  293. resp, err := kvc.DeleteRange(ctx, &pb.DeleteRangeRequest{
  294. Key: []byte(fmt.Sprintf("foo%016x", start)),
  295. RangeEnd: []byte(fmt.Sprintf("foo%016x", end)),
  296. }, grpc.FailFast(false))
  297. if err == nil {
  298. return nil, resp.Deleted
  299. }
  300. return err, 0
  301. }
  302. }