aljoscha commented on a change in pull request #14765: URL: https://github.com/apache/flink/pull/14765#discussion_r565458656
########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksDBFullSnapshotResources.java ########## @@ -0,0 +1,206 @@ +/* + * 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.snapshot; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; +import org.apache.flink.contrib.streaming.state.RocksIteratorWrapper; +import org.apache.flink.contrib.streaming.state.iterator.RocksStatesPerKeyGroupMergeIterator; +import org.apache.flink.contrib.streaming.state.iterator.RocksTransformingIteratorWrapper; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.state.FullSnapshotResources; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyValueStateIterator; +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.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksIterator; +import org.rocksdb.Snapshot; + +import javax.annotation.Nonnegative; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** A {@link FullSnapshotResources} for the RocksDB backend. */ +class RocksDBFullSnapshotResources<K> implements FullSnapshotResources<K> { + private final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots; + private final ResourceGuard.Lease lease; + private final Snapshot snapshot; + private final RocksDB db; + private final List<MetaData> metaData; + + /** Number of bytes in the key-group prefix. */ + @Nonnegative private final int keyGroupPrefixBytes; + + private final KeyGroupRange keyGroupRange; + private final TypeSerializer<K> keySerializer; + private final StreamCompressionDecorator streamCompressionDecorator; + + public RocksDBFullSnapshotResources( + ResourceGuard.Lease lease, + Snapshot snapshot, + List<RocksDBKeyedStateBackend.RocksDbKvStateInfo> metaDataCopy, + List<StateMetaInfoSnapshot> stateMetaInfoSnapshots, + RocksDB db, + int keyGroupPrefixBytes, + KeyGroupRange keyGroupRange, + TypeSerializer<K> keySerializer, + StreamCompressionDecorator streamCompressionDecorator) { + this.lease = lease; + this.snapshot = snapshot; + this.stateMetaInfoSnapshots = stateMetaInfoSnapshots; + this.db = db; + this.keyGroupPrefixBytes = keyGroupPrefixBytes; + this.keyGroupRange = keyGroupRange; + this.keySerializer = keySerializer; + this.streamCompressionDecorator = streamCompressionDecorator; + + // we need to to this in the constructor, i.e. in the synchronous part of the snapshot + // TODO: better yet, we can do it outside the constructor + this.metaData = fillMetaData(metaDataCopy); + } + + private List<MetaData> fillMetaData( + List<RocksDBKeyedStateBackend.RocksDbKvStateInfo> metaDataCopy) { + List<MetaData> metaData = new ArrayList<>(metaDataCopy.size()); + for (RocksDBKeyedStateBackend.RocksDbKvStateInfo rocksDbKvStateInfo : metaDataCopy) { + StateSnapshotTransformer<byte[]> stateSnapshotTransformer = null; + if (rocksDbKvStateInfo.metaInfo instanceof RegisteredKeyValueStateBackendMetaInfo) { + stateSnapshotTransformer = + ((RegisteredKeyValueStateBackendMetaInfo<?, ?>) rocksDbKvStateInfo.metaInfo) + .getStateSnapshotTransformFactory() + .createForSerializedState() + .orElse(null); + } + metaData.add(new MetaData(rocksDbKvStateInfo, stateSnapshotTransformer)); + } + return metaData; + } + + @Override + public KeyValueStateIterator createKVStateIterator() throws IOException { + CloseableRegistry closeableRegistry = new CloseableRegistry(); + + try { + ReadOptions readOptions = new ReadOptions(); + closeableRegistry.registerCloseable(readOptions::close); + readOptions.setSnapshot(snapshot); + + List<Tuple2<RocksIteratorWrapper, Integer>> kvStateIterators = + createKVStateIterators(readOptions); + + for (Tuple2<RocksIteratorWrapper, Integer> iter : kvStateIterators) { Review comment: fixing ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/FullSnapshotAsyncWriter.java ########## @@ -0,0 +1,233 @@ +/* + * 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.api.common.typeutils.base.array.BytePrimitiveArraySerializer; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.function.SupplierWithException; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Objects; + +import static org.apache.flink.runtime.state.FullSnapshotUtil.END_OF_KEY_GROUP_MARK; +import static org.apache.flink.runtime.state.FullSnapshotUtil.hasMetaDataFollowsFlag; +import static org.apache.flink.runtime.state.FullSnapshotUtil.setMetaDataFollowsFlagInKey; + +/** + * An asynchronous writer that can write a full snapshot/savepoint from a {@link + * FullSnapshotResources}. + * + * @param <K> type of the backend keys. + */ +public class FullSnapshotAsyncWriter<K> Review comment: Can do, I have two thoughts on it: - the `RocksFullSnapshotStrategy` still uses this, and not just for savepoints - my latest commit introduces a `SavepointSnapshotStrategy` that uses this. In my head the `FullSnapshotAsyncWriter` is an implementation detail that happens to also be used for savepoints Agreed that `Full` is not very clear here. Also, I'm not at all set on the naming. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
