Open nicktrav opened 1 year ago
As discussed on slack, we should only rotate the mutable memtable if it is non-empty: https://github.com/cockroachdb/pebble/blob/master/ingest.go#L849. If it is empty, then the ingestion cannot overlap with it, and we should be able to place the ingested flushable beneath the current mutable memtable.
I think too many memtables, even if empty, can cause write stalls: https://github.com/cockroachdb/pebble/blob/master/db.go#L2013.
@bananabrick I agree, we should only rotate the mutable memtable if it's non-empty. I imagine we'll need to avoid calling makeRoomForWrite
here:
https://github.com/cockroachdb/pebble/blob/2be11668575156bb3ba15c25592f0f4614a420e9/ingest.go#L855
I think this is a little more complex than I hoped. If we don't rotate the mutable memtable, then the ingested flushable, which will have a log number greater than the mutable memtable, will be placed before the mutable memtable in the flushable queue. This breaks the flush invariant that memtables which are flushed should have increasing log numbers.
Can we handle this here: https://github.com/cockroachdb/pebble/blob/c24246ffc0123cb66aac8f7199400be4da351fd4/ingest.go#L759-L782
and if currMem.empty()
, do not recycleWAL()
and instead perform surgery over the d.mu.mem.queue
to place the ingested flushable followed by the current memtable? That way the ingested flushable adopts the current mutable memtable's log number and the current mutable memtable stays the mutable memtable.
My understanding is that normally, with a non-empty mutable memtable M with log number L1 we:
When M
is empty, we could instead:
M
from the queue, and append it after the flushable ingest batch, this time associated with WAL L2. We might need to take care to update the memTable
to have an appropriate logSeqNum
value (ratcheted to include the flushable ingest's seqnum allocations).One small advantage of the above is that avoids a single unnecessary WAL allocation (and corresponding fsync of the data directory).
One small advantage of the above is that avoids a single unnecessary WAL allocation (and corresponding fsync of the data directory).
I was opting for the delay memtable memory allocation solution instead. That way empty memtables do not have to count towards write stalls. This problem also exists for flushable batches, which will also be solved.
We might need to take care to update the memTable to have an appropriate logSeqNum value.
The problem with this is that the logSeqNum can be read concurrently. Technically it doesn't matter because the memtable is empty, but it is unusual. It's also not as simple as updating an atomic variable, because we need to run a few instructions atomically(update logSeqNum, update logNum, swap positions in the flushable queue). On top of that, this approach also makes the memtable queue not be in sync with the the queue of memtables in an older read state. I can't think of why it would need to be, but I was running into a test failure related to refcounting which I didn't dig into, which might be related.
What do you think?
That way empty memtables do not have to count towards write stalls. This problem also exists for flushable batches, which will also be solved.
Isn't this true in both scenarios? Only the mutable memtable may be empty. Flushable batches are by definition non-empty (they're at least 1/2 the memtable size).
It's also not as simple as updating an atomic variable, because we need to run a few instructions atomically(update logSeqNum, update logNum, swap positions in the flushable queue).
I think the places where these need to be done atomically together all use db.mu
for atomicity.
The problem with this is that the logSeqNum can be read concurrently.
Is this when constructing an iterator? Are there any other concurrent readers? I'm now wondering if there's even any requirement to update the memtable's logSeqNum? Its primary use is to determine whether or not the memtable may contain visible keys, which the original logSeqNum
would still do a reasonable job at. If need be, I think we could always add a separate initLogSeqNum
that's constant for the lifetime of a memtable and is the value read concurrently by iterator construction.
I'm not sure which approach is cleaner in practice, just want to consider both.
Isn't this true in both scenarios? Only the mutable memtable may be empty. Flushable batches are by definition non-empty (they're at least 1/2 the memtable size).
Yea, empty memtables can be generated even when flushable batches are created, because we rotate the mutable memtable. I think the condition for creating flushable batches is that the potential size of the flushable batches in the memtable > d.largeBatchThreshold
which is set on Open
. So, flushable batches being created can also lead to empty memtables.
I think the places where these need to be done atomically together all use db.mu for atomicity.
I was worried about the case where we perform a read on logSeqNum
before all the fields are updated atomically. I believe we don't hold DB.mu
when performing these reads. It doesn't really matter, because the memtable is empty anyway. But I wanted to avoid subtle edge cases we have to reason about to ensure correctness.
Is this when constructing an iterator? Are there any other concurrent readers? Yea, it can be read during compaction picking, but only while holding
DB.mu
throughDB.getEarliestUnflushedSeqNumLocked
.If need be, I think we could always add a separate initLogSeqNum that's constant for the lifetime of a memtable and is the value read concurrently by iterator construction.
This makes sense to me. We don't have to worry about the DB.getEarliestUnflushedSeqNumLocked
because those reads only happen with the DB.mu
.
I think I'll stick to the delayed memory allocation solution because it applies to both flushable batches and ingestions.
I just realized that the error in the issue isn't due to flushable ingestions, and we might not have to fix this.
Consider the example in the issue:
[non-empty memtable, ingested table A, ingested table B, empty-memtable]
The empty memtable is actually the mutable memtable. The test(TestMetrics) forces a flush, which rotates the mutable memtable even if it is empty and causes the empty memtable flush error.
But https://github.com/cockroachdb/pebble/blame/master/ingest.go#L850 suggests that if we allow a Options.MemTableStopWritesThreshold
of 3, then we'll only allow flushable ingestions if there's either one or two entries in the flushable queue.
If there's one entry in the flushable queue, and a flushable ingestion is happening, then the one memtable in the queue must be non-empty.
So, the only case where a flushable ingestion could lead to an empty memtable in the queue, which then contributes to the write stall is something like:
[memtable with key a, empty memtable, ingested flushable with key a, empty mutable memtable]
But this has to be exceedingly rare. The first memtable in the queue must be full, for a new empty memtable to be created. And then the flushable ingestion must occur before any write to the new empty memtable.
I believe we set MemTableStopWritesThreshold
to 4 in cockroach. The case that I'm concerned about is two flusable ingests in quick succession, leaving an empty memtable in between.
The case that I'm concerned about is two flusable ingests in quick succession, leaving an empty memtable in between.
This still might not be a problem. Note that if there's two flushable ingestions, then the flushable queue has two entries(the flushable ingestions) which don't contribute to write stalls. So, if the MemtableStopWritesThreshold
is 4, then the write stall limit would be stall_limit := 4 * opts.MemtableSize
. But after two successive flushable ingests, the queue would look like [memtable, flushable ingest, empty memtable, flushable ingest, empty memtable]
. So, there's three memtables, which have allocated at most 3 * opts.MemtableSize
, which is less than stall_limit
. On top of that, there's an entire empty memtable which can receive writes.
In general, I think there can be at most floor(MemtableStopWritesThreshold/2) flushable ingestions in the flushable queue, because each flushable ingestion also leads to a new mutable memtable. So, half the entries in the queue, won't contribute to writes stalls.
This still might not be a problem.
If I'm understanding you correctly, by not be a problem, you mean won't immediately cause a memtable stall?
I think I agree that we shouldn't do much here to prevent memtable stalls. Practically speaking, the problem of memtable stalls possibly becoming more likely with flushable ingests wouldn't be moved much by solving for the case of an empty mutable memtable. It's more likely we'll see mutable memtables with just a few keys. We'd need #2413 or the like to improve the situation there.
I do think we need to fix the fact that an empty memtable be queued and flushed, resulting in a background error:
2023/03/08 14:44:53 [JOB 21] flush error: pebble: empty table
Are we sure the above is harmless? Is the empty memtable removed from the queue, or does this error prevent flushing subsequent memtables? Regardless, to avoid confusion, we should avoid logging an error in this situation.
If I'm understanding you correctly, by not be a problem, you mean won't immediately cause a memtable stall?
Yes.
I do think we need to fix the fact that an empty memtable be queued and flushed, resulting in a background error
That error is only set in the FlushInfo
in the flush1
function. Even if that error is set in the FlushInfo
the function, the flush1
function can still return a nil
error. I believe it's used for logging more than anything else.
Is the empty memtable removed from the queue, or does this error prevent flushing subsequent memtables?
No, it doesn't prevent future memtable flushes. Even before flushable ingests, we could have empty memtables flushed. I agree that logging it as an actual error is odd.
I wonder how much, if any, this double WAL latency increases p99 latency of commits. WAL rotations are not as cheap as we'd like them to be, suffering multiple fsyncs (#2540).
When a table is ingested as a flushable, new flushables are added to the flushable queue for each table in the ingestion that overlaps. The memtable is then rotated, adding a new memtable to the end of the queue.
Consider a single non-empty memtable, followed by two tables ingested as flushable. After the ingested tables are added, the memtable queue resembles:
The ingestion triggers a flush, which first flushes the non-empty memtable to L0, followed by ingestions of tables A and B into their appropriate levels. The empty memtable is then flushed, which emits the following error to the event logger:
While this error is harmless, it points to a performance issue where we unnecessarily flush the empty memtable.
See #2387 for a minimal reproducer.
Jira issue: PEBBLE-219