You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/11/28 14:03:43 UTC

[GitHub] [flink] jiexray opened a new pull request, #21410: [FLINK-29776][flink-statebackend-changelogmigrate changelog unit tests to junit5

jiexray opened a new pull request, #21410:
URL: https://github.com/apache/flink/pull/21410

   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   *(For example: This pull request makes task deployment go through the blob server, rather than through RPC. That way we avoid re-transferring them on each deployment (during recovery).)*
   
   
   ## Brief change log
   
   *(for example:)*
     - *The TaskInfo is stored in the blob store on job creation time as a persistent artifact*
     - *Deployments RPC transmits only the blob storage reference*
     - *TaskManagers retrieve the TaskInfo from the blob cache*
   
   
   ## Verifying this change
   
   Please make sure both new and modified tests in this PR follows the conventions defined in our code quality guide: https://flink.apache.org/contributing/code-style-and-quality-common.html#testing
   
   *(Please pick either of the following options)*
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This change is already covered by existing tests, such as *(please describe tests)*.
   
   *(or)*
   
   This change added tests and can be verified as follows:
   
   *(example:)*
     - *Added integration tests for end-to-end deployment with large payloads (100MB)*
     - *Extended integration test for recovery after master (JobManager) failure*
     - *Added test that validates that TaskInfo is transferred only once across recoveries*
     - *Manually verified the change by running a 4 node cluster with 2 JobManagers and 4 TaskManagers, a stateful streaming program, and killing one JobManager and two TaskManagers during the execution, verifying that recovery happens correctly.*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / no)
     - The serializers: (yes / no / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / no / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know)
     - The S3 file system connector: (yes / no / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / no)
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1044823025


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -959,24 +947,44 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E
 
             // on the second restore, since the custom serializer will be used for
             // deserialization, we expect the deliberate failure to be thrown
-            expectedException.expect(
-                    anyOf(
-                            isA(ExpectedKryoTestException.class),
-                            Matchers.<Throwable>hasProperty(
-                                    "cause", isA(ExpectedKryoTestException.class))));
-
-            // state backends that eagerly deserializes (such as the memory state backend) will fail
-            // here
-            backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
-
-            state =
-                    backend.getPartitionedState(
-                            VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-
-            backend.setCurrentKey(1);
-            // state backends that lazily deserializes (such as RocksDB) will fail here
-            state.value();
-
+            assertThatThrownBy(
+                            () -> {
+                                // state backends that eagerly deserializes (such as the memory
+                                // state backend) will fail
+                                // here
+                                CheckpointableKeyedStateBackend<Integer> restoreBackend = null;
+                                try {
+                                    restoreBackend =
+                                            restoreKeyedBackend(
+                                                    IntSerializer.INSTANCE, snapshot2, env);
+
+                                    ValueState<TestPojo> restoreState =
+                                            restoreBackend.getPartitionedState(
+                                                    VoidNamespace.INSTANCE,
+                                                    VoidNamespaceSerializer.INSTANCE,
+                                                    new ValueStateDescriptor<>("id", pojoType));
+
+                                    restoreBackend.setCurrentKey(1);
+                                    // state backends that lazily deserializes (such as RocksDB)
+                                    // will fail here
+                                    restoreState.value();
+
+                                    restoreBackend.dispose();
+                                } finally {
+                                    if (restoreBackend != null) {
+                                        IOUtils.closeQuietly(restoreBackend);
+                                        restoreBackend.dispose();
+                                    }
+                                }
+                            })
+                    .satisfiesAnyOf(
+                            actual ->
+                                    assertThat(actual)
+                                            .isInstanceOf(ExpectedKryoTestException.class),
+                            actual ->
+                                    assertThat(actual)
+                                            .hasFieldOrProperty("cause")

Review Comment:
   there is `.hasCauseInstanceOf` method in assertJ e.g.
   ```java
   assertThatThrownBy(() -> ...)
   .hasCauseInstanceOf(ExpectedCauseException.class);
   ```
   will not it help here?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047225701


##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java:
##########
@@ -453,12 +452,12 @@ public void testCompletingSnapshot() throws Exception {
 
             SnapshotResult<KeyedStateHandle> snapshotResult = snapshot.get();
             KeyedStateHandle keyedStateHandle = snapshotResult.getJobManagerOwnedSnapshot();
-            assertNotNull(keyedStateHandle);
-            assertTrue(keyedStateHandle.getStateSize() > 0);
-            assertEquals(2, keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
+            assertThat(keyedStateHandle).isNotNull();
+            assertThat(keyedStateHandle.getStateSize() > 0).isTrue();

Review Comment:
   ```suggestion
               assertThat(keyedStateHandle.getStateSize()).isGreaterThan(0);
   ```
   could be simplified



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "fredia (via GitHub)" <gi...@apache.org>.
fredia commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1173491827


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java:
##########
@@ -35,24 +34,22 @@
  * Tests for the keyed state backend and operator state backend, as created by the {@link
  * FsStateBackend}.
  */
-@RunWith(Parameterized.class)
 public class FileStateBackendTest extends StateBackendTestBase<FsStateBackend> {

Review Comment:
   Should we remove `public` here?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1329208363

   Hi, @piotrp @Myasuka Would you mind helping to check this flink-statebackend-changelog JUnit5 Migration ? Thank you very much.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1343908038

   Thank you @snuyanzin for your comments, I will fix them as soon as possible.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1694921585

   @Myasuka @fredia I have resolved all conversations? Could you have a look at this pr again.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1415322517

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1345145853

   Hello @snuyanzin, I have fixed your comments with a new commit. The modifications includes:
   
   - Remove public statement from test function
   - Replace `ExceptionUtils.findThrowable` with `assertThatThrownBy`
   - Replace `TestLogger` in junit4 with `TestLoggerExtension` in junit5


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1094683643


##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java:
##########
@@ -148,13 +145,16 @@ public static List<Object[]> modes() {
     private final RocksDBResourceContainer optionsContainer = new RocksDBResourceContainer();
 
     public void prepareRocksDB() throws Exception {
-        String dbPath = new File(TEMP_FOLDER.newFolder(), DB_INSTANCE_DIR_STRING).getAbsolutePath();
+        File dbPath = new File(tmpDbPath, DB_INSTANCE_DIR_STRING);
+        if (!dbPath.exists()) {
+            dbPath.mkdirs();

Review Comment:
   Thank you for pointing it out. In fact, this "manually creating path" is not required. The root problem is caused by the wrong "static" definition for field `tmpDbPath`.
   
   Some background: "Static" in JUnit5 `@TempDir`  has special meaning. Static field means that the temporary directory is shared among **ALL** test methods. In contrast, non-static field means each test would use its own temporary directory.
   
   In this utcase, each test should have an exclusive temporary directory for db. Thus, I change `tmpDbPath` to non-static field, and remove the "manually creating path". Every test is passed.
   
   By the way, the checkpoint path (`tmpCheckpointPath`) is shared by all tests.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1094695776


##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java:
##########
@@ -61,17 +62,7 @@
 
     private Map.Entry<UK, UV> loggingMapEntry(
             Map.Entry<UK, UV> entry, KvStateChangeLogger<Map<UK, UV>, N> changeLogger, N ns) {
-        return new Map.Entry<UK, UV>() {
-            @Override
-            public UK getKey() {
-                return entry.getKey();
-            }
-
-            @Override
-            public UV getValue() {
-                return entry.getValue();
-            }
-
+        return new AbstractMap.SimpleEntry<UK, UV>(entry.getKey(), entry.getValue()) {

Review Comment:
   The `assertThat(iterator.next()).isEqualTo(el);` in `ChangelogMapStateTest` uses `Object#equals` to compare two `Map.Entry`. The unnamed class `Map.Entry<UK, UV>` here uses the default `Object#equals()`, and fails the equal assertion.
   
   Alternatively, I add a basic `equal()` implementation for this unnamed Entry:
   ```
   @Override
   public boolean equals(Object o) {
       if (!(o instanceof Map.Entry)) {
           return false;
       }
       Map.Entry<?, ?> e = (Map.Entry<?, ?>)o;
       return Objects.equals(entry.getKey(), e.getKey()) && Objects.equals(entry.getValue(), e.getValue());
   }
   ```
   and remove the extension of `AbstractMap.SimpleEntry`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "fredia (via GitHub)" <gi...@apache.org>.
fredia commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1307253751


##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java:
##########
@@ -21,33 +21,37 @@
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.state.StateBackendMigrationTestBase;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.io.TempDir;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Collection;
+import java.util.List;
 
 /** Tests for the partitioned state part of {@link RocksDBStateBackend}. */
-@RunWith(Parameterized.class)
 public class RocksDBStateBackendMigrationTest
         extends StateBackendMigrationTestBase<RocksDBStateBackend> {
+    // Store it because we need it for the cleanup test.
+    public static String dbPath;

Review Comment:
   why mark `dbPath` as `public static`?



##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendLoadingTest.java:
##########
@@ -58,75 +58,73 @@
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TernaryBoolean;
 
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
 
 import javax.annotation.Nonnull;
 
 import java.util.Collection;
 import java.util.Collections;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.junit.jupiter.api.Assertions.assertSame;

Review Comment:
   Can replace `assertSame` with assertj?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendMigrationTest.java:
##########
@@ -18,18 +18,25 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
-import java.io.File;
+import java.util.Collection;
+import java.util.Collections;
 
 /**
  * Tests for the keyed state backend and operator state backend, as created by the {@link
  * FsStateBackend}.
  */
 public class FileStateBackendMigrationTest extends StateBackendMigrationTestBase<FsStateBackend> {

Review Comment:
   Remove `public`?



##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendMigrationTest.java:
##########
@@ -22,25 +22,30 @@
 import org.apache.flink.runtime.state.StateBackendMigrationTestBase;
 import org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage;
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 import org.apache.flink.util.function.SupplierWithException;
 
-import org.junit.ClassRule;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.io.TempDir;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 
 /** Tests for the partitioned state part of {@link EmbeddedRocksDBStateBackend}. */
-@RunWith(Parameterized.class)
 public class EmbeddedRocksDBStateBackendMigrationTest

Review Comment:
   Remove `public`?



##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java:
##########
@@ -46,18 +46,17 @@
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
 import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory;
 import org.apache.flink.runtime.util.BlockingCheckpointOutputStream;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.function.SupplierWithException;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.filefilter.IOFileFilter;
-import org.junit.After;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
 import org.mockito.Mockito;

Review Comment:
   It would be better to remove the `Mockito`.
   https://flink.apache.org/how-to-contribute/code-style-and-quality-common/#avoid-mockito---use-reusable-test-implementations



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1094833849


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendMigrationTest.java:
##########
@@ -24,28 +24,26 @@
 import org.apache.flink.runtime.state.StateBackendMigrationTestBase;
 import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
 import java.util.Arrays;
-import java.util.List;
+import java.util.Collection;
 import java.util.function.Supplier;
 
 /** Tests for the partitioned state part of {@link ChangelogStateBackend}. */
-@RunWith(Parameterized.class)
 public class ChangelogStateBackendMigrationTest

Review Comment:
   `@ExtendWith(ParameterizedTestExtension.class)` cannot use package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1043363020


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -74,6 +75,7 @@
  * are either compatible or requiring state migration after restoring the state backends.
  */
 @SuppressWarnings("serial")
+@ExtendWith(ParameterizedTestExtension.class)
 public abstract class StateBackendMigrationTestBase<B extends StateBackend> extends TestLogger {

Review Comment:
   `TestLogger` is a JUnit4 rule/extension
   Here should be used `META-INF/services/org.junit.jupiter.api.extension.Extension` 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1043364332


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -98,14 +100,14 @@ protected boolean supportsKeySerializerCheck() {
         return true;
     }
 
-    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    @TempDir public static File tempFolder;
 
-    @Before
+    @BeforeEach
     public void before() {
         env = MockEnvironment.builder().build();
     }
 
-    @After
+    @AfterEach
     public void after() {

Review Comment:
   ```suggestion
       void after() {
   ```
   Could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -156,9 +158,10 @@ public void testKeyedValueStateRegistrationFailsIfNewStateSerializerIsIncompatib
 
             fail("should have failed");
         } catch (Exception expected) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess this could be replaced with usage of `assertThatThrownBy`
   ```java
    assertThatThrownBy(
                           () ->
                                   testKeyedValueStateUpgrade(
                                           new ValueStateDescriptor<>(
                                                   stateName, new TestType.V1TestTypeSerializer()),
                                           new ValueStateDescriptor<>(
                                                   stateName,
                                                   new TestType.IncompatibleTestTypeSerializer())))
                   .isInstanceOf(StateMigrationException.class);
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -381,7 +385,7 @@ public void testKeyedMapStateAsIs() throws Exception {
                         stateName, IntSerializer.INSTANCE, new TestType.V1TestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedMapStateStateMigration() throws Exception {

Review Comment:
   ```suggestion
       void testKeyedMapStateStateMigration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -540,7 +545,7 @@ private void testKeyedMapStateUpgrade(
     //  Tests for keyed priority queue state
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testPriorityQueueStateCreationFailsIfNewSerializerIsNotCompatible()

Review Comment:
   ```suggestion
       void testPriorityQueueStateCreationFailsIfNewSerializerIsNotCompatible()
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -98,14 +100,14 @@ protected boolean supportsKeySerializerCheck() {
         return true;
     }
 
-    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    @TempDir public static File tempFolder;
 
-    @Before
+    @BeforeEach
     public void before() {

Review Comment:
   ```suggestion
       void before() {
   ```
   Could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -593,13 +598,14 @@ public void testStateBackendRestoreFailsIfNewKeySerializerRequiresMigration() th
             fail("should have failed");
         } catch (Exception expected) {
             // the new key serializer requires migration; this should fail the restore
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testStateBackendRestoreSucceedsIfNewKeySerializerRequiresReconfiguration()

Review Comment:
   ```suggestion
       void testStateBackendRestoreSucceedsIfNewKeySerializerRequiresReconfiguration()
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -119,7 +121,7 @@ public void after() {
     //  Tests for keyed ValueState
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testKeyedValueStateMigration() throws Exception {

Review Comment:
   ```suggestion
       void testKeyedValueStateMigration() throws Exception {
   ```
   Could be package private
   



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -249,7 +252,7 @@ public void testKeyedListStateMigration() throws Exception {
                         new TestType.V2TestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedListStateSerializerReconfiguration() throws Exception {

Review Comment:
   ```suggestion
       void testKeyedListStateSerializerReconfiguration() throws Exception {
   ```
   Could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -131,7 +133,7 @@ public void testKeyedValueStateMigration() throws Exception {
                         new TestType.V2TestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedValueStateSerializerReconfiguration() throws Exception {

Review Comment:
   ```suggestion
       void testKeyedValueStateSerializerReconfiguration() throws Exception {
   ```
   Could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -144,7 +146,7 @@ public void testKeyedValueStateSerializerReconfiguration() throws Exception {
                         new TestType.ReconfigurationRequiringTestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedValueStateRegistrationFailsIfNewStateSerializerIsIncompatible() {

Review Comment:
   ```suggestion
       void testKeyedValueStateRegistrationFailsIfNewStateSerializerIsIncompatible() {
   ```
   Could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -690,7 +697,7 @@ private void testKeySerializerUpgrade(
     //  Tests for namespace serializer in keyed state backends
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerRequiresMigration()

Review Comment:
   ```suggestion
       void testKeyedStateRegistrationFailsIfNewNamespaceSerializerRequiresMigration()
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -262,7 +265,7 @@ public void testKeyedListStateSerializerReconfiguration() throws Exception {
                         new TestType.ReconfigurationRequiringTestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedListStateRegistrationFailsIfNewStateSerializerIsIncompatible() {

Review Comment:
   ```suggestion
       void testKeyedListStateRegistrationFailsIfNewStateSerializerIsIncompatible() {
   ```
   Could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -274,9 +277,10 @@ public void testKeyedListStateRegistrationFailsIfNewStateSerializerIsIncompatibl
 
             fail("should have failed");
         } catch (Exception expected) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   i guess it could be replaced with usage of `assertThatThrownBy`
   something like
   ```java
           assertThatThrownBy(
                           () ->
                                   testKeyedListStateUpgrade(
                                           new ListStateDescriptor<>(
                                                   stateName, new TestType.V1TestTypeSerializer()),
                                           new ListStateDescriptor<>(
                                                   stateName,
                                                   new TestType.IncompatibleTestTypeSerializer())))
                   .isInstanceOf(StateMigrationException.class);
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -800,7 +809,7 @@ private void testNamespaceSerializerUpgrade(
     //  Operator state backend partitionable list state tests
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testOperatorParitionableListStateMigration() throws Exception {

Review Comment:
   ```suggestion
       void testOperatorParitionableListStateMigration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -237,7 +240,7 @@ private void testKeyedValueStateUpgrade(
     //  Tests for keyed ListState
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testKeyedListStateMigration() throws Exception {

Review Comment:
   ```suggestion
       void testKeyedListStateMigration() throws Exception {
   ```
   Could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -370,7 +374,7 @@ private void testKeyedListStateUpgrade(
     //  Tests for keyed MapState
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testKeyedMapStateAsIs() throws Exception {

Review Comment:
   ```suggestion
       void testKeyedMapStateAsIs() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -395,7 +399,7 @@ public void testKeyedMapStateStateMigration() throws Exception {
                         new TestType.V2TestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedMapStateSerializerReconfiguration() throws Exception {

Review Comment:
   ```suggestion
       void testKeyedMapStateSerializerReconfiguration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -909,7 +918,7 @@ public void testOperatorUnionListStateMigration() throws Exception {
                         new TestType.V2TestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testOperatorUnionListStateSerializerReconfiguration() throws Exception {

Review Comment:
   ```suggestion
       void testOperatorUnionListStateSerializerReconfiguration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -839,8 +848,8 @@ public void testOperatorParitionableListStateRegistrationFailsIfNewSerializerIsI
 
             fail("should have failed.");
         } catch (Exception e) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent());
+            assertThat(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -921,7 +930,7 @@ public void testOperatorUnionListStateSerializerReconfiguration() throws Excepti
                         new TestType.ReconfigurationRequiringTestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testOperatorUnionListStateRegistrationFailsIfNewSerializerIsIncompatible() {

Review Comment:
   ```suggestion
       void testOperatorUnionListStateRegistrationFailsIfNewSerializerIsIncompatible() {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1024,7 +1033,7 @@ public void testBroadcastStateKeyMigration() throws Exception {
                         IntSerializer.INSTANCE));
     }
 
-    @Test
+    @TestTemplate
     public void testBroadcastStateValueSerializerReconfiguration() throws Exception {

Review Comment:
   ```suggestion
       void testBroadcastStateValueSerializerReconfiguration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -409,7 +413,7 @@ public void testKeyedMapStateSerializerReconfiguration() throws Exception {
                         new TestType.ReconfigurationRequiringTestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedMapStateRegistrationFailsIfNewStateSerializerIsIncompatible() {

Review Comment:
   ```suggestion
       void testKeyedMapStateRegistrationFailsIfNewStateSerializerIsIncompatible() {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -424,9 +428,10 @@ public void testKeyedMapStateRegistrationFailsIfNewStateSerializerIsIncompatible
                             new TestType.IncompatibleTestTypeSerializer()));
             fail("should have failed");
         } catch (Exception expected) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1038,7 +1047,7 @@ public void testBroadcastStateValueSerializerReconfiguration() throws Exception
                         new TestType.ReconfigurationRequiringTestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testBroadcastStateKeySerializerReconfiguration() throws Exception {

Review Comment:
   ```suggestion
       void testBroadcastStateKeySerializerReconfiguration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1052,7 +1061,7 @@ public void testBroadcastStateKeySerializerReconfiguration() throws Exception {
                         IntSerializer.INSTANCE));
     }
 
-    @Test
+    @TestTemplate
     public void testBroadcastStateRegistrationFailsIfNewValueSerializerIsIncompatible() {

Review Comment:
   ```suggestion
       void testBroadcastStateRegistrationFailsIfNewValueSerializerIsIncompatible() {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -996,7 +1005,7 @@ private void testOperatorUnionListStateUpgrade(
     //  Operator state backend broadcast state tests
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testBroadcastStateValueMigration() throws Exception {

Review Comment:
   ```suggestion
       void testBroadcastStateValueMigration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1068,12 +1077,12 @@ public void testBroadcastStateRegistrationFailsIfNewValueSerializerIsIncompatibl
 
             fail("should have failed.");
         } catch (Exception e) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent());
+            assertThat(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent())
+                    .isTrue();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testBroadcastStateRegistrationFailsIfNewKeySerializerIsIncompatible() {

Review Comment:
   ```suggestion
       void testBroadcastStateRegistrationFailsIfNewKeySerializerIsIncompatible() {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -935,8 +944,8 @@ public void testOperatorUnionListStateRegistrationFailsIfNewSerializerIsIncompat
 
             fail("should have failed.");
         } catch (Exception e) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent());
+            assertThat(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1010,7 +1019,7 @@ public void testBroadcastStateValueMigration() throws Exception {
                         new TestType.V2TestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testBroadcastStateKeyMigration() throws Exception {

Review Comment:
   ```suggestion
       void testBroadcastStateKeyMigration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -583,7 +588,7 @@ public void testPriorityQueueStateCreationFailsIfNewSerializerIsNotCompatible()
     //  Tests for key serializer in keyed state backends
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testStateBackendRestoreFailsIfNewKeySerializerRequiresMigration() throws Exception {

Review Comment:
   ```suggestion
       void testStateBackendRestoreFailsIfNewKeySerializerRequiresMigration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1216,7 +1224,7 @@ public void testStateMigrationAfterChangingTTL() throws Exception {
         testKeyedValueStateUpgrade(initialAccessDescriptor, newAccessDescriptorAfterRestore);
     }
 
-    @Test
+    @TestTemplate
     public void testStateMigrationAfterChangingTTLFromEnablingToDisabling() {

Review Comment:
   ```suggestion
       void testStateMigrationAfterChangingTTLFromEnablingToDisabling() {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -572,8 +577,8 @@ public void testPriorityQueueStateCreationFailsIfNewSerializerIsNotCompatible()
 
             fail("should have failed");
         } catch (Exception e) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent());
+            assertThat(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy` 



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1068,12 +1077,12 @@ public void testBroadcastStateRegistrationFailsIfNewValueSerializerIsIncompatibl
 
             fail("should have failed.");
         } catch (Exception e) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent());
+            assertThat(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent())
+                    .isTrue();

Review Comment:
   i guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1231,13 +1239,14 @@ public void testStateMigrationAfterChangingTTLFromEnablingToDisabling() {
             testKeyedValueStateUpgrade(initialAccessDescriptor, newAccessDescriptorAfterRestore);
             fail("should have failed");
         } catch (Exception expected) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -593,13 +598,14 @@ public void testStateBackendRestoreFailsIfNewKeySerializerRequiresMigration() th
             fail("should have failed");
         } catch (Exception expected) {
             // the new key serializer requires migration; this should fail the restore
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1231,13 +1239,14 @@ public void testStateMigrationAfterChangingTTLFromEnablingToDisabling() {
             testKeyedValueStateUpgrade(initialAccessDescriptor, newAccessDescriptorAfterRestore);
             fail("should have failed");
         } catch (Exception expected) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testStateMigrationAfterChangingTTLFromDisablingToEnabling() {

Review Comment:
   ```suggestion
       void testStateMigrationAfterChangingTTLFromDisablingToEnabling() {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -619,9 +625,10 @@ public void testStateBackendRestoreFailsIfNewKeySerializerIsIncompatible() throw
             fail("should have failed");
         } catch (Exception expected) {
             // the new key serializer is incompatible; this should fail the restore
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1089,8 +1098,8 @@ public void testBroadcastStateRegistrationFailsIfNewKeySerializerIsIncompatible(
 
             fail("should have failed.");
         } catch (Exception e) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent());
+            assertThat(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1195,12 +1204,11 @@ public void internalCopySerializerTest(
                         previousSerializer.snapshotConfiguration());
         testProvider.registerNewSerializerForRestoredState(newSerializerForRestoredState);
 
-        assertEquals(
-                testProvider.currentSchemaSerializer().getClass(),
-                internalCopySerializer.getClass());
+        assertThat(internalCopySerializer.getClass())
+                .isEqualTo(testProvider.currentSchemaSerializer().getClass());
     }
 
-    @Test
+    @TestTemplate
     public void testStateMigrationAfterChangingTTL() throws Exception {

Review Comment:
   ```suggestion
       void testStateMigrationAfterChangingTTL() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -1253,9 +1262,10 @@ public void testStateMigrationAfterChangingTTLFromDisablingToEnabling() {
             testKeyedValueStateUpgrade(initialAccessDescriptor, newAccessDescriptorAfterRestore);
             fail("should have failed");
         } catch (Exception expected) {
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, IllegalStateException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, IllegalStateException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -725,9 +733,10 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerIsIncompatibl
             fail("should have failed");
         } catch (Exception expected) {
             // the new namespace serializer is incompatible; this should fail the restore
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -812,7 +821,7 @@ public void testOperatorParitionableListStateMigration() throws Exception {
                         new TestType.V2TestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testOperatorParitionableListStateSerializerReconfiguration() throws Exception {

Review Comment:
   ```suggestion
       void testOperatorParitionableListStateSerializerReconfiguration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -700,21 +707,22 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerRequiresMigra
             fail("should have failed");
         } catch (Exception expected) {
             // the new namespace serializer requires migration; this should fail the restore
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();

Review Comment:
   I guess it could be replaced with usage of `assertThatThrownBy`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -897,7 +906,7 @@ private void testOperatorPartitionableListStateUpgrade(
     //  Operator state backend union list state tests
     // -------------------------------------------------------------------------------
 
-    @Test
+    @TestTemplate
     public void testOperatorUnionListStateMigration() throws Exception {

Review Comment:
   ```suggestion
       void testOperatorUnionListStateMigration() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -824,7 +833,7 @@ public void testOperatorParitionableListStateSerializerReconfiguration() throws
                         new TestType.ReconfigurationRequiringTestTypeSerializer()));
     }
 
-    @Test
+    @TestTemplate
     public void testOperatorParitionableListStateRegistrationFailsIfNewSerializerIsIncompatible()

Review Comment:
   ```suggestion
       void testOperatorParitionableListStateRegistrationFailsIfNewSerializerIsIncompatible()
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -608,7 +614,7 @@ public void testStateBackendRestoreSucceedsIfNewKeySerializerRequiresReconfigura
                 new TestType.ReconfigurationRequiringTestTypeSerializer());
     }
 
-    @Test
+    @TestTemplate
     public void testStateBackendRestoreFailsIfNewKeySerializerIsIncompatible() throws Exception {

Review Comment:
   ```suggestion
       void testStateBackendRestoreFailsIfNewKeySerializerIsIncompatible() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -700,21 +707,22 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerRequiresMigra
             fail("should have failed");
         } catch (Exception expected) {
             // the new namespace serializer requires migration; this should fail the restore
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedStateRegistrationSucceedsIfNewNamespaceSerializerRequiresReconfiguration()
             throws Exception {
         testNamespaceSerializerUpgrade(
                 new TestType.V1TestTypeSerializer(),
                 new TestType.ReconfigurationRequiringTestTypeSerializer());
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerIsIncompatible()

Review Comment:
   ```suggestion
       void testKeyedStateRegistrationFailsIfNewNamespaceSerializerIsIncompatible()
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -700,21 +707,22 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerRequiresMigra
             fail("should have failed");
         } catch (Exception expected) {
             // the new namespace serializer requires migration; this should fail the restore
-            Assert.assertTrue(
-                    ExceptionUtils.findThrowable(expected, StateMigrationException.class)
-                            .isPresent());
+            assertThat(
+                            ExceptionUtils.findThrowable(expected, StateMigrationException.class)
+                                    .isPresent())
+                    .isTrue();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testKeyedStateRegistrationSucceedsIfNewNamespaceSerializerRequiresReconfiguration()

Review Comment:
   ```suggestion
       void testKeyedStateRegistrationSucceedsIfNewNamespaceSerializerRequiresReconfiguration()
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1308122484


##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java:
##########
@@ -46,18 +46,17 @@
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
 import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory;
 import org.apache.flink.runtime.util.BlockingCheckpointOutputStream;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.function.SupplierWithException;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.filefilter.IOFileFilter;
-import org.junit.After;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
 import org.mockito.Mockito;

Review Comment:
   Good point. I think this would better be changed in another pr.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1094833452


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackendTest.java:
##########
@@ -38,22 +38,23 @@
 import org.apache.flink.runtime.state.ttl.mock.MockKeyedStateBackend.MockSnapshotSupplier;
 import org.apache.flink.runtime.state.ttl.mock.MockKeyedStateBackendBuilder;
 import org.apache.flink.state.changelog.ChangelogStateBackendTestUtils.DummyCheckpointingStorageAccess;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameter;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.concurrent.RunnableFuture;
 
 import static java.util.Collections.emptyList;
-import static org.junit.Assert.assertEquals;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** {@link ChangelogKeyedStateBackend} test. */
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
 public class ChangelogKeyedStateBackendTest {

Review Comment:
   `@ExtendWith(ParameterizedTestExtension.class)` cannot use package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047219146


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogPqStateTest.java:
##########
@@ -40,57 +40,55 @@
 
 import static java.util.Collections.emptyList;
 import static java.util.Collections.singletonList;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** ChangelogKeyGroupedPriorityQueue Test. */
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class ChangelogPqStateTest {

Review Comment:
   ```suggestion
   class ChangelogPqStateTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047208624


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -152,17 +137,16 @@
  * <p>NOTE: Please ensure to close and dispose any created keyed state backend in tests.
  */
 @SuppressWarnings("serial")
-public abstract class StateBackendTestBase<B extends AbstractStateBackend> extends TestLogger {
+@ExtendWith({ParameterizedTestExtension.class, TestLoggerExtension.class})

Review Comment:
   No, I mean not `TestLoggerExtension`
   Using the `META-INF/services/org.junit.jupiter.api.extension.Extension` is the better approach here.
   for more details have a look at
   https://github.com/apache/flink/commit/ecf67c67dbcf633c4533f8cb719f32512c19d0ce#diff-d96a3b8d09f526305595964a0010cf113e59999006534653bf45b9971b66241f



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]migrate changelog unit tests to junit5

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1329180126

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9d9dcdf62074f325fa164ec9233c11a9c6baec5e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9d9dcdf62074f325fa164ec9233c11a9c6baec5e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9d9dcdf62074f325fa164ec9233c11a9c6baec5e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047220616


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendLoadingTest.java:
##########
@@ -58,75 +58,73 @@
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TernaryBoolean;
 
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
 
 import javax.annotation.Nonnull;
 
 import java.util.Collection;
 import java.util.Collections;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.junit.jupiter.api.Assertions.assertSame;
 
 /** Verify Changelog StateBackend is properly loaded. */
 public class ChangelogStateBackendLoadingTest {
-    @Rule public final TemporaryFolder tmp = new TemporaryFolder();
-
     private final ClassLoader cl = getClass().getClassLoader();
 
     private final String backendKey = StateBackendOptions.STATE_BACKEND.key();
 
     @Test
-    public void testLoadingDefault() throws Exception {
+    void testLoadingDefault() throws Exception {
         final StateBackend backend =
                 StateBackendLoader.fromApplicationOrConfigOrDefault(
                         null, TernaryBoolean.UNDEFINED, config(), cl, null);
         final CheckpointStorage storage =
                 CheckpointStorageLoader.load(null, null, backend, config(), cl, null);
-
-        assertTrue(backend instanceof HashMapStateBackend);
+        assertThat(backend instanceof HashMapStateBackend).isTrue();

Review Comment:
   ```suggestion
           assertThat(backend).isInstanceOf(HashMapStateBackend.class);
   ```
   could be simplified



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047215124


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateEmbeddedRocksDBStateBackendTest.java:
##########
@@ -33,22 +33,22 @@
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.state.TestTaskStateManager;
 
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
 
+import java.io.File;
 import java.io.IOException;
 
 /** Tests for {@link ChangelogStateBackend} delegating {@link EmbeddedRocksDBStateBackend}. */
 public class ChangelogDelegateEmbeddedRocksDBStateBackendTest

Review Comment:
   ```suggestion
   class ChangelogDelegateEmbeddedRocksDBStateBackendTest
   ```
   could be package private



##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateFileStateBackendTest.java:
##########
@@ -35,20 +35,20 @@
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
 
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
 
+import java.io.File;
 import java.io.IOException;
 
 /** Tests for {@link ChangelogStateBackend} delegating {@link FsStateBackend}. */
 public class ChangelogDelegateFileStateBackendTest extends FileStateBackendTest {

Review Comment:
   ```suggestion
   class ChangelogDelegateFileStateBackendTest extends FileStateBackendTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1094665507


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendMigrationTest.java:
##########
@@ -18,18 +18,24 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
-import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
 
 /**
  * Tests for the keyed state backend and operator state backend, as created by the {@link
  * FsStateBackend}.
  */
 public class FileStateBackendMigrationTest extends StateBackendMigrationTestBase<FsStateBackend> {
 
+    @Parameters
+    public static Collection<Object> modes() {
+        return Arrays.asList("just to please junit5 parameter test in base class");

Review Comment:
   I have changed the expression in the parameter list to enhanace reabability.
   
   ```
       @Parameters
       public static Collection<Object> modes() {
           // current test does not need to be parameterized
           return Collections.singletonList("no-use");
       }
   ```



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1581609186

   > Hi @jiexray I wonder whether you would like to continue working on this PR or not
   
   Thank you for you reminding. I have resolved all converasation, and I will rebase my pr this week.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047216705


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateStateTest.java:
##########
@@ -32,74 +32,77 @@
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 import org.apache.flink.util.IOUtils;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
 import java.util.List;
 import java.util.function.Supplier;
 
 import static org.apache.flink.state.changelog.ChangelogStateBackendTestUtils.DummyCheckpointingStorageAccess;
 import static org.apache.flink.state.changelog.ChangelogStateBackendTestUtils.createKeyedBackend;
-import static org.junit.Assert.assertSame;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** Tests for {@link ChangelogStateBackend} delegating state accesses. */
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
 public class ChangelogDelegateStateTest {

Review Comment:
   ```suggestion
   class ChangelogDelegateStateTest {
   ```
   could be package private



##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackendTest.java:
##########
@@ -38,22 +38,23 @@
 import org.apache.flink.runtime.state.ttl.mock.MockKeyedStateBackend.MockSnapshotSupplier;
 import org.apache.flink.runtime.state.ttl.mock.MockKeyedStateBackendBuilder;
 import org.apache.flink.state.changelog.ChangelogStateBackendTestUtils.DummyCheckpointingStorageAccess;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameter;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.concurrent.RunnableFuture;
 
 import static java.util.Collections.emptyList;
-import static org.junit.Assert.assertEquals;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** {@link ChangelogKeyedStateBackend} test. */
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
 public class ChangelogKeyedStateBackendTest {

Review Comment:
   ```suggestion
   class ChangelogKeyedStateBackendTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047222757


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendMigrationTest.java:
##########
@@ -24,28 +24,26 @@
 import org.apache.flink.runtime.state.StateBackendMigrationTestBase;
 import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
 import java.util.Arrays;
-import java.util.List;
+import java.util.Collection;
 import java.util.function.Supplier;
 
 /** Tests for the partitioned state part of {@link ChangelogStateBackend}. */
-@RunWith(Parameterized.class)
 public class ChangelogStateBackendMigrationTest

Review Comment:
   ```suggestion
   class ChangelogStateBackendMigrationTest
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1414118862

   @roncohen Thank you for you comments. I have fixed some problems in the commit https://github.com/apache/flink/pull/21410/commits/944bf2a8fe21628d279df006fbc417934a122381.
   
   However, I am still confused about the problem related to `TestLoggerExtension`. I have replied it just after your comment.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1094833168


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateStateTest.java:
##########
@@ -32,74 +32,77 @@
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 import org.apache.flink.util.IOUtils;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
 import java.util.List;
 import java.util.function.Supplier;
 
 import static org.apache.flink.state.changelog.ChangelogStateBackendTestUtils.DummyCheckpointingStorageAccess;
 import static org.apache.flink.state.changelog.ChangelogStateBackendTestUtils.createKeyedBackend;
-import static org.junit.Assert.assertSame;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** Tests for {@link ChangelogStateBackend} delegating state accesses. */
-@RunWith(Parameterized.class)
+@ExtendWith(ParameterizedTestExtension.class)
 public class ChangelogDelegateStateTest {

Review Comment:
   `@ExtendWith(ParameterizedTestExtension.class)` cannot use package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1581650923

   @flinkbot  run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] masteryhx commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "masteryhx (via GitHub)" <gi...@apache.org>.
masteryhx commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1306881957


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateEmbeddedRocksDBStateBackendTest.java:
##########
@@ -33,22 +33,21 @@
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.state.TestTaskStateManager;
 
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
 
+import java.io.File;
 import java.io.IOException;
 
 /** Tests for {@link ChangelogStateBackend} delegating {@link EmbeddedRocksDBStateBackend}. */
-public class ChangelogDelegateEmbeddedRocksDBStateBackendTest
-        extends EmbeddedRocksDBStateBackendTest {
+class ChangelogDelegateEmbeddedRocksDBStateBackendTest extends EmbeddedRocksDBStateBackendTest {
 
-    @Rule public final TemporaryFolder temp = new TemporaryFolder();
+    @TempDir static File tmPath;

Review Comment:
   ```suggestion
       @TempDir static File tempFile;
   ```



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java:
##########
@@ -83,6 +84,16 @@ public UV setValue(UV value) {
                 }
                 return oldValue;
             }
+
+            @Override
+            public boolean equals(Object o) {

Review Comment:
   This fix should not be related to Junit5 migration, right?
   It's better to seprate them into 2 commits.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1044638321


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -1080,23 +1088,41 @@ public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throw
 
             // on the second restore, since the custom serializer will be used for
             // deserialization, we expect the deliberate failure to be thrown
-            expectedException.expect(
-                    anyOf(
-                            isA(ExpectedKryoTestException.class),
-                            Matchers.<Throwable>hasProperty(
-                                    "cause", isA(ExpectedKryoTestException.class))));
-
-            // state backends that eagerly deserializes (such as the memory state backend) will fail
-            // here
-            backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
-
-            state =
-                    backend.getPartitionedState(
-                            VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-
-            backend.setCurrentKey(1);
-            // state backends that lazily deserializes (such as RocksDB) will fail here
-            state.value();
+            assertThatThrownBy(
+                            () -> {
+                                // state backends that eagerly deserializes (such as the memory
+                                // state backend) will fail
+                                // here
+                                CheckpointableKeyedStateBackend<Integer> restoreBackend = null;
+                                try {
+                                    restoreBackend =
+                                            restoreKeyedBackend(
+                                                    IntSerializer.INSTANCE, snapshot2, env);
+
+                                    ValueState<TestPojo> restoreState =
+                                            restoreBackend.getPartitionedState(
+                                                    VoidNamespace.INSTANCE,
+                                                    VoidNamespaceSerializer.INSTANCE,
+                                                    new ValueStateDescriptor<>("id", pojoType));
+
+                                    restoreBackend.setCurrentKey(1);
+                                    // state backends that lazily deserializes (such as RocksDB)
+                                    // will fail here
+                                    restoreState.value();
+                                } finally {
+                                    if (restoreBackend != null) {
+                                        restoreBackend.dispose();
+                                    }
+                                }
+                            })
+                    .satisfiesAnyOf(
+                            actual ->
+                                    assertThat(actual)
+                                            .isInstanceOf(ExpectedKryoTestException.class),
+                            actual ->
+                                    assertThat(actual)
+                                            .hasFieldOrProperty("cause")

Review Comment:
   The same condition as mentioned in the former question.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] Myasuka commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "Myasuka (via GitHub)" <gi...@apache.org>.
Myasuka commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1518927059

   @jiexray Please click `Resolve converasation` button if you think you already addressed the comment.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047221724


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendLoadingTest.java:
##########
@@ -229,14 +234,14 @@ public void testLoadingRocksDBStateBackend() throws Exception {
     }
 
     @Test
-    public void testEnableChangelogStateBackendInStreamExecutionEnvironment() throws Exception {
+    void testEnableChangelogStateBackendInStreamExecutionEnvironment() throws Exception {
         StreamExecutionEnvironment env = getEnvironment();
         assertStateBackendAndChangelogInEnvironmentAndStreamGraphAndJobGraph(
                 env, TernaryBoolean.UNDEFINED, null);
 
         // set back and force
         env.setStateBackend(new MemoryStateBackend());
-        assertTrue(env.getStateBackend() instanceof MemoryStateBackend);
+        assertThat(env.getStateBackend() instanceof MemoryStateBackend).isTrue();

Review Comment:
   ```suggestion
           assertThat(env.getStateBackend()).isInstanceOf(MemoryStateBackend.class);
   ```
   could be simplified



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047217368


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogListStateTest.java:
##########
@@ -38,50 +38,48 @@
 
 import static java.util.Collections.emptyList;
 import static java.util.Collections.singletonList;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** ChangelogListState Test. */
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class ChangelogListStateTest {

Review Comment:
   ```suggestion
   class ChangelogListStateTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047227202


##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java:
##########
@@ -529,7 +528,7 @@ public void testDisposeDeletesAllDirectories() throws Exception {
             state.update("Hello");
 
             // more than just the root directory
-            assertTrue(allFilesInDbDir.size() > 1);
+            assertThat(allFilesInDbDir.size() > 1).isTrue();

Review Comment:
   ```suggestion
               assertThat(allFilesInDbDir).hasSizeGreaterThan(1);
   ```
   could be simplified



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1094832054


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -152,17 +137,16 @@
  * <p>NOTE: Please ensure to close and dispose any created keyed state backend in tests.
  */
 @SuppressWarnings("serial")
-public abstract class StateBackendTestBase<B extends AbstractStateBackend> extends TestLogger {
+@ExtendWith({ParameterizedTestExtension.class, TestLoggerExtension.class})

Review Comment:
   Thank you for your advice.
   
   However, I find that `META-INF/services/org.junit.jupiter.api.extension.Extension` already exists in `flink-runtime/src/test/resources/` directory, and the content in the file is:
   ```
   org.apache.flink.util.TestLoggerExtension
   ```
   
   I think the `TestLoggerExtension.class` I used here is the `org.apache.flink.util.TestLoggerExtension` in the `flink-runtime/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension` 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1043363450


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -98,14 +100,14 @@ protected boolean supportsKeySerializerCheck() {
         return true;
     }
 
-    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    @TempDir public static File tempFolder;

Review Comment:
   ```suggestion
       @TempDir protected static File tempFolder;
   ```
   isn't protected enough here?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1578497338

   Hi @jiexray I wonder whether you would like to continue working on this PR or not


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1520059668

   > @jiexray Please click `Resolve converasation` button if you think you already addressed the comment.
   
   OK, I will re-check all converasation, and click `Resolve converasation`.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1692697385

   @flinkbot  run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1043409381


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -152,16 +137,15 @@
  * <p>NOTE: Please ensure to close and dispose any created keyed state backend in tests.
  */
 @SuppressWarnings("serial")
+@ExtendWith(ParameterizedTestExtension.class)
 public abstract class StateBackendTestBase<B extends AbstractStateBackend> extends TestLogger {
 
-    @Rule public final ExpectedException expectedException = ExpectedException.none();
-
-    @Before
+    @BeforeEach
     public void before() throws Exception {

Review Comment:
   ```suggestion
       void before() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -276,7 +260,7 @@ protected <K> CheckpointableKeyedStateBackend<K> restoreKeyedBackend(
                         new CloseableRegistry());
     }
 
-    @Test
+    @TestTemplate
     public void testEnableStateLatencyTracking() throws Exception {

Review Comment:
   ```suggestion
       void testEnableStateLatencyTracking() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -152,16 +137,15 @@
  * <p>NOTE: Please ensure to close and dispose any created keyed state backend in tests.
  */
 @SuppressWarnings("serial")
+@ExtendWith(ParameterizedTestExtension.class)
 public abstract class StateBackendTestBase<B extends AbstractStateBackend> extends TestLogger {
 
-    @Rule public final ExpectedException expectedException = ExpectedException.none();
-
-    @Before
+    @BeforeEach
     public void before() throws Exception {
         env = buildMockEnv();
     }
 
-    @After
+    @AfterEach
     public void after() {

Review Comment:
   ```suggestion
       void after() {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -303,34 +287,35 @@ public void testEnableStateLatencyTracking() throws Exception {
                             ? ((TestableKeyedStateBackend<Integer>) keyedStateBackend)
                                     .getDelegatedKeyedStateBackend(true)
                             : keyedStateBackend;
-            Assert.assertTrue(
-                    ((AbstractKeyedStateBackend<Integer>) nested)
-                            .getLatencyTrackingStateConfig()
-                            .isEnabled());
+            assertThat(
+                            ((AbstractKeyedStateBackend<Integer>) nested)
+                                    .getLatencyTrackingStateConfig()
+                                    .isEnabled())
+                    .isTrue();
         } finally {
             IOUtils.closeQuietly(keyedStateBackend);
             keyedStateBackend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testIsSafeToReuseState() throws Exception {
         CheckpointableKeyedStateBackend<Integer> backend =
                 createKeyedBackend(IntSerializer.INSTANCE);
         try {
-            Assert.assertEquals(isSafeToReuseKVState(), backend.isSafeToReuseKVState());
+            assertThat(backend.isSafeToReuseKVState()).isEqualTo(isSafeToReuseKVState());
         } finally {
             IOUtils.closeQuietly(backend);
             backend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testKeyGroupedInternalPriorityQueue() throws Exception {

Review Comment:
   ```suggestion
       void testKeyGroupedInternalPriorityQueue() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -644,14 +629,14 @@ public void testBackendUsesRegisteredKryoDefaultSerializerUsingGetOrCreate() thr
                 }
             }
 
-            assertTrue("Didn't see the expected Kryo exception.", numExceptions > 0);
+            assertThat(numExceptions > 0).as("Didn't see the expected Kryo exception.").isTrue();
         } finally {
             IOUtils.closeQuietly(backend);
             backend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testBackendUsesRegisteredKryoSerializer() throws Exception {

Review Comment:
   ```suggestion
       void testBackendUsesRegisteredKryoSerializer() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -446,19 +429,19 @@ public void testGetKeys() throws Exception {
                 for (int expectedKey = namespace1ElementsNum;
                         expectedKey < namespace1ElementsNum + namespace2ElementsNum;
                         expectedKey++) {
-                    assertTrue(actualIterator.hasNext());
-                    assertEquals(expectedKey, actualIterator.nextInt());
+                    assertThat(actualIterator.hasNext()).isTrue();
+                    assertThat(actualIterator.nextInt()).isEqualTo(expectedKey);
                 }
 
-                assertFalse(actualIterator.hasNext());
+                assertThat(actualIterator.hasNext()).isFalse();
             }
         } finally {
             IOUtils.closeQuietly(backend);
             backend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testGetKeysAndNamespaces() throws Exception {

Review Comment:
   ```suggestion
       void testGetKeysAndNamespaces() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -1080,23 +1088,41 @@ public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throw
 
             // on the second restore, since the custom serializer will be used for
             // deserialization, we expect the deliberate failure to be thrown
-            expectedException.expect(
-                    anyOf(
-                            isA(ExpectedKryoTestException.class),
-                            Matchers.<Throwable>hasProperty(
-                                    "cause", isA(ExpectedKryoTestException.class))));
-
-            // state backends that eagerly deserializes (such as the memory state backend) will fail
-            // here
-            backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
-
-            state =
-                    backend.getPartitionedState(
-                            VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-
-            backend.setCurrentKey(1);
-            // state backends that lazily deserializes (such as RocksDB) will fail here
-            state.value();
+            assertThatThrownBy(
+                            () -> {
+                                // state backends that eagerly deserializes (such as the memory
+                                // state backend) will fail
+                                // here
+                                CheckpointableKeyedStateBackend<Integer> restoreBackend = null;
+                                try {
+                                    restoreBackend =
+                                            restoreKeyedBackend(
+                                                    IntSerializer.INSTANCE, snapshot2, env);
+
+                                    ValueState<TestPojo> restoreState =
+                                            restoreBackend.getPartitionedState(
+                                                    VoidNamespace.INSTANCE,
+                                                    VoidNamespaceSerializer.INSTANCE,
+                                                    new ValueStateDescriptor<>("id", pojoType));
+
+                                    restoreBackend.setCurrentKey(1);
+                                    // state backends that lazily deserializes (such as RocksDB)
+                                    // will fail here
+                                    restoreState.value();
+                                } finally {
+                                    if (restoreBackend != null) {
+                                        restoreBackend.dispose();
+                                    }
+                                }
+                            })
+                    .satisfiesAnyOf(
+                            actual ->
+                                    assertThat(actual)
+                                            .isInstanceOf(ExpectedKryoTestException.class),
+                            actual ->
+                                    assertThat(actual)
+                                            .hasFieldOrProperty("cause")

Review Comment:
   Could you please clarify why should we check it?
   In my opinion `Throwable` has `cause` field and this check will always pass for anything extending `Throwable`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -996,7 +1004,7 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E
      *
      * @throws Exception expects {@link ExpectedKryoTestException} to be thrown.
      */
-    @Test
+    @TestTemplate
     public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throws Exception {

Review Comment:
   ```suggestion
       void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -570,14 +555,14 @@ public void testBackendUsesRegisteredKryoDefaultSerializer() throws Exception {
                 }
             }
 
-            assertTrue("Didn't see the expected Kryo exception.", numExceptions > 0);
+            assertThat(numExceptions > 0).as("Didn't see the expected Kryo exception.").isTrue();
         } finally {
             IOUtils.closeQuietly(backend);
             backend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     @SuppressWarnings("unchecked")
     public void testBackendUsesRegisteredKryoDefaultSerializerUsingGetOrCreate() throws Exception {

Review Comment:
   ```suggestion
       void testBackendUsesRegisteredKryoDefaultSerializerUsingGetOrCreate() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -355,43 +340,41 @@ public void testKeyGroupedInternalPriorityQueue(boolean addAll) throws Exception
             if (addAll) {
                 priorityQueue.addAll(asList(elements));
             } else {
-                assertTrue(priorityQueue.add(elements[0]));
-                assertTrue(priorityQueue.add(elements[1]));
-                assertFalse(priorityQueue.add(elements[2]));
-                assertFalse(priorityQueue.add(elements[3]));
-                assertFalse(priorityQueue.add(elements[4]));
+                assertThat(priorityQueue.add(elements[0])).isTrue();
+                assertThat(priorityQueue.add(elements[1])).isTrue();
+                assertThat(priorityQueue.add(elements[2])).isFalse();
+                assertThat(priorityQueue.add(elements[3])).isFalse();
+                assertThat(priorityQueue.add(elements[4])).isFalse();
             }
-            assertFalse(priorityQueue.isEmpty());
-            assertThat(
-                    priorityQueue.getSubsetForKeyGroup(1),
-                    containsInAnyOrder(elementA42, elementA44));
-            assertThat(
-                    priorityQueue.getSubsetForKeyGroup(8),
-                    containsInAnyOrder(elementB1, elementB3));
+            assertThat(priorityQueue.isEmpty()).isFalse();
+            assertThat(priorityQueue.getSubsetForKeyGroup(1))
+                    .containsExactlyInAnyOrder(elementA42, elementA44);
+            assertThat(priorityQueue.getSubsetForKeyGroup(8))
+                    .containsExactlyInAnyOrder(elementB1, elementB3);
 
-            assertThat(priorityQueue.peek(), equalTo(elementB1));
-            assertThat(priorityQueue.poll(), equalTo(elementB1));
-            assertThat(priorityQueue.peek(), equalTo(elementB3));
+            assertThat(priorityQueue.peek()).isEqualTo(elementB1);
+            assertThat(priorityQueue.poll()).isEqualTo(elementB1);
+            assertThat(priorityQueue.peek()).isEqualTo(elementB3);
 
             List<TestType> actualList = new ArrayList<>();
             try (CloseableIterator<TestType> iterator = priorityQueue.iterator()) {
                 iterator.forEachRemaining(actualList::add);
             }
 
-            assertThat(actualList, containsInAnyOrder(elementB3, elementA42, elementA44));
+            assertThat(actualList).containsExactlyInAnyOrder(elementB3, elementA42, elementA44);
 
-            assertEquals(3, priorityQueue.size());
+            assertThat(priorityQueue.size()).isEqualTo(3);
 
-            assertFalse(priorityQueue.remove(elementB1));
-            assertTrue(priorityQueue.remove(elementB3));
-            assertThat(priorityQueue.peek(), equalTo(elementA42));
+            assertThat(priorityQueue.remove(elementB1)).isFalse();
+            assertThat(priorityQueue.remove(elementB3)).isTrue();
+            assertThat(priorityQueue.peek()).isEqualTo(elementA42);
         } finally {
             IOUtils.closeQuietly(backend);
             backend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testGetKeys() throws Exception {

Review Comment:
   ```suggestion
       void testGetKeys() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -303,34 +287,35 @@ public void testEnableStateLatencyTracking() throws Exception {
                             ? ((TestableKeyedStateBackend<Integer>) keyedStateBackend)
                                     .getDelegatedKeyedStateBackend(true)
                             : keyedStateBackend;
-            Assert.assertTrue(
-                    ((AbstractKeyedStateBackend<Integer>) nested)
-                            .getLatencyTrackingStateConfig()
-                            .isEnabled());
+            assertThat(
+                            ((AbstractKeyedStateBackend<Integer>) nested)
+                                    .getLatencyTrackingStateConfig()
+                                    .isEnabled())
+                    .isTrue();
         } finally {
             IOUtils.closeQuietly(keyedStateBackend);
             keyedStateBackend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testIsSafeToReuseState() throws Exception {

Review Comment:
   ```suggestion
       void testIsSafeToReuseState() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -506,7 +491,7 @@ public void testGetKeysAndNamespaces() throws Exception {
         }
     }
 
-    @Test
+    @TestTemplate
     @SuppressWarnings("unchecked")
     public void testBackendUsesRegisteredKryoDefaultSerializer() throws Exception {

Review Comment:
   ```suggestion
       void testBackendUsesRegisteredKryoDefaultSerializer() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -303,34 +287,35 @@ public void testEnableStateLatencyTracking() throws Exception {
                             ? ((TestableKeyedStateBackend<Integer>) keyedStateBackend)
                                     .getDelegatedKeyedStateBackend(true)
                             : keyedStateBackend;
-            Assert.assertTrue(
-                    ((AbstractKeyedStateBackend<Integer>) nested)
-                            .getLatencyTrackingStateConfig()
-                            .isEnabled());
+            assertThat(
+                            ((AbstractKeyedStateBackend<Integer>) nested)
+                                    .getLatencyTrackingStateConfig()
+                                    .isEnabled())
+                    .isTrue();
         } finally {
             IOUtils.closeQuietly(keyedStateBackend);
             keyedStateBackend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testIsSafeToReuseState() throws Exception {
         CheckpointableKeyedStateBackend<Integer> backend =
                 createKeyedBackend(IntSerializer.INSTANCE);
         try {
-            Assert.assertEquals(isSafeToReuseKVState(), backend.isSafeToReuseKVState());
+            assertThat(backend.isSafeToReuseKVState()).isEqualTo(isSafeToReuseKVState());
         } finally {
             IOUtils.closeQuietly(backend);
             backend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     public void testKeyGroupedInternalPriorityQueue() throws Exception {
         testKeyGroupedInternalPriorityQueue(false);
     }
 
-    @Test
+    @TestTemplate
     public void testKeyGroupedInternalPriorityQueueAddAll() throws Exception {

Review Comment:
   ```suggestion
       void testKeyGroupedInternalPriorityQueueAddAll() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -872,7 +860,7 @@ public void testKryoRegisteringRestoreResilienceWithRegisteredType() throws Exce
      *
      * @throws Exception expects {@link ExpectedKryoTestException} to be thrown.
      */
-    @Test
+    @TestTemplate
     @SuppressWarnings("unchecked")
     public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws Exception {

Review Comment:
   ```suggestion
       void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -152,16 +137,15 @@
  * <p>NOTE: Please ensure to close and dispose any created keyed state backend in tests.
  */
 @SuppressWarnings("serial")
+@ExtendWith(ParameterizedTestExtension.class)
 public abstract class StateBackendTestBase<B extends AbstractStateBackend> extends TestLogger {

Review Comment:
   `TestLogger` is a JUnit4 rule/extension
   Here should be used `META-INF/services/org.junit.jupiter.api.extension.Extension`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -959,24 +947,44 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E
 
             // on the second restore, since the custom serializer will be used for
             // deserialization, we expect the deliberate failure to be thrown
-            expectedException.expect(
-                    anyOf(
-                            isA(ExpectedKryoTestException.class),
-                            Matchers.<Throwable>hasProperty(
-                                    "cause", isA(ExpectedKryoTestException.class))));
-
-            // state backends that eagerly deserializes (such as the memory state backend) will fail
-            // here
-            backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
-
-            state =
-                    backend.getPartitionedState(
-                            VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-
-            backend.setCurrentKey(1);
-            // state backends that lazily deserializes (such as RocksDB) will fail here
-            state.value();
-
+            assertThatThrownBy(
+                            () -> {
+                                // state backends that eagerly deserializes (such as the memory
+                                // state backend) will fail
+                                // here
+                                CheckpointableKeyedStateBackend<Integer> restoreBackend = null;
+                                try {
+                                    restoreBackend =
+                                            restoreKeyedBackend(
+                                                    IntSerializer.INSTANCE, snapshot2, env);
+
+                                    ValueState<TestPojo> restoreState =
+                                            restoreBackend.getPartitionedState(
+                                                    VoidNamespace.INSTANCE,
+                                                    VoidNamespaceSerializer.INSTANCE,
+                                                    new ValueStateDescriptor<>("id", pojoType));
+
+                                    restoreBackend.setCurrentKey(1);
+                                    // state backends that lazily deserializes (such as RocksDB)
+                                    // will fail here
+                                    restoreState.value();
+
+                                    restoreBackend.dispose();
+                                } finally {
+                                    if (restoreBackend != null) {
+                                        IOUtils.closeQuietly(restoreBackend);
+                                        restoreBackend.dispose();
+                                    }
+                                }
+                            })
+                    .satisfiesAnyOf(
+                            actual ->
+                                    assertThat(actual)
+                                            .isInstanceOf(ExpectedKryoTestException.class),
+                            actual ->
+                                    assertThat(actual)
+                                            .hasFieldOrProperty("cause")

Review Comment:
   Could you please clarify why should we check it?
   In my opinion `Throwable` has `cause` field and this check will always pass for anything extending `Throwable`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -711,14 +697,14 @@ public void testBackendUsesRegisteredKryoSerializer() throws Exception {
                 }
             }
 
-            assertTrue("Didn't see the expected Kryo exception.", numExceptions > 0);
+            assertThat(numExceptions > 0).as("Didn't see the expected Kryo exception.").isTrue();
         } finally {
             IOUtils.closeQuietly(backend);
             backend.dispose();
         }
     }
 
-    @Test
+    @TestTemplate
     @SuppressWarnings("unchecked")
     public void testBackendUsesRegisteredKryoSerializerUsingGetOrCreate() throws Exception {

Review Comment:
   ```suggestion
       void testBackendUsesRegisteredKryoSerializerUsingGetOrCreate() throws Exception {
   ```
   could be package private



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -796,15 +783,16 @@ public void testBackendUsesRegisteredKryoSerializerUsingGetOrCreate() throws Exc
      * <p>This test should not fail, because de- / serialization of the state should not be
      * performed with Kryo's default {@link com.esotericsoftware.kryo.serializers.FieldSerializer}.
      */
-    @Test
+    @TestTemplate
     public void testKryoRegisteringRestoreResilienceWithRegisteredType() throws Exception {

Review Comment:
   ```suggestion
       void testKryoRegisteringRestoreResilienceWithRegisteredType() throws Exception {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] Myasuka commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
Myasuka commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1045708784


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendMigrationTest.java:
##########
@@ -18,18 +18,24 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
 
-import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
 
 /**
  * Tests for the keyed state backend and operator state backend, as created by the {@link
  * FsStateBackend}.
  */
 public class FileStateBackendMigrationTest extends StateBackendMigrationTestBase<FsStateBackend> {
 
+    @Parameters
+    public static Collection<Object> modes() {
+        return Arrays.asList("just to please junit5 parameter test in base class");

Review Comment:
   I don't think this sentence could be understood well.



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java:
##########
@@ -61,17 +62,7 @@
 
     private Map.Entry<UK, UV> loggingMapEntry(
             Map.Entry<UK, UV> entry, KvStateChangeLogger<Map<UK, UV>, N> changeLogger, N ns) {
-        return new Map.Entry<UK, UV>() {
-            @Override
-            public UK getKey() {
-                return entry.getKey();
-            }
-
-            @Override
-            public UV getValue() {
-                return entry.getValue();
-            }
-
+        return new AbstractMap.SimpleEntry<UK, UV>(entry.getKey(), entry.getValue()) {

Review Comment:
   I don't this change has relationship with junit5 refactor.



##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java:
##########
@@ -148,13 +145,16 @@ public static List<Object[]> modes() {
     private final RocksDBResourceContainer optionsContainer = new RocksDBResourceContainer();
 
     public void prepareRocksDB() throws Exception {
-        String dbPath = new File(TEMP_FOLDER.newFolder(), DB_INSTANCE_DIR_STRING).getAbsolutePath();
+        File dbPath = new File(tmpDbPath, DB_INSTANCE_DIR_STRING);
+        if (!dbPath.exists()) {
+            dbPath.mkdirs();

Review Comment:
   Why we need this after switching junit5?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047209085


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java:
##########
@@ -74,7 +74,8 @@
  * are either compatible or requiring state migration after restoring the state backends.
  */
 @SuppressWarnings("serial")
-public abstract class StateBackendMigrationTestBase<B extends StateBackend> extends TestLogger {
+@ExtendWith({ParameterizedTestExtension.class, TestLoggerExtension.class})

Review Comment:
   No, I mean not `TestLoggerExtension`
   Using the `META-INF/services/org.junit.jupiter.api.extension.Extension` is the better approach here.
   for more details have a look at
   https://github.com/apache/flink/commit/ecf67c67dbcf633c4533f8cb719f32512c19d0ce#diff-d96a3b8d09f526305595964a0010cf113e59999006534653bf45b9971b66241f



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047227892


##########
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java:
##########
@@ -539,11 +538,11 @@ public void testDisposeDeletesAllDirectories() throws Exception {
                         new File(dbPath), new AcceptAllFilter(), new AcceptAllFilter());
 
         // just the root directory left
-        assertEquals(1, allFilesInDbDir.size());
+        assertThat(allFilesInDbDir.size()).isEqualTo(1);

Review Comment:
   ```suggestion
          assertThat(allFilesInDbDir).hasSize(1);
   ```
   could be simplified



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047215604


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateHashMapInMemoryTest.java:
##########
@@ -22,17 +22,12 @@
 import org.apache.flink.runtime.state.TestTaskStateManager;
 import org.apache.flink.runtime.state.changelog.inmemory.InMemoryStateChangelogStorage;
 
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
 /**
  * Tests for {@link ChangelogStateBackend} using {@link InMemoryStateChangelogStorage} and
  * delegating {@link HashMapStateBackendTest}.
  */
 public class ChangelogDelegateHashMapInMemoryTest extends ChangelogDelegateHashMapTest {

Review Comment:
   ```suggestion
   class ChangelogDelegateHashMapInMemoryTest extends ChangelogDelegateHashMapTest {
   ```
   could be package private



##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateHashMapTest.java:
##########
@@ -32,19 +32,19 @@
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.state.TestTaskStateManager;
 
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
 
+import java.io.File;
 import java.io.IOException;
 
 /** Tests for {@link ChangelogStateBackend} delegating {@link HashMapStateBackendTest}. */
 public class ChangelogDelegateHashMapTest extends HashMapStateBackendTest {

Review Comment:
   ```suggestion
   class ChangelogDelegateHashMapTest extends HashMapStateBackendTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047216223


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogDelegateMemoryStateBackendTest.java:
##########
@@ -35,20 +35,20 @@
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
 
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.io.TempDir;
 
+import java.io.File;
 import java.io.IOException;
 
 /** Tests for {@link ChangelogStateBackend} delegating {@link MemoryStateBackend}. */
 public class ChangelogDelegateMemoryStateBackendTest extends MemoryStateBackendTest {

Review Comment:
   ```suggestion
   class ChangelogDelegateMemoryStateBackendTest extends MemoryStateBackendTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047219953


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendLoadingTest.java:
##########
@@ -58,75 +58,73 @@
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TernaryBoolean;
 
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
 
 import javax.annotation.Nonnull;
 
 import java.util.Collection;
 import java.util.Collections;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.junit.jupiter.api.Assertions.assertSame;
 
 /** Verify Changelog StateBackend is properly loaded. */
 public class ChangelogStateBackendLoadingTest {

Review Comment:
   ```suggestion
   class ChangelogStateBackendLoadingTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] snuyanzin commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
snuyanzin commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1047217629


##########
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogMapStateTest.java:
##########
@@ -36,80 +36,80 @@
 
 import static java.util.Collections.emptyMap;
 import static java.util.Collections.singletonMap;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** ChangelogMapState Test. */
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class ChangelogMapStateTest {

Review Comment:
   ```suggestion
   class ChangelogMapStateTest {
   ```
   could be package private



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by "jiexray (via GitHub)" <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1414028883

   @Myasuka Thank you for your comments. I have fixed them in a new commit. https://github.com/apache/flink/pull/21410/commits/bf87b002395acf1c6d0a606f6699544087769bd7


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
jiexray commented on PR #21410:
URL: https://github.com/apache/flink/pull/21410#issuecomment-1329197680

   Hello @PatrickRen @ruanhang1993, I am trying to migrate junit tests in flink-statebackend-changelog. Some base tests for changelog are in flink-runtime. The two base test classes are `StateBackendTestBase` and `StateBackendMigrationTestBase`. Could you help me to resolve the conflict.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1044637743


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -959,24 +947,44 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E
 
             // on the second restore, since the custom serializer will be used for
             // deserialization, we expect the deliberate failure to be thrown
-            expectedException.expect(
-                    anyOf(
-                            isA(ExpectedKryoTestException.class),
-                            Matchers.<Throwable>hasProperty(
-                                    "cause", isA(ExpectedKryoTestException.class))));
-
-            // state backends that eagerly deserializes (such as the memory state backend) will fail
-            // here
-            backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
-
-            state =
-                    backend.getPartitionedState(
-                            VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-
-            backend.setCurrentKey(1);
-            // state backends that lazily deserializes (such as RocksDB) will fail here
-            state.value();
-
+            assertThatThrownBy(
+                            () -> {
+                                // state backends that eagerly deserializes (such as the memory
+                                // state backend) will fail
+                                // here
+                                CheckpointableKeyedStateBackend<Integer> restoreBackend = null;
+                                try {
+                                    restoreBackend =
+                                            restoreKeyedBackend(
+                                                    IntSerializer.INSTANCE, snapshot2, env);
+
+                                    ValueState<TestPojo> restoreState =
+                                            restoreBackend.getPartitionedState(
+                                                    VoidNamespace.INSTANCE,
+                                                    VoidNamespaceSerializer.INSTANCE,
+                                                    new ValueStateDescriptor<>("id", pojoType));
+
+                                    restoreBackend.setCurrentKey(1);
+                                    // state backends that lazily deserializes (such as RocksDB)
+                                    // will fail here
+                                    restoreState.value();
+
+                                    restoreBackend.dispose();
+                                } finally {
+                                    if (restoreBackend != null) {
+                                        IOUtils.closeQuietly(restoreBackend);
+                                        restoreBackend.dispose();
+                                    }
+                                }
+                            })
+                    .satisfiesAnyOf(
+                            actual ->
+                                    assertThat(actual)
+                                            .isInstanceOf(ExpectedKryoTestException.class),
+                            actual ->
+                                    assertThat(actual)
+                                            .hasFieldOrProperty("cause")

Review Comment:
   This is because some catch blocks wrap the inner exception, i.e., `ExpectedKryoTestException` (thrown by `CustomKryoTestSerializer`).
   
   For example, `HeapKeyedStateBackendBuilder#restoreState` wrap `ExpectedKryoTestException` in `BackendBuildingException`.
   
   Thus, we need to check the `cause` field of the thrown exception.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] jiexray commented on a diff in pull request #21410: [FLINK-29776][flink-statebackend-changelog][JUnit5 Migration]Module: flink-statebackend-changelog.

Posted by GitBox <gi...@apache.org>.
jiexray commented on code in PR #21410:
URL: https://github.com/apache/flink/pull/21410#discussion_r1044988468


##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java:
##########
@@ -959,24 +947,44 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E
 
             // on the second restore, since the custom serializer will be used for
             // deserialization, we expect the deliberate failure to be thrown
-            expectedException.expect(
-                    anyOf(
-                            isA(ExpectedKryoTestException.class),
-                            Matchers.<Throwable>hasProperty(
-                                    "cause", isA(ExpectedKryoTestException.class))));
-
-            // state backends that eagerly deserializes (such as the memory state backend) will fail
-            // here
-            backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
-
-            state =
-                    backend.getPartitionedState(
-                            VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-
-            backend.setCurrentKey(1);
-            // state backends that lazily deserializes (such as RocksDB) will fail here
-            state.value();
-
+            assertThatThrownBy(
+                            () -> {
+                                // state backends that eagerly deserializes (such as the memory
+                                // state backend) will fail
+                                // here
+                                CheckpointableKeyedStateBackend<Integer> restoreBackend = null;
+                                try {
+                                    restoreBackend =
+                                            restoreKeyedBackend(
+                                                    IntSerializer.INSTANCE, snapshot2, env);
+
+                                    ValueState<TestPojo> restoreState =
+                                            restoreBackend.getPartitionedState(
+                                                    VoidNamespace.INSTANCE,
+                                                    VoidNamespaceSerializer.INSTANCE,
+                                                    new ValueStateDescriptor<>("id", pojoType));
+
+                                    restoreBackend.setCurrentKey(1);
+                                    // state backends that lazily deserializes (such as RocksDB)
+                                    // will fail here
+                                    restoreState.value();
+
+                                    restoreBackend.dispose();
+                                } finally {
+                                    if (restoreBackend != null) {
+                                        IOUtils.closeQuietly(restoreBackend);
+                                        restoreBackend.dispose();
+                                    }
+                                }
+                            })
+                    .satisfiesAnyOf(
+                            actual ->
+                                    assertThat(actual)
+                                            .isInstanceOf(ExpectedKryoTestException.class),
+                            actual ->
+                                    assertThat(actual)
+                                            .hasFieldOrProperty("cause")

Review Comment:
   Yes, I think `hasCauseInstanceOf()` is sufficient for the assertion, and `.hasFieldOrProperty("cause")` can be removed. The assertion code will become:
   
   ```
   .satisfiesAnyOf(
           actual ->
                   assertThat(actual)
                           .isInstanceOf(ExpectedKryoTestException.class),
           actual ->
                   assertThat(actual)
                           .hasCauseInstanceOf(ExpectedKryoTestException.class));
   ```



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org