-
Notifications
You must be signed in to change notification settings - Fork 440
RATIS-1882. Support read-after-write consistency #913
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
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 the change looks good. Just a very minor comment inlined.
Also, spent some time to add another unit test; see
https://issues.apache.org/jira/secure/attachment/13062665/TestReadAfterWrite.patch
Could you include it?
| final CompletableFuture<Long> future = ref != null? ref.get(): null; | ||
| if (future != null) { | ||
| return future; | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just found that these few lines can be simplified as below.
if (ref != null) {
return ref.get();
}There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
|
|
||
| @Override | ||
| public CompletableFuture<RaftClientReply> sendReadAfterWrite(Message message) { | ||
| return send(RaftClientRequest.readAfterWriteConsistentRequestType(), message, null); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@SzyWilliam , Should sendReadAfterWrite use UnorderedAsync.send(..)? Indeed, all the linearizable reads should use UnorderedAsync.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's true! Thanks for pointing out, I'll change all the linearizable reads to use UnorderedAsync
|
@szetszwo Thanks very much for the patch! It's good to know |
szetszwo
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's use OrderedAsync due to the test failure. Could you also include the following?
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
index 4b0c5166..220694ce 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
@@ -227,7 +227,10 @@ public class RaftClientRequest extends RaftClientMessage {
case MESSAGESTREAM:
return toString(getMessageStream());
case READ:
- return "RO";
+ final ReadRequestTypeProto read = getRead();
+ return read.getReadAfterWriteConsistent()? "RaW"
+ : read.getPreferNonLinearizable()? "RO(pNL)"
+ : "RO";
case STALEREAD:
return "StaleRead(" + getStaleRead().getMinIndex() + ")";
case WATCH:
diff --git a/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java b/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java
index f8f2405c..f515628c 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java
@@ -128,8 +128,10 @@ public class TestReadAfterWrite
LOG.info("readAfterWrite.get");
try {
// readAfterWrite should time out
- readAfterWrite.get(100, TimeUnit.MILLISECONDS);
- Assert.fail();
+ final RaftClientReply reply = readAfterWrite.get(100, TimeUnit.MILLISECONDS);
+ final DoubleValue result = (DoubleValue) Expression.Utils.bytes2Expression(
+ reply.getMessage().getContent().toByteArray(), 0);
+ Assert.fail("result=" + result + ", reply=" + reply);
} catch (TimeoutException e) {
LOG.info("Good", e);
}| @Override | ||
| public CompletableFuture<RaftClientReply> sendReadAfterWrite(Message message) { | ||
| return send(RaftClientRequest.readAfterWriteConsistentRequestType(), message, null); | ||
| return UnorderedAsync.send(RaftClientRequest.readAfterWriteConsistentRequestType(), message, null, client); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, it actually not working since UnorderedAsync use a different channel. It can arrive the server earlier than the previous write request and get a small readIndex. TestReadAfterWrite will fail.
- Using unordered async
2023-09-04 11:16:59,888 DEBUG server.RaftServer$Division (LeaderStateImpl.java:getReadIndex(1099)) - readIndex=1, readAfterWriteConsistentIndex=1
- Using ordered async
2023-09-04 11:11:36,917 DEBUG server.RaftServer$Division (LeaderStateImpl.java:getReadIndex(1099)) - readIndex=3, readAfterWriteConsistentIndex=3
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You are right. Just investigated on OrderedAsync and UnorderedAsync and now I know why we should choose OrderedAsync. Thanks very much for your time spent and the detailed explanation!
|
Cool! I'm checking this out. |
|
May I ask what is the target release version of this feature? |
|
We should include this to both 3.0.0 and the next branch 2 releases. |
### What changes were proposed in this pull request? Bump Ratis version from 2.5.1 to 3.0.1. Address incompatible changes: - RATIS-589. Eliminate buffer copying in SegmentedRaftLogOutputStream.(apache/ratis#964) - RATIS-1677. Do not auto format RaftStorage in RECOVER.(apache/ratis#718) - RATIS-1710. Refactor metrics api and implementation to separated modules. (apache/ratis#749) ### Why are the changes needed? Bump Ratis version from 2.5.1 to 3.0.1. Ratis has released v3.0.0, v3.0.1, which release note refers to [3.0.0](https://ratis.apache.org/post/3.0.0.html), [3.0.1](https://ratis.apache.org/post/3.0.1.html). The 3.0.x version include new features like pluggable metrics and lease read, etc, some improvements and bugfixes including: - 3.0.0: Change list of ratis 3.0.0 In total, there are roughly 100 commits diffing from 2.5.1 including: - Incompatible Changes - RaftStorage Auto-Format - RATIS-1677. Do not auto format RaftStorage in RECOVER. (apache/ratis#718) - RATIS-1694. Fix the compatibility issue of RATIS-1677. (apache/ratis#731) - RATIS-1871. Auto format RaftStorage when there is only one directory configured. (apache/ratis#903) - Pluggable Ratis-Metrics (RATIS-1688) - RATIS-1689. Remove the use of the thirdparty Gauge. (apache/ratis#728) - RATIS-1692. Remove the use of the thirdparty Counter. (apache/ratis#732) - RATIS-1693. Remove the use of the thirdparty Timer. (apache/ratis#734) - RATIS-1703. Move MetricsReporting and JvmMetrics to impl. (apache/ratis#741) - RATIS-1704. Fix SuppressWarnings(“VisibilityModifier”) in RatisMetrics. (apache/ratis#742) - RATIS-1710. Refactor metrics api and implementation to separated modules. (apache/ratis#749) - RATIS-1712. Add a dropwizard 3 implementation of ratis-metrics-api. (apache/ratis#751) - RATIS-1391. Update library dropwizard.metrics version to 4.x (apache/ratis#632) - RATIS-1601. Use the shaded dropwizard metrics and remove the dependency (apache/ratis#671) - Streaming Protocol Change - RATIS-1569. Move the asyncRpcApi.sendForward(..) call to the client side. (apache/ratis#635) - New Features - Leader Lease (RATIS-1864) - RATIS-1865. Add leader lease bound ratio configuration (apache/ratis#897) - RATIS-1866. Maintain leader lease after AppendEntries (apache/ratis#898) - RATIS-1894. Implement ReadOnly based on leader lease (apache/ratis#925) - RATIS-1882. Support read-after-write consistency (apache/ratis#913) - StateMachine API - RATIS-1874. Add notifyLeaderReady function in IStateMachine (apache/ratis#906) - RATIS-1897. Make TransactionContext available in DataApi.write(..). (apache/ratis#930) - New Configuration Properties - RATIS-1862. Add the parameter whether to take Snapshot when stopping to adapt to different services (apache/ratis#896) - RATIS-1930. Add a conf for enable/disable majority-add. (apache/ratis#961) - RATIS-1918. Introduces parameters that separately control the shutdown of RaftServerProxy by JVMPauseMonitor. (apache/ratis#950) - RATIS-1636. Support re-config ratis properties (apache/ratis#800) - RATIS-1860. Add ratis-shell cmd to generate a new raft-meta.conf. (apache/ratis#901) - Improvements & Bug Fixes - Netty - RATIS-1898. Netty should use EpollEventLoopGroup by default (apache/ratis#931) - RATIS-1899. Use EpollEventLoopGroup for Netty Proxies (apache/ratis#932) - RATIS-1921. Shared worker group in WorkerGroupGetter should be closed. (apache/ratis#955) - RATIS-1923. Netty: atomic operations require side-effect-free functions. (apache/ratis#956) - RaftServer - RATIS-1924. Increase the default of raft.server.log.segment.size.max. (apache/ratis#957) - RATIS-1892. Unify the lifetime of the RaftServerProxy thread pool (apache/ratis#923) - RATIS-1889. NoSuchMethodError: RaftServerMetricsImpl.addNumPendingRequestsGauge apache/ratis#922 (apache/ratis#922) - RATIS-761. Handle writeStateMachineData failure in leader. (apache/ratis#927) - RATIS-1902. The snapshot index is set incorrectly in InstallSnapshotReplyProto. (apache/ratis#933) - RATIS-1912. Fix infinity election when perform membership change. (apache/ratis#954) - RATIS-1858. Follower keeps logging first election timeout. (apache/ratis#894) - 3.0.1:This is a bugfix release. See the [changes between 3.0.0 and 3.0.1](apache/ratis@ratis-3.0.0...ratis-3.0.1) releases. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Cluster manual test. Closes #2480 from SteNicholas/CELEBORN-1400. Authored-by: SteNicholas <programgeek@163.com> Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
What changes were proposed in this pull request?
Support read-after-write consistency.
Thanks @szetszwo very much for the patch. Only made minor changes and added unit tests to the patch.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-1882
How was this patch tested?
New unit tests.