watchable_store.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531
  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. *store
  38. // mu protects watcher groups and batches. It should never be locked
  39. // before locking store.mu to avoid deadlock.
  40. mu sync.RWMutex
  41. // victims are watcher batches that were blocked on the watch channel
  42. victims []watcherBatch
  43. victimc chan struct{}
  44. // contains all unsynced watchers that needs to sync with events that have happened
  45. unsynced watcherGroup
  46. // contains all synced watchers that are in sync with the progress of the store.
  47. // The key of the map is the key that the watcher watches on.
  48. synced watcherGroup
  49. stopc chan struct{}
  50. wg sync.WaitGroup
  51. }
  52. // cancelFunc updates unsynced and synced maps when running
  53. // cancel operations.
  54. type cancelFunc func()
  55. func New(b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter) ConsistentWatchableKV {
  56. return newWatchableStore(b, le, ig)
  57. }
  58. func newWatchableStore(b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter) *watchableStore {
  59. s := &watchableStore{
  60. store: NewStore(b, le, ig),
  61. victimc: make(chan struct{}, 1),
  62. unsynced: newWatcherGroup(),
  63. synced: newWatcherGroup(),
  64. stopc: make(chan struct{}),
  65. }
  66. s.store.ReadView = &readView{s}
  67. s.store.WriteView = &writeView{s}
  68. if s.le != nil {
  69. // use this store as the deleter so revokes trigger watch events
  70. s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write() })
  71. }
  72. s.wg.Add(2)
  73. go s.syncWatchersLoop()
  74. go s.syncVictimsLoop()
  75. return s
  76. }
  77. func (s *watchableStore) Close() error {
  78. close(s.stopc)
  79. s.wg.Wait()
  80. return s.store.Close()
  81. }
  82. func (s *watchableStore) NewWatchStream() WatchStream {
  83. watchStreamGauge.Inc()
  84. return &watchStream{
  85. watchable: s,
  86. ch: make(chan WatchResponse, chanBufLen),
  87. cancels: make(map[WatchID]cancelFunc),
  88. watchers: make(map[WatchID]*watcher),
  89. }
  90. }
  91. func (s *watchableStore) watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse, fcs ...FilterFunc) (*watcher, cancelFunc) {
  92. wa := &watcher{
  93. key: key,
  94. end: end,
  95. minRev: startRev,
  96. id: id,
  97. ch: ch,
  98. fcs: fcs,
  99. }
  100. s.mu.Lock()
  101. s.revMu.RLock()
  102. synced := startRev > s.store.currentRev || startRev == 0
  103. if synced {
  104. wa.minRev = s.store.currentRev + 1
  105. if startRev > wa.minRev {
  106. wa.minRev = startRev
  107. }
  108. }
  109. if synced {
  110. s.synced.add(wa)
  111. } else {
  112. slowWatcherGauge.Inc()
  113. s.unsynced.add(wa)
  114. }
  115. s.revMu.RUnlock()
  116. s.mu.Unlock()
  117. watcherGauge.Inc()
  118. return wa, func() { s.cancelWatcher(wa) }
  119. }
  120. // cancelWatcher removes references of the watcher from the watchableStore
  121. func (s *watchableStore) cancelWatcher(wa *watcher) {
  122. for {
  123. s.mu.Lock()
  124. if s.unsynced.delete(wa) {
  125. slowWatcherGauge.Dec()
  126. break
  127. } else if s.synced.delete(wa) {
  128. break
  129. } else if wa.compacted {
  130. break
  131. }
  132. if !wa.victim {
  133. panic("watcher not victim but not in watch groups")
  134. }
  135. var victimBatch watcherBatch
  136. for _, wb := range s.victims {
  137. if wb[wa] != nil {
  138. victimBatch = wb
  139. break
  140. }
  141. }
  142. if victimBatch != nil {
  143. slowWatcherGauge.Dec()
  144. delete(victimBatch, wa)
  145. break
  146. }
  147. // victim being processed so not accessible; retry
  148. s.mu.Unlock()
  149. time.Sleep(time.Millisecond)
  150. }
  151. watcherGauge.Dec()
  152. s.mu.Unlock()
  153. }
  154. func (s *watchableStore) Restore(b backend.Backend) error {
  155. s.mu.Lock()
  156. defer s.mu.Unlock()
  157. err := s.store.Restore(b)
  158. if err != nil {
  159. return err
  160. }
  161. for wa := range s.synced.watchers {
  162. wa.restore = true
  163. s.unsynced.add(wa)
  164. }
  165. s.synced = newWatcherGroup()
  166. return nil
  167. }
  168. // syncWatchersLoop syncs the watcher in the unsynced map every 100ms.
  169. func (s *watchableStore) syncWatchersLoop() {
  170. defer s.wg.Done()
  171. for {
  172. s.mu.RLock()
  173. st := time.Now()
  174. lastUnsyncedWatchers := s.unsynced.size()
  175. s.mu.RUnlock()
  176. unsyncedWatchers := 0
  177. if lastUnsyncedWatchers > 0 {
  178. unsyncedWatchers = s.syncWatchers()
  179. }
  180. syncDuration := time.Since(st)
  181. waitDuration := 100 * time.Millisecond
  182. // more work pending?
  183. if unsyncedWatchers != 0 && lastUnsyncedWatchers > unsyncedWatchers {
  184. // be fair to other store operations by yielding time taken
  185. waitDuration = syncDuration
  186. }
  187. select {
  188. case <-time.After(waitDuration):
  189. case <-s.stopc:
  190. return
  191. }
  192. }
  193. }
  194. // syncVictimsLoop tries to write precomputed watcher responses to
  195. // watchers that had a blocked watcher channel
  196. func (s *watchableStore) syncVictimsLoop() {
  197. defer s.wg.Done()
  198. for {
  199. for s.moveVictims() != 0 {
  200. // try to update all victim watchers
  201. }
  202. s.mu.RLock()
  203. isEmpty := len(s.victims) == 0
  204. s.mu.RUnlock()
  205. var tickc <-chan time.Time
  206. if !isEmpty {
  207. tickc = time.After(10 * time.Millisecond)
  208. }
  209. select {
  210. case <-tickc:
  211. case <-s.victimc:
  212. case <-s.stopc:
  213. return
  214. }
  215. }
  216. }
  217. // moveVictims tries to update watches with already pending event data
  218. func (s *watchableStore) moveVictims() (moved int) {
  219. s.mu.Lock()
  220. victims := s.victims
  221. s.victims = nil
  222. s.mu.Unlock()
  223. var newVictim watcherBatch
  224. for _, wb := range victims {
  225. // try to send responses again
  226. for w, eb := range wb {
  227. // watcher has observed the store up to, but not including, w.minRev
  228. rev := w.minRev - 1
  229. if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
  230. pendingEventsGauge.Add(float64(len(eb.evs)))
  231. } else {
  232. if newVictim == nil {
  233. newVictim = make(watcherBatch)
  234. }
  235. newVictim[w] = eb
  236. continue
  237. }
  238. moved++
  239. }
  240. // assign completed victim watchers to unsync/sync
  241. s.mu.Lock()
  242. s.store.revMu.RLock()
  243. curRev := s.store.currentRev
  244. for w, eb := range wb {
  245. if newVictim != nil && newVictim[w] != nil {
  246. // couldn't send watch response; stays victim
  247. continue
  248. }
  249. w.victim = false
  250. if eb.moreRev != 0 {
  251. w.minRev = eb.moreRev
  252. }
  253. if w.minRev <= curRev {
  254. s.unsynced.add(w)
  255. } else {
  256. slowWatcherGauge.Dec()
  257. s.synced.add(w)
  258. }
  259. }
  260. s.store.revMu.RUnlock()
  261. s.mu.Unlock()
  262. }
  263. if len(newVictim) > 0 {
  264. s.mu.Lock()
  265. s.victims = append(s.victims, newVictim)
  266. s.mu.Unlock()
  267. }
  268. return moved
  269. }
  270. // syncWatchers syncs unsynced watchers by:
  271. // 1. choose a set of watchers from the unsynced watcher group
  272. // 2. iterate over the set to get the minimum revision and remove compacted watchers
  273. // 3. use minimum revision to get all key-value pairs and send those events to watchers
  274. // 4. remove synced watchers in set from unsynced group and move to synced group
  275. func (s *watchableStore) syncWatchers() int {
  276. s.mu.Lock()
  277. defer s.mu.Unlock()
  278. if s.unsynced.size() == 0 {
  279. return 0
  280. }
  281. s.store.revMu.RLock()
  282. defer s.store.revMu.RUnlock()
  283. // in order to find key-value pairs from unsynced watchers, we need to
  284. // find min revision index, and these revisions can be used to
  285. // query the backend store of key-value pairs
  286. curRev := s.store.currentRev
  287. compactionRev := s.store.compactMainRev
  288. wg, minRev := s.unsynced.choose(maxWatchersPerSync, curRev, compactionRev)
  289. minBytes, maxBytes := newRevBytes(), newRevBytes()
  290. revToBytes(revision{main: minRev}, minBytes)
  291. revToBytes(revision{main: curRev + 1}, maxBytes)
  292. // UnsafeRange returns keys and values. And in boltdb, keys are revisions.
  293. // values are actual key-value pairs in backend.
  294. tx := s.store.b.ReadTx()
  295. tx.Lock()
  296. revs, vs := tx.UnsafeRange(keyBucketName, minBytes, maxBytes, 0)
  297. evs := kvsToEvents(wg, revs, vs)
  298. tx.Unlock()
  299. var victims watcherBatch
  300. wb := newWatcherBatch(wg, evs)
  301. for w := range wg.watchers {
  302. w.minRev = curRev + 1
  303. eb, ok := wb[w]
  304. if !ok {
  305. // bring un-notified watcher to synced
  306. s.synced.add(w)
  307. s.unsynced.delete(w)
  308. continue
  309. }
  310. if eb.moreRev != 0 {
  311. w.minRev = eb.moreRev
  312. }
  313. if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: curRev}) {
  314. pendingEventsGauge.Add(float64(len(eb.evs)))
  315. } else {
  316. if victims == nil {
  317. victims = make(watcherBatch)
  318. }
  319. w.victim = true
  320. }
  321. if w.victim {
  322. victims[w] = eb
  323. } else {
  324. if eb.moreRev != 0 {
  325. // stay unsynced; more to read
  326. continue
  327. }
  328. s.synced.add(w)
  329. }
  330. s.unsynced.delete(w)
  331. }
  332. s.addVictim(victims)
  333. vsz := 0
  334. for _, v := range s.victims {
  335. vsz += len(v)
  336. }
  337. slowWatcherGauge.Set(float64(s.unsynced.size() + vsz))
  338. return s.unsynced.size()
  339. }
  340. // kvsToEvents gets all events for the watchers from all key-value pairs
  341. func kvsToEvents(wg *watcherGroup, revs, vals [][]byte) (evs []mvccpb.Event) {
  342. for i, v := range vals {
  343. var kv mvccpb.KeyValue
  344. if err := kv.Unmarshal(v); err != nil {
  345. plog.Panicf("cannot unmarshal event: %v", err)
  346. }
  347. if !wg.contains(string(kv.Key)) {
  348. continue
  349. }
  350. ty := mvccpb.PUT
  351. if isTombstone(revs[i]) {
  352. ty = mvccpb.DELETE
  353. // patch in mod revision so watchers won't skip
  354. kv.ModRevision = bytesToRev(revs[i]).main
  355. }
  356. evs = append(evs, mvccpb.Event{Kv: &kv, Type: ty})
  357. }
  358. return evs
  359. }
  360. // notify notifies the fact that given event at the given rev just happened to
  361. // watchers that watch on the key of the event.
  362. func (s *watchableStore) notify(rev int64, evs []mvccpb.Event) {
  363. var victim watcherBatch
  364. for w, eb := range newWatcherBatch(&s.synced, evs) {
  365. if eb.revs != 1 {
  366. plog.Panicf("unexpected multiple revisions in notification")
  367. }
  368. if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
  369. pendingEventsGauge.Add(float64(len(eb.evs)))
  370. } else {
  371. // move slow watcher to victims
  372. w.minRev = rev + 1
  373. if victim == nil {
  374. victim = make(watcherBatch)
  375. }
  376. w.victim = true
  377. victim[w] = eb
  378. s.synced.delete(w)
  379. slowWatcherGauge.Inc()
  380. }
  381. }
  382. s.addVictim(victim)
  383. }
  384. func (s *watchableStore) addVictim(victim watcherBatch) {
  385. if victim == nil {
  386. return
  387. }
  388. s.victims = append(s.victims, victim)
  389. select {
  390. case s.victimc <- struct{}{}:
  391. default:
  392. }
  393. }
  394. func (s *watchableStore) rev() int64 { return s.store.Rev() }
  395. func (s *watchableStore) progress(w *watcher) {
  396. s.mu.RLock()
  397. defer s.mu.RUnlock()
  398. if _, ok := s.synced.watchers[w]; ok {
  399. w.send(WatchResponse{WatchID: w.id, Revision: s.rev()})
  400. // If the ch is full, this watcher is receiving events.
  401. // We do not need to send progress at all.
  402. }
  403. }
  404. type watcher struct {
  405. // the watcher key
  406. key []byte
  407. // end indicates the end of the range to watch.
  408. // If end is set, the watcher is on a range.
  409. end []byte
  410. // victim is set when ch is blocked and undergoing victim processing
  411. victim bool
  412. // compacted is set when the watcher is removed because of compaction
  413. compacted bool
  414. // restore is true when the watcher is being restored from leader snapshot
  415. // which means that this watcher has just been moved from "synced" to "unsynced"
  416. // watcher group, possibly with a future revision when it was first added
  417. // to the synced watcher
  418. // "unsynced" watcher revision must always be <= current revision,
  419. // except when the watcher were to be moved from "synced" watcher group
  420. restore bool
  421. // minRev is the minimum revision update the watcher will accept
  422. minRev int64
  423. id WatchID
  424. fcs []FilterFunc
  425. // a chan to send out the watch response.
  426. // The chan might be shared with other watchers.
  427. ch chan<- WatchResponse
  428. }
  429. func (w *watcher) send(wr WatchResponse) bool {
  430. progressEvent := len(wr.Events) == 0
  431. if len(w.fcs) != 0 {
  432. ne := make([]mvccpb.Event, 0, len(wr.Events))
  433. for i := range wr.Events {
  434. filtered := false
  435. for _, filter := range w.fcs {
  436. if filter(wr.Events[i]) {
  437. filtered = true
  438. break
  439. }
  440. }
  441. if !filtered {
  442. ne = append(ne, wr.Events[i])
  443. }
  444. }
  445. wr.Events = ne
  446. }
  447. // if all events are filtered out, we should send nothing.
  448. if !progressEvent && len(wr.Events) == 0 {
  449. return true
  450. }
  451. select {
  452. case w.ch <- wr:
  453. return true
  454. default:
  455. return false
  456. }
  457. }