From dff53056f5eff9a6119e5df38d524f1e19903fc1 Mon Sep 17 00:00:00 2001 From: Clement Date: Tue, 24 Sep 2024 18:06:55 +0800 Subject: [PATCH 1/9] separate maybeCompactRaftLog function to compact raft log independently from snapshots Signed-off-by: Clement --- server/etcdserver/server.go | 24 +++++++++++++++++------- 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index ba3a3f3ffe1..2540cc5b295 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -980,6 +980,7 @@ func (s *EtcdServer) applyAll(ep *etcdProgress, apply *toApply) { <-apply.notifyc s.triggerSnapshot(ep) + s.maybeCompactRaftLog(ep) select { // snapshot requested via send() case m := <-s.r.msgSnapC: @@ -2170,6 +2171,21 @@ func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) { "saved snapshot", zap.Uint64("snapshot-index", snap.Metadata.Index), ) +} + +func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) { + // Retain all log entries up to the latest snapshot index to ensure any member can recover from that snapshot. + // Beyond the snapshot index, preserve the most recent s.Cfg.SnapshotCatchUpEntries entries in memory. + // This allows slow followers to catch up by synchronizing entries instead of requiring a full snapshot transfer. + // Only compact raft log once every N applies + if ep.snapi <= s.Cfg.SnapshotCatchUpEntries { + return + } + + // make sure compacti >= 0, because s.r.raftStorage.Compact(compacti) always returns ErrCompacted if compacti <= 0 + compacti := ep.snapi - s.Cfg.SnapshotCatchUpEntries + + lg := s.Logger() // When sending a snapshot, etcd will pause compaction. // After receives a snapshot, the slow follower needs to get all the entries right after @@ -2181,13 +2197,7 @@ func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) { return } - // keep some in memory log entries for slow followers. - compacti := uint64(1) - if snapi > s.Cfg.SnapshotCatchUpEntries { - compacti = snapi - s.Cfg.SnapshotCatchUpEntries - } - - err = s.r.raftStorage.Compact(compacti) + err := s.r.raftStorage.Compact(compacti) if err != nil { // the compaction was done asynchronously with the progress of raft. // raft log might already been compact. From 9b04e4df459358cb82a83cabbb993b851a299c36 Mon Sep 17 00:00:00 2001 From: Clement Date: Tue, 24 Sep 2024 18:23:20 +0800 Subject: [PATCH 2/9] separate maybeCompactRaftLog function to compact raft log independently from snapshots Signed-off-by: Clement --- server/etcdserver/server.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index 2540cc5b295..378d4579672 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -2177,12 +2177,11 @@ func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) { // Retain all log entries up to the latest snapshot index to ensure any member can recover from that snapshot. // Beyond the snapshot index, preserve the most recent s.Cfg.SnapshotCatchUpEntries entries in memory. // This allows slow followers to catch up by synchronizing entries instead of requiring a full snapshot transfer. - // Only compact raft log once every N applies if ep.snapi <= s.Cfg.SnapshotCatchUpEntries { return } - // make sure compacti >= 0, because s.r.raftStorage.Compact(compacti) always returns ErrCompacted if compacti <= 0 + // make sure compacti > 0, because s.r.raftStorage.Compact(compacti) returns ErrCompacted if compacti = 0 compacti := ep.snapi - s.Cfg.SnapshotCatchUpEntries lg := s.Logger() From 1d08b34a433f6c0f3b2dafbe5f50f3b8f57551cf Mon Sep 17 00:00:00 2001 From: Clement Date: Tue, 24 Sep 2024 18:27:05 +0800 Subject: [PATCH 3/9] separate maybeCompactRaftLog function to compact raft log independently from snapshots Signed-off-by: Clement --- server/etcdserver/server.go | 1 - 1 file changed, 1 deletion(-) diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index 378d4579672..900a1a8e03f 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -2181,7 +2181,6 @@ func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) { return } - // make sure compacti > 0, because s.r.raftStorage.Compact(compacti) returns ErrCompacted if compacti = 0 compacti := ep.snapi - s.Cfg.SnapshotCatchUpEntries lg := s.Logger() From 06a2cb73696413eaf9324a9699b37598bb83bd46 Mon Sep 17 00:00:00 2001 From: Clement Date: Tue, 24 Sep 2024 22:52:07 +0800 Subject: [PATCH 4/9] add compacti field to etcdProgress; add a test case Signed-off-by: Clement --- server/etcdserver/server.go | 12 +++- tests/integration/raft_log_test.go | 95 ++++++++++++++++++++++++++++++ 2 files changed, 106 insertions(+), 1 deletion(-) create mode 100644 tests/integration/raft_log_test.go diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index 900a1a8e03f..19bfc3a8bd9 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -745,6 +745,7 @@ type etcdProgress struct { snapi uint64 appliedt uint64 appliedi uint64 + compacti uint64 } // raftReadyHandler contains a set of EtcdServer operations to be called by raftNode, @@ -764,6 +765,10 @@ func (s *EtcdServer) run() { if err != nil { lg.Panic("failed to get snapshot from Raft storage", zap.Error(err)) } + fi, err := s.r.raftStorage.FirstIndex() + if err != nil { + lg.Panic("failed to get first index from Raft storage", zap.Error(err)) + } // asynchronously accept toApply packets, dispatch progress in-order sched := schedule.NewFIFOScheduler(lg) @@ -813,6 +818,7 @@ func (s *EtcdServer) run() { snapi: sn.Metadata.Index, appliedt: sn.Metadata.Term, appliedi: sn.Metadata.Index, + compacti: fi - 1, } defer func() { @@ -2177,11 +2183,14 @@ func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) { // Retain all log entries up to the latest snapshot index to ensure any member can recover from that snapshot. // Beyond the snapshot index, preserve the most recent s.Cfg.SnapshotCatchUpEntries entries in memory. // This allows slow followers to catch up by synchronizing entries instead of requiring a full snapshot transfer. - if ep.snapi <= s.Cfg.SnapshotCatchUpEntries { + if ep.snapi <= s.Cfg.SnapshotCatchUpEntries { return } compacti := ep.snapi - s.Cfg.SnapshotCatchUpEntries + if compacti <= ep.compacti { + return + } lg := s.Logger() @@ -2196,6 +2205,7 @@ func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) { } err := s.r.raftStorage.Compact(compacti) + ep.compacti = compacti if err != nil { // the compaction was done asynchronously with the progress of raft. // raft log might already been compact. diff --git a/tests/integration/raft_log_test.go b/tests/integration/raft_log_test.go new file mode 100644 index 00000000000..680f3049749 --- /dev/null +++ b/tests/integration/raft_log_test.go @@ -0,0 +1,95 @@ +// Copyright 2024 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package integration + +import ( + "context" + "errors" + "testing" + "time" + + pb "go.etcd.io/etcd/api/v3/etcdserverpb" + "go.etcd.io/etcd/tests/v3/framework/integration" +) + +// TestRaftLogCompaction tests whether raft log snapshot and compaction work correctly. +func TestRaftLogCompaction(t *testing.T) { + integration.BeforeTest(t) + + clus := integration.NewCluster(t, &integration.ClusterConfig{ + Size: 1, + SnapshotCount: 10, + SnapshotCatchUpEntries: 5, + }) + defer clus.Terminate(t) + + mem := clus.Members[0] + kvc := integration.ToGRPC(mem.Client).KV + + // When starting a new cluster with 1 member, the member will have an index of 4. + // TODO: Can someone explain this? + // Currently, if `ep.appliedi-ep.snapi > s.Cfg.SnapshotCount`, + // a raft log snapshot is created, and raft log entries are compacted. + // In this case, it triggers when the index is a multiple of 11. + appliedi := 4 + for ; appliedi <= 10; appliedi++ { + _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}) + if err != nil { + t.Errorf("#%d: couldn't put key (%v)", appliedi, err) + } + } + // The first snapshot and compaction shouldn't happen because the index is less than 11 + expectMemberLogTimeout(t, mem, 5*time.Second, "saved snapshot", 1) + expectMemberLogTimeout(t, mem, time.Second, "compacted Raft logs", 1) + + for ; appliedi <= 11; appliedi++ { + _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}) + if err != nil { + t.Errorf("#%d: couldn't put key (%v)", appliedi, err) + } + } + // The first snapshot and compaction should happen because the index is 11 + expectMemberLog(t, mem, 5*time.Second, "saved snapshot", 1) + expectMemberLog(t, mem, time.Second, "compacted Raft logs", 1) + expectMemberLog(t, mem, time.Second, "\"compact-index\": 6", 1) + + for ; appliedi <= 1100; appliedi++ { + _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}) + if err != nil { + t.Errorf("#%d: couldn't put key (%v)", appliedi, err) + } + } + // With the index at 1100, snapshot and compaction should happen 100 times. + expectMemberLog(t, mem, 5*time.Second, "saved snapshot", 100) + expectMemberLog(t, mem, time.Second, "compacted Raft logs", 100) + expectMemberLog(t, mem, time.Second, "\"compact-index\": 1095", 1) + + // No more snapshot and compaction should happen. + expectMemberLogTimeout(t, mem, 5*time.Second, "saved snapshot", 101) + expectMemberLogTimeout(t, mem, time.Second, "compacted Raft logs", 101) +} + +// expectMemberLogTimeout ensures that the log has fewer than `count` occurrences of `s` before timing out +func expectMemberLogTimeout(t *testing.T, m *integration.Member, timeout time.Duration, s string, count int) { + ctx, cancel := context.WithTimeout(context.TODO(), timeout) + defer cancel() + + _, err := m.LogObserver.Expect(ctx, s, count) + if !errors.Is(err, context.DeadlineExceeded) { + if err != nil { + t.Fatalf("failed to expect (log:%s, count:%v): %v", s, count, err) + } + } +} From 2028c246770e03f728982963158b07d2b7de58de Mon Sep 17 00:00:00 2001 From: Clement Date: Wed, 25 Sep 2024 16:44:04 +0800 Subject: [PATCH 5/9] apply suggestions Signed-off-by: Clement --- server/etcdserver/memory_storage_test.go | 67 ++++++++++++++++++++++++ server/etcdserver/server.go | 12 +++-- 2 files changed, 76 insertions(+), 3 deletions(-) create mode 100644 server/etcdserver/memory_storage_test.go diff --git a/server/etcdserver/memory_storage_test.go b/server/etcdserver/memory_storage_test.go new file mode 100644 index 00000000000..bbbbc978745 --- /dev/null +++ b/server/etcdserver/memory_storage_test.go @@ -0,0 +1,67 @@ +// Copyright 2024 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package etcdserver + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "go.etcd.io/raft/v3" + "go.etcd.io/raft/v3/raftpb" +) + +// TestMemoryStorageCompaction tests that after calling raftStorage.Compact(compacti) +// without errors, the dummy entry becomes {Index: compacti} and +// raftStorage.FirstIndex() returns (compacti+1, nil). +func TestMemoryStorageCompaction(t *testing.T) { + // entries: [ {Index: 0} ] + raftStorage := raft.NewMemoryStorage() + + firstIndex, err := raftStorage.FirstIndex() + assert.NoError(t, err) + assert.Equal(t, uint64(1), firstIndex) + + // after appending, entries should be: + // [ {Index: 0}, {Index: 1}, {Index: 2}, {Index: 3}, {Index: 4}, {Index: 5} ] + appliedIndex := uint64(1) + for ; appliedIndex <= 5; appliedIndex++ { + e := raftpb.Entry{ + Type: raftpb.EntryNormal, + Term: 1, + Index: appliedIndex, + } + err := raftStorage.Append([]raftpb.Entry{e}) + assert.NoError(t, err) + } + + firstIndex, err = raftStorage.FirstIndex() + assert.NoError(t, err) + assert.Equal(t, uint64(1), firstIndex) + + lastIndex, err := raftStorage.LastIndex() + assert.NoError(t, err) + assert.Equal(t, uint64(5), lastIndex) + + // after compacting, entries should be: + // [ {Index: 3}, {Index: 4}, {Index: 5} ] + err = raftStorage.Compact(3) + assert.NoError(t, err) + + + firstIndex, err = raftStorage.FirstIndex() + assert.NoError(t, err) + assert.Equal(t, uint64(3+1), firstIndex) +} diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index 19bfc3a8bd9..b05b6cff186 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -765,7 +765,7 @@ func (s *EtcdServer) run() { if err != nil { lg.Panic("failed to get snapshot from Raft storage", zap.Error(err)) } - fi, err := s.r.raftStorage.FirstIndex() + firstIndex, err := s.r.raftStorage.FirstIndex() if err != nil { lg.Panic("failed to get first index from Raft storage", zap.Error(err)) } @@ -818,7 +818,13 @@ func (s *EtcdServer) run() { snapi: sn.Metadata.Index, appliedt: sn.Metadata.Term, appliedi: sn.Metadata.Index, - compacti: fi - 1, + // compacti is the index from the last time raftStorage.Compact was called + // without errors. + // + // After calling raftStorage.Compact(compacti) without errors, the dummy entry of + // raftStorage becomes {Index: compacti}, and raftStorage.FirstIndex() returns + // (compacti+1, nil). This is validated by TestMemoryStorageCompaction. + compacti: firstIndex - 1, } defer func() { @@ -2205,7 +2211,6 @@ func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) { } err := s.r.raftStorage.Compact(compacti) - ep.compacti = compacti if err != nil { // the compaction was done asynchronously with the progress of raft. // raft log might already been compact. @@ -2214,6 +2219,7 @@ func (s *EtcdServer) maybeCompactRaftLog(ep *etcdProgress) { } lg.Panic("failed to compact", zap.Error(err)) } + ep.compacti = compacti lg.Info( "compacted Raft logs", zap.Uint64("compact-index", compacti), From 96d898579a19b20bfb27d15561604fcf8c48d8da Mon Sep 17 00:00:00 2001 From: Clement Date: Wed, 25 Sep 2024 16:45:31 +0800 Subject: [PATCH 6/9] apply suggestions Signed-off-by: Clement --- server/etcdserver/server.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index b05b6cff186..0463a175a8e 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -765,7 +765,7 @@ func (s *EtcdServer) run() { if err != nil { lg.Panic("failed to get snapshot from Raft storage", zap.Error(err)) } - firstIndex, err := s.r.raftStorage.FirstIndex() + firstRaftIndex, err := s.r.raftStorage.FirstIndex() if err != nil { lg.Panic("failed to get first index from Raft storage", zap.Error(err)) } @@ -824,7 +824,7 @@ func (s *EtcdServer) run() { // After calling raftStorage.Compact(compacti) without errors, the dummy entry of // raftStorage becomes {Index: compacti}, and raftStorage.FirstIndex() returns // (compacti+1, nil). This is validated by TestMemoryStorageCompaction. - compacti: firstIndex - 1, + compacti: firstRaftIndex - 1, } defer func() { From 7f0ada26c6642c3756b4909b4e8e25cbc4b34b21 Mon Sep 17 00:00:00 2001 From: Clement Date: Wed, 25 Sep 2024 17:01:05 +0800 Subject: [PATCH 7/9] apply suggestions Signed-off-by: Clement --- server/etcdserver/memory_storage_test.go | 12 ++++++------ server/etcdserver/server.go | 9 +++------ 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/server/etcdserver/memory_storage_test.go b/server/etcdserver/memory_storage_test.go index bbbbc978745..19e61efc688 100644 --- a/server/etcdserver/memory_storage_test.go +++ b/server/etcdserver/memory_storage_test.go @@ -23,10 +23,9 @@ import ( "go.etcd.io/raft/v3/raftpb" ) -// TestMemoryStorageCompaction tests that after calling raftStorage.Compact(compacti) -// without errors, the dummy entry becomes {Index: compacti} and -// raftStorage.FirstIndex() returns (compacti+1, nil). -func TestMemoryStorageCompaction(t *testing.T) { +// TestMemoryStorageCompactInclusive tests that compacting is inclusive, +// meaning the first index after compaction is larger by one than compacted index. +func TestMemoryStorageCompactInclusive(t *testing.T) { // entries: [ {Index: 0} ] raftStorage := raft.NewMemoryStorage() @@ -57,11 +56,12 @@ func TestMemoryStorageCompaction(t *testing.T) { // after compacting, entries should be: // [ {Index: 3}, {Index: 4}, {Index: 5} ] - err = raftStorage.Compact(3) + compacti := uint64(3) + err = raftStorage.Compact(compacti) assert.NoError(t, err) firstIndex, err = raftStorage.FirstIndex() assert.NoError(t, err) - assert.Equal(t, uint64(3+1), firstIndex) + assert.Equal(t, compacti+1, firstIndex) } diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index 0463a175a8e..165a2a4c0cd 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -818,12 +818,9 @@ func (s *EtcdServer) run() { snapi: sn.Metadata.Index, appliedt: sn.Metadata.Term, appliedi: sn.Metadata.Index, - // compacti is the index from the last time raftStorage.Compact was called - // without errors. - // - // After calling raftStorage.Compact(compacti) without errors, the dummy entry of - // raftStorage becomes {Index: compacti}, and raftStorage.FirstIndex() returns - // (compacti+1, nil). This is validated by TestMemoryStorageCompaction. + // Compaction is inclusive, meaning compact index should be lower by one + // than the first index after compaction. + // This is validated by TestMemoryStorageCompaction. compacti: firstRaftIndex - 1, } From bc9a3fa4edb2df758f0225cad223f65c74a2e3e0 Mon Sep 17 00:00:00 2001 From: Clement Date: Wed, 25 Sep 2024 18:51:19 +0800 Subject: [PATCH 8/9] apply suggestions: at most N occurrences Signed-off-by: Clement --- tests/integration/raft_log_test.go | 71 +++++++++++++++++++----------- 1 file changed, 45 insertions(+), 26 deletions(-) diff --git a/tests/integration/raft_log_test.go b/tests/integration/raft_log_test.go index 680f3049749..54e2080b01f 100644 --- a/tests/integration/raft_log_test.go +++ b/tests/integration/raft_log_test.go @@ -20,6 +20,8 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + pb "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/tests/v3/framework/integration" ) @@ -36,60 +38,77 @@ func TestRaftLogCompaction(t *testing.T) { defer clus.Terminate(t) mem := clus.Members[0] + + // Get applied index of raft log + endpoint := mem.Client.Endpoints()[0] + assert.NotEmpty(t, endpoint) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + status, _ := mem.Client.Status(ctx, endpoint) + appliedi := status.RaftAppliedIndex + // Assume applied index is less than 10, should be fine at this stage + assert.Less(t, appliedi, uint64(10)) + kvc := integration.ToGRPC(mem.Client).KV - // When starting a new cluster with 1 member, the member will have an index of 4. - // TODO: Can someone explain this? - // Currently, if `ep.appliedi-ep.snapi > s.Cfg.SnapshotCount`, - // a raft log snapshot is created, and raft log entries are compacted. - // In this case, it triggers when the index is a multiple of 11. - appliedi := 4 - for ; appliedi <= 10; appliedi++ { + // When applied index is a multiple of 11 (SnapshotCount+1), + // a snapshot is created, and entries are compacted. + // + // increase applied index to 10 + for ; appliedi < 10; appliedi++ { _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}) if err != nil { t.Errorf("#%d: couldn't put key (%v)", appliedi, err) } } // The first snapshot and compaction shouldn't happen because the index is less than 11 - expectMemberLogTimeout(t, mem, 5*time.Second, "saved snapshot", 1) - expectMemberLogTimeout(t, mem, time.Second, "compacted Raft logs", 1) + logOccurredAtMostNTimes(t, mem, 5*time.Second, "saved snapshot", 0) + logOccurredAtMostNTimes(t, mem, time.Second, "compacted Raft logs", 0) - for ; appliedi <= 11; appliedi++ { + // increase applied index to 11 + for ; appliedi < 11; appliedi++ { _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}) if err != nil { t.Errorf("#%d: couldn't put key (%v)", appliedi, err) } } // The first snapshot and compaction should happen because the index is 11 - expectMemberLog(t, mem, 5*time.Second, "saved snapshot", 1) - expectMemberLog(t, mem, time.Second, "compacted Raft logs", 1) + logOccurredAtMostNTimes(t, mem, 5*time.Second, "saved snapshot", 1) + logOccurredAtMostNTimes(t, mem, time.Second, "compacted Raft logs", 1) expectMemberLog(t, mem, time.Second, "\"compact-index\": 6", 1) - for ; appliedi <= 1100; appliedi++ { + // increase applied index to 1100 + for ; appliedi < 1100; appliedi++ { _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}) if err != nil { t.Errorf("#%d: couldn't put key (%v)", appliedi, err) } } - // With the index at 1100, snapshot and compaction should happen 100 times. - expectMemberLog(t, mem, 5*time.Second, "saved snapshot", 100) - expectMemberLog(t, mem, time.Second, "compacted Raft logs", 100) + // With applied index at 1100, snapshot and compaction should happen 100 times. + logOccurredAtMostNTimes(t, mem, 5*time.Second, "saved snapshot", 100) + logOccurredAtMostNTimes(t, mem, time.Second, "compacted Raft logs", 100) expectMemberLog(t, mem, time.Second, "\"compact-index\": 1095", 1) - - // No more snapshot and compaction should happen. - expectMemberLogTimeout(t, mem, 5*time.Second, "saved snapshot", 101) - expectMemberLogTimeout(t, mem, time.Second, "compacted Raft logs", 101) } -// expectMemberLogTimeout ensures that the log has fewer than `count` occurrences of `s` before timing out -func expectMemberLogTimeout(t *testing.T, m *integration.Member, timeout time.Duration, s string, count int) { +// logOccurredAtMostNTimes ensures that the log has exactly `count` occurrences of `s` before timing out, no more, no less. +func logOccurredAtMostNTimes(t *testing.T, m *integration.Member, timeout time.Duration, s string, count int) { ctx, cancel := context.WithTimeout(context.TODO(), timeout) defer cancel() + // The log must have `count` occurrences before timeout _, err := m.LogObserver.Expect(ctx, s, count) - if !errors.Is(err, context.DeadlineExceeded) { - if err != nil { - t.Fatalf("failed to expect (log:%s, count:%v): %v", s, count, err) + if err != nil { + t.Fatalf("failed to expect(log:%s, count:%d): %v", s, count, err) + } + + // The log mustn't have `count+1` occurrences before timeout + lines, err := m.LogObserver.Expect(ctx, s, count+1) + if err != nil { + if errors.Is(err, context.DeadlineExceeded) { + return + } else { + t.Fatalf("failed to expect(log:%s, count:%d): %v", s, count+1, err) } } -} + t.Fatalf("failed: too many occurrences of %s, expect %d, got %d", s, count, len(lines)) +} \ No newline at end of file From f34bee54aa16e237abf67c42ab0326cb223e9cf8 Mon Sep 17 00:00:00 2001 From: Clement Date: Wed, 25 Sep 2024 18:58:09 +0800 Subject: [PATCH 9/9] typo Signed-off-by: Clement --- tests/integration/raft_log_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/raft_log_test.go b/tests/integration/raft_log_test.go index 54e2080b01f..c38a15d8187 100644 --- a/tests/integration/raft_log_test.go +++ b/tests/integration/raft_log_test.go @@ -61,7 +61,7 @@ func TestRaftLogCompaction(t *testing.T) { t.Errorf("#%d: couldn't put key (%v)", appliedi, err) } } - // The first snapshot and compaction shouldn't happen because the index is less than 11 + // The first snapshot and compaction shouldn't happen because applied index is less than 11 logOccurredAtMostNTimes(t, mem, 5*time.Second, "saved snapshot", 0) logOccurredAtMostNTimes(t, mem, time.Second, "compacted Raft logs", 0) @@ -72,7 +72,7 @@ func TestRaftLogCompaction(t *testing.T) { t.Errorf("#%d: couldn't put key (%v)", appliedi, err) } } - // The first snapshot and compaction should happen because the index is 11 + // The first snapshot and compaction should happen because applied index is 11 logOccurredAtMostNTimes(t, mem, 5*time.Second, "saved snapshot", 1) logOccurredAtMostNTimes(t, mem, time.Second, "compacted Raft logs", 1) expectMemberLog(t, mem, time.Second, "\"compact-index\": 6", 1)