Skip to content

Commit

Permalink
Fixes error where the leadership loss prevents the JobResultStore che…
Browse files Browse the repository at this point in the history
…ck from being executed
  • Loading branch information
XComp committed Nov 26, 2024
1 parent 2e7bb4d commit 7d542ae
Showing 1 changed file with 10 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -263,12 +263,19 @@ private void startJobMasterServiceProcessAsync(UUID leaderSessionId) {
jobResultStore.hasJobResultEntryAsync(
getJobID()))
.exceptionally(
handleLeadershipError(
handleLeadershipErrorWithNull(
"verify job result entry",
leaderSessionId,
Boolean.class))
.thenCompose(
hasJobResult -> {
if (hasJobResult == null) {
// no JobResultEntry could be retrieved due
// to leadership loss - no job handling
// should be initiated
return FutureUtils.completedVoidFuture();
}

if (hasJobResult) {
return handleJobAlreadyDoneAsLeader(
leaderSessionId);
Expand Down Expand Up @@ -297,10 +304,10 @@ private CompletableFuture<Void> createNewJobMasterServiceProcessAsLeader(UUID le

private Function<Throwable, Void> handleLeadershipError(
String actionDescription, UUID leaderSessionId) {
return handleLeadershipError(actionDescription, leaderSessionId, Void.class);
return handleLeadershipErrorWithNull(actionDescription, leaderSessionId, Void.class);
}

private <T> Function<Throwable, T> handleLeadershipError(
private <T> Function<Throwable, T> handleLeadershipErrorWithNull(
String actionDescription, UUID leaderSessionId, Class<T> unusedReturnType) {
return error -> {
final Throwable strippedError = ExceptionUtils.stripCompletionException(error);
Expand Down

0 comments on commit 7d542ae

Please sign in to comment.