rocksdb/db/db_impl
fanrui03 67d72fb5dc Fix checkpoint stuck (#7921)
Summary:
## 1. Bug description:

When RocksDB Checkpoint, it may be stuck in `WaitUntilFlushWouldNotStallWrites` method.

## 2. Simple analysis of the reasons:

### 2.1 Configuration parameters:

```yaml
Compaction Style : Universal

max_write_buffer_number : 4
min_write_buffer_number_to_merge : 3
```

Checkpoint is usually very fast. When the Checkpoint is executed, `WaitUntilFlushWouldNotStallWrites` is called. If there are 2 Immutable MemTables, which are less than `min_write_buffer_number_to_merge`, they will not be flushed. But will enter this code.

```c++
// method: GetWriteStallConditionAndCause
if (mutable_cf_options.max_write_buffer_number> 3 &&
              num_unflushed_memtables >=
                  mutable_cf_options.max_write_buffer_number-1) {
     return {WriteStallCondition::kDelayed, WriteStallCause::kMemtableLimit};
}
```

code link: fbed72f03c/db/column_family.cc (L847)

Checkpoint thought there was a FlushJob, but it didn't. So will always wait.

### 2.2 solution:

Increase the restriction: the `number of Immutable MemTable` >= `min_write_buffer_number_to_merge will wait`.

If there are other better solutions, you can correct me.

### 2.3 Code that can reproduce the problem:

https://github.com/1996fanrui/fanrui-learning/blob/flink-1.12/module-java/src/main/java/com/dream/rocksdb/RocksDBCheckpointStuck.java

## 3. Interesting point

This bug will be triggered only when `the number of sorted runs >= level0_file_num_compaction_trigger`.

Because there is a break in WaitUntilFlushWouldNotStallWrites.

```c++
if (cfd->imm()->NumNotFlushed() <
        cfd->ioptions()->min_write_buffer_number_to_merge &&
    vstorage->l0_delay_trigger_count() <
        mutable_cf_options.level0_file_num_compaction_trigger) {
  break;
}
```

code link: fbed72f03c/db/db_impl/db_impl_compaction_flush.cc (L1974)

Universal may have `l0_delay_trigger_count() >= level0_file_num_compaction_trigger`, so this bug is triggered.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/7921

Reviewed By: jay-zhuang

Differential Revision: D26900559

Pulled By: ajkr

fbshipit-source-id: 133c1252dad7393753f04a47590b68c7d8e670df
2021-03-09 02:21:25 -08:00
..
db_impl_compaction_flush.cc Fix checkpoint stuck (#7921) 2021-03-09 02:21:25 -08:00
db_impl_debug.cc Do not set bg error for compaction in retryable IO Error case (#7899) 2021-01-27 17:58:12 -08:00
db_impl_experimental.cc Replace reinterpret_cast with static_cast_with_check (#7067) 2020-07-02 19:25:41 -07:00
db_impl_files.cc Add a SystemClock class to capture the time functions of an Env (#7858) 2021-01-25 22:09:11 -08:00
db_impl_open.cc Break down the amount of data written during flushes/compactions per file type (#8013) 2021-03-02 09:48:00 -08:00
db_impl_readonly.cc Bug fix for status overridden by Status::NotFound in db_impl_readonly (#7972) 2021-02-17 19:35:57 -08:00
db_impl_readonly.h RocksJava - Add errorIfLogFileExists parameter to RocksDB.openReadOnly (#7046) 2020-09-17 15:41:25 -07:00
db_impl_secondary.cc Add a SystemClock class to capture the time functions of an Env (#7858) 2021-01-25 22:09:11 -08:00
db_impl_secondary.h RocksJava - Add errorIfLogFileExists parameter to RocksDB.openReadOnly (#7046) 2020-09-17 15:41:25 -07:00
db_impl_write.cc Add a SystemClock class to capture the time functions of an Env (#7858) 2021-01-25 22:09:11 -08:00
db_impl.cc Extend VerifyFileChecksums API for blob files (#7979) 2021-02-22 22:09:22 -08:00
db_impl.h Extend VerifyFileChecksums API for blob files (#7979) 2021-02-22 22:09:22 -08:00
db_secondary_test.cc Fix assertion failure in bg flush (#7362) 2020-12-02 09:31:14 -08:00