tillrohrmann commented on a change in pull request #18083:
URL: https://github.com/apache/flink/pull/18083#discussion_r786738133



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointUtils.java
##########
@@ -128,4 +132,80 @@ public static void 
configureUncaughtExceptionHandler(Configuration config) {
                 new ClusterUncaughtExceptionHandler(
                         
config.get(ClusterOptions.UNCAUGHT_EXCEPTION_HANDLING)));
     }
+
+    /**
+     * Creates the working directory for the TaskManager process. This method 
ensures that the
+     * working directory exists.
+     *
+     * @param configuration to extract the required settings from
+     * @param resourceId identifying the TaskManager process
+     * @return working directory
+     * @throws IOException if the working directory could not be created
+     */
+    public static WorkingDirectory createTaskManagerWorkingDirectory(
+            Configuration configuration, ResourceID resourceId) throws 
IOException {
+        return WorkingDirectory.create(
+                generateTaskManagerWorkingDirectoryFile(configuration, 
resourceId));
+    }
+
+    /**
+     * Generates the working directory {@link File} for the TaskManager 
process. This method does
+     * not ensure that the working directory exists.
+     *
+     * @param configuration to extract the required settings from
+     * @param resourceId identifying the TaskManager process
+     * @return working directory file
+     */
+    @VisibleForTesting
+    public static File generateTaskManagerWorkingDirectoryFile(
+            Configuration configuration, ResourceID resourceId) {
+        return generateWorkingDirectoryFile(
+                configuration,
+                ClusterOptions.TASK_MANAGER_PROCESS_WORKING_DIR_BASE,
+                "tm_" + resourceId);
+    }
+
+    /**
+     * Generates the working directory {@link File} for the JobManager 
process. This method does not
+     * ensure that the working directory exists.
+     *
+     * @param configuration to extract the required settings from
+     * @param resourceId identifying the JobManager process
+     * @return working directory file
+     */
+    @VisibleForTesting
+    public static File generateJobManagerWorkingDirectoryFile(
+            Configuration configuration, ResourceID resourceId) {
+        return generateWorkingDirectoryFile(
+                configuration,
+                ClusterOptions.JOB_MANAGER_PROCESS_WORKING_DIR_BASE,
+                "jm_" + resourceId);
+    }
+
+    private static File generateWorkingDirectoryFile(
+            Configuration configuration,
+            ConfigOption<String> precedingOption,
+            String workingDirectoryName) {
+        return new File(
+                configuration
+                        .getOptional(precedingOption)
+                        .orElseGet(
+                                () -> 
configuration.get(ClusterOptions.PROCESS_WORKING_DIR_BASE)),

Review comment:
       Actually, I am not sure whether we truly support multiple temp 
directories well enough at this point in time. As a user I would expect that 
Flink could automatically make use of free directories once others a full. 
However, this is currently not the case. Flink will simply pick a temp dir for 
the RocksDB working directory and hope that everything fits in this directory.




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to