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
[FLINK-10935][kubernetes]Implement KubeClient with Faric8 Kubernetes clients #9965
Conversation
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit 685652d (Wed Dec 04 16:43:19 UTC 2019) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
Please see the Pull Request Review Guide for a full explanation of the review process. The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commandsThe @flinkbot bot supports the following commands:
|
CI report:
Bot commandsThe @flinkbot bot supports the following commands:
|
6f90b45
to
86aa5ce
Compare
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/fabric8/FlinkConfigMap.java
Outdated
Show resolved
Hide resolved
...ernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
Outdated
Show resolved
Hide resolved
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
Outdated
Show resolved
Hide resolved
...n/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkMasterDeploymentDecorator.java
Outdated
Show resolved
Hide resolved
...src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/TaskManagerPodDecorator.java
Outdated
Show resolved
Hide resolved
...src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/TaskManagerPodDecorator.java
Outdated
Show resolved
Hide resolved
86aa5ce
to
7851d84
Compare
I have addressed @MalcolmSanders 's comments and updated the PR. |
7851d84
to
d49bb0a
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for open this PR, @wangyang0918 !
Good to see the K8S support in flink. Some comment left.
BTW, I think we could find a better way to deliver relevant configuration. Now, too many classes hold the flinkConfig
.
...ernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
Outdated
Show resolved
Hide resolved
...kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/TaskManagerPodParameter.java
Outdated
Show resolved
Hide resolved
...kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/ActionWatcher.java
Show resolved
Hide resolved
...kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/ActionWatcher.java
Show resolved
Hide resolved
...bernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/FlinkDeployment.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/Resource.java
Outdated
Show resolved
Hide resolved
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
Outdated
Show resolved
Hide resolved
d88f5bb
to
0e88e0e
Compare
9afcc7b
to
b3d9d02
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your working @wangyang0918 ! I left some comments.
/** | ||
* Create kubernetes config map, include flink-conf.yaml, log4j.properties. | ||
*/ | ||
void createConfigMap() throws Exception; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I notice that some of "create" functions are blocking while others are non-blocking. What is the different between them and how is the decision made?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
All the create operation is asynchronous, including createConfigMap createInternalService createFlinkMasterDeployment createTaskManagerPod
. We just add a CompletableFuture
for creating service because we need to make sure the service are created successfully. Then we could get the uuid of internal service and use the rest service to submit job.
*/ | ||
public class ActionWatcher<T extends HasMetadata> implements Watcher<T> { | ||
private final CountDownLatch latch = new CountDownLatch(1); | ||
private final AtomicReference<T> reference = new AtomicReference(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't see any atomic actions like compare-and-set or get-and-set are in used. Why do you instance an AtomicReference
here? Is volatile
sufficient?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I use AtomicReference
just to make sure the atomic operation of set
and get
from multiple threads. Also volatile
is enough here for object reference. AtomicReference
is more clear, if you insist, i will use volatile instead.
private final CountDownLatch latch = new CountDownLatch(1); | ||
private final AtomicReference<T> reference = new AtomicReference(); | ||
private final T resource; | ||
private Action expectedAction; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It could be final
IMO.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will fix it.
*/ | ||
public abstract class Resource<T> { | ||
|
||
protected T internalResource; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
given we have getter for internalResource
and flinkConfig
they could be private final
; just notice their setters are not into used at all.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is code
subclass extends Resource {
super(flinkConfig);
this.internalResource = internalResource;
}
which could be
super(flinkConfig, internalResource);
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice suggestion. I will update the Resource
and implementation class.
The setInternalResource
will be kept. It will be used by Decorator
. Extract real resource from resource, decorate and put it back.
|
||
private final KubernetesClient internalClient; | ||
|
||
private List<Decorator<ConfigMap, FlinkConfigMap>> configMapDecorators; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'd tend to mark these lists as final
when possible
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree. Final
will be better.
/** | ||
* Get the rest endpoints for access outside cluster. | ||
*/ | ||
Endpoint getRestEndpoints(String clusterId); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I notice that it could be Nullable
in Faric8 implementation. The fact that it is nullable and the meaning of null should be documented.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I will add return description for getRestEndpoints
and other methods may return null.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe getRestEndpoint
since only one Endpoint would be returned
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Make sense. I will fix it.
/** | ||
* Get the kubernetes internal service of the given flink clusterId. | ||
*/ | ||
FlinkService getInternalService(String clusterId); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I notice that it could be Nullable
in Faric8 implementation. The fact that it is nullable and the meaning of null should be documented.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will fix it.
try { | ||
config = Config.fromKubeconfig(KubernetesUtils.getContentFromFile(kubeConfigFile)); | ||
} catch (IOException e) { | ||
LOG.error("Load kubernetes config failed.", e); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We don't throw exception here so new DefaultKubernetesClient(config);
could be new DefaultKubernetesClient(null);
. What is the semantic there then? It looks weird we log error but control flow continues.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
An exception should be thrown if we get the content from kube config file error. I will fix it.
also you could rebase the branch on current master especially on merged FLINK-10932 |
06e6b2b
to
816a11a
Compare
@tisonkun Really thanks for your detailed review. I have rebase the master and addressed all your comments. Please take a look. |
816a11a
to
e806910
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for open this PR @wangyang0918 .
I have a few comments for the configs and will later go through the remaining parts.
...ernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
Show resolved
Hide resolved
...ernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
Outdated
Show resolved
Hide resolved
...ernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
Outdated
Show resolved
Hide resolved
...ernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
Show resolved
Hide resolved
...ernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/Resource.java
Outdated
Show resolved
Hide resolved
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/FlinkService.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/FlinkPod.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/FlinkPod.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
Outdated
Show resolved
Hide resolved
...kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/LabelBuilder.java
Outdated
Show resolved
Hide resolved
...kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/LabelBuilder.java
Outdated
Show resolved
Hide resolved
...kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/LabelBuilder.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/Decorator.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/Decorator.java
Outdated
Show resolved
Hide resolved
...etes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ConfigMapDecorator.java
Outdated
Show resolved
Hide resolved
...etes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ConfigMapDecorator.java
Outdated
Show resolved
Hide resolved
...n/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkMasterDeploymentDecorator.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java
Outdated
Show resolved
Hide resolved
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
Show resolved
Hide resolved
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
Show resolved
Hide resolved
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
Show resolved
Hide resolved
/** | ||
* Get the rest endpoints for access outside cluster. | ||
*/ | ||
Endpoint getRestEndpoints(String clusterId); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe getRestEndpoint
since only one Endpoint would be returned
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Endpoint.java
Show resolved
Hide resolved
...-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java
Outdated
Show resolved
Hide resolved
@zhuzhurk Thanks a lot for your detailed review. I will address your comments one by one. |
e806910
to
e85e76c
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for addressing all the comments @wangyang0918 .
The change looks good to me except for some more minor comments.
flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java
Outdated
Show resolved
Hide resolved
flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java
Outdated
Show resolved
Hide resolved
e85e76c
to
d1c41ae
Compare
@zhuzhurk I have addressed all your comments. Please take a look. |
LGTM. +1 to merge it. |
@tisonkun Please have a final check and help to merge. Really thanks for your kind help. -:) |
d1c41ae
to
1e54eb3
Compare
"to the number of slots per TaskManager"); | ||
|
||
public static final ConfigOption<String> CONTAINER_IMAGE_PULL_POLICY = | ||
key("kubernetes.container.image.pullPolicy") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
for consistency, "pull-policy"?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I will fix it.
1e54eb3
to
685652d
Compare
…rnetes. All the basic and necessary config options has been added. Such as, image name, service account, taskmanager resources, etc.
…s resources. The basic resources for active Kubernetes integration include ConfigMap, Deployment, Service, Pod.
…ecorators. The Fabric8FlinkKubeClient will be used to interact with Kubernetes API server to create/delete resources and get the status. The decorators will be used to build the complete Kubernetes resource description.
685652d
to
1b37e8e
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your contribution @wangyang0918 . Merging now...I'll fix two typo as listed below. Be care to modify your following PRs.
.desc(KubernetesConfigOptions.CONTAINER_IMAGE.description().toString()) | ||
.build(); | ||
|
||
public static final Option JOB_MANAGER_MEMEORY_OPTION = Option.builder("jm") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
public static final Option JOB_MANAGER_MEMEORY_OPTION = Option.builder("jm") | |
public static final Option JOB_MANAGER_MEMORY_OPTION = Option.builder("jm") |
typo. I will fix on merging.
.desc("Memory for JobManager Container with optional unit (default: MB)") | ||
.build(); | ||
|
||
public static final Option TASK_MANAGER_MEMEORY_OPTION = Option.builder("tm") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
public static final Option TASK_MANAGER_MEMEORY_OPTION = Option.builder("tm") | |
public static final Option TASK_MANAGER_MEMORY_OPTION = Option.builder("tm") |
typo. I will fix on merging.
} | ||
|
||
@Override | ||
public void onClose(KubernetesClientException e) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why is it ok to simply ignore the onClose
event? Can't it happen that there is a problem (e.g. e != null
)? Shouldn't we then signal this to potential waiters who have called await
@wangyang0918?
What is the purpose of the change
This PR is part of FLINK-9953(Active Kubernetes integration). This PR introduces the
FlinkKubeClient
API and adds fabric8 implementation. The following k8s resource decorators are added to support for basic k8s integration.KubernetesResourceManager
.This PR is based on #9695.
Brief change log
Verifying this change
This change added
Fabric8ClientTest
public api implementation ofFlinkKubeClient
.Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: (no)Documentation