[ https://issues.apache.org/jira/browse/FLINK-10471?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16699441#comment-16699441 ]
ASF GitHub Bot commented on FLINK-10471: ---------------------------------------- azagrebin commented on a change in pull request #7163: [FLINK-10471][State TTL] State TTL cleanup using RocksDb compaction filter URL: https://github.com/apache/flink/pull/7163#discussion_r236383506 ########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ttl/RocksDbTtlCompactFilterUtils.java ########## @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.contrib.streaming.state.ttl; + +import org.apache.flink.annotation.VisibleForTesting; +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.TypeSerializerSnapshot; +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.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.runtime.state.ttl.TtlStateFactory; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; + +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.FlinkCompactionFilter; +import org.rocksdb.InfoLogLevel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.IOException; + +/** RocksDB compaction filter utils for state with TTL. */ +public class RocksDbTtlCompactFilterUtils { + private static final Logger LOG = LoggerFactory.getLogger(FlinkCompactionFilter.class); + + /** Enables RocksDb compaction filter for State with TTL. */ + private final boolean enableTtlCompactionFilter; + + public RocksDbTtlCompactFilterUtils(boolean enableTtlCompactionFilter) { + this.enableTtlCompactionFilter = enableTtlCompactionFilter; + } + + public FlinkCompactionFilter setCompactFilterIfStateTtl( + @Nonnull StateMetaInfoSnapshot stateMetaInfoSnapshot, @Nonnull ColumnFamilyOptions options) { + boolean keyValueState = stateMetaInfoSnapshot.getBackendStateType() == StateMetaInfoSnapshot.BackendStateType.KEY_VALUE; + if (enableTtlCompactionFilter && keyValueState) { + @SuppressWarnings("unchecked") + TypeSerializerSnapshot<?> stateSerializerSnapshot = Preconditions.checkNotNull( + (TypeSerializerSnapshot<?>) stateMetaInfoSnapshot.getTypeSerializerConfigSnapshot( + StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString())); + TypeSerializer<?> serializer = stateSerializerSnapshot.restoreSerializer(); + if (TtlStateFactory.TtlSerializer.isTtlStateSerializer(serializer)) { + FlinkCompactionFilter compactFilter = new FlinkCompactionFilter(createRocksDbNativeLogger()); + //noinspection resource + options.setCompactionFilter(compactFilter); + return compactFilter; + } + } + return null; + } + + public FlinkCompactionFilter setCompactFilterIfStateTtl( + @Nonnull RegisteredStateMetaInfoBase metaInfoBase, @Nonnull ColumnFamilyOptions options) { + if (enableTtlCompactionFilter && metaInfoBase instanceof RegisteredKeyValueStateBackendMetaInfo) { + RegisteredKeyValueStateBackendMetaInfo kvMetaInfoBase = (RegisteredKeyValueStateBackendMetaInfo) metaInfoBase; + if (TtlStateFactory.TtlSerializer.isTtlStateSerializer(kvMetaInfoBase.getStateSerializer())) { + FlinkCompactionFilter compactFilter = new FlinkCompactionFilter(createRocksDbNativeLogger()); + //noinspection resource + options.setCompactionFilter(compactFilter); + return compactFilter; + } + } + return null; + } + + 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, FlinkCompactionFilter compactionFilter) { + StateTtlConfig ttlConfig = stateDesc.getTtlConfig(); + boolean useSystemTime = !useCustomTimeForTtlCompactFilter && + ttlConfig.getTimeCharacteristic() == StateTtlConfig.TimeCharacteristic.ProcessingTime; + configCompactFilter(stateDesc, compactionFilter, useSystemTime); + } + + private void configCompactFilter( + @Nonnull StateDescriptor<?, ?> stateDesc, FlinkCompactionFilter compactionFilter, boolean useSystemTime) { + StateTtlConfig ttlConfig = stateDesc.getTtlConfig(); + if (ttlConfig.isEnabled() && ttlConfig.getCleanupStrategies().inRocksdbCompactFilter()) { + if (!enableTtlCompactionFilter) { + LOG.warn("Cannot configure RocksDB TTL compaction filter for state <{}>: " + + "feature is disabled for the state backend.", stateDesc.getName()); + } + assert compactionFilter != null; + FlinkCompactionFilter.Config config; + if (stateDesc instanceof ListStateDescriptor) { + ListStateDescriptor<?> listStateDesc = (ListStateDescriptor<?>) stateDesc; + int len = listStateDesc.getElementSerializer().getLength(); + if (len > 0) { + config = FlinkCompactionFilter.Config.createForFixedElementList( + 0, ttlConfig.getTtl().toMilliseconds(), useSystemTime, len + 1); + } else { + config = FlinkCompactionFilter.Config.createForList( + 0, ttlConfig.getTtl().toMilliseconds(), useSystemTime, + new ListElementIter<>(listStateDesc.getElementSerializer())); + } + } else { + int timestampOffset = stateDesc instanceof MapStateDescriptor ? 1 : 0; + config = FlinkCompactionFilter.Config.create( + FlinkCompactionFilter.StateType.Value, timestampOffset, ttlConfig.getTtl().toMilliseconds(), useSystemTime); + } + compactionFilter.configure(config); + } + } + + private static class ListElementIter<T> implements FlinkCompactionFilter.ListElementIter { + private final TypeSerializer<T> serializer; + private DataInputDeserializer input; + + private ListElementIter(TypeSerializer<T> serializer) { + this.serializer = serializer; + } + + @Override + public void setListBytes(byte[] bytes) { + input = new DataInputDeserializer(bytes); + } + + @Override + public int nextOffset(int currentOffset) { + try { + serializer.deserialize(input); Review comment: Good point, earlier they were stateless but after optimising and setting the byte value of list before iterating the elements, it can be accessed concurrently. I refactored the code to use factories which produce objects used by single thread. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL cleanup using RocksDb compaction filter > ------------------------------------------------- > > Key: FLINK-10471 > URL: https://issues.apache.org/jira/browse/FLINK-10471 > Project: Flink > Issue Type: New Feature > Components: State Backends, Checkpointing > Affects Versions: 1.7.0 > Reporter: Andrey Zagrebin > Assignee: Andrey Zagrebin > Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > -- This message was sent by Atlassian JIRA (v7.6.3#76005)