2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
// 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 <chrono>
|
|
|
|
#include <string>
|
|
|
|
#include <unordered_map>
|
2016-09-28 02:43:06 +02:00
|
|
|
#include <utility>
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
#include <vector>
|
|
|
|
|
|
|
|
#include "rocksdb/utilities/transaction.h"
|
|
|
|
#include "util/instrumented_mutex.h"
|
|
|
|
#include "util/thread_local.h"
|
|
|
|
#include "utilities/transactions/transaction_impl.h"
|
|
|
|
|
|
|
|
namespace rocksdb {
|
|
|
|
|
|
|
|
class ColumnFamilyHandle;
|
|
|
|
struct LockInfo;
|
|
|
|
struct LockMap;
|
|
|
|
struct LockMapStripe;
|
|
|
|
|
|
|
|
class Slice;
|
2016-02-02 02:07:05 +01:00
|
|
|
class TransactionDBImpl;
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
|
|
|
|
class TransactionLockMgr {
|
|
|
|
public:
|
2016-02-02 02:07:05 +01:00
|
|
|
TransactionLockMgr(TransactionDB* txn_db, size_t default_num_stripes,
|
|
|
|
int64_t max_num_locks,
|
2015-09-08 21:36:48 +02:00
|
|
|
std::shared_ptr<TransactionDBMutexFactory> factory);
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
|
|
|
|
~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.
|
2016-09-28 02:43:06 +02:00
|
|
|
Status TryLock(TransactionImpl* txn, uint32_t column_family_id,
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
const std::string& key, Env* env);
|
|
|
|
|
|
|
|
// Unlock a key locked by TryLock(). txn must be the same Transaction that
|
|
|
|
// locked this key.
|
|
|
|
void UnLock(const TransactionImpl* txn, const TransactionKeyMap* keys,
|
|
|
|
Env* env);
|
|
|
|
void UnLock(TransactionImpl* txn, uint32_t column_family_id,
|
|
|
|
const std::string& key, Env* env);
|
|
|
|
|
2016-09-28 02:43:06 +02:00
|
|
|
using LockStatusData = std::unordered_multimap<uint32_t, KeyLockInfo>;
|
|
|
|
LockStatusData GetLockStatusData();
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
private:
|
2016-02-02 02:07:05 +01:00
|
|
|
TransactionDBImpl* txn_db_impl_;
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
// 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_;
|
|
|
|
|
2016-09-28 10:23:33 +02:00
|
|
|
// 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_.
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
InstrumentedMutex lock_map_mutex_;
|
|
|
|
|
|
|
|
// Map of ColumnFamilyId to locked key info
|
|
|
|
using LockMaps = std::unordered_map<uint32_t, std::shared_ptr<LockMap>>;
|
|
|
|
LockMaps lock_maps_;
|
|
|
|
|
|
|
|
// Thread-local cache of entries in lock_maps_. This is an optimization
|
|
|
|
// to avoid acquiring a mutex in order to look up a LockMap
|
|
|
|
std::unique_ptr<ThreadLocalPtr> lock_maps_cache_;
|
|
|
|
|
2016-09-28 10:23:33 +02:00
|
|
|
// 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.
|
|
|
|
std::unordered_map<TransactionID, int> rev_wait_txn_map_;
|
|
|
|
// Maps from waiter -> waitee.
|
|
|
|
std::unordered_map<TransactionID, TransactionID> wait_txn_map_;
|
|
|
|
|
|
|
|
// Used to allocate mutexes/condvars to use when locking keys
|
|
|
|
std::shared_ptr<TransactionDBMutexFactory> mutex_factory_;
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
bool IsLockExpired(const LockInfo& lock_info, Env* env, uint64_t* wait_time);
|
|
|
|
|
|
|
|
std::shared_ptr<LockMap> GetLockMap(uint32_t column_family_id);
|
|
|
|
|
2016-09-28 02:43:06 +02:00
|
|
|
Status AcquireWithTimeout(TransactionImpl* txn, LockMap* lock_map,
|
|
|
|
LockMapStripe* stripe, uint32_t column_family_id,
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
const std::string& key, Env* env, int64_t timeout,
|
|
|
|
const LockInfo& lock_info);
|
|
|
|
|
|
|
|
Status AcquireLocked(LockMap* lock_map, LockMapStripe* stripe,
|
|
|
|
const std::string& key, Env* env,
|
2016-09-28 02:43:06 +02:00
|
|
|
const LockInfo& lock_info, uint64_t* wait_time,
|
|
|
|
TransactionID* txn_id);
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
|
2016-09-28 10:23:33 +02:00
|
|
|
bool IncrementWaiters(const TransactionImpl* txn, TransactionID wait_id);
|
|
|
|
void DecrementWaiters(const TransactionImpl* txn, TransactionID wait_id);
|
|
|
|
void DecrementWaitersImpl(const TransactionImpl* txn, TransactionID wait_id);
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
// No copying allowed
|
|
|
|
TransactionLockMgr(const TransactionLockMgr&);
|
|
|
|
void operator=(const TransactionLockMgr&);
|
|
|
|
};
|
|
|
|
|
|
|
|
} // namespace rocksdb
|
|
|
|
#endif // ROCKSDB_LITE
|