Zakelly commented on code in PR #25409:
URL: https://github.com/apache/flink/pull/25409#discussion_r1777959585


##########
flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java:
##########
@@ -444,7 +447,7 @@ public void testKeyedCEPOperatorNFAUpdateWithRocksDB() 
throws Exception {
             operator = CepOperatorTestUtilities.getKeyedCepOperator(true, new 
SimpleNFAFactory());
             harness = CepOperatorTestUtilities.getCepTestHarness(operator);
 
-            rocksDBStateBackend = new RocksDBStateBackend(new 
MemoryStateBackend());
+            rocksDBStateBackend = new 
EmbeddedRocksDBStateBackend(TernaryBoolean.TRUE);

Review Comment:
   Should we use the constructor without any parameter, `new 
EmbeddedRocksDBStateBackend();` ?
   
   Same for any other places



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendMigrationTest.java:
##########
@@ -27,14 +28,20 @@
 
 /**
  * Tests for the keyed state backend and operator state backend, as created by 
the {@link
- * FsStateBackend}.
+ * HashMapStateBackend}.
  */
 @ExtendWith(NoOpTestExtension.class)
-public class FileStateBackendMigrationTest extends 
StateBackendMigrationTestBase<FsStateBackend> {
+public class FileStateBackendMigrationTest

Review Comment:
   Rename this test?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java:
##########
@@ -90,9 +82,7 @@ public class StateBackendLoader {
      * factory is instantiated (via its zero-argument constructor) and its 
{@link
      * StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)} 
method is called.
      *
-     * <p>Recognized shortcut names are '{@value 
StateBackendLoader#HASHMAP_STATE_BACKEND_NAME}',
-     * '{@value StateBackendLoader#ROCKSDB_STATE_BACKEND_NAME}' and '{@value
-     * StateBackendLoader#MEMORY_STATE_BACKEND_NAME}' (Deprecated)
+     * <p>Recognized shortcut names are '{@value 
StateBackendLoader#HASHMAP_STATE_BACKEND_NAME}'

Review Comment:
   Don't remove `StateBackendLoader#ROCKSDB_STATE_BACKEND_NAME` here



##########
flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/StreamOperatorSnapshotRestoreTest.java:
##########
@@ -156,16 +155,16 @@ private void 
testOperatorStatesSnapshotRestoreInternal(final int mode) throws Ex
         // 
-------------------------------------------------------------------------- 
snapshot
 
         StateBackend stateBackend;
-        FsStateBackend fsstateBackend = createStateBackendInternal();
+        HashMapStateBackend hashMapStateBackend = createStateBackendInternal();
         switch (stateBackendEnum) {
             case FILE:
-                stateBackend = fsstateBackend;
+                stateBackend = hashMapStateBackend;

Review Comment:
   Should add ` testHarness.setCheckpointStorage(xxx);` in line 212



##########
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java:
##########
@@ -86,153 +83,49 @@ void testApplicationDefinedHasPrecedence() throws 
Exception {
     }
 
     // ------------------------------------------------------------------------
-    //  Memory State Backend
+    //  HashMap State Backend
     // ------------------------------------------------------------------------
 
-    /** Validates loading a memory state backend from the cluster 
configuration. */
+    /** Validates loading a HashMapStateBackend from the cluster 
configuration. */
     @Test
-    void testLoadMemoryStateBackendNoParameters() throws Exception {
+    void testLoadHashMapStateBackendNoParameters() throws Exception {
         // we configure with the explicit string (rather than
         // AbstractStateBackend#X_STATE_BACKEND_NAME)
         // to guard against config-breaking changes of the name
 
-        final Configuration config1 = new Configuration();
-        config1.setString(backendKey, "jobmanager");
-
-        final Configuration config2 = new Configuration();
-        config2.setString(backendKey, 
MemoryStateBackendFactory.class.getName());
+        final Configuration config = new Configuration();
+        config.setString(backendKey, 
HashMapStateBackendFactory.class.getName());
 
-        StateBackend backend1 = 
StateBackendLoader.loadStateBackendFromConfig(config1, cl, null);
-        StateBackend backend2 = 
StateBackendLoader.loadStateBackendFromConfig(config2, cl, null);
+        StateBackend backend = 
StateBackendLoader.loadStateBackendFromConfig(config, cl, null);
 
-        assertThat(backend1).isInstanceOf(MemoryStateBackend.class);
-        assertThat(backend2).isInstanceOf(MemoryStateBackend.class);
+        assertThat(backend).isInstanceOf(HashMapStateBackend.class);
     }
 
     /**
-     * Validates loading a memory state backend with additional parameters 
from the cluster
+     * Validates loading a hashMap state backend with additional parameters 
from the cluster
      * configuration.
      */
     @Test
-    void testLoadMemoryStateWithParameters() throws Exception {
-        final String checkpointDir = new 
Path(TempDirUtils.newFolder(tmp).toURI()).toString();
-        final String savepointDir = new 
Path(TempDirUtils.newFolder(tmp).toURI()).toString();
-        final Path expectedCheckpointPath = new Path(checkpointDir);
-        final Path expectedSavepointPath = new Path(savepointDir);
-
+    void testLoadHashMapStateBackendWithParameters() throws Exception {

Review Comment:
   IIUC, you should remove this Test as it test the Hashmap with parameters, 
but actually there is no parameter set.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to