From 19fcec84d8d7d21e796c7624e521b60d28ee21ed Mon Sep 17 00:00:00 2001 From: Daniel Baumann Date: Sun, 7 Apr 2024 20:45:59 +0200 Subject: Adding upstream version 16.2.11+ds. Signed-off-by: Daniel Baumann --- .../transactions/optimistic_transaction.cc | 187 + .../transactions/optimistic_transaction.h | 101 + .../transactions/optimistic_transaction_db_impl.cc | 111 + .../transactions/optimistic_transaction_db_impl.h | 71 + .../transactions/optimistic_transaction_test.cc | 1535 +++++ .../transactions/pessimistic_transaction.cc | 723 +++ .../transactions/pessimistic_transaction.h | 225 + .../transactions/pessimistic_transaction_db.cc | 632 ++ .../transactions/pessimistic_transaction_db.h | 220 + .../utilities/transactions/snapshot_checker.cc | 49 + .../utilities/transactions/transaction_base.cc | 837 +++ .../utilities/transactions/transaction_base.h | 374 ++ .../transactions/transaction_db_mutex_impl.cc | 135 + .../transactions/transaction_db_mutex_impl.h | 26 + .../utilities/transactions/transaction_lock_mgr.cc | 745 +++ .../utilities/transactions/transaction_lock_mgr.h | 158 + .../utilities/transactions/transaction_test.cc | 6224 ++++++++++++++++++++ .../utilities/transactions/transaction_test.h | 517 ++ .../utilities/transactions/transaction_util.cc | 182 + .../utilities/transactions/transaction_util.h | 103 + .../write_prepared_transaction_test.cc | 3524 +++++++++++ .../utilities/transactions/write_prepared_txn.cc | 473 ++ .../utilities/transactions/write_prepared_txn.h | 119 + .../transactions/write_prepared_txn_db.cc | 998 ++++ .../utilities/transactions/write_prepared_txn_db.h | 1111 ++++ .../write_unprepared_transaction_test.cc | 727 +++ .../utilities/transactions/write_unprepared_txn.cc | 999 ++++ .../utilities/transactions/write_unprepared_txn.h | 341 ++ .../transactions/write_unprepared_txn_db.cc | 468 ++ .../transactions/write_unprepared_txn_db.h | 148 + 30 files changed, 22063 insertions(+) create mode 100644 src/rocksdb/utilities/transactions/optimistic_transaction.cc create mode 100644 src/rocksdb/utilities/transactions/optimistic_transaction.h create mode 100644 src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.cc create mode 100644 src/rocksdb/utilities/transactions/optimistic_transaction_db_impl.h create mode 100644 src/rocksdb/utilities/transactions/optimistic_transaction_test.cc create mode 100644 src/rocksdb/utilities/transactions/pessimistic_transaction.cc create mode 100644 src/rocksdb/utilities/transactions/pessimistic_transaction.h create mode 100644 src/rocksdb/utilities/transactions/pessimistic_transaction_db.cc create mode 100644 src/rocksdb/utilities/transactions/pessimistic_transaction_db.h create mode 100644 src/rocksdb/utilities/transactions/snapshot_checker.cc create mode 100644 src/rocksdb/utilities/transactions/transaction_base.cc create mode 100644 src/rocksdb/utilities/transactions/transaction_base.h create mode 100644 src/rocksdb/utilities/transactions/transaction_db_mutex_impl.cc create mode 100644 src/rocksdb/utilities/transactions/transaction_db_mutex_impl.h create mode 100644 src/rocksdb/utilities/transactions/transaction_lock_mgr.cc create mode 100644 src/rocksdb/utilities/transactions/transaction_lock_mgr.h create mode 100644 src/rocksdb/utilities/transactions/transaction_test.cc create mode 100644 src/rocksdb/utilities/transactions/transaction_test.h create mode 100644 src/rocksdb/utilities/transactions/transaction_util.cc create mode 100644 src/rocksdb/utilities/transactions/transaction_util.h create mode 100644 src/rocksdb/utilities/transactions/write_prepared_transaction_test.cc create mode 100644 src/rocksdb/utilities/transactions/write_prepared_txn.cc create mode 100644 src/rocksdb/utilities/transactions/write_prepared_txn.h create mode 100644 src/rocksdb/utilities/transactions/write_prepared_txn_db.cc create mode 100644 src/rocksdb/utilities/transactions/write_prepared_txn_db.h create mode 100644 src/rocksdb/utilities/transactions/write_unprepared_transaction_test.cc create mode 100644 src/rocksdb/utilities/transactions/write_unprepared_txn.cc create mode 100644 src/rocksdb/utilities/transactions/write_unprepared_txn.h create mode 100644 src/rocksdb/utilities/transactions/write_unprepared_txn_db.cc create mode 100644 src/rocksdb/utilities/transactions/write_unprepared_txn_db.h (limited to 'src/rocksdb/utilities/transactions') 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 + +#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(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(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(txn_db_); + assert(txn_db_impl); + DBImpl* db_impl = static_cast_with_check(db_->GetRootDB()); + assert(db_impl); + const size_t space = txn_db_impl->GetLockBucketsSize(); + std::set lk_idxes; + std::vector> 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(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 +#include +#include +#include + +#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 +#include + +#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 OptimisticTransactionDBImpl::LockBucket( + size_t idx) { + assert(idx < bucketed_locks_.size()); + return std::unique_lock(*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 column_families; + column_families.push_back( + ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options)); + std::vector 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& column_families, + std::vector* 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& column_families, + std::vector* handles, + OptimisticTransactionDB** dbptr) { + Status s; + DB* db; + + std::vector 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(txn) != nullptr); + auto txn_impl = reinterpret_cast(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 +#include +#include + +#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(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 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> 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 +#include +#include + +#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 { + 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 column_families; + std::vector 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(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 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 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 multiget_cfh = {handles[1], handles[2], + handles[0], handles[2]}; + std::vector multiget_keys = {"AAA", "AAAZZZ", "foo", "foo"}; + std::vector values(4); + + std::vector 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 multiget_keys = {"1", "2", "3"}; + std::vector multiget_values; + + std::vector 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::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(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 threads; + + std::function 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 + +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 +#include +#include +#include + +#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 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(txn_db); + db_impl_ = static_cast_with_check(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& 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> 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 +#include +#include +#include +#include +#include +#include + +#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 GetWaitingTxns(uint32_t* column_family_id, + std::string* key) const override { + std::lock_guard lock(wait_mutex_); + std::vector 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 ids, uint32_t column_family_id, + const std::string* key) { + std::lock_guard lock(wait_mutex_); + waiting_txn_ids_ = ids; + waiting_cf_id_ = column_family_id; + waiting_key_ = key; + } + + void ClearWaitingTxn() { + std::lock_guard 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 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 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 +#include +#include +#include + +#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(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( + 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(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( + 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& compaction_enabled_cf_indices, + const std::vector& 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 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(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 column_families; + column_families.push_back( + ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options)); + std::vector 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& column_families, + std::vector* 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 column_families_copy = column_families; + std::vector 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(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* column_families, + std::vector* 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& compaction_enabled_cf_indices, + const std::vector& handles, TransactionDB** dbptr) { + assert(db != nullptr); + assert(dbptr != nullptr); + *dbptr = nullptr; + std::unique_ptr 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& compaction_enabled_cf_indices, + const std::vector& handles, TransactionDB** dbptr) { + assert(db != nullptr); + assert(dbptr != nullptr); + *dbptr = nullptr; + std::unique_ptr 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 lock(map_mutex_); + expirable_transactions_map_.insert({tx_id, tx}); +} + +void PessimisticTransactionDB::RemoveExpirableTransaction(TransactionID tx_id) { + std::lock_guard lock(map_mutex_); + expirable_transactions_map_.erase(tx_id); +} + +bool PessimisticTransactionDB::TryStealingExpiredTransactionLocks( + TransactionID tx_id) { + std::lock_guard 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(txn); + + txn_impl->Reinitialize(this, write_options, txn_options); +} + +Transaction* PessimisticTransactionDB::GetTransactionByName( + const TransactionName& name) { + std::lock_guard lock(name_map_mutex_); + auto it = transactions_.find(name); + if (it == transactions_.end()) { + return nullptr; + } else { + return it->second; + } +} + +void PessimisticTransactionDB::GetAllPreparedTransactions( + std::vector* transv) { + assert(transv); + transv->clear(); + std::lock_guard 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 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 lock(name_map_mutex_); + transactions_[txn->GetName()] = txn; +} + +void PessimisticTransactionDB::UnregisterTransaction(Transaction* txn) { + assert(txn); + std::lock_guard 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 +#include +#include +#include +#include +#include + +#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& compaction_enabled_cf_indices, + const std::vector& 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(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* trans) override; + + TransactionLockMgr::LockStatusData GetLockStatusData() override; + + std::vector 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&) {} + virtual void UpdateCFComparatorMap(ColumnFamilyHandle*) {} + + protected: + DBImpl* db_impl_; + std::shared_ptr 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 + expirable_transactions_map_; + + // map from name to two phase transaction instance + std::mutex name_map_mutex_; + std::unordered_map 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 +#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 + +#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(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(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 notifier) { + snapshot_needed_ = true; + snapshot_notifier_ = notifier; +} + +void TransactionBaseImpl::SetSnapshotIfNeeded() { + if (snapshot_needed_) { + std::shared_ptr 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>()); + } + 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& 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& 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 TransactionBaseImpl::MultiGet( + const ReadOptions& read_options, + const std::vector& column_family, + const std::vector& keys, std::vector* values) { + size_t num_keys = keys.size(); + values->resize(num_keys); + + std::vector 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 TransactionBaseImpl::MultiGetForUpdate( + const ReadOptions& read_options, + const std::vector& column_family, + const std::vector& keys, std::vector* 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(num_keys, s); + } + } + + // TODO(agiardullo): optimize multiget? + std::vector 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 +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(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(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(©cat); +} + +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 +#include +#include + +#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 MultiGet( + const ReadOptions& options, + const std::vector& column_family, + const std::vector& keys, + std::vector* values) override; + + std::vector MultiGet(const ReadOptions& options, + const std::vector& keys, + std::vector* values) override { + return MultiGet(options, std::vector( + 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 MultiGetForUpdate( + const ReadOptions& options, + const std::vector& column_family, + const std::vector& keys, + std::vector* values) override; + + std::vector MultiGetForUpdate( + const ReadOptions& options, const std::vector& keys, + std::vector* values) override { + return MultiGetForUpdate(options, + std::vector( + 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 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 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 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 snapshot_; + bool snapshot_needed_ = false; + std::shared_ptr 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 snapshot, bool snapshot_needed, + std::shared_ptr 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>> + 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 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 +#include +#include +#include + +#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 mutex) override; + + Status WaitFor(std::shared_ptr 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 +TransactionDBMutexFactoryImpl::AllocateMutex() { + return std::shared_ptr(new TransactionDBMutexImpl()); +} + +std::shared_ptr +TransactionDBMutexFactoryImpl::AllocateCondVar() { + return std::shared_ptr(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 mutex) { + auto mutex_impl = reinterpret_cast(mutex.get()); + + std::unique_lock 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 mutex, int64_t timeout_time) { + Status s; + + auto mutex_impl = reinterpret_cast(mutex.get()); + std::unique_lock 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 AllocateMutex() override; + std::shared_ptr 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 + +#include +#include +#include +#include +#include +#include + +#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 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 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 stripe_mutex; + + // Condition Variable per stripe for waiting on a lock + std::shared_ptr 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 keys; +}; + +// Map of #num_stripes LockMapStripes +struct LockMap { + explicit LockMap(size_t num_stripes, + std::shared_ptr 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 lock_cnt{0}; + + std::vector lock_map_stripes_; + + size_t GetStripe(const std::string& key) const; +}; + +void DeadlockInfoBuffer::AddNewPath(DeadlockPath path) { + std::lock_guard 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 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 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 DeadlockInfoBuffer::PrepareBuffer() { + std::lock_guard 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>*>(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 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(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(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 local_caches; + lock_maps_cache_->Scrape(&local_caches, nullptr); + for (auto cache : local_caches) { + delete static_cast(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 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(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(nullptr); + } else { + // Found lock map. Store in thread-local cache and return. + std::shared_ptr& 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 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 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(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& wait_ids) { + std::lock_guard lock(wait_txn_map_mutex_); + DecrementWaitersImpl(txn, wait_ids); +} + +void TransactionLockMgr::DecrementWaitersImpl( + const PessimisticTransaction* txn, + const autovector& 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& wait_ids, const std::string& key, + const uint32_t& cf_id, const bool& exclusive, Env* const env) { + auto id = txn->GetID(); + std::vector queue_parents(static_cast(txn->GetDeadlockDetectDepth())); + std::vector queue_values(static_cast(txn->GetDeadlockDetectDepth())); + std::lock_guard 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(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 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* 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 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 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> 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 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 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 +#include +#include +#include +#include +#include + +#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 paths_buffer_; + uint32_t buffer_idx_; + std::mutex paths_buffer_mutex_; + std::vector 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 PrepareBuffer(); +}; + +struct TrackedTrxInfo { + autovector 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 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; + LockStatusData GetLockStatusData(); + std::vector 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>; + 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 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 rev_wait_txn_map_; + // Maps from waiter -> waitee. + HashMap wait_txn_map_; + DeadlockInfoBuffer dlock_buffer_; + + // Used to allocate mutexes/condvars to use when locking keys + std::shared_ptr mutex_factory_; + + bool IsLockExpired(TransactionID txn_id, const LockInfo& lock_info, Env* env, + uint64_t* wait_time); + + std::shared_ptr 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* 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& wait_ids, + const std::string& key, const uint32_t& cf_id, + const bool& exclusive, Env* const env); + void DecrementWaiters(const PessimisticTransaction* txn, + const autovector& wait_ids); + void DecrementWaitersImpl(const PessimisticTransaction* txn, + const autovector& 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 +#include +#include +#include + +#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(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(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(options.max_write_buffer_size_to_maintain) / + static_cast(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(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 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 expected_txns = {txn1->GetID(), txn2->GetID(), + txn3->GetID()}; + std::vector 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 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 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 threads; + for (uint32_t i = 0; i < 15; i++) { + std::function 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 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 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 threads_shared; + for (uint32_t i = 0; i < 1; i++) { + std::function 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 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 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 threads; + for (uint32_t i = 0; i < len - 1; i++) { + std::function 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 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::this_thread::get_id()); + Random rnd(static_cast(tid)); + std::function 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 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 txns; + DBImpl* db_impl = reinterpret_cast(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(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 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(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(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(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(db)->TEST_Crash(); + s = ReOpenNoDelete(); + ASSERT_OK(s); + assert(db != nullptr); + db_impl = reinterpret_cast(db->GetRootDB()); + + // find trans in list of prepared transactions + std::vector 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 txn_thread_num(0); + + std::function 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(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('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 t_wait_on_prepare(0); + std::atomic t_wait_on_commit(0); + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "WriteThread::JoinBatchGroup:Wait", [&](void* arg) { + auto* writer = reinterpret_cast(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 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(t))); + for (int i = 0; i < 10; i++) { + std::string key(name + "_" + std::string(1, static_cast('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(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(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(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(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(cfa); + auto cfh_b = reinterpret_cast(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(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(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(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 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 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 multiget_cfh = {handles[1], handles[2], + handles[0], handles[2]}; + std::vector multiget_keys = {"AAA", "AAAZZZ", "foo", "foo"}; + std::vector values(4); + std::vector 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 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 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 keys = {"aaa", "bbb", "ccc", "ddd", "eee", "fff", "ggg"}; + std::vector values(keys.size()); + std::vector 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 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 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 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 keys; + for (int i = 0; i < MultiGetContext::MAX_BATCH_SIZE + 32; ++i) { + keys.emplace_back(key_str[i]); + } + std::vector values(keys.size()); + std::vector 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 multiget_keys = {"1", "2", "3"}; + std::vector multiget_values; + + std::vector 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(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 = + std::make_shared(&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 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 handles; + + s = TransactionDB::Open(options, txn_db_options, dbname, column_families, + &handles, &db); + ASSERT_OK(s); + + auto cfh_default = reinterpret_cast(handles[0]); + auto opt_default = *cfh_default->cfd()->GetLatestMutableCFOptions(); + + auto cfh_a = reinterpret_cast(handles[1]); + auto opt_a = *cfh_a->cfd()->GetLatestMutableCFOptions(); + + auto cfh_b = reinterpret_cast(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(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 threads; + std::atomic 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 call_inserter = [&] { + size_t thd_seed = std::hash()(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 call_checker = [&] { + size_t thd_seed = std::hash()(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 call_slow_checker = [&] { + size_t thd_seed = std::hash()(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 call_slow_inserter = [&] { + size_t thd_seed = std::hash()(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(1) << *branch; + return n & filter; + }; + const size_t max_n = static_cast(1) << NUM_BRANCHES; + for (size_t n = 0; n < max_n; n++) { + DBImpl* db_impl = reinterpret_cast(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(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(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(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(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(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 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 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 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 cfds{ + ColumnFamilyDescriptor(kDefaultColumnFamilyName, + ColumnFamilyOptions(options)), + ColumnFamilyDescriptor(cf_name, cf_options), + }; + std::vector 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(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(db->GetRootDB()) + ->TEST_FlushMemTable(true, false, handles[1]); + reinterpret_cast(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(db->GetRootDB()) + ->TEST_FlushMemTable(true, false, handles[1]); + reinterpret_cast(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(db->GetRootDB()) + ->TEST_FlushMemTable(true, false, handles[1]); + reinterpret_cast(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(db->GetRootDB()) + ->TEST_FlushMemTable(true, false, handles[1]); + reinterpret_cast(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(db->GetRootDB()) + ->TEST_FlushMemTable(true, false, handles[1]); + reinterpret_cast(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(db->GetRootDB()); + uint64_t wal_file_id = db_impl->TEST_LogfileNumber(); + std::string fname = LogFileName(dbname, wal_file_id); + reinterpret_cast(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 handles; + std::vector 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 + +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 +#include +#include +#include +#include + +#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& cfs, + std::vector* 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& cfs, + std::vector* handles) { + std::vector 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 compaction_enabled_cf_indices; + std::vector column_families{ColumnFamilyDescriptor( + kDefaultColumnFamilyName, ColumnFamilyOptions(options))}; + + TransactionDB::PrepareWrap(&options, &column_families, + &compaction_enabled_cf_indices); + std::vector 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 linked = {0}; + std::atomic exp_seq = {0}; + std::atomic commit_writes = {0}; + std::atomic expected_commits = {0}; + // Without Prepare, the commit does not write to WAL + std::atomic with_empty_commits = {0}; + std::function 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 txn_t0 = [&](size_t index) { + return txn_t0_with_status(index, Status::OK()); + }; + std::function 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 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 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 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 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(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(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> { + 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> { + 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 +#include +#include + +#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(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 +#include + +#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>; + +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 +#include +#include +#include +#include +#include + +#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 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 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 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(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(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(21); + const size_t INDEX_SIZE = static_cast(1ull << INDEX_BITS); + const CommitEntry64bFormat FORMAT(static_cast(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 is = {{0, 1, INDEX_SIZE / 2 + 1, INDEX_SIZE - 1}}; + // Samples over the numbers that are covered by that many commit bits + std::array 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& snapshots) { + snapshots_ = snapshots; + } + void TakeSnapshot(SequenceNumber seq) { snapshots_.push_back(seq); } + + protected: + const std::vector GetSnapshotListFromDB( + SequenceNumber /* unused */) override { + return snapshots_; + } + + private: + std::vector 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(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& old_snapshots, + const std::vector& 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 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& 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 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& expected_versions) { + std::vector 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> { + 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> { + 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> { + 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(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(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(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 vec) { + std::set 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 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 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 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 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 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 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 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 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 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(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 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 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 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 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(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 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(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(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 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(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(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(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(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(db); + ReadOptions ropt; + PinnableSlice pinnable_val; + WriteOptions write_options; + TransactionOptions txn_options; + std::vector 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(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(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 threads; + + linked = 0; + std::atomic 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(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(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(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(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(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(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(db)->TEST_Crash(); + auto db_impl = reinterpret_cast(db->GetRootDB()); + db_impl->FlushWAL(true); + ReOpenNoDelete(); + WritePreparedTxnDB* wp_db = dynamic_cast(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(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 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 committed_before; + // The set of commit seq numbers to be excluded from IsInSnapshot queries + std::set commit_seqs; + DBImpl* mock_db = new DBImpl(options, dbname); + UpdateTransactionDBOptions(snapshot_cache_bits, commit_cache_bits); + std::unique_ptr 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 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(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(db->GetRootDB()); + db_impl->FlushWAL(true); + dynamic_cast(db)->TEST_Crash(); + ReOpenNoDelete(); + assert(db != nullptr); + wp_db = dynamic_cast(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 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(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(db->GetRootDB()); + // Snapshots to avoid keys get evicted. + std::vector snapshots; + // Keep track of expected sequence number. + SequenceNumber expected_seq = 0; + + auto add_key = [&](std::function 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(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(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 transactions(kNumTransactions, nullptr); + std::vector versions(kNumTransactions, 0); + std::unordered_map current_data; + std::vector snapshots; + std::vector> 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(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 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(db); + DBImpl* db_impl = reinterpret_cast(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(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(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(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 snap = {nullptr}; + std::atomic 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 + +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 +#include +#include + +#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& comparators_; + std::map& handles_; + using CFKeys = std::set; + std::map keys_; + bool rollback_merge_operands_; + ReadOptions roptions_; + RollbackWriteBatchBuilder( + DBImpl* db, WritePreparedTxnDB* wpt_db, SequenceNumber snap_seq, + WriteBatch* dst_batch, + std::map& comparators, + std::map& 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 = ¬_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( + 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 +#include +#include +#include +#include +#include +#include + +#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 +#include +#include +#include +#include + +#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& compaction_enabled_cf_indices, + const std::vector& handles) { + auto dbimpl = static_cast_with_check(GetRootDB()); + assert(dbimpl != nullptr); + auto rtxns = dbimpl->recovered_transactions(); + std::map 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(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& handles) { + auto cf_map = new std::map(); + auto handle_map = new std::map(); + 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(*old_cf_map_ptr); + auto old_handle_map_ptr = handle_map_.get(); + assert(old_handle_map_ptr); + auto handle_map = + new std::map(*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 WritePreparedTxnDB::MultiGet( + const ReadOptions& options, + const std::vector& column_family, + const std::vector& keys, std::vector* values) { + assert(values); + size_t num_keys = keys.size(); + values->resize(num_keys); + + std::vector 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 s, + SequenceNumber min_uncommitted) + : callback(txn_db, sequence, min_uncommitted, kBackedByDBSnapshot), + snapshot(s) {} + + WritePreparedTxnReadCallback callback; + std::shared_ptr snapshot; +}; + +namespace { +static void CleanupWritePreparedTxnDBIterator(void* arg1, void* /*arg2*/) { + delete reinterpret_cast(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 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( + 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(snapshot) + ->min_uncommitted_; + own_snapshot = std::make_shared(db_impl_, snapshot); + } + assert(snapshot_seq != kMaxSequenceNumber); + auto* cfd = reinterpret_cast(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& column_families, + std::vector* iterators) { + constexpr bool ALLOW_BLOB = true; + constexpr bool ALLOW_REFRESH = true; + std::shared_ptr 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( + 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(db_impl_, snapshot); + min_uncommitted = + static_cast_with_check(snapshot) + ->min_uncommitted_; + } + iterators->clear(); + iterators->reserve(column_families.size()); + for (auto* column_family : column_families) { + auto* cfd = reinterpret_cast(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(1)); + snapshot_cache_ = std::unique_ptr[]>( + new std::atomic[SNAPSHOT_CACHE_SIZE] {}); + commit_cache_ = std::unique_ptr[]>( + new std::atomic[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(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(reinterpret_cast(&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(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(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(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 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 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 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& new_snapshots, + const std::vector& 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& 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 +#include +#include +#include +#include +#include +#include + +#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(1ull << SNAPSHOT_CACHE_BITS)), + COMMIT_CACHE_BITS(txn_db_options.wp_commit_cache_bits), + COMMIT_CACHE_SIZE(static_cast(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(1ull << SNAPSHOT_CACHE_BITS)), + COMMIT_CACHE_BITS(txn_db_options.wp_commit_cache_bits), + COMMIT_CACHE_SIZE(static_cast(1ull << COMMIT_CACHE_BITS)), + FORMAT(COMMIT_CACHE_BITS) { + Init(txn_db_options); + } + + virtual ~WritePreparedTxnDB(); + + virtual Status Initialize( + const std::vector& compaction_enabled_cf_indices, + const std::vector& 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 MultiGet( + const ReadOptions& options, + const std::vector& column_family, + const std::vector& keys, + std::vector* 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& column_families, + std::vector* 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(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(64 - PAD_BITS - INDEX_BITS)), + COMMIT_BITS(static_cast(64 - PREP_BITS)), + COMMIT_FILTER(static_cast((1ull << COMMIT_BITS) - 1)), + DELTA_UPPERBOUND(static_cast((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(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( + (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((1ull << format.COMMIT_BITS))); + if (delta == 0) { + return false; // initialized entry would have non-zero delta + } + + assert(indexed_seq < static_cast((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> GetCFComparatorMap() { + return cf_map_; + } + std::shared_ptr> GetCFHandleMap() { + return handle_map_; + } + void UpdateCFComparatorMap( + const std::vector& 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 heap_; + std::priority_queue, std::greater> + erased_heap_; + std::atomic 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 . 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 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& 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& new_snapshots, + const std::vector& 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 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[]> snapshot_cache_; + // 2nd list for storing snapshots. The list sorted in ascending order. + // Thread-safety is provided with snapshots_mutex_. + std::vector 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 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[]> 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 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 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> 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 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 delayed_prepared_commits_; + // Update when delayed_prepared_.empty() changes. Expected to be true + // normally. + std::atomic delayed_prepared_empty_ = {true}; + // Update when old_commit_map_.empty() changes. Expected to be true normally. + std::atomic 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> 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> 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& comparators) + : comparators_(comparators), batches_(1) {} + std::map& comparators_; + using CFKeys = std::set; + std::map 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(snapshot) + ->min_uncommitted_; + *max = static_cast_with_check(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> { + 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> { + 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::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(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( + std::hash()(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 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 counter(0); + + std::function stress_thread = [&](int id) { + size_t tid = std::hash()(std::this_thread::get_id()); + Random64 rnd(static_cast(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 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 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 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(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(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(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(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 + +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 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(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& handles_; + + SavePointBatchHandler( + WriteBatchWithIndex* wb, + const std::map& 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()); + } + 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 = ¬_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()); + } + 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( + 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(arg1); + auto iter = reinterpret_cast(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( + 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& +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 + +#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& 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& 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& 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& 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 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 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>; + 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 unprep_seqs_; + + // This snapshot will be used to read keys at this savepoint if we call + // RollbackToSavePoint. + std::unique_ptr snapshot_; + + SavePoint(const std::map& 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> + flushed_save_points_; + std::unique_ptr> 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 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& comparators_; + std::map& handles_; + using CFKeys = std::set; + std::map keys_; + bool rollback_merge_operands_; + RollbackWriteBatchBuilder( + DBImpl* db, SequenceNumber snap_seq, WriteBatch* dst_batch, + std::map& comparators, + std::map& 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 = ¬_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& compaction_enabled_cf_indices, + const std::vector& handles) { + // TODO(lth): Reduce code duplication in this function. + auto dbimpl = static_cast_with_check(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 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 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(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 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 snapshot; +}; + +namespace { +static void CleanupWriteUnpreparedTxnDBIterator(void* arg1, void* /*arg2*/) { + delete reinterpret_cast(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 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(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(snapshot) + ->min_uncommitted_; + + auto* cfd = reinterpret_cast(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& compaction_enabled_cf_indices, + const std::vector& 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& 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& 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& 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& unprep_seqs_; + SequenceNumber rollback_seq_; +}; + +} // namespace ROCKSDB_NAMESPACE +#endif // ROCKSDB_LITE -- cgit v1.2.3