mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
raft: provide protection against unbounded Raft log growth
The suggested pattern for Raft proposals is that they be retried periodically until they succeed. This turns out to be an issue when a leader cannot commit entries because the leader will continue to append re-proposed entries to its log without committing anything. This can result in the uncommitted tail of a leader's log growing without bound until it is able to commit entries. This change add a safeguard to protect against this case where a leader's log can grow without bound during loss of quorum scenarios. It does so by introducing a new, optional ``MaxUncommittedEntriesSize configuration. This config limits the max aggregate size of uncommitted entries that may be appended to a leader's log. Once this limit is exceeded, proposals will begin to return ErrProposalDropped errors. See cockroachdb/cockroach#27772
This commit is contained in:
@@ -997,3 +997,57 @@ func TestNodeCommitPaginationAfterRestart(t *testing.T) {
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
// TestNodeBoundedLogGrowthWithPartition tests a scenario where a leader is
|
||||
// partitioned from a quorum of nodes. It verifies that the leader's log is
|
||||
// protected from unbounded growth even as new entries continue to be proposed.
|
||||
// This protection is provided by the MaxUncommittedEntriesSize configuration.
|
||||
func TestNodeBoundedLogGrowthWithPartition(t *testing.T) {
|
||||
const maxEntries = 16
|
||||
data := []byte("testdata")
|
||||
testEntry := raftpb.Entry{Data: data}
|
||||
maxEntrySize := uint64(maxEntries * testEntry.Size())
|
||||
|
||||
s := NewMemoryStorage()
|
||||
cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
|
||||
cfg.MaxUncommittedEntriesSize = maxEntrySize
|
||||
r := newRaft(cfg)
|
||||
n := newNode()
|
||||
go n.run(r)
|
||||
defer n.Stop()
|
||||
n.Campaign(context.TODO())
|
||||
|
||||
rd := readyWithTimeout(&n)
|
||||
if len(rd.CommittedEntries) != 1 {
|
||||
t.Fatalf("expected 1 (empty) entry, got %d", len(rd.CommittedEntries))
|
||||
}
|
||||
s.Append(rd.Entries)
|
||||
n.Advance()
|
||||
|
||||
// Simulate a network partition while we make our proposals by never
|
||||
// committing anything. These proposals should not cause the leader's
|
||||
// log to grow indefinitely.
|
||||
for i := 0; i < 1024; i++ {
|
||||
n.Propose(context.TODO(), data)
|
||||
}
|
||||
|
||||
// Check the size of leader's uncommitted log tail. It should not exceed the
|
||||
// MaxUncommittedEntriesSize limit.
|
||||
checkUncommitted := func(exp uint64) {
|
||||
t.Helper()
|
||||
if a := r.uncommittedSize; exp != a {
|
||||
t.Fatalf("expected %d uncommitted entry bytes, found %d", exp, a)
|
||||
}
|
||||
}
|
||||
checkUncommitted(maxEntrySize)
|
||||
|
||||
// Recover from the partition. The uncommitted tail of the Raft log should
|
||||
// disappear as entries are committed.
|
||||
rd = readyWithTimeout(&n)
|
||||
if len(rd.CommittedEntries) != maxEntries {
|
||||
t.Fatalf("expected %d entries, got %d", maxEntries, len(rd.CommittedEntries))
|
||||
}
|
||||
s.Append(rd.Entries)
|
||||
n.Advance()
|
||||
checkUncommitted(0)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user