xintongsong commented on code in PR #20750: URL: https://github.com/apache/flink/pull/20750#discussion_r969365216
########## flink-tests/src/test/java/org/apache/flink/test/runtime/HybridShuffleITCase.java: ########## @@ -0,0 +1,203 @@ +/* + * 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.test.runtime; + +import org.apache.flink.api.common.BatchShuffleMode; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobType; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.testutils.junit.utils.TempDirUtils; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for hybrid shuffle mode. */ +class HybridShuffleITCase { + private static final String RECORD = "hybrid shuffle test"; + + private static final int NUM_TASK_MANAGERS = 2; + + private static final int NUM_SLOTS_PER_TASK_MANAGER = 10; + + private static final int PARALLELISM = NUM_SLOTS_PER_TASK_MANAGER; + + private static final int[] NUM_RECEIVED_RECORDS = new int[PARALLELISM]; + + private static Path tmpDir; + + @BeforeAll + static void setup(@TempDir Path path) throws Exception { + tmpDir = TempDirUtils.newFolder(path, UUID.randomUUID().toString()).toPath(); + } + + @Test + void testHybridFullExchanges() throws Exception { + final int numRecordsToSend = 10000; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.ALL_EXCHANGES_HYBRID_FULL); + JobGraph jobGraph = createJobGraph(numRecordsToSend, false, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridSelectiveExchanges() throws Exception { + final int numRecordsToSend = 10000; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_HYBRID_SELECTIVE); + JobGraph jobGraph = createJobGraph(numRecordsToSend, false, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridFullExchangesRestart() throws Exception { + final int numRecordsToSend = 10; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.ALL_EXCHANGES_HYBRID_FULL); + configuration.set(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 100); Review Comment: We can move this to `getConfiguration()`. ########## flink-tests/src/test/java/org/apache/flink/test/runtime/HybridShuffleITCase.java: ########## @@ -0,0 +1,203 @@ +/* + * 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.test.runtime; + +import org.apache.flink.api.common.BatchShuffleMode; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobType; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.testutils.junit.utils.TempDirUtils; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for hybrid shuffle mode. */ +class HybridShuffleITCase { + private static final String RECORD = "hybrid shuffle test"; + + private static final int NUM_TASK_MANAGERS = 2; + + private static final int NUM_SLOTS_PER_TASK_MANAGER = 10; + + private static final int PARALLELISM = NUM_SLOTS_PER_TASK_MANAGER; + + private static final int[] NUM_RECEIVED_RECORDS = new int[PARALLELISM]; + + private static Path tmpDir; + + @BeforeAll + static void setup(@TempDir Path path) throws Exception { + tmpDir = TempDirUtils.newFolder(path, UUID.randomUUID().toString()).toPath(); + } + + @Test + void testHybridFullExchanges() throws Exception { + final int numRecordsToSend = 10000; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.ALL_EXCHANGES_HYBRID_FULL); + JobGraph jobGraph = createJobGraph(numRecordsToSend, false, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridSelectiveExchanges() throws Exception { + final int numRecordsToSend = 10000; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_HYBRID_SELECTIVE); + JobGraph jobGraph = createJobGraph(numRecordsToSend, false, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridFullExchangesRestart() throws Exception { + final int numRecordsToSend = 10; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.ALL_EXCHANGES_HYBRID_FULL); + configuration.set(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 100); + JobGraph jobGraph = createJobGraph(numRecordsToSend, true, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridSelectiveExchangesRestart() throws Exception { + final int numRecordsToSend = 10; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_HYBRID_SELECTIVE); + configuration.set(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 100); + JobGraph jobGraph = createJobGraph(numRecordsToSend, true, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + private Configuration getConfiguration() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.TMP_DIRS, tmpDir.toString()); + configuration.set(ExecutionOptions.RUNTIME_MODE, RuntimeExecutionMode.BATCH); + configuration.set(NettyShuffleEnvironmentOptions.BATCH_SHUFFLE_COMPRESSION_ENABLED, false); + return configuration; + } + + private JobGraph createJobGraph( + int numRecordsToSend, boolean failExecution, Configuration configuration) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(10, 0L)); + env.setParallelism(NUM_SLOTS_PER_TASK_MANAGER); + + DataStream<String> source = + new DataStreamSource<>( + env, + BasicTypeInfo.STRING_TYPE_INFO, + new StreamSource<>(new StringSource(numRecordsToSend)), + true, + "source", + Boundedness.BOUNDED); + source.rebalance().map(value -> value).shuffle().addSink(new VerifySink(failExecution)); + + StreamGraph streamGraph = env.getStreamGraph(); + streamGraph.setJobType(JobType.BATCH); + return StreamingJobGraphGenerator.createJobGraph(streamGraph); + } + + private void checkAllDataReceived(int numRecordsToSend) { + assertThat(Arrays.stream(NUM_RECEIVED_RECORDS).sum()) + .isEqualTo(numRecordsToSend * PARALLELISM); + } + + private static class StringSource implements ParallelSourceFunction<String> { + private volatile boolean isRunning = true; + private int numRecordsToSend; + + StringSource(int numRecordsToSend) { + this.numRecordsToSend = numRecordsToSend; + } + + @Override + public void run(SourceContext<String> ctx) throws Exception { + while (isRunning && numRecordsToSend-- > 0) { + ctx.collect(RECORD); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class VerifySink extends RichSinkFunction<String> { + private final boolean failExecution; + + VerifySink(boolean failExecution) { + this.failExecution = failExecution; + } + + @Override + public void open(Configuration parameters) throws Exception { + NUM_RECEIVED_RECORDS[getRuntimeContext().getIndexOfThisSubtask()] = 0; + if (!failExecution + || getRuntimeContext().getAttemptNumber() > 0 + || getRuntimeContext().getIndexOfThisSubtask() != 0) { + return; + } + throw new RuntimeException("expected exception."); + } + + @Override + public void invoke(String value, Context context) throws Exception { + NUM_RECEIVED_RECORDS[getRuntimeContext().getIndexOfThisSubtask()]++; + assertThat(value).isEqualTo(RECORD); + } + } Review Comment: And can `createJobGraph` be deduplicated as well? ########## flink-tests/src/test/java/org/apache/flink/test/runtime/HybridShuffleITCase.java: ########## @@ -0,0 +1,203 @@ +/* + * 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.test.runtime; + +import org.apache.flink.api.common.BatchShuffleMode; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobType; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.testutils.junit.utils.TempDirUtils; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for hybrid shuffle mode. */ +class HybridShuffleITCase { + private static final String RECORD = "hybrid shuffle test"; + + private static final int NUM_TASK_MANAGERS = 2; + + private static final int NUM_SLOTS_PER_TASK_MANAGER = 10; + + private static final int PARALLELISM = NUM_SLOTS_PER_TASK_MANAGER; + + private static final int[] NUM_RECEIVED_RECORDS = new int[PARALLELISM]; + + private static Path tmpDir; + + @BeforeAll + static void setup(@TempDir Path path) throws Exception { + tmpDir = TempDirUtils.newFolder(path, UUID.randomUUID().toString()).toPath(); + } + + @Test + void testHybridFullExchanges() throws Exception { + final int numRecordsToSend = 10000; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.ALL_EXCHANGES_HYBRID_FULL); + JobGraph jobGraph = createJobGraph(numRecordsToSend, false, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridSelectiveExchanges() throws Exception { + final int numRecordsToSend = 10000; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_HYBRID_SELECTIVE); + JobGraph jobGraph = createJobGraph(numRecordsToSend, false, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridFullExchangesRestart() throws Exception { + final int numRecordsToSend = 10; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.ALL_EXCHANGES_HYBRID_FULL); + configuration.set(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 100); + JobGraph jobGraph = createJobGraph(numRecordsToSend, true, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + @Test + void testHybridSelectiveExchangesRestart() throws Exception { + final int numRecordsToSend = 10; + Configuration configuration = getConfiguration(); + configuration.set( + ExecutionOptions.BATCH_SHUFFLE_MODE, + BatchShuffleMode.ALL_EXCHANGES_HYBRID_SELECTIVE); + configuration.set(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 100); + JobGraph jobGraph = createJobGraph(numRecordsToSend, true, configuration); + JobGraphRunningUtil.execute( + jobGraph, configuration, NUM_TASK_MANAGERS, NUM_SLOTS_PER_TASK_MANAGER); + checkAllDataReceived(numRecordsToSend); + } + + private Configuration getConfiguration() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.TMP_DIRS, tmpDir.toString()); + configuration.set(ExecutionOptions.RUNTIME_MODE, RuntimeExecutionMode.BATCH); + configuration.set(NettyShuffleEnvironmentOptions.BATCH_SHUFFLE_COMPRESSION_ENABLED, false); + return configuration; + } + + private JobGraph createJobGraph( + int numRecordsToSend, boolean failExecution, Configuration configuration) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(10, 0L)); + env.setParallelism(NUM_SLOTS_PER_TASK_MANAGER); + + DataStream<String> source = + new DataStreamSource<>( + env, + BasicTypeInfo.STRING_TYPE_INFO, + new StreamSource<>(new StringSource(numRecordsToSend)), + true, + "source", + Boundedness.BOUNDED); + source.rebalance().map(value -> value).shuffle().addSink(new VerifySink(failExecution)); + + StreamGraph streamGraph = env.getStreamGraph(); + streamGraph.setJobType(JobType.BATCH); + return StreamingJobGraphGenerator.createJobGraph(streamGraph); + } + + private void checkAllDataReceived(int numRecordsToSend) { + assertThat(Arrays.stream(NUM_RECEIVED_RECORDS).sum()) + .isEqualTo(numRecordsToSend * PARALLELISM); + } + + private static class StringSource implements ParallelSourceFunction<String> { + private volatile boolean isRunning = true; + private int numRecordsToSend; + + StringSource(int numRecordsToSend) { + this.numRecordsToSend = numRecordsToSend; + } + + @Override + public void run(SourceContext<String> ctx) throws Exception { + while (isRunning && numRecordsToSend-- > 0) { + ctx.collect(RECORD); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class VerifySink extends RichSinkFunction<String> { + private final boolean failExecution; + + VerifySink(boolean failExecution) { + this.failExecution = failExecution; + } + + @Override + public void open(Configuration parameters) throws Exception { + NUM_RECEIVED_RECORDS[getRuntimeContext().getIndexOfThisSubtask()] = 0; + if (!failExecution + || getRuntimeContext().getAttemptNumber() > 0 + || getRuntimeContext().getIndexOfThisSubtask() != 0) { + return; + } + throw new RuntimeException("expected exception."); + } + + @Override + public void invoke(String value, Context context) throws Exception { + NUM_RECEIVED_RECORDS[getRuntimeContext().getIndexOfThisSubtask()]++; + assertThat(value).isEqualTo(RECORD); + } + } Review Comment: These classes seem duplications of those in `BlockingShuffleITCase`. We may introduce a `ShuffleITCaseBase` for deduplication. ########## flink-end-to-end-tests/flink-batch-sql-test/src/main/java/org/apache/flink/sql/tests/BatchSQLTestProgram.java: ########## @@ -55,9 +57,10 @@ public static void main(String[] args) throws Exception { ParameterTool params = ParameterTool.fromArgs(args); String outputPath = params.getRequired("outputPath"); String sqlStatement = params.getRequired("sqlStatement"); - + String shuffleMode = params.getRequired("shuffleMode"); TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); - + tEnv.getConfig() + .set(ExecutionOptions.BATCH_SHUFFLE_MODE, BatchShuffleMode.valueOf(shuffleMode)); Review Comment: Let's decouple the program argument from the `BATCH_SHUFFLE_MODE` values. -- 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]
