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


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateBackend.java:
##########
@@ -734,22 +740,36 @@ static void ensureForStIsLoaded(
                         // loaders, but
                         //  apparently not when coming from the same file 
path, so there we go)
 
-                        rocksLibFolder = new File(tempDirParent, 
"rocksdb-lib-" + new AbstractID());
-
-                        // make sure the temp path exists
-                        LOG.debug(
-                                "Attempting to create ForSt native library 
folder {}",
-                                rocksLibFolder);
-                        // noinspection ResultOfMethodCallIgnored
-                        rocksLibFolder.mkdirs();
-
-                        // explicitly load the JNI dependency if it has not 
been loaded before
-                        nativeLibraryLoaderSupplier
-                                .get()
-                                .loadLibrary(rocksLibFolder.getAbsolutePath());
-
-                        // this initialization here should validate that the 
loading succeeded
-                        RocksDB.loadLibrary();
+                        CompletableFuture<Void> future =
+                                FutureUtils.runAsync(
+                                        () -> {
+                                            File libFolder =
+                                                    new File(
+                                                            tempDirParent,
+                                                            "rocksdb-lib-" + 
new AbstractID());
+                                            rocksLibFolder.set(libFolder);
+
+                                            // make sure the temp path exists
+                                            LOG.debug(
+                                                    "Attempting to create 
ForSt native library folder {}",
+                                                    libFolder);
+                                            // noinspection 
ResultOfMethodCallIgnored
+                                            libFolder.mkdirs();
+
+                                            // explicitly load the JNI 
dependency if it has not been
+                                            // loaded before
+                                            nativeLibraryLoaderSupplier
+                                                    .get()
+                                                    
.loadLibrary(libFolder.getAbsolutePath());
+
+                                            // this initialization here should 
validate that the
+                                            // loading succeeded
+                                            RocksDB.loadLibrary();
+                                        },

Review Comment:
   nit: add some description about why load library async?



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStKeyedStateBackend.java:
##########
@@ -330,7 +329,9 @@ public void dispose() {
         }
         synchronized (lock) {
             if (!closed) {

Review Comment:
   Should set closed=true here since `close()` is removed now ?



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