Skip to content

Commit

Permalink
[FLINK-11730] [State Backends] Make HeapKeyedStateBackend follow the …
Browse files Browse the repository at this point in the history
…builder pattern

This closes apache#7866.
  • Loading branch information
carp84 authored and StefanRRichter committed Mar 4, 2019
1 parent eada52b commit 237d07c
Show file tree
Hide file tree
Showing 28 changed files with 1,279 additions and 711 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.flink.queryablestate.client.VoidNamespace;
import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.AbstractStateBackend;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.TestLocalRecoveryConfig;
import org.apache.flink.runtime.state.internal.InternalListState;
Expand Down Expand Up @@ -88,7 +89,7 @@ public void testListSerialization() throws Exception {
TtlTimeProvider.DEFAULT,
new UnregisteredMetricsGroup(),
Collections.emptyList(),
RocksDBStateBackend.getCompressionDecorator(executionConfig),
AbstractStateBackend.getCompressionDecorator(executionConfig),
new CloseableRegistry()
).build();
longHeapKeyedStateBackend.setCurrentKey(key);
Expand Down Expand Up @@ -132,7 +133,7 @@ public void testMapSerialization() throws Exception {
TtlTimeProvider.DEFAULT,
new UnregisteredMetricsGroup(),
Collections.emptyList(),
RocksDBStateBackend.getCompressionDecorator(executionConfig),
AbstractStateBackend.getCompressionDecorator(executionConfig),
new CloseableRegistry()
).build();
longHeapKeyedStateBackend.setCurrentKey(key);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,12 @@
import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.AbstractStateBackend;
import org.apache.flink.runtime.state.BackendBuildingException;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.TestLocalRecoveryConfig;
import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
import org.apache.flink.runtime.state.heap.HeapKeyedStateBackendBuilder;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory;
import org.apache.flink.runtime.state.internal.InternalKvState;
import org.apache.flink.runtime.state.internal.InternalListState;
Expand All @@ -48,6 +51,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -188,22 +192,7 @@ public void testDeserializeValueTooMany2() throws Exception {
@Test
public void testListSerialization() throws Exception {
final long key = 0L;
final KeyGroupRange keyGroupRange = new KeyGroupRange(0, 0);
// objects for heap state list serialisation
final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend =
new HeapKeyedStateBackend<>(
mock(TaskKvStateRegistry.class),
LongSerializer.INSTANCE,
ClassLoader.getSystemClassLoader(),
keyGroupRange.getNumberOfKeyGroups(),
keyGroupRange,
async,
new ExecutionConfig(),
TestLocalRecoveryConfig.disabled(),
new HeapPriorityQueueSetFactory(keyGroupRange, keyGroupRange.getNumberOfKeyGroups(), 128),
TtlTimeProvider.DEFAULT,
new CloseableRegistry());
longHeapKeyedStateBackend.setCurrentKey(key);
final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend = getLongHeapKeyedStateBackend(key);

final InternalListState<Long, VoidNamespace, Long> listState = longHeapKeyedStateBackend.createInternalState(
VoidNamespaceSerializer.INSTANCE,
Expand Down Expand Up @@ -297,31 +286,39 @@ public void testDeserializeListTooShort2() throws Exception {
@Test
public void testMapSerialization() throws Exception {
final long key = 0L;
final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend = getLongHeapKeyedStateBackend(key);

final InternalMapState<Long, VoidNamespace, Long, String> mapState =
(InternalMapState<Long, VoidNamespace, Long, String>)
longHeapKeyedStateBackend.getPartitionedState(
VoidNamespace.INSTANCE,
VoidNamespaceSerializer.INSTANCE,
new MapStateDescriptor<>("test", LongSerializer.INSTANCE, StringSerializer.INSTANCE));

testMapSerialization(key, mapState);
}

private HeapKeyedStateBackend<Long> getLongHeapKeyedStateBackend(final long key) throws BackendBuildingException {
final KeyGroupRange keyGroupRange = new KeyGroupRange(0, 0);
ExecutionConfig executionConfig = new ExecutionConfig();
// objects for heap state list serialisation
final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend =
new HeapKeyedStateBackend<>(
new HeapKeyedStateBackendBuilder<>(
mock(TaskKvStateRegistry.class),
LongSerializer.INSTANCE,
ClassLoader.getSystemClassLoader(),
keyGroupRange.getNumberOfKeyGroups(),
keyGroupRange,
async,
new ExecutionConfig(),
executionConfig,
TtlTimeProvider.DEFAULT,
Collections.emptyList(),
AbstractStateBackend.getCompressionDecorator(executionConfig),
TestLocalRecoveryConfig.disabled(),
new HeapPriorityQueueSetFactory(keyGroupRange, keyGroupRange.getNumberOfKeyGroups(), 128),
TtlTimeProvider.DEFAULT,
new CloseableRegistry());
async,
new CloseableRegistry()).build();
longHeapKeyedStateBackend.setCurrentKey(key);

final InternalMapState<Long, VoidNamespace, Long, String> mapState =
(InternalMapState<Long, VoidNamespace, Long, String>)
longHeapKeyedStateBackend.getPartitionedState(
VoidNamespace.INSTANCE,
VoidNamespaceSerializer.INSTANCE,
new MapStateDescriptor<>("test", LongSerializer.INSTANCE, StringSerializer.INSTANCE));

testMapSerialization(key, mapState);
return longHeapKeyedStateBackend;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.runtime.state;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.fs.CloseableRegistry;
Expand All @@ -42,6 +43,14 @@ public abstract class AbstractStateBackend implements StateBackend, java.io.Seri

private static final long serialVersionUID = 4620415814639230247L;

public static StreamCompressionDecorator getCompressionDecorator(ExecutionConfig executionConfig) {
if (executionConfig != null && executionConfig.isUseSnapshotCompression()) {
return SnappyStreamCompressionDecorator.INSTANCE;
} else {
return UncompressedStreamCompressionDecorator.INSTANCE;
}
}

// ------------------------------------------------------------------------
// State Backend - State-Holding Backends
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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: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;

/**
* Interface for restore operation.
*
* @param <R> Generic type of the restore result.
*/
public interface RestoreOperation<R> {
/**
* Restores state that was previously snapshot-ed from the provided state handles.
*/
R restore() throws Exception;
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,15 +30,17 @@
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
import org.apache.flink.runtime.state.AbstractStateBackend;
import org.apache.flink.runtime.state.BackendBuildingException;
import org.apache.flink.runtime.state.CheckpointStorage;
import org.apache.flink.runtime.state.ConfigurableStateBackend;
import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
import org.apache.flink.runtime.state.KeyedStateHandle;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.KeyedStateHandle;
import org.apache.flink.runtime.state.LocalRecoveryConfig;
import org.apache.flink.runtime.state.OperatorStateBackend;
import org.apache.flink.runtime.state.TaskStateManager;
import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
import org.apache.flink.runtime.state.heap.HeapKeyedStateBackendBuilder;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory;
import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
import org.apache.flink.util.TernaryBoolean;
Expand Down Expand Up @@ -463,25 +465,27 @@ public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
TtlTimeProvider ttlTimeProvider,
MetricGroup metricGroup,
@Nonnull Collection<KeyedStateHandle> stateHandles,
CloseableRegistry cancelStreamRegistry) {
CloseableRegistry cancelStreamRegistry) throws BackendBuildingException {

TaskStateManager taskStateManager = env.getTaskStateManager();
LocalRecoveryConfig localRecoveryConfig = taskStateManager.createLocalRecoveryConfig();
HeapPriorityQueueSetFactory priorityQueueSetFactory =
new HeapPriorityQueueSetFactory(keyGroupRange, numberOfKeyGroups, 128);

return new HeapKeyedStateBackend<>(
kvStateRegistry,
keySerializer,
env.getUserClassLoader(),
numberOfKeyGroups,
keyGroupRange,
isUsingAsynchronousSnapshots(),
env.getExecutionConfig(),
localRecoveryConfig,
priorityQueueSetFactory,
ttlTimeProvider,
cancelStreamRegistry);
return new HeapKeyedStateBackendBuilder<>(
kvStateRegistry,
keySerializer,
env.getUserClassLoader(),
numberOfKeyGroups,
keyGroupRange,
env.getExecutionConfig(),
ttlTimeProvider,
stateHandles,
AbstractStateBackend.getCompressionDecorator(env.getExecutionConfig()),
localRecoveryConfig,
priorityQueueSetFactory,
isUsingAsynchronousSnapshots(),
cancelStreamRegistry).build();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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: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.heap;

import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;

/**
* Asynchronous behavior for heap snapshot strategy.
*
* @param <K> The data type that the serializer serializes.
*/
class AsyncSnapshotStrategySynchronicityBehavior<K> implements SnapshotStrategySynchronicityBehavior<K> {

@Override
public boolean isAsynchronous() {
return true;
}

@Override
public <N, V> StateTable<K, N, V> newStateTable(
InternalKeyContext<K> keyContext,
RegisteredKeyValueStateBackendMetaInfo<N, V> newMetaInfo) {
return new CopyOnWriteStateTable<>(keyContext, newMetaInfo);
}
}
Loading

0 comments on commit 237d07c

Please sign in to comment.