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

814 lines
30 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
//
// Repairer does best effort recovery to recover as much data as possible after
// a disaster without compromising consistency. It does not guarantee bringing
// the database to a time consistent state.
//
// Repair process is broken into 4 phases:
// (a) Find files
// (b) Convert logs to tables
// (c) Extract metadata
// (d) Write Descriptor
//
// (a) Find files
//
// The repairer goes through all the files in the directory, and classifies them
// based on their file name. Any file that cannot be identified by name will be
// ignored.
//
// (b) Convert logs to table
//
// Every log file that is active is replayed. All sections of the file where the
// checksum does not match is skipped over. We intentionally give preference to
// data consistency.
//
// (c) Extract metadata
//
// We scan every table to compute
// (1) smallest/largest for the table
// (2) largest sequence number in the table
// (3) oldest blob file referred to by the table (if applicable)
//
// If we are unable to scan the file, then we ignore the table.
//
// (d) Write Descriptor
//
// We generate descriptor contents:
// - log number is set to zero
// - next-file-number is set to 1 + largest file number we found
// - last-sequence-number is set to largest sequence# found across
// all tables (see 2c)
// - compaction pointers are cleared
// - every table file is added at level 0
//
// Possible optimization 1:
// (a) Compute total size and use to pick appropriate max-level M
// (b) Sort tables by largest sequence# in the table
// (c) For each table: if it overlaps earlier table, place in level-0,
// else place in level-M.
// (d) We can provide options for time consistent recovery and unsafe recovery
// (ignore checksum failure when applicable)
// Possible optimization 2:
// Store per-table metadata (smallest, largest, largest-seq#, ...)
// in the table's meta section to speed up ScanTable.
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
#include "db/version_builder.h"
#ifndef ROCKSDB_LITE
#include <cinttypes>
#include "db/builder.h"
#include "db/db_impl/db_impl.h"
#include "db/dbformat.h"
#include "db/log_reader.h"
#include "db/log_writer.h"
#include "db/memtable.h"
#include "db/table_cache.h"
#include "db/version_edit.h"
#include "db/write_batch_internal.h"
#include "file/filename.h"
#include "file/writable_file_writer.h"
#include "logging/logging.h"
#include "options/cf_options.h"
#include "rocksdb/comparator.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/options.h"
#include "rocksdb/write_buffer_manager.h"
#include "table/scoped_arena_iterator.h"
#include "table/unique_id_impl.h"
#include "util/string_util.h"
namespace ROCKSDB_NAMESPACE {
namespace {
class Repairer {
public:
Repairer(const std::string& dbname, const DBOptions& db_options,
const std::vector<ColumnFamilyDescriptor>& column_families,
const ColumnFamilyOptions& default_cf_opts,
const ColumnFamilyOptions& unknown_cf_opts, bool create_unknown_cfs)
: dbname_(dbname),
db_session_id_(DBImpl::GenerateDbSessionId(db_options.env)),
env_(db_options.env),
file_options_(),
db_options_(SanitizeOptions(dbname_, db_options)),
immutable_db_options_(ImmutableDBOptions(db_options_)),
icmp_(default_cf_opts.comparator),
default_cf_opts_(
SanitizeOptions(immutable_db_options_, default_cf_opts)),
default_iopts_(
ImmutableOptions(immutable_db_options_, default_cf_opts_)),
unknown_cf_opts_(
SanitizeOptions(immutable_db_options_, unknown_cf_opts)),
create_unknown_cfs_(create_unknown_cfs),
raw_table_cache_(
// TableCache can be small since we expect each table to be opened
// once.
NewLRUCache(10, db_options_.table_cache_numshardbits)),
table_cache_(new TableCache(default_iopts_, &file_options_,
raw_table_cache_.get(),
/*block_cache_tracer=*/nullptr,
/*io_tracer=*/nullptr, db_session_id_)),
wb_(db_options_.db_write_buffer_size),
wc_(db_options_.delayed_write_rate),
vset_(dbname_, &immutable_db_options_, file_options_,
raw_table_cache_.get(), &wb_, &wc_,
/*block_cache_tracer=*/nullptr, /*io_tracer=*/nullptr,
/*db_id=*/"", db_session_id_),
next_file_number_(1),
db_lock_(nullptr),
closed_(false) {
for (const auto& cfd : column_families) {
cf_name_to_opts_[cfd.name] = cfd.options;
}
}
const ColumnFamilyOptions* GetColumnFamilyOptions(
const std::string& cf_name) {
if (cf_name_to_opts_.find(cf_name) == cf_name_to_opts_.end()) {
if (create_unknown_cfs_) {
return &unknown_cf_opts_;
}
return nullptr;
}
return &cf_name_to_opts_[cf_name];
}
// Adds a column family to the VersionSet with cf_options_ and updates
// manifest.
Status AddColumnFamily(const std::string& cf_name, uint32_t cf_id) {
const auto* cf_opts = GetColumnFamilyOptions(cf_name);
if (cf_opts == nullptr) {
return Status::Corruption("Encountered unknown column family with name=" +
cf_name + ", id=" + std::to_string(cf_id));
}
Options opts(db_options_, *cf_opts);
MutableCFOptions mut_cf_opts(opts);
VersionEdit edit;
edit.SetComparatorName(opts.comparator->Name());
edit.SetLogNumber(0);
edit.SetColumnFamily(cf_id);
ColumnFamilyData* cfd;
cfd = nullptr;
edit.AddColumnFamily(cf_name);
mutex_.Lock();
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
std::unique_ptr<FSDirectory> db_dir;
Status status = env_->GetFileSystem()->NewDirectory(dbname_, IOOptions(),
&db_dir, nullptr);
if (status.ok()) {
status = vset_.LogAndApply(cfd, mut_cf_opts, &edit, &mutex_, db_dir.get(),
false /* new_descriptor_log */, cf_opts);
}
mutex_.Unlock();
return status;
}
Status Close() {
Status s = Status::OK();
if (!closed_) {
if (db_lock_ != nullptr) {
s = env_->UnlockFile(db_lock_);
db_lock_ = nullptr;
}
closed_ = true;
}
return s;
}
~Repairer() { Close().PermitUncheckedError(); }
Status Run() {
Status status = env_->LockFile(LockFileName(dbname_), &db_lock_);
if (!status.ok()) {
return status;
}
status = FindFiles();
DBImpl* db_impl = nullptr;
if (status.ok()) {
// Discard older manifests and start a fresh one
for (size_t i = 0; i < manifests_.size(); i++) {
ArchiveFile(dbname_ + "/" + manifests_[i]);
}
// Just create a DBImpl temporarily so we can reuse NewDB()
db_impl = new DBImpl(db_options_, dbname_);
status = db_impl->NewDB(/*new_filenames=*/nullptr);
}
delete db_impl;
if (status.ok()) {
// Recover using the fresh manifest created by NewDB()
status =
vset_.Recover({{kDefaultColumnFamilyName, default_cf_opts_}}, false);
}
if (status.ok()) {
// Need to scan existing SST files first so the column families are
// created before we process WAL files
ExtractMetaData();
// ExtractMetaData() uses table_fds_ to know which SST files' metadata to
// extract -- we need to clear it here since metadata for existing SST
// files has been extracted already
table_fds_.clear();
ConvertLogFilesToTables();
ExtractMetaData();
status = AddTables();
}
if (status.ok()) {
uint64_t bytes = 0;
for (size_t i = 0; i < tables_.size(); i++) {
bytes += tables_[i].meta.fd.GetFileSize();
}
ROCKS_LOG_WARN(db_options_.info_log,
"**** Repaired rocksdb %s; "
"recovered %" ROCKSDB_PRIszt " files; %" PRIu64
" bytes. "
"Some data may have been lost. "
"****",
dbname_.c_str(), tables_.size(), bytes);
}
return status;
}
private:
struct TableInfo {
FileMetaData meta;
uint32_t column_family_id;
std::string column_family_name;
};
std::string const dbname_;
std::string db_session_id_;
Env* const env_;
const FileOptions file_options_;
const DBOptions db_options_;
const ImmutableDBOptions immutable_db_options_;
const InternalKeyComparator icmp_;
const ColumnFamilyOptions default_cf_opts_;
const ImmutableOptions default_iopts_; // table_cache_ holds reference
const ColumnFamilyOptions unknown_cf_opts_;
const bool create_unknown_cfs_;
std::shared_ptr<Cache> raw_table_cache_;
std::unique_ptr<TableCache> table_cache_;
WriteBufferManager wb_;
WriteController wc_;
VersionSet vset_;
std::unordered_map<std::string, ColumnFamilyOptions> cf_name_to_opts_;
InstrumentedMutex mutex_;
std::vector<std::string> manifests_;
std::vector<FileDescriptor> table_fds_;
std::vector<uint64_t> logs_;
std::vector<TableInfo> tables_;
uint64_t next_file_number_;
// Lock over the persistent DB state. Non-nullptr iff successfully
// acquired.
FileLock* db_lock_;
bool closed_;
Status FindFiles() {
std::vector<std::string> filenames;
bool found_file = false;
std::vector<std::string> to_search_paths;
for (size_t path_id = 0; path_id < db_options_.db_paths.size(); path_id++) {
to_search_paths.push_back(db_options_.db_paths[path_id].path);
}
// search wal_dir if user uses a customize wal_dir
bool same = immutable_db_options_.IsWalDirSameAsDBPath(dbname_);
if (!same) {
to_search_paths.push_back(immutable_db_options_.wal_dir);
}
for (size_t path_id = 0; path_id < to_search_paths.size(); path_id++) {
ROCKS_LOG_INFO(db_options_.info_log, "Searching path %s\n",
to_search_paths[path_id].c_str());
Status status = env_->GetChildren(to_search_paths[path_id], &filenames);
if (!status.ok()) {
return status;
}
if (!filenames.empty()) {
found_file = true;
}
uint64_t number;
FileType type;
for (size_t i = 0; i < filenames.size(); i++) {
if (ParseFileName(filenames[i], &number, &type)) {
if (type == kDescriptorFile) {
manifests_.push_back(filenames[i]);
} else {
if (number + 1 > next_file_number_) {
next_file_number_ = number + 1;
}
if (type == kWalFile) {
logs_.push_back(number);
} else if (type == kTableFile) {
table_fds_.emplace_back(number, static_cast<uint32_t>(path_id),
0);
} else {
// Ignore other files
}
}
}
}
}
if (!found_file) {
return Status::Corruption(dbname_, "repair found no files");
}
return Status::OK();
}
void ConvertLogFilesToTables() {
const auto& wal_dir = immutable_db_options_.GetWalDir();
for (size_t i = 0; i < logs_.size(); i++) {
// we should use LogFileName(wal_dir, logs_[i]) here. user might uses
// wal_dir option.
std::string logname = LogFileName(wal_dir, logs_[i]);
Status status = ConvertLogToTable(wal_dir, logs_[i]);
if (!status.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Log #%" PRIu64 ": ignoring conversion error: %s",
logs_[i], status.ToString().c_str());
}
ArchiveFile(logname);
}
}
Status ConvertLogToTable(const std::string& wal_dir, uint64_t log) {
struct LogReporter : public log::Reader::Reporter {
Env* env;
std::shared_ptr<Logger> info_log;
uint64_t lognum;
void Corruption(size_t bytes, const Status& s) override {
// We print error messages for corruption, but continue repairing.
ROCKS_LOG_ERROR(info_log, "Log #%" PRIu64 ": dropping %d bytes; %s",
lognum, static_cast<int>(bytes), s.ToString().c_str());
}
};
// Open the log file
std::string logname = LogFileName(wal_dir, log);
const auto& fs = env_->GetFileSystem();
std::unique_ptr<SequentialFileReader> lfile_reader;
Status status = SequentialFileReader::Create(
fs, logname, fs->OptimizeForLogRead(file_options_), &lfile_reader,
Support read rate-limiting in SequentialFileReader (#9973) Summary: Added rate limiter and read rate-limiting support to SequentialFileReader. I've updated call sites to SequentialFileReader::Read with appropriate IO priority (or left a TODO and specified IO_TOTAL for now). The PR is separated into four commits: the first one added the rate-limiting support, but with some fixes in the unit test since the number of request bytes from rate limiter in SequentialFileReader are not accurate (there is overcharge at EOF). The second commit fixed this by allowing SequentialFileReader to check file size and determine how many bytes are left in the file to read. The third commit added benchmark related code. The fourth commit moved the logic of using file size to avoid overcharging the rate limiter into backup engine (the main user of SequentialFileReader). Pull Request resolved: https://github.com/facebook/rocksdb/pull/9973 Test Plan: - `make check`, backup_engine_test covers usage of SequentialFileReader with rate limiter. - Run db_bench to check if rate limiting is throttling as expected: Verified that reads and writes are together throttled at 2MB/s, and at 0.2MB chunks that are 100ms apart. - Set up: `./db_bench --benchmarks=fillrandom -db=/dev/shm/test_rocksdb` - Benchmark: ``` strace -ttfe read,write ./db_bench --benchmarks=backup -db=/dev/shm/test_rocksdb --backup_rate_limit=2097152 --use_existing_db strace -ttfe read,write ./db_bench --benchmarks=restore -db=/dev/shm/test_rocksdb --restore_rate_limit=2097152 --use_existing_db ``` - db bench on backup and restore to ensure no performance regression. - backup (avg over 50 runs): pre-change: 1.90443e+06 micros/op; post-change: 1.8993e+06 micros/op (improve by 0.2%) - restore (avg over 50 runs): pre-change: 1.79105e+06 micros/op; post-change: 1.78192e+06 micros/op (improve by 0.5%) ``` # Set up ./db_bench --benchmarks=fillrandom -db=/tmp/test_rocksdb -num=10000000 # benchmark TEST_TMPDIR=/tmp/test_rocksdb NUM_RUN=50 for ((j=0;j<$NUM_RUN;j++)) do ./db_bench -db=$TEST_TMPDIR -num=10000000 -benchmarks=backup -use_existing_db | egrep 'backup' # Restore #./db_bench -db=$TEST_TMPDIR -num=10000000 -benchmarks=restore -use_existing_db done > rate_limit.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)}' rate_limit.txt >> rate_limit_2.txt ``` Reviewed By: hx235 Differential Revision: D36327418 Pulled By: cbi42 fbshipit-source-id: e75d4307cff815945482df5ba630c1e88d064691
2 years ago
nullptr /* dbg */, nullptr /* rate limiter */);
if (!status.ok()) {
return status;
}
// Create the log reader.
LogReporter reporter;
reporter.env = env_;
reporter.info_log = db_options_.info_log;
reporter.lognum = log;
// We intentionally make log::Reader do checksumming so that
// corruptions cause entire commits to be skipped instead of
// propagating bad information (like overly large sequence
// numbers).
log::Reader reader(db_options_.info_log, std::move(lfile_reader), &reporter,
Support for single-primary, multi-secondary instances (#4899) Summary: This PR allows RocksDB to run in single-primary, multi-secondary process mode. The writer is a regular RocksDB (e.g. an `DBImpl`) instance playing the role of a primary. Multiple `DBImplSecondary` processes (secondaries) share the same set of SST files, MANIFEST, WAL files with the primary. Secondaries tail the MANIFEST of the primary and apply updates to their own in-memory state of the file system, e.g. `VersionStorageInfo`. This PR has several components: 1. (Originally in #4745). Add a `PathNotFound` subcode to `IOError` to denote the failure when a secondary tries to open a file which has been deleted by the primary. 2. (Similar to #4602). Add `FragmentBufferedReader` to handle partially-read, trailing record at the end of a log from where future read can continue. 3. (Originally in #4710 and #4820). Add implementation of the secondary, i.e. `DBImplSecondary`. 3.1 Tail the primary's MANIFEST during recovery. 3.2 Tail the primary's MANIFEST during normal processing by calling `ReadAndApply`. 3.3 Tailing WAL will be in a future PR. 4. Add an example in 'examples/multi_processes_example.cc' to demonstrate the usage of secondary RocksDB instance in a multi-process setting. Instructions to run the example can be found at the beginning of the source code. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4899 Differential Revision: D14510945 Pulled By: riversand963 fbshipit-source-id: 4ac1c5693e6012ad23f7b4b42d3c374fecbe8886
5 years ago
true /*enable checksum*/, log);
// Initialize per-column family memtables
for (auto* cfd : *vset_.GetColumnFamilySet()) {
cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
kMaxSequenceNumber);
}
auto cf_mems = new ColumnFamilyMemTablesImpl(vset_.GetColumnFamilySet());
// Read all the records and add to a memtable
std::string scratch;
Slice record;
WriteBatch batch;
int counter = 0;
while (reader.ReadRecord(&record, &scratch)) {
if (record.size() < WriteBatchInternal::kHeader) {
reporter.Corruption(record.size(),
Status::Corruption("log record too small"));
continue;
}
Status record_status = WriteBatchInternal::SetContents(&batch, record);
if (record_status.ok()) {
record_status =
WriteBatchInternal::InsertInto(&batch, cf_mems, nullptr, nullptr);
}
if (record_status.ok()) {
counter += WriteBatchInternal::Count(&batch);
} else {
ROCKS_LOG_WARN(db_options_.info_log, "Log #%" PRIu64 ": ignoring %s",
log, record_status.ToString().c_str());
}
}
// Dump a table for each column family with entries in this log file.
for (auto* cfd : *vset_.GetColumnFamilySet()) {
// Do not record a version edit for this conversion to a Table
// since ExtractMetaData() will also generate edits.
MemTable* mem = cfd->mem();
if (mem->IsEmpty()) {
continue;
}
FileMetaData meta;
meta.fd = FileDescriptor(next_file_number_++, 0, 0);
ReadOptions ro;
ro.total_order_seek = true;
Arena arena;
ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
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
7 years ago
const uint64_t current_time = static_cast<uint64_t>(_current_time);
meta.file_creation_time = current_time;
SnapshotChecker* snapshot_checker = DisableGCSnapshotChecker::Instance();
auto write_hint = cfd->CalculateSSTWriteHint(0);
std::vector<std::unique_ptr<FragmentedRangeTombstoneIterator>>
range_del_iters;
Fragment memtable range tombstone in the write path (#10380) Summary: - Right now each read fragments the memtable range tombstones https://github.com/facebook/rocksdb/issues/4808. This PR explores the idea of fragmenting memtable range tombstones in the write path and reads can just read this cached fragmented tombstone without any fragmenting cost. This PR only does the caching for immutable memtable, and does so right before a memtable is added to an immutable memtable list. The fragmentation is done without holding mutex to minimize its performance impact. - db_bench is updated to print out the number of range deletions executed if there is any. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10380 Test Plan: - CI, added asserts in various places to check whether a fragmented range tombstone list should have been constructed. - Benchmark: as this PR only optimizes immutable memtable path, the number of writes in the benchmark is chosen such an immutable memtable is created and range tombstones are in that memtable. ``` single thread: ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=100000 --max_num_range_tombstones=100 multi_thread ./db_bench --benchmarks=fillrandom,readrandom --writes_per_range_tombstone=1 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=15000 --reads=20000 --threads=32 --max_num_range_tombstones=100 ``` Commit 99cdf16464a057ca44de2f747541dedf651bae9e is included in benchmark result. It was an earlier attempt where tombstones are fragmented for each write operation. Reader threads share it using a shared_ptr which would slow down multi-thread read performance as seen in benchmark results. Results are averaged over 5 runs. Single thread result: | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |6.68 |6.57 |6.72 |4.72 |4.79 |4.54 | | 1 |6.67 |6.58 |6.62 |5.41 |4.74 |4.72 | | 10 |6.59 |6.5 |6.56 |7.83 |4.69 |4.59 | | 100 |6.62 |6.75 |6.58 |29.57 |5.04 |5.09 | | 1000 |6.54 |6.82 |6.61 |320.33 |5.22 |5.21 | 32-thread result: note that "Max # tombstones" is per thread. | Max # tombstones | main fillrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | main readrandom micros/op | 99cdf16464a057ca44de2f747541dedf651bae9e | Post PR | | ------------- | ------------- |------------- |------------- |------------- |------------- |------------- | | 0 |234.52 |260.25 |239.42 |5.06 |5.38 |5.09 | | 1 |236.46 |262.0 |231.1 |19.57 |22.14 |5.45 | | 10 |236.95 |263.84 |251.49 |151.73 |21.61 |5.73 | | 100 |268.16 |296.8 |280.13 |2308.52 |22.27 |6.57 | Reviewed By: ajkr Differential Revision: D37916564 Pulled By: cbi42 fbshipit-source-id: 05d6d2e16df26c374c57ddcca13a5bfe9d5b731e
2 years ago
auto range_del_iter = mem->NewRangeTombstoneIterator(
ro, kMaxSequenceNumber, false /* immutable_memtable */);
if (range_del_iter != nullptr) {
range_del_iters.emplace_back(range_del_iter);
}
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
5 years ago
IOStatus io_s;
CompressionOptions default_compression;
TableBuilderOptions tboptions(
*cfd->ioptions(), *cfd->GetLatestMutableCFOptions(),
cfd->internal_comparator(), cfd->int_tbl_prop_collector_factories(),
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
3 years ago
kNoCompression, default_compression, cfd->GetID(), cfd->GetName(),
-1 /* level */, false /* is_bottommost */,
TableFileCreationReason::kRecovery, 0 /* oldest_key_time */,
0 /* file_creation_time */, "DB Repairer" /* db_id */, db_session_id_,
0 /*target_file_size*/, meta.fd.GetNumber());
SeqnoToTimeMapping empty_seqno_time_mapping;
status = BuildTable(
dbname_, /* versions */ nullptr, immutable_db_options_, tboptions,
file_options_, table_cache_.get(), iter.get(),
std::move(range_del_iters), &meta, nullptr /* 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
2 years ago
{}, kMaxSequenceNumber, kMaxSequenceNumber, snapshot_checker,
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
3 years ago
false /* paranoid_file_checks*/, nullptr /* internal_stats */, &io_s,
nullptr /*IOTracer*/, BlobFileCreationReason::kRecovery,
empty_seqno_time_mapping, nullptr /* event_logger */, 0 /* job_id */,
Env::IO_HIGH, nullptr /* table_properties */, write_hint);
ROCKS_LOG_INFO(db_options_.info_log,
"Log #%" PRIu64 ": %d ops saved to Table #%" PRIu64 " %s",
log, counter, meta.fd.GetNumber(),
status.ToString().c_str());
if (status.ok()) {
if (meta.fd.GetFileSize() > 0) {
table_fds_.push_back(meta.fd);
}
} else {
break;
}
}
delete cf_mems;
return status;
}
void ExtractMetaData() {
for (size_t i = 0; i < table_fds_.size(); i++) {
TableInfo t;
t.meta.fd = table_fds_[i];
Status status = ScanTable(&t);
if (!status.ok()) {
std::string fname = TableFileName(
db_options_.db_paths, t.meta.fd.GetNumber(), t.meta.fd.GetPathId());
char file_num_buf[kFormatFileNumberBufSize];
FormatFileNumber(t.meta.fd.GetNumber(), t.meta.fd.GetPathId(),
file_num_buf, sizeof(file_num_buf));
ROCKS_LOG_WARN(db_options_.info_log, "Table #%s: ignoring %s",
file_num_buf, status.ToString().c_str());
ArchiveFile(fname);
} else {
tables_.push_back(t);
}
}
}
Status ScanTable(TableInfo* t) {
std::string fname = TableFileName(
db_options_.db_paths, t->meta.fd.GetNumber(), t->meta.fd.GetPathId());
int counter = 0;
uint64_t file_size;
Status status = env_->GetFileSize(fname, &file_size);
t->meta.fd = FileDescriptor(t->meta.fd.GetNumber(), t->meta.fd.GetPathId(),
file_size);
std::shared_ptr<const TableProperties> props;
if (status.ok()) {
Always verify SST unique IDs on SST file open (#10532) Summary: Although we've been tracking SST unique IDs in the DB manifest unconditionally, checking has been opt-in and with an extra pass at DB::Open time. This changes the behavior of `verify_sst_unique_id_in_manifest` to check unique ID against manifest every time an SST file is opened through table cache (normal DB operations), replacing the explicit pass over files at DB::Open time. This change also enables the option by default and removes the "EXPERIMENTAL" designation. One possible criticism is that the option no longer ensures the integrity of a DB at Open time. This is far from an all-or-nothing issue. Verifying the IDs of all SST files hardly ensures all the data in the DB is readable. (VerifyChecksum is supposed to do that.) Also, with max_open_files=-1 (default, extremely common), all SST files are opened at DB::Open time anyway. Implementation details: * `VerifySstUniqueIdInManifest()` functions are the extra/explicit pass that is now removed. * Unit tests that manipulate/corrupt table properties have to opt out of this check, because that corrupts the "actual" unique id. (And even for testing we don't currently have a mechanism to set "no unique id" in the in-memory file metadata for new files.) * A lot of other unit test churn relates to (a) default checking on, and (b) checking on SST open even without DB::Open (e.g. on flush) * Use `FileMetaData` for more `TableCache` operations (in place of `FileDescriptor`) so that we have access to the unique_id whenever we might need to open an SST file. **There is the possibility of performance impact because we can no longer use the more localized `fd` part of an `FdWithKeyRange` but instead follow the `file_metadata` pointer. However, this change (possible regression) is only done for `GetMemoryUsageByTableReaders`.** * Removed a completely unnecessary constructor overload of `TableReaderOptions` Possible follow-up: * Verification only happens when opening through table cache. Are there more places where this should happen? * Improve error message when there is a file size mismatch vs. manifest (FIXME added in the appropriate place). * I'm not sure there's a justification for `FileDescriptor` to be distinct from `FileMetaData`. * I'm skeptical that `FdWithKeyRange` really still makes sense for optimizing some data locality by duplicating some data in memory, but I could be wrong. * An unnecessary overload of NewTableReader was recently added, in the public API nonetheless (though unusable there). It should be cleaned up to put most things under `TableReaderOptions`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10532 Test Plan: updated unit tests Performance test showing no significant difference (just noise I think): `./db_bench -benchmarks=readwhilewriting[-X10] -num=3000000 -disable_wal=1 -bloom_bits=8 -write_buffer_size=1000000 -target_file_size_base=1000000` Before: readwhilewriting [AVG 10 runs] : 68702 (± 6932) ops/sec After: readwhilewriting [AVG 10 runs] : 68239 (± 7198) ops/sec Reviewed By: jay-zhuang Differential Revision: D38765551 Pulled By: pdillinger fbshipit-source-id: a827a708155f12344ab2a5c16e7701c7636da4c2
2 years ago
status = table_cache_->GetTableProperties(file_options_, icmp_, t->meta,
&props);
}
if (status.ok()) {
auto s =
GetSstInternalUniqueId(props->db_id, props->db_session_id,
props->orig_file_number, &t->meta.unique_id);
if (!s.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Table #%" PRIu64
": unable to get unique id, default to Unknown.",
t->meta.fd.GetNumber());
}
t->column_family_id = static_cast<uint32_t>(props->column_family_id);
if (t->column_family_id ==
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily) {
ROCKS_LOG_WARN(
db_options_.info_log,
"Table #%" PRIu64
": column family unknown (probably due to legacy format); "
"adding to default column family id 0.",
t->meta.fd.GetNumber());
t->column_family_id = 0;
}
if (vset_.GetColumnFamilySet()->GetColumnFamily(t->column_family_id) ==
nullptr) {
status =
AddColumnFamily(props->column_family_name, t->column_family_id);
}
t->meta.oldest_ancester_time = props->creation_time;
}
ColumnFamilyData* cfd = nullptr;
if (status.ok()) {
cfd = vset_.GetColumnFamilySet()->GetColumnFamily(t->column_family_id);
if (cfd->GetName() != props->column_family_name) {
ROCKS_LOG_ERROR(
db_options_.info_log,
"Table #%" PRIu64
": inconsistent column family name '%s'; expected '%s' for column "
"family id %" PRIu32 ".",
t->meta.fd.GetNumber(), props->column_family_name.c_str(),
cfd->GetName().c_str(), t->column_family_id);
status = Status::Corruption(dbname_, "inconsistent column family name");
}
}
if (status.ok()) {
ReadOptions ropts;
ropts.total_order_seek = true;
InternalIterator* iter = table_cache_->NewIterator(
ropts, file_options_, cfd->internal_comparator(), t->meta,
nullptr /* range_del_agg */,
Fast path for detecting unchanged prefix_extractor (#9407) Summary: Fixes a major performance regression in 6.26, where extra CPU is spent in SliceTransform::AsString when reads involve a prefix_extractor (Get, MultiGet, Seek). Common case performance is now better than 6.25. This change creates a "fast path" for verifying that the current prefix extractor is unchanged and compatible with what was used to generate a table file. This fast path detects the common case by pointer comparison on the current prefix_extractor and a "known good" prefix extractor (if applicable) that is saved at the time the table reader is opened. The "known good" prefix extractor is saved as another shared_ptr copy (in an existing field, however) to ensure the pointer is not recycled. When the prefix_extractor has changed to a different instance but same compatible configuration (rare, odd), performance is still a regression compared to 6.25, but this is likely acceptable because of the oddity of such a case. The performance of incompatible prefix_extractor is essentially unchanged. Also fixed a minor case (ForwardIterator) where a prefix_extractor could be used via a raw pointer after being freed as a shared_ptr, if replaced via SetOptions. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9407 Test Plan: ## Performance Populate DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Running head-to-head comparisons simultaneously with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=seekrandom -num=10000000 -duration=20 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12` Below each is compared by ops/sec vs. baseline which is version 6.25 (multiple baseline runs because of variable machine load) v6.26: 4833 vs. 6698 (<- major regression!) v6.27: 4737 vs. 6397 (still) New: 6704 vs. 6461 (better than baseline in common case) Disabled fastpath: 4843 vs. 6389 (e.g. if prefix extractor instance changes but is still compatible) Changed prefix size (no usable filter) in new: 787 vs. 5927 Changed prefix size (no usable filter) in new & baseline: 773 vs. 784 Reviewed By: mrambacher Differential Revision: D33677812 Pulled By: pdillinger fbshipit-source-id: 571d9711c461fb97f957378a061b7e7dbc4d6a76
2 years ago
cfd->GetLatestMutableCFOptions()->prefix_extractor,
/*table_reader_ptr=*/nullptr, /*file_read_hist=*/nullptr,
TableReaderCaller::kRepair, /*arena=*/nullptr, /*skip_filters=*/false,
/*level=*/-1, /*max_file_size_for_l0_meta_pin=*/0,
/*smallest_compaction_key=*/nullptr,
Properly report IO errors when IndexType::kBinarySearchWithFirstKey is used (#6621) Summary: Context: Index type `kBinarySearchWithFirstKey` added the ability for sst file iterator to sometimes report a key from index without reading the corresponding data block. This is useful when sst blocks are cut at some meaningful boundaries (e.g. one block per key prefix), and many seeks land between blocks (e.g. for each prefix, the ranges of keys in different sst files are nearly disjoint, so a typical seek needs to read a data block from only one file even if all files have the prefix). But this added a new error condition, which rocksdb code was really not equipped to deal with: `InternalIterator::value()` may fail with an IO error or Status::Incomplete, but it's just a method returning a Slice, with no way to report error instead. Before this PR, this type of error wasn't handled at all (an empty slice was returned), and kBinarySearchWithFirstKey implementation was considered a prototype. Now that we (LogDevice) have experimented with kBinarySearchWithFirstKey for a while and confirmed that it's really useful, this PR is adding the missing error handling. It's a pretty inconvenient situation implementation-wise. The error needs to be reported from InternalIterator when trying to access value. But there are ~700 call sites of `InternalIterator::value()`, most of which either can't hit the error condition (because the iterator is reading from memtable or from index or something) or wouldn't benefit from the deferred loading of the value (e.g. compaction iterator that reads all values anyway). Adding error handling to all these call sites would needlessly bloat the code. So instead I made the deferred value loading optional: only the call sites that may use deferred loading have to call the new method `PrepareValue()` before calling `value()`. The feature is enabled with a new bool argument `allow_unprepared_value` to a bunch of methods that create iterators (it wouldn't make sense to put it in ReadOptions because it's completely internal to iterators, with virtually no user-visible effect). Lmk if you have better ideas. Note that the deferred value loading only happens for *internal* iterators. The user-visible iterator (DBIter) always prepares the value before returning from Seek/Next/etc. We could go further and add an API to defer that value loading too, but that's most likely not useful for LogDevice, so it doesn't seem worth the complexity for now. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6621 Test Plan: make -j5 check . Will also deploy to some logdevice test clusters and look at stats. Reviewed By: siying Differential Revision: D20786930 Pulled By: al13n321 fbshipit-source-id: 6da77d918bad3780522e918f17f4d5513d3e99ee
4 years ago
/*largest_compaction_key=*/nullptr,
/*allow_unprepared_value=*/false);
ParsedInternalKey parsed;
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
Slice key = iter->key();
Status pik_status =
ParseInternalKey(key, &parsed, db_options_.allow_data_in_errors);
if (!pik_status.ok()) {
ROCKS_LOG_ERROR(db_options_.info_log,
"Table #%" PRIu64 ": unparsable key - %s",
t->meta.fd.GetNumber(), pik_status.getState());
continue;
}
counter++;
status = t->meta.UpdateBoundaries(key, iter->value(), parsed.sequence,
parsed.type);
if (!status.ok()) {
break;
}
}
if (status.ok() && !iter->status().ok()) {
status = iter->status();
}
delete iter;
ROCKS_LOG_INFO(db_options_.info_log, "Table #%" PRIu64 ": %d entries %s",
t->meta.fd.GetNumber(), counter,
status.ToString().c_str());
}
if (status.ok()) {
// XXX/FIXME: This is just basic, naive handling of range tombstones,
// like call to UpdateBoundariesForRange in builder.cc where we assume
// an SST file is a full sorted run. This probably needs the extra logic
// from compaction_job.cc around call to UpdateBoundariesForRange (to
// handle range tombstones extendingg beyond range of other entries).
ReadOptions ropts;
std::unique_ptr<FragmentedRangeTombstoneIterator> r_iter;
status = table_cache_->GetRangeTombstoneIterator(
ropts, cfd->internal_comparator(), t->meta, &r_iter);
if (r_iter) {
r_iter->SeekToFirst();
while (r_iter->Valid()) {
auto tombstone = r_iter->Tombstone();
auto kv = tombstone.Serialize();
t->meta.UpdateBoundariesForRange(
kv.first, tombstone.SerializeEndKey(), tombstone.seq_,
cfd->internal_comparator());
r_iter->Next();
}
}
}
return status;
}
Status AddTables() {
std::unordered_map<uint32_t, std::vector<const TableInfo*>> cf_id_to_tables;
SequenceNumber max_sequence = 0;
for (size_t i = 0; i < tables_.size(); i++) {
cf_id_to_tables[tables_[i].column_family_id].push_back(&tables_[i]);
if (max_sequence < tables_[i].meta.fd.largest_seqno) {
max_sequence = tables_[i].meta.fd.largest_seqno;
}
}
vset_.SetLastAllocatedSequence(max_sequence);
vset_.SetLastPublishedSequence(max_sequence);
vset_.SetLastSequence(max_sequence);
for (const auto& cf_id_and_tables : cf_id_to_tables) {
auto* cfd =
vset_.GetColumnFamilySet()->GetColumnFamily(cf_id_and_tables.first);
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
// Recover files' epoch number using dummy VersionStorageInfo
VersionBuilder dummy_version_builder(
cfd->current()->version_set()->file_options(), cfd->ioptions(),
cfd->table_cache(), cfd->current()->storage_info(),
cfd->current()->version_set(),
cfd->GetFileMetadataCacheReservationManager());
VersionStorageInfo dummy_vstorage(
&cfd->internal_comparator(), cfd->user_comparator(),
cfd->NumberLevels(), cfd->ioptions()->compaction_style,
nullptr /* src_vstorage */, cfd->ioptions()->force_consistency_checks,
EpochNumberRequirement::kMightMissing);
Status s;
VersionEdit dummy_edit;
for (const auto* table : cf_id_and_tables.second) {
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
// TODO(opt): separate out into multiple levels
dummy_edit.AddFile(
0, table->meta.fd.GetNumber(), table->meta.fd.GetPathId(),
table->meta.fd.GetFileSize(), table->meta.smallest,
table->meta.largest, table->meta.fd.smallest_seqno,
table->meta.fd.largest_seqno, table->meta.marked_for_compaction,
table->meta.temperature, table->meta.oldest_blob_file_number,
table->meta.oldest_ancester_time, table->meta.file_creation_time,
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
table->meta.epoch_number, table->meta.file_checksum,
table->meta.file_checksum_func_name, table->meta.unique_id);
}
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
s = dummy_version_builder.Apply(&dummy_edit);
if (s.ok()) {
s = dummy_version_builder.SaveTo(&dummy_vstorage);
Sync dir containing CURRENT after RenameFile on CURRENT as much as possible (#10573) Summary: **Context:** Below crash test revealed a bug that directory containing CURRENT file (short for `dir_contains_current_file` below) was not always get synced after a new CURRENT is created and being called with `RenameFile` as part of the creation. This bug exposes a risk that such un-synced directory containing the updated CURRENT can’t survive a host crash (e.g, power loss) hence get corrupted. This then will be followed by a recovery from a corrupted CURRENT that we don't want. The root-cause is that a nullptr `FSDirectory* dir_contains_current_file` sometimes gets passed-down to `SetCurrentFile()` hence in those case `dir_contains_current_file->FSDirectory::FsyncWithDirOptions()` will be skipped (which otherwise will internally call`Env/FS::SyncDic()` ) ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_data_in_errors=True --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=134.8015470676662 --bottommost_compression_type=disable --cache_size=8388608 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=2 --compaction_ttl=100 --compression_max_dict_buffer_bytes=511 --compression_max_dict_bytes=16384 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=65536 --continuous_verification_interval=0 --data_block_index_type=0 --db=$db --db_write_buffer_size=1048576 --delpercent=5 --delrangepercent=0 --destroy_db_initially=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --expected_values_dir=$exp --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=4 --ingest_external_file_one_in=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=True --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=10000 --max_key_len=3 --max_manifest_file_size=16384 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --mmap_read=1 --nooverwritepercent=1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=1 --partition_pinning=2 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --progress_reports=0 --read_fault_one_in=1000 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --ribbon_starting_level=999 --secondary_cache_fault_one_in=32 --secondary_cache_uri=compressed_secondary_cache://capacity=8388608 --set_options_one_in=10000 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync_fault_injection=1 --target_file_size_base=2097 --target_file_size_multiplier=2 --test_batches_snapshots=1 --top_level_index_pinning=1 --use_full_merge_v1=1 --use_merge=1 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=524288 --write_buffer_size=4194 --writepercent=35 ``` ``` stderr: WARNING: prefix_size is non-zero but memtablerep != prefix_hash db_stress: utilities/fault_injection_fs.cc:748: virtual rocksdb::IOStatus rocksdb::FaultInjectionTestFS::RenameFile(const std::string &, const std::string &, const rocksdb::IOOptions &, rocksdb::IODebugContext *): Assertion `tlist.find(tdn.second) == tlist.end()' failed.` ``` **Summary:** The PR ensured the non-test path pass down a non-null dir containing CURRENT (which is by current RocksDB assumption just db_dir) by doing the following: - Renamed `directory_to_fsync` as `dir_contains_current_file` in `SetCurrentFile()` to tighten the association between this directory and CURRENT file - Changed `SetCurrentFile()` API to require `dir_contains_current_file` being passed-in, instead of making it by default nullptr. - Because `SetCurrentFile()`'s `dir_contains_current_file` is passed down from `VersionSet::LogAndApply()` then `VersionSet::ProcessManifestWrites()` (i.e, think about this as a chain of 3 functions related to MANIFEST update), these 2 functions also got refactored to require `dir_contains_current_file` - Updated the non-test-path callers of these 3 functions to obtain and pass in non-nullptr `dir_contains_current_file`, which by current assumption of RocksDB, is the `FSDirectory* db_dir`. - `db_impl` path will obtain `DBImpl::directories_.getDbDir()` while others with no access to such `directories_` are obtained on the fly by creating such object `FileSystem::NewDirectory(..)` and manage it by unique pointers to ensure short life time. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10573 Test Plan: - `make check` - Passed the repro db_stress command - For future improvement, since we currently don't assert dir containing CURRENT to be non-nullptr due to https://github.com/facebook/rocksdb/pull/10573#pullrequestreview-1087698899, there is still chances that future developers mistakenly pass down nullptr dir containing CURRENT thus resulting skipped sync dir and cause the bug again. Therefore a smarter test (e.g, such as quoted from ajkr "(make) unsynced data loss to be dropping files corresponding to unsynced directory entries") is still needed. Reviewed By: ajkr Differential Revision: D39005886 Pulled By: hx235 fbshipit-source-id: 336fb9090d0cfa6ca3dd580db86268007dde7f5a
2 years ago
}
Sort L0 files by newly introduced epoch_num (#10922) Summary: **Context:** Sorting L0 files by `largest_seqno` has at least two inconvenience: - File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap. - For example, consider the following sequence of events ("key@n" indicates key at seqno "n") - insert k1@1 to memtable m1 - ingest file s1 with k2@2, ingest file s2 with k3@3 - insert k4@4 to m1 - compact files s1, s2 and result in new file s3 of seqno range [2, 3] - flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1 - However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption. - Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption - For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example) - an existing SST s1 contains only k1@1 - insert k1@2 to memtable m1 - ingest file s2 with k3@3, ingest file s3 with k4@4 - insert single delete k5@5 in m1 - flush m1 and result in new file s4 of seqno range [2, 5] - compact s1, s2, s3 and result in new file s5 of seqno range [1, 4] - compact s4 and result in new file s6 of seqno range [2] due to single delete - By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno` Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways: - In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more. - In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption. **Summary:** - Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`. - `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`) - Compaction output file is assigned with the minimum `epoch_number` among input files' - Refit level: reuse refitted file's epoch_number - Other paths needing `epoch_number` treatment: - Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo` - Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`. - Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair). - Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder. - Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery - Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more - Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag` - Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above - Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`. - Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR. - Misc: - update existing tests with `epoch_number` so make check will pass - update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases - assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber() Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922 Test Plan: - `make check` - New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc` - Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930 - [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox` - [Ongoing] normal db stress test - [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761 Reviewed By: ajkr Differential Revision: D41063187 Pulled By: hx235 fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
if (s.ok()) {
dummy_vstorage.RecoverEpochNumbers(cfd);
}
if (s.ok()) {
// Record changes from this repair in VersionEdit, including files with
// recovered epoch numbers
VersionEdit edit;
edit.SetComparatorName(cfd->user_comparator()->Name());
edit.SetLogNumber(0);
edit.SetNextFile(next_file_number_);
edit.SetColumnFamily(cfd->GetID());
for (int level = 0; level < dummy_vstorage.num_levels(); ++level) {
for (FileMetaData* file_meta : dummy_vstorage.LevelFiles(level)) {
edit.AddFile(level, *file_meta);
}
}
// Release resources occupied by the dummy VersionStorageInfo
for (int level = 0; level < dummy_vstorage.num_levels(); ++level) {
for (FileMetaData* file_meta : dummy_vstorage.LevelFiles(level)) {
file_meta->refs--;
if (file_meta->refs <= 0) {
delete file_meta;
}
}
}
// Persist record of changes
assert(next_file_number_ > 0);
vset_.MarkFileNumberUsed(next_file_number_ - 1);
mutex_.Lock();
std::unique_ptr<FSDirectory> db_dir;
s = env_->GetFileSystem()->NewDirectory(dbname_, IOOptions(), &db_dir,
nullptr);
if (s.ok()) {
s = vset_.LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), &edit,
&mutex_, db_dir.get(),
false /* new_descriptor_log */);
}
mutex_.Unlock();
}
if (!s.ok()) {
return s;
}
}
return Status::OK();
}
void ArchiveFile(const std::string& fname) {
// Move into another directory. E.g., for
// dir/foo
// rename to
// dir/lost/foo
const char* slash = strrchr(fname.c_str(), '/');
std::string new_dir;
if (slash != nullptr) {
new_dir.assign(fname.data(), slash - fname.data());
}
new_dir.append("/lost");
env_->CreateDir(new_dir).PermitUncheckedError(); // Ignore error
std::string new_file = new_dir;
new_file.append("/");
new_file.append((slash == nullptr) ? fname.c_str() : slash + 1);
Status s = env_->RenameFile(fname, new_file);
ROCKS_LOG_INFO(db_options_.info_log, "Archiving %s: %s\n", fname.c_str(),
s.ToString().c_str());
}
};
Status GetDefaultCFOptions(
const std::vector<ColumnFamilyDescriptor>& column_families,
ColumnFamilyOptions* res) {
assert(res != nullptr);
auto iter = std::find_if(column_families.begin(), column_families.end(),
[](const ColumnFamilyDescriptor& cfd) {
return cfd.name == kDefaultColumnFamilyName;
});
if (iter == column_families.end()) {
return Status::InvalidArgument(
"column_families", "Must contain entry for default column family");
}
*res = iter->options;
return Status::OK();
}
} // anonymous namespace
Status RepairDB(const std::string& dbname, const DBOptions& db_options,
const std::vector<ColumnFamilyDescriptor>& column_families) {
ColumnFamilyOptions default_cf_opts;
Status status = GetDefaultCFOptions(column_families, &default_cf_opts);
if (!status.ok()) {
return status;
}
Repairer repairer(dbname, db_options, column_families, default_cf_opts,
ColumnFamilyOptions() /* unknown_cf_opts */,
false /* create_unknown_cfs */);
status = repairer.Run();
if (status.ok()) {
status = repairer.Close();
}
return status;
}
Status RepairDB(const std::string& dbname, const DBOptions& db_options,
const std::vector<ColumnFamilyDescriptor>& column_families,
const ColumnFamilyOptions& unknown_cf_opts) {
ColumnFamilyOptions default_cf_opts;
Status status = GetDefaultCFOptions(column_families, &default_cf_opts);
if (!status.ok()) {
return status;
}
Repairer repairer(dbname, db_options, column_families, default_cf_opts,
unknown_cf_opts, true /* create_unknown_cfs */);
status = repairer.Run();
if (status.ok()) {
status = repairer.Close();
}
return status;
}
Status RepairDB(const std::string& dbname, const Options& 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
5 years ago
Options opts(options);
DBOptions db_options(opts);
ColumnFamilyOptions cf_options(opts);
Repairer repairer(dbname, db_options, {}, cf_options /* default_cf_opts */,
cf_options /* unknown_cf_opts */,
true /* create_unknown_cfs */);
Status status = repairer.Run();
if (status.ok()) {
status = repairer.Close();
}
return status;
}
} // namespace ROCKSDB_NAMESPACE
#endif // ROCKSDB_LITE