Summary: Optimistic transactions supporting begin/commit/rollback semantics. Currently relies on checking the memtable to determine if there are any collisions at commit time. Not yet implemented would be a way of enuring the memtable has some minimum amount of history so that we won't fail to commit when the memtable is empty. You should probably start with transaction.h to get an overview of what is currently supported. Test Plan: Added a new test, but still need to look into stress testing. Reviewers: yhchiang, igor, rven, sdong Reviewed By: sdong Subscribers: adamretter, MarkCallaghan, leveldb, dhruba Differential Revision: https://reviews.facebook.net/D33435main
parent
d5a0c0e69b
commit
dc9d70de65
@ -0,0 +1,24 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#pragma once |
||||
|
||||
#include "rocksdb/status.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
class DB; |
||||
|
||||
class WriteCallback { |
||||
public: |
||||
virtual ~WriteCallback() {} |
||||
|
||||
// Will be called while on the write thread before the write executes. If
|
||||
// this function returns a non-OK status, the write will be aborted and this
|
||||
// status will be returned to the caller of DB::Write().
|
||||
virtual Status Callback(DB* db) = 0; |
||||
}; |
||||
|
||||
} // namespace rocksdb
|
@ -0,0 +1,120 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include <string> |
||||
|
||||
#include "db/db_impl.h" |
||||
#include "db/write_callback.h" |
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/write_batch.h" |
||||
#include "util/logging.h" |
||||
#include "util/testharness.h" |
||||
|
||||
using std::string; |
||||
|
||||
namespace rocksdb { |
||||
|
||||
class WriteCallbackTest : public testing::Test { |
||||
public: |
||||
string dbname; |
||||
|
||||
WriteCallbackTest() { |
||||
dbname = test::TmpDir() + "/write_callback_testdb"; |
||||
} |
||||
}; |
||||
|
||||
class WriteCallbackTestWriteCallback1 : public WriteCallback { |
||||
public: |
||||
bool was_called = false; |
||||
|
||||
Status Callback(DB *db) override { |
||||
was_called = true; |
||||
|
||||
// Make sure db is a DBImpl
|
||||
DBImpl* db_impl = dynamic_cast<DBImpl*> (db); |
||||
if (db_impl == nullptr) { |
||||
return Status::InvalidArgument(""); |
||||
} |
||||
|
||||
return Status::OK(); |
||||
} |
||||
}; |
||||
|
||||
class WriteCallbackTestWriteCallback2 : public WriteCallback { |
||||
public: |
||||
Status Callback(DB *db) override { |
||||
return Status::Busy(); |
||||
} |
||||
}; |
||||
|
||||
TEST_F(WriteCallbackTest, WriteCallBackTest) { |
||||
Options options; |
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
string value; |
||||
DB* db; |
||||
DBImpl* db_impl; |
||||
|
||||
options.create_if_missing = true; |
||||
Status s = DB::Open(options, dbname, &db); |
||||
ASSERT_OK(s); |
||||
|
||||
db_impl = dynamic_cast<DBImpl*> (db); |
||||
ASSERT_TRUE(db_impl); |
||||
|
||||
WriteBatch wb; |
||||
|
||||
wb.Put("a", "value.a"); |
||||
wb.Delete("x"); |
||||
|
||||
// Test a simple Write
|
||||
s = db->Write(write_options, &wb); |
||||
ASSERT_OK(s); |
||||
|
||||
s = db->Get(read_options, "a", &value); |
||||
ASSERT_OK(s); |
||||
ASSERT_EQ("value.a", value); |
||||
|
||||
// Test WriteWithCallback
|
||||
WriteCallbackTestWriteCallback1 callback1; |
||||
WriteBatch wb2; |
||||
|
||||
wb2.Put("a", "value.a2"); |
||||
|
||||
s = db_impl->WriteWithCallback(write_options, &wb2, &callback1); |
||||
ASSERT_OK(s); |
||||
ASSERT_TRUE(callback1.was_called); |
||||
|
||||
s = db->Get(read_options, "a", &value); |
||||
ASSERT_OK(s); |
||||
ASSERT_EQ("value.a2", value); |
||||
|
||||
// Test WriteWithCallback for a callback that fails
|
||||
WriteCallbackTestWriteCallback2 callback2; |
||||
WriteBatch wb3; |
||||
|
||||
wb3.Put("a", "value.a3"); |
||||
|
||||
s = db_impl->WriteWithCallback(write_options, &wb3, &callback2); |
||||
ASSERT_NOK(s); |
||||
|
||||
s = db->Get(read_options, "a", &value); |
||||
ASSERT_OK(s); |
||||
ASSERT_EQ("value.a2", value); |
||||
|
||||
delete db; |
||||
DestroyDB(dbname, options); |
||||
} |
||||
|
||||
} // namespace rocksdb
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
return RUN_ALL_TESTS(); |
||||
} |
||||
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,142 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/options.h" |
||||
#include "rocksdb/slice.h" |
||||
#include "rocksdb/utilities/optimistic_transaction.h" |
||||
#include "rocksdb/utilities/optimistic_transaction_db.h" |
||||
|
||||
using namespace rocksdb; |
||||
|
||||
std::string kDBPath = "/tmp/rocksdb_transaction_example"; |
||||
|
||||
int main() { |
||||
// open DB
|
||||
Options options; |
||||
options.create_if_missing = true; |
||||
DB* db; |
||||
OptimisticTransactionDB* txn_db; |
||||
|
||||
Status s = OptimisticTransactionDB::Open(options, kDBPath, &txn_db); |
||||
assert(s.ok()); |
||||
db = txn_db->GetBaseDB(); |
||||
|
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
OptimisticTransactionOptions txn_options; |
||||
std::string value; |
||||
|
||||
////////////////////////////////////////////////////////
|
||||
//
|
||||
// Simple OptimisticTransaction Example ("Read Committed")
|
||||
//
|
||||
////////////////////////////////////////////////////////
|
||||
|
||||
// Start a transaction
|
||||
OptimisticTransaction* txn = txn_db->BeginTransaction(write_options); |
||||
assert(txn); |
||||
|
||||
// Read a key in this transaction
|
||||
s = txn->Get(read_options, "abc", &value); |
||||
assert(s.IsNotFound()); |
||||
|
||||
// Write a key in this transaction
|
||||
txn->Put("abc", "def"); |
||||
|
||||
// Read a key OUTSIDE this transaction. Does not affect txn.
|
||||
s = db->Get(read_options, "abc", &value); |
||||
|
||||
// Write a key OUTSIDE of this transaction.
|
||||
// Does not affect txn since this is an unrelated key. If we wrote key 'abc'
|
||||
// here, the transaction would fail to commit.
|
||||
s = db->Put(write_options, "xyz", "zzz"); |
||||
|
||||
// Commit transaction
|
||||
s = txn->Commit(); |
||||
assert(s.ok()); |
||||
delete txn; |
||||
|
||||
////////////////////////////////////////////////////////
|
||||
//
|
||||
// "Repeatable Read" (Snapshot Isolation) Example
|
||||
// -- Using a single Snapshot
|
||||
//
|
||||
////////////////////////////////////////////////////////
|
||||
|
||||
// Set a snapshot at start of transaction by setting set_snapshot=true
|
||||
txn_options.set_snapshot = true; |
||||
txn = txn_db->BeginTransaction(write_options, txn_options); |
||||
|
||||
const Snapshot* snapshot = txn->GetSnapshot(); |
||||
|
||||
// Write a key OUTSIDE of transaction
|
||||
db->Put(write_options, "abc", "xyz"); |
||||
|
||||
// Read a key using the snapshot
|
||||
read_options.snapshot = snapshot; |
||||
s = txn->GetForUpdate(read_options, "abc", &value); |
||||
assert(value == "def"); |
||||
|
||||
// Attempt to commit transaction
|
||||
s = txn->Commit(); |
||||
|
||||
// Transaction could not commit since the write outside of the txn conflicted
|
||||
// with the read!
|
||||
assert(s.IsBusy()); |
||||
|
||||
delete txn; |
||||
// Clear snapshot from read options since it is no longer valid
|
||||
read_options.snapshot = nullptr; |
||||
snapshot = nullptr; |
||||
|
||||
////////////////////////////////////////////////////////
|
||||
//
|
||||
// "Read Committed" (Monotonic Atomic Views) Example
|
||||
// --Using multiple Snapshots
|
||||
//
|
||||
////////////////////////////////////////////////////////
|
||||
|
||||
// In this example, we set the snapshot multiple times. This is probably
|
||||
// only necessary if you have very strict isolation requirements to
|
||||
// implement.
|
||||
|
||||
// Set a snapshot at start of transaction
|
||||
txn_options.set_snapshot = true; |
||||
txn = txn_db->BeginTransaction(write_options, txn_options); |
||||
|
||||
// Do some reads and writes to key "x"
|
||||
read_options.snapshot = db->GetSnapshot(); |
||||
s = txn->Get(read_options, "x", &value); |
||||
txn->Put("x", "x"); |
||||
|
||||
// Do a write outside of the transaction to key "y"
|
||||
s = db->Put(write_options, "y", "y"); |
||||
|
||||
// Set a new snapshot in the transaction
|
||||
txn->SetSnapshot(); |
||||
read_options.snapshot = db->GetSnapshot(); |
||||
|
||||
// Do some reads and writes to key "y"
|
||||
s = txn->GetForUpdate(read_options, "y", &value); |
||||
txn->Put("y", "y"); |
||||
|
||||
// Commit. Since the snapshot was advanced, the write done outside of the
|
||||
// transaction does not prevent this transaction from Committing.
|
||||
s = txn->Commit(); |
||||
assert(s.ok()); |
||||
delete txn; |
||||
// Clear snapshot from read options since it is no longer valid
|
||||
read_options.snapshot = nullptr; |
||||
|
||||
// Cleanup
|
||||
delete txn_db; |
||||
DestroyDB(kDBPath, options); |
||||
return 0; |
||||
} |
||||
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,233 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#pragma once |
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include <string> |
||||
#include <vector> |
||||
|
||||
#include "rocksdb/comparator.h" |
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/status.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
class OptimisticTransactionDB; |
||||
class WriteBatchWithIndex; |
||||
|
||||
// Provides BEGIN/COMMIT/ROLLBACK transactions for batched writes.
|
||||
//
|
||||
// The current implementation provides optimistic concurrency control.
|
||||
// Transactional reads/writes will not block other operations in the
|
||||
// db. At commit time, the batch of writes will only be written if there have
|
||||
// been no other writes to any keys read or written by this transaction.
|
||||
// Otherwise, the commit will return an error.
|
||||
//
|
||||
// A new optimistic transaction is created by calling
|
||||
// OptimisticTransactionDB::BeginTransaction().
|
||||
// Only reads/writes done through this transaction object will be a part of the
|
||||
// transaction. Any other reads/writes will not be tracked by this
|
||||
// transaction.
|
||||
//
|
||||
// For example, reading data via OptimisticTransaction::GetForUpdate() will
|
||||
// prevent the transaction from committing if this key is written to outside of
|
||||
// this transaction. Any reads done via DB::Get() will not be checked for
|
||||
// conflicts at commit time.
|
||||
//
|
||||
// It is up to the caller to synchronize access to this object.
|
||||
//
|
||||
// See examples/transaction_example.cc for some simple examples.
|
||||
//
|
||||
// TODO(agiardullo): Not yet implemented:
|
||||
// -Transaction support for iterators
|
||||
// -Ensuring memtable holds large enough history to check for conflicts
|
||||
// -Support for using Transactions with DBWithTTL
|
||||
|
||||
// Options to use when starting an Optimistic Transaction
|
||||
struct OptimisticTransactionOptions { |
||||
// Setting set_snapshot=true is the same as calling SetSnapshot().
|
||||
bool set_snapshot = false; |
||||
|
||||
// Should be set if the DB has a non-default comparator.
|
||||
// See comment in WriteBatchWithIndex constructor.
|
||||
const Comparator* cmp = BytewiseComparator(); |
||||
}; |
||||
|
||||
class OptimisticTransaction { |
||||
public: |
||||
virtual ~OptimisticTransaction() {} |
||||
|
||||
// If SetSnapshot() is not called, all keys read/written through this
|
||||
// transaction will only be committed if there have been no writes to
|
||||
// these keys outside of this transaction *since the time each key
|
||||
// was first read/written* in this transaction.
|
||||
//
|
||||
// When SetSnapshot() is called, this transaction will create a Snapshot
|
||||
// to use for conflict validation of all future operations in the transaction.
|
||||
// All future keys read/written will only be committed if there have been
|
||||
// no writes to these keys outside of this transaction *since SetSnapshot()
|
||||
// was called.* Otherwise, Commit() will not succeed.
|
||||
//
|
||||
// It is not necessary to call SetSnapshot() if you only care about other
|
||||
// writes happening on keys *after* they have first been read/written in this
|
||||
// transaction. However, you should set a snapshot if you are concerned
|
||||
// with any other writes happening since a particular time (such as
|
||||
// the start of the transaction).
|
||||
//
|
||||
// SetSnapshot() may be called multiple times if you would like to change
|
||||
// the snapshot used for different operations in this transaction.
|
||||
//
|
||||
// Calling SetSnapshot will not affect the version of Data returned by Get()
|
||||
// methods. See OptimisticTransaction::Get() for more details.
|
||||
//
|
||||
// TODO(agiardullo): add better documentation here once memtable change are
|
||||
// committed
|
||||
virtual void SetSnapshot() = 0; |
||||
|
||||
// Returns the Snapshot created by the last call to SetSnapshot().
|
||||
//
|
||||
// REQUIRED: The returned Snapshot is only valid up until the next time
|
||||
// SetSnapshot() is called or the OptimisticTransaction is deleted.
|
||||
virtual const Snapshot* GetSnapshot() const = 0; |
||||
|
||||
// Write all batched keys to the db atomically if there have not been any
|
||||
// other writes performed on the keys read/written by this transaction.
|
||||
//
|
||||
// Currently, Commit() only checks the memtables to verify that there are no
|
||||
// other writes to these keys. If the memtable's history is not long
|
||||
// enough to verify that there are no conflicts, Commit() will return
|
||||
// a non-OK status.
|
||||
//
|
||||
// Returns OK on success, non-OK on failure.
|
||||
virtual Status Commit() = 0; |
||||
|
||||
// Discard all batched writes in this transaction.
|
||||
virtual void Rollback() = 0; |
||||
|
||||
// This function is similar to DB::Get() except it will also read pending
|
||||
// changes in this transaction.
|
||||
//
|
||||
// If read_options.snapshot is not set, the current version of the key will
|
||||
// be read. Calling SetSnapshot() does not affect the version of the data
|
||||
// returned.
|
||||
//
|
||||
// Note that setting read_options.snapshot will affect what is read from the
|
||||
// DB but will NOT change which keys are read from this transaction (the keys
|
||||
// in this transaction do not yet belong to any snapshot and will be fetched
|
||||
// regardless).
|
||||
//
|
||||
virtual Status Get(const ReadOptions& options, |
||||
ColumnFamilyHandle* column_family, const Slice& key, |
||||
std::string* value) = 0; |
||||
|
||||
virtual Status Get(const ReadOptions& options, const Slice& key, |
||||
std::string* value) = 0; |
||||
|
||||
virtual std::vector<Status> MultiGet( |
||||
const ReadOptions& options, |
||||
const std::vector<ColumnFamilyHandle*>& column_family, |
||||
const std::vector<Slice>& keys, std::vector<std::string>* values) = 0; |
||||
|
||||
virtual std::vector<Status> MultiGet(const ReadOptions& options, |
||||
const std::vector<Slice>& keys, |
||||
std::vector<std::string>* values) = 0; |
||||
|
||||
// Read this key and ensure that this transaction will only
|
||||
// be able to be committed if this key is not written outside this
|
||||
// transaction after it has first been read (or after the snapshot if a
|
||||
// snapshot is set in this transaction).
|
||||
|
||||
// This function is similar to OptimisticTransaction::Get() except it will
|
||||
// affect whether this transaction will be able to be committed.
|
||||
virtual Status GetForUpdate(const ReadOptions& options, |
||||
ColumnFamilyHandle* column_family, |
||||
const Slice& key, std::string* value) = 0; |
||||
|
||||
virtual Status GetForUpdate(const ReadOptions& options, const Slice& key, |
||||
std::string* value) = 0; |
||||
|
||||
virtual std::vector<Status> MultiGetForUpdate( |
||||
const ReadOptions& options, |
||||
const std::vector<ColumnFamilyHandle*>& column_family, |
||||
const std::vector<Slice>& keys, std::vector<std::string>* values) = 0; |
||||
|
||||
virtual std::vector<Status> MultiGetForUpdate( |
||||
const ReadOptions& options, const std::vector<Slice>& keys, |
||||
std::vector<std::string>* values) = 0; |
||||
|
||||
// Put, Merge, and Delete behave similarly to their corresponding
|
||||
// functions in WriteBatch. In addition, this transaction will only
|
||||
// be able to be committed if these keys are not written outside of this
|
||||
// transaction after they have been written by this transaction (or after the
|
||||
// snapshot if a snapshot is set in this transaction).
|
||||
virtual void Put(ColumnFamilyHandle* column_family, const Slice& key, |
||||
const Slice& value) = 0; |
||||
virtual void Put(const Slice& key, const Slice& value) = 0; |
||||
virtual void Put(ColumnFamilyHandle* column_family, const SliceParts& key, |
||||
const SliceParts& value) = 0; |
||||
virtual void Put(const SliceParts& key, const SliceParts& value) = 0; |
||||
|
||||
virtual void Merge(ColumnFamilyHandle* column_family, const Slice& key, |
||||
const Slice& value) = 0; |
||||
virtual void Merge(const Slice& key, const Slice& value) = 0; |
||||
|
||||
virtual void Delete(ColumnFamilyHandle* column_family, const Slice& key) = 0; |
||||
virtual void Delete(const Slice& key) = 0; |
||||
virtual void Delete(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key) = 0; |
||||
virtual void Delete(const SliceParts& key) = 0; |
||||
|
||||
// PutUntracked() will write a Put to the batch of operations to be committed
|
||||
// in this transaction. This write will only happen if this transaction
|
||||
// gets committed successfully. But unlike OptimisticTransaction::Put(),
|
||||
// no conflict checking will be done for this key. So any other writes to
|
||||
// this key outside of this transaction will not prevent this transaction from
|
||||
// committing.
|
||||
virtual void PutUntracked(ColumnFamilyHandle* column_family, const Slice& key, |
||||
const Slice& value) = 0; |
||||
virtual void PutUntracked(const Slice& key, const Slice& value) = 0; |
||||
virtual void PutUntracked(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key, const SliceParts& value) = 0; |
||||
virtual void PutUntracked(const SliceParts& key, const SliceParts& value) = 0; |
||||
|
||||
virtual void MergeUntracked(ColumnFamilyHandle* column_family, |
||||
const Slice& key, const Slice& value) = 0; |
||||
virtual void MergeUntracked(const Slice& key, const Slice& value) = 0; |
||||
|
||||
virtual void DeleteUntracked(ColumnFamilyHandle* column_family, |
||||
const Slice& key) = 0; |
||||
|
||||
virtual void DeleteUntracked(const Slice& key) = 0; |
||||
virtual void DeleteUntracked(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key) = 0; |
||||
virtual void DeleteUntracked(const SliceParts& key) = 0; |
||||
|
||||
// Similar to WriteBatch::PutLogData
|
||||
virtual void PutLogData(const Slice& blob) = 0; |
||||
|
||||
// Fetch the underlying write batch that contains all pending changes to be
|
||||
// committed.
|
||||
//
|
||||
// Note: You should not write or delete anything from the batch directly and
|
||||
// should only use the the functions in the OptimisticTransaction class to
|
||||
// write to this transaction.
|
||||
virtual WriteBatchWithIndex* GetWriteBatch() = 0; |
||||
|
||||
protected: |
||||
// To begin a new transaction, see OptimisticTransactionDB::BeginTransaction()
|
||||
explicit OptimisticTransaction(const OptimisticTransactionDB* db) {} |
||||
OptimisticTransaction() {} |
||||
|
||||
private: |
||||
// No copying allowed
|
||||
OptimisticTransaction(const OptimisticTransaction&); |
||||
void operator=(const OptimisticTransaction&); |
||||
}; |
||||
|
||||
} // namespace rocksdb
|
||||
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,64 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#pragma once |
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include <string> |
||||
#include <vector> |
||||
|
||||
#include "rocksdb/comparator.h" |
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/utilities/optimistic_transaction.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
class OptimisticTransaction; |
||||
|
||||
// Database with Transaction support.
|
||||
//
|
||||
// See optimistic_transaction.h and examples/transaction_example.cc
|
||||
|
||||
class OptimisticTransactionDB { |
||||
public: |
||||
// Open an OptimisticTransactionDB similar to DB::Open().
|
||||
static Status Open(const Options& options, const std::string& dbname, |
||||
OptimisticTransactionDB** dbptr); |
||||
|
||||
static Status Open(const DBOptions& db_options, const std::string& dbname, |
||||
const std::vector<ColumnFamilyDescriptor>& column_families, |
||||
std::vector<ColumnFamilyHandle*>* handles, |
||||
OptimisticTransactionDB** dbptr); |
||||
|
||||
virtual ~OptimisticTransactionDB() {} |
||||
|
||||
// Starts a new OptimisticTransaction. Passing set_snapshot=true has the same
|
||||
// effect
|
||||
// as calling SetSnapshot().
|
||||
//
|
||||
// Caller should delete the returned transaction after calling
|
||||
// Commit() or Rollback().
|
||||
virtual OptimisticTransaction* BeginTransaction( |
||||
const WriteOptions& write_options, |
||||
const OptimisticTransactionOptions& |
||||
txn_options = OptimisticTransactionOptions()) = 0; |
||||
|
||||
// Return the underlying Database that was opened
|
||||
virtual DB* GetBaseDB() = 0; |
||||
|
||||
protected: |
||||
// To Create an OptimisticTransactionDB, call Open()
|
||||
explicit OptimisticTransactionDB(DB* db) {} |
||||
OptimisticTransactionDB() {} |
||||
|
||||
private: |
||||
// No copying allowed
|
||||
OptimisticTransactionDB(const OptimisticTransactionDB&); |
||||
void operator=(const OptimisticTransactionDB&); |
||||
}; |
||||
|
||||
} // namespace rocksdb
|
||||
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,80 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include <string> |
||||
#include <vector> |
||||
|
||||
#include "utilities/transactions/optimistic_transaction_db_impl.h" |
||||
|
||||
#include "db/db_impl.h" |
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/options.h" |
||||
#include "rocksdb/utilities/optimistic_transaction_db.h" |
||||
#include "utilities/transactions/optimistic_transaction_impl.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
OptimisticTransaction* OptimisticTransactionDBImpl::BeginTransaction( |
||||
const WriteOptions& write_options, |
||||
const OptimisticTransactionOptions& txn_options) { |
||||
OptimisticTransaction* txn = |
||||
new OptimisticTransactionImpl(this, write_options, txn_options); |
||||
|
||||
return txn; |
||||
} |
||||
|
||||
Status OptimisticTransactionDB::Open(const Options& options, |
||||
const std::string& dbname, |
||||
OptimisticTransactionDB** dbptr) { |
||||
DBOptions db_options(options); |
||||
ColumnFamilyOptions cf_options(options); |
||||
std::vector<ColumnFamilyDescriptor> column_families; |
||||
column_families.push_back( |
||||
ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options)); |
||||
std::vector<ColumnFamilyHandle*> handles; |
||||
Status s = Open(db_options, dbname, column_families, &handles, dbptr); |
||||
if (s.ok()) { |
||||
assert(handles.size() == 1); |
||||
// i can delete the handle since DBImpl is always holding a reference to
|
||||
// default column family
|
||||
delete handles[0]; |
||||
} |
||||
|
||||
return s; |
||||
} |
||||
|
||||
Status OptimisticTransactionDB::Open( |
||||
const DBOptions& db_options, const std::string& dbname, |
||||
const std::vector<ColumnFamilyDescriptor>& column_families, |
||||
std::vector<ColumnFamilyHandle*>* handles, |
||||
OptimisticTransactionDB** dbptr) { |
||||
Status s; |
||||
DB* db; |
||||
|
||||
std::vector<ColumnFamilyDescriptor> column_families_copy = column_families; |
||||
|
||||
// Enable MemTable History if not already enabled
|
||||
for (auto& column_family : column_families_copy) { |
||||
ColumnFamilyOptions* options = &column_family.options; |
||||
|
||||
if (options->max_write_buffer_number_to_maintain == 0) { |
||||
// Setting to -1 will set the History size to max_write_buffer_number.
|
||||
options->max_write_buffer_number_to_maintain = -1; |
||||
} |
||||
} |
||||
|
||||
s = DB::Open(db_options, dbname, column_families_copy, handles, &db); |
||||
|
||||
if (s.ok()) { |
||||
*dbptr = new OptimisticTransactionDBImpl(db); |
||||
} |
||||
|
||||
return s; |
||||
} |
||||
|
||||
} // namespace rocksdb
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,33 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#pragma once |
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/options.h" |
||||
#include "rocksdb/utilities/optimistic_transaction_db.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
class OptimisticTransactionDBImpl : public OptimisticTransactionDB { |
||||
public: |
||||
explicit OptimisticTransactionDBImpl(DB* db) |
||||
: OptimisticTransactionDB(db), db_(db) {} |
||||
|
||||
~OptimisticTransactionDBImpl() {} |
||||
|
||||
OptimisticTransaction* BeginTransaction( |
||||
const WriteOptions& write_options, |
||||
const OptimisticTransactionOptions& txn_options) override; |
||||
|
||||
DB* GetBaseDB() override { return db_.get(); } |
||||
|
||||
private: |
||||
std::unique_ptr<DB> db_; |
||||
}; |
||||
|
||||
} // namespace rocksdb
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,339 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include "utilities/transactions/optimistic_transaction_impl.h" |
||||
|
||||
#include <string> |
||||
#include <vector> |
||||
|
||||
#include "db/column_family.h" |
||||
#include "db/db_impl.h" |
||||
#include "rocksdb/comparator.h" |
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/status.h" |
||||
#include "rocksdb/utilities/optimistic_transaction_db.h" |
||||
#include "util/string_util.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
struct WriteOptions; |
||||
|
||||
OptimisticTransactionImpl::OptimisticTransactionImpl( |
||||
OptimisticTransactionDB* txn_db, const WriteOptions& write_options, |
||||
const OptimisticTransactionOptions& txn_options) |
||||
: txn_db_(txn_db), |
||||
db_(txn_db->GetBaseDB()), |
||||
write_options_(write_options), |
||||
snapshot_(nullptr), |
||||
write_batch_(txn_options.cmp, 0, true) { |
||||
if (txn_options.set_snapshot) { |
||||
SetSnapshot(); |
||||
} else { |
||||
start_sequence_number_ = db_->GetLatestSequenceNumber(); |
||||
} |
||||
} |
||||
|
||||
OptimisticTransactionImpl::~OptimisticTransactionImpl() { |
||||
tracked_keys_.clear(); |
||||
if (snapshot_ != nullptr) { |
||||
db_->ReleaseSnapshot(snapshot_); |
||||
} |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::SetSnapshot() { |
||||
if (snapshot_ != nullptr) { |
||||
db_->ReleaseSnapshot(snapshot_); |
||||
} |
||||
|
||||
snapshot_ = db_->GetSnapshot(); |
||||
start_sequence_number_ = snapshot_->GetSequenceNumber(); |
||||
} |
||||
|
||||
Status OptimisticTransactionImpl::Commit() { |
||||
// Set up callback which will call CheckTransactionForConflicts() to
|
||||
// check whether this transaction is safe to be committed.
|
||||
OptimisticTransactionCallback callback(this); |
||||
|
||||
DBImpl* db_impl = dynamic_cast<DBImpl*>(db_->GetRootDB()); |
||||
if (db_impl == nullptr) { |
||||
// This should only happen if we support creating transactions from
|
||||
// a StackableDB and someone overrides GetRootDB().
|
||||
return Status::InvalidArgument( |
||||
"DB::GetRootDB() returned an unexpected DB class"); |
||||
} |
||||
|
||||
Status s = db_impl->WriteWithCallback( |
||||
write_options_, write_batch_.GetWriteBatch(), &callback); |
||||
|
||||
if (s.ok()) { |
||||
tracked_keys_.clear(); |
||||
write_batch_.Clear(); |
||||
} |
||||
|
||||
return s; |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::Rollback() { |
||||
tracked_keys_.clear(); |
||||
write_batch_.Clear(); |
||||
} |
||||
|
||||
// Record this key so that we can check it for conflicts at commit time.
|
||||
void OptimisticTransactionImpl::RecordOperation( |
||||
ColumnFamilyHandle* column_family, const Slice& key) { |
||||
uint32_t cfh_id = GetColumnFamilyID(column_family); |
||||
|
||||
SequenceNumber seq; |
||||
if (snapshot_) { |
||||
seq = start_sequence_number_; |
||||
} else { |
||||
seq = db_->GetLatestSequenceNumber(); |
||||
} |
||||
|
||||
std::string key_str = key.ToString(); |
||||
|
||||
auto iter = tracked_keys_[cfh_id].find(key_str); |
||||
if (iter == tracked_keys_[cfh_id].end()) { |
||||
// key not yet seen, store it.
|
||||
tracked_keys_[cfh_id].insert({std::move(key_str), seq}); |
||||
} else { |
||||
SequenceNumber old_seq = iter->second; |
||||
if (seq < old_seq) { |
||||
// Snapshot has changed since we last saw this key, need to
|
||||
// store the earliest seen sequence number.
|
||||
tracked_keys_[cfh_id][key_str] = seq; |
||||
} |
||||
} |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::RecordOperation( |
||||
ColumnFamilyHandle* column_family, const SliceParts& key) { |
||||
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()); |
||||
} |
||||
|
||||
RecordOperation(column_family, str); |
||||
} |
||||
|
||||
Status OptimisticTransactionImpl::Get(const ReadOptions& read_options, |
||||
ColumnFamilyHandle* column_family, |
||||
const Slice& key, std::string* value) { |
||||
return write_batch_.GetFromBatchAndDB(db_, read_options, column_family, key, |
||||
value); |
||||
} |
||||
|
||||
Status OptimisticTransactionImpl::GetForUpdate( |
||||
const ReadOptions& read_options, ColumnFamilyHandle* column_family, |
||||
const Slice& key, std::string* value) { |
||||
// Regardless of whether the Get succeeded, track this key.
|
||||
RecordOperation(column_family, key); |
||||
|
||||
return Get(read_options, column_family, key, value); |
||||
} |
||||
|
||||
std::vector<Status> OptimisticTransactionImpl::MultiGet( |
||||
const ReadOptions& read_options, |
||||
const std::vector<ColumnFamilyHandle*>& column_family, |
||||
const std::vector<Slice>& keys, std::vector<std::string>* values) { |
||||
// Regardless of whether the MultiGet succeeded, track these keys.
|
||||
size_t num_keys = keys.size(); |
||||
values->resize(num_keys); |
||||
|
||||
// TODO(agiardullo): optimize multiget?
|
||||
std::vector<Status> stat_list(num_keys); |
||||
for (size_t i = 0; i < num_keys; ++i) { |
||||
std::string* value = &(*values)[i]; |
||||
stat_list[i] = Get(read_options, column_family[i], keys[i], value); |
||||
} |
||||
|
||||
return stat_list; |
||||
} |
||||
|
||||
std::vector<Status> OptimisticTransactionImpl::MultiGetForUpdate( |
||||
const ReadOptions& read_options, |
||||
const std::vector<ColumnFamilyHandle*>& column_family, |
||||
const std::vector<Slice>& keys, std::vector<std::string>* values) { |
||||
// Regardless of whether the MultiGet succeeded, track these keys.
|
||||
size_t num_keys = keys.size(); |
||||
values->resize(num_keys); |
||||
|
||||
// TODO(agiardullo): optimize multiget?
|
||||
std::vector<Status> stat_list(num_keys); |
||||
for (size_t i = 0; i < num_keys; ++i) { |
||||
// Regardless of whether the Get succeeded, track this key.
|
||||
RecordOperation(column_family[i], keys[i]); |
||||
|
||||
std::string* value = &(*values)[i]; |
||||
stat_list[i] = Get(read_options, column_family[i], keys[i], value); |
||||
} |
||||
|
||||
return stat_list; |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::Put(ColumnFamilyHandle* column_family, |
||||
const Slice& key, const Slice& value) { |
||||
RecordOperation(column_family, key); |
||||
|
||||
write_batch_.Put(column_family, key, value); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::Put(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key, |
||||
const SliceParts& value) { |
||||
RecordOperation(column_family, key); |
||||
|
||||
write_batch_.Put(column_family, key, value); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::Merge(ColumnFamilyHandle* column_family, |
||||
const Slice& key, const Slice& value) { |
||||
RecordOperation(column_family, key); |
||||
|
||||
write_batch_.Merge(column_family, key, value); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::Delete(ColumnFamilyHandle* column_family, |
||||
const Slice& key) { |
||||
RecordOperation(column_family, key); |
||||
|
||||
write_batch_.Delete(column_family, key); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::Delete(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key) { |
||||
RecordOperation(column_family, key); |
||||
|
||||
write_batch_.Delete(column_family, key); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::PutUntracked(ColumnFamilyHandle* column_family, |
||||
const Slice& key, |
||||
const Slice& value) { |
||||
write_batch_.Put(column_family, key, value); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::PutUntracked(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key, |
||||
const SliceParts& value) { |
||||
write_batch_.Put(column_family, key, value); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::MergeUntracked( |
||||
ColumnFamilyHandle* column_family, const Slice& key, const Slice& value) { |
||||
write_batch_.Merge(column_family, key, value); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::DeleteUntracked( |
||||
ColumnFamilyHandle* column_family, const Slice& key) { |
||||
write_batch_.Delete(column_family, key); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::DeleteUntracked( |
||||
ColumnFamilyHandle* column_family, const SliceParts& key) { |
||||
write_batch_.Delete(column_family, key); |
||||
} |
||||
|
||||
void OptimisticTransactionImpl::PutLogData(const Slice& blob) { |
||||
write_batch_.PutLogData(blob); |
||||
} |
||||
|
||||
WriteBatchWithIndex* OptimisticTransactionImpl::GetWriteBatch() { |
||||
return &write_batch_; |
||||
} |
||||
|
||||
// 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 OptimisticTransactionImpl::CheckTransactionForConflicts(DB* db) { |
||||
Status result; |
||||
|
||||
assert(dynamic_cast<DBImpl*>(db) != nullptr); |
||||
auto db_impl = reinterpret_cast<DBImpl*>(db); |
||||
|
||||
for (auto& tracked_keys_iter : tracked_keys_) { |
||||
uint32_t cf_id = tracked_keys_iter.first; |
||||
const auto& keys = tracked_keys_iter.second; |
||||
|
||||
SuperVersion* sv = db_impl->GetAndRefSuperVersion(cf_id); |
||||
if (sv == nullptr) { |
||||
result = |
||||
Status::Busy("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; |
||||
|
||||
// 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 have been flushed recently, we cannot rely on them to tell
|
||||
// whether there have been any recent writes and must fail this
|
||||
// transaction.
|
||||
if (earliest_seq == kMaxSequenceNumber) { |
||||
// The age of this memtable is unknown. Cannot rely on it to check
|
||||
// for recent writes.
|
||||
result = Status::Busy( |
||||
"Could not commit transaction with as the MemTable does not " |
||||
"countain a long enough history to check write at SequenceNumber: ", |
||||
ToString(key_seq)); |
||||
|
||||
} else if (key_seq < earliest_seq) { |
||||
// The age of this memtable is too new to use to check for recent
|
||||
// writes.
|
||||
char msg[255]; |
||||
snprintf( |
||||
msg, sizeof(msg), |
||||
"Could not commit transaction with write at SequenceNumber %lu " |
||||
"as the MemTable only contains changes newer than SequenceNumber " |
||||
"%lu.", |
||||
key_seq, earliest_seq); |
||||
result = Status::Busy(msg); |
||||
} else { |
||||
SequenceNumber seq = kMaxSequenceNumber; |
||||
Status s = db_impl->GetLatestSequenceForKeyFromMemtable(sv, key, &seq); |
||||
if (!s.ok()) { |
||||
result = s; |
||||
} else if (seq != kMaxSequenceNumber && seq > key_seq) { |
||||
result = Status::Busy(); |
||||
} |
||||
} |
||||
|
||||
if (!result.ok()) { |
||||
break; |
||||
} |
||||
} |
||||
|
||||
db_impl->ReturnAndCleanupSuperVersion(cf_id, sv); |
||||
|
||||
if (!result.ok()) { |
||||
break; |
||||
} |
||||
} |
||||
|
||||
return result; |
||||
} |
||||
|
||||
} // namespace rocksdb
|
||||
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,196 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#pragma once |
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include <string> |
||||
#include <unordered_map> |
||||
#include <vector> |
||||
|
||||
#include "db/write_callback.h" |
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/slice.h" |
||||
#include "rocksdb/status.h" |
||||
#include "rocksdb/types.h" |
||||
#include "rocksdb/utilities/optimistic_transaction.h" |
||||
#include "rocksdb/utilities/optimistic_transaction_db.h" |
||||
#include "rocksdb/utilities/write_batch_with_index.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
using TransactionKeyMap = |
||||
std::unordered_map<uint32_t, |
||||
std::unordered_map<std::string, SequenceNumber>>; |
||||
|
||||
class OptimisticTransactionImpl : public OptimisticTransaction { |
||||
public: |
||||
OptimisticTransactionImpl(OptimisticTransactionDB* db, |
||||
const WriteOptions& write_options, |
||||
const OptimisticTransactionOptions& txn_options); |
||||
|
||||
virtual ~OptimisticTransactionImpl(); |
||||
|
||||
Status Commit() override; |
||||
|
||||
void Rollback() override; |
||||
|
||||
Status Get(const ReadOptions& options, ColumnFamilyHandle* column_family, |
||||
const Slice& key, std::string* value) override; |
||||
|
||||
Status Get(const ReadOptions& options, const Slice& key, |
||||
std::string* value) override { |
||||
return Get(options, db_->DefaultColumnFamily(), key, value); |
||||
} |
||||
|
||||
Status GetForUpdate(const ReadOptions& options, |
||||
ColumnFamilyHandle* column_family, const Slice& key, |
||||
std::string* value) override; |
||||
|
||||
Status GetForUpdate(const ReadOptions& options, const Slice& key, |
||||
std::string* value) override { |
||||
return GetForUpdate(options, db_->DefaultColumnFamily(), key, value); |
||||
} |
||||
|
||||
std::vector<Status> MultiGet( |
||||
const ReadOptions& options, |
||||
const std::vector<ColumnFamilyHandle*>& column_family, |
||||
const std::vector<Slice>& keys, |
||||
std::vector<std::string>* values) override; |
||||
|
||||
std::vector<Status> MultiGet(const ReadOptions& options, |
||||
const std::vector<Slice>& keys, |
||||
std::vector<std::string>* values) override { |
||||
return MultiGet(options, std::vector<ColumnFamilyHandle*>( |
||||
keys.size(), db_->DefaultColumnFamily()), |
||||
keys, values); |
||||
} |
||||
|
||||
std::vector<Status> MultiGetForUpdate( |
||||
const ReadOptions& options, |
||||
const std::vector<ColumnFamilyHandle*>& column_family, |
||||
const std::vector<Slice>& keys, |
||||
std::vector<std::string>* values) override; |
||||
|
||||
std::vector<Status> MultiGetForUpdate( |
||||
const ReadOptions& options, const std::vector<Slice>& keys, |
||||
std::vector<std::string>* values) override { |
||||
return MultiGetForUpdate(options, |
||||
std::vector<ColumnFamilyHandle*>( |
||||
keys.size(), db_->DefaultColumnFamily()), |
||||
keys, values); |
||||
} |
||||
|
||||
void Put(ColumnFamilyHandle* column_family, const Slice& key, |
||||
const Slice& value) override; |
||||
void Put(const Slice& key, const Slice& value) override { |
||||
Put(nullptr, key, value); |
||||
} |
||||
|
||||
void Put(ColumnFamilyHandle* column_family, const SliceParts& key, |
||||
const SliceParts& value) override; |
||||
void Put(const SliceParts& key, const SliceParts& value) override { |
||||
Put(nullptr, key, value); |
||||
} |
||||
|
||||
void Merge(ColumnFamilyHandle* column_family, const Slice& key, |
||||
const Slice& value) override; |
||||
void Merge(const Slice& key, const Slice& value) override { |
||||
Merge(nullptr, key, value); |
||||
} |
||||
|
||||
void Delete(ColumnFamilyHandle* column_family, const Slice& key) override; |
||||
void Delete(const Slice& key) override { Delete(nullptr, key); } |
||||
void Delete(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key) override; |
||||
void Delete(const SliceParts& key) override { Delete(nullptr, key); } |
||||
|
||||
void PutUntracked(ColumnFamilyHandle* column_family, const Slice& key, |
||||
const Slice& value) override; |
||||
void PutUntracked(const Slice& key, const Slice& value) override { |
||||
PutUntracked(nullptr, key, value); |
||||
} |
||||
|
||||
void PutUntracked(ColumnFamilyHandle* column_family, const SliceParts& key, |
||||
const SliceParts& value) override; |
||||
void PutUntracked(const SliceParts& key, const SliceParts& value) override { |
||||
PutUntracked(nullptr, key, value); |
||||
} |
||||
|
||||
void MergeUntracked(ColumnFamilyHandle* column_family, const Slice& key, |
||||
const Slice& value) override; |
||||
void MergeUntracked(const Slice& key, const Slice& value) override { |
||||
MergeUntracked(nullptr, key, value); |
||||
} |
||||
|
||||
void DeleteUntracked(ColumnFamilyHandle* column_family, |
||||
const Slice& key) override; |
||||
void DeleteUntracked(const Slice& key) override { |
||||
DeleteUntracked(nullptr, key); |
||||
} |
||||
void DeleteUntracked(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key) override; |
||||
void DeleteUntracked(const SliceParts& key) override { |
||||
DeleteUntracked(nullptr, key); |
||||
} |
||||
|
||||
void PutLogData(const Slice& blob) override; |
||||
|
||||
const TransactionKeyMap* GetTrackedKeys() const { return &tracked_keys_; } |
||||
|
||||
const Snapshot* GetSnapshot() const override { return snapshot_; } |
||||
|
||||
void SetSnapshot() override; |
||||
|
||||
WriteBatchWithIndex* GetWriteBatch() override; |
||||
|
||||
protected: |
||||
OptimisticTransactionDB* const txn_db_; |
||||
DB* db_; |
||||
const WriteOptions write_options_; |
||||
const Snapshot* snapshot_; |
||||
SequenceNumber start_sequence_number_; |
||||
WriteBatchWithIndex write_batch_; |
||||
|
||||
private: |
||||
// Map of Column Family IDs to keys and their sequence numbers
|
||||
TransactionKeyMap tracked_keys_; |
||||
|
||||
friend class OptimisticTransactionCallback; |
||||
|
||||
// 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 RecordOperation(ColumnFamilyHandle* column_family, const Slice& key); |
||||
void RecordOperation(ColumnFamilyHandle* column_family, |
||||
const SliceParts& key); |
||||
|
||||
// No copying allowed
|
||||
OptimisticTransactionImpl(const OptimisticTransactionImpl&); |
||||
void operator=(const OptimisticTransactionImpl&); |
||||
}; |
||||
|
||||
// Used at commit time to trigger transaction validation
|
||||
class OptimisticTransactionCallback : public WriteCallback { |
||||
public: |
||||
explicit OptimisticTransactionCallback(OptimisticTransactionImpl* txn) |
||||
: txn_(txn) {} |
||||
|
||||
Status Callback(DB* db) override { |
||||
return txn_->CheckTransactionForConflicts(db); |
||||
} |
||||
|
||||
private: |
||||
OptimisticTransactionImpl* txn_; |
||||
}; |
||||
|
||||
} // namespace rocksdb
|
||||
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,846 @@ |
||||
// Copyright (c) 2015, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
#include <string> |
||||
|
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/utilities/optimistic_transaction.h" |
||||
#include "rocksdb/utilities/optimistic_transaction_db.h" |
||||
#include "util/logging.h" |
||||
#include "util/testharness.h" |
||||
|
||||
using std::string; |
||||
|
||||
namespace rocksdb { |
||||
|
||||
class OptimisticTransactionTest : public testing::Test { |
||||
public: |
||||
OptimisticTransactionDB* txn_db; |
||||
DB* db; |
||||
string dbname; |
||||
Options options; |
||||
|
||||
OptimisticTransactionTest() { |
||||
options.create_if_missing = true; |
||||
options.max_write_buffer_number = 2; |
||||
dbname = test::TmpDir() + "/optimistic_transaction_testdb"; |
||||
|
||||
DestroyDB(dbname, options); |
||||
Status s = OptimisticTransactionDB::Open(options, dbname, &txn_db); |
||||
assert(s.ok()); |
||||
db = txn_db->GetBaseDB(); |
||||
} |
||||
|
||||
~OptimisticTransactionTest() { |
||||
delete txn_db; |
||||
DestroyDB(dbname, options); |
||||
} |
||||
}; |
||||
|
||||
TEST_F(OptimisticTransactionTest, SuccessTest) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, Slice("foo"), Slice("bar")); |
||||
db->Put(write_options, Slice("foo2"), Slice("bar")); |
||||
|
||||
OptimisticTransaction* 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); |
||||
|
||||
db->Get(read_options, "foo", &value); |
||||
ASSERT_EQ(value, "bar2"); |
||||
|
||||
delete txn; |
||||
} |
||||
|
||||
TEST_F(OptimisticTransactionTest, WriteConflictTest) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, "foo", "bar"); |
||||
db->Put(write_options, "foo2", "bar"); |
||||
|
||||
OptimisticTransaction* 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 = db->Put(write_options, "foo", "barz"); |
||||
ASSERT_OK(s); |
||||
|
||||
s = db->Get(read_options, "foo", &value); |
||||
ASSERT_EQ(value, "barz"); |
||||
|
||||
s = txn->Commit(); |
||||
ASSERT_NOK(s); // Txn should not commit
|
||||
|
||||
// Verify that transaction did not write anything
|
||||
db->Get(read_options, "foo", &value); |
||||
ASSERT_EQ(value, "barz"); |
||||
db->Get(read_options, "foo2", &value); |
||||
ASSERT_EQ(value, "bar"); |
||||
|
||||
delete txn; |
||||
} |
||||
|
||||
TEST_F(OptimisticTransactionTest, WriteConflictTest2) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
OptimisticTransactionOptions txn_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, "foo", "bar"); |
||||
db->Put(write_options, "foo2", "bar"); |
||||
|
||||
txn_options.set_snapshot = true; |
||||
OptimisticTransaction* txn = |
||||
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); |
||||
|
||||
txn->Put("foo", "bar2"); // Conflicts with write done after snapshot taken
|
||||
|
||||
s = db->Get(read_options, "foo", &value); |
||||
ASSERT_EQ(value, "barz"); |
||||
|
||||
s = txn->Commit(); |
||||
ASSERT_NOK(s); // Txn should not commit
|
||||
|
||||
// Verify that transaction did not write anything
|
||||
db->Get(read_options, "foo", &value); |
||||
ASSERT_EQ(value, "barz"); |
||||
db->Get(read_options, "foo2", &value); |
||||
ASSERT_EQ(value, "bar"); |
||||
|
||||
delete txn; |
||||
} |
||||
|
||||
TEST_F(OptimisticTransactionTest, ReadConflictTest) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options, snapshot_read_options; |
||||
OptimisticTransactionOptions txn_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, "foo", "bar"); |
||||
db->Put(write_options, "foo2", "bar"); |
||||
|
||||
txn_options.set_snapshot = true; |
||||
OptimisticTransaction* 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 = db->Put(write_options, "foo", "barz"); |
||||
ASSERT_OK(s); |
||||
|
||||
s = db->Get(read_options, "foo", &value); |
||||
ASSERT_EQ(value, "barz"); |
||||
|
||||
s = txn->Commit(); |
||||
ASSERT_NOK(s); // 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_F(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; |
||||
|
||||
OptimisticTransaction* txn = txn_db->BeginTransaction(write_options); |
||||
ASSERT_TRUE(txn); |
||||
|
||||
txn->Put("x", "y"); |
||||
|
||||
s = txn->Commit(); |
||||
ASSERT_OK(s); |
||||
|
||||
delete txn; |
||||
} |
||||
|
||||
TEST_F(OptimisticTransactionTest, FlushTest) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options, snapshot_read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, Slice("foo"), Slice("bar")); |
||||
db->Put(write_options, Slice("foo2"), Slice("bar")); |
||||
|
||||
OptimisticTransaction* 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 = 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_F(OptimisticTransactionTest, FlushTest2) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options, snapshot_read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, Slice("foo"), Slice("bar")); |
||||
db->Put(write_options, Slice("foo2"), Slice("bar")); |
||||
|
||||
OptimisticTransaction* 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 = db->Put(write_options, "dummy", "dummy"); |
||||
ASSERT_OK(s); |
||||
|
||||
// force a memtable flush
|
||||
FlushOptions flush_ops; |
||||
db->Flush(flush_ops); |
||||
|
||||
// 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
|
||||
db->Flush(flush_ops); |
||||
|
||||
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.
|
||||
db->Flush(flush_ops); |
||||
|
||||
s = txn->Commit(); |
||||
// txn should not commit since MemTableList History is not large enough
|
||||
ASSERT_NOK(s); |
||||
|
||||
db->Get(read_options, "foo", &value); |
||||
ASSERT_EQ(value, "bar"); |
||||
|
||||
delete txn; |
||||
} |
||||
|
||||
TEST_F(OptimisticTransactionTest, NoSnapshotTest) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, "AAA", "bar"); |
||||
|
||||
OptimisticTransaction* txn = txn_db->BeginTransaction(write_options); |
||||
ASSERT_TRUE(txn); |
||||
|
||||
// Modify key after transaction start
|
||||
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_F(OptimisticTransactionTest, MultipleSnapshotTest) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options, snapshot_read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
db->Put(write_options, "AAA", "bar"); |
||||
db->Put(write_options, "BBB", "bar"); |
||||
db->Put(write_options, "CCC", "bar"); |
||||
|
||||
OptimisticTransaction* txn = txn_db->BeginTransaction(write_options); |
||||
ASSERT_TRUE(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
|
||||
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"); |
||||
|
||||
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 = 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 = txn_db->BeginTransaction(write_options); |
||||
|
||||
// Potentially conflicting writes
|
||||
db->Put(write_options, "ZZZ", "zzz"); |
||||
db->Put(write_options, "XXX", "xxx"); |
||||
|
||||
txn->SetSnapshot(); |
||||
|
||||
OptimisticTransactionOptions txn_options; |
||||
txn_options.set_snapshot = true; |
||||
OptimisticTransaction* 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_NOK(s); |
||||
|
||||
delete txn2; |
||||
} |
||||
|
||||
TEST_F(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 = db->CreateColumnFamily(cf_options, "CFA", &cfa); |
||||
ASSERT_OK(s); |
||||
s = db->CreateColumnFamily(cf_options, "CFB", &cfb); |
||||
ASSERT_OK(s); |
||||
|
||||
delete cfa; |
||||
delete cfb; |
||||
delete txn_db; |
||||
|
||||
// open DB with three column families
|
||||
std::vector<ColumnFamilyDescriptor> column_families; |
||||
// have to open default column family
|
||||
column_families.push_back( |
||||
ColumnFamilyDescriptor(kDefaultColumnFamilyName, ColumnFamilyOptions())); |
||||
// open the new column families
|
||||
column_families.push_back( |
||||
ColumnFamilyDescriptor("CFA", ColumnFamilyOptions())); |
||||
column_families.push_back( |
||||
ColumnFamilyDescriptor("CFB", ColumnFamilyOptions())); |
||||
std::vector<ColumnFamilyHandle*> handles; |
||||
s = OptimisticTransactionDB::Open(options, dbname, column_families, &handles, |
||||
&txn_db); |
||||
ASSERT_OK(s); |
||||
db = txn_db->GetBaseDB(); |
||||
|
||||
OptimisticTransaction* txn = txn_db->BeginTransaction(write_options); |
||||
ASSERT_TRUE(txn); |
||||
|
||||
txn->SetSnapshot(); |
||||
snapshot_read_options.snapshot = txn->GetSnapshot(); |
||||
|
||||
txn_options.set_snapshot = true; |
||||
OptimisticTransaction* 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 = db->Write(write_options, &batch); |
||||
ASSERT_OK(s); |
||||
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)); |
||||
|
||||
// Txn should commit
|
||||
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"); |
||||
// 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_NOK(s); |
||||
s = db->Get(read_options, handles[1], "AAAZZZ", &value); |
||||
ASSERT_EQ(value, "barbar"); |
||||
|
||||
delete txn; |
||||
delete txn2; |
||||
|
||||
txn = txn_db->BeginTransaction(write_options, txn_options); |
||||
snapshot_read_options.snapshot = txn->GetSnapshot(); |
||||
|
||||
txn2 = txn_db->BeginTransaction(write_options, txn_options); |
||||
ASSERT_TRUE(txn); |
||||
|
||||
std::vector<ColumnFamilyHandle*> multiget_cfh = {handles[1], handles[2], |
||||
handles[0], handles[2]}; |
||||
std::vector<Slice> multiget_keys = {"AAA", "AAAZZZ", "foo", "foo"}; |
||||
std::vector<std::string> values(4); |
||||
|
||||
std::vector<Status> results = txn->MultiGetForUpdate( |
||||
snapshot_read_options, multiget_cfh, multiget_keys, &values); |
||||
ASSERT_OK(results[0]); |
||||
ASSERT_OK(results[1]); |
||||
ASSERT_OK(results[2]); |
||||
ASSERT_TRUE(results[3].IsNotFound()); |
||||
ASSERT_EQ(values[0], "bar"); |
||||
ASSERT_EQ(values[1], "barbar"); |
||||
ASSERT_EQ(values[2], "foo"); |
||||
|
||||
txn->Delete(handles[2], "ZZZ"); |
||||
txn->Put(handles[2], "ZZZ", "YYY"); |
||||
txn->Put(handles[2], "ZZZ", "YYYY"); |
||||
txn->Delete(handles[2], "ZZZ"); |
||||
txn->Put(handles[2], "AAAZZZ", "barbarbar"); |
||||
|
||||
// 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); |
||||
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_NOK(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_F(OptimisticTransactionTest, EmptyTest) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
s = db->Put(write_options, "aaa", "aaa"); |
||||
ASSERT_OK(s); |
||||
|
||||
OptimisticTransaction* 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 = db->Put(write_options, "aaa", "xxx"); |
||||
s = txn->Commit(); |
||||
ASSERT_NOK(s); |
||||
delete txn; |
||||
} |
||||
|
||||
TEST_F(OptimisticTransactionTest, PredicateManyPreceders) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options1, read_options2; |
||||
OptimisticTransactionOptions txn_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
txn_options.set_snapshot = true; |
||||
OptimisticTransaction* txn1 = |
||||
txn_db->BeginTransaction(write_options, txn_options); |
||||
read_options1.snapshot = txn1->GetSnapshot(); |
||||
|
||||
OptimisticTransaction* txn2 = txn_db->BeginTransaction(write_options); |
||||
txn2->SetSnapshot(); |
||||
read_options2.snapshot = txn2->GetSnapshot(); |
||||
|
||||
std::vector<Slice> multiget_keys = {"1", "2", "3"}; |
||||
std::vector<std::string> multiget_values; |
||||
|
||||
std::vector<Status> results = |
||||
txn1->MultiGetForUpdate(read_options1, multiget_keys, &multiget_values); |
||||
ASSERT_TRUE(results[1].IsNotFound()); |
||||
|
||||
txn2->Put("2", "x"); |
||||
|
||||
s = txn2->Commit(); |
||||
ASSERT_OK(s); |
||||
|
||||
multiget_values.clear(); |
||||
results = |
||||
txn1->MultiGetForUpdate(read_options1, multiget_keys, &multiget_values); |
||||
ASSERT_TRUE(results[1].IsNotFound()); |
||||
|
||||
// should not commit since txn2 wrote a key txn has read
|
||||
s = txn1->Commit(); |
||||
ASSERT_NOK(s); |
||||
|
||||
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_NOK(s); |
||||
|
||||
delete txn1; |
||||
delete txn2; |
||||
} |
||||
|
||||
TEST_F(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
|
||||
|
||||
OptimisticTransaction* txn1 = txn_db->BeginTransaction(write_options); |
||||
OptimisticTransaction* txn2 = txn_db->BeginTransaction(write_options); |
||||
|
||||
txn1->Put("1", "1"); |
||||
txn2->Put("1", "2"); |
||||
|
||||
s = txn1->Commit(); |
||||
ASSERT_OK(s); |
||||
|
||||
s = txn2->Commit(); |
||||
ASSERT_NOK(s); |
||||
|
||||
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_NOK(s); |
||||
|
||||
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_NOK(s); |
||||
|
||||
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 = db->Get(read_options, "1", &value); |
||||
ASSERT_OK(s); |
||||
ASSERT_EQ(value, "8"); |
||||
} |
||||
|
||||
TEST_F(OptimisticTransactionTest, UntrackedWrites) { |
||||
WriteOptions write_options; |
||||
ReadOptions read_options; |
||||
string value; |
||||
Status s; |
||||
|
||||
// Verify transaction rollback works for untracked keys.
|
||||
OptimisticTransaction* txn = txn_db->BeginTransaction(write_options); |
||||
txn->PutUntracked("untracked", "0"); |
||||
txn->Rollback(); |
||||
s = 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 = db->Put(write_options, "untracked", "x"); |
||||
ASSERT_OK(s); |
||||
|
||||
s = txn->Commit(); |
||||
ASSERT_OK(s); |
||||
|
||||
s = 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 = db->Delete(write_options, "tracked"); |
||||
|
||||
s = txn->Commit(); |
||||
ASSERT_NOK(s); |
||||
|
||||
s = db->Get(read_options, "untracked", &value); |
||||
ASSERT_TRUE(s.IsNotFound()); |
||||
|
||||
delete txn; |
||||
} |
||||
|
||||
} // namespace rocksdb
|
||||
|
||||
int main(int argc, char** argv) { |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
return RUN_ALL_TESTS(); |
||||
} |
||||
|
||||
#endif // ROCKSDB_LITE
|
Loading…
Reference in new issue