[GitHub] [flink] 1996fanrui commented on a diff in pull request #23218: [FLINK-32854][flink-runtime][JUnit5 Migration] The state package of flink-runtime module

2023-09-11 Thread via GitHub


1996fanrui commented on code in PR #23218:
URL: https://github.com/apache/flink/pull/23218#discussion_r1321290658


##
flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotDirectoryTest.java:
##
@@ -19,180 +19,167 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.util.FileUtils;
-import org.apache.flink.util.TestLogger;
 
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.UUID;
 
-/** Tests for the {@link SnapshotDirectory}. */
-public class SnapshotDirectoryTest extends TestLogger {
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
-private static TemporaryFolder temporaryFolder;
+/** Tests for the {@link SnapshotDirectory}. */
+class SnapshotDirectoryTest {
 
-@BeforeClass
-public static void beforeClass() throws IOException {
-temporaryFolder = new TemporaryFolder();
-temporaryFolder.create();
-}
-
-@AfterClass
-public static void afterClass() {
-temporaryFolder.delete();
-}
+@TempDir private Path temporaryFolder;
 
 /** Tests if mkdirs for snapshot directories works. */
 @Test
-public void mkdirs() throws Exception {
-File folderRoot = temporaryFolder.getRoot();
+void mkdirs() throws Exception {
+File folderRoot = temporaryFolder.toFile();
 File newFolder = new File(folderRoot, 
String.valueOf(UUID.randomUUID()));
 File innerNewFolder = new File(newFolder, 
String.valueOf(UUID.randomUUID()));
 Path path = innerNewFolder.toPath();
 
-Assert.assertFalse(newFolder.isDirectory());
-Assert.assertFalse(innerNewFolder.isDirectory());
+assertThat(newFolder).doesNotExist();
+assertThat(innerNewFolder).doesNotExist();
 SnapshotDirectory snapshotDirectory = 
SnapshotDirectory.permanent(path);
-Assert.assertFalse(snapshotDirectory.exists());
-Assert.assertFalse(newFolder.isDirectory());
-Assert.assertFalse(innerNewFolder.isDirectory());
-
-Assert.assertTrue(snapshotDirectory.mkdirs());
-Assert.assertTrue(newFolder.isDirectory());
-Assert.assertTrue(innerNewFolder.isDirectory());
-Assert.assertTrue(snapshotDirectory.exists());
+assertThat(snapshotDirectory.exists()).isFalse();
+assertThat(newFolder).doesNotExist();
+assertThat(innerNewFolder).doesNotExist();

Review Comment:
   Thanks for the clarification, sounds good to me!



-- 
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



[GitHub] [flink] 1996fanrui commented on a diff in pull request #23218: [FLINK-32854][flink-runtime][JUnit5 Migration] The state package of flink-runtime module

2023-09-09 Thread via GitHub


1996fanrui commented on code in PR #23218:
URL: https://github.com/apache/flink/pull/23218#discussion_r1316013616


##
flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotDirectoryTest.java:
##
@@ -19,180 +19,167 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.util.FileUtils;
-import org.apache.flink.util.TestLogger;
 
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.UUID;
 
-/** Tests for the {@link SnapshotDirectory}. */
-public class SnapshotDirectoryTest extends TestLogger {
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
-private static TemporaryFolder temporaryFolder;
+/** Tests for the {@link SnapshotDirectory}. */
+class SnapshotDirectoryTest {
 
-@BeforeClass
-public static void beforeClass() throws IOException {
-temporaryFolder = new TemporaryFolder();
-temporaryFolder.create();
-}
-
-@AfterClass
-public static void afterClass() {
-temporaryFolder.delete();
-}
+@TempDir private Path temporaryFolder;
 
 /** Tests if mkdirs for snapshot directories works. */
 @Test
-public void mkdirs() throws Exception {
-File folderRoot = temporaryFolder.getRoot();
+void mkdirs() throws Exception {
+File folderRoot = temporaryFolder.toFile();
 File newFolder = new File(folderRoot, 
String.valueOf(UUID.randomUUID()));
 File innerNewFolder = new File(newFolder, 
String.valueOf(UUID.randomUUID()));
 Path path = innerNewFolder.toPath();
 
-Assert.assertFalse(newFolder.isDirectory());
-Assert.assertFalse(innerNewFolder.isDirectory());
+assertThat(newFolder).doesNotExist();
+assertThat(innerNewFolder).doesNotExist();
 SnapshotDirectory snapshotDirectory = 
SnapshotDirectory.permanent(path);
-Assert.assertFalse(snapshotDirectory.exists());
-Assert.assertFalse(newFolder.isDirectory());
-Assert.assertFalse(innerNewFolder.isDirectory());
-
-Assert.assertTrue(snapshotDirectory.mkdirs());
-Assert.assertTrue(newFolder.isDirectory());
-Assert.assertTrue(innerNewFolder.isDirectory());
-Assert.assertTrue(snapshotDirectory.exists());
+assertThat(snapshotDirectory.exists()).isFalse();
+assertThat(newFolder).doesNotExist();
+assertThat(innerNewFolder).doesNotExist();

Review Comment:
   Why are these 2 asserts changed? The original asserts are :
   
   ```
   Assert.assertFalse(newFolder.isDirectory());
   Assert.assertFalse(innerNewFolder.isDirectory());
   ```
   
   I see this class has a couple of similar changes.



##
flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStorageLoaderTest.java:
##
@@ -35,114 +35,118 @@
 import org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage;
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
 import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
 import org.apache.flink.util.DynamicCodeLoadingException;
-import org.apache.flink.util.TestLogger;
 
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
-import org.hamcrest.Matchers;
 import org.hamcrest.TypeSafeMatcher;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Collection;
 
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.assertj.core.api.HamcrestCondition.matching;
+
 /** This test validates that checkpoint storage is properly loaded from 
configuration. */
-public class CheckpointStorageLoaderTest extends TestLogger {
+class CheckpointStorageLoaderTest {
+
+private final Logger LOG = 
LoggerFactory.getLogger(CheckpointStorageLoaderTest.class);
 
-@Rule public final TemporaryFolder tmp = new TemporaryFolder();
+@TempDir private java.nio.file.Path tmp;
 
 private final ClassLoader cl = getClass().getClassLoader();
 
 @Test
-public void testNoCheckpointStorageDefined() throws Exception {
-Assert.assertFalse(
-CheckpointStorageLoader.fromConfig(new Configuration(), cl, 
null).isPresent());
+void testNoCheckpointStorageDefined() throws Exception {
+assertThat(CheckpointStorageLoader.fromConfig(new Configuration(), 

[GitHub] [flink] 1996fanrui commented on a diff in pull request #23218: [FLINK-32854][flink-runtime][JUnit5 Migration] The state package of flink-runtime module

2023-08-28 Thread via GitHub


1996fanrui commented on code in PR #23218:
URL: https://github.com/apache/flink/pull/23218#discussion_r1307500363


##
flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateOutputCheckpointStreamTest.java:
##
@@ -54,41 +55,40 @@ private OperatorStateHandle writeAllTestKeyGroups(
 }
 
 @Test
-public void testCloseNotPropagated() throws Exception {
+void testCloseNotPropagated() throws Exception {
 OperatorStateCheckpointOutputStream stream = createStream();
 TestMemoryCheckpointOutputStream innerStream =
 (TestMemoryCheckpointOutputStream) stream.getDelegate();
 stream.close();
-Assert.assertFalse(innerStream.isClosed());
+assertThat(innerStream.isClosed()).isFalse();
 innerStream.close();
 }
 
 @Test
-public void testEmptyOperatorStream() throws Exception {
+void testEmptyOperatorStream() throws Exception {
 OperatorStateCheckpointOutputStream stream = createStream();
 TestMemoryCheckpointOutputStream innerStream =
 (TestMemoryCheckpointOutputStream) stream.getDelegate();
 OperatorStateHandle emptyHandle = stream.closeAndGetHandle();
-Assert.assertTrue(innerStream.isClosed());
-Assert.assertEquals(0, stream.getNumberOfPartitions());
-Assert.assertEquals(null, emptyHandle);
+assertThat(innerStream.isClosed()).isTrue();
+assertThat(stream.getNumberOfPartitions()).isZero();
+assertThat(emptyHandle).isNull();
 }
 
 @Test
 public void testWriteReadRoundtrip() throws Exception {

Review Comment:
   ```suggestion
  void testWriteReadRoundtrip() throws Exception {
   ```



##
flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FileStateHandleTest.java:
##
@@ -147,7 +144,7 @@ public void 
testDiscardStateWithDeletionFailureThroughException() throws Excepti
 }
 
 @Test
-public void testDiscardStateWithDeletionFailureThroughReturnValue() throws 
Exception {
+void testDiscardStateWithDeletionFailureThroughReturnValue() throws 
Exception {

Review Comment:
   The public of `testDiscardStateWithDeletionFailureThroughException`can be 
removed.
   
   And the public of `FsCheckpointStreamFactoryTest` class can be removed.



##
flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupsStateHandleTest.java:
##
@@ -20,28 +20,26 @@
 
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** A test for {@link KeyGroupsStateHandle} */
-public class KeyGroupsStateHandleTest {
+class KeyGroupsStateHandleTest {
 
 @Test
-public void testNonEmptyIntersection() {
+void testNonEmptyIntersection() {
 KeyGroupRangeOffsets offsets = new KeyGroupRangeOffsets(0, 7);
 byte[] dummy = new byte[10];
 StreamStateHandle streamHandle = new ByteStreamStateHandle("test", 
dummy);
 KeyGroupsStateHandle handle = new KeyGroupsStateHandle(offsets, 
streamHandle);
 
 KeyGroupRange expectedRange = new KeyGroupRange(0, 3);
 KeyGroupsStateHandle newHandle = handle.getIntersection(expectedRange);
-assertNotNull(newHandle);
-assertEquals(streamHandle, newHandle.getDelegateStateHandle());
-assertEquals(expectedRange, newHandle.getKeyGroupRange());
-assertEquals(handle.getStateHandleId(), newHandle.getStateHandleId());
+assertThat(newHandle).isNotNull();
+assertThat(newHandle.getDelegateStateHandle()).isEqualTo(streamHandle);
+assertThat(newHandle.getKeyGroupRange()).isEqualTo(expectedRange);
+
assertThat(newHandle.getStateHandleId()).isEqualTo(handle.getStateHandleId());
 }
 
 @Test

Review Comment:
   The public can be removed.
   



-- 
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