aljoscha commented on a change in pull request #13502: URL: https://github.com/apache/flink/pull/13502#discussion_r496725604
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/RuntimeExecutionMode.java ########## @@ -0,0 +1,60 @@ +/* + * 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; + +import org.apache.flink.annotation.Internal; + +/** + * Runtime execution mode of DataStream programs. Among other things, this controls task scheduling, + * network shuffle behavior, and time semantics. Some operations will also change their record emission behaviour + * based on the configured execution mode. + * + * @see <a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-134%3A+Batch+execution+for+the+DataStream+API"> + * https://cwiki.apache.org/confluence/display/FLINK/FLIP-134%3A+Batch+execution+for+the+DataStream+API</a> + */ +@Internal +public enum RuntimeExecutionMode { + + /** + * The Pipeline will be executed with Streaming Semantics. All tasks will be deployed before execution starts, + * checkpoints will be enabled, and both processing and event time will be fully supported. + * + * @see <a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-134%3A+Batch+execution+for+the+DataStream+API"> Review comment: We probably don't need the `@see` link for all variants. 😅 ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/WithBoundedness.java ########## @@ -0,0 +1,34 @@ +/* + * 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.transformations; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.Boundedness; + +/** + * An interface to be implemented by transformations that have explicitly set {@link Boundedness}. + */ +@Internal +public interface WithBoundedness { + + /** + * @return The {@link Boundedness} of the implementing entity. Review comment: Maybe this is just a pet peeve but I don't like Javadoc comments that are empty except for a `@return`. I'd just say `Returns the {@link Boundedness} of this {@link Transformation}.` ########## File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/RuntimeExecutionModeDetectionTest.java ########## @@ -0,0 +1,158 @@ +/* + * 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.graph; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.IntegerTypeInfo; +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.runtime.jobgraph.ScheduleMode; +import org.apache.flink.streaming.api.RuntimeExecutionMode; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +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.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests for the detection of the {@link RuntimeExecutionMode runtime execution mode} during + * stream graph translation. + */ +public class RuntimeExecutionModeDetectionTest extends TestLogger { + + @Test + public void testDetectionThroughTransitivePredecessors() { + final SourceTransformation<Integer> bounded = + getSourceTransformation("Bounded Source", Boundedness.BOUNDED); + assertEquals(Boundedness.BOUNDED, bounded.getBoundedness()); + + final SourceTransformation<Integer> unbounded = getSourceTransformation( + "Unbounded Source", Boundedness.CONTINUOUS_UNBOUNDED); + assertEquals(Boundedness.CONTINUOUS_UNBOUNDED, unbounded.getBoundedness()); + + final TwoInputTransformation<Integer, Integer, Integer> resultTransform = new TwoInputTransformation<>( + bounded, + unbounded, + "Test Two Input Transformation", + SimpleOperatorFactory.of(new StreamGraphGeneratorTest.OutputTypeConfigurableOperationWithTwoInputs()), + BasicTypeInfo.INT_TYPE_INFO, + 1); + + final StreamGraph graph = createGraphAndDetectRuntimeExecutionMode(RuntimeExecutionMode.AUTOMATIC, resultTransform); + assertEquals(GlobalDataExchangeMode.ALL_EDGES_PIPELINED, graph.getGlobalDataExchangeMode()); + assertEquals(ScheduleMode.EAGER, graph.getScheduleMode()); + assertTrue(graph.isAllVerticesInSameSlotSharingGroupByDefault()); + } + + @Test + public void testBoundedDetection() { + final SourceTransformation<Integer> bounded = + getSourceTransformation("Bounded Source", Boundedness.BOUNDED); + assertEquals(Boundedness.BOUNDED, bounded.getBoundedness()); + + final StreamGraph graph = createGraphAndDetectRuntimeExecutionMode(RuntimeExecutionMode.AUTOMATIC, bounded); + assertEquals(GlobalDataExchangeMode.POINTWISE_EDGES_PIPELINED, graph.getGlobalDataExchangeMode()); + assertEquals(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST, graph.getScheduleMode()); + assertFalse(graph.isAllVerticesInSameSlotSharingGroupByDefault()); + } + + @Test + public void testUnboundedDetection() { + final SourceTransformation<Integer> unbounded = + getSourceTransformation("Unbounded Source", Boundedness.CONTINUOUS_UNBOUNDED); + assertEquals(Boundedness.CONTINUOUS_UNBOUNDED, unbounded.getBoundedness()); + + final StreamGraph graph = createGraphAndDetectRuntimeExecutionMode(RuntimeExecutionMode.AUTOMATIC, unbounded); + assertEquals(GlobalDataExchangeMode.ALL_EDGES_PIPELINED, graph.getGlobalDataExchangeMode()); + assertEquals(ScheduleMode.EAGER, graph.getScheduleMode()); + assertTrue(graph.isAllVerticesInSameSlotSharingGroupByDefault()); + } + + @Test + public void testMixedDetection() { + final SourceTransformation<Integer> unbounded = + getSourceTransformation("Unbounded Source", Boundedness.CONTINUOUS_UNBOUNDED); + assertEquals(Boundedness.CONTINUOUS_UNBOUNDED, unbounded.getBoundedness()); + + final SourceTransformation<Integer> bounded = + getSourceTransformation("Bounded Source", Boundedness.BOUNDED); + assertEquals(Boundedness.BOUNDED, bounded.getBoundedness()); + + final StreamGraph graph = createGraphAndDetectRuntimeExecutionMode(RuntimeExecutionMode.AUTOMATIC, unbounded); + assertEquals(GlobalDataExchangeMode.ALL_EDGES_PIPELINED, graph.getGlobalDataExchangeMode()); + assertEquals(ScheduleMode.EAGER, graph.getScheduleMode()); + assertTrue(graph.isAllVerticesInSameSlotSharingGroupByDefault()); + } + + @Test + public void testExplicitOverridesDetectedMode() { + final SourceTransformation<Integer> bounded = + getSourceTransformation("Bounded Source", Boundedness.BOUNDED); + assertEquals(Boundedness.BOUNDED, bounded.getBoundedness()); + + final StreamGraph graph = createGraphAndDetectRuntimeExecutionMode(RuntimeExecutionMode.AUTOMATIC, bounded); + assertEquals(GlobalDataExchangeMode.POINTWISE_EDGES_PIPELINED, graph.getGlobalDataExchangeMode()); + assertEquals(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST, graph.getScheduleMode()); + assertFalse(graph.isAllVerticesInSameSlotSharingGroupByDefault()); + + + final StreamGraph streamingGraph = createGraphAndDetectRuntimeExecutionMode(RuntimeExecutionMode.STREAMING, bounded); + assertEquals(GlobalDataExchangeMode.ALL_EDGES_PIPELINED, streamingGraph.getGlobalDataExchangeMode()); + assertEquals(ScheduleMode.EAGER, streamingGraph.getScheduleMode()); + assertTrue(streamingGraph.isAllVerticesInSameSlotSharingGroupByDefault()); + } + + private StreamGraph createGraphAndDetectRuntimeExecutionMode( Review comment: A nit, but this method doesn't detect the runtime mode, it's just an internal detail of `generate()`. Could be just `createStreamGraph(...)`. ---------------------------------------------------------------- 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