2019-12-09 07:49:32 +00:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
|
|
|
// This source code is licensed under both the GPLv2 (found in the
|
|
|
|
// COPYING file in the root directory) and Apache 2.0 License
|
|
|
|
// (found in the LICENSE.Apache file in the root directory).
|
|
|
|
//
|
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors
|
|
|
|
|
|
|
|
#ifdef GFLAGS
|
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include "db_stress_tool/db_stress_stat.h"
|
2021-09-28 21:12:23 +00:00
|
|
|
#include "db_stress_tool/expected_state.h"
|
2020-04-11 00:18:56 +00:00
|
|
|
// SyncPoint is not supported in Released Windows Mode.
|
|
|
|
#if !(defined NDEBUG) || !defined(OS_WIN)
|
|
|
|
#include "test_util/sync_point.h"
|
|
|
|
#endif // !(defined NDEBUG) || !defined(OS_WIN)
|
2019-12-09 07:49:32 +00:00
|
|
|
#include "util/gflags_compat.h"
|
|
|
|
|
|
|
|
DECLARE_uint64(seed);
|
|
|
|
DECLARE_int64(max_key);
|
|
|
|
DECLARE_uint64(log2_keys_per_lock);
|
|
|
|
DECLARE_int32(threads);
|
|
|
|
DECLARE_int32(column_families);
|
|
|
|
DECLARE_int32(nooverwritepercent);
|
2021-09-28 21:12:23 +00:00
|
|
|
DECLARE_string(expected_values_dir);
|
2019-12-09 07:49:32 +00:00
|
|
|
DECLARE_int32(clear_column_family_one_in);
|
|
|
|
DECLARE_bool(test_batches_snapshots);
|
2019-12-20 16:46:52 +00:00
|
|
|
DECLARE_int32(compaction_thread_pool_adjust_interval);
|
|
|
|
DECLARE_int32(continuous_verification_interval);
|
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
|
|
|
DECLARE_bool(error_recovery_with_no_fault_injection);
|
|
|
|
DECLARE_bool(sync_fault_injection);
|
|
|
|
DECLARE_bool(disable_wal);
|
|
|
|
DECLARE_int32(manual_wal_flush_one_in);
|
|
|
|
DECLARE_int32(metadata_read_fault_one_in);
|
|
|
|
DECLARE_int32(metadata_write_fault_one_in);
|
2020-04-11 00:18:56 +00:00
|
|
|
DECLARE_int32(read_fault_one_in);
|
2020-12-17 19:51:04 +00:00
|
|
|
DECLARE_int32(write_fault_one_in);
|
Decouple sync fault and write injection in FaultInjectionTestFS & fix tracing issue under WAL write error injection (#12797)
Summary:
**Context/Summary:**
After injecting write error to WAL, we started to see crash recovery verification failure in prefix recovery. That's because the current tracing implementation traces every write before it writes to WAL even when the WAL write can fail with write error injection. One consequence of that is the traced writes in trace files does not corresponding to write sequence sequence anymore e.g, it has more traced writes that the actual assigned sequence number to successful writes. Therefore https://github.com/facebook/rocksdb/blob/b4a84efb4e842b782e976de5b22a4554c2f76edd/db_stress_tool/expected_state.cc#L674 won't restore the ExpectedState to the correct sequence number we want.
Ideally, we should have a prepare-commit mechanism for tracing just like our ExpectedState so we can ignore the traced write if the write fails later. But for now, to simplify, we simply don't inject WAL error (and metadata write error cuz it could fail write when sync WAL dir fails)
To do so, we need to be able to exclude WAL from write injection but still allow sync fault injection in it to maintain its original sync fault testing coverage. This prompts us to decouple sync fault and write injection in FaultInjectionTestFS. And this is what this PR mainly about.
So now `FaultInjectionTestFS` works as the following:
- If direct_writable is true, then `FaultInjectionTestFS` is bypassed for writable file
- Otherwise, FaultInjectionTestFS` can buffer data for sync fault injection (if inject_unsynced_data_loss_ == true, global settings) and/or inject write error (if MaybeInjectThreadLocalError(), thread-local settings). WAL file can be optionally excluded from write injection
Bonus: better naming of relevant variables
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12797
Test Plan:
- The follow commands failed before this fix but passes after
```
python3 tools/db_crashtest.py --simple blackbox \
--interval=5 \
--preserve_unverified_changes=1 \
--threads=32 \
--disable_auto_compactions=1 \
--WAL_size_limit_MB=0 --WAL_ttl_seconds=0 --acquire_snapshot_one_in=0 --adaptive_readahead=0 --adm_policy=0 --advise_random_on_open=1 --allow_concurrent_memtable_write=0 --allow_data_in_errors=True --allow_fallocate=1 --async_io=0 --auto_readahead_size=0 --avoid_flush_during_recovery=1 --avoid_flush_during_shutdown=0 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=0 --batch_protection_bytes_per_key=0 --bgerror_resume_retry_interval=1000000 --block_align=0 --block_protection_bytes_per_key=4 --block_size=16384 --bloom_before_level=2147483646 --bloom_bits=3.2003682301518492 --bottommost_compression_type=zlib --bottommost_file_compaction_delay=600 --bytes_per_sync=0 --cache_index_and_filter_blocks=1 --cache_index_and_filter_blocks_with_high_priority=1 --cache_size=33554432 --cache_type=fixed_hyper_clock_cache --charge_compression_dictionary_building_buffer=0 --charge_file_metadata=0 --charge_filter_construction=0 --charge_table_reader=1 --check_multiget_consistency=0 --check_multiget_entity_consistency=0 --checkpoint_one_in=0 --checksum_type=kxxHash64 --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=0 --compact_range_one_in=0 --compaction_pri=2 --compaction_readahead_size=0 --compaction_ttl=0 --compress_format_version=1 --compressed_secondary_cache_size=16777216 --compression_checksum=1 --compression_max_dict_buffer_bytes=549755813887 --compression_max_dict_bytes=16384 --compression_parallel_threads=1 --compression_type=none --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --daily_offpeak_time_utc=00:00-23:59 --data_block_index_type=0 \
--db_write_buffer_size=0 --delete_obsolete_files_period_micros=0 --delpercent=0 --delrangepercent=0 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_file_deletions_one_in=0 --disable_manual_compaction_one_in=0 --disable_wal=0 --dump_malloc_stats=0 --enable_checksum_handoff=0 --enable_compaction_filter=0 --enable_custom_split_merge=0 --enable_do_not_compress_roles=1 --enable_index_compression=0 --enable_memtable_insert_with_hint_prefix_extractor=0 --enable_pipelined_write=0 --enable_sst_partitioner_factory=0 --enable_thread_tracking=0 --enable_write_thread_adaptive_yield=0 --error_recovery_with_no_fault_injection=0 --fail_if_options_file_error=0 --fifo_allow_compaction=1 --file_checksum_impl=xxh64 --fill_cache=0 --flush_one_in=100 --format_version=4 --get_all_column_family_metadata_one_in=0 --get_current_wal_file_one_in=0 --get_live_files_apis_one_in=0 --get_properties_of_all_tables_one_in=0 --get_property_one_in=0 --get_sorted_wal_files_one_in=0 --hard_pending_compaction_bytes_limit=274877906944 --high_pri_pool_ratio=0.5 --index_block_restart_interval=9 --index_shortening=1 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=0 --inplace_update_support=0 --iterpercent=0 --key_len_percent_dist=1,30,69 --key_may_exist_one_in=0 --last_level_temperature=kUnknown --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=0 --log2_keys_per_lock=10 --log_file_time_to_roll=0 --log_readahead_size=16777216 --long_running_snapshots=0 --low_pri_pool_ratio=0 --lowest_used_cache_tier=2 --manifest_preallocation_size=0 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=0 --max_auto_readahead_size=524288 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=1000 --max_key_len=3 --memtable_insert_hint_per_batch=0 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.5 --memtable_protection_bytes_per_key=8 --memtable_whole_key_filtering=0 --memtablerep=skip_list --metadata_charge_policy=0 --metadata_read_fault_one_in=0 --metadata_write_fault_one_in=0 --min_write_buffer_number_to_merge=1 --mmap_read=0 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=0 --open_files=-1 --open_metadata_read_fault_one_in=0 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=20000000 \
--optimize_filters_for_hits=1 --optimize_filters_for_memory=1 --optimize_multiget_for_io=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=3 --pause_background_one_in=0 --periodic_compaction_seconds=0 --prefix_size=1 --prefixpercent=0 --prepopulate_block_cache=0 --preserve_internal_time_seconds=0 --progress_reports=0 --promote_l0_one_in=0 --read_amp_bytes_per_bit=0 --read_fault_one_in=0 --readahead_size=0 --readpercent=0 --recycle_log_file_num=0 --reopen=0 --report_bg_io_stats=0 --reset_stats_one_in=1000000 --sample_for_compression=5 --secondary_cache_fault_one_in=0 --secondary_cache_uri= --skip_stats_update_on_db_open=0 --snapshot_hold_ops=100000 --soft_pending_compaction_bytes_limit=68719476736 --sqfc_name=bar --sqfc_version=1 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=10 --stats_history_buffer_size=0 --strict_bytes_per_sync=0 --subcompactions=1 --sync=0 --sync_fault_injection=1 --table_cache_numshardbits=0 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=3 --uncache_aggressiveness=9890 --universal_max_read_amp=-1 --unpartitioned_pinning=3 --use_adaptive_mutex=0 --use_adaptive_mutex_lru=1 --use_attribute_group=0 --use_delta_encoding=0 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=0 --use_multi_cf_iterator=0 --use_multi_get_entity=0 --use_multiget=0 --use_put_entity_one_in=0 --use_sqfc_for_range_queries=0 --use_timed_put_one_in=0 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=0 --verify_checksum_one_in=0 --verify_compression=1 --verify_db_one_in=0 --verify_file_checksums_one_in=0 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=0 --wal_compression=zstd --write_buffer_size=335544320 --write_dbid_to_manifest=1 --write_fault_one_in=100 --writepercent=100
```
- CI
Reviewed By: cbi42
Differential Revision: D58917145
Pulled By: hx235
fbshipit-source-id: b6397036bea035a92341c2b05fb01872db2153d7
2024-06-26 21:56:35 +00:00
|
|
|
DECLARE_bool(exclude_wal_from_write_fault_injection);
|
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
|
|
|
DECLARE_int32(open_metadata_read_fault_one_in);
|
2021-04-28 17:57:11 +00:00
|
|
|
DECLARE_int32(open_metadata_write_fault_one_in);
|
2021-06-30 23:45:44 +00:00
|
|
|
DECLARE_int32(open_write_fault_one_in);
|
2021-07-06 18:04:04 +00:00
|
|
|
DECLARE_int32(open_read_fault_one_in);
|
2019-12-09 07:49:32 +00:00
|
|
|
|
2023-09-05 17:41:29 +00:00
|
|
|
DECLARE_int32(inject_error_severity);
|
2021-07-01 21:15:49 +00:00
|
|
|
|
2020-02-20 20:07:53 +00:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2019-12-09 07:49:32 +00:00
|
|
|
class StressTest;
|
|
|
|
|
|
|
|
// State shared by all concurrent executions of the same benchmark.
|
|
|
|
class SharedState {
|
|
|
|
public:
|
2020-04-11 00:18:56 +00:00
|
|
|
// Errors when reading filter blocks are ignored, so we use a thread
|
|
|
|
// local variable updated via sync points to keep track of errors injected
|
|
|
|
// while reading filter blocks in order to ignore the Get/MultiGet result
|
|
|
|
// for those calls
|
2020-04-24 20:03:08 +00:00
|
|
|
static thread_local bool ignore_read_error;
|
2020-04-11 00:18:56 +00:00
|
|
|
|
2021-07-01 19:22:01 +00:00
|
|
|
SharedState(Env* /*env*/, StressTest* stress_test)
|
2019-12-09 07:49:32 +00:00
|
|
|
: cv_(&mu_),
|
|
|
|
seed_(static_cast<uint32_t>(FLAGS_seed)),
|
|
|
|
max_key_(FLAGS_max_key),
|
|
|
|
log2_keys_per_lock_(static_cast<uint32_t>(FLAGS_log2_keys_per_lock)),
|
2021-12-18 01:30:45 +00:00
|
|
|
num_threads_(0),
|
2019-12-09 07:49:32 +00:00
|
|
|
num_initialized_(0),
|
|
|
|
num_populated_(0),
|
|
|
|
vote_reopen_(0),
|
|
|
|
num_done_(0),
|
|
|
|
start_(false),
|
|
|
|
start_verify_(false),
|
2019-12-20 16:46:52 +00:00
|
|
|
num_bg_threads_(0),
|
2019-12-09 07:49:32 +00:00
|
|
|
should_stop_bg_thread_(false),
|
2019-12-20 16:46:52 +00:00
|
|
|
bg_thread_finished_(0),
|
2019-12-09 07:49:32 +00:00
|
|
|
stress_test_(stress_test),
|
|
|
|
verification_failure_(false),
|
2019-12-20 16:46:52 +00:00
|
|
|
should_stop_test_(false),
|
2022-05-02 20:25:45 +00:00
|
|
|
no_overwrite_ids_(GenerateNoOverwriteIds()),
|
2021-09-28 21:12:23 +00:00
|
|
|
expected_state_manager_(nullptr),
|
2022-07-05 20:30:15 +00:00
|
|
|
printing_verification_results_(false),
|
|
|
|
start_timestamp_(Env::Default()->NowNanos()) {
|
2019-12-09 07:49:32 +00:00
|
|
|
Status status;
|
2021-09-28 21:12:23 +00:00
|
|
|
// TODO: We should introduce a way to explicitly disable verification
|
|
|
|
// during shutdown. When that is disabled and FLAGS_expected_values_dir
|
|
|
|
// is empty (disabling verification at startup), we can skip tracking
|
|
|
|
// expected state. Only then should we permit bypassing the below feature
|
|
|
|
// compatibility checks.
|
|
|
|
if (!FLAGS_expected_values_dir.empty()) {
|
2019-12-09 07:49:32 +00:00
|
|
|
if (!std::atomic<uint32_t>{}.is_lock_free()) {
|
|
|
|
status = Status::InvalidArgument(
|
2021-09-28 21:12:23 +00:00
|
|
|
"Cannot use --expected_values_dir on platforms without lock-free "
|
2019-12-09 07:49:32 +00:00
|
|
|
"std::atomic<uint32_t>");
|
|
|
|
}
|
|
|
|
if (status.ok() && FLAGS_clear_column_family_one_in > 0) {
|
|
|
|
status = Status::InvalidArgument(
|
2021-09-28 21:12:23 +00:00
|
|
|
"Cannot use --expected_values_dir on when "
|
2019-12-09 07:49:32 +00:00
|
|
|
"--clear_column_family_one_in is greater than zero.");
|
|
|
|
}
|
2021-09-28 21:12:23 +00:00
|
|
|
}
|
|
|
|
if (status.ok()) {
|
|
|
|
if (FLAGS_expected_values_dir.empty()) {
|
|
|
|
expected_state_manager_.reset(
|
|
|
|
new AnonExpectedStateManager(FLAGS_max_key, FLAGS_column_families));
|
2019-12-09 07:49:32 +00:00
|
|
|
} else {
|
2021-09-28 21:12:23 +00:00
|
|
|
expected_state_manager_.reset(new FileExpectedStateManager(
|
|
|
|
FLAGS_max_key, FLAGS_column_families, FLAGS_expected_values_dir));
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
2021-09-28 21:12:23 +00:00
|
|
|
status = expected_state_manager_->Open();
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
2021-09-28 21:12:23 +00:00
|
|
|
if (!status.ok()) {
|
|
|
|
fprintf(stderr, "Failed setting up expected state with error: %s\n",
|
|
|
|
status.ToString().c_str());
|
|
|
|
exit(1);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (FLAGS_test_batches_snapshots) {
|
|
|
|
fprintf(stdout, "No lock creation because test_batches_snapshots set\n");
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
long num_locks = static_cast<long>(max_key_ >> log2_keys_per_lock_);
|
|
|
|
if (max_key_ & ((1 << log2_keys_per_lock_) - 1)) {
|
|
|
|
num_locks++;
|
|
|
|
}
|
|
|
|
fprintf(stdout, "Creating %ld locks\n", num_locks * FLAGS_column_families);
|
|
|
|
key_locks_.resize(FLAGS_column_families);
|
|
|
|
|
|
|
|
for (int i = 0; i < FLAGS_column_families; ++i) {
|
2022-02-01 19:46:12 +00:00
|
|
|
key_locks_[i].reset(new port::Mutex[num_locks]);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
2020-04-11 00:18:56 +00:00
|
|
|
if (FLAGS_read_fault_one_in) {
|
2022-05-06 18:17:08 +00:00
|
|
|
#ifdef NDEBUG
|
|
|
|
// Unsupported in release mode because it relies on
|
|
|
|
// `IGNORE_STATUS_IF_ERROR` to distinguish faults not expected to lead to
|
|
|
|
// failure.
|
|
|
|
fprintf(stderr,
|
|
|
|
"Cannot set nonzero value for --read_fault_one_in in "
|
|
|
|
"release mode.");
|
|
|
|
exit(1);
|
|
|
|
#else // NDEBUG
|
2020-04-24 20:03:08 +00:00
|
|
|
SyncPoint::GetInstance()->SetCallBack("FaultInjectionIgnoreError",
|
|
|
|
IgnoreReadErrorCallback);
|
2020-04-11 00:18:56 +00:00
|
|
|
SyncPoint::GetInstance()->EnableProcessing();
|
2022-05-06 18:17:08 +00:00
|
|
|
#endif // NDEBUG
|
2020-04-11 00:18:56 +00:00
|
|
|
}
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
2020-04-11 00:18:56 +00:00
|
|
|
~SharedState() {
|
|
|
|
#ifndef NDEBUG
|
|
|
|
if (FLAGS_read_fault_one_in) {
|
|
|
|
SyncPoint::GetInstance()->ClearAllCallBacks();
|
|
|
|
SyncPoint::GetInstance()->DisableProcessing();
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
}
|
2019-12-09 07:49:32 +00:00
|
|
|
|
|
|
|
port::Mutex* GetMutex() { return &mu_; }
|
|
|
|
|
|
|
|
port::CondVar* GetCondVar() { return &cv_; }
|
|
|
|
|
|
|
|
StressTest* GetStressTest() const { return stress_test_; }
|
|
|
|
|
|
|
|
int64_t GetMaxKey() const { return max_key_; }
|
|
|
|
|
|
|
|
uint32_t GetNumThreads() const { return num_threads_; }
|
|
|
|
|
2022-01-29 18:44:22 +00:00
|
|
|
void SetThreads(int num_threads) { num_threads_ = num_threads; }
|
2021-12-18 01:30:45 +00:00
|
|
|
|
2019-12-09 07:49:32 +00:00
|
|
|
void IncInitialized() { num_initialized_++; }
|
|
|
|
|
|
|
|
void IncOperated() { num_populated_++; }
|
|
|
|
|
|
|
|
void IncDone() { num_done_++; }
|
|
|
|
|
|
|
|
void IncVotedReopen() { vote_reopen_ = (vote_reopen_ + 1) % num_threads_; }
|
|
|
|
|
|
|
|
bool AllInitialized() const { return num_initialized_ >= num_threads_; }
|
|
|
|
|
|
|
|
bool AllOperated() const { return num_populated_ >= num_threads_; }
|
|
|
|
|
|
|
|
bool AllDone() const { return num_done_ >= num_threads_; }
|
|
|
|
|
|
|
|
bool AllVotedReopen() { return (vote_reopen_ == 0); }
|
|
|
|
|
|
|
|
void SetStart() { start_ = true; }
|
|
|
|
|
|
|
|
void SetStartVerify() { start_verify_ = true; }
|
|
|
|
|
|
|
|
bool Started() const { return start_; }
|
|
|
|
|
|
|
|
bool VerifyStarted() const { return start_verify_; }
|
|
|
|
|
|
|
|
void SetVerificationFailure() { verification_failure_.store(true); }
|
|
|
|
|
2019-12-20 16:46:52 +00:00
|
|
|
bool HasVerificationFailedYet() const { return verification_failure_.load(); }
|
|
|
|
|
|
|
|
void SetShouldStopTest() { should_stop_test_.store(true); }
|
|
|
|
|
|
|
|
bool ShouldStopTest() const { return should_stop_test_.load(); }
|
2019-12-09 07:49:32 +00:00
|
|
|
|
2021-09-28 21:12:23 +00:00
|
|
|
// Returns a lock covering `key` in `cf`.
|
2019-12-09 07:49:32 +00:00
|
|
|
port::Mutex* GetMutexForKey(int cf, int64_t key) {
|
2022-02-01 19:46:12 +00:00
|
|
|
return &key_locks_[cf][key >> log2_keys_per_lock_];
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
2021-09-28 21:12:23 +00:00
|
|
|
// Acquires locks for all keys in `cf`.
|
2019-12-09 07:49:32 +00:00
|
|
|
void LockColumnFamily(int cf) {
|
2022-02-01 19:46:12 +00:00
|
|
|
for (int i = 0; i < max_key_ >> log2_keys_per_lock_; ++i) {
|
|
|
|
key_locks_[cf][i].Lock();
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-28 21:12:23 +00:00
|
|
|
// Releases locks for all keys in `cf`.
|
2019-12-09 07:49:32 +00:00
|
|
|
void UnlockColumnFamily(int cf) {
|
2022-02-01 19:46:12 +00:00
|
|
|
for (int i = 0; i < max_key_ >> log2_keys_per_lock_; ++i) {
|
|
|
|
key_locks_[cf][i].Unlock();
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-10-14 21:25:05 +00:00
|
|
|
// Returns a collection of mutex locks covering the key range [start, end) in
|
|
|
|
// `cf`.
|
|
|
|
std::vector<std::unique_ptr<MutexLock>> GetLocksForKeyRange(int cf,
|
|
|
|
int64_t start,
|
|
|
|
int64_t end) {
|
|
|
|
std::vector<std::unique_ptr<MutexLock>> range_locks;
|
|
|
|
|
|
|
|
if (start >= end) {
|
|
|
|
return range_locks;
|
|
|
|
}
|
|
|
|
|
|
|
|
const int64_t start_idx = start >> log2_keys_per_lock_;
|
|
|
|
|
|
|
|
int64_t end_idx = end >> log2_keys_per_lock_;
|
|
|
|
if ((end & ((1 << log2_keys_per_lock_) - 1)) == 0) {
|
|
|
|
--end_idx;
|
|
|
|
}
|
|
|
|
|
|
|
|
for (int64_t idx = start_idx; idx <= end_idx; ++idx) {
|
|
|
|
range_locks.emplace_back(
|
|
|
|
std::make_unique<MutexLock>(&key_locks_[cf][idx]));
|
|
|
|
}
|
|
|
|
|
|
|
|
return range_locks;
|
|
|
|
}
|
|
|
|
|
2021-12-07 21:40:46 +00:00
|
|
|
Status SaveAtAndAfter(DB* db) {
|
|
|
|
return expected_state_manager_->SaveAtAndAfter(db);
|
|
|
|
}
|
|
|
|
|
2021-12-15 20:53:32 +00:00
|
|
|
bool HasHistory() { return expected_state_manager_->HasHistory(); }
|
|
|
|
|
|
|
|
Status Restore(DB* db) { return expected_state_manager_->Restore(db); }
|
|
|
|
|
2021-09-28 21:12:23 +00:00
|
|
|
// Requires external locking covering all keys in `cf`.
|
2019-12-09 07:49:32 +00:00
|
|
|
void ClearColumnFamily(int cf) {
|
2021-09-28 21:12:23 +00:00
|
|
|
return expected_state_manager_->ClearColumnFamily(cf);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Prepare a Put that will be started but not finish yet
|
|
|
|
// This is useful for crash-recovery testing when the process may crash
|
|
|
|
// before updating the corresponding expected value
|
2021-09-28 21:12:23 +00:00
|
|
|
//
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
|
|
// write or delete to the same `key`.
|
|
|
|
PendingExpectedValue PreparePut(int cf, int64_t key) {
|
|
|
|
return expected_state_manager_->PreparePut(cf, key);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Does not requires external locking.
|
|
|
|
ExpectedValue Get(int cf, int64_t key) {
|
2021-09-28 21:12:23 +00:00
|
|
|
return expected_state_manager_->Get(cf, key);
|
|
|
|
}
|
2019-12-09 07:49:32 +00:00
|
|
|
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Prepare a Delete that will be started but not finish yet
|
|
|
|
// This is useful for crash-recovery testing when the process may crash
|
|
|
|
// before updating the corresponding expected value
|
2021-09-28 21:12:23 +00:00
|
|
|
//
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
|
|
// write or delete to the same `key`.
|
|
|
|
PendingExpectedValue PrepareDelete(int cf, int64_t key) {
|
|
|
|
return expected_state_manager_->PrepareDelete(cf, key);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
|
|
// write or delete to the same `key`.
|
|
|
|
PendingExpectedValue PrepareSingleDelete(int cf, int64_t key) {
|
|
|
|
return expected_state_manager_->PrepareSingleDelete(cf, key);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Requires external locking covering keys in `[begin_key, end_key)` in `cf`
|
|
|
|
// to prevent concurrent write or delete to the same `key`.
|
|
|
|
std::vector<PendingExpectedValue> PrepareDeleteRange(int cf,
|
|
|
|
int64_t begin_key,
|
|
|
|
int64_t end_key) {
|
|
|
|
return expected_state_manager_->PrepareDeleteRange(cf, begin_key, end_key);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
2022-05-02 20:25:45 +00:00
|
|
|
bool AllowsOverwrite(int64_t key) const {
|
2019-12-09 07:49:32 +00:00
|
|
|
return no_overwrite_ids_.find(key) == no_overwrite_ids_.end();
|
|
|
|
}
|
|
|
|
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
|
|
// delete to the same `key`.
|
2019-12-09 07:49:32 +00:00
|
|
|
bool Exists(int cf, int64_t key) {
|
2021-09-28 21:12:23 +00:00
|
|
|
return expected_state_manager_->Exists(cf, key);
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
Support parallel read and write/delete to same key in NonBatchedOpsStressTest (#11058)
Summary:
**Context:**
Current `NonBatchedOpsStressTest` does not allow multi-thread read (i.e, Get, Iterator) and write (i.e, Put, Merge) or delete to the same key. Every read or write/delete operation will acquire lock (`GetLocksForKeyRange`) on the target key to gain exclusive access to it. This does not align with RocksDB's nature of allowing multi-thread read and write/delete to the same key, that is concurrent threads can issue read/write/delete to RocksDB without external locking. Therefore this is a gap in our testing coverage.
To close the gap, biggest challenge remains in verifying db value against expected state in presence of parallel read and write/delete. The challenge is due to read/write/delete to the db and read/write to expected state is not within one atomic operation. Therefore we may not know the exact expected state of a certain db read, as by the time we read the expected state for that db read, another write to expected state for another db write to the same key might have changed the expected state.
**Summary:**
Credited to ajkr's idea, we now solve this challenge by breaking the 32-bits expected value of a key into different parts that can be read and write to in parallel.
Basically we divide the 32-bits expected value into `value_base` (corresponding to the previous whole 32 bits but now with some shrinking in the value base range we allow), `pending_write` (i.e, whether there is an ongoing concurrent write), `del_counter` (i.e, number of times a value has been deleted, analogous to value_base for write), `pending_delete` (similar to pending_write) and `deleted` (i.e whether a key is deleted).
Also, we need to use incremental `value_base` instead of random value base as before because we want to control the range of value base a correct db read result can possibly be in presence of parallel read and write. In that way, we can verify the correctness of the read against expected state more easily. This is at the cost of reducing the randomness of the value generated in NonBatchedOpsStressTest we are willing to accept.
(For detailed algorithm of how to use these parts to infer expected state of a key, see the PR)
Misc: hide value_base detail from callers of ExpectedState by abstracting related logics into ExpectedValue class
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11058
Test Plan:
- Manual test of small number of keys (i.e, high chances of parallel read and write/delete to same key) with equally distributed read/write/deleted for 30 min
```
python3 tools/db_crashtest.py --simple {blackbox|whitebox} --sync_fault_injection=1 --skip_verifydb=0 --continuous_verification_interval=1000 --clear_column_family_one_in=0 --max_key=10 --column_families=1 --threads=32 --readpercent=25 --writepercent=25 --nooverwritepercent=0 --iterpercent=25 --verify_iterator_with_expected_state_one_in=1 --num_iterations=5 --delpercent=15 --delrangepercent=10 --range_deletion_width=5 --use_merge={0|1} --use_put_entity_one_in=0 --use_txn=0 --verify_before_write=0 --user_timestamp_size=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --flush_one_in=1000 --get_property_one_in=1000 --ingest_external_file_one_in=100 --backup_one_in=100 --checkpoint_one_in=100 --approximate_size_one_in=0 --acquire_snapshot_one_in=100 --use_multiget=0 --prefixpercent=0 --get_live_files_one_in=1000 --manual_wal_flush_one_in=1000 --pause_background_one_in=1000 --target_file_size_base=524288 --write_buffer_size=524288 --verify_checksum_one_in=1000 --verify_db_one_in=1000
```
- Rehearsal stress test for normal parameter and aggressive parameter to see if such change can find what existing stress test can find (i.e, no regression in testing capability)
- [Ongoing]Try to find new bugs with this change that are not found by current NonBatchedOpsStressTest with no parallel read and write/delete to same key
Reviewed By: ajkr
Differential Revision: D42257258
Pulled By: hx235
fbshipit-source-id: e6fdc18f1fad3753e5ac91731483a644d9b5b6eb
2023-05-15 22:34:22 +00:00
|
|
|
// Sync the `value_base` to the corresponding expected value
|
|
|
|
void SyncPut(int cf, int64_t key, uint32_t value_base) {
|
|
|
|
return expected_state_manager_->SyncPut(cf, key, value_base);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Sync the corresponding expected value to be pending Put
|
|
|
|
void SyncPendingPut(int cf, int64_t key) {
|
|
|
|
return expected_state_manager_->SyncPendingPut(cf, key);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Sync the corresponding expected value to be deleted
|
|
|
|
void SyncDelete(int cf, int64_t key) {
|
|
|
|
return expected_state_manager_->SyncDelete(cf, key);
|
|
|
|
}
|
|
|
|
|
2019-12-09 07:49:32 +00:00
|
|
|
uint32_t GetSeed() const { return seed_; }
|
|
|
|
|
|
|
|
void SetShouldStopBgThread() { should_stop_bg_thread_ = true; }
|
|
|
|
|
2019-12-20 16:46:52 +00:00
|
|
|
bool ShouldStopBgThread() { return should_stop_bg_thread_; }
|
2019-12-09 07:49:32 +00:00
|
|
|
|
2021-12-18 01:30:45 +00:00
|
|
|
void IncBgThreads() { ++num_bg_threads_; }
|
|
|
|
|
2019-12-20 16:46:52 +00:00
|
|
|
void IncBgThreadsFinished() { ++bg_thread_finished_; }
|
2019-12-09 07:49:32 +00:00
|
|
|
|
2019-12-20 16:46:52 +00:00
|
|
|
bool BgThreadsFinished() const {
|
|
|
|
return bg_thread_finished_ == num_bg_threads_;
|
|
|
|
}
|
2019-12-09 07:49:32 +00:00
|
|
|
|
|
|
|
bool ShouldVerifyAtBeginning() const {
|
2021-09-28 21:12:23 +00:00
|
|
|
return !FLAGS_expected_values_dir.empty();
|
2019-12-09 07:49:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool PrintingVerificationResults() {
|
|
|
|
bool tmp = false;
|
|
|
|
return !printing_verification_results_.compare_exchange_strong(
|
|
|
|
tmp, true, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
|
|
|
void FinishPrintingVerificationResults() {
|
|
|
|
printing_verification_results_.store(false, std::memory_order_relaxed);
|
|
|
|
}
|
|
|
|
|
2022-07-05 20:30:15 +00:00
|
|
|
uint64_t GetStartTimestamp() const { return start_timestamp_; }
|
|
|
|
|
Allow TryAgain in db_stress with optimistic txn, and refactoring (#11653)
Summary:
In rare cases, optimistic transaction commit returns TryAgain. This change tolerates that intentional behavior in db_stress, up to a small limit in a row. This way, we don't miss a possible regression with excessive TryAgain, and trying again (rolling back the transaction) should have a well renewed chance of success as the writes will be associated with fresh sequence numbers.
Also, some of the APIs were not clear about Transaction semantics, so I have clarified:
* (Best I can tell....) Destroying a Transaction is safe without calling Rollback() (or at least should be). I don't know why it's a common pattern in our test code and examples to rollback before unconditional destruction. Stress test updated not to call Rollback unnecessarily (to test safe destruction).
* Despite essentially doing what is asked, simply trying Commit() again when it returns TryAgain does not have a chance of success, because of the transaction being bound to the DB state at the time of operations before Commit. Similar logic applies to Busy AFAIK. Commit() API comments updated, and expanded unit test in optimistic_transaction_test.
Also also, because I can't stop myself, I refactored a good portion of the transaction handling code in db_stress.
* Avoid existing and new copy-paste for most transaction interactions with a new ExecuteTransaction (higher-order) function.
* Use unique_ptr (nicely complements removing unnecessary Rollbacks)
* Abstract out a pattern for safely calling std::terminate() and use it in more places. (The TryAgain errors we saw did not have stack traces because of "terminate called recursively".)
Intended follow-up: resurrect use of `FLAGS_rollback_one_in` but also include non-trivial cases
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11653
Test Plan:
this is the test :)
Also, temporarily bypassed the new retry logic and boosted the chance of hitting TryAgain. Quickly reproduced the TryAgain error. Then re-enabled the new retry logic, and was not able to hit the error after running for tens of minutes, even with the boosted chances.
Reviewed By: cbi42
Differential Revision: D47882995
Pulled By: pdillinger
fbshipit-source-id: 21eadb1525423340dbf28d17cf166b9583311a0d
2023-07-28 23:25:29 +00:00
|
|
|
void SafeTerminate() {
|
|
|
|
// Grab mutex so that we don't call terminate while another thread is
|
|
|
|
// attempting to print a stack trace due to the first one
|
|
|
|
MutexLock l(&mu_);
|
|
|
|
std::terminate();
|
|
|
|
}
|
|
|
|
|
2019-12-09 07:49:32 +00:00
|
|
|
private:
|
2022-10-25 21:29:41 +00:00
|
|
|
static void IgnoreReadErrorCallback(void*) { ignore_read_error = true; }
|
2020-04-11 00:18:56 +00:00
|
|
|
|
2022-05-02 20:25:45 +00:00
|
|
|
// Pick random keys in each column family that will not experience overwrite.
|
|
|
|
std::unordered_set<int64_t> GenerateNoOverwriteIds() const {
|
|
|
|
fprintf(stdout, "Choosing random keys with no overwrite\n");
|
|
|
|
// Start with the identity permutation. Subsequent iterations of
|
|
|
|
// for loop below will start with perm of previous for loop
|
|
|
|
std::vector<int64_t> permutation(max_key_);
|
|
|
|
for (int64_t i = 0; i < max_key_; ++i) {
|
|
|
|
permutation[i] = i;
|
|
|
|
}
|
|
|
|
// Now do the Knuth shuffle
|
|
|
|
const int64_t num_no_overwrite_keys =
|
|
|
|
(max_key_ * FLAGS_nooverwritepercent) / 100;
|
|
|
|
// Only need to figure out first num_no_overwrite_keys of permutation
|
|
|
|
std::unordered_set<int64_t> ret;
|
|
|
|
ret.reserve(num_no_overwrite_keys);
|
|
|
|
Random64 rnd(seed_);
|
|
|
|
for (int64_t i = 0; i < num_no_overwrite_keys; i++) {
|
|
|
|
assert(i < max_key_);
|
|
|
|
int64_t rand_index = i + rnd.Next() % (max_key_ - i);
|
|
|
|
// Swap i and rand_index;
|
|
|
|
int64_t temp = permutation[i];
|
|
|
|
permutation[i] = permutation[rand_index];
|
|
|
|
permutation[rand_index] = temp;
|
|
|
|
// Fill no_overwrite_ids_ with the first num_no_overwrite_keys of
|
|
|
|
// permutation
|
|
|
|
ret.insert(permutation[i]);
|
|
|
|
}
|
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
2019-12-09 07:49:32 +00:00
|
|
|
port::Mutex mu_;
|
|
|
|
port::CondVar cv_;
|
|
|
|
const uint32_t seed_;
|
|
|
|
const int64_t max_key_;
|
|
|
|
const uint32_t log2_keys_per_lock_;
|
2021-12-18 01:30:45 +00:00
|
|
|
int num_threads_;
|
2019-12-09 07:49:32 +00:00
|
|
|
long num_initialized_;
|
|
|
|
long num_populated_;
|
|
|
|
long vote_reopen_;
|
|
|
|
long num_done_;
|
|
|
|
bool start_;
|
|
|
|
bool start_verify_;
|
2019-12-20 16:46:52 +00:00
|
|
|
int num_bg_threads_;
|
2019-12-09 07:49:32 +00:00
|
|
|
bool should_stop_bg_thread_;
|
2019-12-20 16:46:52 +00:00
|
|
|
int bg_thread_finished_;
|
2019-12-09 07:49:32 +00:00
|
|
|
StressTest* stress_test_;
|
|
|
|
std::atomic<bool> verification_failure_;
|
2019-12-20 16:46:52 +00:00
|
|
|
std::atomic<bool> should_stop_test_;
|
2019-12-09 07:49:32 +00:00
|
|
|
|
|
|
|
// Keys that should not be overwritten
|
2022-05-02 20:25:45 +00:00
|
|
|
const std::unordered_set<int64_t> no_overwrite_ids_;
|
2019-12-09 07:49:32 +00:00
|
|
|
|
2021-09-28 21:12:23 +00:00
|
|
|
std::unique_ptr<ExpectedStateManager> expected_state_manager_;
|
2022-02-01 19:46:12 +00:00
|
|
|
// Cannot store `port::Mutex` directly in vector since it is not copyable
|
2019-12-09 07:49:32 +00:00
|
|
|
// and storing it in the container may require copying depending on the impl.
|
2022-02-01 19:46:12 +00:00
|
|
|
std::vector<std::unique_ptr<port::Mutex[]>> key_locks_;
|
2019-12-09 07:49:32 +00:00
|
|
|
std::atomic<bool> printing_verification_results_;
|
2022-07-05 20:30:15 +00:00
|
|
|
const uint64_t start_timestamp_;
|
2019-12-09 07:49:32 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
// Per-thread state for concurrent executions of the same benchmark.
|
|
|
|
struct ThreadState {
|
|
|
|
uint32_t tid; // 0..n-1
|
|
|
|
Random rand; // Has different seeds for different threads
|
|
|
|
SharedState* shared;
|
|
|
|
Stats stats;
|
|
|
|
struct SnapshotState {
|
|
|
|
const Snapshot* snapshot;
|
|
|
|
// The cf from which we did a Get at this snapshot
|
|
|
|
int cf_at;
|
|
|
|
// The name of the cf at the time that we did a read
|
|
|
|
std::string cf_at_name;
|
|
|
|
// The key with which we did a Get at this snapshot
|
|
|
|
std::string key;
|
|
|
|
// The status of the Get
|
|
|
|
Status status;
|
|
|
|
// The value of the Get
|
|
|
|
std::string value;
|
|
|
|
// optional state of all keys in the db
|
|
|
|
std::vector<bool>* key_vec;
|
Add user-defined timestamps to db_stress (#8061)
Summary:
Add some basic test for user-defined timestamp to db_stress. Currently,
read with timestamp always tries to read using the current timestamp.
Due to the per-key timestamp-sequence ordering constraint, we only add timestamp-
related tests to the `NonBatchedOpsStressTest` since this test serializes accesses
to the same key and uses a file to cross-check data correctness.
The timestamp feature is not supported in a number of components, e.g. Merge, SingleDelete,
DeleteRange, CompactionFilter, Readonly instance, secondary instance, SST file ingestion, transaction,
etc. Therefore, db_stress should exit if user enables both timestamp and these features at the same
time. The (currently) incompatible features can be found in
`CheckAndSetOptionsForUserTimestamp`.
This PR also fixes a bug triggered when timestamp is enabled together with
`index_type=kBinarySearchWithFirstKey`. This bug fix will also be in another separate PR
with more unit tests coverage. Fixing it here because I do not want to exclude the index type
from crash test.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8061
Test Plan: make crash_test_with_ts
Reviewed By: jay-zhuang
Differential Revision: D27056282
Pulled By: riversand963
fbshipit-source-id: c3e00ad1023fdb9ebbdf9601ec18270c5e2925a9
2021-03-23 12:12:04 +00:00
|
|
|
|
|
|
|
std::string timestamp;
|
2019-12-09 07:49:32 +00:00
|
|
|
};
|
|
|
|
std::queue<std::pair<uint64_t, SnapshotState>> snapshot_queue;
|
|
|
|
|
|
|
|
ThreadState(uint32_t index, SharedState* _shared)
|
|
|
|
: tid(index), rand(1000 + index + _shared->GetSeed()), shared(_shared) {}
|
|
|
|
};
|
2020-02-20 20:07:53 +00:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|
2019-12-09 07:49:32 +00:00
|
|
|
#endif // GFLAGS
|