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

Make calls on cacheArtifacts/reconnect/submit/canel non-blocking #556

Merged
merged 3 commits into from
Sep 14, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ CompletableFuture<TaskExecutorID> getTaskExecutorFor(
*/
CompletableFuture<TaskExecutorGateway> getTaskExecutorGateway(TaskExecutorID taskExecutorID);

CompletableFuture<TaskExecutorGateway> reconnectTaskExecutorGateway(TaskExecutorID taskExecutorID);
CompletableFuture<Ack> reconnectTaskExecutorGateway(TaskExecutorID taskExecutorID);
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: s/reconnect.*Gateway/reconnect


CompletableFuture<TaskExecutorRegistration> getTaskExecutorInfo(String hostName);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@
import io.mantisrx.server.master.resourcecluster.TaskExecutorReport.Occupied;
import io.mantisrx.server.master.resourcecluster.TaskExecutorStatusChange;
import io.mantisrx.server.master.scheduler.JobMessageRouter;
import io.mantisrx.server.worker.TaskExecutorGateway;
import io.mantisrx.server.worker.TaskExecutorGateway.TaskNotFoundException;
import io.mantisrx.shaded.com.google.common.base.Preconditions;
import io.mantisrx.shaded.com.google.common.collect.Comparators;
Expand Down Expand Up @@ -399,7 +398,12 @@ private void onTaskExecutorGatewayReconnectRequest(TaskExecutorGatewayReconnectR
} else {
try {
if (state.isRegistered()) {
sender().tell(state.reconnect().join(), self());
state.reconnect().whenComplete((res, throwable) -> {
if (throwable != null) {
log.error("failed to reconnect to {}", request.getTaskExecutorID(), throwable);
}
});
sender().tell(Ack.getInstance(), self());
} else {
sender().tell(
new Status.Failure(
Expand Down Expand Up @@ -813,11 +817,22 @@ private void onCacheJobArtifactsOnTaskExecutorRequest(CacheJobArtifactsOnTaskExe
TaskExecutorState state = this.executorStateManager.get(request.getTaskExecutorID());
if (state != null && state.isRegistered()) {
try {
TaskExecutorGateway gateway = state.getGateway();
// TODO(fdichiara): store URI directly to avoid remapping for each TE
List<URI> artifacts = jobArtifactsToCache.stream().map(artifactID -> URI.create(artifactID.getResourceID())).collect(Collectors.toList());

gateway.cacheJobArtifacts(new CacheJobArtifactsRequest(artifacts));
state.getGatewayAsync()
.thenComposeAsync(taskExecutorGateway ->
taskExecutorGateway.cacheJobArtifacts(new CacheJobArtifactsRequest(
jobArtifactsToCache
.stream()
.map(artifactID -> URI.create(artifactID.getResourceID()))
.collect(Collectors.toList()))))
.whenComplete((res, throwable) -> {
if (throwable != null) {
log.error("failed to cache artifact on {}", request.getTaskExecutorID(), throwable);
}
else {
log.debug("Acked from cacheJobArtifacts for {}", request.getTaskExecutorID());
}
});
} catch (Exception ex) {
log.warn("Failed to cache job artifacts in task executor {}", request.getTaskExecutorID(), ex);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,13 +204,13 @@ public CompletableFuture<TaskExecutorGateway> getTaskExecutorGateway(
}

@Override
public CompletableFuture<TaskExecutorGateway> reconnectTaskExecutorGateway(
public CompletableFuture<Ack> reconnectTaskExecutorGateway(
TaskExecutorID taskExecutorID) {
return
Patterns
.ask(resourceClusterManagerActor, new TaskExecutorGatewayReconnectRequest(taskExecutorID, clusterID),
askTimeout)
.thenApply(TaskExecutorGateway.class::cast)
.thenApply(Ack.class::cast)
.toCompletableFuture();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,13 @@ protected TaskExecutorGateway getGateway() throws ExecutionException, Interrupte
return this.gateway.get();
}

protected CompletableFuture<TaskExecutorGateway> getGatewayAsync() {
Copy link
Collaborator

Choose a reason for hiding this comment

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

How can this be async?

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: no need to explicitly call this method async since the type already reflects that.

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, get rid of the blocking method.

if (this.gateway == null) {
throw new IllegalStateException("gateway is null");
}
return this.gateway;
}

protected CompletableFuture<TaskExecutorGateway> reconnect() {
this.gateway = rpcService.connect(registration.getTaskExecutorAddress(), TaskExecutorGateway.class)
.whenComplete((gateway, throwable) -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,7 +222,15 @@ private void onFailedToSubmitScheduleRequestEvent(FailedToSubmitScheduleRequestE
Throwables.getStackTraceAsString(event.throwable)));

try {
resourceCluster.reconnectTaskExecutorGateway(event.getTaskExecutorID()).join();
resourceCluster.reconnectTaskExecutorGateway(event.getTaskExecutorID())
.whenComplete((res, throwable) -> {
if (throwable != null) {
log.error("Failed to request reconnect to gateway for {}", event.getTaskExecutorID(), throwable);
}
else {
log.debug("Acked from reconnection request for {}", event.getTaskExecutorID());
}
});
} catch (Exception e) {
log.warn(
"Failed to establish re-connection with the task executor {} on failed schedule request",
Expand Down
Loading