summaryrefslogtreecommitdiffstats
path: root/src/rocksdb/utilities/transactions
diff options
context:
space:
mode:
Diffstat (limited to '')
-rw-r--r--src/rocksdb/utilities/transactions/optimistic_transaction.cc187
-rw-r--r--src/rocksdb/utilities/transactions/optimistic_transaction.h101
-rw-r--r--src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.cc111
-rw-r--r--src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.h71
-rw-r--r--src/rocksdb/utilities/transactions/optimistic_transaction_test.cc1535
-rw-r--r--src/rocksdb/utilities/transactions/pessimistic_transaction.cc723
-rw-r--r--src/rocksdb/utilities/transactions/pessimistic_transaction.h225
-rw-r--r--src/rocksdb/utilities/transactions/pessimistic_transaction_db.cc632
-rw-r--r--src/rocksdb/utilities/transactions/pessimistic_transaction_db.h220
-rw-r--r--src/rocksdb/utilities/transactions/snapshot_checker.cc49
-rw-r--r--src/rocksdb/utilities/transactions/transaction_base.cc837
-rw-r--r--src/rocksdb/utilities/transactions/transaction_base.h374
-rw-r--r--src/rocksdb/utilities/transactions/transaction_db_mutex_impl.cc135
-rw-r--r--src/rocksdb/utilities/transactions/transaction_db_mutex_impl.h26
-rw-r--r--src/rocksdb/utilities/transactions/transaction_lock_mgr.cc745
-rw-r--r--src/rocksdb/utilities/transactions/transaction_lock_mgr.h158
-rw-r--r--src/rocksdb/utilities/transactions/transaction_test.cc6224
-rw-r--r--src/rocksdb/utilities/transactions/transaction_test.h517
-rw-r--r--src/rocksdb/utilities/transactions/transaction_util.cc182
-rw-r--r--src/rocksdb/utilities/transactions/transaction_util.h103
-rw-r--r--src/rocksdb/utilities/transactions/write_prepared_transaction_test.cc3524
-rw-r--r--src/rocksdb/utilities/transactions/write_prepared_txn.cc473
-rw-r--r--src/rocksdb/utilities/transactions/write_prepared_txn.h119
-rw-r--r--src/rocksdb/utilities/transactions/write_prepared_txn_db.cc998
-rw-r--r--src/rocksdb/utilities/transactions/write_prepared_txn_db.h1111
-rw-r--r--src/rocksdb/utilities/transactions/write_unprepared_transaction_test.cc727
-rw-r--r--src/rocksdb/utilities/transactions/write_unprepared_txn.cc999
-rw-r--r--src/rocksdb/utilities/transactions/write_unprepared_txn.h341
-rw-r--r--src/rocksdb/utilities/transactions/write_unprepared_txn_db.cc468
-rw-r--r--src/rocksdb/utilities/transactions/write_unprepared_txn_db.h148
30 files changed, 22063 insertions, 0 deletions
diff --git a/src/rocksdb/utilities/transactions/optimistic_transaction.cc b/src/rocksdb/utilities/transactions/optimistic_transaction.cc
new file mode 100644
index 000000000..b01102bb2
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/optimistic_transaction.cc
@@ -0,0 +1,187 @@
+// 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/optimistic_transaction.h"
+
+#include <string>
+
+#include "db/column_family.h"
+#include "db/db_impl/db_impl.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/db.h"
+#include "rocksdb/status.h"
+#include "rocksdb/utilities/optimistic_transaction_db.h"
+#include "util/cast_util.h"
+#include "util/string_util.h"
+#include "utilities/transactions/transaction_util.h"
+#include "utilities/transactions/optimistic_transaction.h"
+#include "utilities/transactions/optimistic_transaction_db_impl.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct WriteOptions;
+
+OptimisticTransaction::OptimisticTransaction(
+ OptimisticTransactionDB* txn_db, const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options)
+ : TransactionBaseImpl(txn_db->GetBaseDB(), write_options), txn_db_(txn_db) {
+ Initialize(txn_options);
+}
+
+void OptimisticTransaction::Initialize(
+ const OptimisticTransactionOptions& txn_options) {
+ if (txn_options.set_snapshot) {
+ SetSnapshot();
+ }
+}
+
+void OptimisticTransaction::Reinitialize(
+ OptimisticTransactionDB* txn_db, const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options) {
+ TransactionBaseImpl::Reinitialize(txn_db->GetBaseDB(), write_options);
+ Initialize(txn_options);
+}
+
+OptimisticTransaction::~OptimisticTransaction() {}
+
+void OptimisticTransaction::Clear() { TransactionBaseImpl::Clear(); }
+
+Status OptimisticTransaction::Prepare() {
+ return Status::InvalidArgument(
+ "Two phase commit not supported for optimistic transactions.");
+}
+
+Status OptimisticTransaction::Commit() {
+ auto txn_db_impl = static_cast_with_check<OptimisticTransactionDBImpl,
+ OptimisticTransactionDB>(txn_db_);
+ assert(txn_db_impl);
+ switch (txn_db_impl->GetValidatePolicy()) {
+ case OccValidationPolicy::kValidateParallel:
+ return CommitWithParallelValidate();
+ case OccValidationPolicy::kValidateSerial:
+ return CommitWithSerialValidate();
+ default:
+ assert(0);
+ }
+ // unreachable, just void compiler complain
+ return Status::OK();
+}
+
+Status OptimisticTransaction::CommitWithSerialValidate() {
+ // Set up callback which will call CheckTransactionForConflicts() to
+ // check whether this transaction is safe to be committed.
+ OptimisticTransactionCallback callback(this);
+
+ DBImpl* db_impl = static_cast_with_check<DBImpl, DB>(db_->GetRootDB());
+
+ Status s = db_impl->WriteWithCallback(
+ write_options_, GetWriteBatch()->GetWriteBatch(), &callback);
+
+ if (s.ok()) {
+ Clear();
+ }
+
+ return s;
+}
+
+Status OptimisticTransaction::CommitWithParallelValidate() {
+ auto txn_db_impl = static_cast_with_check<OptimisticTransactionDBImpl,
+ OptimisticTransactionDB>(txn_db_);
+ assert(txn_db_impl);
+ DBImpl* db_impl = static_cast_with_check<DBImpl, DB>(db_->GetRootDB());
+ assert(db_impl);
+ const size_t space = txn_db_impl->GetLockBucketsSize();
+ std::set<size_t> lk_idxes;
+ std::vector<std::unique_lock<std::mutex>> lks;
+ for (auto& cfit : GetTrackedKeys()) {
+ for (auto& keyit : cfit.second) {
+ lk_idxes.insert(fastrange64(GetSliceNPHash64(keyit.first), space));
+ }
+ }
+ // NOTE: in a single txn, all bucket-locks are taken in ascending order.
+ // In this way, txns from different threads all obey this rule so that
+ // deadlock can be avoided.
+ for (auto v : lk_idxes) {
+ lks.emplace_back(txn_db_impl->LockBucket(v));
+ }
+
+ Status s = TransactionUtil::CheckKeysForConflicts(db_impl, GetTrackedKeys(),
+ true /* cache_only */);
+ if (!s.ok()) {
+ return s;
+ }
+
+ s = db_impl->Write(write_options_, GetWriteBatch()->GetWriteBatch());
+ if (s.ok()) {
+ Clear();
+ }
+
+ return s;
+}
+
+Status OptimisticTransaction::Rollback() {
+ Clear();
+ return Status::OK();
+}
+
+// Record this key so that we can check it for conflicts at commit time.
+//
+// 'exclusive' is unused for OptimisticTransaction.
+Status OptimisticTransaction::TryLock(ColumnFamilyHandle* column_family,
+ const Slice& key, bool read_only,
+ bool exclusive, const bool do_validate,
+ const bool assume_tracked) {
+ assert(!assume_tracked); // not supported
+ (void)assume_tracked;
+ if (!do_validate) {
+ return Status::OK();
+ }
+ uint32_t cfh_id = GetColumnFamilyID(column_family);
+
+ SetSnapshotIfNeeded();
+
+ SequenceNumber seq;
+ if (snapshot_) {
+ seq = snapshot_->GetSequenceNumber();
+ } else {
+ seq = db_->GetLatestSequenceNumber();
+ }
+
+ std::string key_str = key.ToString();
+
+ TrackKey(cfh_id, key_str, seq, read_only, exclusive);
+
+ // Always return OK. Confilct checking will happen at commit time.
+ return Status::OK();
+}
+
+// Returns OK if it is safe to commit this transaction. Returns Status::Busy
+// if there are read or write conflicts that would prevent us from committing OR
+// if we can not determine whether there would be any such conflicts.
+//
+// Should only be called on writer thread in order to avoid any race conditions
+// in detecting write conflicts.
+Status OptimisticTransaction::CheckTransactionForConflicts(DB* db) {
+ Status result;
+
+ auto db_impl = static_cast_with_check<DBImpl, DB>(db);
+
+ // Since we are on the write thread and do not want to block other writers,
+ // we will do a cache-only conflict check. This can result in TryAgain
+ // getting returned if there is not sufficient memtable history to check
+ // for conflicts.
+ return TransactionUtil::CheckKeysForConflicts(db_impl, GetTrackedKeys(),
+ true /* cache_only */);
+}
+
+Status OptimisticTransaction::SetName(const TransactionName& /* unused */) {
+ return Status::InvalidArgument("Optimistic transactions cannot be named.");
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/optimistic_transaction.h b/src/rocksdb/utilities/transactions/optimistic_transaction.h
new file mode 100644
index 000000000..c337de2af
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/optimistic_transaction.h
@@ -0,0 +1,101 @@
+// 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 <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/optimistic_transaction_db.h"
+#include "rocksdb/utilities/write_batch_with_index.h"
+#include "utilities/transactions/transaction_base.h"
+#include "utilities/transactions/transaction_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class OptimisticTransaction : public TransactionBaseImpl {
+ public:
+ OptimisticTransaction(OptimisticTransactionDB* db,
+ const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options);
+ // No copying allowed
+ OptimisticTransaction(const OptimisticTransaction&) = delete;
+ void operator=(const OptimisticTransaction&) = delete;
+
+ virtual ~OptimisticTransaction();
+
+ void Reinitialize(OptimisticTransactionDB* txn_db,
+ const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options);
+
+ Status Prepare() override;
+
+ Status Commit() override;
+
+ Status Rollback() override;
+
+ Status SetName(const TransactionName& name) override;
+
+ protected:
+ Status TryLock(ColumnFamilyHandle* column_family, const Slice& key,
+ bool read_only, bool exclusive, const bool do_validate = true,
+ const bool assume_tracked = false) override;
+
+ private:
+ ROCKSDB_FIELD_UNUSED OptimisticTransactionDB* const txn_db_;
+
+ friend class OptimisticTransactionCallback;
+
+ void Initialize(const OptimisticTransactionOptions& txn_options);
+
+ // Returns OK if it is safe to commit this transaction. Returns Status::Busy
+ // if there are read or write conflicts that would prevent us from committing
+ // OR if we can not determine whether there would be any such conflicts.
+ //
+ // Should only be called on writer thread.
+ Status CheckTransactionForConflicts(DB* db);
+
+ void Clear() override;
+
+ void UnlockGetForUpdate(ColumnFamilyHandle* /* unused */,
+ const Slice& /* unused */) override {
+ // Nothing to unlock.
+ }
+
+ Status CommitWithSerialValidate();
+
+ Status CommitWithParallelValidate();
+};
+
+// Used at commit time to trigger transaction validation
+class OptimisticTransactionCallback : public WriteCallback {
+ public:
+ explicit OptimisticTransactionCallback(OptimisticTransaction* txn)
+ : txn_(txn) {}
+
+ Status Callback(DB* db) override {
+ return txn_->CheckTransactionForConflicts(db);
+ }
+
+ bool AllowWriteBatching() override { return false; }
+
+ private:
+ OptimisticTransaction* txn_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.cc b/src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.cc
new file mode 100644
index 000000000..bffb3d5ed
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.cc
@@ -0,0 +1,111 @@
+// 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/optimistic_transaction_db_impl.h"
+
+#include <string>
+#include <vector>
+
+#include "db/db_impl/db_impl.h"
+#include "rocksdb/db.h"
+#include "rocksdb/options.h"
+#include "rocksdb/utilities/optimistic_transaction_db.h"
+#include "utilities/transactions/optimistic_transaction.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+Transaction* OptimisticTransactionDBImpl::BeginTransaction(
+ const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options, Transaction* old_txn) {
+ if (old_txn != nullptr) {
+ ReinitializeTransaction(old_txn, write_options, txn_options);
+ return old_txn;
+ } else {
+ return new OptimisticTransaction(this, write_options, txn_options);
+ }
+}
+
+std::unique_lock<std::mutex> OptimisticTransactionDBImpl::LockBucket(
+ size_t idx) {
+ assert(idx < bucketed_locks_.size());
+ return std::unique_lock<std::mutex>(*bucketed_locks_[idx]);
+}
+
+Status OptimisticTransactionDB::Open(const Options& options,
+ const std::string& dbname,
+ OptimisticTransactionDB** 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 = Open(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 OptimisticTransactionDB::Open(
+ const DBOptions& db_options, const std::string& dbname,
+ const std::vector<ColumnFamilyDescriptor>& column_families,
+ std::vector<ColumnFamilyHandle*>* handles,
+ OptimisticTransactionDB** dbptr) {
+ return OptimisticTransactionDB::Open(db_options,
+ OptimisticTransactionDBOptions(), dbname,
+ column_families, handles, dbptr);
+}
+
+Status OptimisticTransactionDB::Open(
+ const DBOptions& db_options,
+ const OptimisticTransactionDBOptions& occ_options,
+ const std::string& dbname,
+ const std::vector<ColumnFamilyDescriptor>& column_families,
+ std::vector<ColumnFamilyHandle*>* handles,
+ OptimisticTransactionDB** dbptr) {
+ Status s;
+ DB* db;
+
+ std::vector<ColumnFamilyDescriptor> column_families_copy = column_families;
+
+ // Enable MemTable History if not already enabled
+ for (auto& column_family : column_families_copy) {
+ ColumnFamilyOptions* options = &column_family.options;
+
+ if (options->max_write_buffer_size_to_maintain == 0 &&
+ options->max_write_buffer_number_to_maintain == 0) {
+ // Setting to -1 will set the History size to
+ // max_write_buffer_number * write_buffer_size.
+ options->max_write_buffer_size_to_maintain = -1;
+ }
+ }
+
+ s = DB::Open(db_options, dbname, column_families_copy, handles, &db);
+
+ if (s.ok()) {
+ *dbptr = new OptimisticTransactionDBImpl(db, occ_options);
+ }
+
+ return s;
+}
+
+void OptimisticTransactionDBImpl::ReinitializeTransaction(
+ Transaction* txn, const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options) {
+ assert(dynamic_cast<OptimisticTransaction*>(txn) != nullptr);
+ auto txn_impl = reinterpret_cast<OptimisticTransaction*>(txn);
+
+ txn_impl->Reinitialize(this, write_options, txn_options);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.h b/src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.h
new file mode 100644
index 000000000..d895d49b8
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.h
@@ -0,0 +1,71 @@
+// 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 <mutex>
+#include <vector>
+#include <algorithm>
+
+#include "rocksdb/db.h"
+#include "rocksdb/options.h"
+#include "rocksdb/utilities/optimistic_transaction_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class OptimisticTransactionDBImpl : public OptimisticTransactionDB {
+ public:
+ explicit OptimisticTransactionDBImpl(
+ DB* db, const OptimisticTransactionDBOptions& occ_options,
+ bool take_ownership = true)
+ : OptimisticTransactionDB(db),
+ db_owner_(take_ownership),
+ validate_policy_(occ_options.validate_policy) {
+ if (validate_policy_ == OccValidationPolicy::kValidateParallel) {
+ uint32_t bucket_size = std::max(16u, occ_options.occ_lock_buckets);
+ bucketed_locks_.reserve(bucket_size);
+ for (size_t i = 0; i < bucket_size; ++i) {
+ bucketed_locks_.emplace_back(
+ std::unique_ptr<std::mutex>(new std::mutex));
+ }
+ }
+ }
+
+ ~OptimisticTransactionDBImpl() {
+ // Prevent this stackable from destroying
+ // base db
+ if (!db_owner_) {
+ db_ = nullptr;
+ }
+ }
+
+ Transaction* BeginTransaction(const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options,
+ Transaction* old_txn) override;
+
+ size_t GetLockBucketsSize() const { return bucketed_locks_.size(); }
+
+ OccValidationPolicy GetValidatePolicy() const { return validate_policy_; }
+
+ std::unique_lock<std::mutex> LockBucket(size_t idx);
+
+ private:
+ // NOTE: used in validation phase. Each key is hashed into some
+ // bucket. We then take the lock in the hash value order to avoid deadlock.
+ std::vector<std::unique_ptr<std::mutex>> bucketed_locks_;
+
+ bool db_owner_;
+
+ const OccValidationPolicy validate_policy_;
+
+ void ReinitializeTransaction(Transaction* txn,
+ const WriteOptions& write_options,
+ const OptimisticTransactionOptions& txn_options =
+ OptimisticTransactionOptions());
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/optimistic_transaction_test.cc b/src/rocksdb/utilities/transactions/optimistic_transaction_test.cc
new file mode 100644
index 000000000..63c1a255c
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/optimistic_transaction_test.cc
@@ -0,0 +1,1535 @@
+// 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 <functional>
+#include <string>
+#include <thread>
+
+#include "db/db_impl/db_impl.h"
+#include "logging/logging.h"
+#include "port/port.h"
+#include "rocksdb/db.h"
+#include "rocksdb/perf_context.h"
+#include "rocksdb/utilities/optimistic_transaction_db.h"
+#include "rocksdb/utilities/transaction.h"
+#include "test_util/sync_point.h"
+#include "test_util/testharness.h"
+#include "test_util/transaction_test_util.h"
+#include "util/crc32c.h"
+#include "util/random.h"
+
+using std::string;
+
+namespace ROCKSDB_NAMESPACE {
+
+class OptimisticTransactionTest
+ : public testing::Test,
+ public testing::WithParamInterface<OccValidationPolicy> {
+ public:
+ OptimisticTransactionDB* txn_db;
+ string dbname;
+ Options options;
+
+ OptimisticTransactionTest() {
+ options.create_if_missing = true;
+ options.max_write_buffer_number = 2;
+ options.max_write_buffer_size_to_maintain = 1600;
+ dbname = test::PerThreadDBPath("optimistic_transaction_testdb");
+
+ DestroyDB(dbname, options);
+ Open();
+ }
+ ~OptimisticTransactionTest() override {
+ delete txn_db;
+ DestroyDB(dbname, options);
+ }
+
+ void Reopen() {
+ delete txn_db;
+ txn_db = nullptr;
+ Open();
+ }
+
+private:
+ void Open() {
+ ColumnFamilyOptions cf_options(options);
+ OptimisticTransactionDBOptions occ_opts;
+ occ_opts.validate_policy = GetParam();
+ std::vector<ColumnFamilyDescriptor> column_families;
+ std::vector<ColumnFamilyHandle*> handles;
+ column_families.push_back(
+ ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
+ Status s =
+ OptimisticTransactionDB::Open(DBOptions(options), occ_opts, dbname,
+ column_families, &handles, &txn_db);
+
+ assert(s.ok());
+ assert(txn_db != nullptr);
+ assert(handles.size() == 1);
+ delete handles[0];
+ }
+};
+
+TEST_P(OptimisticTransactionTest, SuccessTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, Slice("foo"), Slice("bar"));
+ txn_db->Put(write_options, Slice("foo2"), Slice("bar"));
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ txn->GetForUpdate(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ txn->Put(Slice("foo"), Slice("bar2"));
+
+ txn->GetForUpdate(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, WriteConflictTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, "foo", "bar");
+ txn_db->Put(write_options, "foo2", "bar");
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ txn->Put("foo", "bar2");
+
+ // This Put outside of a transaction will conflict with the previous write
+ s = txn_db->Put(write_options, "foo", "barz");
+ ASSERT_OK(s);
+
+ s = txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+ ASSERT_EQ(1, txn->GetNumKeys());
+
+ s = txn->Commit();
+ ASSERT_TRUE(s.IsBusy()); // Txn should not commit
+
+ // Verify that transaction did not write anything
+ txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+ txn_db->Get(read_options, "foo2", &value);
+ ASSERT_EQ(value, "bar");
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, WriteConflictTest2) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, "foo", "bar");
+ txn_db->Put(write_options, "foo2", "bar");
+
+ txn_options.set_snapshot = true;
+ Transaction* txn = txn_db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ // This Put outside of a transaction will conflict with a later write
+ s = txn_db->Put(write_options, "foo", "barz");
+ ASSERT_OK(s);
+
+ txn->Put("foo", "bar2"); // Conflicts with write done after snapshot taken
+
+ s = txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+
+ s = txn->Commit();
+ ASSERT_TRUE(s.IsBusy()); // Txn should not commit
+
+ // Verify that transaction did not write anything
+ txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+ txn_db->Get(read_options, "foo2", &value);
+ ASSERT_EQ(value, "bar");
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, ReadConflictTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, "foo", "bar");
+ txn_db->Put(write_options, "foo2", "bar");
+
+ txn_options.set_snapshot = true;
+ Transaction* txn = txn_db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ // This Put outside of a transaction will conflict with the previous read
+ s = txn_db->Put(write_options, "foo", "barz");
+ ASSERT_OK(s);
+
+ s = txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+
+ s = txn->Commit();
+ ASSERT_TRUE(s.IsBusy()); // Txn should not commit
+
+ // Verify that transaction did not write anything
+ txn->GetForUpdate(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+ txn->GetForUpdate(read_options, "foo2", &value);
+ ASSERT_EQ(value, "bar");
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, TxnOnlyTest) {
+ // Test to make sure transactions work when there are no other writes in an
+ // empty db.
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ txn->Put("x", "y");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, FlushTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, Slice("foo"), Slice("bar"));
+ txn_db->Put(write_options, Slice("foo2"), Slice("bar"));
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ txn->Put(Slice("foo"), Slice("bar2"));
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ // Put a random key so we have a memtable to flush
+ s = txn_db->Put(write_options, "dummy", "dummy");
+ ASSERT_OK(s);
+
+ // force a memtable flush
+ FlushOptions flush_ops;
+ txn_db->Flush(flush_ops);
+
+ s = txn->Commit();
+ // txn should commit since the flushed table is still in MemtableList History
+ ASSERT_OK(s);
+
+ txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, FlushTest2) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, Slice("foo"), Slice("bar"));
+ txn_db->Put(write_options, Slice("foo2"), Slice("bar"));
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ txn->Put(Slice("foo"), Slice("bar2"));
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ // Put a random key so we have a MemTable to flush
+ s = txn_db->Put(write_options, "dummy", "dummy");
+ ASSERT_OK(s);
+
+ // force a memtable flush
+ FlushOptions flush_ops;
+ txn_db->Flush(flush_ops);
+
+ // Put a random key so we have a MemTable to flush
+ s = txn_db->Put(write_options, "dummy", "dummy2");
+ ASSERT_OK(s);
+
+ // force a memtable flush
+ txn_db->Flush(flush_ops);
+
+ s = txn_db->Put(write_options, "dummy", "dummy3");
+ ASSERT_OK(s);
+
+ // force a memtable flush
+ // Since our test db has max_write_buffer_number=2, this flush will cause
+ // the first memtable to get purged from the MemtableList history.
+ txn_db->Flush(flush_ops);
+
+ s = txn->Commit();
+ // txn should not commit since MemTableList History is not large enough
+ ASSERT_TRUE(s.IsTryAgain());
+
+ txn_db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ delete txn;
+}
+
+// Trigger the condition where some old memtables are skipped when doing
+// TransactionUtil::CheckKey(), and make sure the result is still correct.
+TEST_P(OptimisticTransactionTest, CheckKeySkipOldMemtable) {
+ const int kAttemptHistoryMemtable = 0;
+ const int kAttemptImmMemTable = 1;
+ for (int attempt = kAttemptHistoryMemtable; attempt <= kAttemptImmMemTable;
+ attempt++) {
+ options.max_write_buffer_number_to_maintain = 3;
+ Reopen();
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ ReadOptions snapshot_read_options;
+ ReadOptions snapshot_read_options2;
+ string value;
+ Status s;
+
+ ASSERT_OK(txn_db->Put(write_options, Slice("foo"), Slice("bar")));
+ ASSERT_OK(txn_db->Put(write_options, Slice("foo2"), Slice("bar")));
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn != nullptr);
+
+ Transaction* txn2 = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn2 != nullptr);
+
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+ ASSERT_OK(txn->GetForUpdate(snapshot_read_options, "foo", &value));
+ ASSERT_EQ(value, "bar");
+ ASSERT_OK(txn->Put(Slice("foo"), Slice("bar2")));
+
+ snapshot_read_options2.snapshot = txn2->GetSnapshot();
+ ASSERT_OK(txn2->GetForUpdate(snapshot_read_options2, "foo2", &value));
+ ASSERT_EQ(value, "bar");
+ ASSERT_OK(txn2->Put(Slice("foo2"), Slice("bar2")));
+
+ // txn updates "foo" and txn2 updates "foo2", and now a write is
+ // issued for "foo", which conflicts with txn but not txn2
+ ASSERT_OK(txn_db->Put(write_options, "foo", "bar"));
+
+ if (attempt == kAttemptImmMemTable) {
+ // For the second attempt, hold flush from beginning. The memtable
+ // will be switched to immutable after calling TEST_SwitchMemtable()
+ // while CheckKey() is called.
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"OptimisticTransactionTest.CheckKeySkipOldMemtable",
+ "FlushJob::Start"}});
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ }
+
+ // force a memtable flush. The memtable should still be kept
+ FlushOptions flush_ops;
+ if (attempt == kAttemptHistoryMemtable) {
+ ASSERT_OK(txn_db->Flush(flush_ops));
+ } else {
+ assert(attempt == kAttemptImmMemTable);
+ DBImpl* db_impl = static_cast<DBImpl*>(txn_db->GetRootDB());
+ db_impl->TEST_SwitchMemtable();
+ }
+ uint64_t num_imm_mems;
+ ASSERT_TRUE(txn_db->GetIntProperty(DB::Properties::kNumImmutableMemTable,
+ &num_imm_mems));
+ if (attempt == kAttemptHistoryMemtable) {
+ ASSERT_EQ(0, num_imm_mems);
+ } else {
+ assert(attempt == kAttemptImmMemTable);
+ ASSERT_EQ(1, num_imm_mems);
+ }
+
+ // Put something in active memtable
+ ASSERT_OK(txn_db->Put(write_options, Slice("foo3"), Slice("bar")));
+
+ // Create txn3 after flushing, when this transaction is commited,
+ // only need to check the active memtable
+ Transaction* txn3 = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn3 != nullptr);
+
+ // Commit both of txn and txn2. txn will conflict but txn2 will
+ // pass. In both ways, both memtables are queried.
+ SetPerfLevel(PerfLevel::kEnableCount);
+
+ get_perf_context()->Reset();
+ s = txn->Commit();
+ // We should have checked two memtables
+ ASSERT_EQ(2, get_perf_context()->get_from_memtable_count);
+ // txn should fail because of conflict, even if the memtable
+ // has flushed, because it is still preserved in history.
+ ASSERT_TRUE(s.IsBusy());
+
+ get_perf_context()->Reset();
+ s = txn2->Commit();
+ // We should have checked two memtables
+ ASSERT_EQ(2, get_perf_context()->get_from_memtable_count);
+ ASSERT_TRUE(s.ok());
+
+ txn3->Put(Slice("foo2"), Slice("bar2"));
+ get_perf_context()->Reset();
+ s = txn3->Commit();
+ // txn3 is created after the active memtable is created, so that is the only
+ // memtable to check.
+ ASSERT_EQ(1, get_perf_context()->get_from_memtable_count);
+ ASSERT_TRUE(s.ok());
+
+ TEST_SYNC_POINT("OptimisticTransactionTest.CheckKeySkipOldMemtable");
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+
+ SetPerfLevel(PerfLevel::kDisable);
+
+ delete txn;
+ delete txn2;
+ delete txn3;
+ }
+}
+
+TEST_P(OptimisticTransactionTest, NoSnapshotTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, "AAA", "bar");
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ // Modify key after transaction start
+ txn_db->Put(write_options, "AAA", "bar1");
+
+ // Read and write without a snapshot
+ txn->GetForUpdate(read_options, "AAA", &value);
+ ASSERT_EQ(value, "bar1");
+ txn->Put("AAA", "bar2");
+
+ // Should commit since read/write was done after data changed
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ txn->GetForUpdate(read_options, "AAA", &value);
+ ASSERT_EQ(value, "bar2");
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, MultipleSnapshotTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, "AAA", "bar");
+ txn_db->Put(write_options, "BBB", "bar");
+ txn_db->Put(write_options, "CCC", "bar");
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ txn_db->Put(write_options, "AAA", "bar1");
+
+ // Read and write without a snapshot
+ txn->GetForUpdate(read_options, "AAA", &value);
+ ASSERT_EQ(value, "bar1");
+ txn->Put("AAA", "bar2");
+
+ // Modify BBB before snapshot is taken
+ txn_db->Put(write_options, "BBB", "bar1");
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ // Read and write with snapshot
+ txn->GetForUpdate(snapshot_read_options, "BBB", &value);
+ ASSERT_EQ(value, "bar1");
+ txn->Put("BBB", "bar2");
+
+ txn_db->Put(write_options, "CCC", "bar1");
+
+ // Set a new snapshot
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ // Read and write with snapshot
+ txn->GetForUpdate(snapshot_read_options, "CCC", &value);
+ ASSERT_EQ(value, "bar1");
+ txn->Put("CCC", "bar2");
+
+ s = txn->GetForUpdate(read_options, "AAA", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = txn->GetForUpdate(read_options, "BBB", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = txn->GetForUpdate(read_options, "CCC", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+
+ s = txn_db->Get(read_options, "AAA", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar1");
+ s = txn_db->Get(read_options, "BBB", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar1");
+ s = txn_db->Get(read_options, "CCC", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar1");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = txn_db->Get(read_options, "AAA", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = txn_db->Get(read_options, "BBB", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = txn_db->Get(read_options, "CCC", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+
+ // verify that we track multiple writes to the same key at different snapshots
+ delete txn;
+ txn = txn_db->BeginTransaction(write_options);
+
+ // Potentially conflicting writes
+ txn_db->Put(write_options, "ZZZ", "zzz");
+ txn_db->Put(write_options, "XXX", "xxx");
+
+ txn->SetSnapshot();
+
+ OptimisticTransactionOptions txn_options;
+ txn_options.set_snapshot = true;
+ Transaction* txn2 = txn_db->BeginTransaction(write_options, txn_options);
+ txn2->SetSnapshot();
+
+ // This should not conflict in txn since the snapshot is later than the
+ // previous write (spoiler alert: it will later conflict with txn2).
+ txn->Put("ZZZ", "zzzz");
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+
+ // This will conflict since the snapshot is earlier than another write to ZZZ
+ txn2->Put("ZZZ", "xxxxx");
+
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ delete txn2;
+}
+
+TEST_P(OptimisticTransactionTest, ColumnFamiliesTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ ColumnFamilyHandle *cfa, *cfb;
+ ColumnFamilyOptions cf_options;
+
+ // Create 2 new column families
+ s = txn_db->CreateColumnFamily(cf_options, "CFA", &cfa);
+ ASSERT_OK(s);
+ s = txn_db->CreateColumnFamily(cf_options, "CFB", &cfb);
+ ASSERT_OK(s);
+
+ delete cfa;
+ delete cfb;
+ delete txn_db;
+ txn_db = nullptr;
+
+ // open DB with three column families
+ std::vector<ColumnFamilyDescriptor> column_families;
+ // have to open default column family
+ column_families.push_back(
+ ColumnFamilyDescriptor(kDefaultColumnFamilyName, ColumnFamilyOptions()));
+ // open the new column families
+ column_families.push_back(
+ ColumnFamilyDescriptor("CFA", ColumnFamilyOptions()));
+ column_families.push_back(
+ ColumnFamilyDescriptor("CFB", ColumnFamilyOptions()));
+ std::vector<ColumnFamilyHandle*> handles;
+ s = OptimisticTransactionDB::Open(options, dbname, column_families, &handles,
+ &txn_db);
+ ASSERT_OK(s);
+ assert(txn_db != nullptr);
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn_options.set_snapshot = true;
+ Transaction* txn2 = txn_db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ // Write some data to the db
+ WriteBatch batch;
+ batch.Put("foo", "foo");
+ batch.Put(handles[1], "AAA", "bar");
+ batch.Put(handles[1], "AAAZZZ", "bar");
+ s = txn_db->Write(write_options, &batch);
+ ASSERT_OK(s);
+ txn_db->Delete(write_options, handles[1], "AAAZZZ");
+
+ // These keys do no conflict with existing writes since they're in
+ // different column families
+ txn->Delete("AAA");
+ txn->GetForUpdate(snapshot_read_options, handles[1], "foo", &value);
+ Slice key_slice("AAAZZZ");
+ Slice value_slices[2] = {Slice("bar"), Slice("bar")};
+ txn->Put(handles[2], SliceParts(&key_slice, 1), SliceParts(value_slices, 2));
+
+ ASSERT_EQ(3, txn->GetNumKeys());
+
+ // Txn should commit
+ s = txn->Commit();
+ ASSERT_OK(s);
+ s = txn_db->Get(read_options, "AAA", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn_db->Get(read_options, handles[2], "AAAZZZ", &value);
+ ASSERT_EQ(value, "barbar");
+
+ Slice key_slices[3] = {Slice("AAA"), Slice("ZZ"), Slice("Z")};
+ Slice value_slice("barbarbar");
+ // This write will cause a conflict with the earlier batch write
+ txn2->Put(handles[1], SliceParts(key_slices, 3), SliceParts(&value_slice, 1));
+
+ txn2->Delete(handles[2], "XXX");
+ txn2->Delete(handles[1], "XXX");
+ s = txn2->GetForUpdate(snapshot_read_options, handles[1], "AAA", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Verify txn did not commit
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsBusy());
+ s = txn_db->Get(read_options, handles[1], "AAAZZZ", &value);
+ ASSERT_EQ(value, "barbar");
+
+ delete txn;
+ delete txn2;
+
+ txn = txn_db->BeginTransaction(write_options, txn_options);
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn2 = txn_db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ std::vector<ColumnFamilyHandle*> multiget_cfh = {handles[1], handles[2],
+ handles[0], handles[2]};
+ std::vector<Slice> multiget_keys = {"AAA", "AAAZZZ", "foo", "foo"};
+ std::vector<std::string> values(4);
+
+ std::vector<Status> results = txn->MultiGetForUpdate(
+ snapshot_read_options, multiget_cfh, multiget_keys, &values);
+ ASSERT_OK(results[0]);
+ ASSERT_OK(results[1]);
+ ASSERT_OK(results[2]);
+ ASSERT_TRUE(results[3].IsNotFound());
+ ASSERT_EQ(values[0], "bar");
+ ASSERT_EQ(values[1], "barbar");
+ ASSERT_EQ(values[2], "foo");
+
+ txn->Delete(handles[2], "ZZZ");
+ txn->Put(handles[2], "ZZZ", "YYY");
+ txn->Put(handles[2], "ZZZ", "YYYY");
+ txn->Delete(handles[2], "ZZZ");
+ txn->Put(handles[2], "AAAZZZ", "barbarbar");
+
+ ASSERT_EQ(5, txn->GetNumKeys());
+
+ // Txn should commit
+ s = txn->Commit();
+ ASSERT_OK(s);
+ s = txn_db->Get(read_options, handles[2], "ZZZ", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Put a key which will conflict with the next txn using the previous snapshot
+ txn_db->Put(write_options, handles[2], "foo", "000");
+
+ results = txn2->MultiGetForUpdate(snapshot_read_options, multiget_cfh,
+ multiget_keys, &values);
+ ASSERT_OK(results[0]);
+ ASSERT_OK(results[1]);
+ ASSERT_OK(results[2]);
+ ASSERT_TRUE(results[3].IsNotFound());
+ ASSERT_EQ(values[0], "bar");
+ ASSERT_EQ(values[1], "barbar");
+ ASSERT_EQ(values[2], "foo");
+
+ // Verify Txn Did not Commit
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn_db->DropColumnFamily(handles[1]);
+ ASSERT_OK(s);
+ s = txn_db->DropColumnFamily(handles[2]);
+ ASSERT_OK(s);
+
+ delete txn;
+ delete txn2;
+
+ for (auto handle : handles) {
+ delete handle;
+ }
+}
+
+TEST_P(OptimisticTransactionTest, EmptyTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ s = txn_db->Put(write_options, "aaa", "aaa");
+ ASSERT_OK(s);
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ txn = txn_db->BeginTransaction(write_options);
+ txn->Rollback();
+ delete txn;
+
+ txn = txn_db->BeginTransaction(write_options);
+ s = txn->GetForUpdate(read_options, "aaa", &value);
+ ASSERT_EQ(value, "aaa");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ txn = txn_db->BeginTransaction(write_options);
+ txn->SetSnapshot();
+ s = txn->GetForUpdate(read_options, "aaa", &value);
+ ASSERT_EQ(value, "aaa");
+
+ s = txn_db->Put(write_options, "aaa", "xxx");
+ s = txn->Commit();
+ ASSERT_TRUE(s.IsBusy());
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, PredicateManyPreceders) {
+ WriteOptions write_options;
+ ReadOptions read_options1, read_options2;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ txn_options.set_snapshot = true;
+ Transaction* txn1 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ Transaction* txn2 = txn_db->BeginTransaction(write_options);
+ txn2->SetSnapshot();
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ std::vector<Slice> multiget_keys = {"1", "2", "3"};
+ std::vector<std::string> multiget_values;
+
+ std::vector<Status> results =
+ txn1->MultiGetForUpdate(read_options1, multiget_keys, &multiget_values);
+ ASSERT_TRUE(results[1].IsNotFound());
+
+ txn2->Put("2", "x");
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ multiget_values.clear();
+ results =
+ txn1->MultiGetForUpdate(read_options1, multiget_keys, &multiget_values);
+ ASSERT_TRUE(results[1].IsNotFound());
+
+ // should not commit since txn2 wrote a key txn has read
+ s = txn1->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ txn1->Put("4", "x");
+
+ txn2->Delete("4");
+
+ // txn1 can commit since txn2's delete hasn't happened yet (it's just batched)
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options2, "4", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // txn2 cannot commit since txn1 changed "4"
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(OptimisticTransactionTest, LostUpdate) {
+ WriteOptions write_options;
+ ReadOptions read_options, read_options1, read_options2;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ // Test 2 transactions writing to the same key in multiple orders and
+ // with/without snapshots
+
+ Transaction* txn1 = txn_db->BeginTransaction(write_options);
+ Transaction* txn2 = txn_db->BeginTransaction(write_options);
+
+ txn1->Put("1", "1");
+ txn2->Put("1", "2");
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ delete txn1;
+ delete txn2;
+
+ txn_options.set_snapshot = true;
+ txn1 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ txn1->Put("1", "3");
+ txn2->Put("1", "4");
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ txn1->Put("1", "5");
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ txn2->Put("1", "6");
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = txn_db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ txn1->Put("1", "5");
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ txn2->SetSnapshot();
+ txn2->Put("1", "6");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = txn_db->BeginTransaction(write_options);
+ txn2 = txn_db->BeginTransaction(write_options);
+
+ txn1->Put("1", "7");
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ txn2->Put("1", "8");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+ delete txn2;
+
+ s = txn_db->Get(read_options, "1", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "8");
+}
+
+TEST_P(OptimisticTransactionTest, UntrackedWrites) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ // Verify transaction rollback works for untracked keys.
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ txn->PutUntracked("untracked", "0");
+ txn->Rollback();
+ s = txn_db->Get(read_options, "untracked", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+ txn = txn_db->BeginTransaction(write_options);
+
+ txn->Put("tracked", "1");
+ txn->PutUntracked("untracked", "1");
+ txn->MergeUntracked("untracked", "2");
+ txn->DeleteUntracked("untracked");
+
+ // Write to the untracked key outside of the transaction and verify
+ // it doesn't prevent the transaction from committing.
+ s = txn_db->Put(write_options, "untracked", "x");
+ ASSERT_OK(s);
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = txn_db->Get(read_options, "untracked", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+ txn = txn_db->BeginTransaction(write_options);
+
+ txn->Put("tracked", "10");
+ txn->PutUntracked("untracked", "A");
+
+ // Write to tracked key outside of the transaction and verify that the
+ // untracked keys are not written when the commit fails.
+ s = txn_db->Delete(write_options, "tracked");
+
+ s = txn->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn_db->Get(read_options, "untracked", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, IteratorTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ // Write some keys to the db
+ s = txn_db->Put(write_options, "A", "a");
+ ASSERT_OK(s);
+
+ s = txn_db->Put(write_options, "G", "g");
+ ASSERT_OK(s);
+
+ s = txn_db->Put(write_options, "F", "f");
+ ASSERT_OK(s);
+
+ s = txn_db->Put(write_options, "C", "c");
+ ASSERT_OK(s);
+
+ s = txn_db->Put(write_options, "D", "d");
+ ASSERT_OK(s);
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ // Write some keys in a txn
+ s = txn->Put("B", "b");
+ ASSERT_OK(s);
+
+ s = txn->Put("H", "h");
+ ASSERT_OK(s);
+
+ s = txn->Delete("D");
+ ASSERT_OK(s);
+
+ s = txn->Put("E", "e");
+ ASSERT_OK(s);
+
+ txn->SetSnapshot();
+ const Snapshot* snapshot = txn->GetSnapshot();
+
+ // Write some keys to the db after the snapshot
+ s = txn_db->Put(write_options, "BB", "xx");
+ ASSERT_OK(s);
+
+ s = txn_db->Put(write_options, "C", "xx");
+ ASSERT_OK(s);
+
+ read_options.snapshot = snapshot;
+ Iterator* iter = txn->GetIterator(read_options);
+ ASSERT_OK(iter->status());
+ iter->SeekToFirst();
+
+ // Read all keys via iter and lock them all
+ std::string results[] = {"a", "b", "c", "e", "f", "g", "h"};
+ for (int i = 0; i < 7; i++) {
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(results[i], iter->value().ToString());
+
+ s = txn->GetForUpdate(read_options, iter->key(), nullptr);
+ ASSERT_OK(s);
+
+ iter->Next();
+ }
+ ASSERT_FALSE(iter->Valid());
+
+ iter->Seek("G");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("g", iter->value().ToString());
+
+ iter->Prev();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("f", iter->value().ToString());
+
+ iter->Seek("D");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("e", iter->value().ToString());
+
+ iter->Seek("C");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("c", iter->value().ToString());
+
+ iter->Next();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("e", iter->value().ToString());
+
+ iter->Seek("");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("a", iter->value().ToString());
+
+ iter->Seek("X");
+ ASSERT_OK(iter->status());
+ ASSERT_FALSE(iter->Valid());
+
+ iter->SeekToLast();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("h", iter->value().ToString());
+
+ // key "C" was modified in the db after txn's snapshot. txn will not commit.
+ s = txn->Commit();
+ ASSERT_TRUE(s.IsBusy());
+
+ delete iter;
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, SavepointTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ Transaction* txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->RollbackToSavePoint();
+ ASSERT_TRUE(s.IsNotFound());
+
+ txn->SetSavePoint(); // 1
+
+ ASSERT_OK(txn->RollbackToSavePoint()); // Rollback to beginning of txn
+ s = txn->RollbackToSavePoint();
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Put("B", "b");
+ ASSERT_OK(s);
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = txn_db->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b", value);
+
+ delete txn;
+ txn = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Put("B", "bb");
+ ASSERT_OK(s);
+
+ s = txn->Put("C", "c");
+ ASSERT_OK(s);
+
+ txn->SetSavePoint(); // 2
+
+ s = txn->Delete("B");
+ ASSERT_OK(s);
+
+ s = txn->Put("C", "cc");
+ ASSERT_OK(s);
+
+ s = txn->Put("D", "d");
+ ASSERT_OK(s);
+
+ ASSERT_OK(txn->RollbackToSavePoint()); // Rollback to 2
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("bb", value);
+
+ s = txn->Get(read_options, "C", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("c", value);
+
+ s = txn->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Put("E", "e");
+ ASSERT_OK(s);
+
+ // Rollback to beginning of txn
+ s = txn->RollbackToSavePoint();
+ ASSERT_TRUE(s.IsNotFound());
+ txn->Rollback();
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b", value);
+
+ s = txn->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "E", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Put("A", "aa");
+ ASSERT_OK(s);
+
+ s = txn->Put("F", "f");
+ ASSERT_OK(s);
+
+ txn->SetSavePoint(); // 3
+ txn->SetSavePoint(); // 4
+
+ s = txn->Put("G", "g");
+ ASSERT_OK(s);
+
+ s = txn->Delete("F");
+ ASSERT_OK(s);
+
+ s = txn->Delete("B");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("aa", value);
+
+ s = txn->Get(read_options, "F", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ ASSERT_OK(txn->RollbackToSavePoint()); // Rollback to 3
+
+ s = txn->Get(read_options, "F", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("f", value);
+
+ s = txn->Get(read_options, "G", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = txn_db->Get(read_options, "F", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("f", value);
+
+ s = txn_db->Get(read_options, "G", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn_db->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("aa", value);
+
+ s = txn_db->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b", value);
+
+ s = txn_db->Get(read_options, "C", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn_db->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn_db->Get(read_options, "E", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+}
+
+TEST_P(OptimisticTransactionTest, UndoGetForUpdateTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ OptimisticTransactionOptions txn_options;
+ string value;
+ Status s;
+
+ txn_db->Put(write_options, "A", "");
+
+ Transaction* txn1 = txn_db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn1);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+
+ Transaction* txn2 = txn_db->BeginTransaction(write_options);
+ txn2->Put("A", "x");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ // Verify that txn1 can commit since A isn't conflict checked
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+
+ txn1 = txn_db->BeginTransaction(write_options);
+ txn1->Put("A", "a");
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+
+ txn2 = txn_db->BeginTransaction(write_options);
+ txn2->Put("A", "x");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ // Verify that txn1 cannot commit since A will still be conflict checked
+ s = txn1->Commit();
+ ASSERT_TRUE(s.IsBusy());
+ delete txn1;
+
+ txn1 = txn_db->BeginTransaction(write_options);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+
+ txn2 = txn_db->BeginTransaction(write_options);
+ txn2->Put("A", "x");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ // Verify that txn1 cannot commit since A will still be conflict checked
+ s = txn1->Commit();
+ ASSERT_TRUE(s.IsBusy());
+ delete txn1;
+
+ txn1 = txn_db->BeginTransaction(write_options);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("A");
+
+ txn2 = txn_db->BeginTransaction(write_options);
+ txn2->Put("A", "x");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ // Verify that txn1 can commit since A isn't conflict checked
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+
+ txn1 = txn_db->BeginTransaction(write_options);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint();
+ txn1->UndoGetForUpdate("A");
+
+ txn2 = txn_db->BeginTransaction(write_options);
+ txn2->Put("A", "x");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ // Verify that txn1 cannot commit since A will still be conflict checked
+ s = txn1->Commit();
+ ASSERT_TRUE(s.IsBusy());
+ delete txn1;
+
+ txn1 = txn_db->BeginTransaction(write_options);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint();
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ txn1->UndoGetForUpdate("A");
+
+ txn2 = txn_db->BeginTransaction(write_options);
+ txn2->Put("A", "x");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ // Verify that txn1 cannot commit since A will still be conflict checked
+ s = txn1->Commit();
+ ASSERT_TRUE(s.IsBusy());
+ delete txn1;
+
+ txn1 = txn_db->BeginTransaction(write_options);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint();
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ txn1->UndoGetForUpdate("A");
+
+ txn1->RollbackToSavePoint();
+ txn1->UndoGetForUpdate("A");
+
+ txn2 = txn_db->BeginTransaction(write_options);
+ txn2->Put("A", "x");
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ // Verify that txn1 can commit since A isn't conflict checked
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+}
+
+namespace {
+Status OptimisticTransactionStressTestInserter(OptimisticTransactionDB* db,
+ const size_t num_transactions,
+ const size_t num_sets,
+ const size_t num_keys_per_set) {
+ size_t seed = std::hash<std::thread::id>()(std::this_thread::get_id());
+ Random64 _rand(seed);
+ WriteOptions write_options;
+ ReadOptions read_options;
+ OptimisticTransactionOptions txn_options;
+ txn_options.set_snapshot = true;
+
+ RandomTransactionInserter inserter(&_rand, write_options, read_options,
+ num_keys_per_set,
+ static_cast<uint16_t>(num_sets));
+
+ for (size_t t = 0; t < num_transactions; t++) {
+ bool success = inserter.OptimisticTransactionDBInsert(db, txn_options);
+ if (!success) {
+ // unexpected failure
+ return inserter.GetLastStatus();
+ }
+ }
+
+ // Make sure at least some of the transactions succeeded. It's ok if
+ // some failed due to write-conflicts.
+ if (inserter.GetFailureCount() > num_transactions / 2) {
+ return Status::TryAgain("Too many transactions failed! " +
+ std::to_string(inserter.GetFailureCount()) + " / " +
+ std::to_string(num_transactions));
+ }
+
+ return Status::OK();
+}
+} // namespace
+
+TEST_P(OptimisticTransactionTest, OptimisticTransactionStressTest) {
+ const size_t num_threads = 4;
+ const size_t num_transactions_per_thread = 10000;
+ const size_t num_sets = 3;
+ const size_t num_keys_per_set = 100;
+ // Setting the key-space to be 100 keys should cause enough write-conflicts
+ // to make this test interesting.
+
+ std::vector<port::Thread> threads;
+
+ std::function<void()> call_inserter = [&] {
+ ASSERT_OK(OptimisticTransactionStressTestInserter(
+ txn_db, num_transactions_per_thread, num_sets, num_keys_per_set));
+ };
+
+ // Create N threads that use RandomTransactionInserter to write
+ // many transactions.
+ for (uint32_t i = 0; i < num_threads; i++) {
+ threads.emplace_back(call_inserter);
+ }
+
+ // Wait for all threads to run
+ for (auto& t : threads) {
+ t.join();
+ }
+
+ // Verify that data is consistent
+ Status s = RandomTransactionInserter::Verify(txn_db, num_sets);
+ ASSERT_OK(s);
+}
+
+TEST_P(OptimisticTransactionTest, SequenceNumberAfterRecoverTest) {
+ WriteOptions write_options;
+ OptimisticTransactionOptions transaction_options;
+
+ Transaction* transaction(txn_db->BeginTransaction(write_options, transaction_options));
+ Status s = transaction->Put("foo", "val");
+ ASSERT_OK(s);
+ s = transaction->Put("foo2", "val");
+ ASSERT_OK(s);
+ s = transaction->Put("foo3", "val");
+ ASSERT_OK(s);
+ s = transaction->Commit();
+ ASSERT_OK(s);
+ delete transaction;
+
+ Reopen();
+ transaction = txn_db->BeginTransaction(write_options, transaction_options);
+ s = transaction->Put("bar", "val");
+ ASSERT_OK(s);
+ s = transaction->Put("bar2", "val");
+ ASSERT_OK(s);
+ s = transaction->Commit();
+ ASSERT_OK(s);
+
+ delete transaction;
+}
+
+INSTANTIATE_TEST_CASE_P(
+ InstanceOccGroup, OptimisticTransactionTest,
+ testing::Values(OccValidationPolicy::kValidateSerial,
+ OccValidationPolicy::kValidateParallel));
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(
+ stderr,
+ "SKIPPED as optimistic_transaction is not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // !ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/pessimistic_transaction.cc b/src/rocksdb/utilities/transactions/pessimistic_transaction.cc
new file mode 100644
index 000000000..5ae5fed08
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/pessimistic_transaction.cc
@@ -0,0 +1,723 @@
+// 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.h"
+
+#include <map>
+#include <set>
+#include <string>
+#include <vector>
+
+#include "db/column_family.h"
+#include "db/db_impl/db_impl.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/db.h"
+#include "rocksdb/snapshot.h"
+#include "rocksdb/status.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "test_util/sync_point.h"
+#include "util/cast_util.h"
+#include "util/string_util.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+#include "utilities/transactions/transaction_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct WriteOptions;
+
+std::atomic<TransactionID> PessimisticTransaction::txn_id_counter_(1);
+
+TransactionID PessimisticTransaction::GenTxnID() {
+ return txn_id_counter_.fetch_add(1);
+}
+
+PessimisticTransaction::PessimisticTransaction(
+ TransactionDB* txn_db, const WriteOptions& write_options,
+ const TransactionOptions& txn_options, const bool init)
+ : TransactionBaseImpl(txn_db->GetRootDB(), write_options),
+ txn_db_impl_(nullptr),
+ expiration_time_(0),
+ txn_id_(0),
+ waiting_cf_id_(0),
+ waiting_key_(nullptr),
+ lock_timeout_(0),
+ deadlock_detect_(false),
+ deadlock_detect_depth_(0),
+ skip_concurrency_control_(false) {
+ txn_db_impl_ =
+ static_cast_with_check<PessimisticTransactionDB, TransactionDB>(txn_db);
+ db_impl_ = static_cast_with_check<DBImpl, DB>(db_);
+ if (init) {
+ Initialize(txn_options);
+ }
+}
+
+void PessimisticTransaction::Initialize(const TransactionOptions& txn_options) {
+ txn_id_ = GenTxnID();
+
+ txn_state_ = STARTED;
+
+ deadlock_detect_ = txn_options.deadlock_detect;
+ deadlock_detect_depth_ = txn_options.deadlock_detect_depth;
+ write_batch_.SetMaxBytes(txn_options.max_write_batch_size);
+ skip_concurrency_control_ = txn_options.skip_concurrency_control;
+
+ lock_timeout_ = txn_options.lock_timeout * 1000;
+ if (lock_timeout_ < 0) {
+ // Lock timeout not set, use default
+ lock_timeout_ =
+ txn_db_impl_->GetTxnDBOptions().transaction_lock_timeout * 1000;
+ }
+
+ if (txn_options.expiration >= 0) {
+ expiration_time_ = start_time_ + txn_options.expiration * 1000;
+ } else {
+ expiration_time_ = 0;
+ }
+
+ if (txn_options.set_snapshot) {
+ SetSnapshot();
+ }
+
+ if (expiration_time_ > 0) {
+ txn_db_impl_->InsertExpirableTransaction(txn_id_, this);
+ }
+ use_only_the_last_commit_time_batch_for_recovery_ =
+ txn_options.use_only_the_last_commit_time_batch_for_recovery;
+}
+
+PessimisticTransaction::~PessimisticTransaction() {
+ txn_db_impl_->UnLock(this, &GetTrackedKeys());
+ if (expiration_time_ > 0) {
+ txn_db_impl_->RemoveExpirableTransaction(txn_id_);
+ }
+ if (!name_.empty() && txn_state_ != COMMITED) {
+ txn_db_impl_->UnregisterTransaction(this);
+ }
+}
+
+void PessimisticTransaction::Clear() {
+ txn_db_impl_->UnLock(this, &GetTrackedKeys());
+ TransactionBaseImpl::Clear();
+}
+
+void PessimisticTransaction::Reinitialize(
+ TransactionDB* txn_db, const WriteOptions& write_options,
+ const TransactionOptions& txn_options) {
+ if (!name_.empty() && txn_state_ != COMMITED) {
+ txn_db_impl_->UnregisterTransaction(this);
+ }
+ TransactionBaseImpl::Reinitialize(txn_db->GetRootDB(), write_options);
+ Initialize(txn_options);
+}
+
+bool PessimisticTransaction::IsExpired() const {
+ if (expiration_time_ > 0) {
+ if (db_->GetEnv()->NowMicros() >= expiration_time_) {
+ // Transaction is expired.
+ return true;
+ }
+ }
+
+ return false;
+}
+
+WriteCommittedTxn::WriteCommittedTxn(TransactionDB* txn_db,
+ const WriteOptions& write_options,
+ const TransactionOptions& txn_options)
+ : PessimisticTransaction(txn_db, write_options, txn_options){};
+
+Status PessimisticTransaction::CommitBatch(WriteBatch* batch) {
+ TransactionKeyMap keys_to_unlock;
+ Status s = LockBatch(batch, &keys_to_unlock);
+
+ if (!s.ok()) {
+ return s;
+ }
+
+ bool can_commit = false;
+
+ if (IsExpired()) {
+ s = Status::Expired();
+ } else if (expiration_time_ > 0) {
+ TransactionState expected = STARTED;
+ can_commit = std::atomic_compare_exchange_strong(&txn_state_, &expected,
+ AWAITING_COMMIT);
+ } else if (txn_state_ == STARTED) {
+ // lock stealing is not a concern
+ can_commit = true;
+ }
+
+ if (can_commit) {
+ txn_state_.store(AWAITING_COMMIT);
+ s = CommitBatchInternal(batch);
+ if (s.ok()) {
+ txn_state_.store(COMMITED);
+ }
+ } else if (txn_state_ == LOCKS_STOLEN) {
+ s = Status::Expired();
+ } else {
+ s = Status::InvalidArgument("Transaction is not in state for commit.");
+ }
+
+ txn_db_impl_->UnLock(this, &keys_to_unlock);
+
+ return s;
+}
+
+Status PessimisticTransaction::Prepare() {
+ Status s;
+
+ if (name_.empty()) {
+ return Status::InvalidArgument(
+ "Cannot prepare a transaction that has not been named.");
+ }
+
+ if (IsExpired()) {
+ return Status::Expired();
+ }
+
+ bool can_prepare = false;
+
+ if (expiration_time_ > 0) {
+ // must concern ourselves with expiraton and/or lock stealing
+ // need to compare/exchange bc locks could be stolen under us here
+ TransactionState expected = STARTED;
+ can_prepare = std::atomic_compare_exchange_strong(&txn_state_, &expected,
+ AWAITING_PREPARE);
+ } else if (txn_state_ == STARTED) {
+ // expiration and lock stealing is not possible
+ can_prepare = true;
+ }
+
+ if (can_prepare) {
+ txn_state_.store(AWAITING_PREPARE);
+ // transaction can't expire after preparation
+ expiration_time_ = 0;
+ assert(log_number_ == 0 ||
+ txn_db_impl_->GetTxnDBOptions().write_policy == WRITE_UNPREPARED);
+
+ s = PrepareInternal();
+ if (s.ok()) {
+ txn_state_.store(PREPARED);
+ }
+ } else if (txn_state_ == LOCKS_STOLEN) {
+ s = Status::Expired();
+ } else if (txn_state_ == PREPARED) {
+ s = Status::InvalidArgument("Transaction has already been prepared.");
+ } else if (txn_state_ == COMMITED) {
+ s = Status::InvalidArgument("Transaction has already been committed.");
+ } else if (txn_state_ == ROLLEDBACK) {
+ s = Status::InvalidArgument("Transaction has already been rolledback.");
+ } else {
+ s = Status::InvalidArgument("Transaction is not in state for commit.");
+ }
+
+ return s;
+}
+
+Status WriteCommittedTxn::PrepareInternal() {
+ WriteOptions write_options = write_options_;
+ write_options.disableWAL = false;
+ WriteBatchInternal::MarkEndPrepare(GetWriteBatch()->GetWriteBatch(), name_);
+ class MarkLogCallback : public PreReleaseCallback {
+ public:
+ MarkLogCallback(DBImpl* db, bool two_write_queues)
+ : db_(db), two_write_queues_(two_write_queues) {
+ (void)two_write_queues_; // to silence unused private field warning
+ }
+ virtual Status Callback(SequenceNumber, bool is_mem_disabled,
+ uint64_t log_number, size_t /*index*/,
+ size_t /*total*/) override {
+#ifdef NDEBUG
+ (void)is_mem_disabled;
+#endif
+ assert(log_number != 0);
+ assert(!two_write_queues_ || is_mem_disabled); // implies the 2nd queue
+ db_->logs_with_prep_tracker()->MarkLogAsContainingPrepSection(log_number);
+ return Status::OK();
+ }
+
+ private:
+ DBImpl* db_;
+ bool two_write_queues_;
+ } mark_log_callback(db_impl_,
+ db_impl_->immutable_db_options().two_write_queues);
+
+ WriteCallback* const kNoWriteCallback = nullptr;
+ const uint64_t kRefNoLog = 0;
+ const bool kDisableMemtable = true;
+ SequenceNumber* const KIgnoreSeqUsed = nullptr;
+ const size_t kNoBatchCount = 0;
+ Status s = db_impl_->WriteImpl(
+ write_options, GetWriteBatch()->GetWriteBatch(), kNoWriteCallback,
+ &log_number_, kRefNoLog, kDisableMemtable, KIgnoreSeqUsed, kNoBatchCount,
+ &mark_log_callback);
+ return s;
+}
+
+Status PessimisticTransaction::Commit() {
+ Status s;
+ bool commit_without_prepare = false;
+ bool commit_prepared = false;
+
+ if (IsExpired()) {
+ return Status::Expired();
+ }
+
+ if (expiration_time_ > 0) {
+ // we must atomicaly compare and exchange the state here because at
+ // this state in the transaction it is possible for another thread
+ // to change our state out from under us in the even that we expire and have
+ // our locks stolen. In this case the only valid state is STARTED because
+ // a state of PREPARED would have a cleared expiration_time_.
+ TransactionState expected = STARTED;
+ commit_without_prepare = std::atomic_compare_exchange_strong(
+ &txn_state_, &expected, AWAITING_COMMIT);
+ TEST_SYNC_POINT("TransactionTest::ExpirableTransactionDataRace:1");
+ } else if (txn_state_ == PREPARED) {
+ // expiration and lock stealing is not a concern
+ commit_prepared = true;
+ } else if (txn_state_ == STARTED) {
+ // expiration and lock stealing is not a concern
+ commit_without_prepare = true;
+ // TODO(myabandeh): what if the user mistakenly forgets prepare? We should
+ // add an option so that the user explictly express the intention of
+ // skipping the prepare phase.
+ }
+
+ if (commit_without_prepare) {
+ assert(!commit_prepared);
+ if (WriteBatchInternal::Count(GetCommitTimeWriteBatch()) > 0) {
+ s = Status::InvalidArgument(
+ "Commit-time batch contains values that will not be committed.");
+ } else {
+ txn_state_.store(AWAITING_COMMIT);
+ if (log_number_ > 0) {
+ dbimpl_->logs_with_prep_tracker()->MarkLogAsHavingPrepSectionFlushed(
+ log_number_);
+ }
+ s = CommitWithoutPrepareInternal();
+ if (!name_.empty()) {
+ txn_db_impl_->UnregisterTransaction(this);
+ }
+ Clear();
+ if (s.ok()) {
+ txn_state_.store(COMMITED);
+ }
+ }
+ } else if (commit_prepared) {
+ txn_state_.store(AWAITING_COMMIT);
+
+ s = CommitInternal();
+
+ if (!s.ok()) {
+ ROCKS_LOG_WARN(db_impl_->immutable_db_options().info_log,
+ "Commit write failed");
+ return s;
+ }
+
+ // FindObsoleteFiles must now look to the memtables
+ // to determine what prep logs must be kept around,
+ // not the prep section heap.
+ assert(log_number_ > 0);
+ dbimpl_->logs_with_prep_tracker()->MarkLogAsHavingPrepSectionFlushed(
+ log_number_);
+ txn_db_impl_->UnregisterTransaction(this);
+
+ Clear();
+ txn_state_.store(COMMITED);
+ } else if (txn_state_ == LOCKS_STOLEN) {
+ s = Status::Expired();
+ } else if (txn_state_ == COMMITED) {
+ s = Status::InvalidArgument("Transaction has already been committed.");
+ } else if (txn_state_ == ROLLEDBACK) {
+ s = Status::InvalidArgument("Transaction has already been rolledback.");
+ } else {
+ s = Status::InvalidArgument("Transaction is not in state for commit.");
+ }
+
+ return s;
+}
+
+Status WriteCommittedTxn::CommitWithoutPrepareInternal() {
+ uint64_t seq_used = kMaxSequenceNumber;
+ auto s =
+ db_impl_->WriteImpl(write_options_, GetWriteBatch()->GetWriteBatch(),
+ /*callback*/ nullptr, /*log_used*/ nullptr,
+ /*log_ref*/ 0, /*disable_memtable*/ false, &seq_used);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ if (s.ok()) {
+ SetId(seq_used);
+ }
+ return s;
+}
+
+Status WriteCommittedTxn::CommitBatchInternal(WriteBatch* batch, size_t) {
+ uint64_t seq_used = kMaxSequenceNumber;
+ auto s = db_impl_->WriteImpl(write_options_, batch, /*callback*/ nullptr,
+ /*log_used*/ nullptr, /*log_ref*/ 0,
+ /*disable_memtable*/ false, &seq_used);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ if (s.ok()) {
+ SetId(seq_used);
+ }
+ return s;
+}
+
+Status WriteCommittedTxn::CommitInternal() {
+ // We take the commit-time batch and append the Commit marker.
+ // The Memtable will ignore the Commit marker in non-recovery mode
+ WriteBatch* working_batch = GetCommitTimeWriteBatch();
+ WriteBatchInternal::MarkCommit(working_batch, name_);
+
+ // any operations appended to this working_batch will be ignored from WAL
+ working_batch->MarkWalTerminationPoint();
+
+ // insert prepared batch into Memtable only skipping WAL.
+ // Memtable will ignore BeginPrepare/EndPrepare markers
+ // in non recovery mode and simply insert the values
+ WriteBatchInternal::Append(working_batch, GetWriteBatch()->GetWriteBatch());
+
+ uint64_t seq_used = kMaxSequenceNumber;
+ auto s =
+ db_impl_->WriteImpl(write_options_, working_batch, /*callback*/ nullptr,
+ /*log_used*/ nullptr, /*log_ref*/ log_number_,
+ /*disable_memtable*/ false, &seq_used);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ if (s.ok()) {
+ SetId(seq_used);
+ }
+ return s;
+}
+
+Status PessimisticTransaction::Rollback() {
+ Status s;
+ if (txn_state_ == PREPARED) {
+ txn_state_.store(AWAITING_ROLLBACK);
+
+ s = RollbackInternal();
+
+ if (s.ok()) {
+ // we do not need to keep our prepared section around
+ assert(log_number_ > 0);
+ dbimpl_->logs_with_prep_tracker()->MarkLogAsHavingPrepSectionFlushed(
+ log_number_);
+ Clear();
+ txn_state_.store(ROLLEDBACK);
+ }
+ } else if (txn_state_ == STARTED) {
+ if (log_number_ > 0) {
+ assert(txn_db_impl_->GetTxnDBOptions().write_policy == WRITE_UNPREPARED);
+ assert(GetId() > 0);
+ s = RollbackInternal();
+
+ if (s.ok()) {
+ dbimpl_->logs_with_prep_tracker()->MarkLogAsHavingPrepSectionFlushed(
+ log_number_);
+ }
+ }
+ // prepare couldn't have taken place
+ Clear();
+ } else if (txn_state_ == COMMITED) {
+ s = Status::InvalidArgument("This transaction has already been committed.");
+ } else {
+ s = Status::InvalidArgument(
+ "Two phase transaction is not in state for rollback.");
+ }
+
+ return s;
+}
+
+Status WriteCommittedTxn::RollbackInternal() {
+ WriteBatch rollback_marker;
+ WriteBatchInternal::MarkRollback(&rollback_marker, name_);
+ auto s = db_impl_->WriteImpl(write_options_, &rollback_marker);
+ return s;
+}
+
+Status PessimisticTransaction::RollbackToSavePoint() {
+ if (txn_state_ != STARTED) {
+ return Status::InvalidArgument("Transaction is beyond state for rollback.");
+ }
+
+ // Unlock any keys locked since last transaction
+ const std::unique_ptr<TransactionKeyMap>& keys =
+ GetTrackedKeysSinceSavePoint();
+
+ if (keys) {
+ txn_db_impl_->UnLock(this, keys.get());
+ }
+
+ return TransactionBaseImpl::RollbackToSavePoint();
+}
+
+// Lock all keys in this batch.
+// On success, caller should unlock keys_to_unlock
+Status PessimisticTransaction::LockBatch(WriteBatch* batch,
+ TransactionKeyMap* keys_to_unlock) {
+ class Handler : public WriteBatch::Handler {
+ public:
+ // Sorted map of column_family_id to sorted set of keys.
+ // Since LockBatch() always locks keys in sorted order, it cannot deadlock
+ // with itself. We're not using a comparator here since it doesn't matter
+ // what the sorting is as long as it's consistent.
+ std::map<uint32_t, std::set<std::string>> keys_;
+
+ Handler() {}
+
+ void RecordKey(uint32_t column_family_id, const Slice& key) {
+ std::string key_str = key.ToString();
+
+ auto& cfh_keys = keys_[column_family_id];
+ auto iter = cfh_keys.find(key_str);
+ if (iter == cfh_keys.end()) {
+ // key not yet seen, store it.
+ cfh_keys.insert({std::move(key_str)});
+ }
+ }
+
+ Status PutCF(uint32_t column_family_id, const Slice& key,
+ const Slice& /* unused */) override {
+ RecordKey(column_family_id, key);
+ return Status::OK();
+ }
+ Status MergeCF(uint32_t column_family_id, const Slice& key,
+ const Slice& /* unused */) override {
+ RecordKey(column_family_id, key);
+ return Status::OK();
+ }
+ Status DeleteCF(uint32_t column_family_id, const Slice& key) override {
+ RecordKey(column_family_id, key);
+ return Status::OK();
+ }
+ };
+
+ // Iterating on this handler will add all keys in this batch into keys
+ Handler handler;
+ batch->Iterate(&handler);
+
+ Status s;
+
+ // Attempt to lock all keys
+ for (const auto& cf_iter : handler.keys_) {
+ uint32_t cfh_id = cf_iter.first;
+ auto& cfh_keys = cf_iter.second;
+
+ for (const auto& key_iter : cfh_keys) {
+ const std::string& key = key_iter;
+
+ s = txn_db_impl_->TryLock(this, cfh_id, key, true /* exclusive */);
+ if (!s.ok()) {
+ break;
+ }
+ TrackKey(keys_to_unlock, cfh_id, std::move(key), kMaxSequenceNumber,
+ false, true /* exclusive */);
+ }
+
+ if (!s.ok()) {
+ break;
+ }
+ }
+
+ if (!s.ok()) {
+ txn_db_impl_->UnLock(this, keys_to_unlock);
+ }
+
+ return s;
+}
+
+// Attempt to lock this key.
+// Returns OK if the key has been successfully locked. Non-ok, otherwise.
+// If check_shapshot is true and this transaction has a snapshot set,
+// this key will only be locked if there have been no writes to this key since
+// the snapshot time.
+Status PessimisticTransaction::TryLock(ColumnFamilyHandle* column_family,
+ const Slice& key, bool read_only,
+ bool exclusive, const bool do_validate,
+ const bool assume_tracked) {
+ assert(!assume_tracked || !do_validate);
+ Status s;
+ if (UNLIKELY(skip_concurrency_control_)) {
+ return s;
+ }
+ uint32_t cfh_id = GetColumnFamilyID(column_family);
+ std::string key_str = key.ToString();
+ bool previously_locked;
+ bool lock_upgrade = false;
+
+ // lock this key if this transactions hasn't already locked it
+ SequenceNumber tracked_at_seq = kMaxSequenceNumber;
+
+ const auto& tracked_keys = GetTrackedKeys();
+ const auto tracked_keys_cf = tracked_keys.find(cfh_id);
+ if (tracked_keys_cf == tracked_keys.end()) {
+ previously_locked = false;
+ } else {
+ auto iter = tracked_keys_cf->second.find(key_str);
+ if (iter == tracked_keys_cf->second.end()) {
+ previously_locked = false;
+ } else {
+ if (!iter->second.exclusive && exclusive) {
+ lock_upgrade = true;
+ }
+ previously_locked = true;
+ tracked_at_seq = iter->second.seq;
+ }
+ }
+
+ // Lock this key if this transactions hasn't already locked it or we require
+ // an upgrade.
+ if (!previously_locked || lock_upgrade) {
+ s = txn_db_impl_->TryLock(this, cfh_id, key_str, exclusive);
+ }
+
+ SetSnapshotIfNeeded();
+
+ // Even though we do not care about doing conflict checking for this write,
+ // we still need to take a lock to make sure we do not cause a conflict with
+ // some other write. However, we do not need to check if there have been
+ // any writes since this transaction's snapshot.
+ // TODO(agiardullo): could optimize by supporting shared txn locks in the
+ // future
+ if (!do_validate || snapshot_ == nullptr) {
+ if (assume_tracked && !previously_locked) {
+ s = Status::InvalidArgument(
+ "assume_tracked is set but it is not tracked yet");
+ }
+ // Need to remember the earliest sequence number that we know that this
+ // key has not been modified after. This is useful if this same
+ // transaction
+ // later tries to lock this key again.
+ if (tracked_at_seq == kMaxSequenceNumber) {
+ // Since we haven't checked a snapshot, we only know this key has not
+ // been modified since after we locked it.
+ // Note: when last_seq_same_as_publish_seq_==false this is less than the
+ // latest allocated seq but it is ok since i) this is just a heuristic
+ // used only as a hint to avoid actual check for conflicts, ii) this would
+ // cause a false positive only if the snapthot is taken right after the
+ // lock, which would be an unusual sequence.
+ tracked_at_seq = db_->GetLatestSequenceNumber();
+ }
+ } else {
+ // If a snapshot is set, we need to make sure the key hasn't been modified
+ // since the snapshot. This must be done after we locked the key.
+ // If we already have validated an earilier snapshot it must has been
+ // reflected in tracked_at_seq and ValidateSnapshot will return OK.
+ if (s.ok()) {
+ s = ValidateSnapshot(column_family, key, &tracked_at_seq);
+
+ if (!s.ok()) {
+ // Failed to validate key
+ if (!previously_locked) {
+ // Unlock key we just locked
+ if (lock_upgrade) {
+ s = txn_db_impl_->TryLock(this, cfh_id, key_str,
+ false /* exclusive */);
+ assert(s.ok());
+ } else {
+ txn_db_impl_->UnLock(this, cfh_id, key.ToString());
+ }
+ }
+ }
+ }
+ }
+
+ if (s.ok()) {
+ // We must track all the locked keys so that we can unlock them later. If
+ // the key is already locked, this func will update some stats on the
+ // tracked key. It could also update the tracked_at_seq if it is lower
+ // than the existing tracked key seq. These stats are necessary for
+ // RollbackToSavePoint to determine whether a key can be safely removed
+ // from tracked_keys_. Removal can only be done if a key was only locked
+ // during the current savepoint.
+ //
+ // Recall that if assume_tracked is true, we assume that TrackKey has been
+ // called previously since the last savepoint, with the same exclusive
+ // setting, and at a lower sequence number, so skipping here should be
+ // safe.
+ if (!assume_tracked) {
+ TrackKey(cfh_id, key_str, tracked_at_seq, read_only, exclusive);
+ } else {
+#ifndef NDEBUG
+ assert(tracked_keys_cf->second.count(key_str) > 0);
+ const auto& info = tracked_keys_cf->second.find(key_str)->second;
+ assert(info.seq <= tracked_at_seq);
+ assert(info.exclusive == exclusive);
+#endif
+ }
+ }
+
+ return s;
+}
+
+// Return OK() if this key has not been modified more recently than the
+// transaction snapshot_.
+// tracked_at_seq is the global seq at which we either locked the key or already
+// have done ValidateSnapshot.
+Status PessimisticTransaction::ValidateSnapshot(
+ ColumnFamilyHandle* column_family, const Slice& key,
+ SequenceNumber* tracked_at_seq) {
+ assert(snapshot_);
+
+ SequenceNumber snap_seq = snapshot_->GetSequenceNumber();
+ if (*tracked_at_seq <= snap_seq) {
+ // If the key has been previous validated (or locked) at a sequence number
+ // earlier than the current snapshot's sequence number, we already know it
+ // has not been modified aftter snap_seq either.
+ return Status::OK();
+ }
+ // Otherwise we have either
+ // 1: tracked_at_seq == kMaxSequenceNumber, i.e., first time tracking the key
+ // 2: snap_seq < tracked_at_seq: last time we lock the key was via
+ // do_validate=false which means we had skipped ValidateSnapshot. In both
+ // cases we should do ValidateSnapshot now.
+
+ *tracked_at_seq = snap_seq;
+
+ ColumnFamilyHandle* cfh =
+ column_family ? column_family : db_impl_->DefaultColumnFamily();
+
+ return TransactionUtil::CheckKeyForConflicts(
+ db_impl_, cfh, key.ToString(), snap_seq, false /* cache_only */);
+}
+
+bool PessimisticTransaction::TryStealingLocks() {
+ assert(IsExpired());
+ TransactionState expected = STARTED;
+ return std::atomic_compare_exchange_strong(&txn_state_, &expected,
+ LOCKS_STOLEN);
+}
+
+void PessimisticTransaction::UnlockGetForUpdate(
+ ColumnFamilyHandle* column_family, const Slice& key) {
+ txn_db_impl_->UnLock(this, GetColumnFamilyID(column_family), key.ToString());
+}
+
+Status PessimisticTransaction::SetName(const TransactionName& name) {
+ Status s;
+ if (txn_state_ == STARTED) {
+ if (name_.length()) {
+ s = Status::InvalidArgument("Transaction has already been named.");
+ } else if (txn_db_impl_->GetTransactionByName(name) != nullptr) {
+ s = Status::InvalidArgument("Transaction name must be unique.");
+ } else if (name.length() < 1 || name.length() > 512) {
+ s = Status::InvalidArgument(
+ "Transaction name length must be between 1 and 512 chars.");
+ } else {
+ name_ = name;
+ txn_db_impl_->RegisterTransaction(this);
+ }
+ } else {
+ s = Status::InvalidArgument("Transaction is beyond state for naming.");
+ }
+ return s;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/pessimistic_transaction.h b/src/rocksdb/utilities/transactions/pessimistic_transaction.h
new file mode 100644
index 000000000..8f2c84405
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/pessimistic_transaction.h
@@ -0,0 +1,225 @@
+// 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;
+
+ virtual ~PessimisticTransaction();
+
+ 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_; }
+
+ protected:
+ // Refer to
+ // TransactionOptions::use_only_the_last_commit_time_batch_for_recovery
+ bool use_only_the_last_commit_time_batch_for_recovery_ = 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);
+
+ Status LockBatch(WriteBatch* batch, TransactionKeyMap* 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_;
+
+ 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 TransactionLockMgr::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;
+
+ virtual ~WriteCommittedTxn() {}
+
+ 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
diff --git a/src/rocksdb/utilities/transactions/pessimistic_transaction_db.cc b/src/rocksdb/utilities/transactions/pessimistic_transaction_db.cc
new file mode 100644
index 000000000..30d5b79f6
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/pessimistic_transaction_db.cc
@@ -0,0 +1,632 @@
+// 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>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "db/db_impl/db_impl.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>(db)),
+ txn_db_options_(txn_db_options),
+ lock_mgr_(this, txn_db_options_.num_stripes, txn_db_options.max_num_locks,
+ txn_db_options_.max_num_deadlocks,
+ txn_db_options_.custom_mutex_factory
+ ? txn_db_options_.custom_mutex_factory
+ : std::shared_ptr<TransactionDBMutexFactory>(
+ new TransactionDBMutexFactoryImpl())) {
+ 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>(db->GetRootDB())),
+ txn_db_options_(txn_db_options),
+ lock_mgr_(this, txn_db_options_.num_stripes, txn_db_options.max_num_locks,
+ txn_db_options_.max_num_deadlocks,
+ txn_db_options_.custom_mutex_factory
+ ? txn_db_options_.custom_mutex_factory
+ : std::shared_ptr<TransactionDBMutexFactory>(
+ new TransactionDBMutexFactoryImpl())) {
+ 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_.
+ }
+}
+
+Status PessimisticTransactionDB::VerifyCFOptions(const ColumnFamilyOptions&) {
+ 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, DB>(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;
+ 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()) {
+ ROCKS_LOG_WARN(db->GetDBOptions().info_log,
+ "Transaction write_policy is %" PRId32,
+ static_cast<int>(txn_db_options.write_policy));
+ s = WrapDB(db, txn_db_options, compaction_enabled_cf_indices, *handles,
+ dbptr);
+ }
+ if (!s.ok()) {
+ // just in case it was not deleted (and not set to nullptr).
+ delete db;
+ }
+ 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;
+
+ 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;
+}
+
+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) {
+ assert(db != nullptr);
+ assert(dbptr != nullptr);
+ *dbptr = nullptr;
+ std::unique_ptr<PessimisticTransactionDB> txn_db;
+ 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();
+ }
+ return s;
+}
+
+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) {
+ assert(db != nullptr);
+ assert(dbptr != nullptr);
+ *dbptr = nullptr;
+ std::unique_ptr<PessimisticTransactionDB> txn_db;
+
+ 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();
+ }
+ return s;
+}
+
+// Let TransactionLockMgr know that this column family exists so it can
+// allocate a LockMap for it.
+void PessimisticTransactionDB::AddColumnFamily(
+ const ColumnFamilyHandle* handle) {
+ lock_mgr_.AddColumnFamily(handle->GetID());
+}
+
+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_mgr_.AddColumnFamily((*handle)->GetID());
+ UpdateCFComparatorMap(*handle);
+ }
+
+ return s;
+}
+
+// Let TransactionLockMgr 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_mgr_.RemoveColumnFamily(column_family->GetID());
+ }
+
+ return s;
+}
+
+Status PessimisticTransactionDB::TryLock(PessimisticTransaction* txn,
+ uint32_t cfh_id,
+ const std::string& key,
+ bool exclusive) {
+ return lock_mgr_.TryLock(txn, cfh_id, key, GetEnv(), exclusive);
+}
+
+void PessimisticTransactionDB::UnLock(PessimisticTransaction* txn,
+ const TransactionKeyMap* keys) {
+ lock_mgr_.UnLock(txn, keys, GetEnv());
+}
+
+void PessimisticTransactionDB::UnLock(PessimisticTransaction* txn,
+ uint32_t cfh_id, const std::string& key) {
+ lock_mgr_.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) {
+ Status 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) {
+ Status 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) {
+ Status 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) {
+ Status 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) {
+ 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) {
+ 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, Transaction>(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);
+ }
+ }
+}
+
+TransactionLockMgr::LockStatusData
+PessimisticTransactionDB::GetLockStatusData() {
+ return lock_mgr_.GetLockStatusData();
+}
+
+std::vector<DeadlockPath> PessimisticTransactionDB::GetDeadlockInfoBuffer() {
+ return lock_mgr_.GetDeadlockInfoBuffer();
+}
+
+void PessimisticTransactionDB::SetDeadlockInfoBufferSize(uint32_t target_size) {
+ lock_mgr_.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);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/pessimistic_transaction_db.h b/src/rocksdb/utilities/transactions/pessimistic_transaction_db.h
new file mode 100644
index 000000000..39346dddd
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/pessimistic_transaction_db.h
@@ -0,0 +1,220 @@
+// 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 <mutex>
+#include <queue>
+#include <set>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "db/db_iter.h"
+#include "db/read_callback.h"
+#include "db/snapshot_checker.h"
+#include "rocksdb/db.h"
+#include "rocksdb/options.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "util/cast_util.h"
+#include "utilities/transactions/pessimistic_transaction.h"
+#include "utilities/transactions/transaction_lock_mgr.h"
+#include "utilities/transactions/write_prepared_txn.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class PessimisticTransactionDB : public TransactionDB {
+ public:
+ explicit PessimisticTransactionDB(DB* db,
+ const TransactionDBOptions& txn_db_options);
+
+ explicit PessimisticTransactionDB(StackableDB* db,
+ const TransactionDBOptions& txn_db_options);
+
+ virtual ~PessimisticTransactionDB();
+
+ virtual const Snapshot* GetSnapshot() override { return db_->GetSnapshot(); }
+
+ virtual Status Initialize(
+ const std::vector<size_t>& compaction_enabled_cf_indices,
+ const std::vector<ColumnFamilyHandle*>& handles);
+
+ Transaction* BeginTransaction(const WriteOptions& write_options,
+ const TransactionOptions& txn_options,
+ Transaction* old_txn) override = 0;
+
+ using StackableDB::Put;
+ virtual Status Put(const WriteOptions& options,
+ ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& val) override;
+
+ using StackableDB::Delete;
+ virtual Status Delete(const WriteOptions& wopts,
+ ColumnFamilyHandle* column_family,
+ const Slice& key) override;
+
+ using StackableDB::SingleDelete;
+ virtual Status SingleDelete(const WriteOptions& wopts,
+ ColumnFamilyHandle* column_family,
+ const Slice& key) override;
+
+ using StackableDB::Merge;
+ virtual Status Merge(const WriteOptions& options,
+ ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& value) override;
+
+ using TransactionDB::Write;
+ virtual Status Write(const WriteOptions& opts, WriteBatch* updates) override;
+ inline Status WriteWithConcurrencyControl(const WriteOptions& opts,
+ WriteBatch* updates) {
+ // Need to lock all keys in this batch to prevent write conflicts with
+ // concurrent transactions.
+ Transaction* txn = BeginInternalTransaction(opts);
+ txn->DisableIndexing();
+
+ auto txn_impl =
+ static_cast_with_check<PessimisticTransaction, Transaction>(txn);
+
+ // Since commitBatch sorts the keys before locking, concurrent Write()
+ // operations will not cause a deadlock.
+ // In order to avoid a deadlock with a concurrent Transaction, Transactions
+ // should use a lock timeout.
+ Status s = txn_impl->CommitBatch(updates);
+
+ delete txn;
+
+ return s;
+ }
+
+ using StackableDB::CreateColumnFamily;
+ virtual Status CreateColumnFamily(const ColumnFamilyOptions& options,
+ const std::string& column_family_name,
+ ColumnFamilyHandle** handle) override;
+
+ using StackableDB::DropColumnFamily;
+ virtual Status DropColumnFamily(ColumnFamilyHandle* column_family) override;
+
+ Status TryLock(PessimisticTransaction* txn, uint32_t cfh_id,
+ const std::string& key, bool exclusive);
+
+ void UnLock(PessimisticTransaction* txn, const TransactionKeyMap* keys);
+ void UnLock(PessimisticTransaction* txn, uint32_t cfh_id,
+ const std::string& key);
+
+ void AddColumnFamily(const ColumnFamilyHandle* handle);
+
+ static TransactionDBOptions ValidateTxnDBOptions(
+ const TransactionDBOptions& txn_db_options);
+
+ const TransactionDBOptions& GetTxnDBOptions() const {
+ return txn_db_options_;
+ }
+
+ void InsertExpirableTransaction(TransactionID tx_id,
+ PessimisticTransaction* tx);
+ void RemoveExpirableTransaction(TransactionID tx_id);
+
+ // If transaction is no longer available, locks can be stolen
+ // If transaction is available, try stealing locks directly from transaction
+ // It is the caller's responsibility to ensure that the referred transaction
+ // is expirable (GetExpirationTime() > 0) and that it is expired.
+ bool TryStealingExpiredTransactionLocks(TransactionID tx_id);
+
+ Transaction* GetTransactionByName(const TransactionName& name) override;
+
+ void RegisterTransaction(Transaction* txn);
+ void UnregisterTransaction(Transaction* txn);
+
+ // not thread safe. current use case is during recovery (single thread)
+ void GetAllPreparedTransactions(std::vector<Transaction*>* trans) override;
+
+ TransactionLockMgr::LockStatusData GetLockStatusData() override;
+
+ std::vector<DeadlockPath> GetDeadlockInfoBuffer() override;
+ void SetDeadlockInfoBufferSize(uint32_t target_size) override;
+
+ // The default implementation does nothing. The actual implementation is moved
+ // to the child classes that actually need this information. This was due to
+ // an odd performance drop we observed when the added std::atomic member to
+ // the base class even when the subclass do not read it in the fast path.
+ virtual void UpdateCFComparatorMap(const std::vector<ColumnFamilyHandle*>&) {}
+ virtual void UpdateCFComparatorMap(ColumnFamilyHandle*) {}
+
+ protected:
+ DBImpl* db_impl_;
+ std::shared_ptr<Logger> info_log_;
+ const TransactionDBOptions txn_db_options_;
+
+ void ReinitializeTransaction(
+ Transaction* txn, const WriteOptions& write_options,
+ const TransactionOptions& txn_options = TransactionOptions());
+
+ virtual Status VerifyCFOptions(const ColumnFamilyOptions& cf_options);
+
+ private:
+ friend class WritePreparedTxnDB;
+ friend class WritePreparedTxnDBMock;
+ friend class WriteUnpreparedTxn;
+ friend class TransactionTest_DoubleCrashInRecovery_Test;
+ friend class TransactionTest_DoubleEmptyWrite_Test;
+ friend class TransactionTest_DuplicateKeys_Test;
+ friend class TransactionTest_PersistentTwoPhaseTransactionTest_Test;
+ friend class TransactionTest_TwoPhaseDoubleRecoveryTest_Test;
+ friend class TransactionTest_TwoPhaseOutOfOrderDelete_Test;
+ friend class TransactionStressTest_TwoPhaseLongPrepareTest_Test;
+ friend class WriteUnpreparedTransactionTest_RecoveryTest_Test;
+ friend class WriteUnpreparedTransactionTest_MarkLogWithPrepSection_Test;
+ TransactionLockMgr lock_mgr_;
+
+ // Must be held when adding/dropping column families.
+ InstrumentedMutex column_family_mutex_;
+ Transaction* BeginInternalTransaction(const WriteOptions& options);
+
+ // Used to ensure that no locks are stolen from an expirable transaction
+ // that has started a commit. Only transactions with an expiration time
+ // should be in this map.
+ std::mutex map_mutex_;
+ std::unordered_map<TransactionID, PessimisticTransaction*>
+ expirable_transactions_map_;
+
+ // map from name to two phase transaction instance
+ std::mutex name_map_mutex_;
+ std::unordered_map<TransactionName, Transaction*> transactions_;
+
+ // Signal that we are testing a crash scenario. Some asserts could be relaxed
+ // in such cases.
+ virtual void TEST_Crash() {}
+};
+
+// A PessimisticTransactionDB that writes the data to the DB after the commit.
+// In this way the DB only contains the committed data.
+class WriteCommittedTxnDB : public PessimisticTransactionDB {
+ public:
+ explicit WriteCommittedTxnDB(DB* db,
+ const TransactionDBOptions& txn_db_options)
+ : PessimisticTransactionDB(db, txn_db_options) {}
+
+ explicit WriteCommittedTxnDB(StackableDB* db,
+ const TransactionDBOptions& txn_db_options)
+ : PessimisticTransactionDB(db, txn_db_options) {}
+
+ virtual ~WriteCommittedTxnDB() {}
+
+ Transaction* BeginTransaction(const WriteOptions& write_options,
+ const TransactionOptions& txn_options,
+ Transaction* old_txn) override;
+
+ // Optimized version of ::Write that makes use of skip_concurrency_control
+ // hint
+ using TransactionDB::Write;
+ virtual Status Write(const WriteOptions& opts,
+ const TransactionDBWriteOptimizations& optimizations,
+ WriteBatch* updates) override;
+ virtual Status Write(const WriteOptions& opts, WriteBatch* updates) override;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/snapshot_checker.cc b/src/rocksdb/utilities/transactions/snapshot_checker.cc
new file mode 100644
index 000000000..9c43bef43
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/snapshot_checker.cc
@@ -0,0 +1,49 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#include "db/snapshot_checker.h"
+
+#ifdef ROCKSDB_LITE
+#include <assert.h>
+#endif // ROCKSDB_LITE
+
+#include "utilities/transactions/write_prepared_txn_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+#ifdef ROCKSDB_LITE
+WritePreparedSnapshotChecker::WritePreparedSnapshotChecker(
+ WritePreparedTxnDB* /*txn_db*/) {}
+
+SnapshotCheckerResult WritePreparedSnapshotChecker::CheckInSnapshot(
+ SequenceNumber /*sequence*/, SequenceNumber /*snapshot_sequence*/) const {
+ // Should never be called in LITE mode.
+ assert(false);
+ return SnapshotCheckerResult::kInSnapshot;
+}
+
+#else
+
+WritePreparedSnapshotChecker::WritePreparedSnapshotChecker(
+ WritePreparedTxnDB* txn_db)
+ : txn_db_(txn_db){};
+
+SnapshotCheckerResult WritePreparedSnapshotChecker::CheckInSnapshot(
+ SequenceNumber sequence, SequenceNumber snapshot_sequence) const {
+ bool snapshot_released = false;
+ // TODO(myabandeh): set min_uncommitted
+ bool in_snapshot = txn_db_->IsInSnapshot(
+ sequence, snapshot_sequence, kMinUnCommittedSeq, &snapshot_released);
+ if (snapshot_released) {
+ return SnapshotCheckerResult::kSnapshotReleased;
+ }
+ return in_snapshot ? SnapshotCheckerResult::kInSnapshot
+ : SnapshotCheckerResult::kNotInSnapshot;
+}
+
+#endif // ROCKSDB_LITE
+DisableGCSnapshotChecker DisableGCSnapshotChecker::instance_;
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/utilities/transactions/transaction_base.cc b/src/rocksdb/utilities/transactions/transaction_base.cc
new file mode 100644
index 000000000..805d4ab36
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_base.cc
@@ -0,0 +1,837 @@
+// 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 "rocksdb/comparator.h"
+#include "rocksdb/db.h"
+#include "rocksdb/status.h"
+#include "util/cast_util.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+TransactionBaseImpl::TransactionBaseImpl(DB* db,
+ const WriteOptions& write_options)
+ : db_(db),
+ dbimpl_(static_cast_with_check<DBImpl, DB>(db)),
+ write_options_(write_options),
+ cmp_(GetColumnFamilyUserComparator(db->DefaultColumnFamily())),
+ start_time_(db_->GetEnv()->NowMicros()),
+ write_batch_(cmp_, 0, true, 0),
+ 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();
+ tracked_keys_.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_ = db_->GetEnv()->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_);
+ 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
+ const TransactionKeyMap& key_map = save_point.new_keys_;
+ for (const auto& key_map_iter : key_map) {
+ uint32_t column_family_id = key_map_iter.first;
+ auto& keys = key_map_iter.second;
+
+ auto& cf_tracked_keys = tracked_keys_[column_family_id];
+
+ for (const auto& key_iter : keys) {
+ const std::string& key = key_iter.first;
+ uint32_t num_reads = key_iter.second.num_reads;
+ uint32_t num_writes = key_iter.second.num_writes;
+
+ auto tracked_keys_iter = cf_tracked_keys.find(key);
+ assert(tracked_keys_iter != cf_tracked_keys.end());
+
+ // Decrement the total reads/writes of this key by the number of
+ // reads/writes done since the last SavePoint.
+ if (num_reads > 0) {
+ assert(tracked_keys_iter->second.num_reads >= num_reads);
+ tracked_keys_iter->second.num_reads -= num_reads;
+ }
+ if (num_writes > 0) {
+ assert(tracked_keys_iter->second.num_writes >= num_writes);
+ tracked_keys_iter->second.num_writes -= num_writes;
+ }
+ if (tracked_keys_iter->second.num_reads == 0 &&
+ tracked_keys_iter->second.num_writes == 0) {
+ cf_tracked_keys.erase(tracked_keys_iter);
+ }
+ }
+ }
+
+ 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;
+ std::swap(top, save_points_->top());
+ save_points_->pop();
+
+ const TransactionKeyMap& curr_cf_key_map = top.new_keys_;
+ TransactionKeyMap& prev_cf_key_map = save_points_->top().new_keys_;
+
+ for (const auto& curr_cf_key_iter : curr_cf_key_map) {
+ uint32_t column_family_id = curr_cf_key_iter.first;
+ const std::unordered_map<std::string, TransactionKeyMapInfo>& curr_keys =
+ curr_cf_key_iter.second;
+
+ // If cfid was not previously tracked, just copy everything over.
+ auto prev_keys_iter = prev_cf_key_map.find(column_family_id);
+ if (prev_keys_iter == prev_cf_key_map.end()) {
+ prev_cf_key_map.emplace(curr_cf_key_iter);
+ } else {
+ std::unordered_map<std::string, TransactionKeyMapInfo>& prev_keys =
+ prev_keys_iter->second;
+ for (const auto& key_iter : curr_keys) {
+ const std::string& key = key_iter.first;
+ const TransactionKeyMapInfo& info = key_iter.second;
+ // If key was not previously tracked, just copy the whole struct over.
+ // Otherwise, some merging needs to occur.
+ auto prev_info = prev_keys.find(key);
+ if (prev_info == prev_keys.end()) {
+ prev_keys.emplace(key_iter);
+ } else {
+ prev_info->second.Merge(info);
+ }
+ }
+ }
+ }
+ }
+
+ 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) {
+ std::string* value = values ? &(*values)[i] : nullptr;
+ stat_list[i] = Get(read_options, column_family[i], keys[i], value);
+ }
+
+ 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) {
+ std::string* value = values ? &(*values)[i] : nullptr;
+ stat_list[i] = Get(read_options, column_family[i], keys[i], value);
+ }
+
+ return stat_list;
+}
+
+Iterator* TransactionBaseImpl::GetIterator(const ReadOptions& read_options) {
+ Iterator* db_iter = db_->NewIterator(read_options);
+ assert(db_iter);
+
+ return write_batch_.NewIteratorWithBase(db_iter);
+}
+
+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) {
+ write_batch_.PutLogData(blob);
+}
+
+WriteBatchWithIndex* TransactionBaseImpl::GetWriteBatch() {
+ return &write_batch_;
+}
+
+uint64_t TransactionBaseImpl::GetElapsedTime() const {
+ return (db_->GetEnv()->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 {
+ uint64_t count = 0;
+
+ // sum up locked keys in all column families
+ for (const auto& key_map_iter : tracked_keys_) {
+ const auto& keys = key_map_iter.second;
+ count += keys.size();
+ }
+
+ return count;
+}
+
+void TransactionBaseImpl::TrackKey(uint32_t cfh_id, const std::string& key,
+ SequenceNumber seq, bool read_only,
+ bool exclusive) {
+ // Update map of all tracked keys for this transaction
+ TrackKey(&tracked_keys_, cfh_id, key, seq, read_only, exclusive);
+
+ if (save_points_ != nullptr && !save_points_->empty()) {
+ // Update map of tracked keys in this SavePoint
+ TrackKey(&save_points_->top().new_keys_, cfh_id, key, seq, read_only,
+ exclusive);
+ }
+}
+
+// Add a key to the given TransactionKeyMap
+// seq for pessimistic transactions is the sequence number from which we know
+// there has not been a concurrent update to the key.
+void TransactionBaseImpl::TrackKey(TransactionKeyMap* key_map, uint32_t cfh_id,
+ const std::string& key, SequenceNumber seq,
+ bool read_only, bool exclusive) {
+ auto& cf_key_map = (*key_map)[cfh_id];
+#ifdef __cpp_lib_unordered_map_try_emplace
+ // use c++17's try_emplace if available, to avoid rehashing the key
+ // in case it is not already in the map
+ auto result = cf_key_map.try_emplace(key, seq);
+ auto iter = result.first;
+ if (!result.second && seq < iter->second.seq) {
+ // Now tracking this key with an earlier sequence number
+ iter->second.seq = seq;
+ }
+#else
+ auto iter = cf_key_map.find(key);
+ if (iter == cf_key_map.end()) {
+ auto result = cf_key_map.emplace(key, TransactionKeyMapInfo(seq));
+ iter = result.first;
+ } else if (seq < iter->second.seq) {
+ // Now tracking this key with an earlier sequence number
+ iter->second.seq = seq;
+ }
+#endif
+ // else we do not update the seq. The smaller the tracked seq, the stronger it
+ // the guarantee since it implies from the seq onward there has not been a
+ // concurrent update to the key. So we update the seq if it implies stronger
+ // guarantees, i.e., if it is smaller than the existing tracked seq.
+
+ if (read_only) {
+ iter->second.num_reads++;
+ } else {
+ iter->second.num_writes++;
+ }
+ iter->second.exclusive |= exclusive;
+}
+
+std::unique_ptr<TransactionKeyMap>
+TransactionBaseImpl::GetTrackedKeysSinceSavePoint() {
+ if (save_points_ != nullptr && !save_points_->empty()) {
+ // Examine the number of reads/writes performed on all keys written
+ // since the last SavePoint and compare to the total number of reads/writes
+ // for each key.
+ TransactionKeyMap* result = new TransactionKeyMap();
+ for (const auto& key_map_iter : save_points_->top().new_keys_) {
+ uint32_t column_family_id = key_map_iter.first;
+ auto& keys = key_map_iter.second;
+
+ auto& cf_tracked_keys = tracked_keys_[column_family_id];
+
+ for (const auto& key_iter : keys) {
+ const std::string& key = key_iter.first;
+ uint32_t num_reads = key_iter.second.num_reads;
+ uint32_t num_writes = key_iter.second.num_writes;
+
+ auto total_key_info = cf_tracked_keys.find(key);
+ assert(total_key_info != cf_tracked_keys.end());
+ assert(total_key_info->second.num_reads >= num_reads);
+ assert(total_key_info->second.num_writes >= num_writes);
+
+ if (total_key_info->second.num_reads == num_reads &&
+ total_key_info->second.num_writes == num_writes) {
+ // All the reads/writes to this key were done in the last savepoint.
+ bool read_only = (num_writes == 0);
+ TrackKey(result, column_family_id, key, key_iter.second.seq,
+ read_only, key_iter.second.exclusive);
+ }
+ }
+ }
+ return std::unique_ptr<TransactionKeyMap>(result);
+ }
+
+ // No SavePoint
+ return nullptr;
+}
+
+// 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) {
+ uint32_t column_family_id = GetColumnFamilyID(column_family);
+ auto& cf_tracked_keys = tracked_keys_[column_family_id];
+ std::string key_str = key.ToString();
+ bool can_decrement = false;
+ bool can_unlock __attribute__((__unused__)) = false;
+
+ if (save_points_ != nullptr && !save_points_->empty()) {
+ // Check if this key was fetched ForUpdate in this SavePoint
+ auto& cf_savepoint_keys = save_points_->top().new_keys_[column_family_id];
+
+ auto savepoint_iter = cf_savepoint_keys.find(key_str);
+ if (savepoint_iter != cf_savepoint_keys.end()) {
+ if (savepoint_iter->second.num_reads > 0) {
+ savepoint_iter->second.num_reads--;
+ can_decrement = true;
+
+ if (savepoint_iter->second.num_reads == 0 &&
+ savepoint_iter->second.num_writes == 0) {
+ // No other GetForUpdates or write on this key in this SavePoint
+ cf_savepoint_keys.erase(savepoint_iter);
+ can_unlock = true;
+ }
+ }
+ }
+ } else {
+ // No SavePoint set
+ can_decrement = true;
+ can_unlock = true;
+ }
+
+ // We can only decrement the read count for this key if we were able to
+ // decrement the read count in the current SavePoint, OR if there is no
+ // SavePoint set.
+ if (can_decrement) {
+ auto key_iter = cf_tracked_keys.find(key_str);
+
+ if (key_iter != cf_tracked_keys.end()) {
+ if (key_iter->second.num_reads > 0) {
+ key_iter->second.num_reads--;
+
+ if (key_iter->second.num_reads == 0 &&
+ key_iter->second.num_writes == 0) {
+ // No other GetForUpdates or writes on this key
+ assert(can_unlock);
+ cf_tracked_keys.erase(key_iter);
+ 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 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
diff --git a/src/rocksdb/utilities/transactions/transaction_base.h b/src/rocksdb/utilities/transactions/transaction_base.h
new file mode 100644
index 000000000..f279676c6
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_base.h
@@ -0,0 +1,374 @@
+// 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 <stack>
+#include <string>
+#include <vector>
+
+#include "db/write_batch_internal.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_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class TransactionBaseImpl : public Transaction {
+ public:
+ TransactionBaseImpl(DB* db, const WriteOptions& write_options);
+
+ virtual ~TransactionBaseImpl();
+
+ // Remove pending operations queued in this transaction.
+ virtual void Clear();
+
+ void Reinitialize(DB* db, const WriteOptions& write_options);
+
+ // Called before executing Put, Merge, Delete, and GetForUpdate. If TryLock
+ // returns non-OK, the Put/Merge/Delete/GetForUpdate will be failed.
+ // do_validate will be false if called from PutUntracked, DeleteUntracked,
+ // MergeUntracked, or GetForUpdate(do_validate=false)
+ virtual Status TryLock(ColumnFamilyHandle* column_family, const Slice& key,
+ bool read_only, bool exclusive,
+ const bool do_validate = true,
+ const bool assume_tracked = false) = 0;
+
+ void SetSavePoint() override;
+
+ Status RollbackToSavePoint() override;
+
+ Status PopSavePoint() override;
+
+ using Transaction::Get;
+ Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family,
+ const Slice& key, std::string* value) override;
+
+ Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family,
+ const Slice& key, PinnableSlice* value) override;
+
+ Status Get(const ReadOptions& options, const Slice& key,
+ std::string* value) override {
+ return Get(options, db_->DefaultColumnFamily(), key, value);
+ }
+
+ using Transaction::GetForUpdate;
+ Status GetForUpdate(const ReadOptions& options,
+ ColumnFamilyHandle* column_family, const Slice& key,
+ std::string* value, bool exclusive,
+ const bool do_validate) override;
+
+ Status GetForUpdate(const ReadOptions& options,
+ ColumnFamilyHandle* column_family, const Slice& key,
+ PinnableSlice* pinnable_val, bool exclusive,
+ const bool do_validate) override;
+
+ Status GetForUpdate(const ReadOptions& options, const Slice& key,
+ std::string* value, bool exclusive,
+ const bool do_validate) override {
+ return GetForUpdate(options, db_->DefaultColumnFamily(), key, value,
+ exclusive, do_validate);
+ }
+
+ using Transaction::MultiGet;
+ std::vector<Status> MultiGet(
+ const ReadOptions& options,
+ const std::vector<ColumnFamilyHandle*>& column_family,
+ const std::vector<Slice>& keys,
+ std::vector<std::string>* values) override;
+
+ std::vector<Status> MultiGet(const ReadOptions& options,
+ const std::vector<Slice>& keys,
+ std::vector<std::string>* values) override {
+ return MultiGet(options, std::vector<ColumnFamilyHandle*>(
+ keys.size(), db_->DefaultColumnFamily()),
+ keys, values);
+ }
+
+ void MultiGet(const ReadOptions& options, ColumnFamilyHandle* column_family,
+ const size_t num_keys, const Slice* keys, PinnableSlice* values,
+ Status* statuses, const bool sorted_input = false) override;
+
+ using Transaction::MultiGetForUpdate;
+ std::vector<Status> MultiGetForUpdate(
+ const ReadOptions& options,
+ const std::vector<ColumnFamilyHandle*>& column_family,
+ const std::vector<Slice>& keys,
+ std::vector<std::string>* values) override;
+
+ std::vector<Status> MultiGetForUpdate(
+ const ReadOptions& options, const std::vector<Slice>& keys,
+ std::vector<std::string>* values) override {
+ return MultiGetForUpdate(options,
+ std::vector<ColumnFamilyHandle*>(
+ keys.size(), db_->DefaultColumnFamily()),
+ keys, values);
+ }
+
+ Iterator* GetIterator(const ReadOptions& read_options) override;
+ Iterator* GetIterator(const ReadOptions& read_options,
+ ColumnFamilyHandle* column_family) override;
+
+ Status Put(ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& value, const bool assume_tracked = false) override;
+ Status Put(const Slice& key, const Slice& value) override {
+ return Put(nullptr, key, value);
+ }
+
+ Status Put(ColumnFamilyHandle* column_family, const SliceParts& key,
+ const SliceParts& value,
+ const bool assume_tracked = false) override;
+ Status Put(const SliceParts& key, const SliceParts& value) override {
+ return Put(nullptr, key, value);
+ }
+
+ Status Merge(ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& value, const bool assume_tracked = false) override;
+ Status Merge(const Slice& key, const Slice& value) override {
+ return Merge(nullptr, key, value);
+ }
+
+ Status Delete(ColumnFamilyHandle* column_family, const Slice& key,
+ const bool assume_tracked = false) override;
+ Status Delete(const Slice& key) override { return Delete(nullptr, key); }
+ Status Delete(ColumnFamilyHandle* column_family, const SliceParts& key,
+ const bool assume_tracked = false) override;
+ Status Delete(const SliceParts& key) override { return Delete(nullptr, key); }
+
+ Status SingleDelete(ColumnFamilyHandle* column_family, const Slice& key,
+ const bool assume_tracked = false) override;
+ Status SingleDelete(const Slice& key) override {
+ return SingleDelete(nullptr, key);
+ }
+ Status SingleDelete(ColumnFamilyHandle* column_family, const SliceParts& key,
+ const bool assume_tracked = false) override;
+ Status SingleDelete(const SliceParts& key) override {
+ return SingleDelete(nullptr, key);
+ }
+
+ Status PutUntracked(ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& value) override;
+ Status PutUntracked(const Slice& key, const Slice& value) override {
+ return PutUntracked(nullptr, key, value);
+ }
+
+ Status PutUntracked(ColumnFamilyHandle* column_family, const SliceParts& key,
+ const SliceParts& value) override;
+ Status PutUntracked(const SliceParts& key, const SliceParts& value) override {
+ return PutUntracked(nullptr, key, value);
+ }
+
+ Status MergeUntracked(ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& value) override;
+ Status MergeUntracked(const Slice& key, const Slice& value) override {
+ return MergeUntracked(nullptr, key, value);
+ }
+
+ Status DeleteUntracked(ColumnFamilyHandle* column_family,
+ const Slice& key) override;
+ Status DeleteUntracked(const Slice& key) override {
+ return DeleteUntracked(nullptr, key);
+ }
+ Status DeleteUntracked(ColumnFamilyHandle* column_family,
+ const SliceParts& key) override;
+ Status DeleteUntracked(const SliceParts& key) override {
+ return DeleteUntracked(nullptr, key);
+ }
+
+ Status SingleDeleteUntracked(ColumnFamilyHandle* column_family,
+ const Slice& key) override;
+ Status SingleDeleteUntracked(const Slice& key) override {
+ return SingleDeleteUntracked(nullptr, key);
+ }
+
+ void PutLogData(const Slice& blob) override;
+
+ WriteBatchWithIndex* GetWriteBatch() override;
+
+ virtual void SetLockTimeout(int64_t /*timeout*/) override { /* Do nothing */
+ }
+
+ const Snapshot* GetSnapshot() const override {
+ return snapshot_ ? snapshot_.get() : nullptr;
+ }
+
+ virtual void SetSnapshot() override;
+ void SetSnapshotOnNextOperation(
+ std::shared_ptr<TransactionNotifier> notifier = nullptr) override;
+
+ void ClearSnapshot() override {
+ snapshot_.reset();
+ snapshot_needed_ = false;
+ snapshot_notifier_ = nullptr;
+ }
+
+ void DisableIndexing() override { indexing_enabled_ = false; }
+
+ void EnableIndexing() override { indexing_enabled_ = true; }
+
+ uint64_t GetElapsedTime() const override;
+
+ uint64_t GetNumPuts() const override;
+
+ uint64_t GetNumDeletes() const override;
+
+ uint64_t GetNumMerges() const override;
+
+ uint64_t GetNumKeys() const override;
+
+ void UndoGetForUpdate(ColumnFamilyHandle* column_family,
+ const Slice& key) override;
+ void UndoGetForUpdate(const Slice& key) override {
+ return UndoGetForUpdate(nullptr, key);
+ };
+
+ // Get list of keys in this transaction that must not have any conflicts
+ // with writes in other transactions.
+ const TransactionKeyMap& GetTrackedKeys() const { return tracked_keys_; }
+
+ WriteOptions* GetWriteOptions() override { return &write_options_; }
+
+ void SetWriteOptions(const WriteOptions& write_options) override {
+ write_options_ = write_options;
+ }
+
+ // Used for memory management for snapshot_
+ void ReleaseSnapshot(const Snapshot* snapshot, DB* db);
+
+ // iterates over the given batch and makes the appropriate inserts.
+ // used for rebuilding prepared transactions after recovery.
+ virtual Status RebuildFromWriteBatch(WriteBatch* src_batch) override;
+
+ WriteBatch* GetCommitTimeWriteBatch() override;
+
+ protected:
+ // Add a key to the list of tracked keys.
+ //
+ // seqno is the earliest seqno this key was involved with this transaction.
+ // readonly should be set to true if no data was written for this key
+ void TrackKey(uint32_t cfh_id, const std::string& key, SequenceNumber seqno,
+ bool readonly, bool exclusive);
+
+ // Helper function to add a key to the given TransactionKeyMap
+ static void TrackKey(TransactionKeyMap* key_map, uint32_t cfh_id,
+ const std::string& key, SequenceNumber seqno,
+ bool readonly, bool exclusive);
+
+ // Called when UndoGetForUpdate determines that this key can be unlocked.
+ virtual void UnlockGetForUpdate(ColumnFamilyHandle* column_family,
+ const Slice& key) = 0;
+
+ std::unique_ptr<TransactionKeyMap> GetTrackedKeysSinceSavePoint();
+
+ // Sets a snapshot if SetSnapshotOnNextOperation() has been called.
+ void SetSnapshotIfNeeded();
+
+ // Initialize write_batch_ for 2PC by inserting Noop.
+ inline void InitWriteBatch(bool clear = false) {
+ if (clear) {
+ write_batch_.Clear();
+ }
+ assert(write_batch_.GetDataSize() == WriteBatchInternal::kHeader);
+ WriteBatchInternal::InsertNoop(write_batch_.GetWriteBatch());
+ }
+
+ DB* db_;
+ DBImpl* dbimpl_;
+
+ WriteOptions write_options_;
+
+ const Comparator* cmp_;
+
+ // Stores that time the txn was constructed, in microseconds.
+ uint64_t start_time_;
+
+ // Stores the current snapshot that was set by SetSnapshot or null if
+ // no snapshot is currently set.
+ std::shared_ptr<const Snapshot> snapshot_;
+
+ // Count of various operations pending in this transaction
+ uint64_t num_puts_ = 0;
+ uint64_t num_deletes_ = 0;
+ uint64_t num_merges_ = 0;
+
+ struct SavePoint {
+ std::shared_ptr<const Snapshot> snapshot_;
+ bool snapshot_needed_ = false;
+ std::shared_ptr<TransactionNotifier> snapshot_notifier_;
+ uint64_t num_puts_ = 0;
+ uint64_t num_deletes_ = 0;
+ uint64_t num_merges_ = 0;
+
+ // Record all keys tracked since the last savepoint
+ TransactionKeyMap new_keys_;
+
+ SavePoint(std::shared_ptr<const Snapshot> snapshot, bool snapshot_needed,
+ std::shared_ptr<TransactionNotifier> snapshot_notifier,
+ uint64_t num_puts, uint64_t num_deletes, uint64_t num_merges)
+ : snapshot_(snapshot),
+ snapshot_needed_(snapshot_needed),
+ snapshot_notifier_(snapshot_notifier),
+ num_puts_(num_puts),
+ num_deletes_(num_deletes),
+ num_merges_(num_merges) {}
+
+ SavePoint() = default;
+ };
+
+ // Records writes pending in this transaction
+ WriteBatchWithIndex write_batch_;
+
+ // Map from column_family_id to map of keys that are involved in this
+ // transaction.
+ // For Pessimistic Transactions this is the list of locked keys.
+ // Optimistic Transactions will wait till commit time to do conflict checking.
+ TransactionKeyMap tracked_keys_;
+
+ // Stack of the Snapshot saved at each save point. Saved snapshots may be
+ // nullptr if there was no snapshot at the time SetSavePoint() was called.
+ std::unique_ptr<std::stack<TransactionBaseImpl::SavePoint,
+ autovector<TransactionBaseImpl::SavePoint>>>
+ save_points_;
+
+ private:
+ friend class WritePreparedTxn;
+ // Extra data to be persisted with the commit. Note this is only used when
+ // prepare phase is not skipped.
+ WriteBatch commit_time_batch_;
+
+ // If true, future Put/Merge/Deletes will be indexed in the
+ // WriteBatchWithIndex.
+ // If false, future Put/Merge/Deletes will be inserted directly into the
+ // underlying WriteBatch and not indexed in the WriteBatchWithIndex.
+ bool indexing_enabled_;
+
+ // SetSnapshotOnNextOperation() has been called and the snapshot has not yet
+ // been reset.
+ bool snapshot_needed_ = false;
+
+ // SetSnapshotOnNextOperation() has been called and the caller would like
+ // a notification through the TransactionNotifier interface
+ std::shared_ptr<TransactionNotifier> snapshot_notifier_ = nullptr;
+
+ Status TryLock(ColumnFamilyHandle* column_family, const SliceParts& key,
+ bool read_only, bool exclusive, const bool do_validate = true,
+ const bool assume_tracked = false);
+
+ WriteBatchBase* GetBatchForWrite();
+ void SetSnapshotInternal(const Snapshot* snapshot);
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/transaction_db_mutex_impl.cc b/src/rocksdb/utilities/transactions/transaction_db_mutex_impl.cc
new file mode 100644
index 000000000..345c4be90
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_db_mutex_impl.cc
@@ -0,0 +1,135 @@
+// 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_db_mutex_impl.h"
+
+#include <chrono>
+#include <condition_variable>
+#include <functional>
+#include <mutex>
+
+#include "rocksdb/utilities/transaction_db_mutex.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class TransactionDBMutexImpl : public TransactionDBMutex {
+ public:
+ TransactionDBMutexImpl() {}
+ ~TransactionDBMutexImpl() override {}
+
+ Status Lock() override;
+
+ Status TryLockFor(int64_t timeout_time) override;
+
+ void UnLock() override { mutex_.unlock(); }
+
+ friend class TransactionDBCondVarImpl;
+
+ private:
+ std::mutex mutex_;
+};
+
+class TransactionDBCondVarImpl : public TransactionDBCondVar {
+ public:
+ TransactionDBCondVarImpl() {}
+ ~TransactionDBCondVarImpl() override {}
+
+ Status Wait(std::shared_ptr<TransactionDBMutex> mutex) override;
+
+ Status WaitFor(std::shared_ptr<TransactionDBMutex> mutex,
+ int64_t timeout_time) override;
+
+ void Notify() override { cv_.notify_one(); }
+
+ void NotifyAll() override { cv_.notify_all(); }
+
+ private:
+ std::condition_variable cv_;
+};
+
+std::shared_ptr<TransactionDBMutex>
+TransactionDBMutexFactoryImpl::AllocateMutex() {
+ return std::shared_ptr<TransactionDBMutex>(new TransactionDBMutexImpl());
+}
+
+std::shared_ptr<TransactionDBCondVar>
+TransactionDBMutexFactoryImpl::AllocateCondVar() {
+ return std::shared_ptr<TransactionDBCondVar>(new TransactionDBCondVarImpl());
+}
+
+Status TransactionDBMutexImpl::Lock() {
+ mutex_.lock();
+ return Status::OK();
+}
+
+Status TransactionDBMutexImpl::TryLockFor(int64_t timeout_time) {
+ bool locked = true;
+
+ if (timeout_time == 0) {
+ locked = mutex_.try_lock();
+ } else {
+ // Previously, this code used a std::timed_mutex. However, this was changed
+ // due to known bugs in gcc versions < 4.9.
+ // https://gcc.gnu.org/bugzilla/show_bug.cgi?id=54562
+ //
+ // Since this mutex isn't held for long and only a single mutex is ever
+ // held at a time, it is reasonable to ignore the lock timeout_time here
+ // and only check it when waiting on the condition_variable.
+ mutex_.lock();
+ }
+
+ if (!locked) {
+ // timeout acquiring mutex
+ return Status::TimedOut(Status::SubCode::kMutexTimeout);
+ }
+
+ return Status::OK();
+}
+
+Status TransactionDBCondVarImpl::Wait(
+ std::shared_ptr<TransactionDBMutex> mutex) {
+ auto mutex_impl = reinterpret_cast<TransactionDBMutexImpl*>(mutex.get());
+
+ std::unique_lock<std::mutex> lock(mutex_impl->mutex_, std::adopt_lock);
+ cv_.wait(lock);
+
+ // Make sure unique_lock doesn't unlock mutex when it destructs
+ lock.release();
+
+ return Status::OK();
+}
+
+Status TransactionDBCondVarImpl::WaitFor(
+ std::shared_ptr<TransactionDBMutex> mutex, int64_t timeout_time) {
+ Status s;
+
+ auto mutex_impl = reinterpret_cast<TransactionDBMutexImpl*>(mutex.get());
+ std::unique_lock<std::mutex> lock(mutex_impl->mutex_, std::adopt_lock);
+
+ if (timeout_time < 0) {
+ // If timeout is negative, do not use a timeout
+ cv_.wait(lock);
+ } else {
+ auto duration = std::chrono::microseconds(timeout_time);
+ auto cv_status = cv_.wait_for(lock, duration);
+
+ // Check if the wait stopped due to timing out.
+ if (cv_status == std::cv_status::timeout) {
+ s = Status::TimedOut(Status::SubCode::kMutexTimeout);
+ }
+ }
+
+ // Make sure unique_lock doesn't unlock mutex when it destructs
+ lock.release();
+
+ // CV was signaled, or we spuriously woke up (but didn't time out)
+ return s;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/transaction_db_mutex_impl.h b/src/rocksdb/utilities/transactions/transaction_db_mutex_impl.h
new file mode 100644
index 000000000..fbee92832
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_db_mutex_impl.h
@@ -0,0 +1,26 @@
+// 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 "rocksdb/utilities/transaction_db_mutex.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class TransactionDBMutex;
+class TransactionDBCondVar;
+
+// Default implementation of TransactionDBMutexFactory. May be overridden
+// by TransactionDBOptions.custom_mutex_factory.
+class TransactionDBMutexFactoryImpl : public TransactionDBMutexFactory {
+ public:
+ std::shared_ptr<TransactionDBMutex> AllocateMutex() override;
+ std::shared_ptr<TransactionDBCondVar> AllocateCondVar() override;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/transaction_lock_mgr.cc b/src/rocksdb/utilities/transactions/transaction_lock_mgr.cc
new file mode 100644
index 000000000..82b614033
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_lock_mgr.cc
@@ -0,0 +1,745 @@
+// 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_lock_mgr.h"
+
+#include <cinttypes>
+
+#include <algorithm>
+#include <condition_variable>
+#include <functional>
+#include <mutex>
+#include <string>
+#include <vector>
+
+#include "monitoring/perf_context_imp.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/utilities/transaction_db_mutex.h"
+#include "test_util/sync_point.h"
+#include "util/cast_util.h"
+#include "util/hash.h"
+#include "util/thread_local.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct LockInfo {
+ bool exclusive;
+ autovector<TransactionID> txn_ids;
+
+ // Transaction locks are not valid after this time in us
+ uint64_t expiration_time;
+
+ LockInfo(TransactionID id, uint64_t time, bool ex)
+ : exclusive(ex), expiration_time(time) {
+ txn_ids.push_back(id);
+ }
+ LockInfo(const LockInfo& lock_info)
+ : exclusive(lock_info.exclusive),
+ txn_ids(lock_info.txn_ids),
+ expiration_time(lock_info.expiration_time) {}
+};
+
+struct LockMapStripe {
+ explicit LockMapStripe(std::shared_ptr<TransactionDBMutexFactory> factory) {
+ stripe_mutex = factory->AllocateMutex();
+ stripe_cv = factory->AllocateCondVar();
+ assert(stripe_mutex);
+ assert(stripe_cv);
+ }
+
+ // Mutex must be held before modifying keys map
+ std::shared_ptr<TransactionDBMutex> stripe_mutex;
+
+ // Condition Variable per stripe for waiting on a lock
+ std::shared_ptr<TransactionDBCondVar> stripe_cv;
+
+ // Locked keys mapped to the info about the transactions that locked them.
+ // TODO(agiardullo): Explore performance of other data structures.
+ std::unordered_map<std::string, LockInfo> keys;
+};
+
+// Map of #num_stripes LockMapStripes
+struct LockMap {
+ explicit LockMap(size_t num_stripes,
+ std::shared_ptr<TransactionDBMutexFactory> factory)
+ : num_stripes_(num_stripes) {
+ lock_map_stripes_.reserve(num_stripes);
+ for (size_t i = 0; i < num_stripes; i++) {
+ LockMapStripe* stripe = new LockMapStripe(factory);
+ lock_map_stripes_.push_back(stripe);
+ }
+ }
+
+ ~LockMap() {
+ for (auto stripe : lock_map_stripes_) {
+ delete stripe;
+ }
+ }
+
+ // Number of sepearate LockMapStripes to create, each with their own Mutex
+ const size_t num_stripes_;
+
+ // Count of keys that are currently locked in this column family.
+ // (Only maintained if TransactionLockMgr::max_num_locks_ is positive.)
+ std::atomic<int64_t> lock_cnt{0};
+
+ std::vector<LockMapStripe*> lock_map_stripes_;
+
+ size_t GetStripe(const std::string& key) const;
+};
+
+void DeadlockInfoBuffer::AddNewPath(DeadlockPath path) {
+ std::lock_guard<std::mutex> lock(paths_buffer_mutex_);
+
+ if (paths_buffer_.empty()) {
+ return;
+ }
+
+ paths_buffer_[buffer_idx_] = std::move(path);
+ buffer_idx_ = (buffer_idx_ + 1) % paths_buffer_.size();
+}
+
+void DeadlockInfoBuffer::Resize(uint32_t target_size) {
+ std::lock_guard<std::mutex> lock(paths_buffer_mutex_);
+
+ paths_buffer_ = Normalize();
+
+ // Drop the deadlocks that will no longer be needed ater the normalize
+ if (target_size < paths_buffer_.size()) {
+ paths_buffer_.erase(
+ paths_buffer_.begin(),
+ paths_buffer_.begin() + (paths_buffer_.size() - target_size));
+ buffer_idx_ = 0;
+ }
+ // Resize the buffer to the target size and restore the buffer's idx
+ else {
+ auto prev_size = paths_buffer_.size();
+ paths_buffer_.resize(target_size);
+ buffer_idx_ = (uint32_t)prev_size;
+ }
+}
+
+std::vector<DeadlockPath> DeadlockInfoBuffer::Normalize() {
+ auto working = paths_buffer_;
+
+ if (working.empty()) {
+ return working;
+ }
+
+ // Next write occurs at a nonexistent path's slot
+ if (paths_buffer_[buffer_idx_].empty()) {
+ working.resize(buffer_idx_);
+ } else {
+ std::rotate(working.begin(), working.begin() + buffer_idx_, working.end());
+ }
+
+ return working;
+}
+
+std::vector<DeadlockPath> DeadlockInfoBuffer::PrepareBuffer() {
+ std::lock_guard<std::mutex> lock(paths_buffer_mutex_);
+
+ // Reversing the normalized vector returns the latest deadlocks first
+ auto working = Normalize();
+ std::reverse(working.begin(), working.end());
+
+ return working;
+}
+
+namespace {
+void UnrefLockMapsCache(void* ptr) {
+ // Called when a thread exits or a ThreadLocalPtr gets destroyed.
+ auto lock_maps_cache =
+ static_cast<std::unordered_map<uint32_t, std::shared_ptr<LockMap>>*>(ptr);
+ delete lock_maps_cache;
+}
+} // anonymous namespace
+
+TransactionLockMgr::TransactionLockMgr(
+ TransactionDB* txn_db, size_t default_num_stripes, int64_t max_num_locks,
+ uint32_t max_num_deadlocks,
+ std::shared_ptr<TransactionDBMutexFactory> mutex_factory)
+ : txn_db_impl_(nullptr),
+ default_num_stripes_(default_num_stripes),
+ max_num_locks_(max_num_locks),
+ lock_maps_cache_(new ThreadLocalPtr(&UnrefLockMapsCache)),
+ dlock_buffer_(max_num_deadlocks),
+ mutex_factory_(mutex_factory) {
+ assert(txn_db);
+ txn_db_impl_ =
+ static_cast_with_check<PessimisticTransactionDB, TransactionDB>(txn_db);
+}
+
+TransactionLockMgr::~TransactionLockMgr() {}
+
+size_t LockMap::GetStripe(const std::string& key) const {
+ assert(num_stripes_ > 0);
+ return fastrange64(GetSliceNPHash64(key), num_stripes_);
+}
+
+void TransactionLockMgr::AddColumnFamily(uint32_t column_family_id) {
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ if (lock_maps_.find(column_family_id) == lock_maps_.end()) {
+ lock_maps_.emplace(column_family_id,
+ std::make_shared<LockMap>(default_num_stripes_, mutex_factory_));
+ } else {
+ // column_family already exists in lock map
+ assert(false);
+ }
+}
+
+void TransactionLockMgr::RemoveColumnFamily(uint32_t column_family_id) {
+ // Remove lock_map for this column family. Since the lock map is stored
+ // as a shared ptr, concurrent transactions can still keep using it
+ // until they release their references to it.
+ {
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ auto lock_maps_iter = lock_maps_.find(column_family_id);
+ assert(lock_maps_iter != lock_maps_.end());
+
+ lock_maps_.erase(lock_maps_iter);
+ } // lock_map_mutex_
+
+ // Clear all thread-local caches
+ autovector<void*> local_caches;
+ lock_maps_cache_->Scrape(&local_caches, nullptr);
+ for (auto cache : local_caches) {
+ delete static_cast<LockMaps*>(cache);
+ }
+}
+
+// Look up the LockMap std::shared_ptr for a given column_family_id.
+// Note: The LockMap is only valid as long as the caller is still holding on
+// to the returned std::shared_ptr.
+std::shared_ptr<LockMap> TransactionLockMgr::GetLockMap(
+ uint32_t column_family_id) {
+ // First check thread-local cache
+ if (lock_maps_cache_->Get() == nullptr) {
+ lock_maps_cache_->Reset(new LockMaps());
+ }
+
+ auto lock_maps_cache = static_cast<LockMaps*>(lock_maps_cache_->Get());
+
+ auto lock_map_iter = lock_maps_cache->find(column_family_id);
+ if (lock_map_iter != lock_maps_cache->end()) {
+ // Found lock map for this column family.
+ return lock_map_iter->second;
+ }
+
+ // Not found in local cache, grab mutex and check shared LockMaps
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ lock_map_iter = lock_maps_.find(column_family_id);
+ if (lock_map_iter == lock_maps_.end()) {
+ return std::shared_ptr<LockMap>(nullptr);
+ } else {
+ // Found lock map. Store in thread-local cache and return.
+ std::shared_ptr<LockMap>& lock_map = lock_map_iter->second;
+ lock_maps_cache->insert({column_family_id, lock_map});
+
+ return lock_map;
+ }
+}
+
+// Returns true if this lock has expired and can be acquired by another
+// transaction.
+// If false, sets *expire_time to the expiration time of the lock according
+// to Env->GetMicros() or 0 if no expiration.
+bool TransactionLockMgr::IsLockExpired(TransactionID txn_id,
+ const LockInfo& lock_info, Env* env,
+ uint64_t* expire_time) {
+ auto now = env->NowMicros();
+
+ bool expired =
+ (lock_info.expiration_time > 0 && lock_info.expiration_time <= now);
+
+ if (!expired && lock_info.expiration_time > 0) {
+ // return how many microseconds until lock will be expired
+ *expire_time = lock_info.expiration_time;
+ } else {
+ for (auto id : lock_info.txn_ids) {
+ if (txn_id == id) {
+ continue;
+ }
+
+ bool success = txn_db_impl_->TryStealingExpiredTransactionLocks(id);
+ if (!success) {
+ expired = false;
+ break;
+ }
+ *expire_time = 0;
+ }
+ }
+
+ return expired;
+}
+
+Status TransactionLockMgr::TryLock(PessimisticTransaction* txn,
+ uint32_t column_family_id,
+ const std::string& key, Env* env,
+ bool exclusive) {
+ // Lookup lock map for this column family id
+ std::shared_ptr<LockMap> lock_map_ptr = GetLockMap(column_family_id);
+ LockMap* lock_map = lock_map_ptr.get();
+ if (lock_map == nullptr) {
+ char msg[255];
+ snprintf(msg, sizeof(msg), "Column family id not found: %" PRIu32,
+ column_family_id);
+
+ return Status::InvalidArgument(msg);
+ }
+
+ // Need to lock the mutex for the stripe that this key hashes to
+ size_t stripe_num = lock_map->GetStripe(key);
+ assert(lock_map->lock_map_stripes_.size() > stripe_num);
+ LockMapStripe* stripe = lock_map->lock_map_stripes_.at(stripe_num);
+
+ LockInfo lock_info(txn->GetID(), txn->GetExpirationTime(), exclusive);
+ int64_t timeout = txn->GetLockTimeout();
+
+ return AcquireWithTimeout(txn, lock_map, stripe, column_family_id, key, env,
+ timeout, std::move(lock_info));
+}
+
+// Helper function for TryLock().
+Status TransactionLockMgr::AcquireWithTimeout(
+ PessimisticTransaction* txn, LockMap* lock_map, LockMapStripe* stripe,
+ uint32_t column_family_id, const std::string& key, Env* env,
+ int64_t timeout, LockInfo&& lock_info) {
+ Status result;
+ uint64_t end_time = 0;
+
+ if (timeout > 0) {
+ uint64_t start_time = env->NowMicros();
+ end_time = start_time + timeout;
+ }
+
+ if (timeout < 0) {
+ // If timeout is negative, we wait indefinitely to acquire the lock
+ result = stripe->stripe_mutex->Lock();
+ } else {
+ result = stripe->stripe_mutex->TryLockFor(timeout);
+ }
+
+ if (!result.ok()) {
+ // failed to acquire mutex
+ return result;
+ }
+
+ // Acquire lock if we are able to
+ uint64_t expire_time_hint = 0;
+ autovector<TransactionID> wait_ids;
+ result = AcquireLocked(lock_map, stripe, key, env, std::move(lock_info),
+ &expire_time_hint, &wait_ids);
+
+ if (!result.ok() && timeout != 0) {
+ PERF_TIMER_GUARD(key_lock_wait_time);
+ PERF_COUNTER_ADD(key_lock_wait_count, 1);
+ // If we weren't able to acquire the lock, we will keep retrying as long
+ // as the timeout allows.
+ bool timed_out = false;
+ do {
+ // Decide how long to wait
+ int64_t cv_end_time = -1;
+
+ // Check if held lock's expiration time is sooner than our timeout
+ if (expire_time_hint > 0 &&
+ (timeout < 0 || (timeout > 0 && expire_time_hint < end_time))) {
+ // expiration time is sooner than our timeout
+ cv_end_time = expire_time_hint;
+ } else if (timeout >= 0) {
+ cv_end_time = end_time;
+ }
+
+ assert(result.IsBusy() || wait_ids.size() != 0);
+
+ // We are dependent on a transaction to finish, so perform deadlock
+ // detection.
+ if (wait_ids.size() != 0) {
+ if (txn->IsDeadlockDetect()) {
+ if (IncrementWaiters(txn, wait_ids, key, column_family_id,
+ lock_info.exclusive, env)) {
+ result = Status::Busy(Status::SubCode::kDeadlock);
+ stripe->stripe_mutex->UnLock();
+ return result;
+ }
+ }
+ txn->SetWaitingTxn(wait_ids, column_family_id, &key);
+ }
+
+ TEST_SYNC_POINT("TransactionLockMgr::AcquireWithTimeout:WaitingTxn");
+ if (cv_end_time < 0) {
+ // Wait indefinitely
+ result = stripe->stripe_cv->Wait(stripe->stripe_mutex);
+ } else {
+ uint64_t now = env->NowMicros();
+ if (static_cast<uint64_t>(cv_end_time) > now) {
+ result = stripe->stripe_cv->WaitFor(stripe->stripe_mutex,
+ cv_end_time - now);
+ }
+ }
+
+ if (wait_ids.size() != 0) {
+ txn->ClearWaitingTxn();
+ if (txn->IsDeadlockDetect()) {
+ DecrementWaiters(txn, wait_ids);
+ }
+ }
+
+ if (result.IsTimedOut()) {
+ timed_out = true;
+ // Even though we timed out, we will still make one more attempt to
+ // acquire lock below (it is possible the lock expired and we
+ // were never signaled).
+ }
+
+ if (result.ok() || result.IsTimedOut()) {
+ result = AcquireLocked(lock_map, stripe, key, env, std::move(lock_info),
+ &expire_time_hint, &wait_ids);
+ }
+ } while (!result.ok() && !timed_out);
+ }
+
+ stripe->stripe_mutex->UnLock();
+
+ return result;
+}
+
+void TransactionLockMgr::DecrementWaiters(
+ const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids) {
+ std::lock_guard<std::mutex> lock(wait_txn_map_mutex_);
+ DecrementWaitersImpl(txn, wait_ids);
+}
+
+void TransactionLockMgr::DecrementWaitersImpl(
+ const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids) {
+ auto id = txn->GetID();
+ assert(wait_txn_map_.Contains(id));
+ wait_txn_map_.Delete(id);
+
+ for (auto wait_id : wait_ids) {
+ rev_wait_txn_map_.Get(wait_id)--;
+ if (rev_wait_txn_map_.Get(wait_id) == 0) {
+ rev_wait_txn_map_.Delete(wait_id);
+ }
+ }
+}
+
+bool TransactionLockMgr::IncrementWaiters(
+ const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids, const std::string& key,
+ const uint32_t& cf_id, const bool& exclusive, Env* const env) {
+ auto id = txn->GetID();
+ std::vector<int> queue_parents(static_cast<size_t>(txn->GetDeadlockDetectDepth()));
+ std::vector<TransactionID> queue_values(static_cast<size_t>(txn->GetDeadlockDetectDepth()));
+ std::lock_guard<std::mutex> lock(wait_txn_map_mutex_);
+ assert(!wait_txn_map_.Contains(id));
+
+ wait_txn_map_.Insert(id, {wait_ids, cf_id, exclusive, key});
+
+ for (auto wait_id : wait_ids) {
+ if (rev_wait_txn_map_.Contains(wait_id)) {
+ rev_wait_txn_map_.Get(wait_id)++;
+ } else {
+ rev_wait_txn_map_.Insert(wait_id, 1);
+ }
+ }
+
+ // No deadlock if nobody is waiting on self.
+ if (!rev_wait_txn_map_.Contains(id)) {
+ return false;
+ }
+
+ const auto* next_ids = &wait_ids;
+ int parent = -1;
+ int64_t deadlock_time = 0;
+ for (int tail = 0, head = 0; head < txn->GetDeadlockDetectDepth(); head++) {
+ int i = 0;
+ if (next_ids) {
+ for (; i < static_cast<int>(next_ids->size()) &&
+ tail + i < txn->GetDeadlockDetectDepth();
+ i++) {
+ queue_values[tail + i] = (*next_ids)[i];
+ queue_parents[tail + i] = parent;
+ }
+ tail += i;
+ }
+
+ // No more items in the list, meaning no deadlock.
+ if (tail == head) {
+ return false;
+ }
+
+ auto next = queue_values[head];
+ if (next == id) {
+ std::vector<DeadlockInfo> path;
+ while (head != -1) {
+ assert(wait_txn_map_.Contains(queue_values[head]));
+
+ auto extracted_info = wait_txn_map_.Get(queue_values[head]);
+ path.push_back({queue_values[head], extracted_info.m_cf_id,
+ extracted_info.m_exclusive,
+ extracted_info.m_waiting_key});
+ head = queue_parents[head];
+ }
+ env->GetCurrentTime(&deadlock_time);
+ std::reverse(path.begin(), path.end());
+ dlock_buffer_.AddNewPath(DeadlockPath(path, deadlock_time));
+ deadlock_time = 0;
+ DecrementWaitersImpl(txn, wait_ids);
+ return true;
+ } else if (!wait_txn_map_.Contains(next)) {
+ next_ids = nullptr;
+ continue;
+ } else {
+ parent = head;
+ next_ids = &(wait_txn_map_.Get(next).m_neighbors);
+ }
+ }
+
+ // Wait cycle too big, just assume deadlock.
+ env->GetCurrentTime(&deadlock_time);
+ dlock_buffer_.AddNewPath(DeadlockPath(deadlock_time, true));
+ DecrementWaitersImpl(txn, wait_ids);
+ return true;
+}
+
+// Try to lock this key after we have acquired the mutex.
+// Sets *expire_time to the expiration time in microseconds
+// or 0 if no expiration.
+// REQUIRED: Stripe mutex must be held.
+Status TransactionLockMgr::AcquireLocked(LockMap* lock_map,
+ LockMapStripe* stripe,
+ const std::string& key, Env* env,
+ LockInfo&& txn_lock_info,
+ uint64_t* expire_time,
+ autovector<TransactionID>* txn_ids) {
+ assert(txn_lock_info.txn_ids.size() == 1);
+
+ Status result;
+ // Check if this key is already locked
+ auto stripe_iter = stripe->keys.find(key);
+ if (stripe_iter != stripe->keys.end()) {
+ // Lock already held
+ LockInfo& lock_info = stripe_iter->second;
+ assert(lock_info.txn_ids.size() == 1 || !lock_info.exclusive);
+
+ if (lock_info.exclusive || txn_lock_info.exclusive) {
+ if (lock_info.txn_ids.size() == 1 &&
+ lock_info.txn_ids[0] == txn_lock_info.txn_ids[0]) {
+ // The list contains one txn and we're it, so just take it.
+ lock_info.exclusive = txn_lock_info.exclusive;
+ lock_info.expiration_time = txn_lock_info.expiration_time;
+ } else {
+ // Check if it's expired. Skips over txn_lock_info.txn_ids[0] in case
+ // it's there for a shared lock with multiple holders which was not
+ // caught in the first case.
+ if (IsLockExpired(txn_lock_info.txn_ids[0], lock_info, env,
+ expire_time)) {
+ // lock is expired, can steal it
+ lock_info.txn_ids = txn_lock_info.txn_ids;
+ lock_info.exclusive = txn_lock_info.exclusive;
+ lock_info.expiration_time = txn_lock_info.expiration_time;
+ // lock_cnt does not change
+ } else {
+ result = Status::TimedOut(Status::SubCode::kLockTimeout);
+ *txn_ids = lock_info.txn_ids;
+ }
+ }
+ } else {
+ // We are requesting shared access to a shared lock, so just grant it.
+ lock_info.txn_ids.push_back(txn_lock_info.txn_ids[0]);
+ // Using std::max means that expiration time never goes down even when
+ // a transaction is removed from the list. The correct solution would be
+ // to track expiry for every transaction, but this would also work for
+ // now.
+ lock_info.expiration_time =
+ std::max(lock_info.expiration_time, txn_lock_info.expiration_time);
+ }
+ } else { // Lock not held.
+ // Check lock limit
+ if (max_num_locks_ > 0 &&
+ lock_map->lock_cnt.load(std::memory_order_acquire) >= max_num_locks_) {
+ result = Status::Busy(Status::SubCode::kLockLimit);
+ } else {
+ // acquire lock
+ stripe->keys.emplace(key, std::move(txn_lock_info));
+
+ // Maintain lock count if there is a limit on the number of locks
+ if (max_num_locks_) {
+ lock_map->lock_cnt++;
+ }
+ }
+ }
+
+ return result;
+}
+
+void TransactionLockMgr::UnLockKey(const PessimisticTransaction* txn,
+ const std::string& key,
+ LockMapStripe* stripe, LockMap* lock_map,
+ Env* env) {
+#ifdef NDEBUG
+ (void)env;
+#endif
+ TransactionID txn_id = txn->GetID();
+
+ auto stripe_iter = stripe->keys.find(key);
+ if (stripe_iter != stripe->keys.end()) {
+ auto& txns = stripe_iter->second.txn_ids;
+ auto txn_it = std::find(txns.begin(), txns.end(), txn_id);
+ // Found the key we locked. unlock it.
+ if (txn_it != txns.end()) {
+ if (txns.size() == 1) {
+ stripe->keys.erase(stripe_iter);
+ } else {
+ auto last_it = txns.end() - 1;
+ if (txn_it != last_it) {
+ *txn_it = *last_it;
+ }
+ txns.pop_back();
+ }
+
+ if (max_num_locks_ > 0) {
+ // Maintain lock count if there is a limit on the number of locks.
+ assert(lock_map->lock_cnt.load(std::memory_order_relaxed) > 0);
+ lock_map->lock_cnt--;
+ }
+ }
+ } else {
+ // This key is either not locked or locked by someone else. This should
+ // only happen if the unlocking transaction has expired.
+ assert(txn->GetExpirationTime() > 0 &&
+ txn->GetExpirationTime() < env->NowMicros());
+ }
+}
+
+void TransactionLockMgr::UnLock(PessimisticTransaction* txn,
+ uint32_t column_family_id,
+ const std::string& key, Env* env) {
+ std::shared_ptr<LockMap> lock_map_ptr = GetLockMap(column_family_id);
+ LockMap* lock_map = lock_map_ptr.get();
+ if (lock_map == nullptr) {
+ // Column Family must have been dropped.
+ return;
+ }
+
+ // Lock the mutex for the stripe that this key hashes to
+ size_t stripe_num = lock_map->GetStripe(key);
+ assert(lock_map->lock_map_stripes_.size() > stripe_num);
+ LockMapStripe* stripe = lock_map->lock_map_stripes_.at(stripe_num);
+
+ stripe->stripe_mutex->Lock();
+ UnLockKey(txn, key, stripe, lock_map, env);
+ stripe->stripe_mutex->UnLock();
+
+ // Signal waiting threads to retry locking
+ stripe->stripe_cv->NotifyAll();
+}
+
+void TransactionLockMgr::UnLock(const PessimisticTransaction* txn,
+ const TransactionKeyMap* key_map, Env* env) {
+ for (auto& key_map_iter : *key_map) {
+ uint32_t column_family_id = key_map_iter.first;
+ auto& keys = key_map_iter.second;
+
+ std::shared_ptr<LockMap> lock_map_ptr = GetLockMap(column_family_id);
+ LockMap* lock_map = lock_map_ptr.get();
+
+ if (lock_map == nullptr) {
+ // Column Family must have been dropped.
+ return;
+ }
+
+ // Bucket keys by lock_map_ stripe
+ std::unordered_map<size_t, std::vector<const std::string*>> keys_by_stripe(
+ std::max(keys.size(), lock_map->num_stripes_));
+
+ for (auto& key_iter : keys) {
+ const std::string& key = key_iter.first;
+
+ size_t stripe_num = lock_map->GetStripe(key);
+ keys_by_stripe[stripe_num].push_back(&key);
+ }
+
+ // For each stripe, grab the stripe mutex and unlock all keys in this stripe
+ for (auto& stripe_iter : keys_by_stripe) {
+ size_t stripe_num = stripe_iter.first;
+ auto& stripe_keys = stripe_iter.second;
+
+ assert(lock_map->lock_map_stripes_.size() > stripe_num);
+ LockMapStripe* stripe = lock_map->lock_map_stripes_.at(stripe_num);
+
+ stripe->stripe_mutex->Lock();
+
+ for (const std::string* key : stripe_keys) {
+ UnLockKey(txn, *key, stripe, lock_map, env);
+ }
+
+ stripe->stripe_mutex->UnLock();
+
+ // Signal waiting threads to retry locking
+ stripe->stripe_cv->NotifyAll();
+ }
+ }
+}
+
+TransactionLockMgr::LockStatusData TransactionLockMgr::GetLockStatusData() {
+ LockStatusData data;
+ // Lock order here is important. The correct order is lock_map_mutex_, then
+ // for every column family ID in ascending order lock every stripe in
+ // ascending order.
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ std::vector<uint32_t> cf_ids;
+ for (const auto& map : lock_maps_) {
+ cf_ids.push_back(map.first);
+ }
+ std::sort(cf_ids.begin(), cf_ids.end());
+
+ for (auto i : cf_ids) {
+ const auto& stripes = lock_maps_[i]->lock_map_stripes_;
+ // Iterate and lock all stripes in ascending order.
+ for (const auto& j : stripes) {
+ j->stripe_mutex->Lock();
+ for (const auto& it : j->keys) {
+ struct KeyLockInfo info;
+ info.exclusive = it.second.exclusive;
+ info.key = it.first;
+ for (const auto& id : it.second.txn_ids) {
+ info.ids.push_back(id);
+ }
+ data.insert({i, info});
+ }
+ }
+ }
+
+ // Unlock everything. Unlocking order is not important.
+ for (auto i : cf_ids) {
+ const auto& stripes = lock_maps_[i]->lock_map_stripes_;
+ for (const auto& j : stripes) {
+ j->stripe_mutex->UnLock();
+ }
+ }
+
+ return data;
+}
+std::vector<DeadlockPath> TransactionLockMgr::GetDeadlockInfoBuffer() {
+ return dlock_buffer_.PrepareBuffer();
+}
+
+void TransactionLockMgr::Resize(uint32_t target_size) {
+ dlock_buffer_.Resize(target_size);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/transaction_lock_mgr.h b/src/rocksdb/utilities/transactions/transaction_lock_mgr.h
new file mode 100644
index 000000000..b4fd85929
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_lock_mgr.h
@@ -0,0 +1,158 @@
+// 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 <chrono>
+#include <string>
+#include <unordered_map>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "monitoring/instrumented_mutex.h"
+#include "rocksdb/utilities/transaction.h"
+#include "util/autovector.h"
+#include "util/hash_map.h"
+#include "util/thread_local.h"
+#include "utilities/transactions/pessimistic_transaction.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class ColumnFamilyHandle;
+struct LockInfo;
+struct LockMap;
+struct LockMapStripe;
+
+struct DeadlockInfoBuffer {
+ private:
+ std::vector<DeadlockPath> paths_buffer_;
+ uint32_t buffer_idx_;
+ std::mutex paths_buffer_mutex_;
+ std::vector<DeadlockPath> Normalize();
+
+ public:
+ explicit DeadlockInfoBuffer(uint32_t n_latest_dlocks)
+ : paths_buffer_(n_latest_dlocks), buffer_idx_(0) {}
+ void AddNewPath(DeadlockPath path);
+ void Resize(uint32_t target_size);
+ std::vector<DeadlockPath> PrepareBuffer();
+};
+
+struct TrackedTrxInfo {
+ autovector<TransactionID> m_neighbors;
+ uint32_t m_cf_id;
+ bool m_exclusive;
+ std::string m_waiting_key;
+};
+
+class Slice;
+class PessimisticTransactionDB;
+
+class TransactionLockMgr {
+ public:
+ TransactionLockMgr(TransactionDB* txn_db, size_t default_num_stripes,
+ int64_t max_num_locks, uint32_t max_num_deadlocks,
+ std::shared_ptr<TransactionDBMutexFactory> factory);
+ // No copying allowed
+ TransactionLockMgr(const TransactionLockMgr&) = delete;
+ void operator=(const TransactionLockMgr&) = delete;
+
+ ~TransactionLockMgr();
+
+ // Creates a new LockMap for this column family. Caller should guarantee
+ // that this column family does not already exist.
+ void AddColumnFamily(uint32_t column_family_id);
+
+ // Deletes the LockMap for this column family. Caller should guarantee that
+ // this column family is no longer in use.
+ void RemoveColumnFamily(uint32_t column_family_id);
+
+ // Attempt to lock key. If OK status is returned, the caller is responsible
+ // for calling UnLock() on this key.
+ Status TryLock(PessimisticTransaction* txn, uint32_t column_family_id,
+ const std::string& key, Env* env, bool exclusive);
+
+ // Unlock a key locked by TryLock(). txn must be the same Transaction that
+ // locked this key.
+ void UnLock(const PessimisticTransaction* txn, const TransactionKeyMap* keys,
+ Env* env);
+ void UnLock(PessimisticTransaction* txn, uint32_t column_family_id,
+ const std::string& key, Env* env);
+
+ using LockStatusData = std::unordered_multimap<uint32_t, KeyLockInfo>;
+ LockStatusData GetLockStatusData();
+ std::vector<DeadlockPath> GetDeadlockInfoBuffer();
+ void Resize(uint32_t);
+
+ private:
+ PessimisticTransactionDB* txn_db_impl_;
+
+ // Default number of lock map stripes per column family
+ const size_t default_num_stripes_;
+
+ // Limit on number of keys locked per column family
+ const int64_t max_num_locks_;
+
+ // The following lock order must be satisfied in order to avoid deadlocking
+ // ourselves.
+ // - lock_map_mutex_
+ // - stripe mutexes in ascending cf id, ascending stripe order
+ // - wait_txn_map_mutex_
+ //
+ // Must be held when accessing/modifying lock_maps_.
+ InstrumentedMutex lock_map_mutex_;
+
+ // Map of ColumnFamilyId to locked key info
+ using LockMaps = std::unordered_map<uint32_t, std::shared_ptr<LockMap>>;
+ LockMaps lock_maps_;
+
+ // Thread-local cache of entries in lock_maps_. This is an optimization
+ // to avoid acquiring a mutex in order to look up a LockMap
+ std::unique_ptr<ThreadLocalPtr> lock_maps_cache_;
+
+ // Must be held when modifying wait_txn_map_ and rev_wait_txn_map_.
+ std::mutex wait_txn_map_mutex_;
+
+ // Maps from waitee -> number of waiters.
+ HashMap<TransactionID, int> rev_wait_txn_map_;
+ // Maps from waiter -> waitee.
+ HashMap<TransactionID, TrackedTrxInfo> wait_txn_map_;
+ DeadlockInfoBuffer dlock_buffer_;
+
+ // Used to allocate mutexes/condvars to use when locking keys
+ std::shared_ptr<TransactionDBMutexFactory> mutex_factory_;
+
+ bool IsLockExpired(TransactionID txn_id, const LockInfo& lock_info, Env* env,
+ uint64_t* wait_time);
+
+ std::shared_ptr<LockMap> GetLockMap(uint32_t column_family_id);
+
+ Status AcquireWithTimeout(PessimisticTransaction* txn, LockMap* lock_map,
+ LockMapStripe* stripe, uint32_t column_family_id,
+ const std::string& key, Env* env, int64_t timeout,
+ LockInfo&& lock_info);
+
+ Status AcquireLocked(LockMap* lock_map, LockMapStripe* stripe,
+ const std::string& key, Env* env,
+ LockInfo&& lock_info, uint64_t* wait_time,
+ autovector<TransactionID>* txn_ids);
+
+ void UnLockKey(const PessimisticTransaction* txn, const std::string& key,
+ LockMapStripe* stripe, LockMap* lock_map, Env* env);
+
+ bool IncrementWaiters(const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids,
+ const std::string& key, const uint32_t& cf_id,
+ const bool& exclusive, Env* const env);
+ void DecrementWaiters(const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids);
+ void DecrementWaitersImpl(const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids);
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/transaction_test.cc b/src/rocksdb/utilities/transactions/transaction_test.cc
new file mode 100644
index 000000000..bdc2609f5
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_test.cc
@@ -0,0 +1,6224 @@
+// 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_test.h"
+
+#include <algorithm>
+#include <functional>
+#include <string>
+#include <thread>
+
+#include "db/db_impl/db_impl.h"
+#include "rocksdb/db.h"
+#include "rocksdb/options.h"
+#include "rocksdb/perf_context.h"
+#include "rocksdb/utilities/transaction.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "table/mock_table.h"
+#include "test_util/fault_injection_test_env.h"
+#include "test_util/sync_point.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "test_util/transaction_test_util.h"
+#include "util/random.h"
+#include "util/string_util.h"
+#include "utilities/merge_operators.h"
+#include "utilities/merge_operators/string_append/stringappend.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+
+#include "port/port.h"
+
+using std::string;
+
+namespace ROCKSDB_NAMESPACE {
+
+INSTANTIATE_TEST_CASE_P(
+ DBAsBaseDB, TransactionTest,
+ ::testing::Values(
+ std::make_tuple(false, false, WRITE_COMMITTED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_COMMITTED, kOrderedWrite),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite),
+ std::make_tuple(false, false, WRITE_UNPREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_UNPREPARED, kOrderedWrite)));
+INSTANTIATE_TEST_CASE_P(
+ DBAsBaseDB, TransactionStressTest,
+ ::testing::Values(
+ std::make_tuple(false, false, WRITE_COMMITTED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_COMMITTED, kOrderedWrite),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite),
+ std::make_tuple(false, false, WRITE_UNPREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_UNPREPARED, kOrderedWrite)));
+INSTANTIATE_TEST_CASE_P(
+ StackableDBAsBaseDB, TransactionTest,
+ ::testing::Values(
+ std::make_tuple(true, true, WRITE_COMMITTED, kOrderedWrite),
+ std::make_tuple(true, true, WRITE_PREPARED, kOrderedWrite),
+ std::make_tuple(true, true, WRITE_UNPREPARED, kOrderedWrite)));
+
+// MySQLStyleTransactionTest takes far too long for valgrind to run.
+#ifndef ROCKSDB_VALGRIND_RUN
+INSTANTIATE_TEST_CASE_P(
+ MySQLStyleTransactionTest, MySQLStyleTransactionTest,
+ ::testing::Values(
+ std::make_tuple(false, false, WRITE_COMMITTED, kOrderedWrite, false),
+ std::make_tuple(false, true, WRITE_COMMITTED, kOrderedWrite, false),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, false),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, true),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, false),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, true),
+ std::make_tuple(false, false, WRITE_UNPREPARED, kOrderedWrite, false),
+ std::make_tuple(false, false, WRITE_UNPREPARED, kOrderedWrite, true),
+ std::make_tuple(false, true, WRITE_UNPREPARED, kOrderedWrite, false),
+ std::make_tuple(false, true, WRITE_UNPREPARED, kOrderedWrite, true),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, false),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, true)));
+#endif // ROCKSDB_VALGRIND_RUN
+
+TEST_P(TransactionTest, DoubleEmptyWrite) {
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = false;
+
+ WriteBatch batch;
+
+ ASSERT_OK(db->Write(write_options, &batch));
+ ASSERT_OK(db->Write(write_options, &batch));
+
+ // Also test committing empty transactions in 2PC
+ TransactionOptions txn_options;
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Prepare());
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+
+ // Also test that it works during recovery
+ txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid2"));
+ txn0->Put(Slice("foo0"), Slice("bar0a"));
+ ASSERT_OK(txn0->Prepare());
+ delete txn0;
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ASSERT_OK(ReOpenNoDelete());
+ assert(db != nullptr);
+ txn0 = db->GetTransactionByName("xid2");
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+}
+
+TEST_P(TransactionTest, SuccessTest) {
+ ASSERT_OK(db->ResetStats());
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+
+ ASSERT_OK(db->Put(write_options, Slice("foo"), Slice("bar")));
+ ASSERT_OK(db->Put(write_options, Slice("foo2"), Slice("bar")));
+
+ Transaction* txn = db->BeginTransaction(write_options, TransactionOptions());
+ ASSERT_TRUE(txn);
+
+ ASSERT_EQ(0, txn->GetNumPuts());
+ ASSERT_LE(0, txn->GetID());
+
+ ASSERT_OK(txn->GetForUpdate(read_options, "foo", &value));
+ ASSERT_EQ(value, "bar");
+
+ ASSERT_OK(txn->Put(Slice("foo"), Slice("bar2")));
+
+ ASSERT_EQ(1, txn->GetNumPuts());
+
+ ASSERT_OK(txn->GetForUpdate(read_options, "foo", &value));
+ ASSERT_EQ(value, "bar2");
+
+ ASSERT_OK(txn->Commit());
+
+ ASSERT_OK(db->Get(read_options, "foo", &value));
+ ASSERT_EQ(value, "bar2");
+
+ delete txn;
+}
+
+// The test clarifies the contract of do_validate and assume_tracked
+// in GetForUpdate and Put/Merge/Delete
+TEST_P(TransactionTest, AssumeExclusiveTracked) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+ TransactionOptions txn_options;
+ txn_options.lock_timeout = 1;
+ const bool EXCLUSIVE = true;
+ const bool DO_VALIDATE = true;
+ const bool ASSUME_LOCKED = true;
+
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+ txn->SetSnapshot();
+
+ // commit a value after the snapshot is taken
+ ASSERT_OK(db->Put(write_options, Slice("foo"), Slice("bar")));
+
+ // By default write should fail to the commit after our snapshot
+ s = txn->GetForUpdate(read_options, "foo", &value, EXCLUSIVE);
+ ASSERT_TRUE(s.IsBusy());
+ // But the user could direct the db to skip validating the snapshot. The read
+ // value then should be the most recently committed
+ ASSERT_OK(
+ txn->GetForUpdate(read_options, "foo", &value, EXCLUSIVE, !DO_VALIDATE));
+ ASSERT_EQ(value, "bar");
+
+ // Although ValidateSnapshot is skipped the key must have still got locked
+ s = db->Put(write_options, Slice("foo"), Slice("bar"));
+ ASSERT_TRUE(s.IsTimedOut());
+
+ // By default the write operations should fail due to the commit after the
+ // snapshot
+ s = txn->Put(Slice("foo"), Slice("bar1"));
+ ASSERT_TRUE(s.IsBusy());
+ s = txn->Put(db->DefaultColumnFamily(), Slice("foo"), Slice("bar1"),
+ !ASSUME_LOCKED);
+ ASSERT_TRUE(s.IsBusy());
+ // But the user could direct the db that it already assumes exclusive lock on
+ // the key due to the previous GetForUpdate call.
+ ASSERT_OK(txn->Put(db->DefaultColumnFamily(), Slice("foo"), Slice("bar1"),
+ ASSUME_LOCKED));
+ ASSERT_OK(txn->Merge(db->DefaultColumnFamily(), Slice("foo"), Slice("bar2"),
+ ASSUME_LOCKED));
+ ASSERT_OK(
+ txn->Delete(db->DefaultColumnFamily(), Slice("foo"), ASSUME_LOCKED));
+ ASSERT_OK(txn->SingleDelete(db->DefaultColumnFamily(), Slice("foo"),
+ ASSUME_LOCKED));
+
+ txn->Rollback();
+ delete txn;
+}
+
+// This test clarifies the contract of ValidateSnapshot
+TEST_P(TransactionTest, ValidateSnapshotTest) {
+ for (bool with_flush : {true}) {
+ for (bool with_2pc : {true}) {
+ ASSERT_OK(ReOpen());
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+
+ assert(db != nullptr);
+ Transaction* txn1 =
+ db->BeginTransaction(write_options, TransactionOptions());
+ ASSERT_TRUE(txn1);
+ ASSERT_OK(txn1->Put(Slice("foo"), Slice("bar1")));
+ if (with_2pc) {
+ ASSERT_OK(txn1->SetName("xid1"));
+ ASSERT_OK(txn1->Prepare());
+ }
+
+ if (with_flush) {
+ auto db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ db_impl->TEST_FlushMemTable(true);
+ // Make sure the flushed memtable is not kept in memory
+ int max_memtable_in_history =
+ std::max(
+ options.max_write_buffer_number,
+ static_cast<int>(options.max_write_buffer_size_to_maintain) /
+ static_cast<int>(options.write_buffer_size)) +
+ 1;
+ for (int i = 0; i < max_memtable_in_history; i++) {
+ db->Put(write_options, Slice("key"), Slice("value"));
+ db_impl->TEST_FlushMemTable(true);
+ }
+ }
+
+ Transaction* txn2 =
+ db->BeginTransaction(write_options, TransactionOptions());
+ ASSERT_TRUE(txn2);
+ txn2->SetSnapshot();
+
+ ASSERT_OK(txn1->Commit());
+ delete txn1;
+
+ auto pes_txn2 = dynamic_cast<PessimisticTransaction*>(txn2);
+ // Test the simple case where the key is not tracked yet
+ auto trakced_seq = kMaxSequenceNumber;
+ auto s = pes_txn2->ValidateSnapshot(db->DefaultColumnFamily(), "foo",
+ &trakced_seq);
+ ASSERT_TRUE(s.IsBusy());
+ delete txn2;
+ }
+ }
+}
+
+TEST_P(TransactionTest, WaitingTxn) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ string value;
+ Status s;
+
+ txn_options.lock_timeout = 1;
+ s = db->Put(write_options, Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+
+ /* create second cf */
+ ColumnFamilyHandle* cfa;
+ ColumnFamilyOptions cf_options;
+ s = db->CreateColumnFamily(cf_options, "CFA", &cfa);
+ ASSERT_OK(s);
+ s = db->Put(write_options, cfa, Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ TransactionID id1 = txn1->GetID();
+ ASSERT_TRUE(txn1);
+ ASSERT_TRUE(txn2);
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "TransactionLockMgr::AcquireWithTimeout:WaitingTxn", [&](void* /*arg*/) {
+ std::string key;
+ uint32_t cf_id;
+ std::vector<TransactionID> wait = txn2->GetWaitingTxns(&cf_id, &key);
+ ASSERT_EQ(key, "foo");
+ ASSERT_EQ(wait.size(), 1);
+ ASSERT_EQ(wait[0], id1);
+ ASSERT_EQ(cf_id, 0U);
+ });
+
+ get_perf_context()->Reset();
+ // lock key in default cf
+ s = txn1->GetForUpdate(read_options, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+ ASSERT_EQ(get_perf_context()->key_lock_wait_count, 0);
+
+ // lock key in cfa
+ s = txn1->GetForUpdate(read_options, cfa, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+ ASSERT_EQ(get_perf_context()->key_lock_wait_count, 0);
+
+ auto lock_data = db->GetLockStatusData();
+ // Locked keys exist in both column family.
+ ASSERT_EQ(lock_data.size(), 2);
+
+ auto cf_iterator = lock_data.begin();
+
+ // The iterator points to an unordered_multimap
+ // thus the test can not assume any particular order.
+
+ // Column family is 1 or 0 (cfa).
+ if (cf_iterator->first != 1 && cf_iterator->first != 0) {
+ FAIL();
+ }
+ // The locked key is "foo" and is locked by txn1
+ ASSERT_EQ(cf_iterator->second.key, "foo");
+ ASSERT_EQ(cf_iterator->second.ids.size(), 1);
+ ASSERT_EQ(cf_iterator->second.ids[0], txn1->GetID());
+
+ cf_iterator++;
+
+ // Column family is 0 (default) or 1.
+ if (cf_iterator->first != 1 && cf_iterator->first != 0) {
+ FAIL();
+ }
+ // The locked key is "foo" and is locked by txn1
+ ASSERT_EQ(cf_iterator->second.key, "foo");
+ ASSERT_EQ(cf_iterator->second.ids.size(), 1);
+ ASSERT_EQ(cf_iterator->second.ids[0], txn1->GetID());
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ s = txn2->GetForUpdate(read_options, "foo", &value);
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+ ASSERT_EQ(get_perf_context()->key_lock_wait_count, 1);
+ ASSERT_GE(get_perf_context()->key_lock_wait_time, 0);
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ delete cfa;
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, SharedLocks) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ Status s;
+
+ txn_options.lock_timeout = 1;
+ s = db->Put(write_options, Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ Transaction* txn3 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+ ASSERT_TRUE(txn2);
+ ASSERT_TRUE(txn3);
+
+ // Test shared access between txns
+ s = txn1->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn3->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ auto lock_data = db->GetLockStatusData();
+ ASSERT_EQ(lock_data.size(), 1);
+
+ auto cf_iterator = lock_data.begin();
+ ASSERT_EQ(cf_iterator->second.key, "foo");
+
+ // We compare whether the set of txns locking this key is the same. To do
+ // this, we need to sort both vectors so that the comparison is done
+ // correctly.
+ std::vector<TransactionID> expected_txns = {txn1->GetID(), txn2->GetID(),
+ txn3->GetID()};
+ std::vector<TransactionID> lock_txns = cf_iterator->second.ids;
+ ASSERT_EQ(expected_txns, lock_txns);
+ ASSERT_FALSE(cf_iterator->second.exclusive);
+
+ txn1->Rollback();
+ txn2->Rollback();
+ txn3->Rollback();
+
+ // Test txn1 and txn2 sharing a lock and txn3 trying to obtain it.
+ s = txn1->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn3->GetForUpdate(read_options, "foo", nullptr);
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+
+ txn1->UndoGetForUpdate("foo");
+ s = txn3->GetForUpdate(read_options, "foo", nullptr);
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+
+ txn2->UndoGetForUpdate("foo");
+ s = txn3->GetForUpdate(read_options, "foo", nullptr);
+ ASSERT_OK(s);
+
+ txn1->Rollback();
+ txn2->Rollback();
+ txn3->Rollback();
+
+ // Test txn1 and txn2 sharing a lock and txn2 trying to upgrade lock.
+ s = txn1->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options, "foo", nullptr);
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+
+ txn1->UndoGetForUpdate("foo");
+ s = txn2->GetForUpdate(read_options, "foo", nullptr);
+ ASSERT_OK(s);
+
+ ASSERT_OK(txn1->Rollback());
+ ASSERT_OK(txn2->Rollback());
+
+ // Test txn1 trying to downgrade its lock.
+ s = txn1->GetForUpdate(read_options, "foo", nullptr, true /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+
+ // Should still fail after "downgrading".
+ s = txn1->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+
+ txn1->Rollback();
+ txn2->Rollback();
+
+ // Test txn1 holding an exclusive lock and txn2 trying to obtain shared
+ // access.
+ s = txn1->GetForUpdate(read_options, "foo", nullptr);
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+
+ txn1->UndoGetForUpdate("foo");
+ s = txn2->GetForUpdate(read_options, "foo", nullptr, false /* exclusive */);
+ ASSERT_OK(s);
+
+ delete txn1;
+ delete txn2;
+ delete txn3;
+}
+
+TEST_P(TransactionTest, DeadlockCycleShared) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+
+ txn_options.lock_timeout = 1000000;
+ txn_options.deadlock_detect = true;
+
+ // Set up a wait for chain like this:
+ //
+ // Tn -> T(n*2)
+ // Tn -> T(n*2 + 1)
+ //
+ // So we have:
+ // T1 -> T2 -> T4 ...
+ // | |> T5 ...
+ // |> T3 -> T6 ...
+ // |> T7 ...
+ // up to T31, then T[16 - 31] -> T1.
+ // Note that Tn holds lock on floor(n / 2).
+
+ std::vector<Transaction*> txns(31);
+
+ for (uint32_t i = 0; i < 31; i++) {
+ txns[i] = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txns[i]);
+ auto s = txns[i]->GetForUpdate(read_options, ToString((i + 1) / 2), nullptr,
+ false /* exclusive */);
+ ASSERT_OK(s);
+ }
+
+ std::atomic<uint32_t> checkpoints(0);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "TransactionLockMgr::AcquireWithTimeout:WaitingTxn",
+ [&](void* /*arg*/) { checkpoints.fetch_add(1); });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ // We want the leaf transactions to block and hold everyone back.
+ std::vector<port::Thread> threads;
+ for (uint32_t i = 0; i < 15; i++) {
+ std::function<void()> blocking_thread = [&, i] {
+ auto s = txns[i]->GetForUpdate(read_options, ToString(i + 1), nullptr,
+ true /* exclusive */);
+ ASSERT_OK(s);
+ txns[i]->Rollback();
+ delete txns[i];
+ };
+ threads.emplace_back(blocking_thread);
+ }
+
+ // Wait until all threads are waiting on each other.
+ while (checkpoints.load() != 15) {
+ /* sleep override */
+ std::this_thread::sleep_for(std::chrono::milliseconds(100));
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ // Complete the cycle T[16 - 31] -> T1
+ for (uint32_t i = 15; i < 31; i++) {
+ auto s =
+ txns[i]->GetForUpdate(read_options, "0", nullptr, true /* exclusive */);
+ ASSERT_TRUE(s.IsDeadlock());
+
+ // Calculate next buffer len, plateau at 5 when 5 records are inserted.
+ const uint32_t curr_dlock_buffer_len_ =
+ (i - 14 > kInitialMaxDeadlocks) ? kInitialMaxDeadlocks : (i - 14);
+
+ auto dlock_buffer = db->GetDeadlockInfoBuffer();
+ ASSERT_EQ(dlock_buffer.size(), curr_dlock_buffer_len_);
+ auto dlock_entry = dlock_buffer[0].path;
+ ASSERT_EQ(dlock_entry.size(), kInitialMaxDeadlocks);
+ int64_t pre_deadlock_time = dlock_buffer[0].deadlock_time;
+ int64_t cur_deadlock_time = 0;
+ for (auto const& dl_path_rec : dlock_buffer) {
+ cur_deadlock_time = dl_path_rec.deadlock_time;
+ ASSERT_NE(cur_deadlock_time, 0);
+ ASSERT_TRUE(cur_deadlock_time <= pre_deadlock_time);
+ pre_deadlock_time = cur_deadlock_time;
+ }
+
+ int64_t curr_waiting_key = 0;
+
+ // Offset of each txn id from the root of the shared dlock tree's txn id.
+ int64_t offset_root = dlock_entry[0].m_txn_id - 1;
+ // Offset of the final entry in the dlock path from the root's txn id.
+ TransactionID leaf_id =
+ dlock_entry[dlock_entry.size() - 1].m_txn_id - offset_root;
+
+ for (auto it = dlock_entry.rbegin(); it != dlock_entry.rend(); ++it) {
+ auto dl_node = *it;
+ ASSERT_EQ(dl_node.m_txn_id, offset_root + leaf_id);
+ ASSERT_EQ(dl_node.m_cf_id, 0U);
+ ASSERT_EQ(dl_node.m_waiting_key, ToString(curr_waiting_key));
+ ASSERT_EQ(dl_node.m_exclusive, true);
+
+ if (curr_waiting_key == 0) {
+ curr_waiting_key = leaf_id;
+ }
+ curr_waiting_key /= 2;
+ leaf_id /= 2;
+ }
+ }
+
+ // Rollback the leaf transaction.
+ for (uint32_t i = 15; i < 31; i++) {
+ txns[i]->Rollback();
+ delete txns[i];
+ }
+
+ for (auto& t : threads) {
+ t.join();
+ }
+
+ // Downsize the buffer and verify the 3 latest deadlocks are preserved.
+ auto dlock_buffer_before_resize = db->GetDeadlockInfoBuffer();
+ db->SetDeadlockInfoBufferSize(3);
+ auto dlock_buffer_after_resize = db->GetDeadlockInfoBuffer();
+ ASSERT_EQ(dlock_buffer_after_resize.size(), 3);
+
+ for (uint32_t i = 0; i < dlock_buffer_after_resize.size(); i++) {
+ for (uint32_t j = 0; j < dlock_buffer_after_resize[i].path.size(); j++) {
+ ASSERT_EQ(dlock_buffer_after_resize[i].path[j].m_txn_id,
+ dlock_buffer_before_resize[i].path[j].m_txn_id);
+ }
+ }
+
+ // Upsize the buffer and verify the 3 latest dealocks are preserved.
+ dlock_buffer_before_resize = db->GetDeadlockInfoBuffer();
+ db->SetDeadlockInfoBufferSize(5);
+ dlock_buffer_after_resize = db->GetDeadlockInfoBuffer();
+ ASSERT_EQ(dlock_buffer_after_resize.size(), 3);
+
+ for (uint32_t i = 0; i < dlock_buffer_before_resize.size(); i++) {
+ for (uint32_t j = 0; j < dlock_buffer_before_resize[i].path.size(); j++) {
+ ASSERT_EQ(dlock_buffer_after_resize[i].path[j].m_txn_id,
+ dlock_buffer_before_resize[i].path[j].m_txn_id);
+ }
+ }
+
+ // Downsize to 0 and verify the size is consistent.
+ dlock_buffer_before_resize = db->GetDeadlockInfoBuffer();
+ db->SetDeadlockInfoBufferSize(0);
+ dlock_buffer_after_resize = db->GetDeadlockInfoBuffer();
+ ASSERT_EQ(dlock_buffer_after_resize.size(), 0);
+
+ // Upsize from 0 to verify the size is persistent.
+ dlock_buffer_before_resize = db->GetDeadlockInfoBuffer();
+ db->SetDeadlockInfoBufferSize(3);
+ dlock_buffer_after_resize = db->GetDeadlockInfoBuffer();
+ ASSERT_EQ(dlock_buffer_after_resize.size(), 0);
+
+ // Contrived case of shared lock of cycle size 2 to verify that a shared
+ // lock causing a deadlock is correctly reported as "shared" in the buffer.
+ std::vector<Transaction*> txns_shared(2);
+
+ // Create a cycle of size 2.
+ for (uint32_t i = 0; i < 2; i++) {
+ txns_shared[i] = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txns_shared[i]);
+ auto s = txns_shared[i]->GetForUpdate(read_options, ToString(i), nullptr);
+ ASSERT_OK(s);
+ }
+
+ std::atomic<uint32_t> checkpoints_shared(0);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "TransactionLockMgr::AcquireWithTimeout:WaitingTxn",
+ [&](void* /*arg*/) { checkpoints_shared.fetch_add(1); });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ std::vector<port::Thread> threads_shared;
+ for (uint32_t i = 0; i < 1; i++) {
+ std::function<void()> blocking_thread = [&, i] {
+ auto s =
+ txns_shared[i]->GetForUpdate(read_options, ToString(i + 1), nullptr);
+ ASSERT_OK(s);
+ txns_shared[i]->Rollback();
+ delete txns_shared[i];
+ };
+ threads_shared.emplace_back(blocking_thread);
+ }
+
+ // Wait until all threads are waiting on each other.
+ while (checkpoints_shared.load() != 1) {
+ /* sleep override */
+ std::this_thread::sleep_for(std::chrono::milliseconds(100));
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ // Complete the cycle T2 -> T1 with a shared lock.
+ auto s = txns_shared[1]->GetForUpdate(read_options, "0", nullptr, false);
+ ASSERT_TRUE(s.IsDeadlock());
+
+ auto dlock_buffer = db->GetDeadlockInfoBuffer();
+
+ // Verify the size of the buffer and the single path.
+ ASSERT_EQ(dlock_buffer.size(), 1);
+ ASSERT_EQ(dlock_buffer[0].path.size(), 2);
+
+ // Verify the exclusivity field of the transactions in the deadlock path.
+ ASSERT_TRUE(dlock_buffer[0].path[0].m_exclusive);
+ ASSERT_FALSE(dlock_buffer[0].path[1].m_exclusive);
+ txns_shared[1]->Rollback();
+ delete txns_shared[1];
+
+ for (auto& t : threads_shared) {
+ t.join();
+ }
+}
+
+#ifndef ROCKSDB_VALGRIND_RUN
+TEST_P(TransactionStressTest, DeadlockCycle) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+
+ // offset by 2 from the max depth to test edge case
+ const uint32_t kMaxCycleLength = 52;
+
+ txn_options.lock_timeout = 1000000;
+ txn_options.deadlock_detect = true;
+
+ for (uint32_t len = 2; len < kMaxCycleLength; len++) {
+ // Set up a long wait for chain like this:
+ //
+ // T1 -> T2 -> T3 -> ... -> Tlen
+
+ std::vector<Transaction*> txns(len);
+
+ for (uint32_t i = 0; i < len; i++) {
+ txns[i] = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txns[i]);
+ auto s = txns[i]->GetForUpdate(read_options, ToString(i), nullptr);
+ ASSERT_OK(s);
+ }
+
+ std::atomic<uint32_t> checkpoints(0);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "TransactionLockMgr::AcquireWithTimeout:WaitingTxn",
+ [&](void* /*arg*/) { checkpoints.fetch_add(1); });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ // We want the last transaction in the chain to block and hold everyone
+ // back.
+ std::vector<port::Thread> threads;
+ for (uint32_t i = 0; i < len - 1; i++) {
+ std::function<void()> blocking_thread = [&, i] {
+ auto s = txns[i]->GetForUpdate(read_options, ToString(i + 1), nullptr);
+ ASSERT_OK(s);
+ txns[i]->Rollback();
+ delete txns[i];
+ };
+ threads.emplace_back(blocking_thread);
+ }
+
+ // Wait until all threads are waiting on each other.
+ while (checkpoints.load() != len - 1) {
+ /* sleep override */
+ std::this_thread::sleep_for(std::chrono::milliseconds(100));
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ // Complete the cycle Tlen -> T1
+ auto s = txns[len - 1]->GetForUpdate(read_options, "0", nullptr);
+ ASSERT_TRUE(s.IsDeadlock());
+
+ const uint32_t dlock_buffer_size_ = (len - 1 > 5) ? 5 : (len - 1);
+ uint32_t curr_waiting_key = 0;
+ TransactionID curr_txn_id = txns[0]->GetID();
+
+ auto dlock_buffer = db->GetDeadlockInfoBuffer();
+ ASSERT_EQ(dlock_buffer.size(), dlock_buffer_size_);
+ uint32_t check_len = len;
+ bool check_limit_flag = false;
+
+ // Special case for a deadlock path that exceeds the maximum depth.
+ if (len > 50) {
+ check_len = 0;
+ check_limit_flag = true;
+ }
+ auto dlock_entry = dlock_buffer[0].path;
+ ASSERT_EQ(dlock_entry.size(), check_len);
+ ASSERT_EQ(dlock_buffer[0].limit_exceeded, check_limit_flag);
+
+ int64_t pre_deadlock_time = dlock_buffer[0].deadlock_time;
+ int64_t cur_deadlock_time = 0;
+ for (auto const& dl_path_rec : dlock_buffer) {
+ cur_deadlock_time = dl_path_rec.deadlock_time;
+ ASSERT_NE(cur_deadlock_time, 0);
+ ASSERT_TRUE(cur_deadlock_time <= pre_deadlock_time);
+ pre_deadlock_time = cur_deadlock_time;
+ }
+
+ // Iterates backwards over path verifying decreasing txn_ids.
+ for (auto it = dlock_entry.rbegin(); it != dlock_entry.rend(); ++it) {
+ auto dl_node = *it;
+ ASSERT_EQ(dl_node.m_txn_id, len + curr_txn_id - 1);
+ ASSERT_EQ(dl_node.m_cf_id, 0u);
+ ASSERT_EQ(dl_node.m_waiting_key, ToString(curr_waiting_key));
+ ASSERT_EQ(dl_node.m_exclusive, true);
+
+ curr_txn_id--;
+ if (curr_waiting_key == 0) {
+ curr_waiting_key = len;
+ }
+ curr_waiting_key--;
+ }
+
+ // Rollback the last transaction.
+ txns[len - 1]->Rollback();
+ delete txns[len - 1];
+
+ for (auto& t : threads) {
+ t.join();
+ }
+ }
+}
+
+TEST_P(TransactionStressTest, DeadlockStress) {
+ const uint32_t NUM_TXN_THREADS = 10;
+ const uint32_t NUM_KEYS = 100;
+ const uint32_t NUM_ITERS = 10000;
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+
+ txn_options.lock_timeout = 1000000;
+ txn_options.deadlock_detect = true;
+ std::vector<std::string> keys;
+
+ for (uint32_t i = 0; i < NUM_KEYS; i++) {
+ db->Put(write_options, Slice(ToString(i)), Slice(""));
+ keys.push_back(ToString(i));
+ }
+
+ size_t tid = std::hash<std::thread::id>()(std::this_thread::get_id());
+ Random rnd(static_cast<uint32_t>(tid));
+ std::function<void(uint32_t)> stress_thread = [&](uint32_t seed) {
+ std::default_random_engine g(seed);
+
+ Transaction* txn;
+ for (uint32_t i = 0; i < NUM_ITERS; i++) {
+ txn = db->BeginTransaction(write_options, txn_options);
+ auto random_keys = keys;
+ std::shuffle(random_keys.begin(), random_keys.end(), g);
+
+ // Lock keys in random order.
+ for (const auto& k : random_keys) {
+ // Lock mostly for shared access, but exclusive 1/4 of the time.
+ auto s =
+ txn->GetForUpdate(read_options, k, nullptr, txn->GetID() % 4 == 0);
+ if (!s.ok()) {
+ ASSERT_TRUE(s.IsDeadlock());
+ txn->Rollback();
+ break;
+ }
+ }
+
+ delete txn;
+ }
+ };
+
+ std::vector<port::Thread> threads;
+ for (uint32_t i = 0; i < NUM_TXN_THREADS; i++) {
+ threads.emplace_back(stress_thread, rnd.Next());
+ }
+
+ for (auto& t : threads) {
+ t.join();
+ }
+}
+#endif // ROCKSDB_VALGRIND_RUN
+
+TEST_P(TransactionTest, CommitTimeBatchFailTest) {
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+
+ std::string value;
+ Status s;
+
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+
+ ASSERT_OK(txn1->GetCommitTimeWriteBatch()->Put("cat", "dog"));
+
+ s = txn1->Put("foo", "bar");
+ ASSERT_OK(s);
+
+ // fails due to non-empty commit-time batch
+ s = txn1->Commit();
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ delete txn1;
+}
+
+TEST_P(TransactionTest, LogMarkLeakTest) {
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ options.write_buffer_size = 1024;
+ ASSERT_OK(ReOpenNoDelete());
+ assert(db != nullptr);
+ Random rnd(47);
+ std::vector<Transaction*> txns;
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ // At the beginning there should be no log containing prepare data
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+ for (size_t i = 0; i < 100; i++) {
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn->SetName("xid" + ToString(i)));
+ ASSERT_OK(txn->Put(Slice("foo" + ToString(i)), Slice("bar")));
+ ASSERT_OK(txn->Prepare());
+ ASSERT_GT(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+ if (rnd.OneIn(5)) {
+ txns.push_back(txn);
+ } else {
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ }
+ db_impl->TEST_FlushMemTable(true);
+ }
+ for (auto txn : txns) {
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ }
+ // At the end there should be no log left containing prepare data
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+ // Make sure that the underlying data structures are properly truncated and
+ // cause not leak
+ ASSERT_EQ(db_impl->TEST_PreparedSectionCompletedSize(), 0);
+ ASSERT_EQ(db_impl->TEST_LogsWithPrepSize(), 0);
+}
+
+TEST_P(TransactionTest, SimpleTwoPhaseTransactionTest) {
+ for (bool cwb4recovery : {true, false}) {
+ ASSERT_OK(ReOpen());
+ WriteOptions write_options;
+ ReadOptions read_options;
+
+ TransactionOptions txn_options;
+ txn_options.use_only_the_last_commit_time_batch_for_recovery = cwb4recovery;
+
+ string value;
+ Status s;
+
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ s = txn->SetName("xid");
+ ASSERT_OK(s);
+
+ ASSERT_EQ(db->GetTransactionByName("xid"), txn);
+
+ // transaction put
+ s = txn->Put(Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+ ASSERT_EQ(1, txn->GetNumPuts());
+
+ // regular db put
+ s = db->Put(write_options, Slice("foo2"), Slice("bar2"));
+ ASSERT_OK(s);
+ ASSERT_EQ(1, txn->GetNumPuts());
+
+ // regular db read
+ db->Get(read_options, "foo2", &value);
+ ASSERT_EQ(value, "bar2");
+
+ // commit time put
+ txn->GetCommitTimeWriteBatch()->Put(Slice("gtid"), Slice("dogs"));
+ txn->GetCommitTimeWriteBatch()->Put(Slice("gtid2"), Slice("cats"));
+
+ // nothing has been prepped yet
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ // data not im mem yet
+ s = db->Get(read_options, Slice("foo"), &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = db->Get(read_options, Slice("gtid"), &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // find trans in list of prepared transactions
+ std::vector<Transaction*> prepared_trans;
+ db->GetAllPreparedTransactions(&prepared_trans);
+ ASSERT_EQ(prepared_trans.size(), 1);
+ ASSERT_EQ(prepared_trans.front()->GetName(), "xid");
+
+ auto log_containing_prep =
+ db_impl->TEST_FindMinLogContainingOutstandingPrep();
+ ASSERT_GT(log_containing_prep, 0);
+
+ // make commit
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ // value is now available
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+
+ if (!cwb4recovery) {
+ s = db->Get(read_options, "gtid", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "dogs");
+
+ s = db->Get(read_options, "gtid2", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "cats");
+ }
+
+ // we already committed
+ s = txn->Commit();
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // no longer is prepared results
+ db->GetAllPreparedTransactions(&prepared_trans);
+ ASSERT_EQ(prepared_trans.size(), 0);
+ ASSERT_EQ(db->GetTransactionByName("xid"), nullptr);
+
+ // heap should not care about prepared section anymore
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ // but now our memtable should be referencing the prep section
+ ASSERT_GE(log_containing_prep, db_impl->MinLogNumberToKeep());
+ ASSERT_EQ(log_containing_prep,
+ db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // In these modes memtable do not ref the prep sections
+ ASSERT_EQ(0, db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ default:
+ assert(false);
+ }
+
+ db_impl->TEST_FlushMemTable(true);
+ // After flush the recoverable state must be visible
+ if (cwb4recovery) {
+ s = db->Get(read_options, "gtid", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "dogs");
+
+ s = db->Get(read_options, "gtid2", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "cats");
+ }
+
+ // after memtable flush we can now relese the log
+ ASSERT_GT(db_impl->MinLogNumberToKeep(), log_containing_prep);
+ ASSERT_EQ(0, db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+
+ delete txn;
+
+ if (cwb4recovery) {
+ // kill and reopen to trigger recovery
+ s = ReOpenNoDelete();
+ ASSERT_OK(s);
+ assert(db != nullptr);
+ s = db->Get(read_options, "gtid", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "dogs");
+
+ s = db->Get(read_options, "gtid2", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "cats");
+ }
+ }
+}
+
+TEST_P(TransactionTest, TwoPhaseNameTest) {
+ Status s;
+
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ Transaction* txn3 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn3);
+ delete txn3;
+
+ // cant prepare txn without name
+ s = txn1->Prepare();
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // name too short
+ s = txn1->SetName("");
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // name too long
+ s = txn1->SetName(std::string(513, 'x'));
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // valid set name
+ s = txn1->SetName("name1");
+ ASSERT_OK(s);
+
+ // cant have duplicate name
+ s = txn2->SetName("name1");
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // shouldn't be able to prepare
+ s = txn2->Prepare();
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // valid name set
+ s = txn2->SetName("name2");
+ ASSERT_OK(s);
+
+ // cant reset name
+ s = txn2->SetName("name3");
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ ASSERT_EQ(txn1->GetName(), "name1");
+ ASSERT_EQ(txn2->GetName(), "name2");
+
+ s = txn1->Prepare();
+ ASSERT_OK(s);
+
+ // can't rename after prepare
+ s = txn1->SetName("name4");
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ txn1->Rollback();
+ txn2->Rollback();
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, TwoPhaseEmptyWriteTest) {
+ for (bool cwb4recovery : {true, false}) {
+ for (bool test_with_empty_wal : {true, false}) {
+ if (!cwb4recovery && test_with_empty_wal) {
+ continue;
+ }
+ ASSERT_OK(ReOpen());
+ Status s;
+ std::string value;
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ txn_options.use_only_the_last_commit_time_batch_for_recovery =
+ cwb4recovery;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ s = txn1->SetName("joe");
+ ASSERT_OK(s);
+
+ s = txn2->SetName("bob");
+ ASSERT_OK(s);
+
+ s = txn1->Prepare();
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+
+ txn2->GetCommitTimeWriteBatch()->Put(Slice("foo"), Slice("bar"));
+
+ s = txn2->Prepare();
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ delete txn2;
+ if (!cwb4recovery) {
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+ } else {
+ if (test_with_empty_wal) {
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ db_impl->TEST_FlushMemTable(true);
+ // After flush the state must be visible
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+ }
+ db->FlushWAL(true);
+ // kill and reopen to trigger recovery
+ s = ReOpenNoDelete();
+ ASSERT_OK(s);
+ assert(db != nullptr);
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+ }
+ }
+ }
+}
+
+#ifndef ROCKSDB_VALGRIND_RUN
+TEST_P(TransactionStressTest, TwoPhaseExpirationTest) {
+ Status s;
+
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ txn_options.expiration = 500; // 500ms
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+ ASSERT_TRUE(txn1);
+
+ s = txn1->SetName("joe");
+ ASSERT_OK(s);
+ s = txn2->SetName("bob");
+ ASSERT_OK(s);
+
+ s = txn1->Prepare();
+ ASSERT_OK(s);
+
+ /* sleep override */
+ std::this_thread::sleep_for(std::chrono::milliseconds(1000));
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->Prepare();
+ ASSERT_EQ(s, Status::Expired());
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, TwoPhaseRollbackTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+
+ TransactionOptions txn_options;
+
+ std::string value;
+ Status s;
+
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ s = txn->SetName("xid");
+ ASSERT_OK(s);
+
+ // transaction put
+ s = txn->Put(Slice("tfoo"), Slice("tbar"));
+ ASSERT_OK(s);
+
+ // value is readable form txn
+ s = txn->Get(read_options, Slice("tfoo"), &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "tbar");
+
+ // issue rollback
+ s = txn->Rollback();
+ ASSERT_OK(s);
+
+ // value is nolonger readable
+ s = txn->Get(read_options, Slice("tfoo"), &value);
+ ASSERT_TRUE(s.IsNotFound());
+ ASSERT_EQ(txn->GetNumPuts(), 0);
+
+ // put new txn values
+ s = txn->Put(Slice("tfoo2"), Slice("tbar2"));
+ ASSERT_OK(s);
+
+ // new value is readable from txn
+ s = txn->Get(read_options, Slice("tfoo2"), &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "tbar2");
+
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ // flush to next wal
+ s = db->Put(write_options, Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+ db_impl->TEST_FlushMemTable(true);
+
+ // issue rollback (marker written to WAL)
+ s = txn->Rollback();
+ ASSERT_OK(s);
+
+ // value is nolonger readable
+ s = txn->Get(read_options, Slice("tfoo2"), &value);
+ ASSERT_TRUE(s.IsNotFound());
+ ASSERT_EQ(txn->GetNumPuts(), 0);
+
+ // make commit
+ s = txn->Commit();
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // try rollback again
+ s = txn->Rollback();
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, PersistentTwoPhaseTransactionTest) {
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = false;
+ ReadOptions read_options;
+
+ TransactionOptions txn_options;
+
+ std::string value;
+ Status s;
+
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ s = txn->SetName("xid");
+ ASSERT_OK(s);
+
+ ASSERT_EQ(db->GetTransactionByName("xid"), txn);
+
+ // transaction put
+ s = txn->Put(Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+ ASSERT_EQ(1, txn->GetNumPuts());
+
+ // txn read
+ s = txn->Get(read_options, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+
+ // regular db put
+ s = db->Put(write_options, Slice("foo2"), Slice("bar2"));
+ ASSERT_OK(s);
+ ASSERT_EQ(1, txn->GetNumPuts());
+
+ db_impl->TEST_FlushMemTable(true);
+
+ // regular db read
+ db->Get(read_options, "foo2", &value);
+ ASSERT_EQ(value, "bar2");
+
+ // nothing has been prepped yet
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+
+ // prepare
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ // still not available to db
+ s = db->Get(read_options, Slice("foo"), &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ db->FlushWAL(false);
+ delete txn;
+ // kill and reopen
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ s = ReOpenNoDelete();
+ ASSERT_OK(s);
+ assert(db != nullptr);
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+
+ // find trans in list of prepared transactions
+ std::vector<Transaction*> prepared_trans;
+ db->GetAllPreparedTransactions(&prepared_trans);
+ ASSERT_EQ(prepared_trans.size(), 1);
+
+ txn = prepared_trans.front();
+ ASSERT_TRUE(txn);
+ ASSERT_EQ(txn->GetName(), "xid");
+ ASSERT_EQ(db->GetTransactionByName("xid"), txn);
+
+ // log has been marked
+ auto log_containing_prep =
+ db_impl->TEST_FindMinLogContainingOutstandingPrep();
+ ASSERT_GT(log_containing_prep, 0);
+
+ // value is readable from txn
+ s = txn->Get(read_options, "foo", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar");
+
+ // make commit
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ // value is now available
+ db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ // we already committed
+ s = txn->Commit();
+ ASSERT_EQ(s, Status::InvalidArgument());
+
+ // no longer is prepared results
+ prepared_trans.clear();
+ db->GetAllPreparedTransactions(&prepared_trans);
+ ASSERT_EQ(prepared_trans.size(), 0);
+
+ // transaction should no longer be visible
+ ASSERT_EQ(db->GetTransactionByName("xid"), nullptr);
+
+ // heap should not care about prepared section anymore
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ // but now our memtable should be referencing the prep section
+ ASSERT_EQ(log_containing_prep,
+ db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ ASSERT_GE(log_containing_prep, db_impl->MinLogNumberToKeep());
+
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // In these modes memtable do not ref the prep sections
+ ASSERT_EQ(0, db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ default:
+ assert(false);
+ }
+
+ // Add a dummy record to memtable before a flush. Otherwise, the
+ // memtable will be empty and flush will be skipped.
+ s = db->Put(write_options, Slice("foo3"), Slice("bar3"));
+ ASSERT_OK(s);
+
+ db_impl->TEST_FlushMemTable(true);
+
+ // after memtable flush we can now release the log
+ ASSERT_GT(db_impl->MinLogNumberToKeep(), log_containing_prep);
+ ASSERT_EQ(0, db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+
+ delete txn;
+
+ // deleting transaction should unregister transaction
+ ASSERT_EQ(db->GetTransactionByName("xid"), nullptr);
+}
+#endif // ROCKSDB_VALGRIND_RUN
+
+// TODO this test needs to be updated with serial commits
+TEST_P(TransactionTest, DISABLED_TwoPhaseMultiThreadTest) {
+ // mix transaction writes and regular writes
+ const uint32_t NUM_TXN_THREADS = 50;
+ std::atomic<uint32_t> txn_thread_num(0);
+
+ std::function<void()> txn_write_thread = [&]() {
+ uint32_t id = txn_thread_num.fetch_add(1);
+
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = false;
+ TransactionOptions txn_options;
+ txn_options.lock_timeout = 1000000;
+ if (id % 2 == 0) {
+ txn_options.expiration = 1000000;
+ }
+ TransactionName name("xid_" + std::string(1, 'A' + static_cast<char>(id)));
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn->SetName(name));
+ for (int i = 0; i < 10; i++) {
+ std::string key(name + "_" + std::string(1, static_cast<char>('A' + i)));
+ ASSERT_OK(txn->Put(key, "val"));
+ }
+ ASSERT_OK(txn->Prepare());
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ };
+
+ // assure that all thread are in the same write group
+ std::atomic<uint32_t> t_wait_on_prepare(0);
+ std::atomic<uint32_t> t_wait_on_commit(0);
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "WriteThread::JoinBatchGroup:Wait", [&](void* arg) {
+ auto* writer = reinterpret_cast<WriteThread::Writer*>(arg);
+
+ if (writer->ShouldWriteToWAL()) {
+ t_wait_on_prepare.fetch_add(1);
+ // wait for friends
+ while (t_wait_on_prepare.load() < NUM_TXN_THREADS) {
+ env->SleepForMicroseconds(10);
+ }
+ } else if (writer->ShouldWriteToMemtable()) {
+ t_wait_on_commit.fetch_add(1);
+ // wait for friends
+ while (t_wait_on_commit.load() < NUM_TXN_THREADS) {
+ env->SleepForMicroseconds(10);
+ }
+ } else {
+ FAIL();
+ }
+ });
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ // do all the writes
+ std::vector<port::Thread> threads;
+ for (uint32_t i = 0; i < NUM_TXN_THREADS; i++) {
+ threads.emplace_back(txn_write_thread);
+ }
+ for (auto& t : threads) {
+ t.join();
+ }
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+ for (uint32_t t = 0; t < NUM_TXN_THREADS; t++) {
+ TransactionName name("xid_" + std::string(1, 'A' + static_cast<char>(t)));
+ for (int i = 0; i < 10; i++) {
+ std::string key(name + "_" + std::string(1, static_cast<char>('A' + i)));
+ s = db->Get(read_options, key, &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "val");
+ }
+ }
+}
+
+TEST_P(TransactionStressTest, TwoPhaseLongPrepareTest) {
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = false;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ s = txn->SetName("bob");
+ ASSERT_OK(s);
+
+ // transaction put
+ s = txn->Put(Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+
+ // prepare
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ delete txn;
+
+ for (int i = 0; i < 1000; i++) {
+ std::string key(i, 'k');
+ std::string val(1000, 'v');
+ assert(db != nullptr);
+ s = db->Put(write_options, key, val);
+ ASSERT_OK(s);
+
+ if (i % 29 == 0) {
+ // crash
+ env->SetFilesystemActive(false);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ReOpenNoDelete();
+ } else if (i % 37 == 0) {
+ // close
+ ReOpenNoDelete();
+ }
+ }
+
+ // commit old txn
+ txn = db->GetTransactionByName("bob");
+ ASSERT_TRUE(txn);
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ // verify data txn data
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(value, "bar");
+
+ // verify non txn data
+ for (int i = 0; i < 1000; i++) {
+ std::string key(i, 'k');
+ std::string val(1000, 'v');
+ s = db->Get(read_options, key, &value);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(value, val);
+ }
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, TwoPhaseSequenceTest) {
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = false;
+ ReadOptions read_options;
+
+ TransactionOptions txn_options;
+
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ s = txn->SetName("xid");
+ ASSERT_OK(s);
+
+ // transaction put
+ s = txn->Put(Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+ s = txn->Put(Slice("foo2"), Slice("bar2"));
+ ASSERT_OK(s);
+ s = txn->Put(Slice("foo3"), Slice("bar3"));
+ ASSERT_OK(s);
+ s = txn->Put(Slice("foo4"), Slice("bar4"));
+ ASSERT_OK(s);
+
+ // prepare
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ // make commit
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+
+ // kill and reopen
+ env->SetFilesystemActive(false);
+ ReOpenNoDelete();
+ assert(db != nullptr);
+
+ // value is now available
+ s = db->Get(read_options, "foo4", &value);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(value, "bar4");
+}
+
+TEST_P(TransactionTest, TwoPhaseDoubleRecoveryTest) {
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = false;
+ ReadOptions read_options;
+
+ TransactionOptions txn_options;
+
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ s = txn->SetName("a");
+ ASSERT_OK(s);
+
+ // transaction put
+ s = txn->Put(Slice("foo"), Slice("bar"));
+ ASSERT_OK(s);
+
+ // prepare
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ delete txn;
+
+ // kill and reopen
+ env->SetFilesystemActive(false);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ReOpenNoDelete();
+
+ // commit old txn
+ txn = db->GetTransactionByName("a");
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(value, "bar");
+
+ delete txn;
+
+ txn = db->BeginTransaction(write_options, txn_options);
+ s = txn->SetName("b");
+ ASSERT_OK(s);
+
+ s = txn->Put(Slice("foo2"), Slice("bar2"));
+ ASSERT_OK(s);
+
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+
+ // kill and reopen
+ env->SetFilesystemActive(false);
+ ReOpenNoDelete();
+ assert(db != nullptr);
+
+ // value is now available
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(value, "bar");
+
+ s = db->Get(read_options, "foo2", &value);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(value, "bar2");
+}
+
+TEST_P(TransactionTest, TwoPhaseLogRollingTest) {
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+
+ Status s;
+ std::string v;
+ ColumnFamilyHandle *cfa, *cfb;
+
+ // Create 2 new column families
+ ColumnFamilyOptions cf_options;
+ s = db->CreateColumnFamily(cf_options, "CFA", &cfa);
+ ASSERT_OK(s);
+ s = db->CreateColumnFamily(cf_options, "CFB", &cfb);
+ ASSERT_OK(s);
+
+ WriteOptions wopts;
+ wopts.disableWAL = false;
+ wopts.sync = true;
+
+ TransactionOptions topts1;
+ Transaction* txn1 = db->BeginTransaction(wopts, topts1);
+ s = txn1->SetName("xid1");
+ ASSERT_OK(s);
+
+ TransactionOptions topts2;
+ Transaction* txn2 = db->BeginTransaction(wopts, topts2);
+ s = txn2->SetName("xid2");
+ ASSERT_OK(s);
+
+ // transaction put in two column families
+ s = txn1->Put(cfa, "ka1", "va1");
+ ASSERT_OK(s);
+
+ // transaction put in two column families
+ s = txn2->Put(cfa, "ka2", "va2");
+ ASSERT_OK(s);
+ s = txn2->Put(cfb, "kb2", "vb2");
+ ASSERT_OK(s);
+
+ // write prep section to wal
+ s = txn1->Prepare();
+ ASSERT_OK(s);
+
+ // our log should be in the heap
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(),
+ txn1->GetLogNumber());
+ ASSERT_EQ(db_impl->TEST_LogfileNumber(), txn1->GetLastLogNumber());
+
+ // flush default cf to crate new log
+ s = db->Put(wopts, "foo", "bar");
+ ASSERT_OK(s);
+ s = db_impl->TEST_FlushMemTable(true);
+ ASSERT_OK(s);
+
+ // make sure we are on a new log
+ ASSERT_GT(db_impl->TEST_LogfileNumber(), txn1->GetLastLogNumber());
+
+ // put txn2 prep section in this log
+ s = txn2->Prepare();
+ ASSERT_OK(s);
+ ASSERT_EQ(db_impl->TEST_LogfileNumber(), txn2->GetLastLogNumber());
+
+ // heap should still see first log
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(),
+ txn1->GetLogNumber());
+
+ // commit txn1
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ // heap should now show txn2s log
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(),
+ txn2->GetLogNumber());
+
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ // we should see txn1s log refernced by the memtables
+ ASSERT_EQ(txn1->GetLogNumber(),
+ db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // In these modes memtable do not ref the prep sections
+ ASSERT_EQ(0, db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ default:
+ assert(false);
+ }
+
+ // flush default cf to crate new log
+ s = db->Put(wopts, "foo", "bar2");
+ ASSERT_OK(s);
+ s = db_impl->TEST_FlushMemTable(true);
+ ASSERT_OK(s);
+
+ // make sure we are on a new log
+ ASSERT_GT(db_impl->TEST_LogfileNumber(), txn2->GetLastLogNumber());
+
+ // commit txn2
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ // heap should not show any logs
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ // should show the first txn log
+ ASSERT_EQ(txn1->GetLogNumber(),
+ db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // In these modes memtable do not ref the prep sections
+ ASSERT_EQ(0, db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ default:
+ assert(false);
+ }
+
+ // flush only cfa memtable
+ s = db_impl->TEST_FlushMemTable(true, false, cfa);
+ ASSERT_OK(s);
+
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ // should show the first txn log
+ ASSERT_EQ(txn2->GetLogNumber(),
+ db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // In these modes memtable do not ref the prep sections
+ ASSERT_EQ(0, db_impl->TEST_FindMinPrepLogReferencedByMemTable());
+ break;
+ default:
+ assert(false);
+ }
+
+ // flush only cfb memtable
+ s = db_impl->TEST_FlushMemTable(true, false, cfb);
+ ASSERT_OK(s);
+
+ // should show not dependency on logs
+ ASSERT_EQ(db_impl->TEST_FindMinPrepLogReferencedByMemTable(), 0);
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+
+ delete txn1;
+ delete txn2;
+ delete cfa;
+ delete cfb;
+}
+
+TEST_P(TransactionTest, TwoPhaseLogRollingTest2) {
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+
+ Status s;
+ ColumnFamilyHandle *cfa, *cfb;
+
+ ColumnFamilyOptions cf_options;
+ s = db->CreateColumnFamily(cf_options, "CFA", &cfa);
+ ASSERT_OK(s);
+ s = db->CreateColumnFamily(cf_options, "CFB", &cfb);
+ ASSERT_OK(s);
+
+ WriteOptions wopts;
+ wopts.disableWAL = false;
+ wopts.sync = true;
+
+ auto cfh_a = reinterpret_cast<ColumnFamilyHandleImpl*>(cfa);
+ auto cfh_b = reinterpret_cast<ColumnFamilyHandleImpl*>(cfb);
+
+ TransactionOptions topts1;
+ Transaction* txn1 = db->BeginTransaction(wopts, topts1);
+ s = txn1->SetName("xid1");
+ ASSERT_OK(s);
+ s = txn1->Put(cfa, "boys", "girls1");
+ ASSERT_OK(s);
+
+ Transaction* txn2 = db->BeginTransaction(wopts, topts1);
+ s = txn2->SetName("xid2");
+ ASSERT_OK(s);
+ s = txn2->Put(cfb, "up", "down1");
+ ASSERT_OK(s);
+
+ // prepre transaction in LOG A
+ s = txn1->Prepare();
+ ASSERT_OK(s);
+
+ // prepre transaction in LOG A
+ s = txn2->Prepare();
+ ASSERT_OK(s);
+
+ // regular put so that mem table can actually be flushed for log rolling
+ s = db->Put(wopts, "cats", "dogs1");
+ ASSERT_OK(s);
+
+ auto prepare_log_no = txn1->GetLastLogNumber();
+
+ // roll to LOG B
+ s = db_impl->TEST_FlushMemTable(true);
+ ASSERT_OK(s);
+
+ // now we pause background work so that
+ // imm()s are not flushed before we can check their status
+ s = db_impl->PauseBackgroundWork();
+ ASSERT_OK(s);
+
+ ASSERT_GT(db_impl->TEST_LogfileNumber(), prepare_log_no);
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ // This cf is empty and should ref the latest log
+ ASSERT_GT(cfh_a->cfd()->GetLogNumber(), prepare_log_no);
+ ASSERT_EQ(cfh_a->cfd()->GetLogNumber(), db_impl->TEST_LogfileNumber());
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // This cf is not flushed yet and should ref the log that has its data
+ ASSERT_EQ(cfh_a->cfd()->GetLogNumber(), prepare_log_no);
+ break;
+ default:
+ assert(false);
+ }
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(),
+ txn1->GetLogNumber());
+ ASSERT_EQ(db_impl->TEST_FindMinPrepLogReferencedByMemTable(), 0);
+
+ // commit in LOG B
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ ASSERT_EQ(db_impl->TEST_FindMinPrepLogReferencedByMemTable(),
+ prepare_log_no);
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // In these modes memtable do not ref the prep sections
+ ASSERT_EQ(db_impl->TEST_FindMinPrepLogReferencedByMemTable(), 0);
+ break;
+ default:
+ assert(false);
+ }
+
+ ASSERT_TRUE(!db_impl->TEST_UnableToReleaseOldestLog());
+
+ // request a flush for all column families such that the earliest
+ // alive log file can be killed
+ db_impl->TEST_SwitchWAL();
+ // log cannot be flushed because txn2 has not been commited
+ ASSERT_TRUE(!db_impl->TEST_IsLogGettingFlushed());
+ ASSERT_TRUE(db_impl->TEST_UnableToReleaseOldestLog());
+
+ // assert that cfa has a flush requested
+ ASSERT_TRUE(cfh_a->cfd()->imm()->HasFlushRequested());
+
+ switch (txn_db_options.write_policy) {
+ case WRITE_COMMITTED:
+ // cfb should not be flushed becuse it has no data from LOG A
+ ASSERT_TRUE(!cfh_b->cfd()->imm()->HasFlushRequested());
+ break;
+ case WRITE_PREPARED:
+ case WRITE_UNPREPARED:
+ // cfb should be flushed becuse it has prepared data from LOG A
+ ASSERT_TRUE(cfh_b->cfd()->imm()->HasFlushRequested());
+ break;
+ default:
+ assert(false);
+ }
+
+ // cfb now has data from LOG A
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ db_impl->TEST_SwitchWAL();
+ ASSERT_TRUE(!db_impl->TEST_UnableToReleaseOldestLog());
+
+ // we should see that cfb now has a flush requested
+ ASSERT_TRUE(cfh_b->cfd()->imm()->HasFlushRequested());
+
+ // all data in LOG A resides in a memtable that has been
+ // requested for a flush
+ ASSERT_TRUE(db_impl->TEST_IsLogGettingFlushed());
+
+ delete txn1;
+ delete txn2;
+ delete cfa;
+ delete cfb;
+}
+/*
+ * 1) use prepare to keep first log around to determine starting sequence
+ * during recovery.
+ * 2) insert many values, skipping wal, to increase seqid.
+ * 3) insert final value into wal
+ * 4) recover and see that final value was properly recovered - not
+ * hidden behind improperly summed sequence ids
+ */
+TEST_P(TransactionTest, TwoPhaseOutOfOrderDelete) {
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ WriteOptions wal_on, wal_off;
+ wal_on.sync = true;
+ wal_on.disableWAL = false;
+ wal_off.disableWAL = true;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+
+ std::string value;
+ Status s;
+
+ Transaction* txn1 = db->BeginTransaction(wal_on, txn_options);
+
+ s = txn1->SetName("1");
+ ASSERT_OK(s);
+
+ s = db->Put(wal_on, "first", "first");
+ ASSERT_OK(s);
+
+ s = txn1->Put(Slice("dummy"), Slice("dummy"));
+ ASSERT_OK(s);
+ s = txn1->Prepare();
+ ASSERT_OK(s);
+
+ s = db->Put(wal_off, "cats", "dogs1");
+ ASSERT_OK(s);
+ s = db->Put(wal_off, "cats", "dogs2");
+ ASSERT_OK(s);
+ s = db->Put(wal_off, "cats", "dogs3");
+ ASSERT_OK(s);
+
+ s = db_impl->TEST_FlushMemTable(true);
+ ASSERT_OK(s);
+
+ s = db->Put(wal_on, "cats", "dogs4");
+ ASSERT_OK(s);
+
+ db->FlushWAL(false);
+
+ // kill and reopen
+ env->SetFilesystemActive(false);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ReOpenNoDelete();
+ assert(db != nullptr);
+
+ s = db->Get(read_options, "first", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "first");
+
+ s = db->Get(read_options, "cats", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "dogs4");
+}
+
+TEST_P(TransactionTest, FirstWriteTest) {
+ WriteOptions write_options;
+
+ // Test conflict checking against the very first write to a db.
+ // The transaction's snapshot will have seq 1 and the following write
+ // will have sequence 1.
+ Status s = db->Put(write_options, "A", "a");
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ txn->SetSnapshot();
+
+ ASSERT_OK(s);
+
+ s = txn->Put("A", "b");
+ ASSERT_OK(s);
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, FirstWriteTest2) {
+ WriteOptions write_options;
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ txn->SetSnapshot();
+
+ // Test conflict checking against the very first write to a db.
+ // The transaction's snapshot is a seq 0 while the following write
+ // will have sequence 1.
+ Status s = db->Put(write_options, "A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Put("A", "b");
+ ASSERT_TRUE(s.IsBusy());
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, WriteOptionsTest) {
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = true;
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ ASSERT_TRUE(txn->GetWriteOptions()->sync);
+
+ write_options.sync = false;
+ txn->SetWriteOptions(write_options);
+ ASSERT_FALSE(txn->GetWriteOptions()->sync);
+ ASSERT_TRUE(txn->GetWriteOptions()->disableWAL);
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, WriteConflictTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ db->Put(write_options, "foo", "A");
+ db->Put(write_options, "foo2", "B");
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->Put("foo", "A2");
+ ASSERT_OK(s);
+
+ s = txn->Put("foo2", "B2");
+ ASSERT_OK(s);
+
+ // This Put outside of a transaction will conflict with the previous write
+ s = db->Put(write_options, "foo", "xxx");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "A");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "A2");
+ db->Get(read_options, "foo2", &value);
+ ASSERT_EQ(value, "B2");
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, WriteConflictTest2) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ std::string value;
+ Status s;
+
+ db->Put(write_options, "foo", "bar");
+
+ txn_options.set_snapshot = true;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ // This Put outside of a transaction will conflict with a later write
+ s = db->Put(write_options, "foo", "barz");
+ ASSERT_OK(s);
+
+ s = txn->Put("foo2", "X");
+ ASSERT_OK(s);
+
+ s = txn->Put("foo",
+ "bar2"); // Conflicts with write done after snapshot taken
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn->Put("foo3", "Y");
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+
+ ASSERT_EQ(2, txn->GetNumKeys());
+
+ s = txn->Commit();
+ ASSERT_OK(s); // Txn should commit, but only write foo2 and foo3
+
+ // Verify that transaction wrote foo2 and foo3 but not foo
+ db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "barz");
+
+ db->Get(read_options, "foo2", &value);
+ ASSERT_EQ(value, "X");
+
+ db->Get(read_options, "foo3", &value);
+ ASSERT_EQ(value, "Y");
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, ReadConflictTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ TransactionOptions txn_options;
+ std::string value;
+ Status s;
+
+ db->Put(write_options, "foo", "bar");
+ db->Put(write_options, "foo2", "bar");
+
+ txn_options.set_snapshot = true;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ // This Put outside of a transaction will conflict with the previous read
+ s = db->Put(write_options, "foo", "barz");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ s = txn->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, TxnOnlyTest) {
+ // Test to make sure transactions work when there are no other writes in an
+ // empty db.
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->Put("x", "y");
+ ASSERT_OK(s);
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, FlushTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ std::string value;
+ Status s;
+
+ db->Put(write_options, Slice("foo"), Slice("bar"));
+ db->Put(write_options, Slice("foo2"), Slice("bar"));
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ s = txn->Put(Slice("foo"), Slice("bar2"));
+ ASSERT_OK(s);
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ // Put a random key so we have a memtable to flush
+ s = db->Put(write_options, "dummy", "dummy");
+ ASSERT_OK(s);
+
+ // force a memtable flush
+ FlushOptions flush_ops;
+ db->Flush(flush_ops);
+
+ s = txn->Commit();
+ // txn should commit since the flushed table is still in MemtableList History
+ ASSERT_OK(s);
+
+ db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, FlushTest2) {
+ const size_t num_tests = 3;
+
+ for (size_t n = 0; n < num_tests; n++) {
+ // Test different table factories
+ switch (n) {
+ case 0:
+ break;
+ case 1:
+ options.table_factory.reset(new mock::MockTableFactory());
+ break;
+ case 2: {
+ PlainTableOptions pt_opts;
+ pt_opts.hash_table_ratio = 0;
+ options.table_factory.reset(NewPlainTableFactory(pt_opts));
+ break;
+ }
+ }
+
+ Status s = ReOpen();
+ ASSERT_OK(s);
+ assert(db != nullptr);
+
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ TransactionOptions txn_options;
+ string value;
+
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+
+ db->Put(write_options, Slice("foo"), Slice("bar"));
+ db->Put(write_options, Slice("foo2"), Slice("bar2"));
+ db->Put(write_options, Slice("foo3"), Slice("bar3"));
+
+ txn_options.set_snapshot = true;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar");
+
+ s = txn->Put(Slice("foo"), Slice("bar2"));
+ ASSERT_OK(s);
+
+ txn->GetForUpdate(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+ // verify foo is locked by txn
+ s = db->Delete(write_options, "foo");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = db->Put(write_options, "Z", "z");
+ ASSERT_OK(s);
+ s = db->Put(write_options, "dummy", "dummy");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "S", "s");
+ ASSERT_OK(s);
+ s = db->SingleDelete(write_options, "S");
+ ASSERT_OK(s);
+
+ s = txn->Delete("S");
+ // Should fail after encountering a write to S in memtable
+ ASSERT_TRUE(s.IsBusy());
+
+ // force a memtable flush
+ s = db_impl->TEST_FlushMemTable(true);
+ ASSERT_OK(s);
+
+ // Put a random key so we have a MemTable to flush
+ s = db->Put(write_options, "dummy", "dummy2");
+ ASSERT_OK(s);
+
+ // force a memtable flush
+ ASSERT_OK(db_impl->TEST_FlushMemTable(true));
+
+ s = db->Put(write_options, "dummy", "dummy3");
+ ASSERT_OK(s);
+
+ // force a memtable flush
+ // Since our test db has max_write_buffer_number=2, this flush will cause
+ // the first memtable to get purged from the MemtableList history.
+ ASSERT_OK(db_impl->TEST_FlushMemTable(true));
+
+ s = txn->Put("X", "Y");
+ // Should succeed after verifying there is no write to X in SST file
+ ASSERT_OK(s);
+
+ s = txn->Put("Z", "zz");
+ // Should fail after encountering a write to Z in SST file
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn->GetForUpdate(read_options, "foo2", &value);
+ // should succeed since key was written before txn started
+ ASSERT_OK(s);
+ // verify foo2 is locked by txn
+ s = db->Delete(write_options, "foo2");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = txn->Delete("S");
+ // Should fail after encountering a write to S in SST file
+ ASSERT_TRUE(s.IsBusy());
+
+ // Write a bunch of keys to db to force a compaction
+ Random rnd(47);
+ for (int i = 0; i < 1000; i++) {
+ s = db->Put(write_options, std::to_string(i),
+ test::CompressibleString(&rnd, 0.8, 100, &value));
+ ASSERT_OK(s);
+ }
+
+ s = txn->Put("X", "yy");
+ // Should succeed after verifying there is no write to X in SST file
+ ASSERT_OK(s);
+
+ s = txn->Put("Z", "zzz");
+ // Should fail after encountering a write to Z in SST file
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn->Delete("S");
+ // Should fail after encountering a write to S in SST file
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn->GetForUpdate(read_options, "foo3", &value);
+ // should succeed since key was written before txn started
+ ASSERT_OK(s);
+ // verify foo3 is locked by txn
+ s = db->Delete(write_options, "foo3");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ db_impl->TEST_WaitForCompact();
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ // Transaction should only write the keys that succeeded.
+ s = db->Get(read_options, "foo", &value);
+ ASSERT_EQ(value, "bar2");
+
+ s = db->Get(read_options, "X", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("yy", value);
+
+ s = db->Get(read_options, "Z", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("z", value);
+
+ delete txn;
+ }
+}
+
+TEST_P(TransactionTest, NoSnapshotTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ db->Put(write_options, "AAA", "bar");
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ // Modify key after transaction start
+ db->Put(write_options, "AAA", "bar1");
+
+ // Read and write without a snap
+ txn->GetForUpdate(read_options, "AAA", &value);
+ ASSERT_EQ(value, "bar1");
+ s = txn->Put("AAA", "bar2");
+ ASSERT_OK(s);
+
+ // Should commit since read/write was done after data changed
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ txn->GetForUpdate(read_options, "AAA", &value);
+ ASSERT_EQ(value, "bar2");
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, MultipleSnapshotTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ std::string value;
+ Status s;
+
+ ASSERT_OK(db->Put(write_options, "AAA", "bar"));
+ ASSERT_OK(db->Put(write_options, "BBB", "bar"));
+ ASSERT_OK(db->Put(write_options, "CCC", "bar"));
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ db->Put(write_options, "AAA", "bar1");
+
+ // Read and write without a snapshot
+ ASSERT_OK(txn->GetForUpdate(read_options, "AAA", &value));
+ ASSERT_EQ(value, "bar1");
+ s = txn->Put("AAA", "bar2");
+ ASSERT_OK(s);
+
+ // Modify BBB before snapshot is taken
+ ASSERT_OK(db->Put(write_options, "BBB", "bar1"));
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ // Read and write with snapshot
+ ASSERT_OK(txn->GetForUpdate(snapshot_read_options, "BBB", &value));
+ ASSERT_EQ(value, "bar1");
+ s = txn->Put("BBB", "bar2");
+ ASSERT_OK(s);
+
+ ASSERT_OK(db->Put(write_options, "CCC", "bar1"));
+
+ // Set a new snapshot
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ // Read and write with snapshot
+ txn->GetForUpdate(snapshot_read_options, "CCC", &value);
+ ASSERT_EQ(value, "bar1");
+ s = txn->Put("CCC", "bar2");
+ ASSERT_OK(s);
+
+ s = txn->GetForUpdate(read_options, "AAA", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = txn->GetForUpdate(read_options, "BBB", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = txn->GetForUpdate(read_options, "CCC", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+
+ s = db->Get(read_options, "AAA", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar1");
+ s = db->Get(read_options, "BBB", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar1");
+ s = db->Get(read_options, "CCC", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar1");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "AAA", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = db->Get(read_options, "BBB", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+ s = db->Get(read_options, "CCC", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "bar2");
+
+ // verify that we track multiple writes to the same key at different snapshots
+ delete txn;
+ txn = db->BeginTransaction(write_options);
+
+ // Potentially conflicting writes
+ db->Put(write_options, "ZZZ", "zzz");
+ db->Put(write_options, "XXX", "xxx");
+
+ txn->SetSnapshot();
+
+ TransactionOptions txn_options;
+ txn_options.set_snapshot = true;
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ txn2->SetSnapshot();
+
+ // This should not conflict in txn since the snapshot is later than the
+ // previous write (spoiler alert: it will later conflict with txn2).
+ s = txn->Put("ZZZ", "zzzz");
+ ASSERT_OK(s);
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+
+ // This will conflict since the snapshot is earlier than another write to ZZZ
+ s = txn2->Put("ZZZ", "xxxxx");
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "ZZZ", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "zzzz");
+
+ delete txn2;
+}
+
+TEST_P(TransactionTest, ColumnFamiliesTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ TransactionOptions txn_options;
+ string value;
+ Status s;
+
+ ColumnFamilyHandle *cfa, *cfb;
+ ColumnFamilyOptions cf_options;
+
+ // Create 2 new column families
+ s = db->CreateColumnFamily(cf_options, "CFA", &cfa);
+ ASSERT_OK(s);
+ s = db->CreateColumnFamily(cf_options, "CFB", &cfb);
+ ASSERT_OK(s);
+
+ delete cfa;
+ delete cfb;
+ delete db;
+ db = nullptr;
+
+ // open DB with three column families
+ std::vector<ColumnFamilyDescriptor> column_families;
+ // have to open default column family
+ column_families.push_back(
+ ColumnFamilyDescriptor(kDefaultColumnFamilyName, ColumnFamilyOptions()));
+ // open the new column families
+ column_families.push_back(
+ ColumnFamilyDescriptor("CFA", ColumnFamilyOptions()));
+ column_families.push_back(
+ ColumnFamilyDescriptor("CFB", ColumnFamilyOptions()));
+
+ std::vector<ColumnFamilyHandle*> handles;
+
+ ASSERT_OK(ReOpenNoDelete(column_families, &handles));
+ assert(db != nullptr);
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn_options.set_snapshot = true;
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ // Write some data to the db
+ WriteBatch batch;
+ batch.Put("foo", "foo");
+ batch.Put(handles[1], "AAA", "bar");
+ batch.Put(handles[1], "AAAZZZ", "bar");
+ s = db->Write(write_options, &batch);
+ ASSERT_OK(s);
+ db->Delete(write_options, handles[1], "AAAZZZ");
+
+ // These keys do not conflict with existing writes since they're in
+ // different column families
+ s = txn->Delete("AAA");
+ ASSERT_OK(s);
+ s = txn->GetForUpdate(snapshot_read_options, handles[1], "foo", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ Slice key_slice("AAAZZZ");
+ Slice value_slices[2] = {Slice("bar"), Slice("bar")};
+ s = txn->Put(handles[2], SliceParts(&key_slice, 1),
+ SliceParts(value_slices, 2));
+ ASSERT_OK(s);
+ ASSERT_EQ(3, txn->GetNumKeys());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ s = db->Get(read_options, "AAA", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = db->Get(read_options, handles[2], "AAAZZZ", &value);
+ ASSERT_EQ(value, "barbar");
+
+ Slice key_slices[3] = {Slice("AAA"), Slice("ZZ"), Slice("Z")};
+ Slice value_slice("barbarbar");
+
+ s = txn2->Delete(handles[2], "XXX");
+ ASSERT_OK(s);
+ s = txn2->Delete(handles[1], "XXX");
+ ASSERT_OK(s);
+
+ // This write will cause a conflict with the earlier batch write
+ s = txn2->Put(handles[1], SliceParts(key_slices, 3),
+ SliceParts(&value_slice, 1));
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ // In the above the latest change to AAAZZZ in handles[1] is delete.
+ s = db->Get(read_options, handles[1], "AAAZZZ", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+ delete txn2;
+
+ txn = db->BeginTransaction(write_options, txn_options);
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ std::vector<ColumnFamilyHandle*> multiget_cfh = {handles[1], handles[2],
+ handles[0], handles[2]};
+ std::vector<Slice> multiget_keys = {"AAA", "AAAZZZ", "foo", "foo"};
+ std::vector<std::string> values(4);
+ std::vector<Status> results = txn->MultiGetForUpdate(
+ snapshot_read_options, multiget_cfh, multiget_keys, &values);
+ ASSERT_OK(results[0]);
+ ASSERT_OK(results[1]);
+ ASSERT_OK(results[2]);
+ ASSERT_TRUE(results[3].IsNotFound());
+ ASSERT_EQ(values[0], "bar");
+ ASSERT_EQ(values[1], "barbar");
+ ASSERT_EQ(values[2], "foo");
+
+ s = txn->SingleDelete(handles[2], "ZZZ");
+ ASSERT_OK(s);
+ s = txn->Put(handles[2], "ZZZ", "YYY");
+ ASSERT_OK(s);
+ s = txn->Put(handles[2], "ZZZ", "YYYY");
+ ASSERT_OK(s);
+ s = txn->Delete(handles[2], "ZZZ");
+ ASSERT_OK(s);
+ s = txn->Put(handles[2], "AAAZZZ", "barbarbar");
+ ASSERT_OK(s);
+
+ ASSERT_EQ(5, txn->GetNumKeys());
+
+ // Txn should commit
+ s = txn->Commit();
+ ASSERT_OK(s);
+ s = db->Get(read_options, handles[2], "ZZZ", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Put a key which will conflict with the next txn using the previous snapshot
+ db->Put(write_options, handles[2], "foo", "000");
+
+ results = txn2->MultiGetForUpdate(snapshot_read_options, multiget_cfh,
+ multiget_keys, &values);
+ // All results should fail since there was a conflict
+ ASSERT_TRUE(results[0].IsBusy());
+ ASSERT_TRUE(results[1].IsBusy());
+ ASSERT_TRUE(results[2].IsBusy());
+ ASSERT_TRUE(results[3].IsBusy());
+
+ s = db->Get(read_options, handles[2], "foo", &value);
+ ASSERT_EQ(value, "000");
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->DropColumnFamily(handles[1]);
+ ASSERT_OK(s);
+ s = db->DropColumnFamily(handles[2]);
+ ASSERT_OK(s);
+
+ delete txn;
+ delete txn2;
+
+ for (auto handle : handles) {
+ delete handle;
+ }
+}
+
+TEST_P(TransactionTest, MultiGetBatchedTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ TransactionOptions txn_options;
+ string value;
+ Status s;
+
+ ColumnFamilyHandle* cf;
+ ColumnFamilyOptions cf_options;
+
+ // Create a new column families
+ s = db->CreateColumnFamily(cf_options, "CF", &cf);
+ ASSERT_OK(s);
+
+ delete cf;
+ delete db;
+ db = nullptr;
+
+ // open DB with three column families
+ std::vector<ColumnFamilyDescriptor> column_families;
+ // have to open default column family
+ column_families.push_back(
+ ColumnFamilyDescriptor(kDefaultColumnFamilyName, ColumnFamilyOptions()));
+ // open the new column families
+ cf_options.merge_operator = MergeOperators::CreateStringAppendOperator();
+ column_families.push_back(ColumnFamilyDescriptor("CF", cf_options));
+
+ std::vector<ColumnFamilyHandle*> handles;
+
+ options.merge_operator = MergeOperators::CreateStringAppendOperator();
+ ASSERT_OK(ReOpenNoDelete(column_families, &handles));
+ assert(db != nullptr);
+
+ // Write some data to the db
+ WriteBatch batch;
+ batch.Put(handles[1], "aaa", "val1");
+ batch.Put(handles[1], "bbb", "val2");
+ batch.Put(handles[1], "ccc", "val3");
+ batch.Put(handles[1], "ddd", "foo");
+ batch.Put(handles[1], "eee", "val5");
+ batch.Put(handles[1], "fff", "val6");
+ batch.Merge(handles[1], "ggg", "foo");
+ s = db->Write(write_options, &batch);
+ ASSERT_OK(s);
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+
+ txn_options.set_snapshot = true;
+ // Write some data to the db
+ s = txn->Delete(handles[1], "bbb");
+ ASSERT_OK(s);
+ s = txn->Put(handles[1], "ccc", "val3_new");
+ ASSERT_OK(s);
+ s = txn->Merge(handles[1], "ddd", "bar");
+ ASSERT_OK(s);
+
+ std::vector<Slice> keys = {"aaa", "bbb", "ccc", "ddd", "eee", "fff", "ggg"};
+ std::vector<PinnableSlice> values(keys.size());
+ std::vector<Status> statuses(keys.size());
+
+ txn->MultiGet(snapshot_read_options, handles[1], keys.size(), keys.data(),
+ values.data(), statuses.data());
+ ASSERT_TRUE(statuses[0].ok());
+ ASSERT_EQ(values[0], "val1");
+ ASSERT_TRUE(statuses[1].IsNotFound());
+ ASSERT_TRUE(statuses[2].ok());
+ ASSERT_EQ(values[2], "val3_new");
+ ASSERT_TRUE(statuses[3].IsMergeInProgress());
+ ASSERT_TRUE(statuses[4].ok());
+ ASSERT_EQ(values[4], "val5");
+ ASSERT_TRUE(statuses[5].ok());
+ ASSERT_EQ(values[5], "val6");
+ ASSERT_TRUE(statuses[6].ok());
+ ASSERT_EQ(values[6], "foo");
+ delete txn;
+ for (auto handle : handles) {
+ delete handle;
+ }
+}
+
+// This test calls WriteBatchWithIndex::MultiGetFromBatchAndDB with a large
+// number of keys, i.e greater than MultiGetContext::MAX_BATCH_SIZE, which is
+// is 32. This forces autovector allocations in the MultiGet code paths
+// to use std::vector in addition to stack allocations. The MultiGet keys
+// includes Merges, which are handled specially in MultiGetFromBatchAndDB by
+// allocating an autovector of MergeContexts
+TEST_P(TransactionTest, MultiGetLargeBatchedTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ string value;
+ Status s;
+
+ ColumnFamilyHandle* cf;
+ ColumnFamilyOptions cf_options;
+
+ std::vector<std::string> key_str;
+ for (int i = 0; i < 100; ++i) {
+ key_str.emplace_back(std::to_string(i));
+ }
+ // Create a new column families
+ s = db->CreateColumnFamily(cf_options, "CF", &cf);
+ ASSERT_OK(s);
+
+ delete cf;
+ delete db;
+ db = nullptr;
+
+ // open DB with three column families
+ std::vector<ColumnFamilyDescriptor> column_families;
+ // have to open default column family
+ column_families.push_back(
+ ColumnFamilyDescriptor(kDefaultColumnFamilyName, ColumnFamilyOptions()));
+ // open the new column families
+ cf_options.merge_operator = MergeOperators::CreateStringAppendOperator();
+ column_families.push_back(ColumnFamilyDescriptor("CF", cf_options));
+
+ std::vector<ColumnFamilyHandle*> handles;
+
+ options.merge_operator = MergeOperators::CreateStringAppendOperator();
+ ASSERT_OK(ReOpenNoDelete(column_families, &handles));
+ assert(db != nullptr);
+
+ // Write some data to the db
+ WriteBatch batch;
+ for (int i = 0; i < 3 * MultiGetContext::MAX_BATCH_SIZE; ++i) {
+ std::string val = "val" + std::to_string(i);
+ batch.Put(handles[1], key_str[i], val);
+ }
+ s = db->Write(write_options, &batch);
+ ASSERT_OK(s);
+
+ WriteBatchWithIndex wb;
+ // Write some data to the db
+ s = wb.Delete(handles[1], std::to_string(1));
+ ASSERT_OK(s);
+ s = wb.Put(handles[1], std::to_string(2), "new_val" + std::to_string(2));
+ ASSERT_OK(s);
+ // Write a lot of merges so when we call MultiGetFromBatchAndDB later on,
+ // it is forced to use std::vector in ROCKSDB_NAMESPACE::autovector to
+ // allocate MergeContexts. The number of merges needs to be >
+ // MultiGetContext::MAX_BATCH_SIZE
+ for (int i = 8; i < MultiGetContext::MAX_BATCH_SIZE + 24; ++i) {
+ s = wb.Merge(handles[1], std::to_string(i), "merge");
+ ASSERT_OK(s);
+ }
+
+ // MultiGet a lot of keys in order to force std::vector reallocations
+ std::vector<Slice> keys;
+ for (int i = 0; i < MultiGetContext::MAX_BATCH_SIZE + 32; ++i) {
+ keys.emplace_back(key_str[i]);
+ }
+ std::vector<PinnableSlice> values(keys.size());
+ std::vector<Status> statuses(keys.size());
+
+ wb.MultiGetFromBatchAndDB(db, snapshot_read_options, handles[1], keys.size(), keys.data(),
+ values.data(), statuses.data(), false);
+ for (size_t i =0; i < keys.size(); ++i) {
+ if (i == 1) {
+ ASSERT_TRUE(statuses[1].IsNotFound());
+ } else if (i == 2) {
+ ASSERT_TRUE(statuses[2].ok());
+ ASSERT_EQ(values[2], "new_val" + std::to_string(2));
+ } else if (i >= 8 && i < 56) {
+ ASSERT_TRUE(statuses[i].ok());
+ ASSERT_EQ(values[i], "val" + std::to_string(i) + ",merge");
+ } else {
+ ASSERT_TRUE(statuses[i].ok());
+ if (values[i] != "val" + std::to_string(i)) {
+ ASSERT_EQ(values[i], "val" + std::to_string(i));
+ }
+ }
+ }
+
+ for (auto handle : handles) {
+ delete handle;
+ }
+}
+
+TEST_P(TransactionTest, ColumnFamiliesTest2) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ string value;
+ Status s;
+
+ ColumnFamilyHandle *one, *two;
+ ColumnFamilyOptions cf_options;
+
+ // Create 2 new column families
+ s = db->CreateColumnFamily(cf_options, "ONE", &one);
+ ASSERT_OK(s);
+ s = db->CreateColumnFamily(cf_options, "TWO", &two);
+ ASSERT_OK(s);
+
+ Transaction* txn1 = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn1);
+ Transaction* txn2 = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn2);
+
+ s = txn1->Put(one, "X", "1");
+ ASSERT_OK(s);
+ s = txn1->Put(two, "X", "2");
+ ASSERT_OK(s);
+ s = txn1->Put("X", "0");
+ ASSERT_OK(s);
+
+ s = txn2->Put(one, "X", "11");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ // Drop first column family
+ s = db->DropColumnFamily(one);
+ ASSERT_OK(s);
+
+ // Should fail since column family was dropped.
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+ txn1 = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn1);
+
+ // Should fail since column family was dropped
+ s = txn1->Put(one, "X", "111");
+ ASSERT_TRUE(s.IsInvalidArgument());
+
+ s = txn1->Put(two, "X", "222");
+ ASSERT_OK(s);
+
+ s = txn1->Put("X", "000");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, two, "X", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("222", value);
+
+ s = db->Get(read_options, "X", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("000", value);
+
+ s = db->DropColumnFamily(two);
+ ASSERT_OK(s);
+
+ delete txn1;
+ delete txn2;
+
+ delete one;
+ delete two;
+}
+
+TEST_P(TransactionTest, EmptyTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ string value;
+ Status s;
+
+ s = db->Put(write_options, "aaa", "aaa");
+ ASSERT_OK(s);
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ txn = db->BeginTransaction(write_options);
+ txn->Rollback();
+ delete txn;
+
+ txn = db->BeginTransaction(write_options);
+ s = txn->GetForUpdate(read_options, "aaa", &value);
+ ASSERT_EQ(value, "aaa");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ txn = db->BeginTransaction(write_options);
+ txn->SetSnapshot();
+
+ s = txn->GetForUpdate(read_options, "aaa", &value);
+ ASSERT_EQ(value, "aaa");
+
+ // Conflicts with previous GetForUpdate
+ s = db->Put(write_options, "aaa", "xxx");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ // transaction expired!
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+}
+
+TEST_P(TransactionTest, PredicateManyPreceders) {
+ WriteOptions write_options;
+ ReadOptions read_options1, read_options2;
+ TransactionOptions txn_options;
+ string value;
+ Status s;
+
+ txn_options.set_snapshot = true;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ Transaction* txn2 = db->BeginTransaction(write_options);
+ txn2->SetSnapshot();
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ std::vector<Slice> multiget_keys = {"1", "2", "3"};
+ std::vector<std::string> multiget_values;
+
+ std::vector<Status> results =
+ txn1->MultiGetForUpdate(read_options1, multiget_keys, &multiget_values);
+ ASSERT_TRUE(results[1].IsNotFound());
+
+ s = txn2->Put("2", "x"); // Conflict's with txn1's MultiGetForUpdate
+ ASSERT_TRUE(s.IsTimedOut());
+
+ txn2->Rollback();
+
+ multiget_values.clear();
+ results =
+ txn1->MultiGetForUpdate(read_options1, multiget_keys, &multiget_values);
+ ASSERT_TRUE(results[1].IsNotFound());
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ s = txn1->Put("4", "x");
+ ASSERT_OK(s);
+
+ s = txn2->Delete("4"); // conflict
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->GetForUpdate(read_options2, "4", &value);
+ ASSERT_TRUE(s.IsBusy());
+
+ txn2->Rollback();
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, LostUpdate) {
+ WriteOptions write_options;
+ ReadOptions read_options, read_options1, read_options2;
+ TransactionOptions txn_options;
+ std::string value;
+ Status s;
+
+ // Test 2 transactions writing to the same key in multiple orders and
+ // with/without snapshots
+
+ Transaction* txn1 = db->BeginTransaction(write_options);
+ Transaction* txn2 = db->BeginTransaction(write_options);
+
+ s = txn1->Put("1", "1");
+ ASSERT_OK(s);
+
+ s = txn2->Put("1", "2"); // conflict
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "1", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("1", value);
+
+ delete txn1;
+ delete txn2;
+
+ txn_options.set_snapshot = true;
+ txn1 = db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ s = txn1->Put("1", "3");
+ ASSERT_OK(s);
+ s = txn2->Put("1", "4"); // conflict
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "1", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("3", value);
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ s = txn1->Put("1", "5");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->Put("1", "6");
+ ASSERT_TRUE(s.IsBusy());
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "1", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("5", value);
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = db->BeginTransaction(write_options, txn_options);
+ read_options1.snapshot = txn1->GetSnapshot();
+
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ read_options2.snapshot = txn2->GetSnapshot();
+
+ s = txn1->Put("1", "7");
+ ASSERT_OK(s);
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ txn2->SetSnapshot();
+ s = txn2->Put("1", "8");
+ ASSERT_OK(s);
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "1", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("8", value);
+
+ delete txn1;
+ delete txn2;
+
+ txn1 = db->BeginTransaction(write_options);
+ txn2 = db->BeginTransaction(write_options);
+
+ s = txn1->Put("1", "9");
+ ASSERT_OK(s);
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->Put("1", "10");
+ ASSERT_OK(s);
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+ delete txn2;
+
+ s = db->Get(read_options, "1", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "10");
+}
+
+TEST_P(TransactionTest, UntrackedWrites) {
+ if (txn_db_options.write_policy == WRITE_UNPREPARED) {
+ // TODO(lth): For WriteUnprepared, validate that untracked writes are
+ // not supported.
+ return;
+ }
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ // Verify transaction rollback works for untracked keys.
+ Transaction* txn = db->BeginTransaction(write_options);
+ txn->SetSnapshot();
+
+ s = txn->PutUntracked("untracked", "0");
+ ASSERT_OK(s);
+ txn->Rollback();
+ s = db->Get(read_options, "untracked", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+ txn = db->BeginTransaction(write_options);
+ txn->SetSnapshot();
+
+ s = db->Put(write_options, "untracked", "x");
+ ASSERT_OK(s);
+
+ // Untracked writes should succeed even though key was written after snapshot
+ s = txn->PutUntracked("untracked", "1");
+ ASSERT_OK(s);
+ s = txn->MergeUntracked("untracked", "2");
+ ASSERT_OK(s);
+ s = txn->DeleteUntracked("untracked");
+ ASSERT_OK(s);
+
+ // Conflict
+ s = txn->Put("untracked", "3");
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "untracked", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, ExpiredTransaction) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ string value;
+ Status s;
+
+ // Set txn expiration timeout to 0 microseconds (expires instantly)
+ txn_options.expiration = 0;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+
+ s = txn1->Put("X", "1");
+ ASSERT_OK(s);
+
+ s = txn1->Put("Y", "1");
+ ASSERT_OK(s);
+
+ Transaction* txn2 = db->BeginTransaction(write_options);
+
+ // txn2 should be able to write to X since txn1 has expired
+ s = txn2->Put("X", "2");
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ s = db->Get(read_options, "X", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("2", value);
+
+ s = txn1->Put("Z", "1");
+ ASSERT_OK(s);
+
+ // txn1 should fail to commit since it is expired
+ s = txn1->Commit();
+ ASSERT_TRUE(s.IsExpired());
+
+ s = db->Get(read_options, "Y", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = db->Get(read_options, "Z", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, ReinitializeTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ std::string value;
+ Status s;
+
+ // Set txn expiration timeout to 0 microseconds (expires instantly)
+ txn_options.expiration = 0;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+
+ // Reinitialize transaction to no long expire
+ txn_options.expiration = -1;
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+
+ s = txn1->Put("Z", "z");
+ ASSERT_OK(s);
+
+ // Should commit since not expired
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+
+ s = txn1->Put("Z", "zz");
+ ASSERT_OK(s);
+
+ // Reinitilize txn1 and verify that Z gets unlocked
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options, nullptr);
+ s = txn2->Put("Z", "zzz");
+ ASSERT_OK(s);
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ s = db->Get(read_options, "Z", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "zzz");
+
+ // Verify snapshots get reinitialized correctly
+ txn1->SetSnapshot();
+ s = txn1->Put("Z", "zzzz");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "Z", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "zzzz");
+
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+ const Snapshot* snapshot = txn1->GetSnapshot();
+ ASSERT_FALSE(snapshot);
+
+ txn_options.set_snapshot = true;
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+ snapshot = txn1->GetSnapshot();
+ ASSERT_TRUE(snapshot);
+
+ s = txn1->Put("Z", "a");
+ ASSERT_OK(s);
+
+ txn1->Rollback();
+
+ s = txn1->Put("Y", "y");
+ ASSERT_OK(s);
+
+ txn_options.set_snapshot = false;
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+ snapshot = txn1->GetSnapshot();
+ ASSERT_FALSE(snapshot);
+
+ s = txn1->Put("X", "x");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "Z", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ(value, "zzzz");
+
+ s = db->Get(read_options, "Y", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+
+ s = txn1->SetName("name");
+ ASSERT_OK(s);
+
+ s = txn1->Prepare();
+ ASSERT_OK(s);
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ txn1 = db->BeginTransaction(write_options, txn_options, txn1);
+
+ s = txn1->SetName("name");
+ ASSERT_OK(s);
+
+ delete txn1;
+}
+
+TEST_P(TransactionTest, Rollback) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+
+ ASSERT_OK(s);
+
+ s = txn1->Put("X", "1");
+ ASSERT_OK(s);
+
+ Transaction* txn2 = db->BeginTransaction(write_options);
+
+ // txn2 should not be able to write to X since txn1 has it locked
+ s = txn2->Put("X", "2");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ txn1->Rollback();
+ delete txn1;
+
+ // txn2 should now be able to write to X
+ s = txn2->Put("X", "3");
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "X", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("3", value);
+
+ delete txn2;
+}
+
+TEST_P(TransactionTest, LockLimitTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ TransactionOptions txn_options;
+ string value;
+ Status s;
+
+ delete db;
+ db = nullptr;
+
+ // Open DB with a lock limit of 3
+ txn_db_options.max_num_locks = 3;
+ ASSERT_OK(ReOpen());
+ assert(db != nullptr);
+ ASSERT_OK(s);
+
+ // Create a txn and verify we can only lock up to 3 keys
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->Put("X", "x");
+ ASSERT_OK(s);
+
+ s = txn->Put("Y", "y");
+ ASSERT_OK(s);
+
+ s = txn->Put("Z", "z");
+ ASSERT_OK(s);
+
+ // lock limit reached
+ s = txn->Put("W", "w");
+ ASSERT_TRUE(s.IsBusy());
+
+ // re-locking same key shouldn't put us over the limit
+ s = txn->Put("X", "xx");
+ ASSERT_OK(s);
+
+ s = txn->GetForUpdate(read_options, "W", &value);
+ ASSERT_TRUE(s.IsBusy());
+ s = txn->GetForUpdate(read_options, "V", &value);
+ ASSERT_TRUE(s.IsBusy());
+
+ // re-locking same key shouldn't put us over the limit
+ s = txn->GetForUpdate(read_options, "Y", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("y", value);
+
+ s = txn->Get(read_options, "W", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ // "X" currently locked
+ s = txn2->Put("X", "x");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ // lock limit reached
+ s = txn2->Put("M", "m");
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "X", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("xx", value);
+
+ s = db->Get(read_options, "W", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Committing txn should release its locks and allow txn2 to proceed
+ s = txn2->Put("X", "x2");
+ ASSERT_OK(s);
+
+ s = txn2->Delete("X");
+ ASSERT_OK(s);
+
+ s = txn2->Put("M", "m");
+ ASSERT_OK(s);
+
+ s = txn2->Put("Z", "z2");
+ ASSERT_OK(s);
+
+ // lock limit reached
+ s = txn2->Delete("Y");
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "Z", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("z2", value);
+
+ s = db->Get(read_options, "Y", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("y", value);
+
+ s = db->Get(read_options, "X", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, IteratorTest) {
+ // This test does writes without snapshot validation, and then tries to create
+ // iterator later, which is unsupported in write unprepared.
+ if (txn_db_options.write_policy == WRITE_UNPREPARED) {
+ return;
+ }
+
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ std::string value;
+ Status s;
+
+ // Write some keys to the db
+ s = db->Put(write_options, "A", "a");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "G", "g");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "F", "f");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "C", "c");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "D", "d");
+ ASSERT_OK(s);
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ // Write some keys in a txn
+ s = txn->Put("B", "b");
+ ASSERT_OK(s);
+
+ s = txn->Put("H", "h");
+ ASSERT_OK(s);
+
+ s = txn->Delete("D");
+ ASSERT_OK(s);
+
+ s = txn->Put("E", "e");
+ ASSERT_OK(s);
+
+ txn->SetSnapshot();
+ const Snapshot* snapshot = txn->GetSnapshot();
+
+ // Write some keys to the db after the snapshot
+ s = db->Put(write_options, "BB", "xx");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "C", "xx");
+ ASSERT_OK(s);
+
+ read_options.snapshot = snapshot;
+ Iterator* iter = txn->GetIterator(read_options);
+ ASSERT_OK(iter->status());
+ iter->SeekToFirst();
+
+ // Read all keys via iter and lock them all
+ std::string results[] = {"a", "b", "c", "e", "f", "g", "h"};
+ for (int i = 0; i < 7; i++) {
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(results[i], iter->value().ToString());
+
+ s = txn->GetForUpdate(read_options, iter->key(), nullptr);
+ if (i == 2) {
+ // "C" was modified after txn's snapshot
+ ASSERT_TRUE(s.IsBusy());
+ } else {
+ ASSERT_OK(s);
+ }
+
+ iter->Next();
+ }
+ ASSERT_FALSE(iter->Valid());
+
+ iter->Seek("G");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("g", iter->value().ToString());
+
+ iter->Prev();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("f", iter->value().ToString());
+
+ iter->Seek("D");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("e", iter->value().ToString());
+
+ iter->Seek("C");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("c", iter->value().ToString());
+
+ iter->Next();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("e", iter->value().ToString());
+
+ iter->Seek("");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("a", iter->value().ToString());
+
+ iter->Seek("X");
+ ASSERT_OK(iter->status());
+ ASSERT_FALSE(iter->Valid());
+
+ iter->SeekToLast();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("h", iter->value().ToString());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete iter;
+ delete txn;
+}
+
+TEST_P(TransactionTest, DisableIndexingTest) {
+ // Skip this test for write unprepared. It does not solely rely on WBWI for
+ // read your own writes, so depending on whether batches are flushed or not,
+ // only some writes will be visible.
+ //
+ // Also, write unprepared does not support creating iterators if there has
+ // been txn->Put() without snapshot validation.
+ if (txn_db_options.write_policy == WRITE_UNPREPARED) {
+ return;
+ }
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ txn->DisableIndexing();
+
+ s = txn->Put("B", "b");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ Iterator* iter = txn->GetIterator(read_options);
+ ASSERT_OK(iter->status());
+
+ iter->Seek("B");
+ ASSERT_OK(iter->status());
+ ASSERT_FALSE(iter->Valid());
+
+ s = txn->Delete("A");
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ txn->EnableIndexing();
+
+ s = txn->Put("B", "bb");
+ ASSERT_OK(s);
+
+ iter->Seek("B");
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("bb", iter->value().ToString());
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("bb", value);
+
+ s = txn->Put("A", "aa");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("aa", value);
+
+ delete iter;
+ delete txn;
+}
+
+TEST_P(TransactionTest, SavepointTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ ASSERT_EQ(0, txn->GetNumPuts());
+
+ s = txn->RollbackToSavePoint();
+ ASSERT_TRUE(s.IsNotFound());
+
+ txn->SetSavePoint(); // 1
+
+ ASSERT_OK(txn->RollbackToSavePoint()); // Rollback to beginning of txn
+ s = txn->RollbackToSavePoint();
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Put("B", "b");
+ ASSERT_OK(s);
+
+ ASSERT_EQ(1, txn->GetNumPuts());
+ ASSERT_EQ(0, txn->GetNumDeletes());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b", value);
+
+ delete txn;
+ txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Put("B", "bb");
+ ASSERT_OK(s);
+
+ s = txn->Put("C", "c");
+ ASSERT_OK(s);
+
+ txn->SetSavePoint(); // 2
+
+ s = txn->Delete("B");
+ ASSERT_OK(s);
+
+ s = txn->Put("C", "cc");
+ ASSERT_OK(s);
+
+ s = txn->Put("D", "d");
+ ASSERT_OK(s);
+
+ ASSERT_EQ(5, txn->GetNumPuts());
+ ASSERT_EQ(1, txn->GetNumDeletes());
+
+ ASSERT_OK(txn->RollbackToSavePoint()); // Rollback to 2
+
+ ASSERT_EQ(3, txn->GetNumPuts());
+ ASSERT_EQ(0, txn->GetNumDeletes());
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("bb", value);
+
+ s = txn->Get(read_options, "C", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("c", value);
+
+ s = txn->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Put("E", "e");
+ ASSERT_OK(s);
+
+ ASSERT_EQ(5, txn->GetNumPuts());
+ ASSERT_EQ(0, txn->GetNumDeletes());
+
+ // Rollback to beginning of txn
+ s = txn->RollbackToSavePoint();
+ ASSERT_TRUE(s.IsNotFound());
+ txn->Rollback();
+
+ ASSERT_EQ(0, txn->GetNumPuts());
+ ASSERT_EQ(0, txn->GetNumDeletes());
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b", value);
+
+ s = txn->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "E", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Put("A", "aa");
+ ASSERT_OK(s);
+
+ s = txn->Put("F", "f");
+ ASSERT_OK(s);
+
+ ASSERT_EQ(2, txn->GetNumPuts());
+ ASSERT_EQ(0, txn->GetNumDeletes());
+
+ txn->SetSavePoint(); // 3
+ txn->SetSavePoint(); // 4
+
+ s = txn->Put("G", "g");
+ ASSERT_OK(s);
+
+ s = txn->SingleDelete("F");
+ ASSERT_OK(s);
+
+ s = txn->Delete("B");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("aa", value);
+
+ s = txn->Get(read_options, "F", &value);
+ // According to db.h, doing a SingleDelete on a key that has been
+ // overwritten will have undefinied behavior. So it is unclear what the
+ // result of fetching "F" should be. The current implementation will
+ // return NotFound in this case.
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Get(read_options, "B", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ ASSERT_EQ(3, txn->GetNumPuts());
+ ASSERT_EQ(2, txn->GetNumDeletes());
+
+ ASSERT_OK(txn->RollbackToSavePoint()); // Rollback to 3
+
+ ASSERT_EQ(2, txn->GetNumPuts());
+ ASSERT_EQ(0, txn->GetNumDeletes());
+
+ s = txn->Get(read_options, "F", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("f", value);
+
+ s = txn->Get(read_options, "G", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "F", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("f", value);
+
+ s = db->Get(read_options, "G", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = db->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("aa", value);
+
+ s = db->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b", value);
+
+ s = db->Get(read_options, "C", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = db->Get(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = db->Get(read_options, "E", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, SavepointTest2) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ Status s;
+
+ txn_options.lock_timeout = 1; // 1 ms
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+
+ s = txn1->Put("A", "");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 1
+
+ s = txn1->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn1->Put("C", "c");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 2
+
+ s = txn1->Put("A", "a");
+ ASSERT_OK(s);
+ s = txn1->Put("B", "b");
+ ASSERT_OK(s);
+
+ ASSERT_OK(txn1->RollbackToSavePoint()); // Rollback to 2
+
+ // Verify that "A" and "C" is still locked while "B" is not
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ s = txn2->Put("A", "a2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b2");
+ ASSERT_OK(s);
+
+ s = txn1->Put("A", "aa");
+ ASSERT_OK(s);
+ s = txn1->Put("B", "bb");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ s = txn1->Put("A", "aaa");
+ ASSERT_OK(s);
+ s = txn1->Put("B", "bbb");
+ ASSERT_OK(s);
+ s = txn1->Put("C", "ccc");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 3
+ ASSERT_OK(txn1->RollbackToSavePoint()); // Rollback to 3
+
+ // Verify that "A", "B", "C" are still locked
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ s = txn2->Put("A", "a2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c2");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ ASSERT_OK(txn1->RollbackToSavePoint()); // Rollback to 1
+
+ // Verify that only "A" is locked
+ s = txn2->Put("A", "a3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b3");
+ ASSERT_OK(s);
+ s = txn2->Put("C", "c3po");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+
+ // Verify "A" "C" "B" are no longer locked
+ s = txn2->Put("A", "a4");
+ ASSERT_OK(s);
+ s = txn2->Put("B", "b4");
+ ASSERT_OK(s);
+ s = txn2->Put("C", "c4");
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+}
+
+TEST_P(TransactionTest, SavepointTest3) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ Status s;
+
+ txn_options.lock_timeout = 1; // 1 ms
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+
+ s = txn1->PopSavePoint(); // No SavePoint present
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn1->Put("A", "");
+ ASSERT_OK(s);
+
+ s = txn1->PopSavePoint(); // Still no SavePoint present
+ ASSERT_TRUE(s.IsNotFound());
+
+ txn1->SetSavePoint(); // 1
+
+ s = txn1->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn1->PopSavePoint(); // Remove 1
+ ASSERT_TRUE(txn1->RollbackToSavePoint().IsNotFound());
+
+ // Verify that "A" is still locked
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ s = txn2->Put("A", "a2");
+ ASSERT_TRUE(s.IsTimedOut());
+ delete txn2;
+
+ txn1->SetSavePoint(); // 2
+
+ s = txn1->Put("B", "b");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 3
+
+ s = txn1->Put("B", "b2");
+ ASSERT_OK(s);
+
+ ASSERT_OK(txn1->RollbackToSavePoint()); // Roll back to 2
+
+ s = txn1->PopSavePoint();
+ ASSERT_OK(s);
+
+ s = txn1->PopSavePoint();
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+
+ std::string value;
+
+ // tnx1 should have modified "A" to "a"
+ s = db->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ // tnx1 should have set "B" to just "b"
+ s = db->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b", value);
+
+ s = db->Get(read_options, "C", &value);
+ ASSERT_TRUE(s.IsNotFound());
+}
+
+TEST_P(TransactionTest, SavepointTest4) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ Status s;
+
+ txn_options.lock_timeout = 1; // 1 ms
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+
+ txn1->SetSavePoint(); // 1
+ s = txn1->Put("A", "a");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 2
+ s = txn1->Put("B", "b");
+ ASSERT_OK(s);
+
+ s = txn1->PopSavePoint(); // Remove 2
+ ASSERT_OK(s);
+
+ // Verify that A/B still exists.
+ std::string value;
+ ASSERT_OK(txn1->Get(read_options, "A", &value));
+ ASSERT_EQ("a", value);
+
+ ASSERT_OK(txn1->Get(read_options, "B", &value));
+ ASSERT_EQ("b", value);
+
+ ASSERT_OK(txn1->RollbackToSavePoint()); // Rollback to 1
+
+ // Verify that everything was rolled back.
+ s = txn1->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn1->Get(read_options, "B", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Nothing should be locked
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ s = txn2->Put("A", "");
+ ASSERT_OK(s);
+
+ s = txn2->Put("B", "");
+ ASSERT_OK(s);
+
+ delete txn2;
+ delete txn1;
+}
+
+TEST_P(TransactionTest, UndoGetForUpdateTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ std::string value;
+ Status s;
+
+ txn_options.lock_timeout = 1; // 1 ms
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+
+ txn1->UndoGetForUpdate("A");
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+
+ txn1 = db->BeginTransaction(write_options, txn_options);
+
+ txn1->UndoGetForUpdate("A");
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Verify that A is locked
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ s = txn2->Put("A", "a");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ txn1->UndoGetForUpdate("A");
+
+ // Verify that A is now unlocked
+ s = txn2->Put("A", "a2");
+ ASSERT_OK(s);
+ txn2->Commit();
+ delete txn2;
+ s = db->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a2", value);
+
+ s = txn1->Delete("A");
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn1->Put("B", "b3");
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "B", &value);
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("B");
+
+ // Verify that A and B are still locked
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ s = txn2->Put("A", "a4");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b4");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ txn1->Rollback();
+ delete txn1;
+
+ // Verify that A and B are no longer locked
+ s = txn2->Put("A", "a5");
+ ASSERT_OK(s);
+ s = txn2->Put("B", "b5");
+ ASSERT_OK(s);
+ s = txn2->Commit();
+ delete txn2;
+ ASSERT_OK(s);
+
+ txn1 = db->BeginTransaction(write_options, txn_options);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "C", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "C", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn1->GetForUpdate(read_options, "B", &value);
+ ASSERT_OK(s);
+ s = txn1->Put("B", "b5");
+ s = txn1->GetForUpdate(read_options, "B", &value);
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("B");
+ txn1->UndoGetForUpdate("C");
+ txn1->UndoGetForUpdate("X");
+
+ // Verify A,B,C are locked
+ txn2 = db->BeginTransaction(write_options, txn_options);
+ s = txn2->Put("A", "a6");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Delete("B");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c6");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("X", "x6");
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("B");
+ txn1->UndoGetForUpdate("C");
+ txn1->UndoGetForUpdate("X");
+
+ // Verify A,B are locked and C is not
+ s = txn2->Put("A", "a6");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Delete("B");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c6");
+ ASSERT_OK(s);
+ s = txn2->Put("X", "x6");
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("B");
+ txn1->UndoGetForUpdate("C");
+ txn1->UndoGetForUpdate("X");
+
+ // Verify B is locked and A and C are not
+ s = txn2->Put("A", "a7");
+ ASSERT_OK(s);
+ s = txn2->Delete("B");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c7");
+ ASSERT_OK(s);
+ s = txn2->Put("X", "x7");
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+}
+
+TEST_P(TransactionTest, UndoGetForUpdateTest2) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ std::string value;
+ Status s;
+
+ s = db->Put(write_options, "A", "");
+ ASSERT_OK(s);
+
+ txn_options.lock_timeout = 1; // 1 ms
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn1);
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "B", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn1->Put("F", "f");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 1
+
+ txn1->UndoGetForUpdate("A");
+
+ s = txn1->GetForUpdate(read_options, "C", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn1->GetForUpdate(read_options, "D", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn1->Put("E", "e");
+ ASSERT_OK(s);
+ s = txn1->GetForUpdate(read_options, "E", &value);
+ ASSERT_OK(s);
+
+ s = txn1->GetForUpdate(read_options, "F", &value);
+ ASSERT_OK(s);
+
+ // Verify A,B,C,D,E,F are still locked
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ s = txn2->Put("A", "a1");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b1");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c1");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("D", "d1");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("E", "e1");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("F", "f1");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ txn1->UndoGetForUpdate("C");
+ txn1->UndoGetForUpdate("E");
+
+ // Verify A,B,D,E,F are still locked and C is not.
+ s = txn2->Put("A", "a2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("D", "d2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("E", "e2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("F", "f2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c2");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 2
+
+ s = txn1->Put("H", "h");
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("B");
+ txn1->UndoGetForUpdate("C");
+ txn1->UndoGetForUpdate("D");
+ txn1->UndoGetForUpdate("E");
+ txn1->UndoGetForUpdate("F");
+ txn1->UndoGetForUpdate("G");
+ txn1->UndoGetForUpdate("H");
+
+ // Verify A,B,D,E,F,H are still locked and C,G are not.
+ s = txn2->Put("A", "a3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("D", "d3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("E", "e3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("F", "f3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("H", "h3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c3");
+ ASSERT_OK(s);
+ s = txn2->Put("G", "g3");
+ ASSERT_OK(s);
+
+ txn1->RollbackToSavePoint(); // rollback to 2
+
+ // Verify A,B,D,E,F are still locked and C,G,H are not.
+ s = txn2->Put("A", "a3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("D", "d3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("E", "e3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("F", "f3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c3");
+ ASSERT_OK(s);
+ s = txn2->Put("G", "g3");
+ ASSERT_OK(s);
+ s = txn2->Put("H", "h3");
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("B");
+ txn1->UndoGetForUpdate("C");
+ txn1->UndoGetForUpdate("D");
+ txn1->UndoGetForUpdate("E");
+ txn1->UndoGetForUpdate("F");
+ txn1->UndoGetForUpdate("G");
+ txn1->UndoGetForUpdate("H");
+
+ // Verify A,B,E,F are still locked and C,D,G,H are not.
+ s = txn2->Put("A", "a3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("E", "e3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("F", "f3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c3");
+ ASSERT_OK(s);
+ s = txn2->Put("D", "d3");
+ ASSERT_OK(s);
+ s = txn2->Put("G", "g3");
+ ASSERT_OK(s);
+ s = txn2->Put("H", "h3");
+ ASSERT_OK(s);
+
+ txn1->RollbackToSavePoint(); // rollback to 1
+
+ // Verify A,B,F are still locked and C,D,E,G,H are not.
+ s = txn2->Put("A", "a3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("B", "b3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("F", "f3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("C", "c3");
+ ASSERT_OK(s);
+ s = txn2->Put("D", "d3");
+ ASSERT_OK(s);
+ s = txn2->Put("E", "e3");
+ ASSERT_OK(s);
+ s = txn2->Put("G", "g3");
+ ASSERT_OK(s);
+ s = txn2->Put("H", "h3");
+ ASSERT_OK(s);
+
+ txn1->UndoGetForUpdate("A");
+ txn1->UndoGetForUpdate("B");
+ txn1->UndoGetForUpdate("C");
+ txn1->UndoGetForUpdate("D");
+ txn1->UndoGetForUpdate("E");
+ txn1->UndoGetForUpdate("F");
+ txn1->UndoGetForUpdate("G");
+ txn1->UndoGetForUpdate("H");
+
+ // Verify F is still locked and A,B,C,D,E,G,H are not.
+ s = txn2->Put("F", "f3");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Put("A", "a3");
+ ASSERT_OK(s);
+ s = txn2->Put("B", "b3");
+ ASSERT_OK(s);
+ s = txn2->Put("C", "c3");
+ ASSERT_OK(s);
+ s = txn2->Put("D", "d3");
+ ASSERT_OK(s);
+ s = txn2->Put("E", "e3");
+ ASSERT_OK(s);
+ s = txn2->Put("G", "g3");
+ ASSERT_OK(s);
+ s = txn2->Put("H", "h3");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, TimeoutTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ delete db;
+ db = nullptr;
+
+ // transaction writes have an infinite timeout,
+ // but we will override this when we start a txn
+ // db writes have infinite timeout
+ txn_db_options.transaction_lock_timeout = -1;
+ txn_db_options.default_lock_timeout = -1;
+
+ s = TransactionDB::Open(options, txn_db_options, dbname, &db);
+ assert(db != nullptr);
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "aaa", "aaa");
+ ASSERT_OK(s);
+
+ TransactionOptions txn_options0;
+ txn_options0.expiration = 100; // 100ms
+ txn_options0.lock_timeout = 50; // txn timeout no longer infinite
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options0);
+
+ s = txn1->GetForUpdate(read_options, "aaa", nullptr);
+ ASSERT_OK(s);
+
+ // Conflicts with previous GetForUpdate.
+ // Since db writes do not have a timeout, this should eventually succeed when
+ // the transaction expires.
+ s = db->Put(write_options, "aaa", "xxx");
+ ASSERT_OK(s);
+
+ ASSERT_GE(txn1->GetElapsedTime(),
+ static_cast<uint64_t>(txn_options0.expiration));
+
+ s = txn1->Commit();
+ ASSERT_TRUE(s.IsExpired()); // expired!
+
+ s = db->Get(read_options, "aaa", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("xxx", value);
+
+ delete txn1;
+ delete db;
+
+ // transaction writes have 10ms timeout,
+ // db writes have infinite timeout
+ txn_db_options.transaction_lock_timeout = 50;
+ txn_db_options.default_lock_timeout = -1;
+
+ s = TransactionDB::Open(options, txn_db_options, dbname, &db);
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "aaa", "aaa");
+ ASSERT_OK(s);
+
+ TransactionOptions txn_options;
+ txn_options.expiration = 100; // 100ms
+ txn1 = db->BeginTransaction(write_options, txn_options);
+
+ s = txn1->GetForUpdate(read_options, "aaa", nullptr);
+ ASSERT_OK(s);
+
+ // Conflicts with previous GetForUpdate.
+ // Since db writes do not have a timeout, this should eventually succeed when
+ // the transaction expires.
+ s = db->Put(write_options, "aaa", "xxx");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_NOK(s); // expired!
+
+ s = db->Get(read_options, "aaa", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("xxx", value);
+
+ delete txn1;
+ txn_options.expiration = 6000000; // 100 minutes
+ txn_options.lock_timeout = 1; // 1ms
+ txn1 = db->BeginTransaction(write_options, txn_options);
+ txn1->SetLockTimeout(100);
+
+ TransactionOptions txn_options2;
+ txn_options2.expiration = 10; // 10ms
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options2);
+ ASSERT_OK(s);
+
+ s = txn2->Put("a", "2");
+ ASSERT_OK(s);
+
+ // txn1 has a lock timeout longer than txn2's expiration, so it will win
+ s = txn1->Delete("a");
+ ASSERT_OK(s);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ // txn2 should be expired out since txn1 waiting until its timeout expired.
+ s = txn2->Commit();
+ ASSERT_TRUE(s.IsExpired());
+
+ delete txn1;
+ delete txn2;
+ txn_options.expiration = 6000000; // 100 minutes
+ txn1 = db->BeginTransaction(write_options, txn_options);
+ txn_options2.expiration = 100000000;
+ txn2 = db->BeginTransaction(write_options, txn_options2);
+
+ s = txn1->Delete("asdf");
+ ASSERT_OK(s);
+
+ // txn2 has a smaller lock timeout than txn1's expiration, so it will time out
+ s = txn2->Delete("asdf");
+ ASSERT_TRUE(s.IsTimedOut());
+ ASSERT_EQ(s.ToString(), "Operation timed out: Timeout waiting to lock key");
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ s = txn2->Put("asdf", "asdf");
+ ASSERT_OK(s);
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+
+ s = db->Get(read_options, "asdf", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("asdf", value);
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(TransactionTest, SingleDeleteTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = txn->SingleDelete("A");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ txn = db->BeginTransaction(write_options);
+
+ s = txn->SingleDelete("A");
+ ASSERT_OK(s);
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ s = db->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ txn = db->BeginTransaction(write_options);
+
+ s = txn->SingleDelete("A");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ s = db->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ txn = db->BeginTransaction(write_options);
+ Transaction* txn2 = db->BeginTransaction(write_options);
+ txn2->SetSnapshot();
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Put("A", "a2");
+ ASSERT_OK(s);
+
+ s = txn->SingleDelete("A");
+ ASSERT_OK(s);
+
+ s = txn->SingleDelete("B");
+ ASSERT_OK(s);
+
+ // According to db.h, doing a SingleDelete on a key that has been
+ // overwritten will have undefinied behavior. So it is unclear what the
+ // result of fetching "A" should be. The current implementation will
+ // return NotFound in this case.
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn2->Put("B", "b");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ // According to db.h, doing a SingleDelete on a key that has been
+ // overwritten will have undefinied behavior. So it is unclear what the
+ // result of fetching "A" should be. The current implementation will
+ // return NotFound in this case.
+ s = db->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = db->Get(read_options, "B", &value);
+ ASSERT_TRUE(s.IsNotFound());
+}
+
+TEST_P(TransactionTest, MergeTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(write_options, TransactionOptions());
+ ASSERT_TRUE(txn);
+
+ s = db->Put(write_options, "A", "a0");
+ ASSERT_OK(s);
+
+ s = txn->Merge("A", "1");
+ ASSERT_OK(s);
+
+ s = txn->Merge("A", "2");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsMergeInProgress());
+
+ s = txn->Put("A", "a");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a", value);
+
+ s = txn->Merge("A", "3");
+ ASSERT_OK(s);
+
+ s = txn->Get(read_options, "A", &value);
+ ASSERT_TRUE(s.IsMergeInProgress());
+
+ TransactionOptions txn_options;
+ txn_options.lock_timeout = 1; // 1 ms
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2);
+
+ // verify that txn has "A" locked
+ s = txn2->Merge("A", "4");
+ ASSERT_TRUE(s.IsTimedOut());
+
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+ delete txn;
+
+ s = db->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a,3", value);
+}
+
+TEST_P(TransactionTest, DeferSnapshotTest) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+ Status s;
+
+ s = db->Put(write_options, "A", "a0");
+ ASSERT_OK(s);
+
+ Transaction* txn1 = db->BeginTransaction(write_options);
+ Transaction* txn2 = db->BeginTransaction(write_options);
+
+ txn1->SetSnapshotOnNextOperation();
+ auto snapshot = txn1->GetSnapshot();
+ ASSERT_FALSE(snapshot);
+
+ s = txn2->Put("A", "a2");
+ ASSERT_OK(s);
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+
+ s = txn1->GetForUpdate(read_options, "A", &value);
+ // Should not conflict with txn2 since snapshot wasn't set until
+ // GetForUpdate was called.
+ ASSERT_OK(s);
+ ASSERT_EQ("a2", value);
+
+ s = txn1->Put("A", "a1");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "B", "b0");
+ ASSERT_OK(s);
+
+ // Cannot lock B since it was written after the snapshot was set
+ s = txn1->Put("B", "b1");
+ ASSERT_TRUE(s.IsBusy());
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+
+ s = db->Get(read_options, "A", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("a1", value);
+
+ s = db->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("b0", value);
+}
+
+TEST_P(TransactionTest, DeferSnapshotTest2) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn1 = db->BeginTransaction(write_options);
+
+ txn1->SetSnapshot();
+
+ s = txn1->Put("A", "a1");
+ ASSERT_OK(s);
+
+ s = db->Put(write_options, "C", "c0");
+ ASSERT_OK(s);
+ s = db->Put(write_options, "D", "d0");
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+
+ txn1->SetSnapshotOnNextOperation();
+
+ s = txn1->Get(snapshot_read_options, "C", &value);
+ // Snapshot was set before C was written
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn1->Get(snapshot_read_options, "D", &value);
+ // Snapshot was set before D was written
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Snapshot should not have changed yet.
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+
+ s = txn1->Get(snapshot_read_options, "C", &value);
+ // Snapshot was set before C was written
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn1->Get(snapshot_read_options, "D", &value);
+ // Snapshot was set before D was written
+ ASSERT_TRUE(s.IsNotFound());
+
+ s = txn1->GetForUpdate(read_options, "C", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("c0", value);
+
+ s = db->Put(write_options, "D", "d00");
+ ASSERT_OK(s);
+
+ // Snapshot is now set
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ s = txn1->Get(snapshot_read_options, "D", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("d0", value);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+ delete txn1;
+}
+
+TEST_P(TransactionTest, DeferSnapshotSavePointTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ std::string value;
+ Status s;
+
+ Transaction* txn1 = db->BeginTransaction(write_options);
+
+ txn1->SetSavePoint(); // 1
+
+ s = db->Put(write_options, "T", "1");
+ ASSERT_OK(s);
+
+ txn1->SetSnapshotOnNextOperation();
+
+ s = db->Put(write_options, "T", "2");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 2
+
+ s = db->Put(write_options, "T", "3");
+ ASSERT_OK(s);
+
+ s = txn1->Put("A", "a");
+ ASSERT_OK(s);
+
+ txn1->SetSavePoint(); // 3
+
+ s = db->Put(write_options, "T", "4");
+ ASSERT_OK(s);
+
+ txn1->SetSnapshot();
+ txn1->SetSnapshotOnNextOperation();
+
+ txn1->SetSavePoint(); // 4
+
+ s = db->Put(write_options, "T", "5");
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ s = txn1->Get(snapshot_read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("4", value);
+
+ s = txn1->Put("A", "a1");
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ s = txn1->Get(snapshot_read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("5", value);
+
+ s = txn1->RollbackToSavePoint(); // Rollback to 4
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ s = txn1->Get(snapshot_read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("4", value);
+
+ s = txn1->RollbackToSavePoint(); // Rollback to 3
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ s = txn1->Get(snapshot_read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("3", value);
+
+ s = txn1->Get(read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("5", value);
+
+ s = txn1->RollbackToSavePoint(); // Rollback to 2
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ ASSERT_FALSE(snapshot_read_options.snapshot);
+ s = txn1->Get(snapshot_read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("5", value);
+
+ s = txn1->Delete("A");
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ ASSERT_TRUE(snapshot_read_options.snapshot);
+ s = txn1->Get(snapshot_read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("5", value);
+
+ s = txn1->RollbackToSavePoint(); // Rollback to 1
+ ASSERT_OK(s);
+
+ s = txn1->Delete("A");
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn1->GetSnapshot();
+ ASSERT_FALSE(snapshot_read_options.snapshot);
+ s = txn1->Get(snapshot_read_options, "T", &value);
+ ASSERT_OK(s);
+ ASSERT_EQ("5", value);
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+}
+
+TEST_P(TransactionTest, SetSnapshotOnNextOperationWithNotification) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ std::string value;
+
+ class Notifier : public TransactionNotifier {
+ private:
+ const Snapshot** snapshot_ptr_;
+
+ public:
+ explicit Notifier(const Snapshot** snapshot_ptr)
+ : snapshot_ptr_(snapshot_ptr) {}
+
+ void SnapshotCreated(const Snapshot* newSnapshot) override {
+ *snapshot_ptr_ = newSnapshot;
+ }
+ };
+
+ std::shared_ptr<Notifier> notifier =
+ std::make_shared<Notifier>(&read_options.snapshot);
+ Status s;
+
+ s = db->Put(write_options, "B", "0");
+ ASSERT_OK(s);
+
+ Transaction* txn1 = db->BeginTransaction(write_options);
+
+ txn1->SetSnapshotOnNextOperation(notifier);
+ ASSERT_FALSE(read_options.snapshot);
+
+ s = db->Put(write_options, "B", "1");
+ ASSERT_OK(s);
+
+ // A Get does not generate the snapshot
+ s = txn1->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_FALSE(read_options.snapshot);
+ ASSERT_EQ(value, "1");
+
+ // Any other operation does
+ s = txn1->Put("A", "0");
+ ASSERT_OK(s);
+
+ // Now change "B".
+ s = db->Put(write_options, "B", "2");
+ ASSERT_OK(s);
+
+ // The original value should still be read
+ s = txn1->Get(read_options, "B", &value);
+ ASSERT_OK(s);
+ ASSERT_TRUE(read_options.snapshot);
+ ASSERT_EQ(value, "1");
+
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ delete txn1;
+}
+
+TEST_P(TransactionTest, ClearSnapshotTest) {
+ WriteOptions write_options;
+ ReadOptions read_options, snapshot_read_options;
+ std::string value;
+ Status s;
+
+ s = db->Put(write_options, "foo", "0");
+ ASSERT_OK(s);
+
+ Transaction* txn = db->BeginTransaction(write_options);
+ ASSERT_TRUE(txn);
+
+ s = db->Put(write_options, "foo", "1");
+ ASSERT_OK(s);
+
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+ ASSERT_FALSE(snapshot_read_options.snapshot);
+
+ // No snapshot created yet
+ s = txn->Get(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "1");
+
+ txn->SetSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+ ASSERT_TRUE(snapshot_read_options.snapshot);
+
+ s = db->Put(write_options, "foo", "2");
+ ASSERT_OK(s);
+
+ // Snapshot was created before change to '2'
+ s = txn->Get(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "1");
+
+ txn->ClearSnapshot();
+ snapshot_read_options.snapshot = txn->GetSnapshot();
+ ASSERT_FALSE(snapshot_read_options.snapshot);
+
+ // Snapshot has now been cleared
+ s = txn->Get(snapshot_read_options, "foo", &value);
+ ASSERT_EQ(value, "2");
+
+ s = txn->Commit();
+ ASSERT_OK(s);
+
+ delete txn;
+}
+
+TEST_P(TransactionTest, ToggleAutoCompactionTest) {
+ Status s;
+
+ ColumnFamilyHandle *cfa, *cfb;
+ ColumnFamilyOptions cf_options;
+
+ // Create 2 new column families
+ s = db->CreateColumnFamily(cf_options, "CFA", &cfa);
+ ASSERT_OK(s);
+ s = db->CreateColumnFamily(cf_options, "CFB", &cfb);
+ ASSERT_OK(s);
+
+ delete cfa;
+ delete cfb;
+ delete db;
+
+ // open DB with three column families
+ std::vector<ColumnFamilyDescriptor> column_families;
+ // have to open default column family
+ column_families.push_back(
+ ColumnFamilyDescriptor(kDefaultColumnFamilyName, ColumnFamilyOptions()));
+ // open the new column families
+ column_families.push_back(
+ ColumnFamilyDescriptor("CFA", ColumnFamilyOptions()));
+ column_families.push_back(
+ ColumnFamilyDescriptor("CFB", ColumnFamilyOptions()));
+
+ ColumnFamilyOptions* cf_opt_default = &column_families[0].options;
+ ColumnFamilyOptions* cf_opt_cfa = &column_families[1].options;
+ ColumnFamilyOptions* cf_opt_cfb = &column_families[2].options;
+ cf_opt_default->disable_auto_compactions = false;
+ cf_opt_cfa->disable_auto_compactions = true;
+ cf_opt_cfb->disable_auto_compactions = false;
+
+ std::vector<ColumnFamilyHandle*> handles;
+
+ s = TransactionDB::Open(options, txn_db_options, dbname, column_families,
+ &handles, &db);
+ ASSERT_OK(s);
+
+ auto cfh_default = reinterpret_cast<ColumnFamilyHandleImpl*>(handles[0]);
+ auto opt_default = *cfh_default->cfd()->GetLatestMutableCFOptions();
+
+ auto cfh_a = reinterpret_cast<ColumnFamilyHandleImpl*>(handles[1]);
+ auto opt_a = *cfh_a->cfd()->GetLatestMutableCFOptions();
+
+ auto cfh_b = reinterpret_cast<ColumnFamilyHandleImpl*>(handles[2]);
+ auto opt_b = *cfh_b->cfd()->GetLatestMutableCFOptions();
+
+ ASSERT_EQ(opt_default.disable_auto_compactions, false);
+ ASSERT_EQ(opt_a.disable_auto_compactions, true);
+ ASSERT_EQ(opt_b.disable_auto_compactions, false);
+
+ for (auto handle : handles) {
+ delete handle;
+ }
+}
+
+TEST_P(TransactionStressTest, ExpiredTransactionDataRace1) {
+ // In this test, txn1 should succeed committing,
+ // as the callback is called after txn1 starts committing.
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"TransactionTest::ExpirableTransactionDataRace:1"}});
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "TransactionTest::ExpirableTransactionDataRace:1", [&](void* /*arg*/) {
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+
+ // Force txn1 to expire
+ /* sleep override */
+ std::this_thread::sleep_for(std::chrono::milliseconds(150));
+
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ Status s;
+ s = txn2->Put("X", "2");
+ ASSERT_TRUE(s.IsTimedOut());
+ s = txn2->Commit();
+ ASSERT_OK(s);
+ delete txn2;
+ });
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+
+ txn_options.expiration = 100;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+
+ Status s;
+ s = txn1->Put("X", "1");
+ ASSERT_OK(s);
+ s = txn1->Commit();
+ ASSERT_OK(s);
+
+ ReadOptions read_options;
+ string value;
+ s = db->Get(read_options, "X", &value);
+ ASSERT_EQ("1", value);
+
+ delete txn1;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+}
+
+#ifndef ROCKSDB_VALGRIND_RUN
+namespace {
+// cmt_delay_ms is the delay between prepare and commit
+// first_id is the id of the first transaction
+Status TransactionStressTestInserter(
+ TransactionDB* db, const size_t num_transactions, const size_t num_sets,
+ const size_t num_keys_per_set, Random64* rand,
+ const uint64_t cmt_delay_ms = 0, const uint64_t first_id = 0) {
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ if (rand->OneIn(2)) {
+ txn_options.use_only_the_last_commit_time_batch_for_recovery = true;
+ }
+ // Inside the inserter we might also retake the snapshot. We do both since two
+ // separte functions are engaged for each.
+ txn_options.set_snapshot = rand->OneIn(2);
+
+ RandomTransactionInserter inserter(
+ rand, write_options, read_options, num_keys_per_set,
+ static_cast<uint16_t>(num_sets), cmt_delay_ms, first_id);
+
+ for (size_t t = 0; t < num_transactions; t++) {
+ bool success = inserter.TransactionDBInsert(db, txn_options);
+ if (!success) {
+ // unexpected failure
+ return inserter.GetLastStatus();
+ }
+ }
+
+ // Make sure at least some of the transactions succeeded. It's ok if
+ // some failed due to write-conflicts.
+ if (num_transactions != 1 &&
+ inserter.GetFailureCount() > num_transactions / 2) {
+ return Status::TryAgain("Too many transactions failed! " +
+ std::to_string(inserter.GetFailureCount()) + " / " +
+ std::to_string(num_transactions));
+ }
+
+ return Status::OK();
+}
+} // namespace
+
+// Worker threads add a number to a key from each set of keys. The checker
+// threads verify that the sum of all keys in each set are equal.
+TEST_P(MySQLStyleTransactionTest, TransactionStressTest) {
+ // Small write buffer to trigger more compactions
+ options.write_buffer_size = 1024;
+ ReOpenNoDelete();
+ const size_t num_workers = 4; // worker threads count
+ const size_t num_checkers = 2; // checker threads count
+ const size_t num_slow_checkers = 2; // checker threads emulating backups
+ const size_t num_slow_workers = 1; // slow worker threads count
+ const size_t num_transactions_per_thread = 10000;
+ const uint16_t num_sets = 3;
+ const size_t num_keys_per_set = 100;
+ // Setting the key-space to be 100 keys should cause enough write-conflicts
+ // to make this test interesting.
+
+ std::vector<port::Thread> threads;
+ std::atomic<uint32_t> finished = {0};
+ bool TAKE_SNAPSHOT = true;
+ uint64_t time_seed = env->NowMicros();
+ printf("time_seed is %" PRIu64 "\n", time_seed); // would help to reproduce
+
+ std::function<void()> call_inserter = [&] {
+ size_t thd_seed = std::hash<std::thread::id>()(std::this_thread::get_id());
+ Random64 rand(time_seed * thd_seed);
+ ASSERT_OK(TransactionStressTestInserter(db, num_transactions_per_thread,
+ num_sets, num_keys_per_set, &rand));
+ finished++;
+ };
+ std::function<void()> call_checker = [&] {
+ size_t thd_seed = std::hash<std::thread::id>()(std::this_thread::get_id());
+ Random64 rand(time_seed * thd_seed);
+ // Verify that data is consistent
+ while (finished < num_workers) {
+ Status s = RandomTransactionInserter::Verify(
+ db, num_sets, num_keys_per_set, TAKE_SNAPSHOT, &rand);
+ ASSERT_OK(s);
+ }
+ };
+ std::function<void()> call_slow_checker = [&] {
+ size_t thd_seed = std::hash<std::thread::id>()(std::this_thread::get_id());
+ Random64 rand(time_seed * thd_seed);
+ // Verify that data is consistent
+ while (finished < num_workers) {
+ uint64_t delay_ms = rand.Uniform(100) + 1;
+ Status s = RandomTransactionInserter::Verify(
+ db, num_sets, num_keys_per_set, TAKE_SNAPSHOT, &rand, delay_ms);
+ ASSERT_OK(s);
+ }
+ };
+ std::function<void()> call_slow_inserter = [&] {
+ size_t thd_seed = std::hash<std::thread::id>()(std::this_thread::get_id());
+ Random64 rand(time_seed * thd_seed);
+ uint64_t id = 0;
+ // Verify that data is consistent
+ while (finished < num_workers) {
+ uint64_t delay_ms = rand.Uniform(500) + 1;
+ ASSERT_OK(TransactionStressTestInserter(db, 1, num_sets, num_keys_per_set,
+ &rand, delay_ms, id++));
+ }
+ };
+
+ for (uint32_t i = 0; i < num_workers; i++) {
+ threads.emplace_back(call_inserter);
+ }
+ for (uint32_t i = 0; i < num_checkers; i++) {
+ threads.emplace_back(call_checker);
+ }
+ if (with_slow_threads_) {
+ for (uint32_t i = 0; i < num_slow_checkers; i++) {
+ threads.emplace_back(call_slow_checker);
+ }
+ for (uint32_t i = 0; i < num_slow_workers; i++) {
+ threads.emplace_back(call_slow_inserter);
+ }
+ }
+
+ // Wait for all threads to finish
+ for (auto& t : threads) {
+ t.join();
+ }
+
+ // Verify that data is consistent
+ Status s = RandomTransactionInserter::Verify(db, num_sets, num_keys_per_set,
+ !TAKE_SNAPSHOT);
+ ASSERT_OK(s);
+}
+#endif // ROCKSDB_VALGRIND_RUN
+
+TEST_P(TransactionTest, MemoryLimitTest) {
+ TransactionOptions txn_options;
+ // Header (12 bytes) + NOOP (1 byte) + 2 * 8 bytes for data.
+ txn_options.max_write_batch_size = 29;
+ // Set threshold to unlimited so that the write batch does not get flushed,
+ // and can hit the memory limit.
+ txn_options.write_batch_flush_threshold = 0;
+ std::string value;
+ Status s;
+
+ Transaction* txn = db->BeginTransaction(WriteOptions(), txn_options);
+ ASSERT_TRUE(txn);
+
+ ASSERT_EQ(0, txn->GetNumPuts());
+ ASSERT_LE(0, txn->GetID());
+
+ s = txn->Put(Slice("a"), Slice("...."));
+ ASSERT_OK(s);
+ ASSERT_EQ(1, txn->GetNumPuts());
+
+ s = txn->Put(Slice("b"), Slice("...."));
+ ASSERT_OK(s);
+ ASSERT_EQ(2, txn->GetNumPuts());
+
+ s = txn->Put(Slice("b"), Slice("...."));
+ ASSERT_TRUE(s.IsMemoryLimit());
+ ASSERT_EQ(2, txn->GetNumPuts());
+
+ txn->Rollback();
+ delete txn;
+}
+
+// This test clarifies the existing expectation from the sequence number
+// algorithm. It could detect mistakes in updating the code but it is not
+// necessarily the one acceptable way. If the algorithm is legitimately changed,
+// this unit test should be updated as well.
+TEST_P(TransactionStressTest, SeqAdvanceTest) {
+ // TODO(myabandeh): must be test with false before new releases
+ const bool short_test = true;
+ WriteOptions wopts;
+ FlushOptions fopt;
+
+ options.disable_auto_compactions = true;
+ ASSERT_OK(ReOpen());
+
+ // Do the test with NUM_BRANCHES branches in it. Each run of a test takes some
+ // of the branches. This is the same as counting a binary number where i-th
+ // bit represents whether we take branch i in the represented by the number.
+ const size_t NUM_BRANCHES = short_test ? 6 : 10;
+ // Helper function that shows if the branch is to be taken in the run
+ // represented by the number n.
+ auto branch_do = [&](size_t n, size_t* branch) {
+ assert(*branch < NUM_BRANCHES);
+ const size_t filter = static_cast<size_t>(1) << *branch;
+ return n & filter;
+ };
+ const size_t max_n = static_cast<size_t>(1) << NUM_BRANCHES;
+ for (size_t n = 0; n < max_n; n++) {
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ size_t branch = 0;
+ auto seq = db_impl->GetLatestSequenceNumber();
+ exp_seq = seq;
+ txn_t0(0);
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+
+ if (branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->Flush(fopt));
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+ }
+ if (!short_test && branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->FlushWAL(true));
+ ASSERT_OK(ReOpenNoDelete());
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ seq = db_impl->GetLatestSequenceNumber();
+ ASSERT_EQ(exp_seq, seq);
+ }
+
+ // Doing it twice might detect some bugs
+ txn_t0(1);
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+
+ txn_t1(0);
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+
+ if (branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->Flush(fopt));
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+ }
+ if (!short_test && branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->FlushWAL(true));
+ ASSERT_OK(ReOpenNoDelete());
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ seq = db_impl->GetLatestSequenceNumber();
+ ASSERT_EQ(exp_seq, seq);
+ }
+
+ txn_t3(0);
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+
+ if (branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->Flush(fopt));
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+ }
+ if (!short_test && branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->FlushWAL(true));
+ ASSERT_OK(ReOpenNoDelete());
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ seq = db_impl->GetLatestSequenceNumber();
+ ASSERT_EQ(exp_seq, seq);
+ }
+
+ txn_t4(0);
+ seq = db_impl->TEST_GetLastVisibleSequence();
+
+ ASSERT_EQ(exp_seq, seq);
+
+ if (branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->Flush(fopt));
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+ }
+ if (!short_test && branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->FlushWAL(true));
+ ASSERT_OK(ReOpenNoDelete());
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ seq = db_impl->GetLatestSequenceNumber();
+ ASSERT_EQ(exp_seq, seq);
+ }
+
+ txn_t2(0);
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+
+ if (branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->Flush(fopt));
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+ }
+ if (!short_test && branch_do(n, &branch)) {
+ ASSERT_OK(db_impl->FlushWAL(true));
+ ASSERT_OK(ReOpenNoDelete());
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ seq = db_impl->GetLatestSequenceNumber();
+ ASSERT_EQ(exp_seq, seq);
+ }
+ ASSERT_OK(ReOpen());
+ }
+}
+
+// Verify that the optimization would not compromize the correctness
+TEST_P(TransactionTest, Optimizations) {
+ size_t comb_cnt = size_t(1) << 2; // 2 is number of optimization vars
+ for (size_t new_comb = 0; new_comb < comb_cnt; new_comb++) {
+ TransactionDBWriteOptimizations optimizations;
+ optimizations.skip_concurrency_control = IsInCombination(0, new_comb);
+ optimizations.skip_duplicate_key_check = IsInCombination(1, new_comb);
+
+ ASSERT_OK(ReOpen());
+ WriteOptions write_options;
+ WriteBatch batch;
+ batch.Put(Slice("k"), Slice("v1"));
+ ASSERT_OK(db->Write(write_options, &batch));
+
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ ASSERT_OK(db->Get(ropt, db->DefaultColumnFamily(), "k", &pinnable_val));
+ ASSERT_TRUE(pinnable_val == ("v1"));
+ }
+}
+
+// A comparator that uses only the first three bytes
+class ThreeBytewiseComparator : public Comparator {
+ public:
+ ThreeBytewiseComparator() {}
+ const char* Name() const override { return "test.ThreeBytewiseComparator"; }
+ int Compare(const Slice& a, const Slice& b) const override {
+ Slice na = Slice(a.data(), a.size() < 3 ? a.size() : 3);
+ Slice nb = Slice(b.data(), b.size() < 3 ? b.size() : 3);
+ return na.compare(nb);
+ }
+ bool Equal(const Slice& a, const Slice& b) const override {
+ Slice na = Slice(a.data(), a.size() < 3 ? a.size() : 3);
+ Slice nb = Slice(b.data(), b.size() < 3 ? b.size() : 3);
+ return na == nb;
+ }
+ // This methods below dont seem relevant to this test. Implement them if
+ // proven othersize.
+ void FindShortestSeparator(std::string* start,
+ const Slice& limit) const override {
+ const Comparator* bytewise_comp = BytewiseComparator();
+ bytewise_comp->FindShortestSeparator(start, limit);
+ }
+ void FindShortSuccessor(std::string* key) const override {
+ const Comparator* bytewise_comp = BytewiseComparator();
+ bytewise_comp->FindShortSuccessor(key);
+ }
+};
+
+#ifndef ROCKSDB_VALGRIND_RUN
+TEST_P(TransactionTest, GetWithoutSnapshot) {
+ WriteOptions write_options;
+ std::atomic<bool> finish = {false};
+ db->Put(write_options, "key", "value");
+ ROCKSDB_NAMESPACE::port::Thread commit_thread([&]() {
+ for (int i = 0; i < 100; i++) {
+ TransactionOptions txn_options;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn->SetName("xid"));
+ ASSERT_OK(txn->Put("key", "overridedvalue"));
+ ASSERT_OK(txn->Put("key", "value"));
+ ASSERT_OK(txn->Prepare());
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ }
+ finish = true;
+ });
+ ROCKSDB_NAMESPACE::port::Thread read_thread([&]() {
+ while (!finish) {
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ ASSERT_OK(db->Get(ropt, db->DefaultColumnFamily(), "key", &pinnable_val));
+ ASSERT_TRUE(pinnable_val == ("value"));
+ }
+ });
+ commit_thread.join();
+ read_thread.join();
+}
+#endif // ROCKSDB_VALGRIND_RUN
+
+// Test that the transactional db can handle duplicate keys in the write batch
+TEST_P(TransactionTest, DuplicateKeys) {
+ ColumnFamilyOptions cf_options;
+ std::string cf_name = "two";
+ ColumnFamilyHandle* cf_handle = nullptr;
+ {
+ ASSERT_OK(db->CreateColumnFamily(cf_options, cf_name, &cf_handle));
+ WriteOptions write_options;
+ WriteBatch batch;
+ batch.Put(Slice("key"), Slice("value"));
+ batch.Put(Slice("key2"), Slice("value2"));
+ // duplicate the keys
+ batch.Put(Slice("key"), Slice("value3"));
+ // duplicate the 2nd key. It should not be counted duplicate since a
+ // sub-patch is cut after the last duplicate.
+ batch.Put(Slice("key2"), Slice("value4"));
+ // duplicate the keys but in a different cf. It should not be counted as
+ // duplicate keys
+ batch.Put(cf_handle, Slice("key"), Slice("value5"));
+
+ ASSERT_OK(db->Write(write_options, &batch));
+
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ auto s = db->Get(ropt, db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("value3"));
+ s = db->Get(ropt, db->DefaultColumnFamily(), "key2", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("value4"));
+ s = db->Get(ropt, cf_handle, "key", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("value5"));
+
+ delete cf_handle;
+ }
+
+ // Test with non-bytewise comparator
+ {
+ ASSERT_OK(ReOpen());
+ std::unique_ptr<const Comparator> comp_gc(new ThreeBytewiseComparator());
+ cf_options.comparator = comp_gc.get();
+ ASSERT_OK(db->CreateColumnFamily(cf_options, cf_name, &cf_handle));
+ WriteOptions write_options;
+ WriteBatch batch;
+ batch.Put(cf_handle, Slice("key"), Slice("value"));
+ // The first three bytes are the same, do it must be counted as duplicate
+ batch.Put(cf_handle, Slice("key2"), Slice("value2"));
+ // check for 2nd duplicate key in cf with non-default comparator
+ batch.Put(cf_handle, Slice("key2b"), Slice("value2b"));
+ ASSERT_OK(db->Write(write_options, &batch));
+
+ // The value must be the most recent value for all the keys equal to "key",
+ // including "key2"
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ ASSERT_OK(db->Get(ropt, cf_handle, "key", &pinnable_val));
+ ASSERT_TRUE(pinnable_val == ("value2b"));
+
+ // Test duplicate keys with rollback
+ TransactionOptions txn_options;
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(cf_handle, Slice("key3"), Slice("value3")));
+ ASSERT_OK(txn0->Merge(cf_handle, Slice("key4"), Slice("value4")));
+ ASSERT_OK(txn0->Rollback());
+ ASSERT_OK(db->Get(ropt, cf_handle, "key5", &pinnable_val));
+ ASSERT_TRUE(pinnable_val == ("value2b"));
+ delete txn0;
+
+ delete cf_handle;
+ cf_options.comparator = BytewiseComparator();
+ }
+
+ for (bool do_prepare : {true, false}) {
+ for (bool do_rollback : {true, false}) {
+ for (bool with_commit_batch : {true, false}) {
+ if (with_commit_batch && !do_prepare) {
+ continue;
+ }
+ if (with_commit_batch && do_rollback) {
+ continue;
+ }
+ ASSERT_OK(ReOpen());
+ ASSERT_OK(db->CreateColumnFamily(cf_options, cf_name, &cf_handle));
+ TransactionOptions txn_options;
+ txn_options.use_only_the_last_commit_time_batch_for_recovery = false;
+ WriteOptions write_options;
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ auto s = txn0->SetName("xid");
+ ASSERT_OK(s);
+ s = txn0->Put(Slice("foo0"), Slice("bar0a"));
+ ASSERT_OK(s);
+ s = txn0->Put(Slice("foo0"), Slice("bar0b"));
+ ASSERT_OK(s);
+ s = txn0->Put(Slice("foo1"), Slice("bar1"));
+ ASSERT_OK(s);
+ s = txn0->Merge(Slice("foo2"), Slice("bar2a"));
+ ASSERT_OK(s);
+ // Repeat a key after the start of a sub-patch. This should not cause a
+ // duplicate in the most recent sub-patch and hence not creating a new
+ // sub-patch.
+ s = txn0->Put(Slice("foo0"), Slice("bar0c"));
+ ASSERT_OK(s);
+ s = txn0->Merge(Slice("foo2"), Slice("bar2b"));
+ ASSERT_OK(s);
+ // duplicate the keys but in a different cf. It should not be counted as
+ // duplicate.
+ s = txn0->Put(cf_handle, Slice("foo0"), Slice("bar0-cf1"));
+ ASSERT_OK(s);
+ s = txn0->Put(Slice("foo3"), Slice("bar3"));
+ ASSERT_OK(s);
+ s = txn0->Merge(Slice("foo3"), Slice("bar3"));
+ ASSERT_OK(s);
+ s = txn0->Put(Slice("foo4"), Slice("bar4"));
+ ASSERT_OK(s);
+ s = txn0->Delete(Slice("foo4"));
+ ASSERT_OK(s);
+ s = txn0->SingleDelete(Slice("foo4"));
+ ASSERT_OK(s);
+ if (do_prepare) {
+ s = txn0->Prepare();
+ ASSERT_OK(s);
+ }
+ if (do_rollback) {
+ // Test rolling back the batch with duplicates
+ s = txn0->Rollback();
+ ASSERT_OK(s);
+ } else {
+ if (with_commit_batch) {
+ assert(do_prepare);
+ auto cb = txn0->GetCommitTimeWriteBatch();
+ // duplicate a key in the original batch
+ // TODO(myabandeh): the behavior of GetCommitTimeWriteBatch
+ // conflicting with the prepared batch is currently undefined and
+ // gives different results in different implementations.
+
+ // s = cb->Put(Slice("foo0"), Slice("bar0d"));
+ // ASSERT_OK(s);
+ // add a new duplicate key
+ s = cb->Put(Slice("foo6"), Slice("bar6a"));
+ ASSERT_OK(s);
+ s = cb->Put(Slice("foo6"), Slice("bar6b"));
+ ASSERT_OK(s);
+ // add a duplicate key that is removed in the same batch
+ s = cb->Put(Slice("foo7"), Slice("bar7a"));
+ ASSERT_OK(s);
+ s = cb->Delete(Slice("foo7"));
+ ASSERT_OK(s);
+ }
+ s = txn0->Commit();
+ ASSERT_OK(s);
+ }
+ delete txn0;
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+
+ if (do_rollback) {
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ s = db->Get(ropt, cf_handle, "foo0", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo1", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo2", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo3", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo4", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ } else {
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar0c"));
+ s = db->Get(ropt, cf_handle, "foo0", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar0-cf1"));
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo1", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar1"));
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo2", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar2a,bar2b"));
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo3", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar3,bar3"));
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo4", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ if (with_commit_batch) {
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo6", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar6b"));
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo7", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ }
+ }
+ delete cf_handle;
+ } // with_commit_batch
+ } // do_rollback
+ } // do_prepare
+
+ if (!options.unordered_write) {
+ // Also test with max_successive_merges > 0. max_successive_merges will not
+ // affect our algorithm for duplicate key insertion but we add the test to
+ // verify that.
+ cf_options.max_successive_merges = 2;
+ cf_options.merge_operator = MergeOperators::CreateStringAppendOperator();
+ ASSERT_OK(ReOpen());
+ db->CreateColumnFamily(cf_options, cf_name, &cf_handle);
+ WriteOptions write_options;
+ // Ensure one value for the key
+ ASSERT_OK(db->Put(write_options, cf_handle, Slice("key"), Slice("value")));
+ WriteBatch batch;
+ // Merge more than max_successive_merges times
+ batch.Merge(cf_handle, Slice("key"), Slice("1"));
+ batch.Merge(cf_handle, Slice("key"), Slice("2"));
+ batch.Merge(cf_handle, Slice("key"), Slice("3"));
+ batch.Merge(cf_handle, Slice("key"), Slice("4"));
+ ASSERT_OK(db->Write(write_options, &batch));
+ ReadOptions read_options;
+ string value;
+ ASSERT_OK(db->Get(read_options, cf_handle, "key", &value));
+ ASSERT_EQ(value, "value,1,2,3,4");
+ delete cf_handle;
+ }
+
+ {
+ // Test that the duplicate detection is not compromised after rolling back
+ // to a save point
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0a")));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0b")));
+ txn0->SetSavePoint();
+ ASSERT_OK(txn0->RollbackToSavePoint());
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+ }
+
+ // Test sucessfull recovery after a crash
+ {
+ ASSERT_OK(ReOpen());
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ ReadOptions ropt;
+ Transaction* txn0;
+ PinnableSlice pinnable_val;
+ Status s;
+
+ std::unique_ptr<const Comparator> comp_gc(new ThreeBytewiseComparator());
+ cf_options.comparator = comp_gc.get();
+ cf_options.merge_operator = MergeOperators::CreateStringAppendOperator();
+ ASSERT_OK(db->CreateColumnFamily(cf_options, cf_name, &cf_handle));
+ delete cf_handle;
+ std::vector<ColumnFamilyDescriptor> cfds{
+ ColumnFamilyDescriptor(kDefaultColumnFamilyName,
+ ColumnFamilyOptions(options)),
+ ColumnFamilyDescriptor(cf_name, cf_options),
+ };
+ std::vector<ColumnFamilyHandle*> handles;
+ ASSERT_OK(ReOpenNoDelete(cfds, &handles));
+
+ ASSERT_OK(db->Put(write_options, "foo0", "init"));
+ ASSERT_OK(db->Put(write_options, "foo1", "init"));
+ ASSERT_OK(db->Put(write_options, handles[1], "foo0", "init"));
+ ASSERT_OK(db->Put(write_options, handles[1], "foo1", "init"));
+
+ // one entry
+ txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0a")));
+ ASSERT_OK(txn0->Prepare());
+ delete txn0;
+ // This will check the asserts inside recovery code
+ ASSERT_OK(db->FlushWAL(true));
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ASSERT_OK(ReOpenNoDelete(cfds, &handles));
+ txn0 = db->GetTransactionByName("xid");
+ ASSERT_TRUE(txn0 != nullptr);
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar0a"));
+
+ // two entries, no duplicate
+ txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(handles[1], Slice("foo0"), Slice("bar0b")));
+ ASSERT_OK(txn0->Put(handles[1], Slice("fol1"), Slice("bar1b")));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0b")));
+ ASSERT_OK(txn0->Put(Slice("foo1"), Slice("bar1b")));
+ ASSERT_OK(txn0->Prepare());
+ delete txn0;
+ // This will check the asserts inside recovery code
+ db->FlushWAL(true);
+ // Flush only cf 1
+ reinterpret_cast<DBImpl*>(db->GetRootDB())
+ ->TEST_FlushMemTable(true, false, handles[1]);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ASSERT_OK(ReOpenNoDelete(cfds, &handles));
+ txn0 = db->GetTransactionByName("xid");
+ ASSERT_TRUE(txn0 != nullptr);
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+ pinnable_val.Reset();
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar0b"));
+ pinnable_val.Reset();
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo1", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar1b"));
+ pinnable_val.Reset();
+ s = db->Get(ropt, handles[1], "foo0", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar0b"));
+ pinnable_val.Reset();
+ s = db->Get(ropt, handles[1], "fol1", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar1b"));
+
+ // one duplicate with ::Put
+ txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(handles[1], Slice("key-nonkey0"), Slice("bar0c")));
+ ASSERT_OK(txn0->Put(handles[1], Slice("key-nonkey1"), Slice("bar1d")));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0c")));
+ ASSERT_OK(txn0->Put(Slice("foo1"), Slice("bar1c")));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0d")));
+ ASSERT_OK(txn0->Prepare());
+ delete txn0;
+ // This will check the asserts inside recovery code
+ ASSERT_OK(db->FlushWAL(true));
+ // Flush only cf 1
+ reinterpret_cast<DBImpl*>(db->GetRootDB())
+ ->TEST_FlushMemTable(true, false, handles[1]);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ASSERT_OK(ReOpenNoDelete(cfds, &handles));
+ txn0 = db->GetTransactionByName("xid");
+ ASSERT_TRUE(txn0 != nullptr);
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+ pinnable_val.Reset();
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar0d"));
+ pinnable_val.Reset();
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo1", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar1c"));
+ pinnable_val.Reset();
+ s = db->Get(ropt, handles[1], "key-nonkey2", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar1d"));
+
+ // Duplicate with ::Put, ::Delete
+ txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(handles[1], Slice("key-nonkey0"), Slice("bar0e")));
+ ASSERT_OK(txn0->Delete(handles[1], Slice("key-nonkey1")));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0e")));
+ ASSERT_OK(txn0->Delete(Slice("foo0")));
+ ASSERT_OK(txn0->Prepare());
+ delete txn0;
+ // This will check the asserts inside recovery code
+ ASSERT_OK(db->FlushWAL(true));
+ // Flush only cf 1
+ reinterpret_cast<DBImpl*>(db->GetRootDB())
+ ->TEST_FlushMemTable(true, false, handles[1]);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ASSERT_OK(ReOpenNoDelete(cfds, &handles));
+ txn0 = db->GetTransactionByName("xid");
+ ASSERT_TRUE(txn0 != nullptr);
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+ pinnable_val.Reset();
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ pinnable_val.Reset();
+ s = db->Get(ropt, handles[1], "key-nonkey2", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Duplicate with ::Put, ::SingleDelete
+ txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(handles[1], Slice("key-nonkey0"), Slice("bar0g")));
+ ASSERT_OK(txn0->SingleDelete(handles[1], Slice("key-nonkey1")));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0e")));
+ ASSERT_OK(txn0->SingleDelete(Slice("foo0")));
+ ASSERT_OK(txn0->Prepare());
+ delete txn0;
+ // This will check the asserts inside recovery code
+ ASSERT_OK(db->FlushWAL(true));
+ // Flush only cf 1
+ reinterpret_cast<DBImpl*>(db->GetRootDB())
+ ->TEST_FlushMemTable(true, false, handles[1]);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ASSERT_OK(ReOpenNoDelete(cfds, &handles));
+ txn0 = db->GetTransactionByName("xid");
+ ASSERT_TRUE(txn0 != nullptr);
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+ pinnable_val.Reset();
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ pinnable_val.Reset();
+ s = db->Get(ropt, handles[1], "key-nonkey2", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Duplicate with ::Put, ::Merge
+ txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(handles[1], Slice("key-nonkey0"), Slice("bar1i")));
+ ASSERT_OK(txn0->Merge(handles[1], Slice("key-nonkey1"), Slice("bar1j")));
+ ASSERT_OK(txn0->Put(Slice("foo0"), Slice("bar0f")));
+ ASSERT_OK(txn0->Merge(Slice("foo0"), Slice("bar0g")));
+ ASSERT_OK(txn0->Prepare());
+ delete txn0;
+ // This will check the asserts inside recovery code
+ ASSERT_OK(db->FlushWAL(true));
+ // Flush only cf 1
+ reinterpret_cast<DBImpl*>(db->GetRootDB())
+ ->TEST_FlushMemTable(true, false, handles[1]);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ ASSERT_OK(ReOpenNoDelete(cfds, &handles));
+ txn0 = db->GetTransactionByName("xid");
+ ASSERT_TRUE(txn0 != nullptr);
+ ASSERT_OK(txn0->Commit());
+ delete txn0;
+ pinnable_val.Reset();
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar0f,bar0g"));
+ pinnable_val.Reset();
+ s = db->Get(ropt, handles[1], "key-nonkey2", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == ("bar1i,bar1j"));
+
+ for (auto h : handles) {
+ delete h;
+ }
+ delete db;
+ db = nullptr;
+ }
+}
+
+// Test that the reseek optimization in iterators will not result in an infinite
+// loop if there are too many uncommitted entries before the snapshot.
+TEST_P(TransactionTest, ReseekOptimization) {
+ WriteOptions write_options;
+ write_options.sync = true;
+ write_options.disableWAL = false;
+ ColumnFamilyDescriptor cfd;
+ db->DefaultColumnFamily()->GetDescriptor(&cfd);
+ auto max_skip = cfd.options.max_sequential_skip_in_iterations;
+
+ ASSERT_OK(db->Put(write_options, Slice("foo0"), Slice("initv")));
+
+ TransactionOptions txn_options;
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ // Duplicate keys will result into separate sequence numbers in WritePrepared
+ // and WriteUnPrepared
+ for (size_t i = 0; i < 2 * max_skip; i++) {
+ ASSERT_OK(txn0->Put(Slice("foo1"), Slice("bar")));
+ }
+ ASSERT_OK(txn0->Prepare());
+ ASSERT_OK(db->Put(write_options, Slice("foo2"), Slice("initv")));
+
+ ReadOptions read_options;
+ // To avoid loops
+ read_options.max_skippable_internal_keys = 10 * max_skip;
+ Iterator* iter = db->NewIterator(read_options);
+ ASSERT_OK(iter->status());
+ size_t cnt = 0;
+ iter->SeekToFirst();
+ while (iter->Valid()) {
+ iter->Next();
+ ASSERT_OK(iter->status());
+ cnt++;
+ }
+ ASSERT_EQ(cnt, 2);
+ cnt = 0;
+ iter->SeekToLast();
+ while (iter->Valid()) {
+ iter->Prev();
+ ASSERT_OK(iter->status());
+ cnt++;
+ }
+ ASSERT_EQ(cnt, 2);
+ delete iter;
+ txn0->Rollback();
+ delete txn0;
+}
+
+// After recovery in kPointInTimeRecovery mode, the corrupted log file remains
+// there. The new log files should be still read succesfully during recovery of
+// the 2nd crash.
+TEST_P(TransactionTest, DoubleCrashInRecovery) {
+ for (const bool manual_wal_flush : {false, true}) {
+ for (const bool write_after_recovery : {false, true}) {
+ options.wal_recovery_mode = WALRecoveryMode::kPointInTimeRecovery;
+ options.manual_wal_flush = manual_wal_flush;
+ ReOpen();
+ std::string cf_name = "two";
+ ColumnFamilyOptions cf_options;
+ ColumnFamilyHandle* cf_handle = nullptr;
+ ASSERT_OK(db->CreateColumnFamily(cf_options, cf_name, &cf_handle));
+
+ // Add a prepare entry to prevent the older logs from being deleted.
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn->SetName("xid"));
+ ASSERT_OK(txn->Put(Slice("foo-prepare"), Slice("bar-prepare")));
+ ASSERT_OK(txn->Prepare());
+
+ FlushOptions flush_ops;
+ db->Flush(flush_ops);
+ // Now we have a log that cannot be deleted
+
+ ASSERT_OK(db->Put(write_options, cf_handle, "foo1", "bar1"));
+ // Flush only the 2nd cf
+ db->Flush(flush_ops, cf_handle);
+
+ // The value is large enough to be touched by the corruption we ingest
+ // below.
+ std::string large_value(400, ' ');
+ // key/value not touched by corruption
+ ASSERT_OK(db->Put(write_options, "foo2", "bar2"));
+ // key/value touched by corruption
+ ASSERT_OK(db->Put(write_options, "foo3", large_value));
+ // key/value not touched by corruption
+ ASSERT_OK(db->Put(write_options, "foo4", "bar4"));
+
+ db->FlushWAL(true);
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ uint64_t wal_file_id = db_impl->TEST_LogfileNumber();
+ std::string fname = LogFileName(dbname, wal_file_id);
+ reinterpret_cast<PessimisticTransactionDB*>(db)->TEST_Crash();
+ delete txn;
+ delete cf_handle;
+ delete db;
+ db = nullptr;
+
+ // Corrupt the last log file in the middle, so that it is not corrupted
+ // in the tail.
+ std::string file_content;
+ ASSERT_OK(ReadFileToString(env, fname, &file_content));
+ file_content[400] = 'h';
+ file_content[401] = 'a';
+ ASSERT_OK(env->DeleteFile(fname));
+ ASSERT_OK(WriteStringToFile(env, file_content, fname, true));
+
+ // Recover from corruption
+ std::vector<ColumnFamilyHandle*> handles;
+ std::vector<ColumnFamilyDescriptor> column_families;
+ column_families.push_back(ColumnFamilyDescriptor(kDefaultColumnFamilyName,
+ ColumnFamilyOptions()));
+ column_families.push_back(
+ ColumnFamilyDescriptor("two", ColumnFamilyOptions()));
+ ASSERT_OK(ReOpenNoDelete(column_families, &handles));
+
+ if (write_after_recovery) {
+ // Write data to the log right after the corrupted log
+ ASSERT_OK(db->Put(write_options, "foo5", large_value));
+ }
+
+ // Persist data written to WAL during recovery or by the last Put
+ db->FlushWAL(true);
+ // 2nd crash to recover while having a valid log after the corrupted one.
+ ASSERT_OK(ReOpenNoDelete(column_families, &handles));
+ assert(db != nullptr);
+ txn = db->GetTransactionByName("xid");
+ ASSERT_TRUE(txn != nullptr);
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ for (auto handle : handles) {
+ delete handle;
+ }
+ }
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(stderr,
+ "SKIPPED as Transactions are not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/transaction_test.h b/src/rocksdb/utilities/transactions/transaction_test.h
new file mode 100644
index 000000000..2e533d379
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_test.h
@@ -0,0 +1,517 @@
+// 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
+
+#include <algorithm>
+#include <cinttypes>
+#include <functional>
+#include <string>
+#include <thread>
+
+#include "db/db_impl/db_impl.h"
+#include "rocksdb/db.h"
+#include "rocksdb/options.h"
+#include "rocksdb/utilities/transaction.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "table/mock_table.h"
+#include "test_util/fault_injection_test_env.h"
+#include "test_util/sync_point.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "test_util/transaction_test_util.h"
+#include "util/random.h"
+#include "util/string_util.h"
+#include "utilities/merge_operators.h"
+#include "utilities/merge_operators/string_append/stringappend.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+#include "utilities/transactions/write_unprepared_txn_db.h"
+
+#include "port/port.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Return true if the ith bit is set in combination represented by comb
+bool IsInCombination(size_t i, size_t comb) { return comb & (size_t(1) << i); }
+
+enum WriteOrdering : bool { kOrderedWrite, kUnorderedWrite };
+
+class TransactionTestBase : public ::testing::Test {
+ public:
+ TransactionDB* db;
+ FaultInjectionTestEnv* env;
+ std::string dbname;
+ Options options;
+
+ TransactionDBOptions txn_db_options;
+ bool use_stackable_db_;
+
+ TransactionTestBase(bool use_stackable_db, bool two_write_queue,
+ TxnDBWritePolicy write_policy,
+ WriteOrdering write_ordering)
+ : db(nullptr), env(nullptr), use_stackable_db_(use_stackable_db) {
+ options.create_if_missing = true;
+ options.max_write_buffer_number = 2;
+ options.write_buffer_size = 4 * 1024;
+ options.unordered_write = write_ordering == kUnorderedWrite;
+ options.level0_file_num_compaction_trigger = 2;
+ options.merge_operator = MergeOperators::CreateFromStringId("stringappend");
+ env = new FaultInjectionTestEnv(Env::Default());
+ options.env = env;
+ options.two_write_queues = two_write_queue;
+ dbname = test::PerThreadDBPath("transaction_testdb");
+
+ DestroyDB(dbname, options);
+ txn_db_options.transaction_lock_timeout = 0;
+ txn_db_options.default_lock_timeout = 0;
+ txn_db_options.write_policy = write_policy;
+ txn_db_options.rollback_merge_operands = true;
+ // This will stress write unprepared, by forcing write batch flush on every
+ // write.
+ txn_db_options.default_write_batch_flush_threshold = 1;
+ // Write unprepared requires all transactions to be named. This setting
+ // autogenerates the name so that existing tests can pass.
+ txn_db_options.autogenerate_name = true;
+ Status s;
+ if (use_stackable_db == false) {
+ s = TransactionDB::Open(options, txn_db_options, dbname, &db);
+ } else {
+ s = OpenWithStackableDB();
+ }
+ assert(s.ok());
+ }
+
+ ~TransactionTestBase() {
+ delete db;
+ db = nullptr;
+ // This is to skip the assert statement in FaultInjectionTestEnv. There
+ // seems to be a bug in btrfs that the makes readdir return recently
+ // unlink-ed files. By using the default fs we simply ignore errors resulted
+ // from attempting to delete such files in DestroyDB.
+ options.env = Env::Default();
+ DestroyDB(dbname, options);
+ delete env;
+ }
+
+ Status ReOpenNoDelete() {
+ delete db;
+ db = nullptr;
+ env->AssertNoOpenFile();
+ env->DropUnsyncedFileData();
+ env->ResetState();
+ Status s;
+ if (use_stackable_db_ == false) {
+ s = TransactionDB::Open(options, txn_db_options, dbname, &db);
+ } else {
+ s = OpenWithStackableDB();
+ }
+ assert(!s.ok() || db != nullptr);
+ return s;
+ }
+
+ Status ReOpenNoDelete(std::vector<ColumnFamilyDescriptor>& cfs,
+ std::vector<ColumnFamilyHandle*>* handles) {
+ for (auto h : *handles) {
+ delete h;
+ }
+ handles->clear();
+ delete db;
+ db = nullptr;
+ env->AssertNoOpenFile();
+ env->DropUnsyncedFileData();
+ env->ResetState();
+ Status s;
+ if (use_stackable_db_ == false) {
+ s = TransactionDB::Open(options, txn_db_options, dbname, cfs, handles,
+ &db);
+ } else {
+ s = OpenWithStackableDB(cfs, handles);
+ }
+ assert(!s.ok() || db != nullptr);
+ return s;
+ }
+
+ Status ReOpen() {
+ delete db;
+ db = nullptr;
+ DestroyDB(dbname, options);
+ Status s;
+ if (use_stackable_db_ == false) {
+ s = TransactionDB::Open(options, txn_db_options, dbname, &db);
+ } else {
+ s = OpenWithStackableDB();
+ }
+ assert(db != nullptr);
+ return s;
+ }
+
+ Status OpenWithStackableDB(std::vector<ColumnFamilyDescriptor>& cfs,
+ std::vector<ColumnFamilyHandle*>* handles) {
+ std::vector<size_t> compaction_enabled_cf_indices;
+ TransactionDB::PrepareWrap(&options, &cfs, &compaction_enabled_cf_indices);
+ DB* root_db = nullptr;
+ Options options_copy(options);
+ 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;
+ Status s = DBImpl::Open(options_copy, dbname, cfs, handles, &root_db,
+ use_seq_per_batch, use_batch_per_txn);
+ StackableDB* stackable_db = new StackableDB(root_db);
+ if (s.ok()) {
+ assert(root_db != nullptr);
+ s = TransactionDB::WrapStackableDB(stackable_db, txn_db_options,
+ compaction_enabled_cf_indices,
+ *handles, &db);
+ }
+ if (!s.ok()) {
+ delete stackable_db;
+ }
+ return s;
+ }
+
+ Status OpenWithStackableDB() {
+ std::vector<size_t> compaction_enabled_cf_indices;
+ std::vector<ColumnFamilyDescriptor> column_families{ColumnFamilyDescriptor(
+ kDefaultColumnFamilyName, ColumnFamilyOptions(options))};
+
+ TransactionDB::PrepareWrap(&options, &column_families,
+ &compaction_enabled_cf_indices);
+ std::vector<ColumnFamilyHandle*> handles;
+ DB* root_db = nullptr;
+ Options options_copy(options);
+ 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;
+ Status s = DBImpl::Open(options_copy, dbname, column_families, &handles,
+ &root_db, use_seq_per_batch, use_batch_per_txn);
+ if (!s.ok()) {
+ delete root_db;
+ return s;
+ }
+ StackableDB* stackable_db = new StackableDB(root_db);
+ assert(root_db != nullptr);
+ assert(handles.size() == 1);
+ s = TransactionDB::WrapStackableDB(stackable_db, txn_db_options,
+ compaction_enabled_cf_indices, handles,
+ &db);
+ delete handles[0];
+ if (!s.ok()) {
+ delete stackable_db;
+ }
+ return s;
+ }
+
+ std::atomic<size_t> linked = {0};
+ std::atomic<size_t> exp_seq = {0};
+ std::atomic<size_t> commit_writes = {0};
+ std::atomic<size_t> expected_commits = {0};
+ // Without Prepare, the commit does not write to WAL
+ std::atomic<size_t> with_empty_commits = {0};
+ std::function<void(size_t, Status)> txn_t0_with_status = [&](size_t index,
+ Status exp_s) {
+ // Test DB's internal txn. It involves no prepare phase nor a commit marker.
+ WriteOptions wopts;
+ auto s = db->Put(wopts, "key" + std::to_string(index), "value");
+ ASSERT_EQ(exp_s, s);
+ if (txn_db_options.write_policy == TxnDBWritePolicy::WRITE_COMMITTED) {
+ // Consume one seq per key
+ exp_seq++;
+ } else {
+ // Consume one seq per batch
+ exp_seq++;
+ if (options.two_write_queues) {
+ // Consume one seq for commit
+ exp_seq++;
+ }
+ }
+ with_empty_commits++;
+ };
+ std::function<void(size_t)> txn_t0 = [&](size_t index) {
+ return txn_t0_with_status(index, Status::OK());
+ };
+ std::function<void(size_t)> txn_t1 = [&](size_t index) {
+ // Testing directly writing a write batch. Functionality-wise it is
+ // equivalent to commit without prepare.
+ WriteBatch wb;
+ auto istr = std::to_string(index);
+ ASSERT_OK(wb.Put("k1" + istr, "v1"));
+ ASSERT_OK(wb.Put("k2" + istr, "v2"));
+ ASSERT_OK(wb.Put("k3" + istr, "v3"));
+ WriteOptions wopts;
+ auto s = db->Write(wopts, &wb);
+ if (txn_db_options.write_policy == TxnDBWritePolicy::WRITE_COMMITTED) {
+ // Consume one seq per key
+ exp_seq += 3;
+ } else {
+ // Consume one seq per batch
+ exp_seq++;
+ if (options.two_write_queues) {
+ // Consume one seq for commit
+ exp_seq++;
+ }
+ }
+ ASSERT_OK(s);
+ with_empty_commits++;
+ };
+ std::function<void(size_t)> txn_t2 = [&](size_t index) {
+ // Commit without prepare. It should write to DB without a commit marker.
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ auto istr = std::to_string(index);
+ ASSERT_OK(txn->SetName("xid" + istr));
+ ASSERT_OK(txn->Put(Slice("foo" + istr), Slice("bar")));
+ ASSERT_OK(txn->Put(Slice("foo2" + istr), Slice("bar2")));
+ ASSERT_OK(txn->Put(Slice("foo3" + istr), Slice("bar3")));
+ ASSERT_OK(txn->Put(Slice("foo4" + istr), Slice("bar4")));
+ ASSERT_OK(txn->Commit());
+ if (txn_db_options.write_policy == TxnDBWritePolicy::WRITE_COMMITTED) {
+ // Consume one seq per key
+ exp_seq += 4;
+ } else if (txn_db_options.write_policy ==
+ TxnDBWritePolicy::WRITE_PREPARED) {
+ // Consume one seq per batch
+ exp_seq++;
+ if (options.two_write_queues) {
+ // Consume one seq for commit
+ exp_seq++;
+ }
+ } else {
+ // Flushed after each key, consume one seq per flushed batch
+ exp_seq += 4;
+ // WriteUnprepared implements CommitWithoutPrepareInternal by simply
+ // calling Prepare then Commit. Consume one seq for the prepare.
+ exp_seq++;
+ }
+ delete txn;
+ with_empty_commits++;
+ };
+ std::function<void(size_t)> txn_t3 = [&](size_t index) {
+ // A full 2pc txn that also involves a commit marker.
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ auto istr = std::to_string(index);
+ ASSERT_OK(txn->SetName("xid" + istr));
+ ASSERT_OK(txn->Put(Slice("foo" + istr), Slice("bar")));
+ ASSERT_OK(txn->Put(Slice("foo2" + istr), Slice("bar2")));
+ ASSERT_OK(txn->Put(Slice("foo3" + istr), Slice("bar3")));
+ ASSERT_OK(txn->Put(Slice("foo4" + istr), Slice("bar4")));
+ ASSERT_OK(txn->Put(Slice("foo5" + istr), Slice("bar5")));
+ expected_commits++;
+ ASSERT_OK(txn->Prepare());
+ commit_writes++;
+ ASSERT_OK(txn->Commit());
+ if (txn_db_options.write_policy == TxnDBWritePolicy::WRITE_COMMITTED) {
+ // Consume one seq per key
+ exp_seq += 5;
+ } else if (txn_db_options.write_policy ==
+ TxnDBWritePolicy::WRITE_PREPARED) {
+ // Consume one seq per batch
+ exp_seq++;
+ // Consume one seq per commit marker
+ exp_seq++;
+ } else {
+ // Flushed after each key, consume one seq per flushed batch
+ exp_seq += 5;
+ // Consume one seq per commit marker
+ exp_seq++;
+ }
+ delete txn;
+ };
+ std::function<void(size_t)> txn_t4 = [&](size_t index) {
+ // A full 2pc txn that also involves a commit marker.
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ auto istr = std::to_string(index);
+ ASSERT_OK(txn->SetName("xid" + istr));
+ ASSERT_OK(txn->Put(Slice("foo" + istr), Slice("bar")));
+ ASSERT_OK(txn->Put(Slice("foo2" + istr), Slice("bar2")));
+ ASSERT_OK(txn->Put(Slice("foo3" + istr), Slice("bar3")));
+ ASSERT_OK(txn->Put(Slice("foo4" + istr), Slice("bar4")));
+ ASSERT_OK(txn->Put(Slice("foo5" + istr), Slice("bar5")));
+ expected_commits++;
+ ASSERT_OK(txn->Prepare());
+ commit_writes++;
+ ASSERT_OK(txn->Rollback());
+ if (txn_db_options.write_policy == TxnDBWritePolicy::WRITE_COMMITTED) {
+ // No seq is consumed for deleting the txn buffer
+ exp_seq += 0;
+ } else if (txn_db_options.write_policy ==
+ TxnDBWritePolicy::WRITE_PREPARED) {
+ // Consume one seq per batch
+ exp_seq++;
+ // Consume one seq per rollback batch
+ exp_seq++;
+ if (options.two_write_queues) {
+ // Consume one seq for rollback commit
+ exp_seq++;
+ }
+ } else {
+ // Flushed after each key, consume one seq per flushed batch
+ exp_seq += 5;
+ // Consume one seq per rollback batch
+ exp_seq++;
+ if (options.two_write_queues) {
+ // Consume one seq for rollback commit
+ exp_seq++;
+ }
+ }
+ delete txn;
+ };
+
+ // Test that we can change write policy after a clean shutdown (which would
+ // empty the WAL)
+ void CrossCompatibilityTest(TxnDBWritePolicy from_policy,
+ TxnDBWritePolicy to_policy, bool empty_wal) {
+ TransactionOptions txn_options;
+ ReadOptions read_options;
+ WriteOptions write_options;
+ uint32_t index = 0;
+ Random rnd(1103);
+ options.write_buffer_size = 1024; // To create more sst files
+ std::unordered_map<std::string, std::string> committed_kvs;
+ Transaction* txn;
+
+ txn_db_options.write_policy = from_policy;
+ if (txn_db_options.write_policy == WRITE_COMMITTED) {
+ options.unordered_write = false;
+ }
+ ReOpen();
+
+ for (int i = 0; i < 1024; i++) {
+ auto istr = std::to_string(index);
+ auto k = Slice("foo-" + istr).ToString();
+ auto v = Slice("bar-" + istr).ToString();
+ // For test the duplicate keys
+ auto v2 = Slice("bar2-" + istr).ToString();
+ auto type = rnd.Uniform(4);
+ switch (type) {
+ case 0:
+ committed_kvs[k] = v;
+ ASSERT_OK(db->Put(write_options, k, v));
+ committed_kvs[k] = v2;
+ ASSERT_OK(db->Put(write_options, k, v2));
+ break;
+ case 1: {
+ WriteBatch wb;
+ committed_kvs[k] = v;
+ wb.Put(k, v);
+ committed_kvs[k] = v2;
+ wb.Put(k, v2);
+ ASSERT_OK(db->Write(write_options, &wb));
+
+ } break;
+ case 2:
+ case 3:
+ txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn->SetName("xid" + istr));
+ committed_kvs[k] = v;
+ ASSERT_OK(txn->Put(k, v));
+ committed_kvs[k] = v2;
+ ASSERT_OK(txn->Put(k, v2));
+
+ if (type == 3) {
+ ASSERT_OK(txn->Prepare());
+ }
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ break;
+ default:
+ assert(0);
+ }
+
+ index++;
+ } // for i
+
+ txn_db_options.write_policy = to_policy;
+ if (txn_db_options.write_policy == WRITE_COMMITTED) {
+ options.unordered_write = false;
+ }
+ auto db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ // Before upgrade/downgrade the WAL must be emptied
+ if (empty_wal) {
+ db_impl->TEST_FlushMemTable();
+ } else {
+ db_impl->FlushWAL(true);
+ }
+ auto s = ReOpenNoDelete();
+ if (empty_wal) {
+ ASSERT_OK(s);
+ } else {
+ // Test that we can detect the WAL that is produced by an incompatible
+ // WritePolicy and fail fast before mis-interpreting the WAL.
+ ASSERT_TRUE(s.IsNotSupported());
+ return;
+ }
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ // Check that WAL is empty
+ VectorLogPtr log_files;
+ db_impl->GetSortedWalFiles(log_files);
+ ASSERT_EQ(0, log_files.size());
+
+ for (auto& kv : committed_kvs) {
+ std::string value;
+ s = db->Get(read_options, kv.first, &value);
+ if (s.IsNotFound()) {
+ printf("key = %s\n", kv.first.c_str());
+ }
+ ASSERT_OK(s);
+ if (kv.second != value) {
+ printf("key = %s\n", kv.first.c_str());
+ }
+ ASSERT_EQ(kv.second, value);
+ }
+ }
+};
+
+class TransactionTest
+ : public TransactionTestBase,
+ virtual public ::testing::WithParamInterface<
+ std::tuple<bool, bool, TxnDBWritePolicy, WriteOrdering>> {
+ public:
+ TransactionTest()
+ : TransactionTestBase(std::get<0>(GetParam()), std::get<1>(GetParam()),
+ std::get<2>(GetParam()), std::get<3>(GetParam())){};
+};
+
+class TransactionStressTest : public TransactionTest {};
+
+class MySQLStyleTransactionTest
+ : public TransactionTestBase,
+ virtual public ::testing::WithParamInterface<
+ std::tuple<bool, bool, TxnDBWritePolicy, WriteOrdering, bool>> {
+ public:
+ MySQLStyleTransactionTest()
+ : TransactionTestBase(std::get<0>(GetParam()), std::get<1>(GetParam()),
+ std::get<2>(GetParam()), std::get<3>(GetParam())),
+ with_slow_threads_(std::get<4>(GetParam())) {
+ if (with_slow_threads_ &&
+ (txn_db_options.write_policy == WRITE_PREPARED ||
+ txn_db_options.write_policy == WRITE_UNPREPARED)) {
+ // The corner case with slow threads involves the caches filling
+ // over which would not happen even with artifial delays. To help
+ // such cases to show up we lower the size of the cache-related data
+ // structures.
+ txn_db_options.wp_snapshot_cache_bits = 1;
+ txn_db_options.wp_commit_cache_bits = 10;
+ options.write_buffer_size = 1024;
+ EXPECT_OK(ReOpen());
+ }
+ };
+
+ protected:
+ // Also emulate slow threads by addin artiftial delays
+ const bool with_slow_threads_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/utilities/transactions/transaction_util.cc b/src/rocksdb/utilities/transactions/transaction_util.cc
new file mode 100644
index 000000000..23532ae42
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_util.cc
@@ -0,0 +1,182 @@
+// 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_util.h"
+
+#include <cinttypes>
+#include <string>
+#include <vector>
+
+#include "db/db_impl/db_impl.h"
+#include "rocksdb/status.h"
+#include "rocksdb/utilities/write_batch_with_index.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+Status TransactionUtil::CheckKeyForConflicts(
+ DBImpl* db_impl, ColumnFamilyHandle* column_family, const std::string& key,
+ SequenceNumber snap_seq, bool cache_only, ReadCallback* snap_checker,
+ SequenceNumber min_uncommitted) {
+ Status result;
+
+ auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
+ auto cfd = cfh->cfd();
+ SuperVersion* sv = db_impl->GetAndRefSuperVersion(cfd);
+
+ if (sv == nullptr) {
+ result = Status::InvalidArgument("Could not access column family " +
+ cfh->GetName());
+ }
+
+ if (result.ok()) {
+ SequenceNumber earliest_seq =
+ db_impl->GetEarliestMemTableSequenceNumber(sv, true);
+
+ result = CheckKey(db_impl, sv, earliest_seq, snap_seq, key, cache_only,
+ snap_checker, min_uncommitted);
+
+ db_impl->ReturnAndCleanupSuperVersion(cfd, sv);
+ }
+
+ return result;
+}
+
+Status TransactionUtil::CheckKey(DBImpl* db_impl, SuperVersion* sv,
+ SequenceNumber earliest_seq,
+ SequenceNumber snap_seq,
+ const std::string& key, bool cache_only,
+ ReadCallback* snap_checker,
+ SequenceNumber min_uncommitted) {
+ // When `min_uncommitted` is provided, keys are not always committed
+ // in sequence number order, and `snap_checker` is used to check whether
+ // specific sequence number is in the database is visible to the transaction.
+ // So `snap_checker` must be provided.
+ assert(min_uncommitted == kMaxSequenceNumber || snap_checker != nullptr);
+
+ Status result;
+ bool need_to_read_sst = false;
+
+ // Since it would be too slow to check the SST files, we will only use
+ // the memtables to check whether there have been any recent writes
+ // to this key after it was accessed in this transaction. But if the
+ // Memtables do not contain a long enough history, we must fail the
+ // transaction.
+ if (earliest_seq == kMaxSequenceNumber) {
+ // The age of this memtable is unknown. Cannot rely on it to check
+ // for recent writes. This error shouldn't happen often in practice as
+ // the Memtable should have a valid earliest sequence number except in some
+ // corner cases (such as error cases during recovery).
+ need_to_read_sst = true;
+
+ if (cache_only) {
+ result = Status::TryAgain(
+ "Transaction could not check for conflicts as the MemTable does not "
+ "contain a long enough history to check write at SequenceNumber: ",
+ ToString(snap_seq));
+ }
+ } else if (snap_seq < earliest_seq || min_uncommitted <= earliest_seq) {
+ // Use <= for min_uncommitted since earliest_seq is actually the largest sec
+ // before this memtable was created
+ need_to_read_sst = true;
+
+ if (cache_only) {
+ // The age of this memtable is too new to use to check for recent
+ // writes.
+ char msg[300];
+ snprintf(msg, sizeof(msg),
+ "Transaction could not check for conflicts for operation at "
+ "SequenceNumber %" PRIu64
+ " as the MemTable only contains changes newer than "
+ "SequenceNumber %" PRIu64
+ ". Increasing the value of the "
+ "max_write_buffer_size_to_maintain option could reduce the "
+ "frequency "
+ "of this error.",
+ snap_seq, earliest_seq);
+ result = Status::TryAgain(msg);
+ }
+ }
+
+ if (result.ok()) {
+ SequenceNumber seq = kMaxSequenceNumber;
+ bool found_record_for_key = false;
+
+ // When min_uncommitted == kMaxSequenceNumber, writes are committed in
+ // sequence number order, so only keys larger than `snap_seq` can cause
+ // conflict.
+ // When min_uncommitted != kMaxSequenceNumber, keys lower than
+ // min_uncommitted will not triggered conflicts, while keys larger than
+ // min_uncommitted might create conflicts, so we need to read them out
+ // from the DB, and call callback to snap_checker to determine. So only
+ // keys lower than min_uncommitted can be skipped.
+ SequenceNumber lower_bound_seq =
+ (min_uncommitted == kMaxSequenceNumber) ? snap_seq : min_uncommitted;
+ Status s = db_impl->GetLatestSequenceForKey(sv, key, !need_to_read_sst,
+ lower_bound_seq, &seq,
+ &found_record_for_key);
+
+ if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
+ result = s;
+ } else if (found_record_for_key) {
+ bool write_conflict = snap_checker == nullptr
+ ? snap_seq < seq
+ : !snap_checker->IsVisible(seq);
+ if (write_conflict) {
+ result = Status::Busy();
+ }
+ }
+ }
+
+ return result;
+}
+
+Status TransactionUtil::CheckKeysForConflicts(DBImpl* db_impl,
+ const TransactionKeyMap& key_map,
+ bool cache_only) {
+ Status result;
+
+ for (auto& key_map_iter : key_map) {
+ uint32_t cf_id = key_map_iter.first;
+ const auto& keys = key_map_iter.second;
+
+ SuperVersion* sv = db_impl->GetAndRefSuperVersion(cf_id);
+ if (sv == nullptr) {
+ result = Status::InvalidArgument("Could not access column family " +
+ ToString(cf_id));
+ break;
+ }
+
+ SequenceNumber earliest_seq =
+ db_impl->GetEarliestMemTableSequenceNumber(sv, true);
+
+ // For each of the keys in this transaction, check to see if someone has
+ // written to this key since the start of the transaction.
+ for (const auto& key_iter : keys) {
+ const auto& key = key_iter.first;
+ const SequenceNumber key_seq = key_iter.second.seq;
+
+ result = CheckKey(db_impl, sv, earliest_seq, key_seq, key, cache_only);
+
+ if (!result.ok()) {
+ break;
+ }
+ }
+
+ db_impl->ReturnAndCleanupSuperVersion(cf_id, sv);
+
+ if (!result.ok()) {
+ break;
+ }
+ }
+
+ return result;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/transaction_util.h b/src/rocksdb/utilities/transactions/transaction_util.h
new file mode 100644
index 000000000..2e48f84a4
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/transaction_util.h
@@ -0,0 +1,103 @@
+// 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 <string>
+#include <unordered_map>
+
+#include "db/dbformat.h"
+#include "db/read_callback.h"
+
+#include "rocksdb/db.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/status.h"
+#include "rocksdb/types.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct TransactionKeyMapInfo {
+ // Earliest sequence number that is relevant to this transaction for this key
+ SequenceNumber seq;
+
+ uint32_t num_writes;
+ uint32_t num_reads;
+
+ bool exclusive;
+
+ explicit TransactionKeyMapInfo(SequenceNumber seq_no)
+ : seq(seq_no), num_writes(0), num_reads(0), exclusive(false) {}
+
+ // Used in PopSavePoint to collapse two savepoints together.
+ void Merge(const TransactionKeyMapInfo& info) {
+ assert(seq <= info.seq);
+ num_reads += info.num_reads;
+ num_writes += info.num_writes;
+ exclusive |= info.exclusive;
+ }
+};
+
+using TransactionKeyMap =
+ std::unordered_map<uint32_t,
+ std::unordered_map<std::string, TransactionKeyMapInfo>>;
+
+class DBImpl;
+struct SuperVersion;
+class WriteBatchWithIndex;
+
+class TransactionUtil {
+ public:
+ // Verifies there have been no commits to this key in the db since this
+ // sequence number.
+ //
+ // If cache_only is true, then this function will not attempt to read any
+ // SST files. This will make it more likely this function will
+ // return an error if it is unable to determine if there are any conflicts.
+ //
+ // See comment of CheckKey() for explanation of `snap_seq`, `snap_checker`
+ // and `min_uncommitted`.
+ //
+ // Returns OK on success, BUSY if there is a conflicting write, or other error
+ // status for any unexpected errors.
+ static Status CheckKeyForConflicts(
+ DBImpl* db_impl, ColumnFamilyHandle* column_family,
+ const std::string& key, SequenceNumber snap_seq, bool cache_only,
+ ReadCallback* snap_checker = nullptr,
+ SequenceNumber min_uncommitted = kMaxSequenceNumber);
+
+ // For each key,SequenceNumber pair in the TransactionKeyMap, this function
+ // will verify there have been no writes to the key in the db since that
+ // sequence number.
+ //
+ // Returns OK on success, BUSY if there is a conflicting write, or other error
+ // status for any unexpected errors.
+ //
+ // REQUIRED: this function should only be called on the write thread or if the
+ // mutex is held.
+ static Status CheckKeysForConflicts(DBImpl* db_impl,
+ const TransactionKeyMap& keys,
+ bool cache_only);
+
+ private:
+ // If `snap_checker` == nullptr, writes are always commited in sequence number
+ // order. All sequence number <= `snap_seq` will not conflict with any
+ // write, and all keys > `snap_seq` of `key` will trigger conflict.
+ // If `snap_checker` != nullptr, writes may not commit in sequence number
+ // order. In this case `min_uncommitted` is a lower bound.
+ // seq < `min_uncommitted`: no conflict
+ // seq > `snap_seq`: applicable to conflict
+ // `min_uncommitted` <= seq <= `snap_seq`: call `snap_checker` to determine.
+ static Status CheckKey(DBImpl* db_impl, SuperVersion* sv,
+ SequenceNumber earliest_seq, SequenceNumber snap_seq,
+ const std::string& key, bool cache_only,
+ ReadCallback* snap_checker = nullptr,
+ SequenceNumber min_uncommitted = kMaxSequenceNumber);
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_prepared_transaction_test.cc b/src/rocksdb/utilities/transactions/write_prepared_transaction_test.cc
new file mode 100644
index 000000000..0171b9716
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_prepared_transaction_test.cc
@@ -0,0 +1,3524 @@
+// 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_test.h"
+
+#include <algorithm>
+#include <atomic>
+#include <cinttypes>
+#include <functional>
+#include <string>
+#include <thread>
+
+#include "db/db_impl/db_impl.h"
+#include "db/dbformat.h"
+#include "rocksdb/db.h"
+#include "rocksdb/options.h"
+#include "rocksdb/types.h"
+#include "rocksdb/utilities/debug.h"
+#include "rocksdb/utilities/transaction.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "table/mock_table.h"
+#include "test_util/fault_injection_test_env.h"
+#include "test_util/sync_point.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "test_util/transaction_test_util.h"
+#include "util/mutexlock.h"
+#include "util/random.h"
+#include "util/string_util.h"
+#include "utilities/merge_operators.h"
+#include "utilities/merge_operators/string_append/stringappend.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+#include "utilities/transactions/write_prepared_txn_db.h"
+
+#include "port/port.h"
+
+using std::string;
+
+namespace ROCKSDB_NAMESPACE {
+
+using CommitEntry = WritePreparedTxnDB::CommitEntry;
+using CommitEntry64b = WritePreparedTxnDB::CommitEntry64b;
+using CommitEntry64bFormat = WritePreparedTxnDB::CommitEntry64bFormat;
+
+TEST(PreparedHeap, BasicsTest) {
+ WritePreparedTxnDB::PreparedHeap heap;
+ {
+ MutexLock ml(heap.push_pop_mutex());
+ heap.push(14l);
+ // Test with one element
+ ASSERT_EQ(14l, heap.top());
+ heap.push(24l);
+ heap.push(34l);
+ // Test that old min is still on top
+ ASSERT_EQ(14l, heap.top());
+ heap.push(44l);
+ heap.push(54l);
+ heap.push(64l);
+ heap.push(74l);
+ heap.push(84l);
+ }
+ // Test that old min is still on top
+ ASSERT_EQ(14l, heap.top());
+ heap.erase(24l);
+ // Test that old min is still on top
+ ASSERT_EQ(14l, heap.top());
+ heap.erase(14l);
+ // Test that the new comes to the top after multiple erase
+ ASSERT_EQ(34l, heap.top());
+ heap.erase(34l);
+ // Test that the new comes to the top after single erase
+ ASSERT_EQ(44l, heap.top());
+ heap.erase(54l);
+ ASSERT_EQ(44l, heap.top());
+ heap.pop(); // pop 44l
+ // Test that the erased items are ignored after pop
+ ASSERT_EQ(64l, heap.top());
+ heap.erase(44l);
+ // Test that erasing an already popped item would work
+ ASSERT_EQ(64l, heap.top());
+ heap.erase(84l);
+ ASSERT_EQ(64l, heap.top());
+ {
+ MutexLock ml(heap.push_pop_mutex());
+ heap.push(85l);
+ heap.push(86l);
+ heap.push(87l);
+ heap.push(88l);
+ heap.push(89l);
+ }
+ heap.erase(87l);
+ heap.erase(85l);
+ heap.erase(89l);
+ heap.erase(86l);
+ heap.erase(88l);
+ // Test top remains the same after a random order of many erases
+ ASSERT_EQ(64l, heap.top());
+ heap.pop();
+ // Test that pop works with a series of random pending erases
+ ASSERT_EQ(74l, heap.top());
+ ASSERT_FALSE(heap.empty());
+ heap.pop();
+ // Test that empty works
+ ASSERT_TRUE(heap.empty());
+}
+
+// This is a scenario reconstructed from a buggy trace. Test that the bug does
+// not resurface again.
+TEST(PreparedHeap, EmptyAtTheEnd) {
+ WritePreparedTxnDB::PreparedHeap heap;
+ {
+ MutexLock ml(heap.push_pop_mutex());
+ heap.push(40l);
+ }
+ ASSERT_EQ(40l, heap.top());
+ // Although not a recommended scenario, we must be resilient against erase
+ // without a prior push.
+ heap.erase(50l);
+ ASSERT_EQ(40l, heap.top());
+ {
+ MutexLock ml(heap.push_pop_mutex());
+ heap.push(60l);
+ }
+ ASSERT_EQ(40l, heap.top());
+
+ heap.erase(60l);
+ ASSERT_EQ(40l, heap.top());
+ heap.erase(40l);
+ ASSERT_TRUE(heap.empty());
+
+ {
+ MutexLock ml(heap.push_pop_mutex());
+ heap.push(40l);
+ }
+ ASSERT_EQ(40l, heap.top());
+ heap.erase(50l);
+ ASSERT_EQ(40l, heap.top());
+ {
+ MutexLock ml(heap.push_pop_mutex());
+ heap.push(60l);
+ }
+ ASSERT_EQ(40l, heap.top());
+
+ heap.erase(40l);
+ // Test that the erase has not emptied the heap (we had a bug doing that)
+ ASSERT_FALSE(heap.empty());
+ ASSERT_EQ(60l, heap.top());
+ heap.erase(60l);
+ ASSERT_TRUE(heap.empty());
+}
+
+// Generate random order of PreparedHeap access and test that the heap will be
+// successfully emptied at the end.
+TEST(PreparedHeap, Concurrent) {
+ const size_t t_cnt = 10;
+ ROCKSDB_NAMESPACE::port::Thread t[t_cnt + 1];
+ WritePreparedTxnDB::PreparedHeap heap;
+ port::RWMutex prepared_mutex;
+ std::atomic<size_t> last;
+
+ for (size_t n = 0; n < 100; n++) {
+ last = 0;
+ t[0] = ROCKSDB_NAMESPACE::port::Thread([&]() {
+ Random rnd(1103);
+ for (size_t seq = 1; seq <= t_cnt; seq++) {
+ // This is not recommended usage but we should be resilient against it.
+ bool skip_push = rnd.OneIn(5);
+ if (!skip_push) {
+ MutexLock ml(heap.push_pop_mutex());
+ std::this_thread::yield();
+ heap.push(seq);
+ last.store(seq);
+ }
+ }
+ });
+ for (size_t i = 1; i <= t_cnt; i++) {
+ t[i] =
+ ROCKSDB_NAMESPACE::port::Thread([&heap, &prepared_mutex, &last, i]() {
+ auto seq = i;
+ do {
+ std::this_thread::yield();
+ } while (last.load() < seq);
+ WriteLock wl(&prepared_mutex);
+ heap.erase(seq);
+ });
+ }
+ for (size_t i = 0; i <= t_cnt; i++) {
+ t[i].join();
+ }
+ ASSERT_TRUE(heap.empty());
+ }
+}
+
+// Test that WriteBatchWithIndex correctly counts the number of sub-batches
+TEST(WriteBatchWithIndex, SubBatchCnt) {
+ ColumnFamilyOptions cf_options;
+ std::string cf_name = "two";
+ DB* db;
+ Options options;
+ options.create_if_missing = true;
+ const std::string dbname = test::PerThreadDBPath("transaction_testdb");
+ DestroyDB(dbname, options);
+ ASSERT_OK(DB::Open(options, dbname, &db));
+ ColumnFamilyHandle* cf_handle = nullptr;
+ ASSERT_OK(db->CreateColumnFamily(cf_options, cf_name, &cf_handle));
+ WriteOptions write_options;
+ size_t batch_cnt = 1;
+ size_t save_points = 0;
+ std::vector<size_t> batch_cnt_at;
+ WriteBatchWithIndex batch(db->DefaultColumnFamily()->GetComparator(), 0, true,
+ 0);
+ ASSERT_EQ(batch_cnt, batch.SubBatchCnt());
+ batch_cnt_at.push_back(batch_cnt);
+ batch.SetSavePoint();
+ save_points++;
+ batch.Put(Slice("key"), Slice("value"));
+ ASSERT_EQ(batch_cnt, batch.SubBatchCnt());
+ batch_cnt_at.push_back(batch_cnt);
+ batch.SetSavePoint();
+ save_points++;
+ batch.Put(Slice("key2"), Slice("value2"));
+ ASSERT_EQ(batch_cnt, batch.SubBatchCnt());
+ // duplicate the keys
+ batch_cnt_at.push_back(batch_cnt);
+ batch.SetSavePoint();
+ save_points++;
+ batch.Put(Slice("key"), Slice("value3"));
+ batch_cnt++;
+ ASSERT_EQ(batch_cnt, batch.SubBatchCnt());
+ // duplicate the 2nd key. It should not be counted duplicate since a
+ // sub-patch is cut after the last duplicate.
+ batch_cnt_at.push_back(batch_cnt);
+ batch.SetSavePoint();
+ save_points++;
+ batch.Put(Slice("key2"), Slice("value4"));
+ ASSERT_EQ(batch_cnt, batch.SubBatchCnt());
+ // duplicate the keys but in a different cf. It should not be counted as
+ // duplicate keys
+ batch_cnt_at.push_back(batch_cnt);
+ batch.SetSavePoint();
+ save_points++;
+ batch.Put(cf_handle, Slice("key"), Slice("value5"));
+ ASSERT_EQ(batch_cnt, batch.SubBatchCnt());
+
+ // Test that the number of sub-batches matches what we count with
+ // SubBatchCounter
+ std::map<uint32_t, const Comparator*> comparators;
+ comparators[0] = db->DefaultColumnFamily()->GetComparator();
+ comparators[cf_handle->GetID()] = cf_handle->GetComparator();
+ SubBatchCounter counter(comparators);
+ ASSERT_OK(batch.GetWriteBatch()->Iterate(&counter));
+ ASSERT_EQ(batch_cnt, counter.BatchCount());
+
+ // Test that RollbackToSavePoint will properly resets the number of
+ // sub-batches
+ for (size_t i = save_points; i > 0; i--) {
+ batch.RollbackToSavePoint();
+ ASSERT_EQ(batch_cnt_at[i - 1], batch.SubBatchCnt());
+ }
+
+ // Test the count is right with random batches
+ {
+ const size_t TOTAL_KEYS = 20; // 20 ~= 10 to cause a few randoms
+ Random rnd(1131);
+ std::string keys[TOTAL_KEYS];
+ for (size_t k = 0; k < TOTAL_KEYS; k++) {
+ int len = static_cast<int>(rnd.Uniform(50));
+ keys[k] = test::RandomKey(&rnd, len);
+ }
+ for (size_t i = 0; i < 1000; i++) { // 1000 random batches
+ WriteBatchWithIndex rndbatch(db->DefaultColumnFamily()->GetComparator(),
+ 0, true, 0);
+ for (size_t k = 0; k < 10; k++) { // 10 key per batch
+ size_t ki = static_cast<size_t>(rnd.Uniform(TOTAL_KEYS));
+ Slice key = Slice(keys[ki]);
+ std::string buffer;
+ Slice value = Slice(test::RandomString(&rnd, 16, &buffer));
+ rndbatch.Put(key, value);
+ }
+ SubBatchCounter batch_counter(comparators);
+ ASSERT_OK(rndbatch.GetWriteBatch()->Iterate(&batch_counter));
+ ASSERT_EQ(rndbatch.SubBatchCnt(), batch_counter.BatchCount());
+ }
+ }
+
+ delete cf_handle;
+ delete db;
+}
+
+TEST(CommitEntry64b, BasicTest) {
+ const size_t INDEX_BITS = static_cast<size_t>(21);
+ const size_t INDEX_SIZE = static_cast<size_t>(1ull << INDEX_BITS);
+ const CommitEntry64bFormat FORMAT(static_cast<size_t>(INDEX_BITS));
+
+ // zero-initialized CommitEntry64b should indicate an empty entry
+ CommitEntry64b empty_entry64b;
+ uint64_t empty_index = 11ul;
+ CommitEntry empty_entry;
+ bool ok = empty_entry64b.Parse(empty_index, &empty_entry, FORMAT);
+ ASSERT_FALSE(ok);
+
+ // the zero entry is reserved for un-initialized entries
+ const size_t MAX_COMMIT = (1 << FORMAT.COMMIT_BITS) - 1 - 1;
+ // Samples over the numbers that are covered by that many index bits
+ std::array<uint64_t, 4> is = {{0, 1, INDEX_SIZE / 2 + 1, INDEX_SIZE - 1}};
+ // Samples over the numbers that are covered by that many commit bits
+ std::array<uint64_t, 4> ds = {{0, 1, MAX_COMMIT / 2 + 1, MAX_COMMIT}};
+ // Iterate over prepare numbers that have i) cover all bits of a sequence
+ // number, and ii) include some bits that fall into the range of index or
+ // commit bits
+ for (uint64_t base = 1; base < kMaxSequenceNumber; base *= 2) {
+ for (uint64_t i : is) {
+ for (uint64_t d : ds) {
+ uint64_t p = base + i + d;
+ for (uint64_t c : {p, p + d / 2, p + d}) {
+ uint64_t index = p % INDEX_SIZE;
+ CommitEntry before(p, c), after;
+ CommitEntry64b entry64b(before, FORMAT);
+ ok = entry64b.Parse(index, &after, FORMAT);
+ ASSERT_TRUE(ok);
+ if (!(before == after)) {
+ printf("base %" PRIu64 " i %" PRIu64 " d %" PRIu64 " p %" PRIu64
+ " c %" PRIu64 " index %" PRIu64 "\n",
+ base, i, d, p, c, index);
+ }
+ ASSERT_EQ(before, after);
+ }
+ }
+ }
+ }
+}
+
+class WritePreparedTxnDBMock : public WritePreparedTxnDB {
+ public:
+ WritePreparedTxnDBMock(DBImpl* db_impl, TransactionDBOptions& opt)
+ : WritePreparedTxnDB(db_impl, opt) {}
+ void SetDBSnapshots(const std::vector<SequenceNumber>& snapshots) {
+ snapshots_ = snapshots;
+ }
+ void TakeSnapshot(SequenceNumber seq) { snapshots_.push_back(seq); }
+
+ protected:
+ const std::vector<SequenceNumber> GetSnapshotListFromDB(
+ SequenceNumber /* unused */) override {
+ return snapshots_;
+ }
+
+ private:
+ std::vector<SequenceNumber> snapshots_;
+};
+
+class WritePreparedTransactionTestBase : public TransactionTestBase {
+ public:
+ WritePreparedTransactionTestBase(bool use_stackable_db, bool two_write_queue,
+ TxnDBWritePolicy write_policy,
+ WriteOrdering write_ordering)
+ : TransactionTestBase(use_stackable_db, two_write_queue, write_policy,
+ write_ordering){};
+
+ protected:
+ void UpdateTransactionDBOptions(size_t snapshot_cache_bits,
+ size_t commit_cache_bits) {
+ txn_db_options.wp_snapshot_cache_bits = snapshot_cache_bits;
+ txn_db_options.wp_commit_cache_bits = commit_cache_bits;
+ }
+ void UpdateTransactionDBOptions(size_t snapshot_cache_bits) {
+ txn_db_options.wp_snapshot_cache_bits = snapshot_cache_bits;
+ }
+ // If expect_update is set, check if it actually updated old_commit_map_. If
+ // it did not and yet suggested not to check the next snapshot, do the
+ // opposite to check if it was not a bad suggestion.
+ void MaybeUpdateOldCommitMapTestWithNext(uint64_t prepare, uint64_t commit,
+ uint64_t snapshot,
+ uint64_t next_snapshot,
+ bool expect_update) {
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ // reset old_commit_map_empty_ so that its value indicate whether
+ // old_commit_map_ was updated
+ wp_db->old_commit_map_empty_ = true;
+ bool check_next = wp_db->MaybeUpdateOldCommitMap(prepare, commit, snapshot,
+ snapshot < next_snapshot);
+ if (expect_update == wp_db->old_commit_map_empty_) {
+ printf("prepare: %" PRIu64 " commit: %" PRIu64 " snapshot: %" PRIu64
+ " next: %" PRIu64 "\n",
+ prepare, commit, snapshot, next_snapshot);
+ }
+ EXPECT_EQ(!expect_update, wp_db->old_commit_map_empty_);
+ if (!check_next && wp_db->old_commit_map_empty_) {
+ // do the opposite to make sure it was not a bad suggestion
+ const bool dont_care_bool = true;
+ wp_db->MaybeUpdateOldCommitMap(prepare, commit, next_snapshot,
+ dont_care_bool);
+ if (!wp_db->old_commit_map_empty_) {
+ printf("prepare: %" PRIu64 " commit: %" PRIu64 " snapshot: %" PRIu64
+ " next: %" PRIu64 "\n",
+ prepare, commit, snapshot, next_snapshot);
+ }
+ EXPECT_TRUE(wp_db->old_commit_map_empty_);
+ }
+ }
+
+ // Test that a CheckAgainstSnapshots thread reading old_snapshots will not
+ // miss a snapshot because of a concurrent update by UpdateSnapshots that is
+ // writing new_snapshots. Both threads are broken at two points. The sync
+ // points to enforce them are specified by a1, a2, b1, and b2. CommitEntry
+ // entry is expected to be vital for one of the snapshots that is common
+ // between the old and new list of snapshots.
+ void SnapshotConcurrentAccessTestInternal(
+ WritePreparedTxnDB* wp_db,
+ const std::vector<SequenceNumber>& old_snapshots,
+ const std::vector<SequenceNumber>& new_snapshots, CommitEntry& entry,
+ SequenceNumber& version, size_t a1, size_t a2, size_t b1, size_t b2) {
+ // First reset the snapshot list
+ const std::vector<SequenceNumber> empty_snapshots;
+ wp_db->old_commit_map_empty_ = true;
+ wp_db->UpdateSnapshots(empty_snapshots, ++version);
+ // Then initialize it with the old_snapshots
+ wp_db->UpdateSnapshots(old_snapshots, ++version);
+
+ // Starting from the first thread, cut each thread at two points
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"WritePreparedTxnDB::CheckAgainstSnapshots:p:" + std::to_string(a1),
+ "WritePreparedTxnDB::UpdateSnapshots:s:start"},
+ {"WritePreparedTxnDB::UpdateSnapshots:p:" + std::to_string(b1),
+ "WritePreparedTxnDB::CheckAgainstSnapshots:s:" + std::to_string(a1)},
+ {"WritePreparedTxnDB::CheckAgainstSnapshots:p:" + std::to_string(a2),
+ "WritePreparedTxnDB::UpdateSnapshots:s:" + std::to_string(b1)},
+ {"WritePreparedTxnDB::UpdateSnapshots:p:" + std::to_string(b2),
+ "WritePreparedTxnDB::CheckAgainstSnapshots:s:" + std::to_string(a2)},
+ {"WritePreparedTxnDB::CheckAgainstSnapshots:p:end",
+ "WritePreparedTxnDB::UpdateSnapshots:s:" + std::to_string(b2)},
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ {
+ ASSERT_TRUE(wp_db->old_commit_map_empty_);
+ ROCKSDB_NAMESPACE::port::Thread t1(
+ [&]() { wp_db->UpdateSnapshots(new_snapshots, version); });
+ ROCKSDB_NAMESPACE::port::Thread t2(
+ [&]() { wp_db->CheckAgainstSnapshots(entry); });
+ t1.join();
+ t2.join();
+ ASSERT_FALSE(wp_db->old_commit_map_empty_);
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+
+ wp_db->old_commit_map_empty_ = true;
+ wp_db->UpdateSnapshots(empty_snapshots, ++version);
+ wp_db->UpdateSnapshots(old_snapshots, ++version);
+ // Starting from the second thread, cut each thread at two points
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"WritePreparedTxnDB::UpdateSnapshots:p:" + std::to_string(a1),
+ "WritePreparedTxnDB::CheckAgainstSnapshots:s:start"},
+ {"WritePreparedTxnDB::CheckAgainstSnapshots:p:" + std::to_string(b1),
+ "WritePreparedTxnDB::UpdateSnapshots:s:" + std::to_string(a1)},
+ {"WritePreparedTxnDB::UpdateSnapshots:p:" + std::to_string(a2),
+ "WritePreparedTxnDB::CheckAgainstSnapshots:s:" + std::to_string(b1)},
+ {"WritePreparedTxnDB::CheckAgainstSnapshots:p:" + std::to_string(b2),
+ "WritePreparedTxnDB::UpdateSnapshots:s:" + std::to_string(a2)},
+ {"WritePreparedTxnDB::UpdateSnapshots:p:end",
+ "WritePreparedTxnDB::CheckAgainstSnapshots:s:" + std::to_string(b2)},
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ {
+ ASSERT_TRUE(wp_db->old_commit_map_empty_);
+ ROCKSDB_NAMESPACE::port::Thread t1(
+ [&]() { wp_db->UpdateSnapshots(new_snapshots, version); });
+ ROCKSDB_NAMESPACE::port::Thread t2(
+ [&]() { wp_db->CheckAgainstSnapshots(entry); });
+ t1.join();
+ t2.join();
+ ASSERT_FALSE(wp_db->old_commit_map_empty_);
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ }
+
+ // Verify value of keys.
+ void VerifyKeys(const std::unordered_map<std::string, std::string>& data,
+ const Snapshot* snapshot = nullptr) {
+ std::string value;
+ ReadOptions read_options;
+ read_options.snapshot = snapshot;
+ for (auto& kv : data) {
+ auto s = db->Get(read_options, kv.first, &value);
+ ASSERT_TRUE(s.ok() || s.IsNotFound());
+ if (s.ok()) {
+ if (kv.second != value) {
+ printf("key = %s\n", kv.first.c_str());
+ }
+ ASSERT_EQ(kv.second, value);
+ } else {
+ ASSERT_EQ(kv.second, "NOT_FOUND");
+ }
+
+ // Try with MultiGet API too
+ std::vector<std::string> values;
+ auto s_vec = db->MultiGet(read_options, {db->DefaultColumnFamily()},
+ {kv.first}, &values);
+ ASSERT_EQ(1, values.size());
+ ASSERT_EQ(1, s_vec.size());
+ s = s_vec[0];
+ ASSERT_TRUE(s.ok() || s.IsNotFound());
+ if (s.ok()) {
+ ASSERT_TRUE(kv.second == values[0]);
+ } else {
+ ASSERT_EQ(kv.second, "NOT_FOUND");
+ }
+ }
+ }
+
+ // Verify all versions of keys.
+ void VerifyInternalKeys(const std::vector<KeyVersion>& expected_versions) {
+ std::vector<KeyVersion> versions;
+ const size_t kMaxKeys = 100000;
+ ASSERT_OK(GetAllKeyVersions(db, expected_versions.front().user_key,
+ expected_versions.back().user_key, kMaxKeys,
+ &versions));
+ ASSERT_EQ(expected_versions.size(), versions.size());
+ for (size_t i = 0; i < versions.size(); i++) {
+ ASSERT_EQ(expected_versions[i].user_key, versions[i].user_key);
+ ASSERT_EQ(expected_versions[i].sequence, versions[i].sequence);
+ ASSERT_EQ(expected_versions[i].type, versions[i].type);
+ if (versions[i].type != kTypeDeletion &&
+ versions[i].type != kTypeSingleDeletion) {
+ ASSERT_EQ(expected_versions[i].value, versions[i].value);
+ }
+ // Range delete not supported.
+ assert(expected_versions[i].type != kTypeRangeDeletion);
+ }
+ }
+};
+
+class WritePreparedTransactionTest
+ : public WritePreparedTransactionTestBase,
+ virtual public ::testing::WithParamInterface<
+ std::tuple<bool, bool, TxnDBWritePolicy, WriteOrdering>> {
+ public:
+ WritePreparedTransactionTest()
+ : WritePreparedTransactionTestBase(
+ std::get<0>(GetParam()), std::get<1>(GetParam()),
+ std::get<2>(GetParam()), std::get<3>(GetParam())){};
+};
+
+#ifndef ROCKSDB_VALGRIND_RUN
+class SnapshotConcurrentAccessTest
+ : public WritePreparedTransactionTestBase,
+ virtual public ::testing::WithParamInterface<std::tuple<
+ bool, bool, TxnDBWritePolicy, WriteOrdering, size_t, size_t>> {
+ public:
+ SnapshotConcurrentAccessTest()
+ : WritePreparedTransactionTestBase(
+ std::get<0>(GetParam()), std::get<1>(GetParam()),
+ std::get<2>(GetParam()), std::get<3>(GetParam())),
+ split_id_(std::get<4>(GetParam())),
+ split_cnt_(std::get<5>(GetParam())){};
+
+ protected:
+ // A test is split into split_cnt_ tests, each identified with split_id_ where
+ // 0 <= split_id_ < split_cnt_
+ size_t split_id_;
+ size_t split_cnt_;
+};
+#endif // ROCKSDB_VALGRIND_RUN
+
+class SeqAdvanceConcurrentTest
+ : public WritePreparedTransactionTestBase,
+ virtual public ::testing::WithParamInterface<std::tuple<
+ bool, bool, TxnDBWritePolicy, WriteOrdering, size_t, size_t>> {
+ public:
+ SeqAdvanceConcurrentTest()
+ : WritePreparedTransactionTestBase(
+ std::get<0>(GetParam()), std::get<1>(GetParam()),
+ std::get<2>(GetParam()), std::get<3>(GetParam())),
+ split_id_(std::get<4>(GetParam())),
+ split_cnt_(std::get<5>(GetParam())){};
+
+ protected:
+ // A test is split into split_cnt_ tests, each identified with split_id_ where
+ // 0 <= split_id_ < split_cnt_
+ size_t split_id_;
+ size_t split_cnt_;
+};
+
+INSTANTIATE_TEST_CASE_P(
+ WritePreparedTransaction, WritePreparedTransactionTest,
+ ::testing::Values(
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite)));
+
+#ifndef ROCKSDB_VALGRIND_RUN
+INSTANTIATE_TEST_CASE_P(
+ TwoWriteQueues, SnapshotConcurrentAccessTest,
+ ::testing::Values(
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 0, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 1, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 2, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 3, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 4, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 5, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 6, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 7, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 8, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 9, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 10, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 11, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 12, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 13, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 14, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 15, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 16, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 17, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 18, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 19, 20),
+
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 0, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 1, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 2, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 3, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 4, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 5, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 6, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 7, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 8, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 9, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 10, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 11, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 12, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 13, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 14, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 15, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 16, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 17, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 18, 20),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 19, 20)));
+
+INSTANTIATE_TEST_CASE_P(
+ OneWriteQueue, SnapshotConcurrentAccessTest,
+ ::testing::Values(
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 0, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 1, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 2, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 3, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 4, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 5, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 6, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 7, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 8, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 9, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 10, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 11, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 12, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 13, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 14, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 15, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 16, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 17, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 18, 20),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 19, 20)));
+
+INSTANTIATE_TEST_CASE_P(
+ TwoWriteQueues, SeqAdvanceConcurrentTest,
+ ::testing::Values(
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 0, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 1, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 2, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 3, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 4, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 5, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 6, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 7, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 8, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kOrderedWrite, 9, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 0, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 1, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 2, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 3, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 4, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 5, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 6, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 7, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 8, 10),
+ std::make_tuple(false, true, WRITE_PREPARED, kUnorderedWrite, 9, 10)));
+
+INSTANTIATE_TEST_CASE_P(
+ OneWriteQueue, SeqAdvanceConcurrentTest,
+ ::testing::Values(
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 0, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 1, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 2, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 3, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 4, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 5, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 6, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 7, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 8, 10),
+ std::make_tuple(false, false, WRITE_PREPARED, kOrderedWrite, 9, 10)));
+#endif // ROCKSDB_VALGRIND_RUN
+
+TEST_P(WritePreparedTransactionTest, CommitMap) {
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ assert(wp_db);
+ assert(wp_db->db_impl_);
+ size_t size = wp_db->COMMIT_CACHE_SIZE;
+ CommitEntry c = {5, 12}, e;
+ bool evicted = wp_db->AddCommitEntry(c.prep_seq % size, c, &e);
+ ASSERT_FALSE(evicted);
+
+ // Should be able to read the same value
+ CommitEntry64b dont_care;
+ bool found = wp_db->GetCommitEntry(c.prep_seq % size, &dont_care, &e);
+ ASSERT_TRUE(found);
+ ASSERT_EQ(c, e);
+ // Should be able to distinguish between overlapping entries
+ found = wp_db->GetCommitEntry((c.prep_seq + size) % size, &dont_care, &e);
+ ASSERT_TRUE(found);
+ ASSERT_NE(c.prep_seq + size, e.prep_seq);
+ // Should be able to detect non-existent entry
+ found = wp_db->GetCommitEntry((c.prep_seq + 1) % size, &dont_care, &e);
+ ASSERT_FALSE(found);
+
+ // Reject an invalid exchange
+ CommitEntry e2 = {c.prep_seq + size, c.commit_seq + size};
+ CommitEntry64b e2_64b(e2, wp_db->FORMAT);
+ bool exchanged = wp_db->ExchangeCommitEntry(e2.prep_seq % size, e2_64b, e);
+ ASSERT_FALSE(exchanged);
+ // check whether it did actually reject that
+ found = wp_db->GetCommitEntry(e2.prep_seq % size, &dont_care, &e);
+ ASSERT_TRUE(found);
+ ASSERT_EQ(c, e);
+
+ // Accept a valid exchange
+ CommitEntry64b c_64b(c, wp_db->FORMAT);
+ CommitEntry e3 = {c.prep_seq + size, c.commit_seq + size + 1};
+ exchanged = wp_db->ExchangeCommitEntry(c.prep_seq % size, c_64b, e3);
+ ASSERT_TRUE(exchanged);
+ // check whether it did actually accepted that
+ found = wp_db->GetCommitEntry(c.prep_seq % size, &dont_care, &e);
+ ASSERT_TRUE(found);
+ ASSERT_EQ(e3, e);
+
+ // Rewrite an entry
+ CommitEntry e4 = {e3.prep_seq + size, e3.commit_seq + size + 1};
+ evicted = wp_db->AddCommitEntry(e4.prep_seq % size, e4, &e);
+ ASSERT_TRUE(evicted);
+ ASSERT_EQ(e3, e);
+ found = wp_db->GetCommitEntry(e4.prep_seq % size, &dont_care, &e);
+ ASSERT_TRUE(found);
+ ASSERT_EQ(e4, e);
+}
+
+TEST_P(WritePreparedTransactionTest, MaybeUpdateOldCommitMap) {
+ // If prepare <= snapshot < commit we should keep the entry around since its
+ // nonexistence could be interpreted as committed in the snapshot while it is
+ // not true. We keep such entries around by adding them to the
+ // old_commit_map_.
+ uint64_t p /*prepare*/, c /*commit*/, s /*snapshot*/, ns /*next_snapshot*/;
+ p = 10l, c = 15l, s = 20l, ns = 21l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+ // If we do not expect the old commit map to be updated, try also with a next
+ // snapshot that is expected to update the old commit map. This would test
+ // that MaybeUpdateOldCommitMap would not prevent us from checking the next
+ // snapshot that must be checked.
+ p = 10l, c = 15l, s = 20l, ns = 11l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+
+ p = 10l, c = 20l, s = 20l, ns = 19l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+ p = 10l, c = 20l, s = 20l, ns = 21l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+
+ p = 20l, c = 20l, s = 20l, ns = 21l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+ p = 20l, c = 20l, s = 20l, ns = 19l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+
+ p = 10l, c = 25l, s = 20l, ns = 21l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, true);
+
+ p = 20l, c = 25l, s = 20l, ns = 21l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, true);
+
+ p = 21l, c = 25l, s = 20l, ns = 22l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+ p = 21l, c = 25l, s = 20l, ns = 19l;
+ MaybeUpdateOldCommitMapTestWithNext(p, c, s, ns, false);
+}
+
+// Trigger the condition where some old memtables are skipped when doing
+// TransactionUtil::CheckKey(), and make sure the result is still correct.
+TEST_P(WritePreparedTransactionTest, CheckKeySkipOldMemtable) {
+ const int kAttemptHistoryMemtable = 0;
+ const int kAttemptImmMemTable = 1;
+ for (int attempt = kAttemptHistoryMemtable; attempt <= kAttemptImmMemTable;
+ attempt++) {
+ options.max_write_buffer_number_to_maintain = 3;
+ ReOpen();
+
+ WriteOptions write_options;
+ ReadOptions read_options;
+ TransactionOptions txn_options;
+ txn_options.set_snapshot = true;
+ string value;
+ Status s;
+
+ ASSERT_OK(db->Put(write_options, Slice("foo"), Slice("bar")));
+ ASSERT_OK(db->Put(write_options, Slice("foo2"), Slice("bar")));
+
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn != nullptr);
+ ASSERT_OK(txn->SetName("txn"));
+
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn2 != nullptr);
+ ASSERT_OK(txn2->SetName("txn2"));
+
+ // This transaction is created to cause potential conflict.
+ Transaction* txn_x = db->BeginTransaction(write_options);
+ ASSERT_OK(txn_x->SetName("txn_x"));
+ ASSERT_OK(txn_x->Put(Slice("foo"), Slice("bar3")));
+ ASSERT_OK(txn_x->Prepare());
+
+ // Create snapshots after the prepare, but there should still
+ // be a conflict when trying to read "foo".
+
+ if (attempt == kAttemptImmMemTable) {
+ // For the second attempt, hold flush from beginning. The memtable
+ // will be switched to immutable after calling TEST_SwitchMemtable()
+ // while CheckKey() is called.
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"WritePreparedTransactionTest.CheckKeySkipOldMemtable",
+ "FlushJob::Start"}});
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ }
+
+ // force a memtable flush. The memtable should still be kept
+ FlushOptions flush_ops;
+ if (attempt == kAttemptHistoryMemtable) {
+ ASSERT_OK(db->Flush(flush_ops));
+ } else {
+ assert(attempt == kAttemptImmMemTable);
+ DBImpl* db_impl = static_cast<DBImpl*>(db->GetRootDB());
+ db_impl->TEST_SwitchMemtable();
+ }
+ uint64_t num_imm_mems;
+ ASSERT_TRUE(db->GetIntProperty(DB::Properties::kNumImmutableMemTable,
+ &num_imm_mems));
+ if (attempt == kAttemptHistoryMemtable) {
+ ASSERT_EQ(0, num_imm_mems);
+ } else {
+ assert(attempt == kAttemptImmMemTable);
+ ASSERT_EQ(1, num_imm_mems);
+ }
+
+ // Put something in active memtable
+ ASSERT_OK(db->Put(write_options, Slice("foo3"), Slice("bar")));
+
+ // Create txn3 after flushing, but this transaction also needs to
+ // check all memtables because of they contains uncommitted data.
+ Transaction* txn3 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn3 != nullptr);
+ ASSERT_OK(txn3->SetName("txn3"));
+
+ // Commit the pending write
+ ASSERT_OK(txn_x->Commit());
+
+ // Commit txn, txn2 and tx3. txn and tx3 will conflict but txn2 will
+ // pass. In all cases, both memtables are queried.
+ SetPerfLevel(PerfLevel::kEnableCount);
+ get_perf_context()->Reset();
+ ASSERT_TRUE(txn3->GetForUpdate(read_options, "foo", &value).IsBusy());
+ // We should have checked two memtables, active and either immutable
+ // or history memtable, depending on the test case.
+ ASSERT_EQ(2, get_perf_context()->get_from_memtable_count);
+
+ get_perf_context()->Reset();
+ ASSERT_TRUE(txn->GetForUpdate(read_options, "foo", &value).IsBusy());
+ // We should have checked two memtables, active and either immutable
+ // or history memtable, depending on the test case.
+ ASSERT_EQ(2, get_perf_context()->get_from_memtable_count);
+
+ get_perf_context()->Reset();
+ ASSERT_OK(txn2->GetForUpdate(read_options, "foo2", &value));
+ ASSERT_EQ(value, "bar");
+ // We should have checked two memtables, and since there is no
+ // conflict, another Get() will be made and fetch the data from
+ // DB. If it is in immutable memtable, two extra memtable reads
+ // will be issued. If it is not (in history), only one will
+ // be made, which is to the active memtable.
+ if (attempt == kAttemptHistoryMemtable) {
+ ASSERT_EQ(3, get_perf_context()->get_from_memtable_count);
+ } else {
+ assert(attempt == kAttemptImmMemTable);
+ ASSERT_EQ(4, get_perf_context()->get_from_memtable_count);
+ }
+
+ Transaction* txn4 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_TRUE(txn4 != nullptr);
+ ASSERT_OK(txn4->SetName("txn4"));
+ get_perf_context()->Reset();
+ ASSERT_OK(txn4->GetForUpdate(read_options, "foo", &value));
+ if (attempt == kAttemptHistoryMemtable) {
+ // Active memtable will be checked in snapshot validation and when
+ // getting the value.
+ ASSERT_EQ(2, get_perf_context()->get_from_memtable_count);
+ } else {
+ // Only active memtable will be checked in snapshot validation but
+ // both of active and immutable snapshot will be queried when
+ // getting the value.
+ assert(attempt == kAttemptImmMemTable);
+ ASSERT_EQ(3, get_perf_context()->get_from_memtable_count);
+ }
+
+ ASSERT_OK(txn2->Commit());
+ ASSERT_OK(txn4->Commit());
+
+ TEST_SYNC_POINT("WritePreparedTransactionTest.CheckKeySkipOldMemtable");
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+
+ SetPerfLevel(PerfLevel::kDisable);
+
+ delete txn;
+ delete txn2;
+ delete txn3;
+ delete txn4;
+ delete txn_x;
+ }
+}
+
+// Reproduce the bug with two snapshots with the same seuqence number and test
+// that the release of the first snapshot will not affect the reads by the other
+// snapshot
+TEST_P(WritePreparedTransactionTest, DoubleSnapshot) {
+ TransactionOptions txn_options;
+ Status s;
+
+ // Insert initial value
+ ASSERT_OK(db->Put(WriteOptions(), "key", "value1"));
+
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ Transaction* txn =
+ wp_db->BeginTransaction(WriteOptions(), txn_options, nullptr);
+ ASSERT_OK(txn->SetName("txn"));
+ ASSERT_OK(txn->Put("key", "value2"));
+ ASSERT_OK(txn->Prepare());
+ // Three snapshots with the same seq number
+ const Snapshot* snapshot0 = wp_db->GetSnapshot();
+ const Snapshot* snapshot1 = wp_db->GetSnapshot();
+ const Snapshot* snapshot2 = wp_db->GetSnapshot();
+ ASSERT_OK(txn->Commit());
+ SequenceNumber cache_size = wp_db->COMMIT_CACHE_SIZE;
+ SequenceNumber overlap_seq = txn->GetId() + cache_size;
+ delete txn;
+
+ // 4th snapshot with a larger seq
+ const Snapshot* snapshot3 = wp_db->GetSnapshot();
+ // Cause an eviction to advance max evicted seq number
+ // This also fetches the 4 snapshots from db since their seq is lower than the
+ // new max
+ wp_db->AddCommitted(overlap_seq, overlap_seq);
+
+ ReadOptions ropt;
+ // It should see the value before commit
+ ropt.snapshot = snapshot2;
+ PinnableSlice pinnable_val;
+ s = wp_db->Get(ropt, wp_db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == "value1");
+ pinnable_val.Reset();
+
+ wp_db->ReleaseSnapshot(snapshot1);
+
+ // It should still see the value before commit
+ s = wp_db->Get(ropt, wp_db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == "value1");
+ pinnable_val.Reset();
+
+ // Cause an eviction to advance max evicted seq number and trigger updating
+ // the snapshot list
+ overlap_seq += cache_size;
+ wp_db->AddCommitted(overlap_seq, overlap_seq);
+
+ // It should still see the value before commit
+ s = wp_db->Get(ropt, wp_db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_OK(s);
+ ASSERT_TRUE(pinnable_val == "value1");
+ pinnable_val.Reset();
+
+ wp_db->ReleaseSnapshot(snapshot0);
+ wp_db->ReleaseSnapshot(snapshot2);
+ wp_db->ReleaseSnapshot(snapshot3);
+}
+
+size_t UniqueCnt(std::vector<SequenceNumber> vec) {
+ std::set<SequenceNumber> aset;
+ for (auto i : vec) {
+ aset.insert(i);
+ }
+ return aset.size();
+}
+// Test that the entries in old_commit_map_ get garbage collected properly
+TEST_P(WritePreparedTransactionTest, OldCommitMapGC) {
+ const size_t snapshot_cache_bits = 0;
+ const size_t commit_cache_bits = 0;
+ DBImpl* mock_db = new DBImpl(options, dbname);
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ std::unique_ptr<WritePreparedTxnDBMock> wp_db(
+ new WritePreparedTxnDBMock(mock_db, txn_db_options));
+
+ SequenceNumber seq = 0;
+ // Take the first snapshot that overlaps with two txn
+ auto prep_seq = ++seq;
+ wp_db->AddPrepared(prep_seq);
+ auto prep_seq2 = ++seq;
+ wp_db->AddPrepared(prep_seq2);
+ auto snap_seq1 = seq;
+ wp_db->TakeSnapshot(snap_seq1);
+ auto commit_seq = ++seq;
+ wp_db->AddCommitted(prep_seq, commit_seq);
+ wp_db->RemovePrepared(prep_seq);
+ auto commit_seq2 = ++seq;
+ wp_db->AddCommitted(prep_seq2, commit_seq2);
+ wp_db->RemovePrepared(prep_seq2);
+ // Take the 2nd and 3rd snapshot that overlap with the same txn
+ prep_seq = ++seq;
+ wp_db->AddPrepared(prep_seq);
+ auto snap_seq2 = seq;
+ wp_db->TakeSnapshot(snap_seq2);
+ seq++;
+ auto snap_seq3 = seq;
+ wp_db->TakeSnapshot(snap_seq3);
+ seq++;
+ commit_seq = ++seq;
+ wp_db->AddCommitted(prep_seq, commit_seq);
+ wp_db->RemovePrepared(prep_seq);
+ // Make sure max_evicted_seq_ will be larger than 2nd snapshot by evicting the
+ // only item in the commit_cache_ via another commit.
+ prep_seq = ++seq;
+ wp_db->AddPrepared(prep_seq);
+ commit_seq = ++seq;
+ wp_db->AddCommitted(prep_seq, commit_seq);
+ wp_db->RemovePrepared(prep_seq);
+
+ // Verify that the evicted commit entries for all snapshots are in the
+ // old_commit_map_
+ {
+ ASSERT_FALSE(wp_db->old_commit_map_empty_.load());
+ ReadLock rl(&wp_db->old_commit_map_mutex_);
+ ASSERT_EQ(3, wp_db->old_commit_map_.size());
+ ASSERT_EQ(2, UniqueCnt(wp_db->old_commit_map_[snap_seq1]));
+ ASSERT_EQ(1, UniqueCnt(wp_db->old_commit_map_[snap_seq2]));
+ ASSERT_EQ(1, UniqueCnt(wp_db->old_commit_map_[snap_seq3]));
+ }
+
+ // Verify that the 2nd snapshot is cleaned up after the release
+ wp_db->ReleaseSnapshotInternal(snap_seq2);
+ {
+ ASSERT_FALSE(wp_db->old_commit_map_empty_.load());
+ ReadLock rl(&wp_db->old_commit_map_mutex_);
+ ASSERT_EQ(2, wp_db->old_commit_map_.size());
+ ASSERT_EQ(2, UniqueCnt(wp_db->old_commit_map_[snap_seq1]));
+ ASSERT_EQ(1, UniqueCnt(wp_db->old_commit_map_[snap_seq3]));
+ }
+
+ // Verify that the 1st snapshot is cleaned up after the release
+ wp_db->ReleaseSnapshotInternal(snap_seq1);
+ {
+ ASSERT_FALSE(wp_db->old_commit_map_empty_.load());
+ ReadLock rl(&wp_db->old_commit_map_mutex_);
+ ASSERT_EQ(1, wp_db->old_commit_map_.size());
+ ASSERT_EQ(1, UniqueCnt(wp_db->old_commit_map_[snap_seq3]));
+ }
+
+ // Verify that the 3rd snapshot is cleaned up after the release
+ wp_db->ReleaseSnapshotInternal(snap_seq3);
+ {
+ ASSERT_TRUE(wp_db->old_commit_map_empty_.load());
+ ReadLock rl(&wp_db->old_commit_map_mutex_);
+ ASSERT_EQ(0, wp_db->old_commit_map_.size());
+ }
+}
+
+TEST_P(WritePreparedTransactionTest, CheckAgainstSnapshots) {
+ std::vector<SequenceNumber> snapshots = {100l, 200l, 300l, 400l, 500l,
+ 600l, 700l, 800l, 900l};
+ const size_t snapshot_cache_bits = 2;
+ const uint64_t cache_size = 1ul << snapshot_cache_bits;
+ // Safety check to express the intended size in the test. Can be adjusted if
+ // the snapshots lists changed.
+ assert((1ul << snapshot_cache_bits) * 2 + 1 == snapshots.size());
+ DBImpl* mock_db = new DBImpl(options, dbname);
+ UpdateTransactionDBOptions(snapshot_cache_bits);
+ std::unique_ptr<WritePreparedTxnDBMock> wp_db(
+ new WritePreparedTxnDBMock(mock_db, txn_db_options));
+ SequenceNumber version = 1000l;
+ ASSERT_EQ(0, wp_db->snapshots_total_);
+ wp_db->UpdateSnapshots(snapshots, version);
+ ASSERT_EQ(snapshots.size(), wp_db->snapshots_total_);
+ // seq numbers are chosen so that we have two of them between each two
+ // snapshots. If the diff of two consecutive seq is more than 5, there is a
+ // snapshot between them.
+ std::vector<SequenceNumber> seqs = {50l, 55l, 150l, 155l, 250l, 255l, 350l,
+ 355l, 450l, 455l, 550l, 555l, 650l, 655l,
+ 750l, 755l, 850l, 855l, 950l, 955l};
+ assert(seqs.size() > 1);
+ for (size_t i = 0; i < seqs.size() - 1; i++) {
+ wp_db->old_commit_map_empty_ = true; // reset
+ CommitEntry commit_entry = {seqs[i], seqs[i + 1]};
+ wp_db->CheckAgainstSnapshots(commit_entry);
+ // Expect update if there is snapshot in between the prepare and commit
+ bool expect_update = commit_entry.commit_seq - commit_entry.prep_seq > 5 &&
+ commit_entry.commit_seq >= snapshots.front() &&
+ commit_entry.prep_seq <= snapshots.back();
+ ASSERT_EQ(expect_update, !wp_db->old_commit_map_empty_);
+ }
+
+ // Test that search will include multiple snapshot from snapshot cache
+ {
+ // exclude first and last item in the cache
+ CommitEntry commit_entry = {snapshots.front() + 1,
+ snapshots[cache_size - 1] - 1};
+ wp_db->old_commit_map_empty_ = true; // reset
+ wp_db->old_commit_map_.clear();
+ wp_db->CheckAgainstSnapshots(commit_entry);
+ ASSERT_EQ(wp_db->old_commit_map_.size(), cache_size - 2);
+ }
+
+ // Test that search will include multiple snapshot from old snapshots
+ {
+ // include two in the middle
+ CommitEntry commit_entry = {snapshots[cache_size] + 1,
+ snapshots[cache_size + 2] + 1};
+ wp_db->old_commit_map_empty_ = true; // reset
+ wp_db->old_commit_map_.clear();
+ wp_db->CheckAgainstSnapshots(commit_entry);
+ ASSERT_EQ(wp_db->old_commit_map_.size(), 2);
+ }
+
+ // Test that search will include both snapshot cache and old snapshots
+ // Case 1: includes all in snapshot cache
+ {
+ CommitEntry commit_entry = {snapshots.front() - 1, snapshots.back() + 1};
+ wp_db->old_commit_map_empty_ = true; // reset
+ wp_db->old_commit_map_.clear();
+ wp_db->CheckAgainstSnapshots(commit_entry);
+ ASSERT_EQ(wp_db->old_commit_map_.size(), snapshots.size());
+ }
+
+ // Case 2: includes all snapshot caches except the smallest
+ {
+ CommitEntry commit_entry = {snapshots.front() + 1, snapshots.back() + 1};
+ wp_db->old_commit_map_empty_ = true; // reset
+ wp_db->old_commit_map_.clear();
+ wp_db->CheckAgainstSnapshots(commit_entry);
+ ASSERT_EQ(wp_db->old_commit_map_.size(), snapshots.size() - 1);
+ }
+
+ // Case 3: includes only the largest of snapshot cache
+ {
+ CommitEntry commit_entry = {snapshots[cache_size - 1] - 1,
+ snapshots.back() + 1};
+ wp_db->old_commit_map_empty_ = true; // reset
+ wp_db->old_commit_map_.clear();
+ wp_db->CheckAgainstSnapshots(commit_entry);
+ ASSERT_EQ(wp_db->old_commit_map_.size(), snapshots.size() - cache_size + 1);
+ }
+}
+
+// This test is too slow for travis
+#ifndef TRAVIS
+#ifndef ROCKSDB_VALGRIND_RUN
+// Test that CheckAgainstSnapshots will not miss a live snapshot if it is run in
+// parallel with UpdateSnapshots.
+TEST_P(SnapshotConcurrentAccessTest, SnapshotConcurrentAccess) {
+ // We have a sync point in the method under test after checking each snapshot.
+ // If you increase the max number of snapshots in this test, more sync points
+ // in the methods must also be added.
+ const std::vector<SequenceNumber> snapshots = {10l, 20l, 30l, 40l, 50l,
+ 60l, 70l, 80l, 90l, 100l};
+ const size_t snapshot_cache_bits = 2;
+ // Safety check to express the intended size in the test. Can be adjusted if
+ // the snapshots lists changed.
+ assert((1ul << snapshot_cache_bits) * 2 + 2 == snapshots.size());
+ SequenceNumber version = 1000l;
+ // Choose the cache size so that the new snapshot list could replace all the
+ // existing items in the cache and also have some overflow.
+ DBImpl* mock_db = new DBImpl(options, dbname);
+ UpdateTransactionDBOptions(snapshot_cache_bits);
+ std::unique_ptr<WritePreparedTxnDBMock> wp_db(
+ new WritePreparedTxnDBMock(mock_db, txn_db_options));
+ const size_t extra = 2;
+ size_t loop_id = 0;
+ // Add up to extra items that do not fit into the cache
+ for (size_t old_size = 1; old_size <= wp_db->SNAPSHOT_CACHE_SIZE + extra;
+ old_size++) {
+ const std::vector<SequenceNumber> old_snapshots(
+ snapshots.begin(), snapshots.begin() + old_size);
+
+ // Each member of old snapshot might or might not appear in the new list. We
+ // create a common_snapshots for each combination.
+ size_t new_comb_cnt = size_t(1) << old_size;
+ for (size_t new_comb = 0; new_comb < new_comb_cnt; new_comb++, loop_id++) {
+ if (loop_id % split_cnt_ != split_id_) continue;
+ printf("."); // To signal progress
+ fflush(stdout);
+ std::vector<SequenceNumber> common_snapshots;
+ for (size_t i = 0; i < old_snapshots.size(); i++) {
+ if (IsInCombination(i, new_comb)) {
+ common_snapshots.push_back(old_snapshots[i]);
+ }
+ }
+ // And add some new snapshots to the common list
+ for (size_t added_snapshots = 0;
+ added_snapshots <= snapshots.size() - old_snapshots.size();
+ added_snapshots++) {
+ std::vector<SequenceNumber> new_snapshots = common_snapshots;
+ for (size_t i = 0; i < added_snapshots; i++) {
+ new_snapshots.push_back(snapshots[old_snapshots.size() + i]);
+ }
+ for (auto it = common_snapshots.begin(); it != common_snapshots.end();
+ ++it) {
+ auto snapshot = *it;
+ // Create a commit entry that is around the snapshot and thus should
+ // be not be discarded
+ CommitEntry entry = {static_cast<uint64_t>(snapshot - 1),
+ snapshot + 1};
+ // The critical part is when iterating the snapshot cache. Afterwards,
+ // we are operating under the lock
+ size_t a_range =
+ std::min(old_snapshots.size(), wp_db->SNAPSHOT_CACHE_SIZE) + 1;
+ size_t b_range =
+ std::min(new_snapshots.size(), wp_db->SNAPSHOT_CACHE_SIZE) + 1;
+ // Break each thread at two points
+ for (size_t a1 = 1; a1 <= a_range; a1++) {
+ for (size_t a2 = a1 + 1; a2 <= a_range; a2++) {
+ for (size_t b1 = 1; b1 <= b_range; b1++) {
+ for (size_t b2 = b1 + 1; b2 <= b_range; b2++) {
+ SnapshotConcurrentAccessTestInternal(
+ wp_db.get(), old_snapshots, new_snapshots, entry, version,
+ a1, a2, b1, b2);
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ printf("\n");
+}
+#endif // ROCKSDB_VALGRIND_RUN
+#endif // TRAVIS
+
+// This test clarifies the contract of AdvanceMaxEvictedSeq method
+TEST_P(WritePreparedTransactionTest, AdvanceMaxEvictedSeqBasic) {
+ DBImpl* mock_db = new DBImpl(options, dbname);
+ std::unique_ptr<WritePreparedTxnDBMock> wp_db(
+ new WritePreparedTxnDBMock(mock_db, txn_db_options));
+
+ // 1. Set the initial values for max, prepared, and snapshots
+ SequenceNumber zero_max = 0l;
+ // Set the initial list of prepared txns
+ const std::vector<SequenceNumber> initial_prepared = {10, 30, 50, 100,
+ 150, 200, 250};
+ for (auto p : initial_prepared) {
+ wp_db->AddPrepared(p);
+ }
+ // This updates the max value and also set old prepared
+ SequenceNumber init_max = 100;
+ wp_db->AdvanceMaxEvictedSeq(zero_max, init_max);
+ const std::vector<SequenceNumber> initial_snapshots = {20, 40};
+ wp_db->SetDBSnapshots(initial_snapshots);
+ // This will update the internal cache of snapshots from the DB
+ wp_db->UpdateSnapshots(initial_snapshots, init_max);
+
+ // 2. Invoke AdvanceMaxEvictedSeq
+ const std::vector<SequenceNumber> latest_snapshots = {20, 110, 220, 300};
+ wp_db->SetDBSnapshots(latest_snapshots);
+ SequenceNumber new_max = 200;
+ wp_db->AdvanceMaxEvictedSeq(init_max, new_max);
+
+ // 3. Verify that the state matches with AdvanceMaxEvictedSeq contract
+ // a. max should be updated to new_max
+ ASSERT_EQ(wp_db->max_evicted_seq_, new_max);
+ // b. delayed prepared should contain every txn <= max and prepared should
+ // only contain txns > max
+ auto it = initial_prepared.begin();
+ for (; it != initial_prepared.end() && *it <= new_max; ++it) {
+ ASSERT_EQ(1, wp_db->delayed_prepared_.erase(*it));
+ }
+ ASSERT_TRUE(wp_db->delayed_prepared_.empty());
+ for (; it != initial_prepared.end() && !wp_db->prepared_txns_.empty();
+ ++it, wp_db->prepared_txns_.pop()) {
+ ASSERT_EQ(*it, wp_db->prepared_txns_.top());
+ }
+ ASSERT_TRUE(it == initial_prepared.end());
+ ASSERT_TRUE(wp_db->prepared_txns_.empty());
+ // c. snapshots should contain everything below new_max
+ auto sit = latest_snapshots.begin();
+ for (size_t i = 0; sit != latest_snapshots.end() && *sit <= new_max &&
+ i < wp_db->snapshots_total_;
+ sit++, i++) {
+ ASSERT_TRUE(i < wp_db->snapshots_total_);
+ // This test is in small scale and the list of snapshots are assumed to be
+ // within the cache size limit. This is just a safety check to double check
+ // that assumption.
+ ASSERT_TRUE(i < wp_db->SNAPSHOT_CACHE_SIZE);
+ ASSERT_EQ(*sit, wp_db->snapshot_cache_[i]);
+ }
+}
+
+// A new snapshot should always be always larger than max_evicted_seq_
+// Otherwise the snapshot does not go through AdvanceMaxEvictedSeq
+TEST_P(WritePreparedTransactionTest, NewSnapshotLargerThanMax) {
+ WriteOptions woptions;
+ TransactionOptions txn_options;
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ Transaction* txn0 = db->BeginTransaction(woptions, txn_options);
+ ASSERT_OK(txn0->Put(Slice("key"), Slice("value")));
+ ASSERT_OK(txn0->Commit());
+ const SequenceNumber seq = txn0->GetId(); // is also prepare seq
+ delete txn0;
+ std::vector<Transaction*> txns;
+ // Inc seq without committing anything
+ for (int i = 0; i < 10; i++) {
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ ASSERT_OK(txn->SetName("xid" + std::to_string(i)));
+ ASSERT_OK(txn->Put(Slice("key" + std::to_string(i)), Slice("value")));
+ ASSERT_OK(txn->Prepare());
+ txns.push_back(txn);
+ }
+
+ // The new commit is seq + 10
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ auto snap = wp_db->GetSnapshot();
+ const SequenceNumber last_seq = snap->GetSequenceNumber();
+ wp_db->ReleaseSnapshot(snap);
+ ASSERT_LT(seq, last_seq);
+ // Otherwise our test is not effective
+ ASSERT_LT(last_seq - seq, wp_db->INC_STEP_FOR_MAX_EVICTED);
+
+ // Evict seq out of commit cache
+ const SequenceNumber overwrite_seq = seq + wp_db->COMMIT_CACHE_SIZE;
+ // Check that the next write could make max go beyond last
+ auto last_max = wp_db->max_evicted_seq_.load();
+ wp_db->AddCommitted(overwrite_seq, overwrite_seq);
+ // Check that eviction has advanced the max
+ ASSERT_LT(last_max, wp_db->max_evicted_seq_.load());
+ // Check that the new max has not advanced the last seq
+ ASSERT_LT(wp_db->max_evicted_seq_.load(), last_seq);
+ for (auto txn : txns) {
+ txn->Rollback();
+ delete txn;
+ }
+}
+
+// A new snapshot should always be always larger than max_evicted_seq_
+// In very rare cases max could be below last published seq. Test that
+// taking snapshot will wait for max to catch up.
+TEST_P(WritePreparedTransactionTest, MaxCatchupWithNewSnapshot) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 0; // only 1 entry => frequent eviction
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WriteOptions woptions;
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+
+ const int writes = 50;
+ const int batch_cnt = 4;
+ ROCKSDB_NAMESPACE::port::Thread t1([&]() {
+ for (int i = 0; i < writes; i++) {
+ WriteBatch batch;
+ // For duplicate keys cause 4 commit entries, each evicting an entry that
+ // is not published yet, thus causing max evicted seq go higher than last
+ // published.
+ for (int b = 0; b < batch_cnt; b++) {
+ batch.Put("foo", "foo");
+ }
+ db->Write(woptions, &batch);
+ }
+ });
+
+ ROCKSDB_NAMESPACE::port::Thread t2([&]() {
+ while (wp_db->max_evicted_seq_ == 0) { // wait for insert thread
+ std::this_thread::yield();
+ }
+ for (int i = 0; i < 10; i++) {
+ SequenceNumber max_lower_bound = wp_db->max_evicted_seq_;
+ auto snap = db->GetSnapshot();
+ if (snap->GetSequenceNumber() != 0) {
+ // Value of max_evicted_seq_ when snapshot was taken in unknown. We thus
+ // compare with the lower bound instead as an approximation.
+ ASSERT_LT(max_lower_bound, snap->GetSequenceNumber());
+ } // seq 0 is ok to be less than max since nothing is visible to it
+ db->ReleaseSnapshot(snap);
+ }
+ });
+
+ t1.join();
+ t2.join();
+
+ // Make sure that the test has worked and seq number has advanced as we
+ // thought
+ auto snap = db->GetSnapshot();
+ ASSERT_GT(snap->GetSequenceNumber(), batch_cnt * writes - 1);
+ db->ReleaseSnapshot(snap);
+}
+
+// Test that reads without snapshots would not hit an undefined state
+TEST_P(WritePreparedTransactionTest, MaxCatchupWithUnbackedSnapshot) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 0; // only 1 entry => frequent eviction
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WriteOptions woptions;
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+
+ const int writes = 50;
+ ROCKSDB_NAMESPACE::port::Thread t1([&]() {
+ for (int i = 0; i < writes; i++) {
+ WriteBatch batch;
+ batch.Put("key", "foo");
+ db->Write(woptions, &batch);
+ }
+ });
+
+ ROCKSDB_NAMESPACE::port::Thread t2([&]() {
+ while (wp_db->max_evicted_seq_ == 0) { // wait for insert thread
+ std::this_thread::yield();
+ }
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ TransactionOptions txn_options;
+ for (int i = 0; i < 10; i++) {
+ auto s = db->Get(ropt, db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_TRUE(s.ok() || s.IsTryAgain());
+ pinnable_val.Reset();
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ s = txn->Get(ropt, db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_TRUE(s.ok() || s.IsTryAgain());
+ pinnable_val.Reset();
+ std::vector<std::string> values;
+ auto s_vec =
+ txn->MultiGet(ropt, {db->DefaultColumnFamily()}, {"key"}, &values);
+ ASSERT_EQ(1, values.size());
+ ASSERT_EQ(1, s_vec.size());
+ s = s_vec[0];
+ ASSERT_TRUE(s.ok() || s.IsTryAgain());
+ Slice key("key");
+ txn->MultiGet(ropt, db->DefaultColumnFamily(), 1, &key, &pinnable_val, &s,
+ true);
+ ASSERT_TRUE(s.ok() || s.IsTryAgain());
+ delete txn;
+ }
+ });
+
+ t1.join();
+ t2.join();
+
+ // Make sure that the test has worked and seq number has advanced as we
+ // thought
+ auto snap = db->GetSnapshot();
+ ASSERT_GT(snap->GetSequenceNumber(), writes - 1);
+ db->ReleaseSnapshot(snap);
+}
+
+// Check that old_commit_map_ cleanup works correctly if the snapshot equals
+// max_evicted_seq_.
+TEST_P(WritePreparedTransactionTest, CleanupSnapshotEqualToMax) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 0; // only 1 entry => frequent eviction
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WriteOptions woptions;
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ // Insert something to increase seq
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ auto snap = db->GetSnapshot();
+ auto snap_seq = snap->GetSequenceNumber();
+ // Another insert should trigger eviction + load snapshot from db
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ // This is the scenario that we check agaisnt
+ ASSERT_EQ(snap_seq, wp_db->max_evicted_seq_);
+ // old_commit_map_ now has some data that needs gc
+ ASSERT_EQ(1, wp_db->snapshots_total_);
+ ASSERT_EQ(1, wp_db->old_commit_map_.size());
+
+ db->ReleaseSnapshot(snap);
+
+ // Another insert should trigger eviction + load snapshot from db
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+
+ // the snapshot and related metadata must be properly garbage collected
+ ASSERT_EQ(0, wp_db->snapshots_total_);
+ ASSERT_TRUE(wp_db->snapshots_all_.empty());
+ ASSERT_EQ(0, wp_db->old_commit_map_.size());
+}
+
+TEST_P(WritePreparedTransactionTest, AdvanceSeqByOne) {
+ auto snap = db->GetSnapshot();
+ auto seq1 = snap->GetSequenceNumber();
+ db->ReleaseSnapshot(snap);
+
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ wp_db->AdvanceSeqByOne();
+
+ snap = db->GetSnapshot();
+ auto seq2 = snap->GetSequenceNumber();
+ db->ReleaseSnapshot(snap);
+
+ ASSERT_LT(seq1, seq2);
+}
+
+// Test that the txn Initilize calls the overridden functions
+TEST_P(WritePreparedTransactionTest, TxnInitialize) {
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ ASSERT_OK(db->Put(write_options, "key", "value"));
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(Slice("key"), Slice("value1")));
+ ASSERT_OK(txn0->Prepare());
+
+ // SetSnapshot is overridden to update min_uncommitted_
+ txn_options.set_snapshot = true;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ auto snap = txn1->GetSnapshot();
+ auto snap_impl = reinterpret_cast<const SnapshotImpl*>(snap);
+ // If ::Initialize calls the overriden SetSnapshot, min_uncommitted_ must be
+ // udpated
+ ASSERT_GT(snap_impl->min_uncommitted_, kMinUnCommittedSeq);
+
+ txn0->Rollback();
+ txn1->Rollback();
+ delete txn0;
+ delete txn1;
+}
+
+// This tests that transactions with duplicate keys perform correctly after max
+// is advancing their prepared sequence numbers. This will not be the case if
+// for example the txn does not add the prepared seq for the second sub-batch to
+// the PreparedHeap structure.
+TEST_P(WritePreparedTransactionTest, AdvanceMaxEvictedSeqWithDuplicates) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 1; // disable commit cache
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn0->SetName("xid"));
+ ASSERT_OK(txn0->Put(Slice("key"), Slice("value1")));
+ ASSERT_OK(txn0->Put(Slice("key"), Slice("value2")));
+ ASSERT_OK(txn0->Prepare());
+
+ ASSERT_OK(db->Put(write_options, "key2", "value"));
+ // Will cause max advance due to disabled commit cache
+ ASSERT_OK(db->Put(write_options, "key3", "value"));
+
+ auto s = db->Get(ropt, db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ delete txn0;
+
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ wp_db->db_impl_->FlushWAL(true);
+ wp_db->TEST_Crash();
+ ReOpenNoDelete();
+ assert(db != nullptr);
+ s = db->Get(ropt, db->DefaultColumnFamily(), "key", &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+
+ txn0 = db->GetTransactionByName("xid");
+ ASSERT_OK(txn0->Rollback());
+ delete txn0;
+}
+
+#ifndef ROCKSDB_VALGRIND_RUN
+// Stress SmallestUnCommittedSeq, which reads from both prepared_txns_ and
+// delayed_prepared_, when is run concurrently with advancing max_evicted_seq,
+// which moves prepared txns from prepared_txns_ to delayed_prepared_.
+TEST_P(WritePreparedTransactionTest, SmallestUnCommittedSeq) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 1; // disable commit cache
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ std::vector<Transaction*> txns, committed_txns;
+
+ const int cnt = 100;
+ for (int i = 0; i < cnt; i++) {
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn->SetName("xid" + ToString(i)));
+ auto key = "key1" + ToString(i);
+ auto value = "value1" + ToString(i);
+ ASSERT_OK(txn->Put(Slice(key), Slice(value)));
+ ASSERT_OK(txn->Prepare());
+ txns.push_back(txn);
+ }
+
+ port::Mutex mutex;
+ Random rnd(1103);
+ ROCKSDB_NAMESPACE::port::Thread commit_thread([&]() {
+ for (int i = 0; i < cnt; i++) {
+ uint32_t index = rnd.Uniform(cnt - i);
+ Transaction* txn;
+ {
+ MutexLock l(&mutex);
+ txn = txns[index];
+ txns.erase(txns.begin() + index);
+ }
+ // Since commit cache is practically disabled, commit results in immediate
+ // advance in max_evicted_seq_ and subsequently moving some prepared txns
+ // to delayed_prepared_.
+ txn->Commit();
+ committed_txns.push_back(txn);
+ }
+ });
+ ROCKSDB_NAMESPACE::port::Thread read_thread([&]() {
+ while (1) {
+ MutexLock l(&mutex);
+ if (txns.empty()) {
+ break;
+ }
+ auto min_uncommitted = wp_db->SmallestUnCommittedSeq();
+ ASSERT_LE(min_uncommitted, (*txns.begin())->GetId());
+ }
+ });
+
+ commit_thread.join();
+ read_thread.join();
+ for (auto txn : committed_txns) {
+ delete txn;
+ }
+}
+#endif // ROCKSDB_VALGRIND_RUN
+
+TEST_P(SeqAdvanceConcurrentTest, SeqAdvanceConcurrent) {
+ // Given the sequential run of txns, with this timeout we should never see a
+ // deadlock nor a timeout unless we have a key conflict, which should be
+ // almost infeasible.
+ txn_db_options.transaction_lock_timeout = 1000;
+ txn_db_options.default_lock_timeout = 1000;
+ ReOpen();
+ FlushOptions fopt;
+
+ // Number of different txn types we use in this test
+ const size_t type_cnt = 5;
+ // The size of the first write group
+ // TODO(myabandeh): This should be increase for pre-release tests
+ const size_t first_group_size = 2;
+ // Total number of txns we run in each test
+ // TODO(myabandeh): This should be increase for pre-release tests
+ const size_t txn_cnt = first_group_size + 1;
+
+ size_t base[txn_cnt + 1] = {
+ 1,
+ };
+ for (size_t bi = 1; bi <= txn_cnt; bi++) {
+ base[bi] = base[bi - 1] * type_cnt;
+ }
+ const size_t max_n = static_cast<size_t>(std::pow(type_cnt, txn_cnt));
+ printf("Number of cases being tested is %" ROCKSDB_PRIszt "\n", max_n);
+ for (size_t n = 0; n < max_n; n++, ReOpen()) {
+ if (n % split_cnt_ != split_id_) continue;
+ if (n % 1000 == 0) {
+ printf("Tested %" ROCKSDB_PRIszt " cases so far\n", n);
+ }
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ auto seq = db_impl->TEST_GetLastVisibleSequence();
+ with_empty_commits = 0;
+ exp_seq = seq;
+ // This is increased before writing the batch for commit
+ commit_writes = 0;
+ // This is increased before txn starts linking if it expects to do a commit
+ // eventually
+ expected_commits = 0;
+ std::vector<port::Thread> threads;
+
+ linked = 0;
+ std::atomic<bool> batch_formed(false);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "WriteThread::EnterAsBatchGroupLeader:End",
+ [&](void* /*arg*/) { batch_formed = true; });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "WriteThread::JoinBatchGroup:Wait", [&](void* /*arg*/) {
+ linked++;
+ if (linked == 1) {
+ // Wait until the others are linked too.
+ while (linked < first_group_size) {
+ }
+ } else if (linked == 1 + first_group_size) {
+ // Make the 2nd batch of the rest of writes plus any followup
+ // commits from the first batch
+ while (linked < txn_cnt + commit_writes) {
+ }
+ }
+ // Then we will have one or more batches consisting of follow-up
+ // commits from the 2nd batch. There is a bit of non-determinism here
+ // but it should be tolerable.
+ });
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ for (size_t bi = 0; bi < txn_cnt; bi++) {
+ // get the bi-th digit in number system based on type_cnt
+ size_t d = (n % base[bi + 1]) / base[bi];
+ switch (d) {
+ case 0:
+ threads.emplace_back(txn_t0, bi);
+ break;
+ case 1:
+ threads.emplace_back(txn_t1, bi);
+ break;
+ case 2:
+ threads.emplace_back(txn_t2, bi);
+ break;
+ case 3:
+ threads.emplace_back(txn_t3, bi);
+ break;
+ case 4:
+ threads.emplace_back(txn_t3, bi);
+ break;
+ default:
+ assert(false);
+ }
+ // wait to be linked
+ while (linked.load() <= bi) {
+ }
+ // after a queue of size first_group_size
+ if (bi + 1 == first_group_size) {
+ while (!batch_formed) {
+ }
+ // to make it more deterministic, wait until the commits are linked
+ while (linked.load() <= bi + expected_commits) {
+ }
+ }
+ }
+ for (auto& t : threads) {
+ t.join();
+ }
+ if (options.two_write_queues) {
+ // In this case none of the above scheduling tricks to deterministically
+ // form merged batches works because the writes go to separate queues.
+ // This would result in different write groups in each run of the test. We
+ // still keep the test since although non-deterministic and hard to debug,
+ // it is still useful to have.
+ // TODO(myabandeh): Add a deterministic unit test for two_write_queues
+ }
+
+ // Check if memtable inserts advanced seq number as expected
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_EQ(exp_seq, seq);
+
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ // Check if recovery preserves the last sequence number
+ db_impl->FlushWAL(true);
+ ReOpenNoDelete();
+ assert(db != nullptr);
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ seq = db_impl->TEST_GetLastVisibleSequence();
+ ASSERT_LE(exp_seq, seq + with_empty_commits);
+
+ // Check if flush preserves the last sequence number
+ db_impl->Flush(fopt);
+ seq = db_impl->GetLatestSequenceNumber();
+ ASSERT_LE(exp_seq, seq + with_empty_commits);
+
+ // Check if recovery after flush preserves the last sequence number
+ db_impl->FlushWAL(true);
+ ReOpenNoDelete();
+ assert(db != nullptr);
+ db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ seq = db_impl->GetLatestSequenceNumber();
+ ASSERT_LE(exp_seq, seq + with_empty_commits);
+ }
+}
+
+// Run a couple of different txns among them some uncommitted. Restart the db at
+// a couple points to check whether the list of uncommitted txns are recovered
+// properly.
+TEST_P(WritePreparedTransactionTest, BasicRecovery) {
+ options.disable_auto_compactions = true;
+ ReOpen();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+
+ txn_t0(0);
+
+ TransactionOptions txn_options;
+ WriteOptions write_options;
+ size_t index = 1000;
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ auto istr0 = std::to_string(index);
+ auto s = txn0->SetName("xid" + istr0);
+ ASSERT_OK(s);
+ s = txn0->Put(Slice("foo0" + istr0), Slice("bar0" + istr0));
+ ASSERT_OK(s);
+ s = txn0->Prepare();
+ auto prep_seq_0 = txn0->GetId();
+
+ txn_t1(0);
+
+ index++;
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ auto istr1 = std::to_string(index);
+ s = txn1->SetName("xid" + istr1);
+ ASSERT_OK(s);
+ s = txn1->Put(Slice("foo1" + istr1), Slice("bar"));
+ ASSERT_OK(s);
+ s = txn1->Prepare();
+ auto prep_seq_1 = txn1->GetId();
+
+ txn_t2(0);
+
+ ReadOptions ropt;
+ PinnableSlice pinnable_val;
+ // Check the value is not committed before restart
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0" + istr0, &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ pinnable_val.Reset();
+
+ delete txn0;
+ delete txn1;
+ wp_db->db_impl_->FlushWAL(true);
+ wp_db->TEST_Crash();
+ ReOpenNoDelete();
+ assert(db != nullptr);
+ wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ // After recovery, all the uncommitted txns (0 and 1) should be inserted into
+ // delayed_prepared_
+ ASSERT_TRUE(wp_db->prepared_txns_.empty());
+ ASSERT_FALSE(wp_db->delayed_prepared_empty_);
+ ASSERT_LE(prep_seq_0, wp_db->max_evicted_seq_);
+ ASSERT_LE(prep_seq_1, wp_db->max_evicted_seq_);
+ {
+ ReadLock rl(&wp_db->prepared_mutex_);
+ ASSERT_EQ(2, wp_db->delayed_prepared_.size());
+ ASSERT_TRUE(wp_db->delayed_prepared_.find(prep_seq_0) !=
+ wp_db->delayed_prepared_.end());
+ ASSERT_TRUE(wp_db->delayed_prepared_.find(prep_seq_1) !=
+ wp_db->delayed_prepared_.end());
+ }
+
+ // Check the value is still not committed after restart
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0" + istr0, &pinnable_val);
+ ASSERT_TRUE(s.IsNotFound());
+ pinnable_val.Reset();
+
+ txn_t3(0);
+
+ // Test that a recovered txns will be properly marked committed for the next
+ // recovery
+ txn1 = db->GetTransactionByName("xid" + istr1);
+ ASSERT_NE(txn1, nullptr);
+ txn1->Commit();
+ delete txn1;
+
+ index++;
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ auto istr2 = std::to_string(index);
+ s = txn2->SetName("xid" + istr2);
+ ASSERT_OK(s);
+ s = txn2->Put(Slice("foo2" + istr2), Slice("bar"));
+ ASSERT_OK(s);
+ s = txn2->Prepare();
+ auto prep_seq_2 = txn2->GetId();
+
+ delete txn2;
+ wp_db->db_impl_->FlushWAL(true);
+ wp_db->TEST_Crash();
+ ReOpenNoDelete();
+ assert(db != nullptr);
+ wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ ASSERT_TRUE(wp_db->prepared_txns_.empty());
+ ASSERT_FALSE(wp_db->delayed_prepared_empty_);
+
+ // 0 and 2 are prepared and 1 is committed
+ {
+ ReadLock rl(&wp_db->prepared_mutex_);
+ ASSERT_EQ(2, wp_db->delayed_prepared_.size());
+ const auto& end = wp_db->delayed_prepared_.end();
+ ASSERT_NE(wp_db->delayed_prepared_.find(prep_seq_0), end);
+ ASSERT_EQ(wp_db->delayed_prepared_.find(prep_seq_1), end);
+ ASSERT_NE(wp_db->delayed_prepared_.find(prep_seq_2), end);
+ }
+ ASSERT_LE(prep_seq_0, wp_db->max_evicted_seq_);
+ ASSERT_LE(prep_seq_2, wp_db->max_evicted_seq_);
+
+ // Commit all the remaining txns
+ txn0 = db->GetTransactionByName("xid" + istr0);
+ ASSERT_NE(txn0, nullptr);
+ txn0->Commit();
+ txn2 = db->GetTransactionByName("xid" + istr2);
+ ASSERT_NE(txn2, nullptr);
+ txn2->Commit();
+
+ // Check the value is committed after commit
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0" + istr0, &pinnable_val);
+ ASSERT_TRUE(s.ok());
+ ASSERT_TRUE(pinnable_val == ("bar0" + istr0));
+ pinnable_val.Reset();
+
+ delete txn0;
+ delete txn2;
+ wp_db->db_impl_->FlushWAL(true);
+ ReOpenNoDelete();
+ assert(db != nullptr);
+ wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ ASSERT_TRUE(wp_db->prepared_txns_.empty());
+ ASSERT_TRUE(wp_db->delayed_prepared_empty_);
+
+ // Check the value is still committed after recovery
+ s = db->Get(ropt, db->DefaultColumnFamily(), "foo0" + istr0, &pinnable_val);
+ ASSERT_TRUE(s.ok());
+ ASSERT_TRUE(pinnable_val == ("bar0" + istr0));
+ pinnable_val.Reset();
+}
+
+// After recovery the commit map is empty while the max is set. The code would
+// go through a different path which requires a separate test. Test that the
+// committed data before the restart is visible to all snapshots.
+TEST_P(WritePreparedTransactionTest, IsInSnapshotEmptyMap) {
+ for (bool end_with_prepare : {false, true}) {
+ ReOpen();
+ WriteOptions woptions;
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ SequenceNumber prepare_seq = kMaxSequenceNumber;
+ if (end_with_prepare) {
+ TransactionOptions txn_options;
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ ASSERT_OK(txn->SetName("xid0"));
+ ASSERT_OK(txn->Prepare());
+ prepare_seq = txn->GetId();
+ delete txn;
+ }
+ dynamic_cast<WritePreparedTxnDB*>(db)->TEST_Crash();
+ auto db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ db_impl->FlushWAL(true);
+ ReOpenNoDelete();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ assert(wp_db != nullptr);
+ ASSERT_GT(wp_db->max_evicted_seq_, 0); // max after recovery
+ // Take a snapshot right after recovery
+ const Snapshot* snap = db->GetSnapshot();
+ auto snap_seq = snap->GetSequenceNumber();
+ ASSERT_GT(snap_seq, 0);
+
+ for (SequenceNumber seq = 0;
+ seq <= wp_db->max_evicted_seq_ && seq != prepare_seq; seq++) {
+ ASSERT_TRUE(wp_db->IsInSnapshot(seq, snap_seq));
+ }
+ if (end_with_prepare) {
+ ASSERT_FALSE(wp_db->IsInSnapshot(prepare_seq, snap_seq));
+ }
+ // trivial check
+ ASSERT_FALSE(wp_db->IsInSnapshot(snap_seq + 1, snap_seq));
+
+ db->ReleaseSnapshot(snap);
+
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ // Take a snapshot after some writes
+ snap = db->GetSnapshot();
+ snap_seq = snap->GetSequenceNumber();
+ for (SequenceNumber seq = 0;
+ seq <= wp_db->max_evicted_seq_ && seq != prepare_seq; seq++) {
+ ASSERT_TRUE(wp_db->IsInSnapshot(seq, snap_seq));
+ }
+ if (end_with_prepare) {
+ ASSERT_FALSE(wp_db->IsInSnapshot(prepare_seq, snap_seq));
+ }
+ // trivial check
+ ASSERT_FALSE(wp_db->IsInSnapshot(snap_seq + 1, snap_seq));
+
+ db->ReleaseSnapshot(snap);
+ }
+}
+
+// Shows the contract of IsInSnapshot when called on invalid/released snapshots
+TEST_P(WritePreparedTransactionTest, IsInSnapshotReleased) {
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ WriteOptions woptions;
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ // snap seq = 1
+ const Snapshot* snap1 = db->GetSnapshot();
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ ASSERT_OK(db->Put(woptions, "key", "value"));
+ // snap seq = 3
+ const Snapshot* snap2 = db->GetSnapshot();
+ const SequenceNumber seq = 1;
+ // Evict seq out of commit cache
+ size_t overwrite_seq = wp_db->COMMIT_CACHE_SIZE + seq;
+ wp_db->AddCommitted(overwrite_seq, overwrite_seq);
+ SequenceNumber snap_seq;
+ uint64_t min_uncommitted = kMinUnCommittedSeq;
+ bool released;
+
+ released = false;
+ snap_seq = snap1->GetSequenceNumber();
+ ASSERT_LE(seq, snap_seq);
+ // Valid snapshot lower than max
+ ASSERT_LE(snap_seq, wp_db->max_evicted_seq_);
+ ASSERT_TRUE(wp_db->IsInSnapshot(seq, snap_seq, min_uncommitted, &released));
+ ASSERT_FALSE(released);
+
+ released = false;
+ snap_seq = snap1->GetSequenceNumber();
+ // Invaid snapshot lower than max
+ ASSERT_LE(snap_seq + 1, wp_db->max_evicted_seq_);
+ ASSERT_TRUE(
+ wp_db->IsInSnapshot(seq, snap_seq + 1, min_uncommitted, &released));
+ ASSERT_TRUE(released);
+
+ db->ReleaseSnapshot(snap1);
+
+ released = false;
+ // Released snapshot lower than max
+ ASSERT_TRUE(wp_db->IsInSnapshot(seq, snap_seq, min_uncommitted, &released));
+ // The release does not take affect until the next max advance
+ ASSERT_FALSE(released);
+
+ released = false;
+ // Invaid snapshot lower than max
+ ASSERT_TRUE(
+ wp_db->IsInSnapshot(seq, snap_seq + 1, min_uncommitted, &released));
+ ASSERT_TRUE(released);
+
+ // This make the snapshot release to reflect in txn db structures
+ wp_db->AdvanceMaxEvictedSeq(wp_db->max_evicted_seq_,
+ wp_db->max_evicted_seq_ + 1);
+
+ released = false;
+ // Released snapshot lower than max
+ ASSERT_TRUE(wp_db->IsInSnapshot(seq, snap_seq, min_uncommitted, &released));
+ ASSERT_TRUE(released);
+
+ released = false;
+ // Invaid snapshot lower than max
+ ASSERT_TRUE(
+ wp_db->IsInSnapshot(seq, snap_seq + 1, min_uncommitted, &released));
+ ASSERT_TRUE(released);
+
+ snap_seq = snap2->GetSequenceNumber();
+
+ released = false;
+ // Unreleased snapshot lower than max
+ ASSERT_TRUE(wp_db->IsInSnapshot(seq, snap_seq, min_uncommitted, &released));
+ ASSERT_FALSE(released);
+
+ db->ReleaseSnapshot(snap2);
+}
+
+// Test WritePreparedTxnDB's IsInSnapshot against different ordering of
+// snapshot, max_committed_seq_, prepared, and commit entries.
+TEST_P(WritePreparedTransactionTest, IsInSnapshot) {
+ WriteOptions wo;
+ // Use small commit cache to trigger lots of eviction and fast advance of
+ // max_evicted_seq_
+ const size_t commit_cache_bits = 3;
+ // Same for snapshot cache size
+ const size_t snapshot_cache_bits = 2;
+
+ // Take some preliminary snapshots first. This is to stress the data structure
+ // that holds the old snapshots as it will be designed to be efficient when
+ // only a few snapshots are below the max_evicted_seq_.
+ for (int max_snapshots = 1; max_snapshots < 20; max_snapshots++) {
+ // Leave some gap between the preliminary snapshots and the final snapshot
+ // that we check. This should test for also different overlapping scenarios
+ // between the last snapshot and the commits.
+ for (int max_gap = 1; max_gap < 10; max_gap++) {
+ // Since we do not actually write to db, we mock the seq as it would be
+ // increased by the db. The only exception is that we need db seq to
+ // advance for our snapshots. for which we apply a dummy put each time we
+ // increase our mock of seq.
+ uint64_t seq = 0;
+ // At each step we prepare a txn and then we commit it in the next txn.
+ // This emulates the consecutive transactions that write to the same key
+ uint64_t cur_txn = 0;
+ // Number of snapshots taken so far
+ int num_snapshots = 0;
+ // Number of gaps applied so far
+ int gap_cnt = 0;
+ // The final snapshot that we will inspect
+ uint64_t snapshot = 0;
+ bool found_committed = false;
+ // To stress the data structure that maintain prepared txns, at each cycle
+ // we add a new prepare txn. These do not mean to be committed for
+ // snapshot inspection.
+ std::set<uint64_t> prepared;
+ // We keep the list of txns committed before we take the last snapshot.
+ // These should be the only seq numbers that will be found in the snapshot
+ std::set<uint64_t> committed_before;
+ // The set of commit seq numbers to be excluded from IsInSnapshot queries
+ std::set<uint64_t> commit_seqs;
+ DBImpl* mock_db = new DBImpl(options, dbname);
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ std::unique_ptr<WritePreparedTxnDBMock> wp_db(
+ new WritePreparedTxnDBMock(mock_db, txn_db_options));
+ // We continue until max advances a bit beyond the snapshot.
+ while (!snapshot || wp_db->max_evicted_seq_ < snapshot + 100) {
+ // do prepare for a transaction
+ seq++;
+ wp_db->AddPrepared(seq);
+ prepared.insert(seq);
+
+ // If cur_txn is not started, do prepare for it.
+ if (!cur_txn) {
+ seq++;
+ cur_txn = seq;
+ wp_db->AddPrepared(cur_txn);
+ } else { // else commit it
+ seq++;
+ wp_db->AddCommitted(cur_txn, seq);
+ wp_db->RemovePrepared(cur_txn);
+ commit_seqs.insert(seq);
+ if (!snapshot) {
+ committed_before.insert(cur_txn);
+ }
+ cur_txn = 0;
+ }
+
+ if (num_snapshots < max_snapshots - 1) {
+ // Take preliminary snapshots
+ wp_db->TakeSnapshot(seq);
+ num_snapshots++;
+ } else if (gap_cnt < max_gap) {
+ // Wait for some gap before taking the final snapshot
+ gap_cnt++;
+ } else if (!snapshot) {
+ // Take the final snapshot if it is not already taken
+ snapshot = seq;
+ wp_db->TakeSnapshot(snapshot);
+ num_snapshots++;
+ }
+
+ // If the snapshot is taken, verify seq numbers visible to it. We redo
+ // it at each cycle to test that the system is still sound when
+ // max_evicted_seq_ advances.
+ if (snapshot) {
+ for (uint64_t s = 1;
+ s <= seq && commit_seqs.find(s) == commit_seqs.end(); s++) {
+ bool was_committed =
+ (committed_before.find(s) != committed_before.end());
+ bool is_in_snapshot = wp_db->IsInSnapshot(s, snapshot);
+ if (was_committed != is_in_snapshot) {
+ printf("max_snapshots %d max_gap %d seq %" PRIu64 " max %" PRIu64
+ " snapshot %" PRIu64
+ " gap_cnt %d num_snapshots %d s %" PRIu64 "\n",
+ max_snapshots, max_gap, seq,
+ wp_db->max_evicted_seq_.load(), snapshot, gap_cnt,
+ num_snapshots, s);
+ }
+ ASSERT_EQ(was_committed, is_in_snapshot);
+ found_committed = found_committed || is_in_snapshot;
+ }
+ }
+ }
+ // Safety check to make sure the test actually ran
+ ASSERT_TRUE(found_committed);
+ // As an extra check, check if prepared set will be properly empty after
+ // they are committed.
+ if (cur_txn) {
+ wp_db->AddCommitted(cur_txn, seq);
+ wp_db->RemovePrepared(cur_txn);
+ }
+ for (auto p : prepared) {
+ wp_db->AddCommitted(p, seq);
+ wp_db->RemovePrepared(p);
+ }
+ ASSERT_TRUE(wp_db->delayed_prepared_.empty());
+ ASSERT_TRUE(wp_db->prepared_txns_.empty());
+ }
+ }
+}
+
+void ASSERT_SAME(ReadOptions roptions, TransactionDB* db, Status exp_s,
+ PinnableSlice& exp_v, Slice key) {
+ Status s;
+ PinnableSlice v;
+ s = db->Get(roptions, db->DefaultColumnFamily(), key, &v);
+ ASSERT_TRUE(exp_s == s);
+ ASSERT_TRUE(s.ok() || s.IsNotFound());
+ if (s.ok()) {
+ ASSERT_TRUE(exp_v == v);
+ }
+
+ // Try with MultiGet API too
+ std::vector<std::string> values;
+ auto s_vec =
+ db->MultiGet(roptions, {db->DefaultColumnFamily()}, {key}, &values);
+ ASSERT_EQ(1, values.size());
+ ASSERT_EQ(1, s_vec.size());
+ s = s_vec[0];
+ ASSERT_TRUE(exp_s == s);
+ ASSERT_TRUE(s.ok() || s.IsNotFound());
+ if (s.ok()) {
+ ASSERT_TRUE(exp_v == values[0]);
+ }
+}
+
+void ASSERT_SAME(TransactionDB* db, Status exp_s, PinnableSlice& exp_v,
+ Slice key) {
+ ASSERT_SAME(ReadOptions(), db, exp_s, exp_v, key);
+}
+
+TEST_P(WritePreparedTransactionTest, Rollback) {
+ ReadOptions roptions;
+ WriteOptions woptions;
+ TransactionOptions txn_options;
+ const size_t num_keys = 4;
+ const size_t num_values = 5;
+ for (size_t ikey = 1; ikey <= num_keys; ikey++) {
+ for (size_t ivalue = 0; ivalue < num_values; ivalue++) {
+ for (bool crash : {false, true}) {
+ ReOpen();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ std::string key_str = "key" + ToString(ikey);
+ switch (ivalue) {
+ case 0:
+ break;
+ case 1:
+ ASSERT_OK(db->Put(woptions, key_str, "initvalue1"));
+ break;
+ case 2:
+ ASSERT_OK(db->Merge(woptions, key_str, "initvalue2"));
+ break;
+ case 3:
+ ASSERT_OK(db->Delete(woptions, key_str));
+ break;
+ case 4:
+ ASSERT_OK(db->SingleDelete(woptions, key_str));
+ break;
+ default:
+ assert(0);
+ }
+
+ PinnableSlice v1;
+ auto s1 =
+ db->Get(roptions, db->DefaultColumnFamily(), Slice("key1"), &v1);
+ PinnableSlice v2;
+ auto s2 =
+ db->Get(roptions, db->DefaultColumnFamily(), Slice("key2"), &v2);
+ PinnableSlice v3;
+ auto s3 =
+ db->Get(roptions, db->DefaultColumnFamily(), Slice("key3"), &v3);
+ PinnableSlice v4;
+ auto s4 =
+ db->Get(roptions, db->DefaultColumnFamily(), Slice("key4"), &v4);
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ auto s = txn->SetName("xid0");
+ ASSERT_OK(s);
+ s = txn->Put(Slice("key1"), Slice("value1"));
+ ASSERT_OK(s);
+ s = txn->Merge(Slice("key2"), Slice("value2"));
+ ASSERT_OK(s);
+ s = txn->Delete(Slice("key3"));
+ ASSERT_OK(s);
+ s = txn->SingleDelete(Slice("key4"));
+ ASSERT_OK(s);
+ s = txn->Prepare();
+ ASSERT_OK(s);
+
+ {
+ ReadLock rl(&wp_db->prepared_mutex_);
+ ASSERT_FALSE(wp_db->prepared_txns_.empty());
+ ASSERT_EQ(txn->GetId(), wp_db->prepared_txns_.top());
+ }
+
+ ASSERT_SAME(db, s1, v1, "key1");
+ ASSERT_SAME(db, s2, v2, "key2");
+ ASSERT_SAME(db, s3, v3, "key3");
+ ASSERT_SAME(db, s4, v4, "key4");
+
+ if (crash) {
+ delete txn;
+ auto db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ db_impl->FlushWAL(true);
+ dynamic_cast<WritePreparedTxnDB*>(db)->TEST_Crash();
+ ReOpenNoDelete();
+ assert(db != nullptr);
+ wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ txn = db->GetTransactionByName("xid0");
+ ASSERT_FALSE(wp_db->delayed_prepared_empty_);
+ ReadLock rl(&wp_db->prepared_mutex_);
+ ASSERT_TRUE(wp_db->prepared_txns_.empty());
+ ASSERT_FALSE(wp_db->delayed_prepared_.empty());
+ ASSERT_TRUE(wp_db->delayed_prepared_.find(txn->GetId()) !=
+ wp_db->delayed_prepared_.end());
+ }
+
+ ASSERT_SAME(db, s1, v1, "key1");
+ ASSERT_SAME(db, s2, v2, "key2");
+ ASSERT_SAME(db, s3, v3, "key3");
+ ASSERT_SAME(db, s4, v4, "key4");
+
+ s = txn->Rollback();
+ ASSERT_OK(s);
+
+ {
+ ASSERT_TRUE(wp_db->delayed_prepared_empty_);
+ ReadLock rl(&wp_db->prepared_mutex_);
+ ASSERT_TRUE(wp_db->prepared_txns_.empty());
+ ASSERT_TRUE(wp_db->delayed_prepared_.empty());
+ }
+
+ ASSERT_SAME(db, s1, v1, "key1");
+ ASSERT_SAME(db, s2, v2, "key2");
+ ASSERT_SAME(db, s3, v3, "key3");
+ ASSERT_SAME(db, s4, v4, "key4");
+ delete txn;
+ }
+ }
+ }
+}
+
+TEST_P(WritePreparedTransactionTest, DisableGCDuringRecovery) {
+ // Use large buffer to avoid memtable flush after 1024 insertions
+ options.write_buffer_size = 1024 * 1024;
+ ReOpen();
+ std::vector<KeyVersion> versions;
+ uint64_t seq = 0;
+ for (uint64_t i = 1; i <= 1024; i++) {
+ std::string v = "bar" + ToString(i);
+ ASSERT_OK(db->Put(WriteOptions(), "foo", v));
+ VerifyKeys({{"foo", v}});
+ seq++; // one for the key/value
+ KeyVersion kv = {"foo", v, seq, kTypeValue};
+ if (options.two_write_queues) {
+ seq++; // one for the commit
+ }
+ versions.emplace_back(kv);
+ }
+ std::reverse(std::begin(versions), std::end(versions));
+ VerifyInternalKeys(versions);
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ db_impl->FlushWAL(true);
+ // Use small buffer to ensure memtable flush during recovery
+ options.write_buffer_size = 1024;
+ ReOpenNoDelete();
+ VerifyInternalKeys(versions);
+}
+
+TEST_P(WritePreparedTransactionTest, SequenceNumberZero) {
+ ASSERT_OK(db->Put(WriteOptions(), "foo", "bar"));
+ VerifyKeys({{"foo", "bar"}});
+ const Snapshot* snapshot = db->GetSnapshot();
+ ASSERT_OK(db->Flush(FlushOptions()));
+ // Dummy keys to avoid compaction trivially move files and get around actual
+ // compaction logic.
+ ASSERT_OK(db->Put(WriteOptions(), "a", "dummy"));
+ ASSERT_OK(db->Put(WriteOptions(), "z", "dummy"));
+ ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
+ // Compaction will output keys with sequence number 0, if it is visible to
+ // earliest snapshot. Make sure IsInSnapshot() report sequence number 0 is
+ // visible to any snapshot.
+ VerifyKeys({{"foo", "bar"}});
+ VerifyKeys({{"foo", "bar"}}, snapshot);
+ VerifyInternalKeys({{"foo", "bar", 0, kTypeValue}});
+ db->ReleaseSnapshot(snapshot);
+}
+
+// Compaction should not remove a key if it is not committed, and should
+// proceed with older versions of the key as-if the new version doesn't exist.
+TEST_P(WritePreparedTransactionTest, CompactionShouldKeepUncommittedKeys) {
+ options.disable_auto_compactions = true;
+ ReOpen();
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ // Snapshots to avoid keys get evicted.
+ std::vector<const Snapshot*> snapshots;
+ // Keep track of expected sequence number.
+ SequenceNumber expected_seq = 0;
+
+ auto add_key = [&](std::function<Status()> func) {
+ ASSERT_OK(func());
+ expected_seq++;
+ if (options.two_write_queues) {
+ expected_seq++; // 1 for commit
+ }
+ ASSERT_EQ(expected_seq, db_impl->TEST_GetLastVisibleSequence());
+ snapshots.push_back(db->GetSnapshot());
+ };
+
+ // Each key here represent a standalone test case.
+ add_key([&]() { return db->Put(WriteOptions(), "key1", "value1_1"); });
+ add_key([&]() { return db->Put(WriteOptions(), "key2", "value2_1"); });
+ add_key([&]() { return db->Put(WriteOptions(), "key3", "value3_1"); });
+ add_key([&]() { return db->Put(WriteOptions(), "key4", "value4_1"); });
+ add_key([&]() { return db->Merge(WriteOptions(), "key5", "value5_1"); });
+ add_key([&]() { return db->Merge(WriteOptions(), "key5", "value5_2"); });
+ add_key([&]() { return db->Put(WriteOptions(), "key6", "value6_1"); });
+ add_key([&]() { return db->Put(WriteOptions(), "key7", "value7_1"); });
+ ASSERT_OK(db->Flush(FlushOptions()));
+ add_key([&]() { return db->Delete(WriteOptions(), "key6"); });
+ add_key([&]() { return db->SingleDelete(WriteOptions(), "key7"); });
+
+ auto* transaction = db->BeginTransaction(WriteOptions());
+ ASSERT_OK(transaction->SetName("txn"));
+ ASSERT_OK(transaction->Put("key1", "value1_2"));
+ ASSERT_OK(transaction->Delete("key2"));
+ ASSERT_OK(transaction->SingleDelete("key3"));
+ ASSERT_OK(transaction->Merge("key4", "value4_2"));
+ ASSERT_OK(transaction->Merge("key5", "value5_3"));
+ ASSERT_OK(transaction->Put("key6", "value6_2"));
+ ASSERT_OK(transaction->Put("key7", "value7_2"));
+ // Prepare but not commit.
+ ASSERT_OK(transaction->Prepare());
+ ASSERT_EQ(++expected_seq, db->GetLatestSequenceNumber());
+ ASSERT_OK(db->Flush(FlushOptions()));
+ for (auto* s : snapshots) {
+ db->ReleaseSnapshot(s);
+ }
+ // Dummy keys to avoid compaction trivially move files and get around actual
+ // compaction logic.
+ ASSERT_OK(db->Put(WriteOptions(), "a", "dummy"));
+ ASSERT_OK(db->Put(WriteOptions(), "z", "dummy"));
+ ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
+ VerifyKeys({
+ {"key1", "value1_1"},
+ {"key2", "value2_1"},
+ {"key3", "value3_1"},
+ {"key4", "value4_1"},
+ {"key5", "value5_1,value5_2"},
+ {"key6", "NOT_FOUND"},
+ {"key7", "NOT_FOUND"},
+ });
+ VerifyInternalKeys({
+ {"key1", "value1_2", expected_seq, kTypeValue},
+ {"key1", "value1_1", 0, kTypeValue},
+ {"key2", "", expected_seq, kTypeDeletion},
+ {"key2", "value2_1", 0, kTypeValue},
+ {"key3", "", expected_seq, kTypeSingleDeletion},
+ {"key3", "value3_1", 0, kTypeValue},
+ {"key4", "value4_2", expected_seq, kTypeMerge},
+ {"key4", "value4_1", 0, kTypeValue},
+ {"key5", "value5_3", expected_seq, kTypeMerge},
+ {"key5", "value5_1,value5_2", 0, kTypeValue},
+ {"key6", "value6_2", expected_seq, kTypeValue},
+ {"key7", "value7_2", expected_seq, kTypeValue},
+ });
+ ASSERT_OK(transaction->Commit());
+ VerifyKeys({
+ {"key1", "value1_2"},
+ {"key2", "NOT_FOUND"},
+ {"key3", "NOT_FOUND"},
+ {"key4", "value4_1,value4_2"},
+ {"key5", "value5_1,value5_2,value5_3"},
+ {"key6", "value6_2"},
+ {"key7", "value7_2"},
+ });
+ delete transaction;
+}
+
+// Compaction should keep keys visible to a snapshot based on commit sequence,
+// not just prepare sequence.
+TEST_P(WritePreparedTransactionTest, CompactionShouldKeepSnapshotVisibleKeys) {
+ options.disable_auto_compactions = true;
+ ReOpen();
+ // Keep track of expected sequence number.
+ SequenceNumber expected_seq = 0;
+ auto* txn1 = db->BeginTransaction(WriteOptions());
+ ASSERT_OK(txn1->SetName("txn1"));
+ ASSERT_OK(txn1->Put("key1", "value1_1"));
+ ASSERT_OK(txn1->Prepare());
+ ASSERT_EQ(++expected_seq, db->GetLatestSequenceNumber());
+ ASSERT_OK(txn1->Commit());
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ ASSERT_EQ(++expected_seq, db_impl->TEST_GetLastVisibleSequence());
+ delete txn1;
+ // Take a snapshots to avoid keys get evicted before compaction.
+ const Snapshot* snapshot1 = db->GetSnapshot();
+ auto* txn2 = db->BeginTransaction(WriteOptions());
+ ASSERT_OK(txn2->SetName("txn2"));
+ ASSERT_OK(txn2->Put("key2", "value2_1"));
+ ASSERT_OK(txn2->Prepare());
+ ASSERT_EQ(++expected_seq, db->GetLatestSequenceNumber());
+ // txn1 commit before snapshot2 and it is visible to snapshot2.
+ // txn2 commit after snapshot2 and it is not visible.
+ const Snapshot* snapshot2 = db->GetSnapshot();
+ ASSERT_OK(txn2->Commit());
+ ASSERT_EQ(++expected_seq, db_impl->TEST_GetLastVisibleSequence());
+ delete txn2;
+ // Take a snapshots to avoid keys get evicted before compaction.
+ const Snapshot* snapshot3 = db->GetSnapshot();
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value1_2"));
+ expected_seq++; // 1 for write
+ SequenceNumber seq1 = expected_seq;
+ if (options.two_write_queues) {
+ expected_seq++; // 1 for commit
+ }
+ ASSERT_EQ(expected_seq, db_impl->TEST_GetLastVisibleSequence());
+ ASSERT_OK(db->Put(WriteOptions(), "key2", "value2_2"));
+ expected_seq++; // 1 for write
+ SequenceNumber seq2 = expected_seq;
+ if (options.two_write_queues) {
+ expected_seq++; // 1 for commit
+ }
+ ASSERT_EQ(expected_seq, db_impl->TEST_GetLastVisibleSequence());
+ ASSERT_OK(db->Flush(FlushOptions()));
+ db->ReleaseSnapshot(snapshot1);
+ db->ReleaseSnapshot(snapshot3);
+ // Dummy keys to avoid compaction trivially move files and get around actual
+ // compaction logic.
+ ASSERT_OK(db->Put(WriteOptions(), "a", "dummy"));
+ ASSERT_OK(db->Put(WriteOptions(), "z", "dummy"));
+ ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
+ VerifyKeys({{"key1", "value1_2"}, {"key2", "value2_2"}});
+ VerifyKeys({{"key1", "value1_1"}, {"key2", "NOT_FOUND"}}, snapshot2);
+ VerifyInternalKeys({
+ {"key1", "value1_2", seq1, kTypeValue},
+ // "value1_1" is visible to snapshot2. Also keys at bottom level visible
+ // to earliest snapshot will output with seq = 0.
+ {"key1", "value1_1", 0, kTypeValue},
+ {"key2", "value2_2", seq2, kTypeValue},
+ });
+ db->ReleaseSnapshot(snapshot2);
+}
+
+TEST_P(WritePreparedTransactionTest, SmallestUncommittedOptimization) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 0; // disable commit cache
+ for (bool has_recent_prepare : {true, false}) {
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value1"));
+ auto* transaction =
+ db->BeginTransaction(WriteOptions(), TransactionOptions(), nullptr);
+ ASSERT_OK(transaction->SetName("txn"));
+ ASSERT_OK(transaction->Delete("key1"));
+ ASSERT_OK(transaction->Prepare());
+ // snapshot1 should get min_uncommitted from prepared_txns_ heap.
+ auto snapshot1 = db->GetSnapshot();
+ ASSERT_EQ(transaction->GetId(),
+ ((SnapshotImpl*)snapshot1)->min_uncommitted_);
+ // Add a commit to advance max_evicted_seq and move the prepared transaction
+ // into delayed_prepared_ set.
+ ASSERT_OK(db->Put(WriteOptions(), "key2", "value2"));
+ Transaction* txn2 = nullptr;
+ if (has_recent_prepare) {
+ txn2 =
+ db->BeginTransaction(WriteOptions(), TransactionOptions(), nullptr);
+ ASSERT_OK(txn2->SetName("txn2"));
+ ASSERT_OK(txn2->Put("key3", "value3"));
+ ASSERT_OK(txn2->Prepare());
+ }
+ // snapshot2 should get min_uncommitted from delayed_prepared_ set.
+ auto snapshot2 = db->GetSnapshot();
+ ASSERT_EQ(transaction->GetId(),
+ ((SnapshotImpl*)snapshot1)->min_uncommitted_);
+ ASSERT_OK(transaction->Commit());
+ delete transaction;
+ if (has_recent_prepare) {
+ ASSERT_OK(txn2->Commit());
+ delete txn2;
+ }
+ VerifyKeys({{"key1", "NOT_FOUND"}});
+ VerifyKeys({{"key1", "value1"}}, snapshot1);
+ VerifyKeys({{"key1", "value1"}}, snapshot2);
+ db->ReleaseSnapshot(snapshot1);
+ db->ReleaseSnapshot(snapshot2);
+ }
+}
+
+// Insert two values, v1 and v2, for a key. Between prepare and commit of v2
+// take two snapshots, s1 and s2. Release s1 during compaction.
+// Test to make sure compaction doesn't get confused and think s1 can see both
+// values, and thus compact out the older value by mistake.
+TEST_P(WritePreparedTransactionTest, ReleaseSnapshotDuringCompaction) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 0; // minimum commit cache
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value1_1"));
+ auto* transaction =
+ db->BeginTransaction(WriteOptions(), TransactionOptions(), nullptr);
+ ASSERT_OK(transaction->SetName("txn"));
+ ASSERT_OK(transaction->Put("key1", "value1_2"));
+ ASSERT_OK(transaction->Prepare());
+ auto snapshot1 = db->GetSnapshot();
+ // Increment sequence number.
+ ASSERT_OK(db->Put(WriteOptions(), "key2", "value2"));
+ auto snapshot2 = db->GetSnapshot();
+ ASSERT_OK(transaction->Commit());
+ delete transaction;
+ VerifyKeys({{"key1", "value1_2"}});
+ VerifyKeys({{"key1", "value1_1"}}, snapshot1);
+ VerifyKeys({{"key1", "value1_1"}}, snapshot2);
+ // Add a flush to avoid compaction to fallback to trivial move.
+
+ auto callback = [&](void*) {
+ // Release snapshot1 after CompactionIterator init.
+ // CompactionIterator need to figure out the earliest snapshot
+ // that can see key1:value1_2 is kMaxSequenceNumber, not
+ // snapshot1 or snapshot2.
+ db->ReleaseSnapshot(snapshot1);
+ // Add some keys to advance max_evicted_seq.
+ ASSERT_OK(db->Put(WriteOptions(), "key3", "value3"));
+ ASSERT_OK(db->Put(WriteOptions(), "key4", "value4"));
+ };
+ SyncPoint::GetInstance()->SetCallBack("CompactionIterator:AfterInit",
+ callback);
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ASSERT_OK(db->Flush(FlushOptions()));
+ VerifyKeys({{"key1", "value1_2"}});
+ VerifyKeys({{"key1", "value1_1"}}, snapshot2);
+ db->ReleaseSnapshot(snapshot2);
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+// Insert two values, v1 and v2, for a key. Take two snapshots, s1 and s2,
+// after committing v2. Release s1 during compaction, right after compaction
+// processes v2 and before processes v1. Test to make sure compaction doesn't
+// get confused and believe v1 and v2 are visible to different snapshot
+// (v1 by s2, v2 by s1) and refuse to compact out v1.
+TEST_P(WritePreparedTransactionTest, ReleaseSnapshotDuringCompaction2) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 0; // minimum commit cache
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value1"));
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value2"));
+ SequenceNumber v2_seq = db->GetLatestSequenceNumber();
+ auto* s1 = db->GetSnapshot();
+ // Advance sequence number.
+ ASSERT_OK(db->Put(WriteOptions(), "key2", "dummy"));
+ auto* s2 = db->GetSnapshot();
+
+ int count_value = 0;
+ auto callback = [&](void* arg) {
+ auto* ikey = reinterpret_cast<ParsedInternalKey*>(arg);
+ if (ikey->user_key == "key1") {
+ count_value++;
+ if (count_value == 2) {
+ // Processing v1.
+ db->ReleaseSnapshot(s1);
+ // Add some keys to advance max_evicted_seq and update
+ // old_commit_map.
+ ASSERT_OK(db->Put(WriteOptions(), "key3", "dummy"));
+ ASSERT_OK(db->Put(WriteOptions(), "key4", "dummy"));
+ }
+ }
+ };
+ SyncPoint::GetInstance()->SetCallBack("CompactionIterator:ProcessKV",
+ callback);
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ASSERT_OK(db->Flush(FlushOptions()));
+ // value1 should be compact out.
+ VerifyInternalKeys({{"key1", "value2", v2_seq, kTypeValue}});
+
+ // cleanup
+ db->ReleaseSnapshot(s2);
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+// Insert two values, v1 and v2, for a key. Insert another dummy key
+// so to evict the commit cache for v2, while v1 is still in commit cache.
+// Take two snapshots, s1 and s2. Release s1 during compaction.
+// Since commit cache for v2 is evicted, and old_commit_map don't have
+// s1 (it is released),
+// TODO(myabandeh): how can we be sure that the v2's commit info is evicted
+// (and not v1's)? Instead of putting a dummy, we can directly call
+// AddCommitted(v2_seq + cache_size, ...) to evict v2's entry from commit cache.
+TEST_P(WritePreparedTransactionTest, ReleaseSnapshotDuringCompaction3) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 1; // commit cache size = 2
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+
+ // Add a dummy key to evict v2 commit cache, but keep v1 commit cache.
+ // It also advance max_evicted_seq and can trigger old_commit_map cleanup.
+ auto add_dummy = [&]() {
+ auto* txn_dummy =
+ db->BeginTransaction(WriteOptions(), TransactionOptions(), nullptr);
+ ASSERT_OK(txn_dummy->SetName("txn_dummy"));
+ ASSERT_OK(txn_dummy->Put("dummy", "dummy"));
+ ASSERT_OK(txn_dummy->Prepare());
+ ASSERT_OK(txn_dummy->Commit());
+ delete txn_dummy;
+ };
+
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value1"));
+ auto* txn =
+ db->BeginTransaction(WriteOptions(), TransactionOptions(), nullptr);
+ ASSERT_OK(txn->SetName("txn"));
+ ASSERT_OK(txn->Put("key1", "value2"));
+ ASSERT_OK(txn->Prepare());
+ // TODO(myabandeh): replace it with GetId()?
+ auto v2_seq = db->GetLatestSequenceNumber();
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ auto* s1 = db->GetSnapshot();
+ // Dummy key to advance sequence number.
+ add_dummy();
+ auto* s2 = db->GetSnapshot();
+
+ auto callback = [&](void*) {
+ db->ReleaseSnapshot(s1);
+ // Add some dummy entries to trigger s1 being cleanup from old_commit_map.
+ add_dummy();
+ add_dummy();
+ };
+ SyncPoint::GetInstance()->SetCallBack("CompactionIterator:AfterInit",
+ callback);
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ASSERT_OK(db->Flush(FlushOptions()));
+ // value1 should be compact out.
+ VerifyInternalKeys({{"key1", "value2", v2_seq, kTypeValue}});
+
+ db->ReleaseSnapshot(s2);
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+TEST_P(WritePreparedTransactionTest, ReleaseEarliestSnapshotDuringCompaction) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 0; // minimum commit cache
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value1"));
+ auto* transaction =
+ db->BeginTransaction(WriteOptions(), TransactionOptions(), nullptr);
+ ASSERT_OK(transaction->SetName("txn"));
+ ASSERT_OK(transaction->Delete("key1"));
+ ASSERT_OK(transaction->Prepare());
+ SequenceNumber del_seq = db->GetLatestSequenceNumber();
+ auto snapshot1 = db->GetSnapshot();
+ // Increment sequence number.
+ ASSERT_OK(db->Put(WriteOptions(), "key2", "value2"));
+ auto snapshot2 = db->GetSnapshot();
+ ASSERT_OK(transaction->Commit());
+ delete transaction;
+ VerifyKeys({{"key1", "NOT_FOUND"}});
+ VerifyKeys({{"key1", "value1"}}, snapshot1);
+ VerifyKeys({{"key1", "value1"}}, snapshot2);
+ ASSERT_OK(db->Flush(FlushOptions()));
+
+ auto callback = [&](void* compaction) {
+ // Release snapshot1 after CompactionIterator init.
+ // CompactionIterator need to double check and find out snapshot2 is now
+ // the earliest existing snapshot.
+ if (compaction != nullptr) {
+ db->ReleaseSnapshot(snapshot1);
+ // Add some keys to advance max_evicted_seq.
+ ASSERT_OK(db->Put(WriteOptions(), "key3", "value3"));
+ ASSERT_OK(db->Put(WriteOptions(), "key4", "value4"));
+ }
+ };
+ SyncPoint::GetInstance()->SetCallBack("CompactionIterator:AfterInit",
+ callback);
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ // Dummy keys to avoid compaction trivially move files and get around actual
+ // compaction logic.
+ ASSERT_OK(db->Put(WriteOptions(), "a", "dummy"));
+ ASSERT_OK(db->Put(WriteOptions(), "z", "dummy"));
+ ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
+ // Only verify for key1. Both the put and delete for the key should be kept.
+ // Since the delete tombstone is not visible to snapshot2, we need to keep
+ // at least one version of the key, for write-conflict check.
+ VerifyInternalKeys({{"key1", "", del_seq, kTypeDeletion},
+ {"key1", "value1", 0, kTypeValue}});
+ db->ReleaseSnapshot(snapshot2);
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+// A more complex test to verify compaction/flush should keep keys visible
+// to snapshots.
+TEST_P(WritePreparedTransactionTest,
+ CompactionKeepSnapshotVisibleKeysRandomized) {
+ constexpr size_t kNumTransactions = 10;
+ constexpr size_t kNumIterations = 1000;
+
+ std::vector<Transaction*> transactions(kNumTransactions, nullptr);
+ std::vector<size_t> versions(kNumTransactions, 0);
+ std::unordered_map<std::string, std::string> current_data;
+ std::vector<const Snapshot*> snapshots;
+ std::vector<std::unordered_map<std::string, std::string>> snapshot_data;
+
+ Random rnd(1103);
+ options.disable_auto_compactions = true;
+ ReOpen();
+
+ for (size_t i = 0; i < kNumTransactions; i++) {
+ std::string key = "key" + ToString(i);
+ std::string value = "value0";
+ ASSERT_OK(db->Put(WriteOptions(), key, value));
+ current_data[key] = value;
+ }
+ VerifyKeys(current_data);
+
+ for (size_t iter = 0; iter < kNumIterations; iter++) {
+ auto r = rnd.Next() % (kNumTransactions + 1);
+ if (r < kNumTransactions) {
+ std::string key = "key" + ToString(r);
+ if (transactions[r] == nullptr) {
+ std::string value = "value" + ToString(versions[r] + 1);
+ auto* txn = db->BeginTransaction(WriteOptions());
+ ASSERT_OK(txn->SetName("txn" + ToString(r)));
+ ASSERT_OK(txn->Put(key, value));
+ ASSERT_OK(txn->Prepare());
+ transactions[r] = txn;
+ } else {
+ std::string value = "value" + ToString(++versions[r]);
+ ASSERT_OK(transactions[r]->Commit());
+ delete transactions[r];
+ transactions[r] = nullptr;
+ current_data[key] = value;
+ }
+ } else {
+ auto* snapshot = db->GetSnapshot();
+ VerifyKeys(current_data, snapshot);
+ snapshots.push_back(snapshot);
+ snapshot_data.push_back(current_data);
+ }
+ VerifyKeys(current_data);
+ }
+ // Take a last snapshot to test compaction with uncommitted prepared
+ // transaction.
+ snapshots.push_back(db->GetSnapshot());
+ snapshot_data.push_back(current_data);
+
+ assert(snapshots.size() == snapshot_data.size());
+ for (size_t i = 0; i < snapshots.size(); i++) {
+ VerifyKeys(snapshot_data[i], snapshots[i]);
+ }
+ ASSERT_OK(db->Flush(FlushOptions()));
+ for (size_t i = 0; i < snapshots.size(); i++) {
+ VerifyKeys(snapshot_data[i], snapshots[i]);
+ }
+ // Dummy keys to avoid compaction trivially move files and get around actual
+ // compaction logic.
+ ASSERT_OK(db->Put(WriteOptions(), "a", "dummy"));
+ ASSERT_OK(db->Put(WriteOptions(), "z", "dummy"));
+ ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
+ for (size_t i = 0; i < snapshots.size(); i++) {
+ VerifyKeys(snapshot_data[i], snapshots[i]);
+ }
+ // cleanup
+ for (size_t i = 0; i < kNumTransactions; i++) {
+ if (transactions[i] == nullptr) {
+ continue;
+ }
+ ASSERT_OK(transactions[i]->Commit());
+ delete transactions[i];
+ }
+ for (size_t i = 0; i < snapshots.size(); i++) {
+ db->ReleaseSnapshot(snapshots[i]);
+ }
+}
+
+// Compaction should not apply the optimization to output key with sequence
+// number equal to 0 if the key is not visible to earliest snapshot, based on
+// commit sequence number.
+TEST_P(WritePreparedTransactionTest,
+ CompactionShouldKeepSequenceForUncommittedKeys) {
+ options.disable_auto_compactions = true;
+ ReOpen();
+ // Keep track of expected sequence number.
+ SequenceNumber expected_seq = 0;
+ auto* transaction = db->BeginTransaction(WriteOptions());
+ ASSERT_OK(transaction->SetName("txn"));
+ ASSERT_OK(transaction->Put("key1", "value1"));
+ ASSERT_OK(transaction->Prepare());
+ ASSERT_EQ(++expected_seq, db->GetLatestSequenceNumber());
+ SequenceNumber seq1 = expected_seq;
+ ASSERT_OK(db->Put(WriteOptions(), "key2", "value2"));
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ expected_seq++; // one for data
+ if (options.two_write_queues) {
+ expected_seq++; // one for commit
+ }
+ ASSERT_EQ(expected_seq, db_impl->TEST_GetLastVisibleSequence());
+ ASSERT_OK(db->Flush(FlushOptions()));
+ // Dummy keys to avoid compaction trivially move files and get around actual
+ // compaction logic.
+ ASSERT_OK(db->Put(WriteOptions(), "a", "dummy"));
+ ASSERT_OK(db->Put(WriteOptions(), "z", "dummy"));
+ ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
+ VerifyKeys({
+ {"key1", "NOT_FOUND"},
+ {"key2", "value2"},
+ });
+ VerifyInternalKeys({
+ // "key1" has not been committed. It keeps its sequence number.
+ {"key1", "value1", seq1, kTypeValue},
+ // "key2" is committed and output with seq = 0.
+ {"key2", "value2", 0, kTypeValue},
+ });
+ ASSERT_OK(transaction->Commit());
+ VerifyKeys({
+ {"key1", "value1"},
+ {"key2", "value2"},
+ });
+ delete transaction;
+}
+
+TEST_P(WritePreparedTransactionTest, CommitAndSnapshotDuringCompaction) {
+ options.disable_auto_compactions = true;
+ ReOpen();
+
+ const Snapshot* snapshot = nullptr;
+ ASSERT_OK(db->Put(WriteOptions(), "key1", "value1"));
+ auto* txn = db->BeginTransaction(WriteOptions());
+ ASSERT_OK(txn->SetName("txn"));
+ ASSERT_OK(txn->Put("key1", "value2"));
+ ASSERT_OK(txn->Prepare());
+
+ auto callback = [&](void*) {
+ // Snapshot is taken after compaction start. It should be taken into
+ // consideration for whether to compact out value1.
+ snapshot = db->GetSnapshot();
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ };
+ SyncPoint::GetInstance()->SetCallBack("CompactionIterator:AfterInit",
+ callback);
+ SyncPoint::GetInstance()->EnableProcessing();
+ ASSERT_OK(db->Flush(FlushOptions()));
+ ASSERT_NE(nullptr, snapshot);
+ VerifyKeys({{"key1", "value2"}});
+ VerifyKeys({{"key1", "value1"}}, snapshot);
+ db->ReleaseSnapshot(snapshot);
+}
+
+TEST_P(WritePreparedTransactionTest, Iterate) {
+ auto verify_state = [](Iterator* iter, const std::string& key,
+ const std::string& value) {
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_OK(iter->status());
+ ASSERT_EQ(key, iter->key().ToString());
+ ASSERT_EQ(value, iter->value().ToString());
+ };
+
+ auto verify_iter = [&](const std::string& expected_val) {
+ // Get iterator from a concurrent transaction and make sure it has the
+ // same view as an iterator from the DB.
+ auto* txn = db->BeginTransaction(WriteOptions());
+
+ for (int i = 0; i < 2; i++) {
+ Iterator* iter = (i == 0)
+ ? db->NewIterator(ReadOptions())
+ : txn->GetIterator(ReadOptions());
+ // Seek
+ iter->Seek("foo");
+ verify_state(iter, "foo", expected_val);
+ // Next
+ iter->Seek("a");
+ verify_state(iter, "a", "va");
+ iter->Next();
+ verify_state(iter, "foo", expected_val);
+ // SeekForPrev
+ iter->SeekForPrev("y");
+ verify_state(iter, "foo", expected_val);
+ // Prev
+ iter->SeekForPrev("z");
+ verify_state(iter, "z", "vz");
+ iter->Prev();
+ verify_state(iter, "foo", expected_val);
+ delete iter;
+ }
+ delete txn;
+ };
+
+ ASSERT_OK(db->Put(WriteOptions(), "foo", "v1"));
+ auto* transaction = db->BeginTransaction(WriteOptions());
+ ASSERT_OK(transaction->SetName("txn"));
+ ASSERT_OK(transaction->Put("foo", "v2"));
+ ASSERT_OK(transaction->Prepare());
+ VerifyKeys({{"foo", "v1"}});
+ // dummy keys
+ ASSERT_OK(db->Put(WriteOptions(), "a", "va"));
+ ASSERT_OK(db->Put(WriteOptions(), "z", "vz"));
+ verify_iter("v1");
+ ASSERT_OK(transaction->Commit());
+ VerifyKeys({{"foo", "v2"}});
+ verify_iter("v2");
+ delete transaction;
+}
+
+TEST_P(WritePreparedTransactionTest, IteratorRefreshNotSupported) {
+ Iterator* iter = db->NewIterator(ReadOptions());
+ ASSERT_TRUE(iter->Refresh().IsNotSupported());
+ delete iter;
+}
+
+// Committing an delayed prepared has two non-atomic steps: update commit cache,
+// remove seq from delayed_prepared_. The read in IsInSnapshot also involves two
+// non-atomic steps of checking these two data structures. This test breaks each
+// in the middle to ensure correctness in spite of non-atomic execution.
+// Note: This test is limitted to the case where snapshot is larger than the
+// max_evicted_seq_.
+TEST_P(WritePreparedTransactionTest, NonAtomicCommitOfDelayedPrepared) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 3; // 8 entries
+ for (auto split_read : {true, false}) {
+ std::vector<bool> split_options = {false};
+ if (split_read) {
+ // Also test for break before mutex
+ split_options.push_back(true);
+ }
+ for (auto split_before_mutex : split_options) {
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ DBImpl* db_impl = reinterpret_cast<DBImpl*>(db->GetRootDB());
+ // Fill up the commit cache
+ std::string init_value("value1");
+ for (int i = 0; i < 10; i++) {
+ db->Put(WriteOptions(), Slice("key1"), Slice(init_value));
+ }
+ // Prepare a transaction but do not commit it
+ Transaction* txn =
+ db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn->SetName("xid"));
+ ASSERT_OK(txn->Put(Slice("key1"), Slice("value2")));
+ ASSERT_OK(txn->Prepare());
+ // Commit a bunch of entries to advance max evicted seq and make the
+ // prepared a delayed prepared
+ for (int i = 0; i < 10; i++) {
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ }
+ // The snapshot should not see the delayed prepared entry
+ auto snap = db->GetSnapshot();
+
+ if (split_read) {
+ if (split_before_mutex) {
+ // split before acquiring prepare_mutex_
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"WritePreparedTxnDB::IsInSnapshot:prepared_mutex_:pause",
+ "AtomicCommitOfDelayedPrepared:Commit:before"},
+ {"AtomicCommitOfDelayedPrepared:Commit:after",
+ "WritePreparedTxnDB::IsInSnapshot:prepared_mutex_:resume"}});
+ } else {
+ // split right after reading from the commit cache
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"WritePreparedTxnDB::IsInSnapshot:GetCommitEntry:pause",
+ "AtomicCommitOfDelayedPrepared:Commit:before"},
+ {"AtomicCommitOfDelayedPrepared:Commit:after",
+ "WritePreparedTxnDB::IsInSnapshot:GetCommitEntry:resume"}});
+ }
+ } else { // split commit
+ // split right before removing from delayed_prepared_
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"WritePreparedTxnDB::RemovePrepared:pause",
+ "AtomicCommitOfDelayedPrepared:Read:before"},
+ {"AtomicCommitOfDelayedPrepared:Read:after",
+ "WritePreparedTxnDB::RemovePrepared:resume"}});
+ }
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ROCKSDB_NAMESPACE::port::Thread commit_thread([&]() {
+ TEST_SYNC_POINT("AtomicCommitOfDelayedPrepared:Commit:before");
+ ASSERT_OK(txn->Commit());
+ if (split_before_mutex) {
+ // Do bunch of inserts to evict the commit entry from the cache. This
+ // would prevent the 2nd look into commit cache under prepare_mutex_
+ // to see the commit entry.
+ auto seq = db_impl->TEST_GetLastVisibleSequence();
+ size_t tries = 0;
+ while (wp_db->max_evicted_seq_ < seq && tries < 50) {
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ tries++;
+ };
+ ASSERT_LT(tries, 50);
+ }
+ TEST_SYNC_POINT("AtomicCommitOfDelayedPrepared:Commit:after");
+ delete txn;
+ });
+
+ ROCKSDB_NAMESPACE::port::Thread read_thread([&]() {
+ TEST_SYNC_POINT("AtomicCommitOfDelayedPrepared:Read:before");
+ ReadOptions roptions;
+ roptions.snapshot = snap;
+ PinnableSlice value;
+ auto s = db->Get(roptions, db->DefaultColumnFamily(), "key1", &value);
+ ASSERT_OK(s);
+ // It should not see the commit of delayed prepared
+ ASSERT_TRUE(value == init_value);
+ TEST_SYNC_POINT("AtomicCommitOfDelayedPrepared:Read:after");
+ db->ReleaseSnapshot(snap);
+ });
+
+ read_thread.join();
+ commit_thread.join();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+ } // for split_before_mutex
+ } // for split_read
+}
+
+// When max evicted seq advances a prepared seq, it involves two updates: i)
+// adding prepared seq to delayed_prepared_, ii) updating max_evicted_seq_.
+// ::IsInSnapshot also reads these two values in a non-atomic way. This test
+// ensures correctness if the update occurs after ::IsInSnapshot reads
+// delayed_prepared_empty_ and before it reads max_evicted_seq_.
+// Note: this test focuses on read snapshot larger than max_evicted_seq_.
+TEST_P(WritePreparedTransactionTest, NonAtomicUpdateOfDelayedPrepared) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 3; // 8 entries
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ // Fill up the commit cache
+ std::string init_value("value1");
+ for (int i = 0; i < 10; i++) {
+ db->Put(WriteOptions(), Slice("key1"), Slice(init_value));
+ }
+ // Prepare a transaction but do not commit it
+ Transaction* txn = db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn->SetName("xid"));
+ ASSERT_OK(txn->Put(Slice("key1"), Slice("value2")));
+ ASSERT_OK(txn->Prepare());
+ // Create a gap between prepare seq and snapshot seq
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ // The snapshot should not see the delayed prepared entry
+ auto snap = db->GetSnapshot();
+ ASSERT_LT(txn->GetId(), snap->GetSequenceNumber());
+
+ // split right after reading delayed_prepared_empty_
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"WritePreparedTxnDB::IsInSnapshot:delayed_prepared_empty_:pause",
+ "AtomicUpdateOfDelayedPrepared:before"},
+ {"AtomicUpdateOfDelayedPrepared:after",
+ "WritePreparedTxnDB::IsInSnapshot:delayed_prepared_empty_:resume"}});
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ROCKSDB_NAMESPACE::port::Thread commit_thread([&]() {
+ TEST_SYNC_POINT("AtomicUpdateOfDelayedPrepared:before");
+ // Commit a bunch of entries to advance max evicted seq and make the
+ // prepared a delayed prepared
+ size_t tries = 0;
+ while (wp_db->max_evicted_seq_ < txn->GetId() && tries < 50) {
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ tries++;
+ };
+ ASSERT_LT(tries, 50);
+ // This is the case on which the test focuses
+ ASSERT_LT(wp_db->max_evicted_seq_, snap->GetSequenceNumber());
+ TEST_SYNC_POINT("AtomicUpdateOfDelayedPrepared:after");
+ });
+
+ ROCKSDB_NAMESPACE::port::Thread read_thread([&]() {
+ ReadOptions roptions;
+ roptions.snapshot = snap;
+ PinnableSlice value;
+ auto s = db->Get(roptions, db->DefaultColumnFamily(), "key1", &value);
+ ASSERT_OK(s);
+ // It should not see the uncommitted value of delayed prepared
+ ASSERT_TRUE(value == init_value);
+ db->ReleaseSnapshot(snap);
+ });
+
+ read_thread.join();
+ commit_thread.join();
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+// Eviction from commit cache and update of max evicted seq are two non-atomic
+// steps. Similarly the read of max_evicted_seq_ in ::IsInSnapshot and reading
+// from commit cache are two non-atomic steps. This tests if the update occurs
+// after reading max_evicted_seq_ and before reading the commit cache.
+// Note: the test focuses on snapshot larger than max_evicted_seq_
+TEST_P(WritePreparedTransactionTest, NonAtomicUpdateOfMaxEvictedSeq) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ const size_t commit_cache_bits = 3; // 8 entries
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ // Fill up the commit cache
+ std::string init_value("value1");
+ std::string last_value("value_final");
+ for (int i = 0; i < 10; i++) {
+ db->Put(WriteOptions(), Slice("key1"), Slice(init_value));
+ }
+ // Do an uncommitted write to prevent min_uncommitted optimization
+ Transaction* txn1 =
+ db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn1->SetName("xid1"));
+ ASSERT_OK(txn1->Put(Slice("key0"), last_value));
+ ASSERT_OK(txn1->Prepare());
+ // Do a write with prepare to get the prepare seq
+ Transaction* txn = db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn->SetName("xid"));
+ ASSERT_OK(txn->Put(Slice("key1"), last_value));
+ ASSERT_OK(txn->Prepare());
+ ASSERT_OK(txn->Commit());
+ // Create a gap between commit entry and snapshot seq
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ // The snapshot should see the last commit
+ auto snap = db->GetSnapshot();
+ ASSERT_LE(txn->GetId(), snap->GetSequenceNumber());
+
+ // split right after reading max_evicted_seq_
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
+ {{"WritePreparedTxnDB::IsInSnapshot:max_evicted_seq_:pause",
+ "NonAtomicUpdateOfMaxEvictedSeq:before"},
+ {"NonAtomicUpdateOfMaxEvictedSeq:after",
+ "WritePreparedTxnDB::IsInSnapshot:max_evicted_seq_:resume"}});
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ROCKSDB_NAMESPACE::port::Thread commit_thread([&]() {
+ TEST_SYNC_POINT("NonAtomicUpdateOfMaxEvictedSeq:before");
+ // Commit a bunch of entries to advance max evicted seq beyond txn->GetId()
+ size_t tries = 0;
+ while (wp_db->max_evicted_seq_ < txn->GetId() && tries < 50) {
+ db->Put(WriteOptions(), Slice("key3"), Slice("value3"));
+ tries++;
+ };
+ ASSERT_LT(tries, 50);
+ // This is the case on which the test focuses
+ ASSERT_LT(wp_db->max_evicted_seq_, snap->GetSequenceNumber());
+ TEST_SYNC_POINT("NonAtomicUpdateOfMaxEvictedSeq:after");
+ });
+
+ ROCKSDB_NAMESPACE::port::Thread read_thread([&]() {
+ ReadOptions roptions;
+ roptions.snapshot = snap;
+ PinnableSlice value;
+ auto s = db->Get(roptions, db->DefaultColumnFamily(), "key1", &value);
+ ASSERT_OK(s);
+ // It should see the committed value of the evicted entry
+ ASSERT_TRUE(value == last_value);
+ db->ReleaseSnapshot(snap);
+ });
+
+ read_thread.join();
+ commit_thread.join();
+ delete txn;
+ txn1->Commit();
+ delete txn1;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+// Test when we add a prepared seq when the max_evicted_seq_ already goes beyond
+// that. The test focuses on a race condition between AddPrepared and
+// AdvanceMaxEvictedSeq functions.
+TEST_P(WritePreparedTransactionTest, AddPreparedBeforeMax) {
+ if (!options.two_write_queues) {
+ // This test is only for two write queues
+ return;
+ }
+ const size_t snapshot_cache_bits = 7; // same as default
+ // 1 entry to advance max after the 2nd commit
+ const size_t commit_cache_bits = 0;
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ WritePreparedTxnDB* wp_db = dynamic_cast<WritePreparedTxnDB*>(db);
+ std::string some_value("value_some");
+ std::string uncommitted_value("value_uncommitted");
+ // Prepare two uncommitted transactions
+ Transaction* txn1 =
+ db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn1->SetName("xid1"));
+ ASSERT_OK(txn1->Put(Slice("key1"), some_value));
+ ASSERT_OK(txn1->Prepare());
+ Transaction* txn2 =
+ db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn2->SetName("xid2"));
+ ASSERT_OK(txn2->Put(Slice("key2"), some_value));
+ ASSERT_OK(txn2->Prepare());
+ // Start the txn here so the other thread could get its id
+ Transaction* txn = db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn->SetName("xid"));
+ ASSERT_OK(txn->Put(Slice("key0"), uncommitted_value));
+ port::Mutex txn_mutex_;
+
+ // t1) Insert prepared entry, t2) commit other entries to advance max
+ // evicted sec and finish checking the existing prepared entries, t1)
+ // AddPrepared, t2) update max_evicted_seq_
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"AddPreparedCallback::AddPrepared::begin:pause",
+ "AddPreparedBeforeMax::read_thread:start"},
+ {"AdvanceMaxEvictedSeq::update_max:pause",
+ "AddPreparedCallback::AddPrepared::begin:resume"},
+ {"AddPreparedCallback::AddPrepared::end",
+ "AdvanceMaxEvictedSeq::update_max:resume"},
+ });
+ SyncPoint::GetInstance()->EnableProcessing();
+
+ ROCKSDB_NAMESPACE::port::Thread write_thread([&]() {
+ txn_mutex_.Lock();
+ ASSERT_OK(txn->Prepare());
+ txn_mutex_.Unlock();
+ });
+
+ ROCKSDB_NAMESPACE::port::Thread read_thread([&]() {
+ TEST_SYNC_POINT("AddPreparedBeforeMax::read_thread:start");
+ // Publish seq number with a commit
+ ASSERT_OK(txn1->Commit());
+ // Since the commit cache size is one the 2nd commit evict the 1st one and
+ // invokes AdcanceMaxEvictedSeq
+ ASSERT_OK(txn2->Commit());
+
+ ReadOptions roptions;
+ PinnableSlice value;
+ // The snapshot should not see the uncommitted value from write_thread
+ auto snap = db->GetSnapshot();
+ ASSERT_LT(wp_db->max_evicted_seq_, snap->GetSequenceNumber());
+ // This is the scenario that we test for
+ txn_mutex_.Lock();
+ ASSERT_GT(wp_db->max_evicted_seq_, txn->GetId());
+ txn_mutex_.Unlock();
+ roptions.snapshot = snap;
+ auto s = db->Get(roptions, db->DefaultColumnFamily(), "key0", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ db->ReleaseSnapshot(snap);
+ });
+
+ read_thread.join();
+ write_thread.join();
+ delete txn1;
+ delete txn2;
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+// When an old prepared entry gets committed, there is a gap between the time
+// that it is published and when it is cleaned up from old_prepared_. This test
+// stresses such cases.
+TEST_P(WritePreparedTransactionTest, CommitOfDelayedPrepared) {
+ const size_t snapshot_cache_bits = 7; // same as default
+ for (const size_t commit_cache_bits : {0, 2, 3}) {
+ for (const size_t sub_batch_cnt : {1, 2, 3}) {
+ UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits);
+ ReOpen();
+ std::atomic<const Snapshot*> snap = {nullptr};
+ std::atomic<SequenceNumber> exp_prepare = {0};
+ ROCKSDB_NAMESPACE::port::Thread callback_thread;
+ // Value is synchronized via snap
+ PinnableSlice value;
+ // Take a snapshot after publish and before RemovePrepared:Start
+ auto snap_callback = [&]() {
+ ASSERT_EQ(nullptr, snap.load());
+ snap.store(db->GetSnapshot());
+ ReadOptions roptions;
+ roptions.snapshot = snap.load();
+ auto s = db->Get(roptions, db->DefaultColumnFamily(), "key2", &value);
+ ASSERT_OK(s);
+ };
+ auto callback = [&](void* param) {
+ SequenceNumber prep_seq = *((SequenceNumber*)param);
+ if (prep_seq == exp_prepare.load()) { // only for write_thread
+ // We need to spawn a thread to avoid deadlock since getting a
+ // snpashot might end up calling AdvanceSeqByOne which needs joining
+ // the write queue.
+ callback_thread = ROCKSDB_NAMESPACE::port::Thread(snap_callback);
+ TEST_SYNC_POINT("callback:end");
+ }
+ };
+ // Wait for the first snapshot be taken in GetSnapshotInternal. Although
+ // it might be updated before GetSnapshotInternal finishes but this should
+ // cover most of the cases.
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"WritePreparedTxnDB::GetSnapshotInternal:first", "callback:end"},
+ });
+ SyncPoint::GetInstance()->SetCallBack("RemovePrepared:Start", callback);
+ SyncPoint::GetInstance()->EnableProcessing();
+ // Thread to cause frequent evictions
+ ROCKSDB_NAMESPACE::port::Thread eviction_thread([&]() {
+ // Too many txns might cause commit_seq - prepare_seq in another thread
+ // to go beyond DELTA_UPPERBOUND
+ for (int i = 0; i < 25 * (1 << commit_cache_bits); i++) {
+ db->Put(WriteOptions(), Slice("key1"), Slice("value1"));
+ }
+ });
+ ROCKSDB_NAMESPACE::port::Thread write_thread([&]() {
+ for (int i = 0; i < 25 * (1 << commit_cache_bits); i++) {
+ Transaction* txn =
+ db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn->SetName("xid"));
+ std::string val_str = "value" + ToString(i);
+ for (size_t b = 0; b < sub_batch_cnt; b++) {
+ ASSERT_OK(txn->Put(Slice("key2"), val_str));
+ }
+ ASSERT_OK(txn->Prepare());
+ // Let an eviction to kick in
+ std::this_thread::yield();
+
+ exp_prepare.store(txn->GetId());
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ // Wait for the snapshot taking that is triggered by
+ // RemovePrepared:Start callback
+ callback_thread.join();
+
+ // Read with the snapshot taken before delayed_prepared_ cleanup
+ ReadOptions roptions;
+ roptions.snapshot = snap.load();
+ ASSERT_NE(nullptr, roptions.snapshot);
+ PinnableSlice value2;
+ auto s =
+ db->Get(roptions, db->DefaultColumnFamily(), "key2", &value2);
+ ASSERT_OK(s);
+ // It should see its own write
+ ASSERT_TRUE(val_str == value2);
+ // The value read by snapshot should not change
+ ASSERT_STREQ(value2.ToString().c_str(), value.ToString().c_str());
+
+ db->ReleaseSnapshot(roptions.snapshot);
+ snap.store(nullptr);
+ }
+ });
+ write_thread.join();
+ eviction_thread.join();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+ }
+ }
+}
+
+// Test that updating the commit map will not affect the existing snapshots
+TEST_P(WritePreparedTransactionTest, AtomicCommit) {
+ for (bool skip_prepare : {true, false}) {
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
+ {"WritePreparedTxnDB::AddCommitted:start",
+ "AtomicCommit::GetSnapshot:start"},
+ {"AtomicCommit::Get:end",
+ "WritePreparedTxnDB::AddCommitted:start:pause"},
+ {"WritePreparedTxnDB::AddCommitted:end", "AtomicCommit::Get2:start"},
+ {"AtomicCommit::Get2:end",
+ "WritePreparedTxnDB::AddCommitted:end:pause:"},
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ ROCKSDB_NAMESPACE::port::Thread write_thread([&]() {
+ if (skip_prepare) {
+ db->Put(WriteOptions(), Slice("key"), Slice("value"));
+ } else {
+ Transaction* txn =
+ db->BeginTransaction(WriteOptions(), TransactionOptions());
+ ASSERT_OK(txn->SetName("xid"));
+ ASSERT_OK(txn->Put(Slice("key"), Slice("value")));
+ ASSERT_OK(txn->Prepare());
+ ASSERT_OK(txn->Commit());
+ delete txn;
+ }
+ });
+ ROCKSDB_NAMESPACE::port::Thread read_thread([&]() {
+ ReadOptions roptions;
+ TEST_SYNC_POINT("AtomicCommit::GetSnapshot:start");
+ roptions.snapshot = db->GetSnapshot();
+ PinnableSlice val;
+ auto s = db->Get(roptions, db->DefaultColumnFamily(), "key", &val);
+ TEST_SYNC_POINT("AtomicCommit::Get:end");
+ TEST_SYNC_POINT("AtomicCommit::Get2:start");
+ ASSERT_SAME(roptions, db, s, val, "key");
+ TEST_SYNC_POINT("AtomicCommit::Get2:end");
+ db->ReleaseSnapshot(roptions.snapshot);
+ });
+ read_thread.join();
+ write_thread.join();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ }
+}
+
+// Test that we can change write policy from WriteCommitted to WritePrepared
+// after a clean shutdown (which would empty the WAL)
+TEST_P(WritePreparedTransactionTest, WP_WC_DBBackwardCompatibility) {
+ bool empty_wal = true;
+ CrossCompatibilityTest(WRITE_COMMITTED, WRITE_PREPARED, empty_wal);
+}
+
+// Test that we fail fast if WAL is not emptied between changing the write
+// policy from WriteCommitted to WritePrepared
+TEST_P(WritePreparedTransactionTest, WP_WC_WALBackwardIncompatibility) {
+ bool empty_wal = true;
+ CrossCompatibilityTest(WRITE_COMMITTED, WRITE_PREPARED, !empty_wal);
+}
+
+// Test that we can change write policy from WritePrepare back to WriteCommitted
+// after a clean shutdown (which would empty the WAL)
+TEST_P(WritePreparedTransactionTest, WC_WP_ForwardCompatibility) {
+ bool empty_wal = true;
+ CrossCompatibilityTest(WRITE_PREPARED, WRITE_COMMITTED, empty_wal);
+}
+
+// Test that we fail fast if WAL is not emptied between changing the write
+// policy from WriteCommitted to WritePrepared
+TEST_P(WritePreparedTransactionTest, WC_WP_WALForwardIncompatibility) {
+ bool empty_wal = true;
+ CrossCompatibilityTest(WRITE_PREPARED, WRITE_COMMITTED, !empty_wal);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(stderr,
+ "SKIPPED as Transactions are not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_prepared_txn.cc b/src/rocksdb/utilities/transactions/write_prepared_txn.cc
new file mode 100644
index 000000000..216d83555
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_prepared_txn.cc
@@ -0,0 +1,473 @@
+// 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/write_prepared_txn.h"
+
+#include <cinttypes>
+#include <map>
+#include <set>
+
+#include "db/column_family.h"
+#include "db/db_impl/db_impl.h"
+#include "rocksdb/db.h"
+#include "rocksdb/status.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "util/cast_util.h"
+#include "utilities/transactions/pessimistic_transaction.h"
+#include "utilities/transactions/write_prepared_txn_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct WriteOptions;
+
+WritePreparedTxn::WritePreparedTxn(WritePreparedTxnDB* txn_db,
+ const WriteOptions& write_options,
+ const TransactionOptions& txn_options)
+ : PessimisticTransaction(txn_db, write_options, txn_options, false),
+ wpt_db_(txn_db) {
+ // Call Initialize outside PessimisticTransaction constructor otherwise it
+ // would skip overridden functions in WritePreparedTxn since they are not
+ // defined yet in the constructor of PessimisticTransaction
+ Initialize(txn_options);
+}
+
+void WritePreparedTxn::Initialize(const TransactionOptions& txn_options) {
+ PessimisticTransaction::Initialize(txn_options);
+ prepare_batch_cnt_ = 0;
+}
+
+void WritePreparedTxn::MultiGet(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ const size_t num_keys, const Slice* keys,
+ PinnableSlice* values, Status* statuses,
+ const bool sorted_input) {
+ SequenceNumber min_uncommitted, snap_seq;
+ const SnapshotBackup backed_by_snapshot =
+ wpt_db_->AssignMinMaxSeqs(options.snapshot, &min_uncommitted, &snap_seq);
+ WritePreparedTxnReadCallback callback(wpt_db_, snap_seq, min_uncommitted,
+ backed_by_snapshot);
+ write_batch_.MultiGetFromBatchAndDB(db_, options, column_family, num_keys,
+ keys, values, statuses, sorted_input,
+ &callback);
+ if (UNLIKELY(!callback.valid() ||
+ !wpt_db_->ValidateSnapshot(snap_seq, backed_by_snapshot))) {
+ wpt_db_->WPRecordTick(TXN_GET_TRY_AGAIN);
+ for (size_t i = 0; i < num_keys; i++) {
+ statuses[i] = Status::TryAgain();
+ }
+ }
+}
+
+Status WritePreparedTxn::Get(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ const Slice& key, PinnableSlice* pinnable_val) {
+ SequenceNumber min_uncommitted, snap_seq;
+ const SnapshotBackup backed_by_snapshot =
+ wpt_db_->AssignMinMaxSeqs(options.snapshot, &min_uncommitted, &snap_seq);
+ WritePreparedTxnReadCallback callback(wpt_db_, snap_seq, min_uncommitted,
+ backed_by_snapshot);
+ auto res = write_batch_.GetFromBatchAndDB(db_, options, column_family, key,
+ pinnable_val, &callback);
+ if (LIKELY(callback.valid() &&
+ wpt_db_->ValidateSnapshot(callback.max_visible_seq(),
+ backed_by_snapshot))) {
+ return res;
+ } else {
+ wpt_db_->WPRecordTick(TXN_GET_TRY_AGAIN);
+ return Status::TryAgain();
+ }
+}
+
+Iterator* WritePreparedTxn::GetIterator(const ReadOptions& options) {
+ // Make sure to get iterator from WritePrepareTxnDB, not the root db.
+ Iterator* db_iter = wpt_db_->NewIterator(options);
+ assert(db_iter);
+
+ return write_batch_.NewIteratorWithBase(db_iter);
+}
+
+Iterator* WritePreparedTxn::GetIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family) {
+ // Make sure to get iterator from WritePrepareTxnDB, not the root db.
+ Iterator* db_iter = wpt_db_->NewIterator(options, column_family);
+ assert(db_iter);
+
+ return write_batch_.NewIteratorWithBase(column_family, db_iter);
+}
+
+Status WritePreparedTxn::PrepareInternal() {
+ WriteOptions write_options = write_options_;
+ write_options.disableWAL = false;
+ const bool WRITE_AFTER_COMMIT = true;
+ const bool kFirstPrepareBatch = true;
+ WriteBatchInternal::MarkEndPrepare(GetWriteBatch()->GetWriteBatch(), name_,
+ !WRITE_AFTER_COMMIT);
+ // For each duplicate key we account for a new sub-batch
+ prepare_batch_cnt_ = GetWriteBatch()->SubBatchCnt();
+ // Having AddPrepared in the PreReleaseCallback allows in-order addition of
+ // prepared entries to PreparedHeap and hence enables an optimization. Refer to
+ // SmallestUnCommittedSeq for more details.
+ AddPreparedCallback add_prepared_callback(
+ wpt_db_, db_impl_, prepare_batch_cnt_,
+ db_impl_->immutable_db_options().two_write_queues, kFirstPrepareBatch);
+ const bool DISABLE_MEMTABLE = true;
+ uint64_t seq_used = kMaxSequenceNumber;
+ Status s = db_impl_->WriteImpl(
+ write_options, GetWriteBatch()->GetWriteBatch(),
+ /*callback*/ nullptr, &log_number_, /*log ref*/ 0, !DISABLE_MEMTABLE,
+ &seq_used, prepare_batch_cnt_, &add_prepared_callback);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ auto prepare_seq = seq_used;
+ SetId(prepare_seq);
+ return s;
+}
+
+Status WritePreparedTxn::CommitWithoutPrepareInternal() {
+ // For each duplicate key we account for a new sub-batch
+ const size_t batch_cnt = GetWriteBatch()->SubBatchCnt();
+ return CommitBatchInternal(GetWriteBatch()->GetWriteBatch(), batch_cnt);
+}
+
+Status WritePreparedTxn::CommitBatchInternal(WriteBatch* batch,
+ size_t batch_cnt) {
+ return wpt_db_->WriteInternal(write_options_, batch, batch_cnt, this);
+}
+
+Status WritePreparedTxn::CommitInternal() {
+ ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
+ "CommitInternal prepare_seq: %" PRIu64, GetID());
+ // We take the commit-time batch and append the Commit marker.
+ // The Memtable will ignore the Commit marker in non-recovery mode
+ WriteBatch* working_batch = GetCommitTimeWriteBatch();
+ const bool empty = working_batch->Count() == 0;
+ WriteBatchInternal::MarkCommit(working_batch, name_);
+
+ const bool for_recovery = use_only_the_last_commit_time_batch_for_recovery_;
+ if (!empty && for_recovery) {
+ // When not writing to memtable, we can still cache the latest write batch.
+ // The cached batch will be written to memtable in WriteRecoverableState
+ // during FlushMemTable
+ WriteBatchInternal::SetAsLastestPersistentState(working_batch);
+ }
+
+ auto prepare_seq = GetId();
+ const bool includes_data = !empty && !for_recovery;
+ assert(prepare_batch_cnt_);
+ size_t commit_batch_cnt = 0;
+ if (UNLIKELY(includes_data)) {
+ ROCKS_LOG_WARN(db_impl_->immutable_db_options().info_log,
+ "Duplicate key overhead");
+ SubBatchCounter counter(*wpt_db_->GetCFComparatorMap());
+ auto s = working_batch->Iterate(&counter);
+ assert(s.ok());
+ commit_batch_cnt = counter.BatchCount();
+ }
+ const bool disable_memtable = !includes_data;
+ const bool do_one_write =
+ !db_impl_->immutable_db_options().two_write_queues || disable_memtable;
+ WritePreparedCommitEntryPreReleaseCallback update_commit_map(
+ wpt_db_, db_impl_, prepare_seq, prepare_batch_cnt_, commit_batch_cnt);
+ // This is to call AddPrepared on CommitTimeWriteBatch
+ const bool kFirstPrepareBatch = true;
+ AddPreparedCallback add_prepared_callback(
+ wpt_db_, db_impl_, commit_batch_cnt,
+ db_impl_->immutable_db_options().two_write_queues, !kFirstPrepareBatch);
+ PreReleaseCallback* pre_release_callback;
+ if (do_one_write) {
+ pre_release_callback = &update_commit_map;
+ } else {
+ pre_release_callback = &add_prepared_callback;
+ }
+ uint64_t seq_used = kMaxSequenceNumber;
+ // Since the prepared batch is directly written to memtable, there is already
+ // a connection between the memtable and its WAL, so there is no need to
+ // redundantly reference the log that contains the prepared data.
+ const uint64_t zero_log_number = 0ull;
+ size_t batch_cnt = UNLIKELY(commit_batch_cnt) ? commit_batch_cnt : 1;
+ auto s = db_impl_->WriteImpl(write_options_, working_batch, nullptr, nullptr,
+ zero_log_number, disable_memtable, &seq_used,
+ batch_cnt, pre_release_callback);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ const SequenceNumber commit_batch_seq = seq_used;
+ if (LIKELY(do_one_write || !s.ok())) {
+ if (UNLIKELY(!db_impl_->immutable_db_options().two_write_queues &&
+ s.ok())) {
+ // Note: RemovePrepared should be called after WriteImpl that publishsed
+ // the seq. Otherwise SmallestUnCommittedSeq optimization breaks.
+ wpt_db_->RemovePrepared(prepare_seq, prepare_batch_cnt_);
+ } // else RemovePrepared is called from within PreReleaseCallback
+ if (UNLIKELY(!do_one_write)) {
+ assert(!s.ok());
+ // Cleanup the prepared entry we added with add_prepared_callback
+ wpt_db_->RemovePrepared(commit_batch_seq, commit_batch_cnt);
+ }
+ return s;
+ } // else do the 2nd write to publish seq
+ // Note: the 2nd write comes with a performance penality. So if we have too
+ // many of commits accompanied with ComitTimeWriteBatch and yet we cannot
+ // enable use_only_the_last_commit_time_batch_for_recovery_ optimization,
+ // two_write_queues should be disabled to avoid many additional writes here.
+ const size_t kZeroData = 0;
+ // Update commit map only from the 2nd queue
+ WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_aux_batch(
+ wpt_db_, db_impl_, prepare_seq, prepare_batch_cnt_, kZeroData,
+ commit_batch_seq, commit_batch_cnt);
+ WriteBatch empty_batch;
+ empty_batch.PutLogData(Slice());
+ // In the absence of Prepare markers, use Noop as a batch separator
+ WriteBatchInternal::InsertNoop(&empty_batch);
+ const bool DISABLE_MEMTABLE = true;
+ const size_t ONE_BATCH = 1;
+ const uint64_t NO_REF_LOG = 0;
+ s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr,
+ NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
+ &update_commit_map_with_aux_batch);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ if (UNLIKELY(!db_impl_->immutable_db_options().two_write_queues)) {
+ if (s.ok()) {
+ // Note: RemovePrepared should be called after WriteImpl that publishsed
+ // the seq. Otherwise SmallestUnCommittedSeq optimization breaks.
+ wpt_db_->RemovePrepared(prepare_seq, prepare_batch_cnt_);
+ }
+ wpt_db_->RemovePrepared(commit_batch_seq, commit_batch_cnt);
+ } // else RemovePrepared is called from within PreReleaseCallback
+ return s;
+}
+
+Status WritePreparedTxn::RollbackInternal() {
+ ROCKS_LOG_WARN(db_impl_->immutable_db_options().info_log,
+ "RollbackInternal prepare_seq: %" PRIu64, GetId());
+ WriteBatch rollback_batch;
+ assert(GetId() != kMaxSequenceNumber);
+ assert(GetId() > 0);
+ auto cf_map_shared_ptr = wpt_db_->GetCFHandleMap();
+ auto cf_comp_map_shared_ptr = wpt_db_->GetCFComparatorMap();
+ auto read_at_seq = kMaxSequenceNumber;
+ ReadOptions roptions;
+ // to prevent callback's seq to be overrriden inside DBImpk::Get
+ roptions.snapshot = wpt_db_->GetMaxSnapshot();
+ struct RollbackWriteBatchBuilder : public WriteBatch::Handler {
+ DBImpl* db_;
+ WritePreparedTxnReadCallback callback;
+ WriteBatch* rollback_batch_;
+ std::map<uint32_t, const Comparator*>& comparators_;
+ std::map<uint32_t, ColumnFamilyHandle*>& handles_;
+ using CFKeys = std::set<Slice, SetComparator>;
+ std::map<uint32_t, CFKeys> keys_;
+ bool rollback_merge_operands_;
+ ReadOptions roptions_;
+ RollbackWriteBatchBuilder(
+ DBImpl* db, WritePreparedTxnDB* wpt_db, SequenceNumber snap_seq,
+ WriteBatch* dst_batch,
+ std::map<uint32_t, const Comparator*>& comparators,
+ std::map<uint32_t, ColumnFamilyHandle*>& handles,
+ bool rollback_merge_operands, ReadOptions _roptions)
+ : db_(db),
+ callback(wpt_db, snap_seq), // disable min_uncommitted optimization
+ rollback_batch_(dst_batch),
+ comparators_(comparators),
+ handles_(handles),
+ rollback_merge_operands_(rollback_merge_operands),
+ roptions_(_roptions) {}
+
+ Status Rollback(uint32_t cf, const Slice& key) {
+ Status s;
+ CFKeys& cf_keys = keys_[cf];
+ if (cf_keys.size() == 0) { // just inserted
+ auto cmp = comparators_[cf];
+ keys_[cf] = CFKeys(SetComparator(cmp));
+ }
+ auto it = cf_keys.insert(key);
+ if (it.second ==
+ false) { // second is false if a element already existed.
+ return s;
+ }
+
+ PinnableSlice pinnable_val;
+ bool not_used;
+ auto cf_handle = handles_[cf];
+ DBImpl::GetImplOptions get_impl_options;
+ get_impl_options.column_family = cf_handle;
+ get_impl_options.value = &pinnable_val;
+ get_impl_options.value_found = &not_used;
+ get_impl_options.callback = &callback;
+ s = db_->GetImpl(roptions_, key, get_impl_options);
+ assert(s.ok() || s.IsNotFound());
+ if (s.ok()) {
+ s = rollback_batch_->Put(cf_handle, key, pinnable_val);
+ assert(s.ok());
+ } else if (s.IsNotFound()) {
+ // There has been no readable value before txn. By adding a delete we
+ // make sure that there will be none afterwards either.
+ s = rollback_batch_->Delete(cf_handle, key);
+ assert(s.ok());
+ } else {
+ // Unexpected status. Return it to the user.
+ }
+ return s;
+ }
+
+ Status PutCF(uint32_t cf, const Slice& key, const Slice& /*val*/) override {
+ return Rollback(cf, key);
+ }
+
+ Status DeleteCF(uint32_t cf, const Slice& key) override {
+ return Rollback(cf, key);
+ }
+
+ Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
+ return Rollback(cf, key);
+ }
+
+ Status MergeCF(uint32_t cf, const Slice& key,
+ const Slice& /*val*/) override {
+ if (rollback_merge_operands_) {
+ return Rollback(cf, key);
+ } else {
+ return Status::OK();
+ }
+ }
+
+ Status MarkNoop(bool) override { return Status::OK(); }
+ Status MarkBeginPrepare(bool) override { return Status::OK(); }
+ Status MarkEndPrepare(const Slice&) override { return Status::OK(); }
+ Status MarkCommit(const Slice&) override { return Status::OK(); }
+ Status MarkRollback(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+
+ protected:
+ bool WriteAfterCommit() const override { return false; }
+ } rollback_handler(db_impl_, wpt_db_, read_at_seq, &rollback_batch,
+ *cf_comp_map_shared_ptr.get(), *cf_map_shared_ptr.get(),
+ wpt_db_->txn_db_options_.rollback_merge_operands,
+ roptions);
+ auto s = GetWriteBatch()->GetWriteBatch()->Iterate(&rollback_handler);
+ assert(s.ok());
+ if (!s.ok()) {
+ return s;
+ }
+ // The Rollback marker will be used as a batch separator
+ WriteBatchInternal::MarkRollback(&rollback_batch, name_);
+ bool do_one_write = !db_impl_->immutable_db_options().two_write_queues;
+ const bool DISABLE_MEMTABLE = true;
+ const uint64_t NO_REF_LOG = 0;
+ uint64_t seq_used = kMaxSequenceNumber;
+ const size_t ONE_BATCH = 1;
+ const bool kFirstPrepareBatch = true;
+ // We commit the rolled back prepared batches. Although this is
+ // counter-intuitive, i) it is safe to do so, since the prepared batches are
+ // already canceled out by the rollback batch, ii) adding the commit entry to
+ // CommitCache will allow us to benefit from the existing mechanism in
+ // CommitCache that keeps an entry evicted due to max advance and yet overlaps
+ // with a live snapshot around so that the live snapshot properly skips the
+ // entry even if its prepare seq is lower than max_evicted_seq_.
+ AddPreparedCallback add_prepared_callback(
+ wpt_db_, db_impl_, ONE_BATCH,
+ db_impl_->immutable_db_options().two_write_queues, !kFirstPrepareBatch);
+ WritePreparedCommitEntryPreReleaseCallback update_commit_map(
+ wpt_db_, db_impl_, GetId(), prepare_batch_cnt_, ONE_BATCH);
+ PreReleaseCallback* pre_release_callback;
+ if (do_one_write) {
+ pre_release_callback = &update_commit_map;
+ } else {
+ pre_release_callback = &add_prepared_callback;
+ }
+ // Note: the rollback batch does not need AddPrepared since it is written to
+ // DB in one shot. min_uncommitted still works since it requires capturing
+ // data that is written to DB but not yet committed, while
+ // the rollback batch commits with PreReleaseCallback.
+ s = db_impl_->WriteImpl(write_options_, &rollback_batch, nullptr, nullptr,
+ NO_REF_LOG, !DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
+ pre_release_callback);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ if (!s.ok()) {
+ return s;
+ }
+ if (do_one_write) {
+ assert(!db_impl_->immutable_db_options().two_write_queues);
+ wpt_db_->RemovePrepared(GetId(), prepare_batch_cnt_);
+ return s;
+ } // else do the 2nd write for commit
+ uint64_t rollback_seq = seq_used;
+ ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
+ "RollbackInternal 2nd write rollback_seq: %" PRIu64,
+ rollback_seq);
+ // Commit the batch by writing an empty batch to the queue that will release
+ // the commit sequence number to readers.
+ WritePreparedRollbackPreReleaseCallback update_commit_map_with_prepare(
+ wpt_db_, db_impl_, GetId(), rollback_seq, prepare_batch_cnt_);
+ WriteBatch empty_batch;
+ empty_batch.PutLogData(Slice());
+ // In the absence of Prepare markers, use Noop as a batch separator
+ WriteBatchInternal::InsertNoop(&empty_batch);
+ s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr,
+ NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
+ &update_commit_map_with_prepare);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
+ "RollbackInternal (status=%s) commit: %" PRIu64,
+ s.ToString().c_str(), GetId());
+ // TODO(lth): For WriteUnPrepared that rollback is called frequently,
+ // RemovePrepared could be moved to the callback to reduce lock contention.
+ if (s.ok()) {
+ wpt_db_->RemovePrepared(GetId(), prepare_batch_cnt_);
+ }
+ // Note: RemovePrepared for prepared batch is called from within
+ // PreReleaseCallback
+ wpt_db_->RemovePrepared(rollback_seq, ONE_BATCH);
+
+ return s;
+}
+
+Status WritePreparedTxn::ValidateSnapshot(ColumnFamilyHandle* column_family,
+ const Slice& key,
+ SequenceNumber* tracked_at_seq) {
+ assert(snapshot_);
+
+ SequenceNumber min_uncommitted =
+ static_cast_with_check<const SnapshotImpl, const Snapshot>(
+ snapshot_.get())
+ ->min_uncommitted_;
+ SequenceNumber snap_seq = snapshot_->GetSequenceNumber();
+ // tracked_at_seq is either max or the last snapshot with which this key was
+ // trackeed so there is no need to apply the IsInSnapshot to this comparison
+ // here as tracked_at_seq is not a prepare seq.
+ if (*tracked_at_seq <= snap_seq) {
+ // If the key has been previous validated at a sequence number earlier
+ // than the curent snapshot's sequence number, we already know it has not
+ // been modified.
+ return Status::OK();
+ }
+
+ *tracked_at_seq = snap_seq;
+
+ ColumnFamilyHandle* cfh =
+ column_family ? column_family : db_impl_->DefaultColumnFamily();
+
+ WritePreparedTxnReadCallback snap_checker(wpt_db_, snap_seq, min_uncommitted,
+ kBackedByDBSnapshot);
+ return TransactionUtil::CheckKeyForConflicts(db_impl_, cfh, key.ToString(),
+ snap_seq, false /* cache_only */,
+ &snap_checker, min_uncommitted);
+}
+
+void WritePreparedTxn::SetSnapshot() {
+ const bool kForWWConflictCheck = true;
+ SnapshotImpl* snapshot = wpt_db_->GetSnapshotInternal(kForWWConflictCheck);
+ SetSnapshotInternal(snapshot);
+}
+
+Status WritePreparedTxn::RebuildFromWriteBatch(WriteBatch* src_batch) {
+ auto ret = PessimisticTransaction::RebuildFromWriteBatch(src_batch);
+ prepare_batch_cnt_ = GetWriteBatch()->SubBatchCnt();
+ return ret;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_prepared_txn.h b/src/rocksdb/utilities/transactions/write_prepared_txn.h
new file mode 100644
index 000000000..30d9bdb99
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_prepared_txn.h
@@ -0,0 +1,119 @@
+// 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/pessimistic_transaction.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+#include "utilities/transactions/transaction_base.h"
+#include "utilities/transactions/transaction_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class WritePreparedTxnDB;
+
+// This impl could write to DB also uncommitted data and then later tell apart
+// committed data from uncommitted data. Uncommitted data could be after the
+// Prepare phase in 2PC (WritePreparedTxn) or before that
+// (WriteUnpreparedTxnImpl).
+class WritePreparedTxn : public PessimisticTransaction {
+ public:
+ WritePreparedTxn(WritePreparedTxnDB* db, const WriteOptions& write_options,
+ const TransactionOptions& txn_options);
+ // No copying allowed
+ WritePreparedTxn(const WritePreparedTxn&) = delete;
+ void operator=(const WritePreparedTxn&) = delete;
+
+ virtual ~WritePreparedTxn() {}
+
+ // To make WAL commit markers visible, the snapshot will be based on the last
+ // seq in the WAL that is also published, LastPublishedSequence, as opposed to
+ // the last seq in the memtable.
+ using Transaction::Get;
+ virtual Status Get(const ReadOptions& options,
+ ColumnFamilyHandle* column_family, const Slice& key,
+ PinnableSlice* value) override;
+
+ using Transaction::MultiGet;
+ virtual void MultiGet(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ const size_t num_keys, const Slice* keys,
+ PinnableSlice* values, Status* statuses,
+ const bool sorted_input = false) override;
+
+ // Note: The behavior is undefined in presence of interleaved writes to the
+ // same transaction.
+ // To make WAL commit markers visible, the snapshot will be
+ // based on the last seq in the WAL that is also published,
+ // LastPublishedSequence, as opposed to the last seq in the memtable.
+ using Transaction::GetIterator;
+ virtual Iterator* GetIterator(const ReadOptions& options) override;
+ virtual Iterator* GetIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family) override;
+
+ virtual void SetSnapshot() override;
+
+ protected:
+ void Initialize(const TransactionOptions& txn_options) override;
+ // Override the protected SetId to make it visible to the friend class
+ // WritePreparedTxnDB
+ inline void SetId(uint64_t id) override { Transaction::SetId(id); }
+
+ private:
+ friend class WritePreparedTransactionTest_BasicRecoveryTest_Test;
+ friend class WritePreparedTxnDB;
+ friend class WriteUnpreparedTxnDB;
+ friend class WriteUnpreparedTxn;
+
+ Status PrepareInternal() override;
+
+ Status CommitWithoutPrepareInternal() override;
+
+ Status CommitBatchInternal(WriteBatch* batch, size_t batch_cnt) override;
+
+ // Since the data is already written to memtables at the Prepare phase, the
+ // commit entails writing only a commit marker in the WAL. The sequence number
+ // of the commit marker is then the commit timestamp of the transaction. To
+ // make WAL commit markers visible, the snapshot will be based on the last seq
+ // in the WAL that is also published, LastPublishedSequence, as opposed to the
+ // last seq in the memtable.
+ Status CommitInternal() override;
+
+ Status RollbackInternal() override;
+
+ virtual Status ValidateSnapshot(ColumnFamilyHandle* column_family,
+ const Slice& key,
+ SequenceNumber* tracked_at_seq) override;
+
+ virtual Status RebuildFromWriteBatch(WriteBatch* src_batch) override;
+
+ WritePreparedTxnDB* wpt_db_;
+ // Number of sub-batches in prepare
+ size_t prepare_batch_cnt_ = 0;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_prepared_txn_db.cc b/src/rocksdb/utilities/transactions/write_prepared_txn_db.cc
new file mode 100644
index 000000000..051fae554
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_prepared_txn_db.cc
@@ -0,0 +1,998 @@
+// 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/write_prepared_txn_db.h"
+
+#include <algorithm>
+#include <cinttypes>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "db/arena_wrapped_db_iter.h"
+#include "db/db_impl/db_impl.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 "util/string_util.h"
+#include "utilities/transactions/pessimistic_transaction.h"
+#include "utilities/transactions/transaction_db_mutex_impl.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+Status WritePreparedTxnDB::Initialize(
+ const std::vector<size_t>& compaction_enabled_cf_indices,
+ const std::vector<ColumnFamilyHandle*>& handles) {
+ auto dbimpl = static_cast_with_check<DBImpl, DB>(GetRootDB());
+ assert(dbimpl != nullptr);
+ auto rtxns = dbimpl->recovered_transactions();
+ std::map<SequenceNumber, SequenceNumber> ordered_seq_cnt;
+ for (auto rtxn : rtxns) {
+ // There should only one batch for WritePrepared policy.
+ assert(rtxn.second->batches_.size() == 1);
+ const auto& seq = rtxn.second->batches_.begin()->first;
+ const auto& batch_info = rtxn.second->batches_.begin()->second;
+ auto cnt = batch_info.batch_cnt_ ? batch_info.batch_cnt_ : 1;
+ ordered_seq_cnt[seq] = cnt;
+ }
+ // AddPrepared must be called in order
+ for (auto seq_cnt : ordered_seq_cnt) {
+ auto seq = seq_cnt.first;
+ auto cnt = seq_cnt.second;
+ for (size_t i = 0; i < cnt; i++) {
+ AddPrepared(seq + i);
+ }
+ }
+ SequenceNumber prev_max = max_evicted_seq_;
+ SequenceNumber last_seq = db_impl_->GetLatestSequenceNumber();
+ AdvanceMaxEvictedSeq(prev_max, last_seq);
+ // Create a gap between max and the next snapshot. This simplifies the logic
+ // in IsInSnapshot by not having to consider the special case of max ==
+ // snapshot after recovery. This is tested in IsInSnapshotEmptyMapTest.
+ if (last_seq) {
+ db_impl_->versions_->SetLastAllocatedSequence(last_seq + 1);
+ db_impl_->versions_->SetLastSequence(last_seq + 1);
+ db_impl_->versions_->SetLastPublishedSequence(last_seq + 1);
+ }
+
+ db_impl_->SetSnapshotChecker(new WritePreparedSnapshotChecker(this));
+ // A callback to commit a single sub-batch
+ class CommitSubBatchPreReleaseCallback : public PreReleaseCallback {
+ public:
+ explicit CommitSubBatchPreReleaseCallback(WritePreparedTxnDB* db)
+ : db_(db) {}
+ Status Callback(SequenceNumber commit_seq,
+ bool is_mem_disabled __attribute__((__unused__)), uint64_t,
+ size_t /*index*/, size_t /*total*/) override {
+ assert(!is_mem_disabled);
+ db_->AddCommitted(commit_seq, commit_seq);
+ return Status::OK();
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ };
+ db_impl_->SetRecoverableStatePreReleaseCallback(
+ new CommitSubBatchPreReleaseCallback(this));
+
+ auto s = PessimisticTransactionDB::Initialize(compaction_enabled_cf_indices,
+ handles);
+ return s;
+}
+
+Status WritePreparedTxnDB::VerifyCFOptions(
+ const ColumnFamilyOptions& cf_options) {
+ Status s = PessimisticTransactionDB::VerifyCFOptions(cf_options);
+ if (!s.ok()) {
+ return s;
+ }
+ if (!cf_options.memtable_factory->CanHandleDuplicatedKey()) {
+ return Status::InvalidArgument(
+ "memtable_factory->CanHandleDuplicatedKey() cannot be false with "
+ "WritePrpeared transactions");
+ }
+ return Status::OK();
+}
+
+Transaction* WritePreparedTxnDB::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 WritePreparedTxn(this, write_options, txn_options);
+ }
+}
+
+Status WritePreparedTxnDB::Write(const WriteOptions& opts,
+ WriteBatch* updates) {
+ if (txn_db_options_.skip_concurrency_control) {
+ // Skip locking the rows
+ const size_t UNKNOWN_BATCH_CNT = 0;
+ WritePreparedTxn* NO_TXN = nullptr;
+ return WriteInternal(opts, updates, UNKNOWN_BATCH_CNT, NO_TXN);
+ } else {
+ return PessimisticTransactionDB::WriteWithConcurrencyControl(opts, updates);
+ }
+}
+
+Status WritePreparedTxnDB::Write(
+ const WriteOptions& opts,
+ const TransactionDBWriteOptimizations& optimizations, WriteBatch* updates) {
+ if (optimizations.skip_concurrency_control) {
+ // Skip locking the rows
+ const size_t UNKNOWN_BATCH_CNT = 0;
+ const size_t ONE_BATCH_CNT = 1;
+ const size_t batch_cnt = optimizations.skip_duplicate_key_check
+ ? ONE_BATCH_CNT
+ : UNKNOWN_BATCH_CNT;
+ WritePreparedTxn* NO_TXN = nullptr;
+ return WriteInternal(opts, updates, batch_cnt, NO_TXN);
+ } else {
+ // TODO(myabandeh): Make use of skip_duplicate_key_check hint
+ // Fall back to unoptimized version
+ return PessimisticTransactionDB::WriteWithConcurrencyControl(opts, updates);
+ }
+}
+
+Status WritePreparedTxnDB::WriteInternal(const WriteOptions& write_options_orig,
+ WriteBatch* batch, size_t batch_cnt,
+ WritePreparedTxn* txn) {
+ ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
+ "CommitBatchInternal");
+ if (batch->Count() == 0) {
+ // Otherwise our 1 seq per batch logic will break since there is no seq
+ // increased for this batch.
+ return Status::OK();
+ }
+ if (batch_cnt == 0) { // not provided, then compute it
+ // TODO(myabandeh): add an option to allow user skipping this cost
+ SubBatchCounter counter(*GetCFComparatorMap());
+ auto s = batch->Iterate(&counter);
+ assert(s.ok());
+ batch_cnt = counter.BatchCount();
+ WPRecordTick(TXN_DUPLICATE_KEY_OVERHEAD);
+ ROCKS_LOG_DETAILS(info_log_, "Duplicate key overhead: %" PRIu64 " batches",
+ static_cast<uint64_t>(batch_cnt));
+ }
+ assert(batch_cnt);
+
+ bool do_one_write = !db_impl_->immutable_db_options().two_write_queues;
+ WriteOptions write_options(write_options_orig);
+ // In the absence of Prepare markers, use Noop as a batch separator
+ WriteBatchInternal::InsertNoop(batch);
+ const bool DISABLE_MEMTABLE = true;
+ const uint64_t no_log_ref = 0;
+ uint64_t seq_used = kMaxSequenceNumber;
+ const size_t ZERO_PREPARES = 0;
+ const bool kSeperatePrepareCommitBatches = true;
+ // Since this is not 2pc, there is no need for AddPrepared but having it in
+ // the PreReleaseCallback enables an optimization. Refer to
+ // SmallestUnCommittedSeq for more details.
+ AddPreparedCallback add_prepared_callback(
+ this, db_impl_, batch_cnt,
+ db_impl_->immutable_db_options().two_write_queues,
+ !kSeperatePrepareCommitBatches);
+ WritePreparedCommitEntryPreReleaseCallback update_commit_map(
+ this, db_impl_, kMaxSequenceNumber, ZERO_PREPARES, batch_cnt);
+ PreReleaseCallback* pre_release_callback;
+ if (do_one_write) {
+ pre_release_callback = &update_commit_map;
+ } else {
+ pre_release_callback = &add_prepared_callback;
+ }
+ auto s = db_impl_->WriteImpl(write_options, batch, nullptr, nullptr,
+ no_log_ref, !DISABLE_MEMTABLE, &seq_used,
+ batch_cnt, pre_release_callback);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ uint64_t prepare_seq = seq_used;
+ if (txn != nullptr) {
+ txn->SetId(prepare_seq);
+ }
+ if (!s.ok()) {
+ return s;
+ }
+ if (do_one_write) {
+ return s;
+ } // else do the 2nd write for commit
+ ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
+ "CommitBatchInternal 2nd write prepare_seq: %" PRIu64,
+ prepare_seq);
+ // Commit the batch by writing an empty batch to the 2nd queue that will
+ // release the commit sequence number to readers.
+ const size_t ZERO_COMMITS = 0;
+ WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_prepare(
+ this, db_impl_, prepare_seq, batch_cnt, ZERO_COMMITS);
+ WriteBatch empty_batch;
+ write_options.disableWAL = true;
+ write_options.sync = false;
+ const size_t ONE_BATCH = 1; // Just to inc the seq
+ s = db_impl_->WriteImpl(write_options, &empty_batch, nullptr, nullptr,
+ no_log_ref, DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
+ &update_commit_map_with_prepare);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ // Note: RemovePrepared is called from within PreReleaseCallback
+ return s;
+}
+
+Status WritePreparedTxnDB::Get(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ const Slice& key, PinnableSlice* value) {
+ SequenceNumber min_uncommitted, snap_seq;
+ const SnapshotBackup backed_by_snapshot =
+ AssignMinMaxSeqs(options.snapshot, &min_uncommitted, &snap_seq);
+ WritePreparedTxnReadCallback callback(this, snap_seq, min_uncommitted,
+ backed_by_snapshot);
+ bool* dont_care = nullptr;
+ DBImpl::GetImplOptions get_impl_options;
+ get_impl_options.column_family = column_family;
+ get_impl_options.value = value;
+ get_impl_options.value_found = dont_care;
+ get_impl_options.callback = &callback;
+ auto res = db_impl_->GetImpl(options, key, get_impl_options);
+ if (LIKELY(callback.valid() && ValidateSnapshot(callback.max_visible_seq(),
+ backed_by_snapshot))) {
+ return res;
+ } else {
+ WPRecordTick(TXN_GET_TRY_AGAIN);
+ return Status::TryAgain();
+ }
+}
+
+void WritePreparedTxnDB::UpdateCFComparatorMap(
+ const std::vector<ColumnFamilyHandle*>& handles) {
+ auto cf_map = new std::map<uint32_t, const Comparator*>();
+ auto handle_map = new std::map<uint32_t, ColumnFamilyHandle*>();
+ for (auto h : handles) {
+ auto id = h->GetID();
+ const Comparator* comparator = h->GetComparator();
+ (*cf_map)[id] = comparator;
+ if (id != 0) {
+ (*handle_map)[id] = h;
+ } else {
+ // The pointer to the default cf handle in the handles will be deleted.
+ // Use the pointer maintained by the db instead.
+ (*handle_map)[id] = DefaultColumnFamily();
+ }
+ }
+ cf_map_.reset(cf_map);
+ handle_map_.reset(handle_map);
+}
+
+void WritePreparedTxnDB::UpdateCFComparatorMap(ColumnFamilyHandle* h) {
+ auto old_cf_map_ptr = cf_map_.get();
+ assert(old_cf_map_ptr);
+ auto cf_map = new std::map<uint32_t, const Comparator*>(*old_cf_map_ptr);
+ auto old_handle_map_ptr = handle_map_.get();
+ assert(old_handle_map_ptr);
+ auto handle_map =
+ new std::map<uint32_t, ColumnFamilyHandle*>(*old_handle_map_ptr);
+ auto id = h->GetID();
+ const Comparator* comparator = h->GetComparator();
+ (*cf_map)[id] = comparator;
+ (*handle_map)[id] = h;
+ cf_map_.reset(cf_map);
+ handle_map_.reset(handle_map);
+}
+
+
+std::vector<Status> WritePreparedTxnDB::MultiGet(
+ const ReadOptions& options,
+ const std::vector<ColumnFamilyHandle*>& column_family,
+ const std::vector<Slice>& keys, std::vector<std::string>* values) {
+ assert(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) {
+ std::string* value = values ? &(*values)[i] : nullptr;
+ stat_list[i] = this->Get(options, column_family[i], keys[i], value);
+ }
+ return stat_list;
+}
+
+// Struct to hold ownership of snapshot and read callback for iterator cleanup.
+struct WritePreparedTxnDB::IteratorState {
+ IteratorState(WritePreparedTxnDB* txn_db, SequenceNumber sequence,
+ std::shared_ptr<ManagedSnapshot> s,
+ SequenceNumber min_uncommitted)
+ : callback(txn_db, sequence, min_uncommitted, kBackedByDBSnapshot),
+ snapshot(s) {}
+
+ WritePreparedTxnReadCallback callback;
+ std::shared_ptr<ManagedSnapshot> snapshot;
+};
+
+namespace {
+static void CleanupWritePreparedTxnDBIterator(void* arg1, void* /*arg2*/) {
+ delete reinterpret_cast<WritePreparedTxnDB::IteratorState*>(arg1);
+}
+} // anonymous namespace
+
+Iterator* WritePreparedTxnDB::NewIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family) {
+ constexpr bool ALLOW_BLOB = true;
+ constexpr bool ALLOW_REFRESH = true;
+ std::shared_ptr<ManagedSnapshot> own_snapshot = nullptr;
+ SequenceNumber snapshot_seq = kMaxSequenceNumber;
+ SequenceNumber min_uncommitted = 0;
+ if (options.snapshot != nullptr) {
+ snapshot_seq = options.snapshot->GetSequenceNumber();
+ min_uncommitted =
+ static_cast_with_check<const SnapshotImpl, const Snapshot>(
+ options.snapshot)
+ ->min_uncommitted_;
+ } else {
+ auto* snapshot = GetSnapshot();
+ // We take a snapshot to make sure that the related data in the commit map
+ // are not deleted.
+ snapshot_seq = snapshot->GetSequenceNumber();
+ min_uncommitted =
+ static_cast_with_check<const SnapshotImpl, const Snapshot>(snapshot)
+ ->min_uncommitted_;
+ own_snapshot = std::make_shared<ManagedSnapshot>(db_impl_, snapshot);
+ }
+ assert(snapshot_seq != kMaxSequenceNumber);
+ auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
+ auto* state =
+ new IteratorState(this, snapshot_seq, own_snapshot, min_uncommitted);
+ auto* db_iter =
+ db_impl_->NewIteratorImpl(options, cfd, snapshot_seq, &state->callback,
+ !ALLOW_BLOB, !ALLOW_REFRESH);
+ db_iter->RegisterCleanup(CleanupWritePreparedTxnDBIterator, state, nullptr);
+ return db_iter;
+}
+
+Status WritePreparedTxnDB::NewIterators(
+ const ReadOptions& options,
+ const std::vector<ColumnFamilyHandle*>& column_families,
+ std::vector<Iterator*>* iterators) {
+ constexpr bool ALLOW_BLOB = true;
+ constexpr bool ALLOW_REFRESH = true;
+ std::shared_ptr<ManagedSnapshot> own_snapshot = nullptr;
+ SequenceNumber snapshot_seq = kMaxSequenceNumber;
+ SequenceNumber min_uncommitted = 0;
+ if (options.snapshot != nullptr) {
+ snapshot_seq = options.snapshot->GetSequenceNumber();
+ min_uncommitted = static_cast_with_check<const SnapshotImpl, const Snapshot>(
+ options.snapshot)
+ ->min_uncommitted_;
+ } else {
+ auto* snapshot = GetSnapshot();
+ // We take a snapshot to make sure that the related data in the commit map
+ // are not deleted.
+ snapshot_seq = snapshot->GetSequenceNumber();
+ own_snapshot = std::make_shared<ManagedSnapshot>(db_impl_, snapshot);
+ min_uncommitted =
+ static_cast_with_check<const SnapshotImpl, const Snapshot>(snapshot)
+ ->min_uncommitted_;
+ }
+ iterators->clear();
+ iterators->reserve(column_families.size());
+ for (auto* column_family : column_families) {
+ auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
+ auto* state =
+ new IteratorState(this, snapshot_seq, own_snapshot, min_uncommitted);
+ auto* db_iter =
+ db_impl_->NewIteratorImpl(options, cfd, snapshot_seq, &state->callback,
+ !ALLOW_BLOB, !ALLOW_REFRESH);
+ db_iter->RegisterCleanup(CleanupWritePreparedTxnDBIterator, state, nullptr);
+ iterators->push_back(db_iter);
+ }
+ return Status::OK();
+}
+
+void WritePreparedTxnDB::Init(const TransactionDBOptions& /* unused */) {
+ // Adcance max_evicted_seq_ no more than 100 times before the cache wraps
+ // around.
+ INC_STEP_FOR_MAX_EVICTED =
+ std::max(COMMIT_CACHE_SIZE / 100, static_cast<size_t>(1));
+ snapshot_cache_ = std::unique_ptr<std::atomic<SequenceNumber>[]>(
+ new std::atomic<SequenceNumber>[SNAPSHOT_CACHE_SIZE] {});
+ commit_cache_ = std::unique_ptr<std::atomic<CommitEntry64b>[]>(
+ new std::atomic<CommitEntry64b>[COMMIT_CACHE_SIZE] {});
+ dummy_max_snapshot_.number_ = kMaxSequenceNumber;
+}
+
+void WritePreparedTxnDB::CheckPreparedAgainstMax(SequenceNumber new_max,
+ bool locked) {
+ // When max_evicted_seq_ advances, move older entries from prepared_txns_
+ // to delayed_prepared_. This guarantees that if a seq is lower than max,
+ // then it is not in prepared_txns_ and save an expensive, synchronized
+ // lookup from a shared set. delayed_prepared_ is expected to be empty in
+ // normal cases.
+ ROCKS_LOG_DETAILS(
+ info_log_,
+ "CheckPreparedAgainstMax prepared_txns_.empty() %d top: %" PRIu64,
+ prepared_txns_.empty(),
+ prepared_txns_.empty() ? 0 : prepared_txns_.top());
+ const SequenceNumber prepared_top = prepared_txns_.top();
+ const bool empty = prepared_top == kMaxSequenceNumber;
+ // Preliminary check to avoid the synchronization cost
+ if (!empty && prepared_top <= new_max) {
+ if (locked) {
+ // Needed to avoid double locking in pop().
+ prepared_txns_.push_pop_mutex()->Unlock();
+ }
+ WriteLock wl(&prepared_mutex_);
+ // Need to fetch fresh values of ::top after mutex is acquired
+ while (!prepared_txns_.empty() && prepared_txns_.top() <= new_max) {
+ auto to_be_popped = prepared_txns_.top();
+ delayed_prepared_.insert(to_be_popped);
+ ROCKS_LOG_WARN(info_log_,
+ "prepared_mutex_ overhead %" PRIu64 " (prep=%" PRIu64
+ " new_max=%" PRIu64,
+ static_cast<uint64_t>(delayed_prepared_.size()),
+ to_be_popped, new_max);
+ delayed_prepared_empty_.store(false, std::memory_order_release);
+ // Update prepared_txns_ after updating delayed_prepared_empty_ otherwise
+ // there will be a point in time that the entry is neither in
+ // prepared_txns_ nor in delayed_prepared_, which will not be checked if
+ // delayed_prepared_empty_ is false.
+ prepared_txns_.pop();
+ }
+ if (locked) {
+ prepared_txns_.push_pop_mutex()->Lock();
+ }
+ }
+}
+
+void WritePreparedTxnDB::AddPrepared(uint64_t seq, bool locked) {
+ ROCKS_LOG_DETAILS(info_log_, "Txn %" PRIu64 " Preparing with max %" PRIu64,
+ seq, max_evicted_seq_.load());
+ TEST_SYNC_POINT("AddPrepared::begin:pause");
+ TEST_SYNC_POINT("AddPrepared::begin:resume");
+ if (!locked) {
+ prepared_txns_.push_pop_mutex()->Lock();
+ }
+ prepared_txns_.push_pop_mutex()->AssertHeld();
+ prepared_txns_.push(seq);
+ auto new_max = future_max_evicted_seq_.load();
+ if (UNLIKELY(seq <= new_max)) {
+ // This should not happen in normal case
+ ROCKS_LOG_ERROR(
+ info_log_,
+ "Added prepare_seq is not larger than max_evicted_seq_: %" PRIu64
+ " <= %" PRIu64,
+ seq, new_max);
+ CheckPreparedAgainstMax(new_max, true /*locked*/);
+ }
+ if (!locked) {
+ prepared_txns_.push_pop_mutex()->Unlock();
+ }
+ TEST_SYNC_POINT("AddPrepared::end");
+}
+
+void WritePreparedTxnDB::AddCommitted(uint64_t prepare_seq, uint64_t commit_seq,
+ uint8_t loop_cnt) {
+ ROCKS_LOG_DETAILS(info_log_, "Txn %" PRIu64 " Committing with %" PRIu64,
+ prepare_seq, commit_seq);
+ TEST_SYNC_POINT("WritePreparedTxnDB::AddCommitted:start");
+ TEST_SYNC_POINT("WritePreparedTxnDB::AddCommitted:start:pause");
+ auto indexed_seq = prepare_seq % COMMIT_CACHE_SIZE;
+ CommitEntry64b evicted_64b;
+ CommitEntry evicted;
+ bool to_be_evicted = GetCommitEntry(indexed_seq, &evicted_64b, &evicted);
+ if (LIKELY(to_be_evicted)) {
+ assert(evicted.prep_seq != prepare_seq);
+ auto prev_max = max_evicted_seq_.load(std::memory_order_acquire);
+ ROCKS_LOG_DETAILS(info_log_,
+ "Evicting %" PRIu64 ",%" PRIu64 " with max %" PRIu64,
+ evicted.prep_seq, evicted.commit_seq, prev_max);
+ if (prev_max < evicted.commit_seq) {
+ auto last = db_impl_->GetLastPublishedSequence(); // could be 0
+ SequenceNumber max_evicted_seq;
+ if (LIKELY(evicted.commit_seq < last)) {
+ assert(last > 0);
+ // Inc max in larger steps to avoid frequent updates
+ max_evicted_seq =
+ std::min(evicted.commit_seq + INC_STEP_FOR_MAX_EVICTED, last - 1);
+ } else {
+ // legit when a commit entry in a write batch overwrite the previous one
+ max_evicted_seq = evicted.commit_seq;
+ }
+ ROCKS_LOG_DETAILS(info_log_,
+ "%lu Evicting %" PRIu64 ",%" PRIu64 " with max %" PRIu64
+ " => %lu",
+ prepare_seq, evicted.prep_seq, evicted.commit_seq,
+ prev_max, max_evicted_seq);
+ AdvanceMaxEvictedSeq(prev_max, max_evicted_seq);
+ }
+ // After each eviction from commit cache, check if the commit entry should
+ // be kept around because it overlaps with a live snapshot.
+ CheckAgainstSnapshots(evicted);
+ if (UNLIKELY(!delayed_prepared_empty_.load(std::memory_order_acquire))) {
+ WriteLock wl(&prepared_mutex_);
+ for (auto dp : delayed_prepared_) {
+ if (dp == evicted.prep_seq) {
+ // This is a rare case that txn is committed but prepared_txns_ is not
+ // cleaned up yet. Refer to delayed_prepared_commits_ definition for
+ // why it should be kept updated.
+ delayed_prepared_commits_[evicted.prep_seq] = evicted.commit_seq;
+ ROCKS_LOG_DEBUG(info_log_,
+ "delayed_prepared_commits_[%" PRIu64 "]=%" PRIu64,
+ evicted.prep_seq, evicted.commit_seq);
+ break;
+ }
+ }
+ }
+ }
+ bool succ =
+ ExchangeCommitEntry(indexed_seq, evicted_64b, {prepare_seq, commit_seq});
+ if (UNLIKELY(!succ)) {
+ ROCKS_LOG_ERROR(info_log_,
+ "ExchangeCommitEntry failed on [%" PRIu64 "] %" PRIu64
+ ",%" PRIu64 " retrying...",
+ indexed_seq, prepare_seq, commit_seq);
+ // A very rare event, in which the commit entry is updated before we do.
+ // Here we apply a very simple solution of retrying.
+ if (loop_cnt > 100) {
+ throw std::runtime_error("Infinite loop in AddCommitted!");
+ }
+ AddCommitted(prepare_seq, commit_seq, ++loop_cnt);
+ return;
+ }
+ TEST_SYNC_POINT("WritePreparedTxnDB::AddCommitted:end");
+ TEST_SYNC_POINT("WritePreparedTxnDB::AddCommitted:end:pause");
+}
+
+void WritePreparedTxnDB::RemovePrepared(const uint64_t prepare_seq,
+ const size_t batch_cnt) {
+ TEST_SYNC_POINT_CALLBACK(
+ "RemovePrepared:Start",
+ const_cast<void*>(reinterpret_cast<const void*>(&prepare_seq)));
+ TEST_SYNC_POINT("WritePreparedTxnDB::RemovePrepared:pause");
+ TEST_SYNC_POINT("WritePreparedTxnDB::RemovePrepared:resume");
+ ROCKS_LOG_DETAILS(info_log_,
+ "RemovePrepared %" PRIu64 " cnt: %" ROCKSDB_PRIszt,
+ prepare_seq, batch_cnt);
+ WriteLock wl(&prepared_mutex_);
+ for (size_t i = 0; i < batch_cnt; i++) {
+ prepared_txns_.erase(prepare_seq + i);
+ bool was_empty = delayed_prepared_.empty();
+ if (!was_empty) {
+ delayed_prepared_.erase(prepare_seq + i);
+ auto it = delayed_prepared_commits_.find(prepare_seq + i);
+ if (it != delayed_prepared_commits_.end()) {
+ ROCKS_LOG_DETAILS(info_log_, "delayed_prepared_commits_.erase %" PRIu64,
+ prepare_seq + i);
+ delayed_prepared_commits_.erase(it);
+ }
+ bool is_empty = delayed_prepared_.empty();
+ if (was_empty != is_empty) {
+ delayed_prepared_empty_.store(is_empty, std::memory_order_release);
+ }
+ }
+ }
+}
+
+bool WritePreparedTxnDB::GetCommitEntry(const uint64_t indexed_seq,
+ CommitEntry64b* entry_64b,
+ CommitEntry* entry) const {
+ *entry_64b = commit_cache_[static_cast<size_t>(indexed_seq)].load(std::memory_order_acquire);
+ bool valid = entry_64b->Parse(indexed_seq, entry, FORMAT);
+ return valid;
+}
+
+bool WritePreparedTxnDB::AddCommitEntry(const uint64_t indexed_seq,
+ const CommitEntry& new_entry,
+ CommitEntry* evicted_entry) {
+ CommitEntry64b new_entry_64b(new_entry, FORMAT);
+ CommitEntry64b evicted_entry_64b = commit_cache_[static_cast<size_t>(indexed_seq)].exchange(
+ new_entry_64b, std::memory_order_acq_rel);
+ bool valid = evicted_entry_64b.Parse(indexed_seq, evicted_entry, FORMAT);
+ return valid;
+}
+
+bool WritePreparedTxnDB::ExchangeCommitEntry(const uint64_t indexed_seq,
+ CommitEntry64b& expected_entry_64b,
+ const CommitEntry& new_entry) {
+ auto& atomic_entry = commit_cache_[static_cast<size_t>(indexed_seq)];
+ CommitEntry64b new_entry_64b(new_entry, FORMAT);
+ bool succ = atomic_entry.compare_exchange_strong(
+ expected_entry_64b, new_entry_64b, std::memory_order_acq_rel,
+ std::memory_order_acquire);
+ return succ;
+}
+
+void WritePreparedTxnDB::AdvanceMaxEvictedSeq(const SequenceNumber& prev_max,
+ const SequenceNumber& new_max) {
+ ROCKS_LOG_DETAILS(info_log_,
+ "AdvanceMaxEvictedSeq overhead %" PRIu64 " => %" PRIu64,
+ prev_max, new_max);
+ // Declare the intention before getting snapshot from the DB. This helps a
+ // concurrent GetSnapshot to wait to catch up with future_max_evicted_seq_ if
+ // it has not already. Otherwise the new snapshot is when we ask DB for
+ // snapshots smaller than future max.
+ auto updated_future_max = prev_max;
+ while (updated_future_max < new_max &&
+ !future_max_evicted_seq_.compare_exchange_weak(
+ updated_future_max, new_max, std::memory_order_acq_rel,
+ std::memory_order_relaxed)) {
+ };
+
+ CheckPreparedAgainstMax(new_max, false /*locked*/);
+
+ // With each change to max_evicted_seq_ fetch the live snapshots behind it.
+ // We use max as the version of snapshots to identify how fresh are the
+ // snapshot list. This works because the snapshots are between 0 and
+ // max, so the larger the max, the more complete they are.
+ SequenceNumber new_snapshots_version = new_max;
+ std::vector<SequenceNumber> snapshots;
+ bool update_snapshots = false;
+ if (new_snapshots_version > snapshots_version_) {
+ // This is to avoid updating the snapshots_ if it already updated
+ // with a more recent vesion by a concrrent thread
+ update_snapshots = true;
+ // We only care about snapshots lower then max
+ snapshots = GetSnapshotListFromDB(new_max);
+ }
+ if (update_snapshots) {
+ UpdateSnapshots(snapshots, new_snapshots_version);
+ if (!snapshots.empty()) {
+ WriteLock wl(&old_commit_map_mutex_);
+ for (auto snap : snapshots) {
+ // This allows IsInSnapshot to tell apart the reads from in valid
+ // snapshots from the reads from committed values in valid snapshots.
+ old_commit_map_[snap];
+ }
+ old_commit_map_empty_.store(false, std::memory_order_release);
+ }
+ }
+ auto updated_prev_max = prev_max;
+ TEST_SYNC_POINT("AdvanceMaxEvictedSeq::update_max:pause");
+ TEST_SYNC_POINT("AdvanceMaxEvictedSeq::update_max:resume");
+ while (updated_prev_max < new_max &&
+ !max_evicted_seq_.compare_exchange_weak(updated_prev_max, new_max,
+ std::memory_order_acq_rel,
+ std::memory_order_relaxed)) {
+ };
+}
+
+const Snapshot* WritePreparedTxnDB::GetSnapshot() {
+ const bool kForWWConflictCheck = true;
+ return GetSnapshotInternal(!kForWWConflictCheck);
+}
+
+SnapshotImpl* WritePreparedTxnDB::GetSnapshotInternal(
+ bool for_ww_conflict_check) {
+ // Note: for this optimization setting the last sequence number and obtaining
+ // the smallest uncommitted seq should be done atomically. However to avoid
+ // the mutex overhead, we call SmallestUnCommittedSeq BEFORE taking the
+ // snapshot. Since we always updated the list of unprepared seq (via
+ // AddPrepared) AFTER the last sequence is updated, this guarantees that the
+ // smallest uncommitted seq that we pair with the snapshot is smaller or equal
+ // the value that would be obtained otherwise atomically. That is ok since
+ // this optimization works as long as min_uncommitted is less than or equal
+ // than the smallest uncommitted seq when the snapshot was taken.
+ auto min_uncommitted = WritePreparedTxnDB::SmallestUnCommittedSeq();
+ SnapshotImpl* snap_impl = db_impl_->GetSnapshotImpl(for_ww_conflict_check);
+ TEST_SYNC_POINT("WritePreparedTxnDB::GetSnapshotInternal:first");
+ assert(snap_impl);
+ SequenceNumber snap_seq = snap_impl->GetSequenceNumber();
+ // Note: Check against future_max_evicted_seq_ (in contrast with
+ // max_evicted_seq_) in case there is a concurrent AdvanceMaxEvictedSeq.
+ if (UNLIKELY(snap_seq != 0 && snap_seq <= future_max_evicted_seq_)) {
+ // There is a very rare case in which the commit entry evicts another commit
+ // entry that is not published yet thus advancing max evicted seq beyond the
+ // last published seq. This case is not likely in real-world setup so we
+ // handle it with a few retries.
+ size_t retry = 0;
+ SequenceNumber max;
+ while ((max = future_max_evicted_seq_.load()) != 0 &&
+ snap_impl->GetSequenceNumber() <= max && retry < 100) {
+ ROCKS_LOG_WARN(info_log_,
+ "GetSnapshot snap: %" PRIu64 " max: %" PRIu64
+ " retry %" ROCKSDB_PRIszt,
+ snap_impl->GetSequenceNumber(), max, retry);
+ ReleaseSnapshot(snap_impl);
+ // Wait for last visible seq to catch up with max, and also go beyond it
+ // by one.
+ AdvanceSeqByOne();
+ snap_impl = db_impl_->GetSnapshotImpl(for_ww_conflict_check);
+ assert(snap_impl);
+ retry++;
+ }
+ assert(snap_impl->GetSequenceNumber() > max);
+ if (snap_impl->GetSequenceNumber() <= max) {
+ throw std::runtime_error(
+ "Snapshot seq " + ToString(snap_impl->GetSequenceNumber()) +
+ " after " + ToString(retry) +
+ " retries is still less than futre_max_evicted_seq_" + ToString(max));
+ }
+ }
+ EnhanceSnapshot(snap_impl, min_uncommitted);
+ ROCKS_LOG_DETAILS(
+ db_impl_->immutable_db_options().info_log,
+ "GetSnapshot %" PRIu64 " ww:%" PRIi32 " min_uncommitted: %" PRIu64,
+ snap_impl->GetSequenceNumber(), for_ww_conflict_check, min_uncommitted);
+ TEST_SYNC_POINT("WritePreparedTxnDB::GetSnapshotInternal:end");
+ return snap_impl;
+}
+
+void WritePreparedTxnDB::AdvanceSeqByOne() {
+ // Inserting an empty value will i) let the max evicted entry to be
+ // published, i.e., max == last_published, increase the last published to
+ // be one beyond max, i.e., max < last_published.
+ WriteOptions woptions;
+ TransactionOptions txn_options;
+ Transaction* txn0 = BeginTransaction(woptions, txn_options, nullptr);
+ std::hash<std::thread::id> hasher;
+ char name[64];
+ snprintf(name, 64, "txn%" ROCKSDB_PRIszt, hasher(std::this_thread::get_id()));
+ assert(strlen(name) < 64 - 1);
+ Status s = txn0->SetName(name);
+ assert(s.ok());
+ if (s.ok()) {
+ // Without prepare it would simply skip the commit
+ s = txn0->Prepare();
+ }
+ assert(s.ok());
+ if (s.ok()) {
+ s = txn0->Commit();
+ }
+ assert(s.ok());
+ delete txn0;
+}
+
+const std::vector<SequenceNumber> WritePreparedTxnDB::GetSnapshotListFromDB(
+ SequenceNumber max) {
+ ROCKS_LOG_DETAILS(info_log_, "GetSnapshotListFromDB with max %" PRIu64, max);
+ InstrumentedMutexLock dblock(db_impl_->mutex());
+ db_impl_->mutex()->AssertHeld();
+ return db_impl_->snapshots().GetAll(nullptr, max);
+}
+
+void WritePreparedTxnDB::ReleaseSnapshotInternal(
+ const SequenceNumber snap_seq) {
+ // TODO(myabandeh): relax should enough since the synchronizatin is already
+ // done by snapshots_mutex_ under which this function is called.
+ if (snap_seq <= max_evicted_seq_.load(std::memory_order_acquire)) {
+ // Then this is a rare case that transaction did not finish before max
+ // advances. It is expected for a few read-only backup snapshots. For such
+ // snapshots we might have kept around a couple of entries in the
+ // old_commit_map_. Check and do garbage collection if that is the case.
+ bool need_gc = false;
+ {
+ WPRecordTick(TXN_OLD_COMMIT_MAP_MUTEX_OVERHEAD);
+ ROCKS_LOG_WARN(info_log_, "old_commit_map_mutex_ overhead for %" PRIu64,
+ snap_seq);
+ ReadLock rl(&old_commit_map_mutex_);
+ auto prep_set_entry = old_commit_map_.find(snap_seq);
+ need_gc = prep_set_entry != old_commit_map_.end();
+ }
+ if (need_gc) {
+ WPRecordTick(TXN_OLD_COMMIT_MAP_MUTEX_OVERHEAD);
+ ROCKS_LOG_WARN(info_log_, "old_commit_map_mutex_ overhead for %" PRIu64,
+ snap_seq);
+ WriteLock wl(&old_commit_map_mutex_);
+ old_commit_map_.erase(snap_seq);
+ old_commit_map_empty_.store(old_commit_map_.empty(),
+ std::memory_order_release);
+ }
+ }
+}
+
+void WritePreparedTxnDB::CleanupReleasedSnapshots(
+ const std::vector<SequenceNumber>& new_snapshots,
+ const std::vector<SequenceNumber>& old_snapshots) {
+ auto newi = new_snapshots.begin();
+ auto oldi = old_snapshots.begin();
+ for (; newi != new_snapshots.end() && oldi != old_snapshots.end();) {
+ assert(*newi >= *oldi); // cannot have new snapshots with lower seq
+ if (*newi == *oldi) { // still not released
+ auto value = *newi;
+ while (newi != new_snapshots.end() && *newi == value) {
+ newi++;
+ }
+ while (oldi != old_snapshots.end() && *oldi == value) {
+ oldi++;
+ }
+ } else {
+ assert(*newi > *oldi); // *oldi is released
+ ReleaseSnapshotInternal(*oldi);
+ oldi++;
+ }
+ }
+ // Everything remained in old_snapshots is released and must be cleaned up
+ for (; oldi != old_snapshots.end(); oldi++) {
+ ReleaseSnapshotInternal(*oldi);
+ }
+}
+
+void WritePreparedTxnDB::UpdateSnapshots(
+ const std::vector<SequenceNumber>& snapshots,
+ const SequenceNumber& version) {
+ ROCKS_LOG_DETAILS(info_log_, "UpdateSnapshots with version %" PRIu64,
+ version);
+ TEST_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:p:start");
+ TEST_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:s:start");
+#ifndef NDEBUG
+ size_t sync_i = 0;
+#endif
+ ROCKS_LOG_DETAILS(info_log_, "snapshots_mutex_ overhead");
+ WriteLock wl(&snapshots_mutex_);
+ snapshots_version_ = version;
+ // We update the list concurrently with the readers.
+ // Both new and old lists are sorted and the new list is subset of the
+ // previous list plus some new items. Thus if a snapshot repeats in
+ // both new and old lists, it will appear upper in the new list. So if
+ // we simply insert the new snapshots in order, if an overwritten item
+ // is still valid in the new list is either written to the same place in
+ // the array or it is written in a higher palce before it gets
+ // overwritten by another item. This guarantess a reader that reads the
+ // list bottom-up will eventaully see a snapshot that repeats in the
+ // update, either before it gets overwritten by the writer or
+ // afterwards.
+ size_t i = 0;
+ auto it = snapshots.begin();
+ for (; it != snapshots.end() && i < SNAPSHOT_CACHE_SIZE; ++it, ++i) {
+ snapshot_cache_[i].store(*it, std::memory_order_release);
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:p:", ++sync_i);
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:s:", sync_i);
+ }
+#ifndef NDEBUG
+ // Release the remaining sync points since they are useless given that the
+ // reader would also use lock to access snapshots
+ for (++sync_i; sync_i <= 10; ++sync_i) {
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:p:", sync_i);
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:s:", sync_i);
+ }
+#endif
+ snapshots_.clear();
+ for (; it != snapshots.end(); ++it) {
+ // Insert them to a vector that is less efficient to access
+ // concurrently
+ snapshots_.push_back(*it);
+ }
+ // Update the size at the end. Otherwise a parallel reader might read
+ // items that are not set yet.
+ snapshots_total_.store(snapshots.size(), std::memory_order_release);
+
+ // Note: this must be done after the snapshots data structures are updated
+ // with the new list of snapshots.
+ CleanupReleasedSnapshots(snapshots, snapshots_all_);
+ snapshots_all_ = snapshots;
+
+ TEST_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:p:end");
+ TEST_SYNC_POINT("WritePreparedTxnDB::UpdateSnapshots:s:end");
+}
+
+void WritePreparedTxnDB::CheckAgainstSnapshots(const CommitEntry& evicted) {
+ TEST_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:p:start");
+ TEST_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:s:start");
+#ifndef NDEBUG
+ size_t sync_i = 0;
+#endif
+ // First check the snapshot cache that is efficient for concurrent access
+ auto cnt = snapshots_total_.load(std::memory_order_acquire);
+ // The list might get updated concurrently as we are reading from it. The
+ // reader should be able to read all the snapshots that are still valid
+ // after the update. Since the survived snapshots are written in a higher
+ // place before gets overwritten the reader that reads bottom-up will
+ // eventully see it.
+ const bool next_is_larger = true;
+ // We will set to true if the border line snapshot suggests that.
+ bool search_larger_list = false;
+ size_t ip1 = std::min(cnt, SNAPSHOT_CACHE_SIZE);
+ for (; 0 < ip1; ip1--) {
+ SequenceNumber snapshot_seq =
+ snapshot_cache_[ip1 - 1].load(std::memory_order_acquire);
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:p:",
+ ++sync_i);
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:s:", sync_i);
+ if (ip1 == SNAPSHOT_CACHE_SIZE) { // border line snapshot
+ // snapshot_seq < commit_seq => larger_snapshot_seq <= commit_seq
+ // then later also continue the search to larger snapshots
+ search_larger_list = snapshot_seq < evicted.commit_seq;
+ }
+ if (!MaybeUpdateOldCommitMap(evicted.prep_seq, evicted.commit_seq,
+ snapshot_seq, !next_is_larger)) {
+ break;
+ }
+ }
+#ifndef NDEBUG
+ // Release the remaining sync points before accquiring the lock
+ for (++sync_i; sync_i <= 10; ++sync_i) {
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:p:", sync_i);
+ TEST_IDX_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:s:", sync_i);
+ }
+#endif
+ TEST_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:p:end");
+ TEST_SYNC_POINT("WritePreparedTxnDB::CheckAgainstSnapshots:s:end");
+ if (UNLIKELY(SNAPSHOT_CACHE_SIZE < cnt && search_larger_list)) {
+ // Then access the less efficient list of snapshots_
+ WPRecordTick(TXN_SNAPSHOT_MUTEX_OVERHEAD);
+ ROCKS_LOG_WARN(info_log_,
+ "snapshots_mutex_ overhead for <%" PRIu64 ",%" PRIu64
+ "> with %" ROCKSDB_PRIszt " snapshots",
+ evicted.prep_seq, evicted.commit_seq, cnt);
+ ReadLock rl(&snapshots_mutex_);
+ // Items could have moved from the snapshots_ to snapshot_cache_ before
+ // accquiring the lock. To make sure that we do not miss a valid snapshot,
+ // read snapshot_cache_ again while holding the lock.
+ for (size_t i = 0; i < SNAPSHOT_CACHE_SIZE; i++) {
+ SequenceNumber snapshot_seq =
+ snapshot_cache_[i].load(std::memory_order_acquire);
+ if (!MaybeUpdateOldCommitMap(evicted.prep_seq, evicted.commit_seq,
+ snapshot_seq, next_is_larger)) {
+ break;
+ }
+ }
+ for (auto snapshot_seq_2 : snapshots_) {
+ if (!MaybeUpdateOldCommitMap(evicted.prep_seq, evicted.commit_seq,
+ snapshot_seq_2, next_is_larger)) {
+ break;
+ }
+ }
+ }
+}
+
+bool WritePreparedTxnDB::MaybeUpdateOldCommitMap(
+ const uint64_t& prep_seq, const uint64_t& commit_seq,
+ const uint64_t& snapshot_seq, const bool next_is_larger = true) {
+ // If we do not store an entry in old_commit_map_ we assume it is committed in
+ // all snapshots. If commit_seq <= snapshot_seq, it is considered already in
+ // the snapshot so we need not to keep the entry around for this snapshot.
+ if (commit_seq <= snapshot_seq) {
+ // continue the search if the next snapshot could be smaller than commit_seq
+ return !next_is_larger;
+ }
+ // then snapshot_seq < commit_seq
+ if (prep_seq <= snapshot_seq) { // overlapping range
+ WPRecordTick(TXN_OLD_COMMIT_MAP_MUTEX_OVERHEAD);
+ ROCKS_LOG_WARN(info_log_,
+ "old_commit_map_mutex_ overhead for %" PRIu64
+ " commit entry: <%" PRIu64 ",%" PRIu64 ">",
+ snapshot_seq, prep_seq, commit_seq);
+ WriteLock wl(&old_commit_map_mutex_);
+ old_commit_map_empty_.store(false, std::memory_order_release);
+ auto& vec = old_commit_map_[snapshot_seq];
+ vec.insert(std::upper_bound(vec.begin(), vec.end(), prep_seq), prep_seq);
+ // We need to store it once for each overlapping snapshot. Returning true to
+ // continue the search if there is more overlapping snapshot.
+ return true;
+ }
+ // continue the search if the next snapshot could be larger than prep_seq
+ return next_is_larger;
+}
+
+WritePreparedTxnDB::~WritePreparedTxnDB() {
+ // At this point there could be running compaction/flush holding a
+ // SnapshotChecker, which holds a pointer back to WritePreparedTxnDB.
+ // Make sure those jobs finished before destructing WritePreparedTxnDB.
+ if (!db_impl_->shutting_down_) {
+ db_impl_->CancelAllBackgroundWork(true /*wait*/);
+ }
+}
+
+void SubBatchCounter::InitWithComp(const uint32_t cf) {
+ auto cmp = comparators_[cf];
+ keys_[cf] = CFKeys(SetComparator(cmp));
+}
+
+void SubBatchCounter::AddKey(const uint32_t cf, const Slice& key) {
+ CFKeys& cf_keys = keys_[cf];
+ if (cf_keys.size() == 0) { // just inserted
+ InitWithComp(cf);
+ }
+ auto it = cf_keys.insert(key);
+ if (it.second == false) { // second is false if a element already existed.
+ batches_++;
+ keys_.clear();
+ InitWithComp(cf);
+ keys_[cf].insert(key);
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_prepared_txn_db.h b/src/rocksdb/utilities/transactions/write_prepared_txn_db.h
new file mode 100644
index 000000000..964b72689
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_prepared_txn_db.h
@@ -0,0 +1,1111 @@
+// 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 <cinttypes>
+#include <mutex>
+#include <queue>
+#include <set>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "db/db_iter.h"
+#include "db/pre_release_callback.h"
+#include "db/read_callback.h"
+#include "db/snapshot_checker.h"
+#include "rocksdb/db.h"
+#include "rocksdb/options.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "util/cast_util.h"
+#include "util/set_comparator.h"
+#include "util/string_util.h"
+#include "utilities/transactions/pessimistic_transaction.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+#include "utilities/transactions/transaction_lock_mgr.h"
+#include "utilities/transactions/write_prepared_txn.h"
+
+namespace ROCKSDB_NAMESPACE {
+enum SnapshotBackup : bool { kUnbackedByDBSnapshot, kBackedByDBSnapshot };
+
+// A PessimisticTransactionDB that writes data to DB after prepare phase of 2PC.
+// In this way some data in the DB might not be committed. The DB provides
+// mechanisms to tell such data apart from committed data.
+class WritePreparedTxnDB : public PessimisticTransactionDB {
+ public:
+ explicit WritePreparedTxnDB(DB* db,
+ const TransactionDBOptions& txn_db_options)
+ : PessimisticTransactionDB(db, txn_db_options),
+ SNAPSHOT_CACHE_BITS(txn_db_options.wp_snapshot_cache_bits),
+ SNAPSHOT_CACHE_SIZE(static_cast<size_t>(1ull << SNAPSHOT_CACHE_BITS)),
+ COMMIT_CACHE_BITS(txn_db_options.wp_commit_cache_bits),
+ COMMIT_CACHE_SIZE(static_cast<size_t>(1ull << COMMIT_CACHE_BITS)),
+ FORMAT(COMMIT_CACHE_BITS) {
+ Init(txn_db_options);
+ }
+
+ explicit WritePreparedTxnDB(StackableDB* db,
+ const TransactionDBOptions& txn_db_options)
+ : PessimisticTransactionDB(db, txn_db_options),
+ SNAPSHOT_CACHE_BITS(txn_db_options.wp_snapshot_cache_bits),
+ SNAPSHOT_CACHE_SIZE(static_cast<size_t>(1ull << SNAPSHOT_CACHE_BITS)),
+ COMMIT_CACHE_BITS(txn_db_options.wp_commit_cache_bits),
+ COMMIT_CACHE_SIZE(static_cast<size_t>(1ull << COMMIT_CACHE_BITS)),
+ FORMAT(COMMIT_CACHE_BITS) {
+ Init(txn_db_options);
+ }
+
+ virtual ~WritePreparedTxnDB();
+
+ virtual Status Initialize(
+ const std::vector<size_t>& compaction_enabled_cf_indices,
+ const std::vector<ColumnFamilyHandle*>& handles) override;
+
+ Transaction* BeginTransaction(const WriteOptions& write_options,
+ const TransactionOptions& txn_options,
+ Transaction* old_txn) override;
+
+ using TransactionDB::Write;
+ Status Write(const WriteOptions& opts, WriteBatch* updates) override;
+
+ // Optimized version of ::Write that receives more optimization request such
+ // as skip_concurrency_control.
+ using PessimisticTransactionDB::Write;
+ Status Write(const WriteOptions& opts, const TransactionDBWriteOptimizations&,
+ WriteBatch* updates) override;
+
+ // Write the batch to the underlying DB and mark it as committed. Could be
+ // used by both directly from TxnDB or through a transaction.
+ Status WriteInternal(const WriteOptions& write_options, WriteBatch* batch,
+ size_t batch_cnt, WritePreparedTxn* txn);
+
+ using DB::Get;
+ virtual Status Get(const ReadOptions& options,
+ ColumnFamilyHandle* column_family, const Slice& key,
+ PinnableSlice* value) override;
+
+ using DB::MultiGet;
+ virtual std::vector<Status> MultiGet(
+ const ReadOptions& options,
+ const std::vector<ColumnFamilyHandle*>& column_family,
+ const std::vector<Slice>& keys,
+ std::vector<std::string>* values) override;
+
+ using DB::NewIterator;
+ virtual Iterator* NewIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family) override;
+
+ using DB::NewIterators;
+ virtual Status NewIterators(
+ const ReadOptions& options,
+ const std::vector<ColumnFamilyHandle*>& column_families,
+ std::vector<Iterator*>* iterators) override;
+
+ // Check whether the transaction that wrote the value with sequence number seq
+ // is visible to the snapshot with sequence number snapshot_seq.
+ // Returns true if commit_seq <= snapshot_seq
+ // If the snapshot_seq is already released and snapshot_seq <= max, sets
+ // *snap_released to true and returns true as well.
+ inline bool IsInSnapshot(uint64_t prep_seq, uint64_t snapshot_seq,
+ uint64_t min_uncommitted = kMinUnCommittedSeq,
+ bool* snap_released = nullptr) const {
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " min_uncommitted %" PRIu64,
+ prep_seq, snapshot_seq, min_uncommitted);
+ assert(min_uncommitted >= kMinUnCommittedSeq);
+ // Caller is responsible to initialize snap_released.
+ assert(snap_released == nullptr || *snap_released == false);
+ // Here we try to infer the return value without looking into prepare list.
+ // This would help avoiding synchronization over a shared map.
+ // TODO(myabandeh): optimize this. This sequence of checks must be correct
+ // but not necessary efficient
+ if (prep_seq == 0) {
+ // Compaction will output keys to bottom-level with sequence number 0 if
+ // it is visible to the earliest snapshot.
+ ROCKS_LOG_DETAILS(
+ info_log_, "IsInSnapshot %" PRIu64 " in %" PRIu64 " returns %" PRId32,
+ prep_seq, snapshot_seq, 1);
+ return true;
+ }
+ if (snapshot_seq < prep_seq) {
+ // snapshot_seq < prep_seq <= commit_seq => snapshot_seq < commit_seq
+ ROCKS_LOG_DETAILS(
+ info_log_, "IsInSnapshot %" PRIu64 " in %" PRIu64 " returns %" PRId32,
+ prep_seq, snapshot_seq, 0);
+ return false;
+ }
+ if (prep_seq < min_uncommitted) {
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " returns %" PRId32
+ " because of min_uncommitted %" PRIu64,
+ prep_seq, snapshot_seq, 1, min_uncommitted);
+ return true;
+ }
+ // Commit of delayed prepared has two non-atomic steps: add to commit cache,
+ // remove from delayed prepared. Our reads from these two is also
+ // non-atomic. By looking into commit cache first thus we might not find the
+ // prep_seq neither in commit cache not in delayed_prepared_. To fix that i)
+ // we check if there was any delayed prepared BEFORE looking into commit
+ // cache, ii) if there was, we complete the search steps to be these: i)
+ // commit cache, ii) delayed prepared, commit cache again. In this way if
+ // the first query to commit cache missed the commit, the 2nd will catch it.
+ bool was_empty;
+ SequenceNumber max_evicted_seq_lb, max_evicted_seq_ub;
+ CommitEntry64b dont_care;
+ auto indexed_seq = prep_seq % COMMIT_CACHE_SIZE;
+ size_t repeats = 0;
+ do {
+ repeats++;
+ assert(repeats < 100);
+ if (UNLIKELY(repeats >= 100)) {
+ throw std::runtime_error(
+ "The read was intrupted 100 times by update to max_evicted_seq_. "
+ "This is unexpected in all setups");
+ }
+ max_evicted_seq_lb = max_evicted_seq_.load(std::memory_order_acquire);
+ TEST_SYNC_POINT(
+ "WritePreparedTxnDB::IsInSnapshot:max_evicted_seq_:pause");
+ TEST_SYNC_POINT(
+ "WritePreparedTxnDB::IsInSnapshot:max_evicted_seq_:resume");
+ was_empty = delayed_prepared_empty_.load(std::memory_order_acquire);
+ TEST_SYNC_POINT(
+ "WritePreparedTxnDB::IsInSnapshot:delayed_prepared_empty_:pause");
+ TEST_SYNC_POINT(
+ "WritePreparedTxnDB::IsInSnapshot:delayed_prepared_empty_:resume");
+ CommitEntry cached;
+ bool exist = GetCommitEntry(indexed_seq, &dont_care, &cached);
+ TEST_SYNC_POINT("WritePreparedTxnDB::IsInSnapshot:GetCommitEntry:pause");
+ TEST_SYNC_POINT("WritePreparedTxnDB::IsInSnapshot:GetCommitEntry:resume");
+ if (exist && prep_seq == cached.prep_seq) {
+ // It is committed and also not evicted from commit cache
+ ROCKS_LOG_DETAILS(
+ info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64 " returns %" PRId32,
+ prep_seq, snapshot_seq, cached.commit_seq <= snapshot_seq);
+ return cached.commit_seq <= snapshot_seq;
+ }
+ // else it could be committed but not inserted in the map which could
+ // happen after recovery, or it could be committed and evicted by another
+ // commit, or never committed.
+
+ // At this point we dont know if it was committed or it is still prepared
+ max_evicted_seq_ub = max_evicted_seq_.load(std::memory_order_acquire);
+ if (UNLIKELY(max_evicted_seq_lb != max_evicted_seq_ub)) {
+ continue;
+ }
+ // Note: max_evicted_seq_ when we did GetCommitEntry <= max_evicted_seq_ub
+ if (max_evicted_seq_ub < prep_seq) {
+ // Not evicted from cache and also not present, so must be still
+ // prepared
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " returns %" PRId32,
+ prep_seq, snapshot_seq, 0);
+ return false;
+ }
+ TEST_SYNC_POINT("WritePreparedTxnDB::IsInSnapshot:prepared_mutex_:pause");
+ TEST_SYNC_POINT(
+ "WritePreparedTxnDB::IsInSnapshot:prepared_mutex_:resume");
+ if (!was_empty) {
+ // We should not normally reach here
+ WPRecordTick(TXN_PREPARE_MUTEX_OVERHEAD);
+ ReadLock rl(&prepared_mutex_);
+ ROCKS_LOG_WARN(
+ info_log_, "prepared_mutex_ overhead %" PRIu64 " for %" PRIu64,
+ static_cast<uint64_t>(delayed_prepared_.size()), prep_seq);
+ if (delayed_prepared_.find(prep_seq) != delayed_prepared_.end()) {
+ // This is the order: 1) delayed_prepared_commits_ update, 2) publish
+ // 3) delayed_prepared_ clean up. So check if it is the case of a late
+ // clenaup.
+ auto it = delayed_prepared_commits_.find(prep_seq);
+ if (it == delayed_prepared_commits_.end()) {
+ // Then it is not committed yet
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " returns %" PRId32,
+ prep_seq, snapshot_seq, 0);
+ return false;
+ } else {
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " commit: %" PRIu64 " returns %" PRId32,
+ prep_seq, snapshot_seq, it->second,
+ snapshot_seq <= it->second);
+ return it->second <= snapshot_seq;
+ }
+ } else {
+ // 2nd query to commit cache. Refer to was_empty comment above.
+ exist = GetCommitEntry(indexed_seq, &dont_care, &cached);
+ if (exist && prep_seq == cached.prep_seq) {
+ ROCKS_LOG_DETAILS(
+ info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64 " returns %" PRId32,
+ prep_seq, snapshot_seq, cached.commit_seq <= snapshot_seq);
+ return cached.commit_seq <= snapshot_seq;
+ }
+ max_evicted_seq_ub = max_evicted_seq_.load(std::memory_order_acquire);
+ }
+ }
+ } while (UNLIKELY(max_evicted_seq_lb != max_evicted_seq_ub));
+ // When advancing max_evicted_seq_, we move older entires from prepared to
+ // delayed_prepared_. Also we move evicted entries from commit cache to
+ // old_commit_map_ if it overlaps with any snapshot. Since prep_seq <=
+ // max_evicted_seq_, we have three cases: i) in delayed_prepared_, ii) in
+ // old_commit_map_, iii) committed with no conflict with any snapshot. Case
+ // (i) delayed_prepared_ is checked above
+ if (max_evicted_seq_ub < snapshot_seq) { // then (ii) cannot be the case
+ // only (iii) is the case: committed
+ // commit_seq <= max_evicted_seq_ < snapshot_seq => commit_seq <
+ // snapshot_seq
+ ROCKS_LOG_DETAILS(
+ info_log_, "IsInSnapshot %" PRIu64 " in %" PRIu64 " returns %" PRId32,
+ prep_seq, snapshot_seq, 1);
+ return true;
+ }
+ // else (ii) might be the case: check the commit data saved for this
+ // snapshot. If there was no overlapping commit entry, then it is committed
+ // with a commit_seq lower than any live snapshot, including snapshot_seq.
+ if (old_commit_map_empty_.load(std::memory_order_acquire)) {
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " returns %" PRId32 " released=1",
+ prep_seq, snapshot_seq, 0);
+ assert(snap_released);
+ // This snapshot is not valid anymore. We cannot tell if prep_seq is
+ // committed before or after the snapshot. Return true but also set
+ // snap_released to true.
+ *snap_released = true;
+ return true;
+ }
+ {
+ // We should not normally reach here unless sapshot_seq is old. This is a
+ // rare case and it is ok to pay the cost of mutex ReadLock for such old,
+ // reading transactions.
+ WPRecordTick(TXN_OLD_COMMIT_MAP_MUTEX_OVERHEAD);
+ ReadLock rl(&old_commit_map_mutex_);
+ auto prep_set_entry = old_commit_map_.find(snapshot_seq);
+ bool found = prep_set_entry != old_commit_map_.end();
+ if (found) {
+ auto& vec = prep_set_entry->second;
+ found = std::binary_search(vec.begin(), vec.end(), prep_seq);
+ } else {
+ // coming from compaction
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " returns %" PRId32 " released=1",
+ prep_seq, snapshot_seq, 0);
+ // This snapshot is not valid anymore. We cannot tell if prep_seq is
+ // committed before or after the snapshot. Return true but also set
+ // snap_released to true.
+ assert(snap_released);
+ *snap_released = true;
+ return true;
+ }
+
+ if (!found) {
+ ROCKS_LOG_DETAILS(info_log_,
+ "IsInSnapshot %" PRIu64 " in %" PRIu64
+ " returns %" PRId32,
+ prep_seq, snapshot_seq, 1);
+ return true;
+ }
+ }
+ // (ii) it the case: it is committed but after the snapshot_seq
+ ROCKS_LOG_DETAILS(
+ info_log_, "IsInSnapshot %" PRIu64 " in %" PRIu64 " returns %" PRId32,
+ prep_seq, snapshot_seq, 0);
+ return false;
+ }
+
+ // Add the transaction with prepare sequence seq to the prepared list.
+ // Note: must be called serially with increasing seq on each call.
+ // locked is true if prepared_mutex_ is already locked.
+ void AddPrepared(uint64_t seq, bool locked = false);
+ // Check if any of the prepared txns are less than new max_evicted_seq_. Must
+ // be called with prepared_mutex_ write locked.
+ void CheckPreparedAgainstMax(SequenceNumber new_max, bool locked);
+ // Remove the transaction with prepare sequence seq from the prepared list
+ void RemovePrepared(const uint64_t seq, const size_t batch_cnt = 1);
+ // Add the transaction with prepare sequence prepare_seq and commit sequence
+ // commit_seq to the commit map. loop_cnt is to detect infinite loops.
+ // Note: must be called serially.
+ void AddCommitted(uint64_t prepare_seq, uint64_t commit_seq,
+ uint8_t loop_cnt = 0);
+
+ struct CommitEntry {
+ uint64_t prep_seq;
+ uint64_t commit_seq;
+ CommitEntry() : prep_seq(0), commit_seq(0) {}
+ CommitEntry(uint64_t ps, uint64_t cs) : prep_seq(ps), commit_seq(cs) {}
+ bool operator==(const CommitEntry& rhs) const {
+ return prep_seq == rhs.prep_seq && commit_seq == rhs.commit_seq;
+ }
+ };
+
+ struct CommitEntry64bFormat {
+ explicit CommitEntry64bFormat(size_t index_bits)
+ : INDEX_BITS(index_bits),
+ PREP_BITS(static_cast<size_t>(64 - PAD_BITS - INDEX_BITS)),
+ COMMIT_BITS(static_cast<size_t>(64 - PREP_BITS)),
+ COMMIT_FILTER(static_cast<uint64_t>((1ull << COMMIT_BITS) - 1)),
+ DELTA_UPPERBOUND(static_cast<uint64_t>((1ull << COMMIT_BITS))) {}
+ // Number of higher bits of a sequence number that is not used. They are
+ // used to encode the value type, ...
+ const size_t PAD_BITS = static_cast<size_t>(8);
+ // Number of lower bits from prepare seq that can be skipped as they are
+ // implied by the index of the entry in the array
+ const size_t INDEX_BITS;
+ // Number of bits we use to encode the prepare seq
+ const size_t PREP_BITS;
+ // Number of bits we use to encode the commit seq.
+ const size_t COMMIT_BITS;
+ // Filter to encode/decode commit seq
+ const uint64_t COMMIT_FILTER;
+ // The value of commit_seq - prepare_seq + 1 must be less than this bound
+ const uint64_t DELTA_UPPERBOUND;
+ };
+
+ // Prepare Seq (64 bits) = PAD ... PAD PREP PREP ... PREP INDEX INDEX ...
+ // INDEX Delta Seq (64 bits) = 0 0 0 0 0 0 0 0 0 0 0 0 DELTA DELTA ...
+ // DELTA DELTA Encoded Value = PREP PREP .... PREP PREP DELTA DELTA
+ // ... DELTA DELTA PAD: first bits of a seq that is reserved for tagging and
+ // hence ignored PREP/INDEX: the used bits in a prepare seq number INDEX: the
+ // bits that do not have to be encoded (will be provided externally) DELTA:
+ // prep seq - commit seq + 1 Number of DELTA bits should be equal to number of
+ // index bits + PADs
+ struct CommitEntry64b {
+ constexpr CommitEntry64b() noexcept : rep_(0) {}
+
+ CommitEntry64b(const CommitEntry& entry, const CommitEntry64bFormat& format)
+ : CommitEntry64b(entry.prep_seq, entry.commit_seq, format) {}
+
+ CommitEntry64b(const uint64_t ps, const uint64_t cs,
+ const CommitEntry64bFormat& format) {
+ assert(ps < static_cast<uint64_t>(
+ (1ull << (format.PREP_BITS + format.INDEX_BITS))));
+ assert(ps <= cs);
+ uint64_t delta = cs - ps + 1; // make initialized delta always >= 1
+ // zero is reserved for uninitialized entries
+ assert(0 < delta);
+ assert(delta < format.DELTA_UPPERBOUND);
+ if (delta >= format.DELTA_UPPERBOUND) {
+ throw std::runtime_error(
+ "commit_seq >> prepare_seq. The allowed distance is " +
+ ToString(format.DELTA_UPPERBOUND) + " commit_seq is " +
+ ToString(cs) + " prepare_seq is " + ToString(ps));
+ }
+ rep_ = (ps << format.PAD_BITS) & ~format.COMMIT_FILTER;
+ rep_ = rep_ | delta;
+ }
+
+ // Return false if the entry is empty
+ bool Parse(const uint64_t indexed_seq, CommitEntry* entry,
+ const CommitEntry64bFormat& format) {
+ uint64_t delta = rep_ & format.COMMIT_FILTER;
+ // zero is reserved for uninitialized entries
+ assert(delta < static_cast<uint64_t>((1ull << format.COMMIT_BITS)));
+ if (delta == 0) {
+ return false; // initialized entry would have non-zero delta
+ }
+
+ assert(indexed_seq < static_cast<uint64_t>((1ull << format.INDEX_BITS)));
+ uint64_t prep_up = rep_ & ~format.COMMIT_FILTER;
+ prep_up >>= format.PAD_BITS;
+ const uint64_t& prep_low = indexed_seq;
+ entry->prep_seq = prep_up | prep_low;
+
+ entry->commit_seq = entry->prep_seq + delta - 1;
+ return true;
+ }
+
+ private:
+ uint64_t rep_;
+ };
+
+ // Struct to hold ownership of snapshot and read callback for cleanup.
+ struct IteratorState;
+
+ std::shared_ptr<std::map<uint32_t, const Comparator*>> GetCFComparatorMap() {
+ return cf_map_;
+ }
+ std::shared_ptr<std::map<uint32_t, ColumnFamilyHandle*>> GetCFHandleMap() {
+ return handle_map_;
+ }
+ void UpdateCFComparatorMap(
+ const std::vector<ColumnFamilyHandle*>& handles) override;
+ void UpdateCFComparatorMap(ColumnFamilyHandle* handle) override;
+
+ virtual const Snapshot* GetSnapshot() override;
+ SnapshotImpl* GetSnapshotInternal(bool for_ww_conflict_check);
+
+ protected:
+ virtual Status VerifyCFOptions(
+ const ColumnFamilyOptions& cf_options) override;
+ // Assign the min and max sequence numbers for reading from the db. A seq >
+ // max is not valid, and a seq < min is valid, and a min <= seq < max requires
+ // further checking. Normally max is defined by the snapshot and min is by
+ // minimum uncommitted seq.
+ inline SnapshotBackup AssignMinMaxSeqs(const Snapshot* snapshot,
+ SequenceNumber* min,
+ SequenceNumber* max);
+ // Validate is a snapshot sequence number is still valid based on the latest
+ // db status. backed_by_snapshot specifies if the number is baked by an actual
+ // snapshot object. order specified the memory order with which we load the
+ // atomic variables: relax is enough for the default since we care about last
+ // value seen by same thread.
+ inline bool ValidateSnapshot(
+ const SequenceNumber snap_seq, const SnapshotBackup backed_by_snapshot,
+ std::memory_order order = std::memory_order_relaxed);
+ // Get a dummy snapshot that refers to kMaxSequenceNumber
+ Snapshot* GetMaxSnapshot() { return &dummy_max_snapshot_; }
+
+ private:
+ friend class AddPreparedCallback;
+ friend class PreparedHeap_BasicsTest_Test;
+ friend class PreparedHeap_Concurrent_Test;
+ friend class PreparedHeap_EmptyAtTheEnd_Test;
+ friend class SnapshotConcurrentAccessTest_SnapshotConcurrentAccess_Test;
+ friend class WritePreparedCommitEntryPreReleaseCallback;
+ friend class WritePreparedTransactionTestBase;
+ friend class WritePreparedTxn;
+ friend class WritePreparedTxnDBMock;
+ friend class WritePreparedTransactionTest_AddPreparedBeforeMax_Test;
+ friend class WritePreparedTransactionTest_AdvanceMaxEvictedSeqBasic_Test;
+ friend class
+ WritePreparedTransactionTest_AdvanceMaxEvictedSeqWithDuplicates_Test;
+ friend class WritePreparedTransactionTest_AdvanceSeqByOne_Test;
+ friend class WritePreparedTransactionTest_BasicRecovery_Test;
+ friend class WritePreparedTransactionTest_CheckAgainstSnapshots_Test;
+ friend class WritePreparedTransactionTest_CleanupSnapshotEqualToMax_Test;
+ friend class WritePreparedTransactionTest_ConflictDetectionAfterRecovery_Test;
+ friend class WritePreparedTransactionTest_CommitMap_Test;
+ friend class WritePreparedTransactionTest_DoubleSnapshot_Test;
+ friend class WritePreparedTransactionTest_IsInSnapshotEmptyMap_Test;
+ friend class WritePreparedTransactionTest_IsInSnapshotReleased_Test;
+ friend class WritePreparedTransactionTest_IsInSnapshot_Test;
+ friend class WritePreparedTransactionTest_NewSnapshotLargerThanMax_Test;
+ friend class WritePreparedTransactionTest_MaxCatchupWithNewSnapshot_Test;
+ friend class WritePreparedTransactionTest_MaxCatchupWithUnbackedSnapshot_Test;
+ friend class
+ WritePreparedTransactionTest_NonAtomicCommitOfDelayedPrepared_Test;
+ friend class
+ WritePreparedTransactionTest_NonAtomicUpdateOfDelayedPrepared_Test;
+ friend class WritePreparedTransactionTest_NonAtomicUpdateOfMaxEvictedSeq_Test;
+ friend class WritePreparedTransactionTest_OldCommitMapGC_Test;
+ friend class WritePreparedTransactionTest_Rollback_Test;
+ friend class WritePreparedTransactionTest_SmallestUnCommittedSeq_Test;
+ friend class WriteUnpreparedTxn;
+ friend class WriteUnpreparedTxnDB;
+ friend class WriteUnpreparedTransactionTest_RecoveryTest_Test;
+
+ void Init(const TransactionDBOptions& /* unused */);
+
+ void WPRecordTick(uint32_t ticker_type) const {
+ RecordTick(db_impl_->immutable_db_options_.statistics.get(), ticker_type);
+ }
+
+ // A heap with the amortized O(1) complexity for erase. It uses one extra heap
+ // to keep track of erased entries that are not yet on top of the main heap.
+ class PreparedHeap {
+ // The mutex is required for push and pop from PreparedHeap. ::erase will
+ // use external synchronization via prepared_mutex_.
+ port::Mutex push_pop_mutex_;
+ std::deque<uint64_t> heap_;
+ std::priority_queue<uint64_t, std::vector<uint64_t>, std::greater<uint64_t>>
+ erased_heap_;
+ std::atomic<uint64_t> heap_top_ = {kMaxSequenceNumber};
+ // True when testing crash recovery
+ bool TEST_CRASH_ = false;
+ friend class WritePreparedTxnDB;
+
+ public:
+ ~PreparedHeap() {
+ if (!TEST_CRASH_) {
+ assert(heap_.empty());
+ assert(erased_heap_.empty());
+ }
+ }
+ port::Mutex* push_pop_mutex() { return &push_pop_mutex_; }
+
+ inline bool empty() { return top() == kMaxSequenceNumber; }
+ // Returns kMaxSequenceNumber if empty() and the smallest otherwise.
+ inline uint64_t top() { return heap_top_.load(std::memory_order_acquire); }
+ inline void push(uint64_t v) {
+ push_pop_mutex_.AssertHeld();
+ if (heap_.empty()) {
+ heap_top_.store(v, std::memory_order_release);
+ } else {
+ assert(heap_top_.load() < v);
+ }
+ heap_.push_back(v);
+ }
+ void pop(bool locked = false) {
+ if (!locked) {
+ push_pop_mutex()->Lock();
+ }
+ push_pop_mutex_.AssertHeld();
+ heap_.pop_front();
+ while (!heap_.empty() && !erased_heap_.empty() &&
+ // heap_.top() > erased_heap_.top() could happen if we have erased
+ // a non-existent entry. Ideally the user should not do that but we
+ // should be resilient against it.
+ heap_.front() >= erased_heap_.top()) {
+ if (heap_.front() == erased_heap_.top()) {
+ heap_.pop_front();
+ }
+ uint64_t erased __attribute__((__unused__));
+ erased = erased_heap_.top();
+ erased_heap_.pop();
+ // No duplicate prepare sequence numbers
+ assert(erased_heap_.empty() || erased_heap_.top() != erased);
+ }
+ while (heap_.empty() && !erased_heap_.empty()) {
+ erased_heap_.pop();
+ }
+ heap_top_.store(!heap_.empty() ? heap_.front() : kMaxSequenceNumber,
+ std::memory_order_release);
+ if (!locked) {
+ push_pop_mutex()->Unlock();
+ }
+ }
+ // Concurrrent calls needs external synchronization. It is safe to be called
+ // concurrent to push and pop though.
+ void erase(uint64_t seq) {
+ if (!empty()) {
+ auto top_seq = top();
+ if (seq < top_seq) {
+ // Already popped, ignore it.
+ } else if (top_seq == seq) {
+ pop();
+#ifndef NDEBUG
+ MutexLock ml(push_pop_mutex());
+ assert(heap_.empty() || heap_.front() != seq);
+#endif
+ } else { // top() > seq
+ // Down the heap, remember to pop it later
+ erased_heap_.push(seq);
+ }
+ }
+ }
+ };
+
+ void TEST_Crash() override { prepared_txns_.TEST_CRASH_ = true; }
+
+ // Get the commit entry with index indexed_seq from the commit table. It
+ // returns true if such entry exists.
+ bool GetCommitEntry(const uint64_t indexed_seq, CommitEntry64b* entry_64b,
+ CommitEntry* entry) const;
+
+ // Rewrite the entry with the index indexed_seq in the commit table with the
+ // commit entry <prep_seq, commit_seq>. If the rewrite results into eviction,
+ // sets the evicted_entry and returns true.
+ bool AddCommitEntry(const uint64_t indexed_seq, const CommitEntry& new_entry,
+ CommitEntry* evicted_entry);
+
+ // Rewrite the entry with the index indexed_seq in the commit table with the
+ // commit entry new_entry only if the existing entry matches the
+ // expected_entry. Returns false otherwise.
+ bool ExchangeCommitEntry(const uint64_t indexed_seq,
+ CommitEntry64b& expected_entry,
+ const CommitEntry& new_entry);
+
+ // Increase max_evicted_seq_ from the previous value prev_max to the new
+ // value. This also involves taking care of prepared txns that are not
+ // committed before new_max, as well as updating the list of live snapshots at
+ // the time of updating the max. Thread-safety: this function can be called
+ // concurrently. The concurrent invocations of this function is equivalent to
+ // a serial invocation in which the last invocation is the one with the
+ // largest new_max value.
+ void AdvanceMaxEvictedSeq(const SequenceNumber& prev_max,
+ const SequenceNumber& new_max);
+
+ inline SequenceNumber SmallestUnCommittedSeq() {
+ // Note: We have two lists to look into, but for performance reasons they
+ // are not read atomically. Since CheckPreparedAgainstMax copies the entry
+ // to delayed_prepared_ before removing it from prepared_txns_, to ensure
+ // that a prepared entry will not go unmissed, we look into them in opposite
+ // order: first read prepared_txns_ and then delayed_prepared_.
+
+ // This must be called before calling ::top. This is because the concurrent
+ // thread would call ::RemovePrepared before updating
+ // GetLatestSequenceNumber(). Reading then in opposite order here guarantees
+ // that the ::top that we read would be lower the ::top if we had otherwise
+ // update/read them atomically.
+ auto next_prepare = db_impl_->GetLatestSequenceNumber() + 1;
+ auto min_prepare = prepared_txns_.top();
+ // Since we update the prepare_heap always from the main write queue via
+ // PreReleaseCallback, the prepared_txns_.top() indicates the smallest
+ // prepared data in 2pc transactions. For non-2pc transactions that are
+ // written in two steps, we also update prepared_txns_ at the first step
+ // (via the same mechanism) so that their uncommitted data is reflected in
+ // SmallestUnCommittedSeq.
+ if (!delayed_prepared_empty_.load()) {
+ ReadLock rl(&prepared_mutex_);
+ if (!delayed_prepared_.empty()) {
+ return *delayed_prepared_.begin();
+ }
+ }
+ bool empty = min_prepare == kMaxSequenceNumber;
+ if (empty) {
+ // Since GetLatestSequenceNumber is updated
+ // after prepared_txns_ are, the value of GetLatestSequenceNumber would
+ // reflect any uncommitted data that is not added to prepared_txns_ yet.
+ // Otherwise, if there is no concurrent txn, this value simply reflects
+ // that latest value in the memtable.
+ return next_prepare;
+ } else {
+ return std::min(min_prepare, next_prepare);
+ }
+ }
+
+ // Enhance the snapshot object by recording in it the smallest uncommitted seq
+ inline void EnhanceSnapshot(SnapshotImpl* snapshot,
+ SequenceNumber min_uncommitted) {
+ assert(snapshot);
+ assert(min_uncommitted <= snapshot->number_ + 1);
+ snapshot->min_uncommitted_ = min_uncommitted;
+ }
+
+ virtual const std::vector<SequenceNumber> GetSnapshotListFromDB(
+ SequenceNumber max);
+
+ // Will be called by the public ReleaseSnapshot method. Does the maintenance
+ // internal to WritePreparedTxnDB
+ void ReleaseSnapshotInternal(const SequenceNumber snap_seq);
+
+ // Update the list of snapshots corresponding to the soon-to-be-updated
+ // max_evicted_seq_. Thread-safety: this function can be called concurrently.
+ // The concurrent invocations of this function is equivalent to a serial
+ // invocation in which the last invocation is the one with the largest
+ // version value.
+ void UpdateSnapshots(const std::vector<SequenceNumber>& snapshots,
+ const SequenceNumber& version);
+ // Check the new list of new snapshots against the old one to see if any of
+ // the snapshots are released and to do the cleanup for the released snapshot.
+ void CleanupReleasedSnapshots(
+ const std::vector<SequenceNumber>& new_snapshots,
+ const std::vector<SequenceNumber>& old_snapshots);
+
+ // Check an evicted entry against live snapshots to see if it should be kept
+ // around or it can be safely discarded (and hence assume committed for all
+ // snapshots). Thread-safety: this function can be called concurrently. If it
+ // is called concurrently with multiple UpdateSnapshots, the result is the
+ // same as checking the intersection of the snapshot list before updates with
+ // the snapshot list of all the concurrent updates.
+ void CheckAgainstSnapshots(const CommitEntry& evicted);
+
+ // Add a new entry to old_commit_map_ if prep_seq <= snapshot_seq <
+ // commit_seq. Return false if checking the next snapshot(s) is not needed.
+ // This is the case if none of the next snapshots could satisfy the condition.
+ // next_is_larger: the next snapshot will be a larger value
+ bool MaybeUpdateOldCommitMap(const uint64_t& prep_seq,
+ const uint64_t& commit_seq,
+ const uint64_t& snapshot_seq,
+ const bool next_is_larger);
+
+ // A trick to increase the last visible sequence number by one and also wait
+ // for the in-flight commits to be visible.
+ void AdvanceSeqByOne();
+
+ // The list of live snapshots at the last time that max_evicted_seq_ advanced.
+ // The list stored into two data structures: in snapshot_cache_ that is
+ // efficient for concurrent reads, and in snapshots_ if the data does not fit
+ // into snapshot_cache_. The total number of snapshots in the two lists
+ std::atomic<size_t> snapshots_total_ = {};
+ // The list sorted in ascending order. Thread-safety for writes is provided
+ // with snapshots_mutex_ and concurrent reads are safe due to std::atomic for
+ // each entry. In x86_64 architecture such reads are compiled to simple read
+ // instructions.
+ const size_t SNAPSHOT_CACHE_BITS;
+ const size_t SNAPSHOT_CACHE_SIZE;
+ std::unique_ptr<std::atomic<SequenceNumber>[]> snapshot_cache_;
+ // 2nd list for storing snapshots. The list sorted in ascending order.
+ // Thread-safety is provided with snapshots_mutex_.
+ std::vector<SequenceNumber> snapshots_;
+ // The list of all snapshots: snapshots_ + snapshot_cache_. This list although
+ // redundant but simplifies CleanupOldSnapshots implementation.
+ // Thread-safety is provided with snapshots_mutex_.
+ std::vector<SequenceNumber> snapshots_all_;
+ // The version of the latest list of snapshots. This can be used to avoid
+ // rewriting a list that is concurrently updated with a more recent version.
+ SequenceNumber snapshots_version_ = 0;
+
+ // A heap of prepared transactions. Thread-safety is provided with
+ // prepared_mutex_.
+ PreparedHeap prepared_txns_;
+ const size_t COMMIT_CACHE_BITS;
+ const size_t COMMIT_CACHE_SIZE;
+ const CommitEntry64bFormat FORMAT;
+ // commit_cache_ must be initialized to zero to tell apart an empty index from
+ // a filled one. Thread-safety is provided with commit_cache_mutex_.
+ std::unique_ptr<std::atomic<CommitEntry64b>[]> commit_cache_;
+ // The largest evicted *commit* sequence number from the commit_cache_. If a
+ // seq is smaller than max_evicted_seq_ is might or might not be present in
+ // commit_cache_. So commit_cache_ must first be checked before consulting
+ // with max_evicted_seq_.
+ std::atomic<uint64_t> max_evicted_seq_ = {};
+ // Order: 1) update future_max_evicted_seq_ = new_max, 2)
+ // GetSnapshotListFromDB(new_max), max_evicted_seq_ = new_max. Since
+ // GetSnapshotInternal guarantess that the snapshot seq is larger than
+ // future_max_evicted_seq_, this guarantes that if a snapshot is not larger
+ // than max has already being looked at via a GetSnapshotListFromDB(new_max).
+ std::atomic<uint64_t> future_max_evicted_seq_ = {};
+ // Advance max_evicted_seq_ by this value each time it needs an update. The
+ // larger the value, the less frequent advances we would have. We do not want
+ // it to be too large either as it would cause stalls by doing too much
+ // maintenance work under the lock.
+ size_t INC_STEP_FOR_MAX_EVICTED = 1;
+ // A map from old snapshots (expected to be used by a few read-only txns) to
+ // prepared sequence number of the evicted entries from commit_cache_ that
+ // overlaps with such snapshot. These are the prepared sequence numbers that
+ // the snapshot, to which they are mapped, cannot assume to be committed just
+ // because it is no longer in the commit_cache_. The vector must be sorted
+ // after each update.
+ // Thread-safety is provided with old_commit_map_mutex_.
+ std::map<SequenceNumber, std::vector<SequenceNumber>> old_commit_map_;
+ // A set of long-running prepared transactions that are not finished by the
+ // time max_evicted_seq_ advances their sequence number. This is expected to
+ // be empty normally. Thread-safety is provided with prepared_mutex_.
+ std::set<uint64_t> delayed_prepared_;
+ // Commit of a delayed prepared: 1) update commit cache, 2) update
+ // delayed_prepared_commits_, 3) publish seq, 3) clean up delayed_prepared_.
+ // delayed_prepared_commits_ will help us tell apart the unprepared txns from
+ // the ones that are committed but not cleaned up yet.
+ std::unordered_map<SequenceNumber, SequenceNumber> delayed_prepared_commits_;
+ // Update when delayed_prepared_.empty() changes. Expected to be true
+ // normally.
+ std::atomic<bool> delayed_prepared_empty_ = {true};
+ // Update when old_commit_map_.empty() changes. Expected to be true normally.
+ std::atomic<bool> old_commit_map_empty_ = {true};
+ mutable port::RWMutex prepared_mutex_;
+ mutable port::RWMutex old_commit_map_mutex_;
+ mutable port::RWMutex commit_cache_mutex_;
+ mutable port::RWMutex snapshots_mutex_;
+ // A cache of the cf comparators
+ // Thread safety: since it is a const it is safe to read it concurrently
+ std::shared_ptr<std::map<uint32_t, const Comparator*>> cf_map_;
+ // A cache of the cf handles
+ // Thread safety: since the handle is read-only object it is a const it is
+ // safe to read it concurrently
+ std::shared_ptr<std::map<uint32_t, ColumnFamilyHandle*>> handle_map_;
+ // A dummy snapshot object that refers to kMaxSequenceNumber
+ SnapshotImpl dummy_max_snapshot_;
+};
+
+class WritePreparedTxnReadCallback : public ReadCallback {
+ public:
+ WritePreparedTxnReadCallback(WritePreparedTxnDB* db, SequenceNumber snapshot)
+ : ReadCallback(snapshot),
+ db_(db),
+ backed_by_snapshot_(kBackedByDBSnapshot) {}
+ WritePreparedTxnReadCallback(WritePreparedTxnDB* db, SequenceNumber snapshot,
+ SequenceNumber min_uncommitted,
+ SnapshotBackup backed_by_snapshot)
+ : ReadCallback(snapshot, min_uncommitted),
+ db_(db),
+ backed_by_snapshot_(backed_by_snapshot) {
+ (void)backed_by_snapshot_; // to silence unused private field warning
+ }
+
+ virtual ~WritePreparedTxnReadCallback() {
+ // If it is not backed by snapshot, the caller must check validity
+ assert(valid_checked_ || backed_by_snapshot_ == kBackedByDBSnapshot);
+ }
+
+ // Will be called to see if the seq number visible; if not it moves on to
+ // the next seq number.
+ inline virtual bool IsVisibleFullCheck(SequenceNumber seq) override {
+ auto snapshot = max_visible_seq_;
+ bool snap_released = false;
+ auto ret =
+ db_->IsInSnapshot(seq, snapshot, min_uncommitted_, &snap_released);
+ assert(!snap_released || backed_by_snapshot_ == kUnbackedByDBSnapshot);
+ snap_released_ |= snap_released;
+ return ret;
+ }
+
+ inline bool valid() {
+ valid_checked_ = true;
+ return snap_released_ == false;
+ }
+
+ // TODO(myabandeh): override Refresh when Iterator::Refresh is supported
+ private:
+ WritePreparedTxnDB* db_;
+ // Whether max_visible_seq_ is backed by a snapshot
+ const SnapshotBackup backed_by_snapshot_;
+ bool snap_released_ = false;
+ // Safety check to ensure that the caller has checked invalid statuses
+ bool valid_checked_ = false;
+};
+
+class AddPreparedCallback : public PreReleaseCallback {
+ public:
+ AddPreparedCallback(WritePreparedTxnDB* db, DBImpl* db_impl,
+ size_t sub_batch_cnt, bool two_write_queues,
+ bool first_prepare_batch)
+ : db_(db),
+ db_impl_(db_impl),
+ sub_batch_cnt_(sub_batch_cnt),
+ two_write_queues_(two_write_queues),
+ first_prepare_batch_(first_prepare_batch) {
+ (void)two_write_queues_; // to silence unused private field warning
+ }
+ virtual Status Callback(SequenceNumber prepare_seq,
+ bool is_mem_disabled __attribute__((__unused__)),
+ uint64_t log_number, size_t index,
+ size_t total) override {
+ assert(index < total);
+ // To reduce the cost of lock acquisition competing with the concurrent
+ // prepare requests, lock on the first callback and unlock on the last.
+ const bool do_lock = !two_write_queues_ || index == 0;
+ const bool do_unlock = !two_write_queues_ || index + 1 == total;
+ // Always Prepare from the main queue
+ assert(!two_write_queues_ || !is_mem_disabled); // implies the 1st queue
+ TEST_SYNC_POINT("AddPreparedCallback::AddPrepared::begin:pause");
+ TEST_SYNC_POINT("AddPreparedCallback::AddPrepared::begin:resume");
+ if (do_lock) {
+ db_->prepared_txns_.push_pop_mutex()->Lock();
+ }
+ const bool kLocked = true;
+ for (size_t i = 0; i < sub_batch_cnt_; i++) {
+ db_->AddPrepared(prepare_seq + i, kLocked);
+ }
+ if (do_unlock) {
+ db_->prepared_txns_.push_pop_mutex()->Unlock();
+ }
+ TEST_SYNC_POINT("AddPreparedCallback::AddPrepared::end");
+ if (first_prepare_batch_) {
+ assert(log_number != 0);
+ db_impl_->logs_with_prep_tracker()->MarkLogAsContainingPrepSection(
+ log_number);
+ }
+ return Status::OK();
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ DBImpl* db_impl_;
+ size_t sub_batch_cnt_;
+ bool two_write_queues_;
+ // It is 2PC and this is the first prepare batch. Always the case in 2PC
+ // unless it is WriteUnPrepared.
+ bool first_prepare_batch_;
+};
+
+class WritePreparedCommitEntryPreReleaseCallback : public PreReleaseCallback {
+ public:
+ // includes_data indicates that the commit also writes non-empty
+ // CommitTimeWriteBatch to memtable, which needs to be committed separately.
+ WritePreparedCommitEntryPreReleaseCallback(
+ WritePreparedTxnDB* db, DBImpl* db_impl, SequenceNumber prep_seq,
+ size_t prep_batch_cnt, size_t data_batch_cnt = 0,
+ SequenceNumber aux_seq = kMaxSequenceNumber, size_t aux_batch_cnt = 0)
+ : db_(db),
+ db_impl_(db_impl),
+ prep_seq_(prep_seq),
+ prep_batch_cnt_(prep_batch_cnt),
+ data_batch_cnt_(data_batch_cnt),
+ includes_data_(data_batch_cnt_ > 0),
+ aux_seq_(aux_seq),
+ aux_batch_cnt_(aux_batch_cnt),
+ includes_aux_batch_(aux_batch_cnt > 0) {
+ assert((prep_batch_cnt_ > 0) != (prep_seq == kMaxSequenceNumber)); // xor
+ assert(prep_batch_cnt_ > 0 || data_batch_cnt_ > 0);
+ assert((aux_batch_cnt_ > 0) != (aux_seq == kMaxSequenceNumber)); // xor
+ }
+
+ virtual Status Callback(SequenceNumber commit_seq,
+ bool is_mem_disabled __attribute__((__unused__)),
+ uint64_t, size_t /*index*/,
+ size_t /*total*/) override {
+ // Always commit from the 2nd queue
+ assert(!db_impl_->immutable_db_options().two_write_queues ||
+ is_mem_disabled);
+ assert(includes_data_ || prep_seq_ != kMaxSequenceNumber);
+ // Data batch is what accompanied with the commit marker and affects the
+ // last seq in the commit batch.
+ const uint64_t last_commit_seq = LIKELY(data_batch_cnt_ <= 1)
+ ? commit_seq
+ : commit_seq + data_batch_cnt_ - 1;
+ if (prep_seq_ != kMaxSequenceNumber) {
+ for (size_t i = 0; i < prep_batch_cnt_; i++) {
+ db_->AddCommitted(prep_seq_ + i, last_commit_seq);
+ }
+ } // else there was no prepare phase
+ if (includes_aux_batch_) {
+ for (size_t i = 0; i < aux_batch_cnt_; i++) {
+ db_->AddCommitted(aux_seq_ + i, last_commit_seq);
+ }
+ }
+ if (includes_data_) {
+ assert(data_batch_cnt_);
+ // Commit the data that is accompanied with the commit request
+ for (size_t i = 0; i < data_batch_cnt_; i++) {
+ // For commit seq of each batch use the commit seq of the last batch.
+ // This would make debugging easier by having all the batches having
+ // the same sequence number.
+ db_->AddCommitted(commit_seq + i, last_commit_seq);
+ }
+ }
+ if (db_impl_->immutable_db_options().two_write_queues) {
+ assert(is_mem_disabled); // implies the 2nd queue
+ // Publish the sequence number. We can do that here assuming the callback
+ // is invoked only from one write queue, which would guarantee that the
+ // publish sequence numbers will be in order, i.e., once a seq is
+ // published all the seq prior to that are also publishable.
+ db_impl_->SetLastPublishedSequence(last_commit_seq);
+ // Note RemovePrepared should be called after publishing the seq.
+ // Otherwise SmallestUnCommittedSeq optimization breaks.
+ if (prep_seq_ != kMaxSequenceNumber) {
+ db_->RemovePrepared(prep_seq_, prep_batch_cnt_);
+ } // else there was no prepare phase
+ if (includes_aux_batch_) {
+ db_->RemovePrepared(aux_seq_, aux_batch_cnt_);
+ }
+ }
+ // else SequenceNumber that is updated as part of the write already does the
+ // publishing
+ return Status::OK();
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ DBImpl* db_impl_;
+ // kMaxSequenceNumber if there was no prepare phase
+ SequenceNumber prep_seq_;
+ size_t prep_batch_cnt_;
+ size_t data_batch_cnt_;
+ // Data here is the batch that is written with the commit marker, either
+ // because it is commit without prepare or commit has a CommitTimeWriteBatch.
+ bool includes_data_;
+ // Auxiliary batch (if there is any) is a batch that is written before, but
+ // gets the same commit seq as prepare batch or data batch. This is used in
+ // two write queues where the CommitTimeWriteBatch becomes the aux batch and
+ // we do a separate write to actually commit everything.
+ SequenceNumber aux_seq_;
+ size_t aux_batch_cnt_;
+ bool includes_aux_batch_;
+};
+
+// For two_write_queues commit both the aborted batch and the cleanup batch and
+// then published the seq
+class WritePreparedRollbackPreReleaseCallback : public PreReleaseCallback {
+ public:
+ WritePreparedRollbackPreReleaseCallback(WritePreparedTxnDB* db,
+ DBImpl* db_impl,
+ SequenceNumber prep_seq,
+ SequenceNumber rollback_seq,
+ size_t prep_batch_cnt)
+ : db_(db),
+ db_impl_(db_impl),
+ prep_seq_(prep_seq),
+ rollback_seq_(rollback_seq),
+ prep_batch_cnt_(prep_batch_cnt) {
+ assert(prep_seq != kMaxSequenceNumber);
+ assert(rollback_seq != kMaxSequenceNumber);
+ assert(prep_batch_cnt_ > 0);
+ }
+
+ Status Callback(SequenceNumber commit_seq, bool is_mem_disabled, uint64_t,
+ size_t /*index*/, size_t /*total*/) override {
+ // Always commit from the 2nd queue
+ assert(is_mem_disabled); // implies the 2nd queue
+ assert(db_impl_->immutable_db_options().two_write_queues);
+#ifdef NDEBUG
+ (void)is_mem_disabled;
+#endif
+ const uint64_t last_commit_seq = commit_seq;
+ db_->AddCommitted(rollback_seq_, last_commit_seq);
+ for (size_t i = 0; i < prep_batch_cnt_; i++) {
+ db_->AddCommitted(prep_seq_ + i, last_commit_seq);
+ }
+ db_impl_->SetLastPublishedSequence(last_commit_seq);
+ return Status::OK();
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ DBImpl* db_impl_;
+ SequenceNumber prep_seq_;
+ SequenceNumber rollback_seq_;
+ size_t prep_batch_cnt_;
+};
+
+// Count the number of sub-batches inside a batch. A sub-batch does not have
+// duplicate keys.
+struct SubBatchCounter : public WriteBatch::Handler {
+ explicit SubBatchCounter(std::map<uint32_t, const Comparator*>& comparators)
+ : comparators_(comparators), batches_(1) {}
+ std::map<uint32_t, const Comparator*>& comparators_;
+ using CFKeys = std::set<Slice, SetComparator>;
+ std::map<uint32_t, CFKeys> keys_;
+ size_t batches_;
+ size_t BatchCount() { return batches_; }
+ void AddKey(const uint32_t cf, const Slice& key);
+ void InitWithComp(const uint32_t cf);
+ Status MarkNoop(bool) override { return Status::OK(); }
+ Status MarkEndPrepare(const Slice&) override { return Status::OK(); }
+ Status MarkCommit(const Slice&) override { return Status::OK(); }
+ Status PutCF(uint32_t cf, const Slice& key, const Slice&) override {
+ AddKey(cf, key);
+ return Status::OK();
+ }
+ Status DeleteCF(uint32_t cf, const Slice& key) override {
+ AddKey(cf, key);
+ return Status::OK();
+ }
+ Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
+ AddKey(cf, key);
+ return Status::OK();
+ }
+ Status MergeCF(uint32_t cf, const Slice& key, const Slice&) override {
+ AddKey(cf, key);
+ return Status::OK();
+ }
+ Status MarkBeginPrepare(bool) override { return Status::OK(); }
+ Status MarkRollback(const Slice&) override { return Status::OK(); }
+ bool WriteAfterCommit() const override { return false; }
+};
+
+SnapshotBackup WritePreparedTxnDB::AssignMinMaxSeqs(const Snapshot* snapshot,
+ SequenceNumber* min,
+ SequenceNumber* max) {
+ if (snapshot != nullptr) {
+ *min = static_cast_with_check<const SnapshotImpl, const Snapshot>(snapshot)
+ ->min_uncommitted_;
+ *max = static_cast_with_check<const SnapshotImpl, const Snapshot>(snapshot)
+ ->number_;
+ return kBackedByDBSnapshot;
+ } else {
+ *min = SmallestUnCommittedSeq();
+ *max = 0; // to be assigned later after sv is referenced.
+ return kUnbackedByDBSnapshot;
+ }
+}
+
+bool WritePreparedTxnDB::ValidateSnapshot(
+ const SequenceNumber snap_seq, const SnapshotBackup backed_by_snapshot,
+ std::memory_order order) {
+ if (backed_by_snapshot == kBackedByDBSnapshot) {
+ return true;
+ } else {
+ SequenceNumber max = max_evicted_seq_.load(order);
+ // Validate that max has not advanced the snapshot seq that is not backed
+ // by a real snapshot. This is a very rare case that should not happen in
+ // real workloads.
+ if (UNLIKELY(snap_seq <= max && snap_seq != 0)) {
+ return false;
+ }
+ }
+ return true;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_unprepared_transaction_test.cc b/src/rocksdb/utilities/transactions/write_unprepared_transaction_test.cc
new file mode 100644
index 000000000..8b1613b2e
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_unprepared_transaction_test.cc
@@ -0,0 +1,727 @@
+// 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_test.h"
+#include "utilities/transactions/write_unprepared_txn.h"
+#include "utilities/transactions/write_unprepared_txn_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class WriteUnpreparedTransactionTestBase : public TransactionTestBase {
+ public:
+ WriteUnpreparedTransactionTestBase(bool use_stackable_db,
+ bool two_write_queue,
+ TxnDBWritePolicy write_policy)
+ : TransactionTestBase(use_stackable_db, two_write_queue, write_policy,
+ kOrderedWrite) {}
+};
+
+class WriteUnpreparedTransactionTest
+ : public WriteUnpreparedTransactionTestBase,
+ virtual public ::testing::WithParamInterface<
+ std::tuple<bool, bool, TxnDBWritePolicy>> {
+ public:
+ WriteUnpreparedTransactionTest()
+ : WriteUnpreparedTransactionTestBase(std::get<0>(GetParam()),
+ std::get<1>(GetParam()),
+ std::get<2>(GetParam())){}
+};
+
+INSTANTIATE_TEST_CASE_P(
+ WriteUnpreparedTransactionTest, WriteUnpreparedTransactionTest,
+ ::testing::Values(std::make_tuple(false, false, WRITE_UNPREPARED),
+ std::make_tuple(false, true, WRITE_UNPREPARED)));
+
+enum StressAction { NO_SNAPSHOT, RO_SNAPSHOT, REFRESH_SNAPSHOT };
+class WriteUnpreparedStressTest : public WriteUnpreparedTransactionTestBase,
+ virtual public ::testing::WithParamInterface<
+ std::tuple<bool, StressAction>> {
+ public:
+ WriteUnpreparedStressTest()
+ : WriteUnpreparedTransactionTestBase(false, std::get<0>(GetParam()),
+ WRITE_UNPREPARED),
+ action_(std::get<1>(GetParam())) {}
+ StressAction action_;
+};
+
+INSTANTIATE_TEST_CASE_P(
+ WriteUnpreparedStressTest, WriteUnpreparedStressTest,
+ ::testing::Values(std::make_tuple(false, NO_SNAPSHOT),
+ std::make_tuple(false, RO_SNAPSHOT),
+ std::make_tuple(false, REFRESH_SNAPSHOT),
+ std::make_tuple(true, NO_SNAPSHOT),
+ std::make_tuple(true, RO_SNAPSHOT),
+ std::make_tuple(true, REFRESH_SNAPSHOT)));
+
+TEST_P(WriteUnpreparedTransactionTest, ReadYourOwnWrite) {
+ // The following tests checks whether reading your own write for
+ // a transaction works for write unprepared, when there are uncommitted
+ // values written into DB.
+ auto verify_state = [](Iterator* iter, const std::string& key,
+ const std::string& value) {
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_OK(iter->status());
+ ASSERT_EQ(key, iter->key().ToString());
+ ASSERT_EQ(value, iter->value().ToString());
+ };
+
+ // Test always reseeking vs never reseeking.
+ for (uint64_t max_skip : {0, std::numeric_limits<int>::max()}) {
+ options.max_sequential_skip_in_iterations = max_skip;
+ options.disable_auto_compactions = true;
+ ReOpen();
+
+ TransactionOptions txn_options;
+ WriteOptions woptions;
+ ReadOptions roptions;
+
+ ASSERT_OK(db->Put(woptions, "a", ""));
+ ASSERT_OK(db->Put(woptions, "b", ""));
+
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ WriteUnpreparedTxn* wup_txn = dynamic_cast<WriteUnpreparedTxn*>(txn);
+ txn->SetSnapshot();
+
+ for (int i = 0; i < 5; i++) {
+ std::string stored_value = "v" + ToString(i);
+ ASSERT_OK(txn->Put("a", stored_value));
+ ASSERT_OK(txn->Put("b", stored_value));
+ wup_txn->FlushWriteBatchToDB(false);
+
+ // Test Get()
+ std::string value;
+ ASSERT_OK(txn->Get(roptions, "a", &value));
+ ASSERT_EQ(value, stored_value);
+ ASSERT_OK(txn->Get(roptions, "b", &value));
+ ASSERT_EQ(value, stored_value);
+
+ // Test Next()
+ auto iter = txn->GetIterator(roptions);
+ iter->Seek("a");
+ verify_state(iter, "a", stored_value);
+
+ iter->Next();
+ verify_state(iter, "b", stored_value);
+
+ iter->SeekToFirst();
+ verify_state(iter, "a", stored_value);
+
+ iter->Next();
+ verify_state(iter, "b", stored_value);
+
+ delete iter;
+
+ // Test Prev()
+ iter = txn->GetIterator(roptions);
+ iter->SeekForPrev("b");
+ verify_state(iter, "b", stored_value);
+
+ iter->Prev();
+ verify_state(iter, "a", stored_value);
+
+ iter->SeekToLast();
+ verify_state(iter, "b", stored_value);
+
+ iter->Prev();
+ verify_state(iter, "a", stored_value);
+
+ delete iter;
+ }
+
+ delete txn;
+ }
+}
+
+#ifndef ROCKSDB_VALGRIND_RUN
+TEST_P(WriteUnpreparedStressTest, ReadYourOwnWriteStress) {
+ // This is a stress test where different threads are writing random keys, and
+ // then before committing or aborting the transaction, it validates to see
+ // that it can read the keys it wrote, and the keys it did not write respect
+ // the snapshot. To avoid row lock contention (and simply stressing the
+ // locking system), each thread is mostly only writing to its own set of keys.
+ const uint32_t kNumIter = 1000;
+ const uint32_t kNumThreads = 10;
+ const uint32_t kNumKeys = 5;
+
+ std::default_random_engine rand(static_cast<uint32_t>(
+ std::hash<std::thread::id>()(std::this_thread::get_id())));
+
+ // Test with
+ // 1. no snapshots set
+ // 2. snapshot set on ReadOptions
+ // 3. snapshot set, and refreshing after every write.
+ StressAction a = action_;
+ WriteOptions write_options;
+ txn_db_options.transaction_lock_timeout = -1;
+ options.disable_auto_compactions = true;
+ ReOpen();
+
+ std::vector<std::string> keys;
+ for (uint32_t k = 0; k < kNumKeys * kNumThreads; k++) {
+ keys.push_back("k" + ToString(k));
+ }
+ std::shuffle(keys.begin(), keys.end(), rand);
+
+ // This counter will act as a "sequence number" to help us validate
+ // visibility logic with snapshots. If we had direct access to the seqno of
+ // snapshots and key/values, then we should directly compare those instead.
+ std::atomic<int64_t> counter(0);
+
+ std::function<void(uint32_t)> stress_thread = [&](int id) {
+ size_t tid = std::hash<std::thread::id>()(std::this_thread::get_id());
+ Random64 rnd(static_cast<uint32_t>(tid));
+
+ Transaction* txn;
+ TransactionOptions txn_options;
+ // batch_size of 1 causes writes to DB for every marker.
+ txn_options.write_batch_flush_threshold = 1;
+ ReadOptions read_options;
+
+ for (uint32_t i = 0; i < kNumIter; i++) {
+ std::set<std::string> owned_keys(&keys[id * kNumKeys],
+ &keys[(id + 1) * kNumKeys]);
+ // Add unowned keys to make the workload more interesting, but this
+ // increases row lock contention, so just do it sometimes.
+ if (rnd.OneIn(2)) {
+ owned_keys.insert(keys[rnd.Uniform(kNumKeys * kNumThreads)]);
+ }
+
+ txn = db->BeginTransaction(write_options, txn_options);
+ txn->SetName(ToString(id));
+ txn->SetSnapshot();
+ if (a >= RO_SNAPSHOT) {
+ read_options.snapshot = txn->GetSnapshot();
+ ASSERT_TRUE(read_options.snapshot != nullptr);
+ }
+
+ uint64_t buf[2];
+ buf[0] = id;
+
+ // When scanning through the database, make sure that all unprepared
+ // keys have value >= snapshot and all other keys have value < snapshot.
+ int64_t snapshot_num = counter.fetch_add(1);
+
+ Status s;
+ for (const auto& key : owned_keys) {
+ buf[1] = counter.fetch_add(1);
+ s = txn->Put(key, Slice((const char*)buf, sizeof(buf)));
+ if (!s.ok()) {
+ break;
+ }
+ if (a == REFRESH_SNAPSHOT) {
+ txn->SetSnapshot();
+ read_options.snapshot = txn->GetSnapshot();
+ snapshot_num = counter.fetch_add(1);
+ }
+ }
+
+ // Failure is possible due to snapshot validation. In this case,
+ // rollback and move onto next iteration.
+ if (!s.ok()) {
+ ASSERT_TRUE(s.IsBusy());
+ ASSERT_OK(txn->Rollback());
+ delete txn;
+ continue;
+ }
+
+ auto verify_key = [&owned_keys, &a, &id, &snapshot_num](
+ const std::string& key, const std::string& value) {
+ if (owned_keys.count(key) > 0) {
+ ASSERT_EQ(value.size(), 16);
+
+ // Since this key is part of owned_keys, then this key must be
+ // unprepared by this transaction identified by 'id'
+ ASSERT_EQ(((int64_t*)value.c_str())[0], id);
+ if (a == REFRESH_SNAPSHOT) {
+ // If refresh snapshot is true, then the snapshot is refreshed
+ // after every Put(), meaning that the current snapshot in
+ // snapshot_num must be greater than the "seqno" of any keys
+ // written by the current transaction.
+ ASSERT_LT(((int64_t*)value.c_str())[1], snapshot_num);
+ } else {
+ // If refresh snapshot is not on, then the snapshot was taken at
+ // the beginning of the transaction, meaning all writes must come
+ // after snapshot_num
+ ASSERT_GT(((int64_t*)value.c_str())[1], snapshot_num);
+ }
+ } else if (a >= RO_SNAPSHOT) {
+ // If this is not an unprepared key, just assert that the key
+ // "seqno" is smaller than the snapshot seqno.
+ ASSERT_EQ(value.size(), 16);
+ ASSERT_LT(((int64_t*)value.c_str())[1], snapshot_num);
+ }
+ };
+
+ // Validate Get()/Next()/Prev(). Do only one of them to save time, and
+ // reduce lock contention.
+ switch (rnd.Uniform(3)) {
+ case 0: // Validate Get()
+ {
+ for (const auto& key : keys) {
+ std::string value;
+ s = txn->Get(read_options, Slice(key), &value);
+ if (!s.ok()) {
+ ASSERT_TRUE(s.IsNotFound());
+ ASSERT_EQ(owned_keys.count(key), 0);
+ } else {
+ verify_key(key, value);
+ }
+ }
+ break;
+ }
+ case 1: // Validate Next()
+ {
+ Iterator* iter = txn->GetIterator(read_options);
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ verify_key(iter->key().ToString(), iter->value().ToString());
+ }
+ delete iter;
+ break;
+ }
+ case 2: // Validate Prev()
+ {
+ Iterator* iter = txn->GetIterator(read_options);
+ for (iter->SeekToLast(); iter->Valid(); iter->Prev()) {
+ verify_key(iter->key().ToString(), iter->value().ToString());
+ }
+ delete iter;
+ break;
+ }
+ default:
+ ASSERT_TRUE(false);
+ }
+
+ if (rnd.OneIn(2)) {
+ ASSERT_OK(txn->Commit());
+ } else {
+ ASSERT_OK(txn->Rollback());
+ }
+ delete txn;
+ }
+ };
+
+ std::vector<port::Thread> threads;
+ for (uint32_t i = 0; i < kNumThreads; i++) {
+ threads.emplace_back(stress_thread, i);
+ }
+
+ for (auto& t : threads) {
+ t.join();
+ }
+}
+#endif // ROCKSDB_VALGRIND_RUN
+
+// This tests how write unprepared behaves during recovery when the DB crashes
+// after a transaction has either been unprepared or prepared, and tests if
+// the changes are correctly applied for prepared transactions if we decide to
+// rollback/commit.
+TEST_P(WriteUnpreparedTransactionTest, RecoveryTest) {
+ WriteOptions write_options;
+ write_options.disableWAL = false;
+ TransactionOptions txn_options;
+ std::vector<Transaction*> prepared_trans;
+ WriteUnpreparedTxnDB* wup_db;
+ options.disable_auto_compactions = true;
+
+ enum Action { UNPREPARED, ROLLBACK, COMMIT };
+
+ // batch_size of 1 causes writes to DB for every marker.
+ for (size_t batch_size : {1, 1000000}) {
+ txn_options.write_batch_flush_threshold = batch_size;
+ for (bool empty : {true, false}) {
+ for (Action a : {UNPREPARED, ROLLBACK, COMMIT}) {
+ for (int num_batches = 1; num_batches < 10; num_batches++) {
+ // Reset database.
+ prepared_trans.clear();
+ ReOpen();
+ wup_db = dynamic_cast<WriteUnpreparedTxnDB*>(db);
+ if (!empty) {
+ for (int i = 0; i < num_batches; i++) {
+ ASSERT_OK(db->Put(WriteOptions(), "k" + ToString(i),
+ "before value" + ToString(i)));
+ }
+ }
+
+ // Write num_batches unprepared batches.
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ WriteUnpreparedTxn* wup_txn = dynamic_cast<WriteUnpreparedTxn*>(txn);
+ txn->SetName("xid");
+ for (int i = 0; i < num_batches; i++) {
+ ASSERT_OK(txn->Put("k" + ToString(i), "value" + ToString(i)));
+ if (txn_options.write_batch_flush_threshold == 1) {
+ // WriteUnprepared will check write_batch_flush_threshold and
+ // possibly flush before appending to the write batch. No flush
+ // will happen at the first write because the batch is still
+ // empty, so after k puts, there should be k-1 flushed batches.
+ ASSERT_EQ(wup_txn->GetUnpreparedSequenceNumbers().size(), i);
+ } else {
+ ASSERT_EQ(wup_txn->GetUnpreparedSequenceNumbers().size(), 0);
+ }
+ }
+ if (a == UNPREPARED) {
+ // This is done to prevent the destructor from rolling back the
+ // transaction for us, since we want to pretend we crashed and
+ // test that recovery does the rollback.
+ wup_txn->unprep_seqs_.clear();
+ } else {
+ txn->Prepare();
+ }
+ delete txn;
+
+ // Crash and run recovery code paths.
+ wup_db->db_impl_->FlushWAL(true);
+ wup_db->TEST_Crash();
+ ReOpenNoDelete();
+ assert(db != nullptr);
+
+ db->GetAllPreparedTransactions(&prepared_trans);
+ ASSERT_EQ(prepared_trans.size(), a == UNPREPARED ? 0 : 1);
+ if (a == ROLLBACK) {
+ ASSERT_OK(prepared_trans[0]->Rollback());
+ delete prepared_trans[0];
+ } else if (a == COMMIT) {
+ ASSERT_OK(prepared_trans[0]->Commit());
+ delete prepared_trans[0];
+ }
+
+ Iterator* iter = db->NewIterator(ReadOptions());
+ iter->SeekToFirst();
+ // Check that DB has before values.
+ if (!empty || a == COMMIT) {
+ for (int i = 0; i < num_batches; i++) {
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(iter->key().ToString(), "k" + ToString(i));
+ if (a == COMMIT) {
+ ASSERT_EQ(iter->value().ToString(), "value" + ToString(i));
+ } else {
+ ASSERT_EQ(iter->value().ToString(),
+ "before value" + ToString(i));
+ }
+ iter->Next();
+ }
+ }
+ ASSERT_FALSE(iter->Valid());
+ delete iter;
+ }
+ }
+ }
+ }
+}
+
+// Basic test to see that unprepared batch gets written to DB when batch size
+// is exceeded. It also does some basic checks to see if commit/rollback works
+// as expected for write unprepared.
+TEST_P(WriteUnpreparedTransactionTest, UnpreparedBatch) {
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ const int kNumKeys = 10;
+
+ // batch_size of 1 causes writes to DB for every marker.
+ for (size_t batch_size : {1, 1000000}) {
+ txn_options.write_batch_flush_threshold = batch_size;
+ for (bool prepare : {false, true}) {
+ for (bool commit : {false, true}) {
+ ReOpen();
+ Transaction* txn = db->BeginTransaction(write_options, txn_options);
+ WriteUnpreparedTxn* wup_txn = dynamic_cast<WriteUnpreparedTxn*>(txn);
+ txn->SetName("xid");
+
+ for (int i = 0; i < kNumKeys; i++) {
+ txn->Put("k" + ToString(i), "v" + ToString(i));
+ if (txn_options.write_batch_flush_threshold == 1) {
+ // WriteUnprepared will check write_batch_flush_threshold and
+ // possibly flush before appending to the write batch. No flush will
+ // happen at the first write because the batch is still empty, so
+ // after k puts, there should be k-1 flushed batches.
+ ASSERT_EQ(wup_txn->GetUnpreparedSequenceNumbers().size(), i);
+ } else {
+ ASSERT_EQ(wup_txn->GetUnpreparedSequenceNumbers().size(), 0);
+ }
+ }
+
+ if (prepare) {
+ ASSERT_OK(txn->Prepare());
+ }
+
+ Iterator* iter = db->NewIterator(ReadOptions());
+ iter->SeekToFirst();
+ assert(!iter->Valid());
+ ASSERT_FALSE(iter->Valid());
+ delete iter;
+
+ if (commit) {
+ ASSERT_OK(txn->Commit());
+ } else {
+ ASSERT_OK(txn->Rollback());
+ }
+ delete txn;
+
+ iter = db->NewIterator(ReadOptions());
+ iter->SeekToFirst();
+
+ for (int i = 0; i < (commit ? kNumKeys : 0); i++) {
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(iter->key().ToString(), "k" + ToString(i));
+ ASSERT_EQ(iter->value().ToString(), "v" + ToString(i));
+ iter->Next();
+ }
+ ASSERT_FALSE(iter->Valid());
+ delete iter;
+ }
+ }
+ }
+}
+
+// Test whether logs containing unprepared/prepared batches are kept even
+// after memtable finishes flushing, and whether they are removed when
+// transaction commits/aborts.
+//
+// TODO(lth): Merge with TransactionTest/TwoPhaseLogRollingTest tests.
+TEST_P(WriteUnpreparedTransactionTest, MarkLogWithPrepSection) {
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ // batch_size of 1 causes writes to DB for every marker.
+ txn_options.write_batch_flush_threshold = 1;
+ const int kNumKeys = 10;
+
+ WriteOptions wopts;
+ wopts.sync = true;
+
+ for (bool prepare : {false, true}) {
+ for (bool commit : {false, true}) {
+ ReOpen();
+ auto wup_db = dynamic_cast<WriteUnpreparedTxnDB*>(db);
+ auto db_impl = wup_db->db_impl_;
+
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn1->SetName("xid1"));
+
+ Transaction* txn2 = db->BeginTransaction(write_options, txn_options);
+ ASSERT_OK(txn2->SetName("xid2"));
+
+ // Spread this transaction across multiple log files.
+ for (int i = 0; i < kNumKeys; i++) {
+ ASSERT_OK(txn1->Put("k1" + ToString(i), "v" + ToString(i)));
+ if (i >= kNumKeys / 2) {
+ ASSERT_OK(txn2->Put("k2" + ToString(i), "v" + ToString(i)));
+ }
+
+ if (i > 0) {
+ db_impl->TEST_SwitchWAL();
+ }
+ }
+
+ ASSERT_GT(txn1->GetLogNumber(), 0);
+ ASSERT_GT(txn2->GetLogNumber(), 0);
+
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(),
+ txn1->GetLogNumber());
+ ASSERT_GT(db_impl->TEST_LogfileNumber(), txn1->GetLogNumber());
+
+ if (prepare) {
+ ASSERT_OK(txn1->Prepare());
+ ASSERT_OK(txn2->Prepare());
+ }
+
+ ASSERT_GE(db_impl->TEST_LogfileNumber(), txn1->GetLogNumber());
+ ASSERT_GE(db_impl->TEST_LogfileNumber(), txn2->GetLogNumber());
+
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(),
+ txn1->GetLogNumber());
+ if (commit) {
+ ASSERT_OK(txn1->Commit());
+ } else {
+ ASSERT_OK(txn1->Rollback());
+ }
+
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(),
+ txn2->GetLogNumber());
+
+ if (commit) {
+ ASSERT_OK(txn2->Commit());
+ } else {
+ ASSERT_OK(txn2->Rollback());
+ }
+
+ ASSERT_EQ(db_impl->TEST_FindMinLogContainingOutstandingPrep(), 0);
+
+ delete txn1;
+ delete txn2;
+ }
+ }
+}
+
+TEST_P(WriteUnpreparedTransactionTest, NoSnapshotWrite) {
+ WriteOptions woptions;
+ TransactionOptions txn_options;
+ txn_options.write_batch_flush_threshold = 1;
+
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+
+ // Do some writes with no snapshot
+ ASSERT_OK(txn->Put("a", "a"));
+ ASSERT_OK(txn->Put("b", "b"));
+ ASSERT_OK(txn->Put("c", "c"));
+
+ // Test that it is still possible to create iterators after writes with no
+ // snapshot, if iterator snapshot is fresh enough.
+ ReadOptions roptions;
+ auto iter = txn->GetIterator(roptions);
+ int keys = 0;
+ for (iter->SeekToLast(); iter->Valid(); iter->Prev(), keys++) {
+ ASSERT_OK(iter->status());
+ ASSERT_EQ(iter->key().ToString(), iter->value().ToString());
+ }
+ ASSERT_EQ(keys, 3);
+
+ delete iter;
+ delete txn;
+}
+
+// Test whether write to a transaction while iterating is supported.
+TEST_P(WriteUnpreparedTransactionTest, IterateAndWrite) {
+ WriteOptions woptions;
+ TransactionOptions txn_options;
+ txn_options.write_batch_flush_threshold = 1;
+
+ enum Action { DO_DELETE, DO_UPDATE };
+
+ for (Action a : {DO_DELETE, DO_UPDATE}) {
+ for (int i = 0; i < 100; i++) {
+ ASSERT_OK(db->Put(woptions, ToString(i), ToString(i)));
+ }
+
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ // write_batch_ now contains 1 key.
+ ASSERT_OK(txn->Put("9", "a"));
+
+ ReadOptions roptions;
+ auto iter = txn->GetIterator(roptions);
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ ASSERT_OK(iter->status());
+ if (iter->key() == "9") {
+ ASSERT_EQ(iter->value().ToString(), "a");
+ } else {
+ ASSERT_EQ(iter->key().ToString(), iter->value().ToString());
+ }
+
+ if (a == DO_DELETE) {
+ ASSERT_OK(txn->Delete(iter->key()));
+ } else {
+ ASSERT_OK(txn->Put(iter->key(), "b"));
+ }
+ }
+
+ delete iter;
+ ASSERT_OK(txn->Commit());
+
+ iter = db->NewIterator(roptions);
+ if (a == DO_DELETE) {
+ // Check that db is empty.
+ iter->SeekToFirst();
+ ASSERT_FALSE(iter->Valid());
+ } else {
+ int keys = 0;
+ // Check that all values are updated to b.
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next(), keys++) {
+ ASSERT_OK(iter->status());
+ ASSERT_EQ(iter->value().ToString(), "b");
+ }
+ ASSERT_EQ(keys, 100);
+ }
+
+ delete iter;
+ delete txn;
+ }
+}
+
+TEST_P(WriteUnpreparedTransactionTest, SavePoint) {
+ WriteOptions woptions;
+ TransactionOptions txn_options;
+ txn_options.write_batch_flush_threshold = 1;
+
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ txn->SetSavePoint();
+ ASSERT_OK(txn->Put("a", "a"));
+ ASSERT_OK(txn->Put("b", "b"));
+ ASSERT_OK(txn->Commit());
+
+ ReadOptions roptions;
+ std::string value;
+ ASSERT_OK(txn->Get(roptions, "a", &value));
+ ASSERT_EQ(value, "a");
+ ASSERT_OK(txn->Get(roptions, "b", &value));
+ ASSERT_EQ(value, "b");
+ delete txn;
+}
+
+TEST_P(WriteUnpreparedTransactionTest, UntrackedKeys) {
+ WriteOptions woptions;
+ TransactionOptions txn_options;
+ txn_options.write_batch_flush_threshold = 1;
+
+ Transaction* txn = db->BeginTransaction(woptions, txn_options);
+ auto wb = txn->GetWriteBatch()->GetWriteBatch();
+ ASSERT_OK(txn->Put("a", "a"));
+ ASSERT_OK(wb->Put("a_untrack", "a_untrack"));
+ txn->SetSavePoint();
+ ASSERT_OK(txn->Put("b", "b"));
+ ASSERT_OK(txn->Put("b_untrack", "b_untrack"));
+
+ ReadOptions roptions;
+ std::string value;
+ ASSERT_OK(txn->Get(roptions, "a", &value));
+ ASSERT_EQ(value, "a");
+ ASSERT_OK(txn->Get(roptions, "a_untrack", &value));
+ ASSERT_EQ(value, "a_untrack");
+ ASSERT_OK(txn->Get(roptions, "b", &value));
+ ASSERT_EQ(value, "b");
+ ASSERT_OK(txn->Get(roptions, "b_untrack", &value));
+ ASSERT_EQ(value, "b_untrack");
+
+ // b and b_untrack should be rolled back.
+ ASSERT_OK(txn->RollbackToSavePoint());
+ ASSERT_OK(txn->Get(roptions, "a", &value));
+ ASSERT_EQ(value, "a");
+ ASSERT_OK(txn->Get(roptions, "a_untrack", &value));
+ ASSERT_EQ(value, "a_untrack");
+ auto s = txn->Get(roptions, "b", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn->Get(roptions, "b_untrack", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Everything should be rolled back.
+ ASSERT_OK(txn->Rollback());
+ s = txn->Get(roptions, "a", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn->Get(roptions, "a_untrack", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn->Get(roptions, "b", &value);
+ ASSERT_TRUE(s.IsNotFound());
+ s = txn->Get(roptions, "b_untrack", &value);
+ ASSERT_TRUE(s.IsNotFound());
+
+ delete txn;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(stderr,
+ "SKIPPED as Transactions are not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_unprepared_txn.cc b/src/rocksdb/utilities/transactions/write_unprepared_txn.cc
new file mode 100644
index 000000000..01ec298cf
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_unprepared_txn.cc
@@ -0,0 +1,999 @@
+// 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/write_unprepared_txn.h"
+#include "db/db_impl/db_impl.h"
+#include "util/cast_util.h"
+#include "utilities/transactions/write_unprepared_txn_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+bool WriteUnpreparedTxnReadCallback::IsVisibleFullCheck(SequenceNumber seq) {
+ // Since unprep_seqs maps prep_seq => prepare_batch_cnt, to check if seq is
+ // in unprep_seqs, we have to check if seq is equal to prep_seq or any of
+ // the prepare_batch_cnt seq nums after it.
+ //
+ // TODO(lth): Can be optimized with std::lower_bound if unprep_seqs is
+ // large.
+ for (const auto& it : unprep_seqs_) {
+ if (it.first <= seq && seq < it.first + it.second) {
+ return true;
+ }
+ }
+
+ bool snap_released = false;
+ auto ret =
+ db_->IsInSnapshot(seq, wup_snapshot_, min_uncommitted_, &snap_released);
+ assert(!snap_released || backed_by_snapshot_ == kUnbackedByDBSnapshot);
+ snap_released_ |= snap_released;
+ return ret;
+}
+
+WriteUnpreparedTxn::WriteUnpreparedTxn(WriteUnpreparedTxnDB* txn_db,
+ const WriteOptions& write_options,
+ const TransactionOptions& txn_options)
+ : WritePreparedTxn(txn_db, write_options, txn_options),
+ wupt_db_(txn_db),
+ last_log_number_(0),
+ recovered_txn_(false),
+ largest_validated_seq_(0) {
+ if (txn_options.write_batch_flush_threshold < 0) {
+ write_batch_flush_threshold_ =
+ txn_db_impl_->GetTxnDBOptions().default_write_batch_flush_threshold;
+ } else {
+ write_batch_flush_threshold_ = txn_options.write_batch_flush_threshold;
+ }
+}
+
+WriteUnpreparedTxn::~WriteUnpreparedTxn() {
+ if (!unprep_seqs_.empty()) {
+ assert(log_number_ > 0);
+ assert(GetId() > 0);
+ assert(!name_.empty());
+
+ // We should rollback regardless of GetState, but some unit tests that
+ // test crash recovery run the destructor assuming that rollback does not
+ // happen, so that rollback during recovery can be exercised.
+ if (GetState() == STARTED || GetState() == LOCKS_STOLEN) {
+ auto s = RollbackInternal();
+ assert(s.ok());
+ if (!s.ok()) {
+ ROCKS_LOG_FATAL(
+ wupt_db_->info_log_,
+ "Rollback of WriteUnprepared transaction failed in destructor: %s",
+ s.ToString().c_str());
+ }
+ dbimpl_->logs_with_prep_tracker()->MarkLogAsHavingPrepSectionFlushed(
+ log_number_);
+ }
+ }
+
+ // Call tracked_keys_.clear() so that ~PessimisticTransaction does not
+ // try to unlock keys for recovered transactions.
+ if (recovered_txn_) {
+ tracked_keys_.clear();
+ }
+}
+
+void WriteUnpreparedTxn::Initialize(const TransactionOptions& txn_options) {
+ PessimisticTransaction::Initialize(txn_options);
+ if (txn_options.write_batch_flush_threshold < 0) {
+ write_batch_flush_threshold_ =
+ txn_db_impl_->GetTxnDBOptions().default_write_batch_flush_threshold;
+ } else {
+ write_batch_flush_threshold_ = txn_options.write_batch_flush_threshold;
+ }
+
+ unprep_seqs_.clear();
+ flushed_save_points_.reset(nullptr);
+ unflushed_save_points_.reset(nullptr);
+ recovered_txn_ = false;
+ largest_validated_seq_ = 0;
+ assert(active_iterators_.empty());
+ active_iterators_.clear();
+ untracked_keys_.clear();
+}
+
+Status WriteUnpreparedTxn::HandleWrite(std::function<Status()> do_write) {
+ Status s;
+ if (active_iterators_.empty()) {
+ s = MaybeFlushWriteBatchToDB();
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ s = do_write();
+ if (s.ok()) {
+ if (snapshot_) {
+ largest_validated_seq_ =
+ std::max(largest_validated_seq_, snapshot_->GetSequenceNumber());
+ } else {
+ // TODO(lth): We should use the same number as tracked_at_seq in TryLock,
+ // because what is actually being tracked is the sequence number at which
+ // this key was locked at.
+ largest_validated_seq_ = db_impl_->GetLastPublishedSequence();
+ }
+ }
+ return s;
+}
+
+Status WriteUnpreparedTxn::Put(ColumnFamilyHandle* column_family,
+ const Slice& key, const Slice& value,
+ const bool assume_tracked) {
+ return HandleWrite([&]() {
+ return TransactionBaseImpl::Put(column_family, key, value, assume_tracked);
+ });
+}
+
+Status WriteUnpreparedTxn::Put(ColumnFamilyHandle* column_family,
+ const SliceParts& key, const SliceParts& value,
+ const bool assume_tracked) {
+ return HandleWrite([&]() {
+ return TransactionBaseImpl::Put(column_family, key, value, assume_tracked);
+ });
+}
+
+Status WriteUnpreparedTxn::Merge(ColumnFamilyHandle* column_family,
+ const Slice& key, const Slice& value,
+ const bool assume_tracked) {
+ return HandleWrite([&]() {
+ return TransactionBaseImpl::Merge(column_family, key, value,
+ assume_tracked);
+ });
+}
+
+Status WriteUnpreparedTxn::Delete(ColumnFamilyHandle* column_family,
+ const Slice& key, const bool assume_tracked) {
+ return HandleWrite([&]() {
+ return TransactionBaseImpl::Delete(column_family, key, assume_tracked);
+ });
+}
+
+Status WriteUnpreparedTxn::Delete(ColumnFamilyHandle* column_family,
+ const SliceParts& key,
+ const bool assume_tracked) {
+ return HandleWrite([&]() {
+ return TransactionBaseImpl::Delete(column_family, key, assume_tracked);
+ });
+}
+
+Status WriteUnpreparedTxn::SingleDelete(ColumnFamilyHandle* column_family,
+ const Slice& key,
+ const bool assume_tracked) {
+ return HandleWrite([&]() {
+ return TransactionBaseImpl::SingleDelete(column_family, key,
+ assume_tracked);
+ });
+}
+
+Status WriteUnpreparedTxn::SingleDelete(ColumnFamilyHandle* column_family,
+ const SliceParts& key,
+ const bool assume_tracked) {
+ return HandleWrite([&]() {
+ return TransactionBaseImpl::SingleDelete(column_family, key,
+ assume_tracked);
+ });
+}
+
+// WriteUnpreparedTxn::RebuildFromWriteBatch is only called on recovery. For
+// WriteUnprepared, the write batches have already been written into the
+// database during WAL replay, so all we have to do is just to "retrack" the key
+// so that rollbacks are possible.
+//
+// Calling TryLock instead of TrackKey is also possible, but as an optimization,
+// recovered transactions do not hold locks on their keys. This follows the
+// implementation in PessimisticTransactionDB::Initialize where we set
+// skip_concurrency_control to true.
+Status WriteUnpreparedTxn::RebuildFromWriteBatch(WriteBatch* wb) {
+ struct TrackKeyHandler : public WriteBatch::Handler {
+ WriteUnpreparedTxn* txn_;
+ bool rollback_merge_operands_;
+
+ TrackKeyHandler(WriteUnpreparedTxn* txn, bool rollback_merge_operands)
+ : txn_(txn), rollback_merge_operands_(rollback_merge_operands) {}
+
+ Status PutCF(uint32_t cf, const Slice& key, const Slice&) override {
+ txn_->TrackKey(cf, key.ToString(), kMaxSequenceNumber,
+ false /* read_only */, true /* exclusive */);
+ return Status::OK();
+ }
+
+ Status DeleteCF(uint32_t cf, const Slice& key) override {
+ txn_->TrackKey(cf, key.ToString(), kMaxSequenceNumber,
+ false /* read_only */, true /* exclusive */);
+ return Status::OK();
+ }
+
+ Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
+ txn_->TrackKey(cf, key.ToString(), kMaxSequenceNumber,
+ false /* read_only */, true /* exclusive */);
+ return Status::OK();
+ }
+
+ Status MergeCF(uint32_t cf, const Slice& key, const Slice&) override {
+ if (rollback_merge_operands_) {
+ txn_->TrackKey(cf, key.ToString(), kMaxSequenceNumber,
+ false /* read_only */, true /* exclusive */);
+ }
+ return Status::OK();
+ }
+
+ // Recovered batches do not contain 2PC markers.
+ Status MarkBeginPrepare(bool) override { return Status::InvalidArgument(); }
+
+ Status MarkEndPrepare(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+
+ Status MarkNoop(bool) override { return Status::InvalidArgument(); }
+
+ Status MarkCommit(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+
+ Status MarkRollback(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+ };
+
+ TrackKeyHandler handler(this,
+ wupt_db_->txn_db_options_.rollback_merge_operands);
+ return wb->Iterate(&handler);
+}
+
+Status WriteUnpreparedTxn::MaybeFlushWriteBatchToDB() {
+ const bool kPrepared = true;
+ Status s;
+ if (write_batch_flush_threshold_ > 0 &&
+ write_batch_.GetWriteBatch()->Count() > 0 &&
+ write_batch_.GetDataSize() >
+ static_cast<size_t>(write_batch_flush_threshold_)) {
+ assert(GetState() != PREPARED);
+ s = FlushWriteBatchToDB(!kPrepared);
+ }
+ return s;
+}
+
+Status WriteUnpreparedTxn::FlushWriteBatchToDB(bool prepared) {
+ // If the current write batch contains savepoints, then some special handling
+ // is required so that RollbackToSavepoint can work.
+ //
+ // RollbackToSavepoint is not supported after Prepare() is called, so only do
+ // this for unprepared batches.
+ if (!prepared && unflushed_save_points_ != nullptr &&
+ !unflushed_save_points_->empty()) {
+ return FlushWriteBatchWithSavePointToDB();
+ }
+
+ return FlushWriteBatchToDBInternal(prepared);
+}
+
+Status WriteUnpreparedTxn::FlushWriteBatchToDBInternal(bool prepared) {
+ if (name_.empty()) {
+ assert(!prepared);
+#ifndef NDEBUG
+ static std::atomic_ullong autogen_id{0};
+ // To avoid changing all tests to call SetName, just autogenerate one.
+ if (wupt_db_->txn_db_options_.autogenerate_name) {
+ SetName(std::string("autoxid") + ToString(autogen_id.fetch_add(1)));
+ } else
+#endif
+ {
+ return Status::InvalidArgument("Cannot write to DB without SetName.");
+ }
+ }
+
+ struct UntrackedKeyHandler : public WriteBatch::Handler {
+ WriteUnpreparedTxn* txn_;
+ bool rollback_merge_operands_;
+
+ UntrackedKeyHandler(WriteUnpreparedTxn* txn, bool rollback_merge_operands)
+ : txn_(txn), rollback_merge_operands_(rollback_merge_operands) {}
+
+ Status AddUntrackedKey(uint32_t cf, const Slice& key) {
+ auto str = key.ToString();
+ if (txn_->tracked_keys_[cf].count(str) == 0) {
+ txn_->untracked_keys_[cf].push_back(str);
+ }
+ return Status::OK();
+ }
+
+ Status PutCF(uint32_t cf, const Slice& key, const Slice&) override {
+ return AddUntrackedKey(cf, key);
+ }
+
+ Status DeleteCF(uint32_t cf, const Slice& key) override {
+ return AddUntrackedKey(cf, key);
+ }
+
+ Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
+ return AddUntrackedKey(cf, key);
+ }
+
+ Status MergeCF(uint32_t cf, const Slice& key, const Slice&) override {
+ if (rollback_merge_operands_) {
+ return AddUntrackedKey(cf, key);
+ }
+ return Status::OK();
+ }
+
+ // The only expected 2PC marker is the initial Noop marker.
+ Status MarkNoop(bool empty_batch) override {
+ return empty_batch ? Status::OK() : Status::InvalidArgument();
+ }
+
+ Status MarkBeginPrepare(bool) override { return Status::InvalidArgument(); }
+
+ Status MarkEndPrepare(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+
+ Status MarkCommit(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+
+ Status MarkRollback(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+ };
+
+ UntrackedKeyHandler handler(
+ this, wupt_db_->txn_db_options_.rollback_merge_operands);
+ auto s = GetWriteBatch()->GetWriteBatch()->Iterate(&handler);
+ assert(s.ok());
+
+ // TODO(lth): Reduce duplicate code with WritePrepared prepare logic.
+ WriteOptions write_options = write_options_;
+ write_options.disableWAL = false;
+ const bool WRITE_AFTER_COMMIT = true;
+ const bool first_prepare_batch = log_number_ == 0;
+ // MarkEndPrepare will change Noop marker to the appropriate marker.
+ WriteBatchInternal::MarkEndPrepare(GetWriteBatch()->GetWriteBatch(), name_,
+ !WRITE_AFTER_COMMIT, !prepared);
+ // For each duplicate key we account for a new sub-batch
+ prepare_batch_cnt_ = GetWriteBatch()->SubBatchCnt();
+ // AddPrepared better to be called in the pre-release callback otherwise there
+ // is a non-zero chance of max advancing prepare_seq and readers assume the
+ // data as committed.
+ // Also having it in the PreReleaseCallback allows in-order addition of
+ // prepared entries to PreparedHeap and hence enables an optimization. Refer
+ // to SmallestUnCommittedSeq for more details.
+ AddPreparedCallback add_prepared_callback(
+ wpt_db_, db_impl_, prepare_batch_cnt_,
+ db_impl_->immutable_db_options().two_write_queues, first_prepare_batch);
+ const bool DISABLE_MEMTABLE = true;
+ uint64_t seq_used = kMaxSequenceNumber;
+ // log_number_ should refer to the oldest log containing uncommitted data
+ // from the current transaction. This means that if log_number_ is set,
+ // WriteImpl should not overwrite that value, so set log_used to nullptr if
+ // log_number_ is already set.
+ s = db_impl_->WriteImpl(write_options, GetWriteBatch()->GetWriteBatch(),
+ /*callback*/ nullptr, &last_log_number_,
+ /*log ref*/ 0, !DISABLE_MEMTABLE, &seq_used,
+ prepare_batch_cnt_, &add_prepared_callback);
+ if (log_number_ == 0) {
+ log_number_ = last_log_number_;
+ }
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ auto prepare_seq = seq_used;
+
+ // Only call SetId if it hasn't been set yet.
+ if (GetId() == 0) {
+ SetId(prepare_seq);
+ }
+ // unprep_seqs_ will also contain prepared seqnos since they are treated in
+ // the same way in the prepare/commit callbacks. See the comment on the
+ // definition of unprep_seqs_.
+ unprep_seqs_[prepare_seq] = prepare_batch_cnt_;
+
+ // Reset transaction state.
+ if (!prepared) {
+ prepare_batch_cnt_ = 0;
+ const bool kClear = true;
+ TransactionBaseImpl::InitWriteBatch(kClear);
+ }
+
+ return s;
+}
+
+Status WriteUnpreparedTxn::FlushWriteBatchWithSavePointToDB() {
+ assert(unflushed_save_points_ != nullptr &&
+ unflushed_save_points_->size() > 0);
+ assert(save_points_ != nullptr && save_points_->size() > 0);
+ assert(save_points_->size() >= unflushed_save_points_->size());
+
+ // Handler class for creating an unprepared batch from a savepoint.
+ struct SavePointBatchHandler : public WriteBatch::Handler {
+ WriteBatchWithIndex* wb_;
+ const std::map<uint32_t, ColumnFamilyHandle*>& handles_;
+
+ SavePointBatchHandler(
+ WriteBatchWithIndex* wb,
+ const std::map<uint32_t, ColumnFamilyHandle*>& handles)
+ : wb_(wb), handles_(handles) {}
+
+ Status PutCF(uint32_t cf, const Slice& key, const Slice& value) override {
+ return wb_->Put(handles_.at(cf), key, value);
+ }
+
+ Status DeleteCF(uint32_t cf, const Slice& key) override {
+ return wb_->Delete(handles_.at(cf), key);
+ }
+
+ Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
+ return wb_->SingleDelete(handles_.at(cf), key);
+ }
+
+ Status MergeCF(uint32_t cf, const Slice& key, const Slice& value) override {
+ return wb_->Merge(handles_.at(cf), key, value);
+ }
+
+ // The only expected 2PC marker is the initial Noop marker.
+ Status MarkNoop(bool empty_batch) override {
+ return empty_batch ? Status::OK() : Status::InvalidArgument();
+ }
+
+ Status MarkBeginPrepare(bool) override { return Status::InvalidArgument(); }
+
+ Status MarkEndPrepare(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+
+ Status MarkCommit(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+
+ Status MarkRollback(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+ };
+
+ // The comparator of the default cf is passed in, similar to the
+ // initialization of TransactionBaseImpl::write_batch_. This comparator is
+ // only used if the write batch encounters an invalid cf id, and falls back to
+ // this comparator.
+ WriteBatchWithIndex wb(wpt_db_->DefaultColumnFamily()->GetComparator(), 0,
+ true, 0);
+ // Swap with write_batch_ so that wb contains the complete write batch. The
+ // actual write batch that will be flushed to DB will be built in
+ // write_batch_, and will be read by FlushWriteBatchToDBInternal.
+ std::swap(wb, write_batch_);
+ TransactionBaseImpl::InitWriteBatch();
+
+ size_t prev_boundary = WriteBatchInternal::kHeader;
+ const bool kPrepared = true;
+ for (size_t i = 0; i < unflushed_save_points_->size() + 1; i++) {
+ bool trailing_batch = i == unflushed_save_points_->size();
+ SavePointBatchHandler sp_handler(&write_batch_,
+ *wupt_db_->GetCFHandleMap().get());
+ size_t curr_boundary = trailing_batch ? wb.GetWriteBatch()->GetDataSize()
+ : (*unflushed_save_points_)[i];
+
+ // Construct the partial write batch up to the savepoint.
+ //
+ // Theoretically, a memcpy between the write batches should be sufficient
+ // since the rewriting into the batch should produce the exact same byte
+ // representation. Rebuilding the WriteBatchWithIndex index is still
+ // necessary though, and would imply doing two passes over the batch though.
+ Status s = WriteBatchInternal::Iterate(wb.GetWriteBatch(), &sp_handler,
+ prev_boundary, curr_boundary);
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (write_batch_.GetWriteBatch()->Count() > 0) {
+ // Flush the write batch.
+ s = FlushWriteBatchToDBInternal(!kPrepared);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+
+ if (!trailing_batch) {
+ if (flushed_save_points_ == nullptr) {
+ flushed_save_points_.reset(
+ new autovector<WriteUnpreparedTxn::SavePoint>());
+ }
+ flushed_save_points_->emplace_back(
+ unprep_seqs_, new ManagedSnapshot(db_impl_, wupt_db_->GetSnapshot()));
+ }
+
+ prev_boundary = curr_boundary;
+ const bool kClear = true;
+ TransactionBaseImpl::InitWriteBatch(kClear);
+ }
+
+ unflushed_save_points_->clear();
+ return Status::OK();
+}
+
+Status WriteUnpreparedTxn::PrepareInternal() {
+ const bool kPrepared = true;
+ return FlushWriteBatchToDB(kPrepared);
+}
+
+Status WriteUnpreparedTxn::CommitWithoutPrepareInternal() {
+ if (unprep_seqs_.empty()) {
+ assert(log_number_ == 0);
+ assert(GetId() == 0);
+ return WritePreparedTxn::CommitWithoutPrepareInternal();
+ }
+
+ // TODO(lth): We should optimize commit without prepare to not perform
+ // a prepare under the hood.
+ auto s = PrepareInternal();
+ if (!s.ok()) {
+ return s;
+ }
+ return CommitInternal();
+}
+
+Status WriteUnpreparedTxn::CommitInternal() {
+ // TODO(lth): Reduce duplicate code with WritePrepared commit logic.
+
+ // We take the commit-time batch and append the Commit marker. The Memtable
+ // will ignore the Commit marker in non-recovery mode
+ WriteBatch* working_batch = GetCommitTimeWriteBatch();
+ const bool empty = working_batch->Count() == 0;
+ WriteBatchInternal::MarkCommit(working_batch, name_);
+
+ const bool for_recovery = use_only_the_last_commit_time_batch_for_recovery_;
+ if (!empty && for_recovery) {
+ // When not writing to memtable, we can still cache the latest write batch.
+ // The cached batch will be written to memtable in WriteRecoverableState
+ // during FlushMemTable
+ WriteBatchInternal::SetAsLastestPersistentState(working_batch);
+ }
+
+ const bool includes_data = !empty && !for_recovery;
+ size_t commit_batch_cnt = 0;
+ if (UNLIKELY(includes_data)) {
+ ROCKS_LOG_WARN(db_impl_->immutable_db_options().info_log,
+ "Duplicate key overhead");
+ SubBatchCounter counter(*wpt_db_->GetCFComparatorMap());
+ auto s = working_batch->Iterate(&counter);
+ assert(s.ok());
+ commit_batch_cnt = counter.BatchCount();
+ }
+ const bool disable_memtable = !includes_data;
+ const bool do_one_write =
+ !db_impl_->immutable_db_options().two_write_queues || disable_memtable;
+
+ WriteUnpreparedCommitEntryPreReleaseCallback update_commit_map(
+ wpt_db_, db_impl_, unprep_seqs_, commit_batch_cnt);
+ const bool kFirstPrepareBatch = true;
+ AddPreparedCallback add_prepared_callback(
+ wpt_db_, db_impl_, commit_batch_cnt,
+ db_impl_->immutable_db_options().two_write_queues, !kFirstPrepareBatch);
+ PreReleaseCallback* pre_release_callback;
+ if (do_one_write) {
+ pre_release_callback = &update_commit_map;
+ } else {
+ pre_release_callback = &add_prepared_callback;
+ }
+ uint64_t seq_used = kMaxSequenceNumber;
+ // Since the prepared batch is directly written to memtable, there is
+ // already a connection between the memtable and its WAL, so there is no
+ // need to redundantly reference the log that contains the prepared data.
+ const uint64_t zero_log_number = 0ull;
+ size_t batch_cnt = UNLIKELY(commit_batch_cnt) ? commit_batch_cnt : 1;
+ auto s = db_impl_->WriteImpl(write_options_, working_batch, nullptr, nullptr,
+ zero_log_number, disable_memtable, &seq_used,
+ batch_cnt, pre_release_callback);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ const SequenceNumber commit_batch_seq = seq_used;
+ if (LIKELY(do_one_write || !s.ok())) {
+ if (LIKELY(s.ok())) {
+ // Note RemovePrepared should be called after WriteImpl that publishsed
+ // the seq. Otherwise SmallestUnCommittedSeq optimization breaks.
+ for (const auto& seq : unprep_seqs_) {
+ wpt_db_->RemovePrepared(seq.first, seq.second);
+ }
+ }
+ if (UNLIKELY(!do_one_write)) {
+ wpt_db_->RemovePrepared(commit_batch_seq, commit_batch_cnt);
+ }
+ unprep_seqs_.clear();
+ flushed_save_points_.reset(nullptr);
+ unflushed_save_points_.reset(nullptr);
+ return s;
+ } // else do the 2nd write to publish seq
+
+ // Populate unprep_seqs_ with commit_batch_seq, since we treat data in the
+ // commit write batch as just another "unprepared" batch. This will also
+ // update the unprep_seqs_ in the update_commit_map callback.
+ unprep_seqs_[commit_batch_seq] = commit_batch_cnt;
+
+ // Note: the 2nd write comes with a performance penality. So if we have too
+ // many of commits accompanied with ComitTimeWriteBatch and yet we cannot
+ // enable use_only_the_last_commit_time_batch_for_recovery_ optimization,
+ // two_write_queues should be disabled to avoid many additional writes here.
+
+ // Update commit map only from the 2nd queue
+ WriteBatch empty_batch;
+ empty_batch.PutLogData(Slice());
+ // In the absence of Prepare markers, use Noop as a batch separator
+ WriteBatchInternal::InsertNoop(&empty_batch);
+ const bool DISABLE_MEMTABLE = true;
+ const size_t ONE_BATCH = 1;
+ const uint64_t NO_REF_LOG = 0;
+ s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr,
+ NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
+ &update_commit_map);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ // Note RemovePrepared should be called after WriteImpl that publishsed the
+ // seq. Otherwise SmallestUnCommittedSeq optimization breaks.
+ for (const auto& seq : unprep_seqs_) {
+ wpt_db_->RemovePrepared(seq.first, seq.second);
+ }
+ unprep_seqs_.clear();
+ flushed_save_points_.reset(nullptr);
+ unflushed_save_points_.reset(nullptr);
+ return s;
+}
+
+Status WriteUnpreparedTxn::WriteRollbackKeys(
+ const TransactionKeyMap& tracked_keys, WriteBatchWithIndex* rollback_batch,
+ ReadCallback* callback, const ReadOptions& roptions) {
+ const auto& cf_map = *wupt_db_->GetCFHandleMap();
+ auto WriteRollbackKey = [&](const std::string& key, uint32_t cfid) {
+ const auto& cf_handle = cf_map.at(cfid);
+ PinnableSlice pinnable_val;
+ bool not_used;
+ DBImpl::GetImplOptions get_impl_options;
+ get_impl_options.column_family = cf_handle;
+ get_impl_options.value = &pinnable_val;
+ get_impl_options.value_found = &not_used;
+ get_impl_options.callback = callback;
+ auto s = db_impl_->GetImpl(roptions, key, get_impl_options);
+
+ if (s.ok()) {
+ s = rollback_batch->Put(cf_handle, key, pinnable_val);
+ assert(s.ok());
+ } else if (s.IsNotFound()) {
+ s = rollback_batch->Delete(cf_handle, key);
+ assert(s.ok());
+ } else {
+ return s;
+ }
+
+ return Status::OK();
+ };
+
+ for (const auto& cfkey : tracked_keys) {
+ const auto cfid = cfkey.first;
+ const auto& keys = cfkey.second;
+ for (const auto& pair : keys) {
+ auto s = WriteRollbackKey(pair.first, cfid);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ }
+
+ for (const auto& cfkey : untracked_keys_) {
+ const auto cfid = cfkey.first;
+ const auto& keys = cfkey.second;
+ for (const auto& key : keys) {
+ auto s = WriteRollbackKey(key, cfid);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ }
+
+ return Status::OK();
+}
+
+Status WriteUnpreparedTxn::RollbackInternal() {
+ // TODO(lth): Reduce duplicate code with WritePrepared rollback logic.
+ WriteBatchWithIndex rollback_batch(
+ wpt_db_->DefaultColumnFamily()->GetComparator(), 0, true, 0);
+ assert(GetId() != kMaxSequenceNumber);
+ assert(GetId() > 0);
+ Status s;
+ auto read_at_seq = kMaxSequenceNumber;
+ ReadOptions roptions;
+ // to prevent callback's seq to be overrriden inside DBImpk::Get
+ roptions.snapshot = wpt_db_->GetMaxSnapshot();
+ // Note that we do not use WriteUnpreparedTxnReadCallback because we do not
+ // need to read our own writes when reading prior versions of the key for
+ // rollback.
+ WritePreparedTxnReadCallback callback(wpt_db_, read_at_seq);
+ WriteRollbackKeys(GetTrackedKeys(), &rollback_batch, &callback, roptions);
+
+ // The Rollback marker will be used as a batch separator
+ WriteBatchInternal::MarkRollback(rollback_batch.GetWriteBatch(), name_);
+ bool do_one_write = !db_impl_->immutable_db_options().two_write_queues;
+ const bool DISABLE_MEMTABLE = true;
+ const uint64_t NO_REF_LOG = 0;
+ uint64_t seq_used = kMaxSequenceNumber;
+ // TODO(lth): We write rollback batch all in a single batch here, but this
+ // should be subdivded into multiple batches as well. In phase 2, when key
+ // sets are read from WAL, this will happen naturally.
+ const size_t ONE_BATCH = 1;
+ // We commit the rolled back prepared batches. ALthough this is
+ // counter-intuitive, i) it is safe to do so, since the prepared batches are
+ // already canceled out by the rollback batch, ii) adding the commit entry to
+ // CommitCache will allow us to benefit from the existing mechanism in
+ // CommitCache that keeps an entry evicted due to max advance and yet overlaps
+ // with a live snapshot around so that the live snapshot properly skips the
+ // entry even if its prepare seq is lower than max_evicted_seq_.
+ WriteUnpreparedCommitEntryPreReleaseCallback update_commit_map(
+ wpt_db_, db_impl_, unprep_seqs_, ONE_BATCH);
+ // Note: the rollback batch does not need AddPrepared since it is written to
+ // DB in one shot. min_uncommitted still works since it requires capturing
+ // data that is written to DB but not yet committed, while the roolback
+ // batch commits with PreReleaseCallback.
+ s = db_impl_->WriteImpl(write_options_, rollback_batch.GetWriteBatch(),
+ nullptr, nullptr, NO_REF_LOG, !DISABLE_MEMTABLE,
+ &seq_used, rollback_batch.SubBatchCnt(),
+ do_one_write ? &update_commit_map : nullptr);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ if (!s.ok()) {
+ return s;
+ }
+ if (do_one_write) {
+ for (const auto& seq : unprep_seqs_) {
+ wpt_db_->RemovePrepared(seq.first, seq.second);
+ }
+ unprep_seqs_.clear();
+ flushed_save_points_.reset(nullptr);
+ unflushed_save_points_.reset(nullptr);
+ return s;
+ } // else do the 2nd write for commit
+ uint64_t& prepare_seq = seq_used;
+ ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
+ "RollbackInternal 2nd write prepare_seq: %" PRIu64,
+ prepare_seq);
+ // Commit the batch by writing an empty batch to the queue that will release
+ // the commit sequence number to readers.
+ WriteUnpreparedRollbackPreReleaseCallback update_commit_map_with_prepare(
+ wpt_db_, db_impl_, unprep_seqs_, prepare_seq);
+ WriteBatch empty_batch;
+ empty_batch.PutLogData(Slice());
+ // In the absence of Prepare markers, use Noop as a batch separator
+ WriteBatchInternal::InsertNoop(&empty_batch);
+ s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr,
+ NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
+ &update_commit_map_with_prepare);
+ assert(!s.ok() || seq_used != kMaxSequenceNumber);
+ // Mark the txn as rolled back
+ if (s.ok()) {
+ for (const auto& seq : unprep_seqs_) {
+ wpt_db_->RemovePrepared(seq.first, seq.second);
+ }
+ }
+
+ unprep_seqs_.clear();
+ flushed_save_points_.reset(nullptr);
+ unflushed_save_points_.reset(nullptr);
+ return s;
+}
+
+void WriteUnpreparedTxn::Clear() {
+ if (!recovered_txn_) {
+ txn_db_impl_->UnLock(this, &GetTrackedKeys());
+ }
+ unprep_seqs_.clear();
+ flushed_save_points_.reset(nullptr);
+ unflushed_save_points_.reset(nullptr);
+ recovered_txn_ = false;
+ largest_validated_seq_ = 0;
+ assert(active_iterators_.empty());
+ active_iterators_.clear();
+ untracked_keys_.clear();
+ TransactionBaseImpl::Clear();
+}
+
+void WriteUnpreparedTxn::SetSavePoint() {
+ assert((unflushed_save_points_ ? unflushed_save_points_->size() : 0) +
+ (flushed_save_points_ ? flushed_save_points_->size() : 0) ==
+ (save_points_ ? save_points_->size() : 0));
+ PessimisticTransaction::SetSavePoint();
+ if (unflushed_save_points_ == nullptr) {
+ unflushed_save_points_.reset(new autovector<size_t>());
+ }
+ unflushed_save_points_->push_back(write_batch_.GetDataSize());
+}
+
+Status WriteUnpreparedTxn::RollbackToSavePoint() {
+ assert((unflushed_save_points_ ? unflushed_save_points_->size() : 0) +
+ (flushed_save_points_ ? flushed_save_points_->size() : 0) ==
+ (save_points_ ? save_points_->size() : 0));
+ if (unflushed_save_points_ != nullptr && unflushed_save_points_->size() > 0) {
+ Status s = PessimisticTransaction::RollbackToSavePoint();
+ assert(!s.IsNotFound());
+ unflushed_save_points_->pop_back();
+ return s;
+ }
+
+ if (flushed_save_points_ != nullptr && !flushed_save_points_->empty()) {
+ return RollbackToSavePointInternal();
+ }
+
+ return Status::NotFound();
+}
+
+Status WriteUnpreparedTxn::RollbackToSavePointInternal() {
+ Status s;
+
+ const bool kClear = true;
+ TransactionBaseImpl::InitWriteBatch(kClear);
+
+ assert(flushed_save_points_->size() > 0);
+ WriteUnpreparedTxn::SavePoint& top = flushed_save_points_->back();
+
+ assert(save_points_ != nullptr && save_points_->size() > 0);
+ const TransactionKeyMap& tracked_keys = save_points_->top().new_keys_;
+
+ ReadOptions roptions;
+ roptions.snapshot = top.snapshot_->snapshot();
+ SequenceNumber min_uncommitted =
+ static_cast_with_check<const SnapshotImpl, const Snapshot>(
+ roptions.snapshot)
+ ->min_uncommitted_;
+ SequenceNumber snap_seq = roptions.snapshot->GetSequenceNumber();
+ WriteUnpreparedTxnReadCallback callback(wupt_db_, snap_seq, min_uncommitted,
+ top.unprep_seqs_,
+ kBackedByDBSnapshot);
+ WriteRollbackKeys(tracked_keys, &write_batch_, &callback, roptions);
+
+ const bool kPrepared = true;
+ s = FlushWriteBatchToDBInternal(!kPrepared);
+ assert(s.ok());
+ if (!s.ok()) {
+ return s;
+ }
+
+ // PessimisticTransaction::RollbackToSavePoint will call also call
+ // RollbackToSavepoint on write_batch_. However, write_batch_ is empty and has
+ // no savepoints because this savepoint has already been flushed. Work around
+ // this by setting a fake savepoint.
+ write_batch_.SetSavePoint();
+ s = PessimisticTransaction::RollbackToSavePoint();
+ assert(s.ok());
+ if (!s.ok()) {
+ return s;
+ }
+
+ flushed_save_points_->pop_back();
+ return s;
+}
+
+Status WriteUnpreparedTxn::PopSavePoint() {
+ assert((unflushed_save_points_ ? unflushed_save_points_->size() : 0) +
+ (flushed_save_points_ ? flushed_save_points_->size() : 0) ==
+ (save_points_ ? save_points_->size() : 0));
+ if (unflushed_save_points_ != nullptr && unflushed_save_points_->size() > 0) {
+ Status s = PessimisticTransaction::PopSavePoint();
+ assert(!s.IsNotFound());
+ unflushed_save_points_->pop_back();
+ return s;
+ }
+
+ if (flushed_save_points_ != nullptr && !flushed_save_points_->empty()) {
+ // PessimisticTransaction::PopSavePoint will call also call PopSavePoint on
+ // write_batch_. However, write_batch_ is empty and has no savepoints
+ // because this savepoint has already been flushed. Work around this by
+ // setting a fake savepoint.
+ write_batch_.SetSavePoint();
+ Status s = PessimisticTransaction::PopSavePoint();
+ assert(!s.IsNotFound());
+ flushed_save_points_->pop_back();
+ return s;
+ }
+
+ return Status::NotFound();
+}
+
+void WriteUnpreparedTxn::MultiGet(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ const size_t num_keys, const Slice* keys,
+ PinnableSlice* values, Status* statuses,
+ const bool sorted_input) {
+ SequenceNumber min_uncommitted, snap_seq;
+ const SnapshotBackup backed_by_snapshot =
+ wupt_db_->AssignMinMaxSeqs(options.snapshot, &min_uncommitted, &snap_seq);
+ WriteUnpreparedTxnReadCallback callback(wupt_db_, snap_seq, min_uncommitted,
+ unprep_seqs_, backed_by_snapshot);
+ write_batch_.MultiGetFromBatchAndDB(db_, options, column_family, num_keys,
+ keys, values, statuses, sorted_input,
+ &callback);
+ if (UNLIKELY(!callback.valid() ||
+ !wupt_db_->ValidateSnapshot(snap_seq, backed_by_snapshot))) {
+ wupt_db_->WPRecordTick(TXN_GET_TRY_AGAIN);
+ for (size_t i = 0; i < num_keys; i++) {
+ statuses[i] = Status::TryAgain();
+ }
+ }
+}
+
+Status WriteUnpreparedTxn::Get(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ const Slice& key, PinnableSlice* value) {
+ SequenceNumber min_uncommitted, snap_seq;
+ const SnapshotBackup backed_by_snapshot =
+ wupt_db_->AssignMinMaxSeqs(options.snapshot, &min_uncommitted, &snap_seq);
+ WriteUnpreparedTxnReadCallback callback(wupt_db_, snap_seq, min_uncommitted,
+ unprep_seqs_, backed_by_snapshot);
+ auto res = write_batch_.GetFromBatchAndDB(db_, options, column_family, key,
+ value, &callback);
+ if (LIKELY(callback.valid() &&
+ wupt_db_->ValidateSnapshot(snap_seq, backed_by_snapshot))) {
+ return res;
+ } else {
+ wupt_db_->WPRecordTick(TXN_GET_TRY_AGAIN);
+ return Status::TryAgain();
+ }
+}
+
+namespace {
+static void CleanupWriteUnpreparedWBWIIterator(void* arg1, void* arg2) {
+ auto txn = reinterpret_cast<WriteUnpreparedTxn*>(arg1);
+ auto iter = reinterpret_cast<Iterator*>(arg2);
+ txn->RemoveActiveIterator(iter);
+}
+} // anonymous namespace
+
+Iterator* WriteUnpreparedTxn::GetIterator(const ReadOptions& options) {
+ return GetIterator(options, wupt_db_->DefaultColumnFamily());
+}
+
+Iterator* WriteUnpreparedTxn::GetIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family) {
+ // Make sure to get iterator from WriteUnprepareTxnDB, not the root db.
+ Iterator* db_iter = wupt_db_->NewIterator(options, column_family, this);
+ assert(db_iter);
+
+ auto iter = write_batch_.NewIteratorWithBase(column_family, db_iter);
+ active_iterators_.push_back(iter);
+ iter->RegisterCleanup(CleanupWriteUnpreparedWBWIIterator, this, iter);
+ return iter;
+}
+
+Status WriteUnpreparedTxn::ValidateSnapshot(ColumnFamilyHandle* column_family,
+ const Slice& key,
+ SequenceNumber* tracked_at_seq) {
+ // TODO(lth): Reduce duplicate code with WritePrepared ValidateSnapshot logic.
+ assert(snapshot_);
+
+ SequenceNumber min_uncommitted =
+ static_cast_with_check<const SnapshotImpl, const Snapshot>(
+ snapshot_.get())
+ ->min_uncommitted_;
+ SequenceNumber snap_seq = snapshot_->GetSequenceNumber();
+ // tracked_at_seq is either max or the last snapshot with which this key was
+ // trackeed so there is no need to apply the IsInSnapshot to this comparison
+ // here as tracked_at_seq is not a prepare seq.
+ if (*tracked_at_seq <= snap_seq) {
+ // If the key has been previous validated at a sequence number earlier
+ // than the curent snapshot's sequence number, we already know it has not
+ // been modified.
+ return Status::OK();
+ }
+
+ *tracked_at_seq = snap_seq;
+
+ ColumnFamilyHandle* cfh =
+ column_family ? column_family : db_impl_->DefaultColumnFamily();
+
+ WriteUnpreparedTxnReadCallback snap_checker(
+ wupt_db_, snap_seq, min_uncommitted, unprep_seqs_, kBackedByDBSnapshot);
+ return TransactionUtil::CheckKeyForConflicts(db_impl_, cfh, key.ToString(),
+ snap_seq, false /* cache_only */,
+ &snap_checker, min_uncommitted);
+}
+
+const std::map<SequenceNumber, size_t>&
+WriteUnpreparedTxn::GetUnpreparedSequenceNumbers() {
+ return unprep_seqs_;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_unprepared_txn.h b/src/rocksdb/utilities/transactions/write_unprepared_txn.h
new file mode 100644
index 000000000..30c8f4c55
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_unprepared_txn.h
@@ -0,0 +1,341 @@
+// 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 <set>
+
+#include "utilities/transactions/write_prepared_txn.h"
+#include "utilities/transactions/write_unprepared_txn_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class WriteUnpreparedTxnDB;
+class WriteUnpreparedTxn;
+
+// WriteUnprepared transactions needs to be able to read their own uncommitted
+// writes, and supporting this requires some careful consideration. Because
+// writes in the current transaction may be flushed to DB already, we cannot
+// rely on the contents of WriteBatchWithIndex to determine whether a key should
+// be visible or not, so we have to remember to check the DB for any uncommitted
+// keys that should be visible to us. First, we will need to change the seek to
+// snapshot logic, to seek to max_visible_seq = max(snap_seq, max_unprep_seq).
+// Any key greater than max_visible_seq should not be visible because they
+// cannot be unprepared by the current transaction and they are not in its
+// snapshot.
+//
+// When we seek to max_visible_seq, one of these cases will happen:
+// 1. We hit a unprepared key from the current transaction.
+// 2. We hit a unprepared key from the another transaction.
+// 3. We hit a committed key with snap_seq < seq < max_unprep_seq.
+// 4. We hit a committed key with seq <= snap_seq.
+//
+// IsVisibleFullCheck handles all cases correctly.
+//
+// Other notes:
+// Note that max_visible_seq is only calculated once at iterator construction
+// time, meaning if the same transaction is adding more unprep seqs through
+// writes during iteration, these newer writes may not be visible. This is not a
+// problem for MySQL though because it avoids modifying the index as it is
+// scanning through it to avoid the Halloween Problem. Instead, it scans the
+// index once up front, and modifies based on a temporary copy.
+//
+// In DBIter, there is a "reseek" optimization if the iterator skips over too
+// many keys. However, this assumes that the reseek seeks exactly to the
+// required key. In write unprepared, even after seeking directly to
+// max_visible_seq, some iteration may be required before hitting a visible key,
+// and special precautions must be taken to avoid performing another reseek,
+// leading to an infinite loop.
+//
+class WriteUnpreparedTxnReadCallback : public ReadCallback {
+ public:
+ WriteUnpreparedTxnReadCallback(
+ WritePreparedTxnDB* db, SequenceNumber snapshot,
+ SequenceNumber min_uncommitted,
+ const std::map<SequenceNumber, size_t>& unprep_seqs,
+ SnapshotBackup backed_by_snapshot)
+ // Pass our last uncommitted seq as the snapshot to the parent class to
+ // ensure that the parent will not prematurely filter out own writes. We
+ // will do the exact comparison against snapshots in IsVisibleFullCheck
+ // override.
+ : ReadCallback(CalcMaxVisibleSeq(unprep_seqs, snapshot), min_uncommitted),
+ db_(db),
+ unprep_seqs_(unprep_seqs),
+ wup_snapshot_(snapshot),
+ backed_by_snapshot_(backed_by_snapshot) {
+ (void)backed_by_snapshot_; // to silence unused private field warning
+ }
+
+ virtual ~WriteUnpreparedTxnReadCallback() {
+ // If it is not backed by snapshot, the caller must check validity
+ assert(valid_checked_ || backed_by_snapshot_ == kBackedByDBSnapshot);
+ }
+
+ virtual bool IsVisibleFullCheck(SequenceNumber seq) override;
+
+ inline bool valid() {
+ valid_checked_ = true;
+ return snap_released_ == false;
+ }
+
+ void Refresh(SequenceNumber seq) override {
+ max_visible_seq_ = std::max(max_visible_seq_, seq);
+ wup_snapshot_ = seq;
+ }
+
+ static SequenceNumber CalcMaxVisibleSeq(
+ const std::map<SequenceNumber, size_t>& unprep_seqs,
+ SequenceNumber snapshot_seq) {
+ SequenceNumber max_unprepared = 0;
+ if (unprep_seqs.size()) {
+ max_unprepared =
+ unprep_seqs.rbegin()->first + unprep_seqs.rbegin()->second - 1;
+ }
+ return std::max(max_unprepared, snapshot_seq);
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ const std::map<SequenceNumber, size_t>& unprep_seqs_;
+ SequenceNumber wup_snapshot_;
+ // Whether max_visible_seq_ is backed by a snapshot
+ const SnapshotBackup backed_by_snapshot_;
+ bool snap_released_ = false;
+ // Safety check to ensure that the caller has checked invalid statuses
+ bool valid_checked_ = false;
+};
+
+class WriteUnpreparedTxn : public WritePreparedTxn {
+ public:
+ WriteUnpreparedTxn(WriteUnpreparedTxnDB* db,
+ const WriteOptions& write_options,
+ const TransactionOptions& txn_options);
+
+ virtual ~WriteUnpreparedTxn();
+
+ using TransactionBaseImpl::Put;
+ virtual Status Put(ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& value,
+ const bool assume_tracked = false) override;
+ virtual Status Put(ColumnFamilyHandle* column_family, const SliceParts& key,
+ const SliceParts& value,
+ const bool assume_tracked = false) override;
+
+ using TransactionBaseImpl::Merge;
+ virtual Status Merge(ColumnFamilyHandle* column_family, const Slice& key,
+ const Slice& value,
+ const bool assume_tracked = false) override;
+
+ using TransactionBaseImpl::Delete;
+ virtual Status Delete(ColumnFamilyHandle* column_family, const Slice& key,
+ const bool assume_tracked = false) override;
+ virtual Status Delete(ColumnFamilyHandle* column_family,
+ const SliceParts& key,
+ const bool assume_tracked = false) override;
+
+ using TransactionBaseImpl::SingleDelete;
+ virtual Status SingleDelete(ColumnFamilyHandle* column_family,
+ const Slice& key,
+ const bool assume_tracked = false) override;
+ virtual Status SingleDelete(ColumnFamilyHandle* column_family,
+ const SliceParts& key,
+ const bool assume_tracked = false) override;
+
+ // In WriteUnprepared, untracked writes will break snapshot validation logic.
+ // Snapshot validation will only check the largest sequence number of a key to
+ // see if it was committed or not. However, an untracked unprepared write will
+ // hide smaller committed sequence numbers.
+ //
+ // TODO(lth): Investigate whether it is worth having snapshot validation
+ // validate all values larger than snap_seq. Otherwise, we should return
+ // Status::NotSupported for untracked writes.
+
+ virtual Status RebuildFromWriteBatch(WriteBatch*) override;
+
+ virtual uint64_t GetLastLogNumber() const override {
+ return last_log_number_;
+ }
+
+ void RemoveActiveIterator(Iterator* iter) {
+ active_iterators_.erase(
+ std::remove(active_iterators_.begin(), active_iterators_.end(), iter),
+ active_iterators_.end());
+ }
+
+ protected:
+ void Initialize(const TransactionOptions& txn_options) override;
+
+ Status PrepareInternal() override;
+
+ Status CommitWithoutPrepareInternal() override;
+ Status CommitInternal() override;
+
+ Status RollbackInternal() override;
+
+ void Clear() override;
+
+ void SetSavePoint() override;
+ Status RollbackToSavePoint() override;
+ Status PopSavePoint() override;
+
+ // Get and GetIterator needs to be overridden so that a ReadCallback to
+ // handle read-your-own-write is used.
+ using Transaction::Get;
+ virtual Status Get(const ReadOptions& options,
+ ColumnFamilyHandle* column_family, const Slice& key,
+ PinnableSlice* value) override;
+
+ using Transaction::MultiGet;
+ virtual void MultiGet(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ const size_t num_keys, const Slice* keys,
+ PinnableSlice* values, Status* statuses,
+ const bool sorted_input = false) override;
+
+ using Transaction::GetIterator;
+ virtual Iterator* GetIterator(const ReadOptions& options) override;
+ virtual Iterator* GetIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family) override;
+
+ virtual Status ValidateSnapshot(ColumnFamilyHandle* column_family,
+ const Slice& key,
+ SequenceNumber* tracked_at_seq) override;
+
+ private:
+ friend class WriteUnpreparedTransactionTest_ReadYourOwnWrite_Test;
+ friend class WriteUnpreparedTransactionTest_RecoveryTest_Test;
+ friend class WriteUnpreparedTransactionTest_UnpreparedBatch_Test;
+ friend class WriteUnpreparedTxnDB;
+
+ const std::map<SequenceNumber, size_t>& GetUnpreparedSequenceNumbers();
+ Status WriteRollbackKeys(const TransactionKeyMap& tracked_keys,
+ WriteBatchWithIndex* rollback_batch,
+ ReadCallback* callback, const ReadOptions& roptions);
+
+ Status MaybeFlushWriteBatchToDB();
+ Status FlushWriteBatchToDB(bool prepared);
+ Status FlushWriteBatchToDBInternal(bool prepared);
+ Status FlushWriteBatchWithSavePointToDB();
+ Status RollbackToSavePointInternal();
+ Status HandleWrite(std::function<Status()> do_write);
+
+ // For write unprepared, we check on every writebatch append to see if
+ // write_batch_flush_threshold_ has been exceeded, and then call
+ // FlushWriteBatchToDB if so. This logic is encapsulated in
+ // MaybeFlushWriteBatchToDB.
+ int64_t write_batch_flush_threshold_;
+ WriteUnpreparedTxnDB* wupt_db_;
+
+ // Ordered list of unprep_seq sequence numbers that we have already written
+ // to DB.
+ //
+ // This maps unprep_seq => prepare_batch_cnt for each unprepared batch
+ // written by this transaction.
+ //
+ // Note that this contains both prepared and unprepared batches, since they
+ // are treated similarily in prepare heap/commit map, so it simplifies the
+ // commit callbacks.
+ std::map<SequenceNumber, size_t> unprep_seqs_;
+
+ uint64_t last_log_number_;
+
+ // Recovered transactions have tracked_keys_ populated, but are not actually
+ // locked for efficiency reasons. For recovered transactions, skip unlocking
+ // keys when transaction ends.
+ bool recovered_txn_;
+
+ // Track the largest sequence number at which we performed snapshot
+ // validation. If snapshot validation was skipped because no snapshot was set,
+ // then this is set to GetLastPublishedSequence. This value is useful because
+ // it means that for keys that have unprepared seqnos, we can guarantee that
+ // no committed keys by other transactions can exist between
+ // largest_validated_seq_ and max_unprep_seq. See
+ // WriteUnpreparedTxnDB::NewIterator for an explanation for why this is
+ // necessary for iterator Prev().
+ //
+ // Currently this value only increases during the lifetime of a transaction,
+ // but in some cases, we should be able to restore the previously largest
+ // value when calling RollbackToSavepoint.
+ SequenceNumber largest_validated_seq_;
+
+ using KeySet = std::unordered_map<uint32_t, std::vector<std::string>>;
+ struct SavePoint {
+ // Record of unprep_seqs_ at this savepoint. The set of unprep_seq is
+ // used during RollbackToSavepoint to determine visibility when restoring
+ // old values.
+ //
+ // TODO(lth): Since all unprep_seqs_ sets further down the stack must be
+ // subsets, this can potentially be deduplicated by just storing set
+ // difference. Investigate if this is worth it.
+ std::map<SequenceNumber, size_t> unprep_seqs_;
+
+ // This snapshot will be used to read keys at this savepoint if we call
+ // RollbackToSavePoint.
+ std::unique_ptr<ManagedSnapshot> snapshot_;
+
+ SavePoint(const std::map<SequenceNumber, size_t>& seqs,
+ ManagedSnapshot* snapshot)
+ : unprep_seqs_(seqs), snapshot_(snapshot){};
+ };
+
+ // We have 3 data structures holding savepoint information:
+ // 1. TransactionBaseImpl::save_points_
+ // 2. WriteUnpreparedTxn::flushed_save_points_
+ // 3. WriteUnpreparecTxn::unflushed_save_points_
+ //
+ // TransactionBaseImpl::save_points_ holds information about all write
+ // batches, including the current in-memory write_batch_, or unprepared
+ // batches that have been written out. Its responsibility is just to track
+ // which keys have been modified in every savepoint.
+ //
+ // WriteUnpreparedTxn::flushed_save_points_ holds information about savepoints
+ // set on unprepared batches that have already flushed. It holds the snapshot
+ // and unprep_seqs at that savepoint, so that the rollback process can
+ // determine which keys were visible at that point in time.
+ //
+ // WriteUnpreparecTxn::unflushed_save_points_ holds information about
+ // savepoints on the current in-memory write_batch_. It simply records the
+ // size of the write batch at every savepoint.
+ //
+ // TODO(lth): Remove the redundancy between save_point_boundaries_ and
+ // write_batch_.save_points_.
+ //
+ // Based on this information, here are some invariants:
+ // size(unflushed_save_points_) = size(write_batch_.save_points_)
+ // size(flushed_save_points_) + size(unflushed_save_points_)
+ // = size(save_points_)
+ //
+ std::unique_ptr<autovector<WriteUnpreparedTxn::SavePoint>>
+ flushed_save_points_;
+ std::unique_ptr<autovector<size_t>> unflushed_save_points_;
+
+ // It is currently unsafe to flush a write batch if there are active iterators
+ // created from this transaction. This is because we use WriteBatchWithIndex
+ // to do merging reads from the DB and the write batch. If we flush the write
+ // batch, it is possible that the delta iterator on the iterator will point to
+ // invalid memory.
+ std::vector<Iterator*> active_iterators_;
+
+ // Untracked keys that we have to rollback.
+ //
+ // TODO(lth): Currently we we do not record untracked keys per-savepoint.
+ // This means that when rolling back to savepoints, we have to check all
+ // keys in the current transaction for rollback. Note that this is only
+ // inefficient, but still correct because we take a snapshot at every
+ // savepoint, and we will use that snapshot to construct the rollback batch.
+ // The rollback batch will then contain a reissue of the same marker.
+ //
+ // A more optimal solution would be to only check keys changed since the
+ // last savepoint. Also, it may make sense to merge this into tracked_keys_
+ // and differentiate between tracked but not locked keys to avoid having two
+ // very similar data structures.
+ KeySet untracked_keys_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_unprepared_txn_db.cc b/src/rocksdb/utilities/transactions/write_unprepared_txn_db.cc
new file mode 100644
index 000000000..ca365d044
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_unprepared_txn_db.cc
@@ -0,0 +1,468 @@
+// 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/write_unprepared_txn_db.h"
+#include "db/arena_wrapped_db_iter.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "util/cast_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Instead of reconstructing a Transaction object, and calling rollback on it,
+// we can be more efficient with RollbackRecoveredTransaction by skipping
+// unnecessary steps (eg. updating CommitMap, reconstructing keyset)
+Status WriteUnpreparedTxnDB::RollbackRecoveredTransaction(
+ const DBImpl::RecoveredTransaction* rtxn) {
+ // TODO(lth): Reduce duplicate code with WritePrepared rollback logic.
+ assert(rtxn->unprepared_);
+ auto cf_map_shared_ptr = WritePreparedTxnDB::GetCFHandleMap();
+ auto cf_comp_map_shared_ptr = WritePreparedTxnDB::GetCFComparatorMap();
+ // In theory we could write with disableWAL = true during recovery, and
+ // assume that if we crash again during recovery, we can just replay from
+ // the very beginning. Unfortunately, the XIDs from the application may not
+ // necessarily be unique across restarts, potentially leading to situations
+ // like this:
+ //
+ // BEGIN_PREPARE(unprepared) Put(a) END_PREPARE(xid = 1)
+ // -- crash and recover with Put(a) rolled back as it was not prepared
+ // BEGIN_PREPARE(prepared) Put(b) END_PREPARE(xid = 1)
+ // COMMIT(xid = 1)
+ // -- crash and recover with both a, b
+ //
+ // We could just write the rollback marker, but then we would have to extend
+ // MemTableInserter during recovery to actually do writes into the DB
+ // instead of just dropping the in-memory write batch.
+ //
+ WriteOptions w_options;
+
+ class InvalidSnapshotReadCallback : public ReadCallback {
+ public:
+ InvalidSnapshotReadCallback(SequenceNumber snapshot)
+ : ReadCallback(snapshot) {}
+
+ inline bool IsVisibleFullCheck(SequenceNumber) override {
+ // The seq provided as snapshot is the seq right before we have locked and
+ // wrote to it, so whatever is there, it is committed.
+ return true;
+ }
+
+ // Ignore the refresh request since we are confident that our snapshot seq
+ // is not going to be affected by concurrent compactions (not enabled yet.)
+ void Refresh(SequenceNumber) override {}
+ };
+
+ // Iterate starting with largest sequence number.
+ for (auto it = rtxn->batches_.rbegin(); it != rtxn->batches_.rend(); ++it) {
+ auto last_visible_txn = it->first - 1;
+ const auto& batch = it->second.batch_;
+ WriteBatch rollback_batch;
+
+ struct RollbackWriteBatchBuilder : public WriteBatch::Handler {
+ DBImpl* db_;
+ ReadOptions roptions;
+ InvalidSnapshotReadCallback callback;
+ WriteBatch* rollback_batch_;
+ std::map<uint32_t, const Comparator*>& comparators_;
+ std::map<uint32_t, ColumnFamilyHandle*>& handles_;
+ using CFKeys = std::set<Slice, SetComparator>;
+ std::map<uint32_t, CFKeys> keys_;
+ bool rollback_merge_operands_;
+ RollbackWriteBatchBuilder(
+ DBImpl* db, SequenceNumber snap_seq, WriteBatch* dst_batch,
+ std::map<uint32_t, const Comparator*>& comparators,
+ std::map<uint32_t, ColumnFamilyHandle*>& handles,
+ bool rollback_merge_operands)
+ : db_(db),
+ callback(snap_seq),
+ // disable min_uncommitted optimization
+ rollback_batch_(dst_batch),
+ comparators_(comparators),
+ handles_(handles),
+ rollback_merge_operands_(rollback_merge_operands) {}
+
+ Status Rollback(uint32_t cf, const Slice& key) {
+ Status s;
+ CFKeys& cf_keys = keys_[cf];
+ if (cf_keys.size() == 0) { // just inserted
+ auto cmp = comparators_[cf];
+ keys_[cf] = CFKeys(SetComparator(cmp));
+ }
+ auto res = cf_keys.insert(key);
+ if (res.second ==
+ false) { // second is false if a element already existed.
+ return s;
+ }
+
+ PinnableSlice pinnable_val;
+ bool not_used;
+ auto cf_handle = handles_[cf];
+ DBImpl::GetImplOptions get_impl_options;
+ get_impl_options.column_family = cf_handle;
+ get_impl_options.value = &pinnable_val;
+ get_impl_options.value_found = &not_used;
+ get_impl_options.callback = &callback;
+ s = db_->GetImpl(roptions, key, get_impl_options);
+ assert(s.ok() || s.IsNotFound());
+ if (s.ok()) {
+ s = rollback_batch_->Put(cf_handle, key, pinnable_val);
+ assert(s.ok());
+ } else if (s.IsNotFound()) {
+ // There has been no readable value before txn. By adding a delete we
+ // make sure that there will be none afterwards either.
+ s = rollback_batch_->Delete(cf_handle, key);
+ assert(s.ok());
+ } else {
+ // Unexpected status. Return it to the user.
+ }
+ return s;
+ }
+
+ Status PutCF(uint32_t cf, const Slice& key,
+ const Slice& /*val*/) override {
+ return Rollback(cf, key);
+ }
+
+ Status DeleteCF(uint32_t cf, const Slice& key) override {
+ return Rollback(cf, key);
+ }
+
+ Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
+ return Rollback(cf, key);
+ }
+
+ Status MergeCF(uint32_t cf, const Slice& key,
+ const Slice& /*val*/) override {
+ if (rollback_merge_operands_) {
+ return Rollback(cf, key);
+ } else {
+ return Status::OK();
+ }
+ }
+
+ // Recovered batches do not contain 2PC markers.
+ Status MarkNoop(bool) override { return Status::InvalidArgument(); }
+ Status MarkBeginPrepare(bool) override {
+ return Status::InvalidArgument();
+ }
+ Status MarkEndPrepare(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+ Status MarkCommit(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+ Status MarkRollback(const Slice&) override {
+ return Status::InvalidArgument();
+ }
+ } rollback_handler(db_impl_, last_visible_txn, &rollback_batch,
+ *cf_comp_map_shared_ptr.get(), *cf_map_shared_ptr.get(),
+ txn_db_options_.rollback_merge_operands);
+
+ auto s = batch->Iterate(&rollback_handler);
+ if (!s.ok()) {
+ return s;
+ }
+
+ // The Rollback marker will be used as a batch separator
+ WriteBatchInternal::MarkRollback(&rollback_batch, rtxn->name_);
+
+ const uint64_t kNoLogRef = 0;
+ const bool kDisableMemtable = true;
+ const size_t kOneBatch = 1;
+ uint64_t seq_used = kMaxSequenceNumber;
+ s = db_impl_->WriteImpl(w_options, &rollback_batch, nullptr, nullptr,
+ kNoLogRef, !kDisableMemtable, &seq_used, kOneBatch);
+ if (!s.ok()) {
+ return s;
+ }
+
+ // If two_write_queues, we must manually release the sequence number to
+ // readers.
+ if (db_impl_->immutable_db_options().two_write_queues) {
+ db_impl_->SetLastPublishedSequence(seq_used);
+ }
+ }
+
+ return Status::OK();
+}
+
+Status WriteUnpreparedTxnDB::Initialize(
+ const std::vector<size_t>& compaction_enabled_cf_indices,
+ const std::vector<ColumnFamilyHandle*>& handles) {
+ // TODO(lth): Reduce code duplication in this function.
+ auto dbimpl = static_cast_with_check<DBImpl, DB>(GetRootDB());
+ assert(dbimpl != nullptr);
+
+ db_impl_->SetSnapshotChecker(new WritePreparedSnapshotChecker(this));
+ // A callback to commit a single sub-batch
+ class CommitSubBatchPreReleaseCallback : public PreReleaseCallback {
+ public:
+ explicit CommitSubBatchPreReleaseCallback(WritePreparedTxnDB* db)
+ : db_(db) {}
+ Status Callback(SequenceNumber commit_seq,
+ bool is_mem_disabled __attribute__((__unused__)), uint64_t,
+ size_t /*index*/, size_t /*total*/) override {
+ assert(!is_mem_disabled);
+ db_->AddCommitted(commit_seq, commit_seq);
+ return Status::OK();
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ };
+ db_impl_->SetRecoverableStatePreReleaseCallback(
+ new CommitSubBatchPreReleaseCallback(this));
+
+ // PessimisticTransactionDB::Initialize
+ for (auto cf_ptr : handles) {
+ AddColumnFamily(cf_ptr);
+ }
+ // Verify cf options
+ for (auto handle : handles) {
+ ColumnFamilyDescriptor cfd;
+ Status s = handle->GetDescriptor(&cfd);
+ if (!s.ok()) {
+ return s;
+ }
+ s = VerifyCFOptions(cfd.options);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+
+ // Re-enable compaction for the column families that initially had
+ // compaction enabled.
+ std::vector<ColumnFamilyHandle*> compaction_enabled_cf_handles;
+ compaction_enabled_cf_handles.reserve(compaction_enabled_cf_indices.size());
+ for (auto index : compaction_enabled_cf_indices) {
+ compaction_enabled_cf_handles.push_back(handles[index]);
+ }
+
+ // create 'real' transactions from recovered shell transactions
+ auto rtxns = dbimpl->recovered_transactions();
+ std::map<SequenceNumber, SequenceNumber> ordered_seq_cnt;
+ for (auto rtxn : rtxns) {
+ auto recovered_trx = rtxn.second;
+ assert(recovered_trx);
+ assert(recovered_trx->batches_.size() >= 1);
+ assert(recovered_trx->name_.length());
+
+ // We can only rollback transactions after AdvanceMaxEvictedSeq is called,
+ // but AddPrepared must occur before AdvanceMaxEvictedSeq, which is why
+ // two iterations is required.
+ if (recovered_trx->unprepared_) {
+ continue;
+ }
+
+ WriteOptions w_options;
+ w_options.sync = true;
+ TransactionOptions t_options;
+
+ auto first_log_number = recovered_trx->batches_.begin()->second.log_number_;
+ auto first_seq = recovered_trx->batches_.begin()->first;
+ auto last_prepare_batch_cnt =
+ recovered_trx->batches_.begin()->second.batch_cnt_;
+
+ Transaction* real_trx = BeginTransaction(w_options, t_options, nullptr);
+ assert(real_trx);
+ auto wupt =
+ static_cast_with_check<WriteUnpreparedTxn, Transaction>(real_trx);
+ wupt->recovered_txn_ = true;
+
+ real_trx->SetLogNumber(first_log_number);
+ real_trx->SetId(first_seq);
+ Status s = real_trx->SetName(recovered_trx->name_);
+ if (!s.ok()) {
+ return s;
+ }
+ wupt->prepare_batch_cnt_ = last_prepare_batch_cnt;
+
+ for (auto batch : recovered_trx->batches_) {
+ const auto& seq = batch.first;
+ const auto& batch_info = batch.second;
+ auto cnt = batch_info.batch_cnt_ ? batch_info.batch_cnt_ : 1;
+ assert(batch_info.log_number_);
+
+ ordered_seq_cnt[seq] = cnt;
+ assert(wupt->unprep_seqs_.count(seq) == 0);
+ wupt->unprep_seqs_[seq] = cnt;
+
+ s = wupt->RebuildFromWriteBatch(batch_info.batch_);
+ assert(s.ok());
+ if (!s.ok()) {
+ return s;
+ }
+ }
+
+ const bool kClear = true;
+ wupt->InitWriteBatch(kClear);
+
+ real_trx->SetState(Transaction::PREPARED);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ // AddPrepared must be called in order
+ for (auto seq_cnt : ordered_seq_cnt) {
+ auto seq = seq_cnt.first;
+ auto cnt = seq_cnt.second;
+ for (size_t i = 0; i < cnt; i++) {
+ AddPrepared(seq + i);
+ }
+ }
+
+ SequenceNumber prev_max = max_evicted_seq_;
+ SequenceNumber last_seq = db_impl_->GetLatestSequenceNumber();
+ AdvanceMaxEvictedSeq(prev_max, last_seq);
+ // Create a gap between max and the next snapshot. This simplifies the logic
+ // in IsInSnapshot by not having to consider the special case of max ==
+ // snapshot after recovery. This is tested in IsInSnapshotEmptyMapTest.
+ if (last_seq) {
+ db_impl_->versions_->SetLastAllocatedSequence(last_seq + 1);
+ db_impl_->versions_->SetLastSequence(last_seq + 1);
+ db_impl_->versions_->SetLastPublishedSequence(last_seq + 1);
+ }
+
+ Status s;
+ // Rollback unprepared transactions.
+ for (auto rtxn : rtxns) {
+ auto recovered_trx = rtxn.second;
+ if (recovered_trx->unprepared_) {
+ s = RollbackRecoveredTransaction(recovered_trx);
+ if (!s.ok()) {
+ return s;
+ }
+ continue;
+ }
+ }
+
+ if (s.ok()) {
+ dbimpl->DeleteAllRecoveredTransactions();
+
+ // Compaction should start only after max_evicted_seq_ is set AND recovered
+ // transactions are either added to PrepareHeap or rolled back.
+ s = EnableAutoCompaction(compaction_enabled_cf_handles);
+ }
+
+ return s;
+}
+
+Transaction* WriteUnpreparedTxnDB::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 WriteUnpreparedTxn(this, write_options, txn_options);
+ }
+}
+
+// Struct to hold ownership of snapshot and read callback for iterator cleanup.
+struct WriteUnpreparedTxnDB::IteratorState {
+ IteratorState(WritePreparedTxnDB* txn_db, SequenceNumber sequence,
+ std::shared_ptr<ManagedSnapshot> s,
+ SequenceNumber min_uncommitted, WriteUnpreparedTxn* txn)
+ : callback(txn_db, sequence, min_uncommitted, txn->unprep_seqs_,
+ kBackedByDBSnapshot),
+ snapshot(s) {}
+ SequenceNumber MaxVisibleSeq() { return callback.max_visible_seq(); }
+
+ WriteUnpreparedTxnReadCallback callback;
+ std::shared_ptr<ManagedSnapshot> snapshot;
+};
+
+namespace {
+static void CleanupWriteUnpreparedTxnDBIterator(void* arg1, void* /*arg2*/) {
+ delete reinterpret_cast<WriteUnpreparedTxnDB::IteratorState*>(arg1);
+}
+} // anonymous namespace
+
+Iterator* WriteUnpreparedTxnDB::NewIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ WriteUnpreparedTxn* txn) {
+ // TODO(lth): Refactor so that this logic is shared with WritePrepared.
+ constexpr bool ALLOW_BLOB = true;
+ constexpr bool ALLOW_REFRESH = true;
+ std::shared_ptr<ManagedSnapshot> own_snapshot = nullptr;
+ SequenceNumber snapshot_seq = kMaxSequenceNumber;
+ SequenceNumber min_uncommitted = 0;
+
+ // Currently, the Prev() iterator logic does not work well without snapshot
+ // validation. The logic simply iterates through values of a key in
+ // ascending seqno order, stopping at the first non-visible value and
+ // returning the last visible value.
+ //
+ // For example, if snapshot sequence is 3, and we have the following keys:
+ // foo: v1 1
+ // foo: v2 2
+ // foo: v3 3
+ // foo: v4 4
+ // foo: v5 5
+ //
+ // Then 1, 2, 3 will be visible, but 4 will be non-visible, so we return v3,
+ // which is the last visible value.
+ //
+ // For unprepared transactions, if we have snap_seq = 3, but the current
+ // transaction has unprep_seq 5, then returning the first non-visible value
+ // would be incorrect, as we should return v5, and not v3. The problem is that
+ // there are committed values at snapshot_seq < commit_seq < unprep_seq.
+ //
+ // Snapshot validation can prevent this problem by ensuring that no committed
+ // values exist at snapshot_seq < commit_seq, and thus any value with a
+ // sequence number greater than snapshot_seq must be unprepared values. For
+ // example, if the transaction had a snapshot at 3, then snapshot validation
+ // would be performed during the Put(v5) call. It would find v4, and the Put
+ // would fail with snapshot validation failure.
+ //
+ // TODO(lth): Improve Prev() logic to continue iterating until
+ // max_visible_seq, and then return the last visible value, so that this
+ // restriction can be lifted.
+ const Snapshot* snapshot = nullptr;
+ if (options.snapshot == nullptr) {
+ snapshot = GetSnapshot();
+ own_snapshot = std::make_shared<ManagedSnapshot>(db_impl_, snapshot);
+ } else {
+ snapshot = options.snapshot;
+ }
+
+ snapshot_seq = snapshot->GetSequenceNumber();
+ assert(snapshot_seq != kMaxSequenceNumber);
+ // Iteration is safe as long as largest_validated_seq <= snapshot_seq. We are
+ // guaranteed that for keys that were modified by this transaction (and thus
+ // might have unprepared values), no committed values exist at
+ // largest_validated_seq < commit_seq (or the contrapositive: any committed
+ // value must exist at commit_seq <= largest_validated_seq). This implies
+ // that commit_seq <= largest_validated_seq <= snapshot_seq or commit_seq <=
+ // snapshot_seq. As explained above, the problem with Prev() only happens when
+ // snapshot_seq < commit_seq.
+ //
+ // For keys that were not modified by this transaction, largest_validated_seq_
+ // is meaningless, and Prev() should just work with the existing visibility
+ // logic.
+ if (txn->largest_validated_seq_ > snapshot->GetSequenceNumber() &&
+ !txn->unprep_seqs_.empty()) {
+ ROCKS_LOG_ERROR(info_log_,
+ "WriteUnprepared iterator creation failed since the "
+ "transaction has performed unvalidated writes");
+ return nullptr;
+ }
+ min_uncommitted =
+ static_cast_with_check<const SnapshotImpl, const Snapshot>(snapshot)
+ ->min_uncommitted_;
+
+ auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
+ auto* state =
+ new IteratorState(this, snapshot_seq, own_snapshot, min_uncommitted, txn);
+ auto* db_iter =
+ db_impl_->NewIteratorImpl(options, cfd, state->MaxVisibleSeq(),
+ &state->callback, !ALLOW_BLOB, !ALLOW_REFRESH);
+ db_iter->RegisterCleanup(CleanupWriteUnpreparedTxnDBIterator, state, nullptr);
+ return db_iter;
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/write_unprepared_txn_db.h b/src/rocksdb/utilities/transactions/write_unprepared_txn_db.h
new file mode 100644
index 000000000..ad8e40f94
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/write_unprepared_txn_db.h
@@ -0,0 +1,148 @@
+// 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 "utilities/transactions/write_prepared_txn_db.h"
+#include "utilities/transactions/write_unprepared_txn.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class WriteUnpreparedTxn;
+
+class WriteUnpreparedTxnDB : public WritePreparedTxnDB {
+ public:
+ using WritePreparedTxnDB::WritePreparedTxnDB;
+
+ Status Initialize(const std::vector<size_t>& compaction_enabled_cf_indices,
+ const std::vector<ColumnFamilyHandle*>& handles) override;
+
+ Transaction* BeginTransaction(const WriteOptions& write_options,
+ const TransactionOptions& txn_options,
+ Transaction* old_txn) override;
+
+ // Struct to hold ownership of snapshot and read callback for cleanup.
+ struct IteratorState;
+
+ using WritePreparedTxnDB::NewIterator;
+ Iterator* NewIterator(const ReadOptions& options,
+ ColumnFamilyHandle* column_family,
+ WriteUnpreparedTxn* txn);
+
+ private:
+ Status RollbackRecoveredTransaction(const DBImpl::RecoveredTransaction* rtxn);
+};
+
+class WriteUnpreparedCommitEntryPreReleaseCallback : public PreReleaseCallback {
+ // TODO(lth): Reduce code duplication with
+ // WritePreparedCommitEntryPreReleaseCallback
+ public:
+ // includes_data indicates that the commit also writes non-empty
+ // CommitTimeWriteBatch to memtable, which needs to be committed separately.
+ WriteUnpreparedCommitEntryPreReleaseCallback(
+ WritePreparedTxnDB* db, DBImpl* db_impl,
+ const std::map<SequenceNumber, size_t>& unprep_seqs,
+ size_t data_batch_cnt = 0, bool publish_seq = true)
+ : db_(db),
+ db_impl_(db_impl),
+ unprep_seqs_(unprep_seqs),
+ data_batch_cnt_(data_batch_cnt),
+ includes_data_(data_batch_cnt_ > 0),
+ publish_seq_(publish_seq) {
+ assert(unprep_seqs.size() > 0);
+ }
+
+ virtual Status Callback(SequenceNumber commit_seq,
+ bool is_mem_disabled __attribute__((__unused__)),
+ uint64_t, size_t /*index*/,
+ size_t /*total*/) override {
+ const uint64_t last_commit_seq = LIKELY(data_batch_cnt_ <= 1)
+ ? commit_seq
+ : commit_seq + data_batch_cnt_ - 1;
+ // Recall that unprep_seqs maps (un)prepared_seq => prepare_batch_cnt.
+ for (const auto& s : unprep_seqs_) {
+ for (size_t i = 0; i < s.second; i++) {
+ db_->AddCommitted(s.first + i, last_commit_seq);
+ }
+ }
+
+ if (includes_data_) {
+ assert(data_batch_cnt_);
+ // Commit the data that is accompanied with the commit request
+ for (size_t i = 0; i < data_batch_cnt_; i++) {
+ // For commit seq of each batch use the commit seq of the last batch.
+ // This would make debugging easier by having all the batches having
+ // the same sequence number.
+ db_->AddCommitted(commit_seq + i, last_commit_seq);
+ }
+ }
+ if (db_impl_->immutable_db_options().two_write_queues && publish_seq_) {
+ assert(is_mem_disabled); // implies the 2nd queue
+ // Publish the sequence number. We can do that here assuming the callback
+ // is invoked only from one write queue, which would guarantee that the
+ // publish sequence numbers will be in order, i.e., once a seq is
+ // published all the seq prior to that are also publishable.
+ db_impl_->SetLastPublishedSequence(last_commit_seq);
+ }
+ // else SequenceNumber that is updated as part of the write already does the
+ // publishing
+ return Status::OK();
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ DBImpl* db_impl_;
+ const std::map<SequenceNumber, size_t>& unprep_seqs_;
+ size_t data_batch_cnt_;
+ // Either because it is commit without prepare or it has a
+ // CommitTimeWriteBatch
+ bool includes_data_;
+ // Should the callback also publishes the commit seq number
+ bool publish_seq_;
+};
+
+class WriteUnpreparedRollbackPreReleaseCallback : public PreReleaseCallback {
+ // TODO(lth): Reduce code duplication with
+ // WritePreparedCommitEntryPreReleaseCallback
+ public:
+ WriteUnpreparedRollbackPreReleaseCallback(
+ WritePreparedTxnDB* db, DBImpl* db_impl,
+ const std::map<SequenceNumber, size_t>& unprep_seqs,
+ SequenceNumber rollback_seq)
+ : db_(db),
+ db_impl_(db_impl),
+ unprep_seqs_(unprep_seqs),
+ rollback_seq_(rollback_seq) {
+ assert(unprep_seqs.size() > 0);
+ assert(db_impl_->immutable_db_options().two_write_queues);
+ }
+
+ virtual Status Callback(SequenceNumber commit_seq,
+ bool is_mem_disabled __attribute__((__unused__)),
+ uint64_t, size_t /*index*/,
+ size_t /*total*/) override {
+ assert(is_mem_disabled); // implies the 2nd queue
+ const uint64_t last_commit_seq = commit_seq;
+ db_->AddCommitted(rollback_seq_, last_commit_seq);
+ // Recall that unprep_seqs maps (un)prepared_seq => prepare_batch_cnt.
+ for (const auto& s : unprep_seqs_) {
+ for (size_t i = 0; i < s.second; i++) {
+ db_->AddCommitted(s.first + i, last_commit_seq);
+ }
+ }
+ db_impl_->SetLastPublishedSequence(last_commit_seq);
+ return Status::OK();
+ }
+
+ private:
+ WritePreparedTxnDB* db_;
+ DBImpl* db_impl_;
+ const std::map<SequenceNumber, size_t>& unprep_seqs_;
+ SequenceNumber rollback_seq_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE