kl0u commented on a change in pull request #13647: URL: https://github.com/apache/flink/pull/13647#discussion_r506132003
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java ########## @@ -792,6 +792,12 @@ public void configure(ReadableConfig configuration, ClassLoader classLoader) { }); config.configure(configuration, classLoader); checkpointCfg.configure(configuration); + configuration.getOptional(ExecutionOptions.SORT_INPUTS).ifPresent( Review comment: Do we want to expose these to the user? Even if it is a power user. I lean more towards these being set depending on the selected execution mode in the `StreamGraphGenerator` (see https://github.com/apache/flink/pull/13656). This will also remove the need of passing the configuration to the generator ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java ########## @@ -336,12 +336,11 @@ public void setUserHash(String userHash) { this.userHash = userHash; } - @VisibleForTesting public void setSortedInputs(boolean sortedInputs) { this.sortedInputs = sortedInputs; } - boolean getSortedInputs() { + public boolean getSortedInputs() { Review comment: I think this change is not needed anymore. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java ########## @@ -239,13 +259,28 @@ private void configureStreamGraph(final StreamGraph graph) { graph.setGlobalDataExchangeMode(GlobalDataExchangeMode.POINTWISE_EDGES_PIPELINED); graph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST); setDefaultBufferTimeout(-1); + setBatchStateBackend(graph); } else { graph.setAllVerticesInSameSlotSharingGroupByDefault(true); graph.setGlobalDataExchangeMode(GlobalDataExchangeMode.ALL_EDGES_PIPELINED); graph.setScheduleMode(ScheduleMode.EAGER); } } + private void setBatchStateBackend(StreamGraph graph) { Review comment: Given that we also set the `timerService`, maybe give a more descriptive name? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/MultiInputTransformationTranslator.java ########## @@ -51,7 +51,12 @@ protected Collection<Integer> translateForBatchInternal( final AbstractMultipleInputTransformation<OUT> transformation, final Context context) { - return translateInternal(transformation, context); + Collection<Integer> ids = translateInternal(transformation, context); Review comment: In some places you use the name `ids` and in other translators `nodeIds`. What about make them uniform? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java ########## @@ -112,7 +117,7 @@ private final ExecutionConfig executionConfig; private final CheckpointConfig checkpointConfig; - + private final ReadableConfig configuration; Review comment: What about leaving some empty lines above and below? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/BatchExecutionUtils.java ########## @@ -0,0 +1,98 @@ +/* + * 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.streaming.runtime.translators; + +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.streaming.api.graph.StreamNode; +import org.apache.flink.streaming.api.graph.TransformationTranslator; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.InputSelectable; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A utility class for applying sorting inputs. + */ +class BatchExecutionUtils { + private static final Logger LOG = LoggerFactory.getLogger(BatchExecutionUtils.class); + + static void applySortingInputs( + int transformationId, + TransformationTranslator.Context context) { + StreamNode node = context.getStreamGraph().getStreamNode(transformationId); + boolean sortInputs = context.getGraphGeneratorConfig().get(ExecutionOptions.SORT_INPUTS); + boolean isInputSelectable = isInputSelectable(node); + + adjustChainingStrategy(node); + + checkState( + !isInputSelectable || !sortInputs, + "Batch state backend and sorting inputs are not supported in graphs with an InputSelectable operator." + ); + + if (sortInputs) { + LOG.debug("Enabling sorting inputs for an operator {}.", node); + node.setSortedInputs(true); + Map<ManagedMemoryUseCase, Integer> operatorScopeUseCaseWeights = new HashMap<>(); + operatorScopeUseCaseWeights.put(ManagedMemoryUseCase.BATCH_OP, 1); + node.setManagedMemoryUseCaseWeights( + operatorScopeUseCaseWeights, + Collections.emptySet() + ); + } + } + + @SuppressWarnings("rawtypes") + private static boolean isInputSelectable(StreamNode node) { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + Class<? extends StreamOperator> operatorClass = node.getOperatorFactory() + .getStreamOperatorClass(classLoader); + return InputSelectable.class.isAssignableFrom(operatorClass); + } + + private static void adjustChainingStrategy(StreamNode node) { + StreamOperatorFactory<?> operatorFactory = node.getOperatorFactory(); + ChainingStrategy currentChainingStrategy = operatorFactory.getChainingStrategy(); + switch (currentChainingStrategy) { + case ALWAYS: + case HEAD_WITH_SOURCES: + LOG.debug( + "Setting chaining strategy for an operator {}, because of the BATCH execution mode.", Review comment: What about "Setting chaining strategy to HEAD for operator {}, because of the BATCH execution mode."? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org