watch_test.go 32 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156
  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. "context"
  17. "fmt"
  18. "math/rand"
  19. "reflect"
  20. "sort"
  21. "testing"
  22. "time"
  23. "go.etcd.io/etcd/clientv3"
  24. "go.etcd.io/etcd/etcdserver/api/v3rpc"
  25. "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
  26. "go.etcd.io/etcd/integration"
  27. mvccpb "go.etcd.io/etcd/mvcc/mvccpb"
  28. "go.etcd.io/etcd/pkg/testutil"
  29. "google.golang.org/grpc/metadata"
  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 for KV operations 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.Errorf("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.Errorf("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.Errorf("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.Errorf("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.Errorf("watcher unexpectedly closed")
  123. }
  124. t.Errorf("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 _, 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, ok2 := <-wctx.ch:
  290. if ok2 {
  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. func TestWatchResumeInitRev(t *testing.T) {
  313. defer testutil.AfterTest(t)
  314. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  315. defer clus.Terminate(t)
  316. cli := clus.Client(0)
  317. if _, err := cli.Put(context.TODO(), "b", "2"); err != nil {
  318. t.Fatal(err)
  319. }
  320. if _, err := cli.Put(context.TODO(), "a", "3"); err != nil {
  321. t.Fatal(err)
  322. }
  323. // if resume is broken, it'll pick up this key first instead of a=3
  324. if _, err := cli.Put(context.TODO(), "a", "4"); err != nil {
  325. t.Fatal(err)
  326. }
  327. wch := clus.Client(0).Watch(context.Background(), "a", clientv3.WithRev(1), clientv3.WithCreatedNotify())
  328. if resp, ok := <-wch; !ok || resp.Header.Revision != 4 {
  329. t.Fatalf("got (%v, %v), expected create notification rev=4", resp, ok)
  330. }
  331. // pause wch
  332. clus.Members[0].DropConnections()
  333. clus.Members[0].PauseConnections()
  334. select {
  335. case resp, ok := <-wch:
  336. t.Skipf("wch should block, got (%+v, %v); drop not fast enough", resp, ok)
  337. case <-time.After(100 * time.Millisecond):
  338. }
  339. // resume wch
  340. clus.Members[0].UnpauseConnections()
  341. select {
  342. case resp, ok := <-wch:
  343. if !ok {
  344. t.Fatal("unexpected watch close")
  345. }
  346. if len(resp.Events) == 0 {
  347. t.Fatal("expected event on watch")
  348. }
  349. if string(resp.Events[0].Kv.Value) != "3" {
  350. t.Fatalf("expected value=3, got event %+v", resp.Events[0])
  351. }
  352. case <-time.After(5 * time.Second):
  353. t.Fatal("watch timed out")
  354. }
  355. }
  356. // TestWatchResumeCompacted checks that the watcher gracefully closes in case
  357. // that it tries to resume to a revision that's been compacted out of the store.
  358. // Since the watcher's server restarts with stale data, the watcher will receive
  359. // either a compaction error or all keys by staying in sync before the compaction
  360. // is finally applied.
  361. func TestWatchResumeCompacted(t *testing.T) {
  362. defer testutil.AfterTest(t)
  363. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  364. defer clus.Terminate(t)
  365. // create a waiting watcher at rev 1
  366. w := clus.Client(0)
  367. wch := w.Watch(context.Background(), "foo", clientv3.WithRev(1))
  368. select {
  369. case w := <-wch:
  370. t.Errorf("unexpected message from wch %v", w)
  371. default:
  372. }
  373. clus.Members[0].Stop(t)
  374. ticker := time.After(time.Second * 10)
  375. for clus.WaitLeader(t) <= 0 {
  376. select {
  377. case <-ticker:
  378. t.Fatalf("failed to wait for new leader")
  379. default:
  380. time.Sleep(10 * time.Millisecond)
  381. }
  382. }
  383. // put some data and compact away
  384. numPuts := 5
  385. kv := clus.Client(1)
  386. for i := 0; i < numPuts; i++ {
  387. if _, err := kv.Put(context.TODO(), "foo", "bar"); err != nil {
  388. t.Fatal(err)
  389. }
  390. }
  391. if _, err := kv.Compact(context.TODO(), 3); err != nil {
  392. t.Fatal(err)
  393. }
  394. clus.Members[0].Restart(t)
  395. // since watch's server isn't guaranteed to be synced with the cluster when
  396. // the watch resumes, there is a window where the watch can stay synced and
  397. // read off all events; if the watcher misses the window, it will go out of
  398. // sync and get a compaction error.
  399. wRev := int64(2)
  400. for int(wRev) <= numPuts+1 {
  401. var wresp clientv3.WatchResponse
  402. var ok bool
  403. select {
  404. case wresp, ok = <-wch:
  405. if !ok {
  406. t.Fatalf("expected wresp, but got closed channel")
  407. }
  408. case <-time.After(5 * time.Second):
  409. t.Fatalf("compacted watch timed out")
  410. }
  411. for _, ev := range wresp.Events {
  412. if ev.Kv.ModRevision != wRev {
  413. t.Fatalf("expected modRev %v, got %+v", wRev, ev)
  414. }
  415. wRev++
  416. }
  417. if wresp.Err() == nil {
  418. continue
  419. }
  420. if wresp.Err() != rpctypes.ErrCompacted {
  421. t.Fatalf("wresp.Err() expected %v, got %+v", rpctypes.ErrCompacted, wresp.Err())
  422. }
  423. break
  424. }
  425. if int(wRev) > numPuts+1 {
  426. // got data faster than the compaction
  427. return
  428. }
  429. // received compaction error; ensure the channel closes
  430. select {
  431. case wresp, ok := <-wch:
  432. if ok {
  433. t.Fatalf("expected closed channel, but got %v", wresp)
  434. }
  435. case <-time.After(5 * time.Second):
  436. t.Fatalf("timed out waiting for channel close")
  437. }
  438. }
  439. // TestWatchCompactRevision ensures the CompactRevision error is given on a
  440. // compaction event ahead of a watcher.
  441. func TestWatchCompactRevision(t *testing.T) {
  442. defer testutil.AfterTest(t)
  443. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  444. defer clus.Terminate(t)
  445. // set some keys
  446. kv := clus.RandClient()
  447. for i := 0; i < 5; i++ {
  448. if _, err := kv.Put(context.TODO(), "foo", "bar"); err != nil {
  449. t.Fatal(err)
  450. }
  451. }
  452. w := clus.RandClient()
  453. if _, err := kv.Compact(context.TODO(), 4); err != nil {
  454. t.Fatal(err)
  455. }
  456. wch := w.Watch(context.Background(), "foo", clientv3.WithRev(2))
  457. // get compacted error message
  458. wresp, ok := <-wch
  459. if !ok {
  460. t.Fatalf("expected wresp, but got closed channel")
  461. }
  462. if wresp.Err() != rpctypes.ErrCompacted {
  463. t.Fatalf("wresp.Err() expected %v, but got %v", rpctypes.ErrCompacted, wresp.Err())
  464. }
  465. if !wresp.Canceled {
  466. t.Fatalf("wresp.Canceled expected true, got %+v", wresp)
  467. }
  468. // ensure the channel is closed
  469. if wresp, ok = <-wch; ok {
  470. t.Fatalf("expected closed channel, but got %v", wresp)
  471. }
  472. }
  473. func TestWatchWithProgressNotify(t *testing.T) { testWatchWithProgressNotify(t, true) }
  474. func TestWatchWithProgressNotifyNoEvent(t *testing.T) { testWatchWithProgressNotify(t, false) }
  475. func testWatchWithProgressNotify(t *testing.T, watchOnPut bool) {
  476. defer testutil.AfterTest(t)
  477. // accelerate report interval so test terminates quickly
  478. oldpi := v3rpc.GetProgressReportInterval()
  479. // using atomics to avoid race warnings
  480. v3rpc.SetProgressReportInterval(3 * time.Second)
  481. pi := 3 * time.Second
  482. defer func() { v3rpc.SetProgressReportInterval(oldpi) }()
  483. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  484. defer clus.Terminate(t)
  485. wc := clus.RandClient()
  486. opts := []clientv3.OpOption{clientv3.WithProgressNotify()}
  487. if watchOnPut {
  488. opts = append(opts, clientv3.WithPrefix())
  489. }
  490. rch := wc.Watch(context.Background(), "foo", opts...)
  491. select {
  492. case resp := <-rch: // wait for notification
  493. if len(resp.Events) != 0 {
  494. t.Fatalf("resp.Events expected none, got %+v", resp.Events)
  495. }
  496. case <-time.After(2 * pi):
  497. t.Fatalf("watch response expected in %v, but timed out", pi)
  498. }
  499. kvc := clus.RandClient()
  500. if _, err := kvc.Put(context.TODO(), "foox", "bar"); err != nil {
  501. t.Fatal(err)
  502. }
  503. select {
  504. case resp := <-rch:
  505. if resp.Header.Revision != 2 {
  506. t.Fatalf("resp.Header.Revision expected 2, got %d", resp.Header.Revision)
  507. }
  508. if watchOnPut { // wait for put if watch on the put key
  509. ev := []*clientv3.Event{{Type: clientv3.EventTypePut,
  510. Kv: &mvccpb.KeyValue{Key: []byte("foox"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1}}}
  511. if !reflect.DeepEqual(ev, resp.Events) {
  512. t.Fatalf("expected %+v, got %+v", ev, resp.Events)
  513. }
  514. } else if len(resp.Events) != 0 { // wait for notification otherwise
  515. t.Fatalf("expected no events, but got %+v", resp.Events)
  516. }
  517. case <-time.After(time.Duration(1.5 * float64(pi))):
  518. t.Fatalf("watch response expected in %v, but timed out", pi)
  519. }
  520. }
  521. func TestWatchRequestProgress(t *testing.T) {
  522. testCases := []struct {
  523. name string
  524. watchers []string
  525. }{
  526. {"0-watcher", []string{}},
  527. {"1-watcher", []string{"/"}},
  528. {"2-watcher", []string{"/", "/"}},
  529. }
  530. for _, c := range testCases {
  531. t.Run(c.name, func(t *testing.T) {
  532. defer testutil.AfterTest(t)
  533. watchTimeout := 3 * time.Second
  534. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  535. defer clus.Terminate(t)
  536. wc := clus.RandClient()
  537. var watchChans []clientv3.WatchChan
  538. for _, prefix := range c.watchers {
  539. watchChans = append(watchChans, wc.Watch(context.Background(), prefix, clientv3.WithPrefix()))
  540. }
  541. _, err := wc.Put(context.Background(), "/a", "1")
  542. if err != nil {
  543. t.Fatal(err)
  544. }
  545. for _, rch := range watchChans {
  546. select {
  547. case resp := <-rch: // wait for notification
  548. if len(resp.Events) != 1 {
  549. t.Fatalf("resp.Events expected 1, got %d", len(resp.Events))
  550. }
  551. case <-time.After(watchTimeout):
  552. t.Fatalf("watch response expected in %v, but timed out", watchTimeout)
  553. }
  554. }
  555. // put a value not being watched to increment revision
  556. _, err = wc.Put(context.Background(), "x", "1")
  557. if err != nil {
  558. t.Fatal(err)
  559. }
  560. err = wc.RequestProgress(context.Background())
  561. if err != nil {
  562. t.Fatal(err)
  563. }
  564. // verify all watch channels receive a progress notify
  565. for _, rch := range watchChans {
  566. select {
  567. case resp := <-rch:
  568. if !resp.IsProgressNotify() {
  569. t.Fatalf("expected resp.IsProgressNotify() == true")
  570. }
  571. if resp.Header.Revision != 3 {
  572. t.Fatalf("resp.Header.Revision expected 3, got %d", resp.Header.Revision)
  573. }
  574. case <-time.After(watchTimeout):
  575. t.Fatalf("progress response expected in %v, but timed out", watchTimeout)
  576. }
  577. }
  578. })
  579. }
  580. }
  581. func TestWatchEventType(t *testing.T) {
  582. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  583. defer cluster.Terminate(t)
  584. client := cluster.RandClient()
  585. ctx := context.Background()
  586. watchChan := client.Watch(ctx, "/", clientv3.WithPrefix())
  587. if _, err := client.Put(ctx, "/toDelete", "foo"); err != nil {
  588. t.Fatalf("Put failed: %v", err)
  589. }
  590. if _, err := client.Put(ctx, "/toDelete", "bar"); err != nil {
  591. t.Fatalf("Put failed: %v", err)
  592. }
  593. if _, err := client.Delete(ctx, "/toDelete"); err != nil {
  594. t.Fatalf("Delete failed: %v", err)
  595. }
  596. lcr, err := client.Lease.Grant(ctx, 1)
  597. if err != nil {
  598. t.Fatalf("lease create failed: %v", err)
  599. }
  600. if _, err := client.Put(ctx, "/toExpire", "foo", clientv3.WithLease(lcr.ID)); err != nil {
  601. t.Fatalf("Put failed: %v", err)
  602. }
  603. tests := []struct {
  604. et mvccpb.Event_EventType
  605. isCreate bool
  606. isModify bool
  607. }{{
  608. et: clientv3.EventTypePut,
  609. isCreate: true,
  610. }, {
  611. et: clientv3.EventTypePut,
  612. isModify: true,
  613. }, {
  614. et: clientv3.EventTypeDelete,
  615. }, {
  616. et: clientv3.EventTypePut,
  617. isCreate: true,
  618. }, {
  619. et: clientv3.EventTypeDelete,
  620. }}
  621. var res []*clientv3.Event
  622. for {
  623. select {
  624. case wres := <-watchChan:
  625. res = append(res, wres.Events...)
  626. case <-time.After(10 * time.Second):
  627. t.Fatalf("Should receive %d events and then break out loop", len(tests))
  628. }
  629. if len(res) == len(tests) {
  630. break
  631. }
  632. }
  633. for i, tt := range tests {
  634. ev := res[i]
  635. if tt.et != ev.Type {
  636. t.Errorf("#%d: event type want=%s, get=%s", i, tt.et, ev.Type)
  637. }
  638. if tt.isCreate && !ev.IsCreate() {
  639. t.Errorf("#%d: event should be CreateEvent", i)
  640. }
  641. if tt.isModify && !ev.IsModify() {
  642. t.Errorf("#%d: event should be ModifyEvent", i)
  643. }
  644. }
  645. }
  646. func TestWatchErrConnClosed(t *testing.T) {
  647. defer testutil.AfterTest(t)
  648. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  649. defer clus.Terminate(t)
  650. cli := clus.Client(0)
  651. donec := make(chan struct{})
  652. go func() {
  653. defer close(donec)
  654. ch := cli.Watch(context.TODO(), "foo")
  655. if wr := <-ch; !isCanceled(wr.Err()) {
  656. t.Errorf("expected context canceled, got %v", wr.Err())
  657. }
  658. }()
  659. if err := cli.ActiveConnection().Close(); err != nil {
  660. t.Fatal(err)
  661. }
  662. clus.TakeClient(0)
  663. select {
  664. case <-time.After(integration.RequestWaitTimeout):
  665. t.Fatal("wc.Watch took too long")
  666. case <-donec:
  667. }
  668. }
  669. func TestWatchAfterClose(t *testing.T) {
  670. defer testutil.AfterTest(t)
  671. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  672. defer clus.Terminate(t)
  673. cli := clus.Client(0)
  674. clus.TakeClient(0)
  675. if err := cli.Close(); err != nil {
  676. t.Fatal(err)
  677. }
  678. donec := make(chan struct{})
  679. go func() {
  680. cli.Watch(context.TODO(), "foo")
  681. if err := cli.Close(); err != nil && err != context.Canceled {
  682. t.Errorf("expected %v, got %v", context.Canceled, err)
  683. }
  684. close(donec)
  685. }()
  686. select {
  687. case <-time.After(integration.RequestWaitTimeout):
  688. t.Fatal("wc.Watch took too long")
  689. case <-donec:
  690. }
  691. }
  692. // TestWatchWithRequireLeader checks the watch channel closes when no leader.
  693. func TestWatchWithRequireLeader(t *testing.T) {
  694. defer testutil.AfterTest(t)
  695. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
  696. defer clus.Terminate(t)
  697. // Put a key for the non-require leader watch to read as an event.
  698. // The watchers will be on member[0]; put key through member[0] to
  699. // ensure that it receives the update so watching after killing quorum
  700. // is guaranteed to have the key.
  701. liveClient := clus.Client(0)
  702. if _, err := liveClient.Put(context.TODO(), "foo", "bar"); err != nil {
  703. t.Fatal(err)
  704. }
  705. clus.Members[1].Stop(t)
  706. clus.Members[2].Stop(t)
  707. clus.Client(1).Close()
  708. clus.Client(2).Close()
  709. clus.TakeClient(1)
  710. clus.TakeClient(2)
  711. // wait for election timeout, then member[0] will not have a leader.
  712. tickDuration := 10 * time.Millisecond
  713. // existing streams need three elections before they're torn down; wait until 5 elections cycle
  714. // so proxy tests receive a leader loss event on its existing watch before creating a new watch.
  715. time.Sleep(time.Duration(5*clus.Members[0].ElectionTicks) * tickDuration)
  716. chLeader := liveClient.Watch(clientv3.WithRequireLeader(context.TODO()), "foo", clientv3.WithRev(1))
  717. chNoLeader := liveClient.Watch(context.TODO(), "foo", clientv3.WithRev(1))
  718. select {
  719. case resp, ok := <-chLeader:
  720. if !ok {
  721. t.Fatalf("expected %v watch channel, got closed channel", rpctypes.ErrNoLeader)
  722. }
  723. if resp.Err() != rpctypes.ErrNoLeader {
  724. t.Fatalf("expected %v watch response error, got %+v", rpctypes.ErrNoLeader, resp)
  725. }
  726. case <-time.After(integration.RequestWaitTimeout):
  727. t.Fatal("watch without leader took too long to close")
  728. }
  729. select {
  730. case resp, ok := <-chLeader:
  731. if ok {
  732. t.Fatalf("expected closed channel, got response %v", resp)
  733. }
  734. case <-time.After(integration.RequestWaitTimeout):
  735. t.Fatal("waited too long for channel to close")
  736. }
  737. if _, ok := <-chNoLeader; !ok {
  738. t.Fatalf("expected response, got closed channel")
  739. }
  740. }
  741. // TestWatchWithFilter checks that watch filtering works.
  742. func TestWatchWithFilter(t *testing.T) {
  743. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  744. defer cluster.Terminate(t)
  745. client := cluster.RandClient()
  746. ctx := context.Background()
  747. wcNoPut := client.Watch(ctx, "a", clientv3.WithFilterPut())
  748. wcNoDel := client.Watch(ctx, "a", clientv3.WithFilterDelete())
  749. if _, err := client.Put(ctx, "a", "abc"); err != nil {
  750. t.Fatal(err)
  751. }
  752. if _, err := client.Delete(ctx, "a"); err != nil {
  753. t.Fatal(err)
  754. }
  755. npResp := <-wcNoPut
  756. if len(npResp.Events) != 1 || npResp.Events[0].Type != clientv3.EventTypeDelete {
  757. t.Fatalf("expected delete event, got %+v", npResp.Events)
  758. }
  759. ndResp := <-wcNoDel
  760. if len(ndResp.Events) != 1 || ndResp.Events[0].Type != clientv3.EventTypePut {
  761. t.Fatalf("expected put event, got %+v", ndResp.Events)
  762. }
  763. select {
  764. case resp := <-wcNoPut:
  765. t.Fatalf("unexpected event on filtered put (%+v)", resp)
  766. case resp := <-wcNoDel:
  767. t.Fatalf("unexpected event on filtered delete (%+v)", resp)
  768. case <-time.After(100 * time.Millisecond):
  769. }
  770. }
  771. // TestWatchWithCreatedNotification checks that WithCreatedNotify returns a
  772. // Created watch response.
  773. func TestWatchWithCreatedNotification(t *testing.T) {
  774. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  775. defer cluster.Terminate(t)
  776. client := cluster.RandClient()
  777. ctx := context.Background()
  778. createC := client.Watch(ctx, "a", clientv3.WithCreatedNotify())
  779. resp := <-createC
  780. if !resp.Created {
  781. t.Fatalf("expected created event, got %v", resp)
  782. }
  783. }
  784. // TestWatchWithCreatedNotificationDropConn ensures that
  785. // a watcher with created notify does not post duplicate
  786. // created events from disconnect.
  787. func TestWatchWithCreatedNotificationDropConn(t *testing.T) {
  788. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  789. defer cluster.Terminate(t)
  790. client := cluster.RandClient()
  791. wch := client.Watch(context.Background(), "a", clientv3.WithCreatedNotify())
  792. resp := <-wch
  793. if !resp.Created {
  794. t.Fatalf("expected created event, got %v", resp)
  795. }
  796. cluster.Members[0].DropConnections()
  797. // check watch channel doesn't post another watch response.
  798. select {
  799. case wresp := <-wch:
  800. t.Fatalf("got unexpected watch response: %+v\n", wresp)
  801. case <-time.After(time.Second):
  802. // watcher may not reconnect by the time it hits the select,
  803. // so it wouldn't have a chance to filter out the second create event
  804. }
  805. }
  806. // TestWatchCancelOnServer ensures client watcher cancels propagate back to the server.
  807. func TestWatchCancelOnServer(t *testing.T) {
  808. cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  809. defer cluster.Terminate(t)
  810. client := cluster.RandClient()
  811. numWatches := 10
  812. // The grpc proxy starts watches to detect leadership after the proxy server
  813. // returns as started; to avoid racing on the proxy's internal watches, wait
  814. // until require leader watches get create responses to ensure the leadership
  815. // watches have started.
  816. for {
  817. ctx, cancel := context.WithCancel(clientv3.WithRequireLeader(context.TODO()))
  818. ww := client.Watch(ctx, "a", clientv3.WithCreatedNotify())
  819. wresp := <-ww
  820. cancel()
  821. if wresp.Err() == nil {
  822. break
  823. }
  824. }
  825. cancels := make([]context.CancelFunc, numWatches)
  826. for i := 0; i < numWatches; i++ {
  827. // force separate streams in client
  828. md := metadata.Pairs("some-key", fmt.Sprintf("%d", i))
  829. mctx := metadata.NewOutgoingContext(context.Background(), md)
  830. ctx, cancel := context.WithCancel(mctx)
  831. cancels[i] = cancel
  832. w := client.Watch(ctx, fmt.Sprintf("%d", i), clientv3.WithCreatedNotify())
  833. <-w
  834. }
  835. // get max watches; proxy tests have leadership watches, so total may be >numWatches
  836. maxWatches, _ := cluster.Members[0].Metric("etcd_debugging_mvcc_watcher_total")
  837. // cancel all and wait for cancels to propagate to etcd server
  838. for i := 0; i < numWatches; i++ {
  839. cancels[i]()
  840. }
  841. time.Sleep(time.Second)
  842. minWatches, err := cluster.Members[0].Metric("etcd_debugging_mvcc_watcher_total")
  843. if err != nil {
  844. t.Fatal(err)
  845. }
  846. maxWatchV, minWatchV := 0, 0
  847. n, serr := fmt.Sscanf(maxWatches+" "+minWatches, "%d %d", &maxWatchV, &minWatchV)
  848. if n != 2 || serr != nil {
  849. t.Fatalf("expected n=2 and err=nil, got n=%d and err=%v", n, serr)
  850. }
  851. if maxWatchV-minWatchV < numWatches {
  852. t.Fatalf("expected %d canceled watchers, got %d", numWatches, maxWatchV-minWatchV)
  853. }
  854. }
  855. // TestWatchOverlapContextCancel stresses the watcher stream teardown path by
  856. // creating/canceling watchers to ensure that new watchers are not taken down
  857. // by a torn down watch stream. The sort of race that's being detected:
  858. // 1. create w1 using a cancelable ctx with %v as "ctx"
  859. // 2. cancel ctx
  860. // 3. watcher client begins tearing down watcher grpc stream since no more watchers
  861. // 3. start creating watcher w2 using a new "ctx" (not canceled), attaches to old grpc stream
  862. // 4. watcher client finishes tearing down stream on "ctx"
  863. // 5. w2 comes back canceled
  864. func TestWatchOverlapContextCancel(t *testing.T) {
  865. f := func(clus *integration.ClusterV3) {}
  866. testWatchOverlapContextCancel(t, f)
  867. }
  868. func TestWatchOverlapDropConnContextCancel(t *testing.T) {
  869. f := func(clus *integration.ClusterV3) {
  870. clus.Members[0].DropConnections()
  871. }
  872. testWatchOverlapContextCancel(t, f)
  873. }
  874. func testWatchOverlapContextCancel(t *testing.T, f func(*integration.ClusterV3)) {
  875. defer testutil.AfterTest(t)
  876. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  877. defer clus.Terminate(t)
  878. n := 100
  879. ctxs, ctxc := make([]context.Context, 5), make([]chan struct{}, 5)
  880. for i := range ctxs {
  881. // make unique stream
  882. md := metadata.Pairs("some-key", fmt.Sprintf("%d", i))
  883. ctxs[i] = metadata.NewOutgoingContext(context.Background(), md)
  884. // limits the maximum number of outstanding watchers per stream
  885. ctxc[i] = make(chan struct{}, 2)
  886. }
  887. // issue concurrent watches on "abc" with cancel
  888. cli := clus.RandClient()
  889. if _, err := cli.Put(context.TODO(), "abc", "def"); err != nil {
  890. t.Fatal(err)
  891. }
  892. ch := make(chan struct{}, n)
  893. for i := 0; i < n; i++ {
  894. go func() {
  895. defer func() { ch <- struct{}{} }()
  896. idx := rand.Intn(len(ctxs))
  897. ctx, cancel := context.WithCancel(ctxs[idx])
  898. ctxc[idx] <- struct{}{}
  899. wch := cli.Watch(ctx, "abc", clientv3.WithRev(1))
  900. f(clus)
  901. select {
  902. case _, ok := <-wch:
  903. if !ok {
  904. t.Errorf("unexpected closed channel %p", wch)
  905. }
  906. // may take a second or two to reestablish a watcher because of
  907. // grpc back off policies for disconnects
  908. case <-time.After(5 * time.Second):
  909. t.Errorf("timed out waiting for watch on %p", wch)
  910. }
  911. // randomize how cancel overlaps with watch creation
  912. if rand.Intn(2) == 0 {
  913. <-ctxc[idx]
  914. cancel()
  915. } else {
  916. cancel()
  917. <-ctxc[idx]
  918. }
  919. }()
  920. }
  921. // join on watches
  922. for i := 0; i < n; i++ {
  923. select {
  924. case <-ch:
  925. case <-time.After(5 * time.Second):
  926. t.Fatalf("timed out waiting for completed watch")
  927. }
  928. }
  929. }
  930. // TestWatchCancelAndCloseClient ensures that canceling a watcher then immediately
  931. // closing the client does not return a client closing error.
  932. func TestWatchCancelAndCloseClient(t *testing.T) {
  933. defer testutil.AfterTest(t)
  934. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  935. defer clus.Terminate(t)
  936. cli := clus.Client(0)
  937. ctx, cancel := context.WithCancel(context.Background())
  938. wch := cli.Watch(ctx, "abc")
  939. donec := make(chan struct{})
  940. go func() {
  941. defer close(donec)
  942. select {
  943. case wr, ok := <-wch:
  944. if ok {
  945. t.Errorf("expected closed watch after cancel(), got resp=%+v err=%v", wr, wr.Err())
  946. }
  947. case <-time.After(5 * time.Second):
  948. t.Error("timed out waiting for closed channel")
  949. }
  950. }()
  951. cancel()
  952. if err := cli.Close(); err != nil {
  953. t.Fatal(err)
  954. }
  955. <-donec
  956. clus.TakeClient(0)
  957. }
  958. // TestWatchStressResumeClose establishes a bunch of watchers, disconnects
  959. // to put them in resuming mode, cancels them so some resumes by cancel fail,
  960. // then closes the watcher interface to ensure correct clean up.
  961. func TestWatchStressResumeClose(t *testing.T) {
  962. defer testutil.AfterTest(t)
  963. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  964. defer clus.Terminate(t)
  965. cli := clus.Client(0)
  966. ctx, cancel := context.WithCancel(context.Background())
  967. // add more watches than can be resumed before the cancel
  968. wchs := make([]clientv3.WatchChan, 2000)
  969. for i := range wchs {
  970. wchs[i] = cli.Watch(ctx, "abc")
  971. }
  972. clus.Members[0].DropConnections()
  973. cancel()
  974. if err := cli.Close(); err != nil {
  975. t.Fatal(err)
  976. }
  977. clus.TakeClient(0)
  978. }
  979. // TestWatchCancelDisconnected ensures canceling a watcher works when
  980. // its grpc stream is disconnected / reconnecting.
  981. func TestWatchCancelDisconnected(t *testing.T) {
  982. defer testutil.AfterTest(t)
  983. clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1})
  984. defer clus.Terminate(t)
  985. cli := clus.Client(0)
  986. ctx, cancel := context.WithCancel(context.Background())
  987. // add more watches than can be resumed before the cancel
  988. wch := cli.Watch(ctx, "abc")
  989. clus.Members[0].Stop(t)
  990. cancel()
  991. select {
  992. case <-wch:
  993. case <-time.After(time.Second):
  994. t.Fatal("took too long to cancel disconnected watcher")
  995. }
  996. }
  997. // TestWatchClose ensures that close does not return error
  998. func TestWatchClose(t *testing.T) {
  999. runWatchTest(t, testWatchClose)
  1000. }
  1001. func testWatchClose(t *testing.T, wctx *watchctx) {
  1002. ctx, cancel := context.WithCancel(context.Background())
  1003. wch := wctx.w.Watch(ctx, "a")
  1004. cancel()
  1005. if wch == nil {
  1006. t.Fatalf("expected watcher channel, got nil")
  1007. }
  1008. if wctx.w.Close() != nil {
  1009. t.Fatalf("watch did not close successfully")
  1010. }
  1011. wresp, ok := <-wch
  1012. if ok {
  1013. t.Fatalf("read wch got %v; expected closed channel", wresp)
  1014. }
  1015. }