Skip to content

Commit

Permalink
[FLINK-31974] Do not crash jobmanager on Kubernetes client errors
Browse files Browse the repository at this point in the history
  • Loading branch information
gyfora authored and reswqa committed Jun 1, 2023
1 parent 7457251 commit 3b9f7cf
Show file tree
Hide file tree
Showing 2 changed files with 35 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,8 @@
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.concurrent.FutureUtils;

import io.fabric8.kubernetes.client.KubernetesClientException;

import javax.annotation.Nullable;

import java.io.File;
Expand All @@ -64,6 +66,7 @@
import java.util.Set;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;

/** Implementation of {@link ResourceManagerDriver} for Kubernetes deployment. */
public class KubernetesResourceManagerDriver
Expand Down Expand Up @@ -219,6 +222,10 @@ public CompletableFuture<KubernetesWorkerNode> requestResource(
if (t == null) {
return null;
}
// Unwrap CompletionException cause if any
if (t instanceof CompletionException && t.getCause() != null) {
t = t.getCause();
}
if (t instanceof CancellationException) {

requestResourceFutures.remove(taskManagerPod.getName());
Expand All @@ -228,8 +235,9 @@ public CompletableFuture<KubernetesWorkerNode> requestResource(
podName);
stopPod(taskManagerPod.getName());
}
} else if (t instanceof RetryableException) {
// ignore
} else if (t instanceof RetryableException
|| t instanceof KubernetesClientException) {
// ignore transient / retriable errors
} else {
log.error("Error completing resource request.", t);
ExceptionUtils.rethrow(t);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.kubernetes;

import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.core.testutils.FlinkAssertions;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.configuration.KubernetesResourceManagerDriverConfiguration;
import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
Expand All @@ -37,13 +38,15 @@
import org.apache.flink.util.concurrent.FutureUtils;

import io.fabric8.kubernetes.api.model.ResourceRequirements;
import io.fabric8.kubernetes.client.KubernetesClientException;
import org.junit.jupiter.api.Test;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;

Expand Down Expand Up @@ -223,6 +226,28 @@ void testFatalHandleError() throws Exception {
};
}

@Test
void testKubernetesExceptionHandling() throws Exception {
new Context() {
{
runTest(
() ->
FlinkAssertions.assertThatFuture(
runInMainThread(
() -> {
getDriver()
.requestResource(
TASK_EXECUTOR_PROCESS_SPEC)
.completeExceptionally(
new CompletionException(
new KubernetesClientException(
"test")));
}))
.eventuallySucceeds());
}
};
}

@Test
void testRecoverPreviousAttemptWorkersPodTerminated() throws Exception {
new Context() {
Expand Down

0 comments on commit 3b9f7cf

Please sign in to comment.