Skip to content

Conversation

@szetszwo
Copy link
Contributor

@szetszwo szetszwo requested a review from SzyWilliam September 29, 2023 18:32
Copy link
Member

@SzyWilliam SzyWilliam left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 the changes looks good to me! 🚀

*
* @return a future for the write task
*/
default CompletableFuture<?> write(LogEntryProto entry, TransactionContext context) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we add a @Nullable annotation here to notify user the context will be null when the peer is a follower?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a good point! Follower currently creates context in RaftServerImpl.applyLogToStateMachine. We should create a context instead of passing null and then the context can be reused. Let me think about it

Copy link
Member

@SzyWilliam SzyWilliam left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @szetszwo for the update! The patch looks great ;) There is one place that I don't quite get, could you help to explain?

CompletableFuture<ByteString> future = null;
if (stateMachine != null) {
future = stateMachine.data().read(entry).exceptionally(ex -> {
future = stateMachine.data().read(entry, server.getTransactionContext(entry, false)).exceptionally(ex -> {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The TransactionManager manages the TransactionContext for non-leaders, while PendingRequests manages the context for the leader, am I getting it right?
Normally when DataApi.read is called, this peer is leader and is going to replicate the full log to non-leaders. Why do we need to call getTransactionContext here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right that the context should be obtained from LeaderState. Let me update the change.

@SzyWilliam
Copy link
Member

testTransactionContextIsPassedBack failed constantly. In my local env,

java.lang.AssertionError
org.apache.ratis.statemachine.TestStateMachine$SMTransactionContext.applyTransaction(TestStateMachine.java:94)
	at org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:1842)
	at org.apache.ratis.server.impl.StateMachineUpdater.applyLog(StateMachineUpdater.java:244)
	at org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:186)
	at java.lang.Thread.run(Thread.java:750)

Guess it's related to getTransactionContext. The leader somehow cannot obtain the client request from transaction context.

@szetszwo
Copy link
Contributor Author

@SzyWilliam , you are right that there is a bug in getTransactionContext. It should also check leader when createNew == true.

@szetszwo
Copy link
Contributor Author

Forced push for resolving a conflict.

Copy link
Member

@SzyWilliam SzyWilliam left a 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 to me. Thanks @szetszwo again for the big patch!

@SzyWilliam SzyWilliam merged commit ba8e62c into apache:master Oct 12, 2023
@szetszwo
Copy link
Contributor Author

@SzyWilliam , thanks a lot for carefully reviewing this!

RexXiong pushed a commit to apache/celeborn that referenced this pull request May 30, 2024
### 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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants