dawidwys commented on a change in pull request #13656:
URL: https://github.com/apache/flink/pull/13656#discussion_r506100673



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RuntimeOptions.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.api.environment;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.streaming.api.RuntimeExecutionMode;
+
+/**
+ * Execution {@link ConfigOption options} for configuring the runtime behavior 
of the pipeline.
+ *
+ * @see RuntimeExecutionMode
+ */
+@PublicEvolving
+public class RuntimeOptions {
+
+       public static final ConfigOption<RuntimeExecutionMode> RUNTIME_MODE =
+                       ConfigOptions.key("execution.runtime-mode")
+                                       .enumType(RuntimeExecutionMode.class)

Review comment:
       nit: Can't we put it into the `ExecutionOptions`? Do we need a separate 
group for it?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
##########
@@ -244,19 +245,30 @@ private void configureStreamGraph(final StreamGraph 
graph) {
        }
 
        private boolean shouldExecuteInBatchMode(final RuntimeExecutionMode 
configuredMode) {
+               final boolean existsUnboundedSource = existsContinuousSource();
+
+               if (configuredMode == RuntimeExecutionMode.BATCH && 
existsUnboundedSource) {
+                       throw new IllegalArgumentException(
+                                       "Detected an UNBOUNDED source with the 
'" + RuntimeOptions.RUNTIME_MODE.key() + "' set to 'BATCH'. " +
+                                                       "This combination is 
not allowed, please set the '" + RuntimeOptions.RUNTIME_MODE.key() +
+                                                       "' to STREAMING or 
AUTOMATIC");
+               }
+
                if (checkNotNull(configuredMode) != 
RuntimeExecutionMode.AUTOMATIC) {
                        return configuredMode == RuntimeExecutionMode.BATCH;
                }
+               return !existsUnboundedSource;
+       }
 
-               final boolean continuousSourceExists = transformations
+       private boolean existsContinuousSource() {

Review comment:
       nit: `existsUnboundedSource`

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
##########
@@ -244,19 +245,30 @@ private void configureStreamGraph(final StreamGraph 
graph) {
        }
 
        private boolean shouldExecuteInBatchMode(final RuntimeExecutionMode 
configuredMode) {
+               final boolean existsUnboundedSource = existsContinuousSource();
+
+               if (configuredMode == RuntimeExecutionMode.BATCH && 
existsUnboundedSource) {

Review comment:
       nit: `checkArgument`? I am undecided myself, but shall we use 
`IllegalArgumentException` or `IllegalStateException` in those cases?

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorExecutionModeDetectionTest.java
##########
@@ -25,31 +25,109 @@
 import org.apache.flink.api.connector.source.Boundedness;
 import org.apache.flink.api.connector.source.mocks.MockSource;
 import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.execution.JobClient;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.streaming.api.RuntimeExecutionMode;
 import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import org.apache.flink.streaming.api.environment.RuntimeOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
 import org.apache.flink.streaming.api.operators.SourceOperatorFactory;
 import org.apache.flink.streaming.api.transformations.SourceTransformation;
 import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
 import org.apache.flink.util.TestLogger;
 
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertThat;
 
 /**
  * Tests for the detection of the {@link RuntimeExecutionMode runtime 
execution mode} during
  * stream graph translation.
  */
 public class StreamGraphGeneratorExecutionModeDetectionTest extends TestLogger 
{
 
+       @Test
+       public void 
testExecutionModePropagationFromEnvWithDefaultAndBoundedSource() throws 
Exception {
+               final StreamExecutionEnvironment environment = new 
TestExecutionEnvironment(
+                               GlobalDataExchangeMode.ALL_EDGES_PIPELINED,
+                               ScheduleMode.EAGER,
+                               true);
+
+               environment
+                               .fromSource(
+                                               new 
MockSource(Boundedness.BOUNDED, 100),
+                                               
WatermarkStrategy.noWatermarks(),
+                                               "bounded-source")
+                               .print();
+               environment.executeAsync();

Review comment:
       Can't we do `environment.getStreamGraph` and assert that `StreamGraph`? 
I find it way easier to figure out what a test verifies if the assertions are 
part of the test body.




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


Reply via email to