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 2016/10/27 09:06:17 UTC

flink git commit: [FLINK-4579] [RocksDB backend] Add StateBackendFactory and config shortcut for the RocksDB Backend

Repository: flink
Updated Branches:
  refs/heads/master b5938b052 -> fc730bb07


[FLINK-4579] [RocksDB backend] Add StateBackendFactory and config shortcut for the RocksDB Backend

This also now packages the RocksDB state backend into the Flink distribution

This closes #2482


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/fc730bb0
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/fc730bb0
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/fc730bb0

Branch: refs/heads/master
Commit: fc730bb079fe43b550954a91d981b9996c2be4c1
Parents: b5938b0
Author: Jark Wu <wu...@alibaba-inc.com>
Authored: Fri Sep 9 13:38:26 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 27 11:02:12 2016 +0200

----------------------------------------------------------------------
 docs/dev/state_backends.md                      | 18 +----
 docs/setup/config.md                            |  2 +
 .../flink-statebackend-rocksdb/pom.xml          |  2 +
 .../state/RocksDBStateBackendFactory.java       | 74 ++++++++++++++++++++
 .../state/RocksDBStateBackendFactoryTest.java   | 36 ++++++++++
 flink-dist/pom.xml                              |  6 ++
 flink-dist/src/main/resources/flink-conf.yaml   |  4 +-
 .../main/resources/archetype-resources/pom.xml  |  1 +
 .../main/resources/archetype-resources/pom.xml  |  1 +
 .../streaming/runtime/tasks/StreamTask.java     |  5 ++
 10 files changed, 131 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/docs/dev/state_backends.md
----------------------------------------------------------------------
diff --git a/docs/dev/state_backends.md b/docs/dev/state_backends.md
index 70e472d..31ebb6f 100644
--- a/docs/dev/state_backends.md
+++ b/docs/dev/state_backends.md
@@ -98,21 +98,6 @@ This allows keeping very large state, compared to the FsStateBackend that keeps
 This also means, however, that the maximum throughput that can be achieved will be lower with
 this state backend.
 
-**NOTE:** To use the RocksDBStateBackend you also have to add the correct maven dependency to your
-project:
-
-{% highlight xml %}
-<dependency>
-  <groupId>org.apache.flink</groupId>
-  <artifactId>flink-statebackend-rocksdb{{ site.scala_version_suffix }}</artifactId>
-  <version>{{site.version }}</version>
-</dependency>
-{% endhighlight %}
-
-The backend is currently not part of the binary distribution. See
-[here]({{ site.baseurl}}/dev/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution)
-for an explanation of how to include it for cluster execution.
-
 ## Configuring a State Backend
 
 State backends can be configured per job. In addition, you can define a default state backend to be used when the
@@ -144,7 +129,8 @@ env.setStateBackend(new FsStateBackend("hdfs://namenode:40010/flink/checkpoints"
 A default state backend can be configured in the `flink-conf.yaml`, using the configuration key `state.backend`.
 
 Possible values for the config entry are *jobmanager* (MemoryStateBackend), *filesystem* (FsStateBackend), or the fully qualified class
-name of the class that implements the state backend factory [FsStateBackendFactory](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java).
+name of the class that implements the state backend factory [FsStateBackendFactory](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java),
+such as `org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory` for RocksDBStateBackend.
 
 In the case where the default state backend is set to *filesystem*, the entry `state.backend.fs.checkpointdir` defines the directory where the checkpoint data will be stored.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/docs/setup/config.md
----------------------------------------------------------------------
diff --git a/docs/setup/config.md b/docs/setup/config.md
index 9567eb1..a887655 100644
--- a/docs/setup/config.md
+++ b/docs/setup/config.md
@@ -151,6 +151,8 @@ will be used under the directory specified by jobmanager.web.tmpdir.
 
 - `state.backend.fs.checkpointdir`: Directory for storing checkpoints in a Flink supported filesystem. Note: State backend must be accessible from the JobManager, use `file://` only for local setups.
 
+- `state.backend.rocksdb.checkpointdir`:  The local directory for storing RocksDB files, or a list of directories separated by the systems directory delimiter (for example \u2018:\u2019 (colon) on Linux/Unix). (DEFAULT value is `taskmanager.tmp.dirs`)
+
 - `high-availability.zookeeper.storageDir`: Required for HA. Directory for storing JobManager metadata; this is persisted in the state backend and only a pointer to this state is stored in ZooKeeper. Exactly like the checkpoint directory it must be accessible from the JobManager and a local filesystem should only be used for local deployments. Previously this key was named `recovery.zookeeper.storageDir`.
 
 - `blob.storage.directory`: Directory for storing blobs (such as user jar's) on the TaskManagers.

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-contrib/flink-statebackend-rocksdb/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/pom.xml b/flink-contrib/flink-statebackend-rocksdb/pom.xml
index 314b83f..ef5e563 100644
--- a/flink-contrib/flink-statebackend-rocksdb/pom.xml
+++ b/flink-contrib/flink-statebackend-rocksdb/pom.xml
@@ -44,12 +44,14 @@ under the License.
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java_2.10</artifactId>
 			<version>${project.version}</version>
+			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-clients_2.10</artifactId>
 			<version>${project.version}</version>
+			<scope>provided</scope>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java
new file mode 100644
index 0000000..5002272
--- /dev/null
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java
@@ -0,0 +1,74 @@
+/*
+ * 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.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.StateBackendFactory;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+/**
+ * A factory that creates an {@link org.apache.flink.contrib.streaming.state.RocksDBStateBackend}
+ * from a configuration.
+ */
+public class RocksDBStateBackendFactory implements StateBackendFactory<FsStateBackend> {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(RocksDBStateBackendFactory.class);
+
+	private static final long serialVersionUID = 4906988360901930371L;
+
+	/** The key under which the config stores the directory where checkpoints should be stored */
+	public static final String CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.fs.checkpointdir";
+	/** The key under which the config stores the directory where RocksDB should be stored */
+	public static final String ROCKSDB_CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.rocksdb.checkpointdir";
+
+	@Override
+	public AbstractStateBackend createFromConfig(Configuration config) throws Exception {
+		String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
+		String rocksdbLocalPath = config.getString(ROCKSDB_CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
+
+		if (checkpointDirURI == null) {
+			throw new IllegalConfigurationException(
+				"Cannot create the RocksDB state backend: The configuration does not specify the " +
+				"checkpoint directory '" + CHECKPOINT_DIRECTORY_URI_CONF_KEY + '\'');
+		}
+
+		try {
+			Path path = new Path(checkpointDirURI);
+			RocksDBStateBackend backend = new RocksDBStateBackend(path.toUri());
+			if (rocksdbLocalPath != null) {
+				String[] directories = rocksdbLocalPath.split(",|" + File.pathSeparator);
+				backend.setDbStoragePaths(directories);
+			}
+			LOG.info("State backend is set to RocksDB (configured DB storage paths {}, checkpoints to filesystem {} ) ",
+					backend.getDbStoragePaths(), path);
+
+			return backend;
+		}
+		catch (IllegalArgumentException e) {
+			throw new Exception("Cannot initialize RocksDB State Backend with URI '"
+									+ checkpointDirURI + '.', e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java
new file mode 100644
index 0000000..9eb662a
--- /dev/null
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactoryTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class RocksDBStateBackendFactoryTest {
+
+	@Test
+	public void testFactoryName() {
+		// construct the name such that it will not be automatically adjusted on refactorings
+		String factoryName = "org.apache.flink.contrib.streaming.state.Roc";
+		factoryName += "ksDBStateBackendFactory";
+
+		// !!! if this fails, the code in StreamTask.createStateBackend() must be adjusted
+		assertEquals(factoryName, RocksDBStateBackendFactory.class.getName());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 010e9b5..319f6af 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -125,6 +125,12 @@ under the License.
 			<artifactId>flink-mesos_2.10</artifactId>
 			<version>${project.version}</version>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-statebackend-rocksdb_2.10</artifactId>
+			<version>${project.version}</version>
+		</dependency>
 		
 	</dependencies>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-dist/src/main/resources/flink-conf.yaml
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml
index ad916e8..58efe12 100644
--- a/flink-dist/src/main/resources/flink-conf.yaml
+++ b/flink-dist/src/main/resources/flink-conf.yaml
@@ -79,7 +79,7 @@ jobmanager.web.port: 8081
 # The backend that will be used to store operator state checkpoints if
 # checkpointing is enabled.
 #
-# Supported backends: jobmanager, filesystem, <class-name-of-factory>
+# Supported backends: jobmanager, filesystem, rocksdb, <class-name-of-factory>
 #
 #state.backend: filesystem
 
@@ -172,4 +172,4 @@ jobmanager.web.port: 8081
 
 # Override below configuration to provide custom ZK service name if configured
 #
-# zookeeper.sasl.service-name: zookeeper
\ No newline at end of file
+# zookeeper.sasl.service-name: zookeeper

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
index 9da113f..e62d9d4 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
@@ -184,6 +184,7 @@ under the License.
 									<exclude>org.apache.flink:flink-python</exclude>
 									<exclude>org.apache.flink:flink-metrics-core</exclude>
 									<exclude>org.apache.flink:flink-metrics-jmx</exclude>
+									<exclude>org.apache.flink:flink-statebackend-rocksdb_2.10</exclude>
 
 									<!-- Also exclude very big transitive dependencies of Flink
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
index 5fd41b2..a62cf79 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
@@ -188,6 +188,7 @@ under the License.
 									<exclude>org.apache.flink:flink-python</exclude>
 									<exclude>org.apache.flink:flink-metrics-core</exclude>
 									<exclude>org.apache.flink:flink-metrics-jmx</exclude>
+									<exclude>org.apache.flink:flink-statebackend-rocksdb_2.10</exclude>
 
 									<!-- Also exclude very big transitive dependencies of Flink
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fc730bb0/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 905782b..cd63102 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -664,6 +664,11 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 					stateBackend = backend;
 					break;
 
+				case "rocksdb":
+					backendName = "org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory";
+					// fall through to the 'default' case that uses reflection to load the backend
+					// that way we can keep RocksDB in a separate module
+
 				default:
 					try {
 						@SuppressWarnings("rawtypes")