rocksdb/db/event_helpers.cc

334 lines
12 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#include "db/event_helpers.h"
#include "rocksdb/convenience.h"
#include "rocksdb/listener.h"
#include "rocksdb/utilities/customizable_util.h"
namespace ROCKSDB_NAMESPACE {
Status EventListener::CreateFromString(const ConfigOptions& config_options,
const std::string& id,
std::shared_ptr<EventListener>* result) {
return LoadSharedObject<EventListener>(config_options, id, result);
}
namespace {
template <class T>
inline T SafeDivide(T a, T b) {
return b == 0 ? 0 : a / b;
}
} // anonymous namespace
void EventHelpers::AppendCurrentTime(JSONWriter* jwriter) {
*jwriter << "time_micros"
<< std::chrono::duration_cast<std::chrono::microseconds>(
std::chrono::system_clock::now().time_since_epoch())
.count();
}
void EventHelpers::NotifyTableFileCreationStarted(
const std::vector<std::shared_ptr<EventListener>>& listeners,
const std::string& db_name, const std::string& cf_name,
const std::string& file_path, int job_id, TableFileCreationReason reason) {
if (listeners.empty()) {
return;
}
TableFileCreationBriefInfo info;
info.db_name = db_name;
info.cf_name = cf_name;
info.file_path = file_path;
info.job_id = job_id;
info.reason = reason;
for (auto& listener : listeners) {
listener->OnTableFileCreationStarted(info);
}
}
void EventHelpers::NotifyOnBackgroundError(
const std::vector<std::shared_ptr<EventListener>>& listeners,
Auto recovery from out of space errors (#4164) Summary: This commit implements automatic recovery from a Status::NoSpace() error during background operations such as write callback, flush and compaction. The broad design is as follows - 1. Compaction errors are treated as soft errors and don't put the database in read-only mode. A compaction is delayed until enough free disk space is available to accomodate the compaction outputs, which is estimated based on the input size. This means that users can continue to write, and we rely on the WriteController to delay or stop writes if the compaction debt becomes too high due to persistent low disk space condition 2. Errors during write callback and flush are treated as hard errors, i.e the database is put in read-only mode and goes back to read-write only fater certain recovery actions are taken. 3. Both types of recovery rely on the SstFileManagerImpl to poll for sufficient disk space. We assume that there is a 1-1 mapping between an SFM and the underlying OS storage container. For cases where multiple DBs are hosted on a single storage container, the user is expected to allocate a single SFM instance and use the same one for all the DBs. If no SFM is specified by the user, DBImpl::Open() will allocate one, but this will be one per DB and each DB will recover independently. The recovery implemented by SFM is as follows - a) On the first occurance of an out of space error during compaction, subsequent compactions will be delayed until the disk free space check indicates enough available space. The required space is computed as the sum of input sizes. b) The free space check requirement will be removed once the amount of free space is greater than the size reserved by in progress compactions when the first error occured c) If the out of space error is a hard error, a background thread in SFM will poll for sufficient headroom before triggering the recovery of the database and putting it in write-only mode. The headroom is calculated as the sum of the write_buffer_size of all the DB instances associated with the SFM 4. EventListener callbacks will be called at the start and completion of automatic recovery. Users can disable the auto recov ery in the start callback, and later initiate it manually by calling DB::Resume() Todo: 1. More extensive testing 2. Add disk full condition to db_stress (follow-on PR) Pull Request resolved: https://github.com/facebook/rocksdb/pull/4164 Differential Revision: D9846378 Pulled By: anand1976 fbshipit-source-id: 80ea875dbd7f00205e19c82215ff6e37da10da4a
2018-09-15 20:36:19 +00:00
BackgroundErrorReason reason, Status* bg_error, InstrumentedMutex* db_mutex,
bool* auto_recovery) {
if (listeners.empty()) {
return;
}
db_mutex->AssertHeld();
// release lock while notifying events
db_mutex->Unlock();
for (auto& listener : listeners) {
listener->OnBackgroundError(reason, bg_error);
bg_error->PermitUncheckedError();
Auto recovery from out of space errors (#4164) Summary: This commit implements automatic recovery from a Status::NoSpace() error during background operations such as write callback, flush and compaction. The broad design is as follows - 1. Compaction errors are treated as soft errors and don't put the database in read-only mode. A compaction is delayed until enough free disk space is available to accomodate the compaction outputs, which is estimated based on the input size. This means that users can continue to write, and we rely on the WriteController to delay or stop writes if the compaction debt becomes too high due to persistent low disk space condition 2. Errors during write callback and flush are treated as hard errors, i.e the database is put in read-only mode and goes back to read-write only fater certain recovery actions are taken. 3. Both types of recovery rely on the SstFileManagerImpl to poll for sufficient disk space. We assume that there is a 1-1 mapping between an SFM and the underlying OS storage container. For cases where multiple DBs are hosted on a single storage container, the user is expected to allocate a single SFM instance and use the same one for all the DBs. If no SFM is specified by the user, DBImpl::Open() will allocate one, but this will be one per DB and each DB will recover independently. The recovery implemented by SFM is as follows - a) On the first occurance of an out of space error during compaction, subsequent compactions will be delayed until the disk free space check indicates enough available space. The required space is computed as the sum of input sizes. b) The free space check requirement will be removed once the amount of free space is greater than the size reserved by in progress compactions when the first error occured c) If the out of space error is a hard error, a background thread in SFM will poll for sufficient headroom before triggering the recovery of the database and putting it in write-only mode. The headroom is calculated as the sum of the write_buffer_size of all the DB instances associated with the SFM 4. EventListener callbacks will be called at the start and completion of automatic recovery. Users can disable the auto recov ery in the start callback, and later initiate it manually by calling DB::Resume() Todo: 1. More extensive testing 2. Add disk full condition to db_stress (follow-on PR) Pull Request resolved: https://github.com/facebook/rocksdb/pull/4164 Differential Revision: D9846378 Pulled By: anand1976 fbshipit-source-id: 80ea875dbd7f00205e19c82215ff6e37da10da4a
2018-09-15 20:36:19 +00:00
if (*auto_recovery) {
listener->OnErrorRecoveryBegin(reason, *bg_error, auto_recovery);
}
}
db_mutex->Lock();
}
void EventHelpers::LogAndNotifyTableFileCreationFinished(
EventLogger* event_logger,
const std::vector<std::shared_ptr<EventListener>>& listeners,
const std::string& db_name, const std::string& cf_name,
const std::string& file_path, int job_id, const FileDescriptor& fd,
uint64_t oldest_blob_file_number, const TableProperties& table_properties,
TableFileCreationReason reason, const Status& s,
const std::string& file_checksum,
const std::string& file_checksum_func_name) {
if (s.ok() && event_logger) {
JSONWriter jwriter;
AppendCurrentTime(&jwriter);
jwriter << "cf_name" << cf_name << "job" << job_id << "event"
<< "table_file_creation"
<< "file_number" << fd.GetNumber() << "file_size"
Fix a race condition in WAL tracking causing DB open failure (#9715) Summary: There is a race condition if WAL tracking in the MANIFEST is enabled in a database that disables 2PC. The race condition is between two background flush threads trying to install flush results to the MANIFEST. Consider an example database with two column families: "default" (cfd0) and "cf1" (cfd1). Initially, both column families have one mutable (active) memtable whose data backed by 6.log. 1. Trigger a manual flush for "cf1", creating a 7.log 2. Insert another key to "default", and trigger flush for "default", creating 8.log 3. BgFlushThread1 finishes writing 9.sst 4. BgFlushThread2 finishes writing 10.sst ``` Time BgFlushThread1 BgFlushThread2 | mutex_.Lock() | precompute min_wal_to_keep as 6 | mutex_.Unlock() | mutex_.Lock() | precompute min_wal_to_keep as 6 | join MANIFEST write queue and mutex_.Unlock() | write to MANIFEST | mutex_.Lock() | cfd1->log_number = 7 | Signal bg_flush_2 and mutex_.Unlock() | wake up and mutex_.Lock() | cfd0->log_number = 8 | FindObsoleteFiles() with job_context->log_number == 7 | mutex_.Unlock() | PurgeObsoleteFiles() deletes 6.log V ``` As shown in the above, BgFlushThread2 thinks that the min wal to keep is 6.log because "cf1" has unflushed data in 6.log (cf1.log_number=6). Similarly, BgThread1 thinks that min wal to keep is also 6.log because "default" has unflushed data (default.log_number=6). No WAL deletion will be written to MANIFEST because 6 is equal to `versions_->wals_.min_wal_number_to_keep`, due to https://github.com/facebook/rocksdb/blob/7.1.fb/db/memtable_list.cc#L513:L514. The bg flush thread that finishes last will perform file purging. `job_context.log_number` will be evaluated as 7, i.e. the min wal that contains unflushed data, causing 6.log to be deleted. However, MANIFEST thinks 6.log should still exist. If you close the db at this point, you won't be able to re-open it if `track_and_verify_wal_in_manifest` is true. We must handle the case of multiple bg flush threads, and it is difficult for one bg flush thread to know the correct min wal number until the other bg flush threads have finished committing to the manifest and updated the `cfd::log_number`. To fix this issue, we rename an existing variable `min_log_number_to_keep_2pc` to `min_log_number_to_keep`, and use it to track WAL file deletion in non-2pc mode as well. This variable is updated only 1) during recovery with mutex held, or 2) in the MANIFEST write thread. `min_log_number_to_keep` means RocksDB will delete WALs below it, although there may be WALs above it which are also obsolete. Formally, we will have [min_wal_to_keep, max_obsolete_wal]. During recovery, we make sure that only WALs above max_obsolete_wal are checked and added back to `alive_log_files_`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9715 Test Plan: ``` make check ``` Also ran stress test below (with asan) to make sure it completes successfully. ``` TEST_TMPDIR=/dev/shm/rocksdb OPT=-g ASAN_OPTIONS=disable_coredump=0 \ CRASH_TEST_EXT_ARGS=--compression_type=zstd SKIP_FORMAT_BUCK_CHECKS=1 \ make J=52 -j52 blackbox_asan_crash_test ``` Reviewed By: ltamasi Differential Revision: D34984412 Pulled By: riversand963 fbshipit-source-id: c7b21a8d84751bb55ea79c9f387103d21b231005
2022-03-24 02:41:31 +00:00
<< fd.GetFileSize() << "file_checksum"
<< Slice(file_checksum).ToString(true) << "file_checksum_func_name"
<< file_checksum_func_name << "smallest_seqno" << fd.smallest_seqno
<< "largest_seqno" << fd.largest_seqno;
// table_properties
{
jwriter << "table_properties";
jwriter.StartObject();
// basic properties:
jwriter << "data_size" << table_properties.data_size << "index_size"
<< table_properties.index_size << "index_partitions"
<< table_properties.index_partitions << "top_level_index_size"
<< table_properties.top_level_index_size
<< "index_key_is_user_key"
<< table_properties.index_key_is_user_key
<< "index_value_is_delta_encoded"
<< table_properties.index_value_is_delta_encoded << "filter_size"
<< table_properties.filter_size << "raw_key_size"
<< table_properties.raw_key_size << "raw_average_key_size"
<< SafeDivide(table_properties.raw_key_size,
table_properties.num_entries)
<< "raw_value_size" << table_properties.raw_value_size
<< "raw_average_value_size"
<< SafeDivide(table_properties.raw_value_size,
table_properties.num_entries)
<< "num_data_blocks" << table_properties.num_data_blocks
<< "num_entries" << table_properties.num_entries
<< "num_filter_entries" << table_properties.num_filter_entries
<< "num_deletions" << table_properties.num_deletions
<< "num_merge_operands" << table_properties.num_merge_operands
Fix wrong info log printing for num_range_deletions (#5617) Summary: num_range_deletions printing is wrong in this log line: 2019/07/18-12:59:15.309271 7f869f9ff700 EVENT_LOG_v1 {"time_micros": 1563479955309228, "cf_name": "5", "job": 955, "event": "table_file_creation", "file_number": 34579, "file_size": 2239842, "table_properties": {"data_size": 1988792, "index_size": 3067, "index_partitions": 0, "top_level_index_size": 0, "index_key_is_user_key": 0, "index_value_is_delta_encoded": 1, "filter_size": 170821, "raw_key_size": 1951792, "raw_average_key_size": 16, "raw_value_size": 1731720, "raw_average_value_size": 14, "num_data_blocks": 199, "num_entries": 121987, "num_deletions": 15184, "num_merge_operands": 86512, "num_range_deletions": 86512, "format_version": 0, "fixed_key_len": 0, "filter_policy": "rocksdb.BuiltinBloomFilter", "column_family_name": "5", "column_family_id": 5, "comparator": "leveldb.BytewiseComparator", "merge_operator": "PutOperator", "prefix_extractor_name": "rocksdb.FixedPrefix.7", "property_collectors": "[]", "compression": "ZSTD", "compression_options": "window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; ", "creation_time": 1563479951, "oldest_key_time": 0, "file_creation_time": 1563479954}} It actually prints "num_merge_operands" number. Fix it. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5617 Test Plan: Just build. Differential Revision: D16453110 fbshipit-source-id: fc1024b3cd5650312ed47a1379f0d2cf8b2d8a8f
2019-07-24 02:34:56 +00:00
<< "num_range_deletions" << table_properties.num_range_deletions
<< "format_version" << table_properties.format_version
<< "fixed_key_len" << table_properties.fixed_key_len
<< "filter_policy" << table_properties.filter_policy_name
<< "column_family_name" << table_properties.column_family_name
<< "column_family_id" << table_properties.column_family_id
<< "comparator" << table_properties.comparator_name
<< "user_defined_timestamps_persisted"
<< table_properties.user_defined_timestamps_persisted
<< "key_largest_seqno" << table_properties.key_largest_seqno
<< "merge_operator" << table_properties.merge_operator_name
<< "prefix_extractor_name"
<< table_properties.prefix_extractor_name << "property_collectors"
<< table_properties.property_collectors_names << "compression"
<< table_properties.compression_name << "compression_options"
<< table_properties.compression_options << "creation_time"
<< table_properties.creation_time << "oldest_key_time"
<< table_properties.oldest_key_time << "file_creation_time"
<< table_properties.file_creation_time
<< "slow_compression_estimated_data_size"
<< table_properties.slow_compression_estimated_data_size
<< "fast_compression_estimated_data_size"
<< table_properties.fast_compression_estimated_data_size
<< "db_id" << table_properties.db_id << "db_session_id"
Embed original file number in SST table properties (#8686) Summary: I very recently realized that with https://github.com/facebook/rocksdb/issues/8669 we cannot later add file numbers to external SST files (so that more can share db session ids for better uniqueness properties), because of forward compatibility. We would have a version of RocksDB that assumes session IDs are unique on external SST files and therefore can't really break that invariant in future files. This change adds a table property for "orig_file_number" which is populated by normal SST files and also external SST files generated by SstFileWriter. SstFileWriter now keeps a db_session_id for life of the object and increments its own file numbers for embedding in table properties. (They are arguably "fake" file numbers because these numbers and not embedded in the file name.) While updating block_based_table_builder, I removed several unnecessary fields from Rep, because following the pattern would have created another unnecessary field. This change also updates block_based_table_reader to use this new property when available, which means that for newer SST files, we can determine the stable/original <db_session_id,file_number> unique identifier using just the file contents, not the file name. (It's a bit complicated; detailed comments in block_based_table_reader.) Also added DB host id to properties listing by sst_dump, which could be useful in debugging. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8686 Test Plan: majorly overhauled StableCacheKeys test for this change Reviewed By: zhichao-cao Differential Revision: D30457742 Pulled By: pdillinger fbshipit-source-id: 2e5ae7dddeb94fb9d8eac8a928486aed8b8cd445
2021-08-21 03:39:52 +00:00
<< table_properties.db_session_id << "orig_file_number"
<< table_properties.orig_file_number << "seqno_to_time_mapping";
if (table_properties.seqno_to_time_mapping.empty()) {
jwriter << "N/A";
} else {
SeqnoToTimeMapping tmp;
Fix/cleanup SeqnoToTimeMapping (#12253) Summary: The SeqnoToTimeMapping class (RocksDB internal) used by the preserve_internal_time_seconds / preclude_last_level_data_seconds options was essentially in a prototype state with some significant flaws that would risk biting us some day. This is a big, complicated change because both the implementation and the behavioral requirements of the class needed to be upgraded together. In short, this makes SeqnoToTimeMapping more internally responsible for maintaining good invariants, so that callers don't easily encounter dangerous scenarios. * Some API functions were confusingly named and structured, so I fully refactored the APIs to use clear naming (e.g. `DecodeFrom` and `CopyFromSeqnoRange`), object states, function preconditions, etc. * Previously the object could informally be sorted / compacted or not, and there was limited checking or enforcement on these states. Now there's a well-defined "enforced" state that is consistently checked in debug mode for applicable operations. (I attempted to create a separate "builder" class for unenforced states, but IIRC found that more cumbersome for existing uses than it was worth.) * Previously operations would coalesce data in a way that was better for `GetProximalTimeBeforeSeqno` than for `GetProximalSeqnoBeforeTime` which is odd because the latter is the only one used by DB code currently (what is the seqno cut-off for data definitely older than this given time?). This is now reversed to consistently favor `GetProximalSeqnoBeforeTime`, with that logic concentrated in one place: `SeqnoToTimeMapping::SeqnoTimePair::Merge()`. Unfortunately, a lot of unit test logic was specifically testing the old, suboptimal behavior. * Previously, the natural behavior of SeqnoToTimeMapping was to THROW AWAY data needed to get reasonable answers to the important `GetProximalSeqnoBeforeTime` queries. This is because SeqnoToTimeMapping only had a FIFO policy for staying within the entry capacity (except in aggregate+sort+serialize mode). If the DB wasn't extremely careful to avoid gathering too many time mappings, it could lose track of where the seqno cutoff was for cold data (`GetProximalSeqnoBeforeTime()` returning 0) and preventing all further data migration to the cold tier--until time passes etc. for mappings to catch up with FIFO purging of them. (The problem is not so acute because SST files contain relevant snapshots of the mappings, but the problem would apply to long-lived memtables.) * Now the SeqnoToTimeMapping class has fully-integrated smarts for keeping a sufficiently complete history, within capacity limits, to give good answers to `GetProximalSeqnoBeforeTime` queries. * Fixes old `// FIXME: be smarter about how we erase to avoid data falling off the front prematurely.` * Fix an apparent bug in how entries are selected for storing into SST files. Previously, it only selected entries within the seqno range of the file, but that would easily leave a gap at the beginning of the timeline for data in the file for the purposes of answering GetProximalXXX queries with reasonable accuracy. This could probably lead to the same problem discussed above in naively throwing away entries in FIFO order in the old SeqnoToTimeMapping. The updated testing of GetProximalSeqnoBeforeTime in BasicSeqnoToTimeMapping relies on the fixed behavior. * Fix a potential compaction CPU efficiency/scaling issue in which each compaction output file would iterate over and sort all seqno-to-time mappings from all compaction input files. Now we distill the input file entries to a constant size before processing each compaction output file. Intended follow-up (me or others): * Expand some direct testing of SeqnoToTimeMapping APIs. Here I've focused on updating existing tests to make sense. * There are likely more gaps in availability of needed SeqnoToTimeMapping data when the DB shuts down and is restarted, at least with WAL. * The data tracked in the DB could be kept more accurate and limited if it used the oldest seqno of unflushed data. This might require some more API refactoring. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12253 Test Plan: unit tests updated Reviewed By: jowlyzhang Differential Revision: D52913733 Pulled By: pdillinger fbshipit-source-id: 020737fcbbe6212f6701191a6ab86565054c9593
2024-01-20 05:50:38 +00:00
Status status = tmp.DecodeFrom(table_properties.seqno_to_time_mapping);
if (status.ok()) {
jwriter << tmp.ToHumanString();
} else {
jwriter << "Invalid";
}
}
// user collected properties
for (const auto& prop : table_properties.readable_properties) {
jwriter << prop.first << prop.second;
}
jwriter.EndObject();
}
if (oldest_blob_file_number != kInvalidBlobFileNumber) {
jwriter << "oldest_blob_file_number" << oldest_blob_file_number;
}
jwriter.EndObject();
event_logger->Log(jwriter);
}
if (listeners.empty()) {
return;
}
TableFileCreationInfo info;
info.db_name = db_name;
info.cf_name = cf_name;
info.file_path = file_path;
info.file_size = fd.file_size;
info.job_id = job_id;
info.table_properties = table_properties;
info.reason = reason;
info.status = s;
info.file_checksum = file_checksum;
info.file_checksum_func_name = file_checksum_func_name;
for (auto& listener : listeners) {
listener->OnTableFileCreated(info);
}
info.status.PermitUncheckedError();
}
void EventHelpers::LogAndNotifyTableFileDeletion(
EventLogger* event_logger, int job_id, uint64_t file_number,
const std::string& file_path, const Status& status,
const std::string& dbname,
const std::vector<std::shared_ptr<EventListener>>& listeners) {
JSONWriter jwriter;
AppendCurrentTime(&jwriter);
jwriter << "job" << job_id << "event"
<< "table_file_deletion"
<< "file_number" << file_number;
if (!status.ok()) {
jwriter << "status" << status.ToString();
}
jwriter.EndObject();
event_logger->Log(jwriter);
if (listeners.empty()) {
return;
}
TableFileDeletionInfo info;
info.db_name = dbname;
info.job_id = job_id;
info.file_path = file_path;
info.status = status;
for (auto& listener : listeners) {
listener->OnTableFileDeleted(info);
}
info.status.PermitUncheckedError();
}
void EventHelpers::NotifyOnErrorRecoveryEnd(
Auto recovery from out of space errors (#4164) Summary: This commit implements automatic recovery from a Status::NoSpace() error during background operations such as write callback, flush and compaction. The broad design is as follows - 1. Compaction errors are treated as soft errors and don't put the database in read-only mode. A compaction is delayed until enough free disk space is available to accomodate the compaction outputs, which is estimated based on the input size. This means that users can continue to write, and we rely on the WriteController to delay or stop writes if the compaction debt becomes too high due to persistent low disk space condition 2. Errors during write callback and flush are treated as hard errors, i.e the database is put in read-only mode and goes back to read-write only fater certain recovery actions are taken. 3. Both types of recovery rely on the SstFileManagerImpl to poll for sufficient disk space. We assume that there is a 1-1 mapping between an SFM and the underlying OS storage container. For cases where multiple DBs are hosted on a single storage container, the user is expected to allocate a single SFM instance and use the same one for all the DBs. If no SFM is specified by the user, DBImpl::Open() will allocate one, but this will be one per DB and each DB will recover independently. The recovery implemented by SFM is as follows - a) On the first occurance of an out of space error during compaction, subsequent compactions will be delayed until the disk free space check indicates enough available space. The required space is computed as the sum of input sizes. b) The free space check requirement will be removed once the amount of free space is greater than the size reserved by in progress compactions when the first error occured c) If the out of space error is a hard error, a background thread in SFM will poll for sufficient headroom before triggering the recovery of the database and putting it in write-only mode. The headroom is calculated as the sum of the write_buffer_size of all the DB instances associated with the SFM 4. EventListener callbacks will be called at the start and completion of automatic recovery. Users can disable the auto recov ery in the start callback, and later initiate it manually by calling DB::Resume() Todo: 1. More extensive testing 2. Add disk full condition to db_stress (follow-on PR) Pull Request resolved: https://github.com/facebook/rocksdb/pull/4164 Differential Revision: D9846378 Pulled By: anand1976 fbshipit-source-id: 80ea875dbd7f00205e19c82215ff6e37da10da4a
2018-09-15 20:36:19 +00:00
const std::vector<std::shared_ptr<EventListener>>& listeners,
const Status& old_bg_error, const Status& new_bg_error,
InstrumentedMutex* db_mutex) {
if (!listeners.empty()) {
db_mutex->AssertHeld();
// Make copies before releasing mutex to avoid race.
Status old_bg_error_cp = old_bg_error;
Status new_bg_error_cp = new_bg_error;
// release lock while notifying events
db_mutex->Unlock();
Fix for RecoverFromRetryableBGIOError starting with recovery_in_prog_ false (#11991) Summary: cbi42 helped investigation and found a potential scenario where `RecoverFromRetryableBGIOError()` may start with `recovery_in_prog_ ` set as false. (and other booleans like `bg_error_` and `soft_error_no_bg_work_`) **Thread 1** - `StartRecoverFromRetryableBGIOError()`): (mutex held) sets `recovery_in_prog_ = true` **Thread 1's `recovery_thread_`** - (waits for mutex and acquires it) - `RecoverFromRetryableBGIOError()` -> `ResumeImpl()` -> `ClearBGError()`: sets `recovery_in_prog_ = false` - `ClearBGError()` -> `NotifyOnErrorRecoveryEnd()`: releases `mutex` **Thread 2** - `StartRecoverFromRetryableBGIOError()`): (mutex held) sets `recovery_in_prog_ = true` - Waits for Thread 1 (`recovery_thread_`) to finish **Thread 1's `recovery_thread_`** - re-lock mutex in `NotifyOnErrorRecoveryEnd()` - Still inside `RecoverFromRetryableBGIOError()`: sets `recovery_in_prog_ = false` - Done **Thread 2's `recovery_thread_`** - recovery thread started with `recovery_in_prog_` set as `false` # Fix - Remove double-clearing `bg_error_`, `recovery_in_prog_` and other fields after `ResumeImpl()` already returned `OK()`. - Minor typo and linter fixes in `DBErrorHandlingFSTest` Pull Request resolved: https://github.com/facebook/rocksdb/pull/11991 Test Plan: - `DBErrorHandlingFSTest::MultipleRecoveryThreads` added to reproduce the scenario. - Adding `assert(recovery_in_prog_);` at the start of `ErrorHandler::RecoverFromRetryableBGIOError()` fails the test without the fix and succeeds with the fix as expected. Reviewed By: cbi42 Differential Revision: D50506113 Pulled By: jaykorean fbshipit-source-id: 6dabe01e9ecd3fc50bbe9019587f2f4858bed9c6
2023-10-31 23:13:36 +00:00
TEST_SYNC_POINT("NotifyOnErrorRecoveryEnd:MutexUnlocked:1");
TEST_SYNC_POINT("NotifyOnErrorRecoveryEnd:MutexUnlocked:2");
for (auto& listener : listeners) {
BackgroundErrorRecoveryInfo info;
info.old_bg_error = old_bg_error_cp;
info.new_bg_error = new_bg_error_cp;
listener->OnErrorRecoveryCompleted(old_bg_error_cp);
listener->OnErrorRecoveryEnd(info);
info.old_bg_error.PermitUncheckedError();
info.new_bg_error.PermitUncheckedError();
}
db_mutex->Lock();
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: https://github.com/facebook/rocksdb/pull/11865 Test Plan: * Added repro unit tests. Reviewed By: anand1976 Differential Revision: D49484922 Pulled By: cbi42 fbshipit-source-id: 25b536c08f4e02e7f1d0f86571663737d2b5d53d
2023-09-21 22:31:29 +00:00
} else {
old_bg_error.PermitUncheckedError();
Auto recovery from out of space errors (#4164) Summary: This commit implements automatic recovery from a Status::NoSpace() error during background operations such as write callback, flush and compaction. The broad design is as follows - 1. Compaction errors are treated as soft errors and don't put the database in read-only mode. A compaction is delayed until enough free disk space is available to accomodate the compaction outputs, which is estimated based on the input size. This means that users can continue to write, and we rely on the WriteController to delay or stop writes if the compaction debt becomes too high due to persistent low disk space condition 2. Errors during write callback and flush are treated as hard errors, i.e the database is put in read-only mode and goes back to read-write only fater certain recovery actions are taken. 3. Both types of recovery rely on the SstFileManagerImpl to poll for sufficient disk space. We assume that there is a 1-1 mapping between an SFM and the underlying OS storage container. For cases where multiple DBs are hosted on a single storage container, the user is expected to allocate a single SFM instance and use the same one for all the DBs. If no SFM is specified by the user, DBImpl::Open() will allocate one, but this will be one per DB and each DB will recover independently. The recovery implemented by SFM is as follows - a) On the first occurance of an out of space error during compaction, subsequent compactions will be delayed until the disk free space check indicates enough available space. The required space is computed as the sum of input sizes. b) The free space check requirement will be removed once the amount of free space is greater than the size reserved by in progress compactions when the first error occured c) If the out of space error is a hard error, a background thread in SFM will poll for sufficient headroom before triggering the recovery of the database and putting it in write-only mode. The headroom is calculated as the sum of the write_buffer_size of all the DB instances associated with the SFM 4. EventListener callbacks will be called at the start and completion of automatic recovery. Users can disable the auto recov ery in the start callback, and later initiate it manually by calling DB::Resume() Todo: 1. More extensive testing 2. Add disk full condition to db_stress (follow-on PR) Pull Request resolved: https://github.com/facebook/rocksdb/pull/4164 Differential Revision: D9846378 Pulled By: anand1976 fbshipit-source-id: 80ea875dbd7f00205e19c82215ff6e37da10da4a
2018-09-15 20:36:19 +00:00
}
}
void EventHelpers::NotifyBlobFileCreationStarted(
const std::vector<std::shared_ptr<EventListener>>& listeners,
const std::string& db_name, const std::string& cf_name,
const std::string& file_path, int job_id,
BlobFileCreationReason creation_reason) {
if (listeners.empty()) {
return;
}
BlobFileCreationBriefInfo info(db_name, cf_name, file_path, job_id,
creation_reason);
for (const auto& listener : listeners) {
listener->OnBlobFileCreationStarted(info);
}
}
void EventHelpers::LogAndNotifyBlobFileCreationFinished(
EventLogger* event_logger,
const std::vector<std::shared_ptr<EventListener>>& listeners,
const std::string& db_name, const std::string& cf_name,
const std::string& file_path, int job_id, uint64_t file_number,
BlobFileCreationReason creation_reason, const Status& s,
const std::string& file_checksum,
const std::string& file_checksum_func_name, uint64_t total_blob_count,
uint64_t total_blob_bytes) {
if (s.ok() && event_logger) {
JSONWriter jwriter;
AppendCurrentTime(&jwriter);
jwriter << "cf_name" << cf_name << "job" << job_id << "event"
<< "blob_file_creation"
<< "file_number" << file_number << "total_blob_count"
<< total_blob_count << "total_blob_bytes" << total_blob_bytes
<< "file_checksum" << file_checksum << "file_checksum_func_name"
<< file_checksum_func_name << "status" << s.ToString();
jwriter.EndObject();
event_logger->Log(jwriter);
}
if (listeners.empty()) {
return;
}
BlobFileCreationInfo info(db_name, cf_name, file_path, job_id,
creation_reason, total_blob_count, total_blob_bytes,
s, file_checksum, file_checksum_func_name);
for (const auto& listener : listeners) {
listener->OnBlobFileCreated(info);
}
info.status.PermitUncheckedError();
}
void EventHelpers::LogAndNotifyBlobFileDeletion(
EventLogger* event_logger,
const std::vector<std::shared_ptr<EventListener>>& listeners, int job_id,
uint64_t file_number, const std::string& file_path, const Status& status,
const std::string& dbname) {
if (event_logger) {
JSONWriter jwriter;
AppendCurrentTime(&jwriter);
jwriter << "job" << job_id << "event"
<< "blob_file_deletion"
<< "file_number" << file_number;
if (!status.ok()) {
jwriter << "status" << status.ToString();
}
jwriter.EndObject();
event_logger->Log(jwriter);
}
if (listeners.empty()) {
return;
}
BlobFileDeletionInfo info(dbname, file_path, job_id, status);
for (const auto& listener : listeners) {
listener->OnBlobFileDeleted(info);
}
info.status.PermitUncheckedError();
}
} // namespace ROCKSDB_NAMESPACE