Skip to content

Commit f7d10a8

Browse files
authored
feat: implement fast-failover for MessageRecvManager and DataClientManager (#243)
1 parent 31b63b2 commit f7d10a8

File tree

7 files changed

+58
-37
lines changed

7 files changed

+58
-37
lines changed

computer-core/src/main/java/org/apache/hugegraph/computer/core/network/DataClientManager.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -118,10 +118,9 @@ public void onChannelInactive(ConnectionId connectionId) {
118118
@Override
119119
public void exceptionCaught(TransportException cause,
120120
ConnectionId connectionId) {
121-
// TODO: implement failover
122121
LOG.error("Channel for connectionId {} occurred exception",
123122
connectionId, cause);
124-
DataClientManager.this.connManager.closeClient(connectionId);
123+
DataClientManager.this.sender.transportExceptionCaught(cause, connectionId);
125124
}
126125
}
127126
}

computer-core/src/main/java/org/apache/hugegraph/computer/core/network/netty/ChannelFutureListenerOnWrite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ public void onDone(Channel channel, ChannelFuture future) {
5555
}
5656
}
5757

58-
public void onSuccess(Channel channel, ChannelFuture future) {
58+
public void onSuccess(Channel channel, ChannelFuture future) {
5959
if (LOG.isDebugEnabled()) {
6060
LOG.debug("Successfully send data to '{}'",
6161
TransportUtil.remoteAddress(channel));

computer-core/src/main/java/org/apache/hugegraph/computer/core/receiver/MessageRecvManager.java

Lines changed: 28 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,11 @@
1818
package org.apache.hugegraph.computer.core.receiver;
1919

2020
import java.util.Map;
21-
import java.util.concurrent.CountDownLatch;
21+
import java.util.concurrent.ExecutionException;
2222
import java.util.concurrent.TimeUnit;
23+
import java.util.concurrent.CompletableFuture;
24+
import java.util.concurrent.TimeoutException;
25+
import java.util.concurrent.atomic.AtomicInteger;
2326

2427
import org.apache.hugegraph.computer.core.common.ComputerContext;
2528
import org.apache.hugegraph.computer.core.common.Constants;
@@ -60,7 +63,9 @@ public class MessageRecvManager implements Manager, MessageHandler {
6063

6164
private int workerCount;
6265
private int expectedFinishMessages;
63-
private CountDownLatch finishMessagesLatch;
66+
private CompletableFuture<Void> finishMessagesFuture;
67+
private AtomicInteger finishMessagesCount;
68+
6469
private long waitFinishMessagesTimeout;
6570
private long superstep;
6671

@@ -71,6 +76,7 @@ public MessageRecvManager(ComputerContext context,
7176
this.fileManager = fileManager;
7277
this.sortManager = sortManager;
7378
this.superstep = Constants.INPUT_SUPERSTEP;
79+
this.finishMessagesCount = new AtomicInteger();
7480
}
7581

7682
@Override
@@ -90,8 +96,9 @@ public void init(Config config) {
9096
this.workerCount = config.get(ComputerOptions.JOB_WORKERS_COUNT);
9197
// One for vertex and one for edge.
9298
this.expectedFinishMessages = this.workerCount * 2;
93-
this.finishMessagesLatch = new CountDownLatch(
94-
this.expectedFinishMessages);
99+
this.finishMessagesFuture = new CompletableFuture<>();
100+
this.finishMessagesCount.set(this.expectedFinishMessages);
101+
95102
this.waitFinishMessagesTimeout = config.get(
96103
ComputerOptions.WORKER_WAIT_FINISH_MESSAGES_TIMEOUT);
97104
}
@@ -103,8 +110,9 @@ public void beforeSuperstep(Config config, int superstep) {
103110
this.messagePartitions = new ComputeMessageRecvPartitions(
104111
this.context, fileGenerator, this.sortManager);
105112
this.expectedFinishMessages = this.workerCount;
106-
this.finishMessagesLatch = new CountDownLatch(
107-
this.expectedFinishMessages);
113+
this.finishMessagesFuture = new CompletableFuture<>();
114+
this.finishMessagesCount.set(this.expectedFinishMessages);
115+
108116
this.superstep = superstep;
109117

110118
if (this.superstep == Constants.INPUT_SUPERSTEP + 1) {
@@ -138,35 +146,21 @@ public void onChannelInactive(ConnectionId connectionId) {
138146
@Override
139147
public void exceptionCaught(TransportException cause,
140148
ConnectionId connectionId) {
141-
// TODO: implement failover
142-
LOG.warn("Exception caught for connection:{}, root cause:",
149+
LOG.error("Exception caught for connection:{}, root cause:",
143150
connectionId, cause);
151+
this.finishMessagesFuture.completeExceptionally(cause);
144152
}
145153

146154
public void waitReceivedAllMessages() {
147155
try {
148-
boolean status = this.finishMessagesLatch.await(
149-
this.waitFinishMessagesTimeout,
150-
TimeUnit.MILLISECONDS);
151-
if (!status) {
152-
throw new ComputerException(
153-
"Expect %s finish-messages received in %s ms, " +
154-
"%s absence in superstep %s",
155-
this.expectedFinishMessages,
156-
this.waitFinishMessagesTimeout,
157-
this.finishMessagesLatch.getCount(),
158-
this.superstep);
159-
}
160-
} catch (InterruptedException e) {
161-
throw new ComputerException(
162-
"Thread is interrupted while waiting %s " +
163-
"finish-messages received in %s ms, " +
164-
"%s absence in superstep %s",
165-
e,
166-
this.expectedFinishMessages,
167-
this.waitFinishMessagesTimeout,
168-
this.finishMessagesLatch.getCount(),
169-
this.superstep);
156+
this.finishMessagesFuture.get(this.waitFinishMessagesTimeout, TimeUnit.MILLISECONDS);
157+
} catch (TimeoutException e) {
158+
throw new ComputerException("Time out while waiting %s finish-messages " +
159+
"received in %s ms in superstep %s",
160+
this.expectedFinishMessages, this.waitFinishMessagesTimeout, this.superstep, e);
161+
} catch (InterruptedException | ExecutionException e) {
162+
throw new ComputerException("Error while waiting %s finish-messages in superstep %s",
163+
this.expectedFinishMessages, this.superstep, e);
170164
}
171165
}
172166

@@ -214,7 +208,10 @@ public void onStarted(ConnectionId connectionId) {
214208
@Override
215209
public void onFinished(ConnectionId connectionId) {
216210
LOG.debug("ConnectionId {} finished", connectionId);
217-
this.finishMessagesLatch.countDown();
211+
int currentCount = this.finishMessagesCount.decrementAndGet();
212+
if (currentCount == 0) {
213+
this.finishMessagesFuture.complete(null);
214+
}
218215
}
219216

220217
/**

computer-core/src/main/java/org/apache/hugegraph/computer/core/sender/MessageSendManager.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -154,7 +154,7 @@ public void startSend(MessageType type) {
154154
}
155155

156156
/**
157-
* Finsih send message, send the last buffer and put an END signal
157+
* Finish send message, send the last buffer and put an END signal
158158
* into queue
159159
* @param type the message type
160160
*/
@@ -277,10 +277,10 @@ private void sendControlMessageToWorkers(Set<Integer> workerIds,
277277
}
278278
} catch (TimeoutException e) {
279279
throw new ComputerException("Timeout(%sms) to wait for " +
280-
"controling message(%s) to finished",
280+
"controlling message(%s) to finished",
281281
e, timeout, type);
282282
} catch (InterruptedException | ExecutionException e) {
283-
throw new ComputerException("Failed to wait for controling " +
283+
throw new ComputerException("Failed to wait for controlling " +
284284
"message(%s) to finished", e, type);
285285
}
286286
}

computer-core/src/main/java/org/apache/hugegraph/computer/core/sender/MessageSender.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@
1919

2020
import java.util.concurrent.CompletableFuture;
2121

22+
import org.apache.hugegraph.computer.core.common.exception.TransportException;
23+
import org.apache.hugegraph.computer.core.network.ConnectionId;
2224
import org.apache.hugegraph.computer.core.network.message.MessageType;
2325

2426
public interface MessageSender {
@@ -37,4 +39,10 @@ CompletableFuture<Void> send(int workerId, MessageType type)
3739
* @param message message payload
3840
*/
3941
void send(int workerId, QueuedMessage message) throws InterruptedException;
42+
43+
/**
44+
* Invoked when the channel associated with the given connectionId has
45+
* an exception is thrown processing message.
46+
*/
47+
void transportExceptionCaught(TransportException cause, ConnectionId connectionId);
4048
}

computer-core/src/main/java/org/apache/hugegraph/computer/core/sender/QueuedMessageSender.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import org.apache.hugegraph.computer.core.common.exception.TransportException;
2525
import org.apache.hugegraph.computer.core.config.ComputerOptions;
2626
import org.apache.hugegraph.computer.core.config.Config;
27+
import org.apache.hugegraph.computer.core.network.ConnectionId;
2728
import org.apache.hugegraph.computer.core.network.TransportClient;
2829
import org.apache.hugegraph.computer.core.network.message.MessageType;
2930
import org.apache.hugegraph.concurrent.BarrierEvent;
@@ -103,6 +104,15 @@ public void send(int workerId, QueuedMessage message)
103104
channel.queue.put(message);
104105
}
105106

107+
@Override
108+
public void transportExceptionCaught(TransportException cause, ConnectionId connectionId) {
109+
for (WorkerChannel channel : this.channels) {
110+
if (channel.client.connectionId().equals(connectionId)) {
111+
channel.futureRef.get().completeExceptionally(cause);
112+
}
113+
}
114+
}
115+
106116
public Runnable notBusyNotifier() {
107117
/*
108118
* DataClientHandler.sendAvailable() will call it when client

computer-test/src/main/java/org/apache/hugegraph/computer/core/compute/MockMessageSender.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@
1919

2020
import java.util.concurrent.CompletableFuture;
2121

22+
import org.apache.hugegraph.computer.core.common.exception.TransportException;
23+
import org.apache.hugegraph.computer.core.network.ConnectionId;
2224
import org.apache.hugegraph.computer.core.network.message.MessageType;
2325
import org.apache.hugegraph.computer.core.sender.MessageSender;
2426
import org.apache.hugegraph.computer.core.sender.QueuedMessage;
@@ -36,4 +38,9 @@ public CompletableFuture<Void> send(int workerId, MessageType type) {
3638
public void send(int workerId, QueuedMessage message) {
3739
// pass
3840
}
41+
42+
@Override
43+
public void transportExceptionCaught(TransportException cause, ConnectionId connectionId) {
44+
// pass
45+
}
3946
}

0 commit comments

Comments
 (0)