Skip to content

Commit

Permalink
[FLINK-21344] Handle heap timers in Rocks state
Browse files Browse the repository at this point in the history
We serialize the heap timers into the same format as if they were
actually stored in RocksDB instead of storing them in a raw operator
state. It lets users change between using heap and RocksDB timers.
  • Loading branch information
dawidwys committed Feb 24, 2021
1 parent be628c6 commit a9fef44
Show file tree
Hide file tree
Showing 17 changed files with 696 additions and 107 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/*
* 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.runtime.state;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSet;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSnapshotRestoreWrapper;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.StateMigrationException;

import javax.annotation.Nonnull;

import java.util.Map;

/** Manages creating heap priority queues along with their counterpart meta info. */
@Internal
public class HeapPriorityQueuesManager {

private final Map<String, HeapPriorityQueueSnapshotRestoreWrapper<?>> registeredPQStates;
private final HeapPriorityQueueSetFactory priorityQueueSetFactory;
private final KeyGroupRange keyGroupRange;
private final int numberOfKeyGroups;

public HeapPriorityQueuesManager(
Map<String, HeapPriorityQueueSnapshotRestoreWrapper<?>> registeredPQStates,
HeapPriorityQueueSetFactory priorityQueueSetFactory,
KeyGroupRange keyGroupRange,
int numberOfKeyGroups) {
this.registeredPQStates = registeredPQStates;
this.priorityQueueSetFactory = priorityQueueSetFactory;
this.keyGroupRange = keyGroupRange;
this.numberOfKeyGroups = numberOfKeyGroups;
}

@SuppressWarnings("unchecked")
@Nonnull
public <T extends HeapPriorityQueueElement & PriorityComparable<? super T> & Keyed<?>>
KeyGroupedInternalPriorityQueue<T> createOrUpdate(
@Nonnull String stateName,
@Nonnull TypeSerializer<T> byteOrderedElementSerializer) {

final HeapPriorityQueueSnapshotRestoreWrapper<T> existingState =
(HeapPriorityQueueSnapshotRestoreWrapper<T>) registeredPQStates.get(stateName);

if (existingState != null) {
TypeSerializerSchemaCompatibility<T> compatibilityResult =
existingState
.getMetaInfo()
.updateElementSerializer(byteOrderedElementSerializer);

if (compatibilityResult.isIncompatible()) {
throw new FlinkRuntimeException(
new StateMigrationException(
"For heap backends, the new priority queue serializer must not be incompatible."));
} else {
registeredPQStates.put(
stateName,
existingState.forUpdatedSerializer(byteOrderedElementSerializer));
}

return existingState.getPriorityQueue();
} else {
final RegisteredPriorityQueueStateBackendMetaInfo<T> metaInfo =
new RegisteredPriorityQueueStateBackendMetaInfo<>(
stateName, byteOrderedElementSerializer);
return createInternal(metaInfo);
}
}

@Nonnull
private <T extends HeapPriorityQueueElement & PriorityComparable<? super T> & Keyed<?>>
KeyGroupedInternalPriorityQueue<T> createInternal(
RegisteredPriorityQueueStateBackendMetaInfo<T> metaInfo) {

final String stateName = metaInfo.getName();
final HeapPriorityQueueSet<T> priorityQueue =
priorityQueueSetFactory.create(stateName, metaInfo.getElementSerializer());

HeapPriorityQueueSnapshotRestoreWrapper<T> wrapper =
new HeapPriorityQueueSnapshotRestoreWrapper<>(
priorityQueue,
metaInfo,
KeyExtractorFunction.forKeyedObjects(),
keyGroupRange,
numberOfKeyGroups);

registeredPQStates.put(stateName, wrapper);
return priorityQueue;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,15 +35,14 @@
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.KeyExtractorFunction;
import org.apache.flink.runtime.state.HeapPriorityQueuesManager;
import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
import org.apache.flink.runtime.state.Keyed;
import org.apache.flink.runtime.state.KeyedStateFunction;
import org.apache.flink.runtime.state.KeyedStateHandle;
import org.apache.flink.runtime.state.LocalRecoveryConfig;
import org.apache.flink.runtime.state.PriorityComparable;
import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;
import org.apache.flink.runtime.state.RegisteredPriorityQueueStateBackendMetaInfo;
import org.apache.flink.runtime.state.SnapshotResult;
import org.apache.flink.runtime.state.SnapshotStrategyRunner;
import org.apache.flink.runtime.state.StateSnapshotRestore;
Expand Down Expand Up @@ -96,9 +95,6 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
/** Map of registered Key/Value states. */
private final Map<String, StateTable<K, ?, ?>> registeredKVStates;

/** Map of registered priority queue set states. */
private final Map<String, HeapPriorityQueueSnapshotRestoreWrapper<?>> registeredPQStates;

/** The configuration for local recovery. */
private final LocalRecoveryConfig localRecoveryConfig;

Expand All @@ -113,7 +109,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
private final StateTableFactory<K> stateTableFactory;

/** Factory for state that is organized as priority queue. */
private final HeapPriorityQueueSetFactory priorityQueueSetFactory;
private final HeapPriorityQueuesManager priorityQueuesManager;

public HeapKeyedStateBackend(
TaskKvStateRegistry kvStateRegistry,
Expand Down Expand Up @@ -141,80 +137,30 @@ public HeapKeyedStateBackend(
keyGroupCompressionDecorator,
keyContext);
this.registeredKVStates = registeredKVStates;
this.registeredPQStates = registeredPQStates;
this.localRecoveryConfig = localRecoveryConfig;
this.priorityQueueSetFactory = priorityQueueSetFactory;
this.checkpointStrategyRunner = checkpointStrategyRunner;
this.savepointStrategyRunner = savepointStrategyRunner;
this.stateTableFactory = stateTableFactory;
this.priorityQueuesManager =
new HeapPriorityQueuesManager(
registeredPQStates,
priorityQueueSetFactory,
keyContext.getKeyGroupRange(),
keyContext.getNumberOfKeyGroups());
LOG.info("Initializing heap keyed state backend with stream factory.");
}

// ------------------------------------------------------------------------
// state backend operations
// ------------------------------------------------------------------------

@SuppressWarnings("unchecked")
@Nonnull
@Override
public <T extends HeapPriorityQueueElement & PriorityComparable<? super T> & Keyed<?>>
KeyGroupedInternalPriorityQueue<T> create(
@Nonnull String stateName,
@Nonnull TypeSerializer<T> byteOrderedElementSerializer) {

final HeapPriorityQueueSnapshotRestoreWrapper<T> existingState =
(HeapPriorityQueueSnapshotRestoreWrapper<T>) registeredPQStates.get(stateName);

if (existingState != null) {
// TODO we implement the simple way of supporting the current functionality, mimicking
// keyed state
// because this should be reworked in FLINK-9376 and then we should have a common
// algorithm over
// StateMetaInfoSnapshot that avoids this code duplication.

TypeSerializerSchemaCompatibility<T> compatibilityResult =
existingState
.getMetaInfo()
.updateElementSerializer(byteOrderedElementSerializer);

if (compatibilityResult.isIncompatible()) {
throw new FlinkRuntimeException(
new StateMigrationException(
"For heap backends, the new priority queue serializer must not be incompatible."));
} else {
registeredPQStates.put(
stateName,
existingState.forUpdatedSerializer(byteOrderedElementSerializer));
}

return existingState.getPriorityQueue();
} else {
final RegisteredPriorityQueueStateBackendMetaInfo<T> metaInfo =
new RegisteredPriorityQueueStateBackendMetaInfo<>(
stateName, byteOrderedElementSerializer);
return createInternal(metaInfo);
}
}

@Nonnull
private <T extends HeapPriorityQueueElement & PriorityComparable<? super T> & Keyed<?>>
KeyGroupedInternalPriorityQueue<T> createInternal(
RegisteredPriorityQueueStateBackendMetaInfo<T> metaInfo) {

final String stateName = metaInfo.getName();
final HeapPriorityQueueSet<T> priorityQueue =
priorityQueueSetFactory.create(stateName, metaInfo.getElementSerializer());

HeapPriorityQueueSnapshotRestoreWrapper<T> wrapper =
new HeapPriorityQueueSnapshotRestoreWrapper<>(
priorityQueue,
metaInfo,
KeyExtractorFunction.forKeyedObjects(),
keyGroupRange,
numberOfKeyGroups);

registeredPQStates.put(stateName, wrapper);
return priorityQueue;
return priorityQueuesManager.createOrUpdate(stateName, byteOrderedElementSerializer);
}

private <N, V> StateTable<K, N, V> tryRegisterStateTable(
Expand Down Expand Up @@ -409,7 +355,6 @@ public String toString() {

/** Returns the total number of state entries across all keys/namespaces. */
@VisibleForTesting
@SuppressWarnings("unchecked")
@Override
public int numKeyValueStateEntries() {
int sum = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,9 +115,10 @@ Map<Integer, StateMetaInfoSnapshot> createOrCheckStateForMetaInfo(
return kvStatesById;
}

private <T extends HeapPriorityQueueElement & PriorityComparable & Keyed>
@SuppressWarnings({"unchecked", "rawtypes"})
private <T extends HeapPriorityQueueElement & PriorityComparable<? super T> & Keyed<?>>
HeapPriorityQueueSnapshotRestoreWrapper<T> createInternal(
RegisteredPriorityQueueStateBackendMetaInfo<T> metaInfo) {
RegisteredPriorityQueueStateBackendMetaInfo metaInfo) {

final String stateName = metaInfo.getName();
final HeapPriorityQueueSet<T> priorityQueue =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.flink.runtime.state.KeyGroupPartitioner;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.RegisteredPriorityQueueStateBackendMetaInfo;
import org.apache.flink.runtime.state.StateSnapshot;
import org.apache.flink.runtime.state.StateSnapshotKeyGroupReader;
import org.apache.flink.runtime.state.StateSnapshotRestore;

Expand Down Expand Up @@ -61,10 +60,10 @@ public HeapPriorityQueueSnapshotRestoreWrapper(
@SuppressWarnings("unchecked")
@Nonnull
@Override
public StateSnapshot stateSnapshot() {
public HeapPriorityQueueStateSnapshot<T> stateSnapshot() {
final T[] queueDump =
(T[]) priorityQueue.toArray(new HeapPriorityQueueElement[priorityQueue.size()]);
return new HeapPriorityQueueStateSnapshot<>(
return new HeapPriorityQueueStateSnapshot<T>(
queueDump,
keyExtractorFunction,
metaInfo.deepCopy(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,11 @@ public StateMetaInfoSnapshot getMetaInfoSnapshot() {
return metaInfo.snapshot();
}

@Nonnull
public RegisteredPriorityQueueStateBackendMetaInfo<T> getMetaInfo() {
return metaInfo;
}

@Override
public void release() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -164,8 +164,8 @@ private void readKeyGroupStateData(
@SuppressWarnings("unchecked")
private void readPriorityQueue(KeyGroupEntry groupEntry, StateMetaInfoSnapshot infoSnapshot)
throws IOException {
DataInputDeserializer keyDeserializer = new DataInputDeserializer(groupEntry.getKey());
keyDeserializer.skipBytesToRead(keyGroupPrefixBytes);
entryKeyDeserializer.setBuffer(groupEntry.getKey());
entryKeyDeserializer.skipBytesToRead(keyGroupPrefixBytes);
HeapPriorityQueueSnapshotRestoreWrapper<HeapPriorityQueueElement>
priorityQueueSnapshotRestoreWrapper =
(HeapPriorityQueueSnapshotRestoreWrapper<HeapPriorityQueueElement>)
Expand All @@ -174,7 +174,7 @@ private void readPriorityQueue(KeyGroupEntry groupEntry, StateMetaInfoSnapshot i
priorityQueueSnapshotRestoreWrapper
.getMetaInfo()
.getElementSerializer()
.deserialize(keyDeserializer);
.deserialize(entryKeyDeserializer);
HeapPriorityQueueSet<HeapPriorityQueueElement> priorityQueue =
priorityQueueSnapshotRestoreWrapper.getPriorityQueue();
priorityQueue.add(timer);
Expand Down
Loading

0 comments on commit a9fef44

Please sign in to comment.