wangyang0918 commented on a change in pull request #11427: [FLINK-15790][k8s]
Make FlinkKubeClient and its implementations asynchronous
URL: https://github.com/apache/flink/pull/11427#discussion_r406291138
##########
File path:
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
##########
@@ -61,158 +61,180 @@
private final String clusterId;
private final String nameSpace;
- public Fabric8FlinkKubeClient(Configuration flinkConfig,
KubernetesClient client) {
+ private final ExecutorWrapper executorWrapper;
+
+ public Fabric8FlinkKubeClient(Configuration flinkConfig,
KubernetesClient client, ExecutorWrapper executorWrapper) {
this.flinkConfig = checkNotNull(flinkConfig);
this.internalClient = checkNotNull(client);
this.clusterId =
checkNotNull(flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID));
this.nameSpace =
flinkConfig.getString(KubernetesConfigOptions.NAMESPACE);
+
+ this.executorWrapper = executorWrapper;
}
@Override
- public void createJobManagerComponent(KubernetesJobManagerSpecification
kubernetesJMSpec) {
+ public CompletableFuture<Void>
createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
final Deployment deployment = kubernetesJMSpec.getDeployment();
final List<HasMetadata> accompanyingResources =
kubernetesJMSpec.getAccompanyingResources();
// create Deployment
LOG.debug("Start to create deployment with spec {}",
deployment.getSpec().toString());
- final Deployment createdDeployment = this.internalClient
- .apps()
- .deployments()
- .inNamespace(this.nameSpace)
- .create(deployment);
-
- // Note that we should use the uid of the created Deployment
for the OwnerReference.
- setOwnerReference(createdDeployment, accompanyingResources);
- this.internalClient
- .resourceList(accompanyingResources)
- .inNamespace(this.nameSpace)
- .createOrReplace();
+ return CompletableFuture.runAsync(() -> {
+ final Deployment createdDeployment = this.internalClient
+ .apps()
+ .deployments()
+ .inNamespace(this.nameSpace)
+ .create(deployment);
+
+ // Note that we should use the uid of the created
Deployment for the OwnerReference.
+ setOwnerReference(createdDeployment,
accompanyingResources);
+
+ this.internalClient
+ .resourceList(accompanyingResources)
+ .inNamespace(this.nameSpace)
+ .createOrReplace();
+ }, executorWrapper.getExecutor());
}
@Override
public void createTaskManagerPod(KubernetesPod kubernetesPod) {
- final Deployment masterDeployment = this.internalClient
- .apps()
- .deployments()
- .inNamespace(this.nameSpace)
- .withName(KubernetesUtils.getDeploymentName(clusterId))
- .get();
-
- if (masterDeployment == null) {
- throw new RuntimeException(
- "Failed to find Deployment named " + clusterId
+ " in namespace " + this.nameSpace);
- }
+ CompletableFuture.runAsync(() -> {
Review comment:
After checking the `fabric8`, `okhttp` client and some information about K8s
apiserver, i believe that the internal `requestTimeout` and `connectionTimeout`
could completely solve the "retry too often" problems. Since the connection may
be normal and the apiserver return with some error code(`403, 404, 500`, etc.)
immediately. And the retry request will flood to the apiserver.
So i think we still need to add a retry internal in
`KubernetesResourceManager`. I have leave the default value to 3s. Also i add a
test `testCreateTaskManagerPodFailedAndRetry` for creating taskmanager pod
failed and retry.
@azagrebin Please have a look at your convenience.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services