mirror of
https://github.com/facebook/rocksdb.git
synced 2024-11-27 20:43:57 +00:00
5e0584bd73
Summary: Currently the stress test does not support restoring expected state (to a specific sequence number) when there is unsynced data loss during the reopen phase. This causes a few internal stress test failure with errors like inconsistent value. This PR disables dropping unsynced data during reopen to avoid failures due to this issue. We can re-enable later after we decide to support unsynced data loss during DB reopen in stress test. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11731 Test Plan: * Running this test a few times can fail for inconsistent value before this change ``` ./db_stress --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --allow_concurrent_memtable_write=1 --allow_data_in_errors=True --async_io=0 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=0 --batch_protection_bytes_per_key=0 --block_protection_bytes_per_key=8 --block_size=16384 --bloom_bits=20.57166126835524 --bottommost_compression_type=disable --bytes_per_sync=262144 --cache_index_and_filter_blocks=1 --cache_size=8388608 --cache_type=auto_hyper_clock_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=1 --charge_filter_construction=0 --charge_table_reader=1 --checkpoint_one_in=0 --checksum_type=kxxHash --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000000 --compact_range_one_in=1000000 --compaction_pri=3 --compaction_style=1 --compaction_ttl=100 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=zstd --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --data_block_index_type=0 --db=/dev/shm/rocksdb_test/rocksdb_crashtest_whitebox --db_write_buffer_size=0 --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_wal=0 --enable_compaction_filter=0 --enable_pipelined_write=1 --enable_thread_tracking=0 --expected_values_dir=/dev/shm/rocksdb_test/rocksdb_crashtest_expected --fail_if_options_file_error=1 --fifo_allow_compaction=1 --file_checksum_impl=big --flush_one_in=1000000 --format_version=3 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=6 --index_type=3 --ingest_external_file_one_in=0 --initial_auto_readahead_size=16384 --iterpercent=10 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=1 --lock_wal_one_in=1000000 --log2_keys_per_lock=10 --long_running_snapshots=1 --manual_wal_flush_one_in=1000000 --mark_for_compaction_one_file_in=10 --max_auto_readahead_size=0 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=25000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=16777216 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_max_range_deletions=100 --memtable_prefix_bloom_size_ratio=0 --memtable_protection_bytes_per_key=1 --memtable_whole_key_filtering=0 --memtablerep=skip_list --min_write_buffer_number_to_merge=2 --mmap_read=0 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=1 --open_files=-1 --open_metadata_write_fault_one_in=0 --open_read_fault_one_in=5 --open_write_fault_one_in=0 --ops_per_thread=200000 --optimize_filters_for_memory=0 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=3 --pause_background_one_in=1000000 --periodic_compaction_seconds=10 --prefix_size=-1 --prefixpercent=0 --prepopulate_block_cache=1 --preserve_internal_time_seconds=0 --progress_reports=0 --read_fault_one_in=1000 --readahead_size=524288 --readpercent=50 --recycle_log_file_num=0 --reopen=20 --ribbon_starting_level=0 --secondary_cache_fault_one_in=32 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=0 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=10 --subcompactions=3 --sync=0 --sync_fault_injection=1 --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=2 --unpartitioned_pinning=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=1 --use_merge=0 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=1 --user_timestamp_size=0 --value_size_mult=32 --verify_checksum=1 --verify_checksum_one_in=1000000 --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=524288 --wal_compression=zstd --write_buffer_size=33554432 --write_dbid_to_manifest=1 --writepercent=35``` Reviewed By: hx235 Differential Revision: D48537494 Pulled By: cbi42 fbshipit-source-id: ddae21b9bb6ee8d67229121f58513e95f7ef6d8d
333 lines
14 KiB
C++
333 lines
14 KiB
C++
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
|
// This source code is licensed under both the GPLv2 (found in the
|
|
// COPYING file in the root directory) and Apache 2.0 License
|
|
// (found in the LICENSE.Apache file in the root directory).
|
|
//
|
|
// Copyright (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_common.h"
|
|
#include "db_stress_tool/db_stress_shared_state.h"
|
|
|
|
namespace ROCKSDB_NAMESPACE {
|
|
class SystemClock;
|
|
class Transaction;
|
|
class TransactionDB;
|
|
class OptimisticTransactionDB;
|
|
struct TransactionDBOptions;
|
|
|
|
class StressTest {
|
|
public:
|
|
StressTest();
|
|
|
|
virtual ~StressTest();
|
|
|
|
std::shared_ptr<Cache> NewCache(size_t capacity, int32_t num_shard_bits);
|
|
|
|
static std::vector<std::string> GetBlobCompressionTags();
|
|
|
|
bool BuildOptionsTable();
|
|
|
|
void InitDb(SharedState*);
|
|
// The initialization work is split into two parts to avoid a circular
|
|
// dependency with `SharedState`.
|
|
virtual void FinishInitDb(SharedState*);
|
|
void TrackExpectedState(SharedState* shared);
|
|
void OperateDb(ThreadState* thread);
|
|
virtual void VerifyDb(ThreadState* thread) const = 0;
|
|
virtual void ContinuouslyVerifyDb(ThreadState* /*thread*/) const = 0;
|
|
void PrintStatistics();
|
|
|
|
protected:
|
|
Status AssertSame(DB* db, ColumnFamilyHandle* cf,
|
|
ThreadState::SnapshotState& snap_state);
|
|
|
|
// Currently PreloadDb has to be single-threaded.
|
|
void PreloadDbAndReopenAsReadOnly(int64_t number_of_keys,
|
|
SharedState* shared);
|
|
|
|
Status SetOptions(ThreadState* thread);
|
|
|
|
// For transactionsDB, there can be txns prepared but not yet committeed
|
|
// right before previous stress run crash.
|
|
// They will be recovered and processed through
|
|
// ProcessRecoveredPreparedTxnsHelper on the start of current stress run.
|
|
void ProcessRecoveredPreparedTxns(SharedState* shared);
|
|
|
|
// Default implementation will first update ExpectedState to be
|
|
// `SharedState::UNKNOWN` for each keys in `txn` and then randomly
|
|
// commit or rollback `txn`.
|
|
virtual void ProcessRecoveredPreparedTxnsHelper(Transaction* txn,
|
|
SharedState* shared);
|
|
|
|
// ExecuteTransaction is recommended instead
|
|
Status NewTxn(WriteOptions& write_opts,
|
|
std::unique_ptr<Transaction>* out_txn);
|
|
Status CommitTxn(Transaction& txn, ThreadState* thread = nullptr);
|
|
|
|
// Creates a transaction, executes `ops`, and tries to commit
|
|
Status ExecuteTransaction(WriteOptions& write_opts, ThreadState* thread,
|
|
std::function<Status(Transaction&)>&& ops);
|
|
|
|
virtual void MaybeClearOneColumnFamily(ThreadState* /* thread */) {}
|
|
|
|
virtual bool ShouldAcquireMutexOnKey() const { return false; }
|
|
|
|
// Returns true if DB state is tracked by the stress test.
|
|
virtual bool IsStateTracked() const = 0;
|
|
|
|
virtual std::vector<int> GenerateColumnFamilies(
|
|
const int /* num_column_families */, int rand_column_family) const {
|
|
return {rand_column_family};
|
|
}
|
|
|
|
virtual std::vector<int64_t> GenerateKeys(int64_t rand_key) const {
|
|
return {rand_key};
|
|
}
|
|
|
|
virtual Status TestGet(ThreadState* thread, const ReadOptions& read_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
virtual std::vector<Status> TestMultiGet(
|
|
ThreadState* thread, const ReadOptions& read_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
virtual void TestGetEntity(ThreadState* thread, const ReadOptions& read_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
virtual void TestMultiGetEntity(ThreadState* thread,
|
|
const ReadOptions& read_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
virtual Status TestPrefixScan(ThreadState* thread,
|
|
const ReadOptions& read_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
virtual Status TestPut(ThreadState* thread, WriteOptions& write_opts,
|
|
const ReadOptions& read_opts,
|
|
const std::vector<int>& cf_ids,
|
|
const std::vector<int64_t>& keys,
|
|
char (&value)[100]) = 0;
|
|
|
|
virtual Status TestDelete(ThreadState* thread, WriteOptions& write_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
virtual Status TestDeleteRange(ThreadState* thread, WriteOptions& write_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
virtual void TestIngestExternalFile(
|
|
ThreadState* thread, const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys) = 0;
|
|
|
|
// Issue compact range, starting with start_key, whose integer value
|
|
// is rand_key.
|
|
virtual void TestCompactRange(ThreadState* thread, int64_t rand_key,
|
|
const Slice& start_key,
|
|
ColumnFamilyHandle* column_family);
|
|
|
|
// Calculate a hash value for all keys in range [start_key, end_key]
|
|
// at a certain snapshot.
|
|
uint32_t GetRangeHash(ThreadState* thread, const Snapshot* snapshot,
|
|
ColumnFamilyHandle* column_family,
|
|
const Slice& start_key, const Slice& end_key);
|
|
|
|
// Return a column family handle that mirrors what is pointed by
|
|
// `column_family_id`, which will be used to validate data to be correct.
|
|
// By default, the column family itself will be returned.
|
|
virtual ColumnFamilyHandle* GetControlCfh(ThreadState* /* thread*/,
|
|
int column_family_id) {
|
|
return column_families_[column_family_id];
|
|
}
|
|
|
|
// Generated a list of keys that close to boundaries of SST keys.
|
|
// If there isn't any SST file in the DB, return empty list.
|
|
std::vector<std::string> GetWhiteBoxKeys(ThreadState* thread, DB* db,
|
|
ColumnFamilyHandle* cfh,
|
|
size_t num_keys);
|
|
|
|
// Given a key K, this creates an iterator which scans to K and then
|
|
// does a random sequence of Next/Prev operations.
|
|
virtual Status TestIterate(ThreadState* thread, const ReadOptions& read_opts,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys);
|
|
|
|
virtual Status TestIterateAgainstExpected(
|
|
ThreadState* /* thread */, const ReadOptions& /* read_opts */,
|
|
const std::vector<int>& /* rand_column_families */,
|
|
const std::vector<int64_t>& /* rand_keys */) {
|
|
return Status::NotSupported();
|
|
}
|
|
|
|
// Enum used by VerifyIterator() to identify the mode to validate.
|
|
enum LastIterateOp {
|
|
kLastOpSeek,
|
|
kLastOpSeekForPrev,
|
|
kLastOpNextOrPrev,
|
|
kLastOpSeekToFirst,
|
|
kLastOpSeekToLast
|
|
};
|
|
|
|
// Compare the two iterator, iter and cmp_iter are in the same position,
|
|
// unless iter might be made invalidate or undefined because of
|
|
// upper or lower bounds, or prefix extractor.
|
|
// Will flag failure if the verification fails.
|
|
// diverged = true if the two iterator is already diverged.
|
|
// True if verification passed, false if not.
|
|
// op_logs is the information to print when validation fails.
|
|
void VerifyIterator(ThreadState* thread, ColumnFamilyHandle* cmp_cfh,
|
|
const ReadOptions& ro, Iterator* iter, Iterator* cmp_iter,
|
|
LastIterateOp op, const Slice& seek_key,
|
|
const std::string& op_logs, bool* diverged);
|
|
|
|
virtual Status TestBackupRestore(ThreadState* thread,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys);
|
|
|
|
virtual Status TestCheckpoint(ThreadState* thread,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys);
|
|
|
|
void TestCompactFiles(ThreadState* thread, ColumnFamilyHandle* column_family);
|
|
|
|
Status TestFlush(const std::vector<int>& rand_column_families);
|
|
|
|
Status TestPauseBackground(ThreadState* thread);
|
|
|
|
void TestAcquireSnapshot(ThreadState* thread, int rand_column_family,
|
|
const std::string& keystr, uint64_t i);
|
|
|
|
Status MaybeReleaseSnapshots(ThreadState* thread, uint64_t i);
|
|
Status VerifyGetLiveFiles() const;
|
|
Status VerifyGetSortedWalFiles() const;
|
|
Status VerifyGetCurrentWalFile() const;
|
|
void TestGetProperty(ThreadState* thread) const;
|
|
|
|
virtual Status TestApproximateSize(
|
|
ThreadState* thread, uint64_t iteration,
|
|
const std::vector<int>& rand_column_families,
|
|
const std::vector<int64_t>& rand_keys);
|
|
|
|
virtual Status TestCustomOperations(
|
|
ThreadState* /*thread*/,
|
|
const std::vector<int>& /*rand_column_families*/) {
|
|
return Status::NotSupported("TestCustomOperations() must be overridden");
|
|
}
|
|
|
|
void VerificationAbort(SharedState* shared, std::string msg, Status s) const;
|
|
|
|
void VerificationAbort(SharedState* shared, std::string msg, int cf,
|
|
int64_t key) const;
|
|
|
|
void VerificationAbort(SharedState* shared, std::string msg, int cf,
|
|
int64_t key, Slice value_from_db,
|
|
Slice value_from_expected) const;
|
|
|
|
void VerificationAbort(SharedState* shared, int cf, int64_t key,
|
|
const Slice& value, const WideColumns& columns) const;
|
|
|
|
static std::string DebugString(const Slice& value,
|
|
const WideColumns& columns);
|
|
|
|
void PrintEnv() const;
|
|
|
|
void Open(SharedState* shared, bool reopen = false);
|
|
|
|
void Reopen(ThreadState* thread);
|
|
|
|
virtual void RegisterAdditionalListeners() {}
|
|
|
|
virtual void PrepareTxnDbOptions(SharedState* /*shared*/,
|
|
TransactionDBOptions& /*txn_db_opts*/) {}
|
|
|
|
// Returns whether the timestamp of read_opts is updated.
|
|
bool MaybeUseOlderTimestampForPointLookup(ThreadState* thread,
|
|
std::string& ts_str,
|
|
Slice& ts_slice,
|
|
ReadOptions& read_opts);
|
|
|
|
void MaybeUseOlderTimestampForRangeScan(ThreadState* thread,
|
|
std::string& ts_str, Slice& ts_slice,
|
|
ReadOptions& read_opts);
|
|
|
|
std::shared_ptr<Cache> cache_;
|
|
std::shared_ptr<Cache> compressed_cache_;
|
|
std::shared_ptr<const FilterPolicy> filter_policy_;
|
|
DB* db_;
|
|
TransactionDB* txn_db_;
|
|
OptimisticTransactionDB* optimistic_txn_db_;
|
|
|
|
// Currently only used in MultiOpsTxnsStressTest
|
|
std::atomic<DB*> db_aptr_;
|
|
|
|
Options options_;
|
|
SystemClock* clock_;
|
|
std::vector<ColumnFamilyHandle*> column_families_;
|
|
std::vector<std::string> column_family_names_;
|
|
std::atomic<int> new_column_family_name_;
|
|
int num_times_reopened_;
|
|
std::unordered_map<std::string, std::vector<std::string>> options_table_;
|
|
std::vector<std::string> options_index_;
|
|
std::atomic<bool> db_preload_finished_;
|
|
|
|
// Fields used for continuous verification from another thread
|
|
DB* cmp_db_;
|
|
std::vector<ColumnFamilyHandle*> cmp_cfhs_;
|
|
bool is_db_stopped_;
|
|
};
|
|
|
|
// Load options from OPTIONS file and populate `options`.
|
|
extern bool InitializeOptionsFromFile(Options& options);
|
|
|
|
// Initialize `options` using command line arguments.
|
|
// When this function is called, `cache`, `block_cache_compressed`,
|
|
// `filter_policy` have all been initialized. Therefore, we just pass them as
|
|
// input arguments.
|
|
extern void InitializeOptionsFromFlags(
|
|
const std::shared_ptr<Cache>& cache,
|
|
const std::shared_ptr<const FilterPolicy>& filter_policy, Options& options);
|
|
|
|
// Initialize `options` on which `InitializeOptionsFromFile()` and
|
|
// `InitializeOptionsFromFlags()` have both been called already.
|
|
// There are two cases.
|
|
// Case 1: OPTIONS file is not specified. Command line arguments have been used
|
|
// to initialize `options`. InitializeOptionsGeneral() will use
|
|
// `cache` and `filter_policy` to initialize
|
|
// corresponding fields of `options`. InitializeOptionsGeneral() will
|
|
// also set up other fields of `options` so that stress test can run.
|
|
// Examples include `create_if_missing` and
|
|
// `create_missing_column_families`, etc.
|
|
// Case 2: OPTIONS file is specified. It is possible that, after loading from
|
|
// the given OPTIONS files, some shared object fields are still not
|
|
// initialized because they are not set in the OPTIONS file. In this
|
|
// case, if command line arguments indicate that the user wants to set
|
|
// up such shared objects, e.g. block cache, compressed block cache,
|
|
// row cache, filter policy, then InitializeOptionsGeneral() will honor
|
|
// the user's choice, thus passing `cache`,
|
|
// `filter_policy` as input arguments.
|
|
//
|
|
// InitializeOptionsGeneral() must not overwrite fields of `options` loaded
|
|
// from OPTIONS file.
|
|
extern void InitializeOptionsGeneral(
|
|
const std::shared_ptr<Cache>& cache,
|
|
const std::shared_ptr<const FilterPolicy>& filter_policy, Options& options);
|
|
|
|
// If no OPTIONS file is specified, set up `options` so that we can test
|
|
// user-defined timestamp which requires `-user_timestamp_size=8`.
|
|
// This function also checks for known (currently) incompatible features with
|
|
// user-defined timestamp.
|
|
extern void CheckAndSetOptionsForUserTimestamp(Options& options);
|
|
|
|
} // namespace ROCKSDB_NAMESPACE
|
|
#endif // GFLAGS
|