Skip to content

Commit

Permalink
[FLINK-21505] Factor snapshot resource creation out of RocksFullSnaps…
Browse files Browse the repository at this point in the history
…hotStrategy

This will allow us to create snapshot resources for a savepoint without
using a snapshot strategy.
  • Loading branch information
aljoscha authored and dawidwys committed Mar 1, 2021
1 parent 43ec46d commit 04f8abb
Show file tree
Hide file tree
Showing 2 changed files with 54 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;
import org.apache.flink.runtime.state.StateSnapshotTransformer;
import org.apache.flink.runtime.state.StreamCompressionDecorator;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSnapshotRestoreWrapper;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueStateSnapshot;
import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
import org.apache.flink.util.IOUtils;
Expand All @@ -48,6 +49,7 @@

import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;

/** A {@link FullSnapshotResources} for the RocksDB backend. */
Expand Down Expand Up @@ -92,6 +94,53 @@ public RocksDBFullSnapshotResources(
this.metaData = fillMetaData(metaDataCopy);
}

public static <K> RocksDBFullSnapshotResources<K> create(
final LinkedHashMap<String, RocksDBKeyedStateBackend.RocksDbKvStateInfo>
kvStateInformation,
final LinkedHashMap<String, HeapPriorityQueueSnapshotRestoreWrapper<?>>
registeredPQStates,
final RocksDB db,
final ResourceGuard rocksDBResourceGuard,
final KeyGroupRange keyGroupRange,
final TypeSerializer<K> keySerializer,
final int keyGroupPrefixBytes,
final StreamCompressionDecorator keyGroupCompressionDecorator)
throws IOException {

final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots =
new ArrayList<>(kvStateInformation.size());
final List<RocksDBKeyedStateBackend.RocksDbKvStateInfo> metaDataCopy =
new ArrayList<>(kvStateInformation.size());

for (RocksDBKeyedStateBackend.RocksDbKvStateInfo stateInfo : kvStateInformation.values()) {
// snapshot meta info
stateMetaInfoSnapshots.add(stateInfo.metaInfo.snapshot());
metaDataCopy.add(stateInfo);
}

List<HeapPriorityQueueStateSnapshot<?>> heapPriorityQueuesSnapshots =
new ArrayList<>(registeredPQStates.size());
for (HeapPriorityQueueSnapshotRestoreWrapper<?> stateInfo : registeredPQStates.values()) {
stateMetaInfoSnapshots.add(stateInfo.getMetaInfo().snapshot());
heapPriorityQueuesSnapshots.add(stateInfo.stateSnapshot());
}

final ResourceGuard.Lease lease = rocksDBResourceGuard.acquireResource();
final Snapshot snapshot = db.getSnapshot();

return new RocksDBFullSnapshotResources<>(
lease,
snapshot,
metaDataCopy,
heapPriorityQueuesSnapshots,
stateMetaInfoSnapshots,
db,
keyGroupPrefixBytes,
keyGroupRange,
keySerializer,
keyGroupCompressionDecorator);
}

private List<MetaData> fillMetaData(
List<RocksDBKeyedStateBackend.RocksDbKvStateInfo> metaDataCopy) {
List<MetaData> metaData = new ArrayList<>(metaDataCopy.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,22 +32,17 @@
import org.apache.flink.runtime.state.SnapshotResult;
import org.apache.flink.runtime.state.StreamCompressionDecorator;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueSnapshotRestoreWrapper;
import org.apache.flink.runtime.state.heap.HeapPriorityQueueStateSnapshot;
import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
import org.apache.flink.util.ResourceGuard;
import org.apache.flink.util.function.SupplierWithException;

import org.rocksdb.RocksDB;
import org.rocksdb.Snapshot;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nonnegative;
import javax.annotation.Nonnull;

import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;

/**
* Snapshot strategy to create full snapshots of {@link
Expand Down Expand Up @@ -97,37 +92,14 @@ public RocksFullSnapshotStrategy(

@Override
public FullSnapshotResources<K> syncPrepareResources(long checkpointId) throws Exception {

final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots =
new ArrayList<>(kvStateInformation.size());
final List<RocksDbKvStateInfo> metaDataCopy = new ArrayList<>(kvStateInformation.size());

for (RocksDbKvStateInfo stateInfo : kvStateInformation.values()) {
// snapshot meta info
stateMetaInfoSnapshots.add(stateInfo.metaInfo.snapshot());
metaDataCopy.add(stateInfo);
}

List<HeapPriorityQueueStateSnapshot<?>> heapPriorityQueuesSnapshots =
new ArrayList<>(registeredPQStates.size());
for (HeapPriorityQueueSnapshotRestoreWrapper<?> stateInfo : registeredPQStates.values()) {
stateMetaInfoSnapshots.add(stateInfo.getMetaInfo().snapshot());
heapPriorityQueuesSnapshots.add(stateInfo.stateSnapshot());
}

final ResourceGuard.Lease lease = rocksDBResourceGuard.acquireResource();
final Snapshot snapshot = db.getSnapshot();

return new RocksDBFullSnapshotResources<>(
lease,
snapshot,
metaDataCopy,
heapPriorityQueuesSnapshots,
stateMetaInfoSnapshots,
return RocksDBFullSnapshotResources.create(
kvStateInformation,
registeredPQStates,
db,
keyGroupPrefixBytes,
rocksDBResourceGuard,
keyGroupRange,
keySerializer,
keyGroupPrefixBytes,
keyGroupCompressionDecorator);
}

Expand Down

0 comments on commit 04f8abb

Please sign in to comment.