XComp commented on code in PR #24471: URL: https://github.com/apache/flink/pull/24471#discussion_r1525110715
########## flink-end-to-end-tests/flink-batch-sql-test/src/test/java/org/apache/flink/sql/tests/BatchSQLTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.sql.tests; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.resources.ResourceTestUtils; +import org.apache.flink.test.util.JobSubmission; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; Review Comment: Your mixing up JUnit5 and JUnit4 classes/methods. The community agreed to move forward with JUnit5/Assertj for new tests. It's therefore advised to stick to this dependencies for newly added tests. ########## flink-end-to-end-tests/flink-batch-sql-test/src/test/java/org/apache/flink/sql/tests/BatchSQLTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.sql.tests; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.resources.ResourceTestUtils; +import org.apache.flink.test.util.JobSubmission; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** End-to-End tests for Batch SQL tests. */ +@RunWith(Parameterized.class) +public class BatchSQLTest extends TestLogger { + private static final Logger LOG = LoggerFactory.getLogger(BatchSQLTest.class); + private static final Path testJar = ResourceTestUtils.getResource("/BatchSQLTestProgram.jar"); + private static final String testMainClass = "org.apache.flink.sql.tests.BatchSQLTestProgram"; + private static final Path sqlPath = + ResourceTestUtils.getResource("resources/sql-job-query.sql"); + + @Parameterized.Parameters(name = "shuffleType[{0}]") + public static Collection<String> shuffleTypes() { + return Arrays.asList("blocking", "hybrid_full", "hybrid_selective"); + } + + @Rule public final TemporaryFolder tmp = new TemporaryFolder(); + + @Rule + public final FlinkResource flink = + new LocalStandaloneFlinkResourceFactory() Review Comment: `MiniClusterExtension` is probably what you want to use for JUnit5 ########## flink-end-to-end-tests/flink-batch-sql-test/src/test/resources/log4j2-test.properties: ########## @@ -0,0 +1,31 @@ +# +# 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. +# +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level=OFF +rootLogger.appenderRef.test.ref=TestLogger +appender.testlogger.name=TestLogger +appender.testlogger.type=CONSOLE +appender.testlogger.target=SYSTEM_ERR +appender.testlogger.layout.type=PatternLayout +appender.testlogger.layout.pattern=%-4r [%t] %-5p %c %x - %m%n +# Uncomment to enable codegen logging +#loggers = testlogger +#logger.testlogger.name =org.apache.flink.table.planner.codegen +#logger.testlogger.level = TRACE +#logger.testlogger.appenderRefs = TestLogger Review Comment: Is this needed? ########## flink-end-to-end-tests/flink-batch-sql-test/src/test/java/org/apache/flink/sql/tests/BatchSQLTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.sql.tests; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.resources.ResourceTestUtils; +import org.apache.flink.test.util.JobSubmission; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** End-to-End tests for Batch SQL tests. */ +@RunWith(Parameterized.class) +public class BatchSQLTest extends TestLogger { + private static final Logger LOG = LoggerFactory.getLogger(BatchSQLTest.class); + private static final Path testJar = ResourceTestUtils.getResource("/BatchSQLTestProgram.jar"); + private static final String testMainClass = "org.apache.flink.sql.tests.BatchSQLTestProgram"; + private static final Path sqlPath = + ResourceTestUtils.getResource("resources/sql-job-query.sql"); + + @Parameterized.Parameters(name = "shuffleType[{0}]") + public static Collection<String> shuffleTypes() { + return Arrays.asList("blocking", "hybrid_full", "hybrid_selective"); + } + + @Rule public final TemporaryFolder tmp = new TemporaryFolder(); + + @Rule + public final FlinkResource flink = + new LocalStandaloneFlinkResourceFactory() + .create( + FlinkResourceSetup.builder() + .addConfiguration(getConfiguration()) + .build()); + + private Path result; + private final String shuffleType; + + public BatchSQLTest(String shuffleType) { + this.shuffleType = shuffleType; + } + + private static Configuration getConfiguration() { + Configuration conf = new Configuration(); + conf.setString("taskmanager.numberOfTaskSlots", "1"); + return conf; + } + + @Before + public void before() throws Exception { + Path tmpPath = tmp.getRoot().toPath(); + this.result = tmpPath.resolve(String.format("result-%s", UUID.randomUUID())); + LOG.info("Result for this test will be stored at: {}", this.result); + } + + private void runAndCheck(String expected) throws Exception { + LOG.info("Starting cluster"); + try (ClusterController clusterController = flink.startCluster(1)) { + + LOG.info("Submitting job"); + JobSubmission.JobSubmissionBuilder builder = + new JobSubmission.JobSubmissionBuilder(testJar) + .setMainClass(testMainClass) + .setDetached(false) + .setParallelism(2); + + builder.addArgument("-sqlStatement", new String(Files.readAllBytes(sqlPath))); + builder.addArgument("-outputPath", this.result.toString()); + builder.addArgument("-shuffleType", shuffleType); + + clusterController.submitJob(builder.build(), Duration.ofMinutes(1)); + + LOG.info("Job finished"); + String actual = new String(Files.readAllBytes(this.result)); + LOG.debug(actual); + + assertEquals(expected, actual); + } + } + + @Test + public void testIt() throws Exception { + runAndCheck( Review Comment: What about inlining `runAndCheck`? It looks like the method is only used once. :thinking: ########## flink-end-to-end-tests/flink-batch-sql-test/src/test/java/org/apache/flink/sql/tests/BatchSQLTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.sql.tests; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.resources.ResourceTestUtils; +import org.apache.flink.test.util.JobSubmission; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** End-to-End tests for Batch SQL tests. */ +@RunWith(Parameterized.class) +public class BatchSQLTest extends TestLogger { + private static final Logger LOG = LoggerFactory.getLogger(BatchSQLTest.class); + private static final Path testJar = ResourceTestUtils.getResource("/BatchSQLTestProgram.jar"); + private static final String testMainClass = "org.apache.flink.sql.tests.BatchSQLTestProgram"; + private static final Path sqlPath = + ResourceTestUtils.getResource("resources/sql-job-query.sql"); + + @Parameterized.Parameters(name = "shuffleType[{0}]") + public static Collection<String> shuffleTypes() { + return Arrays.asList("blocking", "hybrid_full", "hybrid_selective"); + } + + @Rule public final TemporaryFolder tmp = new TemporaryFolder(); + + @Rule + public final FlinkResource flink = + new LocalStandaloneFlinkResourceFactory() + .create( + FlinkResourceSetup.builder() + .addConfiguration(getConfiguration()) + .build()); + + private Path result; + private final String shuffleType; + + public BatchSQLTest(String shuffleType) { + this.shuffleType = shuffleType; + } + + private static Configuration getConfiguration() { + Configuration conf = new Configuration(); + conf.setString("taskmanager.numberOfTaskSlots", "1"); Review Comment: ```suggestion conf.set(TaskManagerOptions.NUM_TASK_SLOTS, 1); ``` nit: Using the actual classes makes refactorings easier. ########## flink-end-to-end-tests/flink-batch-sql-test/src/test/java/org/apache/flink/sql/tests/BatchSQLTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.sql.tests; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.resources.ResourceTestUtils; +import org.apache.flink.test.util.JobSubmission; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** End-to-End tests for Batch SQL tests. */ +@RunWith(Parameterized.class) +public class BatchSQLTest extends TestLogger { + private static final Logger LOG = LoggerFactory.getLogger(BatchSQLTest.class); + private static final Path testJar = ResourceTestUtils.getResource("/BatchSQLTestProgram.jar"); + private static final String testMainClass = "org.apache.flink.sql.tests.BatchSQLTestProgram"; + private static final Path sqlPath = + ResourceTestUtils.getResource("resources/sql-job-query.sql"); + + @Parameterized.Parameters(name = "shuffleType[{0}]") + public static Collection<String> shuffleTypes() { + return Arrays.asList("blocking", "hybrid_full", "hybrid_selective"); + } + + @Rule public final TemporaryFolder tmp = new TemporaryFolder(); + + @Rule + public final FlinkResource flink = + new LocalStandaloneFlinkResourceFactory() + .create( + FlinkResourceSetup.builder() + .addConfiguration(getConfiguration()) + .build()); + + private Path result; + private final String shuffleType; + + public BatchSQLTest(String shuffleType) { + this.shuffleType = shuffleType; + } + + private static Configuration getConfiguration() { + Configuration conf = new Configuration(); + conf.setString("taskmanager.numberOfTaskSlots", "1"); + return conf; + } + + @Before + public void before() throws Exception { + Path tmpPath = tmp.getRoot().toPath(); + this.result = tmpPath.resolve(String.format("result-%s", UUID.randomUUID())); + LOG.info("Result for this test will be stored at: {}", this.result); + } + + private void runAndCheck(String expected) throws Exception { + LOG.info("Starting cluster"); + try (ClusterController clusterController = flink.startCluster(1)) { + + LOG.info("Submitting job"); + JobSubmission.JobSubmissionBuilder builder = + new JobSubmission.JobSubmissionBuilder(testJar) + .setMainClass(testMainClass) + .setDetached(false) + .setParallelism(2); + + builder.addArgument("-sqlStatement", new String(Files.readAllBytes(sqlPath))); + builder.addArgument("-outputPath", this.result.toString()); + builder.addArgument("-shuffleType", shuffleType); + + clusterController.submitJob(builder.build(), Duration.ofMinutes(1)); + + LOG.info("Job finished"); + String actual = new String(Files.readAllBytes(this.result)); + LOG.debug(actual); + + assertEquals(expected, actual); + } + } + + @Test + public void testIt() throws Exception { Review Comment: Could we make the test method name more descriptive? ########## flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java: ########## @@ -234,10 +234,7 @@ public JobID submitJob(final JobSubmission jobSubmission, Duration timeout) thro LOG.info("Running {}.", commands.stream().collect(Collectors.joining(" "))); - final Pattern pattern = - jobSubmission.isDetached() - ? Pattern.compile("Job has been submitted with JobID (.*)") - : Pattern.compile("Job with JobID (.*) has finished."); + final Pattern pattern = Pattern.compile("Job has been submitted with JobID (.*)"); Review Comment: Why is this change necessary? :thinking: ########## flink-end-to-end-tests/flink-batch-sql-test/src/test/java/org/apache/flink/sql/tests/BatchSQLTest.java: ########## @@ -0,0 +1,124 @@ +/* + * 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.sql.tests; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.resources.ResourceTestUtils; +import org.apache.flink.test.util.JobSubmission; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** End-to-End tests for Batch SQL tests. */ +@RunWith(Parameterized.class) +public class BatchSQLTest extends TestLogger { + private static final Logger LOG = LoggerFactory.getLogger(BatchSQLTest.class); + private static final Path testJar = ResourceTestUtils.getResource("/BatchSQLTestProgram.jar"); + private static final String testMainClass = "org.apache.flink.sql.tests.BatchSQLTestProgram"; + private static final Path sqlPath = + ResourceTestUtils.getResource("resources/sql-job-query.sql"); + + @Parameterized.Parameters(name = "shuffleType[{0}]") + public static Collection<String> shuffleTypes() { + return Arrays.asList("blocking", "hybrid_full", "hybrid_selective"); + } + + @Rule public final TemporaryFolder tmp = new TemporaryFolder(); + + @Rule + public final FlinkResource flink = + new LocalStandaloneFlinkResourceFactory() + .create( + FlinkResourceSetup.builder() + .addConfiguration(getConfiguration()) + .build()); + + private Path result; + private final String shuffleType; + + public BatchSQLTest(String shuffleType) { + this.shuffleType = shuffleType; + } + + private static Configuration getConfiguration() { + Configuration conf = new Configuration(); + conf.setString("taskmanager.numberOfTaskSlots", "1"); + return conf; + } + + @Before + public void before() throws Exception { Review Comment: ```suggestion public void before() { ``` -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org