From 57ddd27524b127a0bbfe461846a6ac8f4e590b5b Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 9 Sep 2016 13:38:26 +0800 Subject: [PATCH 1/2] [FLINK-4579] [StateBackend] Add StateBackendFactory for RocksDB Backend --- docs/dev/state_backends.md | 18 +----- .../state/RocksDBStateBackendFactory.java | 57 +++++++++++++++++++ flink-dist/pom.xml | 6 ++ .../streaming/runtime/tasks/StreamTask.java | 5 +- 4 files changed, 68 insertions(+), 18 deletions(-) create mode 100644 flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java diff --git a/docs/dev/state_backends.md b/docs/dev/state_backends.md index e5b9c2a2f8737..ee821fd302db0 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 %} - - org.apache.flink - flink-statebackend-rocksdb{{ site.scala_version_suffix }} - {{site.version }} - -{% 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. 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 0000000000000..4cfcee89637c6 --- /dev/null +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java @@ -0,0 +1,57 @@ +/* + * 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; + +/** + * A factory that creates an {@link org.apache.flink.contrib.streaming.state.RocksDBStateBackend} + * from a configuration. + */ +public class RocksDBStateBackendFactory implements StateBackendFactory { + + 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"; + + @Override + public AbstractStateBackend createFromConfig(Configuration config) throws Exception { + String checkpointDirURI = config.getString(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); + return new RocksDBStateBackend(path.toUri()); + } + catch (IllegalArgumentException e) { + throw new Exception("Cannot initialize RocksDB State Backend with URI '" + + checkpointDirURI + '.', e); + } + } +} diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index ec84adc00c1fe..3b27184c84f33 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -125,6 +125,12 @@ under the License. flink-mesos_2.10 ${project.version} + + + org.apache.flink + flink-statebackend-rocksdb_2.10 + ${project.version} + 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 9c2650911e852..0639ac6c92122 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 @@ -726,8 +726,7 @@ private AbstractStateBackend createStateBackend() throws Exception { backendName = "jobmanager"; } - backendName = backendName.toLowerCase(); - switch (backendName) { + switch (backendName.toLowerCase()) { case "jobmanager": LOG.info("State backend is set to heap memory (checkpoint to jobmanager)"); stateBackend = new MemoryStateBackend(); @@ -747,6 +746,8 @@ private AbstractStateBackend createStateBackend() throws Exception { Class.forName(backendName, false, getUserCodeClassLoader()).asSubclass(StateBackendFactory.class); stateBackend = ((StateBackendFactory) clazz.newInstance()).createFromConfig(flinkConfig); + + LOG.info("State backend is set to " + backendName); } catch (ClassNotFoundException e) { throw new IllegalConfigurationException("Cannot find configured state backend: " + backendName); } catch (ClassCastException e) { From ca5cafb0315191ee30f354d885cbff4ec99d619e Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 23 Sep 2016 14:29:28 +0800 Subject: [PATCH 2/2] Add a shortcut (rocksdb) for configuring RocksDB state backend. --- docs/setup/config.md | 2 ++ .../state/RocksDBStateBackendFactory.java | 19 ++++++++++++++++++- flink-dist/src/main/resources/flink-conf.yaml | 2 +- .../streaming/runtime/tasks/StreamTask.java | 4 ++++ 4 files changed, 25 insertions(+), 2 deletions(-) diff --git a/docs/setup/config.md b/docs/setup/config.md index 51475ccadb638..0d5d2cd771a47 100644 --- a/docs/setup/config.md +++ b/docs/setup/config.md @@ -134,6 +134,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 ‘:’ (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. 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 index 4cfcee89637c6..5002272c34767 100644 --- 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 @@ -23,6 +23,10 @@ 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} @@ -30,14 +34,19 @@ */ public class RocksDBStateBackendFactory implements StateBackendFactory { + 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( @@ -47,7 +56,15 @@ public AbstractStateBackend createFromConfig(Configuration config) throws Except try { Path path = new Path(checkpointDirURI); - return new RocksDBStateBackend(path.toUri()); + 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 '" diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index 27fd84a560e6e..9e6f3d406fe8a 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, +# Supported backends: jobmanager, filesystem, rocksdb, # #state.backend: filesystem 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 0639ac6c92122..ba59529ba5c95 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 @@ -739,6 +739,10 @@ private AbstractStateBackend createStateBackend() throws Exception { stateBackend = backend; break; + case "rocksdb": + // use reflect to keep rocksdb module separate + backendName = "org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory"; + default: try { @SuppressWarnings("rawtypes")