etcd/mvcc/watchable_store.go

549 lines
12 KiB
Go
Raw Normal View History

2016-05-13 06:50:33 +03:00
// Copyright 2015 The etcd Authors
2015-09-15 23:54:11 +03:00
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
2016-04-25 22:32:58 +03:00
package mvcc
import (
"sync"
"time"
2016-01-05 21:16:50 +03:00
"github.com/coreos/etcd/lease"
2016-04-25 22:32:58 +03:00
"github.com/coreos/etcd/mvcc/backend"
"github.com/coreos/etcd/mvcc/mvccpb"
)
const (
// chanBufLen is the length of the buffered chan
// for sending out watched events.
// TODO: find a good buf value. 1024 is just a random one that
// seems to be reasonable.
chanBufLen = 1024
// maxWatchersPerSync is the number of watchers to sync in a single batch
maxWatchersPerSync = 512
)
type watchable interface {
2016-02-26 19:55:28 +03:00
watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse) (*watcher, cancelFunc)
2016-03-03 09:34:54 +03:00
progress(w *watcher)
rev() int64
}
type watchableStore struct {
mu sync.Mutex
*store
// victims are watcher batches that were blocked on the watch channel
victims []watcherBatch
victimc chan struct{}
2016-01-03 07:20:22 +03:00
// contains all unsynced watchers that needs to sync with events that have happened
2016-02-26 19:55:28 +03:00
unsynced watcherGroup
storage: use map for watchable store unsynced This is for `TODO: use map to reduce cancel cost`. I switched slice to map, and benchmark results show that map implementation performs better, as follows: ``` [1]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 215212 1307 -99.39% BenchmarkWatchableStoreUnsyncedCancel-2 120453 710 -99.41% BenchmarkWatchableStoreUnsyncedCancel-4 120765 748 -99.38% BenchmarkWatchableStoreUnsyncedCancel-8 121391 719 -99.41% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 200 1 -99.50% BenchmarkWatchableStoreUnsyncedCancel-2 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [2]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 212550 1117 -99.47% BenchmarkWatchableStoreUnsyncedCancel-2 120927 691 -99.43% BenchmarkWatchableStoreUnsyncedCancel-4 120752 699 -99.42% BenchmarkWatchableStoreUnsyncedCancel-8 121012 688 -99.43% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 197 1 -99.49% BenchmarkWatchableStoreUnsyncedCancel-2 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [3]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 214268 1183 -99.45% BenchmarkWatchableStoreUnsyncedCancel-2 120763 759 -99.37% BenchmarkWatchableStoreUnsyncedCancel-4 120321 708 -99.41% BenchmarkWatchableStoreUnsyncedCancel-8 121628 680 -99.44% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 200 1 -99.50% BenchmarkWatchableStoreUnsyncedCancel-2 139 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [4]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 208332 1089 -99.48% BenchmarkWatchableStoreUnsyncedCancel-2 121011 691 -99.43% BenchmarkWatchableStoreUnsyncedCancel-4 120678 681 -99.44% BenchmarkWatchableStoreUnsyncedCancel-8 121303 721 -99.41% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 194 1 -99.48% BenchmarkWatchableStoreUnsyncedCancel-2 139 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 139 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [5]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 211900 1097 -99.48% BenchmarkWatchableStoreUnsyncedCancel-2 121795 753 -99.38% BenchmarkWatchableStoreUnsyncedCancel-4 123182 700 -99.43% BenchmarkWatchableStoreUnsyncedCancel-8 122820 688 -99.44% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 198 1 -99.49% BenchmarkWatchableStoreUnsyncedCancel-2 140 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 141 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 141 0 -100.00% ```
2015-10-21 22:06:27 +03:00
2016-01-03 07:20:22 +03:00
// 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.
2016-02-26 19:55:28 +03:00
synced watcherGroup
stopc chan struct{}
wg sync.WaitGroup
}
// cancelFunc updates unsynced and synced maps when running
// cancel operations.
type cancelFunc func()
2016-03-30 21:37:55 +03:00
func New(b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter) ConsistentWatchableKV {
return newWatchableStore(b, le, ig)
}
func newWatchableStore(b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter) *watchableStore {
s := &watchableStore{
2016-03-30 21:37:55 +03:00
store: NewStore(b, le, ig),
victimc: make(chan struct{}, 1),
2016-02-26 19:55:28 +03:00
unsynced: newWatcherGroup(),
synced: newWatcherGroup(),
storage: use map for watchable store unsynced This is for `TODO: use map to reduce cancel cost`. I switched slice to map, and benchmark results show that map implementation performs better, as follows: ``` [1]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 215212 1307 -99.39% BenchmarkWatchableStoreUnsyncedCancel-2 120453 710 -99.41% BenchmarkWatchableStoreUnsyncedCancel-4 120765 748 -99.38% BenchmarkWatchableStoreUnsyncedCancel-8 121391 719 -99.41% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 200 1 -99.50% BenchmarkWatchableStoreUnsyncedCancel-2 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [2]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 212550 1117 -99.47% BenchmarkWatchableStoreUnsyncedCancel-2 120927 691 -99.43% BenchmarkWatchableStoreUnsyncedCancel-4 120752 699 -99.42% BenchmarkWatchableStoreUnsyncedCancel-8 121012 688 -99.43% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 197 1 -99.49% BenchmarkWatchableStoreUnsyncedCancel-2 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [3]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 214268 1183 -99.45% BenchmarkWatchableStoreUnsyncedCancel-2 120763 759 -99.37% BenchmarkWatchableStoreUnsyncedCancel-4 120321 708 -99.41% BenchmarkWatchableStoreUnsyncedCancel-8 121628 680 -99.44% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 200 1 -99.50% BenchmarkWatchableStoreUnsyncedCancel-2 139 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 138 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [4]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 208332 1089 -99.48% BenchmarkWatchableStoreUnsyncedCancel-2 121011 691 -99.43% BenchmarkWatchableStoreUnsyncedCancel-4 120678 681 -99.44% BenchmarkWatchableStoreUnsyncedCancel-8 121303 721 -99.41% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 194 1 -99.48% BenchmarkWatchableStoreUnsyncedCancel-2 139 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 139 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 139 0 -100.00% [5]: benchmark old ns/op new ns/op delta BenchmarkWatchableStoreUnsyncedCancel 211900 1097 -99.48% BenchmarkWatchableStoreUnsyncedCancel-2 121795 753 -99.38% BenchmarkWatchableStoreUnsyncedCancel-4 123182 700 -99.43% BenchmarkWatchableStoreUnsyncedCancel-8 122820 688 -99.44% benchmark old allocs new allocs delta BenchmarkWatchableStoreUnsyncedCancel 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-2 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-4 0 0 +0.00% BenchmarkWatchableStoreUnsyncedCancel-8 0 0 +0.00% benchmark old bytes new bytes delta BenchmarkWatchableStoreUnsyncedCancel 198 1 -99.49% BenchmarkWatchableStoreUnsyncedCancel-2 140 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-4 141 0 -100.00% BenchmarkWatchableStoreUnsyncedCancel-8 141 0 -100.00% ```
2015-10-21 22:06:27 +03:00
stopc: make(chan struct{}),
}
if s.le != nil {
// use this store as the deleter so revokes trigger watch events
s.le.SetRangeDeleter(s)
}
s.wg.Add(2)
2016-01-03 07:20:22 +03:00
go s.syncWatchersLoop()
go s.syncVictimsLoop()
return s
}
2016-01-05 21:16:50 +03:00
func (s *watchableStore) Put(key, value []byte, lease lease.LeaseID) (rev int64) {
s.mu.Lock()
defer s.mu.Unlock()
2016-01-05 02:12:59 +03:00
rev = s.store.Put(key, value, lease)
changes := s.store.getChanges()
if len(changes) != 1 {
2016-05-21 08:30:50 +03:00
plog.Panicf("unexpected len(changes) != 1 after put")
}
2016-04-25 22:32:58 +03:00
ev := mvccpb.Event{
Type: mvccpb.PUT,
Kv: &changes[0],
}
2016-04-25 22:32:58 +03:00
s.notify(rev, []mvccpb.Event{ev})
return rev
}
func (s *watchableStore) DeleteRange(key, end []byte) (n, rev int64) {
s.mu.Lock()
defer s.mu.Unlock()
n, rev = s.store.DeleteRange(key, end)
changes := s.store.getChanges()
if len(changes) != int(n) {
2016-05-21 08:30:50 +03:00
plog.Panicf("unexpected len(changes) != n after deleteRange")
}
if n == 0 {
return n, rev
}
2016-04-25 22:32:58 +03:00
evs := make([]mvccpb.Event, n)
2016-04-29 19:24:46 +03:00
for i := range changes {
2016-04-25 22:32:58 +03:00
evs[i] = mvccpb.Event{
Type: mvccpb.DELETE,
2016-04-29 19:24:46 +03:00
Kv: &changes[i]}
evs[i].Kv.ModRevision = rev
}
2016-02-03 10:02:15 +03:00
s.notify(rev, evs)
return n, rev
}
func (s *watchableStore) TxnBegin() int64 {
s.mu.Lock()
return s.store.TxnBegin()
}
func (s *watchableStore) TxnEnd(txnID int64) error {
err := s.store.TxnEnd(txnID)
if err != nil {
return err
}
changes := s.getChanges()
if len(changes) == 0 {
s.mu.Unlock()
return nil
}
rev := s.store.Rev()
2016-04-25 22:32:58 +03:00
evs := make([]mvccpb.Event, len(changes))
for i, change := range changes {
switch change.CreateRevision {
case 0:
2016-04-25 22:32:58 +03:00
evs[i] = mvccpb.Event{
Type: mvccpb.DELETE,
Kv: &changes[i]}
evs[i].Kv.ModRevision = rev
default:
2016-04-25 22:32:58 +03:00
evs[i] = mvccpb.Event{
Type: mvccpb.PUT,
Kv: &changes[i]}
}
}
s.notify(rev, evs)
s.mu.Unlock()
return nil
}
func (s *watchableStore) Close() error {
close(s.stopc)
s.wg.Wait()
return s.store.Close()
}
func (s *watchableStore) NewWatchStream() WatchStream {
watchStreamGauge.Inc()
return &watchStream{
watchable: s,
ch: make(chan WatchResponse, chanBufLen),
cancels: make(map[WatchID]cancelFunc),
2016-03-03 09:34:54 +03:00
watchers: make(map[WatchID]*watcher),
}
}
2016-02-26 19:55:28 +03:00
func (s *watchableStore) watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse) (*watcher, cancelFunc) {
s.mu.Lock()
defer s.mu.Unlock()
2016-01-03 07:20:22 +03:00
wa := &watcher{
key: key,
end: end,
minRev: startRev,
id: id,
ch: ch,
}
s.store.mu.Lock()
synced := startRev > s.store.currentRev.main || startRev == 0
if synced {
wa.minRev = s.store.currentRev.main + 1
if startRev > wa.minRev {
wa.minRev = startRev
2016-03-07 04:41:30 +03:00
}
}
s.store.mu.Unlock()
if synced {
s.synced.add(wa)
} else {
2016-01-03 07:20:22 +03:00
slowWatcherGauge.Inc()
2016-02-03 07:06:53 +03:00
s.unsynced.add(wa)
}
2016-01-03 07:20:22 +03:00
watcherGauge.Inc()
return wa, func() { s.cancelWatcher(wa) }
}
// cancelWatcher removes references of the watcher from the watchableStore
func (s *watchableStore) cancelWatcher(wa *watcher) {
for {
s.mu.Lock()
2016-02-03 07:06:53 +03:00
if s.unsynced.delete(wa) {
2016-01-03 07:20:22 +03:00
slowWatcherGauge.Dec()
break
2016-05-19 23:51:51 +03:00
} else if s.synced.delete(wa) {
break
} else if wa.compacted {
break
}
if !wa.victim {
panic("watcher not victim but not in watch groups")
}
var victimBatch watcherBatch
for _, wb := range s.victims {
if wb[wa] != nil {
victimBatch = wb
break
}
}
if victimBatch != nil {
slowWatcherGauge.Dec()
delete(victimBatch, wa)
break
}
2016-05-19 23:51:51 +03:00
// victim being processed so not accessible; retry
s.mu.Unlock()
time.Sleep(time.Millisecond)
}
watcherGauge.Dec()
s.mu.Unlock()
}
2016-02-01 08:42:39 +03:00
// syncWatchersLoop syncs the watcher in the unsynced map every 100ms.
2016-01-03 07:20:22 +03:00
func (s *watchableStore) syncWatchersLoop() {
defer s.wg.Done()
for {
s.mu.Lock()
st := time.Now()
lastUnsyncedWatchers := s.unsynced.size()
2016-01-03 07:20:22 +03:00
s.syncWatchers()
unsyncedWatchers := s.unsynced.size()
s.mu.Unlock()
syncDuration := time.Since(st)
waitDuration := 100 * time.Millisecond
// more work pending?
if unsyncedWatchers != 0 && lastUnsyncedWatchers > unsyncedWatchers {
// be fair to other store operations by yielding time taken
waitDuration = syncDuration
}
select {
case <-time.After(waitDuration):
case <-s.stopc:
return
}
}
}
// syncVictimsLoop tries to write precomputed watcher responses to
// watchers that had a blocked watcher channel
func (s *watchableStore) syncVictimsLoop() {
defer s.wg.Done()
for {
for s.moveVictims() != 0 {
// try to update all victim watchers
}
s.mu.Lock()
isEmpty := len(s.victims) == 0
s.mu.Unlock()
var tickc <-chan time.Time
if !isEmpty {
tickc = time.After(10 * time.Millisecond)
}
select {
case <-tickc:
case <-s.victimc:
case <-s.stopc:
return
}
}
}
// moveVictims tries to update watches with already pending event data
func (s *watchableStore) moveVictims() (moved int) {
s.mu.Lock()
victims := s.victims
s.victims = nil
s.mu.Unlock()
var newVictim watcherBatch
for _, wb := range victims {
// try to send responses again
for w, eb := range wb {
// watcher has observed the store up to, but not including, w.minRev
rev := w.minRev - 1
select {
case w.ch <- WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}:
pendingEventsGauge.Add(float64(len(eb.evs)))
default:
if newVictim == nil {
newVictim = make(watcherBatch)
}
newVictim[w] = eb
continue
}
moved++
}
// assign completed victim watchers to unsync/sync
s.mu.Lock()
s.store.mu.Lock()
curRev := s.store.currentRev.main
for w, eb := range wb {
if newVictim != nil && newVictim[w] != nil {
// couldn't send watch response; stays victim
continue
}
w.victim = false
if eb.moreRev != 0 {
w.minRev = eb.moreRev
}
if w.minRev <= curRev {
s.unsynced.add(w)
} else {
slowWatcherGauge.Dec()
s.synced.add(w)
}
}
s.store.mu.Unlock()
s.mu.Unlock()
}
if len(newVictim) > 0 {
s.mu.Lock()
s.victims = append(s.victims, newVictim)
s.mu.Unlock()
}
return moved
}
// syncWatchers syncs unsynced watchers by:
// 1. choose a set of watchers from the unsynced watcher group
// 2. iterate over the set to get the minimum revision and remove compacted watchers
// 3. use minimum revision to get all key-value pairs and send those events to watchers
// 4. remove synced watchers in set from unsynced group and move to synced group
2016-01-03 07:20:22 +03:00
func (s *watchableStore) syncWatchers() {
2016-02-26 19:55:28 +03:00
if s.unsynced.size() == 0 {
return
}
s.store.mu.Lock()
defer s.store.mu.Unlock()
2016-01-03 07:20:22 +03:00
// 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
curRev := s.store.currentRev.main
2016-02-26 19:55:28 +03:00
compactionRev := s.store.compactMainRev
wg, minRev := s.unsynced.choose(maxWatchersPerSync, curRev, compactionRev)
minBytes, maxBytes := newRevBytes(), newRevBytes()
revToBytes(revision{main: minRev}, minBytes)
revToBytes(revision{main: curRev + 1}, maxBytes)
// UnsafeRange returns keys and values. And in boltdb, keys are revisions.
// values are actual key-value pairs in backend.
tx := s.store.b.BatchTx()
tx.Lock()
revs, vs := tx.UnsafeRange(keyBucketName, minBytes, maxBytes, 0)
evs := kvsToEvents(wg, revs, vs)
tx.Unlock()
var victims watcherBatch
wb := newWatcherBatch(wg, evs)
for w := range wg.watchers {
w.minRev = curRev + 1
eb, ok := wb[w]
if !ok {
// bring un-notified watcher to synced
s.synced.add(w)
s.unsynced.delete(w)
continue
}
2016-03-03 09:34:54 +03:00
if eb.moreRev != 0 {
w.minRev = eb.moreRev
}
select {
case w.ch <- WatchResponse{WatchID: w.id, Events: eb.evs, Revision: curRev}:
pendingEventsGauge.Add(float64(len(eb.evs)))
default:
if victims == nil {
victims = make(watcherBatch)
}
w.victim = true
}
if w.victim {
victims[w] = eb
} else {
if eb.moreRev != 0 {
// stay unsynced; more to read
continue
}
s.synced.add(w)
}
s.unsynced.delete(w)
}
s.addVictim(victims)
vsz := 0
for _, v := range s.victims {
vsz += len(v)
}
slowWatcherGauge.Set(float64(s.unsynced.size() + vsz))
}
// kvsToEvents gets all events for the watchers from all key-value pairs
2016-04-25 22:32:58 +03:00
func kvsToEvents(wg *watcherGroup, revs, vals [][]byte) (evs []mvccpb.Event) {
for i, v := range vals {
2016-04-25 22:32:58 +03:00
var kv mvccpb.KeyValue
if err := kv.Unmarshal(v); err != nil {
2016-05-21 08:30:50 +03:00
plog.Panicf("cannot unmarshal event: %v", err)
}
2016-02-26 19:55:28 +03:00
if !wg.contains(string(kv.Key)) {
continue
}
2016-04-25 22:32:58 +03:00
ty := mvccpb.PUT
if isTombstone(revs[i]) {
2016-04-25 22:32:58 +03:00
ty = mvccpb.DELETE
// patch in mod revision so watchers won't skip
kv.ModRevision = bytesToRev(revs[i]).main
}
2016-04-25 22:32:58 +03:00
evs = append(evs, mvccpb.Event{Kv: &kv, Type: ty})
}
return evs
}
// notify notifies the fact that given event at the given rev just happened to
2016-01-03 07:20:22 +03:00
// watchers that watch on the key of the event.
2016-04-25 22:32:58 +03:00
func (s *watchableStore) notify(rev int64, evs []mvccpb.Event) {
var victim watcherBatch
2016-02-26 19:55:28 +03:00
for w, eb := range newWatcherBatch(&s.synced, evs) {
if eb.revs != 1 {
2016-05-21 08:30:50 +03:00
plog.Panicf("unexpected multiple revisions in notification")
2016-02-26 19:55:28 +03:00
}
select {
case w.ch <- WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}:
2016-02-26 19:55:28 +03:00
pendingEventsGauge.Add(float64(len(eb.evs)))
default:
// move slow watcher to victims
w.minRev = rev + 1
if victim == nil {
victim = make(watcherBatch)
}
w.victim = true
victim[w] = eb
2016-02-26 19:55:28 +03:00
s.synced.delete(w)
slowWatcherGauge.Inc()
}
}
s.addVictim(victim)
}
func (s *watchableStore) addVictim(victim watcherBatch) {
if victim == nil {
return
}
s.victims = append(s.victims, victim)
select {
case s.victimc <- struct{}{}:
default:
}
}
func (s *watchableStore) rev() int64 { return s.store.Rev() }
2016-03-03 09:34:54 +03:00
func (s *watchableStore) progress(w *watcher) {
s.mu.Lock()
defer s.mu.Unlock()
if _, ok := s.synced.watchers[w]; ok {
select {
case w.ch <- WatchResponse{WatchID: w.id, Revision: s.rev()}:
default:
// If the ch is full, this watcher is receiving events.
// We do not need to send progress at all.
}
}
}
2016-01-03 07:20:22 +03:00
type watcher struct {
// the watcher key
key []byte
2016-02-26 19:55:28 +03:00
// end indicates the end of the range to watch.
// If end is set, the watcher is on a range.
end []byte
2016-03-07 04:41:30 +03:00
// victim is set when ch is blocked and undergoing victim processing
victim bool
// compacted is set when the watcher is removed because of compaction
compacted bool
// minRev is the minimum revision update the watcher will accept
minRev int64
id WatchID
// a chan to send out the watch response.
2016-01-03 07:20:22 +03:00
// The chan might be shared with other watchers.
ch chan<- WatchResponse
}