rkhachatryan commented on a change in pull request #14943:
URL: https://github.com/apache/flink/pull/14943#discussion_r587380224



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackendForTest.java
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.runtime.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+
+/** A keyed state backend interface for internal testing purpose. */
+@Internal
+public interface KeyedStateBackendForTest {

Review comment:
       How about `TestableKeyedStateBackend`?
   
   And marking the interface itself `@VisibleForTesting`?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
##########
@@ -173,6 +186,22 @@ public static StateBackend loadStateBackendFromConfig(
         }
     }
 
+    public static StateBackend loadWrappedStateBackendFromConfig(
+            ReadableConfig config, ClassLoader classLoader, @Nullable Logger 
logger)
+            throws IllegalConfigurationException, DynamicCodeLoadingException, 
IOException {

Review comment:
       This changes adds two new public methods, increasing the total number of 
methods to get a backend to four. Ideally, I think there should be just one 
public method.
   
   I see that two existing methods are now marked with `@VisibleForTesting` but 
that's not clear enough I think.
   
   Instead, why not
   1. Make existing methods private and add some suffix like `private 
loadStateBackendFromConfig*Internal`
   2. Add two corresponding methods (`loadStateBackendFromConfig` and 
`fromApplicationOrConfigOrDefault`) which would call original ones and wrap if 
needed
   3. No changes to callers required
   ?
   

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java
##########
@@ -173,6 +186,22 @@ public static StateBackend loadStateBackendFromConfig(
         }
     }
 
+    public static StateBackend loadWrappedStateBackendFromConfig(
+            ReadableConfig config, ClassLoader classLoader, @Nullable Logger 
logger)
+            throws IllegalConfigurationException, DynamicCodeLoadingException, 
IOException {
+
+        checkNotNull(config, "config");
+        checkNotNull(classLoader, "classLoader");
+
+        final StateBackend backend = loadStateBackendFromConfig(config, 
classLoader, logger);
+
+        if (config.get(CheckpointingOptions.ENABLE_STATE_CHANGE_LOG)) {
+            return loadChangelogStateBackend(backend, classLoader);

Review comment:
       I think the check whether `backend` is already delegating is missing 
here (`loadStateBackendFromConfig` can construct one using factory class).

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendLoadingTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.state.changelog;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
+import org.apache.flink.runtime.state.CheckpointStorageAccess;
+import org.apache.flink.runtime.state.CheckpointStorageLoader;
+import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
+import org.apache.flink.runtime.state.ConfigurableStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendLoader;
+import org.apache.flink.runtime.state.delegate.DelegatingStateBackend;
+import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import javax.annotation.Nonnull;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/** Verify Changelog StateBackend is properly loaded. */
+@RunWith(Parameterized.class)
+public class ChangelogStateBackendLoadingTest {
+    @Rule public final TemporaryFolder tmp = new TemporaryFolder();
+
+    @Parameterized.Parameters
+    public static List<Boolean> loadFromConfigOnly() {
+        return Arrays.asList(true, false);
+    }
+
+    @Parameterized.Parameter public boolean configOnly;
+
+    private final ClassLoader cl = getClass().getClassLoader();
+
+    private final String backendKey = CheckpointingOptions.STATE_BACKEND.key();
+
+    @Test
+    public void testLoadingDefault() throws Exception {

Review comment:
       I think this is actually a test for `StateBackendLoader` so I'd prefer 
to move it to `StateBackendLoadingTest`.
   
   Testing of RocksDb wrapping won't work, but I don't think we need to test 
for all combinations.

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/test/java/org/apache/flink/state/changelog/ChangelogStateBackendLoadingTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.state.changelog;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.CheckpointStorage;
+import org.apache.flink.runtime.state.CheckpointStorageAccess;
+import org.apache.flink.runtime.state.CheckpointStorageLoader;
+import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
+import org.apache.flink.runtime.state.ConfigurableStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateBackendLoader;
+import org.apache.flink.runtime.state.delegate.DelegatingStateBackend;
+import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import javax.annotation.Nonnull;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/** Verify Changelog StateBackend is properly loaded. */
+@RunWith(Parameterized.class)
+public class ChangelogStateBackendLoadingTest {
+    @Rule public final TemporaryFolder tmp = new TemporaryFolder();
+
+    @Parameterized.Parameters
+    public static List<Boolean> loadFromConfigOnly() {
+        return Arrays.asList(true, false);
+    }
+
+    @Parameterized.Parameter public boolean configOnly;
+
+    private final ClassLoader cl = getClass().getClassLoader();
+
+    private final String backendKey = CheckpointingOptions.STATE_BACKEND.key();
+
+    @Test
+    public void testLoadingDefault() throws Exception {
+        final StateBackend backend =
+                
StateBackendLoader.loadStateBackendFromApplicationOrConfigOrDefault(
+                        null, config(), cl, null);
+        final CheckpointStorage storage =
+                CheckpointStorageLoader.load(null, null, backend, config(), 
cl, null);
+
+        assertDelegateStateBackend(
+                backend, HashMapStateBackend.class, storage, 
JobManagerCheckpointStorage.class);
+    }
+
+    @Test
+    public void testLoadingMemoryStateBackendFromConfig() throws Exception {
+        final Configuration config = config("jobmanager");
+        final StateBackend backend = loadStateBackend(config);
+        final CheckpointStorage storage =
+                CheckpointStorageLoader.load(null, null, backend, config, cl, 
null);
+
+        assertDelegateStateBackend(
+                backend, MemoryStateBackend.class, storage, 
MemoryStateBackend.class);
+    }
+
+    @Test
+    public void testLoadingFsStateBackendFromConfig() throws Exception {
+        final Configuration config = config("filesystem");
+        final StateBackend backend = loadStateBackend(config);
+        final CheckpointStorage storage =
+                CheckpointStorageLoader.load(null, null, backend, config, cl, 
null);
+
+        assertDelegateStateBackend(
+                backend, HashMapStateBackend.class, storage, 
JobManagerCheckpointStorage.class);
+    }
+
+    @Test
+    public void testLoadingHashMapStateBackendFromConfig() throws Exception {
+        final Configuration config = config("hashmap");
+        final StateBackend backend = loadStateBackend(config);
+        final CheckpointStorage storage =
+                CheckpointStorageLoader.load(null, null, backend, config(), 
cl, null);
+
+        assertDelegateStateBackend(
+                backend, HashMapStateBackend.class, storage, 
JobManagerCheckpointStorage.class);
+    }
+
+    @Test
+    public void testLoadingRocksDBStateBackendFromConfig() throws Exception {
+        final Configuration config = config("rocksdb");
+        final StateBackend backend = loadStateBackend(config);
+        final CheckpointStorage storage =
+                CheckpointStorageLoader.load(null, null, backend, config(), 
cl, null);
+
+        assertDelegateStateBackend(
+                backend,
+                EmbeddedRocksDBStateBackend.class,
+                storage,
+                JobManagerCheckpointStorage.class);
+    }

Review comment:
       This code is duplicated many times in this class.
   I think duplication can be eliminated by extracting test parameters (backend 
name, backend class, storage class).
   But as I said above, I don't think we need to test all these combinations.




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

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


Reply via email to