zhuzhurk commented on code in PR #27755:
URL: https://github.com/apache/flink/pull/27755#discussion_r2980340509


##########
flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java:
##########
@@ -153,7 +153,7 @@ public JobGraph(String jobName) {
     public JobGraph(@Nullable JobID jobId, @Nullable ApplicationID 
applicationId, String jobName) {
         this.jobID = jobId == null ? new JobID() : jobId;
         this.jobName = jobName == null ? "(unnamed job)" : jobName;
-        this.applicationId = applicationId;
+        this.applicationId = applicationId == null ? new ApplicationID() : 
applicationId;

Review Comment:
   What is this change for? Seems to may introduce quite some behavior changes.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java:
##########
@@ -96,6 +97,7 @@ public void close() {
     public CompletableFuture<Void> allocateSlot(
             InstanceID instanceId,
             JobID jobId,
+            ApplicationID applicationId,

Review Comment:
   `checkNotNull` is needed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java:
##########
@@ -505,6 +573,56 @@ public void run() {
                         }
                     }
                 }
+
+                // also clean up application blobs
+                runApplicationsCleanup();
+            }
+        }
+    }
+
+    private void runApplicationsCleanup() {

Review Comment:
   Is it possible to extract a common method to clean up expired job blobs and 
application blobs?



##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java:
##########
@@ -1302,6 +1308,11 @@ public Optional<ApplicationID> getApplicationId() {
         return Optional.ofNullable(applicationId);
     }
 
+    public void addUserJarToSkip(String userJarNames) {
+        LOG.info("Add user jar to skip uploading: {}", userJarNames);
+        this.userJarsToSkip.add(userJarNames);

Review Comment:
   Maybe also remove the matched user jars that are already added? It should 
not assume the order of calling these two methods.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java:
##########
@@ -185,17 +189,40 @@ private void registerJobWithExpiry(JobID jobId, long 
expiryTimeout) {
         }
     }
 
+    private void registerDetectedApplications() throws IOException {
+        if (storageDir.deref().exists()) {
+            final Collection<ApplicationID> applicationIds =
+                    
BlobUtils.listExistingApplications(storageDir.deref().toPath());
+
+            final long expiryTimeout = System.currentTimeMillis() + 
cleanupInterval;
+            for (ApplicationID applicationId : applicationIds) {
+                registerApplicationWithExpiry(applicationId, expiryTimeout);
+            }
+        }
+    }
+
+    private void registerApplicationWithExpiry(ApplicationID applicationId, 
long expiryTimeout) {
+        checkNotNull(applicationId);
+        synchronized (jobRefCounters) {

Review Comment:
   Why is it using `jobRefCounters` to guard `applicationRefCounters`?



##########
flink-clients/src/main/java/org/apache/flink/client/program/JarInfo.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.client.program;
+
+import org.apache.flink.runtime.blob.PermanentBlobKey;
+
+import java.io.Serializable;
+
+/** Info about a jar. */
+public class JarInfo implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    public final String jarName;
+    public final PermanentBlobKey jarBlobKey;
+
+    public JarInfo(String jarName, PermanentBlobKey jarBlobKey) {
+        this.jarName = jarName;

Review Comment:
   checkNotNull



##########
flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java:
##########
@@ -265,7 +270,19 @@ private URLClassLoader createUserCodeClassLoader(
                 int count = 0;
                 // add URLs to locally cached JAR files
                 for (PermanentBlobKey key : requiredJarFiles) {
-                    libraryURLs[count] = blobService.getFile(jobId, 
key).toURI().toURL();
+                    try {
+                        // first try job specific libraries
+                        libraryURLs[count] = blobService.getFile(jobId, 
key).toURI().toURL();
+                    } catch (Exception e) {
+                        // then try application specific libraries
+                        LOG.info(
+                                "Cannot get job specific blob {} for job {}. 
Checking application specific blob with application id {}.",

Review Comment:
   The log looks like reporting a problem, which is a bit confusing as this is 
an intended optimization.



##########
flink-clients/src/main/java/org/apache/flink/client/program/JarInfo.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.client.program;
+
+import org.apache.flink.runtime.blob.PermanentBlobKey;
+
+import java.io.Serializable;
+
+/** Info about a jar. */
+public class JarInfo implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    public final String jarName;

Review Comment:
   -> private



##########
flink-runtime/src/main/java/org/apache/flink/runtime/blob/PermanentBlobCache.java:
##########
@@ -208,33 +235,59 @@ public void registerJob(JobID jobId) {
                 ref.keepUntil = -1;
             }
             ++ref.references;
+
+            // also register under application because the job may use 
application-level blobs
+            RefCount applicationRef = 
applicationRefCounters.get(applicationId);
+            if (applicationRef == null) {
+                applicationRef = new RefCount();
+                applicationRefCounters.put(applicationId, applicationRef);
+            } else {
+                // reset cleanup timeout
+                applicationRef.keepUntil = -1;
+            }
+            ++applicationRef.references;
         }
     }
 
     /**
      * Unregisters use of job-related BLOBs and allow them to be released.
      *
      * @param jobId ID of the job this blob belongs to
-     * @see #registerJob(JobID)
+     * @see #registerJob(JobID, ApplicationID)
      */
     @Override
-    public void releaseJob(JobID jobId) {
+    public void releaseJob(JobID jobId, ApplicationID applicationId) {
         checkNotNull(jobId);
 
         synchronized (jobRefCounters) {
+            String warning =
+                    "improper use of releaseJob() without a matching number of 
registerJob() calls for jobId "
+                            + jobId;
+
             RefCount ref = jobRefCounters.get(jobId);
 
             if (ref == null || ref.references == 0) {
-                log.warn(
-                        "improper use of releaseJob() without a matching 
number of registerJob() calls for jobId "
-                                + jobId);
+                log.warn(warning);
                 return;
             }
 
             --ref.references;
             if (ref.references == 0) {
                 ref.keepUntil = System.currentTimeMillis() + cleanupInterval;
             }
+
+            // make sure application related data can be cleaned up
+            RefCount applicationRef = 
applicationRefCounters.get(applicationId);
+
+            if (applicationRef == null || applicationRef.references == 0) {
+                log.warn(warning);

Review Comment:
   It's better to make the log different for trouble shooting.



##########
flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java:
##########
@@ -1302,6 +1308,11 @@ public Optional<ApplicationID> getApplicationId() {
         return Optional.ofNullable(applicationId);
     }
 
+    public void addUserJarToSkip(String userJarNames) {
+        LOG.info("Add user jar to skip uploading: {}", userJarNames);

Review Comment:
   userJarNames -> userJarName



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