Skip to content

Commit

Permalink
ledgerFragment check and results keep order (#3504)
Browse files Browse the repository at this point in the history
### Motivation
When `LedgerChecker` checks Ledger's fragments, it does not keep the order, and the results of the check are not kept in order, which works correctly most of the time.

I'm not sure if we should keep the order of checks and final results, but I observed a flaky-test like this:
https://github.com/apache/bookkeeper/actions/runs/3124965464/jobs/5068848112

I have printed some verbose logs as follows:
testShouldGetTwoFrgamentsIfTwoBookiesFailedInSameEnsemble In the case of success and failure:

### When the test execution is successful:
2022-09-26T15:00:13,790 - INFO  - [Time-limited test:LedgerChecker@373] - checkLedger nowing.
2022-09-26T15:00:13,792 - INFO  - [Time-limited test:LedgerChecker@398] - LedgerCheck fragments before[Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50466, 127.0.0.1:50474, 127.0.0.1:50476], Closed: true)]
2022-09-26T15:00:13,793 - INFO  - [Time-limited test:LedgerChecker@430] - not execute check fragments. curEnsemble [127.0.0.1:50466, 127.0.0.1:50486, 127.0.0.1:50476], curEntryId 1
**2022-09-26T15:00:13,809 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker$1@440] - check2Fragments [Fragment(LedgerID: 0, **FirstEntryID: 0[0]**, LastKnownEntryID: 0[0], Host: [127.0.0.1:50466, 127.0.0.1:50474, 127.0.0.1:50476], Closed: true), Fragment(LedgerID: 0, **FirstEntryID: 1[1]**, LastKnownEntryID: 1[1], Host: [127.0.0.1:50466, 127.0.0.1:50476, 127.0.0.1:50486], Closed: false)]** 
2022-09-26T15:00:13,810 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker@479] - Checking fragment Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50466, 127.0.0.1:50474, 127.0.0.1:50476], Closed: true)
2022-09-26T15:00:13,813 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker$FullLedgerCallback@360] - operationComplete Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50466, 127.0.0.1:50474], Closed: true), -8
2022-09-26T15:00:13,813 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker@479] - Checking fragment Fragment(LedgerID: 0, FirstEntryID: 1[1], LastKnownEntryID: 1[1], Host: [127.0.0.1:50466, 127.0.0.1:50476, 127.0.0.1:50486], Closed: false)
2022-09-26T15:00:13,817 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker$FullLedgerCallback@360] - operationComplete Fragment(LedgerID: 0, FirstEntryID: 1[-1], LastKnownEntryID: 1[-1], Host: [127.0.0.1:50466], Closed: false), -8
**2022-09-26T15:00:13,817 - INFO  - [Time-limited test:TestLedgerChecker@534] - waitAndGetResult [Fragment(LedgerID: 0, **FirstEntryID: 0[0]**, LastKnownEntryID: 0[0], Host: [127.0.0.1:50466, 127.0.0.1:50474], Closed: true), Fragment(LedgerID: 0, **FirstEntryID: 1[-1]**, LastKnownEntryID: 1[-1], Host: [127.0.0.1:50466], Closed: false)]** 
2022-09-26T15:00:13,818 - INFO  - [Time-limited test:TestLedgerChecker@201] - other Ensemble {0=[127.0.0.1:50466, 127.0.0.1:50474, 127.0.0.1:50476], 1=[127.0.0.1:50466, 127.0.0.1:50486, 127.0.0.1:50476]}
2022-09-26T15:00:13,818 - INFO  - [Time-limited test:TestLedgerChecker@206] - unreplicated fragment: Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50466, 127.0.0.1:50474], Closed: true)
2022-09-26T15:00:13,819 - INFO  - [Time-limited test:TestLedgerChecker@206] - unreplicated fragment: Fragment(LedgerID: 0, FirstEntryID: 1[-1], LastKnownEntryID: 1[-1], Host: [127.0.0.1:50466], Closed: false)
2022-09-26T15:00:13,819 - INFO  - [Time-limited test:TestLedgerChecker@212] - bookies fragments [127.0.0.1:50466, 127.0.0.1:50474, 127.0.0.1:50476], [0, 1]
2022-09-26T15:00:13,819 - INFO  - [Time-limited test:BookKeeperClusterTestCase@199] - TearDown

### When test execution failed:
2022-09-26T14:58:09,687 - INFO  - [Time-limited test:LedgerChecker@373] - checkLedger nowing.
2022-09-26T14:58:09,689 - INFO  - [Time-limited test:LedgerChecker@398] - LedgerCheck fragments before[Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50105, 127.0.0.1:50107, 127.0.0.1:50109], Closed: true)]
2022-09-26T14:58:09,690 - INFO  - [Time-limited test:LedgerChecker@430] - not execute check fragments. curEnsemble [127.0.0.1:50105, 127.0.0.1:50122, 127.0.0.1:50107], curEntryId 1
**2022-09-26T14:58:09,704 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker$1@440] - check2Fragments [Fragment(LedgerID: 0, **FirstEntryID: 1[1]**, LastKnownEntryID: 1[1], Host: [127.0.0.1:50105, 127.0.0.1:50122, 127.0.0.1:50107], Closed: false), Fragment(LedgerID: 0, **FirstEntryID: 0[0]**, LastKnownEntryID: 0[0], Host: [127.0.0.1:50105, 127.0.0.1:50107, 127.0.0.1:50109], Closed: true)]** 
2022-09-26T14:58:09,705 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker@479] - Checking fragment Fragment(LedgerID: 0, FirstEntryID: 1[1], LastKnownEntryID: 1[1], Host: [127.0.0.1:50105, 127.0.0.1:50122, 127.0.0.1:50107], Closed: false)
2022-09-26T14:58:09,707 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker@479] - Checking fragment Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50105, 127.0.0.1:50107, 127.0.0.1:50109], Closed: true)
2022-09-26T14:58:09,708 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker$FullLedgerCallback@360] - operationComplete Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50105, 127.0.0.1:50109], Closed: true), -8
2022-09-26T14:58:09,708 - INFO  - [BookKeeperClientWorker-OrderedExecutor-0-0:LedgerChecker$FullLedgerCallback@360] - operationComplete Fragment(LedgerID: 0, FirstEntryID: 1[-1], LastKnownEntryID: 1[-1], Host: [127.0.0.1:50105], Closed: false), -8
**2022-09-26T14:58:09,709 - INFO  - [Time-limited test:TestLedgerChecker@534] - waitAndGetResult [Fragment(LedgerID: 0, **FirstEntryID: 1[-1]**, LastKnownEntryID: 1[-1], Host: [127.0.0.1:50105], Closed: false), Fragment(LedgerID: 0, **FirstEntryID: 0[0]**, LastKnownEntryID: 0[0], Host: [127.0.0.1:50105, 127.0.0.1:50109], Closed: true)]**
2022-09-26T14:58:09,709 - INFO  - [Time-limited test:TestLedgerChecker@201] - other Ensemble {0=[127.0.0.1:50105, 127.0.0.1:50109, 127.0.0.1:50107], 1=[127.0.0.1:50105, 127.0.0.1:50122, 127.0.0.1:50107]}
2022-09-26T14:58:09,709 - INFO  - [Time-limited test:TestLedgerChecker@206] - unreplicated fragment: Fragment(LedgerID: 0, FirstEntryID: 1[-1], LastKnownEntryID: 1[-1], Host: [127.0.0.1:50105], Closed: false)
2022-09-26T14:58:09,709 - INFO  - [Time-limited test:TestLedgerChecker@206] - unreplicated fragment: Fragment(LedgerID: 0, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host: [127.0.0.1:50105, 127.0.0.1:50109], Closed: true)
2022-09-26T14:58:09,709 - INFO  - [Time-limited test:TestLedgerChecker@212] - bookies fragments [127.0.0.1:50105, 127.0.0.1:50122, 127.0.0.1:50107], [0]
2022-09-26T14:58:09,710 - INFO  - [Time-limited test:BookKeeperClusterTestCase@199] - TearDown

### Changes
Replace `HashSet` with `LinkedHashSet`
  • Loading branch information
wenbingshen committed Sep 28, 2022
1 parent 1e387f8 commit 74a87e2
Showing 1 changed file with 3 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.netty.buffer.ByteBuf;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
Expand Down Expand Up @@ -343,7 +344,7 @@ private static class FullLedgerCallback implements

FullLedgerCallback(long numFragments,
GenericCallback<Set<LedgerFragment>> cb) {
badFragments = new HashSet<LedgerFragment>();
badFragments = new LinkedHashSet<>();
this.numFragments = new AtomicLong(numFragments);
this.cb = cb;
}
Expand Down Expand Up @@ -375,7 +376,7 @@ public void checkLedger(final LedgerHandle lh,
final GenericCallback<Set<LedgerFragment>> cb,
long percentageOfLedgerFragmentToBeVerified) {
// build a set of all fragment replicas
final Set<LedgerFragment> fragments = new HashSet<LedgerFragment>();
final Set<LedgerFragment> fragments = new LinkedHashSet<>();

Long curEntryId = null;
List<BookieId> curEnsemble = null;
Expand Down

0 comments on commit 74a87e2

Please sign in to comment.