RocMarshal commented on code in PR #23635:
URL: https://github.com/apache/flink/pull/23635#discussion_r1378352765


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/BalancedPreferredSlotSharingStrategy.java:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This strategy tries to get a balanced tasks scheduling. Execution vertices, 
which are belong to
+ * the same SlotSharingGroup, tend to be put in the same 
ExecutionSlotSharingGroup. Co-location
+ * constraints are ignored at present.
+ */
+class BalancedPreferredSlotSharingStrategy extends AbstractSlotSharingStrategy

Review Comment:
   SGTM~



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/BalancedPreferredSlotSharingStrategy.java:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This strategy tries to get a balanced tasks scheduling. Execution vertices, 
which are belong to
+ * the same SlotSharingGroup, tend to be put in the same 
ExecutionSlotSharingGroup. Co-location
+ * constraints are ignored at present.

Review Comment:
   I'll update the header comments.



##########
flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java:
##########
@@ -708,6 +708,20 @@ public class TaskManagerOptions {
                             "Time we wait for the timers in milliseconds to 
finish all pending timer threads"
                                     + " when the stream task is cancelled.");
 
+    public static final ConfigOption<TaskManagerLoadBalanceMode> 
TASK_MANAGER_LOAD_BALANCE_MODE =
+            ConfigOptions.key("taskmanager.load-balance.mode")
+                    .enumType(TaskManagerLoadBalanceMode.class)
+                    .defaultValue(TaskManagerLoadBalanceMode.NONE)
+                    .withDescription(
+                            "The load balance mode of taskmanager when 
processing scheduling tasks.");

Review Comment:
   👌 



##########
flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java:
##########
@@ -708,6 +708,20 @@ public class TaskManagerOptions {
                             "Time we wait for the timers in milliseconds to 
finish all pending timer threads"
                                     + " when the stream task is cancelled.");
 
+    public static final ConfigOption<TaskManagerLoadBalanceMode> 
TASK_MANAGER_LOAD_BALANCE_MODE =
+            ConfigOptions.key("taskmanager.load-balance.mode")
+                    .enumType(TaskManagerLoadBalanceMode.class)
+                    .defaultValue(TaskManagerLoadBalanceMode.NONE)
+                    .withDescription(
+                            "The load balance mode of taskmanager when 
processing scheduling tasks.");

Review Comment:
   👌 



##########
flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java:
##########
@@ -708,6 +708,20 @@ public class TaskManagerOptions {
                             "Time we wait for the timers in milliseconds to 
finish all pending timer threads"
                                     + " when the stream task is cancelled.");
 
+    public static final ConfigOption<TaskManagerLoadBalanceMode> 
TASK_MANAGER_LOAD_BALANCE_MODE =

Review Comment:
   Thank you very much for the precise comments !
   
   I'd prefer the second and after a little adjustment as follows:
   1.  Add `@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER)`
   2. Temporarily comment out 'SLOT' mode in the `TaskManagerLoadBalanceMode`
   3.  Add the description about the configuration item like follows:
         ```
              The load balance mode of taskmanager when processing scheduling 
tasks.
   
              'NONE' means that the scheduler does not consider any dimensional 
balance when scheduling tasks.
              'TASKS' means that the scheduler prioritizes ensuring that the 
number of tasks on each TM is balanced when scheduling tasks.
   
              Please view FLIP-370 for more details.
         ```
   4. After all PRs merged, we'll add the 'SLOTS' mode back, supplement the 
description and deprecate slot-spread-out. the added description would like 
follows:
         ```
             'SLOTS' indicates that the scheduler prioritizes and balances the 
use of each TM's slot when scheduling tasks.
         ```
   
   Please let me know what's your opinion~



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