question on TTLs and compaction scheduling

229 views
Skip to first unread message

Ben Clay

unread,
Mar 5, 2019, 6:50:47 PM3/5/19
to roc...@googlegroups.com, Donghua Liu, manhattan-rocksdb
Folks-

We use RocksDB with merge operators and user-defined compaction filters.

Reading https://github.com/facebook/rocksdb/wiki/Leveled-Compaction#ttl we were wondering: Does enabling the TTL setting on the column family merely schedule compaction (using the user-defined compaction filters), or does it do that AND scrub away data older than the TTL without passing control to the user-defined compaction filters?  We would like to take advantage of the monitoring and scheduling from the ColumnFamily TTL feature but have our own user-defined compaction filters actually choose what data is removed.

Any guidance here would be appreciated.

Thanks!
-Ben

Ben Clay

unread,
Mar 5, 2019, 6:53:30 PM3/5/19
to roc...@googlegroups.com, Donghua Liu, manhattan-rocksdb
This is for Level compaction FWIW.

-Ben

Sagar Vemuri

unread,
Mar 6, 2019, 5:29:58 PM3/6/19
to Ben Clay, rocksdb, Donghua Liu, manhattan-rocksdb
The files picked up with Level+TTL compaction go through the regular compaction process -- i.e user-defined compaction filters will be invoked if they exist.

-Sagar

--
You received this message because you are subscribed to the Google Groups "rocksdb" group.
To unsubscribe from this group and stop receiving emails from it, send an email to rocksdb+u...@googlegroups.com.
To post to this group, send email to roc...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/rocksdb/CALB2BQ8oZukmN-79ojPaWWnmyBsc6xg9yz67w6xaqhorzS6xpQ%40mail.gmail.com.
For more options, visit https://groups.google.com/d/optout.

Ben Clay

unread,
Mar 6, 2019, 5:38:14 PM3/6/19
to Sagar Vemuri, rocksdb, Donghua Liu, manhattan-rocksdb
OK good to know.  So conversely, if I wanted data to actually expire out after a set TTL, I would set a nonzero TTL on the column family options and then write my own compaction filter to introspect the value (or where-ever I store a timestamp) to decide how old the item is?  Or put another way - if I want RocksDB to apply a simple TTL universally across the DB, how might that be enforced?

-Ben

Sagar Vemuri

unread,
Mar 7, 2019, 4:54:09 PM3/7/19
to Ben Clay, rocksdb, Donghua Liu, manhattan-rocksdb
 So conversely, if I wanted data to actually expire out after a set TTL, I would set a nonzero TTL on the column family options and then write my own compaction filter to introspect the value (or where-ever I store a timestamp) to decide how old the item is?
 
Yeah, that's the direction to go -- using compaction filters to introspect and make a decision, when a TTL-expired file is picked for compaction. 
Out of curiosity, could you provide a little more information about how your timestamps are stored, and what do you use them for? We are looking at how best to support timestamps in RocksDB, and wanted to see if your use-case could be handled better in the future. 

if I want RocksDB to apply a simple TTL universally across the DB, how might that be enforced?
Do you mean, making use of the current `ttl` option across column families?  Or something else? 

Ben Clay

unread,
Mar 13, 2019, 1:23:59 PM3/13/19
to Sagar Vemuri, rocksdb, Donghua Liu, manhattan-rocksdb
 So conversely, if I wanted data to actually expire out after a set TTL, I would set a nonzero TTL on the column family options and then write my own compaction filter to introspect the value (or where-ever I store a timestamp) to decide how old the item is?
 
Yeah, that's the direction to go -- using compaction filters to introspect and make a decision, when a TTL-expired file is picked for compaction. 
Out of curiosity, could you provide a little more information about how your timestamps are stored, and what do you use them for? We are looking at how best to support timestamps in RocksDB, and wanted to see if your use-case could be handled better in the future. 

We have item-level TTLs that we embed in the value along with other relevant fields. Specifically, we embed expiry time, unpack that in our custom compaction filter and either delete the item completely to a tombstone or demote to a tombstone marker under the same key depending on customer requirements.  We may not be the best model for a general-purpose use case but that's what we do.
 
if I want RocksDB to apply a simple TTL universally across the DB, how might that be enforced?
Do you mean, making use of the current `ttl` option across column families?  Or something else? 

I was remembering the gist of this wiki page but not the specifics: https://github.com/facebook/rocksdb/wiki/Time-to-Live  Our concern is simply triggering compaction when the files are old but putting the onus of removal decisions entirely on our user-provided compaction filters.  From reading that wiki it looks like we can set the `ttl` option on the column family but not call `DBWithTTL::Open` to accomplish our goals.

Thanks for the info as always!

-Ben

Ben Clay

unread,
Mar 16, 2019, 1:15:16 PM3/16/19
to Sagar Vemuri, rocksdb, Donghua Liu, manhattan-rocksdb
Following up here on the implementation of `ColumnFamilyOptions.ttl`: from this method I see that L_max is not being chosen for TTL-based compaction, which aligns with the description on the wiki:

This also has the (good) side-effect of all the data in the non-bottommost level being newer than ttl, and all data in the bottommost level older than ttl.

I can understand why L_max is skipped if the goal is simply to reduce space bloat due to stale data.  However, are there other reasons why L_max is skipped? I see that we are marking (potentially) L_max files for compaction as part of  `bottommost_files_` management, but my familiarity with level compaction internals is limited.

-Ben

Ben Clay

unread,
Mar 21, 2019, 1:27:20 PM3/21/19
to Sagar Vemuri, rocksdb, Donghua Liu, manhattan-rocksdb
Ping on this - would love some clarity on why L_max files are being skipped for TTL triggering.

-Ben

Ben Clay

unread,
Apr 4, 2019, 10:33:34 AM4/4/19
to Sagar Vemuri, rocksdb, Donghua Liu, manhattan-rocksdb
To close the loop on this, I spoke offline with Sagar and he told me:

Yes, Lmax was intentionally avoided from TTL compaction originally. But due to some new use-cases at our end in the last month or so, I have been working on adding Lmax too to be included now (behind an option). Keep an eye out for my PR in the next few days. 

-Ben 
Reply all
Reply to author
Forward
0 new messages