2018-05-31 17:42:44 +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).
|
|
|
|
|
|
|
|
|
|
|
|
#include "utilities/transactions/write_unprepared_txn_db.h"
|
2022-10-25 21:15:22 +00:00
|
|
|
|
2019-09-19 19:32:33 +00:00
|
|
|
#include "db/arena_wrapped_db_iter.h"
|
2018-05-31 17:42:44 +00:00
|
|
|
#include "rocksdb/utilities/transaction_db.h"
|
2018-06-27 19:05:29 +00:00
|
|
|
#include "util/cast_util.h"
|
2018-05-31 17:42:44 +00:00
|
|
|
|
2020-02-20 20:07:53 +00:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2018-05-31 17:42:44 +00:00
|
|
|
|
2018-07-07 00:17:36 +00:00
|
|
|
// Instead of reconstructing a Transaction object, and calling rollback on it,
|
|
|
|
// we can be more efficient with RollbackRecoveredTransaction by skipping
|
|
|
|
// unnecessary steps (eg. updating CommitMap, reconstructing keyset)
|
|
|
|
Status WriteUnpreparedTxnDB::RollbackRecoveredTransaction(
|
|
|
|
const DBImpl::RecoveredTransaction* rtxn) {
|
|
|
|
// TODO(lth): Reduce duplicate code with WritePrepared rollback logic.
|
|
|
|
assert(rtxn->unprepared_);
|
|
|
|
auto cf_map_shared_ptr = WritePreparedTxnDB::GetCFHandleMap();
|
|
|
|
auto cf_comp_map_shared_ptr = WritePreparedTxnDB::GetCFComparatorMap();
|
2020-02-14 02:43:16 +00:00
|
|
|
// In theory we could write with disableWAL = true during recovery, and
|
|
|
|
// assume that if we crash again during recovery, we can just replay from
|
|
|
|
// the very beginning. Unfortunately, the XIDs from the application may not
|
|
|
|
// necessarily be unique across restarts, potentially leading to situations
|
|
|
|
// like this:
|
|
|
|
//
|
|
|
|
// BEGIN_PREPARE(unprepared) Put(a) END_PREPARE(xid = 1)
|
|
|
|
// -- crash and recover with Put(a) rolled back as it was not prepared
|
|
|
|
// BEGIN_PREPARE(prepared) Put(b) END_PREPARE(xid = 1)
|
|
|
|
// COMMIT(xid = 1)
|
|
|
|
// -- crash and recover with both a, b
|
|
|
|
//
|
|
|
|
// We could just write the rollback marker, but then we would have to extend
|
|
|
|
// MemTableInserter during recovery to actually do writes into the DB
|
|
|
|
// instead of just dropping the in-memory write batch.
|
|
|
|
//
|
Group SST write in flush, compaction and db open with new stats (#11910)
Summary:
## Context/Summary
Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity.
For that, this PR does the following:
- Tag different write IOs by passing down and converting WriteOptions to IOOptions
- Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS
Some related code refactory to make implementation cleaner:
- Blob stats
- Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info.
- Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write.
- Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority
- Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification
- Build table
- TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables
- Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder.
This parameter is used for dynamically changing file io priority for flush, see https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more
- Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority
## Test
### db bench
Flush
```
./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100
rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
```
compaction, db oopen
```
Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench
Run:./db_bench --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1
rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279
rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213
rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66
```
blob stats - just to make sure they aren't broken by this PR
```
Integrated Blob DB
Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench
Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1
pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600
rocksdb.blobdb.blob.file.synced COUNT : 1
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842
post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164
rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same)
```
```
Stacked Blob DB
Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench
pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876
rocksdb.blobdb.blob.file.synced COUNT : 8
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445
post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164
rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same)
```
### Rehearsal CI stress test
Trigger 3 full runs of all our CI stress tests
### Performance
Flush
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark; enable_statistics = true
Pre-pr: avg 507515519.3 ns
497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908,
Post-pr: avg 511971266.5 ns, regressed 0.88%
502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408,
```
Compaction
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark
Pre-pr: avg 495346098.30 ns
492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846
Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97%
502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007
```
Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats)
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark
Pre-pr: avg 3848.10 ns
3814,3838,3839,3848,3854,3854,3854,3860,3860,3860
Post-pr: avg 3874.20 ns, regressed 0.68%
3863,3867,3871,3874,3875,3877,3877,3877,3880,3881
```
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910
Reviewed By: ajkr
Differential Revision: D49788060
Pulled By: hx235
fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 23:29:23 +00:00
|
|
|
// TODO: plumb Env::IOActivity, Env::IOPriority
|
2018-07-07 00:17:36 +00:00
|
|
|
WriteOptions w_options;
|
|
|
|
|
2019-01-08 22:43:24 +00:00
|
|
|
class InvalidSnapshotReadCallback : public ReadCallback {
|
|
|
|
public:
|
2019-04-12 21:36:36 +00:00
|
|
|
InvalidSnapshotReadCallback(SequenceNumber snapshot)
|
|
|
|
: ReadCallback(snapshot) {}
|
|
|
|
|
|
|
|
inline bool IsVisibleFullCheck(SequenceNumber) override {
|
|
|
|
// The seq provided as snapshot is the seq right before we have locked and
|
|
|
|
// wrote to it, so whatever is there, it is committed.
|
|
|
|
return true;
|
2019-01-08 22:43:24 +00:00
|
|
|
}
|
|
|
|
|
2019-04-12 21:36:36 +00:00
|
|
|
// Ignore the refresh request since we are confident that our snapshot seq
|
|
|
|
// is not going to be affected by concurrent compactions (not enabled yet.)
|
|
|
|
void Refresh(SequenceNumber) override {}
|
2019-01-08 22:43:24 +00:00
|
|
|
};
|
|
|
|
|
2018-07-07 00:17:36 +00:00
|
|
|
// Iterate starting with largest sequence number.
|
2019-05-15 20:14:18 +00:00
|
|
|
for (auto it = rtxn->batches_.rbegin(); it != rtxn->batches_.rend(); ++it) {
|
2018-07-07 00:17:36 +00:00
|
|
|
auto last_visible_txn = it->first - 1;
|
|
|
|
const auto& batch = it->second.batch_;
|
2022-06-17 06:10:07 +00:00
|
|
|
WriteBatch rollback_batch(0 /* reserved_bytes */, 0 /* max_bytes */,
|
|
|
|
w_options.protection_bytes_per_key,
|
|
|
|
0 /* default_cf_ts_sz */);
|
2018-07-07 00:17:36 +00:00
|
|
|
|
|
|
|
struct RollbackWriteBatchBuilder : public WriteBatch::Handler {
|
|
|
|
DBImpl* db_;
|
|
|
|
ReadOptions roptions;
|
2019-01-08 22:43:24 +00:00
|
|
|
InvalidSnapshotReadCallback callback;
|
2018-07-07 00:17:36 +00:00
|
|
|
WriteBatch* rollback_batch_;
|
|
|
|
std::map<uint32_t, const Comparator*>& comparators_;
|
|
|
|
std::map<uint32_t, ColumnFamilyHandle*>& handles_;
|
|
|
|
using CFKeys = std::set<Slice, SetComparator>;
|
|
|
|
std::map<uint32_t, CFKeys> keys_;
|
|
|
|
bool rollback_merge_operands_;
|
|
|
|
RollbackWriteBatchBuilder(
|
2019-04-12 21:36:36 +00:00
|
|
|
DBImpl* db, SequenceNumber snap_seq, WriteBatch* dst_batch,
|
2018-07-07 00:17:36 +00:00
|
|
|
std::map<uint32_t, const Comparator*>& comparators,
|
|
|
|
std::map<uint32_t, ColumnFamilyHandle*>& handles,
|
|
|
|
bool rollback_merge_operands)
|
|
|
|
: db_(db),
|
2019-04-12 21:36:36 +00:00
|
|
|
callback(snap_seq),
|
2019-04-02 21:43:03 +00:00
|
|
|
// disable min_uncommitted optimization
|
2018-07-07 00:17:36 +00:00
|
|
|
rollback_batch_(dst_batch),
|
|
|
|
comparators_(comparators),
|
|
|
|
handles_(handles),
|
|
|
|
rollback_merge_operands_(rollback_merge_operands) {}
|
|
|
|
|
|
|
|
Status Rollback(uint32_t cf, const Slice& key) {
|
|
|
|
Status s;
|
|
|
|
CFKeys& cf_keys = keys_[cf];
|
|
|
|
if (cf_keys.size() == 0) { // just inserted
|
|
|
|
auto cmp = comparators_[cf];
|
|
|
|
keys_[cf] = CFKeys(SetComparator(cmp));
|
|
|
|
}
|
|
|
|
auto res = cf_keys.insert(key);
|
|
|
|
if (res.second ==
|
|
|
|
false) { // second is false if a element already existed.
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
PinnableSlice pinnable_val;
|
|
|
|
bool not_used;
|
|
|
|
auto cf_handle = handles_[cf];
|
New API to get all merge operands for a Key (#5604)
Summary:
This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases:
1. Update subset of columns and read subset of columns -
Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU.
2. Updating very few attributes in a value which is a JSON-like document -
Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge.
----------------------------------------------------------------------------------------------------
API :
Status GetMergeOperands(
const ReadOptions& options, ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* merge_operands,
GetMergeOperandsOptions* get_merge_operands_options,
int* number_of_operands)
Example usage :
int size = 100;
int number_of_operands = 0;
std::vector<PinnableSlice> values(size);
GetMergeOperandsOptions merge_operands_info;
db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands);
Description :
Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion.
merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604
Test Plan:
Added unit test and perf test in db_bench that can be run using the command:
./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist
Differential Revision: D16657366
Pulled By: vjnadimpalli
fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
2019-08-06 21:22:34 +00:00
|
|
|
DBImpl::GetImplOptions get_impl_options;
|
|
|
|
get_impl_options.column_family = cf_handle;
|
|
|
|
get_impl_options.value = &pinnable_val;
|
|
|
|
get_impl_options.value_found = ¬_used;
|
|
|
|
get_impl_options.callback = &callback;
|
|
|
|
s = db_->GetImpl(roptions, key, get_impl_options);
|
2018-07-07 00:17:36 +00:00
|
|
|
assert(s.ok() || s.IsNotFound());
|
|
|
|
if (s.ok()) {
|
|
|
|
s = rollback_batch_->Put(cf_handle, key, pinnable_val);
|
|
|
|
assert(s.ok());
|
|
|
|
} else if (s.IsNotFound()) {
|
|
|
|
// There has been no readable value before txn. By adding a delete we
|
|
|
|
// make sure that there will be none afterwards either.
|
|
|
|
s = rollback_batch_->Delete(cf_handle, key);
|
|
|
|
assert(s.ok());
|
|
|
|
} else {
|
|
|
|
// Unexpected status. Return it to the user.
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status PutCF(uint32_t cf, const Slice& key,
|
|
|
|
const Slice& /*val*/) override {
|
|
|
|
return Rollback(cf, key);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DeleteCF(uint32_t cf, const Slice& key) override {
|
|
|
|
return Rollback(cf, key);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
|
|
|
|
return Rollback(cf, key);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status MergeCF(uint32_t cf, const Slice& key,
|
|
|
|
const Slice& /*val*/) override {
|
|
|
|
if (rollback_merge_operands_) {
|
|
|
|
return Rollback(cf, key);
|
|
|
|
} else {
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Recovered batches do not contain 2PC markers.
|
|
|
|
Status MarkNoop(bool) override { return Status::InvalidArgument(); }
|
|
|
|
Status MarkBeginPrepare(bool) override {
|
|
|
|
return Status::InvalidArgument();
|
|
|
|
}
|
|
|
|
Status MarkEndPrepare(const Slice&) override {
|
|
|
|
return Status::InvalidArgument();
|
|
|
|
}
|
|
|
|
Status MarkCommit(const Slice&) override {
|
|
|
|
return Status::InvalidArgument();
|
|
|
|
}
|
|
|
|
Status MarkRollback(const Slice&) override {
|
|
|
|
return Status::InvalidArgument();
|
|
|
|
}
|
2019-04-12 21:36:36 +00:00
|
|
|
} rollback_handler(db_impl_, last_visible_txn, &rollback_batch,
|
2018-07-07 00:17:36 +00:00
|
|
|
*cf_comp_map_shared_ptr.get(), *cf_map_shared_ptr.get(),
|
2018-07-24 07:09:18 +00:00
|
|
|
txn_db_options_.rollback_merge_operands);
|
2018-07-07 00:17:36 +00:00
|
|
|
|
|
|
|
auto s = batch->Iterate(&rollback_handler);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
// The Rollback marker will be used as a batch separator
|
2020-12-10 05:19:55 +00:00
|
|
|
s = WriteBatchInternal::MarkRollback(&rollback_batch, rtxn->name_);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
2018-07-07 00:17:36 +00:00
|
|
|
|
|
|
|
const uint64_t kNoLogRef = 0;
|
|
|
|
const bool kDisableMemtable = true;
|
|
|
|
const size_t kOneBatch = 1;
|
|
|
|
uint64_t seq_used = kMaxSequenceNumber;
|
|
|
|
s = db_impl_->WriteImpl(w_options, &rollback_batch, nullptr, nullptr,
|
2024-06-01 02:30:19 +00:00
|
|
|
nullptr, kNoLogRef, !kDisableMemtable, &seq_used,
|
|
|
|
kOneBatch);
|
2018-07-07 00:17:36 +00:00
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
// If two_write_queues, we must manually release the sequence number to
|
|
|
|
// readers.
|
|
|
|
if (db_impl_->immutable_db_options().two_write_queues) {
|
|
|
|
db_impl_->SetLastPublishedSequence(seq_used);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status WriteUnpreparedTxnDB::Initialize(
|
|
|
|
const std::vector<size_t>& compaction_enabled_cf_indices,
|
|
|
|
const std::vector<ColumnFamilyHandle*>& handles) {
|
|
|
|
// TODO(lth): Reduce code duplication in this function.
|
2020-04-29 20:06:27 +00:00
|
|
|
auto dbimpl = static_cast_with_check<DBImpl>(GetRootDB());
|
2018-07-07 00:17:36 +00:00
|
|
|
assert(dbimpl != nullptr);
|
|
|
|
|
|
|
|
db_impl_->SetSnapshotChecker(new WritePreparedSnapshotChecker(this));
|
|
|
|
// A callback to commit a single sub-batch
|
|
|
|
class CommitSubBatchPreReleaseCallback : public PreReleaseCallback {
|
|
|
|
public:
|
|
|
|
explicit CommitSubBatchPreReleaseCallback(WritePreparedTxnDB* db)
|
|
|
|
: db_(db) {}
|
2019-04-02 22:14:41 +00:00
|
|
|
Status Callback(SequenceNumber commit_seq,
|
2019-06-10 18:47:16 +00:00
|
|
|
bool is_mem_disabled __attribute__((__unused__)), uint64_t,
|
|
|
|
size_t /*index*/, size_t /*total*/) override {
|
2018-07-07 00:17:36 +00:00
|
|
|
assert(!is_mem_disabled);
|
|
|
|
db_->AddCommitted(commit_seq, commit_seq);
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
WritePreparedTxnDB* db_;
|
|
|
|
};
|
|
|
|
db_impl_->SetRecoverableStatePreReleaseCallback(
|
|
|
|
new CommitSubBatchPreReleaseCallback(this));
|
|
|
|
|
|
|
|
// PessimisticTransactionDB::Initialize
|
|
|
|
for (auto cf_ptr : handles) {
|
|
|
|
AddColumnFamily(cf_ptr);
|
|
|
|
}
|
|
|
|
// Verify cf options
|
|
|
|
for (auto handle : handles) {
|
|
|
|
ColumnFamilyDescriptor cfd;
|
|
|
|
Status s = handle->GetDescriptor(&cfd);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
s = VerifyCFOptions(cfd.options);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Re-enable compaction for the column families that initially had
|
|
|
|
// compaction enabled.
|
|
|
|
std::vector<ColumnFamilyHandle*> compaction_enabled_cf_handles;
|
|
|
|
compaction_enabled_cf_handles.reserve(compaction_enabled_cf_indices.size());
|
|
|
|
for (auto index : compaction_enabled_cf_indices) {
|
|
|
|
compaction_enabled_cf_handles.push_back(handles[index]);
|
|
|
|
}
|
|
|
|
|
|
|
|
// create 'real' transactions from recovered shell transactions
|
|
|
|
auto rtxns = dbimpl->recovered_transactions();
|
2019-06-12 02:52:08 +00:00
|
|
|
std::map<SequenceNumber, SequenceNumber> ordered_seq_cnt;
|
2024-03-04 18:08:32 +00:00
|
|
|
for (const auto& rtxn : rtxns) {
|
2018-07-07 00:17:36 +00:00
|
|
|
auto recovered_trx = rtxn.second;
|
|
|
|
assert(recovered_trx);
|
|
|
|
assert(recovered_trx->batches_.size() >= 1);
|
|
|
|
assert(recovered_trx->name_.length());
|
|
|
|
|
|
|
|
// We can only rollback transactions after AdvanceMaxEvictedSeq is called,
|
|
|
|
// but AddPrepared must occur before AdvanceMaxEvictedSeq, which is why
|
|
|
|
// two iterations is required.
|
|
|
|
if (recovered_trx->unprepared_) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
Group SST write in flush, compaction and db open with new stats (#11910)
Summary:
## Context/Summary
Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity.
For that, this PR does the following:
- Tag different write IOs by passing down and converting WriteOptions to IOOptions
- Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS
Some related code refactory to make implementation cleaner:
- Blob stats
- Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info.
- Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write.
- Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority
- Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification
- Build table
- TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables
- Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder.
This parameter is used for dynamically changing file io priority for flush, see https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more
- Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority
## Test
### db bench
Flush
```
./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100
rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377
rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
```
compaction, db oopen
```
Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench
Run:./db_bench --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1
rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279
rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0
rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213
rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66
```
blob stats - just to make sure they aren't broken by this PR
```
Integrated Blob DB
Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench
Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1
pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600
rocksdb.blobdb.blob.file.synced COUNT : 1
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842
post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164
rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same)
```
```
Stacked Blob DB
Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench
pre-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876
rocksdb.blobdb.blob.file.synced COUNT : 8
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445
post-PR:
rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924
- COUNT is higher and values are smaller as it includes header and footer write
- COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164
rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same)
rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same)
```
### Rehearsal CI stress test
Trigger 3 full runs of all our CI stress tests
### Performance
Flush
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark; enable_statistics = true
Pre-pr: avg 507515519.3 ns
497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908,
Post-pr: avg 511971266.5 ns, regressed 0.88%
502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408,
```
Compaction
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark
Pre-pr: avg 495346098.30 ns
492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846
Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97%
502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007
```
Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats)
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1 --benchmark_repetitions=1000
-- default: 1 thread is used to run benchmark
Pre-pr: avg 3848.10 ns
3814,3838,3839,3848,3854,3854,3854,3860,3860,3860
Post-pr: avg 3874.20 ns, regressed 0.68%
3863,3867,3871,3874,3875,3877,3877,3877,3880,3881
```
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910
Reviewed By: ajkr
Differential Revision: D49788060
Pulled By: hx235
fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 23:29:23 +00:00
|
|
|
// TODO: plumb Env::IOActivity, Env::IOPriority
|
2018-07-07 00:17:36 +00:00
|
|
|
WriteOptions w_options;
|
|
|
|
w_options.sync = true;
|
|
|
|
TransactionOptions t_options;
|
|
|
|
|
|
|
|
auto first_log_number = recovered_trx->batches_.begin()->second.log_number_;
|
2018-07-24 07:09:18 +00:00
|
|
|
auto first_seq = recovered_trx->batches_.begin()->first;
|
2018-07-07 00:17:36 +00:00
|
|
|
auto last_prepare_batch_cnt =
|
|
|
|
recovered_trx->batches_.begin()->second.batch_cnt_;
|
|
|
|
|
|
|
|
Transaction* real_trx = BeginTransaction(w_options, t_options, nullptr);
|
|
|
|
assert(real_trx);
|
2020-04-29 20:06:27 +00:00
|
|
|
auto wupt = static_cast_with_check<WriteUnpreparedTxn>(real_trx);
|
2019-07-16 22:19:45 +00:00
|
|
|
wupt->recovered_txn_ = true;
|
2018-07-07 00:17:36 +00:00
|
|
|
|
|
|
|
real_trx->SetLogNumber(first_log_number);
|
2018-07-24 07:09:18 +00:00
|
|
|
real_trx->SetId(first_seq);
|
2019-03-29 20:09:51 +00:00
|
|
|
Status s = real_trx->SetName(recovered_trx->name_);
|
2018-07-07 00:17:36 +00:00
|
|
|
if (!s.ok()) {
|
2019-07-16 22:19:45 +00:00
|
|
|
return s;
|
2018-07-07 00:17:36 +00:00
|
|
|
}
|
|
|
|
wupt->prepare_batch_cnt_ = last_prepare_batch_cnt;
|
|
|
|
|
|
|
|
for (auto batch : recovered_trx->batches_) {
|
|
|
|
const auto& seq = batch.first;
|
|
|
|
const auto& batch_info = batch.second;
|
|
|
|
auto cnt = batch_info.batch_cnt_ ? batch_info.batch_cnt_ : 1;
|
|
|
|
assert(batch_info.log_number_);
|
|
|
|
|
2019-06-12 02:52:08 +00:00
|
|
|
ordered_seq_cnt[seq] = cnt;
|
2018-07-07 00:17:36 +00:00
|
|
|
assert(wupt->unprep_seqs_.count(seq) == 0);
|
|
|
|
wupt->unprep_seqs_[seq] = cnt;
|
2019-07-16 22:19:45 +00:00
|
|
|
|
|
|
|
s = wupt->RebuildFromWriteBatch(batch_info.batch_);
|
2018-07-24 07:09:18 +00:00
|
|
|
assert(s.ok());
|
|
|
|
if (!s.ok()) {
|
2019-07-16 22:19:45 +00:00
|
|
|
return s;
|
2018-07-24 07:09:18 +00:00
|
|
|
}
|
2018-07-07 00:17:36 +00:00
|
|
|
}
|
|
|
|
|
2019-07-31 20:36:22 +00:00
|
|
|
const bool kClear = true;
|
|
|
|
wupt->InitWriteBatch(kClear);
|
2018-07-07 00:17:36 +00:00
|
|
|
|
|
|
|
real_trx->SetState(Transaction::PREPARED);
|
|
|
|
if (!s.ok()) {
|
2019-07-16 22:19:45 +00:00
|
|
|
return s;
|
2018-07-07 00:17:36 +00:00
|
|
|
}
|
|
|
|
}
|
2019-06-12 02:52:08 +00:00
|
|
|
// AddPrepared must be called in order
|
2019-09-20 19:00:55 +00:00
|
|
|
for (auto seq_cnt : ordered_seq_cnt) {
|
2019-06-12 02:52:08 +00:00
|
|
|
auto seq = seq_cnt.first;
|
|
|
|
auto cnt = seq_cnt.second;
|
|
|
|
for (size_t i = 0; i < cnt; i++) {
|
|
|
|
AddPrepared(seq + i);
|
|
|
|
}
|
|
|
|
}
|
2018-07-07 00:17:36 +00:00
|
|
|
|
|
|
|
SequenceNumber prev_max = max_evicted_seq_;
|
|
|
|
SequenceNumber last_seq = db_impl_->GetLatestSequenceNumber();
|
|
|
|
AdvanceMaxEvictedSeq(prev_max, last_seq);
|
2019-03-29 20:09:51 +00:00
|
|
|
// Create a gap between max and the next snapshot. This simplifies the logic
|
|
|
|
// in IsInSnapshot by not having to consider the special case of max ==
|
|
|
|
// snapshot after recovery. This is tested in IsInSnapshotEmptyMapTest.
|
|
|
|
if (last_seq) {
|
|
|
|
db_impl_->versions_->SetLastAllocatedSequence(last_seq + 1);
|
|
|
|
db_impl_->versions_->SetLastSequence(last_seq + 1);
|
|
|
|
db_impl_->versions_->SetLastPublishedSequence(last_seq + 1);
|
|
|
|
}
|
|
|
|
|
2019-04-12 21:36:36 +00:00
|
|
|
Status s;
|
2018-07-07 00:17:36 +00:00
|
|
|
// Rollback unprepared transactions.
|
2024-03-04 18:08:32 +00:00
|
|
|
for (const auto& rtxn : rtxns) {
|
2018-07-07 00:17:36 +00:00
|
|
|
auto recovered_trx = rtxn.second;
|
|
|
|
if (recovered_trx->unprepared_) {
|
|
|
|
s = RollbackRecoveredTransaction(recovered_trx);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (s.ok()) {
|
|
|
|
dbimpl->DeleteAllRecoveredTransactions();
|
2019-04-12 21:36:36 +00:00
|
|
|
|
|
|
|
// Compaction should start only after max_evicted_seq_ is set AND recovered
|
|
|
|
// transactions are either added to PrepareHeap or rolled back.
|
|
|
|
s = EnableAutoCompaction(compaction_enabled_cf_handles);
|
2018-07-07 00:17:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2018-05-31 17:42:44 +00:00
|
|
|
Transaction* WriteUnpreparedTxnDB::BeginTransaction(
|
2018-06-27 19:05:29 +00:00
|
|
|
const WriteOptions& write_options, const TransactionOptions& txn_options,
|
|
|
|
Transaction* old_txn) {
|
2018-05-31 17:42:44 +00:00
|
|
|
if (old_txn != nullptr) {
|
|
|
|
ReinitializeTransaction(old_txn, write_options, txn_options);
|
|
|
|
return old_txn;
|
|
|
|
} else {
|
|
|
|
return new WriteUnpreparedTxn(this, write_options, txn_options);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-27 19:05:29 +00:00
|
|
|
// Struct to hold ownership of snapshot and read callback for iterator cleanup.
|
|
|
|
struct WriteUnpreparedTxnDB::IteratorState {
|
|
|
|
IteratorState(WritePreparedTxnDB* txn_db, SequenceNumber sequence,
|
|
|
|
std::shared_ptr<ManagedSnapshot> s,
|
|
|
|
SequenceNumber min_uncommitted, WriteUnpreparedTxn* txn)
|
2019-08-05 20:30:56 +00:00
|
|
|
: callback(txn_db, sequence, min_uncommitted, txn->unprep_seqs_,
|
|
|
|
kBackedByDBSnapshot),
|
2019-07-24 17:21:18 +00:00
|
|
|
snapshot(s) {}
|
2019-04-02 21:43:03 +00:00
|
|
|
SequenceNumber MaxVisibleSeq() { return callback.max_visible_seq(); }
|
2018-06-27 19:05:29 +00:00
|
|
|
|
|
|
|
WriteUnpreparedTxnReadCallback callback;
|
|
|
|
std::shared_ptr<ManagedSnapshot> snapshot;
|
|
|
|
};
|
|
|
|
|
|
|
|
namespace {
|
|
|
|
static void CleanupWriteUnpreparedTxnDBIterator(void* arg1, void* /*arg2*/) {
|
Prefer static_cast in place of most reinterpret_cast (#12308)
Summary:
The following are risks associated with pointer-to-pointer reinterpret_cast:
* Can produce the "wrong result" (crash or memory corruption). IIRC, in theory this can happen for any up-cast or down-cast for a non-standard-layout type, though in practice would only happen for multiple inheritance cases (where the base class pointer might be "inside" the derived object). We don't use multiple inheritance a lot, but we do.
* Can mask useful compiler errors upon code change, including converting between unrelated pointer types that you are expecting to be related, and converting between pointer and scalar types unintentionally.
I can only think of some obscure cases where static_cast could be troublesome when it compiles as a replacement:
* Going through `void*` could plausibly cause unnecessary or broken pointer arithmetic. Suppose we have
`struct Derived: public Base1, public Base2`. If we have `Derived*` -> `void*` -> `Base2*` -> `Derived*` through reinterpret casts, this could plausibly work (though technical UB) assuming the `Base2*` is not dereferenced. Changing to static cast could introduce breaking pointer arithmetic.
* Unnecessary (but safe) pointer arithmetic could arise in a case like `Derived*` -> `Base2*` -> `Derived*` where before the Base2 pointer might not have been dereferenced. This could potentially affect performance.
With some light scripting, I tried replacing pointer-to-pointer reinterpret_casts with static_cast and kept the cases that still compile. Most occurrences of reinterpret_cast have successfully been changed (except for java/ and third-party/). 294 changed, 257 remain.
A couple of related interventions included here:
* Previously Cache::Handle was not actually derived from in the implementations and just used as a `void*` stand-in with reinterpret_cast. Now there is a relationship to allow static_cast. In theory, this could introduce pointer arithmetic (as described above) but is unlikely without multiple inheritance AND non-empty Cache::Handle.
* Remove some unnecessary casts to void* as this is allowed to be implicit (for better or worse).
Most of the remaining reinterpret_casts are for converting to/from raw bytes of objects. We could consider better idioms for these patterns in follow-up work.
I wish there were a way to implement a template variant of static_cast that would only compile if no pointer arithmetic is generated, but best I can tell, this is not possible. AFAIK the best you could do is a dynamic check that the void* conversion after the static cast is unchanged.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12308
Test Plan: existing tests, CI
Reviewed By: ltamasi
Differential Revision: D53204947
Pulled By: pdillinger
fbshipit-source-id: 9de23e618263b0d5b9820f4e15966876888a16e2
2024-02-07 18:44:11 +00:00
|
|
|
delete static_cast<WriteUnpreparedTxnDB::IteratorState*>(arg1);
|
2018-06-27 19:05:29 +00:00
|
|
|
}
|
|
|
|
} // anonymous namespace
|
|
|
|
|
Group rocksdb.sst.read.micros stat by different user read IOActivity + misc (#11444)
Summary:
**Context/Summary:**
- Similar to https://github.com/facebook/rocksdb/pull/11288 but for user read such as `Get(), MultiGet(), DBIterator::XXX(), Verify(File)Checksum()`.
- For this, I refactored some user-facing `MultiGet` calls in `TransactionBase` and various types of `DB` so that it does not call a user-facing `Get()` but `GetImpl()` for passing the `ReadOptions::io_activity` check (see PR conversation)
- New user read stats breakdown are guarded by `kExceptDetailedTimers` since measurement shows they have 4-5% regression to the upstream/main.
- Misc
- More refactoring: with https://github.com/facebook/rocksdb/pull/11288, we complete passing `ReadOptions/IOOptions` to FS level. So we can now replace the previously [added](https://github.com/facebook/rocksdb/pull/9424) `rate_limiter_priority` parameter in `RandomAccessFileReader`'s `Read/MultiRead/Prefetch()` with `IOOptions::rate_limiter_priority`
- Also, `ReadAsync()` call time is measured in `SST_READ_MICRO` now
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11444
Test Plan:
- CI fake db crash/stress test
- Microbenchmarking
**Build** `make clean && ROCKSDB_NO_FBCODE=1 DEBUG_LEVEL=0 make -jN db_basic_bench`
- google benchmark version: https://github.com/google/benchmark/commit/604f6fd3f4b34a84ec4eb4db81d842fa4db829cd
- db_basic_bench_base: upstream
- db_basic_bench_pr: db_basic_bench_base + this PR
- asyncread_db_basic_bench_base: upstream + [db basic bench patch for IteratorNext](https://github.com/facebook/rocksdb/compare/main...hx235:rocksdb:micro_bench_async_read)
- asyncread_db_basic_bench_pr: asyncread_db_basic_bench_base + this PR
**Test**
Get
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{null_stat|base|pr} --benchmark_filter=DBGet/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/negative_query:0/enable_filter:0/mmap:1/threads:1 --benchmark_repetitions=1000
```
Result
```
Coming soon
```
AsyncRead
```
TEST_TMPDIR=/dev/shm ./asyncread_db_basic_bench_{base|pr} --benchmark_filter=IteratorNext/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/async_io:1/include_detailed_timers:0 --benchmark_repetitions=1000 > syncread_db_basic_bench_{base|pr}.out
```
Result
```
Base:
1956,1956,1968,1977,1979,1986,1988,1988,1988,1990,1991,1991,1993,1993,1993,1993,1994,1996,1997,1997,1997,1998,1999,2001,2001,2002,2004,2007,2007,2008,
PR (2.3% regression, due to measuring `SST_READ_MICRO` that wasn't measured before):
1993,2014,2016,2022,2024,2027,2027,2028,2028,2030,2031,2031,2032,2032,2038,2039,2042,2044,2044,2047,2047,2047,2048,2049,2050,2052,2052,2052,2053,2053,
```
Reviewed By: ajkr
Differential Revision: D45918925
Pulled By: hx235
fbshipit-source-id: 58a54560d9ebeb3a59b6d807639692614dad058a
2023-08-09 00:26:50 +00:00
|
|
|
Iterator* WriteUnpreparedTxnDB::NewIterator(const ReadOptions& _read_options,
|
2018-06-27 19:05:29 +00:00
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
WriteUnpreparedTxn* txn) {
|
Group rocksdb.sst.read.micros stat by different user read IOActivity + misc (#11444)
Summary:
**Context/Summary:**
- Similar to https://github.com/facebook/rocksdb/pull/11288 but for user read such as `Get(), MultiGet(), DBIterator::XXX(), Verify(File)Checksum()`.
- For this, I refactored some user-facing `MultiGet` calls in `TransactionBase` and various types of `DB` so that it does not call a user-facing `Get()` but `GetImpl()` for passing the `ReadOptions::io_activity` check (see PR conversation)
- New user read stats breakdown are guarded by `kExceptDetailedTimers` since measurement shows they have 4-5% regression to the upstream/main.
- Misc
- More refactoring: with https://github.com/facebook/rocksdb/pull/11288, we complete passing `ReadOptions/IOOptions` to FS level. So we can now replace the previously [added](https://github.com/facebook/rocksdb/pull/9424) `rate_limiter_priority` parameter in `RandomAccessFileReader`'s `Read/MultiRead/Prefetch()` with `IOOptions::rate_limiter_priority`
- Also, `ReadAsync()` call time is measured in `SST_READ_MICRO` now
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11444
Test Plan:
- CI fake db crash/stress test
- Microbenchmarking
**Build** `make clean && ROCKSDB_NO_FBCODE=1 DEBUG_LEVEL=0 make -jN db_basic_bench`
- google benchmark version: https://github.com/google/benchmark/commit/604f6fd3f4b34a84ec4eb4db81d842fa4db829cd
- db_basic_bench_base: upstream
- db_basic_bench_pr: db_basic_bench_base + this PR
- asyncread_db_basic_bench_base: upstream + [db basic bench patch for IteratorNext](https://github.com/facebook/rocksdb/compare/main...hx235:rocksdb:micro_bench_async_read)
- asyncread_db_basic_bench_pr: asyncread_db_basic_bench_base + this PR
**Test**
Get
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{null_stat|base|pr} --benchmark_filter=DBGet/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/negative_query:0/enable_filter:0/mmap:1/threads:1 --benchmark_repetitions=1000
```
Result
```
Coming soon
```
AsyncRead
```
TEST_TMPDIR=/dev/shm ./asyncread_db_basic_bench_{base|pr} --benchmark_filter=IteratorNext/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/async_io:1/include_detailed_timers:0 --benchmark_repetitions=1000 > syncread_db_basic_bench_{base|pr}.out
```
Result
```
Base:
1956,1956,1968,1977,1979,1986,1988,1988,1988,1990,1991,1991,1993,1993,1993,1993,1994,1996,1997,1997,1997,1998,1999,2001,2001,2002,2004,2007,2007,2008,
PR (2.3% regression, due to measuring `SST_READ_MICRO` that wasn't measured before):
1993,2014,2016,2022,2024,2027,2027,2028,2028,2030,2031,2031,2032,2032,2038,2039,2042,2044,2044,2047,2047,2047,2048,2049,2050,2052,2052,2052,2053,2053,
```
Reviewed By: ajkr
Differential Revision: D45918925
Pulled By: hx235
fbshipit-source-id: 58a54560d9ebeb3a59b6d807639692614dad058a
2023-08-09 00:26:50 +00:00
|
|
|
if (_read_options.io_activity != Env::IOActivity::kUnknown &&
|
|
|
|
_read_options.io_activity != Env::IOActivity::kDBIterator) {
|
2023-04-21 16:07:18 +00:00
|
|
|
return NewErrorIterator(Status::InvalidArgument(
|
Group rocksdb.sst.read.micros stat by different user read IOActivity + misc (#11444)
Summary:
**Context/Summary:**
- Similar to https://github.com/facebook/rocksdb/pull/11288 but for user read such as `Get(), MultiGet(), DBIterator::XXX(), Verify(File)Checksum()`.
- For this, I refactored some user-facing `MultiGet` calls in `TransactionBase` and various types of `DB` so that it does not call a user-facing `Get()` but `GetImpl()` for passing the `ReadOptions::io_activity` check (see PR conversation)
- New user read stats breakdown are guarded by `kExceptDetailedTimers` since measurement shows they have 4-5% regression to the upstream/main.
- Misc
- More refactoring: with https://github.com/facebook/rocksdb/pull/11288, we complete passing `ReadOptions/IOOptions` to FS level. So we can now replace the previously [added](https://github.com/facebook/rocksdb/pull/9424) `rate_limiter_priority` parameter in `RandomAccessFileReader`'s `Read/MultiRead/Prefetch()` with `IOOptions::rate_limiter_priority`
- Also, `ReadAsync()` call time is measured in `SST_READ_MICRO` now
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11444
Test Plan:
- CI fake db crash/stress test
- Microbenchmarking
**Build** `make clean && ROCKSDB_NO_FBCODE=1 DEBUG_LEVEL=0 make -jN db_basic_bench`
- google benchmark version: https://github.com/google/benchmark/commit/604f6fd3f4b34a84ec4eb4db81d842fa4db829cd
- db_basic_bench_base: upstream
- db_basic_bench_pr: db_basic_bench_base + this PR
- asyncread_db_basic_bench_base: upstream + [db basic bench patch for IteratorNext](https://github.com/facebook/rocksdb/compare/main...hx235:rocksdb:micro_bench_async_read)
- asyncread_db_basic_bench_pr: asyncread_db_basic_bench_base + this PR
**Test**
Get
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{null_stat|base|pr} --benchmark_filter=DBGet/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/negative_query:0/enable_filter:0/mmap:1/threads:1 --benchmark_repetitions=1000
```
Result
```
Coming soon
```
AsyncRead
```
TEST_TMPDIR=/dev/shm ./asyncread_db_basic_bench_{base|pr} --benchmark_filter=IteratorNext/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/async_io:1/include_detailed_timers:0 --benchmark_repetitions=1000 > syncread_db_basic_bench_{base|pr}.out
```
Result
```
Base:
1956,1956,1968,1977,1979,1986,1988,1988,1988,1990,1991,1991,1993,1993,1993,1993,1994,1996,1997,1997,1997,1998,1999,2001,2001,2002,2004,2007,2007,2008,
PR (2.3% regression, due to measuring `SST_READ_MICRO` that wasn't measured before):
1993,2014,2016,2022,2024,2027,2027,2028,2028,2030,2031,2031,2032,2032,2038,2039,2042,2044,2044,2047,2047,2047,2048,2049,2050,2052,2052,2052,2053,2053,
```
Reviewed By: ajkr
Differential Revision: D45918925
Pulled By: hx235
fbshipit-source-id: 58a54560d9ebeb3a59b6d807639692614dad058a
2023-08-09 00:26:50 +00:00
|
|
|
"Can only call NewIterator with `ReadOptions::io_activity` is "
|
|
|
|
"`Env::IOActivity::kUnknown` or `Env::IOActivity::kDBIterator`"));
|
|
|
|
}
|
|
|
|
|
|
|
|
ReadOptions read_options(_read_options);
|
|
|
|
if (read_options.io_activity == Env::IOActivity::kUnknown) {
|
|
|
|
read_options.io_activity = Env::IOActivity::kDBIterator;
|
2023-04-21 16:07:18 +00:00
|
|
|
}
|
2018-06-27 19:05:29 +00:00
|
|
|
// TODO(lth): Refactor so that this logic is shared with WritePrepared.
|
2020-12-05 05:28:26 +00:00
|
|
|
constexpr bool expose_blob_index = false;
|
|
|
|
constexpr bool allow_refresh = false;
|
2018-06-27 19:05:29 +00:00
|
|
|
std::shared_ptr<ManagedSnapshot> own_snapshot = nullptr;
|
2019-07-23 15:04:58 +00:00
|
|
|
SequenceNumber snapshot_seq = kMaxSequenceNumber;
|
2018-06-27 19:05:29 +00:00
|
|
|
SequenceNumber min_uncommitted = 0;
|
2019-07-23 15:04:58 +00:00
|
|
|
|
|
|
|
// Currently, the Prev() iterator logic does not work well without snapshot
|
|
|
|
// validation. The logic simply iterates through values of a key in
|
|
|
|
// ascending seqno order, stopping at the first non-visible value and
|
|
|
|
// returning the last visible value.
|
|
|
|
//
|
|
|
|
// For example, if snapshot sequence is 3, and we have the following keys:
|
|
|
|
// foo: v1 1
|
|
|
|
// foo: v2 2
|
|
|
|
// foo: v3 3
|
|
|
|
// foo: v4 4
|
|
|
|
// foo: v5 5
|
|
|
|
//
|
|
|
|
// Then 1, 2, 3 will be visible, but 4 will be non-visible, so we return v3,
|
2019-07-24 17:21:18 +00:00
|
|
|
// which is the last visible value.
|
2019-07-23 15:04:58 +00:00
|
|
|
//
|
|
|
|
// For unprepared transactions, if we have snap_seq = 3, but the current
|
2019-07-24 17:21:18 +00:00
|
|
|
// transaction has unprep_seq 5, then returning the first non-visible value
|
2019-07-23 15:04:58 +00:00
|
|
|
// would be incorrect, as we should return v5, and not v3. The problem is that
|
2019-07-24 17:21:18 +00:00
|
|
|
// there are committed values at snapshot_seq < commit_seq < unprep_seq.
|
2019-07-23 15:04:58 +00:00
|
|
|
//
|
|
|
|
// Snapshot validation can prevent this problem by ensuring that no committed
|
2019-07-24 17:21:18 +00:00
|
|
|
// values exist at snapshot_seq < commit_seq, and thus any value with a
|
|
|
|
// sequence number greater than snapshot_seq must be unprepared values. For
|
|
|
|
// example, if the transaction had a snapshot at 3, then snapshot validation
|
|
|
|
// would be performed during the Put(v5) call. It would find v4, and the Put
|
|
|
|
// would fail with snapshot validation failure.
|
2019-07-23 15:04:58 +00:00
|
|
|
//
|
|
|
|
// TODO(lth): Improve Prev() logic to continue iterating until
|
2019-07-24 17:21:18 +00:00
|
|
|
// max_visible_seq, and then return the last visible value, so that this
|
2019-07-23 15:04:58 +00:00
|
|
|
// restriction can be lifted.
|
|
|
|
const Snapshot* snapshot = nullptr;
|
Group rocksdb.sst.read.micros stat by different user read IOActivity + misc (#11444)
Summary:
**Context/Summary:**
- Similar to https://github.com/facebook/rocksdb/pull/11288 but for user read such as `Get(), MultiGet(), DBIterator::XXX(), Verify(File)Checksum()`.
- For this, I refactored some user-facing `MultiGet` calls in `TransactionBase` and various types of `DB` so that it does not call a user-facing `Get()` but `GetImpl()` for passing the `ReadOptions::io_activity` check (see PR conversation)
- New user read stats breakdown are guarded by `kExceptDetailedTimers` since measurement shows they have 4-5% regression to the upstream/main.
- Misc
- More refactoring: with https://github.com/facebook/rocksdb/pull/11288, we complete passing `ReadOptions/IOOptions` to FS level. So we can now replace the previously [added](https://github.com/facebook/rocksdb/pull/9424) `rate_limiter_priority` parameter in `RandomAccessFileReader`'s `Read/MultiRead/Prefetch()` with `IOOptions::rate_limiter_priority`
- Also, `ReadAsync()` call time is measured in `SST_READ_MICRO` now
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11444
Test Plan:
- CI fake db crash/stress test
- Microbenchmarking
**Build** `make clean && ROCKSDB_NO_FBCODE=1 DEBUG_LEVEL=0 make -jN db_basic_bench`
- google benchmark version: https://github.com/google/benchmark/commit/604f6fd3f4b34a84ec4eb4db81d842fa4db829cd
- db_basic_bench_base: upstream
- db_basic_bench_pr: db_basic_bench_base + this PR
- asyncread_db_basic_bench_base: upstream + [db basic bench patch for IteratorNext](https://github.com/facebook/rocksdb/compare/main...hx235:rocksdb:micro_bench_async_read)
- asyncread_db_basic_bench_pr: asyncread_db_basic_bench_base + this PR
**Test**
Get
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{null_stat|base|pr} --benchmark_filter=DBGet/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/negative_query:0/enable_filter:0/mmap:1/threads:1 --benchmark_repetitions=1000
```
Result
```
Coming soon
```
AsyncRead
```
TEST_TMPDIR=/dev/shm ./asyncread_db_basic_bench_{base|pr} --benchmark_filter=IteratorNext/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/async_io:1/include_detailed_timers:0 --benchmark_repetitions=1000 > syncread_db_basic_bench_{base|pr}.out
```
Result
```
Base:
1956,1956,1968,1977,1979,1986,1988,1988,1988,1990,1991,1991,1993,1993,1993,1993,1994,1996,1997,1997,1997,1998,1999,2001,2001,2002,2004,2007,2007,2008,
PR (2.3% regression, due to measuring `SST_READ_MICRO` that wasn't measured before):
1993,2014,2016,2022,2024,2027,2027,2028,2028,2030,2031,2031,2032,2032,2038,2039,2042,2044,2044,2047,2047,2047,2048,2049,2050,2052,2052,2052,2053,2053,
```
Reviewed By: ajkr
Differential Revision: D45918925
Pulled By: hx235
fbshipit-source-id: 58a54560d9ebeb3a59b6d807639692614dad058a
2023-08-09 00:26:50 +00:00
|
|
|
if (read_options.snapshot == nullptr) {
|
2019-07-23 15:04:58 +00:00
|
|
|
snapshot = GetSnapshot();
|
2018-06-27 19:05:29 +00:00
|
|
|
own_snapshot = std::make_shared<ManagedSnapshot>(db_impl_, snapshot);
|
2019-07-23 15:04:58 +00:00
|
|
|
} else {
|
Group rocksdb.sst.read.micros stat by different user read IOActivity + misc (#11444)
Summary:
**Context/Summary:**
- Similar to https://github.com/facebook/rocksdb/pull/11288 but for user read such as `Get(), MultiGet(), DBIterator::XXX(), Verify(File)Checksum()`.
- For this, I refactored some user-facing `MultiGet` calls in `TransactionBase` and various types of `DB` so that it does not call a user-facing `Get()` but `GetImpl()` for passing the `ReadOptions::io_activity` check (see PR conversation)
- New user read stats breakdown are guarded by `kExceptDetailedTimers` since measurement shows they have 4-5% regression to the upstream/main.
- Misc
- More refactoring: with https://github.com/facebook/rocksdb/pull/11288, we complete passing `ReadOptions/IOOptions` to FS level. So we can now replace the previously [added](https://github.com/facebook/rocksdb/pull/9424) `rate_limiter_priority` parameter in `RandomAccessFileReader`'s `Read/MultiRead/Prefetch()` with `IOOptions::rate_limiter_priority`
- Also, `ReadAsync()` call time is measured in `SST_READ_MICRO` now
Pull Request resolved: https://github.com/facebook/rocksdb/pull/11444
Test Plan:
- CI fake db crash/stress test
- Microbenchmarking
**Build** `make clean && ROCKSDB_NO_FBCODE=1 DEBUG_LEVEL=0 make -jN db_basic_bench`
- google benchmark version: https://github.com/google/benchmark/commit/604f6fd3f4b34a84ec4eb4db81d842fa4db829cd
- db_basic_bench_base: upstream
- db_basic_bench_pr: db_basic_bench_base + this PR
- asyncread_db_basic_bench_base: upstream + [db basic bench patch for IteratorNext](https://github.com/facebook/rocksdb/compare/main...hx235:rocksdb:micro_bench_async_read)
- asyncread_db_basic_bench_pr: asyncread_db_basic_bench_base + this PR
**Test**
Get
```
TEST_TMPDIR=/dev/shm ./db_basic_bench_{null_stat|base|pr} --benchmark_filter=DBGet/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/negative_query:0/enable_filter:0/mmap:1/threads:1 --benchmark_repetitions=1000
```
Result
```
Coming soon
```
AsyncRead
```
TEST_TMPDIR=/dev/shm ./asyncread_db_basic_bench_{base|pr} --benchmark_filter=IteratorNext/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1/async_io:1/include_detailed_timers:0 --benchmark_repetitions=1000 > syncread_db_basic_bench_{base|pr}.out
```
Result
```
Base:
1956,1956,1968,1977,1979,1986,1988,1988,1988,1990,1991,1991,1993,1993,1993,1993,1994,1996,1997,1997,1997,1998,1999,2001,2001,2002,2004,2007,2007,2008,
PR (2.3% regression, due to measuring `SST_READ_MICRO` that wasn't measured before):
1993,2014,2016,2022,2024,2027,2027,2028,2028,2030,2031,2031,2032,2032,2038,2039,2042,2044,2044,2047,2047,2047,2048,2049,2050,2052,2052,2052,2053,2053,
```
Reviewed By: ajkr
Differential Revision: D45918925
Pulled By: hx235
fbshipit-source-id: 58a54560d9ebeb3a59b6d807639692614dad058a
2023-08-09 00:26:50 +00:00
|
|
|
snapshot = read_options.snapshot;
|
2018-06-27 19:05:29 +00:00
|
|
|
}
|
2019-07-23 15:04:58 +00:00
|
|
|
|
|
|
|
snapshot_seq = snapshot->GetSequenceNumber();
|
2018-06-27 19:05:29 +00:00
|
|
|
assert(snapshot_seq != kMaxSequenceNumber);
|
2019-07-23 15:04:58 +00:00
|
|
|
// Iteration is safe as long as largest_validated_seq <= snapshot_seq. We are
|
|
|
|
// guaranteed that for keys that were modified by this transaction (and thus
|
2019-07-24 17:21:18 +00:00
|
|
|
// might have unprepared values), no committed values exist at
|
2019-07-23 15:04:58 +00:00
|
|
|
// largest_validated_seq < commit_seq (or the contrapositive: any committed
|
2019-07-24 17:21:18 +00:00
|
|
|
// value must exist at commit_seq <= largest_validated_seq). This implies
|
2019-07-23 15:04:58 +00:00
|
|
|
// that commit_seq <= largest_validated_seq <= snapshot_seq or commit_seq <=
|
|
|
|
// snapshot_seq. As explained above, the problem with Prev() only happens when
|
|
|
|
// snapshot_seq < commit_seq.
|
|
|
|
//
|
|
|
|
// For keys that were not modified by this transaction, largest_validated_seq_
|
|
|
|
// is meaningless, and Prev() should just work with the existing visibility
|
|
|
|
// logic.
|
|
|
|
if (txn->largest_validated_seq_ > snapshot->GetSequenceNumber() &&
|
|
|
|
!txn->unprep_seqs_.empty()) {
|
|
|
|
ROCKS_LOG_ERROR(info_log_,
|
|
|
|
"WriteUnprepared iterator creation failed since the "
|
|
|
|
"transaction has performed unvalidated writes");
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
min_uncommitted =
|
2020-04-29 20:06:27 +00:00
|
|
|
static_cast_with_check<const SnapshotImpl>(snapshot)->min_uncommitted_;
|
2019-07-23 15:04:58 +00:00
|
|
|
|
Access DBImpl* and CFD* by CFHImpl* in Iterators (#12395)
Summary:
In the current implementation of iterators, `DBImpl*` and `ColumnFamilyData*` are held in `DBIter` and `ArenaWrappedDBIter` for two purposes: tracing and Refresh() API. With the introduction of a new iterator called MultiCfIterator in PR https://github.com/facebook/rocksdb/issues/12153 , which is a cross-column-family iterator that maintains multiple DBIters as child iterators from a consistent database state, we need to make some changes to the existing implementation. The new iterator will still be exposed through the generic Iterator interface with an additional capability to return AttributeGroups (via `attribute_groups()`) which is a list of wide columns grouped by column family. For more information about AttributeGroup, please refer to previous PRs: https://github.com/facebook/rocksdb/issues/11925 #11943, and https://github.com/facebook/rocksdb/issues/11977.
To be able to return AttributeGroup in the default single CF iterator created, access to `ColumnFamilyHandle*` within `DBIter` is necessary. However, this is not currently available in `DBIter`. Since `DBImpl*` and `ColumnFamilyData*` can be easily accessed via `ColumnFamilyHandleImpl*`, we have decided to replace the pointers to `ColumnFamilyData` and `DBImpl` in `DBIter` with a pointer to `ColumnFamilyHandleImpl`.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12395
Test Plan:
# Summary
In the current implementation of iterators, `DBImpl*` and `ColumnFamilyData*` are held in `DBIter` and `ArenaWrappedDBIter` for two purposes: tracing and Refresh() API. With the introduction of a new iterator called MultiCfIterator in PR #12153 , which is a cross-column-family iterator that maintains multiple DBIters as child iterators from a consistent database state, we need to make some changes to the existing implementation. The new iterator will still be exposed through the generic Iterator interface with an additional capability to return AttributeGroups (via `attribute_groups()`) which is a list of wide columns grouped by column family. For more information about AttributeGroup, please refer to previous PRs: #11925 #11943, and #11977.
To be able to return AttributeGroup in the default single CF iterator created, access to `ColumnFamilyHandle*` within `DBIter` is necessary. However, this is not currently available in `DBIter`. Since `DBImpl*` and `ColumnFamilyData*` can be easily accessed via `ColumnFamilyHandleImpl*`, we have decided to replace the pointers to `ColumnFamilyData` and `DBImpl` in `DBIter` with a pointer to `ColumnFamilyHandleImpl`.
# Test Plan
There should be no behavior changes. Existing tests and CI for the correctness tests.
**Test for Perf Regression**
Build
```
$> make -j64 release
```
Setup
```
$> TEST_TMPDIR=/dev/shm/db_bench ./db_bench -benchmarks="filluniquerandom" -key_size=32 -value_size=512 -num=1000000 -compression_type=none
```
Run
```
TEST_TMPDIR=/dev/shm/db_bench ./db_bench -use_existing_db=1 -benchmarks="newiterator,seekrandom" -cache_size=10485760000
```
Before the change
```
DB path: [/dev/shm/db_bench/dbbench]
newiterator : 0.552 micros/op 1810157 ops/sec 0.552 seconds 1000000 operations;
DB path: [/dev/shm/db_bench/dbbench]
seekrandom : 4.502 micros/op 222143 ops/sec 4.502 seconds 1000000 operations; (0 of 1000000 found)
```
After the change
```
DB path: [/dev/shm/db_bench/dbbench]
newiterator : 0.520 micros/op 1924401 ops/sec 0.520 seconds 1000000 operations;
DB path: [/dev/shm/db_bench/dbbench]
seekrandom : 4.532 micros/op 220657 ops/sec 4.532 seconds 1000000 operations; (0 of 1000000 found)
```
Reviewed By: pdillinger
Differential Revision: D54332713
Pulled By: jaykorean
fbshipit-source-id: b28d897ad519e58b1ca82eb068a6319544a4fae5
2024-03-01 18:28:20 +00:00
|
|
|
auto* cfh = static_cast_with_check<ColumnFamilyHandleImpl>(column_family);
|
|
|
|
auto* cfd = cfh->cfd();
|
2018-06-27 19:05:29 +00:00
|
|
|
auto* state =
|
|
|
|
new IteratorState(this, snapshot_seq, own_snapshot, min_uncommitted, txn);
|
2023-09-13 23:34:18 +00:00
|
|
|
SuperVersion* super_version = cfd->GetReferencedSuperVersion(db_impl_);
|
2020-12-05 05:28:26 +00:00
|
|
|
auto* db_iter = db_impl_->NewIteratorImpl(
|
Access DBImpl* and CFD* by CFHImpl* in Iterators (#12395)
Summary:
In the current implementation of iterators, `DBImpl*` and `ColumnFamilyData*` are held in `DBIter` and `ArenaWrappedDBIter` for two purposes: tracing and Refresh() API. With the introduction of a new iterator called MultiCfIterator in PR https://github.com/facebook/rocksdb/issues/12153 , which is a cross-column-family iterator that maintains multiple DBIters as child iterators from a consistent database state, we need to make some changes to the existing implementation. The new iterator will still be exposed through the generic Iterator interface with an additional capability to return AttributeGroups (via `attribute_groups()`) which is a list of wide columns grouped by column family. For more information about AttributeGroup, please refer to previous PRs: https://github.com/facebook/rocksdb/issues/11925 #11943, and https://github.com/facebook/rocksdb/issues/11977.
To be able to return AttributeGroup in the default single CF iterator created, access to `ColumnFamilyHandle*` within `DBIter` is necessary. However, this is not currently available in `DBIter`. Since `DBImpl*` and `ColumnFamilyData*` can be easily accessed via `ColumnFamilyHandleImpl*`, we have decided to replace the pointers to `ColumnFamilyData` and `DBImpl` in `DBIter` with a pointer to `ColumnFamilyHandleImpl`.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/12395
Test Plan:
# Summary
In the current implementation of iterators, `DBImpl*` and `ColumnFamilyData*` are held in `DBIter` and `ArenaWrappedDBIter` for two purposes: tracing and Refresh() API. With the introduction of a new iterator called MultiCfIterator in PR #12153 , which is a cross-column-family iterator that maintains multiple DBIters as child iterators from a consistent database state, we need to make some changes to the existing implementation. The new iterator will still be exposed through the generic Iterator interface with an additional capability to return AttributeGroups (via `attribute_groups()`) which is a list of wide columns grouped by column family. For more information about AttributeGroup, please refer to previous PRs: #11925 #11943, and #11977.
To be able to return AttributeGroup in the default single CF iterator created, access to `ColumnFamilyHandle*` within `DBIter` is necessary. However, this is not currently available in `DBIter`. Since `DBImpl*` and `ColumnFamilyData*` can be easily accessed via `ColumnFamilyHandleImpl*`, we have decided to replace the pointers to `ColumnFamilyData` and `DBImpl` in `DBIter` with a pointer to `ColumnFamilyHandleImpl`.
# Test Plan
There should be no behavior changes. Existing tests and CI for the correctness tests.
**Test for Perf Regression**
Build
```
$> make -j64 release
```
Setup
```
$> TEST_TMPDIR=/dev/shm/db_bench ./db_bench -benchmarks="filluniquerandom" -key_size=32 -value_size=512 -num=1000000 -compression_type=none
```
Run
```
TEST_TMPDIR=/dev/shm/db_bench ./db_bench -use_existing_db=1 -benchmarks="newiterator,seekrandom" -cache_size=10485760000
```
Before the change
```
DB path: [/dev/shm/db_bench/dbbench]
newiterator : 0.552 micros/op 1810157 ops/sec 0.552 seconds 1000000 operations;
DB path: [/dev/shm/db_bench/dbbench]
seekrandom : 4.502 micros/op 222143 ops/sec 4.502 seconds 1000000 operations; (0 of 1000000 found)
```
After the change
```
DB path: [/dev/shm/db_bench/dbbench]
newiterator : 0.520 micros/op 1924401 ops/sec 0.520 seconds 1000000 operations;
DB path: [/dev/shm/db_bench/dbbench]
seekrandom : 4.532 micros/op 220657 ops/sec 4.532 seconds 1000000 operations; (0 of 1000000 found)
```
Reviewed By: pdillinger
Differential Revision: D54332713
Pulled By: jaykorean
fbshipit-source-id: b28d897ad519e58b1ca82eb068a6319544a4fae5
2024-03-01 18:28:20 +00:00
|
|
|
read_options, cfh, super_version, state->MaxVisibleSeq(),
|
2023-09-13 23:34:18 +00:00
|
|
|
&state->callback, expose_blob_index, allow_refresh);
|
2018-06-27 19:05:29 +00:00
|
|
|
db_iter->RegisterCleanup(CleanupWriteUnpreparedTxnDBIterator, state, nullptr);
|
|
|
|
return db_iter;
|
|
|
|
}
|
|
|
|
|
2020-02-20 20:07:53 +00:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|