rocksdb/db/db_impl/db_impl_open.cc

1993 lines
76 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).
//
// 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 <cinttypes>
#include "db/builder.h"
#include "db/db_impl/db_impl.h"
#include "db/error_handler.h"
#include "db/periodic_work_scheduler.h"
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
#include "env/composite_env_wrapper.h"
Add (Live)FileStorageInfo API (#8968) Summary: New classes FileStorageInfo and LiveFileStorageInfo and 'experimental' function DB::GetLiveFilesStorageInfo, which is intended to largely replace several fragmented DB functions needed to create checkpoints and backups. This function is now used to create checkpoints and backups, because it fixes many (probably not all) of the prior complexities of checkpoint not having atomic access to DB metadata. This also ensures strong functional test coverage of the new API. Specifically, much of the old CheckpointImpl::CreateCustomCheckpoint has been migrated to and updated in DBImpl::GetLiveFilesStorageInfo, with the former now calling the latter. Also, the class FileStorageInfo in metadata.h compatibly replaces BackupFileInfo and serves as a new base class for SstFileMetaData. Some old fields of SstFileMetaData are still provided (for now) but deprecated. Although FileStorageInfo::directory is accurate when using db_paths and/or cf_paths, these have never been supported by Checkpoint nor BackupEngine and still are not. This change does now detect these cases and return NotSupported when appropriate. (More work needed for support.) Somehow this change broke ProgressCallbackDuringBackup, but the progress_callback logic was dubious to begin with because it would call the callback based on copy buffer size, not size actually copied. Logic and test updated to track size actually copied per-thread. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8968 Test Plan: tests updated. DB::GetLiveFilesStorageInfo mostly tested by use in CheckpointImpl. DBTest.SnapshotFiles updated to also test GetLiveFilesStorageInfo, including reading the data after DB close. Added CheckpointTest.CheckpointWithDbPath (NotSupported). Reviewed By: siying Differential Revision: D31242045 Pulled By: pdillinger fbshipit-source-id: b183d1ce9799e220daaefd6b3b5365d98de676c0
2021-10-16 19:03:19 +02:00
#include "file/filename.h"
#include "file/read_write_util.h"
#include "file/sst_file_manager_impl.h"
#include "file/writable_file_writer.h"
#include "logging/logging.h"
#include "monitoring/persistent_stats_history.h"
#include "options/options_helper.h"
#include "rocksdb/table.h"
#include "rocksdb/wal_filter.h"
#include "test_util/sync_point.h"
#include "util/rate_limiter.h"
namespace ROCKSDB_NAMESPACE {
Make backups openable as read-only DBs (#8142) Summary: A current limitation of backups is that you don't know the exact database state of when the backup was taken. With this new feature, you can at least inspect the backup's DB state without restoring it by opening it as a read-only DB. Rather than add something like OpenAsReadOnlyDB to the BackupEngine API, which would inhibit opening stackable DB implementations read-only (if/when their APIs support it), we instead provide a DB name and Env that can be used to open as a read-only DB. Possible follow-up work: * Add a version of GetBackupInfo for a single backup. * Let CreateNewBackup return the BackupID of the newly-created backup. Implementation details: Refactored ChrootFileSystem to split off new base class RemapFileSystem, which allows more general remapping of files. We use this base class to implement BackupEngineImpl::RemapSharedFileSystem. To minimize API impact, I decided to just add these fields `name_for_open` and `env_for_open` to those set by GetBackupInfo when include_file_details=true. Creating the RemapSharedFileSystem adds a bit to the memory consumption, perhaps unnecessarily in some cases, but this has been mitigated by (a) only initialize the RemapSharedFileSystem lazily when GetBackupInfo with include_file_details=true is called, and (b) using the existing `shared_ptr<FileInfo>` objects to hold most of the mapping data. To enhance API safety, RemapSharedFileSystem is wrapped by new ReadOnlyFileSystem which rejects any attempts to write. This uncovered a couple of places in which DB::OpenForReadOnly would write to the filesystem, so I fixed these. Added a release note because this affects logging. Additional minor refactoring in backupable_db.cc to support the new functionality. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8142 Test Plan: new test (run with ASAN and UBSAN), added to stress test and ran it for a while with amplified backup_one_in Reviewed By: ajkr Differential Revision: D27535408 Pulled By: pdillinger fbshipit-source-id: 04666d310aa0261ef6b2385c43ca793ce1dfd148
2021-04-06 23:36:45 +02:00
Options SanitizeOptions(const std::string& dbname, const Options& src,
bool read_only) {
auto db_options = SanitizeOptions(dbname, DBOptions(src), read_only);
ImmutableDBOptions immutable_db_options(db_options);
auto cf_options =
SanitizeOptions(immutable_db_options, ColumnFamilyOptions(src));
return Options(db_options, cf_options);
}
Make backups openable as read-only DBs (#8142) Summary: A current limitation of backups is that you don't know the exact database state of when the backup was taken. With this new feature, you can at least inspect the backup's DB state without restoring it by opening it as a read-only DB. Rather than add something like OpenAsReadOnlyDB to the BackupEngine API, which would inhibit opening stackable DB implementations read-only (if/when their APIs support it), we instead provide a DB name and Env that can be used to open as a read-only DB. Possible follow-up work: * Add a version of GetBackupInfo for a single backup. * Let CreateNewBackup return the BackupID of the newly-created backup. Implementation details: Refactored ChrootFileSystem to split off new base class RemapFileSystem, which allows more general remapping of files. We use this base class to implement BackupEngineImpl::RemapSharedFileSystem. To minimize API impact, I decided to just add these fields `name_for_open` and `env_for_open` to those set by GetBackupInfo when include_file_details=true. Creating the RemapSharedFileSystem adds a bit to the memory consumption, perhaps unnecessarily in some cases, but this has been mitigated by (a) only initialize the RemapSharedFileSystem lazily when GetBackupInfo with include_file_details=true is called, and (b) using the existing `shared_ptr<FileInfo>` objects to hold most of the mapping data. To enhance API safety, RemapSharedFileSystem is wrapped by new ReadOnlyFileSystem which rejects any attempts to write. This uncovered a couple of places in which DB::OpenForReadOnly would write to the filesystem, so I fixed these. Added a release note because this affects logging. Additional minor refactoring in backupable_db.cc to support the new functionality. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8142 Test Plan: new test (run with ASAN and UBSAN), added to stress test and ran it for a while with amplified backup_one_in Reviewed By: ajkr Differential Revision: D27535408 Pulled By: pdillinger fbshipit-source-id: 04666d310aa0261ef6b2385c43ca793ce1dfd148
2021-04-06 23:36:45 +02:00
DBOptions SanitizeOptions(const std::string& dbname, const DBOptions& src,
bool read_only) {
DBOptions result(src);
Simplify migration to FileSystem API (#6552) Summary: The current Env/FileSystem API separation has a couple of issues - 1. It requires the user to specify 2 options - ```Options::env``` and ```Options::file_system``` - which means they have to make code changes to benefit from the new APIs. Furthermore, there is a risk of accessing the same APIs in two different ways, through Env in the old way and through FileSystem in the new way. The two may not always match, for example, if env is ```PosixEnv``` and FileSystem is a custom implementation. Any stray RocksDB calls to env will use the ```PosixEnv``` implementation rather than the file_system implementation. 2. There needs to be a simple way for the FileSystem developer to instantiate an Env for backward compatibility purposes. This PR solves the above issues and simplifies the migration in the following ways - 1. Embed a shared_ptr to the ```FileSystem``` in the ```Env```, and remove ```Options::file_system``` as a configurable option. This way, no code changes will be required in application code to benefit from the new API. The default Env constructor uses a ```LegacyFileSystemWrapper``` as the embedded ```FileSystem```. 1a. - This also makes it more robust by ensuring that even if RocksDB has some stray calls to Env APIs rather than FileSystem, they will go through the same object and thus there is no risk of getting out of sync. 2. Provide a ```NewCompositeEnv()``` API that can be used to construct a PosixEnv with a custom FileSystem implementation. This eliminates an indirection to call Env APIs, and relieves the FileSystem developer of the burden of having to implement wrappers for the Env APIs. 3. Add a couple of missing FileSystem APIs - ```SanitizeEnvOptions()``` and ```NewLogger()``` Tests: 1. New unit tests 2. make check and make asan_check Pull Request resolved: https://github.com/facebook/rocksdb/pull/6552 Reviewed By: riversand963 Differential Revision: D20592038 Pulled By: anand1976 fbshipit-source-id: c3801ad4153f96d21d5a3ae26c92ba454d1bf1f7
2020-03-24 05:50:42 +01:00
if (result.env == nullptr) {
result.env = Env::Default();
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
}
// result.max_open_files means an "infinite" open files.
if (result.max_open_files != -1) {
int max_max_open_files = port::GetMaxOpenFiles();
if (max_max_open_files == -1) {
max_max_open_files = 0x400000;
}
ClipToRange(&result.max_open_files, 20, max_max_open_files);
TEST_SYNC_POINT_CALLBACK("SanitizeOptions::AfterChangeMaxOpenFiles",
&result.max_open_files);
}
Make backups openable as read-only DBs (#8142) Summary: A current limitation of backups is that you don't know the exact database state of when the backup was taken. With this new feature, you can at least inspect the backup's DB state without restoring it by opening it as a read-only DB. Rather than add something like OpenAsReadOnlyDB to the BackupEngine API, which would inhibit opening stackable DB implementations read-only (if/when their APIs support it), we instead provide a DB name and Env that can be used to open as a read-only DB. Possible follow-up work: * Add a version of GetBackupInfo for a single backup. * Let CreateNewBackup return the BackupID of the newly-created backup. Implementation details: Refactored ChrootFileSystem to split off new base class RemapFileSystem, which allows more general remapping of files. We use this base class to implement BackupEngineImpl::RemapSharedFileSystem. To minimize API impact, I decided to just add these fields `name_for_open` and `env_for_open` to those set by GetBackupInfo when include_file_details=true. Creating the RemapSharedFileSystem adds a bit to the memory consumption, perhaps unnecessarily in some cases, but this has been mitigated by (a) only initialize the RemapSharedFileSystem lazily when GetBackupInfo with include_file_details=true is called, and (b) using the existing `shared_ptr<FileInfo>` objects to hold most of the mapping data. To enhance API safety, RemapSharedFileSystem is wrapped by new ReadOnlyFileSystem which rejects any attempts to write. This uncovered a couple of places in which DB::OpenForReadOnly would write to the filesystem, so I fixed these. Added a release note because this affects logging. Additional minor refactoring in backupable_db.cc to support the new functionality. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8142 Test Plan: new test (run with ASAN and UBSAN), added to stress test and ran it for a while with amplified backup_one_in Reviewed By: ajkr Differential Revision: D27535408 Pulled By: pdillinger fbshipit-source-id: 04666d310aa0261ef6b2385c43ca793ce1dfd148
2021-04-06 23:36:45 +02:00
if (result.info_log == nullptr && !read_only) {
Status s = CreateLoggerFromOptions(dbname, result, &result.info_log);
if (!s.ok()) {
// No place suitable for logging
result.info_log = nullptr;
}
}
if (!result.write_buffer_manager) {
result.write_buffer_manager.reset(
new WriteBufferManager(result.db_write_buffer_size));
}
auto bg_job_limits = DBImpl::GetBGJobLimits(
result.max_background_flushes, result.max_background_compactions,
result.max_background_jobs, true /* parallelize_compactions */);
result.env->IncBackgroundThreadsIfNeeded(bg_job_limits.max_compactions,
Env::Priority::LOW);
result.env->IncBackgroundThreadsIfNeeded(bg_job_limits.max_flushes,
Env::Priority::HIGH);
if (result.rate_limiter.get() != nullptr) {
if (result.bytes_per_sync == 0) {
result.bytes_per_sync = 1024 * 1024;
}
}
if (result.delayed_write_rate == 0) {
if (result.rate_limiter.get() != nullptr) {
result.delayed_write_rate = result.rate_limiter->GetBytesPerSecond();
}
if (result.delayed_write_rate == 0) {
result.delayed_write_rate = 16 * 1024 * 1024;
}
}
if (result.WAL_ttl_seconds > 0 || result.WAL_size_limit_MB > 0) {
result.recycle_log_file_num = false;
}
if (result.recycle_log_file_num &&
(result.wal_recovery_mode ==
WALRecoveryMode::kTolerateCorruptedTailRecords ||
result.wal_recovery_mode == WALRecoveryMode::kPointInTimeRecovery ||
result.wal_recovery_mode == WALRecoveryMode::kAbsoluteConsistency)) {
// - kTolerateCorruptedTailRecords is inconsistent with recycle log file
// feature. WAL recycling expects recovery success upon encountering a
// corrupt record at the point where new data ends and recycled data
// remains at the tail. However, `kTolerateCorruptedTailRecords` must fail
// upon encountering any such corrupt record, as it cannot differentiate
// between this and a real corruption, which would cause committed updates
// to be truncated -- a violation of the recovery guarantee.
// - kPointInTimeRecovery and kAbsoluteConsistency are incompatible with
// recycle log file feature temporarily due to a bug found introducing a
// hole in the recovered data
// (https://github.com/facebook/rocksdb/pull/7252#issuecomment-673766236).
// Besides this bug, we believe the features are fundamentally compatible.
result.recycle_log_file_num = 0;
}
if (result.db_paths.size() == 0) {
result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
} else if (result.wal_dir.empty()) {
// Use dbname as default
result.wal_dir = dbname;
}
if (!result.wal_dir.empty()) {
// If there is a wal_dir already set, check to see if the wal_dir is the
// same as the dbname AND the same as the db_path[0] (which must exist from
// a few lines ago). If the wal_dir matches both of these values, then clear
// the wal_dir value, which will make wal_dir == dbname. Most likely this
// condition was the result of reading an old options file where we forced
// wal_dir to be set (to dbname).
auto npath = NormalizePath(dbname + "/");
if (npath == NormalizePath(result.wal_dir + "/") &&
npath == NormalizePath(result.db_paths[0].path + "/")) {
result.wal_dir.clear();
}
}
if (!result.wal_dir.empty() && result.wal_dir.back() == '/') {
result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
}
if (result.use_direct_reads && result.compaction_readahead_size == 0) {
TEST_SYNC_POINT_CALLBACK("SanitizeOptions:direct_io", nullptr);
result.compaction_readahead_size = 1024 * 1024 * 2;
}
// Force flush on DB open if 2PC is enabled, since with 2PC we have no
// guarantee that consecutive log files have consecutive sequence id, which
// make recovery complicated.
if (result.allow_2pc) {
result.avoid_flush_during_recovery = false;
}
#ifndef ROCKSDB_LITE
ImmutableDBOptions immutable_db_options(result);
if (!immutable_db_options.IsWalDirSameAsDBPath()) {
// Either the WAL dir and db_paths[0]/db_name are not the same, or we
// cannot tell for sure. In either case, assume they're different and
// explicitly cleanup the trash log files (bypass DeleteScheduler)
// Do this first so even if we end up calling
// DeleteScheduler::CleanupDirectory on the same dir later, it will be
// safe
std::vector<std::string> filenames;
auto wal_dir = immutable_db_options.GetWalDir();
Status s = result.env->GetChildren(wal_dir, &filenames);
s.PermitUncheckedError(); //**TODO: What to do on error?
for (std::string& filename : filenames) {
if (filename.find(".log.trash", filename.length() -
std::string(".log.trash").length()) !=
std::string::npos) {
std::string trash_file = wal_dir + "/" + filename;
result.env->DeleteFile(trash_file).PermitUncheckedError();
}
}
}
// When the DB is stopped, it's possible that there are some .trash files that
// were not deleted yet, when we open the DB we will find these .trash files
// and schedule them to be deleted (or delete immediately if SstFileManager
// was not used)
auto sfm = static_cast<SstFileManagerImpl*>(result.sst_file_manager.get());
for (size_t i = 0; i < result.db_paths.size(); i++) {
DeleteScheduler::CleanupDirectory(result.env, sfm, result.db_paths[i].path)
.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 22:36:19 +02:00
// Create a default SstFileManager for purposes of tracking compaction size
// and facilitating recovery from out of space errors.
if (result.sst_file_manager.get() == nullptr) {
std::shared_ptr<SstFileManager> sst_file_manager(
NewSstFileManager(result.env, result.info_log));
result.sst_file_manager = sst_file_manager;
}
#endif // !ROCKSDB_LITE
// Supported wal compression types
if (!StreamingCompressionTypeSupported(result.wal_compression)) {
result.wal_compression = kNoCompression;
ROCKS_LOG_WARN(result.info_log,
"wal_compression is disabled since only zstd is supported");
}
if (!result.paranoid_checks) {
result.skip_checking_sst_file_sizes_on_db_open = true;
ROCKS_LOG_INFO(result.info_log,
"file size check will be skipped during open.");
}
return result;
}
namespace {
Status ValidateOptionsByTable(
const DBOptions& db_opts,
const std::vector<ColumnFamilyDescriptor>& column_families) {
Status s;
for (auto cf : column_families) {
s = ValidateOptions(db_opts, cf.options);
if (!s.ok()) {
return s;
}
}
return Status::OK();
}
} // namespace
Status DBImpl::ValidateOptions(
const DBOptions& db_options,
const std::vector<ColumnFamilyDescriptor>& column_families) {
Status s;
for (auto& cfd : column_families) {
s = ColumnFamilyData::ValidateOptions(db_options, cfd.options);
if (!s.ok()) {
return s;
}
}
s = ValidateOptions(db_options);
return s;
}
Status DBImpl::ValidateOptions(const DBOptions& db_options) {
if (db_options.db_paths.size() > 4) {
return Status::NotSupported(
"More than four DB paths are not supported yet. ");
}
if (db_options.allow_mmap_reads && db_options.use_direct_reads) {
// Protect against assert in PosixMMapReadableFile constructor
return Status::NotSupported(
"If memory mapped reads (allow_mmap_reads) are enabled "
"then direct I/O reads (use_direct_reads) must be disabled. ");
}
if (db_options.allow_mmap_writes &&
db_options.use_direct_io_for_flush_and_compaction) {
return Status::NotSupported(
"If memory mapped writes (allow_mmap_writes) are enabled "
"then direct I/O writes (use_direct_io_for_flush_and_compaction) must "
"be disabled. ");
}
if (db_options.keep_log_file_num == 0) {
return Status::InvalidArgument("keep_log_file_num must be greater than 0");
}
Unordered Writes (#5218) Summary: Performing unordered writes in rocksdb when unordered_write option is set to true. When enabled the writes to memtable are done without joining any write thread. This offers much higher write throughput since the upcoming writes would not have to wait for the slowest memtable write to finish. The tradeoff is that the writes visible to a snapshot might change over time. If the application cannot tolerate that, it should implement its own mechanisms to work around that. Using TransactionDB with WRITE_PREPARED write policy is one way to achieve that. Doing so increases the max throughput by 2.2x without however compromising the snapshot guarantees. The patch is prepared based on an original by siying Existing unit tests are extended to include unordered_write option. Benchmark Results: ``` TEST_TMPDIR=/dev/shm/ ./db_bench_unordered --benchmarks=fillrandom --threads=32 --num=10000000 -max_write_buffer_number=16 --max_background_jobs=64 --batch_size=8 --writes=3000000 -level0_file_num_compaction_trigger=99999 --level0_slowdown_writes_trigger=99999 --level0_stop_writes_trigger=99999 -enable_pipelined_write=false -disable_auto_compactions --unordered_write=1 ``` With WAL - Vanilla RocksDB: 78.6 MB/s - WRITER_PREPARED with unordered_write: 177.8 MB/s (2.2x) - unordered_write: 368.9 MB/s (4.7x with relaxed snapshot guarantees) Without WAL - Vanilla RocksDB: 111.3 MB/s - WRITER_PREPARED with unordered_write: 259.3 MB/s MB/s (2.3x) - unordered_write: 645.6 MB/s (5.8x with relaxed snapshot guarantees) - WRITER_PREPARED with unordered_write disable concurrency control: 185.3 MB/s MB/s (2.35x) Limitations: - The feature is not yet extended to `max_successive_merges` > 0. The feature is also incompatible with `enable_pipelined_write` = true as well as with `allow_concurrent_memtable_write` = false. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5218 Differential Revision: D15219029 Pulled By: maysamyabandeh fbshipit-source-id: 38f2abc4af8780148c6128acdba2b3227bc81759
2019-05-14 02:43:47 +02:00
if (db_options.unordered_write &&
!db_options.allow_concurrent_memtable_write) {
return Status::InvalidArgument(
"unordered_write is incompatible with !allow_concurrent_memtable_write");
}
if (db_options.unordered_write && db_options.enable_pipelined_write) {
return Status::InvalidArgument(
"unordered_write is incompatible with enable_pipelined_write");
}
if (db_options.atomic_flush && db_options.enable_pipelined_write) {
return Status::InvalidArgument(
"atomic_flush is incompatible with enable_pipelined_write");
}
// TODO remove this restriction
if (db_options.atomic_flush && db_options.best_efforts_recovery) {
return Status::InvalidArgument(
"atomic_flush is currently incompatible with best-efforts recovery");
}
if (db_options.use_direct_io_for_flush_and_compaction &&
0 == db_options.writable_file_max_buffer_size) {
return Status::InvalidArgument(
"writes in direct IO require writable_file_max_buffer_size > 0");
}
return Status::OK();
}
Status DBImpl::NewDB(std::vector<std::string>* new_filenames) {
VersionEdit new_db;
Status s = SetIdentityFile(env_, dbname_);
if (!s.ok()) {
return s;
}
if (immutable_db_options_.write_dbid_to_manifest) {
std::string temp_db_id;
GetDbIdentityFromIdentityFile(&temp_db_id);
new_db.SetDBId(temp_db_id);
}
new_db.SetLogNumber(0);
new_db.SetNextFile(2);
new_db.SetLastSequence(0);
ROCKS_LOG_INFO(immutable_db_options_.info_log, "Creating manifest 1 \n");
const std::string manifest = DescriptorFileName(dbname_, 1);
{
Handle rename() failure in non-local FS (#8192) Summary: In a distributed environment, a file `rename()` operation can succeed on server (remote) side, but the client can somehow return non-ok status to RocksDB. Possible reasons include network partition, connection issue, etc. This happens in `rocksdb::SetCurrentFile()`, which can be called in `LogAndApply() -> ProcessManifestWrites()` if RocksDB tries to switch to a new MANIFEST. We currently always delete the new MANIFEST if an error occurs. This is problematic in distributed world. If the server-side successfully updates the CURRENT file via renaming, then a subsequent `DB::Open()` will try to look for the new MANIFEST and fail. As a fix, we can track the execution result of IO operations on the new MANIFEST. - If IO operations on the new MANIFEST fail, then we know the CURRENT must point to the original MANIFEST. Therefore, it is safe to remove the new MANIFEST. - If IO operations on the new MANIFEST all succeed, but somehow we end up in the clean up code block, then we do not know whether CURRENT points to the new or old MANIFEST. (For local POSIX-compliant FS, it should still point to old MANIFEST, but it does not matter if we keep the new MANIFEST.) Therefore, we keep the new MANIFEST. - Any future `LogAndApply()` will switch to a new MANIFEST and update CURRENT. - If process reopens the db immediately after the failure, then the CURRENT file can point to either the new MANIFEST or the old one, both of which exist. Therefore, recovery can succeed and ignore the other. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8192 Test Plan: make check Reviewed By: zhichao-cao Differential Revision: D27804648 Pulled By: riversand963 fbshipit-source-id: 9c16f2a5ce41bc6aadf085e48449b19ede8423e4
2021-04-20 03:10:23 +02:00
if (fs_->FileExists(manifest, IOOptions(), nullptr).ok()) {
fs_->DeleteFile(manifest, IOOptions(), nullptr).PermitUncheckedError();
}
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
std::unique_ptr<FSWritableFile> file;
FileOptions file_options = fs_->OptimizeForManifestWrite(file_options_);
s = NewWritableFile(fs_.get(), manifest, &file, file_options);
if (!s.ok()) {
return s;
}
FileTypeSet tmp_set = immutable_db_options_.checksum_handoff_file_types;
file->SetPreallocationBlockSize(
immutable_db_options_.manifest_preallocation_size);
std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
std::move(file), manifest, file_options, immutable_db_options_.clock,
io_tracer_, nullptr /* stats */, immutable_db_options_.listeners,
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
nullptr, tmp_set.Contains(FileType::kDescriptorFile),
tmp_set.Contains(FileType::kDescriptorFile)));
log::Writer log(std::move(file_writer), 0, false);
std::string record;
new_db.EncodeTo(&record);
s = log.AddRecord(record);
if (s.ok()) {
s = SyncManifest(&immutable_db_options_, log.file());
}
}
if (s.ok()) {
// Make "CURRENT" file that points to the new manifest file.
s = SetCurrentFile(fs_.get(), dbname_, 1, directories_.GetDbDir());
if (new_filenames) {
new_filenames->emplace_back(
manifest.substr(manifest.find_last_of("/\\") + 1));
}
} else {
Handle rename() failure in non-local FS (#8192) Summary: In a distributed environment, a file `rename()` operation can succeed on server (remote) side, but the client can somehow return non-ok status to RocksDB. Possible reasons include network partition, connection issue, etc. This happens in `rocksdb::SetCurrentFile()`, which can be called in `LogAndApply() -> ProcessManifestWrites()` if RocksDB tries to switch to a new MANIFEST. We currently always delete the new MANIFEST if an error occurs. This is problematic in distributed world. If the server-side successfully updates the CURRENT file via renaming, then a subsequent `DB::Open()` will try to look for the new MANIFEST and fail. As a fix, we can track the execution result of IO operations on the new MANIFEST. - If IO operations on the new MANIFEST fail, then we know the CURRENT must point to the original MANIFEST. Therefore, it is safe to remove the new MANIFEST. - If IO operations on the new MANIFEST all succeed, but somehow we end up in the clean up code block, then we do not know whether CURRENT points to the new or old MANIFEST. (For local POSIX-compliant FS, it should still point to old MANIFEST, but it does not matter if we keep the new MANIFEST.) Therefore, we keep the new MANIFEST. - Any future `LogAndApply()` will switch to a new MANIFEST and update CURRENT. - If process reopens the db immediately after the failure, then the CURRENT file can point to either the new MANIFEST or the old one, both of which exist. Therefore, recovery can succeed and ignore the other. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8192 Test Plan: make check Reviewed By: zhichao-cao Differential Revision: D27804648 Pulled By: riversand963 fbshipit-source-id: 9c16f2a5ce41bc6aadf085e48449b19ede8423e4
2021-04-20 03:10:23 +02:00
fs_->DeleteFile(manifest, IOOptions(), nullptr).PermitUncheckedError();
}
return s;
}
IOStatus DBImpl::CreateAndNewDirectory(
FileSystem* fs, const std::string& dirname,
std::unique_ptr<FSDirectory>* directory) {
// We call CreateDirIfMissing() as the directory may already exist (if we
// are reopening a DB), when this happens we don't want creating the
// directory to cause an error. However, we need to check if creating the
// directory fails or else we may get an obscure message about the lock
// file not existing. One real-world example of this occurring is if
// env->CreateDirIfMissing() doesn't create intermediate directories, e.g.
// when dbname_ is "dir/db" but when "dir" doesn't exist.
IOStatus io_s = fs->CreateDirIfMissing(dirname, IOOptions(), nullptr);
if (!io_s.ok()) {
return io_s;
}
return fs->NewDirectory(dirname, IOOptions(), directory, nullptr);
}
IOStatus Directories::SetDirectories(FileSystem* fs, const std::string& dbname,
const std::string& wal_dir,
const std::vector<DbPath>& data_paths) {
IOStatus io_s = DBImpl::CreateAndNewDirectory(fs, dbname, &db_dir_);
if (!io_s.ok()) {
return io_s;
}
if (!wal_dir.empty() && dbname != wal_dir) {
io_s = DBImpl::CreateAndNewDirectory(fs, wal_dir, &wal_dir_);
if (!io_s.ok()) {
return io_s;
}
}
data_dirs_.clear();
for (auto& p : data_paths) {
const std::string db_path = p.path;
if (db_path == dbname) {
data_dirs_.emplace_back(nullptr);
} else {
std::unique_ptr<FSDirectory> path_directory;
io_s = DBImpl::CreateAndNewDirectory(fs, db_path, &path_directory);
if (!io_s.ok()) {
return io_s;
}
data_dirs_.emplace_back(path_directory.release());
}
}
assert(data_dirs_.size() == data_paths.size());
return IOStatus::OK();
}
Status DBImpl::Recover(
const std::vector<ColumnFamilyDescriptor>& column_families, bool read_only,
bool error_if_wal_file_exists, bool error_if_data_exists_in_wals,
uint64_t* recovered_seq) {
mutex_.AssertHeld();
bool is_new_db = false;
assert(db_lock_ == nullptr);
std::vector<std::string> files_in_dbname;
if (!read_only) {
Status s = directories_.SetDirectories(fs_.get(), dbname_,
immutable_db_options_.wal_dir,
immutable_db_options_.db_paths);
if (!s.ok()) {
return s;
}
s = env_->LockFile(LockFileName(dbname_), &db_lock_);
if (!s.ok()) {
return s;
}
std::string current_fname = CurrentFileName(dbname_);
// Path to any MANIFEST file in the db dir. It does not matter which one.
// Since best-efforts recovery ignores CURRENT file, existence of a
// MANIFEST indicates the recovery to recover existing db. If no MANIFEST
// can be found, a new db will be created.
std::string manifest_path;
if (!immutable_db_options_.best_efforts_recovery) {
s = env_->FileExists(current_fname);
} else {
s = Status::NotFound();
Status io_s = env_->GetChildren(dbname_, &files_in_dbname);
if (!io_s.ok()) {
s = io_s;
files_in_dbname.clear();
}
for (const std::string& file : files_in_dbname) {
uint64_t number = 0;
FileType type = kWalFile; // initialize
if (ParseFileName(file, &number, &type) && type == kDescriptorFile) {
// Found MANIFEST (descriptor log), thus best-efforts recovery does
// not have to treat the db as empty.
s = Status::OK();
manifest_path = dbname_ + "/" + file;
break;
}
}
}
if (s.IsNotFound()) {
if (immutable_db_options_.create_if_missing) {
s = NewDB(&files_in_dbname);
is_new_db = true;
if (!s.ok()) {
return s;
}
} else {
return Status::InvalidArgument(
current_fname, "does not exist (create_if_missing is false)");
}
} else if (s.ok()) {
if (immutable_db_options_.error_if_exists) {
return Status::InvalidArgument(dbname_,
"exists (error_if_exists is true)");
}
} else {
// Unexpected error reading file
assert(s.IsIOError());
return s;
}
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
// Verify compatibility of file_options_ and filesystem
DB:Open should fail on tmpfs when use_direct_reads=true Summary: Before: > $ TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 DB path: [/dev/shm/dbbench] put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument db_bench: tpp.c:84: __pthread_tpp_change_priority: Assertion `new_prio == -1 || (new_prio >= fifo_min_prio && new_prio <= fifo_max_prio)' failed. put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument After: > TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags open error: Not implemented: Direct I/O is not supported by the specified DB. Closes https://github.com/facebook/rocksdb/pull/3539 Differential Revision: D7082658 Pulled By: miasantreble fbshipit-source-id: f9d9c6ec3b5e9e049cab52154940ee101ba4d342
2018-02-26 23:46:12 +01:00
{
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
std::unique_ptr<FSRandomAccessFile> idfile;
FileOptions customized_fs(file_options_);
customized_fs.use_direct_reads |=
DB:Open should fail on tmpfs when use_direct_reads=true Summary: Before: > $ TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 DB path: [/dev/shm/dbbench] put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument db_bench: tpp.c:84: __pthread_tpp_change_priority: Assertion `new_prio == -1 || (new_prio >= fifo_min_prio && new_prio <= fifo_max_prio)' failed. put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument After: > TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags open error: Not implemented: Direct I/O is not supported by the specified DB. Closes https://github.com/facebook/rocksdb/pull/3539 Differential Revision: D7082658 Pulled By: miasantreble fbshipit-source-id: f9d9c6ec3b5e9e049cab52154940ee101ba4d342
2018-02-26 23:46:12 +01:00
immutable_db_options_.use_direct_io_for_flush_and_compaction;
const std::string& fname =
manifest_path.empty() ? current_fname : manifest_path;
s = fs_->NewRandomAccessFile(fname, customized_fs, &idfile, nullptr);
DB:Open should fail on tmpfs when use_direct_reads=true Summary: Before: > $ TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 DB path: [/dev/shm/dbbench] put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument db_bench: tpp.c:84: __pthread_tpp_change_priority: Assertion `new_prio == -1 || (new_prio >= fifo_min_prio && new_prio <= fifo_max_prio)' failed. put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument After: > TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags open error: Not implemented: Direct I/O is not supported by the specified DB. Closes https://github.com/facebook/rocksdb/pull/3539 Differential Revision: D7082658 Pulled By: miasantreble fbshipit-source-id: f9d9c6ec3b5e9e049cab52154940ee101ba4d342
2018-02-26 23:46:12 +01:00
if (!s.ok()) {
std::string error_str = s.ToString();
DB:Open should fail on tmpfs when use_direct_reads=true Summary: Before: > $ TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 DB path: [/dev/shm/dbbench] put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument db_bench: tpp.c:84: __pthread_tpp_change_priority: Assertion `new_prio == -1 || (new_prio >= fifo_min_prio && new_prio <= fifo_max_prio)' failed. put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument After: > TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags open error: Not implemented: Direct I/O is not supported by the specified DB. Closes https://github.com/facebook/rocksdb/pull/3539 Differential Revision: D7082658 Pulled By: miasantreble fbshipit-source-id: f9d9c6ec3b5e9e049cab52154940ee101ba4d342
2018-02-26 23:46:12 +01:00
// Check if unsupported Direct I/O is the root cause
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
customized_fs.use_direct_reads = false;
s = fs_->NewRandomAccessFile(fname, customized_fs, &idfile, nullptr);
DB:Open should fail on tmpfs when use_direct_reads=true Summary: Before: > $ TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 DB path: [/dev/shm/dbbench] put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument db_bench: tpp.c:84: __pthread_tpp_change_priority: Assertion `new_prio == -1 || (new_prio >= fifo_min_prio && new_prio <= fifo_max_prio)' failed. put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument After: > TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags open error: Not implemented: Direct I/O is not supported by the specified DB. Closes https://github.com/facebook/rocksdb/pull/3539 Differential Revision: D7082658 Pulled By: miasantreble fbshipit-source-id: f9d9c6ec3b5e9e049cab52154940ee101ba4d342
2018-02-26 23:46:12 +01:00
if (s.ok()) {
return Status::InvalidArgument(
"Direct I/O is not supported by the specified DB.");
} else {
return Status::InvalidArgument(
"Found options incompatible with filesystem", error_str.c_str());
DB:Open should fail on tmpfs when use_direct_reads=true Summary: Before: > $ TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 DB path: [/dev/shm/dbbench] put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument db_bench: tpp.c:84: __pthread_tpp_change_priority: Assertion `new_prio == -1 || (new_prio >= fifo_min_prio && new_prio <= fifo_max_prio)' failed. put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument put error: IO error: While open a file for random read: /dev/shm/dbbench/000007.sst: Invalid argument After: > TEST_TMPDIR=/dev/shm ./db_bench -use_direct_reads=true -benchmarks=readrandomwriterandom -num=10000000 -reads=100000 -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 -max_background_jobs=12 -readwritepercent=50 -key_size=16 -value_size=48 -threads=32 Initializing RocksDB Options from the specified file Initializing RocksDB Options from command-line flags open error: Not implemented: Direct I/O is not supported by the specified DB. Closes https://github.com/facebook/rocksdb/pull/3539 Differential Revision: D7082658 Pulled By: miasantreble fbshipit-source-id: f9d9c6ec3b5e9e049cab52154940ee101ba4d342
2018-02-26 23:46:12 +01:00
}
}
}
} else if (immutable_db_options_.best_efforts_recovery) {
assert(files_in_dbname.empty());
Status s = env_->GetChildren(dbname_, &files_in_dbname);
if (s.IsNotFound()) {
return Status::InvalidArgument(dbname_,
"does not exist (open for read only)");
} else if (s.IsIOError()) {
return s;
}
assert(s.ok());
}
assert(db_id_.empty());
Status s;
bool missing_table_file = false;
if (!immutable_db_options_.best_efforts_recovery) {
s = versions_->Recover(column_families, read_only, &db_id_);
} else {
assert(!files_in_dbname.empty());
s = versions_->TryRecover(column_families, read_only, files_in_dbname,
&db_id_, &missing_table_file);
if (s.ok()) {
// TryRecover may delete previous column_family_set_.
column_family_memtables_.reset(
new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet()));
}
}
if (!s.ok()) {
return s;
}
s = SetDBId(read_only);
Fix a recovery corner case (#7621) Summary: Consider the following sequence of events: 1. Db flushed an SST with file number N, appended to MANIFEST, and tried to sync the MANIFEST. 2. Syncing MANIFEST failed and db crashed. 3. Db tried to recover with this MANIFEST. In the meantime, no entry about the newly-flushed SST was found in the MANIFEST. Therefore, RocksDB replayed WAL and tried to flush to an SST file reusing the same file number N. This failed because file system does not support overwrite. Then Db deleted this file. 4. Db crashed again. 5. Db tried to recover. When db read the MANIFEST, there was an entry referencing N.sst. This could happen probably because the append in step 1 finally reached the MANIFEST and became visible. Since N.sst had been deleted in step 3, recovery failed. It is possible that N.sst created in step 1 is valid. Although step 3 would still fail since the MANIFEST was not synced properly in step 1 and 2, deleting N.sst would make it impossible for the db to recover even if the remaining part of MANIFEST was appended and visible after step 5. After this PR, in step 3, immediately after recovering from MANIFEST, a new MANIFEST is created, then we find that N.sst is not referenced in the MANIFEST, so we delete it, and we'll not reuse N as file number. Then in step 5, since the new MANIFEST does not contain N.sst, the recovery failure situation in step 5 won't happen. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7621 Test Plan: 1. some tests are updated, because these tests assume that new MANIFEST is created after WAL recovery. 2. a new unit test is added in db_basic_test to simulate step 3. Reviewed By: riversand963 Differential Revision: D24668144 Pulled By: cheng-chang fbshipit-source-id: 90d7487fbad2bc3714f5ede46ea949895b15ae3b
2020-11-08 06:54:55 +01:00
if (s.ok() && !read_only) {
s = DeleteUnreferencedSstFiles();
}
if (immutable_db_options_.paranoid_checks && s.ok()) {
s = CheckConsistency();
}
if (s.ok() && !read_only) {
std::map<std::string, std::shared_ptr<FSDirectory>> created_dirs;
for (auto cfd : *versions_->GetColumnFamilySet()) {
s = cfd->AddDirectories(&created_dirs);
if (!s.ok()) {
return s;
}
}
}
// DB mutex is already held
if (s.ok() && immutable_db_options_.persist_stats_to_disk) {
s = InitPersistStatsColumnFamily();
}
std::vector<std::string> files_in_wal_dir;
if (s.ok()) {
// Initial max_total_in_memory_state_ before recovery wals. Log recovery
// may check this value to decide whether to flush.
max_total_in_memory_state_ = 0;
for (auto cfd : *versions_->GetColumnFamilySet()) {
auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
max_total_in_memory_state_ += mutable_cf_options->write_buffer_size *
mutable_cf_options->max_write_buffer_number;
}
SequenceNumber next_sequence(kMaxSequenceNumber);
default_cf_handle_ = new ColumnFamilyHandleImpl(
versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
// TODO(Zhongyi): handle single_column_family_mode_ when
// persistent_stats is enabled
single_column_family_mode_ =
versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
// Recover from all newer log files than the ones named in the
// descriptor (new log files may have been added by the previous
// incarnation without registering them in the descriptor).
//
// Note that prev_log_number() is no longer used, but we pay
// attention to it in case we are recovering a database
// produced by an older version of rocksdb.
auto wal_dir = immutable_db_options_.GetWalDir();
if (!immutable_db_options_.best_efforts_recovery) {
s = env_->GetChildren(wal_dir, &files_in_wal_dir);
}
if (s.IsNotFound()) {
return Status::InvalidArgument("wal_dir not found", wal_dir);
} else if (!s.ok()) {
return s;
}
std::unordered_map<uint64_t, std::string> wal_files;
for (const auto& file : files_in_wal_dir) {
uint64_t number;
FileType type;
if (ParseFileName(file, &number, &type) && type == kWalFile) {
if (is_new_db) {
return Status::Corruption(
"While creating a new Db, wal_dir contains "
"existing log file: ",
file);
} else {
wal_files[number] = LogFileName(wal_dir, number);
}
}
}
if (immutable_db_options_.track_and_verify_wals_in_manifest) {
if (!immutable_db_options_.best_efforts_recovery) {
// Verify WALs in MANIFEST.
s = versions_->GetWalSet().CheckWals(env_, wal_files);
} // else since best effort recovery does not recover from WALs, no need
// to check WALs.
} else if (!versions_->GetWalSet().GetWals().empty()) {
// Tracking is disabled, clear previously tracked WALs from MANIFEST,
// otherwise, in the future, if WAL tracking is enabled again,
// since the WALs deleted when WAL tracking is disabled are not persisted
// into MANIFEST, WAL check may fail.
VersionEdit edit;
WalNumber max_wal_number =
versions_->GetWalSet().GetWals().rbegin()->first;
edit.DeleteWalsBefore(max_wal_number + 1);
s = versions_->LogAndApplyToDefaultColumnFamily(&edit, &mutex_);
}
if (!s.ok()) {
return s;
}
if (!wal_files.empty()) {
if (error_if_wal_file_exists) {
return Status::Corruption(
"The db was opened in readonly mode with error_if_wal_file_exists"
"flag but a WAL file already exists");
} else if (error_if_data_exists_in_wals) {
for (auto& wal_file : wal_files) {
uint64_t bytes;
s = env_->GetFileSize(wal_file.second, &bytes);
if (s.ok()) {
if (bytes > 0) {
return Status::Corruption(
"error_if_data_exists_in_wals is set but there are data "
" in WAL files.");
}
}
}
}
}
if (!wal_files.empty()) {
// Recover in the order in which the wals were generated
std::vector<uint64_t> wals;
wals.reserve(wal_files.size());
for (const auto& wal_file : wal_files) {
wals.push_back(wal_file.first);
}
std::sort(wals.begin(), wals.end());
bool corrupted_wal_found = false;
s = RecoverLogFiles(wals, &next_sequence, read_only,
&corrupted_wal_found);
if (corrupted_wal_found && recovered_seq != nullptr) {
*recovered_seq = next_sequence;
}
if (!s.ok()) {
// Clear memtables if recovery failed
for (auto cfd : *versions_->GetColumnFamilySet()) {
cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
kMaxSequenceNumber);
}
}
}
}
if (read_only) {
// If we are opening as read-only, we need to update options_file_number_
// to reflect the most recent OPTIONS file. It does not matter for regular
// read-write db instance because options_file_number_ will later be
// updated to versions_->NewFileNumber() in RenameTempFileToOptionsFile.
std::vector<std::string> filenames;
if (s.ok()) {
const std::string normalized_dbname = NormalizePath(dbname_);
const std::string normalized_wal_dir =
NormalizePath(immutable_db_options_.GetWalDir());
if (immutable_db_options_.best_efforts_recovery) {
filenames = std::move(files_in_dbname);
} else if (normalized_dbname == normalized_wal_dir) {
filenames = std::move(files_in_wal_dir);
} else {
s = env_->GetChildren(GetName(), &filenames);
}
}
if (s.ok()) {
uint64_t number = 0;
uint64_t options_file_number = 0;
FileType type;
for (const auto& fname : filenames) {
if (ParseFileName(fname, &number, &type) && type == kOptionsFile) {
options_file_number = std::max(number, options_file_number);
}
}
versions_->options_file_number_ = options_file_number;
Add (Live)FileStorageInfo API (#8968) Summary: New classes FileStorageInfo and LiveFileStorageInfo and 'experimental' function DB::GetLiveFilesStorageInfo, which is intended to largely replace several fragmented DB functions needed to create checkpoints and backups. This function is now used to create checkpoints and backups, because it fixes many (probably not all) of the prior complexities of checkpoint not having atomic access to DB metadata. This also ensures strong functional test coverage of the new API. Specifically, much of the old CheckpointImpl::CreateCustomCheckpoint has been migrated to and updated in DBImpl::GetLiveFilesStorageInfo, with the former now calling the latter. Also, the class FileStorageInfo in metadata.h compatibly replaces BackupFileInfo and serves as a new base class for SstFileMetaData. Some old fields of SstFileMetaData are still provided (for now) but deprecated. Although FileStorageInfo::directory is accurate when using db_paths and/or cf_paths, these have never been supported by Checkpoint nor BackupEngine and still are not. This change does now detect these cases and return NotSupported when appropriate. (More work needed for support.) Somehow this change broke ProgressCallbackDuringBackup, but the progress_callback logic was dubious to begin with because it would call the callback based on copy buffer size, not size actually copied. Logic and test updated to track size actually copied per-thread. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8968 Test Plan: tests updated. DB::GetLiveFilesStorageInfo mostly tested by use in CheckpointImpl. DBTest.SnapshotFiles updated to also test GetLiveFilesStorageInfo, including reading the data after DB close. Added CheckpointTest.CheckpointWithDbPath (NotSupported). Reviewed By: siying Differential Revision: D31242045 Pulled By: pdillinger fbshipit-source-id: b183d1ce9799e220daaefd6b3b5365d98de676c0
2021-10-16 19:03:19 +02:00
uint64_t options_file_size = 0;
if (options_file_number > 0) {
s = env_->GetFileSize(OptionsFileName(GetName(), options_file_number),
&options_file_size);
}
versions_->options_file_size_ = options_file_size;
}
}
return s;
}
Status DBImpl::PersistentStatsProcessFormatVersion() {
mutex_.AssertHeld();
Status s;
// persist version when stats CF doesn't exist
bool should_persist_format_version = !persistent_stats_cfd_exists_;
mutex_.Unlock();
if (persistent_stats_cfd_exists_) {
// Check persistent stats format version compatibility. Drop and recreate
// persistent stats CF if format version is incompatible
uint64_t format_version_recovered = 0;
Status s_format = DecodePersistentStatsVersionNumber(
this, StatsVersionKeyType::kFormatVersion, &format_version_recovered);
uint64_t compatible_version_recovered = 0;
Status s_compatible = DecodePersistentStatsVersionNumber(
this, StatsVersionKeyType::kCompatibleVersion,
&compatible_version_recovered);
// abort reading from existing stats CF if any of following is true:
// 1. failed to read format version or compatible version from disk
// 2. sst's format version is greater than current format version, meaning
// this sst is encoded with a newer RocksDB release, and current compatible
// version is below the sst's compatible version
if (!s_format.ok() || !s_compatible.ok() ||
(kStatsCFCurrentFormatVersion < format_version_recovered &&
kStatsCFCompatibleFormatVersion < compatible_version_recovered)) {
if (!s_format.ok() || !s_compatible.ok()) {
ROCKS_LOG_WARN(
immutable_db_options_.info_log,
"Recreating persistent stats column family since reading "
"persistent stats version key failed. Format key: %s, compatible "
"key: %s",
s_format.ToString().c_str(), s_compatible.ToString().c_str());
} else {
ROCKS_LOG_WARN(
immutable_db_options_.info_log,
"Recreating persistent stats column family due to corrupted or "
"incompatible format version. Recovered format: %" PRIu64
"; recovered format compatible since: %" PRIu64 "\n",
format_version_recovered, compatible_version_recovered);
}
s = DropColumnFamily(persist_stats_cf_handle_);
if (s.ok()) {
s = DestroyColumnFamilyHandle(persist_stats_cf_handle_);
}
ColumnFamilyHandle* handle = nullptr;
if (s.ok()) {
ColumnFamilyOptions cfo;
OptimizeForPersistentStats(&cfo);
s = CreateColumnFamily(cfo, kPersistentStatsColumnFamilyName, &handle);
}
if (s.ok()) {
persist_stats_cf_handle_ = static_cast<ColumnFamilyHandleImpl*>(handle);
// should also persist version here because old stats CF is discarded
should_persist_format_version = true;
}
}
}
if (should_persist_format_version) {
// Persistent stats CF being created for the first time, need to write
// format version key
WriteBatch batch;
if (s.ok()) {
s = batch.Put(persist_stats_cf_handle_, kFormatVersionKeyString,
ToString(kStatsCFCurrentFormatVersion));
}
if (s.ok()) {
s = batch.Put(persist_stats_cf_handle_, kCompatibleVersionKeyString,
ToString(kStatsCFCompatibleFormatVersion));
}
if (s.ok()) {
WriteOptions wo;
wo.low_pri = true;
wo.no_slowdown = true;
wo.sync = false;
s = Write(wo, &batch);
}
}
mutex_.Lock();
return s;
}
Status DBImpl::InitPersistStatsColumnFamily() {
mutex_.AssertHeld();
assert(!persist_stats_cf_handle_);
ColumnFamilyData* persistent_stats_cfd =
versions_->GetColumnFamilySet()->GetColumnFamily(
kPersistentStatsColumnFamilyName);
persistent_stats_cfd_exists_ = persistent_stats_cfd != nullptr;
Status s;
if (persistent_stats_cfd != nullptr) {
// We are recovering from a DB which already contains persistent stats CF,
// the CF is already created in VersionSet::ApplyOneVersionEdit, but
// column family handle was not. Need to explicitly create handle here.
persist_stats_cf_handle_ =
new ColumnFamilyHandleImpl(persistent_stats_cfd, this, &mutex_);
} else {
mutex_.Unlock();
ColumnFamilyHandle* handle = nullptr;
ColumnFamilyOptions cfo;
OptimizeForPersistentStats(&cfo);
s = CreateColumnFamily(cfo, kPersistentStatsColumnFamilyName, &handle);
persist_stats_cf_handle_ = static_cast<ColumnFamilyHandleImpl*>(handle);
mutex_.Lock();
}
return s;
}
// REQUIRES: wal_numbers are sorted in ascending order
Status DBImpl::RecoverLogFiles(const std::vector<uint64_t>& wal_numbers,
SequenceNumber* next_sequence, bool read_only,
bool* corrupted_wal_found) {
struct LogReporter : public log::Reader::Reporter {
Env* env;
Logger* info_log;
const char* fname;
Status* status; // nullptr if immutable_db_options_.paranoid_checks==false
void Corruption(size_t bytes, const Status& s) override {
ROCKS_LOG_WARN(info_log, "%s%s: dropping %d bytes; %s",
(status == nullptr ? "(ignoring error) " : ""), fname,
static_cast<int>(bytes), s.ToString().c_str());
if (status != nullptr && status->ok()) {
*status = s;
}
}
};
mutex_.AssertHeld();
Status status;
std::unordered_map<int, VersionEdit> version_edits;
// no need to refcount because iteration is under mutex
for (auto cfd : *versions_->GetColumnFamilySet()) {
VersionEdit edit;
edit.SetColumnFamily(cfd->GetID());
version_edits.insert({cfd->GetID(), edit});
}
int job_id = next_job_id_.fetch_add(1);
{
auto stream = event_logger_.Log();
stream << "job" << job_id << "event"
<< "recovery_started";
stream << "wal_files";
stream.StartArray();
for (auto wal_number : wal_numbers) {
stream << wal_number;
}
stream.EndArray();
}
#ifndef ROCKSDB_LITE
if (immutable_db_options_.wal_filter != nullptr) {
std::map<std::string, uint32_t> cf_name_id_map;
std::map<uint32_t, uint64_t> cf_lognumber_map;
for (auto cfd : *versions_->GetColumnFamilySet()) {
cf_name_id_map.insert(std::make_pair(cfd->GetName(), cfd->GetID()));
cf_lognumber_map.insert(
std::make_pair(cfd->GetID(), cfd->GetLogNumber()));
}
immutable_db_options_.wal_filter->ColumnFamilyLogNumberMap(cf_lognumber_map,
cf_name_id_map);
}
#endif
bool stop_replay_by_wal_filter = false;
bool stop_replay_for_corruption = false;
bool flushed = false;
uint64_t corrupted_wal_number = kMaxSequenceNumber;
uint64_t min_wal_number = MinLogNumberToKeep();
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 03:41:31 +01:00
if (!allow_2pc()) {
// In non-2pc mode, we skip WALs that do not back unflushed data.
min_wal_number =
std::max(min_wal_number, versions_->MinLogNumberWithUnflushedData());
}
for (auto wal_number : wal_numbers) {
if (wal_number < min_wal_number) {
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
2018-05-04 00:35:11 +02:00
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"Skipping log #%" PRIu64
" since it is older than min log to keep #%" PRIu64,
wal_number, min_wal_number);
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
2018-05-04 00:35:11 +02:00
continue;
}
// The previous incarnation may not have written any MANIFEST
// records after allocating this log number. So we manually
// update the file number allocation counter in VersionSet.
versions_->MarkFileNumberUsed(wal_number);
// Open the log file
std::string fname =
LogFileName(immutable_db_options_.GetWalDir(), wal_number);
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"Recovering log #%" PRIu64 " mode %d", wal_number,
static_cast<int>(immutable_db_options_.wal_recovery_mode));
auto logFileDropped = [this, &fname]() {
uint64_t bytes;
if (env_->GetFileSize(fname, &bytes).ok()) {
auto info_log = immutable_db_options_.info_log.get();
ROCKS_LOG_WARN(info_log, "%s: dropping %d bytes", fname.c_str(),
static_cast<int>(bytes));
}
};
if (stop_replay_by_wal_filter) {
logFileDropped();
continue;
}
std::unique_ptr<SequentialFileReader> file_reader;
{
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
std::unique_ptr<FSSequentialFile> file;
status = fs_->NewSequentialFile(fname,
fs_->OptimizeForLogRead(file_options_),
&file, nullptr);
if (!status.ok()) {
MaybeIgnoreError(&status);
if (!status.ok()) {
return status;
} else {
// Fail with one log file, but that's ok.
// Try next one.
continue;
}
}
file_reader.reset(new SequentialFileReader(
std::move(file), fname, immutable_db_options_.log_readahead_size,
io_tracer_));
}
// Create the log reader.
LogReporter reporter;
reporter.env = env_;
reporter.info_log = immutable_db_options_.info_log.get();
reporter.fname = fname.c_str();
if (!immutable_db_options_.paranoid_checks ||
immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kSkipAnyCorruptedRecords) {
reporter.status = nullptr;
} else {
reporter.status = &status;
}
// We intentially make log::Reader do checksumming even if
// paranoid_checks==false so that corruptions cause entire commits
// to be skipped instead of propagating bad information (like overly
// large sequence numbers).
log::Reader reader(immutable_db_options_.info_log, std::move(file_reader),
&reporter, true /*checksum*/, wal_number);
// Determine if we should tolerate incomplete records at the tail end of the
// Read all the records and add to a memtable
std::string scratch;
Slice record;
WriteBatch batch;
TEST_SYNC_POINT_CALLBACK("DBImpl::RecoverLogFiles:BeforeReadWal",
/*arg=*/nullptr);
while (!stop_replay_by_wal_filter &&
reader.ReadRecord(&record, &scratch,
immutable_db_options_.wal_recovery_mode) &&
status.ok()) {
if (record.size() < WriteBatchInternal::kHeader) {
reporter.Corruption(record.size(),
Status::Corruption("log record too small"));
continue;
}
status = WriteBatchInternal::SetContents(&batch, record);
if (!status.ok()) {
return status;
}
SequenceNumber sequence = WriteBatchInternal::Sequence(&batch);
if (immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kPointInTimeRecovery) {
// In point-in-time recovery mode, if sequence id of log files are
// consecutive, we continue recovery despite corruption. This could
// happen when we open and write to a corrupted DB, where sequence id
// will start from the last sequence id we recovered.
if (sequence == *next_sequence) {
stop_replay_for_corruption = false;
}
if (stop_replay_for_corruption) {
logFileDropped();
break;
}
}
#ifndef ROCKSDB_LITE
if (immutable_db_options_.wal_filter != nullptr) {
WriteBatch new_batch;
bool batch_changed = false;
WalFilter::WalProcessingOption wal_processing_option =
immutable_db_options_.wal_filter->LogRecordFound(
wal_number, fname, batch, &new_batch, &batch_changed);
switch (wal_processing_option) {
case WalFilter::WalProcessingOption::kContinueProcessing:
// do nothing, proceeed normally
break;
case WalFilter::WalProcessingOption::kIgnoreCurrentRecord:
// skip current record
continue;
case WalFilter::WalProcessingOption::kStopReplay:
// skip current record and stop replay
stop_replay_by_wal_filter = true;
continue;
case WalFilter::WalProcessingOption::kCorruptedRecord: {
status =
Status::Corruption("Corruption reported by Wal Filter ",
immutable_db_options_.wal_filter->Name());
MaybeIgnoreError(&status);
if (!status.ok()) {
reporter.Corruption(record.size(), status);
continue;
}
break;
}
default: {
assert(false); // unhandled case
status = Status::NotSupported(
"Unknown WalProcessingOption returned"
" by Wal Filter ",
immutable_db_options_.wal_filter->Name());
MaybeIgnoreError(&status);
if (!status.ok()) {
return status;
} else {
// Ignore the error with current record processing.
continue;
}
}
}
if (batch_changed) {
// Make sure that the count in the new batch is
// within the orignal count.
int new_count = WriteBatchInternal::Count(&new_batch);
int original_count = WriteBatchInternal::Count(&batch);
if (new_count > original_count) {
ROCKS_LOG_FATAL(
immutable_db_options_.info_log,
"Recovering log #%" PRIu64
" mode %d log filter %s returned "
"more records (%d) than original (%d) which is not allowed. "
"Aborting recovery.",
wal_number,
static_cast<int>(immutable_db_options_.wal_recovery_mode),
immutable_db_options_.wal_filter->Name(), new_count,
original_count);
status = Status::NotSupported(
"More than original # of records "
"returned by Wal Filter ",
immutable_db_options_.wal_filter->Name());
return status;
}
// Set the same sequence number in the new_batch
// as the original batch.
WriteBatchInternal::SetSequence(&new_batch,
WriteBatchInternal::Sequence(&batch));
batch = new_batch;
}
}
#endif // ROCKSDB_LITE
// If column family was not found, it might mean that the WAL write
// batch references to the column family that was dropped after the
// insert. We don't want to fail the whole write batch in that case --
// we just ignore the update.
// That's why we set ignore missing column families to true
bool has_valid_writes = false;
status = WriteBatchInternal::InsertInto(
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
&batch, column_family_memtables_.get(), &flush_scheduler_,
&trim_history_scheduler_, true, wal_number, this,
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
false /* concurrent_memtable_writes */, next_sequence,
&has_valid_writes, seq_per_batch_, batch_per_txn_);
MaybeIgnoreError(&status);
if (!status.ok()) {
// We are treating this as a failure while reading since we read valid
// blocks that do not form coherent data
reporter.Corruption(record.size(), status);
continue;
}
if (has_valid_writes && !read_only) {
// we can do this because this is called before client has access to the
// DB and there is only a single thread operating on DB
ColumnFamilyData* cfd;
while ((cfd = flush_scheduler_.TakeNextColumnFamily()) != nullptr) {
cfd->UnrefAndTryDelete();
// If this asserts, it means that InsertInto failed in
// filtering updates to already-flushed column families
assert(cfd->GetLogNumber() <= wal_number);
auto iter = version_edits.find(cfd->GetID());
assert(iter != version_edits.end());
VersionEdit* edit = &iter->second;
status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
if (!status.ok()) {
// Reflect errors immediately so that conditions like full
// file-systems cause the DB::Open() to fail.
return status;
}
flushed = true;
cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
*next_sequence);
}
}
}
if (!status.ok()) {
if (status.IsNotSupported()) {
// We should not treat NotSupported as corruption. It is rather a clear
// sign that we are processing a WAL that is produced by an incompatible
// version of the code.
return status;
}
if (immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kSkipAnyCorruptedRecords) {
// We should ignore all errors unconditionally
status = Status::OK();
} else if (immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kPointInTimeRecovery) {
if (status.IsIOError()) {
ROCKS_LOG_ERROR(immutable_db_options_.info_log,
"IOError during point-in-time reading log #%" PRIu64
" seq #%" PRIu64
". %s. This likely mean loss of synced WAL, "
"thus recovery fails.",
wal_number, *next_sequence,
status.ToString().c_str());
return status;
}
// We should ignore the error but not continue replaying
status = Status::OK();
stop_replay_for_corruption = true;
corrupted_wal_number = wal_number;
if (corrupted_wal_found != nullptr) {
*corrupted_wal_found = true;
}
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"Point in time recovered to log #%" PRIu64
" seq #%" PRIu64,
wal_number, *next_sequence);
} else {
assert(immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kTolerateCorruptedTailRecords ||
immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kAbsoluteConsistency);
return status;
}
}
flush_scheduler_.Clear();
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
trim_history_scheduler_.Clear();
auto last_sequence = *next_sequence - 1;
if ((*next_sequence != kMaxSequenceNumber) &&
(versions_->LastSequence() <= last_sequence)) {
versions_->SetLastAllocatedSequence(last_sequence);
versions_->SetLastPublishedSequence(last_sequence);
versions_->SetLastSequence(last_sequence);
}
}
// Compare the corrupted log number to all columnfamily's current log number.
// Abort Open() if any column family's log number is greater than
// the corrupted log number, which means CF contains data beyond the point of
// corruption. This could during PIT recovery when the WAL is corrupted and
// some (but not all) CFs are flushed
// Exclude the PIT case where no log is dropped after the corruption point.
// This is to cover the case for empty wals after corrupted log, in which we
// don't reset stop_replay_for_corruption.
if (stop_replay_for_corruption == true &&
(immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kPointInTimeRecovery ||
immutable_db_options_.wal_recovery_mode ==
WALRecoveryMode::kTolerateCorruptedTailRecords)) {
for (auto cfd : *versions_->GetColumnFamilySet()) {
// One special case cause cfd->GetLogNumber() > corrupted_wal_number but
// the CF is still consistent: If a new column family is created during
// the flush and the WAL sync fails at the same time, the new CF points to
// the new WAL but the old WAL is curropted. Since the new CF is empty, it
// is still consistent. We add the check of CF sst file size to avoid the
// false positive alert.
// Note that, the check of (cfd->GetLiveSstFilesSize() > 0) may leads to
// the ignorance of a very rare inconsistency case caused in data
// canclation. One CF is empty due to KV deletion. But those operations
// are in the WAL. If the WAL is corrupted, the status of this CF might
// not be consistent with others. However, the consistency check will be
// bypassed due to empty CF.
// TODO: a better and complete implementation is needed to ensure strict
// consistency check in WAL recovery including hanlding the tailing
// issues.
if (cfd->GetLogNumber() > corrupted_wal_number &&
cfd->GetLiveSstFilesSize() > 0) {
ROCKS_LOG_ERROR(immutable_db_options_.info_log,
"Column family inconsistency: SST file contains data"
" beyond the point of corruption.");
return Status::Corruption("SST file is ahead of WALs in CF " +
cfd->GetName());
}
}
}
// True if there's any data in the WALs; if not, we can skip re-processing
// them later
bool data_seen = false;
if (!read_only) {
// no need to refcount since client still doesn't have access
// to the DB and can not drop column families while we iterate
const WalNumber max_wal_number = wal_numbers.back();
for (auto cfd : *versions_->GetColumnFamilySet()) {
auto iter = version_edits.find(cfd->GetID());
assert(iter != version_edits.end());
VersionEdit* edit = &iter->second;
if (cfd->GetLogNumber() > max_wal_number) {
// Column family cfd has already flushed the data
// from all wals. Memtable has to be empty because
// we filter the updates based on wal_number
// (in WriteBatch::InsertInto)
assert(cfd->mem()->GetFirstSequenceNumber() == 0);
assert(edit->NumEntries() == 0);
continue;
}
TEST_SYNC_POINT_CALLBACK(
"DBImpl::RecoverLogFiles:BeforeFlushFinalMemtable", /*arg=*/nullptr);
// flush the final memtable (if non-empty)
if (cfd->mem()->GetFirstSequenceNumber() != 0) {
// If flush happened in the middle of recovery (e.g. due to memtable
// being full), we flush at the end. Otherwise we'll need to record
// where we were on last flush, which make the logic complicated.
if (flushed || !immutable_db_options_.avoid_flush_during_recovery) {
status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
if (!status.ok()) {
// Recovery failed
break;
}
flushed = true;
cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
versions_->LastSequence());
}
data_seen = true;
}
// Update the log number info in the version edit corresponding to this
// column family. Note that the version edits will be written to MANIFEST
// together later.
// writing wal_number in the manifest means that any log file
// with number strongly less than (wal_number + 1) is already
// recovered and should be ignored on next reincarnation.
// Since we already recovered max_wal_number, we want all wals
// with numbers `<= max_wal_number` (includes this one) to be ignored
if (flushed || cfd->mem()->GetFirstSequenceNumber() == 0) {
edit->SetLogNumber(max_wal_number + 1);
}
}
if (status.ok()) {
// we must mark the next log number as used, even though it's
// not actually used. that is because VersionSet assumes
// VersionSet::next_file_number_ always to be strictly greater than any
// log number
versions_->MarkFileNumberUsed(max_wal_number + 1);
autovector<ColumnFamilyData*> cfds;
autovector<const MutableCFOptions*> cf_opts;
autovector<autovector<VersionEdit*>> edit_lists;
for (auto* cfd : *versions_->GetColumnFamilySet()) {
cfds.push_back(cfd);
cf_opts.push_back(cfd->GetLatestMutableCFOptions());
auto iter = version_edits.find(cfd->GetID());
assert(iter != version_edits.end());
edit_lists.push_back({&iter->second});
}
std::unique_ptr<VersionEdit> wal_deletion;
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 03:41:31 +01:00
if (flushed) {
wal_deletion = std::make_unique<VersionEdit>();
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 03:41:31 +01:00
if (immutable_db_options_.track_and_verify_wals_in_manifest) {
wal_deletion->DeleteWalsBefore(max_wal_number + 1);
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 03:41:31 +01:00
}
if (!allow_2pc()) {
// In non-2pc mode, flushing the memtables of the column families
// means we can advance min_log_number_to_keep.
wal_deletion->SetMinLogNumberToKeep(max_wal_number + 1);
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 03:41:31 +01:00
}
edit_lists.back().push_back(wal_deletion.get());
}
// write MANIFEST with update
status = versions_->LogAndApply(cfds, cf_opts, edit_lists, &mutex_,
directories_.GetDbDir(),
/*new_descriptor_log=*/true);
}
}
if (status.ok()) {
if (data_seen && !flushed) {
status = RestoreAliveLogFiles(wal_numbers);
} else {
// If there's no data in the WAL, or we flushed all the data, still
// truncate the log file. If the process goes into a crash loop before
// the file is deleted, the preallocated space will never get freed.
Do not truncate WAL if in read_only mode (#8313) Summary: I noticed ```openat``` system call with ```O_WRONLY``` flag and ```sync_file_range``` and ```truncate``` on WAL file when using ```rocksdb::DB::OpenForReadOnly``` by way of ```db_bench --readonly=true --benchmarks=readseq --use_existing_db=1 --num=1 ...``` Noticed in ```strace``` after seeing the last modification time of the WAL file change after each run (with ```--readonly=true```). I think introduced by https://github.com/facebook/rocksdb/commit/7d7f14480e135a4939ed6903f46b3f7056aa837a from https://github.com/facebook/rocksdb/pull/8122 I added a test to catch the WAL file being truncated and the modification time on it changing. I am not sure if a mock filesystem with mock clock could be used to avoid having to sleep 1.1s. The test could also check the set of files is the same and that the sizes are also unchanged. Before: ``` [ RUN ] DBBasicTest.ReadOnlyReopenMtimeUnchanged db/db_basic_test.cc:182: Failure Expected equality of these values: file_mtime_after_readonly_reopen Which is: 1621611136 file_mtime_before_readonly_reopen Which is: 1621611135 file is: 000010.log [ FAILED ] DBBasicTest.ReadOnlyReopenMtimeUnchanged (1108 ms) ``` After: ``` [ RUN ] DBBasicTest.ReadOnlyReopenMtimeUnchanged [ OK ] DBBasicTest.ReadOnlyReopenMtimeUnchanged (1108 ms) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/8313 Reviewed By: pdillinger Differential Revision: D28656925 Pulled By: jay-zhuang fbshipit-source-id: ea9e215cb53e7c830e76bc5fc75c45e21f12a1d6
2021-05-27 19:24:06 +02:00
const bool truncate = !read_only;
GetLogSizeAndMaybeTruncate(wal_numbers.back(), truncate, nullptr)
.PermitUncheckedError();
}
}
event_logger_.Log() << "job" << job_id << "event"
<< "recovery_finished";
return status;
}
Status DBImpl::GetLogSizeAndMaybeTruncate(uint64_t wal_number, bool truncate,
LogFileNumberSize* log_ptr) {
LogFileNumberSize log(wal_number);
std::string fname =
LogFileName(immutable_db_options_.GetWalDir(), wal_number);
Status s;
// This gets the appear size of the wals, not including preallocated space.
s = env_->GetFileSize(fname, &log.size);
if (s.ok() && truncate) {
std::unique_ptr<FSWritableFile> last_log;
Status truncate_status = fs_->ReopenWritableFile(
fname,
fs_->OptimizeForLogWrite(
file_options_,
BuildDBOptions(immutable_db_options_, mutable_db_options_)),
&last_log, nullptr);
if (truncate_status.ok()) {
truncate_status = last_log->Truncate(log.size, IOOptions(), nullptr);
}
if (truncate_status.ok()) {
truncate_status = last_log->Close(IOOptions(), nullptr);
}
// Not a critical error if fail to truncate.
if (!truncate_status.ok() && !truncate_status.IsNotSupported()) {
ROCKS_LOG_WARN(immutable_db_options_.info_log,
"Failed to truncate log #%" PRIu64 ": %s", wal_number,
truncate_status.ToString().c_str());
}
}
if (log_ptr) {
*log_ptr = log;
}
return s;
}
Status DBImpl::RestoreAliveLogFiles(const std::vector<uint64_t>& wal_numbers) {
if (wal_numbers.empty()) {
return Status::OK();
}
Status s;
mutex_.AssertHeld();
assert(immutable_db_options_.avoid_flush_during_recovery);
if (two_write_queues_) {
log_write_mutex_.Lock();
}
// Mark these as alive so they'll be considered for deletion later by
// FindObsoleteFiles()
total_log_size_ = 0;
log_empty_ = false;
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 03:41:31 +01:00
uint64_t min_wal_with_unflushed_data =
versions_->MinLogNumberWithUnflushedData();
for (auto wal_number : wal_numbers) {
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 03:41:31 +01:00
if (!allow_2pc() && wal_number < min_wal_with_unflushed_data) {
// In non-2pc mode, the WAL files not backing unflushed data are not
// alive, thus should not be added to the alive_log_files_.
continue;
}
// We preallocate space for wals, but then after a crash and restart, those
// preallocated space are not needed anymore. It is likely only the last
// log has such preallocated space, so we only truncate for the last log.
LogFileNumberSize log;
s = GetLogSizeAndMaybeTruncate(
wal_number, /*truncate=*/(wal_number == wal_numbers.back()), &log);
if (!s.ok()) {
break;
}
total_log_size_ += log.size;
alive_log_files_.push_back(log);
}
Fix a TSAN-reported bug caused by concurrent accesss to std::deque (#9686) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9686 According to https://www.cplusplus.com/reference/deque/deque/back/, " The container is accessed (neither the const nor the non-const versions modify the container). The last element is potentially accessed or modified by the caller. Concurrently accessing or modifying other elements is safe. " Also according to https://www.cplusplus.com/reference/deque/deque/pop_front/, " The container is modified. The first element is modified. Concurrently accessing or modifying other elements is safe (although see iterator validity above). " In RocksDB, we never pop the last element of `DBImpl::alive_log_files_`. We have been exploiting this fact and the above two properties when ensuring correctness when `DBImpl::alive_log_files_` may be accessed concurrently. Specifically, it can be accessed in the write path when db mutex is released. Sometimes, the log_mute_ is held. It can also be accessed in `FindObsoleteFiles()` when db mutex is always held. It can also be accessed during recovery when db mutex is also held. Given the fact that we never pop the last element of alive_log_files_, we currently do not acquire additional locks when accessing it in `WriteToWAL()` as follows ``` alive_log_files_.back().AddSize(log_entry.size()); ``` This is problematic. Check source code of deque.h ``` back() _GLIBCXX_NOEXCEPT { __glibcxx_requires_nonempty(); ... } pop_front() _GLIBCXX_NOEXCEPT { ... if (this->_M_impl._M_start._M_cur != this->_M_impl._M_start._M_last - 1) { ... ++this->_M_impl._M_start._M_cur; } ... } ``` `back()` will actually call `__glibcxx_requires_nonempty()` first. If `__glibcxx_requires_nonempty()` is enabled and not an empty macro, it will call `empty()` ``` bool empty() { return this->_M_impl._M_finish == this->_M_impl._M_start; } ``` You can see that it will access `this->_M_impl._M_start`, racing with `pop_front()`. Therefore, TSAN will actually catch the bug in this case. To be able to use TSAN on our library and unit tests, we should always coordinate concurrent accesses to STL containers properly. We need to pass information about db mutex and log mutex into `WriteToWAL()`, otherwise it's impossible to know which mutex to acquire inside the function. To fix this, we can catch the tail of `alive_log_files_` by reference, so that we do not have to call `back()` in `WriteToWAL()`. Reviewed By: pdillinger Differential Revision: D34780309 fbshipit-source-id: 1def9821f0c437f2736c6a26445d75890377889b
2022-03-15 02:49:55 +01:00
alive_log_files_tail_ = alive_log_files_.rbegin();
if (two_write_queues_) {
log_write_mutex_.Unlock();
}
return s;
}
Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
MemTable* mem, VersionEdit* edit) {
mutex_.AssertHeld();
Fix a silent data loss for write-committed txn (#9571) Summary: The following sequence of events can cause silent data loss for write-committed transactions. ``` Time thread 1 bg flush | db->Put("a") | txn = NewTxn() | txn->Put("b", "v") | txn->Prepare() // writes only to 5.log | db->SwitchMemtable() // memtable 1 has "a" | // close 5.log, | // creates 8.log | trigger flush | pick memtable 1 | unlock db mutex | write new sst | txn->ctwb->Put("gtid", "1") // writes 8.log | txn->Commit() // writes to 8.log | // writes to memtable 2 | compute min_log_number_to_keep_2pc, this | will be 8 (incorrect). | | Purge obsolete wals, including 5.log | V ``` At this point, writes of txn exists only in memtable. Close db without flush because db thinks the data in memtable are backed by log. Then reopen, the writes are lost except key-value pair {"gtid"->"1"}, only the commit marker of txn is in 8.log The reason lies in `PrecomputeMinLogNumberToKeep2PC()` which calls `FindMinPrepLogReferencedByMemTable()`. In the above example, when bg flush thread tries to find obsolete wals, it uses the information computed by `PrecomputeMinLogNumberToKeep2PC()`. The return value of `PrecomputeMinLogNumberToKeep2PC()` depends on three components - `PrecomputeMinLogNumberToKeepNon2PC()`. This represents the WAL that has unflushed data. As the name of this method suggests, it does not account for 2PC. Although the keys reside in the prepare section of a previous WAL, the column family references the current WAL when they are actually inserted into the memtable during txn commit. - `prep_tracker->FindMinLogContainingOutstandingPrep()`. This represents the WAL with a prepare section but the txn hasn't committed. - `FindMinPrepLogReferencedByMemTable()`. This represents the WAL on which some memtables (mutable and immutable) depend for their unflushed data. The bug lies in `FindMinPrepLogReferencedByMemTable()`. Originally, this function skips checking the column families that are being flushed, but the unit test added in this PR shows that they should not be. In this unit test, there is only the default column family, and one of its memtables has unflushed data backed by a prepare section in 5.log. We should return this information via `FindMinPrepLogReferencedByMemTable()`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9571 Test Plan: ``` ./transaction_test --gtest_filter=*/TransactionTest.SwitchMemtableDuringPrepareAndCommit_WC/* make check ``` Reviewed By: siying Differential Revision: D34235236 Pulled By: riversand963 fbshipit-source-id: 120eb21a666728a38dda77b96276c6af72b008b1
2022-02-17 08:07:48 +01:00
assert(cfd);
assert(cfd->imm());
// The immutable memtable list must be empty.
assert(std::numeric_limits<uint64_t>::max() ==
cfd->imm()->GetEarliestMemTableID());
const uint64_t start_micros = immutable_db_options_.clock->NowMicros();
FileMetaData meta;
std::vector<BlobFileAddition> blob_file_additions;
std::unique_ptr<std::list<uint64_t>::iterator> pending_outputs_inserted_elem(
new std::list<uint64_t>::iterator(
CaptureCurrentFileNumberInPendingOutputs()));
meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
ReadOptions ro;
ro.total_order_seek = true;
Arena arena;
Status s;
TableProperties table_properties;
{
ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
"[%s] [WriteLevel0TableForRecovery]"
" Level-0 table #%" PRIu64 ": started",
cfd->GetName().c_str(), meta.fd.GetNumber());
// Get the latest mutable cf options while the mutex is still locked
const MutableCFOptions mutable_cf_options =
*cfd->GetLatestMutableCFOptions();
bool paranoid_file_checks =
cfd->GetLatestMutableCFOptions()->paranoid_file_checks;
FIFO Compaction with TTL Summary: Introducing FIFO compactions with TTL. FIFO compaction is based on size only which makes it tricky to enable in production as use cases can have organic growth. A user requested an option to drop files based on the time of their creation instead of the total size. To address that request: - Added a new TTL option to FIFO compaction options. - Updated FIFO compaction score to take TTL into consideration. - Added a new table property, creation_time, to keep track of when the SST file is created. - Creation_time is set as below: - On Flush: Set to the time of flush. - On Compaction: Set to the max creation_time of all the files involved in the compaction. - On Repair and Recovery: Set to the time of repair/recovery. - Old files created prior to this code change will have a creation_time of 0. - FIFO compaction with TTL is enabled when ttl > 0. All files older than ttl will be deleted during compaction. i.e. `if (file.creation_time < (current_time - ttl)) then delete(file)`. This will enable cases where you might want to delete all files older than, say, 1 day. - FIFO compaction will fall back to the prior way of deleting files based on size if: - the creation_time of all files involved in compaction is 0. - the total size (of all SST files combined) does not drop below `compaction_options_fifo.max_table_files_size` even if the files older than ttl are deleted. This feature is not supported if max_open_files != -1 or with table formats other than Block-based. **Test Plan:** Added tests. **Benchmark results:** Base: FIFO with max size: 100MB :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 readwhilewriting : 1.924 micros/op 519858 ops/sec; 13.6 MB/s (1176277 of 5000000 found) ``` With TTL (a low one for testing) :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 --fifo_compaction_ttl=20 readwhilewriting : 1.902 micros/op 525817 ops/sec; 13.7 MB/s (1185057 of 5000000 found) ``` Example Log lines: ``` 2017/06/26-15:17:24.609249 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609177) [db/compaction_picker.cc:1471] [default] FIFO compaction: picking file 40 with creation time 1498515423 for deletion 2017/06/26-15:17:24.609255 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609234) [db/db_impl_compaction_flush.cc:1541] [default] Deleted 1 files ... 2017/06/26-15:17:25.553185 7fd5a61a5800 [DEBUG] [db/db_impl_files.cc:309] [JOB 0] Delete /dev/shm/dbbench/000040.sst type=2 #40 -- OK 2017/06/26-15:17:25.553205 7fd5a61a5800 EVENT_LOG_v1 {"time_micros": 1498515445553199, "job": 0, "event": "table_file_deletion", "file_number": 40} ``` SST Files remaining in the dbbench dir, after db_bench execution completed: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ ls -l /dev/shm//dbbench/*.sst -rw-r--r--. 1 svemuri users 30749887 Jun 26 15:17 /dev/shm//dbbench/000042.sst -rw-r--r--. 1 svemuri users 30768779 Jun 26 15:17 /dev/shm//dbbench/000044.sst -rw-r--r--. 1 svemuri users 30757481 Jun 26 15:17 /dev/shm//dbbench/000046.sst ``` Closes https://github.com/facebook/rocksdb/pull/2480 Differential Revision: D5305116 Pulled By: sagar0 fbshipit-source-id: 3e5cfcf5dd07ed2211b5b37492eb235b45139174
2017-06-28 02:02:20 +02:00
int64_t _current_time = 0;
immutable_db_options_.clock->GetCurrentTime(&_current_time)
.PermitUncheckedError(); // ignore error
FIFO Compaction with TTL Summary: Introducing FIFO compactions with TTL. FIFO compaction is based on size only which makes it tricky to enable in production as use cases can have organic growth. A user requested an option to drop files based on the time of their creation instead of the total size. To address that request: - Added a new TTL option to FIFO compaction options. - Updated FIFO compaction score to take TTL into consideration. - Added a new table property, creation_time, to keep track of when the SST file is created. - Creation_time is set as below: - On Flush: Set to the time of flush. - On Compaction: Set to the max creation_time of all the files involved in the compaction. - On Repair and Recovery: Set to the time of repair/recovery. - Old files created prior to this code change will have a creation_time of 0. - FIFO compaction with TTL is enabled when ttl > 0. All files older than ttl will be deleted during compaction. i.e. `if (file.creation_time < (current_time - ttl)) then delete(file)`. This will enable cases where you might want to delete all files older than, say, 1 day. - FIFO compaction will fall back to the prior way of deleting files based on size if: - the creation_time of all files involved in compaction is 0. - the total size (of all SST files combined) does not drop below `compaction_options_fifo.max_table_files_size` even if the files older than ttl are deleted. This feature is not supported if max_open_files != -1 or with table formats other than Block-based. **Test Plan:** Added tests. **Benchmark results:** Base: FIFO with max size: 100MB :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 readwhilewriting : 1.924 micros/op 519858 ops/sec; 13.6 MB/s (1176277 of 5000000 found) ``` With TTL (a low one for testing) :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 --fifo_compaction_ttl=20 readwhilewriting : 1.902 micros/op 525817 ops/sec; 13.7 MB/s (1185057 of 5000000 found) ``` Example Log lines: ``` 2017/06/26-15:17:24.609249 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609177) [db/compaction_picker.cc:1471] [default] FIFO compaction: picking file 40 with creation time 1498515423 for deletion 2017/06/26-15:17:24.609255 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609234) [db/db_impl_compaction_flush.cc:1541] [default] Deleted 1 files ... 2017/06/26-15:17:25.553185 7fd5a61a5800 [DEBUG] [db/db_impl_files.cc:309] [JOB 0] Delete /dev/shm/dbbench/000040.sst type=2 #40 -- OK 2017/06/26-15:17:25.553205 7fd5a61a5800 EVENT_LOG_v1 {"time_micros": 1498515445553199, "job": 0, "event": "table_file_deletion", "file_number": 40} ``` SST Files remaining in the dbbench dir, after db_bench execution completed: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ ls -l /dev/shm//dbbench/*.sst -rw-r--r--. 1 svemuri users 30749887 Jun 26 15:17 /dev/shm//dbbench/000042.sst -rw-r--r--. 1 svemuri users 30768779 Jun 26 15:17 /dev/shm//dbbench/000044.sst -rw-r--r--. 1 svemuri users 30757481 Jun 26 15:17 /dev/shm//dbbench/000046.sst ``` Closes https://github.com/facebook/rocksdb/pull/2480 Differential Revision: D5305116 Pulled By: sagar0 fbshipit-source-id: 3e5cfcf5dd07ed2211b5b37492eb235b45139174
2017-06-28 02:02:20 +02:00
const uint64_t current_time = static_cast<uint64_t>(_current_time);
meta.oldest_ancester_time = current_time;
FIFO Compaction with TTL Summary: Introducing FIFO compactions with TTL. FIFO compaction is based on size only which makes it tricky to enable in production as use cases can have organic growth. A user requested an option to drop files based on the time of their creation instead of the total size. To address that request: - Added a new TTL option to FIFO compaction options. - Updated FIFO compaction score to take TTL into consideration. - Added a new table property, creation_time, to keep track of when the SST file is created. - Creation_time is set as below: - On Flush: Set to the time of flush. - On Compaction: Set to the max creation_time of all the files involved in the compaction. - On Repair and Recovery: Set to the time of repair/recovery. - Old files created prior to this code change will have a creation_time of 0. - FIFO compaction with TTL is enabled when ttl > 0. All files older than ttl will be deleted during compaction. i.e. `if (file.creation_time < (current_time - ttl)) then delete(file)`. This will enable cases where you might want to delete all files older than, say, 1 day. - FIFO compaction will fall back to the prior way of deleting files based on size if: - the creation_time of all files involved in compaction is 0. - the total size (of all SST files combined) does not drop below `compaction_options_fifo.max_table_files_size` even if the files older than ttl are deleted. This feature is not supported if max_open_files != -1 or with table formats other than Block-based. **Test Plan:** Added tests. **Benchmark results:** Base: FIFO with max size: 100MB :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 readwhilewriting : 1.924 micros/op 519858 ops/sec; 13.6 MB/s (1176277 of 5000000 found) ``` With TTL (a low one for testing) :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 --fifo_compaction_ttl=20 readwhilewriting : 1.902 micros/op 525817 ops/sec; 13.7 MB/s (1185057 of 5000000 found) ``` Example Log lines: ``` 2017/06/26-15:17:24.609249 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609177) [db/compaction_picker.cc:1471] [default] FIFO compaction: picking file 40 with creation time 1498515423 for deletion 2017/06/26-15:17:24.609255 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609234) [db/db_impl_compaction_flush.cc:1541] [default] Deleted 1 files ... 2017/06/26-15:17:25.553185 7fd5a61a5800 [DEBUG] [db/db_impl_files.cc:309] [JOB 0] Delete /dev/shm/dbbench/000040.sst type=2 #40 -- OK 2017/06/26-15:17:25.553205 7fd5a61a5800 EVENT_LOG_v1 {"time_micros": 1498515445553199, "job": 0, "event": "table_file_deletion", "file_number": 40} ``` SST Files remaining in the dbbench dir, after db_bench execution completed: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ ls -l /dev/shm//dbbench/*.sst -rw-r--r--. 1 svemuri users 30749887 Jun 26 15:17 /dev/shm//dbbench/000042.sst -rw-r--r--. 1 svemuri users 30768779 Jun 26 15:17 /dev/shm//dbbench/000044.sst -rw-r--r--. 1 svemuri users 30757481 Jun 26 15:17 /dev/shm//dbbench/000046.sst ``` Closes https://github.com/facebook/rocksdb/pull/2480 Differential Revision: D5305116 Pulled By: sagar0 fbshipit-source-id: 3e5cfcf5dd07ed2211b5b37492eb235b45139174
2017-06-28 02:02:20 +02:00
{
auto write_hint = cfd->CalculateSSTWriteHint(0);
mutex_.Unlock();
SequenceNumber earliest_write_conflict_snapshot;
std::vector<SequenceNumber> snapshot_seqs =
snapshots_.GetAll(&earliest_write_conflict_snapshot);
auto snapshot_checker = snapshot_checker_.get();
if (use_custom_gc_ && snapshot_checker == nullptr) {
snapshot_checker = DisableGCSnapshotChecker::Instance();
}
std::vector<std::unique_ptr<FragmentedRangeTombstoneIterator>>
range_del_iters;
auto range_del_iter =
mem->NewRangeTombstoneIterator(ro, kMaxSequenceNumber);
if (range_del_iter != nullptr) {
range_del_iters.emplace_back(range_del_iter);
}
IOStatus io_s;
TableBuilderOptions tboptions(
*cfd->ioptions(), mutable_cf_options, cfd->internal_comparator(),
cfd->int_tbl_prop_collector_factories(),
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options),
Add more LSM info to FilterBuildingContext (#8246) Summary: Add `num_levels`, `is_bottommost`, and table file creation `reason` to `FilterBuildingContext`, in anticipation of more powerful Bloom-like filter support. To support this, added `is_bottommost` and `reason` to `TableBuilderOptions`, which allowed removing `reason` parameter from `rocksdb::BuildTable`. I attempted to remove `skip_filters` from `TableBuilderOptions`, because filter construction decisions should arise from options, not one-off parameters. I could not completely remove it because the public API for SstFileWriter takes a `skip_filters` parameter, and translating this into an option change would mean awkwardly replacing the table_factory if it is BlockBasedTableFactory with new filter_policy=nullptr option. I marked this public skip_filters option as deprecated because of this oddity. (skip_filters on the read side probably makes sense.) At least `skip_filters` is now largely hidden for users of `TableBuilderOptions` and is no longer used for implementing the optimize_filters_for_hits option. Bringing the logic for that option closer to handling of FilterBuildingContext makes it more obvious that hese two are using the same notion of "bottommost." (Planned: configuration options for Bloom-like filters that generalize `optimize_filters_for_hits`) Recommended follow-up: Try to get away from "bottommost level" naming of things, which is inaccurate (see VersionStorageInfo::RangeMightExistAfterSortedRun), and move to "bottommost run" or just "bottommost." Pull Request resolved: https://github.com/facebook/rocksdb/pull/8246 Test Plan: extended an existing unit test to exercise and check various filter building contexts. Also, existing tests for optimize_filters_for_hits validate some of the "bottommost" handling, which is now closely connected to FilterBuildingContext::is_bottommost through TableBuilderOptions::is_bottommost Reviewed By: mrambacher Differential Revision: D28099346 Pulled By: pdillinger fbshipit-source-id: 2c1072e29c24d4ac404c761a7b7663292372600a
2021-04-30 22:49:24 +02:00
mutable_cf_options.compression_opts, cfd->GetID(), cfd->GetName(),
0 /* level */, false /* is_bottommost */,
TableFileCreationReason::kRecovery, current_time,
0 /* oldest_key_time */, 0 /* file_creation_time */, db_id_,
db_session_id_, 0 /* target_file_size */, meta.fd.GetNumber());
s = BuildTable(
dbname_, versions_.get(), immutable_db_options_, tboptions,
file_options_for_compaction_, cfd->table_cache(), iter.get(),
std::move(range_del_iters), &meta, &blob_file_additions,
CompactionIterator sees consistent view of which keys are committed (#9830) Summary: **This PR does not affect the functionality of `DB` and write-committed transactions.** `CompactionIterator` uses `KeyCommitted(seq)` to determine if a key in the database is committed. As the name 'write-committed' implies, if write-committed policy is used, a key exists in the database only if it is committed. In fact, the implementation of `KeyCommitted()` is as follows: ``` inline bool KeyCommitted(SequenceNumber seq) { // For non-txn-db and write-committed, snapshot_checker_ is always nullptr. return snapshot_checker_ == nullptr || snapshot_checker_->CheckInSnapshot(seq, kMaxSequence) == SnapshotCheckerResult::kInSnapshot; } ``` With that being said, we focus on write-prepared/write-unprepared transactions. A few notes: - A key can exist in the db even if it's uncommitted. Therefore, we rely on `snapshot_checker_` to determine data visibility. We also require that all writes go through transaction API instead of the raw `WriteBatch` + `Write`, thus at most one uncommitted version of one user key can exist in the database. - `CompactionIterator` outputs a key as long as the key is uncommitted. Due to the above reasons, it is possible that `CompactionIterator` decides to output an uncommitted key without doing further checks on the key (`NextFromInput()`). By the time the key is being prepared for output, the key becomes committed because the `snapshot_checker_(seq, kMaxSequence)` becomes true in the implementation of `KeyCommitted()`. Then `CompactionIterator` will try to zero its sequence number and hit assertion error if the key is a tombstone. To fix this issue, we should make the `CompactionIterator` see a consistent view of the input keys. Note that for write-prepared/write-unprepared, the background flush/compaction jobs already take a "job snapshot" before starting processing keys. The job snapshot is released only after the entire flush/compaction finishes. We can use this snapshot to determine whether a key is committed or not with minor change to `KeyCommitted()`. ``` inline bool KeyCommitted(SequenceNumber sequence) { // For non-txn-db and write-committed, snapshot_checker_ is always nullptr. return snapshot_checker_ == nullptr || snapshot_checker_->CheckInSnapshot(sequence, job_snapshot_) == SnapshotCheckerResult::kInSnapshot; } ``` As a result, whether a key is committed or not will remain a constant throughout compaction, causing no trouble for `CompactionIterator`s assertions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9830 Test Plan: make check Reviewed By: ltamasi Differential Revision: D35561162 Pulled By: riversand963 fbshipit-source-id: 0e00d200c195240341cfe6d34cbc86798b315b9f
2022-04-14 20:11:04 +02:00
snapshot_seqs, earliest_write_conflict_snapshot, kMaxSequenceNumber,
snapshot_checker, paranoid_file_checks, cfd->internal_stats(), &io_s,
io_tracer_, BlobFileCreationReason::kRecovery, &event_logger_, job_id,
Env::IO_HIGH, nullptr /* table_properties */, write_hint,
nullptr /*full_history_ts_low*/, &blob_callback_);
LogFlush(immutable_db_options_.info_log);
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
"[%s] [WriteLevel0TableForRecovery]"
" Level-0 table #%" PRIu64 ": %" PRIu64 " bytes %s",
cfd->GetName().c_str(), meta.fd.GetNumber(),
meta.fd.GetFileSize(), s.ToString().c_str());
mutex_.Lock();
// TODO(AR) is this ok?
if (!io_s.ok() && s.ok()) {
s = io_s;
}
}
}
ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
// Note that if file_size is zero, the file has been deleted and
// should not be added to the manifest.
const bool has_output = meta.fd.GetFileSize() > 0;
constexpr int level = 0;
if (s.ok() && has_output) {
edit->AddFile(
level, meta.fd.GetNumber(), meta.fd.GetPathId(), meta.fd.GetFileSize(),
meta.smallest, meta.largest, meta.fd.smallest_seqno,
meta.fd.largest_seqno, meta.marked_for_compaction, meta.temperature,
meta.oldest_blob_file_number, meta.oldest_ancester_time,
meta.file_creation_time, meta.file_checksum,
meta.file_checksum_func_name, meta.min_timestamp, meta.max_timestamp);
for (const auto& blob : blob_file_additions) {
edit->AddBlobFile(blob);
}
}
InternalStats::CompactionStats stats(CompactionReason::kFlush, 1);
stats.micros = immutable_db_options_.clock->NowMicros() - start_micros;
if (has_output) {
stats.bytes_written = meta.fd.GetFileSize();
stats.num_output_files = 1;
}
const auto& blobs = edit->GetBlobFileAdditions();
for (const auto& blob : blobs) {
stats.bytes_written_blob += blob.GetTotalBlobBytes();
}
stats.num_output_files_blob = static_cast<int>(blobs.size());
cfd->internal_stats()->AddCompactionStats(level, Env::Priority::USER, stats);
cfd->internal_stats()->AddCFStats(
InternalStats::BYTES_FLUSHED,
stats.bytes_written + stats.bytes_written_blob);
RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
return s;
}
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
DBOptions db_options(options);
ColumnFamilyOptions cf_options(options);
std::vector<ColumnFamilyDescriptor> column_families;
column_families.push_back(
ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
if (db_options.persist_stats_to_disk) {
column_families.push_back(
ColumnFamilyDescriptor(kPersistentStatsColumnFamilyName, cf_options));
}
std::vector<ColumnFamilyHandle*> handles;
Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
if (s.ok()) {
if (db_options.persist_stats_to_disk) {
assert(handles.size() == 2);
} else {
assert(handles.size() == 1);
}
// i can delete the handle since DBImpl is always holding a reference to
// default column family
if (db_options.persist_stats_to_disk && handles[1] != nullptr) {
delete handles[1];
}
delete handles[0];
}
return s;
}
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
const std::vector<ColumnFamilyDescriptor>& column_families,
std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
const bool kSeqPerBatch = true;
const bool kBatchPerTxn = true;
return DBImpl::Open(db_options, dbname, column_families, handles, dbptr,
!kSeqPerBatch, kBatchPerTxn);
}
// TODO: Implement the trimming in flush code path.
// TODO: Perform trimming before inserting into memtable during recovery.
// TODO: Pick files with max_timestamp > trim_ts by each file's timestamp meta
// info, and handle only these files to reduce io.
Status DB::OpenAndTrimHistory(
const DBOptions& db_options, const std::string& dbname,
const std::vector<ColumnFamilyDescriptor>& column_families,
std::vector<ColumnFamilyHandle*>* handles, DB** dbptr,
std::string trim_ts) {
assert(dbptr != nullptr);
assert(handles != nullptr);
auto validate_options = [&db_options] {
if (db_options.avoid_flush_during_recovery) {
return Status::InvalidArgument(
"avoid_flush_during_recovery incompatible with "
"OpenAndTrimHistory");
}
return Status::OK();
};
auto s = validate_options();
if (!s.ok()) {
return s;
}
DB* db = nullptr;
s = DB::Open(db_options, dbname, column_families, handles, &db);
if (!s.ok()) {
return s;
}
assert(db);
CompactRangeOptions options;
options.bottommost_level_compaction =
BottommostLevelCompaction::kForceOptimized;
auto db_impl = static_cast_with_check<DBImpl>(db);
for (auto handle : *handles) {
assert(handle != nullptr);
auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(handle);
auto cfd = cfh->cfd();
assert(cfd != nullptr);
// Only compact column families with timestamp enabled
if (cfd->user_comparator() != nullptr &&
cfd->user_comparator()->timestamp_size() > 0) {
s = db_impl->CompactRangeInternal(options, handle, nullptr, nullptr,
trim_ts);
if (!s.ok()) {
break;
}
}
}
auto clean_op = [&handles, &db] {
for (auto handle : *handles) {
auto temp_s = db->DestroyColumnFamilyHandle(handle);
assert(temp_s.ok());
}
handles->clear();
delete db;
};
if (!s.ok()) {
clean_op();
return s;
}
*dbptr = db;
return s;
}
IOStatus DBImpl::CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number,
size_t preallocate_block_size,
log::Writer** new_log) {
IOStatus io_s;
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
std::unique_ptr<FSWritableFile> lfile;
DBOptions db_options =
BuildDBOptions(immutable_db_options_, mutable_db_options_);
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
FileOptions opt_file_options =
fs_->OptimizeForLogWrite(file_options_, db_options);
std::string wal_dir = immutable_db_options_.GetWalDir();
std::string log_fname = LogFileName(wal_dir, log_file_num);
if (recycle_log_number) {
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"reusing log %" PRIu64 " from recycle list\n",
recycle_log_number);
std::string old_log_fname = LogFileName(wal_dir, recycle_log_number);
TEST_SYNC_POINT("DBImpl::CreateWAL:BeforeReuseWritableFile1");
TEST_SYNC_POINT("DBImpl::CreateWAL:BeforeReuseWritableFile2");
io_s = fs_->ReuseWritableFile(log_fname, old_log_fname, opt_file_options,
&lfile, /*dbg=*/nullptr);
} else {
io_s = NewWritableFile(fs_.get(), log_fname, &lfile, opt_file_options);
}
if (io_s.ok()) {
lfile->SetWriteLifeTimeHint(CalculateWALWriteHint());
lfile->SetPreallocationBlockSize(preallocate_block_size);
const auto& listeners = immutable_db_options_.listeners;
FileTypeSet tmp_set = immutable_db_options_.checksum_handoff_file_types;
std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
std::move(lfile), log_fname, opt_file_options,
immutable_db_options_.clock, io_tracer_, nullptr /* stats */, listeners,
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
nullptr, tmp_set.Contains(FileType::kWalFile),
tmp_set.Contains(FileType::kWalFile)));
*new_log = new log::Writer(std::move(file_writer), log_file_num,
immutable_db_options_.recycle_log_file_num > 0,
immutable_db_options_.manual_wal_flush,
immutable_db_options_.wal_compression);
io_s = (*new_log)->AddCompressionTypeRecord();
}
return io_s;
}
Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname,
const std::vector<ColumnFamilyDescriptor>& column_families,
std::vector<ColumnFamilyHandle*>* handles, DB** dbptr,
const bool seq_per_batch, const bool batch_per_txn) {
Status s = ValidateOptionsByTable(db_options, column_families);
if (!s.ok()) {
return s;
}
s = ValidateOptions(db_options, column_families);
if (!s.ok()) {
return s;
}
*dbptr = nullptr;
handles->clear();
size_t max_write_buffer_size = 0;
for (auto cf : column_families) {
max_write_buffer_size =
std::max(max_write_buffer_size, cf.options.write_buffer_size);
}
DBImpl* impl = new DBImpl(db_options, dbname, seq_per_batch, batch_per_txn);
s = impl->env_->CreateDirIfMissing(impl->immutable_db_options_.GetWalDir());
if (s.ok()) {
std::vector<std::string> paths;
for (auto& db_path : impl->immutable_db_options_.db_paths) {
paths.emplace_back(db_path.path);
}
for (auto& cf : column_families) {
for (auto& cf_path : cf.options.cf_paths) {
paths.emplace_back(cf_path.path);
}
}
for (auto& path : paths) {
s = impl->env_->CreateDirIfMissing(path);
if (!s.ok()) {
break;
}
}
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 22:36:19 +02:00
// For recovery from NoSpace() error, we can only handle
// the case where the database is stored in a single path
if (paths.size() <= 1) {
impl->error_handler_.EnableAutoRecovery();
}
}
if (s.ok()) {
s = impl->CreateArchivalDirectory();
}
if (!s.ok()) {
delete impl;
return s;
}
impl->wal_in_db_path_ = impl->immutable_db_options_.IsWalDirSameAsDBPath();
impl->mutex_.Lock();
// Handles create_if_missing, error_if_exists
uint64_t recovered_seq(kMaxSequenceNumber);
s = impl->Recover(column_families, false, false, false, &recovered_seq);
if (s.ok()) {
uint64_t new_log_number = impl->versions_->NewFileNumber();
log::Writer* new_log = nullptr;
const size_t preallocate_block_size =
impl->GetWalPreallocateBlockSize(max_write_buffer_size);
s = impl->CreateWAL(new_log_number, 0 /*recycle_log_number*/,
preallocate_block_size, &new_log);
if (s.ok()) {
InstrumentedMutexLock wl(&impl->log_write_mutex_);
impl->logfile_number_ = new_log_number;
assert(new_log != nullptr);
Do not track obsolete WALs in MANIFEST even if they are synced (#7725) Summary: Consider the case: 1. All column families are flushed, so all WALs become obsolete, but no WAL is removed from disk yet because the removal is asynchronous, a VersionEdit is written to MANIFEST indicating that WALs before a certain WAL number are obsolete, let's say this number is 3; 2. `SyncWAL` is called, so all the on-disk WALs are synced, and if track_and_verify_wal_in_manifest=true, the WALs will be tracked in MANIFEST, let's say the WAL numbers are 1 and 2; 3. DB crashes; 4. During DB recovery, when replaying MANIFEST, we first see that WAL with number < 3 are obsolete, then we see that WAL 1 and 2 are synced, so according to current implementation of `WalSet`, the `WalSet` will be recovered to include WAL 1 and 2; 5. WAL 1 and 2 are asynchronously deleted from disk, then the WAL verification algorithm fails with `Corruption: missing WAL`. The above case is reproduced in a new unit test `DBBasicTestTrackWal::DoNotTrackObsoleteWal`. The fix is to maintain the upper bound of the obsolete WAL numbers, any WAL with number less than the maintained number is considered to be obsolete, so shouldn't be tracked even if they are later synced. The number is maintained in `WalSet`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7725 Test Plan: 1. a new unit test `DBBasicTestTrackWal::DoNotTrackObsoleteWal` is added. 2. run `make crash_test` on devserver. Reviewed By: riversand963 Differential Revision: D25238914 Pulled By: cheng-chang fbshipit-source-id: f5dccd57c3d89f19565ec5731f2d42f06d272b72
2020-12-08 19:56:50 +01:00
assert(impl->logs_.empty());
impl->logs_.emplace_back(new_log_number, new_log);
}
if (s.ok()) {
// set column family handles
for (auto cf : column_families) {
auto cfd =
impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
if (cfd != nullptr) {
handles->push_back(
new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
impl->NewThreadStatusCfInfo(cfd);
} else {
if (db_options.create_missing_column_families) {
// missing column family, create it
ColumnFamilyHandle* handle;
impl->mutex_.Unlock();
s = impl->CreateColumnFamily(cf.options, cf.name, &handle);
impl->mutex_.Lock();
if (s.ok()) {
handles->push_back(handle);
} else {
break;
}
} else {
s = Status::InvalidArgument("Column family not found", cf.name);
break;
}
}
}
}
if (s.ok()) {
SuperVersionContext sv_context(/* create_superversion */ true);
for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
impl->InstallSuperVersionAndScheduleWork(
cfd, &sv_context, *cfd->GetLatestMutableCFOptions());
}
sv_context.Clean();
if (impl->two_write_queues_) {
impl->log_write_mutex_.Lock();
}
impl->alive_log_files_.push_back(
DBImpl::LogFileNumberSize(impl->logfile_number_));
Fix a TSAN-reported bug caused by concurrent accesss to std::deque (#9686) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9686 According to https://www.cplusplus.com/reference/deque/deque/back/, " The container is accessed (neither the const nor the non-const versions modify the container). The last element is potentially accessed or modified by the caller. Concurrently accessing or modifying other elements is safe. " Also according to https://www.cplusplus.com/reference/deque/deque/pop_front/, " The container is modified. The first element is modified. Concurrently accessing or modifying other elements is safe (although see iterator validity above). " In RocksDB, we never pop the last element of `DBImpl::alive_log_files_`. We have been exploiting this fact and the above two properties when ensuring correctness when `DBImpl::alive_log_files_` may be accessed concurrently. Specifically, it can be accessed in the write path when db mutex is released. Sometimes, the log_mute_ is held. It can also be accessed in `FindObsoleteFiles()` when db mutex is always held. It can also be accessed during recovery when db mutex is also held. Given the fact that we never pop the last element of alive_log_files_, we currently do not acquire additional locks when accessing it in `WriteToWAL()` as follows ``` alive_log_files_.back().AddSize(log_entry.size()); ``` This is problematic. Check source code of deque.h ``` back() _GLIBCXX_NOEXCEPT { __glibcxx_requires_nonempty(); ... } pop_front() _GLIBCXX_NOEXCEPT { ... if (this->_M_impl._M_start._M_cur != this->_M_impl._M_start._M_last - 1) { ... ++this->_M_impl._M_start._M_cur; } ... } ``` `back()` will actually call `__glibcxx_requires_nonempty()` first. If `__glibcxx_requires_nonempty()` is enabled and not an empty macro, it will call `empty()` ``` bool empty() { return this->_M_impl._M_finish == this->_M_impl._M_start; } ``` You can see that it will access `this->_M_impl._M_start`, racing with `pop_front()`. Therefore, TSAN will actually catch the bug in this case. To be able to use TSAN on our library and unit tests, we should always coordinate concurrent accesses to STL containers properly. We need to pass information about db mutex and log mutex into `WriteToWAL()`, otherwise it's impossible to know which mutex to acquire inside the function. To fix this, we can catch the tail of `alive_log_files_` by reference, so that we do not have to call `back()` in `WriteToWAL()`. Reviewed By: pdillinger Differential Revision: D34780309 fbshipit-source-id: 1def9821f0c437f2736c6a26445d75890377889b
2022-03-15 02:49:55 +01:00
impl->alive_log_files_tail_ = impl->alive_log_files_.rbegin();
if (impl->two_write_queues_) {
impl->log_write_mutex_.Unlock();
}
}
if (s.ok()) {
// In WritePrepared there could be gap in sequence numbers. This breaks
// the trick we use in kPointInTimeRecovery which assumes the first seq in
// the log right after the corrupted log is one larger than the last seq
// we read from the wals. To let this trick keep working, we add a dummy
// entry with the expected sequence to the first log right after recovery.
// In non-WritePrepared case also the new log after recovery could be
// empty, and thus missing the consecutive seq hint to distinguish
// middle-log corruption to corrupted-log-remained-after-recovery. This
// case also will be addressed by a dummy write.
if (recovered_seq != kMaxSequenceNumber) {
WriteBatch empty_batch;
WriteBatchInternal::SetSequence(&empty_batch, recovered_seq);
WriteOptions write_options;
uint64_t log_used, log_size;
log::Writer* log_writer = impl->logs_.back().writer;
Rate-limit automatic WAL flush after each user write (#9607) Summary: **Context:** WAL flush is currently not rate-limited by `Options::rate_limiter`. This PR is to provide rate-limiting to auto WAL flush, the one that automatically happen after each user write operation (i.e, `Options::manual_wal_flush == false`), by adding `WriteOptions::rate_limiter_options`. Note that we are NOT rate-limiting WAL flush that do NOT automatically happen after each user write, such as `Options::manual_wal_flush == true + manual FlushWAL()` (rate-limiting multiple WAL flushes), for the benefits of: - being consistent with [ReadOptions::rate_limiter_priority](https://github.com/facebook/rocksdb/blob/7.0.fb/include/rocksdb/options.h#L515) - being able to turn off some WAL flush's rate-limiting but not all (e.g, turn off specific the WAL flush of a critical user write like a service's heartbeat) `WriteOptions::rate_limiter_options` only accept `Env::IO_USER` and `Env::IO_TOTAL` currently due to an implementation constraint. - The constraint is that we currently queue parallel writes (including WAL writes) based on FIFO policy which does not factor rate limiter priority into this layer's scheduling. If we allow lower priorities such as `Env::IO_HIGH/MID/LOW` and such writes specified with lower priorities occurs before ones specified with higher priorities (even just by a tiny bit in arrival time), the former would have blocked the latter, leading to a "priority inversion" issue and contradictory to what we promise for rate-limiting priority. Therefore we only allow `Env::IO_USER` and `Env::IO_TOTAL` right now before improving that scheduling. A pre-requisite to this feature is to support operation-level rate limiting in `WritableFileWriter`, which is also included in this PR. **Summary:** - Renamed test suite `DBRateLimiterTest to DBRateLimiterOnReadTest` for adding a new test suite - Accept `rate_limiter_priority` in `WritableFileWriter`'s private and public write functions - Passed `WriteOptions::rate_limiter_options` to `WritableFileWriter` in the path of automatic WAL flush. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9607 Test Plan: - Added new unit test to verify existing flush/compaction rate-limiting does not break, since `DBTest, RateLimitingTest` is disabled and current db-level rate-limiting tests focus on read only (e.g, `db_rate_limiter_test`, `DBTest2, RateLimitedCompactionReads`). - Added new unit test `DBRateLimiterOnWriteWALTest, AutoWalFlush` - `strace -ftt -e trace=write ./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -rate_limit_auto_wal_flush=1 -rate_limiter_bytes_per_sec=15 -rate_limiter_refill_period_us=1000000 -write_buffer_size=100000000 -disable_auto_compactions=1 -num=100` - verified that WAL flush(i.e, system-call _write_) were chunked into 15 bytes and each _write_ was roughly 1 second apart - verified the chunking disappeared when `-rate_limit_auto_wal_flush=0` - crash test: `python3 tools/db_crashtest.py blackbox --disable_wal=0 --rate_limit_auto_wal_flush=1 --rate_limiter_bytes_per_sec=10485760 --interval=10` killed as normal **Benchmarked on flush/compaction to ensure no performance regression:** - compaction with rate-limiting (see table 1, avg over 1280-run): pre-change: **915635 micros/op**; post-change: **907350 micros/op (improved by 0.106%)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f compact_bmk_output.txt compact_bmk_output_2.txt dont_care_output.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench --benchmarks=fillrandom -db=$TEST_TMPDIR -disable_auto_compactions=1 -write_buffer_size=6710886 > dont_care_output.txt && ./db_bench --benchmarks=compact -use_existing_db=1 -db=$TEST_TMPDIR -level0_file_num_compaction_trigger=1 -rate_limiter_bytes_per_sec=100000000 | egrep 'compact' done > compact_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' compact_bmk_output.txt >> compact_bmk_output_2.txt done ``` - compaction w/o rate-limiting (see table 2, avg over 640-run): pre-change: **822197 micros/op**; post-change: **823148 micros/op (regressed by 0.12%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` - flush with rate-limiting (see table 3, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench ): pre-change: **745752 micros/op**; post-change: **745331 micros/op (regressed by 0.06 %)** ``` #!/bin/bash TEST_TMPDIR=/dev/shm/testdb START=1 NUM_DATA_ENTRY=8 N=10 rm -f flush_bmk_output.txt flush_bmk_output_2.txt for i in $(eval echo "{$START..$NUM_DATA_ENTRY}") do NUM_RUN=$(($N*(2**($i-1)))) for j in $(eval echo "{$START..$NUM_RUN}") do ./db_bench -db=$TEST_TMPDIR -write_buffer_size=1048576000 -num=1000000 -rate_limiter_bytes_per_sec=100000000 -benchmarks=fillseq,flush | egrep 'flush' done > flush_bmk_output.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' flush_bmk_output.txt >> flush_bmk_output_2.txt done ``` - flush w/o rate-limiting (see table 4, avg over 320-run, run on the [patch](https://github.com/hx235/rocksdb/commit/ee5c6023a9f6533fab9afdc681568daa21da4953) to augment current db_bench): pre-change: **487512 micros/op**, post-change: **485856 micors/ops (improved by 0.34%)** ``` Same as above script, except that -rate_limiter_bytes_per_sec=0 ``` | table 1 - compact with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 896978 | 16046.9 | 901242 | 15670.9 | 0.475373978 20 | 893718 | 15813 | 886505 | 17544.7 | -0.8070778478 40 | 900426 | 23882.2 | 894958 | 15104.5 | -0.6072681153 80 | 906635 | 21761.5 | 903332 | 23948.3 | -0.3643141948 160 | 898632 | 21098.9 | 907583 | 21145 | 0.9960695813 3.20E+02 | 905252 | 22785.5 | 908106 | 25325.5 | 0.3152713278 6.40E+02 | 905213 | 23598.6 | 906741 | 21370.5 | 0.1688000504 **1.28E+03** | **908316** | **23533.1** | **907350** | **24626.8** | **-0.1063506533** average over #-run | 901896.25 | 21064.9625 | 901977.125 | 20592.025 | 0.008967217682 | table 2 - compact w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 811211 | 26996.7 | 807586 | 28456.4 | -0.4468627768 20 | 815465 | 14803.7 | 814608 | 28719.7 | -0.105093413 40 | 809203 | 26187.1 | 797835 | 25492.1 | -1.404839082 80 | 822088 | 28765.3 | 822192 | 32840.4 | 0.01265071379 160 | 821719 | 36344.7 | 821664 | 29544.9 | -0.006693285661 3.20E+02 | 820921 | 27756.4 | 821403 | 28347.7 | 0.05871454135 **6.40E+02** | **822197** | **28960.6** | **823148** | **30055.1** | **0.1156657103** average over #-run | 8.18E+05 | 2.71E+04 | 8.15E+05 | 2.91E+04 | -0.25 | table 3 - flush with rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 741721 | 11770.8 | 740345 | 5949.76 | -0.1855144994 20 | 735169 | 3561.83 | 743199 | 9755.77 | 1.09226586 40 | 743368 | 8891.03 | 742102 | 8683.22 | -0.1703059588 80 | 742129 | 8148.51 | 743417 | 9631.58| 0.1735547324 160 | 749045 | 9757.21 | 746256 | 9191.86 | -0.3723407806 **3.20E+02** | **745752** | **9819.65** | **745331** | **9840.62** | **-0.0564530836** 6.40E+02 | 749006 | 11080.5 | 748173 | 10578.7 | -0.1112140624 average over #-run | 743741.4286 | 9004.218571 | 744117.5714 | 9090.215714 | 0.05057441238 | table 4 - flush w/o rate-limiting| #-run | (pre-change) avg micros/op | std micros/op | (post-change) avg micros/op | std micros/op | change in avg micros/op (%) -- | -- | -- | -- | -- | -- 10 | 477283 | 24719.6 | 473864 | 12379 | -0.7163464863 20 | 486743 | 20175.2 | 502296 | 23931.3 | 3.195320734 40 | 482846 | 15309.2 | 489820 | 22259.5 | 1.444352858 80 | 491490 | 21883.1 | 490071 | 23085.7 | -0.2887139108 160 | 493347 | 28074.3 | 483609 | 21211.7 | -1.973864238 **3.20E+02** | **487512** | **21401.5** | **485856** | **22195.2** | **-0.3396839462** 6.40E+02 | 490307 | 25418.6 | 485435 | 22405.2 | -0.9936631539 average over #-run | 4.87E+05 | 2.24E+04 | 4.87E+05 | 2.11E+04 | 0.00E+00 Reviewed By: ajkr Differential Revision: D34442441 Pulled By: hx235 fbshipit-source-id: 4790f13e1e5c0a95ae1d1cc93ffcf69dc6e78bdd
2022-03-08 22:19:39 +01:00
s = impl->WriteToWAL(empty_batch, log_writer, &log_used, &log_size,
Fix a TSAN-reported bug caused by concurrent accesss to std::deque (#9686) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9686 According to https://www.cplusplus.com/reference/deque/deque/back/, " The container is accessed (neither the const nor the non-const versions modify the container). The last element is potentially accessed or modified by the caller. Concurrently accessing or modifying other elements is safe. " Also according to https://www.cplusplus.com/reference/deque/deque/pop_front/, " The container is modified. The first element is modified. Concurrently accessing or modifying other elements is safe (although see iterator validity above). " In RocksDB, we never pop the last element of `DBImpl::alive_log_files_`. We have been exploiting this fact and the above two properties when ensuring correctness when `DBImpl::alive_log_files_` may be accessed concurrently. Specifically, it can be accessed in the write path when db mutex is released. Sometimes, the log_mute_ is held. It can also be accessed in `FindObsoleteFiles()` when db mutex is always held. It can also be accessed during recovery when db mutex is also held. Given the fact that we never pop the last element of alive_log_files_, we currently do not acquire additional locks when accessing it in `WriteToWAL()` as follows ``` alive_log_files_.back().AddSize(log_entry.size()); ``` This is problematic. Check source code of deque.h ``` back() _GLIBCXX_NOEXCEPT { __glibcxx_requires_nonempty(); ... } pop_front() _GLIBCXX_NOEXCEPT { ... if (this->_M_impl._M_start._M_cur != this->_M_impl._M_start._M_last - 1) { ... ++this->_M_impl._M_start._M_cur; } ... } ``` `back()` will actually call `__glibcxx_requires_nonempty()` first. If `__glibcxx_requires_nonempty()` is enabled and not an empty macro, it will call `empty()` ``` bool empty() { return this->_M_impl._M_finish == this->_M_impl._M_start; } ``` You can see that it will access `this->_M_impl._M_start`, racing with `pop_front()`. Therefore, TSAN will actually catch the bug in this case. To be able to use TSAN on our library and unit tests, we should always coordinate concurrent accesses to STL containers properly. We need to pass information about db mutex and log mutex into `WriteToWAL()`, otherwise it's impossible to know which mutex to acquire inside the function. To fix this, we can catch the tail of `alive_log_files_` by reference, so that we do not have to call `back()` in `WriteToWAL()`. Reviewed By: pdillinger Differential Revision: D34780309 fbshipit-source-id: 1def9821f0c437f2736c6a26445d75890377889b
2022-03-15 02:49:55 +01:00
Env::IO_TOTAL, /*with_db_mutex==*/true);
if (s.ok()) {
// Need to fsync, otherwise it might get lost after a power reset.
s = impl->FlushWAL(false);
if (s.ok()) {
s = log_writer->file()->Sync(impl->immutable_db_options_.use_fsync);
}
}
}
}
}
if (s.ok() && impl->immutable_db_options_.persist_stats_to_disk) {
// try to read format version
s = impl->PersistentStatsProcessFormatVersion();
}
if (s.ok()) {
for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
auto* vstorage = cfd->current()->storage_info();
for (int i = 1; i < vstorage->num_levels(); ++i) {
int num_files = vstorage->NumLevelFiles(i);
if (num_files > 0) {
s = Status::InvalidArgument(
"Not all files are at level 0. Cannot "
"open with FIFO compaction style.");
break;
}
}
}
if (!cfd->mem()->IsSnapshotSupported()) {
impl->is_snapshot_supported_ = false;
}
if (cfd->ioptions()->merge_operator != nullptr &&
!cfd->mem()->IsMergeOperatorSupported()) {
s = Status::InvalidArgument(
"The memtable of column family %s does not support merge operator "
"its options.merge_operator is non-null",
cfd->GetName().c_str());
}
if (!s.ok()) {
break;
}
}
}
TEST_SYNC_POINT("DBImpl::Open:Opened");
Status persist_options_status;
if (s.ok()) {
// Persist RocksDB Options before scheduling the compaction.
// The WriteOptionsFile() will release and lock the mutex internally.
persist_options_status = impl->WriteOptionsFile(
false /*need_mutex_lock*/, false /*need_enter_write_thread*/);
*dbptr = impl;
impl->opened_successfully_ = true;
impl->DeleteObsoleteFiles();
TEST_SYNC_POINT("DBImpl::Open:AfterDeleteFiles");
impl->MaybeScheduleFlushOrCompaction();
} else {
persist_options_status.PermitUncheckedError();
}
impl->mutex_.Unlock();
#ifndef ROCKSDB_LITE
auto sfm = static_cast<SstFileManagerImpl*>(
impl->immutable_db_options_.sst_file_manager.get());
if (s.ok() && sfm) {
// Set Statistics ptr for SstFileManager to dump the stats of
// DeleteScheduler.
sfm->SetStatisticsPtr(impl->immutable_db_options_.statistics);
ROCKS_LOG_INFO(impl->immutable_db_options_.info_log,
"SstFileManager instance %p", sfm);
// Notify SstFileManager about all sst files that already exist in
// db_paths[0] and cf_paths[0] when the DB is opened.
// SstFileManagerImpl needs to know sizes of the files. For files whose size
// we already know (sst files that appear in manifest - typically that's the
// vast majority of all files), we'll pass the size to SstFileManager.
// For all other files SstFileManager will query the size from filesystem.
std::vector<LiveFileMetaData> metadata;
// TODO: Once GetLiveFilesMetaData supports blob files, update the logic
// below to get known_file_sizes for blob files.
impl->mutex_.Lock();
impl->versions_->GetLiveFilesMetaData(&metadata);
impl->mutex_.Unlock();
std::unordered_map<std::string, uint64_t> known_file_sizes;
for (const auto& md : metadata) {
std::string name = md.name;
if (!name.empty() && name[0] == '/') {
name = name.substr(1);
}
known_file_sizes[name] = md.size;
}
std::vector<std::string> paths;
paths.emplace_back(impl->immutable_db_options_.db_paths[0].path);
for (auto& cf : column_families) {
if (!cf.options.cf_paths.empty()) {
paths.emplace_back(cf.options.cf_paths[0].path);
}
}
// Remove duplicate paths.
std::sort(paths.begin(), paths.end());
paths.erase(std::unique(paths.begin(), paths.end()), paths.end());
for (auto& path : paths) {
std::vector<std::string> existing_files;
impl->immutable_db_options_.env->GetChildren(path, &existing_files)
.PermitUncheckedError(); //**TODO: What do to on error?
for (auto& file_name : existing_files) {
uint64_t file_number;
FileType file_type;
std::string file_path = path + "/" + file_name;
if (ParseFileName(file_name, &file_number, &file_type) &&
(file_type == kTableFile || file_type == kBlobFile)) {
// TODO: Check for errors from OnAddFile?
if (known_file_sizes.count(file_name)) {
// We're assuming that each sst file name exists in at most one of
// the paths.
sfm->OnAddFile(file_path, known_file_sizes.at(file_name))
.PermitUncheckedError();
} else {
sfm->OnAddFile(file_path).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 22:36:19 +02:00
// Reserve some disk buffer space. This is a heuristic - when we run out
// of disk space, this ensures that there is atleast write_buffer_size
// amount of free space before we resume DB writes. In low disk space
// conditions, we want to avoid a lot of small L0 files due to frequent
// WAL write failures and resultant forced flushes
sfm->ReserveDiskBuffer(max_write_buffer_size,
impl->immutable_db_options_.db_paths[0].path);
}
#endif // !ROCKSDB_LITE
if (s.ok()) {
ROCKS_LOG_HEADER(impl->immutable_db_options_.info_log, "DB pointer %p",
impl);
LogFlush(impl->immutable_db_options_.info_log);
assert(impl->TEST_WALBufferIsEmpty());
// If the assert above fails then we need to FlushWAL before returning
// control back to the user.
if (!persist_options_status.ok()) {
s = Status::IOError(
"DB::Open() failed --- Unable to persist Options file",
persist_options_status.ToString());
}
}
if (!s.ok()) {
ROCKS_LOG_WARN(impl->immutable_db_options_.info_log,
"DB::Open() failed: %s", s.ToString().c_str());
}
move dump stats to a separate thread (#4382) Summary: Currently statistics are supposed to be dumped to info log at intervals of `options.stats_dump_period_sec`. However the implementation choice was to bind it with compaction thread, meaning if the database has been serving very light traffic, the stats may not get dumped at all. We decided to separate stats dumping into a new timed thread using `TimerQueue`, which is already used in blob_db. This will allow us schedule new timed tasks with more deterministic behavior. Tested with db_bench using `--stats_dump_period_sec=20` in command line: > LOG:2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS ------- LOG:2018/09/17-14:08:05.643286 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS ------- LOG:2018/09/17-14:08:25.691325 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS ------- LOG:2018/09/17-14:08:45.740989 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS ------- LOG content: > 2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS ------- 2018/09/17-14:07:45.575080 7fe99fbfe700 [WARN] [db/db_impl.cc:606] ** DB Stats ** Uptime(secs): 20.0 total, 20.0 interval Cumulative writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5.57 GB, 285.01 MB/s Cumulative WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 GB, 285.01 MB/s Cumulative stall: 00:00:0.012 H:M:S, 0.1 percent Interval writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5700.71 MB, 285.01 MB/s Interval WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 MB, 285.01 MB/s Interval stall: 00:00:0.012 H:M:S, 0.1 percent ** Compaction Stats [default] ** Level Files Size Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop Pull Request resolved: https://github.com/facebook/rocksdb/pull/4382 Differential Revision: D9933051 Pulled By: miasantreble fbshipit-source-id: 6d12bb1e4977674eea4bf2d2ac6d486b814bb2fa
2018-10-09 07:52:58 +02:00
if (s.ok()) {
s = impl->StartPeriodicWorkScheduler();
}
if (!s.ok()) {
for (auto* h : *handles) {
delete h;
}
handles->clear();
delete impl;
*dbptr = nullptr;
}
return s;
}
} // namespace ROCKSDB_NAMESPACE