You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/db/db_impl/db_impl_files.cc

811 lines
30 KiB

// 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).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "db/db_impl/db_impl.h"
#include <cinttypes>
#include <set>
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
#include <unordered_set>
#include "db/event_helpers.h"
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
6 years ago
#include "db/memtable_list.h"
#include "file/file_util.h"
#include "file/filename.h"
#include "file/sst_file_manager_impl.h"
#include "util/autovector.h"
namespace ROCKSDB_NAMESPACE {
uint64_t DBImpl::MinLogNumberToKeep() {
if (allow_2pc()) {
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
6 years ago
return versions_->min_log_number_to_keep_2pc();
} else {
return versions_->MinLogNumberWithUnflushedData();
}
}
uint64_t DBImpl::MinObsoleteSstNumberToKeep() {
mutex_.AssertHeld();
if (!pending_outputs_.empty()) {
return *pending_outputs_.begin();
}
return std::numeric_limits<uint64_t>::max();
}
First step towards handling MANIFEST write error (#6949) Summary: This PR provides preliminary support for handling IO error during MANIFEST write. File write/sync is not guaranteed to be atomic. If we encounter an IOError while writing/syncing to the MANIFEST file, we cannot be sure about the state of the MANIFEST file. The version edits may or may not have reached the file. During cleanup, if we delete the newly-generated SST files referenced by the pending version edit(s), but the version edit(s) actually are persistent in the MANIFEST, then next recovery attempt will process the version edits(s) and then fail since the SST files have already been deleted. One approach is to truncate the MANIFEST after write/sync error, so that it is safe to delete the SST files. However, file truncation may not be supported on certain file systems. Therefore, we take the following approach. If an IOError is detected during MANIFEST write/sync, we disable file deletions for the faulty database. Depending on whether the IOError is retryable (set by underlying file system), either RocksDB or application can call `DB::Resume()`, or simply shutdown and restart. During `Resume()`, RocksDB will try to switch to a new MANIFEST and write all existing in-memory version storage in the new file. If this succeeds, then RocksDB may proceed. If all recovery is completed, then file deletions will be re-enabled. Note that multiple threads can call `LogAndApply()` at the same time, though only one of them will be going through the process MANIFEST write, possibly batching the version edits of other threads. When the leading MANIFEST writer finishes, all of the MANIFEST writing threads in this batch will have the same IOError. They will all call `ErrorHandler::SetBGError()` in which file deletion will be disabled. Possible future directions: - Add an `ErrorContext` structure so that it is easier to pass more info to `ErrorHandler`. Currently, as in this example, a new `BackgroundErrorReason` has to be added. Test plan (dev server): make check Pull Request resolved: https://github.com/facebook/rocksdb/pull/6949 Reviewed By: anand1976 Differential Revision: D22026020 Pulled By: riversand963 fbshipit-source-id: f3c68a2ef45d9b505d0d625c7c5e0c88495b91c8
4 years ago
Status DBImpl::DisableFileDeletions() {
InstrumentedMutexLock l(&mutex_);
return DisableFileDeletionsWithLock();
}
Status DBImpl::DisableFileDeletionsWithLock() {
mutex_.AssertHeld();
++disable_delete_obsolete_files_;
if (disable_delete_obsolete_files_ == 1) {
ROCKS_LOG_INFO(immutable_db_options_.info_log, "File Deletions Disabled");
} else {
ROCKS_LOG_WARN(immutable_db_options_.info_log,
"File Deletions Disabled, but already disabled. Counter: %d",
disable_delete_obsolete_files_);
}
return Status::OK();
}
Status DBImpl::EnableFileDeletions(bool force) {
// Job id == 0 means that this is not our background process, but rather
// user thread
JobContext job_context(0);
bool file_deletion_enabled = false;
{
InstrumentedMutexLock l(&mutex_);
if (force) {
// if force, we need to enable file deletions right away
disable_delete_obsolete_files_ = 0;
} else if (disable_delete_obsolete_files_ > 0) {
--disable_delete_obsolete_files_;
}
if (disable_delete_obsolete_files_ == 0) {
file_deletion_enabled = true;
FindObsoleteFiles(&job_context, true);
bg_cv_.SignalAll();
}
}
if (file_deletion_enabled) {
ROCKS_LOG_INFO(immutable_db_options_.info_log, "File Deletions Enabled");
if (job_context.HaveSomethingToDelete()) {
PurgeObsoleteFiles(job_context);
}
} else {
ROCKS_LOG_WARN(immutable_db_options_.info_log,
"File Deletions Enable, but not really enabled. Counter: %d",
disable_delete_obsolete_files_);
}
job_context.Clean();
LogFlush(immutable_db_options_.info_log);
return Status::OK();
}
bool DBImpl::IsFileDeletionsEnabled() const {
return 0 == disable_delete_obsolete_files_;
}
// * Returns the list of live files in 'sst_live' and 'blob_live'.
// If it's doing full scan:
// * Returns the list of all files in the filesystem in
// 'full_scan_candidate_files'.
// Otherwise, gets obsolete files from VersionSet.
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
// mutable_db_options_.delete_obsolete_files_period_micros
// force = true -- force the full scan
void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
bool no_full_scan) {
mutex_.AssertHeld();
// if deletion is disabled, do nothing
if (disable_delete_obsolete_files_ > 0) {
return;
}
bool doing_the_full_scan = false;
// logic for figuring out if we're doing the full scan
if (no_full_scan) {
doing_the_full_scan = false;
} else if (force ||
mutable_db_options_.delete_obsolete_files_period_micros == 0) {
doing_the_full_scan = true;
} else {
const uint64_t now_micros = env_->NowMicros();
if ((delete_obsolete_files_last_run_ +
mutable_db_options_.delete_obsolete_files_period_micros) <
now_micros) {
doing_the_full_scan = true;
delete_obsolete_files_last_run_ = now_micros;
}
}
// don't delete files that might be currently written to from compaction
// threads
// Since job_context->min_pending_output is set, until file scan finishes,
// mutex_ cannot be released. Otherwise, we might see no min_pending_output
// here but later find newer generated unfinalized files while scanning.
job_context->min_pending_output = MinObsoleteSstNumberToKeep();
// Get obsolete files. This function will also update the list of
// pending files in VersionSet().
versions_->GetObsoleteFiles(
&job_context->sst_delete_files, &job_context->blob_delete_files,
&job_context->manifest_delete_files, job_context->min_pending_output);
// Mark the elements in job_context->sst_delete_files and
// job_context->blob_delete_files as "grabbed for purge" so that other threads
// calling FindObsoleteFiles with full_scan=true will not add these files to
// candidate list for purge.
for (const auto& sst_to_del : job_context->sst_delete_files) {
MarkAsGrabbedForPurge(sst_to_del.metadata->fd.GetNumber());
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
}
for (const auto& blob_file : job_context->blob_delete_files) {
MarkAsGrabbedForPurge(blob_file.GetBlobFileNumber());
}
// store the current filenum, lognum, etc
job_context->manifest_file_number = versions_->manifest_file_number();
job_context->pending_manifest_file_number =
versions_->pending_manifest_file_number();
job_context->log_number = MinLogNumberToKeep();
job_context->prev_log_number = versions_->prev_log_number();
versions_->AddLiveFiles(&job_context->sst_live, &job_context->blob_live);
if (doing_the_full_scan) {
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(),
dbname_);
std::set<std::string> paths;
for (size_t path_id = 0; path_id < immutable_db_options_.db_paths.size();
path_id++) {
paths.insert(immutable_db_options_.db_paths[path_id].path);
}
// Note that if cf_paths is not specified in the ColumnFamilyOptions
// of a particular column family, we use db_paths as the cf_paths
// setting. Hence, there can be multiple duplicates of files from db_paths
// in the following code. The duplicate are removed while identifying
// unique files in PurgeObsoleteFiles.
for (auto cfd : *versions_->GetColumnFamilySet()) {
for (size_t path_id = 0; path_id < cfd->ioptions()->cf_paths.size();
path_id++) {
auto& path = cfd->ioptions()->cf_paths[path_id].path;
if (paths.find(path) == paths.end()) {
paths.insert(path);
}
}
}
for (auto& path : paths) {
// set of all files in the directory. We'll exclude files that are still
// alive in the subsequent processings.
std::vector<std::string> files;
env_->GetChildren(path, &files).PermitUncheckedError(); // Ignore errors
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
for (const std::string& file : files) {
uint64_t number;
FileType type;
// 1. If we cannot parse the file name, we skip;
// 2. If the file with file_number equals number has already been
// grabbed for purge by another compaction job, or it has already been
// schedule for purge, we also skip it if we
// are doing full scan in order to avoid double deletion of the same
// file under race conditions. See
// https://github.com/facebook/rocksdb/issues/3573
if (!ParseFileName(file, &number, info_log_prefix.prefix, &type) ||
!ShouldPurge(number)) {
continue;
}
// TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
job_context->full_scan_candidate_files.emplace_back("/" + file, path);
}
}
// Add log files in wal_dir
if (immutable_db_options_.wal_dir != dbname_) {
std::vector<std::string> log_files;
env_->GetChildren(immutable_db_options_.wal_dir,
&log_files); // Ignore errors
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
for (const std::string& log_file : log_files) {
job_context->full_scan_candidate_files.emplace_back(
log_file, immutable_db_options_.wal_dir);
}
}
// Add info log files in db_log_dir
if (!immutable_db_options_.db_log_dir.empty() &&
immutable_db_options_.db_log_dir != dbname_) {
std::vector<std::string> info_log_files;
// Ignore errors
env_->GetChildren(immutable_db_options_.db_log_dir, &info_log_files);
for (std::string& log_file : info_log_files) {
job_context->full_scan_candidate_files.emplace_back(
log_file, immutable_db_options_.db_log_dir);
}
}
}
// logs_ is empty when called during recovery, in which case there can't yet
// be any tracked obsolete logs
if (!alive_log_files_.empty() && !logs_.empty()) {
uint64_t min_log_number = job_context->log_number;
size_t num_alive_log_files = alive_log_files_.size();
// find newly obsoleted log files
while (alive_log_files_.begin()->number < min_log_number) {
auto& earliest = *alive_log_files_.begin();
if (immutable_db_options_.recycle_log_file_num >
log_recycle_files_.size()) {
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"adding log %" PRIu64 " to recycle list\n",
earliest.number);
log_recycle_files_.push_back(earliest.number);
} else {
job_context->log_delete_files.push_back(earliest.number);
}
if (job_context->size_log_to_delete == 0) {
job_context->prev_total_log_size = total_log_size_;
job_context->num_alive_log_files = num_alive_log_files;
}
job_context->size_log_to_delete += earliest.size;
total_log_size_ -= earliest.size;
if (two_write_queues_) {
log_write_mutex_.Lock();
}
alive_log_files_.pop_front();
if (two_write_queues_) {
log_write_mutex_.Unlock();
}
// Current log should always stay alive since it can't have
// number < MinLogNumber().
assert(alive_log_files_.size());
}
while (!logs_.empty() && logs_.front().number < min_log_number) {
auto& log = logs_.front();
if (log.getting_synced) {
log_sync_cv_.Wait();
// logs_ could have changed while we were waiting.
continue;
}
logs_to_free_.push_back(log.ReleaseWriter());
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
7 years ago
{
InstrumentedMutexLock wl(&log_write_mutex_);
logs_.pop_front();
}
}
// Current log cannot be obsolete.
assert(!logs_.empty());
}
// We're just cleaning up for DB::Write().
assert(job_context->logs_to_free.empty());
job_context->logs_to_free = logs_to_free_;
job_context->log_recycle_files.assign(log_recycle_files_.begin(),
log_recycle_files_.end());
if (job_context->HaveSomethingToDelete()) {
++pending_purge_obsolete_files_;
}
logs_to_free_.clear();
}
namespace {
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
const JobContext::CandidateFileInfo& second) {
if (first.file_name > second.file_name) {
return true;
} else if (first.file_name < second.file_name) {
return false;
} else {
return (first.file_path > second.file_path);
}
}
}; // namespace
// Delete obsolete files and log status and information of file deletion
void DBImpl::DeleteObsoleteFileImpl(int job_id, const std::string& fname,
const std::string& path_to_sync,
FileType type, uint64_t number) {
TEST_SYNC_POINT_CALLBACK("DBImpl::DeleteObsoleteFileImpl::BeforeDeletion",
const_cast<std::string*>(&fname));
Status file_deletion_status;
if (type == kTableFile || type == kBlobFile || type == kLogFile) {
file_deletion_status =
DeleteDBFile(&immutable_db_options_, fname, path_to_sync,
/*force_bg=*/false, /*force_fg=*/!wal_in_db_path_);
} else {
file_deletion_status = env_->DeleteFile(fname);
}
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
TEST_SYNC_POINT_CALLBACK("DBImpl::DeleteObsoleteFileImpl:AfterDeletion",
&file_deletion_status);
if (file_deletion_status.ok()) {
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
"[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", job_id,
fname.c_str(), type, number,
file_deletion_status.ToString().c_str());
} else if (env_->FileExists(fname).IsNotFound()) {
ROCKS_LOG_INFO(
immutable_db_options_.info_log,
"[JOB %d] Tried to delete a non-existing file %s type=%d #%" PRIu64
" -- %s\n",
job_id, fname.c_str(), type, number,
file_deletion_status.ToString().c_str());
} else {
ROCKS_LOG_ERROR(immutable_db_options_.info_log,
"[JOB %d] Failed to delete %s type=%d #%" PRIu64 " -- %s\n",
job_id, fname.c_str(), type, number,
file_deletion_status.ToString().c_str());
}
if (type == kTableFile) {
EventHelpers::LogAndNotifyTableFileDeletion(
&event_logger_, job_id, number, fname, file_deletion_status, GetName(),
immutable_db_options_.listeners);
}
}
// Diffs the files listed in filenames and those that do not
// belong to live files are possibly removed. Also, removes all the
// files in sst_delete_files and log_delete_files.
// It is not necessary to hold the mutex when invoking this method.
void DBImpl::PurgeObsoleteFiles(JobContext& state, bool schedule_only) {
TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:Begin");
// we'd better have sth to delete
assert(state.HaveSomethingToDelete());
// FindObsoleteFiles() should've populated this so nonzero
assert(state.manifest_file_number != 0);
// Now, convert lists to unordered sets, WITHOUT mutex held; set is slow.
std::unordered_set<uint64_t> sst_live_set(state.sst_live.begin(),
state.sst_live.end());
std::unordered_set<uint64_t> blob_live_set(state.blob_live.begin(),
state.blob_live.end());
std::unordered_set<uint64_t> log_recycle_files_set(
state.log_recycle_files.begin(), state.log_recycle_files.end());
auto candidate_files = state.full_scan_candidate_files;
candidate_files.reserve(
candidate_files.size() + state.sst_delete_files.size() +
state.blob_delete_files.size() + state.log_delete_files.size() +
state.manifest_delete_files.size());
// We may ignore the dbname when generating the file names.
for (auto& file : state.sst_delete_files) {
candidate_files.emplace_back(
New API to get all merge operands for a Key (#5604) Summary: This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases: 1. Update subset of columns and read subset of columns - Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU. 2. Updating very few attributes in a value which is a JSON-like document - Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge. ---------------------------------------------------------------------------------------------------- API : Status GetMergeOperands( const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* merge_operands, GetMergeOperandsOptions* get_merge_operands_options, int* number_of_operands) Example usage : int size = 100; int number_of_operands = 0; std::vector<PinnableSlice> values(size); GetMergeOperandsOptions merge_operands_info; db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands); Description : Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion. merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604 Test Plan: Added unit test and perf test in db_bench that can be run using the command: ./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist Differential Revision: D16657366 Pulled By: vjnadimpalli fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
5 years ago
MakeTableFileName(file.metadata->fd.GetNumber()), file.path);
if (file.metadata->table_reader_handle) {
table_cache_->Release(file.metadata->table_reader_handle);
}
file.DeleteMetadata();
}
for (const auto& blob_file : state.blob_delete_files) {
candidate_files.emplace_back(BlobFileName(blob_file.GetBlobFileNumber()),
blob_file.GetPath());
}
for (auto file_num : state.log_delete_files) {
if (file_num > 0) {
candidate_files.emplace_back(LogFileName(file_num),
immutable_db_options_.wal_dir);
}
}
for (const auto& filename : state.manifest_delete_files) {
candidate_files.emplace_back(filename, dbname_);
}
// dedup state.candidate_files so we don't try to delete the same
// file twice
std::sort(candidate_files.begin(), candidate_files.end(),
CompareCandidateFile);
candidate_files.erase(
std::unique(candidate_files.begin(), candidate_files.end()),
candidate_files.end());
if (state.prev_total_log_size > 0) {
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"[JOB %d] Try to delete WAL files size %" PRIu64
", prev total WAL file size %" PRIu64
", number of live WAL files %" ROCKSDB_PRIszt ".\n",
state.job_id, state.size_log_to_delete,
state.prev_total_log_size, state.num_alive_log_files);
}
std::vector<std::string> old_info_log_files;
InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(),
dbname_);
// File numbers of most recent two OPTIONS file in candidate_files (found in
// previos FindObsoleteFiles(full_scan=true))
// At this point, there must not be any duplicate file numbers in
// candidate_files.
uint64_t optsfile_num1 = std::numeric_limits<uint64_t>::min();
uint64_t optsfile_num2 = std::numeric_limits<uint64_t>::min();
for (const auto& candidate_file : candidate_files) {
const std::string& fname = candidate_file.file_name;
uint64_t number;
FileType type;
if (!ParseFileName(fname, &number, info_log_prefix.prefix, &type) ||
type != kOptionsFile) {
continue;
}
if (number > optsfile_num1) {
optsfile_num2 = optsfile_num1;
optsfile_num1 = number;
} else if (number > optsfile_num2) {
optsfile_num2 = number;
}
}
// Close WALs before trying to delete them.
for (const auto w : state.logs_to_free) {
// TODO: maybe check the return value of Close.
auto s = w->Close();
s.PermitUncheckedError();
}
bool own_files = OwnTablesAndLogs();
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
std::unordered_set<uint64_t> files_to_del;
for (const auto& candidate_file : candidate_files) {
const std::string& to_delete = candidate_file.file_name;
uint64_t number;
FileType type;
// Ignore file if we cannot recognize it.
if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
continue;
}
bool keep = true;
switch (type) {
case kLogFile:
keep = ((number >= state.log_number) ||
(number == state.prev_log_number) ||
(log_recycle_files_set.find(number) !=
log_recycle_files_set.end()));
break;
case kDescriptorFile:
// Keep my manifest file, and any newer incarnations'
// (can happen during manifest roll)
keep = (number >= state.manifest_file_number);
break;
case kTableFile:
// If the second condition is not there, this makes
// DontDeletePendingOutputs fail
keep = (sst_live_set.find(number) != sst_live_set.end()) ||
number >= state.min_pending_output;
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
if (!keep) {
files_to_del.insert(number);
}
break;
case kBlobFile:
keep = number >= state.min_pending_output ||
(blob_live_set.find(number) != blob_live_set.end());
if (!keep) {
files_to_del.insert(number);
}
break;
case kTempFile:
// Any temp files that are currently being written to must
// be recorded in pending_outputs_, which is inserted into "live".
// Also, SetCurrentFile creates a temp file when writing out new
// manifest, which is equal to state.pending_manifest_file_number. We
// should not delete that file
//
// TODO(yhchiang): carefully modify the third condition to safely
// remove the temp options files.
keep = (sst_live_set.find(number) != sst_live_set.end()) ||
(blob_live_set.find(number) != blob_live_set.end()) ||
(number == state.pending_manifest_file_number) ||
(to_delete.find(kOptionsFileNamePrefix) != std::string::npos);
break;
case kInfoLogFile:
keep = true;
if (number != 0) {
old_info_log_files.push_back(to_delete);
}
break;
case kOptionsFile:
keep = (number >= optsfile_num2);
TEST_SYNC_POINT_CALLBACK(
"DBImpl::PurgeObsoleteFiles:CheckOptionsFiles:1",
reinterpret_cast<void*>(&number));
TEST_SYNC_POINT_CALLBACK(
"DBImpl::PurgeObsoleteFiles:CheckOptionsFiles:2",
reinterpret_cast<void*>(&keep));
break;
case kCurrentFile:
case kDBLockFile:
case kIdentityFile:
case kMetaDatabase:
keep = true;
break;
}
if (keep) {
continue;
}
std::string fname;
std::string dir_to_sync;
if (type == kTableFile) {
// evict from cache
TableCache::Evict(table_cache_.get(), number);
fname = MakeTableFileName(candidate_file.file_path, number);
dir_to_sync = candidate_file.file_path;
} else if (type == kBlobFile) {
fname = BlobFileName(candidate_file.file_path, number);
dir_to_sync = candidate_file.file_path;
} else {
dir_to_sync =
(type == kLogFile) ? immutable_db_options_.wal_dir : dbname_;
fname = dir_to_sync +
((!dir_to_sync.empty() && dir_to_sync.back() == '/') ||
(!to_delete.empty() && to_delete.front() == '/')
? ""
: "/") +
to_delete;
}
#ifndef ROCKSDB_LITE
if (type == kLogFile && (immutable_db_options_.wal_ttl_seconds > 0 ||
immutable_db_options_.wal_size_limit_mb > 0)) {
wal_manager_.ArchiveWALFile(fname, number);
continue;
}
#endif // !ROCKSDB_LITE
// If I do not own these files, e.g. secondary instance with max_open_files
// = -1, then no need to delete or schedule delete these files since they
// will be removed by their owner, e.g. the primary instance.
if (!own_files) {
continue;
}
if (schedule_only) {
InstrumentedMutexLock guard_lock(&mutex_);
SchedulePendingPurge(fname, dir_to_sync, type, number, state.job_id);
} else {
DeleteObsoleteFileImpl(state.job_id, fname, dir_to_sync, type, number);
}
}
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
{
// After purging obsolete files, remove them from files_grabbed_for_purge_.
InstrumentedMutexLock guard_lock(&mutex_);
autovector<uint64_t> to_be_removed;
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
for (auto fn : files_grabbed_for_purge_) {
if (files_to_del.count(fn) != 0) {
to_be_removed.emplace_back(fn);
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
}
}
for (auto fn : to_be_removed) {
files_grabbed_for_purge_.erase(fn);
}
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
}
// Delete old info log files.
size_t old_info_log_file_count = old_info_log_files.size();
if (old_info_log_file_count != 0 &&
old_info_log_file_count >= immutable_db_options_.keep_log_file_num) {
std::sort(old_info_log_files.begin(), old_info_log_files.end());
size_t end =
old_info_log_file_count - immutable_db_options_.keep_log_file_num;
for (unsigned int i = 0; i <= end; i++) {
std::string& to_delete = old_info_log_files.at(i);
std::string full_path_to_delete =
(immutable_db_options_.db_log_dir.empty()
? dbname_
: immutable_db_options_.db_log_dir) +
"/" + to_delete;
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"[JOB %d] Delete info log file %s\n", state.job_id,
full_path_to_delete.c_str());
Status s = env_->DeleteFile(full_path_to_delete);
if (!s.ok()) {
if (env_->FileExists(full_path_to_delete).IsNotFound()) {
ROCKS_LOG_INFO(
immutable_db_options_.info_log,
"[JOB %d] Tried to delete non-existing info log file %s FAILED "
"-- %s\n",
state.job_id, to_delete.c_str(), s.ToString().c_str());
} else {
ROCKS_LOG_ERROR(immutable_db_options_.info_log,
"[JOB %d] Delete info log file %s FAILED -- %s\n",
state.job_id, to_delete.c_str(),
s.ToString().c_str());
}
}
}
}
#ifndef ROCKSDB_LITE
wal_manager_.PurgeObsoleteWALFiles();
#endif // ROCKSDB_LITE
LogFlush(immutable_db_options_.info_log);
InstrumentedMutexLock l(&mutex_);
--pending_purge_obsolete_files_;
assert(pending_purge_obsolete_files_ >= 0);
if (pending_purge_obsolete_files_ == 0) {
bg_cv_.SignalAll();
}
TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:End");
}
void DBImpl::DeleteObsoleteFiles() {
mutex_.AssertHeld();
JobContext job_context(next_job_id_.fetch_add(1));
FindObsoleteFiles(&job_context, true);
mutex_.Unlock();
if (job_context.HaveSomethingToDelete()) {
PurgeObsoleteFiles(job_context);
}
job_context.Clean();
mutex_.Lock();
}
Fix race condition causing double deletion of ssts Summary: Possible interleaved execution of background compaction thread calling `FindObsoleteFiles (no full scan) / PurgeObsoleteFiles` and user thread calling `FindObsoleteFiles (full scan) / PurgeObsoleteFiles` can lead to race condition on which RocksDB attempts to delete a file twice. The second attempt will fail and return `IO error`. This may occur to other files, but this PR targets sst. Also add a unit test to verify that this PR fixes the issue. The newly added unit test `obsolete_files_test` has a test case for this scenario, implemented in `ObsoleteFilesTest#RaceForObsoleteFileDeletion`. `TestSyncPoint`s are used to coordinate the interleaving the `user_thread` and background compaction thread. They execute as follows ``` timeline user_thread background_compaction thread t1 | FindObsoleteFiles(full_scan=false) t2 | FindObsoleteFiles(full_scan=true) t3 | PurgeObsoleteFiles t4 | PurgeObsoleteFiles V ``` When `user_thread` invokes `FindObsoleteFiles` with full scan, it collects ALL files in RocksDB directory, including the ones that background compaction thread have collected in its job context. Then `user_thread` will see an IO error when trying to delete these files in `PurgeObsoleteFiles` because background compaction thread has already deleted the file in `PurgeObsoleteFiles`. To fix this, we make RocksDB remember which (SST) files have been found by threads after calling `FindObsoleteFiles` (see `DBImpl#files_grabbed_for_purge_`). Therefore, when another thread calls `FindObsoleteFiles` with full scan, it will not collect such files. ajkr could you take a look and comment? Thanks! Closes https://github.com/facebook/rocksdb/pull/3638 Differential Revision: D7384372 Pulled By: riversand963 fbshipit-source-id: 01489516d60012e722ee65a80e1449e589ce26d3
6 years ago
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
6 years ago
uint64_t FindMinPrepLogReferencedByMemTable(
VersionSet* vset, const ColumnFamilyData* cfd_to_flush,
const autovector<MemTable*>& memtables_to_flush) {
uint64_t min_log = 0;
// we must look through the memtables for two phase transactions
// that have been committed but not yet flushed
for (auto loop_cfd : *vset->GetColumnFamilySet()) {
if (loop_cfd->IsDropped() || loop_cfd == cfd_to_flush) {
continue;
}
auto log = loop_cfd->imm()->PrecomputeMinLogContainingPrepSection(
memtables_to_flush);
if (log > 0 && (min_log == 0 || log < min_log)) {
min_log = log;
}
log = loop_cfd->mem()->GetMinLogContainingPrepSection();
if (log > 0 && (min_log == 0 || log < min_log)) {
min_log = log;
}
}
return min_log;
}
uint64_t PrecomputeMinLogNumberToKeep(
VersionSet* vset, const ColumnFamilyData& cfd_to_flush,
autovector<VersionEdit*> edit_list,
const autovector<MemTable*>& memtables_to_flush,
LogsWithPrepTracker* prep_tracker) {
assert(vset != nullptr);
assert(prep_tracker != nullptr);
// Calculate updated min_log_number_to_keep
// Since the function should only be called in 2pc mode, log number in
// the version edit should be sufficient.
// Precompute the min log number containing unflushed data for the column
// family being flushed (`cfd_to_flush`).
uint64_t cf_min_log_number_to_keep = 0;
for (auto& e : edit_list) {
if (e->HasLogNumber()) {
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
6 years ago
cf_min_log_number_to_keep =
std::max(cf_min_log_number_to_keep, e->GetLogNumber());
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
6 years ago
}
}
if (cf_min_log_number_to_keep == 0) {
// No version edit contains information on log number. The log number
// for this column family should stay the same as it is.
cf_min_log_number_to_keep = cfd_to_flush.GetLogNumber();
}
// Get min log number containing unflushed data for other column families.
uint64_t min_log_number_to_keep =
vset->PreComputeMinLogNumberWithUnflushedData(&cfd_to_flush);
if (cf_min_log_number_to_keep != 0) {
min_log_number_to_keep =
std::min(cf_min_log_number_to_keep, min_log_number_to_keep);
}
// if are 2pc we must consider logs containing prepared
// sections of outstanding transactions.
//
// We must check min logs with outstanding prep before we check
// logs references by memtables because a log referenced by the
// first data structure could transition to the second under us.
//
// TODO: iterating over all column families under db mutex.
// should find more optimal solution
auto min_log_in_prep_heap =
prep_tracker->FindMinLogContainingOutstandingPrep();
if (min_log_in_prep_heap != 0 &&
min_log_in_prep_heap < min_log_number_to_keep) {
min_log_number_to_keep = min_log_in_prep_heap;
}
uint64_t min_log_refed_by_mem = FindMinPrepLogReferencedByMemTable(
vset, &cfd_to_flush, memtables_to_flush);
if (min_log_refed_by_mem != 0 &&
min_log_refed_by_mem < min_log_number_to_keep) {
min_log_number_to_keep = min_log_refed_by_mem;
}
return min_log_number_to_keep;
}
Status DBImpl::FinishBestEffortsRecovery() {
mutex_.AssertHeld();
std::vector<std::string> paths;
paths.push_back(NormalizePath(dbname_ + std::string(1, kFilePathSeparator)));
for (const auto& db_path : immutable_db_options_.db_paths) {
paths.push_back(
NormalizePath(db_path.path + std::string(1, kFilePathSeparator)));
}
for (const auto* cfd : *versions_->GetColumnFamilySet()) {
for (const auto& cf_path : cfd->ioptions()->cf_paths) {
paths.push_back(
NormalizePath(cf_path.path + std::string(1, kFilePathSeparator)));
}
}
// Dedup paths
std::sort(paths.begin(), paths.end());
paths.erase(std::unique(paths.begin(), paths.end()), paths.end());
uint64_t next_file_number = versions_->current_next_file_number();
uint64_t largest_file_number = next_file_number;
std::set<std::string> files_to_delete;
for (const auto& path : paths) {
std::vector<std::string> files;
env_->GetChildren(path, &files);
for (const auto& fname : files) {
uint64_t number = 0;
FileType type;
if (!ParseFileName(fname, &number, &type)) {
continue;
}
// path ends with '/' or '\\'
const std::string normalized_fpath = path + fname;
largest_file_number = std::max(largest_file_number, number);
if (type == kTableFile && number >= next_file_number &&
files_to_delete.find(normalized_fpath) == files_to_delete.end()) {
files_to_delete.insert(normalized_fpath);
}
}
}
if (largest_file_number > next_file_number) {
versions_->next_file_number_.store(largest_file_number + 1);
}
VersionEdit edit;
edit.SetNextFile(versions_->next_file_number_.load());
assert(versions_->GetColumnFamilySet());
ColumnFamilyData* default_cfd = versions_->GetColumnFamilySet()->GetDefault();
assert(default_cfd);
// Even if new_descriptor_log is false, we will still switch to a new
// MANIFEST and update CURRENT file, since this is in recovery.
Status s = versions_->LogAndApply(
default_cfd, *default_cfd->GetLatestMutableCFOptions(), &edit, &mutex_,
directories_.GetDbDir(), /*new_descriptor_log*/ false);
if (!s.ok()) {
return s;
}
mutex_.Unlock();
for (const auto& fname : files_to_delete) {
s = env_->DeleteFile(fname);
if (!s.ok()) {
break;
}
}
mutex_.Lock();
return s;
}
} // namespace ROCKSDB_NAMESPACE