fredia commented on code in PR #24682:
URL: https://github.com/apache/flink/pull/24682#discussion_r1577715584


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java:
##########
@@ -0,0 +1,547 @@
+/*
+ * 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.forst;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.runtime.memory.OpaqueMemoryResource;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.BloomFilter;
+import org.rocksdb.Cache;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Filter;
+import org.rocksdb.FlinkEnv;
+import org.rocksdb.IndexType;
+import org.rocksdb.PlainTableConfig;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.Statistics;
+import org.rocksdb.TableFormatConfig;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * The container for ForSt resources, including option factory and shared 
resource among instances.
+ *
+ * <p>This should be the only entrance for ForStStateBackend to get ForSt 
options, and should be
+ * properly (and necessarily) closed to prevent resource leak.
+ */
+public final class ForStResourceContainer implements AutoCloseable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ForStResourceContainer.class);
+
+    private static final String FORST_RELOCATE_LOG_SUFFIX = "_LOG";
+
+    // the filename length limit is 255 on most operating systems
+    private static final int INSTANCE_PATH_LENGTH_LIMIT = 255 - 
FORST_RELOCATE_LOG_SUFFIX.length();
+
+    private static final String DB_DIR_STRING = "db";
+
+    @Nullable private final URI remoteBasePath;
+
+    @Nullable private final URI remoteForStPath;
+
+    @Nullable private final File localBasePath;

Review Comment:
   What is different between `localBasePath` and `localForStPath`?



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStSharedResourcesFactory.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.forst;
+
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.memory.OpaqueMemoryResource;
+import org.apache.flink.runtime.memory.SharedResources;
+import org.apache.flink.util.function.LongFunctionWithException;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import static org.apache.flink.state.forst.ForStOptions.FIX_PER_TM_MEMORY_SIZE;
+
+/**
+ * A factory of {@link ForStSharedResources}. Encapsulates memory share scope 
(e.g. TM, Slot) and
+ * lifecycle (managed/unmanaged).
+ */
+enum ForStSharedResourcesFactory {
+    /** Memory allocated per Slot (shared across slot tasks), managed by 
Flink. */
+    SLOT_SHARED_MANAGED(false, MemoryShareScope.SLOT) {
+        @Override
+        protected OpaqueMemoryResource<ForStSharedResources> createInternal(
+                ForStMemoryConfiguration jobMemoryConfig,
+                String resourceId,
+                Environment env,
+                double memoryFraction,
+                LongFunctionWithException<ForStSharedResources, Exception> 
allocator)
+                throws Exception {
+            return env.getMemoryManager()
+                    .getSharedMemoryResourceForManagedMemory(resourceId, 
allocator, memoryFraction);
+        }
+    },
+    /** Memory allocated per Slot (shared across slot tasks), unmanaged. */
+    SLOT_SHARED_UNMANAGED(false, MemoryShareScope.SLOT) {
+        @Override
+        protected OpaqueMemoryResource<ForStSharedResources> createInternal(
+                ForStMemoryConfiguration jobMemoryConfig,
+                String resourceId,
+                Environment env,
+                double memoryFraction,
+                LongFunctionWithException<ForStSharedResources, Exception> 
allocator)
+                throws Exception {
+            return env.getMemoryManager()
+                    .getExternalSharedMemoryResource(
+                            resourceId,
+                            allocator,
+                            
jobMemoryConfig.getFixedMemoryPerSlot().getBytes());
+        }
+    },
+    /** Memory allocated per TM (shared across all tasks), unmanaged. */
+    TM_SHARED_UNMANAGED(false, MemoryShareScope.TM) {
+        @Override
+        protected OpaqueMemoryResource<ForStSharedResources> createInternal(
+                ForStMemoryConfiguration jobMemoryConfig,
+                String resourceId,
+                Environment env,
+                double memoryFraction,
+                LongFunctionWithException<ForStSharedResources, Exception> 
allocator)
+                throws Exception {
+
+            SharedResources sharedResources = env.getSharedResources();
+            Object leaseHolder = new Object();
+            SharedResources.ResourceAndSize<ForStSharedResources> resource =
+                    sharedResources.getOrAllocateSharedResource(
+                            resourceId, leaseHolder, allocator, 
getTmSharedMemorySize(env));
+            ThrowingRunnable<Exception> disposer =
+                    () -> sharedResources.release(resourceId, leaseHolder, 
unused -> {});
+
+            return new OpaqueMemoryResource<>(resource.resourceHandle(), 
resource.size(), disposer);
+        }
+    };
+
+    private final boolean managed;
+    private final MemoryShareScope shareScope;
+
+    ForStSharedResourcesFactory(boolean managed, MemoryShareScope shareScope) {
+        this.managed = managed;
+        this.shareScope = shareScope;
+    }
+
+    @Nullable
+    public static ForStSharedResourcesFactory from(
+            ForStMemoryConfiguration jobMemoryConfig, Environment env) {
+        if (jobMemoryConfig.isUsingFixedMemoryPerSlot()) {
+            return ForStSharedResourcesFactory.SLOT_SHARED_UNMANAGED;
+        } else if (jobMemoryConfig.isUsingManagedMemory()) {
+            return ForStSharedResourcesFactory.SLOT_SHARED_MANAGED;
+        } else if (getTmSharedMemorySize(env) > 0) {
+            return ForStSharedResourcesFactory.TM_SHARED_UNMANAGED;
+        } else {
+            // not shared and not managed - allocate per column family
+            return null;
+        }
+    }
+
+    public final OpaqueMemoryResource<ForStSharedResources> create(
+            ForStMemoryConfiguration jobMemoryConfig,
+            Environment env,
+            double memoryFraction,
+            Logger logger,
+            ForStMemoryControllerUtils.ForStMemoryFactory forStMemoryFactory)
+            throws Exception {
+        logger.info(
+                "Getting shared memory for ForSt: shareScope={}, managed={}", 
shareScope, managed);
+        return createInternal(
+                jobMemoryConfig,
+                managed ? MANAGED_MEMORY_RESOURCE_ID : 
UNMANAGED_MEMORY_RESOURCE_ID,
+                env,
+                memoryFraction,
+                createAllocator(
+                        shareScope.getConfiguration(jobMemoryConfig, env), 
forStMemoryFactory));
+    }
+
+    protected abstract OpaqueMemoryResource<ForStSharedResources> 
createInternal(
+            ForStMemoryConfiguration jobMemoryConfig,
+            String resourceId,
+            Environment env,
+            double memoryFraction,
+            LongFunctionWithException<ForStSharedResources, Exception> 
allocator)
+            throws Exception;
+
+    private static long getTmSharedMemorySize(Environment env) {
+        return env.getTaskManagerInfo()
+                .getConfiguration()
+                .getOptional(FIX_PER_TM_MEMORY_SIZE)
+                .orElse(MemorySize.ZERO)
+                .getBytes();
+    }
+
+    private static final String MANAGED_MEMORY_RESOURCE_ID = 
"state-rocks-managed-memory";

Review Comment:
   state-forst-managed-memory?
   



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStKeyedStateBackend.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.forst;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.util.Disposable;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.URI;
+
+/**
+ * A KeyedStateBackend that stores its state in {@code ForSt}. This state 
backend can store very
+ * large state that exceeds memory even disk to remote storage. TODO: Support 
to implement the new
+ * interface of KeyedStateBackend
+ */
+public class ForStKeyedStateBackend<K> implements Disposable {

Review Comment:
   Should `ForStKeyedStateBackend` inherit `AsyncKeyedStateBackend`?



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

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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

Reply via email to