server: Extract corruption detection to dedicated struct

Signed-off-by: Marek Siarkowicz <siarkowicz@google.com>
This commit is contained in:
Marek Siarkowicz 2022-06-13 14:20:07 +02:00
parent d32de2c410
commit 35cbdf3961
4 changed files with 120 additions and 74 deletions

View File

@ -247,8 +247,8 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
// newly started member ("memberInitialized==false") // newly started member ("memberInitialized==false")
// does not need corruption check // does not need corruption check
if memberInitialized { if memberInitialized && srvcfg.InitialCorruptCheck {
if err = e.Server.CheckInitialHashKV(); err != nil { if err = etcdserver.NewCorruptionMonitor(e.cfg.logger, e.Server).InitialCheck(); err != nil {
// set "EtcdServer" to nil, so that it does not block on "EtcdServer.Close()" // set "EtcdServer" to nil, so that it does not block on "EtcdServer.Close()"
// (nothing to close since rafthttp transports have not been started) // (nothing to close since rafthttp transports have not been started)

View File

@ -33,33 +33,78 @@ import (
"go.uber.org/zap" "go.uber.org/zap"
) )
// CheckInitialHashKV compares initial hash values with its peers type corruptionMonitor struct {
lg *zap.Logger
hasher Hasher
}
type Hasher interface {
mvcc.Hasher
ReqTimeout() time.Duration
MemberId() types.ID
PeerHashByRev(int64) []*peerHashKVResp
LinearizableReadNotify(context.Context) error
TriggerCorruptAlarm(uint64)
}
func NewCorruptionMonitor(lg *zap.Logger, s *EtcdServer) *corruptionMonitor {
return &corruptionMonitor{
lg: lg,
hasher: hasherAdapter{s},
}
}
type hasherAdapter struct {
*EtcdServer
}
func (h hasherAdapter) MemberId() types.ID {
return h.EtcdServer.ID()
}
func (h hasherAdapter) Hash() (hash uint32, revision int64, err error) {
return h.EtcdServer.KV().Hash()
}
func (h hasherAdapter) HashByRev(rev int64) (hash uint32, revision int64, compactRev int64, err error) {
return h.EtcdServer.KV().HashByRev(rev)
}
func (h hasherAdapter) ReqTimeout() time.Duration {
return h.EtcdServer.Cfg.ReqTimeout()
}
func (h hasherAdapter) PeerHashByRev(rev int64) []*peerHashKVResp {
return h.EtcdServer.getPeerHashKVs(rev)
}
func (h hasherAdapter) TriggerCorruptAlarm(memberID uint64) {
h.EtcdServer.triggerCorruptAlarm(memberID)
}
// InitialCheck compares initial hash values with its peers
// before serving any peer/client traffic. Only mismatch when hashes // before serving any peer/client traffic. Only mismatch when hashes
// are different at requested revision, with same compact revision. // are different at requested revision, with same compact revision.
func (s *EtcdServer) CheckInitialHashKV() error { func (cm *corruptionMonitor) InitialCheck() error {
if !s.Cfg.InitialCorruptCheck {
return nil
}
lg := s.Logger() cm.lg.Info(
lg.Info(
"starting initial corruption check", "starting initial corruption check",
zap.String("local-member-id", s.ID().String()), zap.String("local-member-id", cm.hasher.MemberId().String()),
zap.Duration("timeout", s.Cfg.ReqTimeout()), zap.Duration("timeout", cm.hasher.ReqTimeout()),
) )
h, rev, crev, err := s.kv.HashByRev(0) h, rev, crev, err := cm.hasher.HashByRev(0)
if err != nil { if err != nil {
return fmt.Errorf("%s failed to fetch hash (%v)", s.ID(), err) return fmt.Errorf("%s failed to fetch hash (%v)", cm.hasher.MemberId(), err)
} }
peers := s.getPeerHashKVs(rev) peers := cm.hasher.PeerHashByRev(rev)
mismatch := 0 mismatch := 0
for _, p := range peers { for _, p := range peers {
if p.resp != nil { if p.resp != nil {
peerID := types.ID(p.resp.Header.MemberId) peerID := types.ID(p.resp.Header.MemberId)
fields := []zap.Field{ fields := []zap.Field{
zap.String("local-member-id", s.ID().String()), zap.String("local-member-id", cm.hasher.MemberId().String()),
zap.Int64("local-member-revision", rev), zap.Int64("local-member-revision", rev),
zap.Int64("local-member-compact-revision", crev), zap.Int64("local-member-compact-revision", crev),
zap.Uint32("local-member-hash", h), zap.Uint32("local-member-hash", h),
@ -72,10 +117,10 @@ func (s *EtcdServer) CheckInitialHashKV() error {
if h != p.resp.Hash { if h != p.resp.Hash {
if crev == p.resp.CompactRevision { if crev == p.resp.CompactRevision {
lg.Warn("found different hash values from remote peer", fields...) cm.lg.Warn("found different hash values from remote peer", fields...)
mismatch++ mismatch++
} else { } else {
lg.Warn("found different compact revision values from remote peer", fields...) cm.lg.Warn("found different compact revision values from remote peer", fields...)
} }
} }
@ -85,9 +130,9 @@ func (s *EtcdServer) CheckInitialHashKV() error {
if p.err != nil { if p.err != nil {
switch p.err { switch p.err {
case rpctypes.ErrFutureRev: case rpctypes.ErrFutureRev:
lg.Warn( cm.lg.Warn(
"cannot fetch hash from slow remote peer", "cannot fetch hash from slow remote peer",
zap.String("local-member-id", s.ID().String()), zap.String("local-member-id", cm.hasher.MemberId().String()),
zap.Int64("local-member-revision", rev), zap.Int64("local-member-revision", rev),
zap.Int64("local-member-compact-revision", crev), zap.Int64("local-member-compact-revision", crev),
zap.Uint32("local-member-hash", h), zap.Uint32("local-member-hash", h),
@ -96,9 +141,9 @@ func (s *EtcdServer) CheckInitialHashKV() error {
zap.Error(err), zap.Error(err),
) )
case rpctypes.ErrCompacted: case rpctypes.ErrCompacted:
lg.Warn( cm.lg.Warn(
"cannot fetch hash from remote peer; local member is behind", "cannot fetch hash from remote peer; local member is behind",
zap.String("local-member-id", s.ID().String()), zap.String("local-member-id", cm.hasher.MemberId().String()),
zap.Int64("local-member-revision", rev), zap.Int64("local-member-revision", rev),
zap.Int64("local-member-compact-revision", crev), zap.Int64("local-member-compact-revision", crev),
zap.Uint32("local-member-hash", h), zap.Uint32("local-member-hash", h),
@ -110,61 +155,31 @@ func (s *EtcdServer) CheckInitialHashKV() error {
} }
} }
if mismatch > 0 { if mismatch > 0 {
return fmt.Errorf("%s found data inconsistency with peers", s.ID()) return fmt.Errorf("%s found data inconsistency with peers", cm.hasher.MemberId())
} }
lg.Info( cm.lg.Info(
"initial corruption checking passed; no corruption", "initial corruption checking passed; no corruption",
zap.String("local-member-id", s.ID().String()), zap.String("local-member-id", cm.hasher.MemberId().String()),
) )
return nil return nil
} }
func (s *EtcdServer) monitorKVHash() { func (cm *corruptionMonitor) periodicCheck() error {
t := s.Cfg.CorruptCheckTime h, rev, crev, err := cm.hasher.HashByRev(0)
if t == 0 {
return
}
lg := s.Logger()
lg.Info(
"enabled corruption checking",
zap.String("local-member-id", s.ID().String()),
zap.Duration("interval", t),
)
for {
select {
case <-s.stopping:
return
case <-time.After(t):
}
if !s.isLeader() {
continue
}
if err := s.checkHashKV(); err != nil {
lg.Warn("failed to check hash KV", zap.Error(err))
}
}
}
func (s *EtcdServer) checkHashKV() error {
lg := s.Logger()
h, rev, crev, err := s.kv.HashByRev(0)
if err != nil { if err != nil {
return err return err
} }
peers := s.getPeerHashKVs(rev) peers := cm.hasher.PeerHashByRev(rev)
ctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout()) ctx, cancel := context.WithTimeout(context.Background(), cm.hasher.ReqTimeout())
err = s.linearizableReadNotify(ctx) err = cm.hasher.LinearizableReadNotify(ctx)
cancel() cancel()
if err != nil { if err != nil {
return err return err
} }
h2, rev2, crev2, err := s.kv.HashByRev(0) h2, rev2, crev2, err := cm.hasher.HashByRev(0)
if err != nil { if err != nil {
return err return err
} }
@ -175,11 +190,11 @@ func (s *EtcdServer) checkHashKV() error {
return return
} }
alarmed = true alarmed = true
s.triggerCorruptAlarm(id) cm.hasher.TriggerCorruptAlarm(id)
} }
if h2 != h && rev2 == rev && crev == crev2 { if h2 != h && rev2 == rev && crev == crev2 {
lg.Warn( cm.lg.Warn(
"found hash mismatch", "found hash mismatch",
zap.Int64("revision-1", rev), zap.Int64("revision-1", rev),
zap.Int64("compact-revision-1", crev), zap.Int64("compact-revision-1", crev),
@ -188,7 +203,7 @@ func (s *EtcdServer) checkHashKV() error {
zap.Int64("compact-revision-2", crev2), zap.Int64("compact-revision-2", crev2),
zap.Uint32("hash-2", h2), zap.Uint32("hash-2", h2),
) )
mismatch(uint64(s.ID())) mismatch(uint64(cm.hasher.MemberId()))
} }
checkedCount := 0 checkedCount := 0
@ -201,7 +216,7 @@ func (s *EtcdServer) checkHashKV() error {
// leader expects follower's latest revision less than or equal to leader's // leader expects follower's latest revision less than or equal to leader's
if p.resp.Header.Revision > rev2 { if p.resp.Header.Revision > rev2 {
lg.Warn( cm.lg.Warn(
"revision from follower must be less than or equal to leader's", "revision from follower must be less than or equal to leader's",
zap.Int64("leader-revision", rev2), zap.Int64("leader-revision", rev2),
zap.Int64("follower-revision", p.resp.Header.Revision), zap.Int64("follower-revision", p.resp.Header.Revision),
@ -212,7 +227,7 @@ func (s *EtcdServer) checkHashKV() error {
// leader expects follower's latest compact revision less than or equal to leader's // leader expects follower's latest compact revision less than or equal to leader's
if p.resp.CompactRevision > crev2 { if p.resp.CompactRevision > crev2 {
lg.Warn( cm.lg.Warn(
"compact revision from follower must be less than or equal to leader's", "compact revision from follower must be less than or equal to leader's",
zap.Int64("leader-compact-revision", crev2), zap.Int64("leader-compact-revision", crev2),
zap.Int64("follower-compact-revision", p.resp.CompactRevision), zap.Int64("follower-compact-revision", p.resp.CompactRevision),
@ -223,7 +238,7 @@ func (s *EtcdServer) checkHashKV() error {
// follower's compact revision is leader's old one, then hashes must match // follower's compact revision is leader's old one, then hashes must match
if p.resp.CompactRevision == crev && p.resp.Hash != h { if p.resp.CompactRevision == crev && p.resp.Hash != h {
lg.Warn( cm.lg.Warn(
"same compact revision then hashes must match", "same compact revision then hashes must match",
zap.Int64("leader-compact-revision", crev2), zap.Int64("leader-compact-revision", crev2),
zap.Uint32("leader-hash", h), zap.Uint32("leader-hash", h),
@ -234,7 +249,7 @@ func (s *EtcdServer) checkHashKV() error {
mismatch(id) mismatch(id)
} }
} }
lg.Info("finished peer corruption check", zap.Int("number-of-peers-checked", checkedCount)) cm.lg.Info("finished peer corruption check", zap.Int("number-of-peers-checked", checkedCount))
return nil return nil
} }

View File

@ -2508,6 +2508,34 @@ func (s *EtcdServer) monitorVersions() {
} }
} }
func (s *EtcdServer) monitorKVHash() {
t := s.Cfg.CorruptCheckTime
if t == 0 {
return
}
lg := s.Logger()
lg.Info(
"enabled corruption checking",
zap.String("local-member-id", s.ID().String()),
zap.Duration("interval", t),
)
monitor := NewCorruptionMonitor(lg, s)
for {
select {
case <-s.stopping:
return
case <-time.After(t):
}
if !s.isLeader() {
continue
}
if err := monitor.periodicCheck(); err != nil {
lg.Warn("failed to check hash KV", zap.Error(err))
}
}
}
func (s *EtcdServer) updateClusterVersionV2(ver string) { func (s *EtcdServer) updateClusterVersionV2(ver string) {
lg := s.Logger() lg := s.Logger()

View File

@ -112,6 +112,7 @@ const (
type KV interface { type KV interface {
ReadView ReadView
WriteView WriteView
Hasher
// Read creates a read transaction. // Read creates a read transaction.
Read(mode ReadTxMode, trace *traceutil.Trace) TxnRead Read(mode ReadTxMode, trace *traceutil.Trace) TxnRead
@ -119,12 +120,6 @@ type KV interface {
// Write creates a write transaction. // Write creates a write transaction.
Write(trace *traceutil.Trace) TxnWrite Write(trace *traceutil.Trace) TxnWrite
// Hash computes the hash of the KV's backend.
Hash() (hash uint32, revision int64, err error)
// HashByRev computes the hash of all MVCC revisions up to a given revision.
HashByRev(rev int64) (hash uint32, revision int64, compactRev int64, err error)
// Compact frees all superseded keys with revisions less than rev. // Compact frees all superseded keys with revisions less than rev.
Compact(trace *traceutil.Trace, rev int64) (<-chan struct{}, error) Compact(trace *traceutil.Trace, rev int64) (<-chan struct{}, error)
@ -136,6 +131,14 @@ type KV interface {
Close() error Close() error
} }
type Hasher interface {
// Hash computes the hash of the KV's backend.
Hash() (hash uint32, revision int64, err error)
// HashByRev computes the hash of all MVCC revisions up to a given revision.
HashByRev(rev int64) (hash uint32, revision int64, compactRev int64, err error)
}
// WatchableKV is a KV that can be watched. // WatchableKV is a KV that can be watched.
type WatchableKV interface { type WatchableKV interface {
KV KV