Skip to content

Commit

Permalink
[FLINK-4579] [RocksDB backend] Add StateBackendFactory and config sho…
Browse files Browse the repository at this point in the history
…rtcut for the RocksDB Backend

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

This closes apache#2482
  • Loading branch information
wuchong authored and StephanEwen committed Oct 27, 2016
1 parent b5938b0 commit fc730bb
Show file tree
Hide file tree
Showing 10 changed files with 131 additions and 18 deletions.
18 changes: 2 additions & 16 deletions docs/dev/state_backends.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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.

Expand Down
2 changes: 2 additions & 0 deletions docs/setup/config.md
Original file line number Diff line number Diff line change
Expand Up @@ -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 ‘:’ (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.
Expand Down
2 changes: 2 additions & 0 deletions flink-contrib/flink-statebackend-rocksdb/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -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>
Expand Down
Original file line number Diff line number Diff line change
@@ -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);
}
}
}
Original file line number Diff line number Diff line change
@@ -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());
}
}
6 changes: 6 additions & 0 deletions flink-dist/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -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>

Expand Down
4 changes: 2 additions & 2 deletions flink-dist/src/main/resources/flink-conf.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -172,4 +172,4 @@ jobmanager.web.port: 8081

# Override below configuration to provide custom ZK service name if configured
#
# zookeeper.sasl.service-name: zookeeper
# zookeeper.sasl.service-name: zookeeper
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -664,6 +664,11 @@ private AbstractStateBackend createStateBackend() throws Exception {
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")
Expand Down

0 comments on commit fc730bb

Please sign in to comment.