wanglijie95 commented on code in PR #20080:
URL: https://github.com/apache/flink/pull/20080#discussion_r907258779


##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java:
##########
@@ -270,16 +269,9 @@ public TaskManagerLocation 
getCurrentAssignedResourceLocation() {
         return currentExecution.getAssignedResourceLocation();
     }
 
-    @Nullable
     @Override
-    public ArchivedExecution getPriorExecutionAttempt(int attemptNumber) {
-        synchronized (priorExecutions) {
-            if (attemptNumber >= 0 && attemptNumber < priorExecutions.size()) {
-                return priorExecutions.get(attemptNumber);
-            } else {
-                throw new IllegalArgumentException("attempt does not exist");
-            }
-        }
+    public ExecutionHistory getExecutionHistory() {
+        return executionHistory;

Review Comment:
   I'm not quite sure what the `synchronized` is for,  is it safe to delete it?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionVertex.java:
##########
@@ -100,13 +99,12 @@ public TaskManagerLocation 
getCurrentAssignedResourceLocation() {
         return currentExecution.getAssignedResourceLocation();
     }
 
-    @Nullable
     @Override
-    public ArchivedExecution getPriorExecutionAttempt(int attemptNumber) {
-        if (attemptNumber >= 0 && attemptNumber < priorExecutions.size()) {
-            return priorExecutions.get(attemptNumber);
-        } else {
-            throw new IllegalArgumentException("attempt does not exist");
-        }
+    public ExecutionHistory getExecutionHistory() {
+        return executionHistory;
+    }
+
+    static ExecutionHistory getCopyOfExecutionHistory(ExecutionVertex 
executionVertex) {

Review Comment:
   can be private



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