watchable_store.go 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601
  1. // Copyright 2015 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package mvcc
  15. import (
  16. "sync"
  17. "time"
  18. "github.com/coreos/etcd/lease"
  19. "github.com/coreos/etcd/mvcc/backend"
  20. "github.com/coreos/etcd/mvcc/mvccpb"
  21. )
  22. const (
  23. // chanBufLen is the length of the buffered chan
  24. // for sending out watched events.
  25. // TODO: find a good buf value. 1024 is just a random one that
  26. // seems to be reasonable.
  27. chanBufLen = 1024
  28. // maxWatchersPerSync is the number of watchers to sync in a single batch
  29. maxWatchersPerSync = 512
  30. )
  31. type watchable interface {
  32. watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse, fcs ...FilterFunc) (*watcher, cancelFunc)
  33. progress(w *watcher)
  34. rev() int64
  35. }
  36. type watchableStore struct {
  37. mu sync.Mutex
  38. *store
  39. // victims are watcher batches that were blocked on the watch channel
  40. victims []watcherBatch
  41. victimc chan struct{}
  42. // contains all unsynced watchers that needs to sync with events that have happened
  43. unsynced watcherGroup
  44. // contains all synced watchers that are in sync with the progress of the store.
  45. // The key of the map is the key that the watcher watches on.
  46. synced watcherGroup
  47. stopc chan struct{}
  48. wg sync.WaitGroup
  49. }
  50. // cancelFunc updates unsynced and synced maps when running
  51. // cancel operations.
  52. type cancelFunc func()
  53. func New(b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter) ConsistentWatchableKV {
  54. return newWatchableStore(b, le, ig)
  55. }
  56. func newWatchableStore(b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter) *watchableStore {
  57. s := &watchableStore{
  58. store: NewStore(b, le, ig),
  59. victimc: make(chan struct{}, 1),
  60. unsynced: newWatcherGroup(),
  61. synced: newWatcherGroup(),
  62. stopc: make(chan struct{}),
  63. }
  64. if s.le != nil {
  65. // use this store as the deleter so revokes trigger watch events
  66. s.le.SetRangeDeleter(s)
  67. }
  68. s.wg.Add(2)
  69. go s.syncWatchersLoop()
  70. go s.syncVictimsLoop()
  71. return s
  72. }
  73. func (s *watchableStore) Put(key, value []byte, lease lease.LeaseID) (rev int64) {
  74. s.mu.Lock()
  75. defer s.mu.Unlock()
  76. rev = s.store.Put(key, value, lease)
  77. changes := s.store.getChanges()
  78. if len(changes) != 1 {
  79. plog.Panicf("unexpected len(changes) != 1 after put")
  80. }
  81. ev := mvccpb.Event{
  82. Type: mvccpb.PUT,
  83. Kv: &changes[0],
  84. }
  85. s.notify(rev, []mvccpb.Event{ev})
  86. return rev
  87. }
  88. func (s *watchableStore) DeleteRange(key, end []byte) (n, rev int64) {
  89. s.mu.Lock()
  90. defer s.mu.Unlock()
  91. n, rev = s.store.DeleteRange(key, end)
  92. changes := s.store.getChanges()
  93. if len(changes) != int(n) {
  94. plog.Panicf("unexpected len(changes) != n after deleteRange")
  95. }
  96. if n == 0 {
  97. return n, rev
  98. }
  99. evs := make([]mvccpb.Event, n)
  100. for i := range changes {
  101. evs[i] = mvccpb.Event{
  102. Type: mvccpb.DELETE,
  103. Kv: &changes[i]}
  104. evs[i].Kv.ModRevision = rev
  105. }
  106. s.notify(rev, evs)
  107. return n, rev
  108. }
  109. func (s *watchableStore) TxnBegin() int64 {
  110. s.mu.Lock()
  111. return s.store.TxnBegin()
  112. }
  113. func (s *watchableStore) TxnEnd(txnID int64) error {
  114. err := s.store.TxnEnd(txnID)
  115. if err != nil {
  116. return err
  117. }
  118. changes := s.getChanges()
  119. if len(changes) == 0 {
  120. s.mu.Unlock()
  121. return nil
  122. }
  123. rev := s.store.Rev()
  124. evs := make([]mvccpb.Event, len(changes))
  125. for i, change := range changes {
  126. switch change.CreateRevision {
  127. case 0:
  128. evs[i] = mvccpb.Event{
  129. Type: mvccpb.DELETE,
  130. Kv: &changes[i]}
  131. evs[i].Kv.ModRevision = rev
  132. default:
  133. evs[i] = mvccpb.Event{
  134. Type: mvccpb.PUT,
  135. Kv: &changes[i]}
  136. }
  137. }
  138. s.notify(rev, evs)
  139. s.mu.Unlock()
  140. return nil
  141. }
  142. func (s *watchableStore) Close() error {
  143. close(s.stopc)
  144. s.wg.Wait()
  145. return s.store.Close()
  146. }
  147. func (s *watchableStore) NewWatchStream() WatchStream {
  148. watchStreamGauge.Inc()
  149. return &watchStream{
  150. watchable: s,
  151. ch: make(chan WatchResponse, chanBufLen),
  152. cancels: make(map[WatchID]cancelFunc),
  153. watchers: make(map[WatchID]*watcher),
  154. }
  155. }
  156. func (s *watchableStore) watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse, fcs ...FilterFunc) (*watcher, cancelFunc) {
  157. s.mu.Lock()
  158. defer s.mu.Unlock()
  159. wa := &watcher{
  160. key: key,
  161. end: end,
  162. minRev: startRev,
  163. id: id,
  164. ch: ch,
  165. fcs: fcs,
  166. }
  167. s.store.mu.Lock()
  168. synced := startRev > s.store.currentRev.main || startRev == 0
  169. if synced {
  170. wa.minRev = s.store.currentRev.main + 1
  171. if startRev > wa.minRev {
  172. wa.minRev = startRev
  173. }
  174. }
  175. s.store.mu.Unlock()
  176. if synced {
  177. s.synced.add(wa)
  178. } else {
  179. slowWatcherGauge.Inc()
  180. s.unsynced.add(wa)
  181. }
  182. watcherGauge.Inc()
  183. return wa, func() { s.cancelWatcher(wa) }
  184. }
  185. // cancelWatcher removes references of the watcher from the watchableStore
  186. func (s *watchableStore) cancelWatcher(wa *watcher) {
  187. for {
  188. s.mu.Lock()
  189. if s.unsynced.delete(wa) {
  190. slowWatcherGauge.Dec()
  191. break
  192. } else if s.synced.delete(wa) {
  193. break
  194. } else if wa.compacted {
  195. break
  196. }
  197. if !wa.victim {
  198. panic("watcher not victim but not in watch groups")
  199. }
  200. var victimBatch watcherBatch
  201. for _, wb := range s.victims {
  202. if wb[wa] != nil {
  203. victimBatch = wb
  204. break
  205. }
  206. }
  207. if victimBatch != nil {
  208. slowWatcherGauge.Dec()
  209. delete(victimBatch, wa)
  210. break
  211. }
  212. // victim being processed so not accessible; retry
  213. s.mu.Unlock()
  214. time.Sleep(time.Millisecond)
  215. }
  216. watcherGauge.Dec()
  217. s.mu.Unlock()
  218. }
  219. func (s *watchableStore) Restore(b backend.Backend) error {
  220. s.mu.Lock()
  221. defer s.mu.Unlock()
  222. err := s.store.Restore(b)
  223. if err != nil {
  224. return err
  225. }
  226. for wa := range s.synced.watchers {
  227. wa.restore = true
  228. s.unsynced.add(wa)
  229. }
  230. s.synced = newWatcherGroup()
  231. return nil
  232. }
  233. // syncWatchersLoop syncs the watcher in the unsynced map every 100ms.
  234. func (s *watchableStore) syncWatchersLoop() {
  235. defer s.wg.Done()
  236. for {
  237. s.mu.Lock()
  238. st := time.Now()
  239. lastUnsyncedWatchers := s.unsynced.size()
  240. s.syncWatchers()
  241. unsyncedWatchers := s.unsynced.size()
  242. s.mu.Unlock()
  243. syncDuration := time.Since(st)
  244. waitDuration := 100 * time.Millisecond
  245. // more work pending?
  246. if unsyncedWatchers != 0 && lastUnsyncedWatchers > unsyncedWatchers {
  247. // be fair to other store operations by yielding time taken
  248. waitDuration = syncDuration
  249. }
  250. select {
  251. case <-time.After(waitDuration):
  252. case <-s.stopc:
  253. return
  254. }
  255. }
  256. }
  257. // syncVictimsLoop tries to write precomputed watcher responses to
  258. // watchers that had a blocked watcher channel
  259. func (s *watchableStore) syncVictimsLoop() {
  260. defer s.wg.Done()
  261. for {
  262. for s.moveVictims() != 0 {
  263. // try to update all victim watchers
  264. }
  265. s.mu.Lock()
  266. isEmpty := len(s.victims) == 0
  267. s.mu.Unlock()
  268. var tickc <-chan time.Time
  269. if !isEmpty {
  270. tickc = time.After(10 * time.Millisecond)
  271. }
  272. select {
  273. case <-tickc:
  274. case <-s.victimc:
  275. case <-s.stopc:
  276. return
  277. }
  278. }
  279. }
  280. // moveVictims tries to update watches with already pending event data
  281. func (s *watchableStore) moveVictims() (moved int) {
  282. s.mu.Lock()
  283. victims := s.victims
  284. s.victims = nil
  285. s.mu.Unlock()
  286. var newVictim watcherBatch
  287. for _, wb := range victims {
  288. // try to send responses again
  289. for w, eb := range wb {
  290. // watcher has observed the store up to, but not including, w.minRev
  291. rev := w.minRev - 1
  292. if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
  293. pendingEventsGauge.Add(float64(len(eb.evs)))
  294. } else {
  295. if newVictim == nil {
  296. newVictim = make(watcherBatch)
  297. }
  298. newVictim[w] = eb
  299. continue
  300. }
  301. moved++
  302. }
  303. // assign completed victim watchers to unsync/sync
  304. s.mu.Lock()
  305. s.store.mu.Lock()
  306. curRev := s.store.currentRev.main
  307. for w, eb := range wb {
  308. if newVictim != nil && newVictim[w] != nil {
  309. // couldn't send watch response; stays victim
  310. continue
  311. }
  312. w.victim = false
  313. if eb.moreRev != 0 {
  314. w.minRev = eb.moreRev
  315. }
  316. if w.minRev <= curRev {
  317. s.unsynced.add(w)
  318. } else {
  319. slowWatcherGauge.Dec()
  320. s.synced.add(w)
  321. }
  322. }
  323. s.store.mu.Unlock()
  324. s.mu.Unlock()
  325. }
  326. if len(newVictim) > 0 {
  327. s.mu.Lock()
  328. s.victims = append(s.victims, newVictim)
  329. s.mu.Unlock()
  330. }
  331. return moved
  332. }
  333. // syncWatchers syncs unsynced watchers by:
  334. // 1. choose a set of watchers from the unsynced watcher group
  335. // 2. iterate over the set to get the minimum revision and remove compacted watchers
  336. // 3. use minimum revision to get all key-value pairs and send those events to watchers
  337. // 4. remove synced watchers in set from unsynced group and move to synced group
  338. func (s *watchableStore) syncWatchers() {
  339. if s.unsynced.size() == 0 {
  340. return
  341. }
  342. s.store.mu.Lock()
  343. defer s.store.mu.Unlock()
  344. // in order to find key-value pairs from unsynced watchers, we need to
  345. // find min revision index, and these revisions can be used to
  346. // query the backend store of key-value pairs
  347. curRev := s.store.currentRev.main
  348. compactionRev := s.store.compactMainRev
  349. wg, minRev := s.unsynced.choose(maxWatchersPerSync, curRev, compactionRev)
  350. minBytes, maxBytes := newRevBytes(), newRevBytes()
  351. revToBytes(revision{main: minRev}, minBytes)
  352. revToBytes(revision{main: curRev + 1}, maxBytes)
  353. // UnsafeRange returns keys and values. And in boltdb, keys are revisions.
  354. // values are actual key-value pairs in backend.
  355. tx := s.store.b.BatchTx()
  356. tx.Lock()
  357. revs, vs := tx.UnsafeRange(keyBucketName, minBytes, maxBytes, 0)
  358. evs := kvsToEvents(wg, revs, vs)
  359. tx.Unlock()
  360. var victims watcherBatch
  361. wb := newWatcherBatch(wg, evs)
  362. for w := range wg.watchers {
  363. w.minRev = curRev + 1
  364. eb, ok := wb[w]
  365. if !ok {
  366. // bring un-notified watcher to synced
  367. s.synced.add(w)
  368. s.unsynced.delete(w)
  369. continue
  370. }
  371. if eb.moreRev != 0 {
  372. w.minRev = eb.moreRev
  373. }
  374. if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: curRev}) {
  375. pendingEventsGauge.Add(float64(len(eb.evs)))
  376. } else {
  377. if victims == nil {
  378. victims = make(watcherBatch)
  379. }
  380. w.victim = true
  381. }
  382. if w.victim {
  383. victims[w] = eb
  384. } else {
  385. if eb.moreRev != 0 {
  386. // stay unsynced; more to read
  387. continue
  388. }
  389. s.synced.add(w)
  390. }
  391. s.unsynced.delete(w)
  392. }
  393. s.addVictim(victims)
  394. vsz := 0
  395. for _, v := range s.victims {
  396. vsz += len(v)
  397. }
  398. slowWatcherGauge.Set(float64(s.unsynced.size() + vsz))
  399. }
  400. // kvsToEvents gets all events for the watchers from all key-value pairs
  401. func kvsToEvents(wg *watcherGroup, revs, vals [][]byte) (evs []mvccpb.Event) {
  402. for i, v := range vals {
  403. var kv mvccpb.KeyValue
  404. if err := kv.Unmarshal(v); err != nil {
  405. plog.Panicf("cannot unmarshal event: %v", err)
  406. }
  407. if !wg.contains(string(kv.Key)) {
  408. continue
  409. }
  410. ty := mvccpb.PUT
  411. if isTombstone(revs[i]) {
  412. ty = mvccpb.DELETE
  413. // patch in mod revision so watchers won't skip
  414. kv.ModRevision = bytesToRev(revs[i]).main
  415. }
  416. evs = append(evs, mvccpb.Event{Kv: &kv, Type: ty})
  417. }
  418. return evs
  419. }
  420. // notify notifies the fact that given event at the given rev just happened to
  421. // watchers that watch on the key of the event.
  422. func (s *watchableStore) notify(rev int64, evs []mvccpb.Event) {
  423. var victim watcherBatch
  424. for w, eb := range newWatcherBatch(&s.synced, evs) {
  425. if eb.revs != 1 {
  426. plog.Panicf("unexpected multiple revisions in notification")
  427. }
  428. if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
  429. pendingEventsGauge.Add(float64(len(eb.evs)))
  430. } else {
  431. // move slow watcher to victims
  432. w.minRev = rev + 1
  433. if victim == nil {
  434. victim = make(watcherBatch)
  435. }
  436. w.victim = true
  437. victim[w] = eb
  438. s.synced.delete(w)
  439. slowWatcherGauge.Inc()
  440. }
  441. }
  442. s.addVictim(victim)
  443. }
  444. func (s *watchableStore) addVictim(victim watcherBatch) {
  445. if victim == nil {
  446. return
  447. }
  448. s.victims = append(s.victims, victim)
  449. select {
  450. case s.victimc <- struct{}{}:
  451. default:
  452. }
  453. }
  454. func (s *watchableStore) rev() int64 { return s.store.Rev() }
  455. func (s *watchableStore) progress(w *watcher) {
  456. s.mu.Lock()
  457. defer s.mu.Unlock()
  458. if _, ok := s.synced.watchers[w]; ok {
  459. w.send(WatchResponse{WatchID: w.id, Revision: s.rev()})
  460. // If the ch is full, this watcher is receiving events.
  461. // We do not need to send progress at all.
  462. }
  463. }
  464. type watcher struct {
  465. // the watcher key
  466. key []byte
  467. // end indicates the end of the range to watch.
  468. // If end is set, the watcher is on a range.
  469. end []byte
  470. // victim is set when ch is blocked and undergoing victim processing
  471. victim bool
  472. // compacted is set when the watcher is removed because of compaction
  473. compacted bool
  474. // restore is true when the watcher is being restored from leader snapshot
  475. // which means that this watcher has just been moved from "synced" to "unsynced"
  476. // watcher group, possibly with a future revision when it was first added
  477. // to the synced watcher
  478. // "unsynced" watcher revision must always be <= current revision,
  479. // except when the watcher were to be moved from "synced" watcher group
  480. restore bool
  481. // minRev is the minimum revision update the watcher will accept
  482. minRev int64
  483. id WatchID
  484. fcs []FilterFunc
  485. // a chan to send out the watch response.
  486. // The chan might be shared with other watchers.
  487. ch chan<- WatchResponse
  488. }
  489. func (w *watcher) send(wr WatchResponse) bool {
  490. progressEvent := len(wr.Events) == 0
  491. if len(w.fcs) != 0 {
  492. ne := make([]mvccpb.Event, 0, len(wr.Events))
  493. for i := range wr.Events {
  494. filtered := false
  495. for _, filter := range w.fcs {
  496. if filter(wr.Events[i]) {
  497. filtered = true
  498. break
  499. }
  500. }
  501. if !filtered {
  502. ne = append(ne, wr.Events[i])
  503. }
  504. }
  505. wr.Events = ne
  506. }
  507. // if all events are filtered out, we should send nothing.
  508. if !progressEvent && len(wr.Events) == 0 {
  509. return true
  510. }
  511. select {
  512. case w.ch <- wr:
  513. return true
  514. default:
  515. return false
  516. }
  517. }