rocksdb/utilities/fault_injection_fs.h
Hui Xiao 0d93c8a6ca Decouple sync fault and write injection in FaultInjectionTestFS & fix tracing issue under WAL write error injection (#12797)
Summary:
**Context/Summary:**

After injecting write error to WAL, we started to see crash recovery verification failure in prefix recovery. That's because the current tracing implementation traces every write before it writes to WAL even when the WAL write can fail with write error injection. One consequence of that is the traced writes in trace files does not corresponding to write sequence sequence anymore e.g, it has more traced writes that the actual assigned sequence number to successful writes. Therefore b4a84efb4e/db_stress_tool/expected_state.cc (L674) won't restore the ExpectedState to the correct sequence number we want.

Ideally, we should have a prepare-commit mechanism for tracing just like our ExpectedState so we can ignore the traced write if the write fails later. But for now, to simplify, we simply don't inject WAL error (and metadata write error cuz it could fail write when sync WAL dir fails)

To do so, we need to be able to exclude WAL from write injection but still allow sync fault injection in it to maintain its original sync fault testing coverage. This prompts us to decouple sync fault and write injection in FaultInjectionTestFS. And this is what this PR mainly about.

So now `FaultInjectionTestFS` works as the following:
- If direct_writable is true, then `FaultInjectionTestFS` is bypassed for writable file
- Otherwise, FaultInjectionTestFS` can buffer data for sync fault injection (if inject_unsynced_data_loss_ == true, global settings) and/or inject write error (if MaybeInjectThreadLocalError(), thread-local settings). WAL file can be optionally excluded from write injection

Bonus: better naming of relevant variables

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

Test Plan:
- The follow commands failed before this fix but passes after
```
python3 tools/db_crashtest.py --simple blackbox \
    --interval=5 \
    --preserve_unverified_changes=1 \
    --threads=32 \
    --disable_auto_compactions=1 \
    --WAL_size_limit_MB=0 --WAL_ttl_seconds=0 --acquire_snapshot_one_in=0 --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=0 --auto_readahead_size=0 --avoid_flush_during_recovery=1 --avoid_flush_during_shutdown=0 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=0 --batch_protection_bytes_per_key=0 --bgerror_resume_retry_interval=1000000 --block_align=0 --block_protection_bytes_per_key=4 --block_size=16384 --bloom_before_level=2147483646 --bloom_bits=3.2003682301518492 --bottommost_compression_type=zlib --bottommost_file_compaction_delay=600 --bytes_per_sync=0 --cache_index_and_filter_blocks=1 --cache_index_and_filter_blocks_with_high_priority=1 --cache_size=33554432 --cache_type=fixed_hyper_clock_cache --charge_compression_dictionary_building_buffer=0 --charge_file_metadata=0 --charge_filter_construction=0 --charge_table_reader=1 --check_multiget_consistency=0 --check_multiget_entity_consistency=0 --checkpoint_one_in=0 --checksum_type=kxxHash64 --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=0 --compact_range_one_in=0 --compaction_pri=2 --compaction_readahead_size=0 --compaction_ttl=0 --compress_format_version=1 --compressed_secondary_cache_size=16777216 --compression_checksum=1 --compression_max_dict_buffer_bytes=549755813887 --compression_max_dict_bytes=16384 --compression_parallel_threads=1 --compression_type=none --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --daily_offpeak_time_utc=00:00-23:59 --data_block_index_type=0 \
    --db_write_buffer_size=0 --delete_obsolete_files_period_micros=0 --delpercent=0 --delrangepercent=0 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_file_deletions_one_in=0 --disable_manual_compaction_one_in=0 --disable_wal=0 --dump_malloc_stats=0 --enable_checksum_handoff=0 --enable_compaction_filter=0 --enable_custom_split_merge=0 --enable_do_not_compress_roles=1 --enable_index_compression=0 --enable_memtable_insert_with_hint_prefix_extractor=0 --enable_pipelined_write=0 --enable_sst_partitioner_factory=0 --enable_thread_tracking=0 --enable_write_thread_adaptive_yield=0 --error_recovery_with_no_fault_injection=0 --fail_if_options_file_error=0 --fifo_allow_compaction=1 --file_checksum_impl=xxh64 --fill_cache=0 --flush_one_in=100 --format_version=4 --get_all_column_family_metadata_one_in=0 --get_current_wal_file_one_in=0 --get_live_files_apis_one_in=0 --get_properties_of_all_tables_one_in=0 --get_property_one_in=0 --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=1 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=0 --inplace_update_support=0 --iterpercent=0 --key_len_percent_dist=1,30,69 --key_may_exist_one_in=0 --last_level_temperature=kUnknown --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=0 --log2_keys_per_lock=10 --log_file_time_to_roll=0 --log_readahead_size=16777216 --long_running_snapshots=0 --low_pri_pool_ratio=0 --lowest_used_cache_tier=2 --manifest_preallocation_size=0 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=524288 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=1000 --max_key_len=3 --memtable_insert_hint_per_batch=0 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.5 --memtable_protection_bytes_per_key=8 --memtable_whole_key_filtering=0 --memtablerep=skip_list --metadata_charge_policy=0 --metadata_read_fault_one_in=0 --metadata_write_fault_one_in=0 --min_write_buffer_number_to_merge=1 --mmap_read=0 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=0 --open_files=-1 --open_metadata_read_fault_one_in=0 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=20000000 \
    --optimize_filters_for_hits=1 --optimize_filters_for_memory=1 --optimize_multiget_for_io=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=3 --pause_background_one_in=0 --periodic_compaction_seconds=0 --prefix_size=1 --prefixpercent=0 --prepopulate_block_cache=0 --preserve_internal_time_seconds=0 --progress_reports=0 --promote_l0_one_in=0 --read_amp_bytes_per_bit=0 --read_fault_one_in=0 --readahead_size=0 --readpercent=0 --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=0 --secondary_cache_uri= --skip_stats_update_on_db_open=0 --snapshot_hold_ops=100000 --soft_pending_compaction_bytes_limit=68719476736 --sqfc_name=bar --sqfc_version=1 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=10 --stats_history_buffer_size=0 --strict_bytes_per_sync=0 --subcompactions=1 --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=3 --uncache_aggressiveness=9890 --universal_max_read_amp=-1 --unpartitioned_pinning=3 --use_adaptive_mutex=0 --use_adaptive_mutex_lru=1 --use_attribute_group=0 --use_delta_encoding=0 --use_direct_io_for_flush_and_compaction=0 --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=0 --use_multiget=0 --use_put_entity_one_in=0 --use_sqfc_for_range_queries=0 --use_timed_put_one_in=0 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=0 --verify_checksum_one_in=0 --verify_compression=1 --verify_db_one_in=0 --verify_file_checksums_one_in=0 --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=335544320 --write_dbid_to_manifest=1 --write_fault_one_in=100 --writepercent=100

```
- CI

Reviewed By: cbi42

Differential Revision: D58917145

Pulled By: hx235

fbshipit-source-id: b6397036bea035a92341c2b05fb01872db2153d7
2024-06-26 14:56:35 -07:00

712 lines
26 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;
enum class FaultInjectionIOType {
kRead = 0,
kWrite,
kMetadataRead,
kMetadataWrite,
};
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),
inject_unsynced_data_loss_(false),
read_unsynced_data_(true),
allow_link_open_file_(false),
injected_thread_local_read_error_(DeleteThreadLocalErrorContext),
injected_thread_local_write_error_(DeleteThreadLocalErrorContext),
injected_thread_local_metadata_read_error_(
DeleteThreadLocalErrorContext),
injected_thread_local_metadata_write_error_(
DeleteThreadLocalErrorContext),
ingest_data_corruption_before_write_(false),
checksum_handoff_func_type_(kCRC32c),
fail_get_file_unique_id_(false) {}
virtual ~FaultInjectionTestFS() override { fs_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 FileExists(const std::string& fname, const IOOptions& options,
IODebugContext* dbg) override;
IOStatus GetChildren(const std::string& dir, const IOOptions& options,
std::vector<std::string>* result,
IODebugContext* dbg) override;
IOStatus GetChildrenFileAttributes(const std::string& dir,
const IOOptions& options,
std::vector<FileAttributes>* 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 GetFileModificationTime(const std::string& fname,
const IOOptions& options,
uint64_t* file_mtime,
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;
IOStatus NumFileLinks(const std::string& fname, const IOOptions& options,
uint64_t* count, IODebugContext* dbg) override;
IOStatus AreFilesSame(const std::string& first, const std::string& second,
const IOOptions& options, bool* res,
IODebugContext* dbg) override;
IOStatus GetAbsolutePath(const std::string& db_path, const IOOptions& options,
std::string* output_path,
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() &&
fs_error_.subcode() == IOStatus::SubCode::kNoSpace) {
*disk_free = 0;
} else {
io_s = MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead,
options);
if (io_s.ok()) {
io_s = target()->GetFreeSpace(path, options, disk_free, dbg);
}
}
return io_s;
}
IOStatus IsDirectory(const std::string& path, const IOOptions& options,
bool* is_dir, IODebugContext* dgb) override;
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_;
}
void SetFilesystemActiveNoLock(
bool active, IOStatus error = IOStatus::Corruption("Not active")) {
error.PermitUncheckedError();
filesystem_active_ = active;
if (!active) {
fs_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;
}
// If true, we buffer write data in memory to simulate data loss upon system
// crash by only having process crashes
void SetInjectUnsyncedDataLoss(bool inject) {
MutexLock l(&mutex_);
inject_unsynced_data_loss_ = inject;
}
bool InjectUnsyncedDataLoss() {
MutexLock l(&mutex_);
return inject_unsynced_data_loss_;
}
// 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;
}
bool ShouldIOActivtiesExcludedFromFaultInjection(Env::IOActivity io_activty) {
MutexLock l(&mutex_);
return io_activties_excluded_from_fault_injection.find(io_activty) !=
io_activties_excluded_from_fault_injection.end();
}
void AssertNoOpenFile() { assert(open_managed_files_.empty()); }
IOStatus GetError() { return fs_error_; }
void SetFileSystemIOError(IOStatus io_error) {
MutexLock l(&mutex_);
io_error.PermitUncheckedError();
fs_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,
kUnknown,
};
void SetThreadLocalErrorContext(FaultInjectionIOType type, uint32_t seed,
int one_in, bool retryable,
bool has_data_loss) {
struct ErrorContext* new_ctx = new ErrorContext(seed);
new_ctx->one_in = one_in;
new_ctx->count = 0;
new_ctx->retryable = retryable;
new_ctx->has_data_loss = has_data_loss;
SetErrorContextOfFaultInjectionIOType(type, new_ctx);
}
static void DeleteThreadLocalErrorContext(void* p) {
ErrorContext* ctx = static_cast<ErrorContext*>(p);
delete ctx;
}
IOStatus MaybeInjectThreadLocalError(
FaultInjectionIOType type, const IOOptions& io_options,
const std::string& file_name = "", ErrorOperation op = kUnknown,
Slice* slice = nullptr, bool direct_io = false, char* scratch = nullptr,
bool need_count_increase = false, bool* fault_injected = nullptr);
int GetAndResetInjectedThreadLocalErrorCount(FaultInjectionIOType type) {
ErrorContext* ctx = GetErrorContextFromFaultInjectionIOType(type);
int count = 0;
if (ctx) {
count = ctx->count;
ctx->count = 0;
}
return count;
}
void SetIOActivtiesExcludedFromFaultInjection(
const std::set<Env::IOActivity>& io_activties) {
MutexLock l(&mutex_);
io_activties_excluded_from_fault_injection = io_activties;
}
void SetFileTypesExcludedFromWriteFaultInjection(
const std::set<FileType>& types) {
MutexLock l(&mutex_);
file_types_excluded_from_write_fault_injection_ = types;
}
bool ShouldExcludeFromWriteFaultInjection(const std::string& file_name) {
MutexLock l(&mutex_);
FileType file_type = kTempFile;
uint64_t file_number = 0;
if (!TryParseFileName(file_name, &file_number, &file_type)) {
return false;
}
return file_types_excluded_from_write_fault_injection_.find(file_type) !=
file_types_excluded_from_write_fault_injection_.end();
}
void EnableThreadLocalErrorInjection(FaultInjectionIOType type) {
ErrorContext* ctx = GetErrorContextFromFaultInjectionIOType(type);
if (ctx) {
ctx->enable_error_injection = true;
}
}
void DisableThreadLocalErrorInjection(FaultInjectionIOType type) {
ErrorContext* ctx = GetErrorContextFromFaultInjectionIOType(type);
if (ctx) {
ctx->enable_error_injection = false;
}
}
void PrintInjectedThreadLocalErrorBacktrace(FaultInjectionIOType type);
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 inject_unsynced_data_loss_; // See InjectUnsyncedDataLoss()
bool read_unsynced_data_; // See SetReadUnsyncedData()
bool allow_link_open_file_; // See SetAllowLinkOpenFile()
IOStatus fs_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;
bool has_data_loss;
explicit ErrorContext(uint32_t seed)
: rand(seed),
enable_error_injection(false),
callstack(nullptr),
frames(0),
retryable(false),
has_data_loss(false) {}
~ErrorContext() {
if (callstack) {
free(callstack);
}
}
};
std::set<FileType> file_types_excluded_from_write_fault_injection_;
std::set<Env::IOActivity> io_activties_excluded_from_fault_injection;
ThreadLocalPtr injected_thread_local_read_error_;
ThreadLocalPtr injected_thread_local_write_error_;
ThreadLocalPtr injected_thread_local_metadata_read_error_;
ThreadLocalPtr injected_thread_local_metadata_write_error_;
bool ingest_data_corruption_before_write_;
ChecksumType checksum_handoff_func_type_;
bool fail_get_file_unique_id_;
// 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 MaybeInjectThreadLocalReadError(const IOOptions& io_options,
ErrorOperation op, Slice* slice,
bool direct_io, char* scratch,
bool need_count_increase,
bool* fault_injected);
// 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);
ErrorContext* GetErrorContextFromFaultInjectionIOType(
FaultInjectionIOType type) {
ErrorContext* ctx = nullptr;
switch (type) {
case FaultInjectionIOType::kRead:
ctx = static_cast<struct ErrorContext*>(
injected_thread_local_read_error_.Get());
break;
case FaultInjectionIOType::kWrite:
ctx = static_cast<struct ErrorContext*>(
injected_thread_local_write_error_.Get());
break;
case FaultInjectionIOType::kMetadataRead:
ctx = static_cast<struct ErrorContext*>(
injected_thread_local_metadata_read_error_.Get());
break;
case FaultInjectionIOType::kMetadataWrite:
ctx = static_cast<struct ErrorContext*>(
injected_thread_local_metadata_write_error_.Get());
break;
default:
assert(false);
break;
}
return ctx;
}
void SetErrorContextOfFaultInjectionIOType(FaultInjectionIOType type,
ErrorContext* new_ctx) {
ErrorContext* old_ctx = nullptr;
switch (type) {
case FaultInjectionIOType::kRead:
old_ctx = static_cast<struct ErrorContext*>(
injected_thread_local_read_error_.Swap(new_ctx));
break;
case FaultInjectionIOType::kWrite:
old_ctx = static_cast<struct ErrorContext*>(
injected_thread_local_write_error_.Swap(new_ctx));
break;
case FaultInjectionIOType::kMetadataRead:
old_ctx = static_cast<struct ErrorContext*>(
injected_thread_local_metadata_read_error_.Swap(new_ctx));
break;
case FaultInjectionIOType::kMetadataWrite:
old_ctx = static_cast<struct ErrorContext*>(
injected_thread_local_metadata_write_error_.Swap(new_ctx));
break;
default:
assert(false);
break;
}
if (old_ctx) {
DeleteThreadLocalErrorContext(old_ctx);
}
}
std::string GetErrorMessageFromFaultInjectionIOType(
FaultInjectionIOType type) {
std::string msg = "";
switch (type) {
case FaultInjectionIOType::kRead:
msg = "injected read error";
break;
case FaultInjectionIOType::kWrite:
msg = "injected write error";
break;
case FaultInjectionIOType::kMetadataRead:
msg = "injected metadata read error";
break;
case FaultInjectionIOType::kMetadataWrite:
msg = "injected metadata write error";
break;
default:
assert(false);
break;
}
return msg;
}
};
} // namespace ROCKSDB_NAMESPACE