Sxnan commented on code in PR #20147:
URL: https://github.com/apache/flink/pull/20147#discussion_r926192780


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java:
##########
@@ -2203,13 +2199,21 @@ public StreamGraph getStreamGraph() {
      */
     @Internal
     public StreamGraph getStreamGraph(boolean clearTransformations) {
+        updateCacheTransformation();
         final StreamGraph streamGraph = 
getStreamGraphGenerator(transformations).generate();
         if (clearTransformations) {
             transformations.clear();
         }
         return streamGraph;
     }
 
+    private void updateCacheTransformation() {
+        Set<AbstractID> cachedIntermediateDatasetIds = 
listCachedIntermediateDatasetIds();

Review Comment:
   I think it is RM is the source of truth of the status of the cached result. 
If the client maintains the set of the cached datasets by itself without any 
syncing method with the RM, we can only know if the cached dataset is valid or 
not after submitting the job.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java:
##########
@@ -2203,13 +2199,21 @@ public StreamGraph getStreamGraph() {
      */
     @Internal
     public StreamGraph getStreamGraph(boolean clearTransformations) {
+        updateCacheTransformation();

Review Comment:
   I think querying the cached intermediate dataset right before generating the 
StreamGraph can greatly reduce the opportunity of trying to access an invalid 
cached intermediate dataset.
   If we were to update the cache flag right after the last job is finished, we 
don't know when the next job that consumes the cache will be submitted, and 
during this unknown time gap, some cached intermediate datasets can be 
invalidated.



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