Airblader commented on a change in pull request #16578:
URL: https://github.com/apache/flink/pull/16578#discussion_r675353353



##########
File path: flink-core/src/main/java/org/apache/flink/api/common/ShuffleMode.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.api.common;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.DescribedEnum;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.description.InlineElement;
+
+import static org.apache.flink.configuration.description.TextElement.text;
+
+/**
+ * Mode that defines how data is exchanged between tasks if the shuffling 
behavior has not been set
+ * explicitly for an individual exchange.
+ *
+ * <p>The shuffle mode depends on the configured {@link 
ExecutionOptions#RUNTIME_MODE} and is only
+ * relevant for batch executions on bounded streams.
+ *
+ * <p>In streaming mode, upstream and downstream tasks run simultaneously to 
achieve low latency. An
+ * exchange is always pipelined (i.e. a result record is immediately send to 
and processed by the

Review comment:
       send → sent

##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/ExecutionOptions.java
##########
@@ -38,6 +40,32 @@
                             "Runtime execution mode of DataStream programs. 
Among other things, "
                                     + "this controls task scheduling, network 
shuffle behavior, and time semantics.");
 
+    public static final ConfigOption<ShuffleMode> SHUFFLE_MODE =
+            ConfigOptions.key("execution.shuffle-mode")
+                    .enumType(ShuffleMode.class)
+                    .defaultValue(ShuffleMode.AUTOMATIC)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "Mode that defines how data is 
exchanged between tasks if the shuffling "
+                                                    + "behavior has not been 
set explicitly for an individual exchange. "
+                                                    + "The shuffle mode 
depends on the configured '%s' and is only "
+                                                    + "relevant for batch 
executions on bounded streams.",
+                                            text(RUNTIME_MODE.key()))
+                                    .linebreak()
+                                    .text(
+                                            "In streaming mode, upstream and 
downstream tasks run simultaneously to achieve low latency. "
+                                                    + "An exchange is always 
pipelined (i.e. a result record is immediately send to and "

Review comment:
       Same here

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
##########
@@ -351,34 +356,67 @@ private void configureStreamGraph(final StreamGraph 
graph) {
         graph.setUserArtifacts(userArtifacts);
         graph.setTimeCharacteristic(timeCharacteristic);
         graph.setJobName(jobName);
-        graph.setJobType(shouldExecuteInBatchMode ? JobType.BATCH : 
JobType.STREAMING);
 
         if (shouldExecuteInBatchMode) {
-
-            if (checkpointConfig.isCheckpointingEnabled()) {
-                LOG.info(
-                        "Disabled Checkpointing. Checkpointing is not 
supported and not needed when executing jobs in BATCH mode.");
-                checkpointConfig.disableCheckpointing();
-            }
-
-            graph.setAllVerticesInSameSlotSharingGroupByDefault(false);
-            
graph.setGlobalStreamExchangeMode(GlobalStreamExchangeMode.FORWARD_EDGES_PIPELINED);
+            configureStreamGraphBatch(graph);
             setDefaultBufferTimeout(-1);
-            setBatchStateBackendAndTimerService(graph);
         } else {
-            graph.setStateBackend(stateBackend);
-            
graph.setChangelogStateBackendEnabled(changelogStateBackendEnabled);
-            graph.setCheckpointStorage(checkpointStorage);
-            graph.setSavepointDirectory(savepointDir);
+            configureStreamGraphStreaming(graph);
+        }
+    }
+
+    private void configureStreamGraphBatch(final StreamGraph graph) {
+        graph.setJobType(JobType.BATCH);
+
+        if (checkpointConfig.isCheckpointingEnabled()) {
+            LOG.info(
+                    "Disabled Checkpointing. Checkpointing is not supported 
and not needed when executing jobs in BATCH mode.");
+            checkpointConfig.disableCheckpointing();
+        }
+        setBatchStateBackendAndTimerService(graph);
+
+        
graph.setGlobalStreamExchangeMode(deriveGlobalStreamExchangeModeBatch());
+        graph.setAllVerticesInSameSlotSharingGroupByDefault(false);
+    }
+
+    private void configureStreamGraphStreaming(final StreamGraph graph) {
+        graph.setJobType(JobType.STREAMING);
+
+        graph.setStateBackend(stateBackend);
+        graph.setChangelogStateBackendEnabled(changelogStateBackendEnabled);
+        graph.setCheckpointStorage(checkpointStorage);
+        graph.setSavepointDirectory(savepointDir);
+        
graph.setGlobalStreamExchangeMode(deriveGlobalStreamExchangeModeStreaming());
+    }
+
+    private GlobalStreamExchangeMode deriveGlobalStreamExchangeModeBatch() {
+        final ShuffleMode shuffleMode = 
configuration.get(ExecutionOptions.SHUFFLE_MODE);
+        if (shuffleMode == ShuffleMode.ALL_EXCHANGES_PIPELINED) {

Review comment:
       Why not use switch-case here (and below)? Linting tools / IDEs would 
then also be able to show warnings/errors if the enum is not cased over 
exhaustively.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to