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/utilities/transactions/transaction_base.cc

724 lines
23 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).
#ifndef ROCKSDB_LITE
#include "utilities/transactions/transaction_base.h"
#include <cinttypes>
#include "db/column_family.h"
#include "db/db_impl/db_impl.h"
#include "logging/logging.h"
#include "rocksdb/comparator.h"
#include "rocksdb/db.h"
#include "rocksdb/status.h"
#include "util/cast_util.h"
#include "util/string_util.h"
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
#include "utilities/transactions/lock/lock_tracker.h"
namespace ROCKSDB_NAMESPACE {
Snapshots with user-specified timestamps (#9879) Summary: In RocksDB, keys are associated with (internal) sequence numbers which denote when the keys are written to the database. Sequence numbers in different RocksDB instances are unrelated, thus not comparable. It is nice if we can associate sequence numbers with their corresponding actual timestamps. One thing we can do is to support user-defined timestamp, which allows the applications to specify the format of custom timestamps and encode a timestamp with each key. More details can be found at https://github.com/facebook/rocksdb/wiki/User-defined-Timestamp-%28Experimental%29. This PR provides a different but complementary approach. We can associate rocksdb snapshots (defined in https://github.com/facebook/rocksdb/blob/7.2.fb/include/rocksdb/snapshot.h#L20) with **user-specified** timestamps. Since a snapshot is essentially an object representing a sequence number, this PR establishes a bi-directional mapping between sequence numbers and timestamps. In the past, snapshots are usually taken by readers. The current super-version is grabbed, and a `rocksdb::Snapshot` object is created with the last published sequence number of the super-version. You can see that the reader actually has no good idea of what timestamp to assign to this snapshot, because by the time the `GetSnapshot()` is called, an arbitrarily long period of time may have already elapsed since the last write, which is when the last published sequence number is written. This observation motivates the creation of "timestamped" snapshots on the write path. Currently, this functionality is exposed only to the layer of `TransactionDB`. Application can tell RocksDB to create a snapshot when a transaction commits, effectively associating the last sequence number with a timestamp. It is also assumed that application will ensure any two snapshots with timestamps should satisfy the following: ``` snapshot1.seq < snapshot2.seq iff. snapshot1.ts < snapshot2.ts ``` If the application can guarantee that when a reader takes a timestamped snapshot, there is no active writes going on in the database, then we also allow the user to use a new API `TransactionDB::CreateTimestampedSnapshot()` to create a snapshot with associated timestamp. Code example ```cpp // Create a timestamped snapshot when committing transaction. txn->SetCommitTimestamp(100); txn->SetSnapshotOnNextOperation(); txn->Commit(); // A wrapper API for convenience Status Transaction::CommitAndTryCreateSnapshot( std::shared_ptr<TransactionNotifier> notifier, TxnTimestamp ts, std::shared_ptr<const Snapshot>* ret); // Create a timestamped snapshot if caller guarantees no concurrent writes std::pair<Status, std::shared_ptr<const Snapshot>> snapshot = txn_db->CreateTimestampedSnapshot(100); ``` The snapshots created in this way will be managed by RocksDB with ref-counting and potentially shared with other readers. We provide the following APIs for readers to retrieve a snapshot given a timestamp. ```cpp // Return the timestamped snapshot correponding to given timestamp. If ts is // kMaxTxnTimestamp, then we return the latest timestamped snapshot if present. // Othersise, we return the snapshot whose timestamp is equal to `ts`. If no // such snapshot exists, then we return null. std::shared_ptr<const Snapshot> TransactionDB::GetTimestampedSnapshot(TxnTimestamp ts) const; // Return the latest timestamped snapshot if present. std::shared_ptr<const Snapshot> TransactionDB::GetLatestTimestampedSnapshot() const; ``` We also provide two additional APIs for stats collection and reporting purposes. ```cpp Status TransactionDB::GetAllTimestampedSnapshots( std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; // Return timestamped snapshots whose timestamps fall in [ts_lb, ts_ub) and store them in `snapshots`. Status TransactionDB::GetTimestampedSnapshots( TxnTimestamp ts_lb, TxnTimestamp ts_ub, std::vector<std::shared_ptr<const Snapshot>>& snapshots) const; ``` To prevent the number of timestamped snapshots from growing infinitely, we provide the following API to release timestamped snapshots whose timestamps are older than or equal to a given threshold. ```cpp void TransactionDB::ReleaseTimestampedSnapshotsOlderThan(TxnTimestamp ts); ``` Before shutdown, RocksDB will release all timestamped snapshots. Comparison with user-defined timestamp and how they can be combined: User-defined timestamp persists every key with a timestamp, while timestamped snapshots maintain a volatile mapping between snapshots (sequence numbers) and timestamps. Different internal keys with the same user key but different timestamps will be treated as different by compaction, thus a newer version will not hide older versions (with smaller timestamps) unless they are eligible for garbage collection. In contrast, taking a timestamped snapshot at a certain sequence number and timestamp prevents all the keys visible in this snapshot from been dropped by compaction. Here, visible means (seq < snapshot and most recent). The timestamped snapshot supports the semantics of reading at an exact point in time. Timestamped snapshots can also be used with user-defined timestamp. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9879 Test Plan: ``` make check TEST_TMPDIR=/dev/shm make crash_test_with_txn ``` Reviewed By: siying Differential Revision: D35783919 Pulled By: riversand963 fbshipit-source-id: 586ad905e169189e19d3bfc0cb0177a7239d1bd4
2 years ago
Status Transaction::CommitAndTryCreateSnapshot(
std::shared_ptr<TransactionNotifier> notifier, TxnTimestamp ts,
std::shared_ptr<const Snapshot>* snapshot) {
if (snapshot) {
snapshot->reset();
}
TxnTimestamp commit_ts = GetCommitTimestamp();
if (commit_ts == kMaxTxnTimestamp) {
if (ts == kMaxTxnTimestamp) {
return Status::InvalidArgument("Commit timestamp unset");
} else {
const Status s = SetCommitTimestamp(ts);
if (!s.ok()) {
return s;
}
}
} else if (ts != kMaxTxnTimestamp) {
if (ts != commit_ts) {
// For now we treat this as error.
return Status::InvalidArgument("Different commit ts specified");
}
}
SetSnapshotOnNextOperation(notifier);
Status s = Commit();
if (!s.ok()) {
return s;
}
assert(s.ok());
// If we reach here, we must return ok status for this function.
std::shared_ptr<const Snapshot> new_snapshot = GetTimestampedSnapshot();
if (snapshot) {
*snapshot = new_snapshot;
}
return Status::OK();
}
TransactionBaseImpl::TransactionBaseImpl(
DB* db, const WriteOptions& write_options,
const LockTrackerFactory& lock_tracker_factory)
: db_(db),
dbimpl_(static_cast_with_check<DBImpl>(db)),
write_options_(write_options),
cmp_(GetColumnFamilyUserComparator(db->DefaultColumnFamily())),
lock_tracker_factory_(lock_tracker_factory),
start_time_(dbimpl_->GetSystemClock()->NowMicros()),
write_batch_(cmp_, 0, true, 0),
tracked_locks_(lock_tracker_factory_.Create()),
indexing_enabled_(true) {
assert(dynamic_cast<DBImpl*>(db_) != nullptr);
log_number_ = 0;
if (dbimpl_->allow_2pc()) {
InitWriteBatch();
}
}
TransactionBaseImpl::~TransactionBaseImpl() {
// Release snapshot if snapshot is set
SetSnapshotInternal(nullptr);
}
void TransactionBaseImpl::Clear() {
save_points_.reset(nullptr);
write_batch_.Clear();
commit_time_batch_.Clear();
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
tracked_locks_->Clear();
num_puts_ = 0;
num_deletes_ = 0;
num_merges_ = 0;
if (dbimpl_->allow_2pc()) {
InitWriteBatch();
}
}
void TransactionBaseImpl::Reinitialize(DB* db,
const WriteOptions& write_options) {
Clear();
ClearSnapshot();
id_ = 0;
db_ = db;
name_.clear();
log_number_ = 0;
write_options_ = write_options;
start_time_ = dbimpl_->GetSystemClock()->NowMicros();
indexing_enabled_ = true;
cmp_ = GetColumnFamilyUserComparator(db_->DefaultColumnFamily());
}
void TransactionBaseImpl::SetSnapshot() {
const Snapshot* snapshot = dbimpl_->GetSnapshotForWriteConflictBoundary();
SetSnapshotInternal(snapshot);
}
void TransactionBaseImpl::SetSnapshotInternal(const Snapshot* snapshot) {
// Set a custom deleter for the snapshot_ SharedPtr as the snapshot needs to
// be released, not deleted when it is no longer referenced.
snapshot_.reset(snapshot, std::bind(&TransactionBaseImpl::ReleaseSnapshot,
this, std::placeholders::_1, db_));
snapshot_needed_ = false;
snapshot_notifier_ = nullptr;
}
void TransactionBaseImpl::SetSnapshotOnNextOperation(
std::shared_ptr<TransactionNotifier> notifier) {
snapshot_needed_ = true;
snapshot_notifier_ = notifier;
}
void TransactionBaseImpl::SetSnapshotIfNeeded() {
if (snapshot_needed_) {
std::shared_ptr<TransactionNotifier> notifier = snapshot_notifier_;
SetSnapshot();
if (notifier != nullptr) {
notifier->SnapshotCreated(GetSnapshot());
}
}
}
Status TransactionBaseImpl::TryLock(ColumnFamilyHandle* column_family,
const SliceParts& key, bool read_only,
bool exclusive, const bool do_validate,
const bool assume_tracked) {
size_t key_size = 0;
for (int i = 0; i < key.num_parts; ++i) {
key_size += key.parts[i].size();
}
std::string str;
str.reserve(key_size);
for (int i = 0; i < key.num_parts; ++i) {
str.append(key.parts[i].data(), key.parts[i].size());
}
return TryLock(column_family, str, read_only, exclusive, do_validate,
assume_tracked);
}
void TransactionBaseImpl::SetSavePoint() {
if (save_points_ == nullptr) {
save_points_.reset(
new std::stack<TransactionBaseImpl::SavePoint,
autovector<TransactionBaseImpl::SavePoint>>());
}
save_points_->emplace(snapshot_, snapshot_needed_, snapshot_notifier_,
num_puts_, num_deletes_, num_merges_,
lock_tracker_factory_);
write_batch_.SetSavePoint();
}
Status TransactionBaseImpl::RollbackToSavePoint() {
if (save_points_ != nullptr && save_points_->size() > 0) {
// Restore saved SavePoint
TransactionBaseImpl::SavePoint& save_point = save_points_->top();
snapshot_ = save_point.snapshot_;
snapshot_needed_ = save_point.snapshot_needed_;
snapshot_notifier_ = save_point.snapshot_notifier_;
num_puts_ = save_point.num_puts_;
num_deletes_ = save_point.num_deletes_;
num_merges_ = save_point.num_merges_;
// Rollback batch
Status s = write_batch_.RollbackToSavePoint();
assert(s.ok());
// Rollback any keys that were tracked since the last savepoint
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
tracked_locks_->Subtract(*save_point.new_locks_);
save_points_->pop();
return s;
} else {
assert(write_batch_.RollbackToSavePoint().IsNotFound());
return Status::NotFound();
}
}
Status TransactionBaseImpl::PopSavePoint() {
if (save_points_ == nullptr ||
save_points_->empty()) {
// No SavePoint yet.
assert(write_batch_.PopSavePoint().IsNotFound());
return Status::NotFound();
}
assert(!save_points_->empty());
// If there is another savepoint A below the current savepoint B, then A needs
// to inherit tracked_keys in B so that if we rollback to savepoint A, we
// remember to unlock keys in B. If there is no other savepoint below, then we
// can safely discard savepoint info.
if (save_points_->size() == 1) {
save_points_->pop();
} else {
TransactionBaseImpl::SavePoint top(lock_tracker_factory_);
std::swap(top, save_points_->top());
save_points_->pop();
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
save_points_->top().new_locks_->Merge(*top.new_locks_);
}
return write_batch_.PopSavePoint();
}
Status TransactionBaseImpl::Get(const ReadOptions& read_options,
ColumnFamilyHandle* column_family,
const Slice& key, std::string* value) {
assert(value != nullptr);
PinnableSlice pinnable_val(value);
assert(!pinnable_val.IsPinned());
auto s = Get(read_options, column_family, key, &pinnable_val);
if (s.ok() && pinnable_val.IsPinned()) {
value->assign(pinnable_val.data(), pinnable_val.size());
} // else value is already assigned
return s;
}
Status TransactionBaseImpl::Get(const ReadOptions& read_options,
ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* pinnable_val) {
return write_batch_.GetFromBatchAndDB(db_, read_options, column_family, key,
pinnable_val);
}
Status TransactionBaseImpl::GetForUpdate(const ReadOptions& read_options,
ColumnFamilyHandle* column_family,
const Slice& key, std::string* value,
bool exclusive,
const bool do_validate) {
if (!do_validate && read_options.snapshot != nullptr) {
return Status::InvalidArgument(
"If do_validate is false then GetForUpdate with snapshot is not "
"defined.");
}
Status s =
TryLock(column_family, key, true /* read_only */, exclusive, do_validate);
if (s.ok() && value != nullptr) {
assert(value != nullptr);
PinnableSlice pinnable_val(value);
assert(!pinnable_val.IsPinned());
s = Get(read_options, column_family, key, &pinnable_val);
if (s.ok() && pinnable_val.IsPinned()) {
value->assign(pinnable_val.data(), pinnable_val.size());
} // else value is already assigned
}
return s;
}
Status TransactionBaseImpl::GetForUpdate(const ReadOptions& read_options,
ColumnFamilyHandle* column_family,
const Slice& key,
PinnableSlice* pinnable_val,
bool exclusive,
const bool do_validate) {
if (!do_validate && read_options.snapshot != nullptr) {
return Status::InvalidArgument(
"If do_validate is false then GetForUpdate with snapshot is not "
"defined.");
}
Status s =
TryLock(column_family, key, true /* read_only */, exclusive, do_validate);
if (s.ok() && pinnable_val != nullptr) {
s = Get(read_options, column_family, key, pinnable_val);
}
return s;
}
std::vector<Status> TransactionBaseImpl::MultiGet(
const ReadOptions& read_options,
const std::vector<ColumnFamilyHandle*>& column_family,
const std::vector<Slice>& keys, std::vector<std::string>* values) {
size_t num_keys = keys.size();
values->resize(num_keys);
std::vector<Status> stat_list(num_keys);
for (size_t i = 0; i < num_keys; ++i) {
stat_list[i] = Get(read_options, column_family[i], keys[i], &(*values)[i]);
}
return stat_list;
}
void TransactionBaseImpl::MultiGet(const ReadOptions& read_options,
ColumnFamilyHandle* column_family,
const size_t num_keys, const Slice* keys,
PinnableSlice* values, Status* statuses,
const bool sorted_input) {
write_batch_.MultiGetFromBatchAndDB(db_, read_options, column_family,
num_keys, keys, values, statuses,
sorted_input);
}
std::vector<Status> TransactionBaseImpl::MultiGetForUpdate(
const ReadOptions& read_options,
const std::vector<ColumnFamilyHandle*>& column_family,
const std::vector<Slice>& keys, std::vector<std::string>* values) {
// Regardless of whether the MultiGet succeeded, track these keys.
size_t num_keys = keys.size();
values->resize(num_keys);
// Lock all keys
for (size_t i = 0; i < num_keys; ++i) {
Status s = TryLock(column_family[i], keys[i], true /* read_only */,
true /* exclusive */);
if (!s.ok()) {
// Fail entire multiget if we cannot lock all keys
return std::vector<Status>(num_keys, s);
}
}
// TODO(agiardullo): optimize multiget?
std::vector<Status> stat_list(num_keys);
for (size_t i = 0; i < num_keys; ++i) {
stat_list[i] = Get(read_options, column_family[i], keys[i], &(*values)[i]);
}
return stat_list;
}
Iterator* TransactionBaseImpl::GetIterator(const ReadOptions& read_options) {
Iterator* db_iter = db_->NewIterator(read_options);
assert(db_iter);
Add Merge Operator support to WriteBatchWithIndex (#8135) Summary: The WBWI has two differing modes of operation dependent on the value of the constructor parameter `overwrite_key`. Currently, regardless of the parameter, neither mode performs as expected when using Merge. This PR remedies this by correctly invoking the appropriate Merge Operator before returning results from the WBWI. Examples of issues that exist which are solved by this PR: ## Example 1 with `overwrite_key=false` Currently, from an empty database, the following sequence: ``` Put('k1', 'v1') Merge('k1', 'v2') Get('k1') ``` Incorrectly yields `v2`, that is to say that the Merge behaves like a Put. ## Example 2 with o`verwrite_key=true` Currently, from an empty database, the following sequence: ``` Put('k1', 'v1') Merge('k1', 'v2') Get('k1') ``` Incorrectly yields `ERROR: kMergeInProgress`. ## Example 3 with `overwrite_key=false` Currently, with a database containing `('k1' -> 'v1')`, the following sequence: ``` Merge('k1', 'v2') GetFromBatchAndDB('k1') ``` Incorrectly yields `v1,v2` ## Example 4 with `overwrite_key=true` Currently, with a database containing `('k1' -> 'v1')`, the following sequence: ``` Merge('k1', 'v1') GetFromBatchAndDB('k1') ``` Incorrectly yields `ERROR: kMergeInProgress`. ## Example 5 with `overwrite_key=false` Currently, from an empty database, the following sequence: ``` Put('k1', 'v1') Merge('k1', 'v2') GetFromBatchAndDB('k1') ``` Incorrectly yields `v1,v2` ## Example 6 with `overwrite_key=true` Currently, from an empty database, `('k1' -> 'v1')`, the following sequence: ``` Put('k1', 'v1') Merge('k1', 'v2') GetFromBatchAndDB('k1') ``` Incorrectly yields `ERROR: kMergeInProgress`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8135 Reviewed By: pdillinger Differential Revision: D27657938 Pulled By: mrambacher fbshipit-source-id: 0fbda6bbc66bedeba96a84786d90141d776297df
3 years ago
return write_batch_.NewIteratorWithBase(db_->DefaultColumnFamily(), db_iter,
&read_options);
}
Iterator* TransactionBaseImpl::GetIterator(const ReadOptions& read_options,
ColumnFamilyHandle* column_family) {
Iterator* db_iter = db_->NewIterator(read_options, column_family);
assert(db_iter);
return write_batch_.NewIteratorWithBase(column_family, db_iter,
&read_options);
}
Status TransactionBaseImpl::Put(ColumnFamilyHandle* column_family,
const Slice& key, const Slice& value,
const bool assume_tracked) {
const bool do_validate = !assume_tracked;
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, do_validate, assume_tracked);
if (s.ok()) {
s = GetBatchForWrite()->Put(column_family, key, value);
if (s.ok()) {
num_puts_++;
}
}
return s;
}
Status TransactionBaseImpl::Put(ColumnFamilyHandle* column_family,
const SliceParts& key, const SliceParts& value,
const bool assume_tracked) {
const bool do_validate = !assume_tracked;
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, do_validate, assume_tracked);
if (s.ok()) {
s = GetBatchForWrite()->Put(column_family, key, value);
if (s.ok()) {
num_puts_++;
}
}
return s;
}
Status TransactionBaseImpl::Merge(ColumnFamilyHandle* column_family,
const Slice& key, const Slice& value,
const bool assume_tracked) {
const bool do_validate = !assume_tracked;
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, do_validate, assume_tracked);
if (s.ok()) {
s = GetBatchForWrite()->Merge(column_family, key, value);
if (s.ok()) {
num_merges_++;
}
}
return s;
}
Status TransactionBaseImpl::Delete(ColumnFamilyHandle* column_family,
const Slice& key,
const bool assume_tracked) {
const bool do_validate = !assume_tracked;
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, do_validate, assume_tracked);
if (s.ok()) {
s = GetBatchForWrite()->Delete(column_family, key);
if (s.ok()) {
num_deletes_++;
}
}
return s;
}
Status TransactionBaseImpl::Delete(ColumnFamilyHandle* column_family,
const SliceParts& key,
const bool assume_tracked) {
const bool do_validate = !assume_tracked;
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, do_validate, assume_tracked);
if (s.ok()) {
s = GetBatchForWrite()->Delete(column_family, key);
if (s.ok()) {
num_deletes_++;
}
}
return s;
}
Status TransactionBaseImpl::SingleDelete(ColumnFamilyHandle* column_family,
const Slice& key,
const bool assume_tracked) {
const bool do_validate = !assume_tracked;
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, do_validate, assume_tracked);
if (s.ok()) {
s = GetBatchForWrite()->SingleDelete(column_family, key);
if (s.ok()) {
num_deletes_++;
}
}
return s;
}
Status TransactionBaseImpl::SingleDelete(ColumnFamilyHandle* column_family,
const SliceParts& key,
const bool assume_tracked) {
const bool do_validate = !assume_tracked;
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, do_validate, assume_tracked);
if (s.ok()) {
s = GetBatchForWrite()->SingleDelete(column_family, key);
if (s.ok()) {
num_deletes_++;
}
}
return s;
}
Status TransactionBaseImpl::PutUntracked(ColumnFamilyHandle* column_family,
const Slice& key, const Slice& value) {
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, false /* do_validate */);
if (s.ok()) {
s = GetBatchForWrite()->Put(column_family, key, value);
if (s.ok()) {
num_puts_++;
}
}
return s;
}
Status TransactionBaseImpl::PutUntracked(ColumnFamilyHandle* column_family,
const SliceParts& key,
const SliceParts& value) {
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, false /* do_validate */);
if (s.ok()) {
s = GetBatchForWrite()->Put(column_family, key, value);
if (s.ok()) {
num_puts_++;
}
}
return s;
}
Status TransactionBaseImpl::MergeUntracked(ColumnFamilyHandle* column_family,
const Slice& key,
const Slice& value) {
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, false /* do_validate */);
if (s.ok()) {
s = GetBatchForWrite()->Merge(column_family, key, value);
if (s.ok()) {
num_merges_++;
}
}
return s;
}
Status TransactionBaseImpl::DeleteUntracked(ColumnFamilyHandle* column_family,
const Slice& key) {
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, false /* do_validate */);
if (s.ok()) {
s = GetBatchForWrite()->Delete(column_family, key);
if (s.ok()) {
num_deletes_++;
}
}
return s;
}
Status TransactionBaseImpl::DeleteUntracked(ColumnFamilyHandle* column_family,
const SliceParts& key) {
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, false /* do_validate */);
if (s.ok()) {
s = GetBatchForWrite()->Delete(column_family, key);
if (s.ok()) {
num_deletes_++;
}
}
return s;
}
Status TransactionBaseImpl::SingleDeleteUntracked(
ColumnFamilyHandle* column_family, const Slice& key) {
Status s = TryLock(column_family, key, false /* read_only */,
true /* exclusive */, false /* do_validate */);
if (s.ok()) {
s = GetBatchForWrite()->SingleDelete(column_family, key);
if (s.ok()) {
num_deletes_++;
}
}
return s;
}
void TransactionBaseImpl::PutLogData(const Slice& blob) {
auto s = write_batch_.PutLogData(blob);
(void)s;
assert(s.ok());
}
WriteBatchWithIndex* TransactionBaseImpl::GetWriteBatch() {
return &write_batch_;
}
uint64_t TransactionBaseImpl::GetElapsedTime() const {
return (dbimpl_->GetSystemClock()->NowMicros() - start_time_) / 1000;
}
uint64_t TransactionBaseImpl::GetNumPuts() const { return num_puts_; }
uint64_t TransactionBaseImpl::GetNumDeletes() const { return num_deletes_; }
uint64_t TransactionBaseImpl::GetNumMerges() const { return num_merges_; }
uint64_t TransactionBaseImpl::GetNumKeys() const {
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
return tracked_locks_->GetNumPointLocks();
}
void TransactionBaseImpl::TrackKey(uint32_t cfh_id, const std::string& key,
SequenceNumber seq, bool read_only,
bool exclusive) {
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
PointLockRequest r;
r.column_family_id = cfh_id;
r.key = key;
r.seq = seq;
r.read_only = read_only;
r.exclusive = exclusive;
// Update map of all tracked keys for this transaction
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
tracked_locks_->Track(r);
if (save_points_ != nullptr && !save_points_->empty()) {
// Update map of tracked keys in this SavePoint
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
save_points_->top().new_locks_->Track(r);
}
}
// Gets the write batch that should be used for Put/Merge/Deletes.
//
// Returns either a WriteBatch or WriteBatchWithIndex depending on whether
// DisableIndexing() has been called.
WriteBatchBase* TransactionBaseImpl::GetBatchForWrite() {
if (indexing_enabled_) {
// Use WriteBatchWithIndex
return &write_batch_;
} else {
// Don't use WriteBatchWithIndex. Return base WriteBatch.
return write_batch_.GetWriteBatch();
}
}
void TransactionBaseImpl::ReleaseSnapshot(const Snapshot* snapshot, DB* db) {
if (snapshot != nullptr) {
ROCKS_LOG_DETAILS(dbimpl_->immutable_db_options().info_log,
"ReleaseSnapshot %" PRIu64 " Set",
snapshot->GetSequenceNumber());
db->ReleaseSnapshot(snapshot);
}
}
void TransactionBaseImpl::UndoGetForUpdate(ColumnFamilyHandle* column_family,
const Slice& key) {
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
PointLockRequest r;
r.column_family_id = GetColumnFamilyID(column_family);
r.key = key.ToString();
r.read_only = true;
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
bool can_untrack = false;
if (save_points_ != nullptr && !save_points_->empty()) {
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
// If there is no GetForUpdate of the key in this save point,
// then cannot untrack from the global lock tracker.
UntrackStatus s = save_points_->top().new_locks_->Untrack(r);
can_untrack = (s != UntrackStatus::NOT_TRACKED);
} else {
Replace tracked_keys with a new LockTracker interface in TransactionDB (#7013) Summary: We're going to support more locking protocols such as range lock in transaction. However, in current design, `TransactionBase` has a member `tracked_keys` which assumes that point lock (lock a single key) is used, and is used in snapshot checking (isolation protocol). When using range lock, we may use read committed instead of snapshot checking as the isolation protocol. The most significant usage scenarios of `tracked_keys` are: 1. pessimistic transaction uses it to track the locked keys, and unlock these keys when commit or rollback. 2. optimistic transaction does not lock keys upfront, it only tracks the lock intentions in tracked_keys, and do write conflict checking when commit. 3. each `SavePoint` tracks the keys that are locked since the `SavePoint`, `RollbackToSavePoint` or `PopSavePoint` relies on both the tracked keys in `SavePoint`s and `tracked_keys`. Based on these scenarios, if we can abstract out a `LockTracker` interface to hold a set of tracked locks (can be keys or key ranges), and have methods that can be composed together to implement the scenarios, then `tracked_keys` can be an internal data structure of one implementation of `LockTracker`. See `utilities/transactions/lock/lock_tracker.h` for the detailed interface design, and `utilities/transactions/lock/point_lock_tracker.cc` for the implementation. In the future, a `RangeLockTracker` can be implemented to track range locks without affecting other components. After this PR, a clean interface for lock manager should be possible, and then ideally, we can have pluggable locking protocols. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7013 Test Plan: Run `transaction_test` and `optimistic_transaction_test`. Reviewed By: ajkr Differential Revision: D22163706 Pulled By: cheng-chang fbshipit-source-id: f2860577b5334e31dd2994f5bc6d7c40d502b1b4
4 years ago
// No save point, so can untrack from the global lock tracker.
can_untrack = true;
}
if (can_untrack) {
// If erased from the global tracker, then can unlock the key.
UntrackStatus s = tracked_locks_->Untrack(r);
bool can_unlock = (s == UntrackStatus::REMOVED);
if (can_unlock) {
UnlockGetForUpdate(column_family, key);
}
}
}
Status TransactionBaseImpl::RebuildFromWriteBatch(WriteBatch* src_batch) {
struct IndexedWriteBatchBuilder : public WriteBatch::Handler {
Transaction* txn_;
DBImpl* db_;
IndexedWriteBatchBuilder(Transaction* txn, DBImpl* db)
: txn_(txn), db_(db) {
assert(dynamic_cast<TransactionBaseImpl*>(txn_) != nullptr);
}
Status PutCF(uint32_t cf, const Slice& key, const Slice& val) override {
return txn_->Put(db_->GetColumnFamilyHandle(cf), key, val);
}
Status DeleteCF(uint32_t cf, const Slice& key) override {
return txn_->Delete(db_->GetColumnFamilyHandle(cf), key);
}
Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
return txn_->SingleDelete(db_->GetColumnFamilyHandle(cf), key);
}
Status MergeCF(uint32_t cf, const Slice& key, const Slice& val) override {
return txn_->Merge(db_->GetColumnFamilyHandle(cf), key, val);
}
// this is used for reconstructing prepared transactions upon
// recovery. there should not be any meta markers in the batches
// we are processing.
Status MarkBeginPrepare(bool) override { return Status::InvalidArgument(); }
Status MarkEndPrepare(const Slice&) override {
return Status::InvalidArgument();
}
Status MarkCommit(const Slice&) override {
return Status::InvalidArgument();
}
Status MarkCommitWithTimestamp(const Slice&, const Slice&) override {
return Status::InvalidArgument();
}
Status MarkRollback(const Slice&) override {
return Status::InvalidArgument();
}
};
IndexedWriteBatchBuilder copycat(this, dbimpl_);
return src_batch->Iterate(&copycat);
}
WriteBatch* TransactionBaseImpl::GetCommitTimeWriteBatch() {
return &commit_time_batch_;
}
} // namespace ROCKSDB_NAMESPACE
#endif // ROCKSDB_LITE