zentol commented on code in PR #22169:
URL: https://github.com/apache/flink/pull/22169#discussion_r1146055225


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/WaitingForResources.java:
##########
@@ -144,12 +135,17 @@ public Logger getLogger() {
     }
 
     @Override
-    public void notifyNewResourcesAvailable() {
+    public void onNewResourcesAvailable() {
+        checkDesiredOrSufficientResourcesAvailable();
+    }
+
+    @Override
+    public void onNewResourceRequirements() {
         checkDesiredOrSufficientResourcesAvailable();
     }
 
     private void checkDesiredOrSufficientResourcesAvailable() {
-        if (context.hasDesiredResources(desiredResources)) {
+        if (context.hasDesiredResources()) {

Review Comment:
   yes we could refactor this. It probably won't result in wrong behavior, but 
still.
   
   Instead of having the deadline we could instead schedule a single call to 
`checkDesiredOrSufficientResourcesAvailable` using the timeout as the delay.
   
   Like this:
   ```
          if (context.hasSufficientResources()) {
               if (resourceStabilizationDeadline == null) {
                   resourceStabilizationDeadline =
                           
Deadline.fromNowWithClock(resourceStabilizationTimeout, clock);
                   context.runIfState(
                           this,
                           this::checkDesiredOrSufficientResourcesAvailable,
                           resourceStabilizationDeadline.timeLeft());
               }
               if (resourceStabilizationDeadline.isOverdue()) {
                   createExecutionGraphWithAvailableResources();
               }
           } else {
               // clear deadline due to insufficient resources
               resourceStabilizationDeadline = null;
           }
   ```
   
   Anyway, @rkhachatryan you want to file a ticket for this?



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