server: Extract corruption detection to dedicated struct
Signed-off-by: Marek Siarkowicz <siarkowicz@google.com>
This commit is contained in:
parent
d32de2c410
commit
35cbdf3961
@ -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)
|
||||||
|
|
||||||
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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()
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
Loading…
Reference in New Issue
Block a user