watchable_store.go 12 KB

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