Skip to content

Commit

Permalink
[FLINK-10043] [State Backends] Refactor RocksDBKeyedStateBackend obje…
Browse files Browse the repository at this point in the history
…ct construction/initialization/restore code

This closes apache#7674.
  • Loading branch information
carp84 authored and StefanRRichter committed Feb 22, 2019
1 parent 70107c4 commit 3bf06b9
Show file tree
Hide file tree
Showing 49 changed files with 2,662 additions and 1,321 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,13 @@
import org.apache.flink.runtime.state.CheckpointStorage;
import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.KeyedStateHandle;
import org.apache.flink.runtime.state.OperatorStateBackend;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.runtime.state.ttl.TtlTimeProvider;

import java.io.IOException;
import java.util.Collection;

import static org.apache.flink.util.Preconditions.checkNotNull;

Expand Down Expand Up @@ -64,27 +66,28 @@ public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException

@Override
public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
Environment env,
JobID jobID,
String operatorIdentifier,
TypeSerializer<K> keySerializer,
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
TaskKvStateRegistry kvStateRegistry,
TtlTimeProvider ttlTimeProvider,
MetricGroup metricGroup) throws Exception {
Environment env,
JobID jobID,
String operatorIdentifier,
TypeSerializer<K> keySerializer,
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
TaskKvStateRegistry kvStateRegistry,
TtlTimeProvider ttlTimeProvider,
MetricGroup metricGroup,
Collection<KeyedStateHandle> stateHandles) throws Exception {

return backend.createKeyedStateBackend(
env,
jobID,
operatorIdentifier,
keySerializer,
numberOfKeyGroups,
keyGroupRange,
kvStateRegistry,
this.ttlTimeProvider,
metricGroup
);
env,
jobID,
operatorIdentifier,
keySerializer,
numberOfKeyGroups,
keyGroupRange,
kvStateRegistry,
this.ttlTimeProvider,
metricGroup,
stateHandles);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.contrib.streaming.state.PredefinedOptions;
import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
import org.apache.flink.contrib.streaming.state.RocksDBNativeMetricOptions;
import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackendBuilder;
import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.queryablestate.client.VoidNamespace;
Expand All @@ -40,9 +40,10 @@
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.rocksdb.ColumnFamilyOptions;
import org.rocksdb.DBOptions;

import java.util.Collections;

import static org.mockito.Mockito.mock;

/**
Expand All @@ -68,29 +69,26 @@ public void testListSerialization() throws Exception {
// objects for RocksDB state list serialisation
DBOptions dbOptions = PredefinedOptions.DEFAULT.createDBOptions();
dbOptions.setCreateIfMissing(true);
ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
ExecutionConfig executionConfig = new ExecutionConfig();
final RocksDBKeyedStateBackend<Long> longHeapKeyedStateBackend =
new RocksDBKeyedStateBackend<>(
new RocksDBKeyedStateBackendBuilder<>(
"no-op",
ClassLoader.getSystemClassLoader(),
temporaryFolder.getRoot(),
dbOptions,
stateName -> columnFamilyOptions,
stateName -> PredefinedOptions.DEFAULT.createColumnOptions(),
mock(TaskKvStateRegistry.class),
LongSerializer.INSTANCE,
1,
new KeyGroupRange(0, 0),
new ExecutionConfig(),
false,
1,
executionConfig,
TestLocalRecoveryConfig.disabled(),
RocksDBStateBackend.PriorityQueueStateType.HEAP,
TtlTimeProvider.DEFAULT,
false,
new RocksDBNativeMetricOptions(),
new UnregisteredMetricsGroup()
);
longHeapKeyedStateBackend.restore(null);
new UnregisteredMetricsGroup(),
Collections.emptyList(),
RocksDBStateBackend.getCompressionDecorator(executionConfig)
).build();
longHeapKeyedStateBackend.setCurrentKey(key);

final InternalListState<Long, VoidNamespace, Long> listState = longHeapKeyedStateBackend.createInternalState(VoidNamespaceSerializer.INSTANCE,
Expand All @@ -114,29 +112,26 @@ public void testMapSerialization() throws Exception {
// objects for RocksDB state list serialisation
DBOptions dbOptions = PredefinedOptions.DEFAULT.createDBOptions();
dbOptions.setCreateIfMissing(true);
ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
ExecutionConfig executionConfig = new ExecutionConfig();
final RocksDBKeyedStateBackend<Long> longHeapKeyedStateBackend =
new RocksDBKeyedStateBackend<Long>(
new RocksDBKeyedStateBackendBuilder<>(
"no-op",
ClassLoader.getSystemClassLoader(),
temporaryFolder.getRoot(),
dbOptions,
stateName -> columnFamilyOptions,
stateName -> PredefinedOptions.DEFAULT.createColumnOptions(),
mock(TaskKvStateRegistry.class),
LongSerializer.INSTANCE,
1,
new KeyGroupRange(0, 0),
new ExecutionConfig(),
false,
1,
executionConfig,
TestLocalRecoveryConfig.disabled(),
RocksDBStateBackend.PriorityQueueStateType.HEAP,
TtlTimeProvider.DEFAULT,
false,
new RocksDBNativeMetricOptions(),
new UnregisteredMetricsGroup()
);
longHeapKeyedStateBackend.restore(null);
new UnregisteredMetricsGroup(),
Collections.emptyList(),
RocksDBStateBackend.getCompressionDecorator(executionConfig)
).build();
longHeapKeyedStateBackend.setCurrentKey(key);

final InternalMapState<Long, VoidNamespace, Long, String> mapState =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -770,6 +770,7 @@ private AbstractKeyedStateBackend<Integer> createKeyedStateBackend(KvStateRegist
new KeyGroupRange(0, 0),
registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()),
TtlTimeProvider.DEFAULT,
new UnregisteredMetricsGroup());
new UnregisteredMetricsGroup(),
null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public abstract class AbstractKeyedStateBackend<K> implements
CheckpointListener {

/** {@link StateSerializerProvider} for our key serializer. */
private final StateSerializerProvider<K> keySerializerProvider;
protected final StateSerializerProvider<K> keySerializerProvider;

/** The currently active key. */
private K currentKey;
Expand Down Expand Up @@ -105,25 +105,48 @@ public AbstractKeyedStateBackend(
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider) {
TtlTimeProvider ttlTimeProvider,
CloseableRegistry cancelStreamRegistry) {
this(
kvStateRegistry,
StateSerializerProvider.fromNewRegisteredSerializer(keySerializer),
userCodeClassLoader,
numberOfKeyGroups,
keyGroupRange,
executionConfig,
ttlTimeProvider,
cancelStreamRegistry,
determineStreamCompression(executionConfig)
);
}

public AbstractKeyedStateBackend(
TaskKvStateRegistry kvStateRegistry,
StateSerializerProvider<K> keySerializerProvider,
ClassLoader userCodeClassLoader,
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
CloseableRegistry cancelStreamRegistry,
StreamCompressionDecorator keyGroupCompressionDecorator) {
Preconditions.checkArgument(numberOfKeyGroups >= 1, "NumberOfKeyGroups must be a positive number");
Preconditions.checkArgument(numberOfKeyGroups >= keyGroupRange.getNumberOfKeyGroups(), "The total number of key groups must be at least the number in the key group range assigned to this backend");

this.kvStateRegistry = kvStateRegistry;
this.keySerializerProvider = StateSerializerProvider.fromNewRegisteredSerializer(keySerializer);
this.keySerializerProvider = keySerializerProvider;
this.numberOfKeyGroups = numberOfKeyGroups;
this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader);
this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange);
this.cancelStreamRegistry = new CloseableRegistry();
this.cancelStreamRegistry = cancelStreamRegistry;
this.keyValueStatesByName = new HashMap<>();
this.executionConfig = executionConfig;
this.keyGroupCompressionDecorator = determineStreamCompression(executionConfig);
this.keyGroupCompressionDecorator = keyGroupCompressionDecorator;
this.ttlTimeProvider = Preconditions.checkNotNull(ttlTimeProvider);
this.keySelectionListeners = new HashSet<>();
}

private StreamCompressionDecorator determineStreamCompression(ExecutionConfig executionConfig) {
private static StreamCompressionDecorator determineStreamCompression(ExecutionConfig executionConfig) {
if (executionConfig != null && executionConfig.isUseSnapshotCompression()) {
return SnappyStreamCompressionDecorator.INSTANCE;
} else {
Expand Down Expand Up @@ -337,7 +360,7 @@ public boolean supportsAsynchronousSnapshots() {
}

@VisibleForTesting
StreamCompressionDecorator getKeyGroupCompressionDecorator() {
public StreamCompressionDecorator getKeyGroupCompressionDecorator() {
return keyGroupCompressionDecorator;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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
*
* https://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.runtime.state;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.ttl.TtlTimeProvider;

import javax.annotation.Nonnull;

import java.util.Collection;

/**
* An abstract base implementation of the {@link StateBackendBuilder} interface.
*/
public abstract class AbstractKeyedStateBackendBuilder<K>
implements StateBackendBuilder<AbstractKeyedStateBackend, BackendBuildingException> {
protected final TaskKvStateRegistry kvStateRegistry;
protected final TypeSerializer<K> keySerializer;
protected final StateSerializerProvider<K> keySerializerProvider;
protected final ClassLoader userCodeClassLoader;
protected final int numberOfKeyGroups;
protected final KeyGroupRange keyGroupRange;
protected final ExecutionConfig executionConfig;
protected final TtlTimeProvider ttlTimeProvider;
protected final StreamCompressionDecorator keyGroupCompressionDecorator;
protected final Collection<KeyedStateHandle> restoreStateHandles;

public AbstractKeyedStateBackendBuilder(
TaskKvStateRegistry kvStateRegistry,
TypeSerializer<K> keySerializer,
ClassLoader userCodeClassLoader,
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
@Nonnull Collection<KeyedStateHandle> stateHandles,
StreamCompressionDecorator keyGroupCompressionDecorator) {
this.kvStateRegistry = kvStateRegistry;
this.keySerializer = keySerializer;
this.keySerializerProvider = StateSerializerProvider.fromNewRegisteredSerializer(keySerializer);
this.userCodeClassLoader = userCodeClassLoader;
this.numberOfKeyGroups = numberOfKeyGroups;
this.keyGroupRange = keyGroupRange;
this.executionConfig = executionConfig;
this.ttlTimeProvider = ttlTimeProvider;
this.keyGroupCompressionDecorator = keyGroupCompressionDecorator;
this.restoreStateHandles = stateHandles;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,10 @@
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.ttl.TtlTimeProvider;

import javax.annotation.Nonnull;

import java.io.IOException;
import java.util.Collection;

/**
* An abstract base implementation of the {@link StateBackend} interface.
Expand All @@ -52,7 +55,8 @@ public abstract <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
KeyGroupRange keyGroupRange,
TaskKvStateRegistry kvStateRegistry,
TtlTimeProvider ttlTimeProvider,
MetricGroup metricGroup) throws IOException;
MetricGroup metricGroup,
@Nonnull Collection<KeyedStateHandle> stateHandles) throws IOException;

@Override
public abstract OperatorStateBackend createOperatorStateBackend(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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
*
* https://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.runtime.state;

import java.io.IOException;

/**
* Exceptions which indicate that a state backend building has failed.
*/
public class BackendBuildingException extends IOException {
public BackendBuildingException(String message, Throwable cause) {
super(message, cause);
}
}
Loading

0 comments on commit 3bf06b9

Please sign in to comment.