etcd/mvcc/watchable_store.go

549 lines
13 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"
"go.etcd.io/etcd/api/v3/mvccpb"
"go.etcd.io/etcd/pkg/v3/traceutil"
"go.etcd.io/etcd/v3/etcdserver/cindex"
"go.etcd.io/etcd/v3/lease"
"go.etcd.io/etcd/v3/mvcc/backend"
2020-03-15 16:32:39 +03:00
"go.uber.org/zap"
)
// non-const so modifiable by tests
var (
// chanBufLen is the length of the buffered chan
// for sending out watched events.
// See https://github.com/etcd-io/etcd/issues/11906 for more detail.
chanBufLen = 128
// maxWatchersPerSync is the number of watchers to sync in a single batch
maxWatchersPerSync = 512
)
type watchable interface {
watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse, fcs ...FilterFunc) (*watcher, cancelFunc)
2016-03-03 09:34:54 +03:00
progress(w *watcher)
rev() int64
}
type watchableStore struct {
*store
// mu protects watcher groups and batches. It should never be locked
// before locking store.mu to avoid deadlock.
mu sync.RWMutex
// 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()
2020-03-15 16:32:39 +03:00
func New(lg *zap.Logger, b backend.Backend, le lease.Lessor, ci cindex.ConsistentIndexer, cfg StoreConfig) ConsistentWatchableKV {
return newWatchableStore(lg, b, le, ci, cfg)
2016-03-30 21:37:55 +03:00
}
2020-03-15 16:32:39 +03:00
func newWatchableStore(lg *zap.Logger, b backend.Backend, le lease.Lessor, ci cindex.ConsistentIndexer, cfg StoreConfig) *watchableStore {
2020-02-05 19:18:15 +03:00
if lg == nil {
lg = zap.NewNop()
}
s := &watchableStore{
2020-03-15 16:32:39 +03:00
store: NewStore(lg, b, le, ci, cfg),
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{}),
}
s.store.ReadView = &readView{s}
s.store.WriteView = &writeView{s}
if s.le != nil {
// use this store as the deleter so revokes trigger watch events
s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write(traceutil.TODO()) })
}
s.wg.Add(2)
2016-01-03 07:20:22 +03:00
go s.syncWatchersLoop()
go s.syncVictimsLoop()
return s
}
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),
}
}
func (s *watchableStore) watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse, fcs ...FilterFunc) (*watcher, cancelFunc) {
2016-01-03 07:20:22 +03:00
wa := &watcher{
key: key,
end: end,
minRev: startRev,
id: id,
ch: ch,
fcs: fcs,
}
s.mu.Lock()
s.revMu.RLock()
synced := startRev > s.store.currentRev || startRev == 0
if synced {
wa.minRev = s.store.currentRev + 1
if startRev > wa.minRev {
wa.minRev = startRev
2016-03-07 04:41:30 +03:00
}
}
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)
}
s.revMu.RUnlock()
s.mu.Unlock()
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()
watcherGauge.Dec()
break
2016-05-19 23:51:51 +03:00
} else if s.synced.delete(wa) {
watcherGauge.Dec()
break
} else if wa.compacted {
watcherGauge.Dec()
break
} else if wa.ch == nil {
// already canceled (e.g., cancel/close race)
break
}
if !wa.victim {
s.mu.Unlock()
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()
watcherGauge.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)
}
wa.ch = nil
s.mu.Unlock()
}
func (s *watchableStore) Restore(b backend.Backend) error {
s.mu.Lock()
defer s.mu.Unlock()
err := s.store.Restore(b)
if err != nil {
return err
}
for wa := range s.synced.watchers {
mvcc: fix panic by allowing future revision watcher from restore operation This also happens without gRPC proxy. Fix panic when gRPC proxy leader watcher is restored: ``` go test -v -tags cluster_proxy -cpu 4 -race -run TestV3WatchRestoreSnapshotUnsync === RUN TestV3WatchRestoreSnapshotUnsync panic: watcher minimum revision 9223372036854775805 should not exceed current revision 16 goroutine 156 [running]: github.com/coreos/etcd/mvcc.(*watcherGroup).chooseAll(0xc4202b8720, 0x10, 0xffffffffffffffff, 0x1) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watcher_group.go:242 +0x3b5 github.com/coreos/etcd/mvcc.(*watcherGroup).choose(0xc4202b8720, 0x200, 0x10, 0xffffffffffffffff, 0xc420253378, 0xc420253378) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watcher_group.go:225 +0x289 github.com/coreos/etcd/mvcc.(*watchableStore).syncWatchers(0xc4202b86e0, 0x0) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watchable_store.go:340 +0x237 github.com/coreos/etcd/mvcc.(*watchableStore).syncWatchersLoop(0xc4202b86e0) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watchable_store.go:214 +0x280 created by github.com/coreos/etcd/mvcc.newWatchableStore /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watchable_store.go:90 +0x477 exit status 2 FAIL github.com/coreos/etcd/integration 2.551s ``` gRPC proxy spawns a watcher with a key "proxy-namespace__lostleader" and watch revision "int64(math.MaxInt64 - 2)" to detect leader loss. But, when the partitioned node restores, this watcher triggers panic with "watcher minimum revision ... should not exceed current ...". This check was added a long time ago, by my PR, when there was no gRPC proxy: https://github.com/coreos/etcd/pull/4043#discussion_r48457145 > we can remove this checking actually. it is impossible for a unsynced watching to have a future rev. or we should just panic here. However, now it's possible that a unsynced watcher has a future revision, when it was moved from a synced watcher group through restore operation. This PR adds "restore" flag to indicate that a watcher was moved from the synced watcher group with restore operation. Otherwise, the watcher with future revision in an unsynced watcher group would still panic. Example logs with future revision watcher from restore operation: ``` {"level":"info","ts":1527196358.9057755,"caller":"mvcc/watcher_group.go:261","msg":"choosing future revision watcher from restore operation","watch-key":"proxy-namespace__lostleader","watch-revision":9223372036854775805,"current-revision":16} {"level":"info","ts":1527196358.910349,"caller":"mvcc/watcher_group.go:261","msg":"choosing future revision watcher from restore operation","watch-key":"proxy-namespace__lostleader","watch-revision":9223372036854775805,"current-revision":16} ``` Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
2018-05-25 20:04:54 +03:00
wa.restore = true
s.unsynced.add(wa)
}
s.synced = newWatcherGroup()
return nil
}
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.RLock()
st := time.Now()
lastUnsyncedWatchers := s.unsynced.size()
s.mu.RUnlock()
unsyncedWatchers := 0
if lastUnsyncedWatchers > 0 {
unsyncedWatchers = s.syncWatchers()
}
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.RLock()
isEmpty := len(s.victims) == 0
s.mu.RUnlock()
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
if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
pendingEventsGauge.Add(float64(len(eb.evs)))
} else {
if newVictim == nil {
newVictim = make(watcherBatch)
}
newVictim[w] = eb
continue
}
moved++
}
// assign completed victim watchers to unsync/sync
s.mu.Lock()
s.store.revMu.RLock()
curRev := s.store.currentRev
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.revMu.RUnlock()
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
func (s *watchableStore) syncWatchers() int {
s.mu.Lock()
defer s.mu.Unlock()
2016-02-26 19:55:28 +03:00
if s.unsynced.size() == 0 {
return 0
}
s.store.revMu.RLock()
defer s.store.revMu.RUnlock()
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
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.ReadTx()
tx.RLock()
revs, vs := tx.UnsafeRange(keyBucketName, minBytes, maxBytes, 0)
tx.RUnlock()
var evs []mvccpb.Event
2020-02-05 19:18:15 +03:00
evs = kvsToEvents(s.store.lg, wg, revs, vs)
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
}
if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: curRev}) {
pendingEventsGauge.Add(float64(len(eb.evs)))
} else {
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))
return s.unsynced.size()
}
// kvsToEvents gets all events for the watchers from all key-value pairs
func kvsToEvents(lg *zap.Logger, 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 {
2020-02-05 19:18:15 +03:00
lg.Panic("failed to unmarshal mvccpb.KeyValue", zap.Error(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 {
2020-02-05 19:18:15 +03:00
s.store.lg.Panic(
"unexpected multiple revisions in watch notification",
zap.Int("number-of-revisions", eb.revs),
)
2016-02-26 19:55:28 +03:00
}
if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
2016-02-26 19:55:28 +03:00
pendingEventsGauge.Add(float64(len(eb.evs)))
} else {
// 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.RLock()
defer s.mu.RUnlock()
2016-03-03 09:34:54 +03:00
if _, ok := s.synced.watchers[w]; ok {
w.send(WatchResponse{WatchID: w.id, Revision: s.rev()})
// If the ch is full, this watcher is receiving events.
// We do not need to send progress at all.
2016-03-03 09:34:54 +03:00
}
}
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
mvcc: fix panic by allowing future revision watcher from restore operation This also happens without gRPC proxy. Fix panic when gRPC proxy leader watcher is restored: ``` go test -v -tags cluster_proxy -cpu 4 -race -run TestV3WatchRestoreSnapshotUnsync === RUN TestV3WatchRestoreSnapshotUnsync panic: watcher minimum revision 9223372036854775805 should not exceed current revision 16 goroutine 156 [running]: github.com/coreos/etcd/mvcc.(*watcherGroup).chooseAll(0xc4202b8720, 0x10, 0xffffffffffffffff, 0x1) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watcher_group.go:242 +0x3b5 github.com/coreos/etcd/mvcc.(*watcherGroup).choose(0xc4202b8720, 0x200, 0x10, 0xffffffffffffffff, 0xc420253378, 0xc420253378) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watcher_group.go:225 +0x289 github.com/coreos/etcd/mvcc.(*watchableStore).syncWatchers(0xc4202b86e0, 0x0) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watchable_store.go:340 +0x237 github.com/coreos/etcd/mvcc.(*watchableStore).syncWatchersLoop(0xc4202b86e0) /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watchable_store.go:214 +0x280 created by github.com/coreos/etcd/mvcc.newWatchableStore /home/gyuho/go/src/github.com/coreos/etcd/mvcc/watchable_store.go:90 +0x477 exit status 2 FAIL github.com/coreos/etcd/integration 2.551s ``` gRPC proxy spawns a watcher with a key "proxy-namespace__lostleader" and watch revision "int64(math.MaxInt64 - 2)" to detect leader loss. But, when the partitioned node restores, this watcher triggers panic with "watcher minimum revision ... should not exceed current ...". This check was added a long time ago, by my PR, when there was no gRPC proxy: https://github.com/coreos/etcd/pull/4043#discussion_r48457145 > we can remove this checking actually. it is impossible for a unsynced watching to have a future rev. or we should just panic here. However, now it's possible that a unsynced watcher has a future revision, when it was moved from a synced watcher group through restore operation. This PR adds "restore" flag to indicate that a watcher was moved from the synced watcher group with restore operation. Otherwise, the watcher with future revision in an unsynced watcher group would still panic. Example logs with future revision watcher from restore operation: ``` {"level":"info","ts":1527196358.9057755,"caller":"mvcc/watcher_group.go:261","msg":"choosing future revision watcher from restore operation","watch-key":"proxy-namespace__lostleader","watch-revision":9223372036854775805,"current-revision":16} {"level":"info","ts":1527196358.910349,"caller":"mvcc/watcher_group.go:261","msg":"choosing future revision watcher from restore operation","watch-key":"proxy-namespace__lostleader","watch-revision":9223372036854775805,"current-revision":16} ``` Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
2018-05-25 20:04:54 +03:00
// restore is true when the watcher is being restored from leader snapshot
// which means that this watcher has just been moved from "synced" to "unsynced"
// watcher group, possibly with a future revision when it was first added
// to the synced watcher
// "unsynced" watcher revision must always be <= current revision,
// except when the watcher were to be moved from "synced" watcher group
restore bool
// minRev is the minimum revision update the watcher will accept
minRev int64
id WatchID
fcs []FilterFunc
// 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
}
func (w *watcher) send(wr WatchResponse) bool {
progressEvent := len(wr.Events) == 0
if len(w.fcs) != 0 {
ne := make([]mvccpb.Event, 0, len(wr.Events))
for i := range wr.Events {
filtered := false
for _, filter := range w.fcs {
if filter(wr.Events[i]) {
filtered = true
break
}
}
if !filtered {
ne = append(ne, wr.Events[i])
}
}
wr.Events = ne
}
// if all events are filtered out, we should send nothing.
if !progressEvent && len(wr.Events) == 0 {
return true
}
select {
case w.ch <- wr:
return true
default:
return false
}
}