1996fanrui commented on code in PR #22985:
URL: https://github.com/apache/flink/pull/22985#discussion_r1360222871


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingControllerTest.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.scheduler.adaptive.scalingpolicy;
+
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.adaptive.allocator.VertexParallelism;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.Collections;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link RescalingController}. */
+@ExtendWith(TestLoggerExtension.class)
+public class EnforceParallelismChangeRescalingControllerTest {

Review Comment:
   ```suggestion
   class EnforceParallelismChangeRescalingControllerTest {
   ```
   
   The `public` is not needed for junit5.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingControllerTest.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.scheduler.adaptive.scalingpolicy;
+
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.adaptive.allocator.VertexParallelism;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.Collections;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for the {@link RescalingController}. */
+@ExtendWith(TestLoggerExtension.class)

Review Comment:
   ```suggestion
   ```
   
   
   
   It's not needed, the `TestLoggerExtension` has been added for all test 
classes by default.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java:
##########
@@ -124,23 +158,70 @@ private void handleDeploymentFailure(ExecutionVertex 
executionVertex, JobExcepti
 
     @Override
     public void onNewResourcesAvailable() {
-        maybeRescale();
+        rescaleWhenCooldownPeriodIsOver();
     }
 
     @Override
     public void onNewResourceRequirements() {
-        maybeRescale();
+        rescaleWhenCooldownPeriodIsOver();
+    }
+
+    /** Force rescaling as long as the target parallelism is different from 
the current one. */
+    private void forceRescale() {
+        if (context.shouldRescale(getExecutionGraph(), true)) {
+            getLogger()
+                    .info(
+                            "Added resources are still there after {} 
time({}), force a rescale.",
+                            
JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(),
+                            scalingIntervalMax);
+            context.goToRestarting(
+                    getExecutionGraph(),
+                    getExecutionGraphHandler(),
+                    getOperatorCoordinatorHandler(),
+                    Duration.ofMillis(0L),
+                    getFailures());
+        }
     }
 
+    /**
+     * Rescale the job if {@link Context#shouldRescale} is true. Otherwise, 
force a rescale using
+     * {@link Executing#forceRescale()} after {@link
+     * JobManagerOptions#SCHEDULER_SCALING_INTERVAL_MAX}.
+     */
     private void maybeRescale() {
-        if (context.shouldRescale(getExecutionGraph())) {
-            getLogger().info("Can change the parallelism of job. Restarting 
job.");
+        rescaleScheduled = false;
+        if (context.shouldRescale(getExecutionGraph(), false)) {
+            getLogger().info("Can change the parallelism of the job. 
Restarting the job.");
             context.goToRestarting(
                     getExecutionGraph(),
                     getExecutionGraphHandler(),
                     getOperatorCoordinatorHandler(),
                     Duration.ofMillis(0L),
                     getFailures());
+        } else if (scalingIntervalMax != null) {
+            getLogger()
+                    .info(
+                            "The longer the pipeline runs, the more the 
(small) resource gain is worth the restarting time. "
+                                    + "Last resource added does not meet {}, 
force a rescale after {} time({}) if the resource is still there.",
+                            JobManagerOptions.MIN_PARALLELISM_INCREASE,
+                            
JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(),
+                            scalingIntervalMax);
+            // schedule a force rescale in 
JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX time
+            context.runIfState(this, this::forceRescale, scalingIntervalMax);
+        }

Review Comment:
   I still have a question for `scalingIntervalMax`.
   
   > if minimum scaling requirements are not met but last rescale was done more 
than scaling-interval.max ago, a rescale is forced.
   
   I copy it from FLIP, from it, we can see if last rescale was done more than 
scaling-interval.max ago, we should call forceRescale, right?
   
   For current code, we didn't have this logic. We always schedule it after 
scalingIntervalMax. If the `timeSinceLastRescale()` > `scalingIntervalMax`, the 
delay isn't necessary, right?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java:
##########
@@ -594,8 +683,12 @@ public FailureResult howToHandleFailure(Throwable failure) 
{
         }
 
         @Override
-        public boolean shouldRescale(ExecutionGraph executionGraph) {
-            return canScaleUp.get();
+        public boolean shouldRescale(ExecutionGraph executionGraph, boolean 
forceRescale) {
+            if (forceRescale) {
+                return canScaleUpWithForce;
+            } else {
+                return canScaleUpWithoutForce;

Review Comment:
   From `@Nullable private Boolean canScaleUpWithForce = false;`, we can see 
the `canScaleUpWithForce` can be null.
   
   It will be `NullPointException` once it's null here, because the return 
value is `boolean` instead of `Boolean`.
   
   I prefer remove the `Boolean`, and still mark the default value is false. 
WDYT?
   
   



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java:
##########
@@ -67,13 +77,36 @@ class Executing extends StateWithExecutionGraph implements 
ResourceListener {
         this.context = context;
         Preconditions.checkState(
                 executionGraph.getState() == JobStatus.RUNNING, "Assuming 
running execution graph");
+        this.scalingIntervalMin = scalingIntervalMin;
+        this.scalingIntervalMax = scalingIntervalMax;
+        this.lastRescale =
+                Instant.now(); // Executing is recreated with each restart 
(when we rescale)
+        // we consider the first execution of the pipeline as a rescale event
+        Preconditions.checkState(
+                !scalingIntervalMin.isNegative(),
+                "%s must be positive integer or 0",
+                JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key());
+        if (scalingIntervalMax != null) {
+            Preconditions.checkState(
+                    scalingIntervalMax.compareTo(scalingIntervalMin) > 0,
+                    "%s(%d) must be greater than %s(%d)",
+                    JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(),
+                    scalingIntervalMax,
+                    JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key(),
+                    scalingIntervalMin);
+        }
 
         deploy();
 
         // check if new resources have come available in the meantime
         context.runIfState(this, this::maybeRescale, Duration.ZERO);
     }
 
+    @VisibleForTesting
+    void setLastRescale(Instant lastRescale) {

Review Comment:
   > Changing a production constructor just for a test case is a bad design.
   
   So, do you think change a filed from `final` to `non-final` is a good 
design? If the lastRescale is passed from constructor, the `lastRescale` can be 
final, it's safe and reasonable. The caller of `new Executing()` can call the  
`Instant.now()`.
   
   It is worth mentioning that this suggestion is not created by me, but is a 
standard in the Java industry. It is called by everyone: `Inject dependencies`. 
`Inject dependencies` is also mentioned in Flink's "Code Style and Quality 
Guide", which is in the chapter on `Design for Testability`[1]. Two of them 
describe the current code:
   
   - Reusability becomes easier if constructors don’t create their dependencies 
(the objects assigned to the fields), but accept them as parameters.
   - Effectively, constructors should have no new keyword.
   
   So, I prefer the `lastRescale` is passed the the `Executing` constructor 
instead of new it in the `Executing` constructor.
   
   [1] 
https://flink.apache.org/how-to-contribute/code-style-and-quality-common/#design-for-testability
   



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java:
##########
@@ -67,13 +77,36 @@ class Executing extends StateWithExecutionGraph implements 
ResourceListener {
         this.context = context;
         Preconditions.checkState(
                 executionGraph.getState() == JobStatus.RUNNING, "Assuming 
running execution graph");
+        this.scalingIntervalMin = scalingIntervalMin;
+        this.scalingIntervalMax = scalingIntervalMax;
+        this.lastRescale =
+                Instant.now(); // Executing is recreated with each restart 
(when we rescale)
+        // we consider the first execution of the pipeline as a rescale event
+        Preconditions.checkState(
+                !scalingIntervalMin.isNegative(),
+                "%s must be positive integer or 0",
+                JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key());
+        if (scalingIntervalMax != null) {
+            Preconditions.checkState(
+                    scalingIntervalMax.compareTo(scalingIntervalMin) > 0,
+                    "%s(%d) must be greater than %s(%d)",
+                    JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(),
+                    scalingIntervalMax,
+                    JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key(),
+                    scalingIntervalMin);
+        }
 
         deploy();
 
         // check if new resources have come available in the meantime
         context.runIfState(this, this::maybeRescale, Duration.ZERO);
     }
 
+    @VisibleForTesting
+    void setLastRescale(Instant lastRescale) {

Review Comment:
   > Changing a production constructor just for a test case is a bad design.
   
   So, do you think change a filed from `final` to `non-final` is a good 
design? If the lastRescale is passed from constructor, the `lastRescale` can be 
final, it's safe and reasonable. The caller of `new Executing()` can call the  
`Instant.now()`.
   
   It is worth mentioning that this suggestion is not created by me, but is a 
standard in the Java industry. It is called by everyone: `Inject dependencies`. 
`Inject dependencies` is also mentioned in Flink's "Code Style and Quality 
Guide", which is in the chapter on `Design for Testability`[1]. Two of them 
describe the current code:
   
   - Reusability becomes easier if constructors don’t create their dependencies 
(the objects assigned to the fields), but accept them as parameters.
   - Effectively, constructors should have no new keyword.
   
   So, I prefer the `lastRescale` is passed the the `Executing` constructor 
instead of new it in the `Executing` constructor.
   
   [1] 
https://flink.apache.org/how-to-contribute/code-style-and-quality-common/#design-for-testability
   



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