-
Notifications
You must be signed in to change notification settings - Fork 4.1k
bulk: range cache miss during index backfill causes sustained slowdown #84290
Description
During an index creation on a 6TB table in a cluster with 15 nodes (48 vCPU per instance), I noticed a surprising slowdown. After running for about 12 hours, the index creation slowed to a trickle (~1% of peak ingestion throughput). About an hour later, it sped up to about 10% of peak ingestion throughput.
The distsender.rpc.addsstable.sent metric revealed an interesting pattern. Most nodes had stopped sending AddSSTable requests entirely — they were finished. However, much earlier in the backfill, two of the nodes had fallen off dramatically in their rate of sending AddSSTable requests.
Interestingly, at different times, these two nodes eventually did pick up the rate of sending and quickly finished.
I was able to grab logs during the slow period from each of these nodes: cockroach.log
I was also able to grab logspy with high verbosity on the sst_batcher.go file: full_logspy.txt
We see an interesting pattern here. During the slowdown, the index backfiller appears to be in a linear regime where it repeatedly hits SSTable cannot be added spanning range bounds lines. These are meant to be rare.
In one (of dozens, I got lucky) of my logspy attempts, I was able to catch a pair of log lines that I think hints at the problem:
I220710 09:25:04.277698 12929447 kv/bulk/sst_batcher.go:327 ⋮ [n2,f‹57784e93›,job=0,indexBackfillerProcessor=120] 6954 ‹trade backfill› no cached range desc available to determine sst flush key
I220710 09:25:04.377969 12929447 kv/bulk/sst_batcher.go:407 ⋮ [n2,f‹57784e93›,job=0,indexBackfillerProcessor=120] 7031 ‹trade backfill› flushing 16 MiB SST due to size > 16 MiB
It seems that at some point, the SSTBatcher checked to see whether it should perform a rangeFlush:
cockroach/pkg/kv/bulk/sst_batcher.go
Lines 322 to 328 in 4566e92
| r := b.rc.GetCached(ctx, k, false /* inverted */) | |
| if r != nil { | |
| k := r.Desc().EndKey.AsRawKey() | |
| b.flushKey = k | |
| log.VEventf(ctx, 3, "%s building sstable that will flush before %v", b.name, k) | |
| } else { | |
| log.VEventf(ctx, 2, "%s no cached range desc available to determine sst flush key", b.name) |
When checking, it did not find a cached range descriptor for the first key in a batch. As a result, it decided not to perform a range flush for the current batch. Instead, it waited until the batch was 16 MiB in size before performing a size flush.
This did not go well. At the time of this size flush, the index keyspace had 11971 ranges. As a result, the 16 MiB buffer was split into chunks of about 5 KiB each (according to logspy). We then sent these small AddSSTables one-by-one. Each took about 50ms, so we sent about 20 AddSSTable's per second. This meant that we spent about 160 seconds to flush the 16 MiB buffer.
I think the takeaway here is that a RangeCache miss can undermine #79967, which becomes increasingly important with large data sizes. Perhaps we should replace the call to RangeCache.GetCached with one to RangeCache.Lookup so that we'll perform a RangeCache lookup on a cache misses. We're going to send to the range anyway, so a cache lookup now vs. later shouldn't make a difference.
One thing I don't understand is why we kept hitting range cache misses once in this regime. Perhaps that hints at something else going wrong as well. Or perhaps there was a gap in our range cache that kept causing us to skip range flushes but was also never filled during a size flush.
cc. @dt
Jira issue: CRDB-17573


