wangyang0918 commented on a change in pull request #18119:
URL: https://github.com/apache/flink/pull/18119#discussion_r789291361



##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -240,6 +249,31 @@ private void recoverWorkerNodesFromPreviousAttempts() 
throws ResourceManagerExce
         
getResourceEventHandler().onPreviousAttemptWorkersRecovered(recoveredWorkers);
     }
 
+    private void updateKubernetesServiceTargetPortIfNecessary() throws 
Exception {
+        if (!KubernetesUtils.isHostNetwork(flinkConfig)) {
+            return;
+        }
+        flinkKubeClient.updateServiceTargetPort(
+                KubernetesService.ServiceType.REST_SERVICE,
+                clusterId,
+                Constants.REST_PORT_NAME,
+                
ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl(webInterfaceUrl));

Review comment:
       We also need to call the `.get()` here.

##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java
##########
@@ -201,6 +203,20 @@ KubernetesLeaderElector createLeaderElector(
      */
     KubernetesPod loadPodFromTemplateFile(File podTemplateFile);
 
+    /**
+     * Update the target ports of the given Kubernetes service.
+     *
+     * @param serviceType The ServiceType which needs to be updated
+     * @param portName The port name which needs to be updated
+     * @param targetPort The updated target port
+     * @return

Review comment:
       Missing the return.

##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -240,6 +249,31 @@ private void recoverWorkerNodesFromPreviousAttempts() 
throws ResourceManagerExce
         
getResourceEventHandler().onPreviousAttemptWorkersRecovered(recoveredWorkers);
     }
 
+    private void updateKubernetesServiceTargetPortIfNecessary() throws 
Exception {
+        if (!KubernetesUtils.isHostNetwork(flinkConfig)) {
+            return;
+        }
+        flinkKubeClient.updateServiceTargetPort(

Review comment:
       It will be great if we could add a pre-check here for that the parsed 
the rest port should not be `-1`.

##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -240,6 +249,31 @@ private void recoverWorkerNodesFromPreviousAttempts() 
throws ResourceManagerExce
         
getResourceEventHandler().onPreviousAttemptWorkersRecovered(recoveredWorkers);
     }
 
+    private void updateKubernetesServiceTargetPortIfNecessary() throws 
Exception {
+        if (!KubernetesUtils.isHostNetwork(flinkConfig)) {
+            return;
+        }
+        flinkKubeClient.updateServiceTargetPort(
+                KubernetesService.ServiceType.REST_SERVICE,
+                clusterId,
+                Constants.REST_PORT_NAME,
+                
ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl(webInterfaceUrl));
+        if 
(!HighAvailabilityMode.isHighAvailabilityModeActivated(flinkConfig)) {
+            flinkKubeClient
+                    .updateServiceTargetPort(
+                            KubernetesService.ServiceType.INTERNAL_SERVICE,
+                            clusterId,
+                            Constants.BLOB_SERVER_PORT_NAME,
+                            
Integer.parseInt(flinkConfig.getString(BlobServerOptions.PORT)))
+                    .get();
+            flinkKubeClient.updateServiceTargetPort(

Review comment:
       We also need to call the `.get()` here.




-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to