This is an automated email from the ASF dual-hosted git repository.

rpuch pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 3182188010 IGNITE-22175 Improve handling for RocksDb resources on 
RocksDbKeyValueStorage (#3709)
3182188010 is described below

commit 3182188010d65b820fec45eeda07c363532b1759
Author: Tiago Marques Godinho <tmgodi...@ua.pt>
AuthorDate: Tue May 7 16:45:27 2024 +0100

    IGNITE-22175 Improve handling for RocksDb resources on 
RocksDbKeyValueStorage (#3709)
---
 .../server/persistence/RocksDbKeyValueStorage.java | 47 ++++++++++++++++++----
 1 file changed, 40 insertions(+), 7 deletions(-)

diff --git 
a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
 
b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
index f18d278b7b..7a4102656e 100644
--- 
a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
+++ 
b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
@@ -35,7 +35,6 @@ import static 
org.apache.ignite.internal.metastorage.server.persistence.StorageC
 import static org.apache.ignite.internal.rocksdb.RocksUtils.incrementPrefix;
 import static 
org.apache.ignite.internal.rocksdb.snapshot.ColumnFamilyRange.fullRange;
 import static org.apache.ignite.internal.util.ArrayUtils.LONG_EMPTY_ARRAY;
-import static org.apache.ignite.internal.util.IgniteUtils.closeAll;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.COMPACTION_ERR;
 import static org.apache.ignite.lang.ErrorGroups.MetaStorage.OP_EXECUTION_ERR;
 import static 
org.apache.ignite.lang.ErrorGroups.MetaStorage.RESTORING_STORAGE_ERR;
@@ -95,6 +94,7 @@ import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.jetbrains.annotations.Nullable;
 import org.jetbrains.annotations.TestOnly;
+import org.rocksdb.AbstractNativeReference;
 import org.rocksdb.BlockBasedTableConfig;
 import org.rocksdb.BloomFilter;
 import org.rocksdb.ColumnFamilyDescriptor;
@@ -229,6 +229,9 @@ public class RocksDbKeyValueStorage implements 
KeyValueStorage {
      */
     private final UpdatedEntries updatedEntries = new UpdatedEntries();
 
+    /** Tracks RocksDb resources that must be properly closed. */
+    private List<AbstractNativeReference> rocksResources = new ArrayList<>();
+
     /**
      * Constructor.
      *
@@ -245,17 +248,22 @@ public class RocksDbKeyValueStorage implements 
KeyValueStorage {
 
     @Override
     public void start() {
+        rwLock.writeLock().lock();
+
         try {
             // Delete existing data, relying on the raft's snapshot and log 
playback
             destroyRocksDb();
 
             createDb();
         } catch (IOException | RocksDBException e) {
+            closeRocksResources();
             throw new MetaStorageException(STARTING_STORAGE_ERR, "Failed to 
start the storage", e);
+        } finally {
+            rwLock.writeLock().unlock();
         }
     }
 
-    private static List<ColumnFamilyDescriptor> cfDescriptors() {
+    private List<ColumnFamilyDescriptor> cfDescriptors() {
         Options baseOptions = new Options()
                 .setCreateIfMissing(true)
                 // Lowering the desired number of levels will, on average, 
lead to less lookups in files, making reads faster.
@@ -275,12 +283,16 @@ public class RocksDbKeyValueStorage implements 
KeyValueStorage {
         ColumnFamilyOptions dataFamilyOptions = new 
ColumnFamilyOptions(baseOptions)
                 // The prefix is the revision of an entry, so prefix length is 
the size of a long
                 .useFixedLengthPrefixExtractor(Long.BYTES);
+        this.rocksResources.add(dataFamilyOptions);
 
         ColumnFamilyOptions indexFamilyOptions = new 
ColumnFamilyOptions(baseOptions);
+        this.rocksResources.add(indexFamilyOptions);
 
         ColumnFamilyOptions tsToRevFamilyOptions = new 
ColumnFamilyOptions(baseOptions);
+        this.rocksResources.add(tsToRevFamilyOptions);
 
         ColumnFamilyOptions revToTsFamilyOptions = new 
ColumnFamilyOptions(baseOptions);
+        this.rocksResources.add(revToTsFamilyOptions);
 
         return List.of(
                 new ColumnFamilyDescriptor(DATA.nameAsBytes(), 
dataFamilyOptions),
@@ -290,6 +302,16 @@ public class RocksDbKeyValueStorage implements 
KeyValueStorage {
         );
     }
 
+    protected DBOptions createDbOptions() {
+        DBOptions options = new DBOptions()
+                .setCreateMissingColumnFamilies(true)
+                .setCreateIfMissing(true);
+
+        rocksResources.add(options);
+
+        return options;
+    }
+
     protected void createDb() throws RocksDBException {
         List<ColumnFamilyDescriptor> descriptors = cfDescriptors();
 
@@ -297,11 +319,11 @@ public class RocksDbKeyValueStorage implements 
KeyValueStorage {
 
         var handles = new ArrayList<ColumnFamilyHandle>(descriptors.size());
 
-        options = new DBOptions()
-                .setCreateMissingColumnFamilies(true)
-                .setCreateIfMissing(true);
+        options = createDbOptions();
 
         db = RocksDB.open(options, dbPath.toAbsolutePath().toString(), 
descriptors, handles);
+        rocksResources.add(db);
+        rocksResources.addAll(handles);
 
         data = ColumnFamily.wrap(db, handles.get(0));
 
@@ -352,7 +374,18 @@ public class RocksDbKeyValueStorage implements 
KeyValueStorage {
 
         IgniteUtils.shutdownAndAwaitTermination(snapshotExecutor, 10, 
TimeUnit.SECONDS);
 
-        RocksUtils.closeAll(db, options);
+        rwLock.writeLock().lock();
+        try {
+            closeRocksResources();
+        } finally {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    private void closeRocksResources() {
+        Collections.reverse(rocksResources);
+        RocksUtils.closeAll(rocksResources);
+        this.rocksResources = new ArrayList<>();
     }
 
     @Override
@@ -368,7 +401,7 @@ public class RocksDbKeyValueStorage implements 
KeyValueStorage {
 
         try {
             // there's no way to easily remove all data from RocksDB, so we 
need to re-create it from scratch
-            closeAll(db, options);
+            closeRocksResources();
 
             destroyRocksDb();
 

Reply via email to