[ 
https://issues.apache.org/jira/browse/FLINK-10198?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16609069#comment-16609069
 ] 

ASF GitHub Bot commented on FLINK-10198:
----------------------------------------

StefanRRichter closed pull request #6603: [FLINK-10198][state] Set Env object 
in DBOptions for RocksDB
URL: https://github.com/apache/flink/pull/6603
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
index b8bd73c08e1..4cfea5ed7ef 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
@@ -43,6 +43,7 @@
 
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
+import org.rocksdb.Env;
 import org.rocksdb.NativeLibraryLoader;
 import org.rocksdb.RocksDB;
 import org.slf4j.Logger;
@@ -647,6 +648,7 @@ public DBOptions getDbOptions() {
 
                // add necessary default options
                opt = opt.setCreateIfMissing(true);
+               opt.setEnv(Env.getDefault());
 
                return opt;
        }
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
index b7d05b3455e..662bd89c9ae 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
@@ -48,6 +48,7 @@
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.CompactionStyle;
 import org.rocksdb.DBOptions;
+import org.rocksdb.Env;
 
 import java.io.File;
 import java.util.Collections;
@@ -381,6 +382,28 @@ public void testContinueOnSomeDbDirectoriesMissing() 
throws Exception {
        //  RocksDB Options
        // 
------------------------------------------------------------------------
 
+       @Test
+       public void testSetDefaultEnvInOptions() throws Exception {
+               String checkpointPath = 
tempFolder.newFolder().toURI().toString();
+               RocksDBStateBackend rocksDbBackend = new 
RocksDBStateBackend(checkpointPath);
+
+               rocksDbBackend.setOptions(new OptionsFactory() {
+                       @Override
+                       public DBOptions createDBOptions(DBOptions 
currentOptions) {
+                               return new DBOptions();
+                       }
+
+                       @Override
+                       public ColumnFamilyOptions 
createColumnOptions(ColumnFamilyOptions currentOptions) {
+                               throw new UnsupportedOperationException();
+                       }
+               });
+
+               try (DBOptions options = rocksDbBackend.getDbOptions()) {
+                       assertEquals(Env.getDefault(), options.getEnv());
+               }
+       }
+
        @Test
        public void testPredefinedOptions() throws Exception {
                String checkpointPath = 
tempFolder.newFolder().toURI().toString();


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Set Env object in DBOptions for RocksDB
> ---------------------------------------
>
>                 Key: FLINK-10198
>                 URL: https://issues.apache.org/jira/browse/FLINK-10198
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 1.7.0
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>            Priority: Major
>              Labels: pull-request-available
>
> I think we should consider to always set a default environment when we create 
> the DBOptions.
> See https://github.com/facebook/rocksdb/wiki/rocksdb-basics:
> *Support for Multiple Embedded Databases in the same process*
> A common use-case for RocksDB is that applications inherently partition their 
> data set into logical partitions or shards. This technique benefits 
> application load balancing and fast recovery from faults. This means that a 
> single server process should be able to operate multiple RocksDB databases 
> simultaneously. This is done via an environment object named Env. Among other 
> things, a thread pool is associated with an Env. If applications want to 
> share a common thread pool (for background compactions) among multiple 
> database instances, then it should use the same Env object for opening those 
> databases.
> Similarly, multiple database instances may share the same block cache.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to