Skip to content
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

IGNITE-24484 RAFT heartbeat coalescing #5221

Open
wants to merge 6 commits into
base: main
Choose a base branch
from

Conversation

vldpyatkov
Copy link
Contributor

@@ -88,19 +93,78 @@ public Future<Message> requestVote(final PeerId peerId, final RequestVoteRequest

@Override
public Future<Message> appendEntries(final PeerId peerId, final AppendEntriesRequest request,
final int timeoutMs, final RpcResponseClosure<AppendEntriesResponse> done) {
final int timeoutMs, final RpcResponseClosure<AppendEntriesResponse> done) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Formatting issues here and below.

Copy link
Contributor Author

@vldpyatkov vldpyatkov Mar 10, 2025

Choose a reason for hiding this comment

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

It was formatted by our rules.

Copy link
Contributor

@ascherbakoff ascherbakoff Mar 10, 2025

Choose a reason for hiding this comment

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

Our rules forbid reformatting of jraft package completely

@@ -198,7 +198,7 @@ void executesWrongJobClassOnRemoteNodesAsync(String jobClassName, int errorCode,
JobTarget.anyNode(clusterNode(node(1)), clusterNode(node(2))),
JobDescriptor.builder(jobClassName).units(units()).build(),
null
).get(1, TimeUnit.SECONDS));
).get(10, TimeUnit.SECONDS));
Copy link
Contributor

Choose a reason for hiding this comment

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

Why is this required ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Computer tasks become slower. I do not think exactly why, but running any task takes more time than before the patch. And also compute that sute on TC becomes longer.

Copy link
Contributor

Choose a reason for hiding this comment

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

This is definintely a blocker for PR merging.


if (connect(peerId)) { // Replicator should be started asynchronously by node joined event.
if (isHeartbeatRequest(request)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

This check is already done in the upper level in
org.apache.ignite.raft.jraft.core.Replicator#sendEmptyEntries(boolean, org.apache.ignite.raft.jraft.rpc.RpcResponseClosure<org.apache.ignite.raft.jraft.rpc.RpcRequests.AppendEntriesResponse>, boolean)
Just call sendHeartbeat from this method.

           if (isHeartbeat) {
                ....
                this.heartbeatInFly = this.rpcService.sendHeartbeat(this.options.getPeerId().getEndpoint(), request, coalesce,
                        this.options.getElectionTimeoutMs() / 2, heartbeatDone);
            }

Copy link
Contributor Author

@vldpyatkov vldpyatkov Mar 10, 2025

Choose a reason for hiding this comment

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

I know, but it would be more complicated due to component dependencies. We already use this manner of
checking in AppendEntriesRequestProcessor.

Copy link
Contributor

Choose a reason for hiding this comment

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

It was implemented this way in the PoC without any dependency issues.
Ok. let's keep it as is, but at least remove copypaste by moving isHearbeat(..) to utitlities class.

* @param request Append entries request.
* @return True if that request is heartbeat or false otherwise.
*/
private static boolean isHeartbeatRequest(final AppendEntriesRequest request) {
Copy link
Contributor

Choose a reason for hiding this comment

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

This is not needed. See the comment above.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The same as AppendEntriesRequestProcessor#isHeartbeatRequest

final Executor rpcExecutor) {
final RpcClient rc = this.rpcClient;
public <T extends Message> CompletableFuture<Message> invokeWithDone(
PeerId peerId,
Copy link
Contributor

Choose a reason for hiding this comment

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

Avoid reformatting raft code.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I applied formatting that is used in our entire project.

Copy link
Contributor

Choose a reason for hiding this comment

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

Formatting of jraft code is forbidden by style rules.

*
* @return The future.
*/
CompletableFuture<Message> invokeAsync(
Copy link
Contributor

Choose a reason for hiding this comment

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

I was not able to understand why this interface is introduced.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The interface is requested to use it for coalessing requests. This interface allows us to not immediately send messages to the network. Instead, you can handle the message manually.
org.apache.ignite.raft.jraft.rpc.impl.core.DefaultRaftClientService#sendHeartbeat

scheduler = opts.getScheduler();
messagesFactory = opts.getRaftMessagesFactory();

scheduler.schedule(this::onSentHeartbeat , opts.getElectionTimeoutMs(), TimeUnit.MILLISECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

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

This scheme is not efficient.
Currently each raft group generate it's own timer event in r.startHeartbeatTimer(Utils.nowMs());
Which means events multiplication linear to groups number and excessive CPU usage.
Instead, a single event per all groups can be triggered.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Probably we can create another ticket for this? Or you insist to do this improvement in this one.

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm ok with another ticket.

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