You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2017/01/05 11:17:40 UTC
flink git commit: [FLINK-5408] [RocksDB backend] Uniquify RocksDB JNI
library path to avoid multiple classloader problem
Repository: flink
Updated Branches:
refs/heads/master 29eec70d0 -> 3070ff9a6
[FLINK-5408] [RocksDB backend] Uniquify RocksDB JNI library path to avoid multiple classloader problem
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3070ff9a
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3070ff9a
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3070ff9a
Branch: refs/heads/master
Commit: 3070ff9a6d9de47a4713d4b4952929f8c00043b1
Parents: 29eec70
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Jan 5 00:18:13 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Jan 5 12:02:09 2017 +0100
----------------------------------------------------------------------
.../streaming/state/RocksDBStateBackend.java | 29 ++++--
.../state/RocksDbMultiClassLoaderTest.java | 100 +++++++++++++++++++
.../runtime/state/StateBackendTestBase.java | 3 +-
3 files changed, 123 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/3070ff9a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
index 2109cea..c2e33d4 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.KeyGroupsStateHandle;
import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.util.AbstractID;
import org.rocksdb.ColumnFamilyOptions;
import org.rocksdb.DBOptions;
@@ -70,8 +71,10 @@ public class RocksDBStateBackend extends AbstractStateBackend {
private static final Logger LOG = LoggerFactory.getLogger(RocksDBStateBackend.class);
+ /** The number of (re)tries for loading the RocksDB JNI library */
private static final int ROCKSDB_LIB_LOADING_ATTEMPTS = 3;
+
private static boolean rocksDbInitialized = false;
// ------------------------------------------------------------------------
@@ -476,24 +479,34 @@ public class RocksDBStateBackend extends AbstractStateBackend {
// ------------------------------------------------------------------------
private void ensureRocksDBIsLoaded(String tempDirectory) throws Exception {
- // lock on something that cannot be in the user JAR
- synchronized (org.apache.flink.runtime.taskmanager.Task.class) {
+ synchronized (RocksDBStateBackend.class) {
if (!rocksDbInitialized) {
- final File tempDirFile = new File(tempDirectory);
- final String path = tempDirFile.getAbsolutePath();
-
- LOG.info("Attempting to load RocksDB native library and store it at '{}'", path);
+ final File tempDirParent = new File(tempDirectory).getAbsoluteFile();
+ LOG.info("Attempting to load RocksDB native library and store it under '{}'", tempDirParent);
Throwable lastException = null;
for (int attempt = 1; attempt <= ROCKSDB_LIB_LOADING_ATTEMPTS; attempt++) {
try {
+ // when multiple instances of this class and RocksDB exist in different
+ // class loaders, then we can see the following exception:
+ // "java.lang.UnsatisfiedLinkError: Native Library /path/to/temp/dir/librocksdbjni-linux64.so
+ // already loaded in another class loader"
+
+ // to avoid that, we need to add a random element to the library file path
+ // (I know, seems like an unnecessary hack, since the JVM obviously can handle multiple
+ // instances of the same JNI library being loaded in different class loaders, but
+ // apparently not when coming from the same file path, so there we go)
+
+ final File rocksLibFolder = new File(tempDirParent, "rocksdb-lib-" + new AbstractID());
+
// make sure the temp path exists
+ LOG.debug("Attempting to create RocksDB native library folder {}", rocksLibFolder);
// noinspection ResultOfMethodCallIgnored
- tempDirFile.mkdirs();
+ rocksLibFolder.mkdirs();
// explicitly load the JNI dependency if it has not been loaded before
- NativeLibraryLoader.getInstance().loadLibrary(path);
+ NativeLibraryLoader.getInstance().loadLibrary(rocksLibFolder.getAbsolutePath());
// this initialization here should validate that the loading succeeded
RocksDB.loadLibrary();
http://git-wip-us.apache.org/repos/asf/flink/blob/3070ff9a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDbMultiClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDbMultiClassLoaderTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDbMultiClassLoaderTest.java
new file mode 100644
index 0000000..c53fa3e
--- /dev/null
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDbMultiClassLoaderTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.RocksDB;
+
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+import static org.junit.Assert.*;
+
+/**
+ * This test validates that the RocksDB JNI library loading works properly
+ * in the presence of the RocksDB code being loaded dynamically via reflection.
+ * That can happen when RocksDB is in the user code JAR, or in certain test setups.
+ */
+public class RocksDbMultiClassLoaderTest {
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testTwoSeparateClassLoaders() throws Exception {
+ // collect the libraries / class folders with RocksDB related code: the state backend and RocksDB itself
+ final URL codePath1 = RocksDBStateBackend.class.getProtectionDomain().getCodeSource().getLocation();
+ final URL codePath2 = RocksDB.class.getProtectionDomain().getCodeSource().getLocation();
+
+ final ClassLoader parent = getClass().getClassLoader();
+ final ClassLoader loader1 = new ChildFirstClassLoader(new URL[] { codePath1, codePath2 }, parent);
+ final ClassLoader loader2 = new ChildFirstClassLoader(new URL[] { codePath1, codePath2 }, parent);
+
+ final String className = RocksDBStateBackend.class.getName();
+
+ final Class<?> clazz1 = Class.forName(className, false, loader1);
+ final Class<?> clazz2 = Class.forName(className, false, loader2);
+ assertNotEquals("Test broken - the two reflectively loaded classes are equal", clazz1, clazz2);
+
+ final Object instance1 = clazz1.getConstructor(String.class).newInstance(tmp.newFolder().toURI().toString());
+ final Object instance2 = clazz2.getConstructor(String.class).newInstance(tmp.newFolder().toURI().toString());
+
+ final String tempDir = tmp.newFolder().getAbsolutePath();
+
+ final Method meth1 = clazz1.getDeclaredMethod("ensureRocksDBIsLoaded", String.class);
+ final Method meth2 = clazz2.getDeclaredMethod("ensureRocksDBIsLoaded", String.class);
+ meth1.setAccessible(true);
+ meth2.setAccessible(true);
+
+ // if all is well, these methods can both complete successfully
+ meth1.invoke(instance1, tempDir);
+ meth2.invoke(instance2, tempDir);
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * A variant of the URLClassLoader that first loads from the URLs and only after that from the parent.
+ */
+ private static final class ChildFirstClassLoader extends URLClassLoader {
+
+ private final ClassLoader parent;
+
+ public ChildFirstClassLoader(URL[] urls, ClassLoader parent) {
+ super(urls, null);
+ this.parent = parent;
+ }
+
+ @Override
+ public Class<?> findClass(String name) throws ClassNotFoundException {
+ // first try to load from the URLs
+ // because the URLClassLoader's parent is null, this cannot implicitly load from the parent
+ try {
+ return super.findClass(name);
+ }
+ catch (ClassNotFoundException e) {
+ // not in the URL, check the parent
+ return parent.loadClass(name);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/3070ff9a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index aad84df..5655f1c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -48,6 +48,7 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
import org.apache.flink.runtime.state.heap.AbstractHeapState;
import org.apache.flink.runtime.state.heap.StateTable;
import org.apache.flink.types.IntValue;
+import org.apache.flink.util.TestLogger;
import org.junit.Test;
import java.util.Collections;
@@ -68,7 +69,7 @@ import static org.mockito.Mockito.verify;
* Generic tests for the partitioned state part of {@link AbstractStateBackend}.
*/
@SuppressWarnings("serial")
-public abstract class StateBackendTestBase<B extends AbstractStateBackend> {
+public abstract class StateBackendTestBase<B extends AbstractStateBackend> extends TestLogger {
protected abstract B getStateBackend() throws Exception;