rocksdb/utilities/fault_injection_fs.cc

1357 lines
44 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright 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 simulate
// 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.
#include "utilities/fault_injection_fs.h"
#include <algorithm>
Ensure Close() before LinkFile() for WALs in Checkpoint (#12734) Summary: POSIX semantics for LinkFile (hard links) allow linking a file that is still being written two, with both the source and destination showing any subsequent writes to the source. This may not be practical semantics for some FileSystem implementations such as remote storage. They might only link the flushed or sync-ed file contents at time of LinkFile, or might even have undefined behavior if LinkFile is called on a file still open for write (not yet "sealed"). This change builds on https://github.com/facebook/rocksdb/issues/12731 to bring more hygiene to our handling of WAL files in Checkpoint. Specifically, we now Close WAL files as soon as they are either (a) inactive and fully synced, or (b) inactive and obsolete (so maybe never fully synced), rather than letting Close() happen in handling obsolete files (maybe a background thread). This should not be a performance issue as Close() should be trivial cost relative to other IO ops, but just in case: * We don't Close() while holding a mutex, to avoid blocking, and * The old behavior is available with a new kill switch option `background_close_inactive_wals`. Stacked on https://github.com/facebook/rocksdb/issues/12731 Pull Request resolved: https://github.com/facebook/rocksdb/pull/12734 Test Plan: Extended existing unit test, especially adding a hygiene check to FaultInjectionTestFS to detect LinkFile() on a file still open for writes. FaultInjectionTestFS already has relevant tracking data, and tests can opt out of the new check, as in a smoke test I have left for the old, deprecated functionality `background_close_inactive_wals=true`. Also ran lengthy blackbox_crash_test to ensure the hygiene check is OK with the crash test. (The only place I can find we use LinkFile in production is Checkpoint.) Reviewed By: cbi42 Differential Revision: D58295284 Pulled By: pdillinger fbshipit-source-id: 64d90ed8477e2366c19eaf9c4c5ad60b82cac5c6
2024-06-12 18:48:45 +00:00
#include <cstdio>
#include <functional>
#include <utility>
#include "env/composite_env_wrapper.h"
#include "port/lang.h"
#include "port/stack_trace.h"
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
#include "rocksdb/env.h"
#include "rocksdb/io_status.h"
#include "rocksdb/types.h"
#include "test_util/sync_point.h"
#include "util/coding.h"
#include "util/crc32c.h"
#include "util/mutexlock.h"
#include "util/random.h"
#include "util/string_util.h"
#include "util/xxhash.h"
namespace ROCKSDB_NAMESPACE {
const std::string kNewFileNoOverwrite;
// Assume a filename, and not a directory name like "/foo/bar/"
std::string TestFSGetDirName(const std::string filename) {
size_t found = filename.find_last_of("/\\");
if (found == std::string::npos) {
return "";
} else {
return filename.substr(0, found);
}
}
// Trim the tailing "/" in the end of `str`
std::string TestFSTrimDirname(const std::string& str) {
size_t found = str.find_last_not_of('/');
if (found == std::string::npos) {
return str;
}
return str.substr(0, found + 1);
}
// Return pair <parent directory name, file name> of a full path.
std::pair<std::string, std::string> TestFSGetDirAndName(
const std::string& name) {
std::string dirname = TestFSGetDirName(name);
std::string fname = name.substr(dirname.size() + 1);
return std::make_pair(dirname, fname);
}
// Calculate the checksum of the data with corresponding checksum
// type. If name does not match, no checksum is returned.
void CalculateTypedChecksum(const ChecksumType& checksum_type, const char* data,
size_t size, std::string* checksum) {
if (checksum_type == ChecksumType::kCRC32c) {
uint32_t v_crc32c = crc32c::Extend(0, data, size);
PutFixed32(checksum, v_crc32c);
return;
} else if (checksum_type == ChecksumType::kxxHash) {
uint32_t v = XXH32(data, size, 0);
PutFixed32(checksum, v);
}
}
IOStatus FSFileState::DropUnsyncedData() {
buffer_.resize(0);
return IOStatus::OK();
}
IOStatus FSFileState::DropRandomUnsyncedData(Random* rand) {
int range = static_cast<int>(buffer_.size());
size_t truncated_size = static_cast<size_t>(rand->Uniform(range));
buffer_.resize(truncated_size);
return IOStatus::OK();
}
IOStatus TestFSDirectory::Fsync(const IOOptions& options, IODebugContext* dbg) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, options);
if (!s.ok()) {
return s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
fs_->SyncDir(dirname_);
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = dir_->Fsync(options, dbg);
return s;
}
IOStatus TestFSDirectory::Close(const IOOptions& options, IODebugContext* dbg) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, options);
if (!s.ok()) {
return s;
}
s = dir_->Close(options, dbg);
return s;
}
IOStatus TestFSDirectory::FsyncWithDirOptions(
const IOOptions& options, IODebugContext* dbg,
const DirFsyncOptions& dir_fsync_options) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, options);
if (!s.ok()) {
return s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
fs_->SyncDir(dirname_);
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = dir_->FsyncWithDirOptions(options, dbg, dir_fsync_options);
return s;
}
TestFSWritableFile::TestFSWritableFile(const std::string& fname,
const FileOptions& file_opts,
std::unique_ptr<FSWritableFile>&& f,
FaultInjectionTestFS* fs)
: state_(fname),
file_opts_(file_opts),
target_(std::move(f)),
writable_file_opened_(true),
fs_(fs) {
assert(target_ != nullptr);
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 21:41:14 +00:00
state_.pos_at_last_append_ = 0;
}
TestFSWritableFile::~TestFSWritableFile() {
if (writable_file_opened_) {
Close(IOOptions(), nullptr).PermitUncheckedError();
}
}
IOStatus TestFSWritableFile::Append(const Slice& data, const IOOptions& options,
IODebugContext* dbg) {
MutexLock l(&mutex_);
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s =
fs_->MaybeInjectThreadLocalError(FaultInjectionIOType::kWrite, options);
if (!s.ok()) {
return s;
}
if (target_->use_direct_io()) {
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 21:41:14 +00:00
// TODO(hx235): buffer data for direct IO write to simulate data loss like
// non-direct IO write
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target_->Append(data, options, dbg);
} else {
state_.buffer_.append(data.data(), data.size());
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 21:41:14 +00:00
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (s.ok()) {
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 21:41:14 +00:00
state_.pos_at_last_append_ += data.size();
fs_->WritableFileAppended(state_);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return s;
}
// By setting the IngestDataCorruptionBeforeWrite(), the data corruption is
// simulated.
IOStatus TestFSWritableFile::Append(
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
const Slice& data, const IOOptions& options,
const DataVerificationInfo& verification_info, IODebugContext* dbg) {
MutexLock l(&mutex_);
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
if (fs_->ShouldDataCorruptionBeforeWrite()) {
return IOStatus::Corruption("Data is corrupted!");
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s =
fs_->MaybeInjectThreadLocalError(FaultInjectionIOType::kWrite, options);
if (!s.ok()) {
return s;
}
// Calculate the checksum
std::string checksum;
CalculateTypedChecksum(fs_->GetChecksumHandoffFuncType(), data.data(),
data.size(), &checksum);
if (fs_->GetChecksumHandoffFuncType() != ChecksumType::kNoChecksum &&
checksum != verification_info.checksum.ToString()) {
std::string msg =
"Data is corrupted! Origin data checksum: " +
verification_info.checksum.ToString(true) +
"current data checksum: " + Slice(checksum).ToString(true);
return IOStatus::Corruption(msg);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
if (target_->use_direct_io()) {
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 21:41:14 +00:00
// TODO(hx235): buffer data for direct IO write to simulate data loss like
// non-direct IO write
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target_->Append(data, options, dbg);
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
} else {
state_.buffer_.append(data.data(), data.size());
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 21:41:14 +00:00
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (s.ok()) {
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 21:41:14 +00:00
state_.pos_at_last_append_ += data.size();
fs_->WritableFileAppended(state_);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return s;
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 21:41:14 +00:00
}
IOStatus TestFSWritableFile::Truncate(uint64_t size, const IOOptions& options,
IODebugContext* dbg) {
MutexLock l(&mutex_);
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
IOStatus s =
fs_->MaybeInjectThreadLocalError(FaultInjectionIOType::kWrite, options);
if (!s.ok()) {
return s;
}
s = target_->Truncate(size, options, dbg);
if (s.ok()) {
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 21:41:14 +00:00
state_.pos_at_last_append_ = size;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return s;
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 21:41:14 +00:00
}
IOStatus TestFSWritableFile::PositionedAppend(const Slice& data,
uint64_t offset,
const IOOptions& options,
IODebugContext* dbg) {
MutexLock l(&mutex_);
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
if (fs_->ShouldDataCorruptionBeforeWrite()) {
return IOStatus::Corruption("Data is corrupted!");
}
IOStatus s =
fs_->MaybeInjectThreadLocalError(FaultInjectionIOType::kWrite, options);
if (!s.ok()) {
return s;
}
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 21:41:14 +00:00
// TODO(hx235): buffer data for direct IO write to simulate data loss like
// non-direct IO write
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target_->PositionedAppend(data, offset, options, dbg);
if (s.ok()) {
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 21:41:14 +00:00
state_.pos_at_last_append_ = offset + data.size();
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
fs_->WritableFileAppended(state_);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return s;
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
}
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
IOStatus TestFSWritableFile::PositionedAppend(
const Slice& data, uint64_t offset, const IOOptions& options,
const DataVerificationInfo& verification_info, IODebugContext* dbg) {
MutexLock l(&mutex_);
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
if (fs_->ShouldDataCorruptionBeforeWrite()) {
return IOStatus::Corruption("Data is corrupted!");
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s =
fs_->MaybeInjectThreadLocalError(FaultInjectionIOType::kWrite, options);
if (!s.ok()) {
return s;
}
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
// Calculate the checksum
std::string checksum;
CalculateTypedChecksum(fs_->GetChecksumHandoffFuncType(), data.data(),
data.size(), &checksum);
if (fs_->GetChecksumHandoffFuncType() != ChecksumType::kNoChecksum &&
checksum != verification_info.checksum.ToString()) {
std::string msg =
"Data is corrupted! Origin data checksum: " +
verification_info.checksum.ToString(true) +
"current data checksum: " + Slice(checksum).ToString(true);
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412) Summary: In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead. In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412 Test Plan: make check, add new testing cases. Reviewed By: anand1976 Differential Revision: D29151545 Pulled By: zhichao-cao fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 07:46:33 +00:00
return IOStatus::Corruption(msg);
}
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 21:41:14 +00:00
// TODO(hx235): buffer data for direct IO write to simulate data loss like
// non-direct IO write
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target_->PositionedAppend(data, offset, options, dbg);
if (s.ok()) {
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 21:41:14 +00:00
state_.pos_at_last_append_ = offset + data.size();
fs_->WritableFileAppended(state_);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return s;
}
IOStatus TestFSWritableFile::Close(const IOOptions& options,
IODebugContext* dbg) {
Add missing synchronization in TestFSWritableFile (#10544) Summary: **Context:** ajkr's command revealed an existing TSAN data race between `TestFSWritableFile::Append` and `TestFSWritableFile::Sync` on `TestFSWritableFile::state_` ``` $ make clean && COMPILE_WITH_TSAN=1 make -j56 db_stress $ python3 tools/db_crashtest.py blackbox --simple --duration=3600 --interval=10 --sync_fault_injection=1 --disable_wal=0 --max_key=10000 --checkpoint_one_in=1000 ``` The race is due to concurrent access from [checkpoint's WAL sync](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L324) and [db put's WAL write when ‘sync_fault_injection=1 ‘](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L208) to the `state_` on the same WAL `TestFSWritableFile` under the missing synchronization. ``` WARNING: ThreadSanitizer: data race (pid=11275) Write of size 8 at 0x7b480003d850 by thread T23 (mutexes: write M69230): #0 rocksdb::TestFSWritableFile::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:297 (db_stress+0x716004) https://github.com/facebook/rocksdb/issues/1 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Sync() internal_repo_rocksdb/repo/env/composite_env.cc:154 (db_stress+0x4dfa78) https://github.com/facebook/rocksdb/issues/2 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:280 (db_stress+0x6dfd24) https://github.com/facebook/rocksdb/issues/3 rocksdb::WritableFileWriter::SyncInternal(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:460 (db_stress+0xa1b98c) https://github.com/facebook/rocksdb/issues/4 rocksdb::WritableFileWriter::SyncWithoutFlush(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:435 (db_stress+0xa1e441) https://github.com/facebook/rocksdb/issues/5 rocksdb::DBImpl::SyncWAL() internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1385 (db_stress+0x529458) https://github.com/facebook/rocksdb/issues/6 rocksdb::DBImpl::FlushWAL(bool) internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1339 (db_stress+0x54f82a) https://github.com/facebook/rocksdb/issues/7 rocksdb::DBImpl::GetLiveFilesStorageInfo(rocksdb::LiveFilesStorageInfoOptions const&, std::vector<rocksdb::LiveFileStorageInfo, std::allocator<rocksdb::LiveFileStorageInfo> >*) internal_repo_rocksdb/repo/db/db_filesnapshot.cc:387 (db_stress+0x5c831d) https://github.com/facebook/rocksdb/issues/8 rocksdb::CheckpointImpl::CreateCustomCheckpoint(std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, rocksdb::FileType, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::Temperature)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, unsigned long*, unsigned long, bool) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:214 (db_stress+0x4c0343) https://github.com/facebook/rocksdb/issues/9 rocksdb::CheckpointImpl::CreateCheckpoint(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, unsigned long*) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:123 (db_stress+0x4c237e) https://github.com/facebook/rocksdb/issues/10 rocksdb::StressTest::TestCheckpoint(rocksdb::ThreadState*, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:1699 (db_stress+0x328340) https://github.com/facebook/rocksdb/issues/11 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:825 (db_stress+0x33921f) https://github.com/facebook/rocksdb/issues/12 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/13 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Previous read of size 8 at 0x7b480003d850 by thread T64 (mutexes: write M980798978697532600, write M253744503184415024, write M1262): #0 memcpy <null> (db_stress+0xbc9696) https://github.com/facebook/rocksdb/issues/1 operator= internal_repo_rocksdb/repo/utilities/fault_injection_fs.h:35 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/2 rocksdb::FaultInjectionTestFS::WritableFileAppended(rocksdb::FSFileState const&) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:827 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/3 rocksdb::TestFSWritableFile::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:173 (db_stress+0x7143af) https://github.com/facebook/rocksdb/issues/4 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Append(rocksdb::Slice const&) internal_repo_rocksdb/repo/env/composite_env.cc:115 (db_stress+0x4de3ab) https://github.com/facebook/rocksdb/issues/5 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:248 (db_stress+0x6df44b) https://github.com/facebook/rocksdb/issues/6 rocksdb::WritableFileWriter::WriteBuffered(char const*, unsigned long, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:551 (db_stress+0xa1a953) https://github.com/facebook/rocksdb/issues/7 rocksdb::WritableFileWriter::Flush(rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:327 (db_stress+0xa16ee8) https://github.com/facebook/rocksdb/issues/8 rocksdb::log::Writer::AddRecord(rocksdb::Slice const&, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/db/log_writer.cc:147 (db_stress+0x7f121f) https://github.com/facebook/rocksdb/issues/9 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteBatch const&, rocksdb::log::Writer*, unsigned long*, unsigned long*, rocksdb::Env::IOPriority, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1285 (db_stress+0x695042) https://github.com/facebook/rocksdb/issues/10 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteThread::WriteGroup const&, rocksdb::log::Writer*, unsigned long*, bool, bool, unsigned long, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1328 (db_stress+0x6907e8) https://github.com/facebook/rocksdb/issues/11 rocksdb::DBImpl::PipelinedWriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:731 (db_stress+0x68e8a7) https://github.com/facebook/rocksdb/issues/12 rocksdb::DBImpl::WriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*, unsigned long, rocksdb::PreReleaseCallback*, rocksdb::PostMemTableCallback*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:283 (db_stress+0x688370) https://github.com/facebook/rocksdb/issues/13 rocksdb::DBImpl::Write(rocksdb::WriteOptions const&, rocksdb::WriteBatch*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:126 (db_stress+0x69a7b5) https://github.com/facebook/rocksdb/issues/14 rocksdb::DB::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:2247 (db_stress+0x698634) https://github.com/facebook/rocksdb/issues/15 rocksdb::DBImpl::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:37 (db_stress+0x699868) https://github.com/facebook/rocksdb/issues/16 rocksdb::NonBatchedOpsStressTest::TestPut(rocksdb::ThreadState*, rocksdb::WriteOptions&, rocksdb::ReadOptions const&, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&, char (&) [100], std::unique_ptr<rocksdb::MutexLock, std::default_delete<rocksdb::MutexLock> >&) internal_repo_rocksdb/repo/db_stress_tool/no_batched_ops_stress.cc:681 (db_stress+0x38d20c) https://github.com/facebook/rocksdb/issues/17 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:897 (db_stress+0x3399ec) https://github.com/facebook/rocksdb/issues/18 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/19 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Location is heap block of size 352 at 0x7b480003d800 allocated by thread T23: #0 operator new(unsigned long) <null> (db_stress+0xb685dc) https://github.com/facebook/rocksdb/issues/1 rocksdb::FaultInjectionTestFS::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileOptions const&, std::unique_ptr<rocksdb::FSWritableFile, std::default_delete<rocksdb::FSWritableFile> >*, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:506 (db_stress+0x711192) https://github.com/facebook/rocksdb/issues/2 rocksdb::CompositeEnv::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/env/composite_env.cc:329 (db_stress+0x4d33fa) https://github.com/facebook/rocksdb/issues/3 rocksdb::EnvWrapper::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/include/rocksdb/env.h:1425 (db_stress+0x300662) ... ``` **Summary:** - Added the missing lock in functions mentioned above along with three other functions with a similar need in TestFSWritableFile - Added clarification comment Pull Request resolved: https://github.com/facebook/rocksdb/pull/10544 Test Plan: - Past the above race condition repro Reviewed By: ajkr Differential Revision: D38886634 Pulled By: hx235 fbshipit-source-id: 0571bae9615f35b16fbd8168204607e306b1b486
2022-08-22 22:50:22 +00:00
MutexLock l(&mutex_);
fs_->WritableFileClosed(state_);
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus io_s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, options);
if (!io_s.ok()) {
return io_s;
}
writable_file_opened_ = false;
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
// Drop buffered data that was never synced because close is not a syncing
// mechanism in POSIX file semantics.
state_.buffer_.resize(0);
io_s = target_->Close(options, dbg);
return io_s;
}
IOStatus TestFSWritableFile::Flush(const IOOptions&, IODebugContext*) {
Add missing synchronization in TestFSWritableFile (#10544) Summary: **Context:** ajkr's command revealed an existing TSAN data race between `TestFSWritableFile::Append` and `TestFSWritableFile::Sync` on `TestFSWritableFile::state_` ``` $ make clean && COMPILE_WITH_TSAN=1 make -j56 db_stress $ python3 tools/db_crashtest.py blackbox --simple --duration=3600 --interval=10 --sync_fault_injection=1 --disable_wal=0 --max_key=10000 --checkpoint_one_in=1000 ``` The race is due to concurrent access from [checkpoint's WAL sync](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L324) and [db put's WAL write when ‘sync_fault_injection=1 ‘](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L208) to the `state_` on the same WAL `TestFSWritableFile` under the missing synchronization. ``` WARNING: ThreadSanitizer: data race (pid=11275) Write of size 8 at 0x7b480003d850 by thread T23 (mutexes: write M69230): #0 rocksdb::TestFSWritableFile::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:297 (db_stress+0x716004) https://github.com/facebook/rocksdb/issues/1 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Sync() internal_repo_rocksdb/repo/env/composite_env.cc:154 (db_stress+0x4dfa78) https://github.com/facebook/rocksdb/issues/2 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:280 (db_stress+0x6dfd24) https://github.com/facebook/rocksdb/issues/3 rocksdb::WritableFileWriter::SyncInternal(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:460 (db_stress+0xa1b98c) https://github.com/facebook/rocksdb/issues/4 rocksdb::WritableFileWriter::SyncWithoutFlush(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:435 (db_stress+0xa1e441) https://github.com/facebook/rocksdb/issues/5 rocksdb::DBImpl::SyncWAL() internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1385 (db_stress+0x529458) https://github.com/facebook/rocksdb/issues/6 rocksdb::DBImpl::FlushWAL(bool) internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1339 (db_stress+0x54f82a) https://github.com/facebook/rocksdb/issues/7 rocksdb::DBImpl::GetLiveFilesStorageInfo(rocksdb::LiveFilesStorageInfoOptions const&, std::vector<rocksdb::LiveFileStorageInfo, std::allocator<rocksdb::LiveFileStorageInfo> >*) internal_repo_rocksdb/repo/db/db_filesnapshot.cc:387 (db_stress+0x5c831d) https://github.com/facebook/rocksdb/issues/8 rocksdb::CheckpointImpl::CreateCustomCheckpoint(std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, rocksdb::FileType, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::Temperature)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, unsigned long*, unsigned long, bool) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:214 (db_stress+0x4c0343) https://github.com/facebook/rocksdb/issues/9 rocksdb::CheckpointImpl::CreateCheckpoint(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, unsigned long*) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:123 (db_stress+0x4c237e) https://github.com/facebook/rocksdb/issues/10 rocksdb::StressTest::TestCheckpoint(rocksdb::ThreadState*, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:1699 (db_stress+0x328340) https://github.com/facebook/rocksdb/issues/11 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:825 (db_stress+0x33921f) https://github.com/facebook/rocksdb/issues/12 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/13 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Previous read of size 8 at 0x7b480003d850 by thread T64 (mutexes: write M980798978697532600, write M253744503184415024, write M1262): #0 memcpy <null> (db_stress+0xbc9696) https://github.com/facebook/rocksdb/issues/1 operator= internal_repo_rocksdb/repo/utilities/fault_injection_fs.h:35 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/2 rocksdb::FaultInjectionTestFS::WritableFileAppended(rocksdb::FSFileState const&) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:827 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/3 rocksdb::TestFSWritableFile::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:173 (db_stress+0x7143af) https://github.com/facebook/rocksdb/issues/4 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Append(rocksdb::Slice const&) internal_repo_rocksdb/repo/env/composite_env.cc:115 (db_stress+0x4de3ab) https://github.com/facebook/rocksdb/issues/5 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:248 (db_stress+0x6df44b) https://github.com/facebook/rocksdb/issues/6 rocksdb::WritableFileWriter::WriteBuffered(char const*, unsigned long, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:551 (db_stress+0xa1a953) https://github.com/facebook/rocksdb/issues/7 rocksdb::WritableFileWriter::Flush(rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:327 (db_stress+0xa16ee8) https://github.com/facebook/rocksdb/issues/8 rocksdb::log::Writer::AddRecord(rocksdb::Slice const&, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/db/log_writer.cc:147 (db_stress+0x7f121f) https://github.com/facebook/rocksdb/issues/9 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteBatch const&, rocksdb::log::Writer*, unsigned long*, unsigned long*, rocksdb::Env::IOPriority, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1285 (db_stress+0x695042) https://github.com/facebook/rocksdb/issues/10 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteThread::WriteGroup const&, rocksdb::log::Writer*, unsigned long*, bool, bool, unsigned long, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1328 (db_stress+0x6907e8) https://github.com/facebook/rocksdb/issues/11 rocksdb::DBImpl::PipelinedWriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:731 (db_stress+0x68e8a7) https://github.com/facebook/rocksdb/issues/12 rocksdb::DBImpl::WriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*, unsigned long, rocksdb::PreReleaseCallback*, rocksdb::PostMemTableCallback*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:283 (db_stress+0x688370) https://github.com/facebook/rocksdb/issues/13 rocksdb::DBImpl::Write(rocksdb::WriteOptions const&, rocksdb::WriteBatch*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:126 (db_stress+0x69a7b5) https://github.com/facebook/rocksdb/issues/14 rocksdb::DB::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:2247 (db_stress+0x698634) https://github.com/facebook/rocksdb/issues/15 rocksdb::DBImpl::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:37 (db_stress+0x699868) https://github.com/facebook/rocksdb/issues/16 rocksdb::NonBatchedOpsStressTest::TestPut(rocksdb::ThreadState*, rocksdb::WriteOptions&, rocksdb::ReadOptions const&, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&, char (&) [100], std::unique_ptr<rocksdb::MutexLock, std::default_delete<rocksdb::MutexLock> >&) internal_repo_rocksdb/repo/db_stress_tool/no_batched_ops_stress.cc:681 (db_stress+0x38d20c) https://github.com/facebook/rocksdb/issues/17 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:897 (db_stress+0x3399ec) https://github.com/facebook/rocksdb/issues/18 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/19 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Location is heap block of size 352 at 0x7b480003d800 allocated by thread T23: #0 operator new(unsigned long) <null> (db_stress+0xb685dc) https://github.com/facebook/rocksdb/issues/1 rocksdb::FaultInjectionTestFS::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileOptions const&, std::unique_ptr<rocksdb::FSWritableFile, std::default_delete<rocksdb::FSWritableFile> >*, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:506 (db_stress+0x711192) https://github.com/facebook/rocksdb/issues/2 rocksdb::CompositeEnv::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/env/composite_env.cc:329 (db_stress+0x4d33fa) https://github.com/facebook/rocksdb/issues/3 rocksdb::EnvWrapper::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/include/rocksdb/env.h:1425 (db_stress+0x300662) ... ``` **Summary:** - Added the missing lock in functions mentioned above along with three other functions with a similar need in TestFSWritableFile - Added clarification comment Pull Request resolved: https://github.com/facebook/rocksdb/pull/10544 Test Plan: - Past the above race condition repro Reviewed By: ajkr Differential Revision: D38886634 Pulled By: hx235 fbshipit-source-id: 0571bae9615f35b16fbd8168204607e306b1b486
2022-08-22 22:50:22 +00:00
MutexLock l(&mutex_);
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
return IOStatus::OK();
}
IOStatus TestFSWritableFile::Sync(const IOOptions& options,
IODebugContext* dbg) {
Add missing synchronization in TestFSWritableFile (#10544) Summary: **Context:** ajkr's command revealed an existing TSAN data race between `TestFSWritableFile::Append` and `TestFSWritableFile::Sync` on `TestFSWritableFile::state_` ``` $ make clean && COMPILE_WITH_TSAN=1 make -j56 db_stress $ python3 tools/db_crashtest.py blackbox --simple --duration=3600 --interval=10 --sync_fault_injection=1 --disable_wal=0 --max_key=10000 --checkpoint_one_in=1000 ``` The race is due to concurrent access from [checkpoint's WAL sync](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L324) and [db put's WAL write when ‘sync_fault_injection=1 ‘](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L208) to the `state_` on the same WAL `TestFSWritableFile` under the missing synchronization. ``` WARNING: ThreadSanitizer: data race (pid=11275) Write of size 8 at 0x7b480003d850 by thread T23 (mutexes: write M69230): #0 rocksdb::TestFSWritableFile::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:297 (db_stress+0x716004) https://github.com/facebook/rocksdb/issues/1 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Sync() internal_repo_rocksdb/repo/env/composite_env.cc:154 (db_stress+0x4dfa78) https://github.com/facebook/rocksdb/issues/2 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:280 (db_stress+0x6dfd24) https://github.com/facebook/rocksdb/issues/3 rocksdb::WritableFileWriter::SyncInternal(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:460 (db_stress+0xa1b98c) https://github.com/facebook/rocksdb/issues/4 rocksdb::WritableFileWriter::SyncWithoutFlush(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:435 (db_stress+0xa1e441) https://github.com/facebook/rocksdb/issues/5 rocksdb::DBImpl::SyncWAL() internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1385 (db_stress+0x529458) https://github.com/facebook/rocksdb/issues/6 rocksdb::DBImpl::FlushWAL(bool) internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1339 (db_stress+0x54f82a) https://github.com/facebook/rocksdb/issues/7 rocksdb::DBImpl::GetLiveFilesStorageInfo(rocksdb::LiveFilesStorageInfoOptions const&, std::vector<rocksdb::LiveFileStorageInfo, std::allocator<rocksdb::LiveFileStorageInfo> >*) internal_repo_rocksdb/repo/db/db_filesnapshot.cc:387 (db_stress+0x5c831d) https://github.com/facebook/rocksdb/issues/8 rocksdb::CheckpointImpl::CreateCustomCheckpoint(std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, rocksdb::FileType, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::Temperature)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, unsigned long*, unsigned long, bool) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:214 (db_stress+0x4c0343) https://github.com/facebook/rocksdb/issues/9 rocksdb::CheckpointImpl::CreateCheckpoint(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, unsigned long*) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:123 (db_stress+0x4c237e) https://github.com/facebook/rocksdb/issues/10 rocksdb::StressTest::TestCheckpoint(rocksdb::ThreadState*, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:1699 (db_stress+0x328340) https://github.com/facebook/rocksdb/issues/11 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:825 (db_stress+0x33921f) https://github.com/facebook/rocksdb/issues/12 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/13 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Previous read of size 8 at 0x7b480003d850 by thread T64 (mutexes: write M980798978697532600, write M253744503184415024, write M1262): #0 memcpy <null> (db_stress+0xbc9696) https://github.com/facebook/rocksdb/issues/1 operator= internal_repo_rocksdb/repo/utilities/fault_injection_fs.h:35 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/2 rocksdb::FaultInjectionTestFS::WritableFileAppended(rocksdb::FSFileState const&) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:827 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/3 rocksdb::TestFSWritableFile::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:173 (db_stress+0x7143af) https://github.com/facebook/rocksdb/issues/4 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Append(rocksdb::Slice const&) internal_repo_rocksdb/repo/env/composite_env.cc:115 (db_stress+0x4de3ab) https://github.com/facebook/rocksdb/issues/5 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:248 (db_stress+0x6df44b) https://github.com/facebook/rocksdb/issues/6 rocksdb::WritableFileWriter::WriteBuffered(char const*, unsigned long, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:551 (db_stress+0xa1a953) https://github.com/facebook/rocksdb/issues/7 rocksdb::WritableFileWriter::Flush(rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:327 (db_stress+0xa16ee8) https://github.com/facebook/rocksdb/issues/8 rocksdb::log::Writer::AddRecord(rocksdb::Slice const&, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/db/log_writer.cc:147 (db_stress+0x7f121f) https://github.com/facebook/rocksdb/issues/9 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteBatch const&, rocksdb::log::Writer*, unsigned long*, unsigned long*, rocksdb::Env::IOPriority, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1285 (db_stress+0x695042) https://github.com/facebook/rocksdb/issues/10 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteThread::WriteGroup const&, rocksdb::log::Writer*, unsigned long*, bool, bool, unsigned long, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1328 (db_stress+0x6907e8) https://github.com/facebook/rocksdb/issues/11 rocksdb::DBImpl::PipelinedWriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:731 (db_stress+0x68e8a7) https://github.com/facebook/rocksdb/issues/12 rocksdb::DBImpl::WriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*, unsigned long, rocksdb::PreReleaseCallback*, rocksdb::PostMemTableCallback*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:283 (db_stress+0x688370) https://github.com/facebook/rocksdb/issues/13 rocksdb::DBImpl::Write(rocksdb::WriteOptions const&, rocksdb::WriteBatch*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:126 (db_stress+0x69a7b5) https://github.com/facebook/rocksdb/issues/14 rocksdb::DB::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:2247 (db_stress+0x698634) https://github.com/facebook/rocksdb/issues/15 rocksdb::DBImpl::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:37 (db_stress+0x699868) https://github.com/facebook/rocksdb/issues/16 rocksdb::NonBatchedOpsStressTest::TestPut(rocksdb::ThreadState*, rocksdb::WriteOptions&, rocksdb::ReadOptions const&, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&, char (&) [100], std::unique_ptr<rocksdb::MutexLock, std::default_delete<rocksdb::MutexLock> >&) internal_repo_rocksdb/repo/db_stress_tool/no_batched_ops_stress.cc:681 (db_stress+0x38d20c) https://github.com/facebook/rocksdb/issues/17 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:897 (db_stress+0x3399ec) https://github.com/facebook/rocksdb/issues/18 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/19 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Location is heap block of size 352 at 0x7b480003d800 allocated by thread T23: #0 operator new(unsigned long) <null> (db_stress+0xb685dc) https://github.com/facebook/rocksdb/issues/1 rocksdb::FaultInjectionTestFS::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileOptions const&, std::unique_ptr<rocksdb::FSWritableFile, std::default_delete<rocksdb::FSWritableFile> >*, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:506 (db_stress+0x711192) https://github.com/facebook/rocksdb/issues/2 rocksdb::CompositeEnv::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/env/composite_env.cc:329 (db_stress+0x4d33fa) https://github.com/facebook/rocksdb/issues/3 rocksdb::EnvWrapper::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/include/rocksdb/env.h:1425 (db_stress+0x300662) ... ``` **Summary:** - Added the missing lock in functions mentioned above along with three other functions with a similar need in TestFSWritableFile - Added clarification comment Pull Request resolved: https://github.com/facebook/rocksdb/pull/10544 Test Plan: - Past the above race condition repro Reviewed By: ajkr Differential Revision: D38886634 Pulled By: hx235 fbshipit-source-id: 0571bae9615f35b16fbd8168204607e306b1b486
2022-08-22 22:50:22 +00:00
MutexLock l(&mutex_);
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
if (target_->use_direct_io()) {
// For Direct IO mode, we don't buffer anything in TestFSWritableFile.
// So just return
return IOStatus::OK();
}
IOStatus io_s = target_->Append(state_.buffer_, options, dbg);
state_.buffer_.resize(0);
// Ignore sync errors
target_->Sync(options, dbg).PermitUncheckedError();
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 21:41:14 +00:00
state_.pos_at_last_sync_ = state_.pos_at_last_append_;
fs_->WritableFileSynced(state_);
return io_s;
}
IOStatus TestFSWritableFile::RangeSync(uint64_t offset, uint64_t nbytes,
const IOOptions& options,
IODebugContext* dbg) {
Add missing synchronization in TestFSWritableFile (#10544) Summary: **Context:** ajkr's command revealed an existing TSAN data race between `TestFSWritableFile::Append` and `TestFSWritableFile::Sync` on `TestFSWritableFile::state_` ``` $ make clean && COMPILE_WITH_TSAN=1 make -j56 db_stress $ python3 tools/db_crashtest.py blackbox --simple --duration=3600 --interval=10 --sync_fault_injection=1 --disable_wal=0 --max_key=10000 --checkpoint_one_in=1000 ``` The race is due to concurrent access from [checkpoint's WAL sync](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L324) and [db put's WAL write when ‘sync_fault_injection=1 ‘](https://github.com/facebook/rocksdb/blob/7.4.fb/utilities/fault_injection_fs.cc#L208) to the `state_` on the same WAL `TestFSWritableFile` under the missing synchronization. ``` WARNING: ThreadSanitizer: data race (pid=11275) Write of size 8 at 0x7b480003d850 by thread T23 (mutexes: write M69230): #0 rocksdb::TestFSWritableFile::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:297 (db_stress+0x716004) https://github.com/facebook/rocksdb/issues/1 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Sync() internal_repo_rocksdb/repo/env/composite_env.cc:154 (db_stress+0x4dfa78) https://github.com/facebook/rocksdb/issues/2 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Sync(rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:280 (db_stress+0x6dfd24) https://github.com/facebook/rocksdb/issues/3 rocksdb::WritableFileWriter::SyncInternal(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:460 (db_stress+0xa1b98c) https://github.com/facebook/rocksdb/issues/4 rocksdb::WritableFileWriter::SyncWithoutFlush(bool) internal_repo_rocksdb/repo/file/writable_file_writer.cc:435 (db_stress+0xa1e441) https://github.com/facebook/rocksdb/issues/5 rocksdb::DBImpl::SyncWAL() internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1385 (db_stress+0x529458) https://github.com/facebook/rocksdb/issues/6 rocksdb::DBImpl::FlushWAL(bool) internal_repo_rocksdb/repo/db/db_impl/db_impl.cc:1339 (db_stress+0x54f82a) https://github.com/facebook/rocksdb/issues/7 rocksdb::DBImpl::GetLiveFilesStorageInfo(rocksdb::LiveFilesStorageInfoOptions const&, std::vector<rocksdb::LiveFileStorageInfo, std::allocator<rocksdb::LiveFileStorageInfo> >*) internal_repo_rocksdb/repo/db/db_filesnapshot.cc:387 (db_stress+0x5c831d) https://github.com/facebook/rocksdb/issues/8 rocksdb::CheckpointImpl::CreateCustomCheckpoint(std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, rocksdb::FileType, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::Temperature)>, std::function<rocksdb::Status (std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileType)>, unsigned long*, unsigned long, bool) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:214 (db_stress+0x4c0343) https://github.com/facebook/rocksdb/issues/9 rocksdb::CheckpointImpl::CreateCheckpoint(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, unsigned long, unsigned long*) internal_repo_rocksdb/repo/utilities/checkpoint/checkpoint_impl.cc:123 (db_stress+0x4c237e) https://github.com/facebook/rocksdb/issues/10 rocksdb::StressTest::TestCheckpoint(rocksdb::ThreadState*, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:1699 (db_stress+0x328340) https://github.com/facebook/rocksdb/issues/11 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:825 (db_stress+0x33921f) https://github.com/facebook/rocksdb/issues/12 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/13 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Previous read of size 8 at 0x7b480003d850 by thread T64 (mutexes: write M980798978697532600, write M253744503184415024, write M1262): #0 memcpy <null> (db_stress+0xbc9696) https://github.com/facebook/rocksdb/issues/1 operator= internal_repo_rocksdb/repo/utilities/fault_injection_fs.h:35 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/2 rocksdb::FaultInjectionTestFS::WritableFileAppended(rocksdb::FSFileState const&) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:827 (db_stress+0x70d5f1) https://github.com/facebook/rocksdb/issues/3 rocksdb::TestFSWritableFile::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:173 (db_stress+0x7143af) https://github.com/facebook/rocksdb/issues/4 rocksdb::(anonymous namespace)::CompositeWritableFileWrapper::Append(rocksdb::Slice const&) internal_repo_rocksdb/repo/env/composite_env.cc:115 (db_stress+0x4de3ab) https://github.com/facebook/rocksdb/issues/5 rocksdb::(anonymous namespace)::LegacyWritableFileWrapper::Append(rocksdb::Slice const&, rocksdb::IOOptions const&, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/env/env.cc:248 (db_stress+0x6df44b) https://github.com/facebook/rocksdb/issues/6 rocksdb::WritableFileWriter::WriteBuffered(char const*, unsigned long, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:551 (db_stress+0xa1a953) https://github.com/facebook/rocksdb/issues/7 rocksdb::WritableFileWriter::Flush(rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/file/writable_file_writer.cc:327 (db_stress+0xa16ee8) https://github.com/facebook/rocksdb/issues/8 rocksdb::log::Writer::AddRecord(rocksdb::Slice const&, rocksdb::Env::IOPriority) internal_repo_rocksdb/repo/db/log_writer.cc:147 (db_stress+0x7f121f) https://github.com/facebook/rocksdb/issues/9 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteBatch const&, rocksdb::log::Writer*, unsigned long*, unsigned long*, rocksdb::Env::IOPriority, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1285 (db_stress+0x695042) https://github.com/facebook/rocksdb/issues/10 rocksdb::DBImpl::WriteToWAL(rocksdb::WriteThread::WriteGroup const&, rocksdb::log::Writer*, unsigned long*, bool, bool, unsigned long, rocksdb::DBImpl::LogFileNumberSize&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:1328 (db_stress+0x6907e8) https://github.com/facebook/rocksdb/issues/11 rocksdb::DBImpl::PipelinedWriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:731 (db_stress+0x68e8a7) https://github.com/facebook/rocksdb/issues/12 rocksdb::DBImpl::WriteImpl(rocksdb::WriteOptions const&, rocksdb::WriteBatch*, rocksdb::WriteCallback*, unsigned long*, unsigned long, bool, unsigned long*, unsigned long, rocksdb::PreReleaseCallback*, rocksdb::PostMemTableCallback*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:283 (db_stress+0x688370) https://github.com/facebook/rocksdb/issues/13 rocksdb::DBImpl::Write(rocksdb::WriteOptions const&, rocksdb::WriteBatch*) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:126 (db_stress+0x69a7b5) https://github.com/facebook/rocksdb/issues/14 rocksdb::DB::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:2247 (db_stress+0x698634) https://github.com/facebook/rocksdb/issues/15 rocksdb::DBImpl::Put(rocksdb::WriteOptions const&, rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, rocksdb::Slice const&, rocksdb::Slice const&) internal_repo_rocksdb/repo/db/db_impl/db_impl_write.cc:37 (db_stress+0x699868) https://github.com/facebook/rocksdb/issues/16 rocksdb::NonBatchedOpsStressTest::TestPut(rocksdb::ThreadState*, rocksdb::WriteOptions&, rocksdb::ReadOptions const&, std::vector<int, std::allocator<int> > const&, std::vector<long, std::allocator<long> > const&, char (&) [100], std::unique_ptr<rocksdb::MutexLock, std::default_delete<rocksdb::MutexLock> >&) internal_repo_rocksdb/repo/db_stress_tool/no_batched_ops_stress.cc:681 (db_stress+0x38d20c) https://github.com/facebook/rocksdb/issues/17 rocksdb::StressTest::OperateDb(rocksdb::ThreadState*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_test_base.cc:897 (db_stress+0x3399ec) https://github.com/facebook/rocksdb/issues/18 rocksdb::ThreadBody(void*) internal_repo_rocksdb/repo/db_stress_tool/db_stress_driver.cc:33 (db_stress+0x354857) https://github.com/facebook/rocksdb/issues/19 rocksdb::(anonymous namespace)::StartThreadWrapper(void*) internal_repo_rocksdb/repo/env/env_posix.cc:447 (db_stress+0x6eb2ad) Location is heap block of size 352 at 0x7b480003d800 allocated by thread T23: #0 operator new(unsigned long) <null> (db_stress+0xb685dc) https://github.com/facebook/rocksdb/issues/1 rocksdb::FaultInjectionTestFS::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, rocksdb::FileOptions const&, std::unique_ptr<rocksdb::FSWritableFile, std::default_delete<rocksdb::FSWritableFile> >*, rocksdb::IODebugContext*) internal_repo_rocksdb/repo/utilities/fault_injection_fs.cc:506 (db_stress+0x711192) https://github.com/facebook/rocksdb/issues/2 rocksdb::CompositeEnv::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/env/composite_env.cc:329 (db_stress+0x4d33fa) https://github.com/facebook/rocksdb/issues/3 rocksdb::EnvWrapper::NewWritableFile(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::unique_ptr<rocksdb::WritableFile, std::default_delete<rocksdb::WritableFile> >*, rocksdb::EnvOptions const&) internal_repo_rocksdb/repo/include/rocksdb/env.h:1425 (db_stress+0x300662) ... ``` **Summary:** - Added the missing lock in functions mentioned above along with three other functions with a similar need in TestFSWritableFile - Added clarification comment Pull Request resolved: https://github.com/facebook/rocksdb/pull/10544 Test Plan: - Past the above race condition repro Reviewed By: ajkr Differential Revision: D38886634 Pulled By: hx235 fbshipit-source-id: 0571bae9615f35b16fbd8168204607e306b1b486
2022-08-22 22:50:22 +00:00
MutexLock l(&mutex_);
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
// Assumes caller passes consecutive byte ranges.
uint64_t sync_limit = offset + nbytes;
uint64_t buf_begin =
state_.pos_at_last_sync_ < 0 ? 0 : state_.pos_at_last_sync_;
IOStatus io_s;
if (sync_limit < buf_begin) {
return io_s;
}
uint64_t num_to_sync = std::min(static_cast<uint64_t>(state_.buffer_.size()),
sync_limit - buf_begin);
Slice buf_to_sync(state_.buffer_.data(), num_to_sync);
io_s = target_->Append(buf_to_sync, options, dbg);
state_.buffer_ = state_.buffer_.substr(num_to_sync);
// Ignore sync errors
target_->RangeSync(offset, nbytes, options, dbg).PermitUncheckedError();
state_.pos_at_last_sync_ = offset + num_to_sync;
fs_->WritableFileSynced(state_);
return io_s;
}
TestFSRandomRWFile::TestFSRandomRWFile(const std::string& /*fname*/,
std::unique_ptr<FSRandomRWFile>&& f,
FaultInjectionTestFS* fs)
: target_(std::move(f)), file_opened_(true), fs_(fs) {
assert(target_ != nullptr);
}
TestFSRandomRWFile::~TestFSRandomRWFile() {
if (file_opened_) {
Close(IOOptions(), nullptr).PermitUncheckedError();
}
}
IOStatus TestFSRandomRWFile::Write(uint64_t offset, const Slice& data,
const IOOptions& options,
IODebugContext* dbg) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
return target_->Write(offset, data, options, dbg);
}
IOStatus TestFSRandomRWFile::Read(uint64_t offset, size_t n,
const IOOptions& options, Slice* result,
char* scratch, IODebugContext* dbg) const {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
// TODO (low priority): fs_->ReadUnsyncedData()
return target_->Read(offset, n, options, result, scratch, dbg);
}
IOStatus TestFSRandomRWFile::Close(const IOOptions& options,
IODebugContext* dbg) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
file_opened_ = false;
return target_->Close(options, dbg);
}
IOStatus TestFSRandomRWFile::Flush(const IOOptions& options,
IODebugContext* dbg) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
return target_->Flush(options, dbg);
}
IOStatus TestFSRandomRWFile::Sync(const IOOptions& options,
IODebugContext* dbg) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
return target_->Sync(options, dbg);
}
TestFSRandomAccessFile::TestFSRandomAccessFile(
const std::string& /*fname*/, std::unique_ptr<FSRandomAccessFile>&& f,
FaultInjectionTestFS* fs)
: target_(std::move(f)), fs_(fs) {
assert(target_ != nullptr);
}
IOStatus TestFSRandomAccessFile::Read(uint64_t offset, size_t n,
const IOOptions& options, Slice* result,
char* scratch,
IODebugContext* dbg) const {
Fix potential incorrect result for duplicate key in MultiGet (#12295) Summary: The RocksDB correctness testing has recently discovered a possible, but very unlikely, correctness issue with MultiGet. The issue happens when all of the below conditions are met - 1. Duplicate keys in a MultiGet batch 2. Key matches the last key in a non-zero, non-bottommost level file 3. Final value is not in the file (merge operand, not snapshot visible etc) 4. Multiple entries exist for the key in the file spanning more than 1 data block. This can happen due to snapshots, which would force multiple versions of the key in the file, and they may spill over to another data block 5. Lookup attempt in the SST for the first of the duplicates fails with IO error on a data block (NOT the first data block, but the second or subsequent uncached block), but no errors for the other duplicates 6. Value or merge operand for the key is present in the very next level The problem is, in FilePickerMultiGet, when looking up keys in a level we use FileIndexer and the overlapping file in the current level to determine the search bounds for that key in the file list in the next level. If the next level is empty, the search bounds are reset and we do a full binary search in the next non-empty level's LevelFilesBrief. However, under the conditions https://github.com/facebook/rocksdb/issues/1 and https://github.com/facebook/rocksdb/issues/2 listed above, only the first of the duplicates has its next-level search bounds updated, and the remaining duplicates are skipped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12295 Test Plan: Add unit tests that fail an assertion or return wrong result without the fix Reviewed By: hx235 Differential Revision: D53187634 Pulled By: anand1976 fbshipit-source-id: a5eadf4fede9bbdec784cd993b15e3341436d1ea
2024-02-02 19:48:35 +00:00
TEST_SYNC_POINT("FaultInjectionTestFS::RandomRead");
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kRead, options,
FaultInjectionTestFS::ErrorOperation::kRead, result, use_direct_io(),
scratch, /*need_count_increase=*/true,
/*fault_injected=*/nullptr);
if (!s.ok()) {
return s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target_->Read(offset, n, options, result, scratch, dbg);
// TODO (low priority): fs_->ReadUnsyncedData()
return s;
}
IOStatus TestFSRandomAccessFile::ReadAsync(
FSReadRequest& req, const IOOptions& opts,
std::function<void(FSReadRequest&, void*)> cb, void* cb_arg,
void** io_handle, IOHandleDeleter* del_fn, IODebugContext* /*dbg*/) {
IOStatus res_status;
FSReadRequest res;
IOStatus s;
if (!fs_->IsFilesystemActive()) {
res_status = fs_->GetError();
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
}
if (res_status.ok()) {
res_status = fs_->MaybeInjectThreadLocalError(
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
FaultInjectionIOType::kRead, opts,
FaultInjectionTestFS::ErrorOperation::kRead, &res.result,
use_direct_io(), req.scratch, /*need_count_increase=*/true,
/*fault_injected=*/nullptr);
}
if (res_status.ok()) {
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target_->ReadAsync(req, opts, cb, cb_arg, io_handle, del_fn, nullptr);
// TODO (low priority): fs_->ReadUnsyncedData()
} else {
// If theres no injected error, then cb will be called asynchronously when
// target_ actually finishes the read. But if theres an injected error, it
// needs to immediately call cb(res, cb_arg) s since target_->ReadAsync()
// isnt invoked at all.
res.status = res_status;
cb(res, cb_arg);
}
// We return ReadAsync()'s status intead of injected error status here since
// the return status is not supposed to be the status of the actual IO (i.e,
// the actual async read). The actual status of the IO will be passed to cb()
// callback upon the actual read finishes or like above when injected error
// happens.
return s;
}
IOStatus TestFSRandomAccessFile::MultiRead(FSReadRequest* reqs, size_t num_reqs,
const IOOptions& options,
IODebugContext* dbg) {
if (!fs_->IsFilesystemActive()) {
return fs_->GetError();
}
IOStatus s = target_->MultiRead(reqs, num_reqs, options, dbg);
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
// TODO (low priority): fs_->ReadUnsyncedData()
bool injected_error = false;
for (size_t i = 0; i < num_reqs; i++) {
if (!reqs[i].status.ok()) {
// Already seeing an error.
break;
}
bool this_injected_error;
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
reqs[i].status = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kRead, options,
FaultInjectionTestFS::ErrorOperation::kRead, &(reqs[i].result),
use_direct_io(), reqs[i].scratch,
/*need_count_increase=*/true,
/*fault_injected=*/&this_injected_error);
injected_error |= this_injected_error;
}
if (s.ok()) {
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kRead, options,
FaultInjectionTestFS::ErrorOperation::kMultiRead, nullptr,
use_direct_io(), nullptr, /*need_count_increase=*/!injected_error,
/*fault_injected=*/nullptr);
}
return s;
}
size_t TestFSRandomAccessFile::GetUniqueId(char* id, size_t max_size) const {
if (fs_->ShouldFailGetUniqueId()) {
return 0;
} else {
return target_->GetUniqueId(id, max_size);
}
}
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
void FaultInjectionTestFS::AddUnsyncedToRead(const std::string& fname,
size_t pos, size_t n,
Slice* result, char* scratch) {
// Should be checked prior
assert(result->size() < n);
size_t pos_after = pos + result->size();
MutexLock l(&mutex_);
auto it = db_file_state_.find(fname);
if (it != db_file_state_.end()) {
auto& st = it->second;
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 21:41:14 +00:00
if (st.pos_at_last_append_ > static_cast<ssize_t>(pos_after)) {
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
size_t remaining_requested = n - result->size();
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 21:41:14 +00:00
size_t to_copy =
std::min(remaining_requested,
static_cast<size_t>(st.pos_at_last_append_) - pos_after);
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
size_t buffer_offset = pos_after - static_cast<size_t>(std::max(
st.pos_at_last_sync_, ssize_t{0}));
// Data might have been dropped from buffer
if (st.buffer_.size() > buffer_offset) {
to_copy = std::min(to_copy, st.buffer_.size() - buffer_offset);
if (result->data() != scratch) {
// TODO: this will be needed when supporting random reads
// but not currently used
abort();
// NOTE: might overlap
// std::copy_n(result->data(), result->size(), scratch);
}
std::copy_n(st.buffer_.data() + buffer_offset, to_copy,
scratch + result->size());
*result = Slice(scratch, result->size() + to_copy);
}
}
}
}
IOStatus TestFSSequentialFile::Read(size_t n, const IOOptions& options,
Slice* result, char* scratch,
IODebugContext* dbg) {
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kRead, options,
FaultInjectionTestFS::ErrorOperation::kRead, result, use_direct_io(),
scratch, true /*need_count_increase=*/, nullptr /* fault_injected*/);
if (!s.ok()) {
return s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target()->Read(n, options, result, scratch, dbg);
if (!s.ok()) {
return s;
}
if (fs_->ReadUnsyncedData() && result->size() < n) {
fs_->AddUnsyncedToRead(fname_, read_pos_, n, result, scratch);
}
read_pos_ += result->size();
return s;
}
IOStatus TestFSSequentialFile::PositionedRead(uint64_t offset, size_t n,
const IOOptions& options,
Slice* result, char* scratch,
IODebugContext* dbg) {
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus s = fs_->MaybeInjectThreadLocalError(
FaultInjectionIOType::kRead, options,
FaultInjectionTestFS::ErrorOperation::kRead, result, use_direct_io(),
scratch, true /*need_count_increase=*/, nullptr /* fault_injected */);
if (!s.ok()) {
return s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
s = target()->PositionedRead(offset, n, options, result, scratch, dbg);
// TODO (low priority): fs_->ReadUnsyncedData()
return s;
}
IOStatus FaultInjectionTestFS::NewDirectory(
const std::string& name, const IOOptions& options,
std::unique_ptr<FSDirectory>* result, IODebugContext* dbg) {
std::unique_ptr<FSDirectory> r;
IOStatus io_s = target()->NewDirectory(name, options, &r, dbg);
if (!io_s.ok()) {
return io_s;
}
result->reset(
new TestFSDirectory(this, TestFSTrimDirname(name), r.release()));
return IOStatus::OK();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus FaultInjectionTestFS::FileExists(const std::string& fname,
const IOOptions& options,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->FileExists(fname, options, dbg);
return io_s;
}
IOStatus FaultInjectionTestFS::GetChildren(const std::string& dir,
const IOOptions& options,
std::vector<std::string>* result,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->GetChildren(dir, options, result, dbg);
return io_s;
}
IOStatus FaultInjectionTestFS::GetChildrenFileAttributes(
const std::string& dir, const IOOptions& options,
std::vector<FileAttributes>* result, IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->GetChildrenFileAttributes(dir, options, result, dbg);
return io_s;
}
IOStatus FaultInjectionTestFS::NewWritableFile(
const std::string& fname, const FileOptions& file_opts,
std::unique_ptr<FSWritableFile>* result, IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
if (ShouldUseDiretWritable(fname)) {
return target()->NewWritableFile(fname, file_opts, result, dbg);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus io_s = MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, file_opts.io_options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->NewWritableFile(fname, file_opts, result, dbg);
if (io_s.ok()) {
result->reset(
new TestFSWritableFile(fname, file_opts, std::move(*result), this));
// WritableFileWriter* file is opened
// again then it will be truncated - so forget our saved state.
UntrackFile(fname);
{
MutexLock l(&mutex_);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
open_managed_files_.insert(fname);
auto dir_and_name = TestFSGetDirAndName(fname);
auto& list = dir_to_new_files_since_last_sync_[dir_and_name.first];
// The new file could overwrite an old one. Here we simplify
// the implementation by assuming no file of this name after
// dropping unsynced files.
list[dir_and_name.second] = kNewFileNoOverwrite;
}
}
return io_s;
}
IOStatus FaultInjectionTestFS::ReopenWritableFile(
const std::string& fname, const FileOptions& file_opts,
std::unique_ptr<FSWritableFile>* result, IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
if (ShouldUseDiretWritable(fname)) {
return target()->ReopenWritableFile(fname, file_opts, result, dbg);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus io_s = MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, file_opts.io_options);
if (!io_s.ok()) {
return io_s;
}
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
bool exists;
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus exists_s =
target()->FileExists(fname, IOOptions(), nullptr /* dbg */);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
if (exists_s.IsNotFound()) {
exists = false;
} else if (exists_s.ok()) {
exists = true;
} else {
io_s = exists_s;
exists = false;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (!io_s.ok()) {
return io_s;
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
io_s = target()->ReopenWritableFile(fname, file_opts, result, dbg);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
// Only track files we created. Files created outside of this
// `FaultInjectionTestFS` are not eligible for tracking/data dropping
// (for example, they may contain data a previous db_stress run expects to
// be recovered). This could be extended to track/drop data appended once
// the file is under `FaultInjectionTestFS`'s control.
if (io_s.ok()) {
bool should_track;
{
MutexLock l(&mutex_);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
if (db_file_state_.find(fname) != db_file_state_.end()) {
// It was written by this `FileSystem` earlier.
assert(exists);
should_track = true;
} else if (!exists) {
// It was created by this `FileSystem` just now.
should_track = true;
open_managed_files_.insert(fname);
auto dir_and_name = TestFSGetDirAndName(fname);
auto& list = dir_to_new_files_since_last_sync_[dir_and_name.first];
list[dir_and_name.second] = kNewFileNoOverwrite;
} else {
should_track = false;
}
}
if (should_track) {
result->reset(
new TestFSWritableFile(fname, file_opts, std::move(*result), this));
}
}
return io_s;
}
IOStatus FaultInjectionTestFS::ReuseWritableFile(
const std::string& fname, const std::string& old_fname,
const FileOptions& file_opts, std::unique_ptr<FSWritableFile>* result,
IODebugContext* dbg) {
IOStatus s = RenameFile(old_fname, fname, file_opts.io_options, dbg);
if (!s.ok()) {
return s;
}
return NewWritableFile(fname, file_opts, result, dbg);
}
IOStatus FaultInjectionTestFS::NewRandomRWFile(
const std::string& fname, const FileOptions& file_opts,
std::unique_ptr<FSRandomRWFile>* result, IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
if (ShouldUseDiretWritable(fname)) {
return target()->NewRandomRWFile(fname, file_opts, result, dbg);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus io_s = MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, file_opts.io_options);
if (!io_s.ok()) {
return io_s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
io_s = target()->NewRandomRWFile(fname, file_opts, result, dbg);
if (io_s.ok()) {
result->reset(new TestFSRandomRWFile(fname, std::move(*result), this));
// WritableFileWriter* file is opened
// again then it will be truncated - so forget our saved state.
UntrackFile(fname);
{
MutexLock l(&mutex_);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
open_managed_files_.insert(fname);
auto dir_and_name = TestFSGetDirAndName(fname);
auto& list = dir_to_new_files_since_last_sync_[dir_and_name.first];
// It could be overwriting an old file, but we simplify the
// implementation by ignoring it.
list[dir_and_name.second] = kNewFileNoOverwrite;
}
}
return io_s;
}
IOStatus FaultInjectionTestFS::NewRandomAccessFile(
const std::string& fname, const FileOptions& file_opts,
std::unique_ptr<FSRandomAccessFile>* result, IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s = MaybeInjectThreadLocalError(FaultInjectionIOType::kRead,
file_opts.io_options);
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (!io_s.ok()) {
return io_s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
io_s = target()->NewRandomAccessFile(fname, file_opts, result, dbg);
if (io_s.ok()) {
result->reset(new TestFSRandomAccessFile(fname, std::move(*result), this));
}
return io_s;
}
IOStatus FaultInjectionTestFS::NewSequentialFile(
const std::string& fname, const FileOptions& file_opts,
std::unique_ptr<FSSequentialFile>* result, IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s = MaybeInjectThreadLocalError(FaultInjectionIOType::kRead,
file_opts.io_options);
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (!io_s.ok()) {
return io_s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
io_s = target()->NewSequentialFile(fname, file_opts, result, dbg);
if (io_s.ok()) {
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
result->reset(new TestFSSequentialFile(std::move(*result), this, fname));
}
return io_s;
}
IOStatus FaultInjectionTestFS::DeleteFile(const std::string& f,
const IOOptions& options,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus io_s = MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, options);
if (!io_s.ok()) {
return io_s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
io_s = FileSystemWrapper::DeleteFile(f, options, dbg);
if (io_s.ok()) {
UntrackFile(f);
}
return io_s;
}
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
IOStatus FaultInjectionTestFS::GetFileSize(const std::string& f,
const IOOptions& options,
uint64_t* file_size,
IODebugContext* dbg) {
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->GetFileSize(f, options, file_size, dbg);
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
if (!io_s.ok()) {
return io_s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
if (ReadUnsyncedData()) {
// Need to report flushed size, not synced size
MutexLock l(&mutex_);
auto it = db_file_state_.find(f);
if (it != db_file_state_.end()) {
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 21:41:14 +00:00
*file_size = it->second.pos_at_last_append_;
FaultInjectionTestFS read unsynced data by default (#12729) Summary: 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.) This change makes the "proper" behavior the default behavior, at least for GetFileSize and FSSequentialFile. However, this new functionality is disabled in db_stress because of undiagnosed, unresolved issues. Also removes unused and confusing field `pos_at_last_flush_` This change is needed to support testing a relevant bug fix (in a follow-up diff). Other suggested follow-up: * Fix db_stress not to rely on the old behavior, and fix a related FIXME in db_stress_test_base.cc in LockWAL testing. * Fill in some corner cases in the FileSystem API for reading unsynced data (see new TODO items). * Consider deprecating and removing Flush() API functions from FileSystem APIs. It is not clear to me that there is a supported scenario in which they do anything but confuse API users and developers. If there is a use for them, it doesn't appear to be tested. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12729 Test Plan: applies to all unit tests successfully, just updating the unit test from https://github.com/facebook/rocksdb/issues/12556 due to relying on the errant behavior. Also added a specific unit test Reviewed By: hx235 Differential Revision: D58091835 Pulled By: pdillinger fbshipit-source-id: f47a63b2b000f5875b6293a98577bff663d7fd33
2024-06-04 22:25:23 +00:00
}
}
return io_s;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus FaultInjectionTestFS::GetFileModificationTime(const std::string& fname,
const IOOptions& options,
uint64_t* file_mtime,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->GetFileModificationTime(fname, options, file_mtime, dbg);
return io_s;
}
IOStatus FaultInjectionTestFS::RenameFile(const std::string& s,
const std::string& t,
const IOOptions& options,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus io_s = MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, options);
if (!io_s.ok()) {
return io_s;
}
// We preserve contents of overwritten files up to a size threshold.
// We could keep previous file in another name, but we need to worry about
// garbage collect the those files. We do it if it is needed later.
// We ignore I/O errors here for simplicity.
std::string previous_contents = kNewFileNoOverwrite;
if (target()->FileExists(t, IOOptions(), nullptr).ok()) {
uint64_t file_size;
if (target()->GetFileSize(t, IOOptions(), &file_size, nullptr).ok() &&
file_size < 1024) {
ReadFileToString(target(), t, &previous_contents).PermitUncheckedError();
}
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
io_s = FileSystemWrapper::RenameFile(s, t, options, dbg);
if (io_s.ok()) {
{
MutexLock l(&mutex_);
if (db_file_state_.find(s) != db_file_state_.end()) {
db_file_state_[t] = db_file_state_[s];
db_file_state_.erase(s);
}
auto sdn = TestFSGetDirAndName(s);
auto tdn = TestFSGetDirAndName(t);
if (dir_to_new_files_since_last_sync_[sdn.first].erase(sdn.second) != 0) {
auto& tlist = dir_to_new_files_since_last_sync_[tdn.first];
assert(tlist.find(tdn.second) == tlist.end());
tlist[tdn.second] = previous_contents;
}
}
}
return io_s;
}
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
IOStatus FaultInjectionTestFS::LinkFile(const std::string& s,
const std::string& t,
const IOOptions& options,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus io_s = MaybeInjectThreadLocalError(
FaultInjectionIOType::kMetadataWrite, options);
if (!io_s.ok()) {
return io_s;
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
}
// Using the value in `dir_to_new_files_since_last_sync_` for the source file
// may be a more reasonable choice.
std::string previous_contents = kNewFileNoOverwrite;
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
io_s = FileSystemWrapper::LinkFile(s, t, options, dbg);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
if (io_s.ok()) {
{
MutexLock l(&mutex_);
Ensure Close() before LinkFile() for WALs in Checkpoint (#12734) Summary: POSIX semantics for LinkFile (hard links) allow linking a file that is still being written two, with both the source and destination showing any subsequent writes to the source. This may not be practical semantics for some FileSystem implementations such as remote storage. They might only link the flushed or sync-ed file contents at time of LinkFile, or might even have undefined behavior if LinkFile is called on a file still open for write (not yet "sealed"). This change builds on https://github.com/facebook/rocksdb/issues/12731 to bring more hygiene to our handling of WAL files in Checkpoint. Specifically, we now Close WAL files as soon as they are either (a) inactive and fully synced, or (b) inactive and obsolete (so maybe never fully synced), rather than letting Close() happen in handling obsolete files (maybe a background thread). This should not be a performance issue as Close() should be trivial cost relative to other IO ops, but just in case: * We don't Close() while holding a mutex, to avoid blocking, and * The old behavior is available with a new kill switch option `background_close_inactive_wals`. Stacked on https://github.com/facebook/rocksdb/issues/12731 Pull Request resolved: https://github.com/facebook/rocksdb/pull/12734 Test Plan: Extended existing unit test, especially adding a hygiene check to FaultInjectionTestFS to detect LinkFile() on a file still open for writes. FaultInjectionTestFS already has relevant tracking data, and tests can opt out of the new check, as in a smoke test I have left for the old, deprecated functionality `background_close_inactive_wals=true`. Also ran lengthy blackbox_crash_test to ensure the hygiene check is OK with the crash test. (The only place I can find we use LinkFile in production is Checkpoint.) Reviewed By: cbi42 Differential Revision: D58295284 Pulled By: pdillinger fbshipit-source-id: 64d90ed8477e2366c19eaf9c4c5ad60b82cac5c6
2024-06-12 18:48:45 +00:00
if (!allow_link_open_file_ &&
open_managed_files_.find(s) != open_managed_files_.end()) {
fprintf(stderr, "Attempt to LinkFile while open for write: %s\n",
s.c_str());
abort();
}
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
if (db_file_state_.find(s) != db_file_state_.end()) {
db_file_state_[t] = db_file_state_[s];
}
auto sdn = TestFSGetDirAndName(s);
auto tdn = TestFSGetDirAndName(t);
if (dir_to_new_files_since_last_sync_[sdn.first].find(sdn.second) !=
dir_to_new_files_since_last_sync_[sdn.first].end()) {
auto& tlist = dir_to_new_files_since_last_sync_[tdn.first];
assert(tlist.find(tdn.second) == tlist.end());
tlist[tdn.second] = previous_contents;
}
}
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return io_s;
}
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus FaultInjectionTestFS::NumFileLinks(const std::string& fname,
const IOOptions& options,
uint64_t* count,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->NumFileLinks(fname, options, count, dbg);
return io_s;
}
IOStatus FaultInjectionTestFS::AreFilesSame(const std::string& first,
const std::string& second,
const IOOptions& options, bool* res,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->AreFilesSame(first, second, options, res, dbg);
return io_s;
}
IOStatus FaultInjectionTestFS::GetAbsolutePath(const std::string& db_path,
const IOOptions& options,
std::string* output_path,
IODebugContext* dbg) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->GetAbsolutePath(db_path, options, output_path, dbg);
return io_s;
}
IOStatus FaultInjectionTestFS::IsDirectory(const std::string& path,
const IOOptions& options,
bool* is_dir, IODebugContext* dgb) {
if (!IsFilesystemActive()) {
return GetError();
}
IOStatus io_s =
MaybeInjectThreadLocalError(FaultInjectionIOType::kMetadataRead, options);
if (!io_s.ok()) {
return io_s;
}
io_s = target()->IsDirectory(path, options, is_dir, dgb);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
return io_s;
}
IOStatus FaultInjectionTestFS::Poll(std::vector<void*>& io_handles,
size_t min_completions) {
return target()->Poll(io_handles, min_completions);
}
IOStatus FaultInjectionTestFS::AbortIO(std::vector<void*>& io_handles) {
return target()->AbortIO(io_handles);
}
void FaultInjectionTestFS::WritableFileClosed(const FSFileState& state) {
MutexLock l(&mutex_);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
if (open_managed_files_.find(state.filename_) != open_managed_files_.end()) {
db_file_state_[state.filename_] = state;
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
open_managed_files_.erase(state.filename_);
}
}
void FaultInjectionTestFS::WritableFileSynced(const FSFileState& state) {
MutexLock l(&mutex_);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
if (open_managed_files_.find(state.filename_) != open_managed_files_.end()) {
if (db_file_state_.find(state.filename_) == db_file_state_.end()) {
db_file_state_.insert(std::make_pair(state.filename_, state));
} else {
db_file_state_[state.filename_] = state;
}
}
}
void FaultInjectionTestFS::WritableFileAppended(const FSFileState& state) {
MutexLock l(&mutex_);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
if (open_managed_files_.find(state.filename_) != open_managed_files_.end()) {
if (db_file_state_.find(state.filename_) == db_file_state_.end()) {
db_file_state_.insert(std::make_pair(state.filename_, state));
} else {
db_file_state_[state.filename_] = state;
}
}
}
IOStatus FaultInjectionTestFS::DropUnsyncedFileData() {
IOStatus io_s;
MutexLock l(&mutex_);
for (std::map<std::string, FSFileState>::iterator it = db_file_state_.begin();
io_s.ok() && it != db_file_state_.end(); ++it) {
FSFileState& fs_state = it->second;
if (!fs_state.IsFullySynced()) {
io_s = fs_state.DropUnsyncedData();
}
}
return io_s;
}
IOStatus FaultInjectionTestFS::DropRandomUnsyncedFileData(Random* rnd) {
IOStatus io_s;
MutexLock l(&mutex_);
for (std::map<std::string, FSFileState>::iterator it = db_file_state_.begin();
io_s.ok() && it != db_file_state_.end(); ++it) {
FSFileState& fs_state = it->second;
if (!fs_state.IsFullySynced()) {
io_s = fs_state.DropRandomUnsyncedData(rnd);
}
}
return io_s;
}
IOStatus FaultInjectionTestFS::DeleteFilesCreatedAfterLastDirSync(
const IOOptions& options, IODebugContext* dbg) {
// Because DeleteFile access this container make a copy to avoid deadlock
std::map<std::string, std::map<std::string, std::string>> map_copy;
{
MutexLock l(&mutex_);
map_copy.insert(dir_to_new_files_since_last_sync_.begin(),
dir_to_new_files_since_last_sync_.end());
}
for (auto& pair : map_copy) {
for (auto& file_pair : pair.second) {
if (file_pair.second == kNewFileNoOverwrite) {
IOStatus io_s =
DeleteFile(pair.first + "/" + file_pair.first, options, dbg);
if (!io_s.ok()) {
return io_s;
}
} else {
Group SST write in flush, compaction and db open with new stats (#11910) Summary: ## Context/Summary Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity. For that, this PR does the following: - Tag different write IOs by passing down and converting WriteOptions to IOOptions - Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS Some related code refactory to make implementation cleaner: - Blob stats - Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info. - Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write. - Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority - Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification - Build table - TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables - Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder. This parameter is used for dynamically changing file io priority for flush, see https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more - Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority ## Test ### db bench Flush ``` ./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100 rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 ``` compaction, db oopen ``` Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279 rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213 rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66 ``` blob stats - just to make sure they aren't broken by this PR ``` Integrated Blob DB Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600 rocksdb.blobdb.blob.file.synced COUNT : 1 rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same) ``` ``` Stacked Blob DB Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876 rocksdb.blobdb.blob.file.synced COUNT : 8 rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same) ``` ### Rehearsal CI stress test Trigger 3 full runs of all our CI stress tests ### Performance Flush ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark; enable_statistics = true Pre-pr: avg 507515519.3 ns 497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908, Post-pr: avg 511971266.5 ns, regressed 0.88% 502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408, ``` Compaction ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 495346098.30 ns 492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846 Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97% 502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007 ``` Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats) ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 3848.10 ns 3814,3838,3839,3848,3854,3854,3854,3860,3860,3860 Post-pr: avg 3874.20 ns, regressed 0.68% 3863,3867,3871,3874,3875,3877,3877,3877,3880,3881 ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910 Reviewed By: ajkr Differential Revision: D49788060 Pulled By: hx235 fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 23:29:23 +00:00
IOOptions opts;
IOStatus io_s =
WriteStringToFile(target(), file_pair.second,
Group SST write in flush, compaction and db open with new stats (#11910) Summary: ## Context/Summary Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity. For that, this PR does the following: - Tag different write IOs by passing down and converting WriteOptions to IOOptions - Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS Some related code refactory to make implementation cleaner: - Blob stats - Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info. - Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write. - Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority - Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification - Build table - TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables - Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder. This parameter is used for dynamically changing file io priority for flush, see https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more - Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority ## Test ### db bench Flush ``` ./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100 rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 ``` compaction, db oopen ``` Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279 rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213 rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66 ``` blob stats - just to make sure they aren't broken by this PR ``` Integrated Blob DB Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600 rocksdb.blobdb.blob.file.synced COUNT : 1 rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same) ``` ``` Stacked Blob DB Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876 rocksdb.blobdb.blob.file.synced COUNT : 8 rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same) ``` ### Rehearsal CI stress test Trigger 3 full runs of all our CI stress tests ### Performance Flush ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark; enable_statistics = true Pre-pr: avg 507515519.3 ns 497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908, Post-pr: avg 511971266.5 ns, regressed 0.88% 502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408, ``` Compaction ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 495346098.30 ns 492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846 Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97% 502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007 ``` Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats) ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 3848.10 ns 3814,3838,3839,3848,3854,3854,3854,3860,3860,3860 Post-pr: avg 3874.20 ns, regressed 0.68% 3863,3867,3871,3874,3875,3877,3877,3877,3880,3881 ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910 Reviewed By: ajkr Differential Revision: D49788060 Pulled By: hx235 fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 23:29:23 +00:00
pair.first + "/" + file_pair.first, true, opts);
if (!io_s.ok()) {
return io_s;
}
}
}
}
return IOStatus::OK();
}
void FaultInjectionTestFS::ResetState() {
MutexLock l(&mutex_);
db_file_state_.clear();
dir_to_new_files_since_last_sync_.clear();
SetFilesystemActiveNoLock(true);
}
void FaultInjectionTestFS::UntrackFile(const std::string& f) {
MutexLock l(&mutex_);
auto dir_and_name = TestFSGetDirAndName(f);
dir_to_new_files_since_last_sync_[dir_and_name.first].erase(
dir_and_name.second);
db_file_state_.erase(f);
Protect existing files in `FaultInjectionTest{Env,FS}::ReopenWritableFile()` (#8995) Summary: `FaultInjectionTest{Env,FS}::ReopenWritableFile()` functions were accidentally deleting WALs from previous `db_stress` runs causing verification to fail. They were operating under the assumption that `ReopenWritableFile()` would delete any existing file. It was a reasonable assumption considering the `{Env,FileSystem}::ReopenWritableFile()` documentation stated that would happen. The only problem was neither the implementations we offer nor the "real" clients in RocksDB code followed that contract. So, this PR updates the contract as well as fixing the fault injection client usage. The fault injection change exposed that `ExternalSSTFileBasicTest.SyncFailure` was relying on a fault injection `Env` dropping unsynced data written by a regular `Env`. I changed that test to make its `SstFileWriter` use fault injection `Env`, and also implemented `LinkFile()` in fault injection so the unsynced data is tracked under the new name. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8995 Test Plan: - Verified it fixes the following failure: ``` $ ./db_stress --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --ops_per_thread=1000 --prefixpercent=0 --readpercent=60 --reopen=0 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... $ ./db_stress --avoid_flush_during_recovery=1 --clear_column_family_one_in=0 --column_families=1 --db=/dev/shm/rocksdb_crashtest_whitebox --delpercent=5 --destroy_db_initially=0 --expected_values_dir=/dev/shm/rocksdb_crashtest_expected --iterpercent=10 --key_len_percent_dist=1,30,69 --max_bytes_for_level_base=4194304 --max_key=100000 --max_key_len=3 --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_write_fault_one_in=16 --ops_per_thread=1000 --prefix_size=-1 --prefixpercent=0 --readpercent=50 --sync=1 --target_file_size_base=1048576 --test_batches_snapshots=0 --write_buffer_size=1048576 --writepercent=35 --value_size_mult=33 -threads=1 ... Verification failed for column family 0 key 000000000000001300000000000000857878787878 (1143): Value not found: NotFound: Crash-recovery verification failed :( ... ``` - `make check -j48` Reviewed By: ltamasi Differential Revision: D31495388 Pulled By: ajkr fbshipit-source-id: 7886ccb6a07cb8b78ad7b6c1c341ccf40bb68385
2021-10-11 23:22:10 +00:00
open_managed_files_.erase(f);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus FaultInjectionTestFS::MaybeInjectThreadLocalReadError(
const IOOptions& io_options, ErrorOperation op, Slice* result,
bool direct_io, char* scratch, bool need_count_increase,
bool* fault_injected) {
bool dummy_bool;
bool& ret_fault_injected = fault_injected ? *fault_injected : dummy_bool;
ret_fault_injected = false;
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
ErrorContext* ctx =
static_cast<ErrorContext*>(injected_thread_local_read_error_.Get());
if (ctx == nullptr || !ctx->enable_error_injection || !ctx->one_in ||
Fix nullptr access and race to fault_fs_guard (#12799) Summary: **Context/Summary:** There are a couple places where we forgot to check fault_fs_guard before accessing it. So we can see something like this occasionally ``` =138831==Hint: address points to the zero page. SCARINESS: 10 (null-deref) AddressSanitizer:DEADLYSIGNAL #0 0x18b9e0b in rocksdb::ThreadLocalPtr::Get() const fbcode/internal_repo_rocksdb/repo/util/thread_local.cc:503 https://github.com/facebook/rocksdb/issues/1 0x83d8b7 in rocksdb::StressTest::TestCompactRange(rocksdb::ThreadState*, long, rocksdb::Slice const&, rocksdb::ColumnFamilyHandle*) fbcode/internal_repo_rocksdb/repo/utilities/fault_injection_fs.h ``` Also accessing of `io_activties_exempted_from_fault_injection.find` not fully synced so we see the following ``` WARNING: ThreadSanitizer: data race (pid=90939) Write of size 8 at 0x7b4c000004d0 by thread T762 (mutexes: write M0): #0 std::_Rb_tree<rocksdb::Env::IOActivity, rocksdb::Env::IOActivity, std::_Identity<rocksdb::Env::IOActivity>, std::less<rocksdb::Env::IOActivity>, std::allocator<rocksdb::Env::IOActivity>>::operator=(std::_Rb_tree<rocksdb::Env::IOActivity, rocksdb::Env::IOActivity, std::_Identity<rocksdb::Env::IOActivity>, std::less<rocksdb::Env::IOActivity>, std::allocator<rocksdb::Env::IOActivity>> const&) fbcode/third-party-buck/platform010/build/libgcc/include/c++/trunk/bits/stl_tree.h:208 (db_stress+0x411c32) (BuildId: b803e5aca22c6b080defed8e85b7bfec) https://github.com/facebook/rocksdb/issues/1 rocksdb::DbStressListener::OnErrorRecoveryCompleted(rocksdb::Status) fbcode/third-party-buck/platform010/build/libgcc/include/c++/trunk/bits/stl_set.h:298 (db_stress+0x4112e5) (BuildId: b803e5aca22c6b080defed8e85b7bfec) https://github.com/facebook/rocksdb/issues/2 rocksdb::EventHelpers::NotifyOnErrorRecoveryEnd(std::vector<std::shared_ptr<rocksdb::EventListener>, std::allocator<std::shared_ptr<rocksdb::EventListener>>> const&, rocksdb::Status const&, rocksdb::Status const&, rocksdb::InstrumentedMutex*) fbcode/internal_repo_rocksdb/repo/db/event_helpers.cc:239 (db_stress+0xa09d60) (BuildId: b803e5aca22c6b080defed8e85b7bfec) Previous read of size 8 at 0x7b4c000004d0 by thread T131 (mutexes: write M1): #0 rocksdb::FaultInjectionTestFS::MaybeInjectThreadLocalError(rocksdb::FaultInjectionIOType, rocksdb::IOOptions const&, rocksdb::FaultInjectionTestFS::ErrorOperation, rocksdb::Slice*, bool, char*, bool, bool*) fbcode/third-party-buck/platform010/build/libgcc/include/c++/trunk/bits/stl_tree.h:798 (db_stress+0xf7d0f3) (BuildId: b803e5aca22c6b080defed8e85b7bfec) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/12799 Test Plan: CI Reviewed By: jowlyzhang Differential Revision: D58917449 Pulled By: hx235 fbshipit-source-id: f24fc1acc2a7d91f9f285447a97ba41397f48dbd
2024-06-24 23:10:36 +00:00
ShouldIOActivtiesExemptFromFaultInjection(io_options.io_activity)) {
return IOStatus::OK();
}
IOStatus ret;
if (ctx->rand.OneIn(ctx->one_in)) {
if (ctx->count == 0) {
ctx->message = "";
}
if (need_count_increase) {
ctx->count++;
}
if (ctx->callstack) {
free(ctx->callstack);
}
ctx->callstack = port::SaveStack(&ctx->frames);
if (op != ErrorOperation::kMultiReadSingleReq) {
// Likely non-per read status code for MultiRead
ctx->message += "injected read error; ";
ret_fault_injected = true;
ret = IOStatus::IOError(ctx->message);
} else if (Random::GetTLSInstance()->OneIn(8)) {
assert(result);
// For a small chance, set the failure to status but turn the
// result to be empty, which is supposed to be caught for a check.
*result = Slice();
ctx->message += "injected empty result; ";
ret_fault_injected = true;
} else if (!direct_io && Random::GetTLSInstance()->OneIn(7) &&
scratch != nullptr && result->data() == scratch) {
assert(result);
// With direct I/O, many extra bytes might be read so corrupting
// one byte might not cause checksum mismatch. Skip checksum
// corruption injection.
// We only corrupt data if the result is filled to `scratch`. For other
// cases, the data might not be able to be modified (e.g mmaped files)
// or has unintended side effects.
// For a small chance, set the failure to status but corrupt the
// result in a way that checksum checking is supposed to fail.
// Corrupt the last byte, which is supposed to be a checksum byte
// It would work for CRC. Not 100% sure for xxhash and will adjust
// if it is not the case.
const_cast<char*>(result->data())[result->size() - 1]++;
ctx->message += "injected corrupt last byte; ";
ret_fault_injected = true;
} else {
ctx->message += "injected error result multiget single; ";
ret_fault_injected = true;
ret = IOStatus::IOError(ctx->message);
}
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
ret.SetRetryable(ctx->retryable);
ret.SetDataLoss(ctx->has_data_loss);
return ret;
}
bool FaultInjectionTestFS::TryParseFileName(const std::string& file_name,
uint64_t* number, FileType* type) {
std::size_t found = file_name.find_last_of('/');
std::string file = file_name.substr(found);
return ParseFileName(file, number, type);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus FaultInjectionTestFS::MaybeInjectThreadLocalError(
FaultInjectionIOType type, const IOOptions& io_options, ErrorOperation op,
Slice* result, bool direct_io, char* scratch, bool need_count_increase,
bool* fault_injected) {
if (type == FaultInjectionIOType::kRead) {
return MaybeInjectThreadLocalReadError(io_options, op, result, direct_io,
scratch, need_count_increase,
fault_injected);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
ErrorContext* ctx = GetErrorContextFromFaultInjectionIOType(type);
if (ctx == nullptr || !ctx->enable_error_injection || !ctx->one_in ||
Fix nullptr access and race to fault_fs_guard (#12799) Summary: **Context/Summary:** There are a couple places where we forgot to check fault_fs_guard before accessing it. So we can see something like this occasionally ``` =138831==Hint: address points to the zero page. SCARINESS: 10 (null-deref) AddressSanitizer:DEADLYSIGNAL #0 0x18b9e0b in rocksdb::ThreadLocalPtr::Get() const fbcode/internal_repo_rocksdb/repo/util/thread_local.cc:503 https://github.com/facebook/rocksdb/issues/1 0x83d8b7 in rocksdb::StressTest::TestCompactRange(rocksdb::ThreadState*, long, rocksdb::Slice const&, rocksdb::ColumnFamilyHandle*) fbcode/internal_repo_rocksdb/repo/utilities/fault_injection_fs.h ``` Also accessing of `io_activties_exempted_from_fault_injection.find` not fully synced so we see the following ``` WARNING: ThreadSanitizer: data race (pid=90939) Write of size 8 at 0x7b4c000004d0 by thread T762 (mutexes: write M0): #0 std::_Rb_tree<rocksdb::Env::IOActivity, rocksdb::Env::IOActivity, std::_Identity<rocksdb::Env::IOActivity>, std::less<rocksdb::Env::IOActivity>, std::allocator<rocksdb::Env::IOActivity>>::operator=(std::_Rb_tree<rocksdb::Env::IOActivity, rocksdb::Env::IOActivity, std::_Identity<rocksdb::Env::IOActivity>, std::less<rocksdb::Env::IOActivity>, std::allocator<rocksdb::Env::IOActivity>> const&) fbcode/third-party-buck/platform010/build/libgcc/include/c++/trunk/bits/stl_tree.h:208 (db_stress+0x411c32) (BuildId: b803e5aca22c6b080defed8e85b7bfec) https://github.com/facebook/rocksdb/issues/1 rocksdb::DbStressListener::OnErrorRecoveryCompleted(rocksdb::Status) fbcode/third-party-buck/platform010/build/libgcc/include/c++/trunk/bits/stl_set.h:298 (db_stress+0x4112e5) (BuildId: b803e5aca22c6b080defed8e85b7bfec) https://github.com/facebook/rocksdb/issues/2 rocksdb::EventHelpers::NotifyOnErrorRecoveryEnd(std::vector<std::shared_ptr<rocksdb::EventListener>, std::allocator<std::shared_ptr<rocksdb::EventListener>>> const&, rocksdb::Status const&, rocksdb::Status const&, rocksdb::InstrumentedMutex*) fbcode/internal_repo_rocksdb/repo/db/event_helpers.cc:239 (db_stress+0xa09d60) (BuildId: b803e5aca22c6b080defed8e85b7bfec) Previous read of size 8 at 0x7b4c000004d0 by thread T131 (mutexes: write M1): #0 rocksdb::FaultInjectionTestFS::MaybeInjectThreadLocalError(rocksdb::FaultInjectionIOType, rocksdb::IOOptions const&, rocksdb::FaultInjectionTestFS::ErrorOperation, rocksdb::Slice*, bool, char*, bool, bool*) fbcode/third-party-buck/platform010/build/libgcc/include/c++/trunk/bits/stl_tree.h:798 (db_stress+0xf7d0f3) (BuildId: b803e5aca22c6b080defed8e85b7bfec) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/12799 Test Plan: CI Reviewed By: jowlyzhang Differential Revision: D58917449 Pulled By: hx235 fbshipit-source-id: f24fc1acc2a7d91f9f285447a97ba41397f48dbd
2024-06-24 23:10:36 +00:00
ShouldIOActivtiesExemptFromFaultInjection(io_options.io_activity)) {
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return IOStatus::OK();
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
IOStatus ret;
if (ctx->rand.OneIn(ctx->one_in)) {
ctx->count++;
if (ctx->callstack) {
free(ctx->callstack);
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
ctx->callstack = port::SaveStack(&ctx->frames);
ctx->message = GetErrorMessageFromFaultInjectionIOType(type);
ret = IOStatus::IOError(ctx->message);
ret.SetRetryable(ctx->retryable);
ret.SetDataLoss(ctx->has_data_loss);
if (type == FaultInjectionIOType::kWrite) {
TEST_SYNC_POINT(
"FaultInjectionTestFS::InjectMetadataWriteError:Injected");
}
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
return ret;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
void FaultInjectionTestFS::PrintInjectedThreadLocalErrorBacktrace(
FaultInjectionIOType type) {
#if defined(OS_LINUX)
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
ErrorContext* ctx = GetErrorContextFromFaultInjectionIOType(type);
if (ctx) {
if (type == FaultInjectionIOType::kRead) {
fprintf(stderr, "Injected read error type = %d\n", ctx->type);
}
fprintf(stderr, "Message: %s\n", ctx->message.c_str());
port::PrintAndFreeStack(ctx->callstack, ctx->frames);
ctx->callstack = nullptr;
}
Inject more errors to more files in stress test (#12713) Summary: **Context:** We currently have partial error injection: - DB operation: all read, SST write - DB open: all read, SST write, all metadata write. This PR completes the error injection (with some limitations below): - DB operation & open: all read, all write, all metadata write, all metadata read **Summary:** - Inject retryable metadata read, metadata write error concerning directory (e.g, dir sync, ) or file metadata (e.g, name, size, file creation/deletion...) - Inject retryable errors to all major file types: random access file, sequential file, writable file - Allow db stress test operations to handle above injected errors gracefully without crashing - Change all error injection to thread-local implementation for easier disabling and enabling in the same thread. For example, we can control error handling thread to have no error injection. It's also cleaner in code. - Limitation: compared to before, we now don't have write fault injection for backup/restore CopyOrCreateFiles work threads since they use anonymous background threads as well as read injection for db open bg thread - Add a new flag to test error recovery without error injection so we can test the path where error recovery actually succeeds - Some Refactory & fix to db stress test framework (see PR review comments) - Fix some minor bugs surfaced (see PR review comments) - Limitation: had to disable backup restore with metadata read/write injection since it surfaces too many testing issues. Will add it back later to focus on surfacing actual code/internal bugs first. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12713 Test Plan: - Existing UT - CI with no trivial error failure Reviewed By: pdillinger Differential Revision: D58326608 Pulled By: hx235 fbshipit-source-id: 011b5195aaeb6011641ae0a9194f7f2a0e325ad7
2024-06-19 15:42:00 +00:00
#else
(void)type;
#endif
}
} // namespace ROCKSDB_NAMESPACE