Range Locking: Implementation of range locking (#7506)
Summary: Range Locking - an implementation based on the locktree library - Add a RangeTreeLockManager and RangeTreeLockTracker which implement range locking using the locktree library. - Point locks are handled as locks on single-point ranges. - Add a unit test: range_locking_test Pull Request resolved: https://github.com/facebook/rocksdb/pull/7506 Reviewed By: akankshamahajan15 Differential Revision: D25320703 Pulled By: cheng-chang fbshipit-source-id: f86347384b42ba2b0257d67eca0f45f806b69da7main
parent
f4db3e4119
commit
daab7603f6
@ -0,0 +1,304 @@ |
||||
// 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 |
||||
#ifndef OS_WIN |
||||
|
||||
#include <algorithm> |
||||
#include <functional> |
||||
#include <string> |
||||
#include <thread> |
||||
|
||||
#include "db/db_impl/db_impl.h" |
||||
#include "port/port.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 "utilities/transactions/lock/point/point_lock_manager_test.h" |
||||
#include "utilities/transactions/pessimistic_transaction_db.h" |
||||
#include "utilities/transactions/transaction_test.h" |
||||
|
||||
using std::string; |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
class RangeLockingTest : public ::testing::Test { |
||||
public: |
||||
TransactionDB* db; |
||||
std::string dbname; |
||||
Options options; |
||||
|
||||
std::shared_ptr<RangeLockManagerHandle> range_lock_mgr; |
||||
TransactionDBOptions txn_db_options; |
||||
|
||||
RangeLockingTest() : db(nullptr) { |
||||
options.create_if_missing = true; |
||||
dbname = test::PerThreadDBPath("range_locking_testdb"); |
||||
|
||||
DestroyDB(dbname, options); |
||||
|
||||
range_lock_mgr.reset(NewRangeLockManager(nullptr)); |
||||
txn_db_options.lock_mgr_handle = range_lock_mgr; |
||||
|
||||
auto s = TransactionDB::Open(options, txn_db_options, dbname, &db); |
||||
assert(s.ok()); |
||||
} |
||||
|
||||
~RangeLockingTest() { |
||||
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.
|
||||
DestroyDB(dbname, options); |
||||
} |
||||
|
||||
PessimisticTransaction* NewTxn( |
||||
TransactionOptions txn_opt = TransactionOptions()) { |
||||
Transaction* txn = db->BeginTransaction(WriteOptions(), txn_opt); |
||||
return reinterpret_cast<PessimisticTransaction*>(txn); |
||||
} |
||||
}; |
||||
|
||||
// TODO: set a smaller lock wait timeout so that the test runs faster.
|
||||
TEST_F(RangeLockingTest, BasicRangeLocking) { |
||||
WriteOptions write_options; |
||||
TransactionOptions txn_options; |
||||
std::string value; |
||||
ReadOptions read_options; |
||||
auto cf = db->DefaultColumnFamily(); |
||||
|
||||
Transaction* txn0 = db->BeginTransaction(write_options, txn_options); |
||||
Transaction* txn1 = db->BeginTransaction(write_options, txn_options); |
||||
|
||||
// Get a range lock
|
||||
ASSERT_OK(txn0->GetRangeLock(cf, Endpoint("a"), Endpoint("c"))); |
||||
|
||||
// Check that range Lock inhibits an overlapping range lock
|
||||
{ |
||||
auto s = txn1->GetRangeLock(cf, Endpoint("b"), Endpoint("z")); |
||||
ASSERT_TRUE(s.IsTimedOut()); |
||||
} |
||||
|
||||
// Check that range Lock inhibits an overlapping point lock
|
||||
{ |
||||
auto s = txn1->GetForUpdate(read_options, cf, Slice("b"), &value); |
||||
ASSERT_TRUE(s.IsTimedOut()); |
||||
} |
||||
|
||||
// Get a point lock, check that it inhibits range locks
|
||||
ASSERT_OK(txn0->Put(cf, Slice("n"), Slice("value"))); |
||||
{ |
||||
auto s = txn1->GetRangeLock(cf, Endpoint("m"), Endpoint("p")); |
||||
ASSERT_TRUE(s.IsTimedOut()); |
||||
} |
||||
|
||||
ASSERT_OK(txn0->Commit()); |
||||
txn1->Rollback(); |
||||
|
||||
delete txn0; |
||||
delete txn1; |
||||
} |
||||
|
||||
TEST_F(RangeLockingTest, MyRocksLikeUpdate) { |
||||
WriteOptions write_options; |
||||
TransactionOptions txn_options; |
||||
Transaction* txn0 = db->BeginTransaction(write_options, txn_options); |
||||
auto cf = db->DefaultColumnFamily(); |
||||
Status s; |
||||
|
||||
// Get a range lock for the range we are about to update
|
||||
ASSERT_OK(txn0->GetRangeLock(cf, Endpoint("a"), Endpoint("c"))); |
||||
|
||||
bool try_range_lock_called = false; |
||||
|
||||
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( |
||||
"RangeTreeLockManager::TryRangeLock:enter", |
||||
[&](void* /*arg*/) { try_range_lock_called = true; }); |
||||
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); |
||||
|
||||
// For performance reasons, the following must NOT call lock_mgr->TryLock():
|
||||
// We verify that by checking the value of try_range_lock_called.
|
||||
ASSERT_OK(txn0->Put(cf, Slice("b"), Slice("value"), |
||||
/*assume_tracked=*/true)); |
||||
|
||||
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing(); |
||||
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks(); |
||||
ASSERT_FALSE(try_range_lock_called); |
||||
|
||||
txn0->Rollback(); |
||||
|
||||
delete txn0; |
||||
} |
||||
|
||||
TEST_F(RangeLockingTest, SnapshotValidation) { |
||||
Status s; |
||||
Slice key_slice = Slice("k"); |
||||
ColumnFamilyHandle* cfh = db->DefaultColumnFamily(); |
||||
|
||||
auto txn0 = NewTxn(); |
||||
txn0->Put(key_slice, Slice("initial")); |
||||
txn0->Commit(); |
||||
|
||||
// txn1
|
||||
auto txn1 = NewTxn(); |
||||
txn1->SetSnapshot(); |
||||
std::string val1; |
||||
ASSERT_OK(txn1->Get(ReadOptions(), cfh, key_slice, &val1)); |
||||
ASSERT_EQ(val1, "initial"); |
||||
val1 = val1 + std::string("-txn1"); |
||||
|
||||
ASSERT_OK(txn1->Put(cfh, key_slice, Slice(val1))); |
||||
|
||||
// txn2
|
||||
auto txn2 = NewTxn(); |
||||
txn2->SetSnapshot(); |
||||
std::string val2; |
||||
// This will see the original value as nothing is committed
|
||||
// This is also Get, so it is doesn't acquire any locks.
|
||||
ASSERT_OK(txn2->Get(ReadOptions(), cfh, key_slice, &val2)); |
||||
ASSERT_EQ(val2, "initial"); |
||||
|
||||
// txn1
|
||||
ASSERT_OK(txn1->Commit()); |
||||
|
||||
// txn2
|
||||
val2 = val2 + std::string("-txn2"); |
||||
// Now, this call should do Snapshot Validation and fail:
|
||||
s = txn2->Put(cfh, key_slice, Slice(val2)); |
||||
ASSERT_TRUE(s.IsBusy()); |
||||
|
||||
ASSERT_OK(txn2->Commit()); |
||||
|
||||
delete txn0; |
||||
delete txn1; |
||||
delete txn2; |
||||
} |
||||
|
||||
TEST_F(RangeLockingTest, MultipleTrxLockStatusData) { |
||||
WriteOptions write_options; |
||||
TransactionOptions txn_options; |
||||
auto cf = db->DefaultColumnFamily(); |
||||
|
||||
Transaction* txn0 = db->BeginTransaction(write_options, txn_options); |
||||
Transaction* txn1 = db->BeginTransaction(write_options, txn_options); |
||||
|
||||
// Get a range lock
|
||||
ASSERT_OK(txn0->GetRangeLock(cf, Endpoint("z"), Endpoint("z"))); |
||||
ASSERT_OK(txn1->GetRangeLock(cf, Endpoint("b"), Endpoint("e"))); |
||||
|
||||
auto s = range_lock_mgr->GetRangeLockStatusData(); |
||||
ASSERT_EQ(s.size(), 2); |
||||
for (auto it = s.begin(); it != s.end(); ++it) { |
||||
ASSERT_EQ(it->first, cf->GetID()); |
||||
auto val = it->second; |
||||
ASSERT_FALSE(val.start.inf_suffix); |
||||
ASSERT_FALSE(val.end.inf_suffix); |
||||
ASSERT_TRUE(val.exclusive); |
||||
ASSERT_EQ(val.ids.size(), 1); |
||||
if (val.ids[0] == txn0->GetID()) { |
||||
ASSERT_EQ(val.start.slice, "z"); |
||||
ASSERT_EQ(val.end.slice, "z"); |
||||
} else if (val.ids[0] == txn1->GetID()) { |
||||
ASSERT_EQ(val.start.slice, "b"); |
||||
ASSERT_EQ(val.end.slice, "e"); |
||||
} else { |
||||
FAIL(); // Unknown transaction ID.
|
||||
} |
||||
} |
||||
|
||||
delete txn0; |
||||
delete txn1; |
||||
} |
||||
|
||||
#if defined(__has_feature) |
||||
#if __has_feature(thread_sanitizer) |
||||
#define SKIP_LOCK_ESCALATION_TEST 1 |
||||
#endif |
||||
#endif |
||||
|
||||
#ifndef SKIP_LOCK_ESCALATION_TEST |
||||
TEST_F(RangeLockingTest, BasicLockEscalation) { |
||||
auto cf = db->DefaultColumnFamily(); |
||||
|
||||
auto counters = range_lock_mgr->GetStatus(); |
||||
|
||||
// Initially not using any lock memory
|
||||
ASSERT_EQ(counters.current_lock_memory, 0); |
||||
ASSERT_EQ(counters.escalation_count, 0); |
||||
|
||||
ASSERT_EQ(0, range_lock_mgr->SetMaxLockMemory(2000)); |
||||
|
||||
// Insert until we see lock escalations
|
||||
auto txn = NewTxn(); |
||||
|
||||
// Get the locks until we hit an escalation
|
||||
for (int i = 0; i < 2020; i++) { |
||||
char buf[32]; |
||||
snprintf(buf, sizeof(buf) - 1, "%08d", i); |
||||
ASSERT_OK(txn->GetRangeLock(cf, Endpoint(buf), Endpoint(buf))); |
||||
} |
||||
counters = range_lock_mgr->GetStatus(); |
||||
ASSERT_GT(counters.escalation_count, 0); |
||||
ASSERT_LE(counters.current_lock_memory, 2000); |
||||
|
||||
delete txn; |
||||
} |
||||
#endif |
||||
|
||||
void PointLockManagerTestExternalSetup(PointLockManagerTest* self) { |
||||
self->env_ = Env::Default(); |
||||
self->db_dir_ = test::PerThreadDBPath("point_lock_manager_test"); |
||||
ASSERT_OK(self->env_->CreateDir(self->db_dir_)); |
||||
|
||||
Options opt; |
||||
opt.create_if_missing = true; |
||||
TransactionDBOptions txn_opt; |
||||
txn_opt.transaction_lock_timeout = 0; |
||||
|
||||
auto mutex_factory = std::make_shared<TransactionDBMutexFactoryImpl>(); |
||||
self->locker_.reset(NewRangeLockManager(mutex_factory)->getLockManager()); |
||||
std::shared_ptr<RangeLockManagerHandle> range_lock_mgr = |
||||
std::dynamic_pointer_cast<RangeLockManagerHandle>(self->locker_); |
||||
txn_opt.lock_mgr_handle = range_lock_mgr; |
||||
|
||||
ASSERT_OK(TransactionDB::Open(opt, txn_opt, self->db_dir_, &self->db_)); |
||||
self->wait_sync_point_name_ = "RangeTreeLockManager::TryRangeLock:WaitingTxn"; |
||||
} |
||||
|
||||
INSTANTIATE_TEST_CASE_P(RangeLockManager, AnyLockManagerTest, |
||||
::testing::Values(PointLockManagerTestExternalSetup)); |
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
|
||||
int main(int argc, char** argv) { |
||||
ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); |
||||
::testing::InitGoogleTest(&argc, argv); |
||||
return RUN_ALL_TESTS(); |
||||
} |
||||
|
||||
#else // OS_WIN
|
||||
|
||||
#include <stdio.h> |
||||
int main(int /*argc*/, char** /*argv*/) { |
||||
fprintf(stderr, "skipped as Range Locking is not supported on Windows\n"); |
||||
return 0; |
||||
} |
||||
|
||||
#endif // OS_WIN
|
||||
|
||||
#else |
||||
#include <stdio.h> |
||||
|
||||
int main(int /*argc*/, char** /*argv*/) { |
||||
fprintf(stderr, |
||||
"skipped as transactions are not supported in rocksdb_lite\n"); |
||||
return 0; |
||||
} |
||||
|
||||
#endif // ROCKSDB_LITE
|
@ -1 +0,0 @@ |
||||
Implement LockTracker and LockManager for range lock with range tree. |
@ -0,0 +1,479 @@ |
||||
// 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 |
||||
#ifndef OS_WIN |
||||
|
||||
#include "utilities/transactions/lock/range/range_tree/range_tree_lock_manager.h" |
||||
|
||||
#include <algorithm> |
||||
#include <cinttypes> |
||||
#include <mutex> |
||||
|
||||
#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/lock/range/range_tree/range_tree_lock_tracker.h" |
||||
#include "utilities/transactions/pessimistic_transaction_db.h" |
||||
#include "utilities/transactions/transaction_db_mutex_impl.h" |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
RangeLockManagerHandle* NewRangeLockManager( |
||||
std::shared_ptr<TransactionDBMutexFactory> mutex_factory) { |
||||
std::shared_ptr<TransactionDBMutexFactory> use_factory; |
||||
|
||||
if (mutex_factory) { |
||||
use_factory = mutex_factory; |
||||
} else { |
||||
use_factory.reset(new TransactionDBMutexFactoryImpl()); |
||||
} |
||||
return new RangeTreeLockManager(use_factory); |
||||
} |
||||
|
||||
static const char SUFFIX_INFIMUM = 0x0; |
||||
static const char SUFFIX_SUPREMUM = 0x1; |
||||
|
||||
// Convert Endpoint into an internal format used for storing it in locktree
|
||||
// (DBT structure is used for passing endpoints to locktree and getting back)
|
||||
void serialize_endpoint(const Endpoint& endp, std::string* buf) { |
||||
buf->push_back(endp.inf_suffix ? SUFFIX_SUPREMUM : SUFFIX_INFIMUM); |
||||
buf->append(endp.slice.data(), endp.slice.size()); |
||||
} |
||||
|
||||
// Decode the endpoint from the format it is stored in the locktree (DBT) to
|
||||
// one used outside (EndpointWithString)
|
||||
void deserialize_endpoint(const DBT* dbt, EndpointWithString* endp) { |
||||
assert(dbt->size >= 1); |
||||
const char* dbt_data = (const char*)dbt->data; |
||||
char suffix = dbt_data[0]; |
||||
assert(suffix == SUFFIX_INFIMUM || suffix == SUFFIX_SUPREMUM); |
||||
endp->inf_suffix = (suffix == SUFFIX_SUPREMUM); |
||||
endp->slice.assign(dbt_data + 1, dbt->size - 1); |
||||
} |
||||
|
||||
// Get a range lock on [start_key; end_key] range
|
||||
Status RangeTreeLockManager::TryLock(PessimisticTransaction* txn, |
||||
uint32_t column_family_id, |
||||
const Endpoint& start_endp, |
||||
const Endpoint& end_endp, Env*, |
||||
bool exclusive) { |
||||
toku::lock_request request; |
||||
request.create(mutex_factory_); |
||||
DBT start_key_dbt, end_key_dbt; |
||||
|
||||
TEST_SYNC_POINT("RangeTreeLockManager::TryRangeLock:enter"); |
||||
std::string start_key; |
||||
std::string end_key; |
||||
serialize_endpoint(start_endp, &start_key); |
||||
serialize_endpoint(end_endp, &end_key); |
||||
|
||||
toku_fill_dbt(&start_key_dbt, start_key.data(), start_key.size()); |
||||
toku_fill_dbt(&end_key_dbt, end_key.data(), end_key.size()); |
||||
|
||||
auto lt = GetLockTreeForCF(column_family_id); |
||||
|
||||
// Put the key waited on into request's m_extra. See
|
||||
// wait_callback_for_locktree for details.
|
||||
std::string wait_key(start_endp.slice.data(), start_endp.slice.size()); |
||||
|
||||
request.set(lt.get(), (TXNID)txn, &start_key_dbt, &end_key_dbt, |
||||
exclusive ? toku::lock_request::WRITE : toku::lock_request::READ, |
||||
false /* not a big txn */, &wait_key); |
||||
|
||||
// This is for "periodically wake up and check if the wait is killed" feature
|
||||
// which we are not using.
|
||||
uint64_t killed_time_msec = 0; |
||||
uint64_t wait_time_msec = txn->GetLockTimeout(); |
||||
|
||||
if (wait_time_msec == static_cast<uint64_t>(-1)) { |
||||
// The transaction has no wait timeout. lock_request::wait doesn't support
|
||||
// this, it needs a number of milliseconds to wait. Pass it one year to
|
||||
// be safe.
|
||||
wait_time_msec = uint64_t(1000) * 60 * 60 * 24 * 365; |
||||
} else { |
||||
// convert microseconds to milliseconds
|
||||
wait_time_msec = (wait_time_msec + 500) / 1000; |
||||
} |
||||
|
||||
std::vector<RangeDeadlockInfo> di_path; |
||||
request.m_deadlock_cb = [&](TXNID txnid, bool is_exclusive, |
||||
const DBT* start_dbt, const DBT* end_dbt) { |
||||
EndpointWithString start; |
||||
EndpointWithString end; |
||||
deserialize_endpoint(start_dbt, &start); |
||||
deserialize_endpoint(end_dbt, &end); |
||||
|
||||
di_path.push_back({((PessimisticTransaction*)txnid)->GetID(), |
||||
column_family_id, is_exclusive, std::move(start), |
||||
std::move(end)}); |
||||
}; |
||||
|
||||
request.start(); |
||||
|
||||
const int r = request.wait(wait_time_msec, killed_time_msec, |
||||
nullptr, // killed_callback
|
||||
wait_callback_for_locktree, nullptr); |
||||
|
||||
// Inform the txn that we are no longer waiting:
|
||||
txn->ClearWaitingTxn(); |
||||
|
||||
request.destroy(); |
||||
switch (r) { |
||||
case 0: |
||||
break; // fall through
|
||||
case DB_LOCK_NOTGRANTED: |
||||
return Status::TimedOut(Status::SubCode::kLockTimeout); |
||||
case TOKUDB_OUT_OF_LOCKS: |
||||
return Status::Busy(Status::SubCode::kLockLimit); |
||||
case DB_LOCK_DEADLOCK: { |
||||
std::reverse(di_path.begin(), di_path.end()); |
||||
dlock_buffer_.AddNewPath( |
||||
RangeDeadlockPath(di_path, request.get_start_time())); |
||||
return Status::Busy(Status::SubCode::kDeadlock); |
||||
} |
||||
default: |
||||
assert(0); |
||||
return Status::Busy(Status::SubCode::kLockLimit); |
||||
} |
||||
|
||||
return Status::OK(); |
||||
} |
||||
|
||||
// Wait callback that locktree library will call to inform us about
|
||||
// the lock waits that are in progress.
|
||||
void wait_callback_for_locktree(void*, lock_wait_infos* infos) { |
||||
for (auto wait_info : *infos) { |
||||
auto txn = (PessimisticTransaction*)wait_info.waiter; |
||||
auto cf_id = (ColumnFamilyId)wait_info.ltree->get_dict_id().dictid; |
||||
|
||||
autovector<TransactionID> waitee_ids; |
||||
for (auto waitee : wait_info.waitees) { |
||||
waitee_ids.push_back(((PessimisticTransaction*)waitee)->GetID()); |
||||
} |
||||
txn->SetWaitingTxn(waitee_ids, cf_id, (std::string*)wait_info.m_extra); |
||||
} |
||||
|
||||
// Here we can assume that the locktree code will now wait for some lock
|
||||
TEST_SYNC_POINT("RangeTreeLockManager::TryRangeLock:WaitingTxn"); |
||||
} |
||||
|
||||
void RangeTreeLockManager::UnLock(PessimisticTransaction* txn, |
||||
ColumnFamilyId column_family_id, |
||||
const std::string& key, Env*) { |
||||
auto locktree = GetLockTreeForCF(column_family_id); |
||||
std::string endp_image; |
||||
serialize_endpoint({key.data(), key.size(), false}, &endp_image); |
||||
|
||||
DBT key_dbt; |
||||
toku_fill_dbt(&key_dbt, endp_image.data(), endp_image.size()); |
||||
|
||||
toku::range_buffer range_buf; |
||||
range_buf.create(); |
||||
range_buf.append(&key_dbt, &key_dbt); |
||||
|
||||
locktree->release_locks((TXNID)txn, &range_buf); |
||||
range_buf.destroy(); |
||||
|
||||
toku::lock_request::retry_all_lock_requests( |
||||
locktree.get(), wait_callback_for_locktree, nullptr); |
||||
} |
||||
|
||||
void RangeTreeLockManager::UnLock(PessimisticTransaction* txn, |
||||
const LockTracker& tracker, Env*) { |
||||
const RangeTreeLockTracker* range_tracker = |
||||
static_cast<const RangeTreeLockTracker*>(&tracker); |
||||
|
||||
RangeTreeLockTracker* range_trx_tracker = |
||||
static_cast<RangeTreeLockTracker*>(&txn->GetTrackedLocks()); |
||||
bool all_keys = (range_trx_tracker == range_tracker); |
||||
|
||||
// tracked_locks_->range_list may hold nullptr if the transaction has never
|
||||
// acquired any locks.
|
||||
((RangeTreeLockTracker*)range_tracker)->ReleaseLocks(this, txn, all_keys); |
||||
} |
||||
|
||||
int RangeTreeLockManager::CompareDbtEndpoints(void* arg, const DBT* a_key, |
||||
const DBT* b_key) { |
||||
const char* a = (const char*)a_key->data; |
||||
const char* b = (const char*)b_key->data; |
||||
|
||||
size_t a_len = a_key->size; |
||||
size_t b_len = b_key->size; |
||||
|
||||
size_t min_len = std::min(a_len, b_len); |
||||
|
||||
// Compare the values. The first byte encodes the endpoint type, its value
|
||||
// is either SUFFIX_INFIMUM or SUFFIX_SUPREMUM.
|
||||
Comparator* cmp = (Comparator*)arg; |
||||
int res = cmp->Compare(Slice(a + 1, min_len - 1), Slice(b + 1, min_len - 1)); |
||||
if (!res) { |
||||
if (b_len > min_len) { |
||||
// a is shorter;
|
||||
if (a[0] == SUFFIX_INFIMUM) { |
||||
return -1; //"a is smaller"
|
||||
} else { |
||||
// a is considered padded with 0xFF:FF:FF:FF...
|
||||
return 1; // "a" is bigger
|
||||
} |
||||
} else if (a_len > min_len) { |
||||
// the opposite of the above: b is shorter.
|
||||
if (b[0] == SUFFIX_INFIMUM) { |
||||
return 1; //"b is smaller"
|
||||
} else { |
||||
// b is considered padded with 0xFF:FF:FF:FF...
|
||||
return -1; // "b" is bigger
|
||||
} |
||||
} else { |
||||
// the lengths are equal (and the key values, too)
|
||||
if (a[0] < b[0]) { |
||||
return -1; |
||||
} else if (a[0] > b[0]) { |
||||
return 1; |
||||
} else { |
||||
return 0; |
||||
} |
||||
} |
||||
} else { |
||||
return res; |
||||
} |
||||
} |
||||
|
||||
namespace { |
||||
void UnrefLockTreeMapsCache(void* ptr) { |
||||
// Called when a thread exits or a ThreadLocalPtr gets destroyed.
|
||||
auto lock_tree_map_cache = static_cast< |
||||
std::unordered_map<ColumnFamilyId, std::shared_ptr<locktree>>*>(ptr); |
||||
delete lock_tree_map_cache; |
||||
} |
||||
} // anonymous namespace
|
||||
|
||||
RangeTreeLockManager::RangeTreeLockManager( |
||||
std::shared_ptr<TransactionDBMutexFactory> mutex_factory) |
||||
: mutex_factory_(mutex_factory), |
||||
ltree_lookup_cache_(new ThreadLocalPtr(&UnrefLockTreeMapsCache)), |
||||
dlock_buffer_(10) { |
||||
ltm_.create(on_create, on_destroy, on_escalate, nullptr, mutex_factory_); |
||||
} |
||||
|
||||
void RangeTreeLockManager::SetRangeDeadlockInfoBufferSize( |
||||
uint32_t target_size) { |
||||
dlock_buffer_.Resize(target_size); |
||||
} |
||||
|
||||
void RangeTreeLockManager::Resize(uint32_t target_size) { |
||||
SetRangeDeadlockInfoBufferSize(target_size); |
||||
} |
||||
|
||||
std::vector<RangeDeadlockPath> |
||||
RangeTreeLockManager::GetRangeDeadlockInfoBuffer() { |
||||
return dlock_buffer_.PrepareBuffer(); |
||||
} |
||||
|
||||
std::vector<DeadlockPath> RangeTreeLockManager::GetDeadlockInfoBuffer() { |
||||
std::vector<DeadlockPath> res; |
||||
std::vector<RangeDeadlockPath> data = GetRangeDeadlockInfoBuffer(); |
||||
// report left endpoints
|
||||
for (auto it = data.begin(); it != data.end(); ++it) { |
||||
std::vector<DeadlockInfo> path; |
||||
|
||||
for (auto it2 = it->path.begin(); it2 != it->path.end(); ++it2) { |
||||
path.push_back( |
||||
{it2->m_txn_id, it2->m_cf_id, it2->m_exclusive, it2->m_start.slice}); |
||||
} |
||||
res.push_back(DeadlockPath(path, it->deadlock_time)); |
||||
} |
||||
return res; |
||||
} |
||||
|
||||
// @brief Lock Escalation Callback function
|
||||
//
|
||||
// @param txnid Transaction whose locks got escalated
|
||||
// @param lt Lock Tree where escalation is happening
|
||||
// @param buffer Escalation result: list of locks that this transaction now
|
||||
// owns in this lock tree.
|
||||
// @param void* Callback context
|
||||
void RangeTreeLockManager::on_escalate(TXNID txnid, const locktree* lt, |
||||
const range_buffer& buffer, void*) { |
||||
auto txn = (PessimisticTransaction*)txnid; |
||||
((RangeTreeLockTracker*)&txn->GetTrackedLocks())->ReplaceLocks(lt, buffer); |
||||
} |
||||
|
||||
RangeTreeLockManager::~RangeTreeLockManager() { |
||||
autovector<void*> local_caches; |
||||
ltree_lookup_cache_->Scrape(&local_caches, nullptr); |
||||
for (auto cache : local_caches) { |
||||
delete static_cast<LockTreeMap*>(cache); |
||||
} |
||||
ltree_map_.clear(); // this will call release_lt() for all locktrees
|
||||
ltm_.destroy(); |
||||
} |
||||
|
||||
RangeLockManagerHandle::Counters RangeTreeLockManager::GetStatus() { |
||||
LTM_STATUS_S ltm_status_test; |
||||
ltm_.get_status(<m_status_test); |
||||
Counters res; |
||||
|
||||
// Searching status variable by its string name is how Toku's unit tests
|
||||
// do it (why didn't they make LTM_ESCALATION_COUNT constant visible?)
|
||||
// lookup keyname in status
|
||||
for (int i = 0; i < LTM_STATUS_S::LTM_STATUS_NUM_ROWS; i++) { |
||||
TOKU_ENGINE_STATUS_ROW status = <m_status_test.status[i]; |
||||
if (strcmp(status->keyname, "LTM_ESCALATION_COUNT") == 0) { |
||||
res.escalation_count = status->value.num; |
||||
continue; |
||||
} |
||||
if (strcmp(status->keyname, "LTM_SIZE_CURRENT") == 0) { |
||||
res.current_lock_memory = status->value.num; |
||||
} |
||||
} |
||||
return res; |
||||
} |
||||
|
||||
std::shared_ptr<locktree> RangeTreeLockManager::MakeLockTreePtr(locktree* lt) { |
||||
locktree_manager* ltm = <m_; |
||||
return std::shared_ptr<locktree>(lt, |
||||
[ltm](locktree* p) { ltm->release_lt(p); }); |
||||
} |
||||
|
||||
void RangeTreeLockManager::AddColumnFamily(const ColumnFamilyHandle* cfh) { |
||||
uint32_t column_family_id = cfh->GetID(); |
||||
|
||||
InstrumentedMutexLock l(<ree_map_mutex_); |
||||
if (ltree_map_.find(column_family_id) == ltree_map_.end()) { |
||||
DICTIONARY_ID dict_id = {.dictid = column_family_id}; |
||||
toku::comparator cmp; |
||||
cmp.create(CompareDbtEndpoints, (void*)cfh->GetComparator()); |
||||
toku::locktree* ltree = ltm_.get_lt(dict_id, cmp, |
||||
/* on_create_extra*/ nullptr); |
||||
// This is ok to because get_lt has copied the comparator:
|
||||
cmp.destroy(); |
||||
|
||||
ltree_map_.insert({column_family_id, MakeLockTreePtr(ltree)}); |
||||
} |
||||
} |
||||
|
||||
void RangeTreeLockManager::RemoveColumnFamily(const ColumnFamilyHandle* cfh) { |
||||
uint32_t column_family_id = cfh->GetID(); |
||||
// 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.
|
||||
|
||||
// TODO what if one drops a column family while transaction(s) still have
|
||||
// locks in it?
|
||||
// locktree uses column family'c Comparator* as the criteria to do tree
|
||||
// ordering. If the comparator is gone, we won't even be able to remove the
|
||||
// elements from the locktree.
|
||||
// A possible solution might be to remove everything right now:
|
||||
// - wait until everyone traversing the locktree are gone
|
||||
// - remove everything from the locktree.
|
||||
// - some transactions may have acquired locks in their LockTracker objects.
|
||||
// Arrange something so we don't blow up when they try to release them.
|
||||
// - ...
|
||||
// This use case (drop column family while somebody is using it) doesn't seem
|
||||
// the priority, though.
|
||||
|
||||
{ |
||||
InstrumentedMutexLock l(<ree_map_mutex_); |
||||
|
||||
auto lock_maps_iter = ltree_map_.find(column_family_id); |
||||
assert(lock_maps_iter != ltree_map_.end()); |
||||
ltree_map_.erase(lock_maps_iter); |
||||
} // lock_map_mutex_
|
||||
|
||||
autovector<void*> local_caches; |
||||
ltree_lookup_cache_->Scrape(&local_caches, nullptr); |
||||
for (auto cache : local_caches) { |
||||
delete static_cast<LockTreeMap*>(cache); |
||||
} |
||||
} |
||||
|
||||
std::shared_ptr<locktree> RangeTreeLockManager::GetLockTreeForCF( |
||||
ColumnFamilyId column_family_id) { |
||||
// First check thread-local cache
|
||||
if (ltree_lookup_cache_->Get() == nullptr) { |
||||
ltree_lookup_cache_->Reset(new LockTreeMap()); |
||||
} |
||||
|
||||
auto ltree_map_cache = static_cast<LockTreeMap*>(ltree_lookup_cache_->Get()); |
||||
|
||||
auto it = ltree_map_cache->find(column_family_id); |
||||
if (it != ltree_map_cache->end()) { |
||||
// Found lock map for this column family.
|
||||
return it->second; |
||||
} |
||||
|
||||
// Not found in local cache, grab mutex and check shared LockMaps
|
||||
InstrumentedMutexLock l(<ree_map_mutex_); |
||||
|
||||
it = ltree_map_.find(column_family_id); |
||||
if (it == ltree_map_.end()) { |
||||
return nullptr; |
||||
} else { |
||||
// Found lock map. Store in thread-local cache and return.
|
||||
ltree_map_cache->insert({column_family_id, it->second}); |
||||
return it->second; |
||||
} |
||||
} |
||||
|
||||
struct LOCK_PRINT_CONTEXT { |
||||
RangeLockManagerHandle::RangeLockStatus* data; // Save locks here
|
||||
uint32_t cfh_id; // Column Family whose tree we are traversing
|
||||
}; |
||||
|
||||
// Report left endpoints of the acquired locks
|
||||
LockManager::PointLockStatus RangeTreeLockManager::GetPointLockStatus() { |
||||
PointLockStatus res; |
||||
LockManager::RangeLockStatus data = GetRangeLockStatus(); |
||||
// report left endpoints
|
||||
for (auto it = data.begin(); it != data.end(); ++it) { |
||||
auto& val = it->second; |
||||
res.insert({it->first, {val.start.slice, val.ids, val.exclusive}}); |
||||
} |
||||
return res; |
||||
} |
||||
|
||||
static void push_into_lock_status_data(void* param, const DBT* left, |
||||
const DBT* right, TXNID txnid_arg, |
||||
bool is_shared, TxnidVector* owners) { |
||||
struct LOCK_PRINT_CONTEXT* ctx = (LOCK_PRINT_CONTEXT*)param; |
||||
struct RangeLockInfo info; |
||||
|
||||
info.exclusive = !is_shared; |
||||
|
||||
deserialize_endpoint(left, &info.start); |
||||
deserialize_endpoint(right, &info.end); |
||||
|
||||
if (txnid_arg != TXNID_SHARED) { |
||||
TXNID txnid = ((PessimisticTransaction*)txnid_arg)->GetID(); |
||||
info.ids.push_back(txnid); |
||||
} else { |
||||
for (auto it : *owners) { |
||||
TXNID real_id = ((PessimisticTransaction*)it)->GetID(); |
||||
info.ids.push_back(real_id); |
||||
} |
||||
} |
||||
ctx->data->insert({ctx->cfh_id, info}); |
||||
} |
||||
|
||||
LockManager::RangeLockStatus RangeTreeLockManager::GetRangeLockStatus() { |
||||
LockManager::RangeLockStatus data; |
||||
{ |
||||
InstrumentedMutexLock l(<ree_map_mutex_); |
||||
for (auto it : ltree_map_) { |
||||
LOCK_PRINT_CONTEXT ctx = {&data, it.first}; |
||||
it.second->dump_locks((void*)&ctx, push_into_lock_status_data); |
||||
} |
||||
} |
||||
return data; |
||||
} |
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif // OS_WIN
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,130 @@ |
||||
// 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 |
||||
#ifndef OS_WIN |
||||
|
||||
// For DeadlockInfoBuffer:
|
||||
#include "util/thread_local.h" |
||||
#include "utilities/transactions/lock/point/point_lock_manager.h" |
||||
#include "utilities/transactions/lock/range/range_lock_manager.h" |
||||
|
||||
// Lock Tree library:
|
||||
#include "utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.h" |
||||
#include "utilities/transactions/lock/range/range_tree/lib/locktree/locktree.h" |
||||
#include "utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.h" |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
using namespace toku; |
||||
|
||||
typedef DeadlockInfoBufferTempl<RangeDeadlockPath> RangeDeadlockInfoBuffer; |
||||
|
||||
// A Range Lock Manager that uses PerconaFT's locktree library
|
||||
class RangeTreeLockManager : public RangeLockManagerBase, |
||||
public RangeLockManagerHandle { |
||||
public: |
||||
LockManager* getLockManager() override { return this; } |
||||
|
||||
void AddColumnFamily(const ColumnFamilyHandle* cfh) override; |
||||
void RemoveColumnFamily(const ColumnFamilyHandle* cfh) override; |
||||
|
||||
void Resize(uint32_t) override; |
||||
std::vector<DeadlockPath> GetDeadlockInfoBuffer() override; |
||||
|
||||
std::vector<RangeDeadlockPath> GetRangeDeadlockInfoBuffer() override; |
||||
void SetRangeDeadlockInfoBufferSize(uint32_t target_size) override; |
||||
|
||||
// Get a lock on a range
|
||||
// @note only exclusive locks are currently supported (requesting a
|
||||
// non-exclusive lock will get an exclusive one)
|
||||
using LockManager::TryLock; |
||||
Status TryLock(PessimisticTransaction* txn, ColumnFamilyId column_family_id, |
||||
const Endpoint& start_endp, const Endpoint& end_endp, Env* env, |
||||
bool exclusive) override; |
||||
|
||||
void UnLock(PessimisticTransaction* txn, const LockTracker& tracker, |
||||
Env* env) override; |
||||
void UnLock(PessimisticTransaction* txn, ColumnFamilyId column_family_id, |
||||
const std::string& key, Env* env) override; |
||||
void UnLock(PessimisticTransaction*, ColumnFamilyId, const Endpoint&, |
||||
const Endpoint&, Env*) override { |
||||
// TODO: range unlock does nothing...
|
||||
} |
||||
|
||||
explicit RangeTreeLockManager( |
||||
std::shared_ptr<TransactionDBMutexFactory> mutex_factory); |
||||
|
||||
~RangeTreeLockManager() override; |
||||
|
||||
int SetMaxLockMemory(size_t max_lock_memory) override { |
||||
return ltm_.set_max_lock_memory(max_lock_memory); |
||||
} |
||||
|
||||
size_t GetMaxLockMemory() override { return ltm_.get_max_lock_memory(); } |
||||
|
||||
Counters GetStatus() override; |
||||
|
||||
bool IsPointLockSupported() const override { |
||||
// One could have acquired a point lock (it is reduced to range lock)
|
||||
return true; |
||||
} |
||||
|
||||
PointLockStatus GetPointLockStatus() override; |
||||
|
||||
// This is from LockManager
|
||||
LockManager::RangeLockStatus GetRangeLockStatus() override; |
||||
|
||||
// This has the same meaning as GetRangeLockStatus but is from
|
||||
// RangeLockManagerHandle
|
||||
RangeLockManagerHandle::RangeLockStatus GetRangeLockStatusData() override { |
||||
return GetRangeLockStatus(); |
||||
} |
||||
|
||||
bool IsRangeLockSupported() const override { return true; } |
||||
|
||||
const LockTrackerFactory& GetLockTrackerFactory() const override { |
||||
return RangeTreeLockTrackerFactory::Get(); |
||||
} |
||||
|
||||
// Get the locktree which stores locks for the Column Family with given cf_id
|
||||
std::shared_ptr<locktree> GetLockTreeForCF(ColumnFamilyId cf_id); |
||||
|
||||
private: |
||||
toku::locktree_manager ltm_; |
||||
|
||||
std::shared_ptr<TransactionDBMutexFactory> mutex_factory_; |
||||
|
||||
// Map from cf_id to locktree*. Can only be accessed while holding the
|
||||
// ltree_map_mutex_. Must use a custom deleter that calls ltm_.release_lt
|
||||
using LockTreeMap = |
||||
std::unordered_map<ColumnFamilyId, std::shared_ptr<locktree>>; |
||||
LockTreeMap ltree_map_; |
||||
|
||||
InstrumentedMutex ltree_map_mutex_; |
||||
|
||||
// Per-thread cache of ltree_map_.
|
||||
// (uses the same approach as TransactionLockMgr::lock_maps_cache_)
|
||||
std::unique_ptr<ThreadLocalPtr> ltree_lookup_cache_; |
||||
|
||||
RangeDeadlockInfoBuffer dlock_buffer_; |
||||
|
||||
std::shared_ptr<locktree> MakeLockTreePtr(locktree* lt); |
||||
static int CompareDbtEndpoints(void* arg, const DBT* a_key, const DBT* b_key); |
||||
|
||||
// Callbacks
|
||||
static int on_create(locktree*, void*) { return 0; /* no error */ } |
||||
static void on_destroy(locktree*) {} |
||||
static void on_escalate(TXNID txnid, const locktree* lt, |
||||
const range_buffer& buffer, void* extra); |
||||
}; |
||||
|
||||
void serialize_endpoint(const Endpoint& endp, std::string* buf); |
||||
void wait_callback_for_locktree(void* cdata, lock_wait_infos* infos); |
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif // OS_WIN
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,156 @@ |
||||
// 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 |
||||
#ifndef OS_WIN |
||||
|
||||
#include "utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.h" |
||||
|
||||
#include "utilities/transactions/lock/range/range_tree/range_tree_lock_manager.h" |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
RangeLockList *RangeTreeLockTracker::getOrCreateList() { |
||||
if (range_list_) return range_list_.get(); |
||||
|
||||
// Doesn't exist, create
|
||||
range_list_.reset(new RangeLockList()); |
||||
return range_list_.get(); |
||||
} |
||||
|
||||
void RangeTreeLockTracker::Track(const PointLockRequest &lock_req) { |
||||
DBT key_dbt; |
||||
std::string key; |
||||
serialize_endpoint(Endpoint(lock_req.key, false), &key); |
||||
toku_fill_dbt(&key_dbt, key.data(), key.size()); |
||||
RangeLockList *rl = getOrCreateList(); |
||||
rl->Append(lock_req.column_family_id, &key_dbt, &key_dbt); |
||||
} |
||||
|
||||
void RangeTreeLockTracker::Track(const RangeLockRequest &lock_req) { |
||||
DBT start_dbt, end_dbt; |
||||
std::string start_key, end_key; |
||||
|
||||
serialize_endpoint(lock_req.start_endp, &start_key); |
||||
serialize_endpoint(lock_req.end_endp, &end_key); |
||||
|
||||
toku_fill_dbt(&start_dbt, start_key.data(), start_key.size()); |
||||
toku_fill_dbt(&end_dbt, end_key.data(), end_key.size()); |
||||
|
||||
RangeLockList *rl = getOrCreateList(); |
||||
rl->Append(lock_req.column_family_id, &start_dbt, &end_dbt); |
||||
} |
||||
|
||||
PointLockStatus RangeTreeLockTracker::GetPointLockStatus( |
||||
ColumnFamilyId /*cf_id*/, const std::string & /*key*/) const { |
||||
// This function is not expected to be called as RangeTreeLockTracker::
|
||||
// IsPointLockSupported() returns false. Return the status which indicates
|
||||
// the point is not locked.
|
||||
PointLockStatus p; |
||||
p.locked = false; |
||||
p.exclusive = true; |
||||
p.seq = 0; |
||||
return p; |
||||
} |
||||
|
||||
void RangeTreeLockTracker::Clear() { range_list_.reset(); } |
||||
|
||||
void RangeLockList::Append(ColumnFamilyId cf_id, const DBT *left_key, |
||||
const DBT *right_key) { |
||||
MutexLock l(&mutex_); |
||||
// Only the transaction owner thread calls this function.
|
||||
// The same thread does the lock release, so we can be certain nobody is
|
||||
// releasing the locks concurrently.
|
||||
assert(!releasing_locks_.load()); |
||||
auto it = buffers_.find(cf_id); |
||||
if (it == buffers_.end()) { |
||||
// create a new one
|
||||
it = buffers_.emplace(cf_id, std::make_shared<toku::range_buffer>()).first; |
||||
it->second->create(); |
||||
} |
||||
it->second->append(left_key, right_key); |
||||
} |
||||
|
||||
void RangeLockList::ReleaseLocks(RangeTreeLockManager *mgr, |
||||
PessimisticTransaction *txn, |
||||
bool all_trx_locks) { |
||||
{ |
||||
MutexLock l(&mutex_); |
||||
// The lt->release_locks() call below will walk range_list->buffer_. We
|
||||
// need to prevent lock escalation callback from replacing
|
||||
// range_list->buffer_ while we are doing that.
|
||||
//
|
||||
// Additional complication here is internal mutex(es) in the locktree
|
||||
// (let's call them latches):
|
||||
// - Lock escalation first obtains latches on the lock tree
|
||||
// - Then, it calls RangeTreeLockManager::on_escalate to replace
|
||||
// transaction's range_list->buffer_. = Access to that buffer must be
|
||||
// synchronized, so it will want to acquire the range_list->mutex_.
|
||||
//
|
||||
// While in this function we would want to do the reverse:
|
||||
// - Acquire range_list->mutex_ to prevent access to the range_list.
|
||||
// - Then, lt->release_locks() call will walk through the range_list
|
||||
// - and acquire latches on parts of the lock tree to remove locks from
|
||||
// it.
|
||||
//
|
||||
// How do we avoid the deadlock? The idea is that here we set
|
||||
// releasing_locks_=true, and release the mutex.
|
||||
// All other users of the range_list must:
|
||||
// - Acquire the mutex, then check that releasing_locks_=false.
|
||||
// (the code in this function doesnt do that as there's only one thread
|
||||
// that releases transaction's locks)
|
||||
releasing_locks_.store(true); |
||||
} |
||||
|
||||
for (auto it : buffers_) { |
||||
// Don't try to call release_locks() if the buffer is empty! if we are
|
||||
// not holding any locks, the lock tree might be in the STO-mode with
|
||||
// another transaction, and our attempt to release an empty set of locks
|
||||
// will cause an assertion failure.
|
||||
if (it.second->get_num_ranges()) { |
||||
auto lt_ptr = mgr->GetLockTreeForCF(it.first); |
||||
toku::locktree *lt = lt_ptr.get(); |
||||
|
||||
lt->release_locks((TXNID)txn, it.second.get(), all_trx_locks); |
||||
|
||||
it.second->destroy(); |
||||
it.second->create(); |
||||
|
||||
toku::lock_request::retry_all_lock_requests(lt, |
||||
wait_callback_for_locktree); |
||||
} |
||||
} |
||||
|
||||
Clear(); |
||||
releasing_locks_.store(false); |
||||
} |
||||
|
||||
void RangeLockList::ReplaceLocks(const toku::locktree *lt, |
||||
const toku::range_buffer &buffer) { |
||||
MutexLock l(&mutex_); |
||||
if (releasing_locks_.load()) { |
||||
// Do nothing. The transaction is releasing its locks, so it will not care
|
||||
// about having a correct list of ranges. (In TokuDB,
|
||||
// toku_db_txn_escalate_callback() makes use of this property, too)
|
||||
return; |
||||
} |
||||
|
||||
ColumnFamilyId cf_id = (ColumnFamilyId)lt->get_dict_id().dictid; |
||||
|
||||
auto it = buffers_.find(cf_id); |
||||
it->second->destroy(); |
||||
it->second->create(); |
||||
|
||||
toku::range_buffer::iterator iter(&buffer); |
||||
toku::range_buffer::iterator::record rec; |
||||
while (iter.current(&rec)) { |
||||
it->second->append(rec.get_left_key(), rec.get_right_key()); |
||||
iter.next(); |
||||
} |
||||
} |
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif // OS_WIN
|
||||
#endif // ROCKSDB_LITE
|
@ -0,0 +1,146 @@ |
||||
// 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 <memory> |
||||
#include <string> |
||||
#include <unordered_map> |
||||
|
||||
#include "util/mutexlock.h" |
||||
#include "utilities/transactions/lock/lock_tracker.h" |
||||
#include "utilities/transactions/pessimistic_transaction.h" |
||||
|
||||
// Range Locking:
|
||||
#include "lib/locktree/lock_request.h" |
||||
#include "lib/locktree/locktree.h" |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
class RangeTreeLockManager; |
||||
|
||||
// Storage for locks that are currently held by a transaction.
|
||||
//
|
||||
// Locks are kept in toku::range_buffer because toku::locktree::release_locks()
|
||||
// accepts that as an argument.
|
||||
//
|
||||
// Note: the list of locks may differ slighly from the contents of the lock
|
||||
// tree, due to concurrency between lock acquisition, lock release, and lock
|
||||
// escalation. See MDEV-18227 and RangeTreeLockManager::UnLock for details.
|
||||
// This property is currently harmless.
|
||||
//
|
||||
// Append() and ReleaseLocks() are not thread-safe, as they are expected to be
|
||||
// called only by the owner transaction. ReplaceLocks() is safe to call from
|
||||
// other threads.
|
||||
class RangeLockList { |
||||
public: |
||||
~RangeLockList() { Clear(); } |
||||
|
||||
RangeLockList() : releasing_locks_(false) {} |
||||
|
||||
void Append(ColumnFamilyId cf_id, const DBT* left_key, const DBT* right_key); |
||||
void ReleaseLocks(RangeTreeLockManager* mgr, PessimisticTransaction* txn, |
||||
bool all_trx_locks); |
||||
void ReplaceLocks(const toku::locktree* lt, const toku::range_buffer& buffer); |
||||
|
||||
private: |
||||
void Clear() { |
||||
for (auto it : buffers_) { |
||||
it.second->destroy(); |
||||
} |
||||
buffers_.clear(); |
||||
} |
||||
|
||||
std::unordered_map<ColumnFamilyId, std::shared_ptr<toku::range_buffer>> |
||||
buffers_; |
||||
port::Mutex mutex_; |
||||
std::atomic<bool> releasing_locks_; |
||||
}; |
||||
|
||||
// A LockTracker-based object that is used together with RangeTreeLockManager.
|
||||
class RangeTreeLockTracker : public LockTracker { |
||||
public: |
||||
RangeTreeLockTracker() : range_list_(nullptr) {} |
||||
|
||||
RangeTreeLockTracker(const RangeTreeLockTracker&) = delete; |
||||
RangeTreeLockTracker& operator=(const RangeTreeLockTracker&) = delete; |
||||
|
||||
void Track(const PointLockRequest&) override; |
||||
void Track(const RangeLockRequest&) override; |
||||
|
||||
bool IsPointLockSupported() const override { |
||||
// This indicates that we don't implement GetPointLockStatus()
|
||||
return false; |
||||
} |
||||
bool IsRangeLockSupported() const override { return true; } |
||||
|
||||
// a Not-supported dummy implementation.
|
||||
UntrackStatus Untrack(const RangeLockRequest& /*lock_request*/) override { |
||||
return UntrackStatus::NOT_TRACKED; |
||||
} |
||||
|
||||
UntrackStatus Untrack(const PointLockRequest& /*lock_request*/) override { |
||||
return UntrackStatus::NOT_TRACKED; |
||||
} |
||||
|
||||
// "If this method is not supported, leave it as a no-op."
|
||||
void Merge(const LockTracker&) override {} |
||||
|
||||
// "If this method is not supported, leave it as a no-op."
|
||||
void Subtract(const LockTracker&) override {} |
||||
|
||||
void Clear() override; |
||||
|
||||
// "If this method is not supported, returns nullptr."
|
||||
virtual LockTracker* GetTrackedLocksSinceSavePoint( |
||||
const LockTracker&) const override { |
||||
return nullptr; |
||||
} |
||||
|
||||
PointLockStatus GetPointLockStatus(ColumnFamilyId column_family_id, |
||||
const std::string& key) const override; |
||||
|
||||
// The return value is only used for tests
|
||||
uint64_t GetNumPointLocks() const override { return 0; } |
||||
|
||||
ColumnFamilyIterator* GetColumnFamilyIterator() const override { |
||||
return nullptr; |
||||
} |
||||
|
||||
KeyIterator* GetKeyIterator( |
||||
ColumnFamilyId /*column_family_id*/) const override { |
||||
return nullptr; |
||||
} |
||||
|
||||
void ReleaseLocks(RangeTreeLockManager* mgr, PessimisticTransaction* txn, |
||||
bool all_trx_locks) { |
||||
if (range_list_) range_list_->ReleaseLocks(mgr, txn, all_trx_locks); |
||||
} |
||||
|
||||
void ReplaceLocks(const toku::locktree* lt, |
||||
const toku::range_buffer& buffer) { |
||||
// range_list_ cannot be NULL here
|
||||
range_list_->ReplaceLocks(lt, buffer); |
||||
} |
||||
|
||||
private: |
||||
RangeLockList* getOrCreateList(); |
||||
std::unique_ptr<RangeLockList> range_list_; |
||||
}; |
||||
|
||||
class RangeTreeLockTrackerFactory : public LockTrackerFactory { |
||||
public: |
||||
static const RangeTreeLockTrackerFactory& Get() { |
||||
static const RangeTreeLockTrackerFactory instance; |
||||
return instance; |
||||
} |
||||
|
||||
LockTracker* Create() const override { return new RangeTreeLockTracker(); } |
||||
|
||||
private: |
||||
RangeTreeLockTrackerFactory() {} |
||||
}; |
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
Loading…
Reference in new issue