Closed benesch closed 6 years ago
@petermattis something I've generally wanted to be able to do multiple times was to actually peek into RocksDB while we're in that state. What's the appropriate tool to do so? gdb
et al? Assuming the DBIterSeek is at the bottom of the stack, it's probably waiting on some mutex.
edit: there's no way it's at the bottom of the stack. Rather what I'm trying to say is "what's the full stack there and what's at the bottom of it".
@petermattis something I've generally wanted to be able to do multiple times was to actually peek into RocksDB while we're in that state. What's the appropriate tool to do so? gdb et al? Assuming the DBIterSeek is at the bottom of the stack, it's probably waiting on some mutex.
edit: there's no way it's at the bottom of the stack. Rather what I'm trying to say is "what's the full stack there and what's at the bottom of it".
A CPU profile may show where these goroutines are blocked on the C++ side. It's unfortunate that the you can't extract the stack for a specific goroutine for a CPU profile. Alternately, it would be nice of the normal goroutine dump showed the C++ stacks as well, though I understand the technical reason they currently don't (Go would have to send a SIGPROF signal to all threads, while the current runtime.Stacks
output is produced by looping over the goroutines).
I haven't tried gdb
.
We could use github.com/google/pprof/profile
to parse a CPU profile and look for a sample that contains _Cfunc_DBIterSeek
. Somewhat onerous. Off topic: we should investigate using profiler labels (https://rakyll.org/profiler-labels/) at some point.
By the way, there was code added originally with the consistency queue mechanism to avoid scheduling for newly created ranges, such as after splits and rebalancings. Objections kept that feature out, but perhaps it should be revived. We are too aggressive in scheduling these checks.
@spencerkimball on the other hand, that would've potentially hidden this issue here. Your suggestion fits into https://github.com/cockroachdb/cockroach/issues/26151 in spirit, though.
Ok, I have a theory!
Just to confirm your observation of the cons checker as the root cause, could you go through a few runs with the consistency checker turned off?
Turning off the consistency checker helps quite a bit, but doesn't fix the problem. Other goroutines still get wedged and the CPU is still pegged.
So I turned off the scanner and the problem went poof. (I didn't try running a schema change, but I was able to get the cluster up and running again.) This makes quite a bit of sense: the scanner's pace interval is essentially zero because there are more replicas on each node than there are seconds in the default scan interval (10m, I think?). Every replica scanned triggers a few calls to getQueueLastProcessed/setQueueLastProcessed, each of which create an iterator. When iterator creation is slowed down by range tombstones, that can be a few hundred milliseconds of CPU time right there.
I stopped seeing stuck goroutines after that... except for one.
goroutine 307 [syscall, 4 minutes]:
github.com/cockroachdb/cockroach/pkg/storage/engine._Cfunc_DBIterSeek(0x7fd83ba1ab10, 0xc427fb85a7, 0x7, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
_cgo_gotypes.go:666 +0x74
github.com/cockroachdb/cockroach/pkg/storage/engine.(*rocksDBIterator).Seek.func2(0x7fd83ba1ab10, 0xc427fb85a7, 0x7, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/engine/rocksdb.go:1951 +0xba
github.com/cockroachdb/cockroach/pkg/storage/engine.(*rocksDBIterator).Seek(0xc42f60efc0, 0xc427fb85a7, 0x7, 0x13, 0x0, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/engine/rocksdb.go:1951 +0x16b
github.com/cockroachdb/cockroach/pkg/storage/compactor.(*Compactor).fetchSuggestions.func1(0xc427fb85a0, 0x19, 0x1a, 0x0, 0x0, 0xc4334611c0, 0xf, 0xf, 0x0, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/compactor/compactor.go:309 +0x284
github.com/cockroachdb/cockroach/pkg/storage/engine.dbIterate(0x7fd83bb181b0, 0x2b0c0a0, 0xc420678340, 0xc4202b510a, 0x6, 0x6, 0x0, 0x0, 0xc4202b5110, 0x6, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/engine/rocksdb.go:2448 +0x320
github.com/cockroachdb/cockroach/pkg/storage/engine.(*RocksDB).Iterate(0xc420678340, 0xc4202b510a, 0x6, 0x6, 0x0, 0x0, 0xc4202b5110, 0x6, 0x6, 0x0, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/engine/rocksdb.go:798 +0xbf
github.com/cockroachdb/cockroach/pkg/storage/compactor.(*Compactor).fetchSuggestions(0xc4209ae1e0, 0x2afa760, 0xc427bc50b0, 0xc42031a1e0, 0x26c7338, 0x1d, 0x2afa760, 0x0, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/compactor/compactor.go:295 +0x306
github.com/cockroachdb/cockroach/pkg/storage/compactor.(*Compactor).processSuggestions(0xc4209ae1e0, 0x2afa760, 0xc427bc50b0, 0x0, 0x0, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/compactor/compactor.go:207 +0x11d
github.com/cockroachdb/cockroach/pkg/storage/compactor.(*Compactor).Start.func1(0x2afa760, 0xc429ae75c0)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/compactor/compactor.go:147 +0x1d9
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker.func1(0xc42b0b8c00, 0xc4208085a0, 0xc42bc71c60)
/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:193 +0xe9
created by github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker
/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:186 +0xad
The code I added to the compactor is getting bogged down by range tombstones. As you might imagine, this leads to even more range tombstones. 🤦♂️
On the bright side, because there was only one thread in C++, it was easy to capture a stacktrace:
#0 snappy::SnappyDecompressor::DecompressAllTags<snappy::SnappyArrayWriter> () at /go/src/github.com/cockroachdb/cockroach/c-deps/snappy/snappy.cc:747
#1 InternalUncompressAllTags<snappy::SnappyArrayWriter> () at /go/src/github.com/cockroachdb/cockroach/c-deps/snappy/snappy.cc:844
#2 InternalUncompress<snappy::SnappyArrayWriter> () at /go/src/github.com/cockroachdb/cockroach/c-deps/snappy/snappy.cc:831
#3 0x0000000002011273 in snappy::RawUncompress () at /go/src/github.com/cockroachdb/cockroach/c-deps/snappy/snappy.cc:1197
#4 snappy::RawUncompress () at /go/src/github.com/cockroachdb/cockroach/c-deps/snappy/snappy.cc:1192
#5 0x0000000001ec4d34 in rocksdb::Snappy_Uncompress () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/util/compression.h:190
#6 rocksdb::UncompressBlockContentsForCompressionType () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/format.cc:287
#7 0x0000000001ec4ee9 in rocksdb::UncompressBlockContents () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/format.cc:393
#8 0x0000000001ec0406 in rocksdb::BlockFetcher::ReadBlockContents () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/block_fetcher.cc:221
#9 0x0000000001ead52b in ReadBlockFromFile () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/block_based_table_reader.cc:87
#10 0x0000000001eb006a in rocksdb::BlockBasedTable::MaybeLoadDataBlockToCache ()
at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/block_based_table_reader.cc:1612
#11 0x0000000001eb02fc in rocksdb::BlockBasedTable::NewDataBlockIterator ()
at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/block_based_table_reader.cc:1489
#12 0x0000000001eb0b73 in rocksdb::BlockBasedTableIterator::InitDataBlock ()
at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/block_based_table_reader.cc:1907
#13 0x0000000001eb0d0a in rocksdb::BlockBasedTableIterator::FindKeyForward ()
at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/block_based_table_reader.cc:1928
#14 0x0000000001e41c09 in rocksdb::IteratorWrapper::Next () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/iterator_wrapper.h:61
#15 Next () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/version_set.cc:586
#16 0x0000000001ed07c2 in rocksdb::IteratorWrapper::Next () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/iterator_wrapper.h:61
#17 rocksdb::MergingIterator::Next () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/table/merging_iterator.cc:170
#18 0x0000000001df058c in rocksdb::DBIter::FindNextUserEntryInternal () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/db_iter.cc:591
#19 0x0000000001df1648 in rocksdb::DBIter::FindNextUserEntry () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/db_iter.cc:397
#20 rocksdb::DBIter::Seek () at /go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/db_iter.cc:1180
#21 0x0000000001d2267a in DBIterSeek () at /go/src/github.com/cockroachdb/cockroach/c-deps/libroach/db.cc:549
#22 0x0000000001c88313 in _cgo_cf94bd8886ad_Cfunc_DBIterSeek (v=0xc42f688f48) at cgo-gcc-prolog:572
#23 0x00000000006f47e0 in runtime.asmcgocall () at /usr/local/go/src/runtime/asm_amd64.s:688
#24 0x00000000006f2111 in runtime.exitsyscallfast.func1 () at /usr/local/go/src/runtime/proc.go:3034
#25 0x000000c42bc7f680 in ?? ()
#26 0x00000000006ca580 in ?? () at /usr/local/go/src/runtime/proc.go:1087
#27 0x00007fd800000000 in ?? ()
#28 0x0100000000000150 in ?? ()
I watched one of these seeks climb up to 15m in C++, alternating between adding range tombstones and reading new blocks. Then it dawned on me. This sounds a lot like https://www.cockroachlabs.com/blog/adventures-performance-debugging/, doesn't it? Just on a scale of minutes, instead of seconds.
So here's the theory: if you end up with a bunch of contiguous range deletion tombstones over gigabytes of data, seeking to a key within the beginning of that region is going to be a bad time. To complete the seek RocksDB has to load gigabytes of data into memory and then run every single key through ShouldDelete. Oops. And if you run multiple of these seeks at once... they'll thrash the block cache into oblivion.
@petermattis I'd love to get your help in verifying this theory. I know that our iterators support some integration with RocksDB's perf contexts to get stats about a slow scan.
Seems like the solution is to install an iterate_upper_bound
on basically every single iterator. In most cases we know exactly the bounds we'd like to iterate; we just don't tell RocksDB about them until its too late.
Oh, before I forget, to get a cluster with thousands of range deletion tombstones back online, you need a diff like this to avoid paying the cost of creating 10 new iterators for every replica
diff --git a/pkg/storage/engine/iter.go b/pkg/storage/engine/iter.go
new file mode 100644
index 000000000..c82815933
--- /dev/null
+++ b/pkg/storage/engine/iter.go
@@ -0,0 +1,49 @@
+// Copyright 2018 The Cockroach Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+// implied. See the License for the specific language governing
+// permissions and limitations under the License.
+
+package engine
+
+import (
+ "context"
+
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+)
+
+type IteratorCachingEngine struct {
+ Engine
+ Ctx context.Context // XXX
+ iter *CachedIterator
+}
+
+func (ice *IteratorCachingEngine) NewIterator(opts IterOptions) Iterator {
+ if ice.iter == nil {
+ ice.iter = &CachedIterator{ice.Engine.NewIterator(opts)}
+ }
+ return ice.iter
+}
+
+func (ice *IteratorCachingEngine) Close() {
+ if ice.iter != nil {
+ ice.iter.Close()
+ }
+}
+
+type CachedIterator struct {
+ Iterator
+}
+
+func (ci *CachedIterator) Close() {
+ // noop
+}
diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go
index 34e6d74..b65b800 100644
--- a/pkg/storage/replica.go
+++ b/pkg/storage/replica.go
@@ -655,24 +655,24 @@ func newReplica(rangeID roachpb.RangeID, store *Store) *Replica {
// replicaID should be 0 and the replicaID will be discovered from the
// descriptor.
func NewReplica(
- desc *roachpb.RangeDescriptor, store *Store, replicaID roachpb.ReplicaID,
+ desc *roachpb.RangeDescriptor, store *Store, replicaID roachpb.ReplicaID, e engine.Engine,
) (*Replica, error) {
r := newReplica(desc.RangeID, store)
- return r, r.init(desc, store.Clock(), replicaID)
+ return r, r.init(desc, store.Clock(), replicaID, e)
}
func (r *Replica) init(
- desc *roachpb.RangeDescriptor, clock *hlc.Clock, replicaID roachpb.ReplicaID,
+ desc *roachpb.RangeDescriptor, clock *hlc.Clock, replicaID roachpb.ReplicaID, e engine.Engine,
) error {
r.raftMu.Lock()
defer r.raftMu.Unlock()
r.mu.Lock()
defer r.mu.Unlock()
- return r.initRaftMuLockedReplicaMuLocked(desc, clock, replicaID)
+ return r.initRaftMuLockedReplicaMuLocked(desc, clock, replicaID, e)
}
func (r *Replica) initRaftMuLockedReplicaMuLocked(
- desc *roachpb.RangeDescriptor, clock *hlc.Clock, replicaID roachpb.ReplicaID,
+ desc *roachpb.RangeDescriptor, clock *hlc.Clock, replicaID roachpb.ReplicaID, e engine.Engine,
) error {
ctx := r.AnnotateCtx(context.TODO())
if r.mu.state.Desc != nil && r.isInitializedRLocked() {
@@ -696,7 +696,7 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked(
var err error
- if r.mu.state, err = r.mu.stateLoader.Load(ctx, r.store.Engine(), desc); err != nil {
+ if r.mu.state, err = r.mu.stateLoader.Load(ctx, e, desc); err != nil {
return err
}
@@ -720,13 +720,13 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked(
r.rangeStr.store(0, r.mu.state.Desc)
- r.mu.lastIndex, err = r.mu.stateLoader.LoadLastIndex(ctx, r.store.Engine())
+ r.mu.lastIndex, err = r.mu.stateLoader.LoadLastIndex(ctx, e)
if err != nil {
return err
}
r.mu.lastTerm = invalidLastTerm
- pErr, err := r.mu.stateLoader.LoadReplicaDestroyedError(ctx, r.store.Engine())
+ pErr, err := r.mu.stateLoader.LoadReplicaDestroyedError(ctx, e)
if err != nil {
return err
}
@@ -751,7 +751,7 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked(
return err
}
- r.assertStateLocked(ctx, r.store.Engine())
+ r.assertStateLocked(ctx, e)
return nil
}
diff --git a/pkg/storage/store.go b/pkg/storage/store.go
index 20f6101..705aab1 100644
--- a/pkg/storage/store.go
+++ b/pkg/storage/store.go
@@ -963,9 +963,7 @@ func NewStore(cfg StoreConfig, eng engine.Engine, nodeDesc *roachpb.NodeDescript
if cfg.TestingKnobs.DisableRaftSnapshotQueue {
s.setRaftSnapshotQueueActive(false)
}
- if cfg.TestingKnobs.DisableScanner {
- s.setScannerActive(false)
- }
+ s.setScannerActive(false)
return s
}
@@ -1359,13 +1357,17 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error {
// concurrently. Note that while we can perform this initialization
// concurrently, all of the initialization must be performed before we start
// listening for Raft messages and starting the process Raft loop.
- err = IterateRangeDescriptors(ctx, s.engine,
+ ice := &engine.IteratorCachingEngine{Ctx: ctx, Engine: s.engine}
+ defer ice.Close()
+ err = IterateRangeDescriptors(ctx, ice,
func(desc roachpb.RangeDescriptor) (bool, error) {
if !desc.IsInitialized() {
return false, errors.Errorf("found uninitialized RangeDescriptor: %+v", desc)
}
- rep, err := NewReplica(&desc, s, 0)
+ log.Eventf(ctx, "initializing %s", desc.StartKey)
+
+ rep, err := NewReplica(&desc, s, 0, ice)
if err != nil {
return false, err
}
@@ -2160,7 +2162,7 @@ func splitPostApply(
{
rightRng.mu.Lock()
// Already holding raftMu, see above.
- err := rightRng.initRaftMuLockedReplicaMuLocked(&split.RightDesc, r.store.Clock(), 0)
+ err := rightRng.initRaftMuLockedReplicaMuLocked(&split.RightDesc, r.store.Clock(), 0, r.store.Engine())
rightRng.mu.Unlock()
if err != nil {
log.Fatal(ctx, err)
@@ -3960,7 +3962,7 @@ func (s *Store) tryGetOrCreateReplica(
// TODO(bdarnell): other fields are unknown; need to populate them from
// snapshot.
}
- if err := repl.initRaftMuLockedReplicaMuLocked(desc, s.Clock(), replicaID); err != nil {
+ if err := repl.initRaftMuLockedReplicaMuLocked(desc, s.Clock(), replicaID, s.engine); err != nil {
// Mark the replica as destroyed and remove it from the replicas maps to
// ensure nobody tries to use it
repl.mu.destroyStatus.Set(errors.Wrapf(err, "%s: failed to initialize", repl), destroyReasonRemoved)
and also probably facebook/rocksdb#3800.
@petermattis I'd love to get your help in verifying this theory. I know that our iterators support some integration with RocksDB's perf contexts to get stats about a slow scan.
@tschottdorf Added support for extracting perf context stats during iterator operations. See libroach:ScopedStats
. Currently only a single stat is retrieved, but adding more should be straightforward.
Ok, the results are in. Here's the perf context after a 7m seek:
user_key_comparison_count = 6119910, block_cache_hit_count = 6, block_read_count = 1629046, block_read_byte = 49444945773, block_read_time = 289346125311, block_checksum_time = 7171937629, block_decompress_time = 123770617916, get_read_bytes = 0, multiget_read_bytes = 0, iter_read_bytes = 10266, internal_key_skipped_count = 258, internal_delete_skipped_count = 6117839, internal_recent_skipped_count = 0, internal_merge_count = 0, write_wal_time = 0, get_snapshot_time = 0, get_from_memtable_time = 0, get_from_memtable_count = 0, get_post_process_time = 0, get_from_output_files_time = 0, seek_on_memtable_time = 1354, seek_on_memtable_count = 1, next_on_memtable_count = 0, prev_on_memtable_count = 0, seek_child_seek_time = 35141338, seek_child_seek_count = 6, seek_min_heap_time = 716, seek_internal_seek_time = 35143112, find_next_user_entry_time = 437018704126, write_pre_and_post_process_time = 0, write_memtable_time = 0, db_mutex_lock_nanos = 0, db_condition_wait_nanos = 0, merge_operator_time_nanos = 0, write_delay_time = 0, read_index_block_nanos = 0, read_filter_block_nanos = 0, new_table_block_iter_nanos = 427002309018, new_table_iterator_nanos = 1113030843, block_seek_nanos = 2002893, find_table_nanos = 1019574640, bloom_memtable_hit_count = 0, bloom_memtable_miss_count = 0, bloom_sst_hit_count = 0, bloom_sst_miss_count = 0, key_lock_wait_time = 0, key_lock_wait_count = 0, env_new_sequential_file_nanos = 0, env_new_random_access_file_nanos = 0, env_new_writable_file_nanos = 0, env_reuse_writable_file_nanos = 0, env_new_random_rw_file_nanos = 0, env_new_directory_nanos = 0, env_file_exists_nanos = 0, env_get_children_nanos = 0, env_get_children_file_attributes_nanos = 0, env_delete_file_nanos = 0, env_create_dir_nanos = 0, env_create_dir_if_missing_nanos = 0, env_delete_dir_nanos = 0, env_get_file_size_nanos = 0, env_get_file_modification_time_nanos = 0, env_rename_file_nanos = 0, env_link_file_nanos = 0, env_lock_file_nanos = 0, env_unlock_file_nanos = 0, env_new_logger_nanos = 0,
I'd say my theory is confirmed. That seek read 50GiB of data.
Wow!
@benesch I've been experimenting with your two PRs (#26449 and #26488) patched on top of #26526. Initial experiments still resulted in badness, with 2 nodes pegging the CPU in RangeDelAggregator::AddTombstones
. I haven't dived too far into debugging why #26526 isn't helping more, but I did notice that the rate limiting of ClearRange
operations from the schema changer isn't as effective as we'd like. I tried reduce the batch size from 50 to 10, but that didn't help. What did help was to insert a 1s sleep in between every batch. With this throttling in place, the compaction queue seems to be keeping up:
I'm still seeing 1 node pegging the CPU in RangeDelAggregator::AddTombstones
. Still investigating.
Hrmm, the problematic node's compactor goroutine is stuck in DBCompactRange
:
goroutine 856 [syscall, 21 minutes]:
github.com/cockroachdb/cockroach/pkg/storage/engine._Cfunc_DBCompactRange(0x7fb4a7d182a0, 0xc4296d2bc7, 0x8, 0xc4296d2ff1, 0x8, 0xc429f9dc00, 0x0, 0x0)
_cgo_gotypes.go:336 +0x50
github.com/cockroachdb/cockroach/pkg/storage/engine.(*RocksDB).CompactRange.func1(0x7fb4a7d182a0, 0xc4296d2bc7, 0x8, 0xc4296d2ff1, 0x8, 0x0, 0x0, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/engine/rocksdb.go:889 +0x82
github.com/cockroachdb/cockroach/pkg/storage/engine.(*RocksDB).CompactRange(0xc4201add40, 0xc4296d2bc7, 0x8, 0x15, 0xc4296d2ff1, 0x8, 0xb, 0x0, 0x1b, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/engine/rocksdb.go:889 +0xae
github.com/cockroachdb/cockroach/pkg/storage/compactor.(*Compactor).processCompaction(0xc4204ee240, 0x2b4c360, 0xc428cae180, 0xc4296d2bc7, 0x8, 0x15, 0xc4296d2ff1, 0x8, 0xb, 0x18e13ca8, ...)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/compactor/compactor.go:372 +0x8d0
github.com/cockroachdb/cockroach/pkg/storage/compactor.(*Compactor).processSuggestions(0xc4204ee240, 0x2b4c360, 0xc428cae180, 0x0, 0x0, 0x0)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/compactor/compactor.go:263 +0x77e
github.com/cockroachdb/cockroach/pkg/storage/compactor.(*Compactor).Start.func1(0x2b4c360, 0xc428aa7e30)
/go/src/github.com/cockroachdb/cockroach/pkg/storage/compactor/compactor.go:147 +0x1d9
github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker.func1(0xc428017ab0, 0xc4207341b0, 0xc42a310600)
/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:193 +0xe9
created by github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunWorker
/go/src/github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:186 +0xad
Yeah, I've been running with --vmodule=schema_changer=2,compactor=2,rocksdb=3
in order to see where the compactor's getting stuck.
One other patch you might want to try running with: facebook/rocksdb#3800
Yeah, I was just about to run with rocksdb=3
. Interestingly, in several runs I keep on seeing problems on the same node (n4
, roachprod node 3). The very first compaction it does takes a long time:
I180609 17:02:07.497974 542 storage/compactor/compactor.go:367 [n4,s4,compactor] processing compaction #913-931/1427 (/Table/51/1/37957760-/Table/51/1/38233415/0) for 289 MiB (reasons: size=true used=false avail=false)
I180609 17:09:35.642037 542 storage/compactor/compactor.go:383 [n4,s4,compactor] processed compaction #913-931/1427 (/Table/51/1/37957760-/Table/51/1/38233415/0) for 289 MiB in 7m28.144071063s
Internally, DBCompactRange
only issued 2 compactions (which seems appropriate for compacting 289MiB).
Wow, 7m is so slow. I wonder what's actually going on. The RocksDB logs should be illuminating.
(n4, roachprod node 3)
We've got to start generating these store dumps on clusters started with roachprod start --sequential
.
Something is horked up with the manual compaction. It seems we're compacting to the end of the key space, but I don't see how. The cycle time on debugging this is irritating. Here are some RocksDB compaction logs generated from a single DB::CompactRange
call:
I180609 18:20:00.049641 17 storage/engine/rocksdb.go:93 [rocksdb] [/go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/compaction_job.cc:1497] [default] Compaction start summary: Base version 2 Base level 0, inputs: [10650(270KB)], [10624(8404KB) 10623(8221KB)]
I180609 18:20:00.343483 17 storage/engine/rocksdb.go:93 [rocksdb] [/go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/compaction_job.cc:1497] [default] Compaction start summary: Base version 3 Base level 2, inputs: [10656(8221KB)], [7653(16MB) 7655(3442KB) 7556(8233KB) 7558(6024KB) 7545(8226KB) 7548(8233KB) 9065(3194KB) 9058(5724KB)]
I180609 18:20:01.478244 17 storage/engine/rocksdb.go:93 [rocksdb] [/go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/compaction_job.cc:1497] [default] Compaction start summary: Base version 5 Base level 4, inputs: [10662(2113KB)]
I180609 18:20:01.577256 17 storage/engine/rocksdb.go:93 [rocksdb] [/go/src/github.com/cockroachdb/cockroach/c-deps/rocksdb/db/compaction_job.cc:1497] [default] Compaction start summary: Base version 6 Base level 5, inputs: [10663(2111KB)], [5728(100MB) 5731(100MB) 5733(100MB) 5735(100MB) 5739(100MB) 5741(100MB) 5744(100MB) 5750(100MB) 5752(100MB) 5757(100MB) 5761(100MB) 5764(100MB) 5768(100MB) 5771(100MB) 5777(100MB) 5780(100MB) 5783(100MB) 5786(100MB) 5789(100MB) 5791(100MB) 5794(100MB) 5798(100MB) 5800(100MB) 5803(100MB) 5807(100MB) 5810(100MB) 5813(100MB) 5818(100MB) 5822(100MB) 5823(100MB) 5827(100MB) 5830(100MB) 5834(100MB) 5837(100MB) 5842(100MB) 5845(100MB) 5849(100MB) 5850(100MB) 5854(100MB) 5857(100MB) 5862(100MB) 5864(41MB) 3334(24MB) 3779(18MB) 3784(24MB) 3787(24MB) 3791(1534KB) 3800(24MB) 3812(24MB) 3822(24MB) 3836(6506KB) 3855(20MB) 3874(24MB) 3907(24MB) 3933(24MB) 3968(24MB) 3996(15MB) 4025(24MB) 4054(24MB) 4083(24MB) 4111(24MB) 4139(24MB) 4168(24MB) 4210(14MB) 4246(24MB) 4266(24MB) 4313(24MB) 4339(12MB) 4351(24MB) 4382(19MB) 4406(24MB) 4473(20MB) 4481(24MB) 4497(24MB) 4504(24MB) 4507(24MB) 4511(24MB) 4517(24MB) 4527(24MB) 4536(24MB) 4547(24MB) 4556(17MB) 4565(24MB) 4575(24MB) 4590(24MB) 4599(24MB) 4610(24MB) 4618(7435KB) 4628(24MB) 4639(24MB) 4649(24MB) 4658(10MB) 4669(24MB) 4683(24MB) 4694(16MB) 4704(24MB) 4716(24MB) 4728(24MB) 4735(21MB) 4743(24MB) 4755(24MB) 4766(24MB) 4774(24MB) 3790(24MB) 3795(24MB) 3807(8855KB) 3819(24MB) 3832(24MB) 3851(24MB) 3869(24MB) 3902(24MB) 3927(24MB) 3963(24MB) 3990(24MB) 4019(24MB) 4045(24MB) 4075(24MB) 4106(16MB) 4133(24MB) 4159(24MB) 4203(24MB) 4231(24MB) 4256(24MB) 4287(24MB) 4323(24MB) 4366(24MB) 4417(9607KB) 4394(24MB) 4430(24MB) 4448(24MB) 4476(17MB) 4501(24MB) 4513(24MB) 4525(24MB) 4533(24MB) 4545(24MB) 4555(6683KB) 4567(24MB) 4576(24MB) 4587(24MB) 4597(24MB) 4608(24MB) 4617(24MB) 4627(24MB) 4637(24MB) 4646(933KB) 4656(24MB) 4666(24MB) 4681(24MB) 4692(24MB) 4714(24MB) 4724(24MB) 4733(24MB) 4742(24MB) 4752(24MB) 4764(14MB) 4773(24MB) 4782(24MB) 4790(24MB) 4793(24MB) 4799(24MB) 4804(18MB) 4809(24MB) 4814(24MB) 4819(24MB) 4823(24MB) 4827(24MB) 4831(24MB) 4837(24MB) 4840(24MB) 4845(17MB) 4850(24MB) 4855(24MB) 4859(24MB) 4868(24MB) 4873(24MB) 4877(24MB) 4881(16MB) 4887(24MB) 4891(24MB) 4895(24MB) 4899(24MB) 4904(24MB) 4909(24MB) 4913(24MB) 4916(11MB) 4920(24MB) 4925(24MB) 4929(24MB) 4933(24MB) 4937(24MB) 4941(24MB) 4945(24MB) 4949(20MB) 4955(24MB) 4960(24MB) 4964(24MB) 4967(24MB) 4971(24MB) 4977(24MB) 4981(24MB) 4986(24MB) 4992(24MB)]
Kind of hard to parse, but what is happening is that we're compacting to L0 -> L2, L2 -> L3, L3 -> L4, L4 -> L5 and then L5 -> L6. (The L3 -> L4 compaction isn't represented above). The compactions up until L5 -> L6 are speedy. That last compaction covers way too much of the key space and causes badness.
What did help was to insert a 1s sleep in between every batch. With this throttling in place, the compaction queue seems to be keeping up:
My initial patch had code to send no more than one batch every 5s... except I had a bug and the rate limiting wasn't applied. And yet that experiment was the one successful experiment I've run, so I concluded the rate limiting wasn't necessary. I suspect that one node was just slow enough in processing ClearRange requests—they seemed to be taking several seconds—that there was natural rate limiting. Perhaps your fixes sped up ClearRange requests to the point where rate limiting is actually necessary.
So even with my change to the compaction queue to ignore suggestions spanning live data, a few of the nodes with this store dump have some pending suggestions that they try to act upon:
$ roachprod run benesch-clearrange 'grep "compactor.go.*process" logs/cockroach.log'
benesch-clearrange: grep "compactor.go.*process... 10/10
1:
Process exited with status 1
2: I180609 21:54:38.849945 150 storage/compactor/compactor.go:370 [n8,s8,compactor] processing compaction #49-84/447 (/Table/51/1/1255643-/Table/51/1/2495965/0) for 294 MiB (reasons: size=true used=false avail=false)
I180609 21:54:39.382614 150 storage/compactor/compactor.go:386 [n8,s8,compactor] processed compaction #49-84/447 (/Table/51/1/1255643-/Table/51/1/2495965/0) for 294 MiB in 532.616883ms
I180609 21:54:39.733709 150 storage/compactor/compactor.go:370 [n8,s8,compactor] processing compaction #329-357/447 (/Table/51/1/40766258/0-/Table/51/1/40860289) for 427 MiB (reasons: size=true used=false avail=false)
I180609 21:56:34.559770 150 storage/compactor/compactor.go:386 [n8,s8,compactor] processed compaction #329-357/447 (/Table/51/1/40766258/0-/Table/51/1/40860289) for 427 MiB in 1m54.826041583s
3: I180609 21:55:37.301785 583 storage/compactor/compactor.go:370 [n4,s4,compactor] processing compaction #914-932/1428 (/Table/51/1/37957760-/Table/51/1/38233415/0) for 289 MiB (reasons: size=true used=false avail=false)
I180609 22:04:13.000059 583 storage/compactor/compactor.go:386 [n4,s4,compactor] processed compaction #914-932/1428 (/Table/51/1/37957760-/Table/51/1/38233415/0) for 289 MiB in 8m35.698238964s
I180609 22:04:13.065776 583 storage/compactor/compactor.go:370 [n4,s4,compactor] processing compaction #983-1028/1428 (/Table/51/1/39538412/0-/Table/51/1/39781285) for 362 MiB (reasons: size=true used=false avail=false)
I180609 22:04:13.074273 583 storage/compactor/compactor.go:386 [n4,s4,compactor] processed compaction #983-1028/1428 (/Table/51/1/39538412/0-/Table/51/1/39781285) for 362 MiB in 8.488469ms
I180609 22:04:13.099419 583 storage/compactor/compactor.go:370 [n4,s4,compactor] processing compaction #1037-1066/1428 (/Table/51/1/40770561/0-/Table/51/1/40857017/0) for 398 MiB (reasons: size=true used=false avail=false)
4:
Process exited with status 1
5:
Process exited with status 1
6: I180609 21:55:07.840456 352 storage/compactor/compactor.go:370 [n5,s5,compactor] processing compaction #585-614/1823 (/Table/51/1/19637255-/Table/51/1/19907573/0) for 269 MiB (reasons: size=true used=false avail=false)
I180609 21:56:17.033874 352 storage/compactor/compactor.go:386 [n5,s5,compactor] processed compaction #585-614/1823 (/Table/51/1/19637255-/Table/51/1/19907573/0) for 269 MiB in 1m9.193311445s
7:
Process exited with status 1
8:
Process exited with status 1
9:
Process exited with status 1
10:
Process exited with status 1
Error: Process exited with status 1
It takes n5 1m, n8 2m to process its suggestions, and n4 10m and counting. The problem, as @petermattis discovered, is that even though we're asking for a small range to be compacted, that range is contained within an SST in L5 with a very wide range, spanning over 1k SSTs (!) in L6. Compacting that, unsurprisingly, takes quite a while.
I suspect we're tripping over another problem with range tombstones. I need to do a fuller write up of my understanding of compactions, but the short summary is that RocksDB attempts to align the boundaries of sstables at Ln with those at Ln+1. But somehow it is failing to do so and the result is a super-large compaction (as alluded to above, an L5 -> L6 compaction involving a single file in L5 is overlapping 1247 files in L6).
I've been adding logging to try and understand what RocksDB is doing. Some highlights:
L0 -> L2
/Local/RangeID/1/r/AbortSpan/"911ec3ce-6d25-4713-82e4-2fc6c2951af7" /Table/51/1/46687208
/Table/51/1/23887551/0 /Table/51/1/64697768
L2 -> L3
/Table/51/1/23887551/0 /Table/51/1/64697768/NULL
/Table/51/1/23887551/0 /Table/51/1/27440183/0
/Table/51/1/27440184/0 /Table/51/1/27442206/0
/Table/51/1/27442207/0 /Table/51/1/28781803/0
/Table/51/1/28781804/0 /Table/51/1/28783823/0
/Table/51/1/28783824/0 /Table/51/1/64697768
L3 -> L4 (no-op because no overlapping L4 files)
L4 -> L5
/Table/51/1/28783824/0 /Table/51/1/64697768
/Table/51/1/28783824/0 /Table/51/1/40794462
/Table/51/1/40804278/0 /Table/51/1/64697768
The first line indicates the key range for the input file (and in these compactions there is always a single input file). The indented lines indicate the key ranges for the output files. So we see in the L0->L2 compaction an output file over a fairly large key range. The output file becomes the input for the L2->L3 compaction which generates a few small ranges and then a large range at the end. That last output file again becomes the input for the L4->L5 compaction.
Not shown is that the key range /Table/51/1/40804278/0 /Table/51/1/64697768
overlaps 1247 files in L6.
My current hypothesis is that SubcompactionState::ShouldStopBefore
is correctly indicating good boundaries for the output files (i.e. boundaries that align well with lower levels), but we have a range tombstone at the end of the file which is causing the end-key boundary to be extended in RangeDelAggregator::AddToBuilder
. A bit of support for this theory is that my RocksDB logging indicates the "type" of the key /Table/51/1/64697768
(the end-key for the problematic sstables) as being a range deletion.
@benesch As far as I can tell this is independent of my RocksDB changes. In CompactionJob::FinishCompactionOutputFile
, range tombstones for the last generated file are bounded by the input key range, but that can result in the last file in a level covering too large a range.
I'm going to try reproducing this badness in a smaller test as I'm tired of dealing with the 10-15min debug cycle of the clearrange
roachtest.
@benesch As far as I can tell this is independent of my RocksDB changes. In CompactionJob::FinishCompactionOutputFile, range tombstones for the last generated file are bounded by the input key range, but that can result in the last file in a level covering too large a range.
Yes, it certainly is. I'm running without your RocksDB patches and observing the same too-large compactions.
I'm going to try reproducing this badness in a smaller test as I'm tired of dealing with the 10-15min debug cycle of the clearrange roachtest.
Running on ZFS-formatted volumes turns out to be really easy:
sudo apt-get update
sudo apt-get -qqy install zfsutils-linux
sudo umount /mnt/data
sudo zpool create -f data1 -m /mnt/data1 /dev/sdb
sudo chown -R $(whoami):$(whoami) /mnt/data1
# download store dumps without starting cluster
sudo zfs snapshot data1@pristine
Then you can restore to the pristine state in less than a second with:
sudo zfs rollback data1@pristine
The performance characteristics of ZFS are bound to cause problems for our performance tests, but so far I haven't noticed any problems.
Here's a list of all of the tombstones on that node: https://gist.github.com/benesch/e72fb0a347382ce1eeca4c99232cfd27
The tombstone you mention spans /Table/51/1/64694496/0 - /Table/51/1/64697768. Doesn't seem especially large on its own. It's preceded by quite a few nearly-adjacent tombstones, though.
While stepping out of the shower this morning I thought of a simple hack that might work around the problem. The TLDR of the problem is that compactions do not take into consideration range tombstones when determining the output files. The result is that the first and last output file from a compaction can be extended way too far so that it covers an excessive number of sstables in the next lower level.
The simple hack: whenever we perform a clear range we also add two additional deletions (for individual keys) at the start and end of the tombstone. These key deletions should allow the compaction logic to select output boundaries correctly. Note that the key deletions have to occur after the range deletion so that they have higher sequence numbers and thus won't get deleted as the range tombstone progresses down through the levels.
Adding the key deletions immediately after a clear range is easy. We might also want to have a migration that runs through the suggested compactions at startup and retroactively adds key deletions for any empty suggested compaction. (We can't do this in the compaction queue because a replica might be concurrently rebalanced onto the node).
Ha! That really might just work.
We might be able to skip the migration. Anyone running a 2.0 cluster that’s been balanced for more than 24h will have aged out all of its suggested compactions. We’re the oddballs for continually rolling back to the state immediately after a restore
On Sun, Jun 10, 2018 at 8:33 AM Peter Mattis notifications@github.com wrote:
While stepping out of the shower this morning I thought of a simple hack that might work around the problem. The TLDR of the problem is that compactions do not take into consideration range tombstones when determining the output files. The result is that the first and last output file from a compaction can be extended way too far so that it covers an excessive number of sstables in the next lower level.
The simple hack: whenever we perform a clear range we also add two additional deletions (for individual keys) at the start and end of the tombstone. These key deletions should allow the compaction logic to select output boundaries correctly. Note that the key deletions have to occur after the range deletion so that they have higher sequence numbers and thus won't get deleted as the range tombstone progresses down through the levels.
Adding the key deletions immediately after a clear range is easy. We might also want to have a migration that runs through the suggested compactions at startup and retroactively adds key deletions for any empty suggested compaction. (We can't do this in the compaction queue because a replica might be concurrently rebalanced onto the node).
— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/24029#issuecomment-396045581, or mute the thread https://github.com/notifications/unsubscribe-auth/AA15ICukTibL-QGg3g_8Yx4YtXE-HnFlks5t7RIUgaJpZM4SwswS .
Ha! That really might just work.
So good in theory, but it doesn't seem to have had any effect. I'm still seeing the excessively large L5->L6 compactions. In fact, I'm seeing exactly the same compactions as before. Either my hack has a bug or there is something fouling it up.
So my hack does work, but it didn't seem to help the clearrange
test. I wrote a small test which demonstrates the working of the hack. I'm going to see about pulling this out into a separate PR and then get back to investigating the clearrange
test.
I've also filed https://github.com/facebook/rocksdb/issues/3977 as the range-tombstone/compaction issue deserves to be addressed upstream.
FYI, I built a new bank stores fixture: gs://cockroach-fixtures/workload/bank/version=1.0.0,payload-bytes=10240,ranges=0,rows=65104166,seed=2/stores=10,bin-version=2.0-6/
. This was built using #26576 which means it has both the clear range hack and the restore empty-range fix. Notice that seed=2
and bin-version=2.0-6
.
Awesome! How long did that take?
On Sun, Jun 10, 2018 at 8:28 PM, Peter Mattis notifications@github.com wrote:
FYI, I built a new bank stores fixture: gs://cockroach-fixtures/ workload/bank/version=1.0.0,payload-bytes=10240,ranges=0, rows=65104166,seed=2/stores=10,bin-version=2.0-6/. This was built using
26576 https://github.com/cockroachdb/cockroach/pull/26576 which means
it has both the clear range hack and the restore empty-range fix. Notice that seed=2 and bin-version=2.0-6.
— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/24029#issuecomment-396094266, or mute the thread https://github.com/notifications/unsubscribe-auth/AA15IOkfabJd4CuM06IVszkPBR1IlXxPks5t7bmhgaJpZM4SwswS .
Awesome! How long did that take?
3-4 hours.
So I found what seems to be a huge perf bug in RocksDB's existing range deletion implementation! Opened a PR upstream: https://github.com/facebook/rocksdb/pull/3992
I'm running a clear range test right now with that fix plus https://github.com/benesch/cockroach/tree/iter-upper-bound and results are looking VERY promising:
I'm going to regret making that claim in the morning. CPU usage is quite stable, though:
Please forgive the graph wonkiness. I filed https://github.com/cockroachdb/cockroach/issues/26679 for that. I also had to turn off the consistency checker because I think whatever is destroying the graphs is also violating replica consistency. (The exact same consistency failures are happening in nightly roachtests so I suspect they're unrelated to my RocksDB patch, but still scary timing.)
One last thought: you'll notice that storage available isn't budging in the screenshot above. It is in fact ticking downward as I write this comment.
We'll likely want to continue investigating the fixes for the overlarge compactions we're seeing that wedge up the compaction queue, but mitigating the massive CPU usage of range tombstones really gives us some breathing room.
Just peeked at your cluster... This really does look pretty nice:
That "Capacity" line at the very top deserves a better tool tip by the way... this looks very confusing, though there's probably a good reason it behaves that way.
That "Capacity" line at the very top deserves a better tool tip by the way... this looks very confusing, though there's probably a good reason it behaves that way.
That's due to running on zfs and having a zfs snapshot that is consuming capacity and thus reducing the capacity to the rest of the system as the filesystem "head" diverges from the snapshot.
Ah, TIL. Thanks for the heads up.
I ran a test this morning using https://github.com/petermattis/rocksdb/tree/pmattis/get-overlapping-inputs which changes VersionStorageInfo::GetOverlappingInputs
to not consider the range tombstone sentinel key that sometimes shows up as the largest key in an sstable as being part of the user-key. Gah, I feel at a loss to explain this right now. Here are some pictures:
This also looks promising, but the CPU graph is much less pretty:
Grep'ing logs also showed that there were still a number of large compactions. For example, one compaction took 11m12s. Hmm, looks like this might be the compaction queue:
I180613 11:04:05.066713 234 storage/compactor/compactor.go:367 [n3,s3,compactor] processing compaction #1-2023/2023 (/Table/53/1/39713836-/Table/53/1/58425827) for 58 GiB (reasons: size=true used=true avail=true)
...
I180613 11:15:18.032050 234 storage/compactor/compactor.go:383 [n3,s3,compactor] processed compaction #1-2023/2023 (/Table/53/1/39713836-/Table/53/1/58425827) for 58 GiB in 11m12.965283011s
Putting this here so I don't forget.
There's something funny about how the existing code handles empty range tombstones, i.e., range tombstones from [a, a) or [b, a). Here's a test that I discovered last night that expects such tombstones to be preserved even though they can be dropped on the floor the minute they come through: https://github.com/facebook/rocksdb/blob/1f32dc7d2b6721a1fe13eb515d52e5cd6f110f59/db/db_range_del_test.cc#L489-L515
Here's a case where it's more problematic: https://github.com/facebook/rocksdb/pull/3992/files#diff-de8dc928b3d0f5bf738e5dc2cae5cc6dR114. That collapsed map has two entries for key a! Yikes. Pretty sure that's not handled correctly by ShouldDelete. I suspect it's also mishandled by AddToBuilder.
With the recent improvements to RocksDB range tombstones, the clearrange
roachtest now reliably passes:
Note that the test started slightly after 00:00. This test was using a freshly generated bank
fixture. For some reason, cockroach
was refusing to start using the old store fixtures complaining about the version being too old. This doesn't make any sense and needs to be investigated. I'll file a separate issue when I reproduce.
For some reason, cockroach was refusing to start using the old store fixtures complaining about the version being too old. This doesn't make any sense and needs to be investigated.
Why doesn't it make any sense? I think that fixture was generated with 1.1, and you're running a 2.1 beta.
In that graph, why does available stay constant while capacity decreases?
On Fri, Jul 13, 2018 at 8:40 PM Nikhil Benesch notifications@github.com wrote:
For some reason, cockroach was refusing to start using the old store fixtures complaining about the version being too old. This doesn't make any sense and needs to be investigated.
Why doesn't it make any sense? I think that fixture was generated with 1.1, and this is a 2.1 beta.
— You are receiving this because you were mentioned. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/24029#issuecomment-404985740, or mute the thread https://github.com/notifications/unsubscribe-auth/AF3MTX1xzZnmjyOUp9tin3rkW-pQoIo3ks5uGT32gaJpZM4SwswS .
Why doesn't it make any sense? I think that fixture was generated with 1.1, and you're running a 2.1 beta.
See https://github.com/cockroachdb/cockroach/issues/27525. I generated that fixture with a binary at version v2.0-7.
In that graph, why does available stay constant while capacity decreases?
Zfs. For ease of debugging we take a zfs snapshot after restoring the store fixtures which makes subsequent runs of the test fast. The downside is that as we start to delete data, zfs doesn't actually delete it off disk because it is referenced by the snapshot.
clearrange
has passed 4 nights in a row. I'm closing this issue. https://github.com/cockroachdb/cockroach/issues/26693 remains open to investigate the excessive compactions that can still occur, but do not drive a cluster to its knees.
Wait ~10m for stores to download. Then drop the 2TiB table:
The cluster explodes a few minutes later as RocksDB tombstones pile up. I can no longer execute any SQL queries that read from/write to disk.
Very closely related to #21901, but thought I'd file a separate tracking issue.
/cc @spencerkimball