watch_test.go 30 KB

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