mirror of
https://github.com/facebook/rocksdb.git
synced 2024-11-28 15:33:54 +00:00
408e8d4c85
Summary: **Context/Summary:** We discovered the following false positive in our crash test lately: (1) PUT() writes k/v to WAL but fails in `ApplyWALToManifest()`. The k/v is in the WAL (2) Current stress test logic will rollback the expected state of such k/v since PUT() fails (3) If the DB crashes before recovery finishes and reopens, the WAL will be replayed and the k/v is in the DB while the expected state have been roll-backed. We decided to leave those expected state to be pending until the loop-write of the same key succeeds. Bonus: Now that I realized write to manifest can also fail the write which faces the similar problem as https://github.com/facebook/rocksdb/pull/12797, I decided to disable fault injection on user write per thread (instead of globally) when tracing is needed for prefix recovery; some refactory Pull Request resolved: https://github.com/facebook/rocksdb/pull/12838 Test Plan: Rehearsal CI Run below command (varies on sync_fault_injection=1,0 to verify ExpectedState behavior) for a while to ensure crash recovery validation works fine ``` python3 tools/db_crashtest.py --simple blackbox --interval=30 --WAL_size_limit_MB=0 --WAL_ttl_seconds=0 --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --adm_policy=1 --advise_random_on_open=0 --allow_concurrent_memtable_write=0 --allow_data_in_errors=True --allow_fallocate=0 --async_io=0 --auto_readahead_size=0 --avoid_flush_during_recovery=0 --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=1 --block_protection_bytes_per_key=4 --block_size=16384 --bloom_before_level=4 --bloom_bits=56.810257702625165 --bottommost_compression_type=none --bottommost_file_compaction_delay=0 --bytes_per_sync=262144 --cache_index_and_filter_blocks=1 --cache_index_and_filter_blocks_with_high_priority=1 --cache_size=8388608 --cache_type=auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=1 --charge_filter_construction=1 --charge_table_reader=0 --check_multiget_consistency=0 --check_multiget_entity_consistency=1 --checkpoint_one_in=10000 --checksum_type=kxxHash --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000 --compact_range_one_in=1000 --compaction_pri=4 --compaction_readahead_size=1048576 --compaction_ttl=10 --compress_format_version=1 --compressed_secondary_cache_ratio=0.0 --compressed_secondary_cache_size=0 --compression_checksum=0 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=none --compression_use_zstd_dict_trainer=0 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --daily_offpeak_time_utc=04:00-08:00 --data_block_index_type=1 --db_write_buffer_size=0 --default_temperature=kWarm --default_write_temperature=kCold --delete_obsolete_files_period_micros=30000000 --delpercent=20 --delrangepercent=20 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_file_deletions_one_in=10000 --disable_manual_compaction_one_in=1000000 --disable_wal=0 --dump_malloc_stats=0 --enable_checksum_handoff=1 --enable_compaction_filter=0 --enable_custom_split_merge=0 --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=0 --enable_thread_tracking=0 --enable_write_thread_adaptive_yield=0 --error_recovery_with_no_fault_injection=1 --exclude_wal_from_write_fault_injection=0 --fail_if_options_file_error=1 --fifo_allow_compaction=0 --file_checksum_impl=crc32c --fill_cache=1 --flush_one_in=1000000 --format_version=3 --get_all_column_family_metadata_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_apis_one_in=1000000 --get_properties_of_all_tables_one_in=1000000 --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=4 --index_shortening=2 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=16384 --inplace_update_support=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --key_may_exist_one_in=100 --last_level_temperature=kWarm --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=10000 --log_file_time_to_roll=60 --log_readahead_size=16777216 --long_running_snapshots=1 --low_pri_pool_ratio=0 --lowest_used_cache_tier=0 --manifest_preallocation_size=0 --manual_wal_flush_one_in=0 --mark_for_compaction_one_file_in=10 --max_auto_readahead_size=16384 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=100000 --max_key_len=3 --max_log_file_size=1048576 --max_manifest_file_size=32768 --max_sequential_skip_in_iterations=1 --max_total_wal_size=0 --max_write_batch_group_size_bytes=16 --max_write_buffer_number=10 --max_write_buffer_size_to_maintain=8388608 --memtable_insert_hint_per_batch=1 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.01 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=1 --memtablerep=skip_list --metadata_charge_policy=1 --metadata_read_fault_one_in=0 --metadata_write_fault_one_in=8 --min_write_buffer_number_to_merge=1 --mmap_read=1 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --open_files=-1 --open_metadata_read_fault_one_in=0 --open_metadata_write_fault_one_in=8 --open_read_fault_one_in=0 --open_write_fault_one_in=8 --ops_per_thread=100000000 --optimize_filters_for_hits=1 --optimize_filters_for_memory=1 --optimize_multiget_for_io=1 --paranoid_file_checks=0 --partition_filters=0 --partition_pinning=3 --pause_background_one_in=1000000 --periodic_compaction_seconds=2 --prefix_size=7 --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=1000 --readahead_size=524288 --readpercent=10 --recycle_log_file_num=1 --reopen=0 --report_bg_io_stats=0 --reset_stats_one_in=1000000 --sample_for_compression=0 --secondary_cache_fault_one_in=0 --set_options_one_in=0 --skip_stats_update_on_db_open=1 --snapshot_hold_ops=100000 --soft_pending_compaction_bytes_limit=68719476736 --sqfc_name=foo --sqfc_version=0 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=10 --stats_history_buffer_size=0 --strict_bytes_per_sync=1 --subcompactions=4 --sync=1 --sync_fault_injection=0 --table_cache_numshardbits=6 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=2 --uncache_aggressiveness=239 --universal_max_read_amp=-1 --unpartitioned_pinning=1 --use_adaptive_mutex=1 --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=1 --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=1000000 --verify_compression=0 --verify_db_one_in=100000 --verify_file_checksums_one_in=1000000 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=0 --wal_compression=none --write_buffer_size=33554432 --write_dbid_to_manifest=0 --write_fault_one_in=8 --writepercent=40 ``` Reviewed By: cbi42 Differential Revision: D59377075 Pulled By: hx235 fbshipit-source-id: 91f602fd67e2d339d378cd28b982095fd073dcb6
332 lines
12 KiB
C++
332 lines
12 KiB
C++
// Copyright (c) 2021-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).
|
|
|
|
#ifdef GFLAGS
|
|
|
|
#pragma once
|
|
|
|
#include <stdint.h>
|
|
|
|
#include <atomic>
|
|
#include <memory>
|
|
|
|
#include "db/dbformat.h"
|
|
#include "db_stress_tool/expected_value.h"
|
|
#include "file/file_util.h"
|
|
#include "rocksdb/db.h"
|
|
#include "rocksdb/env.h"
|
|
#include "rocksdb/file_system.h"
|
|
#include "rocksdb/rocksdb_namespace.h"
|
|
#include "rocksdb/types.h"
|
|
#include "util/string_util.h"
|
|
|
|
namespace ROCKSDB_NAMESPACE {
|
|
// `ExpectedState` provides read/write access to expected values stored in
|
|
// `ExpectedState` for every key.
|
|
class ExpectedState {
|
|
public:
|
|
explicit ExpectedState(size_t max_key, size_t num_column_families);
|
|
|
|
virtual ~ExpectedState() {}
|
|
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
virtual Status Open(bool create) = 0;
|
|
|
|
// Requires external locking covering all keys in `cf`.
|
|
void ClearColumnFamily(int cf);
|
|
|
|
// Prepare a Put that will be started but not finished yet
|
|
// This is useful for crash-recovery testing when the process may crash
|
|
// before updating the corresponding expected value
|
|
//
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
// write or delete to the same `key`.
|
|
PendingExpectedValue PreparePut(int cf, int64_t key, bool* prepared);
|
|
|
|
// Does not requires external locking.
|
|
ExpectedValue Get(int cf, int64_t key);
|
|
|
|
// Prepare a Delete that will be started but not finished yet.
|
|
// This is useful for crash-recovery testing when the process may crash
|
|
// before updating the corresponding expected value
|
|
//
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
// write or delete to the same `key`.
|
|
PendingExpectedValue PrepareDelete(int cf, int64_t key, bool* prepared);
|
|
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
// write or delete to the same `key`.
|
|
PendingExpectedValue PrepareSingleDelete(int cf, int64_t key, bool* prepared);
|
|
|
|
// 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,
|
|
bool* prepared);
|
|
|
|
// Update the expected value for start of an incomplete write or delete
|
|
// operation on the key assoicated with this expected value
|
|
void Precommit(int cf, int64_t key, const ExpectedValue& value);
|
|
|
|
// Requires external locking covering `key` in `cf` to prevent concurrent
|
|
// delete to the same `key`.
|
|
bool Exists(int cf, int64_t key);
|
|
|
|
// Sync the `value_base` to the corresponding expected value
|
|
//
|
|
// Requires external locking covering `key` in `cf` or be in single thread
|
|
// to prevent concurrent write or delete to the same `key`
|
|
void SyncPut(int cf, int64_t key, uint32_t value_base);
|
|
|
|
// Sync the corresponding expected value to be pending Put
|
|
//
|
|
// Requires external locking covering `key` in `cf` or be in single thread
|
|
// to prevent concurrent write or delete to the same `key`
|
|
void SyncPendingPut(int cf, int64_t key);
|
|
|
|
// Sync the corresponding expected value to be deleted
|
|
//
|
|
// Requires external locking covering `key` in `cf` or be in single thread
|
|
// to prevent concurrent write or delete to the same `key`
|
|
void SyncDelete(int cf, int64_t key);
|
|
|
|
// Sync the corresponding expected values to be deleted
|
|
//
|
|
// Requires external locking covering keys in `[begin_key, end_key)` in `cf`
|
|
// to prevent concurrent write or delete to the same `key`
|
|
void SyncDeleteRange(int cf, int64_t begin_key, int64_t end_key);
|
|
|
|
private:
|
|
// Does not requires external locking.
|
|
std::atomic<uint32_t>& Value(int cf, int64_t key) const {
|
|
return values_[cf * max_key_ + key];
|
|
}
|
|
|
|
// Does not requires external locking
|
|
ExpectedValue Load(int cf, int64_t key) const {
|
|
return ExpectedValue(Value(cf, key).load());
|
|
}
|
|
|
|
const size_t max_key_;
|
|
const size_t num_column_families_;
|
|
|
|
protected:
|
|
size_t GetValuesLen() const {
|
|
return sizeof(std::atomic<uint32_t>) * num_column_families_ * max_key_;
|
|
}
|
|
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
void Reset();
|
|
|
|
std::atomic<uint32_t>* values_;
|
|
};
|
|
|
|
// A `FileExpectedState` implements `ExpectedState` backed by a file.
|
|
class FileExpectedState : public ExpectedState {
|
|
public:
|
|
explicit FileExpectedState(std::string expected_state_file_path,
|
|
size_t max_key, size_t num_column_families);
|
|
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
Status Open(bool create) override;
|
|
|
|
private:
|
|
const std::string expected_state_file_path_;
|
|
std::unique_ptr<MemoryMappedFileBuffer> expected_state_mmap_buffer_;
|
|
};
|
|
|
|
// An `AnonExpectedState` implements `ExpectedState` backed by a memory
|
|
// allocation.
|
|
class AnonExpectedState : public ExpectedState {
|
|
public:
|
|
explicit AnonExpectedState(size_t max_key, size_t num_column_families);
|
|
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
Status Open(bool create) override;
|
|
|
|
private:
|
|
std::unique_ptr<std::atomic<uint32_t>[]> values_allocation_;
|
|
};
|
|
|
|
// An `ExpectedStateManager` manages data about the expected state of the
|
|
// database. It exposes operations for reading and modifying the latest
|
|
// expected state.
|
|
class ExpectedStateManager {
|
|
public:
|
|
explicit ExpectedStateManager(size_t max_key, size_t num_column_families);
|
|
|
|
virtual ~ExpectedStateManager();
|
|
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
virtual Status Open() = 0;
|
|
|
|
// Saves expected values for the current state of `db` and begins tracking
|
|
// changes. Following a successful `SaveAtAndAfter()`, `Restore()` can be
|
|
// called on the same DB, as long as its state does not roll back to before
|
|
// its current state.
|
|
//
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function. Furthermore, `db` must not be mutated while this function
|
|
// is executing.
|
|
virtual Status SaveAtAndAfter(DB* db) = 0;
|
|
|
|
// Returns true if at least one state of historical expected values can be
|
|
// restored.
|
|
//
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
virtual bool HasHistory() = 0;
|
|
|
|
// Restores expected values according to the current state of `db`. See
|
|
// `SaveAtAndAfter()` for conditions where this can be called.
|
|
//
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function. Furthermore, `db` must not be mutated while this function
|
|
// is executing.
|
|
virtual Status Restore(DB* db) = 0;
|
|
|
|
// Requires external locking covering all keys in `cf`.
|
|
void ClearColumnFamily(int cf) { return latest_->ClearColumnFamily(cf); }
|
|
|
|
// See ExpectedState::PreparePut()
|
|
PendingExpectedValue PreparePut(int cf, int64_t key, bool* prepared) {
|
|
return latest_->PreparePut(cf, key, prepared);
|
|
}
|
|
|
|
// See ExpectedState::Get()
|
|
ExpectedValue Get(int cf, int64_t key) { return latest_->Get(cf, key); }
|
|
|
|
// See ExpectedState::PrepareDelete()
|
|
PendingExpectedValue PrepareDelete(int cf, int64_t key, bool* prepared) {
|
|
return latest_->PrepareDelete(cf, key, prepared);
|
|
}
|
|
|
|
// See ExpectedState::PrepareSingleDelete()
|
|
PendingExpectedValue PrepareSingleDelete(int cf, int64_t key,
|
|
bool* prepared) {
|
|
return latest_->PrepareSingleDelete(cf, key, prepared);
|
|
}
|
|
|
|
// See ExpectedState::PrepareDeleteRange()
|
|
std::vector<PendingExpectedValue> PrepareDeleteRange(int cf,
|
|
int64_t begin_key,
|
|
int64_t end_key,
|
|
bool* prepared) {
|
|
return latest_->PrepareDeleteRange(cf, begin_key, end_key, prepared);
|
|
}
|
|
|
|
// See ExpectedState::Exists()
|
|
bool Exists(int cf, int64_t key) { return latest_->Exists(cf, key); }
|
|
|
|
// See ExpectedState::SyncPut()
|
|
void SyncPut(int cf, int64_t key, uint32_t value_base) {
|
|
return latest_->SyncPut(cf, key, value_base);
|
|
}
|
|
|
|
// See ExpectedState::SyncPendingPut()
|
|
void SyncPendingPut(int cf, int64_t key) {
|
|
return latest_->SyncPendingPut(cf, key);
|
|
}
|
|
|
|
// See ExpectedState::SyncDelete()
|
|
void SyncDelete(int cf, int64_t key) { return latest_->SyncDelete(cf, key); }
|
|
|
|
// See ExpectedState::SyncDeleteRange()
|
|
void SyncDeleteRange(int cf, int64_t begin_key, int64_t end_key) {
|
|
return latest_->SyncDeleteRange(cf, begin_key, end_key);
|
|
}
|
|
|
|
protected:
|
|
const size_t max_key_;
|
|
const size_t num_column_families_;
|
|
std::unique_ptr<ExpectedState> latest_;
|
|
};
|
|
|
|
// A `FileExpectedStateManager` implements an `ExpectedStateManager` backed by
|
|
// a directory of files containing data about the expected state of the
|
|
// database.
|
|
class FileExpectedStateManager : public ExpectedStateManager {
|
|
public:
|
|
explicit FileExpectedStateManager(size_t max_key, size_t num_column_families,
|
|
std::string expected_state_dir_path);
|
|
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
Status Open() override;
|
|
|
|
// See `ExpectedStateManager::SaveAtAndAfter()` API doc.
|
|
//
|
|
// This implementation makes a copy of "LATEST.state" into
|
|
// "<current seqno>.state", and starts a trace in "<current seqno>.trace".
|
|
// Due to using external files, a following `Restore()` can happen even
|
|
// from a different process.
|
|
Status SaveAtAndAfter(DB* db) override;
|
|
|
|
// See `ExpectedStateManager::HasHistory()` API doc.
|
|
bool HasHistory() override;
|
|
|
|
// See `ExpectedStateManager::Restore()` API doc.
|
|
//
|
|
// Say `db->GetLatestSequenceNumber()` was `a` last time `SaveAtAndAfter()`
|
|
// was called and now it is `b`. Then this function replays `b - a` write
|
|
// operations from "`a`.trace" onto "`a`.state", and then copies the resulting
|
|
// file into "LATEST.state".
|
|
Status Restore(DB* db) override;
|
|
|
|
private:
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
Status Clean();
|
|
|
|
std::string GetTempPathForFilename(const std::string& filename);
|
|
std::string GetPathForFilename(const std::string& filename);
|
|
|
|
static const std::string kLatestBasename;
|
|
static const std::string kStateFilenameSuffix;
|
|
static const std::string kTraceFilenameSuffix;
|
|
static const std::string kTempFilenamePrefix;
|
|
static const std::string kTempFilenameSuffix;
|
|
|
|
const std::string expected_state_dir_path_;
|
|
SequenceNumber saved_seqno_ = kMaxSequenceNumber;
|
|
};
|
|
|
|
// An `AnonExpectedStateManager` implements an `ExpectedStateManager` backed by
|
|
// a memory allocation containing data about the expected state of the database.
|
|
class AnonExpectedStateManager : public ExpectedStateManager {
|
|
public:
|
|
explicit AnonExpectedStateManager(size_t max_key, size_t num_column_families);
|
|
|
|
// See `ExpectedStateManager::SaveAtAndAfter()` API doc.
|
|
//
|
|
// This implementation returns `Status::NotSupported` since we do not
|
|
// currently have a need to keep history of expected state within a process.
|
|
Status SaveAtAndAfter(DB* /* db */) override {
|
|
return Status::NotSupported();
|
|
}
|
|
|
|
// See `ExpectedStateManager::HasHistory()` API doc.
|
|
bool HasHistory() override { return false; }
|
|
|
|
// See `ExpectedStateManager::Restore()` API doc.
|
|
//
|
|
// This implementation returns `Status::NotSupported` since we do not
|
|
// currently have a need to keep history of expected state within a process.
|
|
Status Restore(DB* /* db */) override { return Status::NotSupported(); }
|
|
|
|
// Requires external locking preventing concurrent execution with any other
|
|
// member function.
|
|
Status Open() override;
|
|
};
|
|
} // namespace ROCKSDB_NAMESPACE
|
|
|
|
#endif // GFLAGS
|