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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputOutputFormatVertex.java
##########
@@ -24,6 +24,7 @@
 import org.apache.flink.api.common.io.OutputFormat;
 import org.apache.flink.api.common.operators.util.UserCodeWrapper;
 import org.apache.flink.runtime.OperatorIDPair;
+import org.apache.flink.runtime.entrypoint.ClusterEntryPointExceptionUtils;

Review comment:
       It feels wrong that a `Vertex` knows anything from the 
`o.a.f.runtime.entrypoint` package.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntryPointExceptionUtils.java
##########
@@ -61,7 +61,7 @@ private ClusterEntryPointExceptionUtils() {
         * {@code null} if the argument was {@code null}
         */
        @Nullable
-       static Throwable tryEnrichClusterEntryPointError(@Nullable Throwable 
exception) {
+       public static Throwable tryEnrichClusterEntryPointError(@Nullable 
Throwable exception) {

Review comment:
       I think that this method won't add a proper error message in case of an 
out of heap space error.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntryPointExceptionUtils.java
##########
@@ -25,9 +25,9 @@
 import static org.apache.flink.util.ExceptionUtils.tryEnrichOutOfMemoryError;
 
 /**
- * Exception utils to handle and enrich exceptions occurring in TaskManager.
+ * Exception utils to handle and enrich exceptions occurring in the JobManager.

Review comment:
       ```suggestion
    * Exception utils to handle and enrich exceptions occurring in the 
ClusterEntrypoint.
   ```

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputOutputFormatVertex.java
##########
@@ -99,15 +100,16 @@ public void initializeOnMaster(ClassLoader loader) throws 
Exception {
                                        ((InitializeOnMaster) 
outputFormat).initializeGlobal(getParallelism());
                                }
                        }
-
+               } catch (Throwable t) {
+                       throw 
ClusterEntryPointExceptionUtils.tryEnrichClusterEntryPointError(t);

Review comment:
       I think catching the exception here and enriching it is too specific. 
That way we will leave all other places where this can occur out. I believe 
that a better approach is to do it where most of the code paths originate from. 
One idea could be the generation of the `JobManagerRunnerImpl` in the 
`Dispatcher` and its `resultFuture`. This is the point which will be passed by 
job submissions as well as job recoveries.




----------------------------------------------------------------
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:
[email protected]


Reply via email to