Hi Yang,
Why your checkpoint is failed, was that checkpoint expired or failed due to error? Could you paste the jstack result of what are RocksDB doing during checkpoint? BTW, you could also use async-profiler [1] to view what the CPU operation of your actions, this tool could help to view what's RocksDB doing. [1] https://github.com/jvm-profiling-tools/async-profiler Best Yun Tang ________________________________ From: Andrey Zagrebin <azagre...@apache.org> Sent: Friday, December 4, 2020 17:49 To: user <user@flink.apache.org> Subject: Re: Flink 1.9Version State TTL parameter configuration it does not work Hi Yang, (redirecting this to user mailing list as this is not a dev question) I am not sure why the state loading is stuck after enabling the compaction filter but the background cleanup of RocksDB state with TTL will not work without activating the filter. This happens on RocksDB opening in Flink, before any state is created and it starts to load. Which version of Flink do you use? Did you try to enable the filter without starting from the checkpoint, basically from the beginning of the job run? Best, Andrey On Fri, Dec 4, 2020 at 11:27 AM Yang Peng <yangpengklf...@gmail.com<mailto:yangpengklf...@gmail.com>> wrote: Hi,I have some questions about state TTL to consult with everybody,the statebackend is rocksdb Below is my code: -----------------code begin------------- private static final String EV_STATE_FLAG = "EV_EID_FLAG"; StateTtlConfig ttlConfig = StateTtlConfig .newBuilder(Time.minutes(60)) .updateTtlOnCreateAndWrite() .neverReturnExpired() .cleanupInRocksdbCompactFilter(1000) .build(); MapStateDescriptor<String, Integer> eidMapStateDesc = new MapStateDescriptor<>( EV_STATE_FLAG , BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); eidMapStateDesc.enableTimeToLive(ttlConfig); eidMapState = getRuntimeContext().getMapState(eidMapStateDesc); -----------------code end----------------- I have set the TTL of the state is 60mins, But after 12 hours, through the monitor of rocksdb metric , we found that the sst file of CF:EV_EID_FLAG has been increasing, and there is no decreasing trend. Later we found some information from the taskmanager log:*WARN org.rocksdb.FlinkCompactionFilter - Cannot configure RocksDB TTL compaction filter for state < EV_EID_FLAG >: feature is disabled for the state backend* After I added "*state.backend.rocksdb.ttl.compaction.filter.**enabled: true*" this parameter, the warn information disappeared, but ater the project completed some checkpoints ,The next checkpoint will always fail, I checked the jstack command and found that the fail checkpoint was stuck in acquiring state ,disk io is idle;remove the " *state.backend.rocksdb.ttl.compaction.filter.**enabled: true"* the parameter,the project will resume the checkpoint. So I’m asking everyone here. Is my usage method wrong?