|
|
@@ -16,8 +16,6 @@ package storage
|
|
|
|
|
|
import (
|
|
|
"log"
|
|
|
- "math"
|
|
|
- "strings"
|
|
|
"sync"
|
|
|
"time"
|
|
|
|
|
|
@@ -34,103 +32,8 @@ const (
|
|
|
chanBufLen = 1024
|
|
|
)
|
|
|
|
|
|
-var (
|
|
|
- // watchBatchMaxRevs is the maximum distinct revisions that
|
|
|
- // may be sent to an unsynced watcher at a time. Declared as
|
|
|
- // var instead of const for testing purposes.
|
|
|
- watchBatchMaxRevs = 1000
|
|
|
-)
|
|
|
-
|
|
|
-type eventBatch struct {
|
|
|
- // evs is a batch of revision-ordered events
|
|
|
- evs []storagepb.Event
|
|
|
- // revs is the minimum unique revisions observed for this batch
|
|
|
- revs int
|
|
|
- // moreRev is first revision with more events following this batch
|
|
|
- moreRev int64
|
|
|
-}
|
|
|
-
|
|
|
-type (
|
|
|
- watcherSetByKey map[string]watcherSet
|
|
|
- watcherSet map[*watcher]struct{}
|
|
|
- watcherBatch map[*watcher]*eventBatch
|
|
|
-)
|
|
|
-
|
|
|
-func (eb *eventBatch) add(ev storagepb.Event) {
|
|
|
- if eb.revs > watchBatchMaxRevs {
|
|
|
- // maxed out batch size
|
|
|
- return
|
|
|
- }
|
|
|
-
|
|
|
- if len(eb.evs) == 0 {
|
|
|
- // base case
|
|
|
- eb.revs = 1
|
|
|
- eb.evs = append(eb.evs, ev)
|
|
|
- return
|
|
|
- }
|
|
|
-
|
|
|
- // revision accounting
|
|
|
- ebRev := eb.evs[len(eb.evs)-1].Kv.ModRevision
|
|
|
- evRev := ev.Kv.ModRevision
|
|
|
- if evRev > ebRev {
|
|
|
- eb.revs++
|
|
|
- if eb.revs > watchBatchMaxRevs {
|
|
|
- eb.moreRev = evRev
|
|
|
- return
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- eb.evs = append(eb.evs, ev)
|
|
|
-}
|
|
|
-
|
|
|
-func (wb watcherBatch) add(w *watcher, ev storagepb.Event) {
|
|
|
- eb := wb[w]
|
|
|
- if eb == nil {
|
|
|
- eb = &eventBatch{}
|
|
|
- wb[w] = eb
|
|
|
- }
|
|
|
- eb.add(ev)
|
|
|
-}
|
|
|
-
|
|
|
-func (w watcherSet) add(wa *watcher) {
|
|
|
- if _, ok := w[wa]; ok {
|
|
|
- panic("add watcher twice!")
|
|
|
- }
|
|
|
- w[wa] = struct{}{}
|
|
|
-}
|
|
|
-
|
|
|
-func (w watcherSetByKey) add(wa *watcher) {
|
|
|
- set := w[string(wa.key)]
|
|
|
- if set == nil {
|
|
|
- set = make(watcherSet)
|
|
|
- w[string(wa.key)] = set
|
|
|
- }
|
|
|
- set.add(wa)
|
|
|
-}
|
|
|
-
|
|
|
-func (w watcherSetByKey) getSetByKey(key string) (watcherSet, bool) {
|
|
|
- set, ok := w[key]
|
|
|
- return set, ok
|
|
|
-}
|
|
|
-
|
|
|
-func (w watcherSetByKey) delete(wa *watcher) bool {
|
|
|
- k := string(wa.key)
|
|
|
- if v, ok := w[k]; ok {
|
|
|
- if _, ok := v[wa]; ok {
|
|
|
- delete(v, wa)
|
|
|
- // if there is nothing in the set,
|
|
|
- // remove the set
|
|
|
- if len(v) == 0 {
|
|
|
- delete(w, k)
|
|
|
- }
|
|
|
- return true
|
|
|
- }
|
|
|
- }
|
|
|
- return false
|
|
|
-}
|
|
|
-
|
|
|
type watchable interface {
|
|
|
- watch(key []byte, prefix bool, startRev int64, id WatchID, ch chan<- WatchResponse) (*watcher, cancelFunc)
|
|
|
+ watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse) (*watcher, cancelFunc)
|
|
|
rev() int64
|
|
|
}
|
|
|
|
|
|
@@ -140,11 +43,11 @@ type watchableStore struct {
|
|
|
*store
|
|
|
|
|
|
// contains all unsynced watchers that needs to sync with events that have happened
|
|
|
- unsynced watcherSetByKey
|
|
|
+ unsynced watcherGroup
|
|
|
|
|
|
// contains all synced watchers that are in sync with the progress of the store.
|
|
|
// The key of the map is the key that the watcher watches on.
|
|
|
- synced watcherSetByKey
|
|
|
+ synced watcherGroup
|
|
|
|
|
|
stopc chan struct{}
|
|
|
wg sync.WaitGroup
|
|
|
@@ -157,8 +60,8 @@ type cancelFunc func()
|
|
|
func newWatchableStore(b backend.Backend, le lease.Lessor) *watchableStore {
|
|
|
s := &watchableStore{
|
|
|
store: NewStore(b, le),
|
|
|
- unsynced: make(watcherSetByKey),
|
|
|
- synced: make(watcherSetByKey),
|
|
|
+ unsynced: newWatcherGroup(),
|
|
|
+ synced: newWatcherGroup(),
|
|
|
stopc: make(chan struct{}),
|
|
|
}
|
|
|
if s.le != nil {
|
|
|
@@ -268,16 +171,16 @@ func (s *watchableStore) NewWatchStream() WatchStream {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (s *watchableStore) watch(key []byte, prefix bool, startRev int64, id WatchID, ch chan<- WatchResponse) (*watcher, cancelFunc) {
|
|
|
+func (s *watchableStore) watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse) (*watcher, cancelFunc) {
|
|
|
s.mu.Lock()
|
|
|
defer s.mu.Unlock()
|
|
|
|
|
|
wa := &watcher{
|
|
|
- key: key,
|
|
|
- prefix: prefix,
|
|
|
- cur: startRev,
|
|
|
- id: id,
|
|
|
- ch: ch,
|
|
|
+ key: key,
|
|
|
+ end: end,
|
|
|
+ cur: startRev,
|
|
|
+ id: id,
|
|
|
+ ch: ch,
|
|
|
}
|
|
|
|
|
|
s.store.mu.Lock()
|
|
|
@@ -342,15 +245,16 @@ func (s *watchableStore) syncWatchers() {
|
|
|
s.store.mu.Lock()
|
|
|
defer s.store.mu.Unlock()
|
|
|
|
|
|
- if len(s.unsynced) == 0 {
|
|
|
+ if s.unsynced.size() == 0 {
|
|
|
return
|
|
|
}
|
|
|
|
|
|
// in order to find key-value pairs from unsynced watchers, we need to
|
|
|
// find min revision index, and these revisions can be used to
|
|
|
// query the backend store of key-value pairs
|
|
|
- prefixes, minRev := s.scanUnsync()
|
|
|
curRev := s.store.currentRev.main
|
|
|
+ compactionRev := s.store.compactMainRev
|
|
|
+ minRev := s.unsynced.scanMinRev(curRev, compactionRev)
|
|
|
minBytes, maxBytes := newRevBytes(), newRevBytes()
|
|
|
revToBytes(revision{main: minRev}, minBytes)
|
|
|
revToBytes(revision{main: curRev + 1}, maxBytes)
|
|
|
@@ -360,10 +264,10 @@ func (s *watchableStore) syncWatchers() {
|
|
|
tx := s.store.b.BatchTx()
|
|
|
tx.Lock()
|
|
|
revs, vs := tx.UnsafeRange(keyBucketName, minBytes, maxBytes, 0)
|
|
|
- evs := kvsToEvents(revs, vs, s.unsynced, prefixes)
|
|
|
+ evs := kvsToEvents(&s.unsynced, revs, vs)
|
|
|
tx.Unlock()
|
|
|
|
|
|
- for w, eb := range newWatcherBatch(s.unsynced, evs) {
|
|
|
+ for w, eb := range newWatcherBatch(&s.unsynced, evs) {
|
|
|
select {
|
|
|
// s.store.Rev also uses Lock, so just return directly
|
|
|
case w.ch <- WatchResponse{WatchID: w.id, Events: eb.evs, Revision: s.store.currentRev.main}:
|
|
|
@@ -383,56 +287,18 @@ func (s *watchableStore) syncWatchers() {
|
|
|
s.unsynced.delete(w)
|
|
|
}
|
|
|
|
|
|
- slowWatcherGauge.Set(float64(len(s.unsynced)))
|
|
|
-}
|
|
|
-
|
|
|
-func (s *watchableStore) scanUnsync() (prefixes map[string]struct{}, minRev int64) {
|
|
|
- curRev := s.store.currentRev.main
|
|
|
- compactionRev := s.store.compactMainRev
|
|
|
-
|
|
|
- prefixes = make(map[string]struct{})
|
|
|
- minRev = int64(math.MaxInt64)
|
|
|
- for _, set := range s.unsynced {
|
|
|
- for w := range set {
|
|
|
- k := string(w.key)
|
|
|
-
|
|
|
- if w.cur > curRev {
|
|
|
- panic("watcher current revision should not exceed current revision")
|
|
|
- }
|
|
|
-
|
|
|
- if w.cur < compactionRev {
|
|
|
- select {
|
|
|
- case w.ch <- WatchResponse{WatchID: w.id, CompactRevision: compactionRev}:
|
|
|
- s.unsynced.delete(w)
|
|
|
- default:
|
|
|
- // retry next time
|
|
|
- }
|
|
|
- continue
|
|
|
- }
|
|
|
-
|
|
|
- if minRev > w.cur {
|
|
|
- minRev = w.cur
|
|
|
- }
|
|
|
-
|
|
|
- if w.prefix {
|
|
|
- prefixes[k] = struct{}{}
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- return prefixes, minRev
|
|
|
+ slowWatcherGauge.Set(float64(s.unsynced.size()))
|
|
|
}
|
|
|
|
|
|
// kvsToEvents gets all events for the watchers from all key-value pairs
|
|
|
-func kvsToEvents(revs, vals [][]byte, wsk watcherSetByKey, pfxs map[string]struct{}) (evs []storagepb.Event) {
|
|
|
+func kvsToEvents(wg *watcherGroup, revs, vals [][]byte) (evs []storagepb.Event) {
|
|
|
for i, v := range vals {
|
|
|
var kv storagepb.KeyValue
|
|
|
if err := kv.Unmarshal(v); err != nil {
|
|
|
log.Panicf("storage: cannot unmarshal event: %v", err)
|
|
|
}
|
|
|
|
|
|
- k := string(kv.Key)
|
|
|
- if _, ok := wsk.getSetByKey(k); !ok && !matchPrefix(k, pfxs) {
|
|
|
+ if !wg.contains(string(kv.Key)) {
|
|
|
continue
|
|
|
}
|
|
|
|
|
|
@@ -450,26 +316,19 @@ func kvsToEvents(revs, vals [][]byte, wsk watcherSetByKey, pfxs map[string]struc
|
|
|
// notify notifies the fact that given event at the given rev just happened to
|
|
|
// watchers that watch on the key of the event.
|
|
|
func (s *watchableStore) notify(rev int64, evs []storagepb.Event) {
|
|
|
- we := newWatcherBatch(s.synced, evs)
|
|
|
- for _, wm := range s.synced {
|
|
|
- for w := range wm {
|
|
|
- eb, ok := we[w]
|
|
|
- if !ok {
|
|
|
- continue
|
|
|
- }
|
|
|
- if eb.revs != 1 {
|
|
|
- panic("unexpected multiple revisions in notification")
|
|
|
- }
|
|
|
- select {
|
|
|
- case w.ch <- WatchResponse{WatchID: w.id, Events: eb.evs, Revision: s.Rev()}:
|
|
|
- pendingEventsGauge.Add(float64(len(eb.evs)))
|
|
|
- default:
|
|
|
- // move slow watcher to unsynced
|
|
|
- w.cur = rev
|
|
|
- s.unsynced.add(w)
|
|
|
- delete(wm, w)
|
|
|
- slowWatcherGauge.Inc()
|
|
|
- }
|
|
|
+ for w, eb := range newWatcherBatch(&s.synced, evs) {
|
|
|
+ if eb.revs != 1 {
|
|
|
+ panic("unexpected multiple revisions in notification")
|
|
|
+ }
|
|
|
+ select {
|
|
|
+ case w.ch <- WatchResponse{WatchID: w.id, Events: eb.evs, Revision: s.Rev()}:
|
|
|
+ pendingEventsGauge.Add(float64(len(eb.evs)))
|
|
|
+ default:
|
|
|
+ // move slow watcher to unsynced
|
|
|
+ w.cur = rev
|
|
|
+ s.unsynced.add(w)
|
|
|
+ s.synced.delete(w)
|
|
|
+ slowWatcherGauge.Inc()
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
@@ -479,9 +338,9 @@ func (s *watchableStore) rev() int64 { return s.store.Rev() }
|
|
|
type watcher struct {
|
|
|
// the watcher key
|
|
|
key []byte
|
|
|
- // prefix indicates if watcher is on a key or a prefix.
|
|
|
- // If prefix is true, the watcher is on a prefix.
|
|
|
- prefix bool
|
|
|
+ // end indicates the end of the range to watch.
|
|
|
+ // If end is set, the watcher is on a range.
|
|
|
+ end []byte
|
|
|
// cur is the current watcher revision.
|
|
|
// If cur is behind the current revision of the KV,
|
|
|
// watcher is unsynced and needs to catch up.
|
|
|
@@ -492,42 +351,3 @@ type watcher struct {
|
|
|
// The chan might be shared with other watchers.
|
|
|
ch chan<- WatchResponse
|
|
|
}
|
|
|
-
|
|
|
-// newWatcherBatch maps watchers to their matched events. It enables quick
|
|
|
-// events look up by watcher.
|
|
|
-func newWatcherBatch(sm watcherSetByKey, evs []storagepb.Event) watcherBatch {
|
|
|
- wb := make(watcherBatch)
|
|
|
- for _, ev := range evs {
|
|
|
- key := string(ev.Kv.Key)
|
|
|
-
|
|
|
- // check all prefixes of the key to notify all corresponded watchers
|
|
|
- for i := 0; i <= len(key); i++ {
|
|
|
- for w := range sm[key[:i]] {
|
|
|
- // don't double notify
|
|
|
- if ev.Kv.ModRevision < w.cur {
|
|
|
- continue
|
|
|
- }
|
|
|
-
|
|
|
- // the watcher needs to be notified when either it watches prefix or
|
|
|
- // the key is exactly matched.
|
|
|
- if !w.prefix && i != len(ev.Kv.Key) {
|
|
|
- continue
|
|
|
- }
|
|
|
- wb.add(w, ev)
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- return wb
|
|
|
-}
|
|
|
-
|
|
|
-// matchPrefix returns true if key has any matching prefix
|
|
|
-// from prefixes map.
|
|
|
-func matchPrefix(key string, prefixes map[string]struct{}) bool {
|
|
|
- for p := range prefixes {
|
|
|
- if strings.HasPrefix(key, p) {
|
|
|
- return true
|
|
|
- }
|
|
|
- }
|
|
|
- return false
|
|
|
-}
|