-
Notifications
You must be signed in to change notification settings - Fork 438
RATIS-2403. Support leader batch write to improve linearizable follower read throughput #1362
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from 4 commits
4a8a402
534240e
5898f39
7f58060
36f8903
bfc5182
589117b
f1a2bae
e595eab
60380ee
5128716
19321e3
511e7df
302b53e
c69b2b5
fb11a13
57a7a64
17d20ec
676c52b
cea0549
2cf81b8
85dd858
bcec68e
e9fe28b
ece8ab6
632e455
78db698
d48bcc9
a078432
45c1903
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ | |
| import org.apache.ratis.retry.RetryPolicy; | ||
| import org.apache.ratis.server.RaftServer; | ||
| import org.apache.ratis.server.RaftServerConfigKeys; | ||
| import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; | ||
| import org.apache.ratis.server.impl.MiniRaftCluster; | ||
| import org.apache.ratis.util.Slf4jUtils; | ||
| import org.apache.ratis.util.TimeDuration; | ||
|
|
@@ -60,7 +61,7 @@ public abstract class LinearizableReadTests<CLUSTER extends MiniRaftCluster> | |
|
|
||
| public abstract boolean isLeaderLeaseEnabled(); | ||
|
|
||
| public abstract boolean readIndexAppliedIndexEnabled(); | ||
| public abstract Type readIndexType(); | ||
|
|
||
| public abstract void assertRaftProperties(RaftProperties properties); | ||
|
|
||
|
|
@@ -77,7 +78,7 @@ public void setup() { | |
| CounterStateMachine.setProperties(p); | ||
| RaftServerConfigKeys.Read.setOption(p, LINEARIZABLE); | ||
| RaftServerConfigKeys.Read.setLeaderLeaseEnabled(p, isLeaderLeaseEnabled()); | ||
| RaftServerConfigKeys.Read.ReadIndex.setAppliedIndexEnabled(p, readIndexAppliedIndexEnabled()); | ||
| RaftServerConfigKeys.Read.ReadIndex.setType(p, readIndexType()); | ||
| } | ||
|
|
||
| @Test | ||
|
|
@@ -143,10 +144,12 @@ static <C extends MiniRaftCluster> void runTestFollowerLinearizableRead(C cluste | |
|
|
||
| @Test | ||
| public void testFollowerLinearizableReadParallel() throws Exception { | ||
| runWithNewCluster(LinearizableReadTests::runTestFollowerReadOnlyParallel); | ||
| final Type type = readIndexType(); | ||
| runWithNewCluster(cluster -> runTestFollowerReadOnlyParallel(type, cluster)); | ||
| } | ||
|
|
||
| static <C extends MiniRaftCluster> void runTestFollowerReadOnlyParallel(C cluster) throws Exception { | ||
| static <C extends MiniRaftCluster> void runTestFollowerReadOnlyParallel(Type readIndexType, C cluster) | ||
| throws Exception { | ||
| final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); | ||
|
|
||
| final List<RaftServer.Division> followers = cluster.getFollowers(); | ||
|
|
@@ -169,8 +172,17 @@ static <C extends MiniRaftCluster> void runTestFollowerReadOnlyParallel(C cluste | |
| writeReplies.add(new Reply(count, leaderClient.async().send(WAIT_AND_INCREMENT))); | ||
| Thread.sleep(100); | ||
|
|
||
| assertReplyExact(count, f0Client.io().sendReadOnly(QUERY, f0)); | ||
| f1Replies.add(new Reply(count, f1Client.async().sendReadOnly(QUERY, f1))); | ||
| if (readIndexType == Type.REPLIED_INDEX) { | ||
| // With REPLIED_INDEX the read index only advances after the leader has applied the | ||
| // transaction and the reply batch is flushed. WAIT_AND_INCREMENT takes 500 ms in | ||
| // the state machine but we only waited 100 ms, so its reply has not been generated | ||
| // yet and the follower read may only see the preceding sync INCREMENT (count - 1). | ||
| assertReplyAtLeast(count - 1, f0Client.io().sendReadOnly(QUERY, f0)); | ||
| f1Replies.add(new Reply(count - 1, f1Client.async().sendReadOnly(QUERY, f1))); | ||
| } else { | ||
| assertReplyExact(count, f0Client.io().sendReadOnly(QUERY, f0)); | ||
| f1Replies.add(new Reply(count, f1Client.async().sendReadOnly(QUERY, f1))); | ||
| } | ||
|
||
| } | ||
|
|
||
| for (int i = 0; i < n; i++) { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.