-
Notifications
You must be signed in to change notification settings - Fork 556
perf: support user-defined "guards" for sstable boundaries #517
Description
When flushing and compacting sstables, Pebble needs to decide how to break up the resulting output into sstables. We currently use two signals: sstable size, and the boundaries of sstables in the "grandparent" level (i.e. the level below the output level). These signals are reasonable for randomly distributed writes, but can cause problems for skewed writes. Consider the following LSM structure:
L0 a------------------------z
L1 ab de gh jk mn pq st vw yz
L0 contains a single sstable spanning [a,z]. L1 contains 9 sstables. A compaction from L0 to L1 will need to include all of the L1 sstables. It is possible that L0 only contains 2 keys: a and z. This problem is easy to construct if a and z are written to the memtable near each other in time and then flushed as flushing currently always produces a single sstable.
This situation appears to arise in practice within CRDB due to the presence of "local" vs "global" keys. Most operations involve only "global" keys. When a "local" key is operated on, it will end up generating an L0 sstable that spans a large swath of the global key space.
The idea behind "guards" is to allow the user (i.e. CockroachDB) some control over sstable boundaries. CockroachDB would define guards at the boundary between the local and global keyspace. It may also define guards at the boundary between SQL tables. Such guards would ensure that we're segregating sstables along the guard boundaries so that an L0 sstable can't cover a huge fraction of the key space.
Note that "guards" would almost certainly be specified via a callback and not upfront. That is, we'd want to allow the user to specify a callback which can return true if there should be an sstable boundary between two user keys.
Jira issue: PEBBLE-178