rocksdb/utilities/fault_injection_fs.h
Hui Xiao d0259c2c98 Enable reading un-synced data in db stress test (#12752)
Summary:
**Context/Summary:**
There are a few blockers to enabling reading un-synced data in db stress test
(1) GetFileSize() will always return 0 for file written under direct IO because we don't track the last flushed position for `TestFSWritableFile` under direct IO. So it will surface as
```
Verification failed: VerifyChecksum failed: Corruption: file is too short (0 bytes) to be an sstable: /tmp/rocksdb_crashtest_blackbox4deg_c5e/000009.sst
db_stress: db_stress_tool/db_stress_test_base.cc:518: void rocksdb::StressTest::ProcessStatus(rocksdb::SharedState*, std::string, const rocksdb::Status&, bool) const: Assertion `false' failed.
Received signal 6 (Aborted)
Invoking GDB for stack trace...
```
(2) A couple minor FIXME in left in https://github.com/facebook/rocksdb/pull/12729.

This PR fixed (1) and (2) and enabled reading un-synced data in stress test.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12752

Test Plan:
- The following command failed before this PR and passed after.

```
./db_stress --WAL_size_limit_MB=1 --WAL_ttl_seconds=0 --acquire_snapshot_one_in=100 --adaptive_readahead=0 --adm_policy=0 --advise_random_on_open=1 --allow_concurrent_memtable_write=0 --allow_data_in_errors=True --allow_fallocate=1 --async_io=1 --atomic_flush=1 --auto_readahead_size=1 --avoid_flush_during_recovery=0 --avoid_flush_during_shutdown=1 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=0 --bgerror_resume_retry_interval=10000 --block_align=0 --block_protection_bytes_per_key=4 --block_size=16384 --bloom_before_level=2147483647 --bloom_bits=37.92024930098943 --bottommost_compression_type=disable --bottommost_file_compaction_delay=0 --bytes_per_sync=0 --cache_index_and_filter_blocks=1 --cache_index_and_filter_blocks_with_high_priority=0 --cache_size=8388608 --cache_type=auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=0 --charge_filter_construction=0 --charge_table_reader=1 --check_multiget_consistency=0 --check_multiget_entity_consistency=0 --checkpoint_one_in=1000000 --checksum_type=kXXH3 --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000 --compact_range_one_in=1000 --compaction_pri=3 --compaction_readahead_size=0 --compaction_ttl=10 --compress_format_version=2 --compressed_secondary_cache_size=8388608 --compression_checksum=1 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=zlib --compression_use_zstd_dict_trainer=0 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --daily_offpeak_time_utc= --data_block_index_type=0 --db=/tmp/rocksdb_crashtest_blackbox4deg_c5e --db_write_buffer_size=0 --default_temperature=kWarm --default_write_temperature=kHot --delete_obsolete_files_period_micros=30000000 --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_file_deletions_one_in=1000000 --disable_manual_compaction_one_in=10000 --disable_wal=1 --dump_malloc_stats=0 --enable_checksum_handoff=0 --enable_compaction_filter=0 --enable_custom_split_merge=1 --enable_do_not_compress_roles=0 --enable_index_compression=1 --enable_memtable_insert_with_hint_prefix_extractor=0 --enable_pipelined_write=0 --enable_sst_partitioner_factory=1 --enable_thread_tracking=0 --enable_write_thread_adaptive_yield=0 --expected_values_dir=/tmp/rocksdb_crashtest_expected_8whyhdxm --fail_if_options_file_error=0 --fifo_allow_compaction=1 --file_checksum_impl=xxh64 --fill_cache=1 --flush_one_in=1000 --format_version=4 --get_all_column_family_metadata_one_in=10000 --get_current_wal_file_one_in=0 --get_live_files_apis_one_in=1000000 --get_properties_of_all_tables_one_in=100000 --get_property_one_in=100000 --get_sorted_wal_files_one_in=0 --hard_pending_compaction_bytes_limit=274877906944 --high_pri_pool_ratio=0.5 --index_block_restart_interval=9 --index_shortening=0 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=0 --inplace_update_support=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --key_may_exist_one_in=100000 --last_level_temperature=kUnknown --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=100 --log_file_time_to_roll=0 --log_readahead_size=0 --long_running_snapshots=0 --low_pri_pool_ratio=0 --lowest_used_cache_tier=0 --manifest_preallocation_size=5120 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=0 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=100000 --max_key_len=3 --max_log_file_size=0 --max_manifest_file_size=1073741824 --max_sequential_skip_in_iterations=2 --max_total_wal_size=0 --max_write_batch_group_size_bytes=1048576 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=2097152 --memtable_insert_hint_per_batch=1 --memtable_max_range_deletions=100 --memtable_prefix_bloom_size_ratio=0.001 --memtable_protection_bytes_per_key=0 --memtable_whole_key_filtering=0 --memtablerep=skip_list --metadata_charge_policy=1 --min_write_buffer_number_to_merge=2 --mmap_read=0 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=16 --ops_per_thread=100000000 --optimize_filters_for_hits=1 --optimize_filters_for_memory=0 --optimize_multiget_for_io=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=1 --pause_background_one_in=1000000 --periodic_compaction_seconds=1000 --prefix_size=5 --prefixpercent=5 --prepopulate_block_cache=1 --preserve_internal_time_seconds=0 --progress_reports=0 --promote_l0_one_in=0 --read_amp_bytes_per_bit=0 --read_fault_one_in=1000 --readahead_size=16384 --readpercent=45 --recycle_log_file_num=0 --reopen=0 --report_bg_io_stats=0 --reset_stats_one_in=1000000 --sample_for_compression=5 --secondary_cache_fault_one_in=32 --secondary_cache_uri= --set_options_one_in=0 --skip_stats_update_on_db_open=0 --snapshot_hold_ops=100000 --soft_pending_compaction_bytes_limit=68719476736 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=1048576 --stats_dump_period_sec=0 --stats_history_buffer_size=0 --strict_bytes_per_sync=0 --subcompactions=2 --sync=0 --sync_fault_injection=1 --table_cache_numshardbits=0 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=0 --uncache_aggressiveness=1 --universal_max_read_amp=-1 --unpartitioned_pinning=0 --use_adaptive_mutex=0 --use_adaptive_mutex_lru=0 --use_attribute_group=1 --use_delta_encoding=1 --use_direct_io_for_flush_and_compaction=1 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=0 --use_multi_cf_iterator=0 --use_multi_get_entity=1 --use_multiget=0 --use_put_entity_one_in=5 --use_timed_put_one_in=0 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=10 --verify_compression=1 --verify_db_one_in=10000 --verify_file_checksums_one_in=10 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=0 --wal_compression=zstd --write_buffer_size=33554432 --write_dbid_to_manifest=0 --write_fault_one_in=0 --writepercent=35

Verification failed: VerifyChecksum failed: Corruption: file is too short (0 bytes) to be an sstable: /tmp/rocksdb_crashtest_blackbox4deg_c5e/000009.sst
db_stress: db_stress_tool/db_stress_test_base.cc:518: void rocksdb::StressTest::ProcessStatus(rocksdb::SharedState*, std::string, const rocksdb::Status&, bool) const: Assertion `false' failed.
Received signal 6 (Aborted)
Invoking GDB for stack trace...
```
- Run python3 tools/db_crashtest.py --simple blackbox --lock_wal_one_in=10 --backup_one_in=10 --sync_fault_injection=0 --use_direct_io_for_flush_and_compaction=0 for 1 hour
- Monitor stress test CI

Reviewed By: pdillinger

Differential Revision: D58395807

Pulled By: hx235

fbshipit-source-id: 7d4b321acc0a0af3501b62dc417a7f6e2d318265
2024-06-18 14:41:14 -07:00

631 lines
23 KiB
C++

// 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 2014 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.
// This test uses a custom FileSystem to keep track of the state of a file
// system the last "Sync". The data being written is cached in a "buffer".
// Only when "Sync" is called, the data will be persistent. It can similate
// file data loss (or entire files) not protected by a "Sync". For any of the
// FileSystem related operations, by specify the "IOStatus Error", a specific
// error can be returned when file system is not activated.
#pragma once
#include <algorithm>
#include <map>
#include <set>
#include <string>
#include "file/filename.h"
#include "rocksdb/file_system.h"
#include "util/mutexlock.h"
#include "util/random.h"
#include "util/thread_local.h"
namespace ROCKSDB_NAMESPACE {
class TestFSWritableFile;
class FaultInjectionTestFS;
struct FSFileState {
std::string filename_;
ssize_t pos_at_last_append_;
ssize_t pos_at_last_sync_;
std::string buffer_;
explicit FSFileState(const std::string& filename)
: filename_(filename), pos_at_last_append_(-1), pos_at_last_sync_(-1) {}
FSFileState() : pos_at_last_append_(-1), pos_at_last_sync_(-1) {}
bool IsFullySynced() const {
return pos_at_last_append_ <= 0 || pos_at_last_append_ == pos_at_last_sync_;
}
IOStatus DropUnsyncedData();
IOStatus DropRandomUnsyncedData(Random* rand);
};
// A wrapper around WritableFileWriter* file
// is written to or sync'ed.
class TestFSWritableFile : public FSWritableFile {
public:
explicit TestFSWritableFile(const std::string& fname,
const FileOptions& file_opts,
std::unique_ptr<FSWritableFile>&& f,
FaultInjectionTestFS* fs);
virtual ~TestFSWritableFile();
IOStatus Append(const Slice& data, const IOOptions&,
IODebugContext*) override;
IOStatus Append(const Slice& data, const IOOptions& options,
const DataVerificationInfo& verification_info,
IODebugContext* dbg) override;
IOStatus Truncate(uint64_t size, const IOOptions& options,
IODebugContext* dbg) override;
IOStatus Close(const IOOptions& options, IODebugContext* dbg) override;
IOStatus Flush(const IOOptions&, IODebugContext*) override;
IOStatus Sync(const IOOptions& options, IODebugContext* dbg) override;
IOStatus RangeSync(uint64_t /*offset*/, uint64_t /*nbytes*/,
const IOOptions& options, IODebugContext* dbg) override;
bool IsSyncThreadSafe() const override { return true; }
IOStatus PositionedAppend(const Slice& data, uint64_t offset,
const IOOptions& options,
IODebugContext* dbg) override;
IOStatus PositionedAppend(const Slice& data, uint64_t offset,
const IOOptions& options,
const DataVerificationInfo& verification_info,
IODebugContext* dbg) override;
size_t GetRequiredBufferAlignment() const override {
return target_->GetRequiredBufferAlignment();
}
bool use_direct_io() const override { return target_->use_direct_io(); }
uint64_t GetFileSize(const IOOptions& options, IODebugContext* dbg) override {
MutexLock l(&mutex_);
return target_->GetFileSize(options, dbg);
}
private:
FSFileState state_; // Need protection by mutex_
FileOptions file_opts_;
std::unique_ptr<FSWritableFile> target_;
bool writable_file_opened_;
FaultInjectionTestFS* fs_;
port::Mutex mutex_;
};
// A wrapper around WritableFileWriter* file
// is written to or sync'ed.
class TestFSRandomRWFile : public FSRandomRWFile {
public:
explicit TestFSRandomRWFile(const std::string& fname,
std::unique_ptr<FSRandomRWFile>&& f,
FaultInjectionTestFS* fs);
virtual ~TestFSRandomRWFile();
IOStatus Write(uint64_t offset, const Slice& data, const IOOptions& options,
IODebugContext* dbg) override;
IOStatus Read(uint64_t offset, size_t n, const IOOptions& options,
Slice* result, char* scratch,
IODebugContext* dbg) const override;
IOStatus Close(const IOOptions& options, IODebugContext* dbg) override;
IOStatus Flush(const IOOptions& options, IODebugContext* dbg) override;
IOStatus Sync(const IOOptions& options, IODebugContext* dbg) override;
size_t GetRequiredBufferAlignment() const override {
return target_->GetRequiredBufferAlignment();
}
bool use_direct_io() const override { return target_->use_direct_io(); }
private:
std::unique_ptr<FSRandomRWFile> target_;
bool file_opened_;
FaultInjectionTestFS* fs_;
};
class TestFSRandomAccessFile : public FSRandomAccessFile {
public:
explicit TestFSRandomAccessFile(const std::string& fname,
std::unique_ptr<FSRandomAccessFile>&& f,
FaultInjectionTestFS* fs);
~TestFSRandomAccessFile() override {}
IOStatus Read(uint64_t offset, size_t n, const IOOptions& options,
Slice* result, char* scratch,
IODebugContext* dbg) const override;
IOStatus ReadAsync(FSReadRequest& req, const IOOptions& opts,
std::function<void(FSReadRequest&, void*)> cb,
void* cb_arg, void** io_handle, IOHandleDeleter* del_fn,
IODebugContext* dbg) override;
IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs,
const IOOptions& options, IODebugContext* dbg) override;
size_t GetRequiredBufferAlignment() const override {
return target_->GetRequiredBufferAlignment();
}
bool use_direct_io() const override { return target_->use_direct_io(); }
size_t GetUniqueId(char* id, size_t max_size) const override;
private:
std::unique_ptr<FSRandomAccessFile> target_;
FaultInjectionTestFS* fs_;
};
class TestFSSequentialFile : public FSSequentialFileOwnerWrapper {
public:
explicit TestFSSequentialFile(std::unique_ptr<FSSequentialFile>&& f,
FaultInjectionTestFS* fs, std::string fname)
: FSSequentialFileOwnerWrapper(std::move(f)),
fs_(fs),
fname_(std::move(fname)) {}
IOStatus Read(size_t n, const IOOptions& options, Slice* result,
char* scratch, IODebugContext* dbg) override;
IOStatus PositionedRead(uint64_t offset, size_t n, const IOOptions& options,
Slice* result, char* scratch,
IODebugContext* dbg) override;
private:
FaultInjectionTestFS* fs_;
std::string fname_;
size_t read_pos_ = 0;
};
class TestFSDirectory : public FSDirectory {
public:
explicit TestFSDirectory(FaultInjectionTestFS* fs, std::string dirname,
FSDirectory* dir)
: fs_(fs), dirname_(std::move(dirname)), dir_(dir) {}
~TestFSDirectory() {}
IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) override;
IOStatus Close(const IOOptions& options, IODebugContext* dbg) override;
IOStatus FsyncWithDirOptions(
const IOOptions& options, IODebugContext* dbg,
const DirFsyncOptions& dir_fsync_options) override;
private:
FaultInjectionTestFS* fs_;
std::string dirname_;
std::unique_ptr<FSDirectory> dir_;
};
class FaultInjectionTestFS : public FileSystemWrapper {
public:
explicit FaultInjectionTestFS(const std::shared_ptr<FileSystem>& base)
: FileSystemWrapper(base),
filesystem_active_(true),
filesystem_writable_(false),
read_unsynced_data_(true),
allow_link_open_file_(false),
thread_local_error_(new ThreadLocalPtr(DeleteThreadLocalErrorContext)),
enable_write_error_injection_(false),
enable_metadata_write_error_injection_(false),
write_error_rand_(0),
write_error_one_in_(0),
metadata_write_error_one_in_(0),
read_error_one_in_(0),
ingest_data_corruption_before_write_(false),
checksum_handoff_func_type_(kCRC32c),
fail_get_file_unique_id_(false) {}
virtual ~FaultInjectionTestFS() { error_.PermitUncheckedError(); }
static const char* kClassName() { return "FaultInjectionTestFS"; }
const char* Name() const override { return kClassName(); }
IOStatus NewDirectory(const std::string& name, const IOOptions& options,
std::unique_ptr<FSDirectory>* result,
IODebugContext* dbg) override;
IOStatus NewWritableFile(const std::string& fname,
const FileOptions& file_opts,
std::unique_ptr<FSWritableFile>* result,
IODebugContext* dbg) override;
IOStatus ReopenWritableFile(const std::string& fname,
const FileOptions& file_opts,
std::unique_ptr<FSWritableFile>* result,
IODebugContext* dbg) override;
IOStatus ReuseWritableFile(const std::string& fname,
const std::string& old_fname,
const FileOptions& file_opts,
std::unique_ptr<FSWritableFile>* result,
IODebugContext* dbg) override;
IOStatus NewRandomRWFile(const std::string& fname,
const FileOptions& file_opts,
std::unique_ptr<FSRandomRWFile>* result,
IODebugContext* dbg) override;
IOStatus NewRandomAccessFile(const std::string& fname,
const FileOptions& file_opts,
std::unique_ptr<FSRandomAccessFile>* result,
IODebugContext* dbg) override;
IOStatus NewSequentialFile(const std::string& f, const FileOptions& file_opts,
std::unique_ptr<FSSequentialFile>* r,
IODebugContext* dbg) override;
IOStatus DeleteFile(const std::string& f, const IOOptions& options,
IODebugContext* dbg) override;
IOStatus GetFileSize(const std::string& f, const IOOptions& options,
uint64_t* file_size, IODebugContext* dbg) override;
IOStatus RenameFile(const std::string& s, const std::string& t,
const IOOptions& options, IODebugContext* dbg) override;
IOStatus LinkFile(const std::string& src, const std::string& target,
const IOOptions& options, IODebugContext* dbg) override;
// Undef to eliminate clash on Windows
#undef GetFreeSpace
IOStatus GetFreeSpace(const std::string& path, const IOOptions& options,
uint64_t* disk_free, IODebugContext* dbg) override {
IOStatus io_s;
if (!IsFilesystemActive() &&
error_.subcode() == IOStatus::SubCode::kNoSpace) {
*disk_free = 0;
} else {
io_s = target()->GetFreeSpace(path, options, disk_free, dbg);
}
return io_s;
}
IOStatus Poll(std::vector<void*>& io_handles,
size_t min_completions) override;
IOStatus AbortIO(std::vector<void*>& io_handles) override;
void WritableFileClosed(const FSFileState& state);
void WritableFileSynced(const FSFileState& state);
void WritableFileAppended(const FSFileState& state);
IOStatus DropUnsyncedFileData();
IOStatus DropRandomUnsyncedFileData(Random* rnd);
IOStatus DeleteFilesCreatedAfterLastDirSync(const IOOptions& options,
IODebugContext* dbg);
void ResetState();
void UntrackFile(const std::string& f);
void SyncDir(const std::string& dirname) {
MutexLock l(&mutex_);
dir_to_new_files_since_last_sync_.erase(dirname);
}
// Setting the filesystem to inactive is the test equivalent to simulating a
// system reset. Setting to inactive will freeze our saved filesystem state so
// that it will stop being recorded. It can then be reset back to the state at
// the time of the reset.
bool IsFilesystemActive() {
MutexLock l(&mutex_);
return filesystem_active_;
}
// Setting filesystem_writable_ makes NewWritableFile. ReopenWritableFile,
// and NewRandomRWFile bypass FaultInjectionTestFS and go directly to the
// target FS
bool IsFilesystemDirectWritable() {
MutexLock l(&mutex_);
return filesystem_writable_;
}
bool ShouldUseDiretWritable(const std::string& file_name) {
MutexLock l(&mutex_);
if (filesystem_writable_) {
return true;
}
FileType file_type = kTempFile;
uint64_t file_number = 0;
if (!TryParseFileName(file_name, &file_number, &file_type)) {
return false;
}
return direct_writable_types_.find(file_type) !=
direct_writable_types_.end();
}
void SetFilesystemActiveNoLock(
bool active, IOStatus error = IOStatus::Corruption("Not active")) {
error.PermitUncheckedError();
filesystem_active_ = active;
if (!active) {
error_ = error;
}
}
void SetFilesystemActive(
bool active, IOStatus error = IOStatus::Corruption("Not active")) {
MutexLock l(&mutex_);
error.PermitUncheckedError();
SetFilesystemActiveNoLock(active, error);
}
void SetFilesystemDirectWritable(bool writable) {
MutexLock l(&mutex_);
filesystem_writable_ = writable;
}
// In places (e.g. GetSortedWals()) RocksDB relies on querying the file size
// or even reading the contents of files currently open for writing, and
// as in POSIX semantics, expects to see the flushed size and contents
// regardless of what has been synced. FaultInjectionTestFS historically
// did not emulate this behavior, only showing synced data from such read
// operations. (Different from FaultInjectionTestEnv--sigh.) Calling this
// function with false restores this historical behavior for testing
// stability, but use of this semantics must be phased out as it is
// inconsistent with expected FileSystem semantics. In other words, this
// functionality is DEPRECATED. Intended to be set after construction and
// unchanged (not thread safe).
void SetReadUnsyncedData(bool read_unsynced_data) {
read_unsynced_data_ = read_unsynced_data;
}
bool ReadUnsyncedData() const { return read_unsynced_data_; }
// FaultInjectionTestFS normally includes a hygiene check for FileSystem
// implementations that only support LinkFile() on closed files (not open
// for write). Setting this to true bypasses the check.
void SetAllowLinkOpenFile(bool allow_link_open_file = true) {
allow_link_open_file_ = allow_link_open_file;
}
void AssertNoOpenFile() { assert(open_managed_files_.empty()); }
IOStatus GetError() { return error_; }
void SetFileSystemIOError(IOStatus io_error) {
MutexLock l(&mutex_);
io_error.PermitUncheckedError();
error_ = io_error;
}
// To simulate the data corruption before data is written in FS
void IngestDataCorruptionBeforeWrite() {
MutexLock l(&mutex_);
ingest_data_corruption_before_write_ = true;
}
void NoDataCorruptionBeforeWrite() {
MutexLock l(&mutex_);
ingest_data_corruption_before_write_ = false;
}
bool ShouldDataCorruptionBeforeWrite() {
MutexLock l(&mutex_);
return ingest_data_corruption_before_write_;
}
void SetChecksumHandoffFuncType(const ChecksumType& func_type) {
MutexLock l(&mutex_);
checksum_handoff_func_type_ = func_type;
}
const ChecksumType& GetChecksumHandoffFuncType() {
MutexLock l(&mutex_);
return checksum_handoff_func_type_;
}
void SetFailGetUniqueId(bool flag) {
MutexLock l(&mutex_);
fail_get_file_unique_id_ = flag;
}
bool ShouldFailGetUniqueId() {
MutexLock l(&mutex_);
return fail_get_file_unique_id_;
}
// Specify what the operation, so we can inject the right type of error
enum ErrorOperation : char {
kRead = 0,
kMultiReadSingleReq = 1,
kMultiRead = 2,
kOpen,
};
// Set thread-local parameters for error injection. The first argument,
// seed is the seed for the random number generator, and one_in determines
// the probability of injecting error (i.e an error is injected with
// 1/one_in probability)
void SetThreadLocalReadErrorContext(uint32_t seed, int one_in,
bool retryable) {
struct ErrorContext* ctx =
static_cast<struct ErrorContext*>(thread_local_error_->Get());
if (ctx == nullptr) {
ctx = new ErrorContext(seed);
thread_local_error_->Reset(ctx);
}
ctx->one_in = one_in;
ctx->count = 0;
ctx->retryable = retryable;
}
static void DeleteThreadLocalErrorContext(void* p) {
ErrorContext* ctx = static_cast<ErrorContext*>(p);
delete ctx;
}
// This is to set the parameters for the write error injection.
// seed is the seed for the random number generator, and one_in determines
// the probability of injecting error (i.e an error is injected with
// 1/one_in probability). For write error, we can specify the error we
// want to inject. Types decides the file types we want to inject the
// error (e.g., Wal files, SST files), which is empty by default.
void SetRandomWriteError(uint32_t seed, int one_in, IOStatus error,
bool inject_for_all_file_types,
const std::vector<FileType>& types) {
MutexLock l(&mutex_);
Random tmp_rand(seed);
error.PermitUncheckedError();
error_ = error;
write_error_rand_ = tmp_rand;
write_error_one_in_ = one_in;
inject_for_all_file_types_ = inject_for_all_file_types;
write_error_allowed_types_ = types;
}
void SetDirectWritableTypes(const std::set<FileType>& types) {
MutexLock l(&mutex_);
direct_writable_types_ = types;
}
void SetRandomMetadataWriteError(int one_in) {
MutexLock l(&mutex_);
metadata_write_error_one_in_ = one_in;
}
// If the value is not 0, it is enabled. Otherwise, it is disabled.
void SetRandomReadError(int one_in) { read_error_one_in_ = one_in; }
bool ShouldInjectRandomReadError() {
auto one_in = read_error_one_in();
return one_in > 0 && Random::GetTLSInstance()->OneIn(one_in);
}
// Inject an write error with randomlized parameter and the predefined
// error type. Only the allowed file types will inject the write error
IOStatus InjectWriteError(const std::string& file_name);
// Ingest error to metadata operations.
IOStatus InjectMetadataWriteError();
// Inject an error. For a READ operation, a status of IOError(), a
// corruption in the contents of scratch, or truncation of slice
// are the types of error with equal probability. For OPEN,
// its always an IOError.
// fault_injected returns whether a fault is injected. It is needed
// because some fault is inected with IOStatus to be OK.
IOStatus InjectThreadSpecificReadError(ErrorOperation op, Slice* slice,
bool direct_io, char* scratch,
bool need_count_increase,
bool* fault_injected);
// Get the count of how many times we injected since the previous call
int GetAndResetErrorCount() {
ErrorContext* ctx = static_cast<ErrorContext*>(thread_local_error_->Get());
int count = 0;
if (ctx != nullptr) {
count = ctx->count;
ctx->count = 0;
}
return count;
}
void EnableErrorInjection() {
ErrorContext* ctx = static_cast<ErrorContext*>(thread_local_error_->Get());
if (ctx) {
ctx->enable_error_injection = true;
}
}
void EnableWriteErrorInjection() {
MutexLock l(&mutex_);
enable_write_error_injection_ = true;
}
void EnableMetadataWriteErrorInjection() {
MutexLock l(&mutex_);
enable_metadata_write_error_injection_ = true;
}
void DisableWriteErrorInjection() {
MutexLock l(&mutex_);
enable_write_error_injection_ = false;
}
void DisableErrorInjection() {
ErrorContext* ctx = static_cast<ErrorContext*>(thread_local_error_->Get());
if (ctx) {
ctx->enable_error_injection = false;
}
}
void DisableMetadataWriteErrorInjection() {
MutexLock l(&mutex_);
enable_metadata_write_error_injection_ = false;
}
int read_error_one_in() const { return read_error_one_in_.load(); }
int write_error_one_in() const { return write_error_one_in_; }
// We capture a backtrace every time a fault is injected, for debugging
// purposes. This call prints the backtrace to stderr and frees the
// saved callstack
void PrintFaultBacktrace();
void AddUnsyncedToRead(const std::string& fname, size_t offset, size_t n,
Slice* result, char* scratch);
private:
port::Mutex mutex_;
std::map<std::string, FSFileState> db_file_state_;
std::set<std::string> open_managed_files_;
// directory -> (file name -> file contents to recover)
// When data is recovered from unsyned parent directory, the files with
// empty file contents to recover is deleted. Those with non-empty ones
// will be recovered to content accordingly.
std::unordered_map<std::string, std::map<std::string, std::string>>
dir_to_new_files_since_last_sync_;
bool filesystem_active_; // Record flushes, syncs, writes
bool filesystem_writable_; // Bypass FaultInjectionTestFS and go directly
// to underlying FS for writable files
bool read_unsynced_data_; // See SetReadUnsyncedData()
bool allow_link_open_file_; // See SetAllowLinkOpenFile()
IOStatus error_;
enum ErrorType : int {
kErrorTypeStatus = 0,
kErrorTypeCorruption,
kErrorTypeTruncated,
kErrorTypeMax
};
struct ErrorContext {
Random rand;
int one_in;
int count;
bool enable_error_injection;
void* callstack;
std::string message;
int frames;
ErrorType type;
bool retryable;
explicit ErrorContext(uint32_t seed)
: rand(seed),
enable_error_injection(false),
callstack(nullptr),
frames(0),
retryable(false) {}
~ErrorContext() {
if (callstack) {
free(callstack);
}
}
};
std::unique_ptr<ThreadLocalPtr> thread_local_error_;
bool enable_write_error_injection_;
bool enable_metadata_write_error_injection_;
Random write_error_rand_;
int write_error_one_in_;
int metadata_write_error_one_in_;
std::atomic<int> read_error_one_in_;
bool inject_for_all_file_types_;
std::vector<FileType> write_error_allowed_types_;
// File types where direct writable is skipped.
std::set<FileType> direct_writable_types_;
bool ingest_data_corruption_before_write_;
ChecksumType checksum_handoff_func_type_;
bool fail_get_file_unique_id_;
// Extract number of type from file name. Return false if failing to fine
// them.
bool TryParseFileName(const std::string& file_name, uint64_t* number,
FileType* type);
};
} // namespace ROCKSDB_NAMESPACE