zhuzhurk commented on code in PR #25790:
URL: https://github.com/apache/flink/pull/25790#discussion_r1894839918


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/OperatorsFinished.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.adaptivebatch;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class represents the information about the finished operators. It 
includes a list of
+ * StreamNode IDs representing the finished operators, and a map associating 
each finished
+ * StreamNode ID with their corresponding produced data size and distribution 
information.
+ */
+public class OperatorsFinished {
+
+    /** A list that holds the IDs of the finished StreamNodes. */
+    private final List<Integer> finishedStreamNodeIds;
+
+    /**
+     * A map that associates each finished StreamNode ID with a list of 
IntermediateResultInfo
+     * objects. The key is the StreamNode ID, and the value is a list of 
IntermediateResultInfo.
+     */
+    private final Map<Integer, List<BlockingResultInfo>> resultInfoMap;
+
+    public OperatorsFinished(
+            List<Integer> finishedStreamNodeIds,
+            Map<Integer, List<BlockingResultInfo>> resultInfoMap) {
+        this.finishedStreamNodeIds = checkNotNull(finishedStreamNodeIds);
+        this.resultInfoMap = checkNotNull(resultInfoMap);
+    }
+
+    public List<Integer> getFinishedStreamNodeIds() {
+        return finishedStreamNodeIds;
+    }
+
+    public Map<Integer, List<BlockingResultInfo>> getResultInfoMap() {
+        return resultInfoMap;

Review Comment:
   Should it be unmodifiable?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AbstractBlockingResultInfo.java:
##########
@@ -72,4 +75,9 @@ public void resetPartitionInfo(int partitionIndex) {
     int getNumOfRecordedPartitions() {
         return subpartitionBytesByPartitionIndex.size();
     }
+
+    @Override
+    public Map<Integer, long[]> getSubpartitionBytesByPartitionIndex() {
+        return new HashMap<>(subpartitionBytesByPartitionIndex);

Review Comment:
   Can we just return an unmodifiable map?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultInfo.java:
##########
@@ -35,6 +35,14 @@ public interface IntermediateResultInfo {
      */
     boolean isBroadcast();
 
+    /**
+     * Indicates whether every downstream consumer needs to consume all 
produced sub-partitions.
+     *
+     * @return true if every downstream consumer needs to consume all produced 
sub-partitions, false
+     *     otherwise.
+     */
+    boolean isEveryConsumerConsumeAllSubPartitions();

Review Comment:
   Is this used to distinguish whether it is a predefined broadcast or a later 
converted broadcast result?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/OperatorsFinished.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.adaptivebatch;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class represents the information about the finished operators. It 
includes a list of
+ * StreamNode IDs representing the finished operators, and a map associating 
each finished
+ * StreamNode ID with their corresponding produced data size and distribution 
information.
+ */
+public class OperatorsFinished {
+
+    /** A list that holds the IDs of the finished StreamNodes. */
+    private final List<Integer> finishedStreamNodeIds;
+
+    /**
+     * A map that associates each finished StreamNode ID with a list of 
IntermediateResultInfo
+     * objects. The key is the StreamNode ID, and the value is a list of 
IntermediateResultInfo.
+     */
+    private final Map<Integer, List<BlockingResultInfo>> resultInfoMap;
+
+    public OperatorsFinished(
+            List<Integer> finishedStreamNodeIds,
+            Map<Integer, List<BlockingResultInfo>> resultInfoMap) {
+        this.finishedStreamNodeIds = checkNotNull(finishedStreamNodeIds);
+        this.resultInfoMap = checkNotNull(resultInfoMap);
+    }
+
+    public List<Integer> getFinishedStreamNodeIds() {
+        return finishedStreamNodeIds;

Review Comment:
   Should it be unmodifiable?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##########
@@ -937,6 +965,26 @@ SpeculativeExecutionHandler 
getSpeculativeExecutionHandler() {
         return speculativeExecutionHandler;
     }
 
+    private void tryUpdateResultInfo(IntermediateDataSetID id, 
DistributionPattern targetPattern) {

Review Comment:
   +1. e.g. what may have being changed and what should be updated accordingly.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##########
@@ -273,6 +273,14 @@ public void onNewJobVerticesAdded(List<JobVertex> 
newVertices, int pendingOperat
 
         // 4. update json plan
         
getExecutionGraph().setJsonPlan(JsonPlanGenerator.generatePlan(getJobGraph()));
+
+        // 5. try aggregate subpartition bytes

Review Comment:
   Comments are needed to explain why this process is needed.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java:
##########
@@ -482,15 +490,29 @@ private void updateResultPartitionBytesMetrics(
                             result.getId(),
                             (ignored, resultInfo) -> {
                                 if (resultInfo == null) {
-                                    resultInfo = 
createFromIntermediateResult(result);
+                                    resultInfo =
+                                            
createFromIntermediateResult(result, new HashMap<>());

Review Comment:
   This change is in the wrong commit.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/StreamGraphOptimizer.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.adaptivebatch;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.graph.StreamGraphContext;
+import org.apache.flink.util.DynamicCodeLoadingException;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@code StreamGraphOptimizer} class is responsible for optimizing a 
StreamGraph based on
+ * runtime information.
+ *
+ * <p>Upon initialization, it obtains a {@code StreamGraphContext} from the 
{@code
+ * AdaptiveGraphManager} and loads the specified optimization strategies. At 
runtime, it applies
+ * these strategies sequentially to the StreamGraph using the provided context 
and information about
+ * finished operators.
+ */
+public class StreamGraphOptimizer {
+
+    private final List<StreamGraphOptimizationStrategy> optimizationStrategies;
+
+    public StreamGraphOptimizer(Configuration jobConfiguration, ClassLoader 
userClassLoader)
+            throws DynamicCodeLoadingException {
+        checkNotNull(jobConfiguration);
+
+        Optional<List<String>> optional =
+                jobConfiguration.getOptional(
+                        
StreamGraphOptimizationStrategy.STREAM_GRAPH_OPTIMIZATION_STRATEGY);
+        if (optional.isPresent()) {
+            optimizationStrategies = 
loadOptimizationStrategies(optional.get(), userClassLoader);
+        } else {
+            optimizationStrategies = new ArrayList<>();
+        }
+    }
+
+    /**
+     * Applies all loaded optimization strategies to the StreamGraph.
+     *
+     * @param operatorsFinished the object containing information about 
finished operators.
+     * @param context the StreamGraphContext providing methods to modify the 
StreamGraph.
+     */
+    public void maybeOptimizeStreamGraph(

Review Comment:
   Maybe rename it as mentioned in another comment?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/StreamGraphOptimizationStrategy.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.adaptivebatch;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.streaming.api.graph.StreamGraphContext;
+
+import java.util.List;
+
+/**
+ * Defines an optimization strategy for StreamGraph. Implementors of this 
interface provide methods
+ * to modify and optimize a StreamGraph based on contexts provided at runtime.
+ */
+@FunctionalInterface
+public interface StreamGraphOptimizationStrategy {
+
+    @Internal
+    ConfigOption<List<String>> STREAM_GRAPH_OPTIMIZATION_STRATEGY =
+            
ConfigOptions.key("execution.batch.adaptive.stream-graph-optimization.strategies")
+                    .stringType()
+                    .asList()
+                    .noDefaultValue()
+                    .withDescription(
+                            "Defines a comma-separated list of fully qualified 
class names "
+                                    + "implementing the 
StreamGraphOptimizationStrategy interface.");
+
+    /**
+     * Tries to optimize the StreamGraph using the provided OperatorsFinished 
and the
+     * StreamGraphContext. The method returns a boolean indicating whether the 
StreamGraph was
+     * successfully optimized.
+     *
+     * @param operatorsFinished the OperatorsFinished object containing 
information about completed
+     *     operators and their produced data size and distribution information.
+     * @param context the StreamGraphContext with a read-only view of a 
StreamGraph, providing
+     *     methods to modify StreamEdges and StreamNodes within the 
StreamGraph.
+     * @return {@code true} if the StreamGraph was successfully optimized; 
{@code false} otherwise.
+     */
+    boolean maybeOptimizeStreamGraph(

Review Comment:
   maybe  name it as `onOperatorsFinished`? in the future there can be other 
optimization actions based on different kinds of events.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/StreamGraphOptimizationStrategy.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.adaptivebatch;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.streaming.api.graph.StreamGraphContext;
+
+import java.util.List;
+
+/**
+ * Defines an optimization strategy for StreamGraph. Implementors of this 
interface provide methods
+ * to modify and optimize a StreamGraph based on contexts provided at runtime.
+ */
+@FunctionalInterface
+public interface StreamGraphOptimizationStrategy {
+
+    @Internal
+    ConfigOption<List<String>> STREAM_GRAPH_OPTIMIZATION_STRATEGY =
+            
ConfigOptions.key("execution.batch.adaptive.stream-graph-optimization.strategies")
+                    .stringType()
+                    .asList()
+                    .noDefaultValue()
+                    .withDescription(
+                            "Defines a comma-separated list of fully qualified 
class names "
+                                    + "implementing the 
StreamGraphOptimizationStrategy interface.");
+
+    /**
+     * Tries to optimize the StreamGraph using the provided OperatorsFinished 
and the

Review Comment:
   maybe {@link OperatorsFinished} and {@link StreamGraphContext}



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AllToAllBlockingResultInfo.java:
##########
@@ -139,7 +153,14 @@ public void resetPartitionInfo(int partitionIndex) {
     }
 
     public List<Long> getAggregatedSubpartitionBytes() {
-        checkState(aggregatedSubpartitionBytes != null, "Not all partition 
infos are ready");
-        return Collections.unmodifiableList(aggregatedSubpartitionBytes);
+        checkState(
+                aggregatedSubpartitionBytes != null
+                        || subpartitionBytesByPartitionIndex.size() == 
numOfPartitions,
+                "Not all partition infos are ready");
+        if (aggregatedSubpartitionBytes == null) {
+            return getAggregatedSubpartitionBytesInternal();

Review Comment:
   Can it just invoke `aggregateSubpartitionBytes()`? Invoking 
`getAggregatedSubpartitionBytesInternal()` will not update the 
`aggregatedSubpartitionBytes` nor clear the 
`subpartitionBytesByPartitionIndex`. 
   This comment also applies to other places where 
`getAggregatedSubpartitionBytesInternal()` is invoked.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AllToAllBlockingResultInfo.java:
##########
@@ -106,31 +117,34 @@ public long getNumBytesProduced(
 
     @Override
     public void recordPartitionInfo(int partitionIndex, ResultPartitionBytes 
partitionBytes) {
-        // Once all partitions are finished, we can convert the subpartition 
bytes to aggregated
-        // value to reduce the space usage, because the distribution of source 
splits does not
-        // affect the distribution of data consumed by downstream tasks of 
ALL_TO_ALL edges(Hashing
-        // or Rebalancing, we do not consider rare cases such as custom 
partitions here).

Review Comment:
   Similar comments are needed, although the condition may have changed 
slightly.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/BlockingResultInfo.java:
##########
@@ -64,4 +64,7 @@ public interface BlockingResultInfo extends 
IntermediateResultInfo {
      * @param partitionIndex the intermediate result partition index
      */
     void resetPartitionInfo(int partitionIndex);
+
+    /** Aggregates the subpartition bytes to reduce space usage. */
+    void aggregateSubpartitionBytes();

Review Comment:
   No need to add it in this common interface. Instead, in 
`AdaptiveBatchScheduler#maybeAggregateSubpartitionBytes()` it should invoke 
this only if it is an `AllToAllBlockingResultInfo`.



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