Skip to content

Commit

Permalink
Patch 8.6.6 (#11886)
Browse files Browse the repository at this point in the history
* Rollback other pending memtable flushes when a flush fails (#11865)

Summary:
when atomic_flush=false, there are certain cases where we try to install memtable results with already deleted SST files. This can happen when the following sequence events happen:
```
Start Flush0 for memtable M0 to SST0
Start Flush1 for memtable M1 to SST1
Flush 1 returns OK, but don't install to MANIFEST and let whoever flushes M0 to take care of it
Flush0 finishes with a retryable IOError, it rollbacks M0, (incorrectly) does not rollback M1, and deletes SST0 and SST1
Starts Flush2 for M0, it does not pick up M1 since it thought M1 is flushed
Flush2 writes SST2 and finishes OK, tries to install SST2 and SST1
Error opening SST1 since it's already deleted with an  error message like the following:

IO error: No such file or directory: While open a file for random read: /tmp/rocksdbtest-501/db_flush_test_3577_4230653031040984171/000011.sst: No such file or directory
```

This happens since:
1. We currently only rollback the memtables that we are flushing in a flush job when atomic_flush=false.
2. Pending output SSTs from previous flushes are deleted since a pending file number is released whenever a flush job is finished no matter of flush status: https://github.com/facebook/rocksdb/blob/f42e70bf561d4be9b6bbe7316d1c2c0c8a3818e6/db/db_impl/db_impl_compaction_flush.cc#L3161

This PR fixes the issue by rollback these pending flushes.

There is another issue where if a new flush for new memtable starts and finishes after Flush0 finishes. Its output may also be deleted (see more in unit test). It is fixed by checking bg error status before installing a memtable result, and rollback if there is an error.

There is a more efficient fix where we just don't release the pending file output number for flushes that delegate installation. It is more efficient since it does not have to rewrite the flush output file. With the fix in this PR, we can end up with a giant file if a lot of memtables are being flushed together. However, the more efficient fix is a bit more complicated to implement (requires associating such pending file numbers with flush job/memtables) and is more risky since it changes normal flush code path.

Pull Request resolved: #11865

Test Plan: * Added repro unit tests.

Reviewed By: anand1976

Differential Revision: D49484922

Pulled By: cbi42

fbshipit-source-id: 25b536c08f4e02e7f1d0f86571663737d2b5d53d

* Fix a bug with atomic_flush that causes DB to stuck after a flush failure (#11872)

Summary:
With atomic_flush=true, a flush job with younger memtables wait for older memtables to be installed before install its memtables. If the flush for older memtables failed, auto-recovery starts a resume thread which can becomes stuck waiting for all background work to finish (including the flush for younger memtables). If a non-recovery flush starts now and tries to flush, it can make the situation worse since it will fail due to background error but never rollback its memtable: https://github.com/facebook/rocksdb/blob/269478ee4618283cd6d710fdfea9687157a259c1/db/db_impl/db_impl_compaction_flush.cc#L725 This prevents any future flush to pick old memtables.

A more detailed repro is in unit test.

This PR fixes this issue by
1. Ensure we rollback memtables if an atomic flush fails due to background error
2. When there is a background error, abort atomic flushes that are waiting for older memtables to be installed
3. Do not schedule non-recovery flushes when there is a background error that stops background work

There was another issue with atomic_flush=true where DB can hang during DB close, see more in #11867. The fix in this PR, specifically fix 2 above, should be enough to resolve it too.

Pull Request resolved: #11872

Test Plan: new unit test.

Reviewed By: jowlyzhang

Differential Revision: D49556867

Pulled By: cbi42

fbshipit-source-id: 4a0210ff28a8552a99ece7fbb0f574fd24b4da3f

* Only flush after recovery for retryable IOError (#11880)

Summary:
#11872 causes a unit test to start failing with the error message below. The cause is that the additional call to `FlushAllColumnFamilies()` in `DBImpl::ResumeImpl()` can run while DB is closing. More detailed explanation: there are two places where we call `ResumeImpl()`:

1. in `ErrorHandler::RecoverFromBGError`, for manual resume or recovery from errors like OutOfSpace through sst file manager, and
2. in `Errorhandler::RecoverFromRetryableBGIOError`, for error recovery from errors like flush failure due to retryable IOError. This is tracked by `ErrorHandler::recovery_thread_`.

Here is how DB close waits for error recovery: https://github.com/facebook/rocksdb/blob/49da91ec097b4efcd8a8e4dc1b287e9f81eb4093/db/db_impl/db_impl.cc#L540-L543

`CancelErrorRecovery()` waits until `recovery_thread_` finishes and `IsRecoveryInProgress()` checks the `recovery_in_prog_` flag. The additional call to `FlushAllColumnFamilies()` in `ResumeImpl()` happens after it clears bg error and the `recovery_in_prog_` flag: https://github.com/facebook/rocksdb/blob/49da91ec097b4efcd8a8e4dc1b287e9f81eb4093/db/db_impl/db_impl.cc#L436-L463. So if `ResumeImpl()` is called in `RecoverFromBGError()`, we can have a thread running `FlushAllColumnFamilies()` while DB is closing and thought that recovery is done.

The fix is to only do the additional call to `FlushAllColumnFamilies()` when doing error recovery through `Errorhandler::RecoverFromRetryableBGIOError` by setting flags in `DBRecoverContext`.

Pull Request resolved: #11880

Test Plan:
`gtest-parallel --repeat=100 --workers=4 ./error_handler_fs_test --gtest_filter="*AutoRecoverFlushError*"` reproduces the error pretty reliably.

```[==========] Running 1 test from 1 test case.
[----------] Global test environment set-up.
[----------] 1 test from DBErrorHandlingFSTest
[ RUN      ] DBErrorHandlingFSTest.AutoRecoverFlushError
error_handler_fs_test: db/column_family.cc:1618: rocksdb::ColumnFamilySet::~ColumnFamilySet(): Assertion `last_ref' failed.
Received signal 6 (Aborted)
...
#10 0x00007fac4409efd6 in __GI___assert_fail (assertion=0x7fac452c0afa "last_ref", file=0x7fac452c9fb5 "db/column_family.cc", line=1618, function=0x7fac452cb950 "rocksdb::ColumnFamilySet::~ColumnFamilySet()") at assert.c:101
101     in assert.c
#11 0x00007fac44b5324f in rocksdb::ColumnFamilySet::~ColumnFamilySet (this=0x7b5400000000) at db/column_family.cc:1618
1618        assert(last_ref);
#12 0x00007fac44e0f047 in std::default_delete<rocksdb::ColumnFamilySet>::operator() (this=0x7b5800000940, __ptr=0x7b5400000000) at /usr/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/unique_ptr.h:85
85              delete __ptr;
#13 std::__uniq_ptr_impl<rocksdb::ColumnFamilySet, std::default_delete<rocksdb::ColumnFamilySet> >::reset (this=0x7b5800000940, __p=0x0) at /usr/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/unique_ptr.h:182
182               _M_deleter()(__old_p);
#14 std::unique_ptr<rocksdb::ColumnFamilySet, std::default_delete<rocksdb::ColumnFamilySet> >::reset (this=0x7b5800000940, __p=0x0) at /usr/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/unique_ptr.h:456
456             _M_t.reset(std::move(__p));
#15 rocksdb::VersionSet::~VersionSet (this=this@entry=0x7b5800000900) at db/version_set.cc:5081
5081      column_family_set_.reset();
#16 0x00007fac44e0f97a in rocksdb::VersionSet::~VersionSet (this=0x7b5800000900) at db/version_set.cc:5078
5078    VersionSet::~VersionSet() {
#17 0x00007fac44bf0b2f in std::default_delete<rocksdb::VersionSet>::operator() (this=0x7b8c00000068, __ptr=0x7b5800000900) at /usr/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/unique_ptr.h:85
85              delete __ptr;
#18 std::__uniq_ptr_impl<rocksdb::VersionSet, std::default_delete<rocksdb::VersionSet> >::reset (this=0x7b8c00000068, __p=0x0) at /usr/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/unique_ptr.h:182
182               _M_deleter()(__old_p);
#19 std::unique_ptr<rocksdb::VersionSet, std::default_delete<rocksdb::VersionSet> >::reset (this=0x7b8c00000068, __p=0x0) at /usr/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/unique_ptr.h:456
456             _M_t.reset(std::move(__p));
#20 rocksdb::DBImpl::CloseHelper (this=this@entry=0x7b8c00000000) at db/db_impl/db_impl.cc:676
676       versions_.reset();
#21 0x00007fac44bf1346 in rocksdb::DBImpl::CloseImpl (this=0x7b8c00000000) at db/db_impl/db_impl.cc:720
720     Status DBImpl::CloseImpl() { return CloseHelper(); }
#22 rocksdb::DBImpl::~DBImpl (this=this@entry=0x7b8c00000000) at db/db_impl/db_impl.cc:738
738       closing_status_ = CloseImpl();
#23 0x00007fac44bf2bba in rocksdb::DBImpl::~DBImpl (this=0x7b8c00000000) at db/db_impl/db_impl.cc:722
722     DBImpl::~DBImpl() {
#24 0x00007fac455444d4 in rocksdb::DBTestBase::Close (this=this@entry=0x7b6c00000000) at db/db_test_util.cc:678
678       delete db_;
#25 0x00007fac455455fb in rocksdb::DBTestBase::TryReopen (this=this@entry=0x7b6c00000000, options=...) at db/db_test_util.cc:707
707       Close();
#26 0x00007fac45543459 in rocksdb::DBTestBase::Reopen (this=0x7ffed74b79a0, options=...) at db/db_test_util.cc:670
670       ASSERT_OK(TryReopen(options));
#27 0x00000000004f2522 in rocksdb::DBErrorHandlingFSTest_AutoRecoverFlushError_Test::TestBody (this=this@entry=0x7b6c00000000) at db/error_handler_fs_test.cc:1224
1224      Reopen(options);
```

Reviewed By: jowlyzhang

Differential Revision: D49579701

Pulled By: cbi42

fbshipit-source-id: 3fc8325e6dde7e7faa8bcad95060cb4e26eda638

* Update HISTORY.md and version.h for 8.6.6
  • Loading branch information
cbi42 committed Sep 25, 2023
1 parent 2c78b1b commit 8c2e91a
Show file tree
Hide file tree
Showing 15 changed files with 513 additions and 46 deletions.
6 changes: 6 additions & 0 deletions HISTORY.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,12 @@
# Rocksdb Change Log
> NOTE: Entries for next release do not go here. Follow instructions in `unreleased_history/README.txt`
## 8.6.6 (09/25/2023)
### Bug Fixes
* Fix a bug with atomic_flush=true that can cause DB to stuck after a flush fails (#11872).
* Fix a bug where RocksDB (with atomic_flush=false) can delete output SST files of pending flushes when a previous concurrent flush fails (#11865). This can result in DB entering read-only state with error message like `IO error: No such file or directory: While open a file for random read: /tmp/rocksdbtest-501/db_flush_test_87732_4230653031040984171/000013.sst`.
* When the compressed secondary cache capacity is reduced to 0, it should be completely disabled. Before this fix, inserts and lookups would still go to the backing `LRUCache` before returning, thus incurring locking overhead. With this fix, inserts and lookups are no-ops and do not add any overhead.

## 8.6.5 (09/15/2023)
### Bug Fixes
* Fixed a bug where `rocksdb.file.read.verify.file.checksums.micros` is not populated.
Expand Down
273 changes: 273 additions & 0 deletions db/db_flush_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3193,6 +3193,279 @@ INSTANTIATE_TEST_CASE_P(DBFlushDirectIOTest, DBFlushDirectIOTest,

INSTANTIATE_TEST_CASE_P(DBAtomicFlushTest, DBAtomicFlushTest, testing::Bool());

TEST_F(DBFlushTest, NonAtomicFlushRollbackPendingFlushes) {
// Fix a bug in when atomic_flush=false.
// The bug can happen as follows:
// Start Flush0 for memtable M0 to SST0
// Start Flush1 for memtable M1 to SST1
// Flush1 returns OK, but don't install to MANIFEST and let whoever flushes
// M0 to take care of it
// Flush0 finishes with a retryable IOError
// - It rollbacks M0, (incorrectly) not M1
// - Deletes SST1 and SST2
//
// Auto-recovery will start Flush2 for M0, it does not pick up M1 since it
// thinks that M1 is flushed
// Flush2 writes SST3 and finishes OK, tries to install SST3 and SST2
// Error opening SST2 since it's already deleted
//
// The fix is to let Flush0 also rollback M1.
Options opts = CurrentOptions();
opts.atomic_flush = false;
opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
opts.max_write_buffer_number = 64;
opts.max_background_flushes = 4;
env_->SetBackgroundThreads(4, Env::HIGH);
DestroyAndReopen(opts);
std::atomic_int flush_count = 0;
SyncPoint::GetInstance()->ClearAllCallBacks();
SyncPoint::GetInstance()->DisableProcessing();
SyncPoint::GetInstance()->SetCallBack(
"FlushJob::WriteLevel0Table:s", [&](void* s_ptr) {
int c = flush_count.fetch_add(1);
if (c == 0) {
Status* s = (Status*)(s_ptr);
IOStatus io_error = IOStatus::IOError("injected foobar");
io_error.SetRetryable(true);
*s = io_error;
TEST_SYNC_POINT("Let mem1 flush start");
TEST_SYNC_POINT("Wait for mem1 flush to finish");
}
});
SyncPoint::GetInstance()->LoadDependency(
{{"Let mem1 flush start", "Mem1 flush starts"},
{"DBImpl::BGWorkFlush:done", "Wait for mem1 flush to finish"},
{"RecoverFromRetryableBGIOError:RecoverSuccess",
"Wait for error recover"}});
// Need first flush to wait for the second flush to finish
SyncPoint::GetInstance()->EnableProcessing();
ASSERT_OK(Put(Key(1), "val1"));
// trigger bg flush mem0
ASSERT_OK(Put(Key(2), "val2"));
TEST_SYNC_POINT("Mem1 flush starts");
// trigger bg flush mem1
ASSERT_OK(Put(Key(3), "val3"));

TEST_SYNC_POINT("Wait for error recover");
ASSERT_EQ(1, NumTableFilesAtLevel(0));
SyncPoint::GetInstance()->ClearAllCallBacks();
SyncPoint::GetInstance()->DisableProcessing();
}

TEST_F(DBFlushTest, AbortNonAtomicFlushWhenBGError) {
// Fix a bug in when atomic_flush=false.
// The bug can happen as follows:
// Start Flush0 for memtable M0 to SST0
// Start Flush1 for memtable M1 to SST1
// Flush1 returns OK, but doesn't install output MANIFEST and let whoever
// flushes M0 to take care of it
// Start Flush2 for memtable M2 to SST2
// Flush0 finishes with a retryable IOError
// - It rollbacks M0 AND M1
// - Deletes SST1 and SST2
// Flush2 finishes, does not rollback M2,
// - releases the pending file number that keeps SST2 alive
// - deletes SST2
//
// Then auto-recovery starts, error opening SST2 when try to install
// flush result
//
// The fix is to let Flush2 rollback M2 if it finds that
// there is a background error.
Options opts = CurrentOptions();
opts.atomic_flush = false;
opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
opts.max_write_buffer_number = 64;
opts.max_background_flushes = 4;
env_->SetBackgroundThreads(4, Env::HIGH);
DestroyAndReopen(opts);
std::atomic_int flush_count = 0;
SyncPoint::GetInstance()->ClearAllCallBacks();
SyncPoint::GetInstance()->DisableProcessing();
SyncPoint::GetInstance()->SetCallBack(
"FlushJob::WriteLevel0Table:s", [&](void* s_ptr) {
int c = flush_count.fetch_add(1);
if (c == 0) {
Status* s = (Status*)(s_ptr);
IOStatus io_error = IOStatus::IOError("injected foobar");
io_error.SetRetryable(true);
*s = io_error;
TEST_SYNC_POINT("Let mem1 flush start");
TEST_SYNC_POINT("Wait for mem1 flush to finish");

TEST_SYNC_POINT("Let mem2 flush start");
TEST_SYNC_POINT("Wait for mem2 to start writing table");
}
});

SyncPoint::GetInstance()->SetCallBack(
"FlushJob::WriteLevel0Table", [&](void* mems) {
autovector<MemTable*>* mems_ptr = (autovector<MemTable*>*)mems;
if ((*mems_ptr)[0]->GetID() == 3) {
TEST_SYNC_POINT("Mem2 flush starts writing table");
TEST_SYNC_POINT("Mem2 flush waits until rollback");
}
});
SyncPoint::GetInstance()->LoadDependency(
{{"Let mem1 flush start", "Mem1 flush starts"},
{"DBImpl::BGWorkFlush:done", "Wait for mem1 flush to finish"},
{"Let mem2 flush start", "Mem2 flush starts"},
{"Mem2 flush starts writing table",
"Wait for mem2 to start writing table"},
{"RollbackMemtableFlush", "Mem2 flush waits until rollback"},
{"RecoverFromRetryableBGIOError:RecoverSuccess",
"Wait for error recover"}});
SyncPoint::GetInstance()->EnableProcessing();

ASSERT_OK(Put(Key(1), "val1"));
// trigger bg flush mem0
ASSERT_OK(Put(Key(2), "val2"));
TEST_SYNC_POINT("Mem1 flush starts");
// trigger bg flush mem1
ASSERT_OK(Put(Key(3), "val3"));

TEST_SYNC_POINT("Mem2 flush starts");
ASSERT_OK(Put(Key(4), "val4"));

TEST_SYNC_POINT("Wait for error recover");
// Recovery flush writes 3 memtables together into 1 file.
ASSERT_EQ(1, NumTableFilesAtLevel(0));
SyncPoint::GetInstance()->ClearAllCallBacks();
SyncPoint::GetInstance()->DisableProcessing();
}

TEST_F(DBFlushTest, NonAtomicNormalFlushAbortWhenBGError) {
Options opts = CurrentOptions();
opts.atomic_flush = false;
opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
opts.max_write_buffer_number = 64;
opts.max_background_flushes = 1;
env_->SetBackgroundThreads(2, Env::HIGH);
DestroyAndReopen(opts);
SyncPoint::GetInstance()->ClearAllCallBacks();
SyncPoint::GetInstance()->DisableProcessing();
std::atomic_int flush_write_table_count = 0;
SyncPoint::GetInstance()->SetCallBack(
"FlushJob::WriteLevel0Table:s", [&](void* s_ptr) {
int c = flush_write_table_count.fetch_add(1);
if (c == 0) {
Status* s = (Status*)(s_ptr);
IOStatus io_error = IOStatus::IOError("injected foobar");
io_error.SetRetryable(true);
*s = io_error;
}
});

SyncPoint::GetInstance()->EnableProcessing();
SyncPoint::GetInstance()->LoadDependency(
{{"Let error recovery start",
"RecoverFromRetryableBGIOError:BeforeStart"},
{"RecoverFromRetryableBGIOError:RecoverSuccess",
"Wait for error recover"}});

ASSERT_OK(Put(Key(1), "val1"));
// trigger bg flush0 for mem0
ASSERT_OK(Put(Key(2), "val2"));
// Not checking status since this wait can finish before flush starts.
dbfull()->TEST_WaitForFlushMemTable().PermitUncheckedError();

// trigger bg flush1 for mem1, should see bg error and abort
// before picking a memtable to flush
ASSERT_OK(Put(Key(3), "val3"));
ASSERT_NOK(dbfull()->TEST_WaitForFlushMemTable());
ASSERT_EQ(0, NumTableFilesAtLevel(0));

TEST_SYNC_POINT("Let error recovery start");
TEST_SYNC_POINT("Wait for error recover");
// Recovery flush writes 2 memtables together into 1 file.
ASSERT_EQ(1, NumTableFilesAtLevel(0));
// 1 for flush 0 and 1 for recovery flush
ASSERT_EQ(2, flush_write_table_count);
SyncPoint::GetInstance()->ClearAllCallBacks();
SyncPoint::GetInstance()->DisableProcessing();
}

TEST_F(DBFlushTest, DBStuckAfterAtomicFlushError) {
// Test for a bug with atomic flush where DB can become stuck
// after a flush error. A repro timeline:
//
// Start Flush0 for mem0
// Start Flush1 for mem1
// Now Flush1 will wait for Flush0 to install mem0
// Flush0 finishes with retryable IOError, rollbacks mem0
// Resume starts and waits for background job to finish, i.e., Flush1
// Fill memtable again, trigger Flush2 for mem0
// Flush2 will get error status, and not rollback mem0, see code in
// https://github.com/facebook/rocksdb/blob/b927ba5936216861c2c35ab68f50ba4a78e65747/db/db_impl/db_impl_compaction_flush.cc#L725
//
// DB is stuck since mem0 can never be picked now
//
// The fix is to rollback mem0 in Flush2, and let Flush1 also abort upon
// background error besides waiting for older memtables to be installed.
// The recovery flush in this case should pick up all memtables
// and write them to a single L0 file.
Options opts = CurrentOptions();
opts.atomic_flush = true;
opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
opts.max_write_buffer_number = 64;
opts.max_background_flushes = 4;
env_->SetBackgroundThreads(4, Env::HIGH);
DestroyAndReopen(opts);

std::atomic_int flush_count = 0;
SyncPoint::GetInstance()->ClearAllCallBacks();
SyncPoint::GetInstance()->DisableProcessing();
SyncPoint::GetInstance()->SetCallBack(
"FlushJob::WriteLevel0Table:s", [&](void* s_ptr) {
int c = flush_count.fetch_add(1);
if (c == 0) {
Status* s = (Status*)(s_ptr);
IOStatus io_error = IOStatus::IOError("injected foobar");
io_error.SetRetryable(true);
*s = io_error;
TEST_SYNC_POINT("Let flush for mem1 start");
// Wait for Flush1 to start waiting to install flush result
TEST_SYNC_POINT("Wait for flush for mem1");
}
});
SyncPoint::GetInstance()->LoadDependency(
{{"Let flush for mem1 start", "Flush for mem1"},
{"DBImpl::AtomicFlushMemTablesToOutputFiles:WaitCV",
"Wait for flush for mem1"},
{"RecoverFromRetryableBGIOError:BeforeStart",
"Wait for resume to start"},
{"Recovery should continue here",
"RecoverFromRetryableBGIOError:BeforeStart2"},
{"RecoverFromRetryableBGIOError:RecoverSuccess",
"Wait for error recover"}});
SyncPoint::GetInstance()->EnableProcessing();
ASSERT_OK(Put(Key(1), "val1"));
// trigger Flush0 for mem0
ASSERT_OK(Put(Key(2), "val2"));

// trigger Flush1 for mem1
TEST_SYNC_POINT("Flush for mem1");
ASSERT_OK(Put(Key(3), "val3"));

// Wait until resume started to schedule another flush
TEST_SYNC_POINT("Wait for resume to start");
// This flush should not be scheduled due to bg error
ASSERT_OK(Put(Key(4), "val4"));

// TEST_WaitForBackgroundWork() returns background error
// after all background work is done.
ASSERT_NOK(dbfull()->TEST_WaitForBackgroundWork());
// Flush should abort and not writing any table
ASSERT_EQ(0, NumTableFilesAtLevel(0));

// Wait until this flush is done.
TEST_SYNC_POINT("Recovery should continue here");
TEST_SYNC_POINT("Wait for error recover");
// error recovery can schedule new flushes, but should not
// encounter error
ASSERT_OK(dbfull()->TEST_WaitForBackgroundWork());
ASSERT_EQ(1, NumTableFilesAtLevel(0));
}
} // namespace ROCKSDB_NAMESPACE

int main(int argc, char** argv) {
Expand Down
22 changes: 22 additions & 0 deletions db/db_impl/db_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -471,6 +471,28 @@ Status DBImpl::ResumeImpl(DBRecoverContext context) {
if (shutdown_initiated_) {
s = Status::ShutdownInProgress();
}
if (s.ok() && context.flush_after_recovery) {
// Since we drop all non-recovery flush requests during recovery,
// and new memtable may fill up during recovery,
// schedule one more round of flush.
FlushOptions flush_opts;
flush_opts.allow_write_stall = false;
flush_opts.wait = false;
Status status = FlushAllColumnFamilies(
flush_opts, FlushReason::kCatchUpAfterErrorRecovery);
if (!status.ok()) {
// FlushAllColumnFamilies internally should take care of setting
// background error if needed.
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"The catch up flush after successful recovery failed [%s]",
s.ToString().c_str());
}
// FlushAllColumnFamilies releases and re-acquires mutex.
if (shutdown_initiated_) {
s = Status::ShutdownInProgress();
}
}

if (s.ok()) {
for (auto cfd : *versions_->GetColumnFamilySet()) {
SchedulePendingCompaction(cfd);
Expand Down
Loading

0 comments on commit 8c2e91a

Please sign in to comment.