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_db.cc

783 lines
26 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/pessimistic_transaction_db.h"
#include <cinttypes>
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
#include <sstream>
#include <string>
#include <unordered_set>
#include <vector>
#include "db/db_impl/db_impl.h"
#include "logging/logging.h"
#include "rocksdb/db.h"
#include "rocksdb/options.h"
#include "rocksdb/utilities/transaction_db.h"
#include "test_util/sync_point.h"
#include "util/cast_util.h"
#include "util/mutexlock.h"
#include "utilities/transactions/pessimistic_transaction.h"
#include "utilities/transactions/transaction_db_mutex_impl.h"
#include "utilities/transactions/write_prepared_txn_db.h"
#include "utilities/transactions/write_unprepared_txn_db.h"
namespace ROCKSDB_NAMESPACE {
PessimisticTransactionDB::PessimisticTransactionDB(
DB* db, const TransactionDBOptions& txn_db_options)
: TransactionDB(db),
db_impl_(static_cast_with_check<DBImpl>(db)),
txn_db_options_(txn_db_options),
lock_manager_(NewLockManager(this, txn_db_options)) {
assert(db_impl_ != nullptr);
info_log_ = db_impl_->GetDBOptions().info_log;
}
// Support initiliazing PessimisticTransactionDB from a stackable db
//
// PessimisticTransactionDB
// ^ ^
// | |
// | +
// | StackableDB
// | ^
// | |
// + +
// DBImpl
// ^
// |(inherit)
// +
// DB
//
PessimisticTransactionDB::PessimisticTransactionDB(
StackableDB* db, const TransactionDBOptions& txn_db_options)
: TransactionDB(db),
db_impl_(static_cast_with_check<DBImpl>(db->GetRootDB())),
txn_db_options_(txn_db_options),
lock_manager_(NewLockManager(this, txn_db_options)) {
assert(db_impl_ != nullptr);
}
PessimisticTransactionDB::~PessimisticTransactionDB() {
while (!transactions_.empty()) {
delete transactions_.begin()->second;
// TODO(myabandeh): this seems to be an unsafe approach as it is not quite
// clear whether delete would also remove the entry from transactions_.
}
}
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 PessimisticTransactionDB::VerifyCFOptions(
const ColumnFamilyOptions& cf_options) {
const Comparator* const ucmp = cf_options.comparator;
assert(ucmp);
size_t ts_sz = ucmp->timestamp_size();
if (0 == ts_sz) {
return Status::OK();
}
if (ts_sz != sizeof(TxnTimestamp)) {
std::ostringstream oss;
oss << "Timestamp of transaction must have " << sizeof(TxnTimestamp)
<< " bytes. CF comparator " << std::string(ucmp->Name())
<< " timestamp size is " << ts_sz << " bytes";
return Status::InvalidArgument(oss.str());
}
Support user-defined timestamps in write-committed txns (#9629) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9629 Pessimistic transactions use pessimistic concurrency control, i.e. locking. Keys are locked upon first operation that writes the key or has the intention of writing. For example, `PessimisticTransaction::Put()`, `PessimisticTransaction::Delete()`, `PessimisticTransaction::SingleDelete()` will write to or delete a key, while `PessimisticTransaction::GetForUpdate()` is used by application to indicate to RocksDB that the transaction has the intention of performing write operation later in the same transaction. Pessimistic transactions support two-phase commit (2PC). A transaction can be `Prepared()`'ed and then `Commit()`. The prepare phase is similar to a promise: once `Prepare()` succeeds, the transaction has acquired the necessary resources to commit. The resources include locks, persistence of WAL, etc. Write-committed transaction is the default pessimistic transaction implementation. In RocksDB write-committed transaction, `Prepare()` will write data to the WAL as a prepare section. `Commit()` will write a commit marker to the WAL and then write data to the memtables. While writing to the memtables, different keys in the transaction's write batch will be assigned different sequence numbers in ascending order. Until commit/rollback, the transaction holds locks on the keys so that no other transaction can write to the same keys. Furthermore, the keys' sequence numbers represent the order in which they are committed and should be made visible. This is convenient for us to implement support for user-defined timestamps. Since column families with and without timestamps can co-exist in the same database, a transaction may or may not involve timestamps. Based on this observation, we add two optional members to each `PessimisticTransaction`, `read_timestamp_` and `commit_timestamp_`. If no key in the transaction's write batch has timestamp, then setting these two variables do not have any effect. For the rest of this commit, we discuss only the cases when these two variables are meaningful. read_timestamp_ is used mainly for validation, and should be set before first call to `GetForUpdate()`. Otherwise, the latter will return non-ok status. `GetForUpdate()` calls `TryLock()` that can verify if another transaction has written the same key since `read_timestamp_` till this call to `GetForUpdate()`. If another transaction has indeed written the same key, then validation fails, and RocksDB allows this transaction to refine `read_timestamp_` by increasing it. Note that a transaction can still use `Get()` with a different timestamp to read, but the result of the read should not be used to determine data that will be written later. commit_timestamp_ must be set after finishing writing and before transaction commit. This applies to both 2PC and non-2PC cases. In the case of 2PC, it's usually set after prepare phase succeeds. We currently require that the commit timestamp be chosen after all keys are locked. This means we disallow the `TransactionDB`-level APIs if user-defined timestamp is used by the transaction. Specifically, calling `PessimisticTransactionDB::Put()`, `PessimisticTransactionDB::Delete()`, `PessimisticTransactionDB::SingleDelete()`, etc. will return non-ok status because they specify timestamps before locking the keys. Users are also prompted to use the `Transaction` APIs when they receive the non-ok status. Reviewed By: ltamasi Differential Revision: D31822445 fbshipit-source-id: b82abf8e230216dc89cc519564a588224a88fd43
3 years ago
if (txn_db_options_.write_policy != WRITE_COMMITTED) {
return Status::NotSupported("Only WriteCommittedTxn supports timestamp");
}
return Status::OK();
}
Status PessimisticTransactionDB::Initialize(
const std::vector<size_t>& compaction_enabled_cf_indices,
const std::vector<ColumnFamilyHandle*>& handles) {
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]);
}
Status s = EnableAutoCompaction(compaction_enabled_cf_handles);
// create 'real' transactions from recovered shell transactions
auto dbimpl = static_cast_with_check<DBImpl>(GetRootDB());
assert(dbimpl != nullptr);
auto rtrxs = dbimpl->recovered_transactions();
for (auto it = rtrxs.begin(); it != rtrxs.end(); ++it) {
auto recovered_trx = it->second;
assert(recovered_trx);
assert(recovered_trx->batches_.size() == 1);
const auto& seq = recovered_trx->batches_.begin()->first;
const auto& batch_info = recovered_trx->batches_.begin()->second;
assert(batch_info.log_number_);
assert(recovered_trx->name_.length());
WriteOptions w_options;
w_options.sync = true;
TransactionOptions t_options;
// This would help avoiding deadlock for keys that although exist in the WAL
// did not go through concurrency control. This includes the merge that
// MyRocks uses for auto-inc columns. It is safe to do so, since (i) if
// there is a conflict between the keys of two transactions that must be
// avoided, it is already avoided by the application, MyRocks, before the
// restart (ii) application, MyRocks, guarntees to rollback/commit the
// recovered transactions before new transactions start.
t_options.skip_concurrency_control = true;
Transaction* real_trx = BeginTransaction(w_options, t_options, nullptr);
assert(real_trx);
real_trx->SetLogNumber(batch_info.log_number_);
assert(seq != kMaxSequenceNumber);
if (GetTxnDBOptions().write_policy != WRITE_COMMITTED) {
real_trx->SetId(seq);
}
s = real_trx->SetName(recovered_trx->name_);
if (!s.ok()) {
break;
}
s = real_trx->RebuildFromWriteBatch(batch_info.batch_);
// WriteCommitted set this to to disable this check that is specific to
// WritePrepared txns
assert(batch_info.batch_cnt_ == 0 ||
real_trx->GetWriteBatch()->SubBatchCnt() == batch_info.batch_cnt_);
real_trx->SetState(Transaction::PREPARED);
if (!s.ok()) {
break;
}
}
if (s.ok()) {
dbimpl->DeleteAllRecoveredTransactions();
}
return s;
}
Transaction* WriteCommittedTxnDB::BeginTransaction(
const WriteOptions& write_options, const TransactionOptions& txn_options,
Transaction* old_txn) {
if (old_txn != nullptr) {
ReinitializeTransaction(old_txn, write_options, txn_options);
return old_txn;
} else {
return new WriteCommittedTxn(this, write_options, txn_options);
}
}
TransactionDBOptions PessimisticTransactionDB::ValidateTxnDBOptions(
const TransactionDBOptions& txn_db_options) {
TransactionDBOptions validated = txn_db_options;
if (txn_db_options.num_stripes == 0) {
validated.num_stripes = 1;
}
return validated;
}
Status TransactionDB::Open(const Options& options,
const TransactionDBOptions& txn_db_options,
const std::string& dbname, TransactionDB** dbptr) {
DBOptions db_options(options);
ColumnFamilyOptions cf_options(options);
std::vector<ColumnFamilyDescriptor> column_families;
column_families.push_back(
ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
std::vector<ColumnFamilyHandle*> handles;
Status s = TransactionDB::Open(db_options, txn_db_options, dbname,
column_families, &handles, dbptr);
if (s.ok()) {
assert(handles.size() == 1);
// i can delete the handle since DBImpl is always holding a reference to
// default column family
delete handles[0];
}
return s;
}
Status TransactionDB::Open(
const DBOptions& db_options, const TransactionDBOptions& txn_db_options,
const std::string& dbname,
const std::vector<ColumnFamilyDescriptor>& column_families,
std::vector<ColumnFamilyHandle*>* handles, TransactionDB** dbptr) {
Status s;
DB* db = nullptr;
Unordered Writes (#5218) Summary: Performing unordered writes in rocksdb when unordered_write option is set to true. When enabled the writes to memtable are done without joining any write thread. This offers much higher write throughput since the upcoming writes would not have to wait for the slowest memtable write to finish. The tradeoff is that the writes visible to a snapshot might change over time. If the application cannot tolerate that, it should implement its own mechanisms to work around that. Using TransactionDB with WRITE_PREPARED write policy is one way to achieve that. Doing so increases the max throughput by 2.2x without however compromising the snapshot guarantees. The patch is prepared based on an original by siying Existing unit tests are extended to include unordered_write option. Benchmark Results: ``` TEST_TMPDIR=/dev/shm/ ./db_bench_unordered --benchmarks=fillrandom --threads=32 --num=10000000 -max_write_buffer_number=16 --max_background_jobs=64 --batch_size=8 --writes=3000000 -level0_file_num_compaction_trigger=99999 --level0_slowdown_writes_trigger=99999 --level0_stop_writes_trigger=99999 -enable_pipelined_write=false -disable_auto_compactions --unordered_write=1 ``` With WAL - Vanilla RocksDB: 78.6 MB/s - WRITER_PREPARED with unordered_write: 177.8 MB/s (2.2x) - unordered_write: 368.9 MB/s (4.7x with relaxed snapshot guarantees) Without WAL - Vanilla RocksDB: 111.3 MB/s - WRITER_PREPARED with unordered_write: 259.3 MB/s MB/s (2.3x) - unordered_write: 645.6 MB/s (5.8x with relaxed snapshot guarantees) - WRITER_PREPARED with unordered_write disable concurrency control: 185.3 MB/s MB/s (2.35x) Limitations: - The feature is not yet extended to `max_successive_merges` > 0. The feature is also incompatible with `enable_pipelined_write` = true as well as with `allow_concurrent_memtable_write` = false. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5218 Differential Revision: D15219029 Pulled By: maysamyabandeh fbshipit-source-id: 38f2abc4af8780148c6128acdba2b3227bc81759
5 years ago
if (txn_db_options.write_policy == WRITE_COMMITTED &&
db_options.unordered_write) {
return Status::NotSupported(
"WRITE_COMMITTED is incompatible with unordered_writes");
}
if (txn_db_options.write_policy == WRITE_UNPREPARED &&
db_options.unordered_write) {
// TODO(lth): support it
return Status::NotSupported(
"WRITE_UNPREPARED is currently incompatible with unordered_writes");
}
if (txn_db_options.write_policy == WRITE_PREPARED &&
db_options.unordered_write && !db_options.two_write_queues) {
return Status::NotSupported(
"WRITE_PREPARED is incompatible with unordered_writes if "
"two_write_queues is not enabled.");
}
std::vector<ColumnFamilyDescriptor> column_families_copy = column_families;
std::vector<size_t> compaction_enabled_cf_indices;
DBOptions db_options_2pc = db_options;
PrepareWrap(&db_options_2pc, &column_families_copy,
&compaction_enabled_cf_indices);
const bool use_seq_per_batch =
txn_db_options.write_policy == WRITE_PREPARED ||
txn_db_options.write_policy == WRITE_UNPREPARED;
const bool use_batch_per_txn =
txn_db_options.write_policy == WRITE_COMMITTED ||
txn_db_options.write_policy == WRITE_PREPARED;
s = DBImpl::Open(db_options_2pc, dbname, column_families_copy, handles, &db,
use_seq_per_batch, use_batch_per_txn);
if (s.ok()) {
Unordered Writes (#5218) Summary: Performing unordered writes in rocksdb when unordered_write option is set to true. When enabled the writes to memtable are done without joining any write thread. This offers much higher write throughput since the upcoming writes would not have to wait for the slowest memtable write to finish. The tradeoff is that the writes visible to a snapshot might change over time. If the application cannot tolerate that, it should implement its own mechanisms to work around that. Using TransactionDB with WRITE_PREPARED write policy is one way to achieve that. Doing so increases the max throughput by 2.2x without however compromising the snapshot guarantees. The patch is prepared based on an original by siying Existing unit tests are extended to include unordered_write option. Benchmark Results: ``` TEST_TMPDIR=/dev/shm/ ./db_bench_unordered --benchmarks=fillrandom --threads=32 --num=10000000 -max_write_buffer_number=16 --max_background_jobs=64 --batch_size=8 --writes=3000000 -level0_file_num_compaction_trigger=99999 --level0_slowdown_writes_trigger=99999 --level0_stop_writes_trigger=99999 -enable_pipelined_write=false -disable_auto_compactions --unordered_write=1 ``` With WAL - Vanilla RocksDB: 78.6 MB/s - WRITER_PREPARED with unordered_write: 177.8 MB/s (2.2x) - unordered_write: 368.9 MB/s (4.7x with relaxed snapshot guarantees) Without WAL - Vanilla RocksDB: 111.3 MB/s - WRITER_PREPARED with unordered_write: 259.3 MB/s MB/s (2.3x) - unordered_write: 645.6 MB/s (5.8x with relaxed snapshot guarantees) - WRITER_PREPARED with unordered_write disable concurrency control: 185.3 MB/s MB/s (2.35x) Limitations: - The feature is not yet extended to `max_successive_merges` > 0. The feature is also incompatible with `enable_pipelined_write` = true as well as with `allow_concurrent_memtable_write` = false. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5218 Differential Revision: D15219029 Pulled By: maysamyabandeh fbshipit-source-id: 38f2abc4af8780148c6128acdba2b3227bc81759
5 years ago
ROCKS_LOG_WARN(db->GetDBOptions().info_log,
"Transaction write_policy is %" PRId32,
static_cast<int>(txn_db_options.write_policy));
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
// if WrapDB return non-ok, db will be deleted in WrapDB() via
// ~StackableDB().
s = WrapDB(db, txn_db_options, compaction_enabled_cf_indices, *handles,
dbptr);
}
return s;
}
void TransactionDB::PrepareWrap(
DBOptions* db_options, std::vector<ColumnFamilyDescriptor>* column_families,
std::vector<size_t>* compaction_enabled_cf_indices) {
compaction_enabled_cf_indices->clear();
// Enable MemTable History if not already enabled
for (size_t i = 0; i < column_families->size(); i++) {
ColumnFamilyOptions* cf_options = &(*column_families)[i].options;
Refactor trimming logic for immutable memtables (#5022) Summary: MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory. We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one. The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming. In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022 Differential Revision: D14394062 Pulled By: miasantreble fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
5 years ago
if (cf_options->max_write_buffer_size_to_maintain == 0 &&
cf_options->max_write_buffer_number_to_maintain == 0) {
// Setting to -1 will set the History size to
// max_write_buffer_number * write_buffer_size.
cf_options->max_write_buffer_size_to_maintain = -1;
}
if (!cf_options->disable_auto_compactions) {
// Disable compactions momentarily to prevent race with DB::Open
cf_options->disable_auto_compactions = true;
compaction_enabled_cf_indices->push_back(i);
}
}
db_options->allow_2pc = true;
}
namespace {
template <typename DBType>
Status WrapAnotherDBInternal(
DBType* db, const TransactionDBOptions& txn_db_options,
const std::vector<size_t>& compaction_enabled_cf_indices,
const std::vector<ColumnFamilyHandle*>& handles, TransactionDB** dbptr) {
assert(db != nullptr);
assert(dbptr != nullptr);
*dbptr = nullptr;
std::unique_ptr<PessimisticTransactionDB> txn_db;
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
// txn_db owns object pointed to by the raw db pointer.
switch (txn_db_options.write_policy) {
case WRITE_UNPREPARED:
txn_db.reset(new WriteUnpreparedTxnDB(
db, PessimisticTransactionDB::ValidateTxnDBOptions(txn_db_options)));
break;
case WRITE_PREPARED:
txn_db.reset(new WritePreparedTxnDB(
db, PessimisticTransactionDB::ValidateTxnDBOptions(txn_db_options)));
break;
case WRITE_COMMITTED:
default:
txn_db.reset(new WriteCommittedTxnDB(
db, PessimisticTransactionDB::ValidateTxnDBOptions(txn_db_options)));
}
txn_db->UpdateCFComparatorMap(handles);
Status s = txn_db->Initialize(compaction_enabled_cf_indices, handles);
// In case of a failure at this point, db is deleted via the txn_db destructor
// and set to nullptr.
if (s.ok()) {
*dbptr = txn_db.release();
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
} else {
for (auto* h : handles) {
delete h;
}
// txn_db still owns db, and ~StackableDB() will be called when txn_db goes
// out of scope, deleting the input db pointer.
ROCKS_LOG_FATAL(db->GetDBOptions().info_log,
"Failed to initialize txn_db: %s", s.ToString().c_str());
}
return s;
}
} // namespace
Status TransactionDB::WrapDB(
// make sure this db is already opened with memtable history enabled,
// auto compaction distabled and 2 phase commit enabled
DB* db, const TransactionDBOptions& txn_db_options,
const std::vector<size_t>& compaction_enabled_cf_indices,
const std::vector<ColumnFamilyHandle*>& handles, TransactionDB** dbptr) {
return WrapAnotherDBInternal(db, txn_db_options,
compaction_enabled_cf_indices, handles, dbptr);
}
Status TransactionDB::WrapStackableDB(
// make sure this stackable_db is already opened with memtable history
// enabled, auto compaction distabled and 2 phase commit enabled
StackableDB* db, const TransactionDBOptions& txn_db_options,
const std::vector<size_t>& compaction_enabled_cf_indices,
const std::vector<ColumnFamilyHandle*>& handles, TransactionDB** dbptr) {
return WrapAnotherDBInternal(db, txn_db_options,
compaction_enabled_cf_indices, handles, dbptr);
}
// Let LockManager know that this column family exists so it can
// allocate a LockMap for it.
void PessimisticTransactionDB::AddColumnFamily(
const ColumnFamilyHandle* handle) {
lock_manager_->AddColumnFamily(handle);
}
Status PessimisticTransactionDB::CreateColumnFamily(
const ColumnFamilyOptions& options, const std::string& column_family_name,
ColumnFamilyHandle** handle) {
InstrumentedMutexLock l(&column_family_mutex_);
Status s = VerifyCFOptions(options);
if (!s.ok()) {
return s;
}
s = db_->CreateColumnFamily(options, column_family_name, handle);
if (s.ok()) {
lock_manager_->AddColumnFamily(*handle);
UpdateCFComparatorMap(*handle);
}
return s;
}
Status PessimisticTransactionDB::CreateColumnFamilies(
const ColumnFamilyOptions& options,
const std::vector<std::string>& column_family_names,
std::vector<ColumnFamilyHandle*>* handles) {
InstrumentedMutexLock l(&column_family_mutex_);
Status s = VerifyCFOptions(options);
if (!s.ok()) {
return s;
}
s = db_->CreateColumnFamilies(options, column_family_names, handles);
if (s.ok()) {
for (auto* handle : *handles) {
lock_manager_->AddColumnFamily(handle);
UpdateCFComparatorMap(handle);
}
}
return s;
}
Status PessimisticTransactionDB::CreateColumnFamilies(
const std::vector<ColumnFamilyDescriptor>& column_families,
std::vector<ColumnFamilyHandle*>* handles) {
InstrumentedMutexLock l(&column_family_mutex_);
for (auto& cf_desc : column_families) {
Status s = VerifyCFOptions(cf_desc.options);
if (!s.ok()) {
return s;
}
}
Status s = db_->CreateColumnFamilies(column_families, handles);
if (s.ok()) {
for (auto* handle : *handles) {
lock_manager_->AddColumnFamily(handle);
UpdateCFComparatorMap(handle);
}
}
return s;
}
// Let LockManager know that it can deallocate the LockMap for this
// column family.
Status PessimisticTransactionDB::DropColumnFamily(
ColumnFamilyHandle* column_family) {
InstrumentedMutexLock l(&column_family_mutex_);
Status s = db_->DropColumnFamily(column_family);
if (s.ok()) {
lock_manager_->RemoveColumnFamily(column_family);
}
return s;
}
Status PessimisticTransactionDB::DropColumnFamilies(
const std::vector<ColumnFamilyHandle*>& column_families) {
InstrumentedMutexLock l(&column_family_mutex_);
Status s = db_->DropColumnFamilies(column_families);
if (s.ok()) {
for (auto* handle : column_families) {
lock_manager_->RemoveColumnFamily(handle);
}
}
return s;
}
Status PessimisticTransactionDB::TryLock(PessimisticTransaction* txn,
uint32_t cfh_id,
const std::string& key,
bool exclusive) {
return lock_manager_->TryLock(txn, cfh_id, key, GetEnv(), exclusive);
}
Status PessimisticTransactionDB::TryRangeLock(PessimisticTransaction* txn,
uint32_t cfh_id,
const Endpoint& start_endp,
const Endpoint& end_endp) {
return lock_manager_->TryLock(txn, cfh_id, start_endp, end_endp, GetEnv(),
/*exclusive=*/true);
}
void PessimisticTransactionDB::UnLock(PessimisticTransaction* txn,
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
const LockTracker& keys) {
lock_manager_->UnLock(txn, keys, GetEnv());
}
void PessimisticTransactionDB::UnLock(PessimisticTransaction* txn,
uint32_t cfh_id, const std::string& key) {
lock_manager_->UnLock(txn, cfh_id, key, GetEnv());
}
// Used when wrapping DB write operations in a transaction
Transaction* PessimisticTransactionDB::BeginInternalTransaction(
const WriteOptions& options) {
TransactionOptions txn_options;
Transaction* txn = BeginTransaction(options, txn_options, nullptr);
// Use default timeout for non-transactional writes
txn->SetLockTimeout(txn_db_options_.default_lock_timeout);
return txn;
}
// All user Put, Merge, Delete, and Write requests must be intercepted to make
// sure that they lock all keys that they are writing to avoid causing conflicts
// with any concurrent transactions. The easiest way to do this is to wrap all
// write operations in a transaction.
//
// Put(), Merge(), and Delete() only lock a single key per call. Write() will
// sort its keys before locking them. This guarantees that TransactionDB write
// methods cannot deadlock with each other (but still could deadlock with a
// Transaction).
Status PessimisticTransactionDB::Put(const WriteOptions& options,
ColumnFamilyHandle* column_family,
const Slice& key, const Slice& val) {
Support user-defined timestamps in write-committed txns (#9629) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9629 Pessimistic transactions use pessimistic concurrency control, i.e. locking. Keys are locked upon first operation that writes the key or has the intention of writing. For example, `PessimisticTransaction::Put()`, `PessimisticTransaction::Delete()`, `PessimisticTransaction::SingleDelete()` will write to or delete a key, while `PessimisticTransaction::GetForUpdate()` is used by application to indicate to RocksDB that the transaction has the intention of performing write operation later in the same transaction. Pessimistic transactions support two-phase commit (2PC). A transaction can be `Prepared()`'ed and then `Commit()`. The prepare phase is similar to a promise: once `Prepare()` succeeds, the transaction has acquired the necessary resources to commit. The resources include locks, persistence of WAL, etc. Write-committed transaction is the default pessimistic transaction implementation. In RocksDB write-committed transaction, `Prepare()` will write data to the WAL as a prepare section. `Commit()` will write a commit marker to the WAL and then write data to the memtables. While writing to the memtables, different keys in the transaction's write batch will be assigned different sequence numbers in ascending order. Until commit/rollback, the transaction holds locks on the keys so that no other transaction can write to the same keys. Furthermore, the keys' sequence numbers represent the order in which they are committed and should be made visible. This is convenient for us to implement support for user-defined timestamps. Since column families with and without timestamps can co-exist in the same database, a transaction may or may not involve timestamps. Based on this observation, we add two optional members to each `PessimisticTransaction`, `read_timestamp_` and `commit_timestamp_`. If no key in the transaction's write batch has timestamp, then setting these two variables do not have any effect. For the rest of this commit, we discuss only the cases when these two variables are meaningful. read_timestamp_ is used mainly for validation, and should be set before first call to `GetForUpdate()`. Otherwise, the latter will return non-ok status. `GetForUpdate()` calls `TryLock()` that can verify if another transaction has written the same key since `read_timestamp_` till this call to `GetForUpdate()`. If another transaction has indeed written the same key, then validation fails, and RocksDB allows this transaction to refine `read_timestamp_` by increasing it. Note that a transaction can still use `Get()` with a different timestamp to read, but the result of the read should not be used to determine data that will be written later. commit_timestamp_ must be set after finishing writing and before transaction commit. This applies to both 2PC and non-2PC cases. In the case of 2PC, it's usually set after prepare phase succeeds. We currently require that the commit timestamp be chosen after all keys are locked. This means we disallow the `TransactionDB`-level APIs if user-defined timestamp is used by the transaction. Specifically, calling `PessimisticTransactionDB::Put()`, `PessimisticTransactionDB::Delete()`, `PessimisticTransactionDB::SingleDelete()`, etc. will return non-ok status because they specify timestamps before locking the keys. Users are also prompted to use the `Transaction` APIs when they receive the non-ok status. Reviewed By: ltamasi Differential Revision: D31822445 fbshipit-source-id: b82abf8e230216dc89cc519564a588224a88fd43
3 years ago
Status s = FailIfCfEnablesTs(this, column_family);
if (!s.ok()) {
return s;
}
Transaction* txn = BeginInternalTransaction(options);
txn->DisableIndexing();
// Since the client didn't create a transaction, they don't care about
// conflict checking for this write. So we just need to do PutUntracked().
s = txn->PutUntracked(column_family, key, val);
if (s.ok()) {
s = txn->Commit();
}
delete txn;
return s;
}
Status PessimisticTransactionDB::Delete(const WriteOptions& wopts,
ColumnFamilyHandle* column_family,
const Slice& key) {
Support user-defined timestamps in write-committed txns (#9629) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9629 Pessimistic transactions use pessimistic concurrency control, i.e. locking. Keys are locked upon first operation that writes the key or has the intention of writing. For example, `PessimisticTransaction::Put()`, `PessimisticTransaction::Delete()`, `PessimisticTransaction::SingleDelete()` will write to or delete a key, while `PessimisticTransaction::GetForUpdate()` is used by application to indicate to RocksDB that the transaction has the intention of performing write operation later in the same transaction. Pessimistic transactions support two-phase commit (2PC). A transaction can be `Prepared()`'ed and then `Commit()`. The prepare phase is similar to a promise: once `Prepare()` succeeds, the transaction has acquired the necessary resources to commit. The resources include locks, persistence of WAL, etc. Write-committed transaction is the default pessimistic transaction implementation. In RocksDB write-committed transaction, `Prepare()` will write data to the WAL as a prepare section. `Commit()` will write a commit marker to the WAL and then write data to the memtables. While writing to the memtables, different keys in the transaction's write batch will be assigned different sequence numbers in ascending order. Until commit/rollback, the transaction holds locks on the keys so that no other transaction can write to the same keys. Furthermore, the keys' sequence numbers represent the order in which they are committed and should be made visible. This is convenient for us to implement support for user-defined timestamps. Since column families with and without timestamps can co-exist in the same database, a transaction may or may not involve timestamps. Based on this observation, we add two optional members to each `PessimisticTransaction`, `read_timestamp_` and `commit_timestamp_`. If no key in the transaction's write batch has timestamp, then setting these two variables do not have any effect. For the rest of this commit, we discuss only the cases when these two variables are meaningful. read_timestamp_ is used mainly for validation, and should be set before first call to `GetForUpdate()`. Otherwise, the latter will return non-ok status. `GetForUpdate()` calls `TryLock()` that can verify if another transaction has written the same key since `read_timestamp_` till this call to `GetForUpdate()`. If another transaction has indeed written the same key, then validation fails, and RocksDB allows this transaction to refine `read_timestamp_` by increasing it. Note that a transaction can still use `Get()` with a different timestamp to read, but the result of the read should not be used to determine data that will be written later. commit_timestamp_ must be set after finishing writing and before transaction commit. This applies to both 2PC and non-2PC cases. In the case of 2PC, it's usually set after prepare phase succeeds. We currently require that the commit timestamp be chosen after all keys are locked. This means we disallow the `TransactionDB`-level APIs if user-defined timestamp is used by the transaction. Specifically, calling `PessimisticTransactionDB::Put()`, `PessimisticTransactionDB::Delete()`, `PessimisticTransactionDB::SingleDelete()`, etc. will return non-ok status because they specify timestamps before locking the keys. Users are also prompted to use the `Transaction` APIs when they receive the non-ok status. Reviewed By: ltamasi Differential Revision: D31822445 fbshipit-source-id: b82abf8e230216dc89cc519564a588224a88fd43
3 years ago
Status s = FailIfCfEnablesTs(this, column_family);
if (!s.ok()) {
return s;
}
Transaction* txn = BeginInternalTransaction(wopts);
txn->DisableIndexing();
// Since the client didn't create a transaction, they don't care about
// conflict checking for this write. So we just need to do
// DeleteUntracked().
s = txn->DeleteUntracked(column_family, key);
if (s.ok()) {
s = txn->Commit();
}
delete txn;
return s;
}
Status PessimisticTransactionDB::SingleDelete(const WriteOptions& wopts,
ColumnFamilyHandle* column_family,
const Slice& key) {
Support user-defined timestamps in write-committed txns (#9629) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9629 Pessimistic transactions use pessimistic concurrency control, i.e. locking. Keys are locked upon first operation that writes the key or has the intention of writing. For example, `PessimisticTransaction::Put()`, `PessimisticTransaction::Delete()`, `PessimisticTransaction::SingleDelete()` will write to or delete a key, while `PessimisticTransaction::GetForUpdate()` is used by application to indicate to RocksDB that the transaction has the intention of performing write operation later in the same transaction. Pessimistic transactions support two-phase commit (2PC). A transaction can be `Prepared()`'ed and then `Commit()`. The prepare phase is similar to a promise: once `Prepare()` succeeds, the transaction has acquired the necessary resources to commit. The resources include locks, persistence of WAL, etc. Write-committed transaction is the default pessimistic transaction implementation. In RocksDB write-committed transaction, `Prepare()` will write data to the WAL as a prepare section. `Commit()` will write a commit marker to the WAL and then write data to the memtables. While writing to the memtables, different keys in the transaction's write batch will be assigned different sequence numbers in ascending order. Until commit/rollback, the transaction holds locks on the keys so that no other transaction can write to the same keys. Furthermore, the keys' sequence numbers represent the order in which they are committed and should be made visible. This is convenient for us to implement support for user-defined timestamps. Since column families with and without timestamps can co-exist in the same database, a transaction may or may not involve timestamps. Based on this observation, we add two optional members to each `PessimisticTransaction`, `read_timestamp_` and `commit_timestamp_`. If no key in the transaction's write batch has timestamp, then setting these two variables do not have any effect. For the rest of this commit, we discuss only the cases when these two variables are meaningful. read_timestamp_ is used mainly for validation, and should be set before first call to `GetForUpdate()`. Otherwise, the latter will return non-ok status. `GetForUpdate()` calls `TryLock()` that can verify if another transaction has written the same key since `read_timestamp_` till this call to `GetForUpdate()`. If another transaction has indeed written the same key, then validation fails, and RocksDB allows this transaction to refine `read_timestamp_` by increasing it. Note that a transaction can still use `Get()` with a different timestamp to read, but the result of the read should not be used to determine data that will be written later. commit_timestamp_ must be set after finishing writing and before transaction commit. This applies to both 2PC and non-2PC cases. In the case of 2PC, it's usually set after prepare phase succeeds. We currently require that the commit timestamp be chosen after all keys are locked. This means we disallow the `TransactionDB`-level APIs if user-defined timestamp is used by the transaction. Specifically, calling `PessimisticTransactionDB::Put()`, `PessimisticTransactionDB::Delete()`, `PessimisticTransactionDB::SingleDelete()`, etc. will return non-ok status because they specify timestamps before locking the keys. Users are also prompted to use the `Transaction` APIs when they receive the non-ok status. Reviewed By: ltamasi Differential Revision: D31822445 fbshipit-source-id: b82abf8e230216dc89cc519564a588224a88fd43
3 years ago
Status s = FailIfCfEnablesTs(this, column_family);
if (!s.ok()) {
return s;
}
Transaction* txn = BeginInternalTransaction(wopts);
txn->DisableIndexing();
// Since the client didn't create a transaction, they don't care about
// conflict checking for this write. So we just need to do
// SingleDeleteUntracked().
s = txn->SingleDeleteUntracked(column_family, key);
if (s.ok()) {
s = txn->Commit();
}
delete txn;
return s;
}
Status PessimisticTransactionDB::Merge(const WriteOptions& options,
ColumnFamilyHandle* column_family,
const Slice& key, const Slice& value) {
Support user-defined timestamps in write-committed txns (#9629) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9629 Pessimistic transactions use pessimistic concurrency control, i.e. locking. Keys are locked upon first operation that writes the key or has the intention of writing. For example, `PessimisticTransaction::Put()`, `PessimisticTransaction::Delete()`, `PessimisticTransaction::SingleDelete()` will write to or delete a key, while `PessimisticTransaction::GetForUpdate()` is used by application to indicate to RocksDB that the transaction has the intention of performing write operation later in the same transaction. Pessimistic transactions support two-phase commit (2PC). A transaction can be `Prepared()`'ed and then `Commit()`. The prepare phase is similar to a promise: once `Prepare()` succeeds, the transaction has acquired the necessary resources to commit. The resources include locks, persistence of WAL, etc. Write-committed transaction is the default pessimistic transaction implementation. In RocksDB write-committed transaction, `Prepare()` will write data to the WAL as a prepare section. `Commit()` will write a commit marker to the WAL and then write data to the memtables. While writing to the memtables, different keys in the transaction's write batch will be assigned different sequence numbers in ascending order. Until commit/rollback, the transaction holds locks on the keys so that no other transaction can write to the same keys. Furthermore, the keys' sequence numbers represent the order in which they are committed and should be made visible. This is convenient for us to implement support for user-defined timestamps. Since column families with and without timestamps can co-exist in the same database, a transaction may or may not involve timestamps. Based on this observation, we add two optional members to each `PessimisticTransaction`, `read_timestamp_` and `commit_timestamp_`. If no key in the transaction's write batch has timestamp, then setting these two variables do not have any effect. For the rest of this commit, we discuss only the cases when these two variables are meaningful. read_timestamp_ is used mainly for validation, and should be set before first call to `GetForUpdate()`. Otherwise, the latter will return non-ok status. `GetForUpdate()` calls `TryLock()` that can verify if another transaction has written the same key since `read_timestamp_` till this call to `GetForUpdate()`. If another transaction has indeed written the same key, then validation fails, and RocksDB allows this transaction to refine `read_timestamp_` by increasing it. Note that a transaction can still use `Get()` with a different timestamp to read, but the result of the read should not be used to determine data that will be written later. commit_timestamp_ must be set after finishing writing and before transaction commit. This applies to both 2PC and non-2PC cases. In the case of 2PC, it's usually set after prepare phase succeeds. We currently require that the commit timestamp be chosen after all keys are locked. This means we disallow the `TransactionDB`-level APIs if user-defined timestamp is used by the transaction. Specifically, calling `PessimisticTransactionDB::Put()`, `PessimisticTransactionDB::Delete()`, `PessimisticTransactionDB::SingleDelete()`, etc. will return non-ok status because they specify timestamps before locking the keys. Users are also prompted to use the `Transaction` APIs when they receive the non-ok status. Reviewed By: ltamasi Differential Revision: D31822445 fbshipit-source-id: b82abf8e230216dc89cc519564a588224a88fd43
3 years ago
Status s = FailIfCfEnablesTs(this, column_family);
if (!s.ok()) {
return s;
}
Transaction* txn = BeginInternalTransaction(options);
txn->DisableIndexing();
// Since the client didn't create a transaction, they don't care about
// conflict checking for this write. So we just need to do
// MergeUntracked().
s = txn->MergeUntracked(column_family, key, value);
if (s.ok()) {
s = txn->Commit();
}
delete txn;
return s;
}
Status PessimisticTransactionDB::Write(const WriteOptions& opts,
WriteBatch* updates) {
return WriteWithConcurrencyControl(opts, updates);
}
Status WriteCommittedTxnDB::Write(const WriteOptions& opts,
WriteBatch* updates) {
Support user-defined timestamps in write-committed txns (#9629) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9629 Pessimistic transactions use pessimistic concurrency control, i.e. locking. Keys are locked upon first operation that writes the key or has the intention of writing. For example, `PessimisticTransaction::Put()`, `PessimisticTransaction::Delete()`, `PessimisticTransaction::SingleDelete()` will write to or delete a key, while `PessimisticTransaction::GetForUpdate()` is used by application to indicate to RocksDB that the transaction has the intention of performing write operation later in the same transaction. Pessimistic transactions support two-phase commit (2PC). A transaction can be `Prepared()`'ed and then `Commit()`. The prepare phase is similar to a promise: once `Prepare()` succeeds, the transaction has acquired the necessary resources to commit. The resources include locks, persistence of WAL, etc. Write-committed transaction is the default pessimistic transaction implementation. In RocksDB write-committed transaction, `Prepare()` will write data to the WAL as a prepare section. `Commit()` will write a commit marker to the WAL and then write data to the memtables. While writing to the memtables, different keys in the transaction's write batch will be assigned different sequence numbers in ascending order. Until commit/rollback, the transaction holds locks on the keys so that no other transaction can write to the same keys. Furthermore, the keys' sequence numbers represent the order in which they are committed and should be made visible. This is convenient for us to implement support for user-defined timestamps. Since column families with and without timestamps can co-exist in the same database, a transaction may or may not involve timestamps. Based on this observation, we add two optional members to each `PessimisticTransaction`, `read_timestamp_` and `commit_timestamp_`. If no key in the transaction's write batch has timestamp, then setting these two variables do not have any effect. For the rest of this commit, we discuss only the cases when these two variables are meaningful. read_timestamp_ is used mainly for validation, and should be set before first call to `GetForUpdate()`. Otherwise, the latter will return non-ok status. `GetForUpdate()` calls `TryLock()` that can verify if another transaction has written the same key since `read_timestamp_` till this call to `GetForUpdate()`. If another transaction has indeed written the same key, then validation fails, and RocksDB allows this transaction to refine `read_timestamp_` by increasing it. Note that a transaction can still use `Get()` with a different timestamp to read, but the result of the read should not be used to determine data that will be written later. commit_timestamp_ must be set after finishing writing and before transaction commit. This applies to both 2PC and non-2PC cases. In the case of 2PC, it's usually set after prepare phase succeeds. We currently require that the commit timestamp be chosen after all keys are locked. This means we disallow the `TransactionDB`-level APIs if user-defined timestamp is used by the transaction. Specifically, calling `PessimisticTransactionDB::Put()`, `PessimisticTransactionDB::Delete()`, `PessimisticTransactionDB::SingleDelete()`, etc. will return non-ok status because they specify timestamps before locking the keys. Users are also prompted to use the `Transaction` APIs when they receive the non-ok status. Reviewed By: ltamasi Differential Revision: D31822445 fbshipit-source-id: b82abf8e230216dc89cc519564a588224a88fd43
3 years ago
Status s = FailIfBatchHasTs(updates);
if (!s.ok()) {
return s;
}
if (txn_db_options_.skip_concurrency_control) {
return db_impl_->Write(opts, updates);
} else {
return WriteWithConcurrencyControl(opts, updates);
}
}
Status WriteCommittedTxnDB::Write(
const WriteOptions& opts,
const TransactionDBWriteOptimizations& optimizations, WriteBatch* updates) {
Support user-defined timestamps in write-committed txns (#9629) Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/9629 Pessimistic transactions use pessimistic concurrency control, i.e. locking. Keys are locked upon first operation that writes the key or has the intention of writing. For example, `PessimisticTransaction::Put()`, `PessimisticTransaction::Delete()`, `PessimisticTransaction::SingleDelete()` will write to or delete a key, while `PessimisticTransaction::GetForUpdate()` is used by application to indicate to RocksDB that the transaction has the intention of performing write operation later in the same transaction. Pessimistic transactions support two-phase commit (2PC). A transaction can be `Prepared()`'ed and then `Commit()`. The prepare phase is similar to a promise: once `Prepare()` succeeds, the transaction has acquired the necessary resources to commit. The resources include locks, persistence of WAL, etc. Write-committed transaction is the default pessimistic transaction implementation. In RocksDB write-committed transaction, `Prepare()` will write data to the WAL as a prepare section. `Commit()` will write a commit marker to the WAL and then write data to the memtables. While writing to the memtables, different keys in the transaction's write batch will be assigned different sequence numbers in ascending order. Until commit/rollback, the transaction holds locks on the keys so that no other transaction can write to the same keys. Furthermore, the keys' sequence numbers represent the order in which they are committed and should be made visible. This is convenient for us to implement support for user-defined timestamps. Since column families with and without timestamps can co-exist in the same database, a transaction may or may not involve timestamps. Based on this observation, we add two optional members to each `PessimisticTransaction`, `read_timestamp_` and `commit_timestamp_`. If no key in the transaction's write batch has timestamp, then setting these two variables do not have any effect. For the rest of this commit, we discuss only the cases when these two variables are meaningful. read_timestamp_ is used mainly for validation, and should be set before first call to `GetForUpdate()`. Otherwise, the latter will return non-ok status. `GetForUpdate()` calls `TryLock()` that can verify if another transaction has written the same key since `read_timestamp_` till this call to `GetForUpdate()`. If another transaction has indeed written the same key, then validation fails, and RocksDB allows this transaction to refine `read_timestamp_` by increasing it. Note that a transaction can still use `Get()` with a different timestamp to read, but the result of the read should not be used to determine data that will be written later. commit_timestamp_ must be set after finishing writing and before transaction commit. This applies to both 2PC and non-2PC cases. In the case of 2PC, it's usually set after prepare phase succeeds. We currently require that the commit timestamp be chosen after all keys are locked. This means we disallow the `TransactionDB`-level APIs if user-defined timestamp is used by the transaction. Specifically, calling `PessimisticTransactionDB::Put()`, `PessimisticTransactionDB::Delete()`, `PessimisticTransactionDB::SingleDelete()`, etc. will return non-ok status because they specify timestamps before locking the keys. Users are also prompted to use the `Transaction` APIs when they receive the non-ok status. Reviewed By: ltamasi Differential Revision: D31822445 fbshipit-source-id: b82abf8e230216dc89cc519564a588224a88fd43
3 years ago
Status s = FailIfBatchHasTs(updates);
if (!s.ok()) {
return s;
}
if (optimizations.skip_concurrency_control) {
return db_impl_->Write(opts, updates);
} else {
return WriteWithConcurrencyControl(opts, updates);
}
}
void PessimisticTransactionDB::InsertExpirableTransaction(
TransactionID tx_id, PessimisticTransaction* tx) {
assert(tx->GetExpirationTime() > 0);
std::lock_guard<std::mutex> lock(map_mutex_);
expirable_transactions_map_.insert({tx_id, tx});
}
void PessimisticTransactionDB::RemoveExpirableTransaction(TransactionID tx_id) {
std::lock_guard<std::mutex> lock(map_mutex_);
expirable_transactions_map_.erase(tx_id);
}
bool PessimisticTransactionDB::TryStealingExpiredTransactionLocks(
TransactionID tx_id) {
std::lock_guard<std::mutex> lock(map_mutex_);
auto tx_it = expirable_transactions_map_.find(tx_id);
if (tx_it == expirable_transactions_map_.end()) {
return true;
}
PessimisticTransaction& tx = *(tx_it->second);
return tx.TryStealingLocks();
}
void PessimisticTransactionDB::ReinitializeTransaction(
Transaction* txn, const WriteOptions& write_options,
const TransactionOptions& txn_options) {
auto txn_impl = static_cast_with_check<PessimisticTransaction>(txn);
txn_impl->Reinitialize(this, write_options, txn_options);
}
Transaction* PessimisticTransactionDB::GetTransactionByName(
const TransactionName& name) {
std::lock_guard<std::mutex> lock(name_map_mutex_);
auto it = transactions_.find(name);
if (it == transactions_.end()) {
return nullptr;
} else {
return it->second;
}
}
void PessimisticTransactionDB::GetAllPreparedTransactions(
std::vector<Transaction*>* transv) {
assert(transv);
transv->clear();
std::lock_guard<std::mutex> lock(name_map_mutex_);
for (auto it = transactions_.begin(); it != transactions_.end(); ++it) {
if (it->second->GetState() == Transaction::PREPARED) {
transv->push_back(it->second);
}
}
}
LockManager::PointLockStatus PessimisticTransactionDB::GetLockStatusData() {
return lock_manager_->GetPointLockStatus();
}
std::vector<DeadlockPath> PessimisticTransactionDB::GetDeadlockInfoBuffer() {
return lock_manager_->GetDeadlockInfoBuffer();
}
void PessimisticTransactionDB::SetDeadlockInfoBufferSize(uint32_t target_size) {
lock_manager_->Resize(target_size);
}
void PessimisticTransactionDB::RegisterTransaction(Transaction* txn) {
assert(txn);
assert(txn->GetName().length() > 0);
assert(GetTransactionByName(txn->GetName()) == nullptr);
assert(txn->GetState() == Transaction::STARTED);
std::lock_guard<std::mutex> lock(name_map_mutex_);
transactions_[txn->GetName()] = txn;
}
void PessimisticTransactionDB::UnregisterTransaction(Transaction* txn) {
assert(txn);
std::lock_guard<std::mutex> lock(name_map_mutex_);
auto it = transactions_.find(txn->GetName());
assert(it != transactions_.end());
transactions_.erase(it);
}
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
std::pair<Status, std::shared_ptr<const Snapshot>>
PessimisticTransactionDB::CreateTimestampedSnapshot(TxnTimestamp ts) {
if (kMaxTxnTimestamp == ts) {
return std::make_pair(Status::InvalidArgument("invalid ts"), nullptr);
}
assert(db_impl_);
return db_impl_->CreateTimestampedSnapshot(kMaxSequenceNumber, ts);
}
std::shared_ptr<const Snapshot>
PessimisticTransactionDB::GetTimestampedSnapshot(TxnTimestamp ts) const {
assert(db_impl_);
return db_impl_->GetTimestampedSnapshot(ts);
}
void PessimisticTransactionDB::ReleaseTimestampedSnapshotsOlderThan(
TxnTimestamp ts) {
assert(db_impl_);
db_impl_->ReleaseTimestampedSnapshotsOlderThan(ts);
}
Status PessimisticTransactionDB::GetTimestampedSnapshots(
TxnTimestamp ts_lb, TxnTimestamp ts_ub,
std::vector<std::shared_ptr<const Snapshot>>& timestamped_snapshots) const {
assert(db_impl_);
return db_impl_->GetTimestampedSnapshots(ts_lb, ts_ub, timestamped_snapshots);
}
Status SnapshotCreationCallback::operator()(SequenceNumber seq,
bool disable_memtable) {
assert(db_impl_);
assert(commit_ts_ != kMaxTxnTimestamp);
const bool two_write_queues =
db_impl_->immutable_db_options().two_write_queues;
assert(!two_write_queues || !disable_memtable);
#ifdef NDEBUG
(void)two_write_queues;
(void)disable_memtable;
#endif
const bool seq_per_batch = db_impl_->seq_per_batch();
if (!seq_per_batch) {
assert(db_impl_->GetLastPublishedSequence() <= seq);
} else {
assert(db_impl_->GetLastPublishedSequence() < seq);
}
// Create a snapshot which can also be used for write conflict checking.
auto ret = db_impl_->CreateTimestampedSnapshot(seq, commit_ts_);
snapshot_creation_status_ = ret.first;
snapshot_ = ret.second;
if (snapshot_creation_status_.ok()) {
assert(snapshot_);
} else {
assert(!snapshot_);
}
if (snapshot_ && snapshot_notifier_) {
snapshot_notifier_->SnapshotCreated(snapshot_.get());
}
return Status::OK();
}
} // namespace ROCKSDB_NAMESPACE
#endif // ROCKSDB_LITE