1996fanrui commented on code in PR #20990: URL: https://github.com/apache/flink/pull/20990#discussion_r1292783464
########## flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBaseJUnit5.java: ########## @@ -0,0 +1,125 @@ +/* + * 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.util; + +import org.apache.flink.client.program.MiniClusterClient; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.util.FileUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +/** + * Base class for unit tests that run multiple tests and want to reuse the same Flink cluster. This + * saves a significant amount of time, since the startup and shutdown of the Flink clusters + * (including actor systems, etc) usually dominates the execution of the actual tests. + * + * <p>To write a unit test against this test base, simply extend it and add one or more regular test + * methods and retrieve the StreamExecutionEnvironment from the context: + * + * <pre> + * {@literal @}Test + * public void someTest() { + * ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + * // test code + * env.execute(); + * } + * + * {@literal @}Test + * public void anotherTest() { + * StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + * // test code + * env.execute(); + * } + * + * </pre> + */ +public abstract class AbstractTestBaseJUnit5 { Review Comment: This PR added 3 based classes: - AbstractTestBaseJUnit5 - JavaProgramTestBaseJUnit5 - MultipleProgramsTestBaseJUnit5 As I understand, they are base test classes, and there are too many tests will use them in the future. So I have a couple of suggestions about them: - I prefer using the separate JIRA to do this. And this PR focues on the hadoop module. - About the suffix, `JUnit5` doesn't make sense to me. I prefer update the old `AbstractTestBase` to `AbstractTestBaseJUnit4`, and mark it to `@Deprecated`. And then adding the `AbstractTestBase` for junit5. - If we add the `JUnit5` suffix for junit5, we still need to remove the suffix after all tests are upgraded to junit5. What do you think? And I want to hear more thoughts from more experts of junit5. Hi @JingGe @XComp , would you mind helping take a look in your free time? thanks~ ########## flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBaseJUnit5.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.util; + +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +import java.util.Arrays; +import java.util.Collection; + +/** + * Base class for unit tests that run multiple tests and want to reuse the same Flink cluster. This + * saves a significant amount of time, since the startup and shutdown of the Flink clusters + * (including actor systems, etc) usually dominates the execution of the actual tests. + * + * <p>To write a unit test against this test base, simply extend it and add one or more regular test + * methods and retrieve the ExecutionEnvironment from the context: + * + * <pre>{@code + * {@literal @}Test + * public void someTest() { + * ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + * // test code + * env.execute(); + * } + * + * {@literal @}Test + * public void anotherTest() { + * ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + * // test code + * env.execute(); + * } + * + * }</pre> + */ +public class MultipleProgramsTestBaseJUnit5 extends AbstractTestBaseJUnit5 { + + /** + * Enum that defines which execution environment to run the next test on: An embedded local + * flink cluster, or the collection execution backend. + */ + public enum TestExecutionMode { Review Comment: If I understand correctly, the enum should be the logic inside of `MultipleProgramsTestBase` or its inheritor. So leaving it here looks good to me. ########## flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopIOFormatsITCase.java: ########## @@ -35,40 +38,54 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.SequenceFileInputFormat; -import org.junit.Assume; -import org.junit.Before; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.io.File; import java.net.URI; import java.util.ArrayList; import java.util.Collection; import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; +import static org.assertj.core.api.Assumptions.assumeThat; /** Integration tests for Hadoop IO formats. */ -@RunWith(Parameterized.class) -public class HadoopIOFormatsITCase extends JavaProgramTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class HadoopIOFormatsITCase extends JavaProgramTestBaseJUnit5 { private static final int NUM_PROGRAMS = 2; - private final int curProgId; + @Parameter private int curProgId; private String[] resultPath; private String[] expectedResult; private String sequenceFileInPath; private String sequenceFileInPathNull; - public HadoopIOFormatsITCase(int curProgId) { - this.curProgId = curProgId; + @BeforeEach + void checkOperatingSystem() { + // FLINK-5164 - see https://wiki.apache.org/hadoop/WindowsProblems + assumeThat(OperatingSystem.isWindows()) + .as("This test can't run successfully on Windows.") + .isFalse(); } - @Before - public void checkOperatingSystem() { - // FLINK-5164 - see https://wiki.apache.org/hadoop/WindowsProblems - Assume.assumeTrue( - "This test can't run successfully on Windows.", !OperatingSystem.isWindows()); + @Override + @TestTemplate + public void testJobWithObjectReuse() { + super.testJobWithoutObjectReuse(); + } + + @Override + @TestTemplate + public void testJobWithoutObjectReuse() { + super.testJobWithoutObjectReuse(); + } + + @Override + @TestTemplate + public void testJobCollectionExecution() { + super.testJobCollectionExecution(); } Review Comment: These methods call super method directly, why need to implement them? ########## flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatTest.java: ########## @@ -137,27 +132,26 @@ public void testFetchNextRecordReaderHasNewValue() throws Exception { setupHadoopInputFormat(new DummyInputFormat(), Job.getInstance(), recordReader); hadoopInputFormat.fetchNext(); - assertThat(hadoopInputFormat.fetched, is(true)); - assertThat(hadoopInputFormat.hasNext, is(true)); + assertThat(hadoopInputFormat.fetched).isTrue(); + assertThat(hadoopInputFormat.hasNext).isTrue(); } @Test - public void testFetchNextRecordReaderThrowsException() throws Exception { + void testFetchNextRecordReaderThrowsException() throws Exception { DummyRecordReader recordReader = mock(DummyRecordReader.class); when(recordReader.nextKeyValue()).thenThrow(new InterruptedException()); HadoopInputFormat<String, Long> hadoopInputFormat = setupHadoopInputFormat(new DummyInputFormat(), Job.getInstance(), recordReader); - exception.expect(IOException.class); - hadoopInputFormat.fetchNext(); + assertThatThrownBy(hadoopInputFormat::fetchNext).isInstanceOf(IOException.class); - assertThat(hadoopInputFormat.hasNext, is(true)); + assertThat(hadoopInputFormat.hasNext).isFalse(); Review Comment: Why changing this assert from `true` to `false`? -- 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]
