flink-user-zh mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Yun Tang <myas...@live.com>
Subject Re: Question about RocksDBStateBackend Compaction Filter state cleanup
Date Tue, 17 Mar 2020 08:04:31 GMT
Hi Lake

Flink leverage RocksDB's background compaction mechanism to filter out-of-TTL entries (by
comparing with current timestamp provided from RocksDB's time_provider) to not let them stay
in newly compacted data.

This would iterator over data entries with FlinkCompactionFilter::FilterV2 [1], and the parameter
'queryTimeAfterNumEntries' in Flink indicates the threshold 'query_time_after_num_entries_'
in FrocksDB  [2]. Once RocksDB iterator more than several entries .e.g 1000, it would call
time_provider to update current timestamp to let the process of cleaning up more eagerly and
accurately.

[1] https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/utilities/flink/flink_compaction_filter.cc#L107
[2] https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/utilities/flink/flink_compaction_filter.h#L140

Best
Yun Tang

________________________________
From: LakeShen <shenleifighting@gmail.com>
Sent: Tuesday, March 17, 2020 15:30
To: dev <dev@flink.apache.org>; user-zh <user-zh@flink.apache.org>; user <user@flink.apache.org>
Subject: Question about RocksDBStateBackend Compaction Filter state cleanup

Hi community ,

I see the flink RocksDBStateBackend state cleanup,now the code like this :


StateTtlConfig ttlConfig = StateTtlConfig
    .newBuilder(Time.seconds(1))
    .cleanupInRocksdbCompactFilter(1000)
    .build();


The default background cleanup for RocksDB backend queries the current timestamp each time
1000 entries have been processed.

What's the meaning of  1000 entries? 1000 different key ?

Thanks to your reply.

Best regards,
LakeShen

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message