wangyang0918 commented on a change in pull request #11323: [FLINK-16439][k8s] 
Make KubernetesResourceManager starts workers using WorkerResourceSpec 
requested by SlotManager
URL: https://github.com/apache/flink/pull/11323#discussion_r410225561
 
 

 ##########
 File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java
 ##########
 @@ -237,57 +230,73 @@ private void recoverWorkerNodesFromPreviousAttempts() 
throws ResourceManagerExce
                        ++currentMaxAttemptId);
        }
 
-       private void requestKubernetesPod() {
-               numPendingPodRequests++;
+       private void requestKubernetesPod(WorkerResourceSpec 
workerResourceSpec) {
+               final KubernetesTaskManagerParameters parameters =
+                       
createKubernetesTaskManagerParameters(workerResourceSpec);
+
+               final KubernetesPod taskManagerPod =
+                       
KubernetesTaskManagerFactory.createTaskManagerComponent(parameters);
+               kubeClient.createTaskManagerPod(taskManagerPod);
+
+               podWorkerResources.put(parameters.getPodName(), 
workerResourceSpec);
+               final int pendingWorkerNum = 
notifyNewWorkerRequested(workerResourceSpec);
 
                log.info("Requesting new TaskManager pod with <{},{}>. Number 
pending requests {}.",
-                       defaultMemoryMB,
-                       defaultCpus,
-                       numPendingPodRequests);
+                       parameters.getTaskManagerMemoryMB(),
+                       parameters.getTaskManagerCPU(),
+                       pendingWorkerNum);
+               log.info("TaskManager {} will be started with {}.", 
parameters.getPodName(), workerResourceSpec);
+       }
+
+       private KubernetesTaskManagerParameters 
createKubernetesTaskManagerParameters(WorkerResourceSpec workerResourceSpec) {
+               final TaskExecutorProcessSpec taskExecutorProcessSpec =
+                       
TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, 
workerResourceSpec);
 
                final String podName = String.format(
                        TASK_MANAGER_POD_FORMAT,
                        clusterId,
                        currentMaxAttemptId,
                        ++currentMaxPodId);
 
+               final ContaineredTaskManagerParameters taskManagerParameters =
+                       ContaineredTaskManagerParameters.create(flinkConfig, 
taskExecutorProcessSpec);
+
                final String dynamicProperties =
                        
BootstrapTools.getDynamicPropertiesAsString(flinkClientConfig, flinkConfig);
 
-               final KubernetesTaskManagerParameters 
kubernetesTaskManagerParameters = new KubernetesTaskManagerParameters(
+               return new KubernetesTaskManagerParameters(
                        flinkConfig,
                        podName,
                        dynamicProperties,
                        taskManagerParameters);
-
-               final KubernetesPod taskManagerPod =
-                       
KubernetesTaskManagerFactory.createTaskManagerComponent(kubernetesTaskManagerParameters);
-
-               log.info("TaskManager {} will be started with {}.", podName, 
taskExecutorProcessSpec);
-               kubeClient.createTaskManagerPod(taskManagerPod);
        }
 
        /**
         * Request new pod if pending pods cannot satisfy pending slot requests.
         */
-       private void requestKubernetesPodIfRequired() {
-               final int requiredTaskManagers = 
getNumberRequiredTaskManagers();
+       private void requestKubernetesPodIfRequired(WorkerResourceSpec 
workerResourceSpec) {
+               final int pendingWorkerNum = 
getNumPendingWorkersFor(workerResourceSpec);
+               int requiredTaskManagers = 
getRequiredResources().get(workerResourceSpec);
 
-               while (requiredTaskManagers > numPendingPodRequests) {
-                       requestKubernetesPod();
+               while (requiredTaskManagers-- > pendingWorkerNum) {
+                       requestKubernetesPod(workerResourceSpec);
                }
        }
 
        private void removePodIfTerminated(KubernetesPod pod) {
                if (pod.isTerminated()) {
                        kubeClient.stopPod(pod.getName());
 
 Review comment:
   I am afraid that the situation is different in YARN. In 
`YarnResourceManager`, when the `AMRMClient` heartbeats failed, may because 
YARN RM internal error, or invalid resource request, or network problems, or 
sth else, we will call `onFatalError` to fail the current jobmanager attempt 
and start a new one. We do not retry in such case.
   
   Maybe we could have the same logics in `KubernetesResourceManager` if we 
confirm that the `Error WatchEvent` is sent only when K8s ApiServer internal 
error, or invalid pod spec. I would give more information in FLINK-17177 after 
dive into the K8s ApiServer implementation.
   
   Till is right, we need to add thorough test coverage for this effort and try 
to move the same logics to `ActiveResouceManager`.

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to