You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/db_stress_tool/db_stress_test_base.h

333 lines
14 KiB

// 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;
Stress/Crash Test for OptimisticTransactionDB (#11513) Summary: Context: OptimisticTransactionDB has not been covered by db_stress (including crash test) like TransactionDB. 1. Adding the following gflag options to to test OptimisticTransactionDB - `use_optimistic_txn`: When true, open OptimisticTransactionDB to test - `occ_validation_policy`: `OccValidationPolicy::kValidateParallel = 1` by default. - `share_occ_lock_buckets`: Use shared occ locks - `occ_lock_bucket_count`: 500 by default. Number of buckets to use for shared occ lock. 2. Opening OptimisticTransactionDB and NewTxn/Commit added per `use_optimistic_txn` flag in `db_stress_test_base.cc` 3. OptimisticTransactionDB blackbox/whitebox test added in crash_test.mk Please note that the existing flag `use_txn` is being used here. When `use_txn == true` and `use_optimistic_txn == false`, we use `TransactionDB` (a.k.a. pessimistic transaction db). When both `use_txn` and `use_optimistic_txn` are true, we use `OptimisticTransactionDB`. If `use_txn == false` but `use_optimistic_txn == true` throw error with message _"You cannot set use_optimistic_txn true while use_txn is false. Please set use_txn true if you want to use OptimisticTransactionDB"_. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11513 Test Plan: **Crash Test** Serial Validation ``` export CRASH_TEST_EXT_ARGS="--use_optimistic_txn=1 --use_txn=1 --use_put_entity_one_in=0 --occ_validation_policy=0" make crash_test -j ``` Parallel Validation (no share bucket) ``` export CRASH_TEST_EXT_ARGS="--use_optimistic_txn=1 --use_txn=1 --use_put_entity_one_in=0 --occ_validation_policy=1 --share_occ_lock_buckets=0" make crash_test -j ``` Parallel Validation (share bucket) ``` export CRASH_TEST_EXT_ARGS="--use_optimistic_txn=1 --use_txn=1 --use_put_entity_one_in=0 --occ_validation_policy=1 --share_occ_lock_buckets=1 --occ_lock_bucket_count=500" make crash_test -j ``` **Stress Test** ``` ./db_stress -use_optimistic_txn -threads=32 ``` Reviewed By: pdillinger Differential Revision: D46547387 Pulled By: jaykorean fbshipit-source-id: ca19819ca6e0281694966998014b40d95d4e5960
1 year ago
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);
Support WriteCommit policy with sync_fault_injection=1 (#10624) Summary: **Context:** Prior to this PR, correctness testing with un-sync data loss [disabled](https://github.com/facebook/rocksdb/pull/10605) transaction (`use_txn=1`) thus all of the `txn_write_policy` . This PR improved that by adding support for one policy - WriteCommit (`txn_write_policy=0`). **Summary:** They key to this support is (a) handle Mark{Begin, End}Prepare/MarkCommit/MarkRollback in constructing ExpectedState under WriteCommit policy correctly and (b) monitor CI jobs and solve any test incompatibility issue till jobs are stable. (b) will be part of the test plan. For (a) - During prepare (i.e, between `MarkBeginPrepare()` and `MarkEndPrepare(xid)`), `ExpectedStateTraceRecordHandler` will buffer all writes by adding all writes to an internal `WriteBatch`. - On `MarkEndPrepare()`, that `WriteBatch` will be associated with the transaction's `xid`. - During the commit (i.e, on `MarkCommit(xid)`), `ExpectedStateTraceRecordHandler` will retrieve and iterate the internal `WriteBatch` and finally apply those writes to `ExpectedState` - During the rollback (i.e, on `MarkRollback(xid)`), `ExpectedStateTraceRecordHandler` will erase the internal `WriteBatch` from the map. For (b) - one major issue described below: - TransactionsDB in db stress recovers prepared-but-not-committed txns from the previous crashed run by randomly committing or rolling back it at the start of the current run, see a historical [PR](https://github.com/facebook/rocksdb/commit/6d06be22c083ccf185fd38dba49fde73b644b4c1) predated correctness testing. - And we will verify those processed keys in a recovered db against their expected state. - However since now we turn on `sync_fault_injection=1` where the expected state is constructed from the trace instead of using the LATEST.state from previous run. The expected state now used to verify those processed keys won't contain UNKNOWN_SENTINEL as they should - see test 1 for a failed case. - Therefore, we decided to manually update its expected state to be UNKNOWN_SENTINEL as part of the processing. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10624 Test Plan: 1. Test exposed the major issue described above. This test will fail without setting UNKNOWN_SENTINEL in expected state during the processing and pass after ``` db=/dev/shm/rocksdb_crashtest_blackbox exp=/dev/shm/rocksdb_crashtest_expected dbt=$db.tmp expt=$exp.tmp rm -rf $db $exp mkdir -p $exp echo "RUN 1" ./db_stress \ --clear_column_family_one_in=0 --column_families=1 --db=$db --delpercent=10 --delrangepercent=0 --destroy_db_initially=0 --expected_values_dir=$exp --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=1000000 --max_key_len=3 --prefixpercent=0 --readpercent=0 --reopen=0 --ops_per_thread=100000000 --test_batches_snapshots=0 --value_size_mult=32 --writepercent=90 \ --use_txn=1 --txn_write_policy=0 --sync_fault_injection=1 & pid=$! sleep 0.2 sleep 20 kill $pid sleep 0.2 echo "RUN 2" ./db_stress \ --clear_column_family_one_in=0 --column_families=1 --db=$db --delpercent=10 --delrangepercent=0 --destroy_db_initially=0 --expected_values_dir=$exp --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=1000000 --max_key_len=3 --prefixpercent=0 --readpercent=0 --reopen=0 --ops_per_thread=100000000 --test_batches_snapshots=0 --value_size_mult=32 --writepercent=90 \ --use_txn=1 --txn_write_policy=0 --sync_fault_injection=1 & pid=$! sleep 0.2 sleep 20 kill $pid sleep 0.2 echo "RUN 3" ./db_stress \ --clear_column_family_one_in=0 --column_families=1 --db=$db --delpercent=10 --delrangepercent=0 --destroy_db_initially=0 --expected_values_dir=$exp --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=1000000 --max_key_len=3 --prefixpercent=0 --readpercent=0 --reopen=0 --ops_per_thread=100000000 --test_batches_snapshots=0 --value_size_mult=32 --writepercent=90 \ --use_txn=1 --txn_write_policy=0 --sync_fault_injection=1 ``` 2. Manual testing to ensure ExpectedState is constructed correctly during recovery by verifying it against previously crashed TransactionDB's WAL. - Run the following command to crash a TransactionDB with WriteCommit policy. Then `./ldb dump_wal` on its WAL file ``` db=/dev/shm/rocksdb_crashtest_blackbox exp=/dev/shm/rocksdb_crashtest_expected rm -rf $db $exp mkdir -p $exp ./db_stress \ --clear_column_family_one_in=0 --column_families=1 --db=$db --delpercent=10 --delrangepercent=0 --destroy_db_initially=0 --expected_values_dir=$exp --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=1000000 --max_key_len=3 --prefixpercent=0 --readpercent=0 --reopen=0 --ops_per_thread=100000000 --test_batches_snapshots=0 --value_size_mult=32 --writepercent=90 \ --use_txn=1 --txn_write_policy=0 --sync_fault_injection=1 & pid=$! sleep 30 kill $pid sleep 1 ``` - Run the following command to verify recovery of the crashed db under debugger. Compare the step-wise result with WAL records (e.g, WriteBatch content, xid, prepare/commit/rollback marker) ``` ./db_stress \ --clear_column_family_one_in=0 --column_families=1 --db=$db --delpercent=10 --delrangepercent=0 --destroy_db_initially=0 --expected_values_dir=$exp --iterpercent=0 --key_len_percent_dist=1,30,69 --max_key=1000000 --max_key_len=3 --prefixpercent=0 --readpercent=0 --reopen=0 --ops_per_thread=100000000 --test_batches_snapshots=0 --value_size_mult=32 --writepercent=90 \ --use_txn=1 --txn_write_policy=0 --sync_fault_injection=1 ``` 3. Automatic testing by triggering all RocksDB stress/crash test jobs for 3 rounds with no failure. Reviewed By: ajkr, riversand963 Differential Revision: D39199373 Pulled By: hx235 fbshipit-source-id: 7a1dec0e3e2ee6ea86ddf5dd19ceb5543a3d6f0c
2 years ago
// 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);
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
1 year ago
// 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);
Add Iterator test against expected state to stress test (#10538) Summary: As mentioned in https://github.com/facebook/rocksdb/pull/5506#issuecomment-506021913, `db_stress` does not have much verification for iterator correctness. It has a `TestIterate()` function, but that is mainly for comparing results between two iterators, one with `total_order_seek` and the other optionally sets auto_prefix, upper/lower bounds. Commit 49a0581ad2462e31aa3f768afa769e0d33390f33 added a new `TestIterateAgainstExpected()` function that compares iterator against expected state. It locks a range of keys, creates an iterator, does a random sequence of `Next/Prev` and compares against expected state. This PR is based on that commit, the main changes include some logs (for easier debugging if a test fails), a forward and backward scan to cover the entire locked key range, and a flag for optionally turning on this version of Iterator testing. Added constraint that the checks against expected state in `TestIterateAgainstExpected()` and in `TestGet()` are only turned on when `--skip_verifydb` flag is not set. Remove the change log introduced in https://github.com/facebook/rocksdb/issues/10553. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10538 Test Plan: Run `db_stress` with `--verify_iterator_with_expected_state_one_in=1`, and a large `--iterpercent` and `--num_iterations`. Checked `op_logs` manually to ensure expected coverage. Tweaked part of the code in https://github.com/facebook/rocksdb/issues/10449 and stress test was able to catch it. - internally run various flavor of crash test Reviewed By: ajkr Differential Revision: D38847269 Pulled By: cbi42 fbshipit-source-id: 8b4402a9bba9f6cfa08051943cd672579d489599
2 years ago
virtual Status TestIterateAgainstExpected(
ThreadState* /* thread */, const ReadOptions& /* read_opts */,
const std::vector<int>& /* rand_column_families */,
const std::vector<int64_t>& /* rand_keys */) {
Add Iterator test against expected state to stress test (#10538) Summary: As mentioned in https://github.com/facebook/rocksdb/pull/5506#issuecomment-506021913, `db_stress` does not have much verification for iterator correctness. It has a `TestIterate()` function, but that is mainly for comparing results between two iterators, one with `total_order_seek` and the other optionally sets auto_prefix, upper/lower bounds. Commit 49a0581ad2462e31aa3f768afa769e0d33390f33 added a new `TestIterateAgainstExpected()` function that compares iterator against expected state. It locks a range of keys, creates an iterator, does a random sequence of `Next/Prev` and compares against expected state. This PR is based on that commit, the main changes include some logs (for easier debugging if a test fails), a forward and backward scan to cover the entire locked key range, and a flag for optionally turning on this version of Iterator testing. Added constraint that the checks against expected state in `TestIterateAgainstExpected()` and in `TestGet()` are only turned on when `--skip_verifydb` flag is not set. Remove the change log introduced in https://github.com/facebook/rocksdb/issues/10553. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10538 Test Plan: Run `db_stress` with `--verify_iterator_with_expected_state_one_in=1`, and a large `--iterpercent` and `--num_iterations`. Checked `op_logs` manually to ensure expected coverage. Tweaked part of the code in https://github.com/facebook/rocksdb/issues/10449 and stress test was able to catch it. - internally run various flavor of crash test Reviewed By: ajkr Differential Revision: D38847269 Pulled By: cbi42 fbshipit-source-id: 8b4402a9bba9f6cfa08051943cd672579d489599
2 years ago
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);
void Reopen(ThreadState* thread);
virtual void RegisterAdditionalListeners() {}
virtual void PrepareTxnDbOptions(SharedState* /*shared*/,
TransactionDBOptions& /*txn_db_opts*/) {}
User-defined timestamp support for `DeleteRange()` (#10661) Summary: Add user-defined timestamp support for range deletion. The new API is `DeleteRange(opt, cf, begin_key, end_key, ts)`. Most of the change is to update the comparator to compare without timestamp. Other than that, major changes are - internal range tombstone data structures (`FragmentedRangeTombstoneList`, `RangeTombstone`, etc.) to store timestamps. - Garbage collection of range tombstones and range tombstone covered keys during compaction. - Get()/MultiGet() to return the timestamp of a range tombstone when needed. - Get/Iterator with range tombstones bounded by readoptions.timestamp. - timestamp crash test now issues DeleteRange by default. Pull Request resolved: https://github.com/facebook/rocksdb/pull/10661 Test Plan: - Added unit test: `make check` - Stress test: `python3 tools/db_crashtest.py --enable_ts whitebox --readpercent=57 --prefixpercent=4 --writepercent=25 -delpercent=5 --iterpercent=5 --delrangepercent=4` - Ran `db_bench` to measure regression when timestamp is not enabled. The tests are for write (with some range deletion) and iterate with DB fitting in memory: `./db_bench--benchmarks=fillrandom,seekrandom --writes_per_range_tombstone=200 --max_write_buffer_number=100 --min_write_buffer_number_to_merge=100 --writes=500000 --reads=500000 --seek_nexts=10 --disable_auto_compactions -disable_wal=true --max_num_range_tombstones=1000`. Did not see consistent regression in no timestamp case. | micros/op | fillrandom | seekrandom | | --- | --- | --- | |main| 2.58 |10.96| |PR 10661| 2.68 |10.63| Reviewed By: riversand963 Differential Revision: D39441192 Pulled By: cbi42 fbshipit-source-id: f05aca3c41605caf110daf0ff405919f300ddec2
2 years ago
// 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_;
Stress/Crash Test for OptimisticTransactionDB (#11513) Summary: Context: OptimisticTransactionDB has not been covered by db_stress (including crash test) like TransactionDB. 1. Adding the following gflag options to to test OptimisticTransactionDB - `use_optimistic_txn`: When true, open OptimisticTransactionDB to test - `occ_validation_policy`: `OccValidationPolicy::kValidateParallel = 1` by default. - `share_occ_lock_buckets`: Use shared occ locks - `occ_lock_bucket_count`: 500 by default. Number of buckets to use for shared occ lock. 2. Opening OptimisticTransactionDB and NewTxn/Commit added per `use_optimistic_txn` flag in `db_stress_test_base.cc` 3. OptimisticTransactionDB blackbox/whitebox test added in crash_test.mk Please note that the existing flag `use_txn` is being used here. When `use_txn == true` and `use_optimistic_txn == false`, we use `TransactionDB` (a.k.a. pessimistic transaction db). When both `use_txn` and `use_optimistic_txn` are true, we use `OptimisticTransactionDB`. If `use_txn == false` but `use_optimistic_txn == true` throw error with message _"You cannot set use_optimistic_txn true while use_txn is false. Please set use_txn true if you want to use OptimisticTransactionDB"_. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11513 Test Plan: **Crash Test** Serial Validation ``` export CRASH_TEST_EXT_ARGS="--use_optimistic_txn=1 --use_txn=1 --use_put_entity_one_in=0 --occ_validation_policy=0" make crash_test -j ``` Parallel Validation (no share bucket) ``` export CRASH_TEST_EXT_ARGS="--use_optimistic_txn=1 --use_txn=1 --use_put_entity_one_in=0 --occ_validation_policy=1 --share_occ_lock_buckets=0" make crash_test -j ``` Parallel Validation (share bucket) ``` export CRASH_TEST_EXT_ARGS="--use_optimistic_txn=1 --use_txn=1 --use_put_entity_one_in=0 --occ_validation_policy=1 --share_occ_lock_buckets=1 --occ_lock_bucket_count=500" make crash_test -j ``` **Stress Test** ``` ./db_stress -use_optimistic_txn -threads=32 ``` Reviewed By: pdillinger Differential Revision: D46547387 Pulled By: jaykorean fbshipit-source-id: ca19819ca6e0281694966998014b40d95d4e5960
1 year ago
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_;
};
Avoid overwriting options loaded from OPTIONS (#9943) Summary: This is similar to https://github.com/facebook/rocksdb/issues/9862, including the following fixes/refactoring: 1. If OPTIONS file is specified via `-options_file`, majority of options will be loaded from the file. We should not overwrite options that have been loaded from the file. Instead, we configure only fields of options which are shared objects and not set by the OPTIONS file. We also configure a few fields, e.g. `create_if_missing` necessary for stress test to run. 2. Refactor options initialization into three functions, `InitializeOptionsFromFile()`, `InitializeOptionsFromFlags()` and `InitializeOptionsGeneral()` similar to db_bench. I hope they can be shared in the future. The high-level logic is as follows: ```cpp if (!InitializeOptionsFromFile(...)) { InitializeOptionsFromFlags(...); } InitializeOptionsGeneral(...); ``` 3. Currently, the setting for `block_cache_compressed` does not seem correct because it by default specifies a size of `numeric_limits<size_t>::max()` ((size_t)-1). According to code comments, `-1` indicates default value, which should be referring to `num_shard_bits` argument. 4. Clarify `fail_if_options_file_error`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9943 Test Plan: 1. make check 2. Run stress tests, and manually check generated OPTIONS file and compare them with input OPTIONS files Reviewed By: jay-zhuang Differential Revision: D36133769 Pulled By: riversand963 fbshipit-source-id: 35dacdc090a0a72c922907170cd132b9ecaa073e
3 years ago
// 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
Avoid overwriting options loaded from OPTIONS (#9943) Summary: This is similar to https://github.com/facebook/rocksdb/issues/9862, including the following fixes/refactoring: 1. If OPTIONS file is specified via `-options_file`, majority of options will be loaded from the file. We should not overwrite options that have been loaded from the file. Instead, we configure only fields of options which are shared objects and not set by the OPTIONS file. We also configure a few fields, e.g. `create_if_missing` necessary for stress test to run. 2. Refactor options initialization into three functions, `InitializeOptionsFromFile()`, `InitializeOptionsFromFlags()` and `InitializeOptionsGeneral()` similar to db_bench. I hope they can be shared in the future. The high-level logic is as follows: ```cpp if (!InitializeOptionsFromFile(...)) { InitializeOptionsFromFlags(...); } InitializeOptionsGeneral(...); ``` 3. Currently, the setting for `block_cache_compressed` does not seem correct because it by default specifies a size of `numeric_limits<size_t>::max()` ((size_t)-1). According to code comments, `-1` indicates default value, which should be referring to `num_shard_bits` argument. 4. Clarify `fail_if_options_file_error`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9943 Test Plan: 1. make check 2. Run stress tests, and manually check generated OPTIONS file and compare them with input OPTIONS files Reviewed By: jay-zhuang Differential Revision: D36133769 Pulled By: riversand963 fbshipit-source-id: 35dacdc090a0a72c922907170cd132b9ecaa073e
3 years ago
// 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`,
Avoid overwriting options loaded from OPTIONS (#9943) Summary: This is similar to https://github.com/facebook/rocksdb/issues/9862, including the following fixes/refactoring: 1. If OPTIONS file is specified via `-options_file`, majority of options will be loaded from the file. We should not overwrite options that have been loaded from the file. Instead, we configure only fields of options which are shared objects and not set by the OPTIONS file. We also configure a few fields, e.g. `create_if_missing` necessary for stress test to run. 2. Refactor options initialization into three functions, `InitializeOptionsFromFile()`, `InitializeOptionsFromFlags()` and `InitializeOptionsGeneral()` similar to db_bench. I hope they can be shared in the future. The high-level logic is as follows: ```cpp if (!InitializeOptionsFromFile(...)) { InitializeOptionsFromFlags(...); } InitializeOptionsGeneral(...); ``` 3. Currently, the setting for `block_cache_compressed` does not seem correct because it by default specifies a size of `numeric_limits<size_t>::max()` ((size_t)-1). According to code comments, `-1` indicates default value, which should be referring to `num_shard_bits` argument. 4. Clarify `fail_if_options_file_error`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9943 Test Plan: 1. make check 2. Run stress tests, and manually check generated OPTIONS file and compare them with input OPTIONS files Reviewed By: jay-zhuang Differential Revision: D36133769 Pulled By: riversand963 fbshipit-source-id: 35dacdc090a0a72c922907170cd132b9ecaa073e
3 years ago
// `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