Skip to content

Commit af9209f

Browse files
committed
etcdserver: keep raft log entries up to snapshot index
Signed-off-by: Clement <[email protected]>
1 parent bd93a00 commit af9209f

File tree

4 files changed

+47
-15
lines changed

4 files changed

+47
-15
lines changed

server/config/config.go

+4
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,10 @@ type ServerConfig struct {
5858
// follower to catch up.
5959
SnapshotCatchUpEntries uint64
6060

61+
// CompactRaftLogEveryNApplies compact raft log once every N applies.
62+
// Minimum value is 1, which means compacting raft log every apply.
63+
CompactRaftLogEveryNApplies uint64
64+
6165
MaxSnapFiles uint
6266
MaxWALFiles uint
6367

server/etcdserver/server.go

+26-7
Original file line numberDiff line numberDiff line change
@@ -84,6 +84,10 @@ const (
8484
// follower to catch up.
8585
DefaultSnapshotCatchUpEntries uint64 = 5000
8686

87+
// DefaultCompactRaftLogEveryNApplies compact raft log once every N applies.
88+
// Minimum value is 1, which means compacting raft log every apply.
89+
DefaultCompactRaftLogEveryNApplies uint64 = 10
90+
8791
StoreClusterPrefix = "/0"
8892
StoreKeysPrefix = "/1"
8993

@@ -569,6 +573,14 @@ func (s *EtcdServer) start() {
569573
)
570574
s.Cfg.SnapshotCatchUpEntries = DefaultSnapshotCatchUpEntries
571575
}
576+
if s.Cfg.CompactRaftLogEveryNApplies == 0 {
577+
lg.Info(
578+
"updating compact raft log every N applies to default",
579+
zap.Uint64("given-compact-raft-log-every-n-applies", s.Cfg.CompactRaftLogEveryNApplies),
580+
zap.Uint64("updated-compact-raft-log-every-n-applies", DefaultCompactRaftLogEveryNApplies),
581+
)
582+
s.Cfg.CompactRaftLogEveryNApplies = DefaultCompactRaftLogEveryNApplies
583+
}
572584

573585
s.w = wait.New()
574586
s.applyWait = wait.NewTimeList()
@@ -980,6 +992,7 @@ func (s *EtcdServer) applyAll(ep *etcdProgress, apply *toApply) {
980992
<-apply.notifyc
981993

982994
s.triggerSnapshot(ep)
995+
s.maybeCompactRaftLog(ep)
983996
select {
984997
// snapshot requested via send()
985998
case m := <-s.r.msgSnapC:
@@ -2170,6 +2183,18 @@ func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) {
21702183
"saved snapshot",
21712184
zap.Uint64("snapshot-index", snap.Metadata.Index),
21722185
)
2186+
}
2187+
2188+
func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) {
2189+
lg := s.Logger()
2190+
2191+
// Keep some in memory log entries for slow followers, while keeping the entries up to snapshot index.
2192+
// Only compact raft log once every N applies
2193+
if ep.appliedi <= ep.snapi+s.Cfg.SnapshotCatchUpEntries || ep.appliedi%s.Cfg.CompactRaftLogEveryNApplies != 0 {
2194+
return
2195+
}
2196+
2197+
compacti := ep.appliedi - s.Cfg.SnapshotCatchUpEntries
21732198

21742199
// When sending a snapshot, etcd will pause compaction.
21752200
// After receives a snapshot, the slow follower needs to get all the entries right after
@@ -2181,13 +2206,7 @@ func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) {
21812206
return
21822207
}
21832208

2184-
// keep some in memory log entries for slow followers.
2185-
compacti := uint64(1)
2186-
if snapi > s.Cfg.SnapshotCatchUpEntries {
2187-
compacti = snapi - s.Cfg.SnapshotCatchUpEntries
2188-
}
2189-
2190-
err = s.r.raftStorage.Compact(compacti)
2209+
err := s.r.raftStorage.Compact(compacti)
21912210
if err != nil {
21922211
// the compaction was done asynchronously with the progress of raft.
21932212
// raft log might already been compact.

tests/framework/integration/cluster.go

+9-2
Original file line numberDiff line numberDiff line change
@@ -148,8 +148,9 @@ type ClusterConfig struct {
148148
MaxTxnOps uint
149149
MaxRequestBytes uint
150150

151-
SnapshotCount uint64
152-
SnapshotCatchUpEntries uint64
151+
SnapshotCount uint64
152+
SnapshotCatchUpEntries uint64
153+
CompactRaftLogEveryNApplies uint64
153154

154155
GRPCKeepAliveMinTime time.Duration
155156
GRPCKeepAliveInterval time.Duration
@@ -276,6 +277,7 @@ func (c *Cluster) mustNewMember(t testutil.TB) *Member {
276277
MaxRequestBytes: c.Cfg.MaxRequestBytes,
277278
SnapshotCount: c.Cfg.SnapshotCount,
278279
SnapshotCatchUpEntries: c.Cfg.SnapshotCatchUpEntries,
280+
CompactRaftLogEveryNApplies: c.Cfg.CompactRaftLogEveryNApplies,
279281
GRPCKeepAliveMinTime: c.Cfg.GRPCKeepAliveMinTime,
280282
GRPCKeepAliveInterval: c.Cfg.GRPCKeepAliveInterval,
281283
GRPCKeepAliveTimeout: c.Cfg.GRPCKeepAliveTimeout,
@@ -601,6 +603,7 @@ type MemberConfig struct {
601603
MaxRequestBytes uint
602604
SnapshotCount uint64
603605
SnapshotCatchUpEntries uint64
606+
CompactRaftLogEveryNApplies uint64
604607
GRPCKeepAliveMinTime time.Duration
605608
GRPCKeepAliveInterval time.Duration
606609
GRPCKeepAliveTimeout time.Duration
@@ -686,6 +689,10 @@ func MustNewMember(t testutil.TB, mcfg MemberConfig) *Member {
686689
if mcfg.SnapshotCatchUpEntries != 0 {
687690
m.SnapshotCatchUpEntries = mcfg.SnapshotCatchUpEntries
688691
}
692+
m.CompactRaftLogEveryNApplies = etcdserver.DefaultCompactRaftLogEveryNApplies
693+
if mcfg.CompactRaftLogEveryNApplies != 0 {
694+
m.CompactRaftLogEveryNApplies = mcfg.CompactRaftLogEveryNApplies
695+
}
689696

690697
// for the purpose of integration testing, simple token is enough
691698
m.AuthToken = "simple"

tests/integration/v3_watch_restore_test.go

+8-6
Original file line numberDiff line numberDiff line change
@@ -55,9 +55,10 @@ func TestV3WatchRestoreSnapshotUnsync(t *testing.T) {
5555
integration.BeforeTest(t)
5656

5757
clus := integration.NewCluster(t, &integration.ClusterConfig{
58-
Size: 3,
59-
SnapshotCount: 10,
60-
SnapshotCatchUpEntries: 5,
58+
Size: 3,
59+
SnapshotCount: 10,
60+
SnapshotCatchUpEntries: 5,
61+
CompactRaftLogEveryNApplies: 10,
6162
})
6263
defer clus.Terminate(t)
6364

@@ -102,11 +103,12 @@ func TestV3WatchRestoreSnapshotUnsync(t *testing.T) {
102103
// elected. Leader will apply 3 MemberAttrSet and 1 ClusterVersionSet
103104
// changes. So member 0 has index 8 in raft log before network
104105
// partition. We need to trigger EtcdServer.snapshot() at least twice.
106+
// Raft log is only compacted when appliedi%CompactRaftLogEveryNApplies==0
105107
//
106-
// SnapshotCount: 10, SnapshotCatchUpEntries: 5
108+
// SnapshotCount: 10, SnapshotCatchUpEntries: 5, CompactRaftLogEveryNApplies: 10
107109
//
108-
// T1: L(snapshot-index: 11, compacted-index: 6), F_m0(index:8)
109-
// T2: L(snapshot-index: 22, compacted-index: 17), F_m0(index:8, out of date)
110+
// T1: L(snapshot-index: 11, compacted-index: 5), F_m0(index:8)
111+
// T2: L(snapshot-index: 22, compacted-index: 15), F_m0(index:8, out of date)
110112
//
111113
// Since there is no way to confirm server has compacted the log, we
112114
// use log monitor to watch and expect "compacted Raft logs" content.

0 commit comments

Comments
 (0)