server: Implement compaction hash checking
Signed-off-by: Marek Siarkowicz <siarkowicz@google.com>
This commit is contained in:
@ -21,7 +21,9 @@ import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
pb "go.etcd.io/etcd/api/v3/etcdserverpb"
|
||||
@ -36,12 +38,16 @@ import (
|
||||
type CorruptionChecker interface {
|
||||
InitialCheck() error
|
||||
PeriodicCheck() error
|
||||
CompactHashCheck()
|
||||
}
|
||||
|
||||
type corruptionChecker struct {
|
||||
lg *zap.Logger
|
||||
|
||||
hasher Hasher
|
||||
|
||||
mux sync.RWMutex
|
||||
latestRevisionChecked int64
|
||||
}
|
||||
|
||||
type Hasher interface {
|
||||
@ -53,10 +59,10 @@ type Hasher interface {
|
||||
TriggerCorruptAlarm(uint64)
|
||||
}
|
||||
|
||||
func NewCorruptionChecker(lg *zap.Logger, s *EtcdServer) *corruptionChecker {
|
||||
func newCorruptionChecker(lg *zap.Logger, s *EtcdServer, storage mvcc.HashStorage) *corruptionChecker {
|
||||
return &corruptionChecker{
|
||||
lg: lg,
|
||||
hasher: hasherAdapter{s, s.KV().HashStorage()},
|
||||
hasher: hasherAdapter{s, storage},
|
||||
}
|
||||
}
|
||||
|
||||
@ -251,6 +257,85 @@ func (cm *corruptionChecker) PeriodicCheck() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (cm *corruptionChecker) CompactHashCheck() {
|
||||
cm.lg.Info("starting compact hash check",
|
||||
zap.String("local-member-id", cm.hasher.MemberId().String()),
|
||||
zap.Duration("timeout", cm.hasher.ReqTimeout()),
|
||||
)
|
||||
hashes := cm.uncheckedRevisions()
|
||||
// Assume that revisions are ordered from largest to smallest
|
||||
for i, hash := range hashes {
|
||||
peers := cm.hasher.PeerHashByRev(hash.Revision)
|
||||
if len(peers) == 0 {
|
||||
continue
|
||||
}
|
||||
peersChecked := 0
|
||||
for _, p := range peers {
|
||||
if p.resp == nil || p.resp.CompactRevision != hash.CompactRevision {
|
||||
continue
|
||||
}
|
||||
|
||||
// follower's compact revision is leader's old one, then hashes must match
|
||||
if p.resp.Hash != hash.Hash {
|
||||
cm.hasher.TriggerCorruptAlarm(uint64(p.id))
|
||||
cm.lg.Error("failed compaction hash check",
|
||||
zap.Int64("revision", hash.Revision),
|
||||
zap.Int64("leader-compact-revision", hash.CompactRevision),
|
||||
zap.Uint32("leader-hash", hash.Hash),
|
||||
zap.Int64("follower-compact-revision", p.resp.CompactRevision),
|
||||
zap.Uint32("follower-hash", p.resp.Hash),
|
||||
zap.String("follower-peer-id", p.id.String()),
|
||||
)
|
||||
return
|
||||
}
|
||||
peersChecked++
|
||||
cm.lg.Info("successfully checked hash on follower",
|
||||
zap.Int64("revision", hash.Revision),
|
||||
zap.String("peer-id", p.id.String()),
|
||||
)
|
||||
}
|
||||
if len(peers) == peersChecked {
|
||||
cm.lg.Info("successfully checked hash on whole cluster",
|
||||
zap.Int("number-of-peers-checked", peersChecked),
|
||||
zap.Int64("revision", hash.Revision),
|
||||
)
|
||||
cm.mux.Lock()
|
||||
if hash.Revision > cm.latestRevisionChecked {
|
||||
cm.latestRevisionChecked = hash.Revision
|
||||
}
|
||||
cm.mux.Unlock()
|
||||
cm.lg.Info("finished compaction hash check", zap.Int("number-of-hashes-checked", i+1))
|
||||
return
|
||||
} else {
|
||||
cm.lg.Warn("skipped checking hash; was not able to check all peers",
|
||||
zap.Int("number-of-peers-checked", peersChecked),
|
||||
zap.Int("number-of-peers", len(peers)),
|
||||
zap.Int64("revision", hash.Revision),
|
||||
)
|
||||
}
|
||||
}
|
||||
cm.lg.Info("finished compaction hash check", zap.Int("number-of-hashes-checked", len(hashes)))
|
||||
return
|
||||
}
|
||||
|
||||
func (cm *corruptionChecker) uncheckedRevisions() []mvcc.KeyValueHash {
|
||||
cm.mux.RLock()
|
||||
lastRevisionChecked := cm.latestRevisionChecked
|
||||
cm.mux.RUnlock()
|
||||
|
||||
hashes := cm.hasher.Hashes()
|
||||
// Sort in descending order
|
||||
sort.Slice(hashes, func(i, j int) bool {
|
||||
return hashes[i].Revision > hashes[j].Revision
|
||||
})
|
||||
for i, hash := range hashes {
|
||||
if hash.Revision <= lastRevisionChecked {
|
||||
return hashes[:i]
|
||||
}
|
||||
}
|
||||
return hashes
|
||||
}
|
||||
|
||||
func (s *EtcdServer) triggerCorruptAlarm(id uint64) {
|
||||
a := &pb.AlarmRequest{
|
||||
MemberID: id,
|
||||
|
@ -221,11 +221,101 @@ func TestPeriodicCheck(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompactHashCheck(t *testing.T) {
|
||||
tcs := []struct {
|
||||
name string
|
||||
hasher fakeHasher
|
||||
lastRevisionChecked int64
|
||||
|
||||
expectError bool
|
||||
expectCorrupt bool
|
||||
expectActions []string
|
||||
expectLastRevisionChecked int64
|
||||
}{
|
||||
{
|
||||
name: "No hashes",
|
||||
expectActions: []string{"MemberId()", "ReqTimeout()", "Hashes()"},
|
||||
},
|
||||
{
|
||||
name: "No peers, check new checked from largest to smallest",
|
||||
hasher: fakeHasher{
|
||||
hashes: []mvcc.KeyValueHash{{Revision: 1}, {Revision: 2}, {Revision: 3}, {Revision: 4}},
|
||||
},
|
||||
lastRevisionChecked: 2,
|
||||
expectActions: []string{"MemberId()", "ReqTimeout()", "Hashes()", "PeerHashByRev(4)", "PeerHashByRev(3)"},
|
||||
expectLastRevisionChecked: 2,
|
||||
},
|
||||
{
|
||||
name: "Peer error",
|
||||
hasher: fakeHasher{
|
||||
hashes: []mvcc.KeyValueHash{{Revision: 1}, {Revision: 2}},
|
||||
peerHashes: []*peerHashKVResp{{err: fmt.Errorf("failed getting hash")}},
|
||||
},
|
||||
expectActions: []string{"MemberId()", "ReqTimeout()", "Hashes()", "PeerHashByRev(2)", "PeerHashByRev(1)"},
|
||||
},
|
||||
{
|
||||
name: "Peer returned different compaction revision is skipped",
|
||||
hasher: fakeHasher{
|
||||
hashes: []mvcc.KeyValueHash{{Revision: 1, CompactRevision: 1}, {Revision: 2, CompactRevision: 2}},
|
||||
peerHashes: []*peerHashKVResp{{resp: &pb.HashKVResponse{CompactRevision: 3}}},
|
||||
},
|
||||
expectActions: []string{"MemberId()", "ReqTimeout()", "Hashes()", "PeerHashByRev(2)", "PeerHashByRev(1)"},
|
||||
},
|
||||
{
|
||||
name: "Peer returned same compaction revision but different hash triggers alarm",
|
||||
hasher: fakeHasher{
|
||||
hashes: []mvcc.KeyValueHash{{Revision: 1, CompactRevision: 1, Hash: 1}, {Revision: 2, CompactRevision: 1, Hash: 2}},
|
||||
peerHashes: []*peerHashKVResp{{peerInfo: peerInfo{id: 42}, resp: &pb.HashKVResponse{CompactRevision: 1, Hash: 3}}},
|
||||
},
|
||||
expectActions: []string{"MemberId()", "ReqTimeout()", "Hashes()", "PeerHashByRev(2)", "TriggerCorruptAlarm(42)"},
|
||||
expectCorrupt: true,
|
||||
},
|
||||
{
|
||||
name: "Peer returned same hash bumps last revision checked",
|
||||
hasher: fakeHasher{
|
||||
hashes: []mvcc.KeyValueHash{{Revision: 1, CompactRevision: 1, Hash: 1}, {Revision: 2, CompactRevision: 1, Hash: 1}},
|
||||
peerHashes: []*peerHashKVResp{{resp: &pb.HashKVResponse{Header: &pb.ResponseHeader{MemberId: 42}, CompactRevision: 1, Hash: 1}}},
|
||||
},
|
||||
expectActions: []string{"MemberId()", "ReqTimeout()", "Hashes()", "PeerHashByRev(2)"},
|
||||
expectLastRevisionChecked: 2,
|
||||
},
|
||||
{
|
||||
name: "Only one peer succeeded check",
|
||||
hasher: fakeHasher{
|
||||
hashes: []mvcc.KeyValueHash{{Revision: 1, CompactRevision: 1, Hash: 1}},
|
||||
peerHashes: []*peerHashKVResp{
|
||||
{resp: &pb.HashKVResponse{Header: &pb.ResponseHeader{MemberId: 42}, CompactRevision: 1, Hash: 1}},
|
||||
{err: fmt.Errorf("failed getting hash")},
|
||||
},
|
||||
},
|
||||
expectActions: []string{"MemberId()", "ReqTimeout()", "Hashes()", "PeerHashByRev(1)"},
|
||||
},
|
||||
}
|
||||
for _, tc := range tcs {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
monitor := corruptionChecker{
|
||||
latestRevisionChecked: tc.lastRevisionChecked,
|
||||
lg: zaptest.NewLogger(t),
|
||||
hasher: &tc.hasher,
|
||||
}
|
||||
monitor.CompactHashCheck()
|
||||
if tc.hasher.alarmTriggered != tc.expectCorrupt {
|
||||
t.Errorf("Unexpected corrupt triggered, got: %v, expected?: %v", tc.hasher.alarmTriggered, tc.expectCorrupt)
|
||||
}
|
||||
if tc.expectLastRevisionChecked != monitor.latestRevisionChecked {
|
||||
t.Errorf("Unexpected last revision checked, got: %v, expected?: %v", monitor.latestRevisionChecked, tc.expectLastRevisionChecked)
|
||||
}
|
||||
assert.Equal(t, tc.expectActions, tc.hasher.actions)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type fakeHasher struct {
|
||||
peerHashes []*peerHashKVResp
|
||||
hashByRevIndex int
|
||||
hashByRevResponses []hashByRev
|
||||
linearizableReadNotify error
|
||||
hashes []mvcc.KeyValueHash
|
||||
|
||||
alarmTriggered bool
|
||||
actions []string
|
||||
@ -251,8 +341,14 @@ func (f *fakeHasher) HashByRev(rev int64) (hash mvcc.KeyValueHash, revision int6
|
||||
return hashByRev.hash, hashByRev.revision, hashByRev.err
|
||||
}
|
||||
|
||||
func (f *fakeHasher) Store(valueHash mvcc.KeyValueHash) {
|
||||
panic("not implemented")
|
||||
func (f *fakeHasher) Store(hash mvcc.KeyValueHash) {
|
||||
f.actions = append(f.actions, fmt.Sprintf("Store(%v)", hash))
|
||||
f.hashes = append(f.hashes, hash)
|
||||
}
|
||||
|
||||
func (f *fakeHasher) Hashes() []mvcc.KeyValueHash {
|
||||
f.actions = append(f.actions, "Hashes()")
|
||||
return f.hashes
|
||||
}
|
||||
|
||||
func (f *fakeHasher) ReqTimeout() time.Duration {
|
||||
|
@ -108,7 +108,8 @@ var (
|
||||
// monitorVersionInterval should be smaller than the timeout
|
||||
// on the connection. Or we will not be able to reuse the connection
|
||||
// (since it will timeout).
|
||||
monitorVersionInterval = rafthttp.ConnWriteTimeout - time.Second
|
||||
monitorVersionInterval = rafthttp.ConnWriteTimeout - time.Second
|
||||
CompactHashCheckInterval = 15 * time.Second
|
||||
|
||||
recommendedMaxRequestBytesString = humanize.Bytes(uint64(recommendedMaxRequestBytes))
|
||||
storeMemberAttributeRegexp = regexp.MustCompile(path.Join(membership.StoreMembersPrefix, "[[:xdigit:]]{1,16}", "attributes"))
|
||||
@ -630,7 +631,7 @@ func NewServer(cfg config.ServerConfig) (srv *EtcdServer, err error) {
|
||||
)
|
||||
}
|
||||
}
|
||||
srv.corruptionChecker = NewCorruptionChecker(cfg.Logger, srv)
|
||||
srv.corruptionChecker = newCorruptionChecker(cfg.Logger, srv, srv.kv.HashStorage())
|
||||
|
||||
srv.authStore = auth.NewAuthStore(srv.Logger(), srv.be, tp, int(cfg.BcryptCost))
|
||||
|
||||
@ -805,6 +806,7 @@ func (s *EtcdServer) Start() {
|
||||
s.GoAttach(s.monitorVersions)
|
||||
s.GoAttach(s.linearizableReadLoop)
|
||||
s.GoAttach(s.monitorKVHash)
|
||||
s.GoAttach(s.monitorCompactHash)
|
||||
s.GoAttach(s.monitorDowngrade)
|
||||
}
|
||||
|
||||
@ -2537,6 +2539,20 @@ func (s *EtcdServer) monitorKVHash() {
|
||||
}
|
||||
}
|
||||
|
||||
func (s *EtcdServer) monitorCompactHash() {
|
||||
for {
|
||||
select {
|
||||
case <-time.After(CompactHashCheckInterval):
|
||||
case <-s.stopping:
|
||||
return
|
||||
}
|
||||
if !s.isLeader() {
|
||||
continue
|
||||
}
|
||||
s.corruptionChecker.CompactHashCheck()
|
||||
}
|
||||
}
|
||||
|
||||
func (s *EtcdServer) updateClusterVersionV2(ver string) {
|
||||
lg := s.Logger()
|
||||
|
||||
|
Reference in New Issue
Block a user