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/pessimistic_transaction.h

244 lines
8.2 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).
#pragma once
#ifndef ROCKSDB_LITE
#include <algorithm>
#include <atomic>
#include <mutex>
#include <stack>
#include <string>
#include <unordered_map>
#include <vector>
#include "db/write_callback.h"
#include "rocksdb/db.h"
#include "rocksdb/slice.h"
#include "rocksdb/snapshot.h"
#include "rocksdb/status.h"
#include "rocksdb/types.h"
#include "rocksdb/utilities/transaction.h"
#include "rocksdb/utilities/transaction_db.h"
#include "rocksdb/utilities/write_batch_with_index.h"
#include "util/autovector.h"
#include "utilities/transactions/transaction_base.h"
#include "utilities/transactions/transaction_util.h"
namespace ROCKSDB_NAMESPACE {
class PessimisticTransactionDB;
// A transaction under pessimistic concurrency control. This class implements
// the locking API and interfaces with the lock manager as well as the
// pessimistic transactional db.
class PessimisticTransaction : public TransactionBaseImpl {
public:
PessimisticTransaction(TransactionDB* db, const WriteOptions& write_options,
const TransactionOptions& txn_options,
const bool init = true);
// No copying allowed
PessimisticTransaction(const PessimisticTransaction&) = delete;
void operator=(const PessimisticTransaction&) = delete;
~PessimisticTransaction() override;
void Reinitialize(TransactionDB* txn_db, const WriteOptions& write_options,
const TransactionOptions& txn_options);
Status Prepare() override;
Status Commit() override;
// It is basically Commit without going through Prepare phase. The write batch
// is also directly provided instead of expecting txn to gradually batch the
// transactions writes to an internal write batch.
Status CommitBatch(WriteBatch* batch);
Status Rollback() override;
Status RollbackToSavePoint() override;
Status SetName(const TransactionName& name) override;
// Generate a new unique transaction identifier
static TransactionID GenTxnID();
TransactionID GetID() const override { return txn_id_; }
std::vector<TransactionID> GetWaitingTxns(uint32_t* column_family_id,
std::string* key) const override {
std::lock_guard<std::mutex> lock(wait_mutex_);
std::vector<TransactionID> ids(waiting_txn_ids_.size());
if (key) *key = waiting_key_ ? *waiting_key_ : "";
if (column_family_id) *column_family_id = waiting_cf_id_;
std::copy(waiting_txn_ids_.begin(), waiting_txn_ids_.end(), ids.begin());
return ids;
}
void SetWaitingTxn(autovector<TransactionID> ids, uint32_t column_family_id,
const std::string* key) {
std::lock_guard<std::mutex> lock(wait_mutex_);
waiting_txn_ids_ = ids;
waiting_cf_id_ = column_family_id;
waiting_key_ = key;
}
void ClearWaitingTxn() {
std::lock_guard<std::mutex> lock(wait_mutex_);
waiting_txn_ids_.clear();
waiting_cf_id_ = 0;
waiting_key_ = nullptr;
}
// Returns the time (in microseconds according to Env->GetMicros())
// that this transaction will be expired. Returns 0 if this transaction does
// not expire.
uint64_t GetExpirationTime() const { return expiration_time_; }
// returns true if this transaction has an expiration_time and has expired.
bool IsExpired() const;
// Returns the number of microseconds a transaction can wait on acquiring a
// lock or -1 if there is no timeout.
int64_t GetLockTimeout() const { return lock_timeout_; }
void SetLockTimeout(int64_t timeout) override {
lock_timeout_ = timeout * 1000;
}
// Returns true if locks were stolen successfully, false otherwise.
bool TryStealingLocks();
bool IsDeadlockDetect() const override { return deadlock_detect_; }
int64_t GetDeadlockDetectDepth() const { return deadlock_detect_depth_; }
virtual Status GetRangeLock(ColumnFamilyHandle* column_family,
const Endpoint& start_key,
const Endpoint& end_key) override;
protected:
// Refer to
// TransactionOptions::use_only_the_last_commit_time_batch_for_recovery
bool use_only_the_last_commit_time_batch_for_recovery_ = false;
// Refer to
// TransactionOptions::skip_prepare
bool skip_prepare_ = false;
virtual Status PrepareInternal() = 0;
virtual Status CommitWithoutPrepareInternal() = 0;
// batch_cnt if non-zero is the number of sub-batches. A sub-batch is a batch
// with no duplicate keys. If zero, then the number of sub-batches is unknown.
virtual Status CommitBatchInternal(WriteBatch* batch,
size_t batch_cnt = 0) = 0;
virtual Status CommitInternal() = 0;
virtual Status RollbackInternal() = 0;
virtual void Initialize(const TransactionOptions& txn_options);
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
Status LockBatch(WriteBatch* batch, LockTracker* keys_to_unlock);
Status TryLock(ColumnFamilyHandle* column_family, const Slice& key,
bool read_only, bool exclusive, const bool do_validate = true,
const bool assume_tracked = false) override;
void Clear() override;
PessimisticTransactionDB* txn_db_impl_;
DBImpl* db_impl_;
// If non-zero, this transaction should not be committed after this time (in
// microseconds according to Env->NowMicros())
uint64_t expiration_time_;
Add commit_timestamp and read_timestamp to Pessimistic transaction (#9537) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9537 Add `Transaction::SetReadTimestampForValidation()` and `Transaction::SetCommitTimestamp()` APIs with default implementation returning `Status::NotSupported()`. Currently, calling these two APIs do not have any effect. Also add checks to `PessimisticTransactionDB` to enforce that column families in the same db either - disable user-defined timestamp - enable 64-bit timestamp Just to clarify, a `PessimisticTransactionDB` can have some column families without timestamps as well as column families that enable timestamp. Each `PessimisticTransaction` can have two optional timestamps, `read_timestamp_` used for additional validation and `commit_timestamp_` which denotes when the transaction commits. For now, we are going to support `WriteCommittedTxn` (in a series of subsequent PRs) Once set, we do not allow decreasing `read_timestamp_`. The `commit_timestamp_` must be greater than `read_timestamp_` for each transaction and must be set before commit, unless the transaction does not involve any column family that enables user-defined timestamp. TransactionDB builds on top of RocksDB core `DB` layer. Though `DB` layer assumes that user-defined timestamps are byte arrays, `TransactionDB` uses uint64_t to store timestamps. When they are passed down, they are still interpreted as byte-arrays by `DB`. Reviewed By: ltamasi Differential Revision: D31567959 fbshipit-source-id: b0b6b69acab5d8e340cf174f33e8b09f1c3d3502
3 years ago
// Timestamp used by the transaction to perform all GetForUpdate.
// Use this timestamp for conflict checking.
// read_timestamp_ == kMaxTxnTimestamp means this transaction has not
// performed any GetForUpdate. It is possible that the transaction has
// performed blind writes or Get, though.
TxnTimestamp read_timestamp_{kMaxTxnTimestamp};
TxnTimestamp commit_timestamp_{kMaxTxnTimestamp};
private:
friend class TransactionTest_ValidateSnapshotTest_Test;
// Used to create unique ids for transactions.
static std::atomic<TransactionID> txn_id_counter_;
// Unique ID for this transaction
TransactionID txn_id_;
// IDs for the transactions that are blocking the current transaction.
//
// empty if current transaction is not waiting.
autovector<TransactionID> waiting_txn_ids_;
// The following two represents the (cf, key) that a transaction is waiting
// on.
//
// If waiting_key_ is not null, then the pointer should always point to
// a valid string object. The reason is that it is only non-null when the
// transaction is blocked in the PointLockManager::AcquireWithTimeout
// function. At that point, the key string object is one of the function
// parameters.
uint32_t waiting_cf_id_;
const std::string* waiting_key_;
// Mutex protecting waiting_txn_ids_, waiting_cf_id_ and waiting_key_.
mutable std::mutex wait_mutex_;
// Timeout in microseconds when locking a key or -1 if there is no timeout.
int64_t lock_timeout_;
// Whether to perform deadlock detection or not.
bool deadlock_detect_;
// Whether to perform deadlock detection or not.
int64_t deadlock_detect_depth_;
// Refer to TransactionOptions::skip_concurrency_control
bool skip_concurrency_control_;
virtual Status ValidateSnapshot(ColumnFamilyHandle* column_family,
const Slice& key,
SequenceNumber* tracked_at_seq);
void UnlockGetForUpdate(ColumnFamilyHandle* column_family,
const Slice& key) override;
};
class WriteCommittedTxn : public PessimisticTransaction {
public:
WriteCommittedTxn(TransactionDB* db, const WriteOptions& write_options,
const TransactionOptions& txn_options);
// No copying allowed
WriteCommittedTxn(const WriteCommittedTxn&) = delete;
void operator=(const WriteCommittedTxn&) = delete;
~WriteCommittedTxn() override {}
Add commit_timestamp and read_timestamp to Pessimistic transaction (#9537) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9537 Add `Transaction::SetReadTimestampForValidation()` and `Transaction::SetCommitTimestamp()` APIs with default implementation returning `Status::NotSupported()`. Currently, calling these two APIs do not have any effect. Also add checks to `PessimisticTransactionDB` to enforce that column families in the same db either - disable user-defined timestamp - enable 64-bit timestamp Just to clarify, a `PessimisticTransactionDB` can have some column families without timestamps as well as column families that enable timestamp. Each `PessimisticTransaction` can have two optional timestamps, `read_timestamp_` used for additional validation and `commit_timestamp_` which denotes when the transaction commits. For now, we are going to support `WriteCommittedTxn` (in a series of subsequent PRs) Once set, we do not allow decreasing `read_timestamp_`. The `commit_timestamp_` must be greater than `read_timestamp_` for each transaction and must be set before commit, unless the transaction does not involve any column family that enables user-defined timestamp. TransactionDB builds on top of RocksDB core `DB` layer. Though `DB` layer assumes that user-defined timestamps are byte arrays, `TransactionDB` uses uint64_t to store timestamps. When they are passed down, they are still interpreted as byte-arrays by `DB`. Reviewed By: ltamasi Differential Revision: D31567959 fbshipit-source-id: b0b6b69acab5d8e340cf174f33e8b09f1c3d3502
3 years ago
Status SetReadTimestampForValidation(TxnTimestamp ts) override;
Status SetCommitTimestamp(TxnTimestamp ts) override;
private:
Status PrepareInternal() override;
Status CommitWithoutPrepareInternal() override;
Status CommitBatchInternal(WriteBatch* batch, size_t batch_cnt) override;
Status CommitInternal() override;
Status RollbackInternal() override;
};
} // namespace ROCKSDB_NAMESPACE
#endif // ROCKSDB_LITE