rocksdb/db_stress_tool/db_stress_compaction_filter.h
Hui Xiao 75a1230ce8 Fix improper ExpectedValute::Exists() usages and disable compaction during VerifyDB() in crash test (#12933)
Summary:
**Context:**
Adding assertion `!PendingPut()&&!PendingDelete()` in `ExpectedValute::Exists()` surfaced a couple improper usages of `ExpectedValute::Exists()` in the crash test
- Commit phase of `ExpectedValue::Delete()`/`SyncDelete()`:
When we issue delete to expected value during commit phase or `SyncDelete()` (used in crash recovery verification) as below, we don't really care about the result.
d458331ee9/db_stress_tool/expected_state.cc (L73)
d458331ee9/db_stress_tool/expected_value.cc (L52)
That means, we don't really need to check for `Exists()` d458331ee9/db_stress_tool/expected_value.cc (L24-L26).
This actually gives an alternative solution to b65e29a4a9 to solve false-positive assertion violation.
- TestMultiGetXX() path: `Exists()` is called without holding the lock as required

https://github.com/facebook/rocksdb/blame/f63428bcc7c42308ec1a84e82787b8d5dbd322ae/db_stress_tool/no_batched_ops_stress.cc#L2688
```
void MaybeAddKeyToTxnForRYW(
      ThreadState* thread, int column_family, int64_t key, Transaction* txn,
      std::unordered_map<std::string, ExpectedValue>& ryw_expected_values) {
    assert(thread);
    assert(txn);

    SharedState* const shared = thread->shared;
    assert(shared);

    if (!shared->AllowsOverwrite(key) && shared->Exists(column_family, key)) {
      // Just do read your write checks for keys that allow overwrites.
      return;
    }

    // With a 1 in 10 probability, insert the just added key in the batch
    // into the transaction. This will create an overlap with the MultiGet
    // keys and exercise some corner cases in the code
    if (thread->rand.OneIn(10)) {
```

f63428bcc7/db_stress_tool/expected_state.h (L74-L76)

The assertion also failed if db stress compaction filter was invoked before crash recovery verification (`VerifyDB()`->`VerifyOrSyncValue()`) finishes.
f63428bcc7/db_stress_tool/db_stress_compaction_filter.h (L53)
It failed because it can encounter a key with pending state when checking for `Exists()` since that key's expected state has not been sync-ed with db state in `VerifyOrSyncValue()`.
f63428bcc7/db_stress_tool/no_batched_ops_stress.cc (L2579-L2591)

**Summary:**
This PR fixes above issues by
- not checking `Exists()` in commit phase/`SyncDelete()`
- using the concurrent version of key existence check like in other read
- conditionally temporarily disabling compaction till after crash recovery verification succeeds()

And add back the assertion `!PendingPut()&&!PendingDelete()`

Pull Request resolved: https://github.com/facebook/rocksdb/pull/12933

Test Plan: Rehearsal CI

Reviewed By: cbi42

Differential Revision: D61214889

Pulled By: hx235

fbshipit-source-id: ef25ba896e64330ddf330182314981516880c3e4
2024-08-15 12:32:59 -07:00

96 lines
3.2 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.
#pragma once
#include "db_stress_tool/db_stress_common.h"
#include "db_stress_tool/db_stress_shared_state.h"
#include "rocksdb/compaction_filter.h"
namespace ROCKSDB_NAMESPACE {
// DbStressCompactionFilter is safe to use with db_stress as it does not perform
// any mutation. It only makes `kRemove` decisions for keys that are already
// non-existent according to the `SharedState`.
class DbStressCompactionFilter : public CompactionFilter {
public:
DbStressCompactionFilter(SharedState* state, int cf_id)
: state_(state), cf_id_(cf_id) {}
Decision FilterV2(int /*level*/, const Slice& key, ValueType /*value_type*/,
const Slice& /*existing_value*/, std::string* /*new_value*/,
std::string* /*skip_until*/) const override {
if (state_ == nullptr) {
return Decision::kKeep;
}
if (key.empty() || ('0' <= key[0] && key[0] <= '9')) {
// It is likely leftover from a test_batches_snapshots run. Below this
// conditional, the test_batches_snapshots key format is not handled
// properly. Just keep it to be safe.
return Decision::kKeep;
}
uint64_t key_num = 0;
{
Slice ukey_without_ts = key;
assert(ukey_without_ts.size() >= FLAGS_user_timestamp_size);
ukey_without_ts.remove_suffix(FLAGS_user_timestamp_size);
[[maybe_unused]] bool ok =
GetIntVal(ukey_without_ts.ToString(), &key_num);
assert(ok);
}
port::Mutex* key_mutex = state_->GetMutexForKey(cf_id_, key_num);
if (!key_mutex->TryLock()) {
return Decision::kKeep;
}
// Reaching here means we acquired the lock.
key_mutex->AssertHeld();
bool key_exists = state_->Exists(cf_id_, key_num);
const bool allow_overwrite = state_->AllowsOverwrite(key_num);
key_mutex->Unlock();
if (!key_exists) {
return allow_overwrite ? Decision::kRemove : Decision::kPurge;
}
return Decision::kKeep;
}
const char* Name() const override { return "DbStressCompactionFilter"; }
private:
SharedState* const state_;
const int cf_id_;
};
class DbStressCompactionFilterFactory : public CompactionFilterFactory {
public:
DbStressCompactionFilterFactory() : state_(nullptr) {}
void SetSharedState(SharedState* state) {
MutexLock state_mutex_guard(&state_mutex_);
state_ = state;
}
std::unique_ptr<CompactionFilter> CreateCompactionFilter(
const CompactionFilter::Context& context) override {
MutexLock state_mutex_guard(&state_mutex_);
return std::unique_ptr<CompactionFilter>(
new DbStressCompactionFilter(state_, context.column_family_id));
}
const char* Name() const override {
return "DbStressCompactionFilterFactory";
}
private:
port::Mutex state_mutex_;
SharedState* state_;
};
} // namespace ROCKSDB_NAMESPACE