diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index 5c0e66fa0e421..8ec1a86fe0758 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -187,7 +187,16 @@ public StreamOperatorStateContext streamOperatorStateContext( try { // -------------- Keyed State Backend -------------- - // TODO: Support KeyedStateBackend for AsyncKeyedStateBackend to unify the logic + keyedStatedBackend = + keyedStatedBackend( + keySerializer, + operatorIdentifierText, + prioritizedOperatorSubtaskStates, + streamTaskCloseableRegistry, + metricGroup, + managedMemoryFraction, + statsCollector, + StateBackend::createKeyedStateBackend); if (stateBackend.supportsAsyncKeyedStateBackend()) { asyncKeyedStateBackend = keyedStatedBackend( @@ -200,16 +209,6 @@ public StreamOperatorStateContext streamOperatorStateContext( statsCollector, StateBackend::createAsyncKeyedStateBackend); } else { - keyedStatedBackend = - keyedStatedBackend( - keySerializer, - operatorIdentifierText, - prioritizedOperatorSubtaskStates, - streamTaskCloseableRegistry, - metricGroup, - managedMemoryFraction, - statsCollector, - StateBackend::createKeyedStateBackend); if (keyedStatedBackend != null) { asyncKeyedStateBackend = new AsyncKeyedStateBackendAdaptor<>(keyedStatedBackend); diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStConfigurableOptions.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStConfigurableOptions.java index d811199e5e01a..bf2b0c9f00915 100644 --- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStConfigurableOptions.java +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStConfigurableOptions.java @@ -30,6 +30,7 @@ import java.io.File; import java.io.Serializable; +import java.time.Duration; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -300,6 +301,25 @@ public class ForStConfigurableOptions implements Serializable { "If true, ForSt will use block-based filter instead of full filter, this only take effect when bloom filter is used. " + "The default value is 'false'."); + public static final ConfigOption COMPACT_FILTER_QUERY_TIME_AFTER_NUM_ENTRIES = + key("state.backend.forst.compaction.filter.query-time-after-num-entries") + .longType() + .defaultValue(1000L) + .withDescription( + "Number of state entries to process by compaction filter before updating current timestamp. " + + "Updating the timestamp more often can improve cleanup speed, " + + "but it decreases compaction performance because it uses JNI calls from native code.The default value is '1000L'."); + + public static final ConfigOption COMPACT_FILTER_PERIODIC_COMPACTION_TIME = + key("state.backend.forst.compaction.filter.periodic-compaction-time") + .durationType() + .defaultValue(Duration.ofDays(30)) + .withDescription( + "Periodic compaction could speed up expired state entries cleanup, especially for state" + + " entries rarely accessed. Files older than this value will be picked up for compaction," + + " and re-written to the same level as they were before. It makes sure a file goes through" + + " compaction filters periodically. 0 means turning off periodic compaction.The default value is '30days'."); + static final ConfigOption[] CANDIDATE_CONFIGS = new ConfigOption[] { // configurable DBOptions diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStDBTtlCompactFiltersManager.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStDBTtlCompactFiltersManager.java new file mode 100644 index 0000000000000..f81119edef1f8 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStDBTtlCompactFiltersManager.java @@ -0,0 +1,259 @@ +/* + * 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.state.forst; + +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.state.StateTtlConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase; +import org.apache.flink.runtime.state.ttl.TtlStateFactory; +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; +import org.apache.flink.runtime.state.ttl.TtlUtils; +import org.apache.flink.runtime.state.ttl.TtlValue; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.Preconditions; + +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.FlinkCompactionFilter; +import org.rocksdb.FlinkCompactionFilter.FlinkCompactionFilterFactory; +import org.rocksdb.InfoLogLevel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.time.Duration; +import java.util.LinkedHashMap; + +/** RocksDB compaction filter utils for state with TTL. */ +public class ForStDBTtlCompactFiltersManager { + private static final Logger LOG = LoggerFactory.getLogger(FlinkCompactionFilter.class); + + private final TtlTimeProvider ttlTimeProvider; + + /** Registered compaction filter factories. */ + private final LinkedHashMap compactionFilterFactories; + + /** Created column family options. */ + private final LinkedHashMap columnFamilyOptionsMap; + + /** + * Number of state entries to process by compaction filter before updating current timestamp. + */ + private final long queryTimeAfterNumEntries; + + /** + * Periodic compaction could speed up expired state entries cleanup, especially for state + * entries rarely accessed. Files older than this value will be picked up for compaction, and + * re-written to the same level as they were before. It makes sure a file goes through + * compaction filters periodically. 0 means turning off periodic compaction. + */ + private final Duration periodicCompactionTime; + + public ForStDBTtlCompactFiltersManager( + TtlTimeProvider ttlTimeProvider, + long queryTimeAfterNumEntries, + Duration periodicCompactionTime) { + this.ttlTimeProvider = ttlTimeProvider; + this.queryTimeAfterNumEntries = queryTimeAfterNumEntries; + this.periodicCompactionTime = periodicCompactionTime; + this.compactionFilterFactories = new LinkedHashMap<>(); + this.columnFamilyOptionsMap = new LinkedHashMap<>(); + } + + public void setAndRegisterCompactFilterIfStateTtl( + @Nonnull RegisteredStateMetaInfoBase metaInfoBase, + @Nonnull ColumnFamilyOptions options) { + + if (metaInfoBase instanceof RegisteredKeyValueStateBackendMetaInfo) { + RegisteredKeyValueStateBackendMetaInfo kvMetaInfoBase = + (RegisteredKeyValueStateBackendMetaInfo) metaInfoBase; + if (TtlStateFactory.TtlSerializer.isTtlStateSerializer( + kvMetaInfoBase.getStateSerializer())) { + createAndSetCompactFilterFactory(metaInfoBase.getName(), options); + } + } + } + + private void createAndSetCompactFilterFactory( + String stateName, @Nonnull ColumnFamilyOptions options) { + + FlinkCompactionFilterFactory compactionFilterFactory = + new FlinkCompactionFilterFactory( + new TimeProviderWrapper(ttlTimeProvider), createRocksDbNativeLogger()); + //noinspection resource + options.setCompactionFilterFactory(compactionFilterFactory); + compactionFilterFactories.put(stateName, compactionFilterFactory); + columnFamilyOptionsMap.put(stateName, options); + } + + private static org.rocksdb.Logger createRocksDbNativeLogger() { + if (LOG.isDebugEnabled()) { + // options are always needed for org.rocksdb.Logger construction (no other constructor) + // the logger level gets configured from the options in native code + try (DBOptions opts = new DBOptions().setInfoLogLevel(InfoLogLevel.DEBUG_LEVEL)) { + return new org.rocksdb.Logger(opts) { + @Override + protected void log(InfoLogLevel infoLogLevel, String logMsg) { + LOG.debug("RocksDB filter native code log: " + logMsg); + } + }; + } + } else { + return null; + } + } + + public void configCompactFilter( + @Nonnull StateDescriptor stateDesc, TypeSerializer stateSerializer) { + StateTtlConfig ttlConfig = stateDesc.getTtlConfig(); + if (ttlConfig.isEnabled() && ttlConfig.getCleanupStrategies().inRocksdbCompactFilter()) { + FlinkCompactionFilterFactory compactionFilterFactory = + compactionFilterFactories.get(stateDesc.getName()); + Preconditions.checkNotNull(compactionFilterFactory); + long ttl = ttlConfig.getTimeToLive().toMillis(); + + ColumnFamilyOptions columnFamilyOptions = + columnFamilyOptionsMap.get(stateDesc.getName()); + Preconditions.checkNotNull(columnFamilyOptions); + + StateTtlConfig.RocksdbCompactFilterCleanupStrategy rocksdbCompactFilterCleanupStrategy = + ttlConfig.getCleanupStrategies().getRocksdbCompactFilterCleanupStrategy(); + + Duration periodicCompactionTime = this.periodicCompactionTime; + long queryTimeAfterNumEntries = this.queryTimeAfterNumEntries; + + if (rocksdbCompactFilterCleanupStrategy != null) { + periodicCompactionTime = + rocksdbCompactFilterCleanupStrategy.getPeriodicCompactionTime(); + queryTimeAfterNumEntries = + rocksdbCompactFilterCleanupStrategy.getQueryTimeAfterNumEntries(); + } + if (periodicCompactionTime != null) { + columnFamilyOptions.setPeriodicCompactionSeconds( + periodicCompactionTime.getSeconds()); + } + + FlinkCompactionFilter.Config config; + if (stateDesc instanceof ListStateDescriptor) { + TypeSerializer elemSerializer = + ((ListSerializer) stateSerializer).getElementSerializer(); + int len = elemSerializer.getLength(); + if (len > 0) { + config = + FlinkCompactionFilter.Config.createForFixedElementList( + ttl, + queryTimeAfterNumEntries, + len + 1); // plus one byte for list element delimiter + } else { + config = + FlinkCompactionFilter.Config.createForList( + ttl, + queryTimeAfterNumEntries, + new ListElementFilterFactory<>(elemSerializer.duplicate())); + } + } else if (stateDesc instanceof MapStateDescriptor) { + config = FlinkCompactionFilter.Config.createForMap(ttl, queryTimeAfterNumEntries); + } else { + config = FlinkCompactionFilter.Config.createForValue(ttl, queryTimeAfterNumEntries); + } + compactionFilterFactory.configure(config); + } + } + + private static class ListElementFilterFactory + implements FlinkCompactionFilter.ListElementFilterFactory { + private final TypeSerializer serializer; + + private ListElementFilterFactory(TypeSerializer serializer) { + this.serializer = serializer; + } + + @Override + public FlinkCompactionFilter.ListElementFilter createListElementFilter() { + return new ListElementFilter<>(serializer); + } + } + + private static class TimeProviderWrapper implements FlinkCompactionFilter.TimeProvider { + private final TtlTimeProvider ttlTimeProvider; + + private TimeProviderWrapper(TtlTimeProvider ttlTimeProvider) { + this.ttlTimeProvider = ttlTimeProvider; + } + + @Override + public long currentTimestamp() { + return ttlTimeProvider.currentTimestamp(); + } + } + + private static class ListElementFilter implements FlinkCompactionFilter.ListElementFilter { + private final TypeSerializer serializer; + private DataInputDeserializer input; + + private ListElementFilter(TypeSerializer serializer) { + this.serializer = serializer; + this.input = new DataInputDeserializer(); + } + + @Override + public int nextUnexpiredOffset(byte[] bytes, long ttl, long currentTimestamp) { + input.setBuffer(bytes); + int lastElementOffset = 0; + while (input.available() > 0) { + try { + long timestamp = nextElementLastAccessTimestamp(); + if (!TtlUtils.expired(timestamp, ttl, currentTimestamp)) { + break; + } + lastElementOffset = input.getPosition(); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Failed to deserialize list element for TTL compaction filter", e); + } + } + return lastElementOffset; + } + + private long nextElementLastAccessTimestamp() throws IOException { + TtlValue ttlValue = (TtlValue) serializer.deserialize(input); + if (input.available() > 0) { + input.skipBytesToRead(1); + } + return ttlValue.getLastAccessTimestamp(); + } + } + + public void disposeAndClearRegisteredCompactionFactories() { + for (FlinkCompactionFilterFactory factory : compactionFilterFactories.values()) { + IOUtils.closeQuietly(factory); + } + compactionFilterFactories.clear(); + columnFamilyOptionsMap.clear(); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStDBWriteBatchWrapper.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStDBWriteBatchWrapper.java index e166cda08e0db..14eff409ab3d4 100644 --- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStDBWriteBatchWrapper.java +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStDBWriteBatchWrapper.java @@ -32,6 +32,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.io.Closeable; import java.util.ArrayList; import java.util.List; @@ -61,6 +62,8 @@ public class ForStDBWriteBatchWrapper implements AutoCloseable { /** List of all objects that we need to close in close(). */ private final List toClose; + private volatile boolean cancelled; + public ForStDBWriteBatchWrapper(@Nonnull RocksDB rocksDB, long writeBatchSize) { this(rocksDB, null, 500, writeBatchSize); } @@ -157,4 +160,12 @@ private void flushIfNeeded() throws RocksDBException { long getDataSize() { return batch.getDataSize(); } + + public void markCancelled() { + this.cancelled = true; + } + + public Closeable getCancelCloseable() { + return this::markCancelled; + } } diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOperationUtils.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOperationUtils.java index d3e3ec7e98db5..c74abb018972c 100644 --- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOperationUtils.java +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOperationUtils.java @@ -18,8 +18,12 @@ package org.apache.flink.state.forst; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.core.fs.ICloseableRegistry; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.memory.OpaqueMemoryResource; +import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase; +import org.apache.flink.state.forst.sync.ForStIteratorWrapper; +import org.apache.flink.state.forst.sync.ForStSyncKeyedStateBackend; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.IOUtils; import org.apache.flink.util.OperatingSystem; @@ -29,21 +33,28 @@ import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.ColumnFamilyOptions; import org.rocksdb.DBOptions; +import org.rocksdb.ExportImportFilesMetaData; +import org.rocksdb.ReadOptions; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.function.Function; /** Utils for ForSt Operations. */ public class ForStOperationUtils { + private static final Logger LOG = LoggerFactory.getLogger(ForStOperationUtils.class); + /** * The name of the merge operator in ForSt. Do not change except you know exactly what you do. */ @@ -163,6 +174,177 @@ public static OpaqueMemoryResource allocateSharedCachesIfC } } + public static ForStIteratorWrapper getForStIterator( + RocksDB db, ColumnFamilyHandle columnFamilyHandle, ReadOptions readOptions) { + return new ForStIteratorWrapper(db.newIterator(columnFamilyHandle, readOptions)); + } + + public static void addColumnFamilyOptionsToCloseLater( + List columnFamilyOptions, ColumnFamilyHandle columnFamilyHandle) { + try { + // IMPORTANT NOTE: Do not call ColumnFamilyHandle#getDescriptor() just to judge if it + // return null and then call it again when it return is not null. That will cause + // task manager native memory used by RocksDB can't be released timely after job + // restart. + // The problem can find in : https://issues.apache.org/jira/browse/FLINK-21986 + if (columnFamilyHandle != null) { + ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyHandle.getDescriptor(); + if (columnFamilyDescriptor != null) { + columnFamilyOptions.add(columnFamilyDescriptor.getOptions()); + } + } + } catch (RocksDBException e) { + // ignore + } + } + + /** + * Creates a state info from a new meta info to use with a k/v state. + * + *

Creates the column family for the state. Sets TTL compaction filter if {@code + * ttlCompactFiltersManager} is not {@code null}. + * + * @param importFilesMetaData if not empty, we import the files specified in the metadata to the + * column family. + */ + public static ForStSyncKeyedStateBackend.ForStDbKvStateInfo createStateInfo( + RegisteredStateMetaInfoBase metaInfoBase, + RocksDB db, + Function columnFamilyOptionsFactory, + @Nullable ForStDBTtlCompactFiltersManager ttlCompactFiltersManager, + @Nullable Long writeBufferManagerCapacity, + List importFilesMetaData, + ICloseableRegistry cancelStreamRegistryForRestore) { + + ColumnFamilyDescriptor columnFamilyDescriptor = + createColumnFamilyDescriptor( + metaInfoBase, + columnFamilyOptionsFactory, + ttlCompactFiltersManager, + writeBufferManagerCapacity); + + try { + ColumnFamilyHandle columnFamilyHandle = createColumnFamily(columnFamilyDescriptor, db); + return new ForStSyncKeyedStateBackend.ForStDbKvStateInfo( + columnFamilyHandle, metaInfoBase); + } catch (Exception ex) { + IOUtils.closeQuietly(columnFamilyDescriptor.getOptions()); + throw new FlinkRuntimeException("Error creating ColumnFamilyHandle.", ex); + } + } + + /** + * Create RocksDB-backed KV-state, including RocksDB ColumnFamily. + * + * @param cancelStreamRegistryForRestore {@link ICloseableRegistry#close closing} it interrupts + * KV state creation + */ + public static ForStSyncKeyedStateBackend.ForStDbKvStateInfo createStateInfo( + RegisteredStateMetaInfoBase metaInfoBase, + RocksDB db, + Function columnFamilyOptionsFactory, + @Nullable ForStDBTtlCompactFiltersManager ttlCompactFiltersManager, + @Nullable Long writeBufferManagerCapacity, + ICloseableRegistry cancelStreamRegistryForRestore) { + return createStateInfo( + metaInfoBase, + db, + columnFamilyOptionsFactory, + ttlCompactFiltersManager, + writeBufferManagerCapacity, + Collections.emptyList(), + cancelStreamRegistryForRestore); + } + + /** + * Creates a column descriptor for a state column family. + * + *

Sets TTL compaction filter if {@code ttlCompactFiltersManager} is not {@code null}. + */ + public static ColumnFamilyDescriptor createColumnFamilyDescriptor( + RegisteredStateMetaInfoBase metaInfoBase, + Function columnFamilyOptionsFactory, + @Nullable ForStDBTtlCompactFiltersManager ttlCompactFiltersManager, + @Nullable Long writeBufferManagerCapacity) { + + byte[] nameBytes = metaInfoBase.getName().getBytes(ConfigConstants.DEFAULT_CHARSET); + Preconditions.checkState( + !Arrays.equals(RocksDB.DEFAULT_COLUMN_FAMILY, nameBytes), + "The chosen state name 'default' collides with the name of the default column family!"); + + ColumnFamilyOptions options = + createColumnFamilyOptions(columnFamilyOptionsFactory, metaInfoBase.getName()); + + if (ttlCompactFiltersManager != null) { + ttlCompactFiltersManager.setAndRegisterCompactFilterIfStateTtl(metaInfoBase, options); + } + + if (writeBufferManagerCapacity != null) { + // It'd be great to perform the check earlier, e.g. when creating write buffer manager. + // Unfortunately the check needs write buffer size that was just calculated. + sanityCheckArenaBlockSize( + options.writeBufferSize(), + options.arenaBlockSize(), + writeBufferManagerCapacity); + } + + return new ColumnFamilyDescriptor(nameBytes, options); + } + + /** + * Logs a warning if the arena block size is too high causing RocksDB to flush constantly. + * Essentially, the condition + * here will always be true. + * + * @param writeBufferSize the size of write buffer (bytes) + * @param arenaBlockSizeConfigured the manually configured arena block size, zero or less means + * not configured + * @param writeBufferManagerCapacity the size of the write buffer manager (bytes) + * @return true if sanity check passes, false otherwise + */ + static boolean sanityCheckArenaBlockSize( + long writeBufferSize, long arenaBlockSizeConfigured, long writeBufferManagerCapacity) { + + long defaultArenaBlockSize = + ForStMemoryControllerUtils.calculateForStDefaultArenaBlockSize(writeBufferSize); + long arenaBlockSize = + arenaBlockSizeConfigured <= 0 ? defaultArenaBlockSize : arenaBlockSizeConfigured; + long mutableLimit = + ForStMemoryControllerUtils.calculateForStMutableLimit(writeBufferManagerCapacity); + if (ForStMemoryControllerUtils.validateArenaBlockSize(arenaBlockSize, mutableLimit)) { + return true; + } else { + LOG.warn( + "ForStStateBackend performance will be poor because of the current Flink memory configuration! " + + "RocksDB will flush memtable constantly, causing high IO and CPU. " + + "Typically the easiest fix is to increase task manager managed memory size. " + + "If running locally, see the parameter taskmanager.memory.managed.size. " + + "Details: arenaBlockSize {} > mutableLimit {} (writeBufferSize = {}, arenaBlockSizeConfigured = {}," + + " defaultArenaBlockSize = {}, writeBufferManagerCapacity = {})", + arenaBlockSize, + mutableLimit, + writeBufferSize, + arenaBlockSizeConfigured, + defaultArenaBlockSize, + writeBufferManagerCapacity); + return false; + } + } + + public static void registerKvStateInformation( + Map kvStateInformation, + ForStNativeMetricMonitor nativeMetricMonitor, + String columnFamilyName, + ForStSyncKeyedStateBackend.ForStDbKvStateInfo registeredColumn) { + + kvStateInformation.put(columnFamilyName, registeredColumn); + if (nativeMetricMonitor != null) { + nativeMetricMonitor.registerColumnFamily( + columnFamilyName, registeredColumn.columnFamilyHandle); + } + } + private static void throwExceptionIfPathLengthExceededOnWindows(String path, Exception cause) throws IOException { // max directory path length on Windows is 247. diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java index 2ff6d2e1534a2..7ab0fb2389848 100644 --- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java @@ -19,6 +19,7 @@ package org.apache.flink.state.forst; import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; @@ -26,6 +27,8 @@ import org.apache.flink.configuration.description.Description; import org.apache.flink.configuration.description.TextElement; +import static org.apache.flink.state.forst.ForStStateBackend.PriorityQueueStateType.ForStDB; + /** Configuration options for the ForStStateBackend. */ @Experimental public class ForStOptions { @@ -130,4 +133,27 @@ public class ForStOptions { + "the partitions that are required to perform the index/filter query. " + "This option only has an effect when '%s' or '%s' are configured.", USE_MANAGED_MEMORY.key(), FIX_PER_SLOT_MEMORY_SIZE.key())); + + /** Choice of timer service implementation. */ + @Documentation.Section(Documentation.Sections.STATE_BACKEND_ROCKSDB) + public static final ConfigOption + TIMER_SERVICE_FACTORY = + ConfigOptions.key("state.backend.forst.timer-service.factory") + .enumType(ForStStateBackend.PriorityQueueStateType.class) + .defaultValue(ForStDB) + .withDescription( + "This determines the factory for timer service state implementation."); + + /** The cache size per key-group for ROCKSDB timer service factory implementation. */ + @Documentation.Section(Documentation.Sections.STATE_BACKEND_ROCKSDB) + public static final ConfigOption FORST_TIMER_SERVICE_FACTORY_CACHE_SIZE = + ConfigOptions.key("state.backend.forst.timer-service.cache-size") + .intType() + .defaultValue(128) + .withDescription( + String.format( + "The cache size per keyGroup of rocksdb timer service factory. This option only has an effect " + + "when '%s' is configured to '%s'. Increasing this value can improve the performance " + + "of rocksdb timer service, but consumes more heap memory at the same time.", + TIMER_SERVICE_FACTORY.key(), ForStDB.name())); } diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java index 52909b3e3a050..66699923bbe1e 100644 --- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java @@ -51,6 +51,7 @@ import java.io.File; import java.io.IOException; import java.nio.file.Files; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -580,4 +581,29 @@ private String resolveRelocatedDbLogPrefix(String instanceForStAbsolutePath) { return instanceForStAbsolutePath.replaceAll("[^a-zA-Z0-9\\-._]", "_") + FORST_RELOCATE_LOG_SUFFIX; } + + /** + * Gets write buffer manager capacity. + * + * @return the capacity of the write buffer manager, or null if write buffer manager is not + * enabled. + */ + public Long getWriteBufferManagerCapacity() { + if (sharedResources == null) { + return null; + } + + return sharedResources.getResourceHandle().getWriteBufferManagerCapacity(); + } + + /** Gets the "queryTimeAfterNumEntries" parameter from the configuration. */ + public Long getQueryTimeAfterNumEntries() { + return internalGetOption( + ForStConfigurableOptions.COMPACT_FILTER_QUERY_TIME_AFTER_NUM_ENTRIES); + } + + /** Gets the "getPeriodicCompactionTime" parameter from the configuration. */ + public Duration getPeriodicCompactionTime() { + return internalGetOption(ForStConfigurableOptions.COMPACT_FILTER_PERIODIC_COMPACTION_TIME); + } } diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateBackend.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateBackend.java index 4fcf7327c126c..2dc2febfd40c8 100644 --- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateBackend.java +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStStateBackend.java @@ -20,11 +20,14 @@ import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DescribedEnum; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.configuration.description.InlineElement; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.memory.OpaqueMemoryResource; @@ -32,8 +35,13 @@ import org.apache.flink.runtime.state.AbstractManagedMemoryStateBackend; import org.apache.flink.runtime.state.ConfigurableStateBackend; import org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.StreamCompressionDecorator; +import org.apache.flink.runtime.state.metrics.LatencyTrackingStateConfig; import org.apache.flink.state.forst.ForStMemoryControllerUtils.ForStMemoryFactory; +import org.apache.flink.state.forst.sync.ForStPriorityQueueConfig; +import org.apache.flink.state.forst.sync.ForStSyncKeyedStateBackendBuilder; import org.apache.flink.util.AbstractID; import org.apache.flink.util.DynamicCodeLoadingException; import org.apache.flink.util.FileUtils; @@ -59,6 +67,8 @@ import java.util.UUID; import java.util.function.Supplier; +import static org.apache.flink.configuration.description.TextElement.text; + /** * A {@link org.apache.flink.runtime.state.StateBackend} that stores its state in a ForSt instance. * This state backend can store very large state that exceeds memory even disk and spills to remote @@ -127,12 +137,18 @@ public class ForStStateBackend extends AbstractManagedMemoryStateBackend /** Factory for Write Buffer Manager and Block Cache. */ private final ForStMemoryFactory forStMemoryFactory; + + /** + * The configuration for rocksdb priorityQueue state settings (priorityQueue state type, etc.). + */ + private final ForStPriorityQueueConfig priorityQueueConfig; // ------------------------------------------------------------------------ /** Creates a new {@code ForStStateBackend} for storing state. */ public ForStStateBackend() { this.nativeMetricOptions = new ForStNativeMetricOptions(); this.memoryConfiguration = new ForStMemoryConfiguration(); + this.priorityQueueConfig = new ForStPriorityQueueConfig(); this.forStMemoryFactory = ForStMemoryFactory.DEFAULT; } @@ -157,6 +173,10 @@ private ForStStateBackend( this.remoteForStDirectory = remoteDirStr == null ? null : new Path(remoteDirStr); } + this.priorityQueueConfig = + ForStPriorityQueueConfig.fromOtherAndConfiguration( + original.priorityQueueConfig, config); + // configure local directories if (original.localForStDirectories != null) { this.localForStDirectories = original.localForStDirectories; @@ -347,8 +367,80 @@ public ForStKeyedStateBackend createAsyncKeyedStateBackend( @Override public AbstractKeyedStateBackend createKeyedStateBackend( - KeyedStateBackendParameters parameters) { - throw new UnsupportedOperationException("Don't support createKeyedStateBackend yet"); + KeyedStateBackendParameters parameters) throws IOException { + Environment env = parameters.getEnv(); + + // first, make sure that the RocksDB JNI library is loaded + // we do this explicitly here to have better error handling + String tempDir = env.getTaskManagerInfo().getTmpWorkingDirectory().getAbsolutePath(); + ensureForStIsLoaded(tempDir); + + // replace all characters that are not legal for filenames with underscore + String fileCompatibleIdentifier = + parameters.getOperatorIdentifier().replaceAll("[^a-zA-Z0-9\\-]", "_"); + + lazyInitializeForJob(env, fileCompatibleIdentifier); + + File instanceBasePath = + new File( + getNextStoragePath(), + "job_" + + jobId + + "_op_" + + fileCompatibleIdentifier + + "_uuid_" + + UUID.randomUUID()); + + LocalRecoveryConfig localRecoveryConfig = + env.getTaskStateManager().createLocalRecoveryConfig(); + + final OpaqueMemoryResource sharedResources = + ForStOperationUtils.allocateSharedCachesIfConfigured( + memoryConfiguration, + env, + parameters.getManagedMemoryFraction(), + LOG, + forStMemoryFactory); + if (sharedResources != null) { + LOG.info("Obtained shared RocksDB cache of size {} bytes", sharedResources.getSize()); + } + final ForStResourceContainer resourceContainer = + createOptionsAndResourceContainer( + sharedResources, + instanceBasePath, + null, + nativeMetricOptions.isStatisticsEnabled()); + + ExecutionConfig executionConfig = env.getExecutionConfig(); + StreamCompressionDecorator keyGroupCompressionDecorator = + getCompressionDecorator(executionConfig); + + LatencyTrackingStateConfig latencyTrackingStateConfig = + latencyTrackingConfigBuilder.setMetricGroup(parameters.getMetricGroup()).build(); + ForStSyncKeyedStateBackendBuilder builder = + new ForStSyncKeyedStateBackendBuilder<>( + parameters.getOperatorIdentifier(), + env.getUserCodeClassLoader().asClassLoader(), + instanceBasePath, + resourceContainer, + stateName -> resourceContainer.getColumnOptions(), + parameters.getKvStateRegistry(), + parameters.getKeySerializer(), + parameters.getNumberOfKeyGroups(), + parameters.getKeyGroupRange(), + executionConfig, + localRecoveryConfig, + priorityQueueConfig, + parameters.getTtlTimeProvider(), + latencyTrackingStateConfig, + parameters.getMetricGroup(), + parameters.getCustomInitializationMetrics(), + parameters.getStateHandles(), + keyGroupCompressionDecorator, + parameters.getCancelStreamRegistry()) + .setNativeMetricOptions( + resourceContainer.getMemoryWatcherOptions(nativeMetricOptions)); + return builder.build(); } @Override @@ -691,4 +783,21 @@ static void resetForStLoadedFlag() throws Exception { initField.setAccessible(true); initField.setBoolean(null, false); } + + /** The options to chose for the type of priority queue state. */ + public enum PriorityQueueStateType implements DescribedEnum { + HEAP(text("Heap-based")), + ForStDB(text("Implementation based on RocksDB")); + + private final InlineElement description; + + PriorityQueueStateType(InlineElement description) { + this.description = description; + } + + @Override + public InlineElement getDescription() { + return description; + } + } } diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStStateKeysIterator.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStStateKeysIterator.java new file mode 100644 index 0000000000000..f59d1880ae165 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStStateKeysIterator.java @@ -0,0 +1,74 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.runtime.state.CompositeKeySerializationUtils; + +import javax.annotation.Nonnull; + +import java.io.IOException; + +/** + * Base class for iterators over RocksDB column families. + * + * @param the type of the iterated objects, which are keys in RocksDB. + */ +@Internal +public abstract class AbstractForStStateKeysIterator implements AutoCloseable { + + @Nonnull protected final ForStIteratorWrapper iterator; + + @Nonnull protected final String state; + + @Nonnull protected final TypeSerializer keySerializer; + + protected final boolean ambiguousKeyPossible; + + protected final int keyGroupPrefixBytes; + + protected final DataInputDeserializer byteArrayDataInputView; + + public AbstractForStStateKeysIterator( + @Nonnull ForStIteratorWrapper iterator, + @Nonnull String state, + @Nonnull TypeSerializer keySerializer, + int keyGroupPrefixBytes, + boolean ambiguousKeyPossible) { + this.iterator = iterator; + this.state = state; + this.keySerializer = keySerializer; + this.keyGroupPrefixBytes = keyGroupPrefixBytes; + this.ambiguousKeyPossible = ambiguousKeyPossible; + this.byteArrayDataInputView = new DataInputDeserializer(); + } + + protected K deserializeKey(byte[] keyBytes, DataInputDeserializer readView) throws IOException { + readView.setBuffer(keyBytes, keyGroupPrefixBytes, keyBytes.length - keyGroupPrefixBytes); + return CompositeKeySerializationUtils.readKey( + keySerializer, byteArrayDataInputView, ambiguousKeyPossible); + } + + @Override + public void close() { + iterator.close(); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStSyncAppendingState.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStSyncAppendingState.java new file mode 100644 index 0000000000000..c9d845f71902b --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStSyncAppendingState.java @@ -0,0 +1,74 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.state.internal.InternalAppendingState; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; + +import java.io.IOException; + +abstract class AbstractForStSyncAppendingState + extends AbstractForStSyncState + implements InternalAppendingState { + + /** + * Creates a new RocksDB backend appending state. + * + * @param columnFamily The RocksDB column family that this state is associated to. + * @param namespaceSerializer The serializer for the namespace. + * @param valueSerializer The serializer for the state. + * @param defaultValue The default value for the state. + * @param backend The backend for which this state is bind to. + */ + protected AbstractForStSyncAppendingState( + ColumnFamilyHandle columnFamily, + TypeSerializer namespaceSerializer, + TypeSerializer valueSerializer, + SV defaultValue, + ForStSyncKeyedStateBackend backend) { + super(columnFamily, namespaceSerializer, valueSerializer, defaultValue, backend); + } + + @Override + public SV getInternal() throws IOException, RocksDBException { + return getInternal(getKeyBytes()); + } + + SV getInternal(byte[] key) throws IOException, RocksDBException { + byte[] valueBytes = backend.db.get(columnFamily, key); + if (valueBytes == null) { + return null; + } + dataInputView.setBuffer(valueBytes); + return valueSerializer.deserialize(dataInputView); + } + + @Override + public void updateInternal(SV valueToStore) throws RocksDBException { + updateInternal(getKeyBytes(), valueToStore); + } + + void updateInternal(byte[] key, SV valueToStore) throws RocksDBException { + // write the new value to RocksDB + backend.db.put(columnFamily, writeOptions, key, getValueBytes(valueToStore)); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStSyncState.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStSyncState.java new file mode 100644 index 0000000000000..e9cea2834fbe1 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/AbstractForStSyncState.java @@ -0,0 +1,243 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder; +import org.apache.flink.runtime.state.internal.InternalKvState; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StateMigrationException; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; +import org.rocksdb.WriteOptions; + +import java.io.IOException; + +/** + * Base class for {@link State} implementations that store state in a RocksDB database. + * + *

State is not stored in this class but in the {@link org.rocksdb.RocksDB} instance that the + * {@link org.apache.flink.state.forst.ForStStateBackend} manages and checkpoints. + * + * @param The type of the key. + * @param The type of the namespace. + * @param The type of values kept internally in state. + */ +public abstract class AbstractForStSyncState implements InternalKvState, State { + + /** Serializer for the namespace. */ + TypeSerializer namespaceSerializer; + + /** Serializer for the state values. */ + TypeSerializer valueSerializer; + + /** The current namespace, which the next value methods will refer to. */ + private N currentNamespace; + + /** Backend that holds the actual RocksDB instance where we store state. */ + protected ForStSyncKeyedStateBackend backend; + + /** The column family of this particular instance of state. */ + protected ColumnFamilyHandle columnFamily; + + protected V defaultValue; + + protected final WriteOptions writeOptions; + + protected final DataOutputSerializer dataOutputView; + + protected final DataInputDeserializer dataInputView; + + private final SerializedCompositeKeyBuilder sharedKeyNamespaceSerializer; + + /** + * Creates a new RocksDB backed state. + * + * @param columnFamily The RocksDB column family that this state is associated to. + * @param namespaceSerializer The serializer for the namespace. + * @param valueSerializer The serializer for the state. + * @param defaultValue The default value for the state. + * @param backend The backend for which this state is bind to. + */ + protected AbstractForStSyncState( + ColumnFamilyHandle columnFamily, + TypeSerializer namespaceSerializer, + TypeSerializer valueSerializer, + V defaultValue, + ForStSyncKeyedStateBackend backend) { + + this.namespaceSerializer = namespaceSerializer; + this.backend = backend; + + this.columnFamily = columnFamily; + + this.writeOptions = backend.getWriteOptions(); + this.valueSerializer = + Preconditions.checkNotNull(valueSerializer, "State value serializer"); + this.defaultValue = defaultValue; + + this.dataOutputView = new DataOutputSerializer(128); + this.dataInputView = new DataInputDeserializer(); + this.sharedKeyNamespaceSerializer = backend.getSharedRocksKeyBuilder(); + } + + // ------------------------------------------------------------------------ + + @Override + public void clear() { + try { + backend.db.delete( + columnFamily, writeOptions, serializeCurrentKeyWithGroupAndNamespace()); + } catch (RocksDBException e) { + throw new FlinkRuntimeException("Error while removing entry from RocksDB", e); + } + } + + @Override + public void setCurrentNamespace(N namespace) { + this.currentNamespace = namespace; + } + + @Override + public byte[] getSerializedValue( + final byte[] serializedKeyAndNamespace, + final TypeSerializer safeKeySerializer, + final TypeSerializer safeNamespaceSerializer, + final TypeSerializer safeValueSerializer) + throws Exception { + + // TODO make KvStateSerializer key-group aware to save this round trip and key-group + // computation + Tuple2 keyAndNamespace = + KvStateSerializer.deserializeKeyAndNamespace( + serializedKeyAndNamespace, safeKeySerializer, safeNamespaceSerializer); + + int keyGroup = + KeyGroupRangeAssignment.assignToKeyGroup( + keyAndNamespace.f0, backend.getNumberOfKeyGroups()); + + SerializedCompositeKeyBuilder keyBuilder = + new SerializedCompositeKeyBuilder<>( + safeKeySerializer, backend.getKeyGroupPrefixBytes(), 32); + keyBuilder.setKeyAndKeyGroup(keyAndNamespace.f0, keyGroup); + byte[] key = keyBuilder.buildCompositeKeyNamespace(keyAndNamespace.f1, namespaceSerializer); + return backend.db.get(columnFamily, key); + } + + byte[] serializeCurrentKeyWithGroupAndNamespacePlusUserKey( + UK userKey, TypeSerializer userKeySerializer) throws IOException { + return sharedKeyNamespaceSerializer.buildCompositeKeyNamesSpaceUserKey( + currentNamespace, namespaceSerializer, userKey, userKeySerializer); + } + + private byte[] serializeValueInternal(T value, TypeSerializer serializer) + throws IOException { + serializer.serialize(value, dataOutputView); + return dataOutputView.getCopyOfBuffer(); + } + + byte[] serializeCurrentKeyWithGroupAndNamespace() { + return sharedKeyNamespaceSerializer.buildCompositeKeyNamespace( + currentNamespace, namespaceSerializer); + } + + byte[] serializeValue(V value) throws IOException { + return serializeValue(value, valueSerializer); + } + + byte[] serializeValueNullSensitive(T value, TypeSerializer serializer) + throws IOException { + dataOutputView.clear(); + dataOutputView.writeBoolean(value == null); + return serializeValueInternal(value, serializer); + } + + byte[] serializeValue(T value, TypeSerializer serializer) throws IOException { + dataOutputView.clear(); + return serializeValueInternal(value, serializer); + } + + public void migrateSerializedValue( + DataInputDeserializer serializedOldValueInput, + DataOutputSerializer serializedMigratedValueOutput, + TypeSerializer priorSerializer, + TypeSerializer newSerializer) + throws StateMigrationException { + + try { + V value = priorSerializer.deserialize(serializedOldValueInput); + newSerializer.serialize(value, serializedMigratedValueOutput); + } catch (Exception e) { + throw new StateMigrationException("Error while trying to migrate RocksDB state.", e); + } + } + + byte[] getKeyBytes() { + return serializeCurrentKeyWithGroupAndNamespace(); + } + + byte[] getValueBytes(V value) { + try { + dataOutputView.clear(); + valueSerializer.serialize(value, dataOutputView); + return dataOutputView.getCopyOfBuffer(); + } catch (IOException e) { + throw new FlinkRuntimeException("Error while serializing value", e); + } + } + + protected V getDefaultValue() { + if (defaultValue != null) { + return valueSerializer.copy(defaultValue); + } else { + return null; + } + } + + protected AbstractForStSyncState setNamespaceSerializer( + TypeSerializer namespaceSerializer) { + this.namespaceSerializer = namespaceSerializer; + return this; + } + + protected AbstractForStSyncState setValueSerializer( + TypeSerializer valueSerializer) { + this.valueSerializer = valueSerializer; + return this; + } + + protected AbstractForStSyncState setDefaultValue(V defaultValue) { + this.defaultValue = defaultValue; + return this; + } + + @Override + public StateIncrementalVisitor getStateIncrementalVisitor( + int recommendedMaxNumberOfReturnedRecords) { + throw new UnsupportedOperationException( + "Global state entry iterator is unsupported for RocksDb backend"); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStDBCachingPriorityQueueSet.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStDBCachingPriorityQueueSet.java new file mode 100644 index 0000000000000..6ae049f39df86 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStDBCachingPriorityQueueSet.java @@ -0,0 +1,527 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.runtime.state.CompositeKeySerializationUtils; +import org.apache.flink.runtime.state.InternalPriorityQueue; +import org.apache.flink.runtime.state.heap.AbstractHeapPriorityQueueElement; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.state.forst.ForStDBWriteBatchWrapper; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.flink.shaded.guava32.com.google.common.primitives.UnsignedBytes; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.TreeSet; + +/** + * A priority queue with set semantics, implemented on top of RocksDB. This uses a {@link TreeSet} + * to cache the bytes of up to the first n elements from RocksDB in memory to reduce interaction + * with RocksDB, in particular seek operations. Cache uses a simple write-through policy. + * + * @param the type of the contained elements in the queue. + */ +public class ForStDBCachingPriorityQueueSet + extends AbstractHeapPriorityQueueElement implements InternalPriorityQueue { + + /** Serialized empty value to insert into RocksDB. */ + private static final byte[] DUMMY_BYTES = new byte[] {}; + + /** The RocksDB instance that serves as store. */ + @Nonnull private final RocksDB db; + + @Nonnull private final ReadOptions readOptions; + + /** Handle to the column family of the RocksDB instance in which the elements are stored. */ + @Nonnull private final ColumnFamilyHandle columnFamilyHandle; + + /** + * Serializer for the contained elements. The lexicographical order of the bytes of serialized + * objects must be aligned with their logical order. + */ + @Nonnull private final TypeSerializer byteOrderProducingSerializer; + + /** Wrapper to batch all writes to RocksDB. */ + @Nonnull private final ForStDBWriteBatchWrapper batchWrapper; + + /** The key-group id in serialized form. */ + @Nonnull private final byte[] groupPrefixBytes; + + /** Output view that helps to serialize elements. */ + @Nonnull private final DataOutputSerializer outputView; + + /** Input view that helps to de-serialize elements. */ + @Nonnull private final DataInputDeserializer inputView; + + /** In memory cache that holds a head-subset of the elements stored in RocksDB. */ + @Nonnull private final OrderedByteArraySetCache orderedCache; + + /** + * This holds the key that we use to seek to the first element in RocksDB, to improve + * seek/iterator performance. + */ + @Nonnull private byte[] seekHint; + + /** Cache for the head element in de-serialized form. */ + @Nullable private E peekCache; + + /** This flag is true iff all elements in RocksDB are also contained in the cache. */ + private boolean allElementsInCache; + + ForStDBCachingPriorityQueueSet( + @Nonnegative int keyGroupId, + @Nonnegative int keyGroupPrefixBytes, + @Nonnull RocksDB db, + @Nonnull ReadOptions readOptions, + @Nonnull ColumnFamilyHandle columnFamilyHandle, + @Nonnull TypeSerializer byteOrderProducingSerializer, + @Nonnull DataOutputSerializer outputStream, + @Nonnull DataInputDeserializer inputStream, + @Nonnull ForStDBWriteBatchWrapper batchWrapper, + @Nonnull OrderedByteArraySetCache orderedByteArraySetCache) { + this.db = db; + this.readOptions = readOptions; + this.columnFamilyHandle = columnFamilyHandle; + this.byteOrderProducingSerializer = byteOrderProducingSerializer; + this.batchWrapper = batchWrapper; + this.outputView = outputStream; + this.inputView = inputStream; + this.orderedCache = orderedByteArraySetCache; + this.allElementsInCache = false; + this.groupPrefixBytes = createKeyGroupBytes(keyGroupId, keyGroupPrefixBytes); + this.seekHint = groupPrefixBytes; + } + + @Nullable + @Override + public E peek() { + + checkRefillCacheFromStore(); + + if (peekCache != null) { + return peekCache; + } + + byte[] firstBytes = orderedCache.peekFirst(); + if (firstBytes != null) { + peekCache = deserializeElement(firstBytes); + return peekCache; + } else { + return null; + } + } + + @Nullable + @Override + public E poll() { + + checkRefillCacheFromStore(); + + final byte[] firstBytes = orderedCache.pollFirst(); + + if (firstBytes == null) { + return null; + } + + // write-through sync + removeFromRocksDB(firstBytes); + + if (orderedCache.isEmpty()) { + seekHint = firstBytes; + } + + if (peekCache != null) { + E fromCache = peekCache; + peekCache = null; + return fromCache; + } else { + return deserializeElement(firstBytes); + } + } + + @Override + public boolean add(@Nonnull E toAdd) { + + checkRefillCacheFromStore(); + + final byte[] toAddBytes = serializeElement(toAdd); + + final boolean cacheFull = orderedCache.isFull(); + + if ((!cacheFull && allElementsInCache) + || OrderedByteArraySetCache.LEXICOGRAPHIC_BYTE_COMPARATOR.compare( + toAddBytes, orderedCache.peekLast()) + < 0) { + + if (cacheFull) { + // we drop the element with lowest priority from the cache + orderedCache.pollLast(); + // the dropped element is now only in the store + allElementsInCache = false; + } + + if (orderedCache.add(toAddBytes)) { + // write-through sync + addToRocksDB(toAddBytes); + if (toAddBytes == orderedCache.peekFirst()) { + peekCache = null; + return true; + } + } + } else { + // we only added to the store + addToRocksDB(toAddBytes); + allElementsInCache = false; + } + return false; + } + + @Override + public boolean remove(@Nonnull E toRemove) { + + checkRefillCacheFromStore(); + + final byte[] oldHead = orderedCache.peekFirst(); + + if (oldHead == null) { + return false; + } + + final byte[] toRemoveBytes = serializeElement(toRemove); + + // write-through sync + removeFromRocksDB(toRemoveBytes); + orderedCache.remove(toRemoveBytes); + + if (orderedCache.isEmpty()) { + seekHint = toRemoveBytes; + peekCache = null; + return true; + } + + if (oldHead != orderedCache.peekFirst()) { + peekCache = null; + return true; + } + + return false; + } + + @Override + public void addAll(@Nullable Collection toAdd) { + + if (toAdd == null) { + return; + } + + for (E element : toAdd) { + add(element); + } + } + + @Override + public boolean isEmpty() { + checkRefillCacheFromStore(); + return orderedCache.isEmpty(); + } + + @Nonnull + @Override + public CloseableIterator iterator() { + return new DeserializingIteratorWrapper(orderedBytesIterator()); + } + + /** + * This implementation comes at a relatively high cost per invocation. It should not be called + * repeatedly when it is clear that the value did not change. Currently this is only truly used + * to realize certain higher-level tests. + */ + @Override + public int size() { + + if (allElementsInCache) { + return orderedCache.size(); + } else { + int count = 0; + try (final RocksBytesIterator iterator = orderedBytesIterator()) { + while (iterator.hasNext()) { + iterator.next(); + ++count; + } + } + return count; + } + } + + @Nonnull + private RocksBytesIterator orderedBytesIterator() { + flushWriteBatch(); + return new RocksBytesIterator( + new ForStIteratorWrapper(db.newIterator(columnFamilyHandle, readOptions))); + } + + /** Ensures that recent writes are flushed and reflect in the RocksDB instance. */ + private void flushWriteBatch() { + try { + batchWrapper.flush(); + } catch (RocksDBException e) { + throw new FlinkRuntimeException(e); + } + } + + private void addToRocksDB(@Nonnull byte[] toAddBytes) { + try { + batchWrapper.put(columnFamilyHandle, toAddBytes, DUMMY_BYTES); + } catch (RocksDBException e) { + throw new FlinkRuntimeException(e); + } + } + + private void removeFromRocksDB(@Nonnull byte[] toRemoveBytes) { + try { + batchWrapper.remove(columnFamilyHandle, toRemoveBytes); + } catch (RocksDBException e) { + throw new FlinkRuntimeException(e); + } + } + + private void checkRefillCacheFromStore() { + if (!allElementsInCache && orderedCache.isEmpty()) { + try (final RocksBytesIterator iterator = orderedBytesIterator()) { + orderedCache.bulkLoadFromOrderedIterator(iterator); + allElementsInCache = !iterator.hasNext(); + } catch (Exception e) { + throw new FlinkRuntimeException( + "Exception while refilling store from iterator.", e); + } + } + } + + private static boolean isPrefixWith(byte[] bytes, byte[] prefixBytes) { + for (int i = 0; i < prefixBytes.length; ++i) { + if (bytes[i] != prefixBytes[i]) { + return false; + } + } + return true; + } + + @Nonnull + private byte[] createKeyGroupBytes(int keyGroupId, int numPrefixBytes) { + + outputView.clear(); + + try { + CompositeKeySerializationUtils.writeKeyGroup(keyGroupId, numPrefixBytes, outputView); + } catch (IOException e) { + throw new FlinkRuntimeException("Could not write key-group bytes.", e); + } + + return outputView.getCopyOfBuffer(); + } + + @Nonnull + private byte[] serializeElement(@Nonnull E element) { + try { + outputView.clear(); + outputView.write(groupPrefixBytes); + byteOrderProducingSerializer.serialize(element, outputView); + return outputView.getCopyOfBuffer(); + } catch (IOException e) { + throw new FlinkRuntimeException("Error while serializing the element.", e); + } + } + + @Nonnull + private E deserializeElement(@Nonnull byte[] bytes) { + try { + final int numPrefixBytes = groupPrefixBytes.length; + inputView.setBuffer(bytes, numPrefixBytes, bytes.length - numPrefixBytes); + return byteOrderProducingSerializer.deserialize(inputView); + } catch (IOException e) { + throw new FlinkRuntimeException("Error while deserializing the element.", e); + } + } + + /** + * Wraps an iterator over byte-arrays with deserialization logic, so that it iterates over + * deserialized objects. + */ + private class DeserializingIteratorWrapper implements CloseableIterator { + + /** The iterator over byte-arrays with the serialized objects. */ + @Nonnull private final CloseableIterator bytesIterator; + + private DeserializingIteratorWrapper(@Nonnull CloseableIterator bytesIterator) { + this.bytesIterator = bytesIterator; + } + + @Override + public void close() throws Exception { + bytesIterator.close(); + } + + @Override + public boolean hasNext() { + return bytesIterator.hasNext(); + } + + @Override + public E next() { + return deserializeElement(bytesIterator.next()); + } + } + + /** + * Adapter between RocksDB iterator and Java iterator. This is also closeable to release the + * native resources after use. + */ + private class RocksBytesIterator implements CloseableIterator { + + /** The RocksDb iterator to which we forward ops. */ + @Nonnull private final ForStIteratorWrapper iterator; + + /** Cache for the current element of the iteration. */ + @Nullable private byte[] currentElement; + + private RocksBytesIterator(@Nonnull ForStIteratorWrapper iterator) { + this.iterator = iterator; + try { + // We use our knowledge about the lower bound to issue a seek that is as close to + // the first element in + // the key-group as possible, i.e. we generate the next possible key after seekHint + // by appending one + // zero-byte. + iterator.seek(Arrays.copyOf(seekHint, seekHint.length + 1)); + currentElement = nextElementIfAvailable(); + } catch (Exception ex) { + // ensure resource cleanup also in the face of (runtime) exceptions in the + // constructor. + iterator.close(); + throw new FlinkRuntimeException("Could not initialize ordered iterator.", ex); + } + } + + @Override + public void close() { + iterator.close(); + } + + @Override + public boolean hasNext() { + return currentElement != null; + } + + @Override + public byte[] next() { + final byte[] returnElement = this.currentElement; + if (returnElement == null) { + throw new NoSuchElementException("Iterator has no more elements!"); + } + iterator.next(); + currentElement = nextElementIfAvailable(); + return returnElement; + } + + private byte[] nextElementIfAvailable() { + final byte[] elementBytes; + return iterator.isValid() + && isPrefixWith((elementBytes = iterator.key()), groupPrefixBytes) + ? elementBytes + : null; + } + } + + /** + * Cache that is organized as an ordered set for byte-arrays. The byte-arrays are sorted in + * lexicographic order of their content. Caches typically have a bounded size. + */ + public interface OrderedByteArraySetCache { + + /** Comparator for byte arrays. */ + Comparator LEXICOGRAPHIC_BYTE_COMPARATOR = + UnsignedBytes.lexicographicalComparator(); + + /** Returns the number of contained elements. */ + int size(); + + /** Returns the maximum number of elements that can be stored in the cache. */ + int maxSize(); + + /** Returns size() == 0. */ + boolean isEmpty(); + + /** Returns size() == maxSize(). */ + boolean isFull(); + + /** + * Adds the given element, if it was not already contained. Returns true iff + * the cache was modified. + */ + boolean add(@Nonnull byte[] toAdd); + + /** + * Removes the given element, if it is contained. Returns true iff the cache + * was modified. + */ + boolean remove(@Nonnull byte[] toRemove); + + /** Returns the first element or null if empty. */ + @Nullable + byte[] peekFirst(); + + /** Returns the last element or null if empty. */ + @Nullable + byte[] peekLast(); + + /** Returns and removes the first element or returns null if empty. */ + @Nullable + byte[] pollFirst(); + + /** Returns and removes the last element or returns null if empty. */ + @Nullable + byte[] pollLast(); + + /** + * Clears the cache and adds up to maxSize() elements from the iterator to the + * cache. Iterator must be ordered in the same order as this cache. + * + * @param orderedIterator iterator with elements in-order. + */ + void bulkLoadFromOrderedIterator(@Nonnull Iterator orderedIterator); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStDBPriorityQueueSetFactory.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStDBPriorityQueueSetFactory.java new file mode 100644 index 0000000000000..de45b6415b087 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStDBPriorityQueueSetFactory.java @@ -0,0 +1,243 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.core.fs.ICloseableRegistry; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.runtime.state.KeyExtractorFunction; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.Keyed; +import org.apache.flink.runtime.state.PriorityComparable; +import org.apache.flink.runtime.state.PriorityComparator; +import org.apache.flink.runtime.state.PriorityQueueSetFactory; +import org.apache.flink.runtime.state.RegisteredPriorityQueueStateBackendMetaInfo; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.state.heap.KeyGroupPartitionedPriorityQueue; +import org.apache.flink.state.forst.ForStDBWriteBatchWrapper; +import org.apache.flink.state.forst.ForStNativeMetricMonitor; +import org.apache.flink.state.forst.ForStOperationUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StateMigrationException; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDB; + +import javax.annotation.Nonnull; + +import java.util.Map; +import java.util.function.Function; + +/** + * Encapsulates the logic and resources in connection with creating priority queue state structures, + * for RocksDB backend. + */ +public class ForStDBPriorityQueueSetFactory implements PriorityQueueSetFactory { + + /** The priorityQueue cache size per key-group. */ + private final int cacheSize; + + /** A shared buffer to serialize elements for the priority queue. */ + @Nonnull private final DataOutputSerializer sharedElementOutView; + + /** A shared buffer to de-serialize elements for the priority queue. */ + @Nonnull private final DataInputDeserializer sharedElementInView; + + private final KeyGroupRange keyGroupRange; + private final int keyGroupPrefixBytes; + private final int numberOfKeyGroups; + private final Map kvStateInformation; + private final RocksDB db; + private final ReadOptions readOptions; + private final ForStDBWriteBatchWrapper writeBatchWrapper; + private final ForStNativeMetricMonitor nativeMetricMonitor; + private final Function columnFamilyOptionsFactory; + private final Long writeBufferManagerCapacity; + + ForStDBPriorityQueueSetFactory( + KeyGroupRange keyGroupRange, + int keyGroupPrefixBytes, + int numberOfKeyGroups, + Map kvStateInformation, + RocksDB db, + ReadOptions readOptions, + ForStDBWriteBatchWrapper writeBatchWrapper, + ForStNativeMetricMonitor nativeMetricMonitor, + Function columnFamilyOptionsFactory, + Long writeBufferManagerCapacity, + int cacheSize) { + this.keyGroupRange = keyGroupRange; + this.keyGroupPrefixBytes = keyGroupPrefixBytes; + this.numberOfKeyGroups = numberOfKeyGroups; + this.kvStateInformation = kvStateInformation; + this.db = db; + this.readOptions = readOptions; + this.writeBatchWrapper = writeBatchWrapper; + this.nativeMetricMonitor = nativeMetricMonitor; + this.columnFamilyOptionsFactory = columnFamilyOptionsFactory; + this.sharedElementOutView = new DataOutputSerializer(128); + this.sharedElementInView = new DataInputDeserializer(); + this.writeBufferManagerCapacity = writeBufferManagerCapacity; + Preconditions.checkArgument(cacheSize > 0); + this.cacheSize = cacheSize; + } + + @Nonnull + @Override + public & Keyed> + KeyGroupedInternalPriorityQueue create( + @Nonnull String stateName, + @Nonnull TypeSerializer byteOrderedElementSerializer) { + return create(stateName, byteOrderedElementSerializer, false); + } + + @Nonnull + @Override + public & Keyed> + KeyGroupedInternalPriorityQueue create( + @Nonnull String stateName, + @Nonnull TypeSerializer byteOrderedElementSerializer, + boolean allowFutureMetadataUpdates) { + + final ForStSyncKeyedStateBackend.ForStDbKvStateInfo stateCFHandle = + tryRegisterPriorityQueueMetaInfo( + stateName, byteOrderedElementSerializer, allowFutureMetadataUpdates); + + final ColumnFamilyHandle columnFamilyHandle = stateCFHandle.columnFamilyHandle; + + return new KeyGroupPartitionedPriorityQueue<>( + KeyExtractorFunction.forKeyedObjects(), + PriorityComparator.forPriorityComparableObjects(), + new KeyGroupPartitionedPriorityQueue.PartitionQueueSetFactory< + T, ForStDBCachingPriorityQueueSet>() { + @Nonnull + @Override + public ForStDBCachingPriorityQueueSet create( + int keyGroupId, + int numKeyGroups, + @Nonnull KeyExtractorFunction keyExtractor, + @Nonnull PriorityComparator elementPriorityComparator) { + TreeOrderedSetCache orderedSetCache = new TreeOrderedSetCache(cacheSize); + return new ForStDBCachingPriorityQueueSet<>( + keyGroupId, + keyGroupPrefixBytes, + db, + readOptions, + columnFamilyHandle, + byteOrderedElementSerializer, + sharedElementOutView, + sharedElementInView, + writeBatchWrapper, + orderedSetCache); + } + }, + keyGroupRange, + numberOfKeyGroups); + } + + @Nonnull + private ForStSyncKeyedStateBackend.ForStDbKvStateInfo tryRegisterPriorityQueueMetaInfo( + @Nonnull String stateName, + @Nonnull TypeSerializer byteOrderedElementSerializer, + boolean allowFutureMetadataUpdates) { + + ForStSyncKeyedStateBackend.ForStDbKvStateInfo stateInfo = kvStateInformation.get(stateName); + + if (stateInfo == null) { + // Currently this class is for timer service and TTL feature is not applicable here, + // so no need to register compact filter when creating column family + RegisteredPriorityQueueStateBackendMetaInfo metaInfo = + new RegisteredPriorityQueueStateBackendMetaInfo<>( + stateName, byteOrderedElementSerializer); + + metaInfo = + allowFutureMetadataUpdates + ? metaInfo.withSerializerUpgradesAllowed() + : metaInfo; + + stateInfo = + ForStOperationUtils.createStateInfo( + metaInfo, + db, + columnFamilyOptionsFactory, + null, + writeBufferManagerCapacity, + // Using ICloseableRegistry.NO_OP here because there is no restore in + // progress; created column families will be closed in dispose() + ICloseableRegistry.NO_OP); + ForStOperationUtils.registerKvStateInformation( + kvStateInformation, nativeMetricMonitor, stateName, stateInfo); + } else { + // 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. + + @SuppressWarnings("unchecked") + RegisteredPriorityQueueStateBackendMetaInfo castedMetaInfo = + (RegisteredPriorityQueueStateBackendMetaInfo) stateInfo.metaInfo; + + TypeSerializer previousElementSerializer = + castedMetaInfo.getPreviousElementSerializer(); + + if (previousElementSerializer != byteOrderedElementSerializer) { + TypeSerializerSchemaCompatibility compatibilityResult = + castedMetaInfo.updateElementSerializer(byteOrderedElementSerializer); + + // Since priority queue elements are written into RocksDB + // as keys prefixed with the key group and namespace, we do not support + // migrating them. Therefore, here we only check for incompatibility. + if (compatibilityResult.isIncompatible()) { + throw new FlinkRuntimeException( + new StateMigrationException( + "The new priority queue serializer must not be incompatible.")); + } + + RegisteredPriorityQueueStateBackendMetaInfo metaInfo = + new RegisteredPriorityQueueStateBackendMetaInfo<>( + stateName, byteOrderedElementSerializer); + + metaInfo = + allowFutureMetadataUpdates + ? metaInfo.withSerializerUpgradesAllowed() + : metaInfo; + + // update meta info with new serializer + stateInfo = + new ForStSyncKeyedStateBackend.ForStDbKvStateInfo( + stateInfo.columnFamilyHandle, metaInfo); + kvStateInformation.put(stateName, stateInfo); + } + } + + return stateInfo; + } + + @VisibleForTesting + public int getCacheSize() { + return cacheSize; + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStIteratorWrapper.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStIteratorWrapper.java new file mode 100644 index 0000000000000..de2c11e8d8fa3 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStIteratorWrapper.java @@ -0,0 +1,132 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.util.FlinkRuntimeException; + +import org.rocksdb.RocksDBException; +import org.rocksdb.RocksIterator; +import org.rocksdb.RocksIteratorInterface; + +import javax.annotation.Nonnull; + +import java.io.Closeable; +import java.nio.ByteBuffer; + +/** + * This class was originally a wrapper around {@link RocksIterator} to check the iterator status for + * all the methods mentioned to require this check in the wiki documentation: seek, next, + * seekToFirst, seekToLast, seekForPrev, and prev. At that time, this was required because the + * iterator may pass the blocks or files it had difficulties in reading (because of IO errors, data + * corruptions or other issues) and continue with the next available keys. The status flag may not + * be OK, even if the iterator is valid. + * + *

However, after 3810 was merged, + * the behaviour had changed. If the iterator is valid, the status() is guaranteed to be OK; If the + * iterator is not valid, there are two possibilities: 1) We have reached the end of the data. And + * in this case, status() is OK; 2) There is an error. In this case, status() is not OK; More + * information can be found here. + */ +public class ForStIteratorWrapper implements RocksIteratorInterface, Closeable { + + private RocksIterator iterator; + + public ForStIteratorWrapper(@Nonnull RocksIterator iterator) { + this.iterator = iterator; + } + + @Override + public boolean isValid() { + boolean isValid = this.iterator.isValid(); + if (!isValid) { + status(); + } + + return isValid; + } + + @Override + public void seekToFirst() { + iterator.seekToFirst(); + } + + @Override + public void seekToLast() { + iterator.seekToLast(); + } + + @Override + public void seek(byte[] target) { + iterator.seek(target); + } + + @Override + public void seekForPrev(byte[] target) { + iterator.seekForPrev(target); + } + + @Override + public void seek(ByteBuffer target) { + iterator.seek(target); + } + + @Override + public void seekForPrev(ByteBuffer target) { + iterator.seekForPrev(target); + } + + @Override + public void next() { + iterator.next(); + } + + @Override + public void prev() { + iterator.prev(); + } + + @Override + public void status() { + try { + iterator.status(); + } catch (RocksDBException ex) { + throw new FlinkRuntimeException("Internal exception found in RocksDB", ex); + } + } + + @Override + public void refresh() throws RocksDBException { + iterator.refresh(); + status(); + } + + public byte[] key() { + return iterator.key(); + } + + public byte[] value() { + return iterator.value(); + } + + @Override + public void close() { + iterator.close(); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStPriorityQueueConfig.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStPriorityQueueConfig.java new file mode 100644 index 0000000000000..a60907ed7afb3 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStPriorityQueueConfig.java @@ -0,0 +1,98 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.state.forst.ForStStateBackend; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +import static org.apache.flink.state.forst.ForStOptions.FORST_TIMER_SERVICE_FACTORY_CACHE_SIZE; +import static org.apache.flink.state.forst.ForStOptions.TIMER_SERVICE_FACTORY; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The configuration of rocksDB priority queue state implementation. */ +public class ForStPriorityQueueConfig implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final int UNDEFINED_ROCKSDB_PRIORITY_QUEUE_SET_CACHE_SIZE = -1; + + /** This determines the type of priority queue state. */ + private @Nullable ForStStateBackend.PriorityQueueStateType priorityQueueStateType; + + /** cache size per keyGroup for rocksDB priority queue state. */ + private int rocksDBPriorityQueueSetCacheSize; + + public ForStPriorityQueueConfig() { + this(null, UNDEFINED_ROCKSDB_PRIORITY_QUEUE_SET_CACHE_SIZE); + } + + public ForStPriorityQueueConfig( + ForStStateBackend.PriorityQueueStateType priorityQueueStateType, + int rocksDBPriorityQueueSetCacheSize) { + this.priorityQueueStateType = priorityQueueStateType; + this.rocksDBPriorityQueueSetCacheSize = rocksDBPriorityQueueSetCacheSize; + } + + /** + * Gets the type of the priority queue state. It will fall back to the default value if it is + * not explicitly set. + */ + public ForStStateBackend.PriorityQueueStateType getPriorityQueueStateType() { + return priorityQueueStateType == null + ? TIMER_SERVICE_FACTORY.defaultValue() + : priorityQueueStateType; + } + + public void setPriorityQueueStateType(ForStStateBackend.PriorityQueueStateType type) { + this.priorityQueueStateType = checkNotNull(type); + } + + /** + * Gets the cache size of rocksDB priority queue set. It will fall back to the default value if + * it is not explicitly set. + */ + public int getRocksDBPriorityQueueSetCacheSize() { + return rocksDBPriorityQueueSetCacheSize == UNDEFINED_ROCKSDB_PRIORITY_QUEUE_SET_CACHE_SIZE + ? FORST_TIMER_SERVICE_FACTORY_CACHE_SIZE.defaultValue() + : rocksDBPriorityQueueSetCacheSize; + } + + public static ForStPriorityQueueConfig fromOtherAndConfiguration( + ForStPriorityQueueConfig other, ReadableConfig config) { + ForStStateBackend.PriorityQueueStateType priorityQueueType = + (null == other.priorityQueueStateType) + ? config.get(TIMER_SERVICE_FACTORY) + : other.priorityQueueStateType; + int cacheSize = + (other.rocksDBPriorityQueueSetCacheSize + == UNDEFINED_ROCKSDB_PRIORITY_QUEUE_SET_CACHE_SIZE) + ? config.get(FORST_TIMER_SERVICE_FACTORY_CACHE_SIZE) + : other.rocksDBPriorityQueueSetCacheSize; + return new ForStPriorityQueueConfig(priorityQueueType, cacheSize); + } + + public static ForStPriorityQueueConfig buildWithPriorityQueueType( + ForStStateBackend.PriorityQueueStateType type) { + return new ForStPriorityQueueConfig( + type, FORST_TIMER_SERVICE_FACTORY_CACHE_SIZE.defaultValue()); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStStateKeysAndNamespaceIterator.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStStateKeysAndNamespaceIterator.java new file mode 100644 index 0000000000000..bc4b53e6aa907 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStStateKeysAndNamespaceIterator.java @@ -0,0 +1,94 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.CompositeKeySerializationUtils; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nonnull; + +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Objects; + +/** + * Adapter class to bridge between {@link ForStIteratorWrapper} and {@link Iterator} to iterate over + * the keys and namespaces. This class is not thread safe. + * + * @param the type of the iterated keys in RocksDB. + * @param the type of the iterated namespaces in RocksDB. + */ +public class ForStStateKeysAndNamespaceIterator extends AbstractForStStateKeysIterator + implements Iterator> { + + @Nonnull private final TypeSerializer namespaceSerializer; + + private Tuple2 nextKeyAndNamespace; + private Tuple2 previousKeyAndNamespace; + + public ForStStateKeysAndNamespaceIterator( + @Nonnull ForStIteratorWrapper iterator, + @Nonnull String state, + @Nonnull TypeSerializer keySerializer, + @Nonnull TypeSerializer namespaceSerializer, + int keyGroupPrefixBytes, + boolean ambiguousKeyPossible) { + super(iterator, state, keySerializer, keyGroupPrefixBytes, ambiguousKeyPossible); + + this.namespaceSerializer = namespaceSerializer; + this.nextKeyAndNamespace = null; + this.previousKeyAndNamespace = null; + } + + @Override + public boolean hasNext() { + try { + while (nextKeyAndNamespace == null && iterator.isValid()) { + + final byte[] keyBytes = iterator.key(); + final K currentKey = deserializeKey(keyBytes, byteArrayDataInputView); + final N currentNamespace = + CompositeKeySerializationUtils.readNamespace( + namespaceSerializer, byteArrayDataInputView, ambiguousKeyPossible); + final Tuple2 currentKeyAndNamespace = Tuple2.of(currentKey, currentNamespace); + if (!Objects.equals(previousKeyAndNamespace, currentKeyAndNamespace)) { + previousKeyAndNamespace = currentKeyAndNamespace; + nextKeyAndNamespace = currentKeyAndNamespace; + } + iterator.next(); + } + } catch (Exception e) { + throw new FlinkRuntimeException("Failed to access state [" + state + "]", e); + } + return nextKeyAndNamespace != null; + } + + @Override + public Tuple2 next() { + if (!hasNext()) { + throw new NoSuchElementException("Failed to access state [" + state + "]"); + } + + Tuple2 tmpKeyAndNamespace = nextKeyAndNamespace; + nextKeyAndNamespace = null; + return tmpKeyAndNamespace; + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStStateKeysIterator.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStStateKeysIterator.java new file mode 100644 index 0000000000000..7f4d57854f60c --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStStateKeysIterator.java @@ -0,0 +1,103 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nonnull; + +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Objects; + +/** + * Adapter class to bridge between {@link ForStIteratorWrapper} and {@link Iterator} to iterate over + * the keys. This class is not thread safe. + * + * @param the type of the iterated objects, which are keys in RocksDB. + */ +public class ForStStateKeysIterator extends AbstractForStStateKeysIterator + implements Iterator { + + @Nonnull private final byte[] namespaceBytes; + + private K nextKey; + private K previousKey; + + public ForStStateKeysIterator( + @Nonnull ForStIteratorWrapper iterator, + @Nonnull String state, + @Nonnull TypeSerializer keySerializer, + int keyGroupPrefixBytes, + boolean ambiguousKeyPossible, + @Nonnull byte[] namespaceBytes) { + super(iterator, state, keySerializer, keyGroupPrefixBytes, ambiguousKeyPossible); + this.namespaceBytes = namespaceBytes; + this.nextKey = null; + this.previousKey = null; + } + + @Override + public boolean hasNext() { + try { + while (nextKey == null && iterator.isValid()) { + + final byte[] keyBytes = iterator.key(); + final K currentKey = deserializeKey(keyBytes, byteArrayDataInputView); + final int namespaceByteStartPos = byteArrayDataInputView.getPosition(); + + if (isMatchingNameSpace(keyBytes, namespaceByteStartPos) + && !Objects.equals(previousKey, currentKey)) { + previousKey = currentKey; + nextKey = currentKey; + } + iterator.next(); + } + } catch (Exception e) { + throw new FlinkRuntimeException("Failed to access state [" + state + "]", e); + } + return nextKey != null; + } + + @Override + public K next() { + if (!hasNext()) { + throw new NoSuchElementException("Failed to access state [" + state + "]"); + } + + K tmpKey = nextKey; + nextKey = null; + return tmpKey; + } + + private boolean isMatchingNameSpace(@Nonnull byte[] key, int beginPos) { + final int namespaceBytesLength = namespaceBytes.length; + final int basicLength = namespaceBytesLength + beginPos; + if (key.length >= basicLength) { + for (int i = 0; i < namespaceBytesLength; ++i) { + if (key[beginPos + i] != namespaceBytes[i]) { + return false; + } + } + return true; + } + return false; + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncAggregatingState.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncAggregatingState.java new file mode 100644 index 0000000000000..64967bf4491fe --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncAggregatingState.java @@ -0,0 +1,197 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.functions.AggregateFunction; +import org.apache.flink.api.common.state.AggregatingState; +import org.apache.flink.api.common.state.AggregatingStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.internal.InternalAggregatingState; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; + +import java.io.IOException; +import java.util.Collection; + +/** + * An {@link AggregatingState} implementation that stores state in RocksDB. + * + * @param The type of the key + * @param The type of the namespace + * @param The type of the values that aggregated into the state + * @param The type of the value stored in the state (the accumulator type) + * @param The type of the value returned from the state + */ +class ForStSyncAggregatingState + extends AbstractForStSyncAppendingState + implements InternalAggregatingState { + + /** User-specified aggregation function. */ + private AggregateFunction aggFunction; + + /** + * Creates a new {@code RocksDBAggregatingState}. + * + * @param columnFamily The RocksDB column family that this state is associated to. + * @param namespaceSerializer The serializer for the namespace. + * @param valueSerializer The serializer for the state. + * @param defaultValue The default value for the state. + * @param aggFunction The aggregate function used for aggregating state. + * @param backend The backend for which this state is bind to. + */ + private ForStSyncAggregatingState( + ColumnFamilyHandle columnFamily, + TypeSerializer namespaceSerializer, + TypeSerializer valueSerializer, + ACC defaultValue, + AggregateFunction aggFunction, + ForStSyncKeyedStateBackend backend) { + + super(columnFamily, namespaceSerializer, valueSerializer, defaultValue, backend); + this.aggFunction = aggFunction; + } + + @Override + public TypeSerializer getKeySerializer() { + return backend.getKeySerializer(); + } + + @Override + public TypeSerializer getNamespaceSerializer() { + return namespaceSerializer; + } + + @Override + public TypeSerializer getValueSerializer() { + return valueSerializer; + } + + @Override + public R get() throws IOException, RocksDBException { + ACC accumulator = getInternal(); + if (accumulator == null) { + return null; + } + return aggFunction.getResult(accumulator); + } + + @Override + public void add(T value) throws IOException, RocksDBException { + byte[] key = getKeyBytes(); + ACC accumulator = getInternal(key); + accumulator = accumulator == null ? aggFunction.createAccumulator() : accumulator; + updateInternal(key, aggFunction.add(value, accumulator)); + } + + @Override + public void mergeNamespaces(N target, Collection sources) + throws IOException, RocksDBException { + if (sources == null || sources.isEmpty()) { + return; + } + + ACC current = null; + + // merge the sources to the target + for (N source : sources) { + if (source != null) { + setCurrentNamespace(source); + final byte[] sourceKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] valueBytes = backend.db.get(columnFamily, sourceKey); + + if (valueBytes != null) { + backend.db.delete(columnFamily, writeOptions, sourceKey); + dataInputView.setBuffer(valueBytes); + ACC value = valueSerializer.deserialize(dataInputView); + + if (current != null) { + current = aggFunction.merge(current, value); + } else { + current = value; + } + } + } + } + + // if something came out of merging the sources, merge it or write it to the target + if (current != null) { + setCurrentNamespace(target); + // create the target full-binary-key + final byte[] targetKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey); + + if (targetValueBytes != null) { + // target also had a value, merge + dataInputView.setBuffer(targetValueBytes); + ACC value = valueSerializer.deserialize(dataInputView); + + current = aggFunction.merge(current, value); + } + + // serialize the resulting value + dataOutputView.clear(); + valueSerializer.serialize(current, dataOutputView); + + // write the resulting value + backend.db.put(columnFamily, writeOptions, targetKey, dataOutputView.getCopyOfBuffer()); + } + } + + ForStSyncAggregatingState setAggFunction( + AggregateFunction aggFunction) { + this.aggFunction = aggFunction; + return this; + } + + @SuppressWarnings("unchecked") + static IS create( + StateDescriptor stateDesc, + Tuple2> + registerResult, + ForStSyncKeyedStateBackend backend) { + return (IS) + new ForStSyncAggregatingState<>( + registerResult.f0, + registerResult.f1.getNamespaceSerializer(), + registerResult.f1.getStateSerializer(), + stateDesc.getDefaultValue(), + ((AggregatingStateDescriptor) stateDesc).getAggregateFunction(), + backend); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static IS update( + StateDescriptor stateDesc, + Tuple2> + registerResult, + IS existingState) { + return (IS) + ((ForStSyncAggregatingState) existingState) + .setAggFunction( + ((AggregatingStateDescriptor) stateDesc).getAggregateFunction()) + .setNamespaceSerializer(registerResult.f1.getNamespaceSerializer()) + .setValueSerializer(registerResult.f1.getStateSerializer()) + .setDefaultValue(stateDesc.getDefaultValue()); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncKeyedStateBackend.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncKeyedStateBackend.java new file mode 100644 index 0000000000000..9c895b12d6ef3 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncKeyedStateBackend.java @@ -0,0 +1,957 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializerSnapshot; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.fs.ICloseableRegistry; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.SnapshotType; +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.CompositeKeySerializationUtils; +import org.apache.flink.runtime.state.HeapPriorityQueuesManager; +import org.apache.flink.runtime.state.InternalKeyContext; +import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.Keyed; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.PriorityComparable; +import org.apache.flink.runtime.state.PriorityQueueSetFactory; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase; +import org.apache.flink.runtime.state.SavepointResources; +import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateSnapshotTransformer.StateSnapshotTransformFactory; +import org.apache.flink.runtime.state.StreamCompressionDecorator; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueSnapshotRestoreWrapper; +import org.apache.flink.runtime.state.metrics.LatencyTrackingStateConfig; +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; +import org.apache.flink.state.forst.ForStDBTtlCompactFiltersManager; +import org.apache.flink.state.forst.ForStDBWriteBatchWrapper; +import org.apache.flink.state.forst.ForStNativeMetricMonitor; +import org.apache.flink.state.forst.ForStOperationUtils; +import org.apache.flink.state.forst.ForStResourceContainer; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; +import org.apache.flink.util.StateMigrationException; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.Snapshot; +import org.rocksdb.WriteOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.RunnableFuture; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * An {@link AbstractKeyedStateBackend} that stores its state in {@code RocksDB} and serializes + * state to streams provided by a {@link org.apache.flink.runtime.state.CheckpointStreamFactory} + * upon checkpointing. This state backend can store very large state that exceeds memory and spills + * to disk. Except for the snapshotting, this class should be accessed as if it is not threadsafe. + * + *

This class follows the rules for closing/releasing native RocksDB resources as described in + + * + * this document. + */ +public class ForStSyncKeyedStateBackend extends AbstractKeyedStateBackend { + + private static final Logger LOG = LoggerFactory.getLogger(ForStSyncKeyedStateBackend.class); + + private static final Map STATE_CREATE_FACTORIES = + Stream.of( + Tuple2.of( + StateDescriptor.Type.VALUE, + (StateCreateFactory) ForStSyncValueState::create), + Tuple2.of( + StateDescriptor.Type.LIST, + (StateCreateFactory) ForStSyncListState::create), + Tuple2.of( + StateDescriptor.Type.MAP, + (StateCreateFactory) ForStSyncMapState::create), + Tuple2.of( + StateDescriptor.Type.AGGREGATING, + (StateCreateFactory) ForStSyncAggregatingState::create), + Tuple2.of( + StateDescriptor.Type.REDUCING, + (StateCreateFactory) ForStSyncReducingState::create)) + .collect(Collectors.toMap(t -> t.f0, t -> t.f1)); + + private static final Map STATE_UPDATE_FACTORIES = + Stream.of( + Tuple2.of( + StateDescriptor.Type.VALUE, + (StateUpdateFactory) ForStSyncValueState::update), + Tuple2.of( + StateDescriptor.Type.LIST, + (StateUpdateFactory) ForStSyncListState::update), + Tuple2.of( + StateDescriptor.Type.MAP, + (StateUpdateFactory) ForStSyncMapState::update), + Tuple2.of( + StateDescriptor.Type.AGGREGATING, + (StateUpdateFactory) ForStSyncAggregatingState::update), + Tuple2.of( + StateDescriptor.Type.REDUCING, + (StateUpdateFactory) ForStSyncReducingState::update)) + .collect(Collectors.toMap(t -> t.f0, t -> t.f1)); + + private interface StateCreateFactory { + IS createState( + StateDescriptor stateDesc, + Tuple2> + registerResult, + ForStSyncKeyedStateBackend backend) + throws Exception; + } + + private interface StateUpdateFactory { + IS updateState( + StateDescriptor stateDesc, + Tuple2> + registerResult, + IS existingState) + throws Exception; + } + + /** Factory function to create column family options from state name. */ + private final Function columnFamilyOptionsFactory; + + /** The container of RocksDB option factory and predefined options. */ + private final ForStResourceContainer optionsContainer; + + /** Path where this configured instance stores its data directory. */ + private final File instanceBasePath; + + /** + * Protects access to RocksDB in other threads, like the checkpointing thread from parallel call + * that disposes the RocksDB object. + */ + private final ResourceGuard rocksDBResourceGuard; + + /** The write options to use in the states. We disable write ahead logging. */ + private final WriteOptions writeOptions; + + /** + * The read options to use when creating iterators. We ensure total order seek in case user + * misuse, see FLINK-17800 for more details. + */ + private final ReadOptions readOptions; + + /** + * The max memory size for one batch in {@link + * org.apache.flink.state.forst.ForStDBWriteBatchWrapper}. + */ + private final long writeBatchSize; + + /** Map of created k/v states. */ + private final Map createdKVStates; + + /** + * Information about the k/v states, maintained in the order as we create them. This is used to + * retrieve the column family that is used for a state and also for sanity checks when + * restoring. + */ + private final LinkedHashMap kvStateInformation; + + private final HeapPriorityQueuesManager heapPriorityQueuesManager; + + /** Number of bytes required to prefix the key groups. */ + private final int keyGroupPrefixBytes; + + /** + * We are not using the default column family for Flink state ops, but we still need to remember + * this handle so that we can close it properly when the backend is closed. Note that the one + * returned by {@link RocksDB#open(String)} is different from that by {@link + * RocksDB#getDefaultColumnFamily()}, probably it's a bug of RocksDB java API. + */ + private final ColumnFamilyHandle defaultColumnFamily; + + /** Shared wrapper for batch writes to the RocksDB instance. */ + private final ForStDBWriteBatchWrapper writeBatchWrapper; + + /** + * The checkpoint snapshot strategy, e.g., if we use full or incremental checkpoints, local + * state, and so on. + */ + // private final RocksDBSnapshotStrategyBase checkpointSnapshotStrategy; + + /** The native metrics monitor. */ + private final ForStNativeMetricMonitor nativeMetricMonitor; + + /** Factory for priority queue state. */ + private final PriorityQueueSetFactory priorityQueueFactory; + + /** + * Helper to build the byte arrays of composite keys to address data in RocksDB. Shared across + * all states. + */ + private final SerializedCompositeKeyBuilder sharedRocksKeyBuilder; + + /** + * Our RocksDB database, this is used by the actual subclasses of {@link AbstractForStSyncState} + * to store state. The different k/v states that we have don't each have their own RocksDB + * instance. They all write to this instance but to their own column family. + */ + protected final RocksDB db; + + // mark whether this backend is already disposed and prevent duplicate disposing + private boolean disposed = false; + + private final ForStDBTtlCompactFiltersManager ttlCompactFiltersManager; + + public ForStSyncKeyedStateBackend( + ClassLoader userCodeClassLoader, + File instanceBasePath, + ForStResourceContainer optionsContainer, + Function columnFamilyOptionsFactory, + TaskKvStateRegistry kvStateRegistry, + TypeSerializer keySerializer, + ExecutionConfig executionConfig, + TtlTimeProvider ttlTimeProvider, + LatencyTrackingStateConfig latencyTrackingStateConfig, + RocksDB db, + LinkedHashMap kvStateInformation, + Map> registeredPQStates, + int keyGroupPrefixBytes, + CloseableRegistry cancelStreamRegistry, + StreamCompressionDecorator keyGroupCompressionDecorator, + ResourceGuard rocksDBResourceGuard, + // RocksDBSnapshotStrategyBase checkpointSnapshotStrategy, + ForStDBWriteBatchWrapper writeBatchWrapper, + ColumnFamilyHandle defaultColumnFamilyHandle, + ForStNativeMetricMonitor nativeMetricMonitor, + SerializedCompositeKeyBuilder sharedRocksKeyBuilder, + PriorityQueueSetFactory priorityQueueFactory, + ForStDBTtlCompactFiltersManager ttlCompactFiltersManager, + InternalKeyContext keyContext, + @Nonnegative long writeBatchSize, + @Nullable CompletableFuture asyncCompactFuture) { + + super( + kvStateRegistry, + keySerializer, + userCodeClassLoader, + executionConfig, + ttlTimeProvider, + latencyTrackingStateConfig, + cancelStreamRegistry, + keyGroupCompressionDecorator, + keyContext); + + this.ttlCompactFiltersManager = ttlCompactFiltersManager; + + // ensure that we use the right merge operator, because other code relies on this + this.columnFamilyOptionsFactory = Preconditions.checkNotNull(columnFamilyOptionsFactory); + + this.optionsContainer = Preconditions.checkNotNull(optionsContainer); + + this.instanceBasePath = Preconditions.checkNotNull(instanceBasePath); + + this.keyGroupPrefixBytes = keyGroupPrefixBytes; + this.kvStateInformation = kvStateInformation; + this.createdKVStates = new HashMap<>(); + + this.writeOptions = optionsContainer.getWriteOptions(); + this.readOptions = optionsContainer.getReadOptions(); + this.writeBatchSize = writeBatchSize; + this.db = db; + this.rocksDBResourceGuard = rocksDBResourceGuard; + // this.checkpointSnapshotStrategy = checkpointSnapshotStrategy; + this.writeBatchWrapper = writeBatchWrapper; + this.defaultColumnFamily = defaultColumnFamilyHandle; + this.nativeMetricMonitor = nativeMetricMonitor; + this.sharedRocksKeyBuilder = sharedRocksKeyBuilder; + this.priorityQueueFactory = priorityQueueFactory; + if (priorityQueueFactory instanceof HeapPriorityQueueSetFactory) { + this.heapPriorityQueuesManager = + new HeapPriorityQueuesManager( + registeredPQStates, + (HeapPriorityQueueSetFactory) priorityQueueFactory, + keyContext.getKeyGroupRange(), + keyContext.getNumberOfKeyGroups()); + } else { + this.heapPriorityQueuesManager = null; + } + } + + @SuppressWarnings("unchecked") + @Override + public Stream getKeys(String state, N namespace) { + ForStDbKvStateInfo columnInfo = kvStateInformation.get(state); + if (columnInfo == null + || !(columnInfo.metaInfo instanceof RegisteredKeyValueStateBackendMetaInfo)) { + return Stream.empty(); + } + + RegisteredKeyValueStateBackendMetaInfo registeredKeyValueStateBackendMetaInfo = + (RegisteredKeyValueStateBackendMetaInfo) columnInfo.metaInfo; + + final TypeSerializer namespaceSerializer = + registeredKeyValueStateBackendMetaInfo.getNamespaceSerializer(); + final DataOutputSerializer namespaceOutputView = new DataOutputSerializer(8); + boolean ambiguousKeyPossible = + CompositeKeySerializationUtils.isAmbiguousKeyPossible( + getKeySerializer(), namespaceSerializer); + final byte[] nameSpaceBytes; + try { + CompositeKeySerializationUtils.writeNameSpace( + namespace, namespaceSerializer, namespaceOutputView, ambiguousKeyPossible); + nameSpaceBytes = namespaceOutputView.getCopyOfBuffer(); + } catch (IOException ex) { + throw new FlinkRuntimeException("Failed to get keys from RocksDB state backend.", ex); + } + + ForStIteratorWrapper iterator = + ForStOperationUtils.getForStIterator( + db, columnInfo.columnFamilyHandle, readOptions); + iterator.seekToFirst(); + + final ForStStateKeysIterator iteratorWrapper = + new ForStStateKeysIterator<>( + iterator, + state, + getKeySerializer(), + keyGroupPrefixBytes, + ambiguousKeyPossible, + nameSpaceBytes); + + Stream targetStream = + StreamSupport.stream( + Spliterators.spliteratorUnknownSize(iteratorWrapper, Spliterator.ORDERED), + false); + return targetStream.onClose(iteratorWrapper::close); + } + + @Override + public Stream> getKeysAndNamespaces(String state) { + ForStDbKvStateInfo columnInfo = kvStateInformation.get(state); + if (columnInfo == null + || !(columnInfo.metaInfo instanceof RegisteredKeyValueStateBackendMetaInfo)) { + return Stream.empty(); + } + + RegisteredKeyValueStateBackendMetaInfo registeredKeyValueStateBackendMetaInfo = + (RegisteredKeyValueStateBackendMetaInfo) columnInfo.metaInfo; + + final TypeSerializer namespaceSerializer = + registeredKeyValueStateBackendMetaInfo.getNamespaceSerializer(); + boolean ambiguousKeyPossible = + CompositeKeySerializationUtils.isAmbiguousKeyPossible( + getKeySerializer(), namespaceSerializer); + + ForStIteratorWrapper iterator = + ForStOperationUtils.getForStIterator( + db, columnInfo.columnFamilyHandle, readOptions); + iterator.seekToFirst(); + + final ForStStateKeysAndNamespaceIterator iteratorWrapper = + new ForStStateKeysAndNamespaceIterator<>( + iterator, + state, + getKeySerializer(), + namespaceSerializer, + keyGroupPrefixBytes, + ambiguousKeyPossible); + + Stream> targetStream = + StreamSupport.stream( + Spliterators.spliteratorUnknownSize(iteratorWrapper, Spliterator.ORDERED), + false); + return targetStream.onClose(iteratorWrapper::close); + } + + @VisibleForTesting + ColumnFamilyHandle getColumnFamilyHandle(String state) { + ForStDbKvStateInfo columnInfo = kvStateInformation.get(state); + return columnInfo != null ? columnInfo.columnFamilyHandle : null; + } + + @Override + public void setCurrentKey(K newKey) { + super.setCurrentKey(newKey); + sharedRocksKeyBuilder.setKeyAndKeyGroup(getCurrentKey(), getCurrentKeyGroupIndex()); + } + + /** Should only be called by one thread, and only after all accesses to the DB happened. */ + @Override + public void dispose() { + if (this.disposed) { + return; + } + super.dispose(); + + // This call will block until all clients that still acquire access to the RocksDB instance + // have released it, + // so that we cannot release the native resources while clients are still working with it in + // parallel. + rocksDBResourceGuard.close(); + + // IMPORTANT: null reference to signal potential async checkpoint workers that the db was + // disposed, as + // working on the disposed object results in SEGFAULTS. + if (db != null) { + IOUtils.closeQuietly(writeBatchWrapper); + + // Metric collection occurs on a background thread. When this method returns + // it is guaranteed that thr RocksDB reference has been invalidated + // and no more metric collection will be attempted against the database. + if (nativeMetricMonitor != null) { + nativeMetricMonitor.close(); + } + + List columnFamilyOptions = + new ArrayList<>(kvStateInformation.values().size()); + + // RocksDB's native memory management requires that *all* CFs (including default) are + // closed before the + // DB is closed. See: + // https://github.com/facebook/rocksdb/wiki/RocksJava-Basics#opening-a-database-with-column-families + // Start with default CF ... + ForStOperationUtils.addColumnFamilyOptionsToCloseLater( + columnFamilyOptions, defaultColumnFamily); + IOUtils.closeQuietly(defaultColumnFamily); + + // ... continue with the ones created by Flink... + for (ForStDbKvStateInfo kvStateInfo : kvStateInformation.values()) { + ForStOperationUtils.addColumnFamilyOptionsToCloseLater( + columnFamilyOptions, kvStateInfo.columnFamilyHandle); + IOUtils.closeQuietly(kvStateInfo.columnFamilyHandle); + } + + // ... and finally close the DB instance ... + IOUtils.closeQuietly(db); + + columnFamilyOptions.forEach(IOUtils::closeQuietly); + + IOUtils.closeQuietly(optionsContainer); + + kvStateInformation.clear(); + + cleanInstanceBasePath(); + } + // todo: rebase after checkpoint pr merged + // IOUtils.closeQuietly(checkpointSnapshotStrategy); + this.disposed = true; + } + + @Nonnull + @Override + public & Keyed> + KeyGroupedInternalPriorityQueue create( + @Nonnull String stateName, + @Nonnull TypeSerializer byteOrderedElementSerializer) { + return create(stateName, byteOrderedElementSerializer, false); + } + + @Override + public & Keyed> + KeyGroupedInternalPriorityQueue create( + @Nonnull String stateName, + @Nonnull TypeSerializer byteOrderedElementSerializer, + boolean allowFutureMetadataUpdates) { + if (this.heapPriorityQueuesManager != null) { + return this.heapPriorityQueuesManager.createOrUpdate( + stateName, byteOrderedElementSerializer, allowFutureMetadataUpdates); + } else { + return priorityQueueFactory.create( + stateName, byteOrderedElementSerializer, allowFutureMetadataUpdates); + } + } + + private void cleanInstanceBasePath() { + LOG.info( + "Closed RocksDB State Backend. Cleaning up RocksDB working directory {}.", + instanceBasePath); + + try { + FileUtils.deleteDirectory(instanceBasePath); + } catch (IOException ex) { + LOG.warn("Could not delete RocksDB working directory: {}", instanceBasePath, ex); + } + } + + // ------------------------------------------------------------------------ + // Getters and Setters + // ------------------------------------------------------------------------ + + public int getKeyGroupPrefixBytes() { + return keyGroupPrefixBytes; + } + + @VisibleForTesting + PriorityQueueSetFactory getPriorityQueueFactory() { + return priorityQueueFactory; + } + + public WriteOptions getWriteOptions() { + return writeOptions; + } + + public ReadOptions getReadOptions() { + return readOptions; + } + + SerializedCompositeKeyBuilder getSharedRocksKeyBuilder() { + return sharedRocksKeyBuilder; + } + + @VisibleForTesting + boolean isDisposed() { + return this.disposed; + } + + /** + * Triggers an asynchronous snapshot of the keyed state backend from RocksDB. This snapshot can + * be canceled and is also stopped when the backend is closed through {@link #dispose()}. For + * each backend, this method must always be called by the same thread. + * + * @param checkpointId The Id of the checkpoint. + * @param timestamp The timestamp of the checkpoint. + * @param streamFactory The factory that we can use for writing our state to streams. + * @param checkpointOptions Options for how to perform this checkpoint. + * @return Future to the state handle of the snapshot data. + * @throws Exception indicating a problem in the synchronous part of the checkpoint. + */ + @Nonnull + @Override + public RunnableFuture> snapshot( + final long checkpointId, + final long timestamp, + @Nonnull final CheckpointStreamFactory streamFactory, + @Nonnull CheckpointOptions checkpointOptions) + throws Exception { + throw new UnsupportedOperationException("This method is not supported."); + } + + @Nonnull + @Override + public SavepointResources savepoint() throws Exception { + throw new UnsupportedOperationException("This method is not supported."); + } + + @Override + public void notifyCheckpointComplete(long completedCheckpointId) throws Exception { + throw new UnsupportedOperationException("This method is not supported."); + } + + @Override + public void notifyCheckpointAborted(long checkpointId) throws Exception { + throw new UnsupportedOperationException("This method is not supported."); + } + + /** + * Registers a k/v state information, which includes its state id, type, RocksDB column family + * handle, and serializers. + * + *

When restoring from a snapshot, we don’t restore the individual k/v states, just the + * global RocksDB database and the list of k/v state information. When a k/v state is first + * requested we check here whether we already have a registered entry for that and return it + * (after some necessary state compatibility checks) or create a new one if it does not exist. + */ + private + Tuple2> + tryRegisterKvStateInformation( + StateDescriptor stateDesc, + TypeSerializer namespaceSerializer, + @Nonnull StateSnapshotTransformFactory snapshotTransformFactory, + boolean allowFutureMetadataUpdates) + throws Exception { + + ForStDbKvStateInfo oldStateInfo = kvStateInformation.get(stateDesc.getName()); + + TypeSerializer stateSerializer = stateDesc.getSerializer(); + + ForStDbKvStateInfo newRocksStateInfo; + RegisteredKeyValueStateBackendMetaInfo newMetaInfo; + if (oldStateInfo != null) { + @SuppressWarnings("unchecked") + RegisteredKeyValueStateBackendMetaInfo castedMetaInfo = + (RegisteredKeyValueStateBackendMetaInfo) oldStateInfo.metaInfo; + + newMetaInfo = + updateRestoredStateMetaInfo( + Tuple2.of(oldStateInfo.columnFamilyHandle, castedMetaInfo), + stateDesc, + namespaceSerializer, + stateSerializer); + + newMetaInfo = + allowFutureMetadataUpdates + ? newMetaInfo.withSerializerUpgradesAllowed() + : newMetaInfo; + + newRocksStateInfo = + new ForStDbKvStateInfo(oldStateInfo.columnFamilyHandle, newMetaInfo); + kvStateInformation.put(stateDesc.getName(), newRocksStateInfo); + } else { + newMetaInfo = + new RegisteredKeyValueStateBackendMetaInfo<>( + stateDesc.getType(), + stateDesc.getName(), + namespaceSerializer, + stateSerializer, + StateSnapshotTransformFactory.noTransform()); + + newMetaInfo = + allowFutureMetadataUpdates + ? newMetaInfo.withSerializerUpgradesAllowed() + : newMetaInfo; + + newRocksStateInfo = + ForStOperationUtils.createStateInfo( + newMetaInfo, + db, + columnFamilyOptionsFactory, + ttlCompactFiltersManager, + optionsContainer.getWriteBufferManagerCapacity(), + // Using ICloseableRegistry.NO_OP here because there is no restore in + // progress; created column families will be closed in dispose() + ICloseableRegistry.NO_OP); + ForStOperationUtils.registerKvStateInformation( + this.kvStateInformation, + this.nativeMetricMonitor, + stateDesc.getName(), + newRocksStateInfo); + } + + // todo: update SnapshotTransformFactory + return Tuple2.of(newRocksStateInfo.columnFamilyHandle, newMetaInfo); + } + + private + RegisteredKeyValueStateBackendMetaInfo updateRestoredStateMetaInfo( + Tuple2> + oldStateInfo, + StateDescriptor stateDesc, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer) + throws Exception { + + RegisteredKeyValueStateBackendMetaInfo restoredKvStateMetaInfo = oldStateInfo.f1; + + // fetch current serializer now because if it is incompatible, we can't access + // it anymore to improve the error message + TypeSerializer previousNamespaceSerializer = + restoredKvStateMetaInfo.getNamespaceSerializer(); + + TypeSerializerSchemaCompatibility s = + restoredKvStateMetaInfo.updateNamespaceSerializer(namespaceSerializer); + if (s.isCompatibleAfterMigration() || s.isIncompatible()) { + throw new StateMigrationException( + "The new namespace serializer (" + + namespaceSerializer + + ") must be compatible with the old namespace serializer (" + + previousNamespaceSerializer + + ")."); + } + + restoredKvStateMetaInfo.checkStateMetaInfo(stateDesc); + + // fetch current serializer now because if it is incompatible, we can't access + // it anymore to improve the error message + TypeSerializer previousStateSerializer = restoredKvStateMetaInfo.getStateSerializer(); + + TypeSerializerSchemaCompatibility newStateSerializerCompatibility = + restoredKvStateMetaInfo.updateStateSerializer(stateSerializer); + if (newStateSerializerCompatibility.isCompatibleAfterMigration()) { + migrateStateValues(stateDesc, oldStateInfo); + } else if (newStateSerializerCompatibility.isIncompatible()) { + throw new StateMigrationException( + "The new state serializer (" + + stateSerializer + + ") must not be incompatible with the old state serializer (" + + previousStateSerializer + + ")."); + } + + return restoredKvStateMetaInfo; + } + + /** + * Migrate only the state value, that is the "value" that is stored in RocksDB. We don't migrate + * the key here, which is made up of key group, key, namespace and map key (in case of + * MapState). + */ + @SuppressWarnings("unchecked") + private void migrateStateValues( + StateDescriptor stateDesc, + Tuple2> stateMetaInfo) + throws Exception { + + if (stateDesc.getType() == StateDescriptor.Type.MAP) { + TypeSerializerSnapshot previousSerializerSnapshot = + stateMetaInfo.f1.getPreviousStateSerializerSnapshot(); + checkState( + previousSerializerSnapshot != null, + "the previous serializer snapshot should exist."); + checkState( + previousSerializerSnapshot instanceof MapSerializerSnapshot, + "previous serializer snapshot should be a MapSerializerSnapshot."); + + TypeSerializer newSerializer = stateMetaInfo.f1.getStateSerializer(); + checkState( + newSerializer instanceof MapSerializer, + "new serializer should be a MapSerializer."); + + MapSerializer mapSerializer = (MapSerializer) newSerializer; + MapSerializerSnapshot mapSerializerSnapshot = + (MapSerializerSnapshot) previousSerializerSnapshot; + if (!checkMapStateKeySchemaCompatibility(mapSerializerSnapshot, mapSerializer)) { + throw new StateMigrationException( + "The new serializer for a MapState requires state migration in order for the job to proceed, since the key schema has changed. However, migration for MapState currently only allows value schema evolutions."); + } + } + + LOG.info( + "Performing state migration for state {} because the state serializer's schema, i.e. serialization format, has changed.", + stateDesc); + + // we need to get an actual state instance because migration is different + // for different state types. For example, ListState needs to deal with + // individual elements + State state = createState(stateDesc, stateMetaInfo); + if (!(state instanceof AbstractForStSyncState)) { + throw new FlinkRuntimeException( + "State should be an AbstractForStSyncState but is " + state); + } + + @SuppressWarnings("unchecked") + AbstractForStSyncState rocksDBState = (AbstractForStSyncState) state; + + Snapshot rocksDBSnapshot = db.getSnapshot(); + try (ForStIteratorWrapper iterator = + ForStOperationUtils.getForStIterator(db, stateMetaInfo.f0, readOptions); + ForStDBWriteBatchWrapper batchWriter = + new ForStDBWriteBatchWrapper(db, getWriteOptions(), getWriteBatchSize()); + Closeable ignored = + cancelStreamRegistry.registerCloseableTemporarily( + writeBatchWrapper.getCancelCloseable())) { + iterator.seekToFirst(); + + DataInputDeserializer serializedValueInput = new DataInputDeserializer(); + DataOutputSerializer migratedSerializedValueOutput = new DataOutputSerializer(512); + while (iterator.isValid()) { + serializedValueInput.setBuffer(iterator.value()); + + rocksDBState.migrateSerializedValue( + serializedValueInput, + migratedSerializedValueOutput, + stateMetaInfo.f1.getPreviousStateSerializer(), + stateMetaInfo.f1.getStateSerializer()); + + batchWriter.put( + stateMetaInfo.f0, + iterator.key(), + migratedSerializedValueOutput.getCopyOfBuffer()); + + migratedSerializedValueOutput.clear(); + iterator.next(); + } + } finally { + db.releaseSnapshot(rocksDBSnapshot); + rocksDBSnapshot.close(); + } + } + + @SuppressWarnings("unchecked") + private static boolean checkMapStateKeySchemaCompatibility( + MapSerializerSnapshot mapStateSerializerSnapshot, + MapSerializer newMapStateSerializer) { + TypeSerializerSnapshot previousKeySerializerSnapshot = + (TypeSerializerSnapshot) mapStateSerializerSnapshot.getKeySerializerSnapshot(); + TypeSerializer newUserKeySerializer = + (TypeSerializer) newMapStateSerializer.getKeySerializer(); + + TypeSerializerSchemaCompatibility keyCompatibility = + newUserKeySerializer + .snapshotConfiguration() + .resolveSchemaCompatibility(previousKeySerializerSnapshot); + return keyCompatibility.isCompatibleAsIs(); + } + + @Override + @Nonnull + public IS createOrUpdateInternalState( + @Nonnull TypeSerializer namespaceSerializer, + @Nonnull StateDescriptor stateDesc, + @Nonnull StateSnapshotTransformFactory snapshotTransformFactory) + throws Exception { + return createOrUpdateInternalState( + namespaceSerializer, stateDesc, snapshotTransformFactory, false); + } + + @Nonnull + @Override + public IS createOrUpdateInternalState( + @Nonnull TypeSerializer namespaceSerializer, + @Nonnull StateDescriptor stateDesc, + @Nonnull StateSnapshotTransformFactory snapshotTransformFactory, + boolean allowFutureMetadataUpdates) + throws Exception { + Tuple2> registerResult = + tryRegisterKvStateInformation( + stateDesc, + namespaceSerializer, + snapshotTransformFactory, + allowFutureMetadataUpdates); + if (!allowFutureMetadataUpdates) { + // Config compact filter only when no future metadata updates + ttlCompactFiltersManager.configCompactFilter( + stateDesc, registerResult.f1.getStateSerializer()); + } + + return createState(stateDesc, registerResult); + } + + private IS createState( + StateDescriptor stateDesc, + Tuple2> + registerResult) + throws Exception { + @SuppressWarnings("unchecked") + IS createdState = (IS) createdKVStates.get(stateDesc.getName()); + if (createdState == null) { + StateCreateFactory stateCreateFactory = STATE_CREATE_FACTORIES.get(stateDesc.getType()); + if (stateCreateFactory == null) { + throw new FlinkRuntimeException(stateNotSupportedMessage(stateDesc)); + } + createdState = + stateCreateFactory.createState( + stateDesc, registerResult, ForStSyncKeyedStateBackend.this); + } else { + StateUpdateFactory stateUpdateFactory = STATE_UPDATE_FACTORIES.get(stateDesc.getType()); + if (stateUpdateFactory == null) { + throw new FlinkRuntimeException(stateNotSupportedMessage(stateDesc)); + } + createdState = stateUpdateFactory.updateState(stateDesc, registerResult, createdState); + } + + createdKVStates.put(stateDesc.getName(), createdState); + return createdState; + } + + private String stateNotSupportedMessage( + StateDescriptor stateDesc) { + return String.format( + "State %s is not supported by %s", stateDesc.getClass(), this.getClass()); + } + + /** Only visible for testing, DO NOT USE. */ + File getInstanceBasePath() { + return instanceBasePath; + } + + @VisibleForTesting + @Override + public int numKeyValueStateEntries() { + int count = 0; + + for (ForStDbKvStateInfo metaInfo : kvStateInformation.values()) { + // TODO maybe filterOrTransform only for k/v states + try (ForStIteratorWrapper rocksIterator = + ForStOperationUtils.getForStIterator( + db, metaInfo.columnFamilyHandle, readOptions)) { + rocksIterator.seekToFirst(); + + while (rocksIterator.isValid()) { + count++; + rocksIterator.next(); + } + } + } + + return count; + } + + @Override + public boolean requiresLegacySynchronousTimerSnapshots(SnapshotType checkpointType) { + return priorityQueueFactory instanceof HeapPriorityQueueSetFactory + && !checkpointType.isSavepoint(); + } + + @Override + public boolean isSafeToReuseKVState() { + return true; + } + + /** Rocks DB specific information about the k/v states. */ + public static class ForStDbKvStateInfo implements AutoCloseable { + public final ColumnFamilyHandle columnFamilyHandle; + public final RegisteredStateMetaInfoBase metaInfo; + + public ForStDbKvStateInfo( + ColumnFamilyHandle columnFamilyHandle, RegisteredStateMetaInfoBase metaInfo) { + this.columnFamilyHandle = columnFamilyHandle; + this.metaInfo = metaInfo; + } + + @Override + public void close() throws Exception { + this.columnFamilyHandle.close(); + } + } + + @Nonnegative + long getWriteBatchSize() { + return writeBatchSize; + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncKeyedStateBackendBuilder.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncKeyedStateBackendBuilder.java new file mode 100644 index 0000000000000..b634ec26137cc --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncKeyedStateBackendBuilder.java @@ -0,0 +1,468 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.state.AbstractKeyedStateBackendBuilder; +import org.apache.flink.runtime.state.BackendBuildingException; +import org.apache.flink.runtime.state.CompositeKeySerializationUtils; +import org.apache.flink.runtime.state.IncrementalKeyedStateHandle.HandleAndLocalPath; +import org.apache.flink.runtime.state.InternalKeyContext; +import org.apache.flink.runtime.state.InternalKeyContextImpl; +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.PriorityQueueSetFactory; +import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.StreamCompressionDecorator; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueSetFactory; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueSnapshotRestoreWrapper; +import org.apache.flink.runtime.state.metrics.LatencyTrackingStateConfig; +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; +import org.apache.flink.runtime.taskmanager.AsyncExceptionHandler; +import org.apache.flink.state.forst.ForStConfigurableOptions; +import org.apache.flink.state.forst.ForStDBTtlCompactFiltersManager; +import org.apache.flink.state.forst.ForStDBWriteBatchWrapper; +import org.apache.flink.state.forst.ForStNativeMetricMonitor; +import org.apache.flink.state.forst.ForStNativeMetricOptions; +import org.apache.flink.state.forst.ForStOperationUtils; +import org.apache.flink.state.forst.ForStResourceContainer; +import org.apache.flink.state.forst.restore.ForStNoneRestoreOperation; +import org.apache.flink.state.forst.restore.ForStRestoreOperation; +import org.apache.flink.state.forst.restore.ForStRestoreResult; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.RocksDB; + +import javax.annotation.Nonnull; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Builder class for {@link org.apache.flink.state.forst.sync.ForStSyncKeyedStateBackend} which + * handles all necessary initializations and clean ups. + * + * @param The data type that the key serializer serializes. + */ +public class ForStSyncKeyedStateBackendBuilder extends AbstractKeyedStateBackendBuilder { + + static final String DB_INSTANCE_DIR_STRING = "db"; + + /** String that identifies the operator that owns this backend. */ + private final String operatorIdentifier; + + /** The configuration of rocksDB priorityQueue state. */ + private final ForStPriorityQueueConfig priorityQueueConfig; + + /** The configuration of local recovery. */ + private final LocalRecoveryConfig localRecoveryConfig; + + /** Factory function to create column family options from state name. */ + private final Function columnFamilyOptionsFactory; + + /** The container of RocksDB option factory and predefined options. */ + private final ForStResourceContainer optionsContainer; + + /** Path where this configured instance stores its data directory. */ + private final File instanceBasePath; + + /** Path where this configured instance stores its RocksDB database. */ + private final File instanceForStDBPath; + + private final MetricGroup metricGroup; + private final StateBackend.CustomInitializationMetrics customInitializationMetrics; + + /** True if incremental checkpointing is enabled. */ + private boolean enableIncrementalCheckpointing; + + /** RocksDB property-based and statistics-based native metrics options. */ + private ForStNativeMetricOptions nativeMetricOptions; + + private int numberOfTransferingThreads; + private long writeBatchSize = + ForStConfigurableOptions.WRITE_BATCH_SIZE.defaultValue().getBytes(); + + private RocksDB injectedTestDB; // for testing + // todo: checkpoint/restore related + private boolean incrementalRestoreAsyncCompactAfterRescale = false; + private boolean rescalingUseDeleteFilesInRange = false; + + private double overlapFractionThreshold = 0.5; + private boolean useIngestDbRestoreMode = false; + private ColumnFamilyHandle injectedDefaultColumnFamilyHandle; // for testing + private AsyncExceptionHandler asyncExceptionHandler; + + public ForStSyncKeyedStateBackendBuilder( + String operatorIdentifier, + ClassLoader userCodeClassLoader, + File instanceBasePath, + ForStResourceContainer optionsContainer, + Function columnFamilyOptionsFactory, + TaskKvStateRegistry kvStateRegistry, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + ExecutionConfig executionConfig, + LocalRecoveryConfig localRecoveryConfig, + ForStPriorityQueueConfig priorityQueueConfig, + TtlTimeProvider ttlTimeProvider, + LatencyTrackingStateConfig latencyTrackingStateConfig, + MetricGroup metricGroup, + StateBackend.CustomInitializationMetrics customInitializationMetrics, + @Nonnull Collection stateHandles, + StreamCompressionDecorator keyGroupCompressionDecorator, + CloseableRegistry cancelStreamRegistry) { + + super( + kvStateRegistry, + keySerializer, + userCodeClassLoader, + numberOfKeyGroups, + keyGroupRange, + executionConfig, + ttlTimeProvider, + latencyTrackingStateConfig, + stateHandles, + keyGroupCompressionDecorator, + cancelStreamRegistry); + + this.operatorIdentifier = operatorIdentifier; + this.priorityQueueConfig = priorityQueueConfig; + this.localRecoveryConfig = localRecoveryConfig; + // ensure that we use the right merge operator, because other code relies on this + this.columnFamilyOptionsFactory = Preconditions.checkNotNull(columnFamilyOptionsFactory); + this.optionsContainer = optionsContainer; + this.instanceBasePath = instanceBasePath; + this.instanceForStDBPath = getInstanceRocksDBPath(instanceBasePath); + this.metricGroup = metricGroup; + this.customInitializationMetrics = customInitializationMetrics; + this.enableIncrementalCheckpointing = false; + this.nativeMetricOptions = new ForStNativeMetricOptions(); + this.numberOfTransferingThreads = 4; + } + + @VisibleForTesting + ForStSyncKeyedStateBackendBuilder( + String operatorIdentifier, + ClassLoader userCodeClassLoader, + File instanceBasePath, + ForStResourceContainer optionsContainer, + Function columnFamilyOptionsFactory, + TaskKvStateRegistry kvStateRegistry, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + ExecutionConfig executionConfig, + LocalRecoveryConfig localRecoveryConfig, + ForStPriorityQueueConfig rocksDBPriorityQueueConfig, + TtlTimeProvider ttlTimeProvider, + LatencyTrackingStateConfig latencyTrackingStateConfig, + MetricGroup metricGroup, + @Nonnull Collection stateHandles, + StreamCompressionDecorator keyGroupCompressionDecorator, + RocksDB injectedTestDB, + ColumnFamilyHandle injectedDefaultColumnFamilyHandle, + CloseableRegistry cancelStreamRegistry) { + this( + operatorIdentifier, + userCodeClassLoader, + instanceBasePath, + optionsContainer, + columnFamilyOptionsFactory, + kvStateRegistry, + keySerializer, + numberOfKeyGroups, + keyGroupRange, + executionConfig, + localRecoveryConfig, + rocksDBPriorityQueueConfig, + ttlTimeProvider, + latencyTrackingStateConfig, + metricGroup, + (key, value) -> {}, + stateHandles, + keyGroupCompressionDecorator, + cancelStreamRegistry); + this.injectedTestDB = injectedTestDB; + this.injectedDefaultColumnFamilyHandle = injectedDefaultColumnFamilyHandle; + } + + public ForStSyncKeyedStateBackendBuilder setNativeMetricOptions( + ForStNativeMetricOptions nativeMetricOptions) { + this.nativeMetricOptions = nativeMetricOptions; + return this; + } + + public ForStSyncKeyedStateBackendBuilder setWriteBatchSize(long writeBatchSize) { + checkArgument(writeBatchSize >= 0, "Write batch size should be non negative."); + this.writeBatchSize = writeBatchSize; + return this; + } + + public static File getInstanceRocksDBPath(File instanceBasePath) { + return new File(instanceBasePath, DB_INSTANCE_DIR_STRING); + } + + private static void checkAndCreateDirectory(File directory) throws IOException { + if (directory.exists()) { + if (!directory.isDirectory()) { + throw new IOException("Not a directory: " + directory); + } + } else if (!directory.mkdirs()) { + throw new IOException( + String.format("Could not create RocksDB data directory at %s.", directory)); + } + } + + @Override + public ForStSyncKeyedStateBackend build() throws BackendBuildingException { + ForStDBWriteBatchWrapper writeBatchWrapper = null; + ColumnFamilyHandle defaultColumnFamilyHandle = null; + ForStNativeMetricMonitor nativeMetricMonitor = null; + CloseableRegistry cancelRegistryForBackend = new CloseableRegistry(); + LinkedHashMap kvStateInformation = + new LinkedHashMap<>(); + LinkedHashMap> registeredPQStates = + new LinkedHashMap<>(); + RocksDB db = null; + ForStRestoreOperation restoreOperation = null; + CompletableFuture asyncCompactAfterRestoreFuture = null; + + ForStDBTtlCompactFiltersManager ttlCompactFiltersManager = + new ForStDBTtlCompactFiltersManager( + ttlTimeProvider, + optionsContainer.getQueryTimeAfterNumEntries(), + optionsContainer.getPeriodicCompactionTime()); + + ResourceGuard rocksDBResourceGuard = new ResourceGuard(); + PriorityQueueSetFactory priorityQueueFactory; + SerializedCompositeKeyBuilder sharedRocksKeyBuilder; + // Number of bytes required to prefix the key groups. + int keyGroupPrefixBytes = + CompositeKeySerializationUtils.computeRequiredBytesInKeyGroupPrefix( + numberOfKeyGroups); + + try { + // Variables for snapshot strategy when incremental checkpoint is enabled + UUID backendUID = UUID.randomUUID(); + SortedMap> materializedSstFiles = new TreeMap<>(); + long lastCompletedCheckpointId = -1L; + prepareDirectories(); + restoreOperation = + getRocksDBRestoreOperation( + keyGroupPrefixBytes, + rocksDBResourceGuard, + cancelStreamRegistry, + cancelRegistryForBackend, + kvStateInformation, + registeredPQStates, + ttlCompactFiltersManager); + ForStRestoreResult restoreResult = restoreOperation.restore(); + db = restoreResult.getDb(); + defaultColumnFamilyHandle = restoreResult.getDefaultColumnFamilyHandle(); + nativeMetricMonitor = restoreResult.getNativeMetricMonitor(); + + writeBatchWrapper = + new ForStDBWriteBatchWrapper( + db, optionsContainer.getWriteOptions(), writeBatchSize); + + // it is important that we only create the key builder after the restore, and not + // before; + // restore operations may reconfigure the key serializer, so accessing the key + // serializer + // only now we can be certain that the key serializer used in the builder is final. + sharedRocksKeyBuilder = + new SerializedCompositeKeyBuilder<>( + keySerializerProvider.currentSchemaSerializer(), + keyGroupPrefixBytes, + 32); + // todo: init snapshot strategy after db is assured to be initialized + + // init priority queue factory + priorityQueueFactory = + initPriorityQueueFactory( + keyGroupPrefixBytes, + kvStateInformation, + db, + writeBatchWrapper, + nativeMetricMonitor); + } catch (Throwable e) { + // Do clean up + List columnFamilyOptions = + new ArrayList<>(kvStateInformation.values().size()); + IOUtils.closeQuietly(cancelRegistryForBackend); + IOUtils.closeQuietly(writeBatchWrapper); + IOUtils.closeQuietly(rocksDBResourceGuard); + ForStOperationUtils.addColumnFamilyOptionsToCloseLater( + columnFamilyOptions, defaultColumnFamilyHandle); + IOUtils.closeQuietly(defaultColumnFamilyHandle); + IOUtils.closeQuietly(nativeMetricMonitor); + for (ForStSyncKeyedStateBackend.ForStDbKvStateInfo kvStateInfo : + kvStateInformation.values()) { + ForStOperationUtils.addColumnFamilyOptionsToCloseLater( + columnFamilyOptions, kvStateInfo.columnFamilyHandle); + IOUtils.closeQuietly(kvStateInfo.columnFamilyHandle); + } + IOUtils.closeQuietly(db); + // it's possible that db has been initialized but later restore steps failed + IOUtils.closeQuietly(restoreOperation); + IOUtils.closeAllQuietly(columnFamilyOptions); + IOUtils.closeQuietly(optionsContainer); + ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories(); + kvStateInformation.clear(); + + try { + FileUtils.deleteDirectory(instanceBasePath); + } catch (Exception ex) { + logger.warn("Failed to delete base path for RocksDB: " + instanceBasePath, ex); + } + // Log and rethrow + if (e instanceof BackendBuildingException) { + throw (BackendBuildingException) e; + } else { + String errMsg = "Caught unexpected exception."; + logger.error(errMsg, e); + throw new BackendBuildingException(errMsg, e); + } + } + InternalKeyContext keyContext = + new InternalKeyContextImpl<>(keyGroupRange, numberOfKeyGroups); + logger.info("Finished building RocksDB keyed state-backend at {}.", instanceBasePath); + return new ForStSyncKeyedStateBackend<>( + this.userCodeClassLoader, + this.instanceBasePath, + this.optionsContainer, + columnFamilyOptionsFactory, + this.kvStateRegistry, + this.keySerializerProvider.currentSchemaSerializer(), + this.executionConfig, + this.ttlTimeProvider, + latencyTrackingStateConfig, + db, + kvStateInformation, + registeredPQStates, + keyGroupPrefixBytes, + cancelRegistryForBackend, + this.keyGroupCompressionDecorator, + rocksDBResourceGuard, + // checkpointStrategy, + writeBatchWrapper, + defaultColumnFamilyHandle, + nativeMetricMonitor, + sharedRocksKeyBuilder, + priorityQueueFactory, + ttlCompactFiltersManager, + keyContext, + writeBatchSize, + asyncCompactAfterRestoreFuture); + } + + private ForStRestoreOperation getRocksDBRestoreOperation( + int keyGroupPrefixBytes, + ResourceGuard rocksDBResourceGuard, + CloseableRegistry cancelStreamRegistryForRestore, + CloseableRegistry cancelRegistryForBackend, + LinkedHashMap kvStateInformation, + LinkedHashMap> registeredPQStates, + ForStDBTtlCompactFiltersManager ttlCompactFiltersManager) { + DBOptions dbOptions = optionsContainer.getDbOptions(); + if (CollectionUtil.isEmptyOrAllElementsNull(restoreStateHandles)) { + return new ForStNoneRestoreOperation( + instanceForStDBPath, + optionsContainer.getDbOptions(), + columnFamilyOptionsFactory, + nativeMetricOptions, + metricGroup); + } + throw new UnsupportedOperationException("Not support restoring yet for ForStStateBackend"); + } + + private PriorityQueueSetFactory initPriorityQueueFactory( + int keyGroupPrefixBytes, + Map kvStateInformation, + RocksDB db, + ForStDBWriteBatchWrapper writeBatchWrapper, + ForStNativeMetricMonitor nativeMetricMonitor) { + PriorityQueueSetFactory priorityQueueFactory; + switch (priorityQueueConfig.getPriorityQueueStateType()) { + case HEAP: + priorityQueueFactory = createHeapQueueFactory(); + break; + case ForStDB: + priorityQueueFactory = + new ForStDBPriorityQueueSetFactory( + keyGroupRange, + keyGroupPrefixBytes, + numberOfKeyGroups, + kvStateInformation, + db, + optionsContainer.getReadOptions(), + writeBatchWrapper, + nativeMetricMonitor, + columnFamilyOptionsFactory, + optionsContainer.getWriteBufferManagerCapacity(), + priorityQueueConfig.getRocksDBPriorityQueueSetCacheSize()); + break; + default: + throw new IllegalArgumentException( + "Unknown priority queue state type: " + + priorityQueueConfig.getPriorityQueueStateType()); + } + return priorityQueueFactory; + } + + private HeapPriorityQueueSetFactory createHeapQueueFactory() { + return new HeapPriorityQueueSetFactory(keyGroupRange, numberOfKeyGroups, 128); + } + + private void prepareDirectories() throws IOException { + checkAndCreateDirectory(instanceBasePath); + if (instanceForStDBPath.exists()) { + // Clear the base directory when the backend is created + // in case something crashed and the backend never reached dispose() + FileUtils.deleteDirectory(instanceBasePath); + } + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncListState.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncListState.java new file mode 100644 index 0000000000000..2a03a11fb3d4d --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncListState.java @@ -0,0 +1,319 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.runtime.state.ListDelimitedSerializer; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.runtime.state.internal.InternalListState; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StateMigrationException; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import static org.apache.flink.runtime.state.StateSnapshotTransformer.CollectionStateSnapshotTransformer.TransformStrategy.STOP_ON_FIRST_INCLUDED; + +/** + * {@link ListState} implementation that stores state in RocksDB. + * + *

{@link org.apache.flink.state.forst.ForStStateBackend} must ensure that we set the {@link + * org.rocksdb.StringAppendOperator} on the column family that we use for our state since we use the + * {@code merge()} call. + * + * @param The type of the key. + * @param The type of the namespace. + * @param The type of the values in the list state. + */ +class ForStSyncListState extends AbstractForStSyncState> + implements InternalListState { + + /** Serializer for the values. */ + private TypeSerializer elementSerializer; + + private final ListDelimitedSerializer listSerializer; + + /** Separator of StringAppendTestOperator in RocksDB. */ + private static final byte DELIMITER = ','; + + /** + * Creates a new {@code RocksDBListState}. + * + * @param columnFamily The RocksDB column family that this state is associated to. + * @param namespaceSerializer The serializer for the namespace. + * @param valueSerializer The serializer for the state. + * @param defaultValue The default value for the state. + * @param backend The backend for which this state is bind to. + */ + private ForStSyncListState( + ColumnFamilyHandle columnFamily, + TypeSerializer namespaceSerializer, + TypeSerializer> valueSerializer, + List defaultValue, + ForStSyncKeyedStateBackend backend) { + + super(columnFamily, namespaceSerializer, valueSerializer, defaultValue, backend); + + ListSerializer castedListSerializer = (ListSerializer) valueSerializer; + this.elementSerializer = castedListSerializer.getElementSerializer(); + this.listSerializer = new ListDelimitedSerializer(); + } + + @Override + public TypeSerializer getKeySerializer() { + return backend.getKeySerializer(); + } + + @Override + public TypeSerializer getNamespaceSerializer() { + return namespaceSerializer; + } + + @Override + public TypeSerializer> getValueSerializer() { + return valueSerializer; + } + + @Override + public Iterable get() throws IOException, RocksDBException { + return getInternal(); + } + + @Override + public List getInternal() throws IOException, RocksDBException { + byte[] key = serializeCurrentKeyWithGroupAndNamespace(); + byte[] valueBytes = backend.db.get(columnFamily, key); + return listSerializer.deserializeList(valueBytes, elementSerializer); + } + + @Override + public void add(V value) throws IOException, RocksDBException { + Preconditions.checkNotNull(value, "You cannot add null to a ListState."); + + backend.db.merge( + columnFamily, + writeOptions, + serializeCurrentKeyWithGroupAndNamespace(), + serializeValue(value, elementSerializer)); + } + + @Override + public void mergeNamespaces(N target, Collection sources) { + if (sources == null || sources.isEmpty()) { + return; + } + + try { + // create the target full-binary-key + setCurrentNamespace(target); + final byte[] targetKey = serializeCurrentKeyWithGroupAndNamespace(); + + // merge the sources to the target + for (N source : sources) { + if (source != null) { + setCurrentNamespace(source); + final byte[] sourceKey = serializeCurrentKeyWithGroupAndNamespace(); + + byte[] valueBytes = backend.db.get(columnFamily, sourceKey); + + if (valueBytes != null) { + backend.db.delete(columnFamily, writeOptions, sourceKey); + backend.db.merge(columnFamily, writeOptions, targetKey, valueBytes); + } + } + } + } catch (Exception e) { + throw new FlinkRuntimeException("Error while merging state in RocksDB", e); + } + } + + @Override + public void update(List valueToStore) throws IOException, RocksDBException { + updateInternal(valueToStore); + } + + @Override + public void updateInternal(List values) throws IOException, RocksDBException { + Preconditions.checkNotNull(values, "List of values to add cannot be null."); + + if (!values.isEmpty()) { + backend.db.put( + columnFamily, + writeOptions, + serializeCurrentKeyWithGroupAndNamespace(), + listSerializer.serializeList(values, elementSerializer)); + } else { + clear(); + } + } + + @Override + public void addAll(List values) throws IOException, RocksDBException { + Preconditions.checkNotNull(values, "List of values to add cannot be null."); + + if (!values.isEmpty()) { + backend.db.merge( + columnFamily, + writeOptions, + serializeCurrentKeyWithGroupAndNamespace(), + listSerializer.serializeList(values, elementSerializer)); + } + } + + @Override + public void migrateSerializedValue( + DataInputDeserializer serializedOldValueInput, + DataOutputSerializer serializedMigratedValueOutput, + TypeSerializer> priorSerializer, + TypeSerializer> newSerializer) + throws StateMigrationException { + + Preconditions.checkArgument(priorSerializer instanceof ListSerializer); + Preconditions.checkArgument(newSerializer instanceof ListSerializer); + + TypeSerializer priorElementSerializer = + ((ListSerializer) priorSerializer).getElementSerializer(); + + TypeSerializer newElementSerializer = + ((ListSerializer) newSerializer).getElementSerializer(); + + try { + while (serializedOldValueInput.available() > 0) { + V element = + ListDelimitedSerializer.deserializeNextElement( + serializedOldValueInput, priorElementSerializer); + newElementSerializer.serialize(element, serializedMigratedValueOutput); + if (serializedOldValueInput.available() > 0) { + serializedMigratedValueOutput.write(DELIMITER); + } + } + } catch (Exception e) { + throw new StateMigrationException( + "Error while trying to migrate RocksDB list state.", e); + } + } + + @Override + protected ForStSyncListState setValueSerializer( + TypeSerializer> valueSerializer) { + super.setValueSerializer(valueSerializer); + this.elementSerializer = ((ListSerializer) valueSerializer).getElementSerializer(); + return this; + } + + @SuppressWarnings("unchecked") + static IS create( + StateDescriptor stateDesc, + Tuple2> + registerResult, + ForStSyncKeyedStateBackend backend) { + return (IS) + new ForStSyncListState<>( + registerResult.f0, + registerResult.f1.getNamespaceSerializer(), + (TypeSerializer>) registerResult.f1.getStateSerializer(), + (List) stateDesc.getDefaultValue(), + backend); + } + + @SuppressWarnings("unchecked") + static IS update( + StateDescriptor stateDesc, + Tuple2> + registerResult, + IS existingState) { + return (IS) + ((ForStSyncListState) existingState) + .setNamespaceSerializer(registerResult.f1.getNamespaceSerializer()) + .setValueSerializer( + (TypeSerializer>) registerResult.f1.getStateSerializer()) + .setDefaultValue((List) stateDesc.getDefaultValue()); + } + + static class StateSnapshotTransformerWrapper implements StateSnapshotTransformer { + private final StateSnapshotTransformer elementTransformer; + private final TypeSerializer elementSerializer; + private final CollectionStateSnapshotTransformer.TransformStrategy transformStrategy; + private final ListDelimitedSerializer listSerializer; + private final DataInputDeserializer in = new DataInputDeserializer(); + + StateSnapshotTransformerWrapper( + StateSnapshotTransformer elementTransformer, + TypeSerializer elementSerializer) { + this.elementTransformer = elementTransformer; + this.elementSerializer = elementSerializer; + this.listSerializer = new ListDelimitedSerializer(); + this.transformStrategy = + elementTransformer instanceof CollectionStateSnapshotTransformer + ? ((CollectionStateSnapshotTransformer) elementTransformer) + .getFilterStrategy() + : CollectionStateSnapshotTransformer.TransformStrategy.TRANSFORM_ALL; + } + + @Override + @Nullable + public byte[] filterOrTransform(@Nullable byte[] value) { + if (value == null) { + return null; + } + List result = new ArrayList<>(); + in.setBuffer(value); + T next; + int prevPosition = 0; + try { + while ((next = + ListDelimitedSerializer.deserializeNextElement( + in, elementSerializer)) + != null) { + T transformedElement = elementTransformer.filterOrTransform(next); + if (transformedElement != null) { + if (transformStrategy == STOP_ON_FIRST_INCLUDED) { + return Arrays.copyOfRange(value, prevPosition, value.length); + } else { + result.add(transformedElement); + } + } + prevPosition = in.getPosition(); + } + return result.isEmpty() + ? null + : listSerializer.serializeList(result, elementSerializer); + } catch (IOException e) { + throw new FlinkRuntimeException("Failed to serialize transformed list", e); + } + } + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncMapState.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncMapState.java new file mode 100644 index 0000000000000..8f3042f0e204b --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncMapState.java @@ -0,0 +1,806 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.SerializedCompositeKeyBuilder; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.runtime.state.internal.InternalMapState; +import org.apache.flink.state.forst.ForStDBWriteBatchWrapper; +import org.apache.flink.state.forst.ForStOperationUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StateMigrationException; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * {@link MapState} implementation that stores state in RocksDB. + * + * @param The type of the key. + * @param The type of the namespace. + * @param The type of the keys in the map state. + * @param The type of the values in the map state. + */ +class ForStSyncMapState extends AbstractForStSyncState> + implements InternalMapState { + + private static final Logger LOG = LoggerFactory.getLogger(ForStSyncMapState.class); + + /** Serializer for the keys and values. */ + private TypeSerializer userKeySerializer; + + private TypeSerializer userValueSerializer; + + /** + * Creates a new {@code RocksDBMapState}. + * + * @param columnFamily The RocksDB column family that this state is associated to. + * @param namespaceSerializer The serializer for the namespace. + * @param valueSerializer The serializer for the state. + * @param defaultValue The default value for the state. + * @param backend The backend for which this state is bind to. + */ + private ForStSyncMapState( + ColumnFamilyHandle columnFamily, + TypeSerializer namespaceSerializer, + TypeSerializer> valueSerializer, + Map defaultValue, + ForStSyncKeyedStateBackend backend) { + + super(columnFamily, namespaceSerializer, valueSerializer, defaultValue, backend); + + Preconditions.checkState( + valueSerializer instanceof MapSerializer, "Unexpected serializer type."); + + MapSerializer castedMapSerializer = (MapSerializer) valueSerializer; + this.userKeySerializer = castedMapSerializer.getKeySerializer(); + this.userValueSerializer = castedMapSerializer.getValueSerializer(); + } + + @Override + public TypeSerializer getKeySerializer() { + return backend.getKeySerializer(); + } + + @Override + public TypeSerializer getNamespaceSerializer() { + return namespaceSerializer; + } + + @Override + public TypeSerializer> getValueSerializer() { + return valueSerializer; + } + + // ------------------------------------------------------------------------ + // MapState Implementation + // ------------------------------------------------------------------------ + + @Override + public UV get(UK userKey) throws IOException, RocksDBException { + byte[] rawKeyBytes = + serializeCurrentKeyWithGroupAndNamespacePlusUserKey(userKey, userKeySerializer); + byte[] rawValueBytes = backend.db.get(columnFamily, rawKeyBytes); + + return (rawValueBytes == null + ? null + : deserializeUserValue(dataInputView, rawValueBytes, userValueSerializer)); + } + + @Override + public void put(UK userKey, UV userValue) throws IOException, RocksDBException { + + byte[] rawKeyBytes = + serializeCurrentKeyWithGroupAndNamespacePlusUserKey(userKey, userKeySerializer); + byte[] rawValueBytes = serializeValueNullSensitive(userValue, userValueSerializer); + + backend.db.put(columnFamily, writeOptions, rawKeyBytes, rawValueBytes); + } + + @Override + public void putAll(Map map) throws IOException, RocksDBException { + if (map == null) { + return; + } + + try (ForStDBWriteBatchWrapper writeBatchWrapper = + new ForStDBWriteBatchWrapper( + backend.db, writeOptions, backend.getWriteBatchSize())) { + for (Map.Entry entry : map.entrySet()) { + byte[] rawKeyBytes = + serializeCurrentKeyWithGroupAndNamespacePlusUserKey( + entry.getKey(), userKeySerializer); + byte[] rawValueBytes = + serializeValueNullSensitive(entry.getValue(), userValueSerializer); + writeBatchWrapper.put(columnFamily, rawKeyBytes, rawValueBytes); + } + } + } + + @Override + public void remove(UK userKey) throws IOException, RocksDBException { + byte[] rawKeyBytes = + serializeCurrentKeyWithGroupAndNamespacePlusUserKey(userKey, userKeySerializer); + + backend.db.delete(columnFamily, writeOptions, rawKeyBytes); + } + + @Override + public boolean contains(UK userKey) throws IOException, RocksDBException { + byte[] rawKeyBytes = + serializeCurrentKeyWithGroupAndNamespacePlusUserKey(userKey, userKeySerializer); + byte[] rawValueBytes = backend.db.get(columnFamily, rawKeyBytes); + + return (rawValueBytes != null); + } + + @Override + public Iterable> entries() { + return this::iterator; + } + + @Override + public Iterable keys() { + final byte[] prefixBytes = serializeCurrentKeyWithGroupAndNamespace(); + + return () -> + new RocksDBMapIterator( + backend.db, + prefixBytes, + userKeySerializer, + userValueSerializer, + dataInputView) { + @Nullable + @Override + public UK next() { + RocksDBMapEntry entry = nextEntry(); + return (entry == null ? null : entry.getKey()); + } + }; + } + + @Override + public Iterable values() { + final byte[] prefixBytes = serializeCurrentKeyWithGroupAndNamespace(); + + return () -> + new RocksDBMapIterator( + backend.db, + prefixBytes, + userKeySerializer, + userValueSerializer, + dataInputView) { + @Override + public UV next() { + RocksDBMapEntry entry = nextEntry(); + return (entry == null ? null : entry.getValue()); + } + }; + } + + @Override + public void migrateSerializedValue( + DataInputDeserializer serializedOldValueInput, + DataOutputSerializer serializedMigratedValueOutput, + TypeSerializer> priorSerializer, + TypeSerializer> newSerializer) + throws StateMigrationException { + + checkArgument(priorSerializer instanceof MapSerializer); + checkArgument(newSerializer instanceof MapSerializer); + + TypeSerializer priorMapValueSerializer = + ((MapSerializer) priorSerializer).getValueSerializer(); + TypeSerializer newMapValueSerializer = + ((MapSerializer) newSerializer).getValueSerializer(); + + try { + boolean isNull = serializedOldValueInput.readBoolean(); + UV mapUserValue = null; + if (!isNull) { + mapUserValue = priorMapValueSerializer.deserialize(serializedOldValueInput); + } + serializedMigratedValueOutput.writeBoolean(mapUserValue == null); + newMapValueSerializer.serialize(mapUserValue, serializedMigratedValueOutput); + } catch (Exception e) { + throw new StateMigrationException( + "Error while trying to migrate RocksDB map state.", e); + } + } + + @Override + public Iterator> iterator() { + final byte[] prefixBytes = serializeCurrentKeyWithGroupAndNamespace(); + + return new RocksDBMapIterator>( + backend.db, prefixBytes, userKeySerializer, userValueSerializer, dataInputView) { + @Override + public Map.Entry next() { + return nextEntry(); + } + }; + } + + @Override + public boolean isEmpty() { + final byte[] prefixBytes = serializeCurrentKeyWithGroupAndNamespace(); + + try (ForStIteratorWrapper iterator = + ForStOperationUtils.getForStIterator( + backend.db, columnFamily, backend.getReadOptions())) { + + iterator.seek(prefixBytes); + + return !iterator.isValid() || !startWithKeyPrefix(prefixBytes, iterator.key()); + } + } + + @Override + public void clear() { + try (ForStIteratorWrapper iterator = + ForStOperationUtils.getForStIterator( + backend.db, columnFamily, backend.getReadOptions()); + ForStDBWriteBatchWrapper rocksDBWriteBatchWrapper = + new ForStDBWriteBatchWrapper( + backend.db, + backend.getWriteOptions(), + backend.getWriteBatchSize())) { + + final byte[] keyPrefixBytes = serializeCurrentKeyWithGroupAndNamespace(); + iterator.seek(keyPrefixBytes); + + while (iterator.isValid()) { + byte[] keyBytes = iterator.key(); + if (startWithKeyPrefix(keyPrefixBytes, keyBytes)) { + rocksDBWriteBatchWrapper.remove(columnFamily, keyBytes); + } else { + break; + } + iterator.next(); + } + } catch (RocksDBException e) { + throw new FlinkRuntimeException("Error while cleaning the state in RocksDB.", e); + } + } + + @Override + protected ForStSyncMapState setValueSerializer( + TypeSerializer> valueSerializer) { + super.setValueSerializer(valueSerializer); + MapSerializer castedMapSerializer = (MapSerializer) valueSerializer; + this.userKeySerializer = castedMapSerializer.getKeySerializer(); + this.userValueSerializer = castedMapSerializer.getValueSerializer(); + return this; + } + + @Override + public byte[] getSerializedValue( + final byte[] serializedKeyAndNamespace, + final TypeSerializer safeKeySerializer, + final TypeSerializer safeNamespaceSerializer, + final TypeSerializer> safeValueSerializer) + throws Exception { + + Preconditions.checkNotNull(serializedKeyAndNamespace); + Preconditions.checkNotNull(safeKeySerializer); + Preconditions.checkNotNull(safeNamespaceSerializer); + Preconditions.checkNotNull(safeValueSerializer); + + // TODO make KvStateSerializer key-group aware to save this round trip and key-group + // computation + Tuple2 keyAndNamespace = + KvStateSerializer.deserializeKeyAndNamespace( + serializedKeyAndNamespace, safeKeySerializer, safeNamespaceSerializer); + + int keyGroup = + KeyGroupRangeAssignment.assignToKeyGroup( + keyAndNamespace.f0, backend.getNumberOfKeyGroups()); + + SerializedCompositeKeyBuilder keyBuilder = + new SerializedCompositeKeyBuilder<>( + safeKeySerializer, backend.getKeyGroupPrefixBytes(), 32); + + keyBuilder.setKeyAndKeyGroup(keyAndNamespace.f0, keyGroup); + + final byte[] keyPrefixBytes = + keyBuilder.buildCompositeKeyNamespace(keyAndNamespace.f1, namespaceSerializer); + + final MapSerializer serializer = (MapSerializer) safeValueSerializer; + + final TypeSerializer dupUserKeySerializer = serializer.getKeySerializer(); + final TypeSerializer dupUserValueSerializer = serializer.getValueSerializer(); + final DataInputDeserializer inputView = new DataInputDeserializer(); + + final Iterator> iterator = + new RocksDBMapIterator>( + backend.db, + keyPrefixBytes, + dupUserKeySerializer, + dupUserValueSerializer, + inputView) { + + @Override + public Map.Entry next() { + return nextEntry(); + } + }; + + // Return null to make the behavior consistent with other backends + if (!iterator.hasNext()) { + return null; + } + + return KvStateSerializer.serializeMap( + () -> iterator, dupUserKeySerializer, dupUserValueSerializer); + } + + // ------------------------------------------------------------------------ + // Serialization Methods + // ------------------------------------------------------------------------ + + private static UK deserializeUserKey( + DataInputDeserializer dataInputView, + int userKeyOffset, + byte[] rawKeyBytes, + TypeSerializer keySerializer) + throws IOException { + dataInputView.setBuffer(rawKeyBytes, userKeyOffset, rawKeyBytes.length - userKeyOffset); + return keySerializer.deserialize(dataInputView); + } + + private static UV deserializeUserValue( + DataInputDeserializer dataInputView, + byte[] rawValueBytes, + TypeSerializer valueSerializer) + throws IOException { + + dataInputView.setBuffer(rawValueBytes); + + boolean isNull = dataInputView.readBoolean(); + + return isNull ? null : valueSerializer.deserialize(dataInputView); + } + + private boolean startWithKeyPrefix(byte[] keyPrefixBytes, byte[] rawKeyBytes) { + if (rawKeyBytes.length < keyPrefixBytes.length) { + return false; + } + + for (int i = keyPrefixBytes.length; --i >= backend.getKeyGroupPrefixBytes(); ) { + if (rawKeyBytes[i] != keyPrefixBytes[i]) { + return false; + } + } + + return true; + } + + // ------------------------------------------------------------------------ + // Internal Classes + // ------------------------------------------------------------------------ + + /** A map entry in RocksDBMapState. */ + private class RocksDBMapEntry implements Map.Entry { + private final RocksDB db; + + /** + * The raw bytes of the key stored in RocksDB. Each user key is stored in RocksDB with the + * format #KeyGroup#Key#Namespace#UserKey. + */ + private final byte[] rawKeyBytes; + + /** The raw bytes of the value stored in RocksDB. */ + private byte[] rawValueBytes; + + /** True if the entry has been deleted. */ + private boolean deleted; + + /** + * The user key and value. The deserialization is performed lazily, i.e. the key and the + * value is deserialized only when they are accessed. + */ + private UK userKey; + + private UV userValue; + + /** The offset of User Key offset in raw key bytes. */ + private final int userKeyOffset; + + private final TypeSerializer keySerializer; + + private final TypeSerializer valueSerializer; + + private final DataInputDeserializer dataInputView; + + RocksDBMapEntry( + @Nonnull final RocksDB db, + @Nonnegative final int userKeyOffset, + @Nonnull final byte[] rawKeyBytes, + @Nonnull final byte[] rawValueBytes, + @Nonnull final TypeSerializer keySerializer, + @Nonnull final TypeSerializer valueSerializer, + @Nonnull DataInputDeserializer dataInputView) { + this.db = db; + + this.userKeyOffset = userKeyOffset; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + + this.rawKeyBytes = rawKeyBytes; + this.rawValueBytes = rawValueBytes; + this.deleted = false; + this.dataInputView = dataInputView; + } + + public void remove() { + deleted = true; + rawValueBytes = null; + + try { + db.delete(columnFamily, writeOptions, rawKeyBytes); + } catch (RocksDBException e) { + throw new FlinkRuntimeException("Error while removing data from RocksDB.", e); + } + } + + @Override + public UK getKey() { + if (userKey == null) { + try { + userKey = + deserializeUserKey( + dataInputView, userKeyOffset, rawKeyBytes, keySerializer); + } catch (IOException e) { + throw new FlinkRuntimeException("Error while deserializing the user key.", e); + } + } + + return userKey; + } + + @Override + public UV getValue() { + if (deleted) { + return null; + } else { + if (userValue == null) { + try { + userValue = + deserializeUserValue(dataInputView, rawValueBytes, valueSerializer); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Error while deserializing the user value.", e); + } + } + + return userValue; + } + } + + @Override + public UV setValue(UV value) { + if (deleted) { + throw new IllegalStateException("The value has already been deleted."); + } + + UV oldValue = getValue(); + + try { + userValue = value; + rawValueBytes = serializeValueNullSensitive(value, valueSerializer); + + db.put(columnFamily, writeOptions, rawKeyBytes, rawValueBytes); + } catch (IOException | RocksDBException e) { + throw new FlinkRuntimeException("Error while putting data into RocksDB.", e); + } + + return oldValue; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof Map.Entry)) { + return false; + } + Map.Entry e = (Map.Entry) o; + return Objects.equals(getKey(), e.getKey()) && Objects.equals(getValue(), e.getValue()); + } + } + + /** An auxiliary utility to scan all entries under the given key. */ + private abstract class RocksDBMapIterator implements Iterator { + + private static final int CACHE_SIZE_LIMIT = 128; + + /** The db where data resides. */ + private final RocksDB db; + + /** + * The prefix bytes of the key being accessed. All entries under the same key have the same + * prefix, hence we can stop iterating once coming across an entry with a different prefix. + */ + @Nonnull private final byte[] keyPrefixBytes; + + /** + * True if all entries have been accessed or the iterator has come across an entry with a + * different prefix. + */ + private boolean expired = false; + + /** A in-memory cache for the entries in the rocksdb. */ + private ArrayList cacheEntries = new ArrayList<>(); + + /** + * The entry pointing to the current position which is last returned by calling {@link + * #nextEntry()}. + */ + private RocksDBMapEntry currentEntry; + + private int cacheIndex = 0; + + private final TypeSerializer keySerializer; + private final TypeSerializer valueSerializer; + private final DataInputDeserializer dataInputView; + + RocksDBMapIterator( + final RocksDB db, + final byte[] keyPrefixBytes, + final TypeSerializer keySerializer, + final TypeSerializer valueSerializer, + DataInputDeserializer dataInputView) { + + this.db = db; + this.keyPrefixBytes = keyPrefixBytes; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.dataInputView = dataInputView; + } + + @Override + public boolean hasNext() { + loadCache(); + + return (cacheIndex < cacheEntries.size()); + } + + @Override + public void remove() { + if (currentEntry == null || currentEntry.deleted) { + throw new IllegalStateException( + "The remove operation must be called after a valid next operation."); + } + + currentEntry.remove(); + } + + final RocksDBMapEntry nextEntry() { + loadCache(); + + if (cacheIndex == cacheEntries.size()) { + if (!expired) { + throw new IllegalStateException(); + } + + return null; + } + + this.currentEntry = cacheEntries.get(cacheIndex); + cacheIndex++; + + return currentEntry; + } + + private void loadCache() { + if (cacheIndex > cacheEntries.size()) { + throw new IllegalStateException(); + } + + // Load cache entries only when the cache is empty and there still exist unread entries + if (cacheIndex < cacheEntries.size() || expired) { + return; + } + + // use try-with-resources to ensure RocksIterator can be release even some runtime + // exception + // occurred in the below code block. + try (ForStIteratorWrapper iterator = + ForStOperationUtils.getForStIterator( + db, columnFamily, backend.getReadOptions())) { + + /* + * The iteration starts from the prefix bytes at the first loading. After #nextEntry() is called, + * the currentEntry points to the last returned entry, and at that time, we will start + * the iterating from currentEntry if reloading cache is needed. + */ + byte[] startBytes = + (currentEntry == null ? keyPrefixBytes : currentEntry.rawKeyBytes); + + cacheEntries.clear(); + cacheIndex = 0; + + iterator.seek(startBytes); + + /* + * If the entry pointing to the current position is not removed, it will be the first entry in the + * new iterating. Skip it to avoid redundant access in such cases. + */ + if (currentEntry != null && !currentEntry.deleted) { + iterator.next(); + } + + while (true) { + if (!iterator.isValid() + || !startWithKeyPrefix(keyPrefixBytes, iterator.key())) { + expired = true; + break; + } + + if (cacheEntries.size() >= CACHE_SIZE_LIMIT) { + break; + } + + RocksDBMapEntry entry = + new RocksDBMapEntry( + db, + keyPrefixBytes.length, + iterator.key(), + iterator.value(), + keySerializer, + valueSerializer, + dataInputView); + + cacheEntries.add(entry); + + iterator.next(); + } + } + } + } + + @SuppressWarnings("unchecked") + static IS create( + StateDescriptor stateDesc, + Tuple2> + registerResult, + ForStSyncKeyedStateBackend backend) { + return (IS) + new ForStSyncMapState<>( + registerResult.f0, + registerResult.f1.getNamespaceSerializer(), + (TypeSerializer>) registerResult.f1.getStateSerializer(), + (Map) stateDesc.getDefaultValue(), + backend); + } + + @SuppressWarnings("unchecked") + static IS update( + StateDescriptor stateDesc, + Tuple2> + registerResult, + IS existingState) { + return (IS) + ((ForStSyncMapState) existingState) + .setNamespaceSerializer(registerResult.f1.getNamespaceSerializer()) + .setValueSerializer( + (TypeSerializer>) + registerResult.f1.getStateSerializer()) + .setDefaultValue((Map) stateDesc.getDefaultValue()); + } + + /** + * RocksDB map state specific byte value transformer wrapper. + * + *

This specific transformer wrapper checks the first byte to detect null user value entries + * and if not null forward the rest of byte array to the original byte value transformer. + */ + static class StateSnapshotTransformerWrapper implements StateSnapshotTransformer { + private static final byte[] NULL_VALUE; + private static final byte NON_NULL_VALUE_PREFIX; + + static { + DataOutputSerializer dov = new DataOutputSerializer(1); + try { + dov.writeBoolean(true); + NULL_VALUE = dov.getCopyOfBuffer(); + dov.clear(); + dov.writeBoolean(false); + NON_NULL_VALUE_PREFIX = dov.getSharedBuffer()[0]; + } catch (IOException e) { + throw new FlinkRuntimeException( + "Failed to serialize boolean flag of map user null value", e); + } + } + + private final StateSnapshotTransformer elementTransformer; + private final DataInputDeserializer div; + + StateSnapshotTransformerWrapper(StateSnapshotTransformer originalTransformer) { + this.elementTransformer = originalTransformer; + this.div = new DataInputDeserializer(); + } + + @Override + @Nullable + public byte[] filterOrTransform(@Nullable byte[] value) { + if (value == null || isNull(value)) { + return NULL_VALUE; + } else { + // we have to skip the first byte indicating null user value + // TODO: optimization here could be to work with slices and not byte arrays + // and copy slice sub-array only when needed + byte[] woNullByte = Arrays.copyOfRange(value, 1, value.length); + byte[] filteredValue = elementTransformer.filterOrTransform(woNullByte); + if (filteredValue == null) { + filteredValue = NULL_VALUE; + } else if (filteredValue != woNullByte) { + filteredValue = prependWithNonNullByte(filteredValue, value); + } else { + filteredValue = value; + } + return filteredValue; + } + } + + private boolean isNull(byte[] value) { + try { + div.setBuffer(value, 0, 1); + return div.readBoolean(); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Failed to deserialize boolean flag of map user null value", e); + } + } + + private static byte[] prependWithNonNullByte(byte[] value, byte[] reuse) { + int len = 1 + value.length; + byte[] result = reuse.length == len ? reuse : new byte[len]; + result[0] = NON_NULL_VALUE_PREFIX; + System.arraycopy(value, 0, result, 1, value.length); + return result; + } + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncReducingState.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncReducingState.java new file mode 100644 index 0000000000000..bdc23ddc66379 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncReducingState.java @@ -0,0 +1,187 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.state.ReducingState; +import org.apache.flink.api.common.state.ReducingStateDescriptor; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.internal.InternalReducingState; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; + +import java.io.IOException; +import java.util.Collection; + +/** + * {@link ReducingState} implementation that stores state in RocksDB. + * + * @param The type of the key. + * @param The type of the namespace. + * @param The type of value that the state state stores. + */ +class ForStSyncReducingState extends AbstractForStSyncAppendingState + implements InternalReducingState { + + /** User-specified reduce function. */ + private ReduceFunction reduceFunction; + + /** + * Creates a new {@code RocksDBReducingState}. + * + * @param columnFamily The RocksDB column family that this state is associated to. + * @param namespaceSerializer The serializer for the namespace. + * @param valueSerializer The serializer for the state. + * @param defaultValue The default value for the state. + * @param reduceFunction The reduce function used for reducing state. + * @param backend The backend for which this state is bind to. + */ + private ForStSyncReducingState( + ColumnFamilyHandle columnFamily, + TypeSerializer namespaceSerializer, + TypeSerializer valueSerializer, + V defaultValue, + ReduceFunction reduceFunction, + ForStSyncKeyedStateBackend backend) { + + super(columnFamily, namespaceSerializer, valueSerializer, defaultValue, backend); + this.reduceFunction = reduceFunction; + } + + @Override + public TypeSerializer getKeySerializer() { + return backend.getKeySerializer(); + } + + @Override + public TypeSerializer getNamespaceSerializer() { + return namespaceSerializer; + } + + @Override + public TypeSerializer getValueSerializer() { + return valueSerializer; + } + + @Override + public V get() throws IOException, RocksDBException { + return getInternal(); + } + + @Override + public void add(V value) throws Exception { + byte[] key = getKeyBytes(); + V oldValue = getInternal(key); + V newValue = oldValue == null ? value : reduceFunction.reduce(oldValue, value); + updateInternal(key, newValue); + } + + @Override + public void mergeNamespaces(N target, Collection sources) throws Exception { + if (sources == null || sources.isEmpty()) { + return; + } + + V current = null; + + // merge the sources to the target + for (N source : sources) { + if (source != null) { + setCurrentNamespace(source); + final byte[] sourceKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] valueBytes = backend.db.get(columnFamily, sourceKey); + + if (valueBytes != null) { + backend.db.delete(columnFamily, writeOptions, sourceKey); + dataInputView.setBuffer(valueBytes); + V value = valueSerializer.deserialize(dataInputView); + + if (current != null) { + current = reduceFunction.reduce(current, value); + } else { + current = value; + } + } + } + } + + // if something came out of merging the sources, merge it or write it to the target + if (current != null) { + // create the target full-binary-key + setCurrentNamespace(target); + final byte[] targetKey = serializeCurrentKeyWithGroupAndNamespace(); + final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey); + + if (targetValueBytes != null) { + dataInputView.setBuffer(targetValueBytes); + // target also had a value, merge + V value = valueSerializer.deserialize(dataInputView); + + current = reduceFunction.reduce(current, value); + } + + // serialize the resulting value + dataOutputView.clear(); + valueSerializer.serialize(current, dataOutputView); + + // write the resulting value + backend.db.put(columnFamily, writeOptions, targetKey, dataOutputView.getCopyOfBuffer()); + } + } + + ForStSyncReducingState setReduceFunction(ReduceFunction reduceFunction) { + this.reduceFunction = reduceFunction; + return this; + } + + @SuppressWarnings("unchecked") + static IS create( + StateDescriptor stateDesc, + Tuple2> + registerResult, + ForStSyncKeyedStateBackend backend) { + return (IS) + new ForStSyncReducingState<>( + registerResult.f0, + registerResult.f1.getNamespaceSerializer(), + registerResult.f1.getStateSerializer(), + stateDesc.getDefaultValue(), + ((ReducingStateDescriptor) stateDesc).getReduceFunction(), + backend); + } + + @SuppressWarnings("unchecked") + static IS update( + StateDescriptor stateDesc, + Tuple2> + registerResult, + IS existingState) { + return (IS) + ((ForStSyncReducingState) existingState) + .setReduceFunction( + ((ReducingStateDescriptor) stateDesc).getReduceFunction()) + .setNamespaceSerializer(registerResult.f1.getNamespaceSerializer()) + .setDefaultValue(stateDesc.getDefaultValue()); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncValueState.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncValueState.java new file mode 100644 index 0000000000000..b1933e3368909 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/ForStSyncValueState.java @@ -0,0 +1,139 @@ +/* + * 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.state.forst.sync; + +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; +import org.apache.flink.runtime.state.internal.InternalValueState; + +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.RocksDBException; + +import java.io.IOException; + +/** + * {@link ValueState} implementation that stores state in RocksDB. + * + * @param The type of the key. + * @param The type of the namespace. + * @param The type of value that the state state stores. + */ +class ForStSyncValueState extends AbstractForStSyncState + implements InternalValueState { + + /** + * Creates a new {@code RocksDBValueState}. + * + * @param columnFamily The RocksDB column family that this state is associated to. + * @param namespaceSerializer The serializer for the namespace. + * @param valueSerializer The serializer for the state. + * @param defaultValue The default value for the state. + * @param backend The backend for which this state is bind to. + */ + private ForStSyncValueState( + ColumnFamilyHandle columnFamily, + TypeSerializer namespaceSerializer, + TypeSerializer valueSerializer, + V defaultValue, + ForStSyncKeyedStateBackend backend) { + + super(columnFamily, namespaceSerializer, valueSerializer, defaultValue, backend); + } + + @Override + public TypeSerializer getKeySerializer() { + return backend.getKeySerializer(); + } + + @Override + public TypeSerializer getNamespaceSerializer() { + return namespaceSerializer; + } + + @Override + public TypeSerializer getValueSerializer() { + return valueSerializer; + } + + @Override + public V value() throws IOException { + try { + byte[] valueBytes = + backend.db.get(columnFamily, serializeCurrentKeyWithGroupAndNamespace()); + + if (valueBytes == null) { + return getDefaultValue(); + } + dataInputView.setBuffer(valueBytes); + return valueSerializer.deserialize(dataInputView); + } catch (RocksDBException e) { + throw new IOException("Error while retrieving data from RocksDB.", e); + } + } + + @Override + public void update(V value) throws IOException { + if (value == null) { + clear(); + return; + } + + try { + backend.db.put( + columnFamily, + writeOptions, + serializeCurrentKeyWithGroupAndNamespace(), + serializeValue(value)); + } catch (RocksDBException e) { + throw new IOException("Error while adding data to RocksDB", e); + } + } + + @SuppressWarnings("unchecked") + static IS create( + StateDescriptor stateDesc, + Tuple2> + registerResult, + ForStSyncKeyedStateBackend backend) { + return (IS) + new ForStSyncValueState<>( + registerResult.f0, + registerResult.f1.getNamespaceSerializer(), + registerResult.f1.getStateSerializer(), + stateDesc.getDefaultValue(), + backend); + } + + @SuppressWarnings("unchecked") + static IS update( + StateDescriptor stateDesc, + Tuple2> + registerResult, + IS existingState) { + return (IS) + ((ForStSyncValueState) existingState) + .setNamespaceSerializer(registerResult.f1.getNamespaceSerializer()) + .setValueSerializer(registerResult.f1.getStateSerializer()) + .setDefaultValue(stateDesc.getDefaultValue()); + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/TreeOrderedSetCache.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/TreeOrderedSetCache.java new file mode 100644 index 0000000000000..d9dd03764f6c8 --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/sync/TreeOrderedSetCache.java @@ -0,0 +1,102 @@ +/* + * 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.state.forst.sync; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.util.Iterator; +import java.util.TreeSet; + +/** Implementation of a lexicographically ordered set of byte-arrays, based on a {@link TreeSet}. */ +public class TreeOrderedSetCache + implements ForStDBCachingPriorityQueueSet.OrderedByteArraySetCache { + + /** Maximum capacity. */ + private final int maxSize; + + @Nonnull private final TreeSet treeSet; + + TreeOrderedSetCache(int maxSize) { + this.maxSize = maxSize; + this.treeSet = new TreeSet<>(LEXICOGRAPHIC_BYTE_COMPARATOR); + } + + @Override + public int size() { + return treeSet.size(); + } + + @Override + public int maxSize() { + return maxSize; + } + + @Override + public boolean isEmpty() { + return treeSet.isEmpty(); + } + + @Override + public boolean isFull() { + return treeSet.size() >= maxSize; + } + + @Override + public boolean add(@Nonnull byte[] toAdd) { + return treeSet.add(toAdd); + } + + @Override + public boolean remove(@Nonnull byte[] toRemove) { + return treeSet.remove(toRemove); + } + + @Nullable + @Override + public byte[] peekFirst() { + return !isEmpty() ? treeSet.first() : null; + } + + @Nullable + @Override + public byte[] peekLast() { + return !isEmpty() ? treeSet.last() : null; + } + + @Nullable + @Override + public byte[] pollFirst() { + return !isEmpty() ? treeSet.pollFirst() : null; + } + + @Nullable + @Override + public byte[] pollLast() { + return !isEmpty() ? treeSet.pollLast() : null; + } + + @Override + public void bulkLoadFromOrderedIterator(@Nonnull Iterator orderedIterator) { + treeSet.clear(); + for (int i = maxSize; --i >= 0 && orderedIterator.hasNext(); ) { + treeSet.add(orderedIterator.next()); + } + } +} diff --git a/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStStateBackendTest.java b/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStStateBackendTest.java new file mode 100644 index 0000000000000..b37ab51d26dbb --- /dev/null +++ b/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/ForStStateBackendTest.java @@ -0,0 +1,95 @@ +/* + * 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.state.forst; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.CheckpointStorage; +import org.apache.flink.runtime.state.ConfigurableStateBackend; +import org.apache.flink.runtime.state.StateBackendTestBase; +import org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage; +import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.apache.flink.testutils.junit.utils.TempDirUtils; +import org.apache.flink.util.function.SupplierWithException; + +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.state.forst.ForStOptions.LOCAL_DIRECTORIES; + +/** Tests for the partitioned state part of {@link ForStStateBackendTest}. */ +@Disabled("ForStStateBackend is not support checkpoint yet, some tests can't run") +@ExtendWith(ParameterizedTestExtension.class) +class ForStStateBackendTest extends StateBackendTestBase { + @TempDir private static java.nio.file.Path tempFolder; + + @Parameters + public static List modes() { + return Arrays.asList( + new Object[][] { + { + (SupplierWithException) + JobManagerCheckpointStorage::new + }, + { + (SupplierWithException) + () -> { + String checkpointPath = + TempDirUtils.newFolder(tempFolder).toURI().toString(); + return new FileSystemCheckpointStorage( + new Path(checkpointPath), 0, -1); + } + } + }); + } + + @Parameter public SupplierWithException storageSupplier; + + @Override + protected CheckpointStorage getCheckpointStorage() throws Exception { + return storageSupplier.get(); + } + + @Override + protected ConfigurableStateBackend getStateBackend() throws Exception { + ForStStateBackend backend = new ForStStateBackend(); + Configuration config = new Configuration(); + config.set(LOCAL_DIRECTORIES, tempFolder.toString()); + backend.configure(config, Thread.currentThread().getContextClassLoader()); + return new ForStStateBackend(); + } + + @Override + protected boolean isSerializerPresenceRequiredOnRestore() { + return false; + } + + @Override + protected boolean supportsAsynchronousSnapshots() { + return true; + } +}