cluster.go 9.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404
  1. // Copyright 2015 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 main
  15. import (
  16. "fmt"
  17. "math/rand"
  18. "net"
  19. "strings"
  20. "time"
  21. "golang.org/x/net/context"
  22. "google.golang.org/grpc"
  23. clientv2 "github.com/coreos/etcd/client"
  24. "github.com/coreos/etcd/clientv3"
  25. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  26. "github.com/coreos/etcd/tools/functional-tester/etcd-agent/client"
  27. )
  28. const peerURLPort = 2380
  29. type cluster struct {
  30. v2Only bool // to be deprecated
  31. agentEndpoints []string
  32. datadir string
  33. stressKeySize int
  34. stressKeySuffixRange int
  35. Size int
  36. Agents []client.Agent
  37. Stressers []Stresser
  38. Names []string
  39. GRPCURLs []string
  40. ClientURLs []string
  41. }
  42. type ClusterStatus struct {
  43. AgentStatuses map[string]client.Status
  44. }
  45. // newCluster starts and returns a new cluster. The caller should call Terminate when finished, to shut it down.
  46. func newCluster(agentEndpoints []string, datadir string, stressKeySize, stressKeySuffixRange int, isV2Only bool) (*cluster, error) {
  47. c := &cluster{
  48. v2Only: isV2Only,
  49. agentEndpoints: agentEndpoints,
  50. datadir: datadir,
  51. stressKeySize: stressKeySize,
  52. stressKeySuffixRange: stressKeySuffixRange,
  53. }
  54. if err := c.Bootstrap(); err != nil {
  55. return nil, err
  56. }
  57. return c, nil
  58. }
  59. func (c *cluster) Bootstrap() error {
  60. size := len(c.agentEndpoints)
  61. agents := make([]client.Agent, size)
  62. names := make([]string, size)
  63. grpcURLs := make([]string, size)
  64. clientURLs := make([]string, size)
  65. peerURLs := make([]string, size)
  66. members := make([]string, size)
  67. for i, u := range c.agentEndpoints {
  68. var err error
  69. agents[i], err = client.NewAgent(u)
  70. if err != nil {
  71. return err
  72. }
  73. names[i] = fmt.Sprintf("etcd-%d", i)
  74. host, _, err := net.SplitHostPort(u)
  75. if err != nil {
  76. return err
  77. }
  78. grpcURLs[i] = fmt.Sprintf("%s:2379", host)
  79. clientURLs[i] = fmt.Sprintf("http://%s:2379", host)
  80. peerURLs[i] = fmt.Sprintf("http://%s:%d", host, peerURLPort)
  81. members[i] = fmt.Sprintf("%s=%s", names[i], peerURLs[i])
  82. }
  83. clusterStr := strings.Join(members, ",")
  84. token := fmt.Sprint(rand.Int())
  85. for i, a := range agents {
  86. flags := []string{
  87. "--name", names[i],
  88. "--data-dir", c.datadir,
  89. "--listen-client-urls", clientURLs[i],
  90. "--advertise-client-urls", clientURLs[i],
  91. "--listen-peer-urls", peerURLs[i],
  92. "--initial-advertise-peer-urls", peerURLs[i],
  93. "--initial-cluster-token", token,
  94. "--initial-cluster", clusterStr,
  95. "--initial-cluster-state", "new",
  96. }
  97. if _, err := a.Start(flags...); err != nil {
  98. // cleanup
  99. for j := 0; j < i; j++ {
  100. agents[j].Terminate()
  101. }
  102. return err
  103. }
  104. }
  105. // TODO: Too intensive stressers can panic etcd member with
  106. // 'out of memory' error. Put rate limits in server side.
  107. stressN := 100
  108. var stressers []Stresser
  109. if c.v2Only {
  110. for _, u := range clientURLs {
  111. s := &stresserV2{
  112. Endpoint: u,
  113. KeySize: c.stressKeySize,
  114. KeySuffixRange: c.stressKeySuffixRange,
  115. N: stressN,
  116. }
  117. go s.Stress()
  118. stressers = append(stressers, s)
  119. }
  120. } else {
  121. for _, u := range grpcURLs {
  122. s := &stresser{
  123. Endpoint: u,
  124. KeySize: c.stressKeySize,
  125. KeySuffixRange: c.stressKeySuffixRange,
  126. N: stressN,
  127. }
  128. go s.Stress()
  129. stressers = append(stressers, s)
  130. }
  131. }
  132. c.Size = size
  133. c.Agents = agents
  134. c.Stressers = stressers
  135. c.Names = names
  136. c.GRPCURLs = grpcURLs
  137. c.ClientURLs = clientURLs
  138. return nil
  139. }
  140. func (c *cluster) WaitHealth() error {
  141. var err error
  142. // wait 60s to check cluster health.
  143. // TODO: set it to a reasonable value. It is set that high because
  144. // follower may use long time to catch up the leader when reboot under
  145. // reasonable workload (https://github.com/coreos/etcd/issues/2698)
  146. healthFunc, urls := setHealthKey, c.GRPCURLs
  147. if c.v2Only {
  148. healthFunc, urls = setHealthKeyV2, c.ClientURLs
  149. }
  150. for i := 0; i < 60; i++ {
  151. err = healthFunc(urls)
  152. if err == nil {
  153. return nil
  154. }
  155. plog.Warningf("#%d setHealthKey error (%v)", i, err)
  156. time.Sleep(time.Second)
  157. }
  158. return err
  159. }
  160. // GetLeader returns the index of leader and error if any.
  161. func (c *cluster) GetLeader() (int, error) {
  162. if c.v2Only {
  163. return 0, nil
  164. }
  165. for i, ep := range c.GRPCURLs {
  166. cli, err := clientv3.New(clientv3.Config{
  167. Endpoints: []string{ep},
  168. DialTimeout: 5 * time.Second,
  169. })
  170. if err != nil {
  171. return 0, err
  172. }
  173. defer cli.Close()
  174. mapi := clientv3.NewMaintenance(cli)
  175. resp, err := mapi.Status(context.Background(), ep)
  176. if err != nil {
  177. return 0, err
  178. }
  179. if resp.Header.MemberId == resp.Leader {
  180. return i, nil
  181. }
  182. }
  183. return 0, fmt.Errorf("no leader found")
  184. }
  185. func (c *cluster) Report() (success, failure int) {
  186. for _, stress := range c.Stressers {
  187. s, f := stress.Report()
  188. success += s
  189. failure += f
  190. }
  191. return
  192. }
  193. func (c *cluster) Cleanup() error {
  194. var lasterr error
  195. for _, a := range c.Agents {
  196. if err := a.Cleanup(); err != nil {
  197. lasterr = err
  198. }
  199. }
  200. for _, s := range c.Stressers {
  201. s.Cancel()
  202. }
  203. return lasterr
  204. }
  205. func (c *cluster) Terminate() {
  206. for _, a := range c.Agents {
  207. a.Terminate()
  208. }
  209. for _, s := range c.Stressers {
  210. s.Cancel()
  211. }
  212. }
  213. func (c *cluster) Status() ClusterStatus {
  214. cs := ClusterStatus{
  215. AgentStatuses: make(map[string]client.Status),
  216. }
  217. for i, a := range c.Agents {
  218. s, err := a.Status()
  219. // TODO: add a.Desc() as a key of the map
  220. desc := c.agentEndpoints[i]
  221. if err != nil {
  222. cs.AgentStatuses[desc] = client.Status{State: "unknown"}
  223. plog.Printf("failed to get the status of agent [%s]", desc)
  224. }
  225. cs.AgentStatuses[desc] = s
  226. }
  227. return cs
  228. }
  229. // setHealthKey sets health key on all given urls.
  230. func setHealthKey(us []string) error {
  231. for _, u := range us {
  232. conn, err := grpc.Dial(u, grpc.WithInsecure(), grpc.WithTimeout(5*time.Second))
  233. if err != nil {
  234. return fmt.Errorf("%v (%s)", err, u)
  235. }
  236. ctx, cancel := context.WithTimeout(context.Background(), time.Second)
  237. kvc := pb.NewKVClient(conn)
  238. _, err = kvc.Put(ctx, &pb.PutRequest{Key: []byte("health"), Value: []byte("good")})
  239. cancel()
  240. conn.Close()
  241. if err != nil {
  242. return fmt.Errorf("%v (%s)", err, u)
  243. }
  244. }
  245. return nil
  246. }
  247. // setHealthKeyV2 sets health key on all given urls.
  248. func setHealthKeyV2(us []string) error {
  249. for _, u := range us {
  250. cfg := clientv2.Config{
  251. Endpoints: []string{u},
  252. }
  253. c, err := clientv2.New(cfg)
  254. if err != nil {
  255. return err
  256. }
  257. ctx, cancel := context.WithTimeout(context.Background(), time.Second)
  258. kapi := clientv2.NewKeysAPI(c)
  259. _, err = kapi.Set(ctx, "health", "good", nil)
  260. cancel()
  261. if err != nil {
  262. return err
  263. }
  264. }
  265. return nil
  266. }
  267. func (c *cluster) getRevisionHash() (map[string]int64, map[string]int64, error) {
  268. revs := make(map[string]int64)
  269. hashes := make(map[string]int64)
  270. for _, u := range c.GRPCURLs {
  271. conn, err := grpc.Dial(u, grpc.WithInsecure(), grpc.WithTimeout(5*time.Second))
  272. if err != nil {
  273. return nil, nil, err
  274. }
  275. m := pb.NewMaintenanceClient(conn)
  276. ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
  277. resp, err := m.Hash(ctx, &pb.HashRequest{})
  278. cancel()
  279. conn.Close()
  280. if err != nil {
  281. return nil, nil, err
  282. }
  283. revs[u] = resp.Header.Revision
  284. hashes[u] = int64(resp.Hash)
  285. }
  286. return revs, hashes, nil
  287. }
  288. func (c *cluster) compactKV(rev int64, timeout time.Duration) (err error) {
  289. if rev <= 0 {
  290. return nil
  291. }
  292. for i, u := range c.GRPCURLs {
  293. conn, derr := grpc.Dial(u, grpc.WithInsecure(), grpc.WithTimeout(5*time.Second))
  294. if derr != nil {
  295. plog.Printf("[compact kv #%d] dial error %v (endpoint %s)", i, derr, u)
  296. err = derr
  297. continue
  298. }
  299. kvc := pb.NewKVClient(conn)
  300. ctx, cancel := context.WithTimeout(context.Background(), timeout)
  301. plog.Printf("[compact kv #%d] starting (endpoint %s)", i, u)
  302. _, cerr := kvc.Compact(ctx, &pb.CompactionRequest{Revision: rev, Physical: true})
  303. cancel()
  304. conn.Close()
  305. succeed := true
  306. if cerr != nil {
  307. if strings.Contains(cerr.Error(), "required revision has been compacted") && i > 0 {
  308. plog.Printf("[compact kv #%d] already compacted (endpoint %s)", i, u)
  309. } else {
  310. plog.Warningf("[compact kv #%d] error %v (endpoint %s)", i, cerr, u)
  311. err = cerr
  312. succeed = false
  313. }
  314. }
  315. if succeed {
  316. plog.Printf("[compact kv #%d] done (endpoint %s)", i, u)
  317. }
  318. }
  319. return err
  320. }
  321. func (c *cluster) checkCompact(rev int64) error {
  322. if rev == 0 {
  323. return nil
  324. }
  325. for _, u := range c.GRPCURLs {
  326. cli, err := clientv3.New(clientv3.Config{
  327. Endpoints: []string{u},
  328. DialTimeout: 5 * time.Second,
  329. })
  330. if err != nil {
  331. return fmt.Errorf("%v (endpoint %s)", err, u)
  332. }
  333. ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
  334. wch := cli.Watch(ctx, "\x00", clientv3.WithFromKey(), clientv3.WithRev(rev-1))
  335. wr, ok := <-wch
  336. cancel()
  337. cli.Close()
  338. if !ok {
  339. return fmt.Errorf("watch channel terminated (endpoint %s)", u)
  340. }
  341. if wr.CompactRevision != rev {
  342. return fmt.Errorf("got compact revision %v, wanted %v (endpoint %s)", wr.CompactRevision, rev, u)
  343. }
  344. }
  345. return nil
  346. }
  347. func (c *cluster) defrag() error {
  348. for _, u := range c.GRPCURLs {
  349. plog.Printf("defragmenting %s\n", u)
  350. conn, err := grpc.Dial(u, grpc.WithInsecure(), grpc.WithTimeout(5*time.Second))
  351. if err != nil {
  352. return err
  353. }
  354. mt := pb.NewMaintenanceClient(conn)
  355. if _, err = mt.Defragment(context.Background(), &pb.DefragmentRequest{}); err != nil {
  356. return err
  357. }
  358. conn.Close()
  359. plog.Printf("defragmented %s\n", u)
  360. }
  361. return nil
  362. }