watch_test.go 27 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974
  1. // Copyright 2016 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 integration
  15. import (
  16. "fmt"
  17. "math/rand"
  18. "reflect"
  19. "sort"
  20. "testing"
  21. "time"
  22. "github.com/coreos/etcd/clientv3"
  23. "github.com/coreos/etcd/etcdserver/api/v3rpc"
  24. "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
  25. "github.com/coreos/etcd/integration"
  26. mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
  27. "github.com/coreos/etcd/pkg/testutil"
  28. "golang.org/x/net/context"
  29. "google.golang.org/grpc"
  30. )
  31. type watcherTest func(*testing.T, *watchctx)
  32. type watchctx struct {
  33. clus *integration.ClusterV3
  34. w clientv3.Watcher
  35. kv clientv3.KV
  36. wclientMember int
  37. kvMember int
  38. ch clientv3.WatchChan
  39. }
  40. func runWatchTest(t *testing.T, f watcherTest) {
  41. defer testutil.AfterTest(t)
  42. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  43. defer clus.Terminate(t)
  44. wclientMember := rand.Intn(3)
  45. w := clus.Client(wclientMember).Watcher
  46. // select a different client from wclient so puts succeed if
  47. // a test knocks out the watcher client
  48. kvMember := rand.Intn(3)
  49. for kvMember == wclientMember {
  50. kvMember = rand.Intn(3)
  51. }
  52. kv := clus.Client(kvMember).KV
  53. wctx := &watchctx{clus, w, kv, wclientMember, kvMember, nil}
  54. f(t, wctx)
  55. }
  56. // TestWatchMultiWatcher modifies multiple keys and observes the changes.
  57. func TestWatchMultiWatcher(t *testing.T) {
  58. runWatchTest(t, testWatchMultiWatcher)
  59. }
  60. func testWatchMultiWatcher(t *testing.T, wctx *watchctx) {
  61. numKeyUpdates := 4
  62. keys := []string{"foo", "bar", "baz"}
  63. donec := make(chan struct{})
  64. readyc := make(chan struct{})
  65. for _, k := range keys {
  66. // key watcher
  67. go func(key string) {
  68. ch := wctx.w.Watch(context.TODO(), key)
  69. if ch == nil {
  70. t.Fatalf("expected watcher channel, got nil")
  71. }
  72. readyc <- struct{}{}
  73. for i := 0; i < numKeyUpdates; i++ {
  74. resp, ok := <-ch
  75. if !ok {
  76. t.Fatalf("watcher unexpectedly closed")
  77. }
  78. v := fmt.Sprintf("%s-%d", key, i)
  79. gotv := string(resp.Events[0].Kv.Value)
  80. if gotv != v {
  81. t.Errorf("#%d: got %s, wanted %s", i, gotv, v)
  82. }
  83. }
  84. donec <- struct{}{}
  85. }(k)
  86. }
  87. // prefix watcher on "b" (bar and baz)
  88. go func() {
  89. prefixc := wctx.w.Watch(context.TODO(), "b", clientv3.WithPrefix())
  90. if prefixc == nil {
  91. t.Fatalf("expected watcher channel, got nil")
  92. }
  93. readyc <- struct{}{}
  94. evs := []*clientv3.Event{}
  95. for i := 0; i < numKeyUpdates*2; i++ {
  96. resp, ok := <-prefixc
  97. if !ok {
  98. t.Fatalf("watcher unexpectedly closed")
  99. }
  100. evs = append(evs, resp.Events...)
  101. }
  102. // check response
  103. expected := []string{}
  104. bkeys := []string{"bar", "baz"}
  105. for _, k := range bkeys {
  106. for i := 0; i < numKeyUpdates; i++ {
  107. expected = append(expected, fmt.Sprintf("%s-%d", k, i))
  108. }
  109. }
  110. got := []string{}
  111. for _, ev := range evs {
  112. got = append(got, string(ev.Kv.Value))
  113. }
  114. sort.Strings(got)
  115. if !reflect.DeepEqual(expected, got) {
  116. t.Errorf("got %v, expected %v", got, expected)
  117. }
  118. // ensure no extra data
  119. select {
  120. case resp, ok := <-prefixc:
  121. if !ok {
  122. t.Fatalf("watcher unexpectedly closed")
  123. }
  124. t.Fatalf("unexpected event %+v", resp)
  125. case <-time.After(time.Second):
  126. }
  127. donec <- struct{}{}
  128. }()
  129. // wait for watcher bring up
  130. for i := 0; i < len(keys)+1; i++ {
  131. <-readyc
  132. }
  133. // generate events
  134. ctx := context.TODO()
  135. for i := 0; i < numKeyUpdates; i++ {
  136. for _, k := range keys {
  137. v := fmt.Sprintf("%s-%d", k, i)
  138. if _, err := wctx.kv.Put(ctx, k, v); err != nil {
  139. t.Fatal(err)
  140. }
  141. }
  142. }
  143. // wait for watcher shutdown
  144. for i := 0; i < len(keys)+1; i++ {
  145. <-donec
  146. }
  147. }
  148. // TestWatchRange tests watcher creates ranges
  149. func TestWatchRange(t *testing.T) {
  150. runWatchTest(t, testWatchRange)
  151. }
  152. func testWatchRange(t *testing.T, wctx *watchctx) {
  153. if wctx.ch = wctx.w.Watch(context.TODO(), "a", clientv3.WithRange("c")); wctx.ch == nil {
  154. t.Fatalf("expected non-nil channel")
  155. }
  156. putAndWatch(t, wctx, "a", "a")
  157. putAndWatch(t, wctx, "b", "b")
  158. putAndWatch(t, wctx, "bar", "bar")
  159. }
  160. // TestWatchReconnRequest tests the send failure path when requesting a watcher.
  161. func TestWatchReconnRequest(t *testing.T) {
  162. runWatchTest(t, testWatchReconnRequest)
  163. }
  164. func testWatchReconnRequest(t *testing.T, wctx *watchctx) {
  165. donec, stopc := make(chan struct{}), make(chan struct{}, 1)
  166. go func() {
  167. timer := time.After(2 * time.Second)
  168. defer close(donec)
  169. // take down watcher connection
  170. for {
  171. wctx.clus.Members[wctx.wclientMember].DropConnections()
  172. select {
  173. case <-timer:
  174. // spinning on close may live lock reconnection
  175. return
  176. case <-stopc:
  177. return
  178. default:
  179. }
  180. }
  181. }()
  182. // should reconnect when requesting watch
  183. if wctx.ch = wctx.w.Watch(context.TODO(), "a"); wctx.ch == nil {
  184. t.Fatalf("expected non-nil channel")
  185. }
  186. // wait for disconnections to stop
  187. stopc <- struct{}{}
  188. <-donec
  189. // spinning on dropping connections may trigger a leader election
  190. // due to resource starvation; l-read to ensure the cluster is stable
  191. ctx, cancel := context.WithTimeout(context.TODO(), 30*time.Second)
  192. if _, err := wctx.kv.Get(ctx, "_"); err != nil {
  193. t.Fatal(err)
  194. }
  195. cancel()
  196. // ensure watcher works
  197. putAndWatch(t, wctx, "a", "a")
  198. }
  199. // TestWatchReconnInit tests watcher resumes correctly if connection lost
  200. // before any data was sent.
  201. func TestWatchReconnInit(t *testing.T) {
  202. runWatchTest(t, testWatchReconnInit)
  203. }
  204. func testWatchReconnInit(t *testing.T, wctx *watchctx) {
  205. if wctx.ch = wctx.w.Watch(context.TODO(), "a"); wctx.ch == nil {
  206. t.Fatalf("expected non-nil channel")
  207. }
  208. wctx.clus.Members[wctx.wclientMember].DropConnections()
  209. // watcher should recover
  210. putAndWatch(t, wctx, "a", "a")
  211. }
  212. // TestWatchReconnRunning tests watcher resumes correctly if connection lost
  213. // after data was sent.
  214. func TestWatchReconnRunning(t *testing.T) {
  215. runWatchTest(t, testWatchReconnRunning)
  216. }
  217. func testWatchReconnRunning(t *testing.T, wctx *watchctx) {
  218. if wctx.ch = wctx.w.Watch(context.TODO(), "a"); wctx.ch == nil {
  219. t.Fatalf("expected non-nil channel")
  220. }
  221. putAndWatch(t, wctx, "a", "a")
  222. // take down watcher connection
  223. wctx.clus.Members[wctx.wclientMember].DropConnections()
  224. // watcher should recover
  225. putAndWatch(t, wctx, "a", "b")
  226. }
  227. // TestWatchCancelImmediate ensures a closed channel is returned
  228. // if the context is cancelled.
  229. func TestWatchCancelImmediate(t *testing.T) {
  230. runWatchTest(t, testWatchCancelImmediate)
  231. }
  232. func testWatchCancelImmediate(t *testing.T, wctx *watchctx) {
  233. ctx, cancel := context.WithCancel(context.Background())
  234. cancel()
  235. wch := wctx.w.Watch(ctx, "a")
  236. select {
  237. case wresp, ok := <-wch:
  238. if ok {
  239. t.Fatalf("read wch got %v; expected closed channel", wresp)
  240. }
  241. default:
  242. t.Fatalf("closed watcher channel should not block")
  243. }
  244. }
  245. // TestWatchCancelInit tests watcher closes correctly after no events.
  246. func TestWatchCancelInit(t *testing.T) {
  247. runWatchTest(t, testWatchCancelInit)
  248. }
  249. func testWatchCancelInit(t *testing.T, wctx *watchctx) {
  250. ctx, cancel := context.WithCancel(context.Background())
  251. if wctx.ch = wctx.w.Watch(ctx, "a"); wctx.ch == nil {
  252. t.Fatalf("expected non-nil watcher channel")
  253. }
  254. cancel()
  255. select {
  256. case <-time.After(time.Second):
  257. t.Fatalf("took too long to cancel")
  258. case _, ok := <-wctx.ch:
  259. if ok {
  260. t.Fatalf("expected watcher channel to close")
  261. }
  262. }
  263. }
  264. // TestWatchCancelRunning tests watcher closes correctly after events.
  265. func TestWatchCancelRunning(t *testing.T) {
  266. runWatchTest(t, testWatchCancelRunning)
  267. }
  268. func testWatchCancelRunning(t *testing.T, wctx *watchctx) {
  269. ctx, cancel := context.WithCancel(context.Background())
  270. if wctx.ch = wctx.w.Watch(ctx, "a"); wctx.ch == nil {
  271. t.Fatalf("expected non-nil watcher channel")
  272. }
  273. if _, err := wctx.kv.Put(ctx, "a", "a"); err != nil {
  274. t.Fatal(err)
  275. }
  276. cancel()
  277. select {
  278. case <-time.After(time.Second):
  279. t.Fatalf("took too long to cancel")
  280. case v, ok := <-wctx.ch:
  281. if !ok {
  282. // closed before getting put; OK
  283. break
  284. }
  285. // got the PUT; should close next
  286. select {
  287. case <-time.After(time.Second):
  288. t.Fatalf("took too long to close")
  289. case v, ok = <-wctx.ch:
  290. if ok {
  291. t.Fatalf("expected watcher channel to close, got %v", v)
  292. }
  293. }
  294. }
  295. }
  296. func putAndWatch(t *testing.T, wctx *watchctx, key, val string) {
  297. if _, err := wctx.kv.Put(context.TODO(), key, val); err != nil {
  298. t.Fatal(err)
  299. }
  300. select {
  301. case <-time.After(5 * time.Second):
  302. t.Fatalf("watch timed out")
  303. case v, ok := <-wctx.ch:
  304. if !ok {
  305. t.Fatalf("unexpected watch close")
  306. }
  307. if string(v.Events[0].Kv.Value) != val {
  308. t.Fatalf("bad value got %v, wanted %v", v.Events[0].Kv.Value, val)
  309. }
  310. }
  311. }
  312. // TestWatchResumeComapcted checks that the watcher gracefully closes in case
  313. // that it tries to resume to a revision that's been compacted out of the store.
  314. // Since the watcher's server restarts with stale data, the watcher will receive
  315. // either a compaction error or all keys by staying in sync before the compaction
  316. // is finally applied.
  317. func TestWatchResumeCompacted(t *testing.T) {
  318. defer testutil.AfterTest(t)
  319. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  320. defer clus.Terminate(t)
  321. // create a waiting watcher at rev 1
  322. w := clus.Client(0)
  323. wch := w.Watch(context.Background(), "foo", clientv3.WithRev(1))
  324. select {
  325. case w := <-wch:
  326. t.Errorf("unexpected message from wch %v", w)
  327. default:
  328. }
  329. clus.Members[0].Stop(t)
  330. ticker := time.After(time.Second * 10)
  331. for clus.WaitLeader(t) <= 0 {
  332. select {
  333. case <-ticker:
  334. t.Fatalf("failed to wait for new leader")
  335. default:
  336. time.Sleep(10 * time.Millisecond)
  337. }
  338. }
  339. // put some data and compact away
  340. numPuts := 5
  341. kv := clus.Client(1)
  342. for i := 0; i < numPuts; i++ {
  343. if _, err := kv.Put(context.TODO(), "foo", "bar"); err != nil {
  344. t.Fatal(err)
  345. }
  346. }
  347. if _, err := kv.Compact(context.TODO(), 3); err != nil {
  348. t.Fatal(err)
  349. }
  350. clus.Members[0].Restart(t)
  351. // since watch's server isn't guaranteed to be synced with the cluster when
  352. // the watch resumes, there is a window where the watch can stay synced and
  353. // read off all events; if the watcher misses the window, it will go out of
  354. // sync and get a compaction error.
  355. wRev := int64(2)
  356. for int(wRev) <= numPuts+1 {
  357. var wresp clientv3.WatchResponse
  358. var ok bool
  359. select {
  360. case wresp, ok = <-wch:
  361. if !ok {
  362. t.Fatalf("expected wresp, but got closed channel")
  363. }
  364. case <-time.After(5 * time.Second):
  365. t.Fatalf("compacted watch timed out")
  366. }
  367. for _, ev := range wresp.Events {
  368. if ev.Kv.ModRevision != wRev {
  369. t.Fatalf("expected modRev %v, got %+v", wRev, ev)
  370. }
  371. wRev++
  372. }
  373. if wresp.Err() == nil {
  374. continue
  375. }
  376. if wresp.Err() != rpctypes.ErrCompacted {
  377. t.Fatalf("wresp.Err() expected %v, but got %v %+v", rpctypes.ErrCompacted, wresp.Err())
  378. }
  379. break
  380. }
  381. if int(wRev) > numPuts+1 {
  382. // got data faster than the compaction
  383. return
  384. }
  385. // received compaction error; ensure the channel closes
  386. select {
  387. case wresp, ok := <-wch:
  388. if ok {
  389. t.Fatalf("expected closed channel, but got %v", wresp)
  390. }
  391. case <-time.After(5 * time.Second):
  392. t.Fatalf("timed out waiting for channel close")
  393. }
  394. }
  395. // TestWatchCompactRevision ensures the CompactRevision error is given on a
  396. // compaction event ahead of a watcher.
  397. func TestWatchCompactRevision(t *testing.T) {
  398. defer testutil.AfterTest(t)
  399. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  400. defer clus.Terminate(t)
  401. // set some keys
  402. kv := clus.RandClient()
  403. for i := 0; i < 5; i++ {
  404. if _, err := kv.Put(context.TODO(), "foo", "bar"); err != nil {
  405. t.Fatal(err)
  406. }
  407. }
  408. w := clus.RandClient()
  409. if _, err := kv.Compact(context.TODO(), 4); err != nil {
  410. t.Fatal(err)
  411. }
  412. wch := w.Watch(context.Background(), "foo", clientv3.WithRev(2))
  413. // get compacted error message
  414. wresp, ok := <-wch
  415. if !ok {
  416. t.Fatalf("expected wresp, but got closed channel")
  417. }
  418. if wresp.Err() != rpctypes.ErrCompacted {
  419. t.Fatalf("wresp.Err() expected %v, but got %v", rpctypes.ErrCompacted, wresp.Err())
  420. }
  421. // ensure the channel is closed
  422. if wresp, ok = <-wch; ok {
  423. t.Fatalf("expected closed channel, but got %v", wresp)
  424. }
  425. }
  426. func TestWatchWithProgressNotify(t *testing.T) { testWatchWithProgressNotify(t, true) }
  427. func TestWatchWithProgressNotifyNoEvent(t *testing.T) { testWatchWithProgressNotify(t, false) }
  428. func testWatchWithProgressNotify(t *testing.T, watchOnPut bool) {
  429. defer testutil.AfterTest(t)
  430. // accelerate report interval so test terminates quickly
  431. oldpi := v3rpc.GetProgressReportInterval()
  432. // using atomics to avoid race warnings
  433. v3rpc.SetProgressReportInterval(3 * time.Second)
  434. pi := 3 * time.Second
  435. defer func() { v3rpc.SetProgressReportInterval(oldpi) }()
  436. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  437. defer clus.Terminate(t)
  438. wc := clus.RandClient()
  439. opts := []clientv3.OpOption{clientv3.WithProgressNotify()}
  440. if watchOnPut {
  441. opts = append(opts, clientv3.WithPrefix())
  442. }
  443. rch := wc.Watch(context.Background(), "foo", opts...)
  444. select {
  445. case resp := <-rch: // wait for notification
  446. if len(resp.Events) != 0 {
  447. t.Fatalf("resp.Events expected none, got %+v", resp.Events)
  448. }
  449. case <-time.After(2 * pi):
  450. t.Fatalf("watch response expected in %v, but timed out", pi)
  451. }
  452. kvc := clus.RandClient()
  453. if _, err := kvc.Put(context.TODO(), "foox", "bar"); err != nil {
  454. t.Fatal(err)
  455. }
  456. select {
  457. case resp := <-rch:
  458. if resp.Header.Revision != 2 {
  459. t.Fatalf("resp.Header.Revision expected 2, got %d", resp.Header.Revision)
  460. }
  461. if watchOnPut { // wait for put if watch on the put key
  462. ev := []*clientv3.Event{{Type: clientv3.EventTypePut,
  463. Kv: &mvccpb.KeyValue{Key: []byte("foox"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1}}}
  464. if !reflect.DeepEqual(ev, resp.Events) {
  465. t.Fatalf("expected %+v, got %+v", ev, resp.Events)
  466. }
  467. } else if len(resp.Events) != 0 { // wait for notification otherwise
  468. t.Fatalf("expected no events, but got %+v", resp.Events)
  469. }
  470. case <-time.After(time.Duration(1.5 * float64(pi))):
  471. t.Fatalf("watch response expected in %v, but timed out", pi)
  472. }
  473. }
  474. func TestWatchEventType(t *testing.T) {
  475. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  476. defer cluster.Terminate(t)
  477. client := cluster.RandClient()
  478. ctx := context.Background()
  479. watchChan := client.Watch(ctx, "/", clientv3.WithPrefix())
  480. if _, err := client.Put(ctx, "/toDelete", "foo"); err != nil {
  481. t.Fatalf("Put failed: %v", err)
  482. }
  483. if _, err := client.Put(ctx, "/toDelete", "bar"); err != nil {
  484. t.Fatalf("Put failed: %v", err)
  485. }
  486. if _, err := client.Delete(ctx, "/toDelete"); err != nil {
  487. t.Fatalf("Delete failed: %v", err)
  488. }
  489. lcr, err := client.Lease.Grant(ctx, 1)
  490. if err != nil {
  491. t.Fatalf("lease create failed: %v", err)
  492. }
  493. if _, err := client.Put(ctx, "/toExpire", "foo", clientv3.WithLease(lcr.ID)); err != nil {
  494. t.Fatalf("Put failed: %v", err)
  495. }
  496. tests := []struct {
  497. et mvccpb.Event_EventType
  498. isCreate bool
  499. isModify bool
  500. }{{
  501. et: clientv3.EventTypePut,
  502. isCreate: true,
  503. }, {
  504. et: clientv3.EventTypePut,
  505. isModify: true,
  506. }, {
  507. et: clientv3.EventTypeDelete,
  508. }, {
  509. et: clientv3.EventTypePut,
  510. isCreate: true,
  511. }, {
  512. et: clientv3.EventTypeDelete,
  513. }}
  514. var res []*clientv3.Event
  515. for {
  516. select {
  517. case wres := <-watchChan:
  518. res = append(res, wres.Events...)
  519. case <-time.After(10 * time.Second):
  520. t.Fatalf("Should receive %d events and then break out loop", len(tests))
  521. }
  522. if len(res) == len(tests) {
  523. break
  524. }
  525. }
  526. for i, tt := range tests {
  527. ev := res[i]
  528. if tt.et != ev.Type {
  529. t.Errorf("#%d: event type want=%s, get=%s", i, tt.et, ev.Type)
  530. }
  531. if tt.isCreate && !ev.IsCreate() {
  532. t.Errorf("#%d: event should be CreateEvent", i)
  533. }
  534. if tt.isModify && !ev.IsModify() {
  535. t.Errorf("#%d: event should be ModifyEvent", i)
  536. }
  537. }
  538. }
  539. func TestWatchErrConnClosed(t *testing.T) {
  540. defer testutil.AfterTest(t)
  541. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  542. defer clus.Terminate(t)
  543. cli := clus.Client(0)
  544. donec := make(chan struct{})
  545. go func() {
  546. defer close(donec)
  547. ch := cli.Watch(context.TODO(), "foo")
  548. if wr := <-ch; grpc.ErrorDesc(wr.Err()) != grpc.ErrClientConnClosing.Error() {
  549. t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, grpc.ErrorDesc(wr.Err()))
  550. }
  551. }()
  552. if err := cli.ActiveConnection().Close(); err != nil {
  553. t.Fatal(err)
  554. }
  555. clus.TakeClient(0)
  556. select {
  557. case <-time.After(3 * time.Second):
  558. t.Fatal("wc.Watch took too long")
  559. case <-donec:
  560. }
  561. }
  562. func TestWatchAfterClose(t *testing.T) {
  563. defer testutil.AfterTest(t)
  564. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  565. defer clus.Terminate(t)
  566. cli := clus.Client(0)
  567. clus.TakeClient(0)
  568. if err := cli.Close(); err != nil {
  569. t.Fatal(err)
  570. }
  571. donec := make(chan struct{})
  572. go func() {
  573. cli.Watch(context.TODO(), "foo")
  574. if err := cli.Close(); err != nil && err != grpc.ErrClientConnClosing {
  575. t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
  576. }
  577. close(donec)
  578. }()
  579. select {
  580. case <-time.After(3 * time.Second):
  581. t.Fatal("wc.Watch took too long")
  582. case <-donec:
  583. }
  584. }
  585. // TestWatchWithRequireLeader checks the watch channel closes when no leader.
  586. func TestWatchWithRequireLeader(t *testing.T) {
  587. defer testutil.AfterTest(t)
  588. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  589. defer clus.Terminate(t)
  590. // Put a key for the non-require leader watch to read as an event.
  591. // The watchers will be on member[0]; put key through member[0] to
  592. // ensure that it receives the update so watching after killing quorum
  593. // is guaranteed to have the key.
  594. liveClient := clus.Client(0)
  595. if _, err := liveClient.Put(context.TODO(), "foo", "bar"); err != nil {
  596. t.Fatal(err)
  597. }
  598. clus.Members[1].Stop(t)
  599. clus.Members[2].Stop(t)
  600. clus.Client(1).Close()
  601. clus.Client(2).Close()
  602. clus.TakeClient(1)
  603. clus.TakeClient(2)
  604. // wait for election timeout, then member[0] will not have a leader.
  605. tickDuration := 10 * time.Millisecond
  606. // existing streams need three elections before they're torn down; wait until 5 elections cycle
  607. // so proxy tests receive a leader loss event on its existing watch before creating a new watch.
  608. time.Sleep(time.Duration(5*clus.Members[0].ElectionTicks) * tickDuration)
  609. chLeader := liveClient.Watch(clientv3.WithRequireLeader(context.TODO()), "foo", clientv3.WithRev(1))
  610. chNoLeader := liveClient.Watch(context.TODO(), "foo", clientv3.WithRev(1))
  611. select {
  612. case resp, ok := <-chLeader:
  613. if !ok {
  614. t.Fatalf("expected %v watch channel, got closed channel", rpctypes.ErrNoLeader)
  615. }
  616. if resp.Err() != rpctypes.ErrNoLeader {
  617. t.Fatalf("expected %v watch response error, got %+v", rpctypes.ErrNoLeader, resp)
  618. }
  619. case <-time.After(3 * time.Second):
  620. t.Fatal("watch without leader took too long to close")
  621. }
  622. select {
  623. case resp, ok := <-chLeader:
  624. if ok {
  625. t.Fatalf("expected closed channel, got response %v", resp)
  626. }
  627. case <-time.After(3 * time.Second):
  628. t.Fatal("waited too long for channel to close")
  629. }
  630. if _, ok := <-chNoLeader; !ok {
  631. t.Fatalf("expected response, got closed channel")
  632. }
  633. }
  634. // TestWatchWithFilter checks that watch filtering works.
  635. func TestWatchWithFilter(t *testing.T) {
  636. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  637. defer cluster.Terminate(t)
  638. client := cluster.RandClient()
  639. ctx := context.Background()
  640. wcNoPut := client.Watch(ctx, "a", clientv3.WithFilterPut())
  641. wcNoDel := client.Watch(ctx, "a", clientv3.WithFilterDelete())
  642. if _, err := client.Put(ctx, "a", "abc"); err != nil {
  643. t.Fatal(err)
  644. }
  645. if _, err := client.Delete(ctx, "a"); err != nil {
  646. t.Fatal(err)
  647. }
  648. npResp := <-wcNoPut
  649. if len(npResp.Events) != 1 || npResp.Events[0].Type != clientv3.EventTypeDelete {
  650. t.Fatalf("expected delete event, got %+v", npResp.Events)
  651. }
  652. ndResp := <-wcNoDel
  653. if len(ndResp.Events) != 1 || ndResp.Events[0].Type != clientv3.EventTypePut {
  654. t.Fatalf("expected put event, got %+v", ndResp.Events)
  655. }
  656. select {
  657. case resp := <-wcNoPut:
  658. t.Fatalf("unexpected event on filtered put (%+v)", resp)
  659. case resp := <-wcNoDel:
  660. t.Fatalf("unexpected event on filtered delete (%+v)", resp)
  661. case <-time.After(100 * time.Millisecond):
  662. }
  663. }
  664. // TestWatchWithCreatedNotification checks that createdNotification works.
  665. func TestWatchWithCreatedNotification(t *testing.T) {
  666. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  667. defer cluster.Terminate(t)
  668. client := cluster.RandClient()
  669. ctx := context.Background()
  670. createC := client.Watch(ctx, "a", clientv3.WithCreatedNotify())
  671. resp := <-createC
  672. if !resp.Created {
  673. t.Fatalf("expected created event, got %v", resp)
  674. }
  675. }
  676. // TestWatchWithCreatedNotificationDropConn ensures that
  677. // a watcher with created notify does not post duplicate
  678. // created events from disconnect.
  679. func TestWatchWithCreatedNotificationDropConn(t *testing.T) {
  680. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  681. defer cluster.Terminate(t)
  682. client := cluster.RandClient()
  683. wch := client.Watch(context.Background(), "a", clientv3.WithCreatedNotify())
  684. resp := <-wch
  685. if !resp.Created {
  686. t.Fatalf("expected created event, got %v", resp)
  687. }
  688. cluster.Members[0].DropConnections()
  689. // try to receive from watch channel again
  690. // ensure it doesn't post another createNotify
  691. select {
  692. case wresp := <-wch:
  693. t.Fatalf("got unexpected watch response: %+v\n", wresp)
  694. case <-time.After(time.Second):
  695. // watcher may not reconnect by the time it hits the select,
  696. // so it wouldn't have a chance to filter out the second create event
  697. }
  698. }
  699. // TestWatchCancelOnServer ensures client watcher cancels propagate back to the server.
  700. func TestWatchCancelOnServer(t *testing.T) {
  701. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  702. defer cluster.Terminate(t)
  703. client := cluster.RandClient()
  704. for i := 0; i < 10; i++ {
  705. ctx, cancel := context.WithTimeout(context.Background(), time.Second)
  706. client.Watch(ctx, "a", clientv3.WithCreatedNotify())
  707. cancel()
  708. }
  709. // wait for cancels to propagate
  710. time.Sleep(time.Second)
  711. watchers, err := cluster.Members[0].Metric("etcd_debugging_mvcc_watcher_total")
  712. if err != nil {
  713. t.Fatal(err)
  714. }
  715. if watchers != "0" {
  716. t.Fatalf("expected 0 watchers, got %q", watchers)
  717. }
  718. }
  719. // TestWatchOverlapContextCancel stresses the watcher stream teardown path by
  720. // creating/canceling watchers to ensure that new watchers are not taken down
  721. // by a torn down watch stream. The sort of race that's being detected:
  722. // 1. create w1 using a cancelable ctx with %v as "ctx"
  723. // 2. cancel ctx
  724. // 3. watcher client begins tearing down watcher grpc stream since no more watchers
  725. // 3. start creating watcher w2 using a new "ctx" (not canceled), attaches to old grpc stream
  726. // 4. watcher client finishes tearing down stream on "ctx"
  727. // 5. w2 comes back canceled
  728. func TestWatchOverlapContextCancel(t *testing.T) {
  729. f := func(clus *integration.ClusterV3) {}
  730. testWatchOverlapContextCancel(t, f)
  731. }
  732. func TestWatchOverlapDropConnContextCancel(t *testing.T) {
  733. f := func(clus *integration.ClusterV3) {
  734. clus.Members[0].DropConnections()
  735. }
  736. testWatchOverlapContextCancel(t, f)
  737. }
  738. func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3)) {
  739. defer testutil.AfterTest(t)
  740. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  741. defer clus.Terminate(t)
  742. // each unique context "%v" has a unique grpc stream
  743. n := 100
  744. ctxs, ctxc := make([]context.Context, 5), make([]chan struct{}, 5)
  745. for i := range ctxs {
  746. // make "%v" unique
  747. ctxs[i] = context.WithValue(context.TODO(), "key", i)
  748. // limits the maximum number of outstanding watchers per stream
  749. ctxc[i] = make(chan struct{}, 2)
  750. }
  751. // issue concurrent watches on "abc" with cancel
  752. cli := clus.RandClient()
  753. if _, err := cli.Put(context.TODO(), "abc", "def"); err != nil {
  754. t.Fatal(err)
  755. }
  756. ch := make(chan struct{}, n)
  757. for i := 0; i < n; i++ {
  758. go func() {
  759. defer func() { ch <- struct{}{} }()
  760. idx := rand.Intn(len(ctxs))
  761. ctx, cancel := context.WithCancel(ctxs[idx])
  762. ctxc[idx] <- struct{}{}
  763. wch := cli.Watch(ctx, "abc", clientv3.WithRev(1))
  764. f(clus)
  765. select {
  766. case _, ok := <-wch:
  767. if !ok {
  768. t.Fatalf("unexpected closed channel %p", wch)
  769. }
  770. // may take a second or two to reestablish a watcher because of
  771. // grpc backoff policies for disconnects
  772. case <-time.After(5 * time.Second):
  773. t.Errorf("timed out waiting for watch on %p", wch)
  774. }
  775. // randomize how cancel overlaps with watch creation
  776. if rand.Intn(2) == 0 {
  777. <-ctxc[idx]
  778. cancel()
  779. } else {
  780. cancel()
  781. <-ctxc[idx]
  782. }
  783. }()
  784. }
  785. // join on watches
  786. for i := 0; i < n; i++ {
  787. select {
  788. case <-ch:
  789. case <-time.After(5 * time.Second):
  790. t.Fatalf("timed out waiting for completed watch")
  791. }
  792. }
  793. }
  794. // TestWatchCanelAndCloseClient ensures that canceling a watcher then immediately
  795. // closing the client does not return a client closing error.
  796. func TestWatchCancelAndCloseClient(t *testing.T) {
  797. defer testutil.AfterTest(t)
  798. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  799. defer clus.Terminate(t)
  800. cli := clus.Client(0)
  801. ctx, cancel := context.WithCancel(context.Background())
  802. wch := cli.Watch(ctx, "abc")
  803. donec := make(chan struct{})
  804. go func() {
  805. defer close(donec)
  806. select {
  807. case wr, ok := <-wch:
  808. if ok {
  809. t.Fatalf("expected closed watch after cancel(), got resp=%+v err=%v", wr, wr.Err())
  810. }
  811. case <-time.After(5 * time.Second):
  812. t.Fatal("timed out waiting for closed channel")
  813. }
  814. }()
  815. cancel()
  816. if err := cli.Close(); err != nil {
  817. t.Fatal(err)
  818. }
  819. <-donec
  820. clus.TakeClient(0)
  821. }
  822. // TestWatchStressResumeClose establishes a bunch of watchers, disconnects
  823. // to put them in resuming mode, cancels them so some resumes by cancel fail,
  824. // then closes the watcher interface to ensure correct clean up.
  825. func TestWatchStressResumeClose(t *testing.T) {
  826. defer testutil.AfterTest(t)
  827. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  828. defer clus.Terminate(t)
  829. cli := clus.Client(0)
  830. ctx, cancel := context.WithCancel(context.Background())
  831. // add more watches than can be resumed before the cancel
  832. wchs := make([]clientv3.WatchChan, 2000)
  833. for i := range wchs {
  834. wchs[i] = cli.Watch(ctx, "abc")
  835. }
  836. clus.Members[0].DropConnections()
  837. cancel()
  838. if err := cli.Close(); err != nil {
  839. t.Fatal(err)
  840. }
  841. clus.TakeClient(0)
  842. }
  843. // TestWatchCancelDisconnected ensures canceling a watcher works when
  844. // its grpc stream is disconnected / reconnecting.
  845. func TestWatchCancelDisconnected(t *testing.T) {
  846. defer testutil.AfterTest(t)
  847. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  848. defer clus.Terminate(t)
  849. cli := clus.Client(0)
  850. ctx, cancel := context.WithCancel(context.Background())
  851. // add more watches than can be resumed before the cancel
  852. wch := cli.Watch(ctx, "abc")
  853. clus.Members[0].Stop(t)
  854. cancel()
  855. select {
  856. case <-wch:
  857. case <-time.After(time.Second):
  858. t.Fatal("took too long to cancel disconnected watcher")
  859. }
  860. }