summaryrefslogtreecommitdiffstats
path: root/src/rocksdb/utilities/transactions/lock
diff options
context:
space:
mode:
authorDaniel Baumann <daniel.baumann@progress-linux.org>2024-04-21 11:54:28 +0000
committerDaniel Baumann <daniel.baumann@progress-linux.org>2024-04-21 11:54:28 +0000
commite6918187568dbd01842d8d1d2c808ce16a894239 (patch)
tree64f88b554b444a49f656b6c656111a145cbbaa28 /src/rocksdb/utilities/transactions/lock
parentInitial commit. (diff)
downloadceph-b26c4052f3542036551aa9dec9caa4226e456195.tar.xz
ceph-b26c4052f3542036551aa9dec9caa4226e456195.zip
Adding upstream version 18.2.2.upstream/18.2.2
Signed-off-by: Daniel Baumann <daniel.baumann@progress-linux.org>
Diffstat (limited to 'src/rocksdb/utilities/transactions/lock')
-rw-r--r--src/rocksdb/utilities/transactions/lock/lock_manager.cc29
-rw-r--r--src/rocksdb/utilities/transactions/lock/lock_manager.h82
-rw-r--r--src/rocksdb/utilities/transactions/lock/lock_tracker.h209
-rw-r--r--src/rocksdb/utilities/transactions/lock/point/point_lock_manager.cc721
-rw-r--r--src/rocksdb/utilities/transactions/lock/point/point_lock_manager.h224
-rw-r--r--src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.cc181
-rw-r--r--src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.h324
-rw-r--r--src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.cc257
-rw-r--r--src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.h99
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_lock_manager.h36
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_locking_test.cc459
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.AGPLv3661
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.APACHEv2174
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.GPLv2339
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/README13
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/db.h76
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/comparator.h138
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/ft-status.h102
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc139
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.h174
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.cc222
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.h141
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.cc527
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.h255
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.cc1023
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.h580
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/manager.cc527
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.cc265
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.h178
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.cc520
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.h302
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.cc120
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.h92
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.cc213
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.h124
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/memory.h215
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_assert_subst.h39
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_atomic.h130
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_external_pthread.h83
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_instrumentation.h286
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_portability.h87
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_pthread.h520
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_race_tools.h179
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h193
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/txn_subst.h27
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc132
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc153
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.h98
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h144
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc201
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.h141
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt.h794
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt_impl.h1295
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/partitioned_counter.h165
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/status.h76
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.cc503
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.h137
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.cc156
-rw-r--r--src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.h146
59 files changed, 15426 insertions, 0 deletions
diff --git a/src/rocksdb/utilities/transactions/lock/lock_manager.cc b/src/rocksdb/utilities/transactions/lock/lock_manager.cc
new file mode 100644
index 000000000..df16b32ad
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/lock_manager.cc
@@ -0,0 +1,29 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#ifndef ROCKSDB_LITE
+
+#include "utilities/transactions/lock/lock_manager.h"
+
+#include "utilities/transactions/lock/point/point_lock_manager.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+std::shared_ptr<LockManager> NewLockManager(PessimisticTransactionDB* db,
+ const TransactionDBOptions& opt) {
+ assert(db);
+ if (opt.lock_mgr_handle) {
+ // A custom lock manager was provided in options
+ auto mgr = opt.lock_mgr_handle->getLockManager();
+ return std::shared_ptr<LockManager>(opt.lock_mgr_handle, mgr);
+ } else {
+ // Use a point lock manager by default
+ return std::shared_ptr<LockManager>(new PointLockManager(db, opt));
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/lock_manager.h b/src/rocksdb/utilities/transactions/lock/lock_manager.h
new file mode 100644
index 000000000..a5ce1948c
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/lock_manager.h
@@ -0,0 +1,82 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#pragma once
+#ifndef ROCKSDB_LITE
+
+#include "rocksdb/types.h"
+#include "rocksdb/utilities/transaction.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "utilities/transactions/lock/lock_tracker.h"
+#include "utilities/transactions/pessimistic_transaction.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class PessimisticTransactionDB;
+
+class LockManager {
+ public:
+ virtual ~LockManager() {}
+
+ // Whether supports locking a specific key.
+ virtual bool IsPointLockSupported() const = 0;
+
+ // Whether supports locking a range of keys.
+ virtual bool IsRangeLockSupported() const = 0;
+
+ // Locks acquired through this LockManager should be tracked by
+ // the LockTrackers created through the returned factory.
+ virtual const LockTrackerFactory& GetLockTrackerFactory() const = 0;
+
+ // Enable locking for the specified column family.
+ // Caller should guarantee that this column family is not already enabled.
+ virtual void AddColumnFamily(const ColumnFamilyHandle* cf) = 0;
+
+ // Disable locking for the specified column family.
+ // Caller should guarantee that this column family is no longer used.
+ virtual void RemoveColumnFamily(const ColumnFamilyHandle* cf) = 0;
+
+ // Attempt to lock a key or a key range. If OK status is returned, the caller
+ // is responsible for calling UnLock() on this key.
+ virtual Status TryLock(PessimisticTransaction* txn,
+ ColumnFamilyId column_family_id,
+ const std::string& key, Env* env, bool exclusive) = 0;
+ // The range [start, end] are inclusive at both sides.
+ virtual Status TryLock(PessimisticTransaction* txn,
+ ColumnFamilyId column_family_id, const Endpoint& start,
+ const Endpoint& end, Env* env, bool exclusive) = 0;
+
+ // Unlock a key or a range locked by TryLock(). txn must be the same
+ // Transaction that locked this key.
+ virtual void UnLock(PessimisticTransaction* txn, const LockTracker& tracker,
+ Env* env) = 0;
+ virtual void UnLock(PessimisticTransaction* txn,
+ ColumnFamilyId column_family_id, const std::string& key,
+ Env* env) = 0;
+ virtual void UnLock(PessimisticTransaction* txn,
+ ColumnFamilyId column_family_id, const Endpoint& start,
+ const Endpoint& end, Env* env) = 0;
+
+ using PointLockStatus = std::unordered_multimap<ColumnFamilyId, KeyLockInfo>;
+ virtual PointLockStatus GetPointLockStatus() = 0;
+
+ using RangeLockStatus =
+ std::unordered_multimap<ColumnFamilyId, RangeLockInfo>;
+ virtual RangeLockStatus GetRangeLockStatus() = 0;
+
+ virtual std::vector<DeadlockPath> GetDeadlockInfoBuffer() = 0;
+
+ virtual void Resize(uint32_t new_size) = 0;
+};
+
+// LockManager should always be constructed through this factory method,
+// instead of constructing through concrete implementations' constructor.
+// Caller owns the returned pointer.
+std::shared_ptr<LockManager> NewLockManager(PessimisticTransactionDB* db,
+ const TransactionDBOptions& opt);
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/lock_tracker.h b/src/rocksdb/utilities/transactions/lock/lock_tracker.h
new file mode 100644
index 000000000..5fa228a82
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/lock_tracker.h
@@ -0,0 +1,209 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#pragma once
+#ifndef ROCKSDB_LITE
+
+#include <memory>
+
+#include "rocksdb/rocksdb_namespace.h"
+#include "rocksdb/status.h"
+#include "rocksdb/types.h"
+#include "rocksdb/utilities/transaction_db.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Request for locking a single key.
+struct PointLockRequest {
+ // The id of the key's column family.
+ ColumnFamilyId column_family_id = 0;
+ // The key to lock.
+ std::string key;
+ // The sequence number from which there is no concurrent update to key.
+ SequenceNumber seq = 0;
+ // Whether the lock is acquired only for read.
+ bool read_only = false;
+ // Whether the lock is in exclusive mode.
+ bool exclusive = true;
+};
+
+// Request for locking a range of keys.
+struct RangeLockRequest {
+ // The id of the key's column family.
+ ColumnFamilyId column_family_id;
+
+ // The range to be locked
+ Endpoint start_endp;
+ Endpoint end_endp;
+};
+
+struct PointLockStatus {
+ // Whether the key is locked.
+ bool locked = false;
+ // Whether the key is locked in exclusive mode.
+ bool exclusive = true;
+ // The sequence number in the tracked PointLockRequest.
+ SequenceNumber seq = 0;
+};
+
+// Return status when calling LockTracker::Untrack.
+enum class UntrackStatus {
+ // The lock is not tracked at all, so no lock to untrack.
+ NOT_TRACKED,
+ // The lock is untracked but not removed from the tracker.
+ UNTRACKED,
+ // The lock is removed from the tracker.
+ REMOVED,
+};
+
+// Tracks the lock requests.
+// In PessimisticTransaction, it tracks the locks acquired through LockMgr;
+// In OptimisticTransaction, since there is no LockMgr, it tracks the lock
+// intention. Not thread-safe.
+class LockTracker {
+ public:
+ virtual ~LockTracker() {}
+
+ // Whether supports locking a specific key.
+ virtual bool IsPointLockSupported() const = 0;
+
+ // Whether supports locking a range of keys.
+ virtual bool IsRangeLockSupported() const = 0;
+
+ // Tracks the acquirement of a lock on key.
+ //
+ // If this method is not supported, leave it as a no-op.
+ virtual void Track(const PointLockRequest& /*lock_request*/) = 0;
+
+ // Untracks the lock on a key.
+ // seq and exclusive in lock_request are not used.
+ //
+ // If this method is not supported, leave it as a no-op and
+ // returns NOT_TRACKED.
+ virtual UntrackStatus Untrack(const PointLockRequest& /*lock_request*/) = 0;
+
+ // Counterpart of Track(const PointLockRequest&) for RangeLockRequest.
+ virtual void Track(const RangeLockRequest& /*lock_request*/) = 0;
+
+ // Counterpart of Untrack(const PointLockRequest&) for RangeLockRequest.
+ virtual UntrackStatus Untrack(const RangeLockRequest& /*lock_request*/) = 0;
+
+ // Merges lock requests tracked in the specified tracker into the current
+ // tracker.
+ //
+ // E.g. for point lock, if a key in tracker is not yet tracked,
+ // track this new key; otherwise, merge the tracked information of the key
+ // such as lock's exclusiveness, read/write statistics.
+ //
+ // If this method is not supported, leave it as a no-op.
+ //
+ // REQUIRED: the specified tracker must be of the same concrete class type as
+ // the current tracker.
+ virtual void Merge(const LockTracker& /*tracker*/) = 0;
+
+ // This is a reverse operation of Merge.
+ //
+ // E.g. for point lock, if a key exists in both current and the sepcified
+ // tracker, then subtract the information (such as read/write statistics) of
+ // the key in the specified tracker from the current tracker.
+ //
+ // If this method is not supported, leave it as a no-op.
+ //
+ // REQUIRED:
+ // The specified tracker must be of the same concrete class type as
+ // the current tracker.
+ // The tracked locks in the specified tracker must be a subset of those
+ // tracked by the current tracker.
+ virtual void Subtract(const LockTracker& /*tracker*/) = 0;
+
+ // Clears all tracked locks.
+ virtual void Clear() = 0;
+
+ // Gets the new locks (excluding the locks that have been tracked before the
+ // save point) tracked since the specified save point, the result is stored
+ // in an internally constructed LockTracker and returned.
+ //
+ // save_point_tracker is the tracker used by a SavePoint to track locks
+ // tracked after creating the SavePoint.
+ //
+ // The implementation should document whether point lock, or range lock, or
+ // both are considered in this method.
+ // If this method is not supported, returns nullptr.
+ //
+ // REQUIRED:
+ // The save_point_tracker must be of the same concrete class type as the
+ // current tracker.
+ // The tracked locks in the specified tracker must be a subset of those
+ // tracked by the current tracker.
+ virtual LockTracker* GetTrackedLocksSinceSavePoint(
+ const LockTracker& /*save_point_tracker*/) const = 0;
+
+ // Gets lock related information of the key.
+ //
+ // If point lock is not supported, always returns LockStatus with
+ // locked=false.
+ virtual PointLockStatus GetPointLockStatus(
+ ColumnFamilyId /*column_family_id*/,
+ const std::string& /*key*/) const = 0;
+
+ // Gets number of tracked point locks.
+ //
+ // If point lock is not supported, always returns 0.
+ virtual uint64_t GetNumPointLocks() const = 0;
+
+ class ColumnFamilyIterator {
+ public:
+ virtual ~ColumnFamilyIterator() {}
+
+ // Whether there are remaining column families.
+ virtual bool HasNext() const = 0;
+
+ // Gets next column family id.
+ //
+ // If HasNext is false, calling this method has undefined behavior.
+ virtual ColumnFamilyId Next() = 0;
+ };
+
+ // Gets an iterator for column families.
+ //
+ // Returned iterator must not be nullptr.
+ // If there is no column family to iterate,
+ // returns an empty non-null iterator.
+ // Caller owns the returned pointer.
+ virtual ColumnFamilyIterator* GetColumnFamilyIterator() const = 0;
+
+ class KeyIterator {
+ public:
+ virtual ~KeyIterator() {}
+
+ // Whether there are remaining keys.
+ virtual bool HasNext() const = 0;
+
+ // Gets the next key.
+ //
+ // If HasNext is false, calling this method has undefined behavior.
+ virtual const std::string& Next() = 0;
+ };
+
+ // Gets an iterator for keys with tracked point locks in the column family.
+ //
+ // The column family must exist.
+ // Returned iterator must not be nullptr.
+ // Caller owns the returned pointer.
+ virtual KeyIterator* GetKeyIterator(
+ ColumnFamilyId /*column_family_id*/) const = 0;
+};
+
+// LockTracker should always be constructed through this factory.
+// Each LockManager owns a LockTrackerFactory.
+class LockTrackerFactory {
+ public:
+ // Caller owns the returned pointer.
+ virtual LockTracker* Create() const = 0;
+ virtual ~LockTrackerFactory() {}
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/point/point_lock_manager.cc b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager.cc
new file mode 100644
index 000000000..b362a164d
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager.cc
@@ -0,0 +1,721 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#ifndef ROCKSDB_LITE
+
+#include "utilities/transactions/lock/point/point_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/pessimistic_transaction_db.h"
+#include "utilities/transactions/transaction_db_mutex_impl.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct LockInfo {
+ bool exclusive;
+ autovector<TransactionID> txn_ids;
+
+ // Transaction locks are not valid after this time in us
+ uint64_t expiration_time;
+
+ LockInfo(TransactionID id, uint64_t time, bool ex)
+ : exclusive(ex), expiration_time(time) {
+ txn_ids.push_back(id);
+ }
+ LockInfo(const LockInfo& lock_info)
+ : exclusive(lock_info.exclusive),
+ txn_ids(lock_info.txn_ids),
+ expiration_time(lock_info.expiration_time) {}
+ void operator=(const LockInfo& lock_info) {
+ exclusive = lock_info.exclusive;
+ txn_ids = lock_info.txn_ids;
+ expiration_time = lock_info.expiration_time;
+ }
+ DECLARE_DEFAULT_MOVES(LockInfo);
+};
+
+struct LockMapStripe {
+ explicit LockMapStripe(std::shared_ptr<TransactionDBMutexFactory> factory) {
+ stripe_mutex = factory->AllocateMutex();
+ stripe_cv = factory->AllocateCondVar();
+ assert(stripe_mutex);
+ assert(stripe_cv);
+ }
+
+ // Mutex must be held before modifying keys map
+ std::shared_ptr<TransactionDBMutex> stripe_mutex;
+
+ // Condition Variable per stripe for waiting on a lock
+ std::shared_ptr<TransactionDBCondVar> stripe_cv;
+
+ // Locked keys mapped to the info about the transactions that locked them.
+ // TODO(agiardullo): Explore performance of other data structures.
+ UnorderedMap<std::string, LockInfo> keys;
+};
+
+// Map of #num_stripes LockMapStripes
+struct LockMap {
+ explicit LockMap(size_t num_stripes,
+ std::shared_ptr<TransactionDBMutexFactory> factory)
+ : num_stripes_(num_stripes) {
+ lock_map_stripes_.reserve(num_stripes);
+ for (size_t i = 0; i < num_stripes; i++) {
+ LockMapStripe* stripe = new LockMapStripe(factory);
+ lock_map_stripes_.push_back(stripe);
+ }
+ }
+
+ ~LockMap() {
+ for (auto stripe : lock_map_stripes_) {
+ delete stripe;
+ }
+ }
+
+ // Number of sepearate LockMapStripes to create, each with their own Mutex
+ const size_t num_stripes_;
+
+ // Count of keys that are currently locked in this column family.
+ // (Only maintained if PointLockManager::max_num_locks_ is positive.)
+ std::atomic<int64_t> lock_cnt{0};
+
+ std::vector<LockMapStripe*> lock_map_stripes_;
+
+ size_t GetStripe(const std::string& key) const;
+};
+
+namespace {
+void UnrefLockMapsCache(void* ptr) {
+ // Called when a thread exits or a ThreadLocalPtr gets destroyed.
+ auto lock_maps_cache =
+ static_cast<UnorderedMap<uint32_t, std::shared_ptr<LockMap>>*>(ptr);
+ delete lock_maps_cache;
+}
+} // anonymous namespace
+
+PointLockManager::PointLockManager(PessimisticTransactionDB* txn_db,
+ const TransactionDBOptions& opt)
+ : txn_db_impl_(txn_db),
+ default_num_stripes_(opt.num_stripes),
+ max_num_locks_(opt.max_num_locks),
+ lock_maps_cache_(new ThreadLocalPtr(&UnrefLockMapsCache)),
+ dlock_buffer_(opt.max_num_deadlocks),
+ mutex_factory_(opt.custom_mutex_factory
+ ? opt.custom_mutex_factory
+ : std::make_shared<TransactionDBMutexFactoryImpl>()) {}
+
+size_t LockMap::GetStripe(const std::string& key) const {
+ assert(num_stripes_ > 0);
+ return FastRange64(GetSliceNPHash64(key), num_stripes_);
+}
+
+void PointLockManager::AddColumnFamily(const ColumnFamilyHandle* cf) {
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ if (lock_maps_.find(cf->GetID()) == lock_maps_.end()) {
+ lock_maps_.emplace(cf->GetID(), std::make_shared<LockMap>(
+ default_num_stripes_, mutex_factory_));
+ } else {
+ // column_family already exists in lock map
+ assert(false);
+ }
+}
+
+void PointLockManager::RemoveColumnFamily(const ColumnFamilyHandle* cf) {
+ // Remove lock_map for this column family. Since the lock map is stored
+ // as a shared ptr, concurrent transactions can still keep using it
+ // until they release their references to it.
+ {
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ auto lock_maps_iter = lock_maps_.find(cf->GetID());
+ if (lock_maps_iter == lock_maps_.end()) {
+ return;
+ }
+
+ lock_maps_.erase(lock_maps_iter);
+ } // lock_map_mutex_
+
+ // Clear all thread-local caches
+ autovector<void*> local_caches;
+ lock_maps_cache_->Scrape(&local_caches, nullptr);
+ for (auto cache : local_caches) {
+ delete static_cast<LockMaps*>(cache);
+ }
+}
+
+// Look up the LockMap std::shared_ptr for a given column_family_id.
+// Note: The LockMap is only valid as long as the caller is still holding on
+// to the returned std::shared_ptr.
+std::shared_ptr<LockMap> PointLockManager::GetLockMap(
+ ColumnFamilyId column_family_id) {
+ // First check thread-local cache
+ if (lock_maps_cache_->Get() == nullptr) {
+ lock_maps_cache_->Reset(new LockMaps());
+ }
+
+ auto lock_maps_cache = static_cast<LockMaps*>(lock_maps_cache_->Get());
+
+ auto lock_map_iter = lock_maps_cache->find(column_family_id);
+ if (lock_map_iter != lock_maps_cache->end()) {
+ // Found lock map for this column family.
+ return lock_map_iter->second;
+ }
+
+ // Not found in local cache, grab mutex and check shared LockMaps
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ lock_map_iter = lock_maps_.find(column_family_id);
+ if (lock_map_iter == lock_maps_.end()) {
+ return std::shared_ptr<LockMap>(nullptr);
+ } else {
+ // Found lock map. Store in thread-local cache and return.
+ std::shared_ptr<LockMap>& lock_map = lock_map_iter->second;
+ lock_maps_cache->insert({column_family_id, lock_map});
+
+ return lock_map;
+ }
+}
+
+// Returns true if this lock has expired and can be acquired by another
+// transaction.
+// If false, sets *expire_time to the expiration time of the lock according
+// to Env->GetMicros() or 0 if no expiration.
+bool PointLockManager::IsLockExpired(TransactionID txn_id,
+ const LockInfo& lock_info, Env* env,
+ uint64_t* expire_time) {
+ if (lock_info.expiration_time == 0) {
+ *expire_time = 0;
+ return false;
+ }
+
+ auto now = env->NowMicros();
+ bool expired = lock_info.expiration_time <= now;
+ if (!expired) {
+ // return how many microseconds until lock will be expired
+ *expire_time = lock_info.expiration_time;
+ } else {
+ for (auto id : lock_info.txn_ids) {
+ if (txn_id == id) {
+ continue;
+ }
+
+ bool success = txn_db_impl_->TryStealingExpiredTransactionLocks(id);
+ if (!success) {
+ expired = false;
+ *expire_time = 0;
+ break;
+ }
+ }
+ }
+
+ return expired;
+}
+
+Status PointLockManager::TryLock(PessimisticTransaction* txn,
+ ColumnFamilyId column_family_id,
+ const std::string& key, Env* env,
+ bool exclusive) {
+ // Lookup lock map for this column family id
+ std::shared_ptr<LockMap> lock_map_ptr = GetLockMap(column_family_id);
+ LockMap* lock_map = lock_map_ptr.get();
+ if (lock_map == nullptr) {
+ char msg[255];
+ snprintf(msg, sizeof(msg), "Column family id not found: %" PRIu32,
+ column_family_id);
+
+ return Status::InvalidArgument(msg);
+ }
+
+ // Need to lock the mutex for the stripe that this key hashes to
+ size_t stripe_num = lock_map->GetStripe(key);
+ assert(lock_map->lock_map_stripes_.size() > stripe_num);
+ LockMapStripe* stripe = lock_map->lock_map_stripes_.at(stripe_num);
+
+ LockInfo lock_info(txn->GetID(), txn->GetExpirationTime(), exclusive);
+ int64_t timeout = txn->GetLockTimeout();
+
+ return AcquireWithTimeout(txn, lock_map, stripe, column_family_id, key, env,
+ timeout, lock_info);
+}
+
+// Helper function for TryLock().
+Status PointLockManager::AcquireWithTimeout(
+ PessimisticTransaction* txn, LockMap* lock_map, LockMapStripe* stripe,
+ ColumnFamilyId column_family_id, const std::string& key, Env* env,
+ int64_t timeout, const LockInfo& lock_info) {
+ Status result;
+ uint64_t end_time = 0;
+
+ if (timeout > 0) {
+ uint64_t start_time = env->NowMicros();
+ end_time = start_time + timeout;
+ }
+
+ if (timeout < 0) {
+ // If timeout is negative, we wait indefinitely to acquire the lock
+ result = stripe->stripe_mutex->Lock();
+ } else {
+ result = stripe->stripe_mutex->TryLockFor(timeout);
+ }
+
+ if (!result.ok()) {
+ // failed to acquire mutex
+ return result;
+ }
+
+ // Acquire lock if we are able to
+ uint64_t expire_time_hint = 0;
+ autovector<TransactionID> wait_ids;
+ result = AcquireLocked(lock_map, stripe, key, env, lock_info,
+ &expire_time_hint, &wait_ids);
+
+ if (!result.ok() && timeout != 0) {
+ PERF_TIMER_GUARD(key_lock_wait_time);
+ PERF_COUNTER_ADD(key_lock_wait_count, 1);
+ // If we weren't able to acquire the lock, we will keep retrying as long
+ // as the timeout allows.
+ bool timed_out = false;
+ do {
+ // Decide how long to wait
+ int64_t cv_end_time = -1;
+ if (expire_time_hint > 0 && end_time > 0) {
+ cv_end_time = std::min(expire_time_hint, end_time);
+ } else if (expire_time_hint > 0) {
+ cv_end_time = expire_time_hint;
+ } else if (end_time > 0) {
+ cv_end_time = end_time;
+ }
+
+ assert(result.IsBusy() || wait_ids.size() != 0);
+
+ // We are dependent on a transaction to finish, so perform deadlock
+ // detection.
+ if (wait_ids.size() != 0) {
+ if (txn->IsDeadlockDetect()) {
+ if (IncrementWaiters(txn, wait_ids, key, column_family_id,
+ lock_info.exclusive, env)) {
+ result = Status::Busy(Status::SubCode::kDeadlock);
+ stripe->stripe_mutex->UnLock();
+ return result;
+ }
+ }
+ txn->SetWaitingTxn(wait_ids, column_family_id, &key);
+ }
+
+ TEST_SYNC_POINT("PointLockManager::AcquireWithTimeout:WaitingTxn");
+ if (cv_end_time < 0) {
+ // Wait indefinitely
+ result = stripe->stripe_cv->Wait(stripe->stripe_mutex);
+ } else {
+ uint64_t now = env->NowMicros();
+ if (static_cast<uint64_t>(cv_end_time) > now) {
+ result = stripe->stripe_cv->WaitFor(stripe->stripe_mutex,
+ cv_end_time - now);
+ }
+ }
+
+ if (wait_ids.size() != 0) {
+ txn->ClearWaitingTxn();
+ if (txn->IsDeadlockDetect()) {
+ DecrementWaiters(txn, wait_ids);
+ }
+ }
+
+ if (result.IsTimedOut()) {
+ timed_out = true;
+ // Even though we timed out, we will still make one more attempt to
+ // acquire lock below (it is possible the lock expired and we
+ // were never signaled).
+ }
+
+ if (result.ok() || result.IsTimedOut()) {
+ result = AcquireLocked(lock_map, stripe, key, env, lock_info,
+ &expire_time_hint, &wait_ids);
+ }
+ } while (!result.ok() && !timed_out);
+ }
+
+ stripe->stripe_mutex->UnLock();
+
+ return result;
+}
+
+void PointLockManager::DecrementWaiters(
+ const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids) {
+ std::lock_guard<std::mutex> lock(wait_txn_map_mutex_);
+ DecrementWaitersImpl(txn, wait_ids);
+}
+
+void PointLockManager::DecrementWaitersImpl(
+ const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids) {
+ auto id = txn->GetID();
+ assert(wait_txn_map_.Contains(id));
+ wait_txn_map_.Delete(id);
+
+ for (auto wait_id : wait_ids) {
+ rev_wait_txn_map_.Get(wait_id)--;
+ if (rev_wait_txn_map_.Get(wait_id) == 0) {
+ rev_wait_txn_map_.Delete(wait_id);
+ }
+ }
+}
+
+bool PointLockManager::IncrementWaiters(
+ const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids, const std::string& key,
+ const uint32_t& cf_id, const bool& exclusive, Env* const env) {
+ auto id = txn->GetID();
+ std::vector<int> queue_parents(
+ static_cast<size_t>(txn->GetDeadlockDetectDepth()));
+ std::vector<TransactionID> queue_values(
+ static_cast<size_t>(txn->GetDeadlockDetectDepth()));
+ std::lock_guard<std::mutex> lock(wait_txn_map_mutex_);
+ assert(!wait_txn_map_.Contains(id));
+
+ wait_txn_map_.Insert(id, {wait_ids, cf_id, exclusive, key});
+
+ for (auto wait_id : wait_ids) {
+ if (rev_wait_txn_map_.Contains(wait_id)) {
+ rev_wait_txn_map_.Get(wait_id)++;
+ } else {
+ rev_wait_txn_map_.Insert(wait_id, 1);
+ }
+ }
+
+ // No deadlock if nobody is waiting on self.
+ if (!rev_wait_txn_map_.Contains(id)) {
+ return false;
+ }
+
+ const auto* next_ids = &wait_ids;
+ int parent = -1;
+ int64_t deadlock_time = 0;
+ for (int tail = 0, head = 0; head < txn->GetDeadlockDetectDepth(); head++) {
+ int i = 0;
+ if (next_ids) {
+ for (; i < static_cast<int>(next_ids->size()) &&
+ tail + i < txn->GetDeadlockDetectDepth();
+ i++) {
+ queue_values[tail + i] = (*next_ids)[i];
+ queue_parents[tail + i] = parent;
+ }
+ tail += i;
+ }
+
+ // No more items in the list, meaning no deadlock.
+ if (tail == head) {
+ return false;
+ }
+
+ auto next = queue_values[head];
+ if (next == id) {
+ std::vector<DeadlockInfo> path;
+ while (head != -1) {
+ assert(wait_txn_map_.Contains(queue_values[head]));
+
+ auto extracted_info = wait_txn_map_.Get(queue_values[head]);
+ path.push_back({queue_values[head], extracted_info.m_cf_id,
+ extracted_info.m_exclusive,
+ extracted_info.m_waiting_key});
+ head = queue_parents[head];
+ }
+ if (!env->GetCurrentTime(&deadlock_time).ok()) {
+ /*
+ TODO(AR) this preserves the current behaviour whilst checking the
+ status of env->GetCurrentTime to ensure that ASSERT_STATUS_CHECKED
+ passes. Should we instead raise an error if !ok() ?
+ */
+ deadlock_time = 0;
+ }
+ std::reverse(path.begin(), path.end());
+ dlock_buffer_.AddNewPath(DeadlockPath(path, deadlock_time));
+ deadlock_time = 0;
+ DecrementWaitersImpl(txn, wait_ids);
+ return true;
+ } else if (!wait_txn_map_.Contains(next)) {
+ next_ids = nullptr;
+ continue;
+ } else {
+ parent = head;
+ next_ids = &(wait_txn_map_.Get(next).m_neighbors);
+ }
+ }
+
+ // Wait cycle too big, just assume deadlock.
+ if (!env->GetCurrentTime(&deadlock_time).ok()) {
+ /*
+ TODO(AR) this preserves the current behaviour whilst checking the status
+ of env->GetCurrentTime to ensure that ASSERT_STATUS_CHECKED passes.
+ Should we instead raise an error if !ok() ?
+ */
+ deadlock_time = 0;
+ }
+ dlock_buffer_.AddNewPath(DeadlockPath(deadlock_time, true));
+ DecrementWaitersImpl(txn, wait_ids);
+ return true;
+}
+
+// Try to lock this key after we have acquired the mutex.
+// Sets *expire_time to the expiration time in microseconds
+// or 0 if no expiration.
+// REQUIRED: Stripe mutex must be held.
+Status PointLockManager::AcquireLocked(LockMap* lock_map, LockMapStripe* stripe,
+ const std::string& key, Env* env,
+ const LockInfo& txn_lock_info,
+ uint64_t* expire_time,
+ autovector<TransactionID>* txn_ids) {
+ assert(txn_lock_info.txn_ids.size() == 1);
+
+ Status result;
+ // Check if this key is already locked
+ auto stripe_iter = stripe->keys.find(key);
+ if (stripe_iter != stripe->keys.end()) {
+ // Lock already held
+ LockInfo& lock_info = stripe_iter->second;
+ assert(lock_info.txn_ids.size() == 1 || !lock_info.exclusive);
+
+ if (lock_info.exclusive || txn_lock_info.exclusive) {
+ if (lock_info.txn_ids.size() == 1 &&
+ lock_info.txn_ids[0] == txn_lock_info.txn_ids[0]) {
+ // The list contains one txn and we're it, so just take it.
+ lock_info.exclusive = txn_lock_info.exclusive;
+ lock_info.expiration_time = txn_lock_info.expiration_time;
+ } else {
+ // Check if it's expired. Skips over txn_lock_info.txn_ids[0] in case
+ // it's there for a shared lock with multiple holders which was not
+ // caught in the first case.
+ if (IsLockExpired(txn_lock_info.txn_ids[0], lock_info, env,
+ expire_time)) {
+ // lock is expired, can steal it
+ lock_info.txn_ids = txn_lock_info.txn_ids;
+ lock_info.exclusive = txn_lock_info.exclusive;
+ lock_info.expiration_time = txn_lock_info.expiration_time;
+ // lock_cnt does not change
+ } else {
+ result = Status::TimedOut(Status::SubCode::kLockTimeout);
+ *txn_ids = lock_info.txn_ids;
+ }
+ }
+ } else {
+ // We are requesting shared access to a shared lock, so just grant it.
+ lock_info.txn_ids.push_back(txn_lock_info.txn_ids[0]);
+ // Using std::max means that expiration time never goes down even when
+ // a transaction is removed from the list. The correct solution would be
+ // to track expiry for every transaction, but this would also work for
+ // now.
+ lock_info.expiration_time =
+ std::max(lock_info.expiration_time, txn_lock_info.expiration_time);
+ }
+ } else { // Lock not held.
+ // Check lock limit
+ if (max_num_locks_ > 0 &&
+ lock_map->lock_cnt.load(std::memory_order_acquire) >= max_num_locks_) {
+ result = Status::Busy(Status::SubCode::kLockLimit);
+ } else {
+ // acquire lock
+ stripe->keys.emplace(key, txn_lock_info);
+
+ // Maintain lock count if there is a limit on the number of locks
+ if (max_num_locks_) {
+ lock_map->lock_cnt++;
+ }
+ }
+ }
+
+ return result;
+}
+
+void PointLockManager::UnLockKey(PessimisticTransaction* txn,
+ const std::string& key, LockMapStripe* stripe,
+ LockMap* lock_map, Env* env) {
+#ifdef NDEBUG
+ (void)env;
+#endif
+ TransactionID txn_id = txn->GetID();
+
+ auto stripe_iter = stripe->keys.find(key);
+ if (stripe_iter != stripe->keys.end()) {
+ auto& txns = stripe_iter->second.txn_ids;
+ auto txn_it = std::find(txns.begin(), txns.end(), txn_id);
+ // Found the key we locked. unlock it.
+ if (txn_it != txns.end()) {
+ if (txns.size() == 1) {
+ stripe->keys.erase(stripe_iter);
+ } else {
+ auto last_it = txns.end() - 1;
+ if (txn_it != last_it) {
+ *txn_it = *last_it;
+ }
+ txns.pop_back();
+ }
+
+ if (max_num_locks_ > 0) {
+ // Maintain lock count if there is a limit on the number of locks.
+ assert(lock_map->lock_cnt.load(std::memory_order_relaxed) > 0);
+ lock_map->lock_cnt--;
+ }
+ }
+ } else {
+ // This key is either not locked or locked by someone else. This should
+ // only happen if the unlocking transaction has expired.
+ assert(txn->GetExpirationTime() > 0 &&
+ txn->GetExpirationTime() < env->NowMicros());
+ }
+}
+
+void PointLockManager::UnLock(PessimisticTransaction* txn,
+ ColumnFamilyId column_family_id,
+ const std::string& key, Env* env) {
+ std::shared_ptr<LockMap> lock_map_ptr = GetLockMap(column_family_id);
+ LockMap* lock_map = lock_map_ptr.get();
+ if (lock_map == nullptr) {
+ // Column Family must have been dropped.
+ return;
+ }
+
+ // Lock the mutex for the stripe that this key hashes to
+ size_t stripe_num = lock_map->GetStripe(key);
+ assert(lock_map->lock_map_stripes_.size() > stripe_num);
+ LockMapStripe* stripe = lock_map->lock_map_stripes_.at(stripe_num);
+
+ stripe->stripe_mutex->Lock().PermitUncheckedError();
+ UnLockKey(txn, key, stripe, lock_map, env);
+ stripe->stripe_mutex->UnLock();
+
+ // Signal waiting threads to retry locking
+ stripe->stripe_cv->NotifyAll();
+}
+
+void PointLockManager::UnLock(PessimisticTransaction* txn,
+ const LockTracker& tracker, Env* env) {
+ std::unique_ptr<LockTracker::ColumnFamilyIterator> cf_it(
+ tracker.GetColumnFamilyIterator());
+ assert(cf_it != nullptr);
+ while (cf_it->HasNext()) {
+ ColumnFamilyId cf = cf_it->Next();
+ std::shared_ptr<LockMap> lock_map_ptr = GetLockMap(cf);
+ LockMap* lock_map = lock_map_ptr.get();
+ if (!lock_map) {
+ // Column Family must have been dropped.
+ return;
+ }
+
+ // Bucket keys by lock_map_ stripe
+ UnorderedMap<size_t, std::vector<const std::string*>> keys_by_stripe(
+ lock_map->num_stripes_);
+ std::unique_ptr<LockTracker::KeyIterator> key_it(
+ tracker.GetKeyIterator(cf));
+ assert(key_it != nullptr);
+ while (key_it->HasNext()) {
+ const std::string& key = key_it->Next();
+ size_t stripe_num = lock_map->GetStripe(key);
+ keys_by_stripe[stripe_num].push_back(&key);
+ }
+
+ // For each stripe, grab the stripe mutex and unlock all keys in this stripe
+ for (auto& stripe_iter : keys_by_stripe) {
+ size_t stripe_num = stripe_iter.first;
+ auto& stripe_keys = stripe_iter.second;
+
+ assert(lock_map->lock_map_stripes_.size() > stripe_num);
+ LockMapStripe* stripe = lock_map->lock_map_stripes_.at(stripe_num);
+
+ stripe->stripe_mutex->Lock().PermitUncheckedError();
+
+ for (const std::string* key : stripe_keys) {
+ UnLockKey(txn, *key, stripe, lock_map, env);
+ }
+
+ stripe->stripe_mutex->UnLock();
+
+ // Signal waiting threads to retry locking
+ stripe->stripe_cv->NotifyAll();
+ }
+ }
+}
+
+PointLockManager::PointLockStatus PointLockManager::GetPointLockStatus() {
+ PointLockStatus data;
+ // Lock order here is important. The correct order is lock_map_mutex_, then
+ // for every column family ID in ascending order lock every stripe in
+ // ascending order.
+ InstrumentedMutexLock l(&lock_map_mutex_);
+
+ std::vector<uint32_t> cf_ids;
+ for (const auto& map : lock_maps_) {
+ cf_ids.push_back(map.first);
+ }
+ std::sort(cf_ids.begin(), cf_ids.end());
+
+ for (auto i : cf_ids) {
+ const auto& stripes = lock_maps_[i]->lock_map_stripes_;
+ // Iterate and lock all stripes in ascending order.
+ for (const auto& j : stripes) {
+ j->stripe_mutex->Lock().PermitUncheckedError();
+ for (const auto& it : j->keys) {
+ struct KeyLockInfo info;
+ info.exclusive = it.second.exclusive;
+ info.key = it.first;
+ for (const auto& id : it.second.txn_ids) {
+ info.ids.push_back(id);
+ }
+ data.insert({i, info});
+ }
+ }
+ }
+
+ // Unlock everything. Unlocking order is not important.
+ for (auto i : cf_ids) {
+ const auto& stripes = lock_maps_[i]->lock_map_stripes_;
+ for (const auto& j : stripes) {
+ j->stripe_mutex->UnLock();
+ }
+ }
+
+ return data;
+}
+
+std::vector<DeadlockPath> PointLockManager::GetDeadlockInfoBuffer() {
+ return dlock_buffer_.PrepareBuffer();
+}
+
+void PointLockManager::Resize(uint32_t target_size) {
+ dlock_buffer_.Resize(target_size);
+}
+
+PointLockManager::RangeLockStatus PointLockManager::GetRangeLockStatus() {
+ return {};
+}
+
+Status PointLockManager::TryLock(PessimisticTransaction* /* txn */,
+ ColumnFamilyId /* cf_id */,
+ const Endpoint& /* start */,
+ const Endpoint& /* end */, Env* /* env */,
+ bool /* exclusive */) {
+ return Status::NotSupported(
+ "PointLockManager does not support range locking");
+}
+
+void PointLockManager::UnLock(PessimisticTransaction* /* txn */,
+ ColumnFamilyId /* cf_id */,
+ const Endpoint& /* start */,
+ const Endpoint& /* end */, Env* /* env */) {
+ // no-op
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/point/point_lock_manager.h b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager.h
new file mode 100644
index 000000000..eeb34f3be
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager.h
@@ -0,0 +1,224 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#pragma once
+#ifndef ROCKSDB_LITE
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "monitoring/instrumented_mutex.h"
+#include "rocksdb/utilities/transaction.h"
+#include "util/autovector.h"
+#include "util/hash_containers.h"
+#include "util/hash_map.h"
+#include "util/thread_local.h"
+#include "utilities/transactions/lock/lock_manager.h"
+#include "utilities/transactions/lock/point/point_lock_tracker.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class ColumnFamilyHandle;
+struct LockInfo;
+struct LockMap;
+struct LockMapStripe;
+
+template <class Path>
+class DeadlockInfoBufferTempl {
+ private:
+ std::vector<Path> paths_buffer_;
+ uint32_t buffer_idx_;
+ std::mutex paths_buffer_mutex_;
+
+ std::vector<Path> Normalize() {
+ auto working = paths_buffer_;
+
+ if (working.empty()) {
+ return working;
+ }
+
+ // Next write occurs at a nonexistent path's slot
+ if (paths_buffer_[buffer_idx_].empty()) {
+ working.resize(buffer_idx_);
+ } else {
+ std::rotate(working.begin(), working.begin() + buffer_idx_,
+ working.end());
+ }
+
+ return working;
+ }
+
+ public:
+ explicit DeadlockInfoBufferTempl(uint32_t n_latest_dlocks)
+ : paths_buffer_(n_latest_dlocks), buffer_idx_(0) {}
+
+ void AddNewPath(Path path) {
+ std::lock_guard<std::mutex> lock(paths_buffer_mutex_);
+
+ if (paths_buffer_.empty()) {
+ return;
+ }
+
+ paths_buffer_[buffer_idx_] = std::move(path);
+ buffer_idx_ = (buffer_idx_ + 1) % paths_buffer_.size();
+ }
+
+ void Resize(uint32_t target_size) {
+ std::lock_guard<std::mutex> lock(paths_buffer_mutex_);
+
+ paths_buffer_ = Normalize();
+
+ // Drop the deadlocks that will no longer be needed ater the normalize
+ if (target_size < paths_buffer_.size()) {
+ paths_buffer_.erase(
+ paths_buffer_.begin(),
+ paths_buffer_.begin() + (paths_buffer_.size() - target_size));
+ buffer_idx_ = 0;
+ }
+ // Resize the buffer to the target size and restore the buffer's idx
+ else {
+ auto prev_size = paths_buffer_.size();
+ paths_buffer_.resize(target_size);
+ buffer_idx_ = (uint32_t)prev_size;
+ }
+ }
+
+ std::vector<Path> PrepareBuffer() {
+ std::lock_guard<std::mutex> lock(paths_buffer_mutex_);
+
+ // Reversing the normalized vector returns the latest deadlocks first
+ auto working = Normalize();
+ std::reverse(working.begin(), working.end());
+
+ return working;
+ }
+};
+
+using DeadlockInfoBuffer = DeadlockInfoBufferTempl<DeadlockPath>;
+
+struct TrackedTrxInfo {
+ autovector<TransactionID> m_neighbors;
+ uint32_t m_cf_id;
+ bool m_exclusive;
+ std::string m_waiting_key;
+};
+
+class PointLockManager : public LockManager {
+ public:
+ PointLockManager(PessimisticTransactionDB* db,
+ const TransactionDBOptions& opt);
+ // No copying allowed
+ PointLockManager(const PointLockManager&) = delete;
+ PointLockManager& operator=(const PointLockManager&) = delete;
+
+ ~PointLockManager() override {}
+
+ bool IsPointLockSupported() const override { return true; }
+
+ bool IsRangeLockSupported() const override { return false; }
+
+ const LockTrackerFactory& GetLockTrackerFactory() const override {
+ return PointLockTrackerFactory::Get();
+ }
+
+ // Creates a new LockMap for this column family. Caller should guarantee
+ // that this column family does not already exist.
+ void AddColumnFamily(const ColumnFamilyHandle* cf) override;
+ // Deletes the LockMap for this column family. Caller should guarantee that
+ // this column family is no longer in use.
+ void RemoveColumnFamily(const ColumnFamilyHandle* cf) override;
+
+ Status TryLock(PessimisticTransaction* txn, ColumnFamilyId column_family_id,
+ const std::string& key, Env* env, bool exclusive) override;
+ Status TryLock(PessimisticTransaction* txn, ColumnFamilyId column_family_id,
+ const Endpoint& start, const Endpoint& end, 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* txn, ColumnFamilyId column_family_id,
+ const Endpoint& start, const Endpoint& end, Env* env) override;
+
+ PointLockStatus GetPointLockStatus() override;
+
+ RangeLockStatus GetRangeLockStatus() override;
+
+ std::vector<DeadlockPath> GetDeadlockInfoBuffer() override;
+
+ void Resize(uint32_t new_size) override;
+
+ private:
+ PessimisticTransactionDB* txn_db_impl_;
+
+ // Default number of lock map stripes per column family
+ const size_t default_num_stripes_;
+
+ // Limit on number of keys locked per column family
+ const int64_t max_num_locks_;
+
+ // The following lock order must be satisfied in order to avoid deadlocking
+ // ourselves.
+ // - lock_map_mutex_
+ // - stripe mutexes in ascending cf id, ascending stripe order
+ // - wait_txn_map_mutex_
+ //
+ // Must be held when accessing/modifying lock_maps_.
+ InstrumentedMutex lock_map_mutex_;
+
+ // Map of ColumnFamilyId to locked key info
+ using LockMaps = UnorderedMap<uint32_t, std::shared_ptr<LockMap>>;
+ LockMaps lock_maps_;
+
+ // Thread-local cache of entries in lock_maps_. This is an optimization
+ // to avoid acquiring a mutex in order to look up a LockMap
+ std::unique_ptr<ThreadLocalPtr> lock_maps_cache_;
+
+ // Must be held when modifying wait_txn_map_ and rev_wait_txn_map_.
+ std::mutex wait_txn_map_mutex_;
+
+ // Maps from waitee -> number of waiters.
+ HashMap<TransactionID, int> rev_wait_txn_map_;
+ // Maps from waiter -> waitee.
+ HashMap<TransactionID, TrackedTrxInfo> wait_txn_map_;
+ DeadlockInfoBuffer dlock_buffer_;
+
+ // Used to allocate mutexes/condvars to use when locking keys
+ std::shared_ptr<TransactionDBMutexFactory> mutex_factory_;
+
+ bool IsLockExpired(TransactionID txn_id, const LockInfo& lock_info, Env* env,
+ uint64_t* wait_time);
+
+ std::shared_ptr<LockMap> GetLockMap(uint32_t column_family_id);
+
+ Status AcquireWithTimeout(PessimisticTransaction* txn, LockMap* lock_map,
+ LockMapStripe* stripe, uint32_t column_family_id,
+ const std::string& key, Env* env, int64_t timeout,
+ const LockInfo& lock_info);
+
+ Status AcquireLocked(LockMap* lock_map, LockMapStripe* stripe,
+ const std::string& key, Env* env,
+ const LockInfo& lock_info, uint64_t* wait_time,
+ autovector<TransactionID>* txn_ids);
+
+ void UnLockKey(PessimisticTransaction* txn, const std::string& key,
+ LockMapStripe* stripe, LockMap* lock_map, Env* env);
+
+ bool IncrementWaiters(const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids,
+ const std::string& key, const uint32_t& cf_id,
+ const bool& exclusive, Env* const env);
+ void DecrementWaiters(const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids);
+ void DecrementWaitersImpl(const PessimisticTransaction* txn,
+ const autovector<TransactionID>& wait_ids);
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.cc b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.cc
new file mode 100644
index 000000000..525fdea71
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.cc
@@ -0,0 +1,181 @@
+// Copyright (c) 2020-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#ifndef ROCKSDB_LITE
+
+#include "utilities/transactions/lock/point/point_lock_manager_test.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// This test is not applicable for Range Lock manager as Range Lock Manager
+// operates on Column Families, not their ids.
+TEST_F(PointLockManagerTest, LockNonExistingColumnFamily) {
+ MockColumnFamilyHandle cf(1024);
+ locker_->RemoveColumnFamily(&cf);
+ auto txn = NewTxn();
+ auto s = locker_->TryLock(txn, 1024, "k", env_, true);
+ ASSERT_TRUE(s.IsInvalidArgument());
+ ASSERT_STREQ(s.getState(), "Column family id not found: 1024");
+ delete txn;
+}
+
+TEST_F(PointLockManagerTest, LockStatus) {
+ MockColumnFamilyHandle cf1(1024), cf2(2048);
+ locker_->AddColumnFamily(&cf1);
+ locker_->AddColumnFamily(&cf2);
+
+ auto txn1 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn1, 1024, "k1", env_, true));
+ ASSERT_OK(locker_->TryLock(txn1, 2048, "k1", env_, true));
+
+ auto txn2 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn2, 1024, "k2", env_, false));
+ ASSERT_OK(locker_->TryLock(txn2, 2048, "k2", env_, false));
+
+ auto s = locker_->GetPointLockStatus();
+ ASSERT_EQ(s.size(), 4u);
+ for (uint32_t cf_id : {1024, 2048}) {
+ ASSERT_EQ(s.count(cf_id), 2u);
+ auto range = s.equal_range(cf_id);
+ for (auto it = range.first; it != range.second; it++) {
+ ASSERT_TRUE(it->second.key == "k1" || it->second.key == "k2");
+ if (it->second.key == "k1") {
+ ASSERT_EQ(it->second.exclusive, true);
+ ASSERT_EQ(it->second.ids.size(), 1u);
+ ASSERT_EQ(it->second.ids[0], txn1->GetID());
+ } else if (it->second.key == "k2") {
+ ASSERT_EQ(it->second.exclusive, false);
+ ASSERT_EQ(it->second.ids.size(), 1u);
+ ASSERT_EQ(it->second.ids[0], txn2->GetID());
+ }
+ }
+ }
+
+ // Cleanup
+ locker_->UnLock(txn1, 1024, "k1", env_);
+ locker_->UnLock(txn1, 2048, "k1", env_);
+ locker_->UnLock(txn2, 1024, "k2", env_);
+ locker_->UnLock(txn2, 2048, "k2", env_);
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_F(PointLockManagerTest, UnlockExclusive) {
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+
+ auto txn1 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k", env_, true));
+ locker_->UnLock(txn1, 1, "k", env_);
+
+ auto txn2 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn2, 1, "k", env_, true));
+
+ // Cleanup
+ locker_->UnLock(txn2, 1, "k", env_);
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_F(PointLockManagerTest, UnlockShared) {
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+
+ auto txn1 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k", env_, false));
+ locker_->UnLock(txn1, 1, "k", env_);
+
+ auto txn2 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn2, 1, "k", env_, true));
+
+ // Cleanup
+ locker_->UnLock(txn2, 1, "k", env_);
+
+ delete txn1;
+ delete txn2;
+}
+
+// This test doesn't work with Range Lock Manager, because Range Lock Manager
+// doesn't support deadlock_detect_depth.
+
+TEST_F(PointLockManagerTest, DeadlockDepthExceeded) {
+ // Tests that when detecting deadlock, if the detection depth is exceeded,
+ // it's also viewed as deadlock.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ TransactionOptions txn_opt;
+ txn_opt.deadlock_detect = true;
+ txn_opt.deadlock_detect_depth = 1;
+ txn_opt.lock_timeout = 1000000;
+ auto txn1 = NewTxn(txn_opt);
+ auto txn2 = NewTxn(txn_opt);
+ auto txn3 = NewTxn(txn_opt);
+ auto txn4 = NewTxn(txn_opt);
+ // "a ->(k) b" means transaction a is waiting for transaction b to release
+ // the held lock on key k.
+ // txn4 ->(k3) -> txn3 ->(k2) txn2 ->(k1) txn1
+ // txn3's deadlock detection will exceed the detection depth 1,
+ // which will be viewed as a deadlock.
+ // NOTE:
+ // txn4 ->(k3) -> txn3 must be set up before
+ // txn3 ->(k2) -> txn2, because to trigger deadlock detection for txn3,
+ // it must have another txn waiting on it, which is txn4 in this case.
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k1", env_, true));
+
+ port::Thread t1 = BlockUntilWaitingTxn(wait_sync_point_name_, [&]() {
+ ASSERT_OK(locker_->TryLock(txn2, 1, "k2", env_, true));
+ // block because txn1 is holding a lock on k1.
+ locker_->TryLock(txn2, 1, "k1", env_, true);
+ });
+
+ ASSERT_OK(locker_->TryLock(txn3, 1, "k3", env_, true));
+
+ port::Thread t2 = BlockUntilWaitingTxn(wait_sync_point_name_, [&]() {
+ // block because txn3 is holding a lock on k1.
+ locker_->TryLock(txn4, 1, "k3", env_, true);
+ });
+
+ auto s = locker_->TryLock(txn3, 1, "k2", env_, true);
+ ASSERT_TRUE(s.IsBusy());
+ ASSERT_EQ(s.subcode(), Status::SubCode::kDeadlock);
+
+ std::vector<DeadlockPath> deadlock_paths = locker_->GetDeadlockInfoBuffer();
+ ASSERT_EQ(deadlock_paths.size(), 1u);
+ ASSERT_TRUE(deadlock_paths[0].limit_exceeded);
+
+ locker_->UnLock(txn1, 1, "k1", env_);
+ locker_->UnLock(txn3, 1, "k3", env_);
+ t1.join();
+ t2.join();
+
+ delete txn4;
+ delete txn3;
+ delete txn2;
+ delete txn1;
+}
+
+INSTANTIATE_TEST_CASE_P(PointLockManager, AnyLockManagerTest,
+ ::testing::Values(nullptr));
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(stderr,
+ "SKIPPED because Transactions are not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.h b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.h
new file mode 100644
index 000000000..ca9f46bf9
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.h
@@ -0,0 +1,324 @@
+// Copyright (c) Meta Platforms, Inc. and affiliates.
+//
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#include "file/file_util.h"
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/utilities/transaction_db.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "utilities/transactions/lock/point/point_lock_manager.h"
+#include "utilities/transactions/pessimistic_transaction_db.h"
+#include "utilities/transactions/transaction_db_mutex_impl.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class MockColumnFamilyHandle : public ColumnFamilyHandle {
+ public:
+ explicit MockColumnFamilyHandle(ColumnFamilyId cf_id) : cf_id_(cf_id) {}
+
+ ~MockColumnFamilyHandle() override {}
+
+ const std::string& GetName() const override { return name_; }
+
+ ColumnFamilyId GetID() const override { return cf_id_; }
+
+ Status GetDescriptor(ColumnFamilyDescriptor*) override {
+ return Status::OK();
+ }
+
+ const Comparator* GetComparator() const override {
+ return BytewiseComparator();
+ }
+
+ private:
+ ColumnFamilyId cf_id_;
+ std::string name_ = "MockCF";
+};
+
+class PointLockManagerTest : public testing::Test {
+ public:
+ void SetUp() override {
+ env_ = Env::Default();
+ db_dir_ = test::PerThreadDBPath("point_lock_manager_test");
+ ASSERT_OK(env_->CreateDir(db_dir_));
+
+ Options opt;
+ opt.create_if_missing = true;
+ TransactionDBOptions txn_opt;
+ txn_opt.transaction_lock_timeout = 0;
+
+ ASSERT_OK(TransactionDB::Open(opt, txn_opt, db_dir_, &db_));
+
+ // CAUTION: This test creates a separate lock manager object (right, NOT
+ // the one that the TransactionDB is using!), and runs tests on it.
+ locker_.reset(new PointLockManager(
+ static_cast<PessimisticTransactionDB*>(db_), txn_opt));
+
+ wait_sync_point_name_ = "PointLockManager::AcquireWithTimeout:WaitingTxn";
+ }
+
+ void TearDown() override {
+ delete db_;
+ EXPECT_OK(DestroyDir(env_, db_dir_));
+ }
+
+ PessimisticTransaction* NewTxn(
+ TransactionOptions txn_opt = TransactionOptions()) {
+ Transaction* txn = db_->BeginTransaction(WriteOptions(), txn_opt);
+ return reinterpret_cast<PessimisticTransaction*>(txn);
+ }
+
+ protected:
+ Env* env_;
+ std::shared_ptr<LockManager> locker_;
+ const char* wait_sync_point_name_;
+ friend void PointLockManagerTestExternalSetup(PointLockManagerTest*);
+
+ private:
+ std::string db_dir_;
+ TransactionDB* db_;
+};
+
+using init_func_t = void (*)(PointLockManagerTest*);
+
+class AnyLockManagerTest : public PointLockManagerTest,
+ public testing::WithParamInterface<init_func_t> {
+ public:
+ void SetUp() override {
+ // If a custom setup function was provided, use it. Otherwise, use what we
+ // have inherited.
+ auto init_func = GetParam();
+ if (init_func)
+ (*init_func)(this);
+ else
+ PointLockManagerTest::SetUp();
+ }
+};
+
+TEST_P(AnyLockManagerTest, ReentrantExclusiveLock) {
+ // Tests that a txn can acquire exclusive lock on the same key repeatedly.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ auto txn = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, true));
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, true));
+
+ // Cleanup
+ locker_->UnLock(txn, 1, "k", env_);
+
+ delete txn;
+}
+
+TEST_P(AnyLockManagerTest, ReentrantSharedLock) {
+ // Tests that a txn can acquire shared lock on the same key repeatedly.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ auto txn = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, false));
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, false));
+
+ // Cleanup
+ locker_->UnLock(txn, 1, "k", env_);
+
+ delete txn;
+}
+
+TEST_P(AnyLockManagerTest, LockUpgrade) {
+ // Tests that a txn can upgrade from a shared lock to an exclusive lock.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ auto txn = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, false));
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, true));
+
+ // Cleanup
+ locker_->UnLock(txn, 1, "k", env_);
+ delete txn;
+}
+
+TEST_P(AnyLockManagerTest, LockDowngrade) {
+ // Tests that a txn can acquire a shared lock after acquiring an exclusive
+ // lock on the same key.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ auto txn = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, true));
+ ASSERT_OK(locker_->TryLock(txn, 1, "k", env_, false));
+
+ // Cleanup
+ locker_->UnLock(txn, 1, "k", env_);
+ delete txn;
+}
+
+TEST_P(AnyLockManagerTest, LockConflict) {
+ // Tests that lock conflicts lead to lock timeout.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ auto txn1 = NewTxn();
+ auto txn2 = NewTxn();
+
+ {
+ // exclusive-exclusive conflict.
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k1", env_, true));
+ auto s = locker_->TryLock(txn2, 1, "k1", env_, true);
+ ASSERT_TRUE(s.IsTimedOut());
+ }
+
+ {
+ // exclusive-shared conflict.
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k2", env_, true));
+ auto s = locker_->TryLock(txn2, 1, "k2", env_, false);
+ ASSERT_TRUE(s.IsTimedOut());
+ }
+
+ {
+ // shared-exclusive conflict.
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k2", env_, false));
+ auto s = locker_->TryLock(txn2, 1, "k2", env_, true);
+ ASSERT_TRUE(s.IsTimedOut());
+ }
+
+ // Cleanup
+ locker_->UnLock(txn1, 1, "k1", env_);
+ locker_->UnLock(txn1, 1, "k2", env_);
+
+ delete txn1;
+ delete txn2;
+}
+
+port::Thread BlockUntilWaitingTxn(const char* sync_point_name,
+ std::function<void()> f) {
+ std::atomic<bool> reached(false);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ sync_point_name, [&](void* /*arg*/) { reached.store(true); });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ port::Thread t(f);
+
+ while (!reached.load()) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(100));
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ return t;
+}
+
+TEST_P(AnyLockManagerTest, SharedLocks) {
+ // Tests that shared locks can be concurrently held by multiple transactions.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ auto txn1 = NewTxn();
+ auto txn2 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k", env_, false));
+ ASSERT_OK(locker_->TryLock(txn2, 1, "k", env_, false));
+
+ // Cleanup
+ locker_->UnLock(txn1, 1, "k", env_);
+ locker_->UnLock(txn2, 1, "k", env_);
+
+ delete txn1;
+ delete txn2;
+}
+
+TEST_P(AnyLockManagerTest, Deadlock) {
+ // Tests that deadlock can be detected.
+ // Deadlock scenario:
+ // txn1 exclusively locks k1, and wants to lock k2;
+ // txn2 exclusively locks k2, and wants to lock k1.
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+ TransactionOptions txn_opt;
+ txn_opt.deadlock_detect = true;
+ txn_opt.lock_timeout = 1000000;
+ auto txn1 = NewTxn(txn_opt);
+ auto txn2 = NewTxn(txn_opt);
+
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k1", env_, true));
+ ASSERT_OK(locker_->TryLock(txn2, 1, "k2", env_, true));
+
+ // txn1 tries to lock k2, will block forever.
+ port::Thread t = BlockUntilWaitingTxn(wait_sync_point_name_, [&]() {
+ // block because txn2 is holding a lock on k2.
+ locker_->TryLock(txn1, 1, "k2", env_, true);
+ });
+
+ auto s = locker_->TryLock(txn2, 1, "k1", env_, true);
+ ASSERT_TRUE(s.IsBusy());
+ ASSERT_EQ(s.subcode(), Status::SubCode::kDeadlock);
+
+ std::vector<DeadlockPath> deadlock_paths = locker_->GetDeadlockInfoBuffer();
+ ASSERT_EQ(deadlock_paths.size(), 1u);
+ ASSERT_FALSE(deadlock_paths[0].limit_exceeded);
+
+ std::vector<DeadlockInfo> deadlocks = deadlock_paths[0].path;
+ ASSERT_EQ(deadlocks.size(), 2u);
+
+ ASSERT_EQ(deadlocks[0].m_txn_id, txn1->GetID());
+ ASSERT_EQ(deadlocks[0].m_cf_id, 1u);
+ ASSERT_TRUE(deadlocks[0].m_exclusive);
+ ASSERT_EQ(deadlocks[0].m_waiting_key, "k2");
+
+ ASSERT_EQ(deadlocks[1].m_txn_id, txn2->GetID());
+ ASSERT_EQ(deadlocks[1].m_cf_id, 1u);
+ ASSERT_TRUE(deadlocks[1].m_exclusive);
+ ASSERT_EQ(deadlocks[1].m_waiting_key, "k1");
+
+ locker_->UnLock(txn2, 1, "k2", env_);
+ t.join();
+
+ // Cleanup
+ locker_->UnLock(txn1, 1, "k1", env_);
+ locker_->UnLock(txn1, 1, "k2", env_);
+ delete txn2;
+ delete txn1;
+}
+
+TEST_P(AnyLockManagerTest, GetWaitingTxns_MultipleTxns) {
+ MockColumnFamilyHandle cf(1);
+ locker_->AddColumnFamily(&cf);
+
+ auto txn1 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn1, 1, "k", env_, false));
+
+ auto txn2 = NewTxn();
+ ASSERT_OK(locker_->TryLock(txn2, 1, "k", env_, false));
+
+ auto txn3 = NewTxn();
+ txn3->SetLockTimeout(10000);
+ port::Thread t1 = BlockUntilWaitingTxn(wait_sync_point_name_, [&]() {
+ ASSERT_OK(locker_->TryLock(txn3, 1, "k", env_, true));
+ locker_->UnLock(txn3, 1, "k", env_);
+ });
+
+ // Ok, now txn3 is waiting for lock on "k", which is owned by two
+ // transactions. Check that GetWaitingTxns reports this correctly
+ uint32_t wait_cf_id;
+ std::string wait_key;
+ auto waiters = txn3->GetWaitingTxns(&wait_cf_id, &wait_key);
+
+ ASSERT_EQ(wait_cf_id, 1u);
+ ASSERT_EQ(wait_key, "k");
+ ASSERT_EQ(waiters.size(), 2);
+ bool waits_correct =
+ (waiters[0] == txn1->GetID() && waiters[1] == txn2->GetID()) ||
+ (waiters[1] == txn1->GetID() && waiters[0] == txn2->GetID());
+ ASSERT_EQ(waits_correct, true);
+
+ // Release locks so txn3 can proceed with execution
+ locker_->UnLock(txn1, 1, "k", env_);
+ locker_->UnLock(txn2, 1, "k", env_);
+
+ // Wait until txn3 finishes
+ t1.join();
+
+ delete txn1;
+ delete txn2;
+ delete txn3;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.cc b/src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.cc
new file mode 100644
index 000000000..6204a8f02
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.cc
@@ -0,0 +1,257 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#ifndef ROCKSDB_LITE
+
+#include "utilities/transactions/lock/point/point_lock_tracker.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+class TrackedKeysColumnFamilyIterator
+ : public LockTracker::ColumnFamilyIterator {
+ public:
+ explicit TrackedKeysColumnFamilyIterator(const TrackedKeys& keys)
+ : tracked_keys_(keys), it_(keys.begin()) {}
+
+ bool HasNext() const override { return it_ != tracked_keys_.end(); }
+
+ ColumnFamilyId Next() override { return (it_++)->first; }
+
+ private:
+ const TrackedKeys& tracked_keys_;
+ TrackedKeys::const_iterator it_;
+};
+
+class TrackedKeysIterator : public LockTracker::KeyIterator {
+ public:
+ TrackedKeysIterator(const TrackedKeys& keys, ColumnFamilyId id)
+ : key_infos_(keys.at(id)), it_(key_infos_.begin()) {}
+
+ bool HasNext() const override { return it_ != key_infos_.end(); }
+
+ const std::string& Next() override { return (it_++)->first; }
+
+ private:
+ const TrackedKeyInfos& key_infos_;
+ TrackedKeyInfos::const_iterator it_;
+};
+
+} // namespace
+
+void PointLockTracker::Track(const PointLockRequest& r) {
+ auto& keys = tracked_keys_[r.column_family_id];
+ auto result = keys.try_emplace(r.key, r.seq);
+ auto it = result.first;
+ if (!result.second && r.seq < it->second.seq) {
+ // Now tracking this key with an earlier sequence number
+ it->second.seq = r.seq;
+ }
+ // else we do not update the seq. The smaller the tracked seq, the stronger it
+ // the guarantee since it implies from the seq onward there has not been a
+ // concurrent update to the key. So we update the seq if it implies stronger
+ // guarantees, i.e., if it is smaller than the existing tracked seq.
+
+ if (r.read_only) {
+ it->second.num_reads++;
+ } else {
+ it->second.num_writes++;
+ }
+
+ it->second.exclusive = it->second.exclusive || r.exclusive;
+}
+
+UntrackStatus PointLockTracker::Untrack(const PointLockRequest& r) {
+ auto cf_keys = tracked_keys_.find(r.column_family_id);
+ if (cf_keys == tracked_keys_.end()) {
+ return UntrackStatus::NOT_TRACKED;
+ }
+
+ auto& keys = cf_keys->second;
+ auto it = keys.find(r.key);
+ if (it == keys.end()) {
+ return UntrackStatus::NOT_TRACKED;
+ }
+
+ bool untracked = false;
+ auto& info = it->second;
+ if (r.read_only) {
+ if (info.num_reads > 0) {
+ info.num_reads--;
+ untracked = true;
+ }
+ } else {
+ if (info.num_writes > 0) {
+ info.num_writes--;
+ untracked = true;
+ }
+ }
+
+ bool removed = false;
+ if (info.num_reads == 0 && info.num_writes == 0) {
+ keys.erase(it);
+ if (keys.empty()) {
+ tracked_keys_.erase(cf_keys);
+ }
+ removed = true;
+ }
+
+ if (removed) {
+ return UntrackStatus::REMOVED;
+ }
+ if (untracked) {
+ return UntrackStatus::UNTRACKED;
+ }
+ return UntrackStatus::NOT_TRACKED;
+}
+
+void PointLockTracker::Merge(const LockTracker& tracker) {
+ const PointLockTracker& t = static_cast<const PointLockTracker&>(tracker);
+ for (const auto& cf_keys : t.tracked_keys_) {
+ ColumnFamilyId cf = cf_keys.first;
+ const auto& keys = cf_keys.second;
+
+ auto current_cf_keys = tracked_keys_.find(cf);
+ if (current_cf_keys == tracked_keys_.end()) {
+ tracked_keys_.emplace(cf_keys);
+ } else {
+ auto& current_keys = current_cf_keys->second;
+ for (const auto& key_info : keys) {
+ const std::string& key = key_info.first;
+ const TrackedKeyInfo& info = key_info.second;
+ // If key was not previously tracked, just copy the whole struct over.
+ // Otherwise, some merging needs to occur.
+ auto current_info = current_keys.find(key);
+ if (current_info == current_keys.end()) {
+ current_keys.emplace(key_info);
+ } else {
+ current_info->second.Merge(info);
+ }
+ }
+ }
+ }
+}
+
+void PointLockTracker::Subtract(const LockTracker& tracker) {
+ const PointLockTracker& t = static_cast<const PointLockTracker&>(tracker);
+ for (const auto& cf_keys : t.tracked_keys_) {
+ ColumnFamilyId cf = cf_keys.first;
+ const auto& keys = cf_keys.second;
+
+ auto& current_keys = tracked_keys_.at(cf);
+ for (const auto& key_info : keys) {
+ const std::string& key = key_info.first;
+ const TrackedKeyInfo& info = key_info.second;
+ uint32_t num_reads = info.num_reads;
+ uint32_t num_writes = info.num_writes;
+
+ auto current_key_info = current_keys.find(key);
+ assert(current_key_info != current_keys.end());
+
+ // Decrement the total reads/writes of this key by the number of
+ // reads/writes done since the last SavePoint.
+ if (num_reads > 0) {
+ assert(current_key_info->second.num_reads >= num_reads);
+ current_key_info->second.num_reads -= num_reads;
+ }
+ if (num_writes > 0) {
+ assert(current_key_info->second.num_writes >= num_writes);
+ current_key_info->second.num_writes -= num_writes;
+ }
+ if (current_key_info->second.num_reads == 0 &&
+ current_key_info->second.num_writes == 0) {
+ current_keys.erase(current_key_info);
+ }
+ }
+ }
+}
+
+LockTracker* PointLockTracker::GetTrackedLocksSinceSavePoint(
+ const LockTracker& save_point_tracker) const {
+ // Examine the number of reads/writes performed on all keys written
+ // since the last SavePoint and compare to the total number of reads/writes
+ // for each key.
+ LockTracker* t = new PointLockTracker();
+ const PointLockTracker& save_point_t =
+ static_cast<const PointLockTracker&>(save_point_tracker);
+ for (const auto& cf_keys : save_point_t.tracked_keys_) {
+ ColumnFamilyId cf = cf_keys.first;
+ const auto& keys = cf_keys.second;
+
+ auto& current_keys = tracked_keys_.at(cf);
+ for (const auto& key_info : keys) {
+ const std::string& key = key_info.first;
+ const TrackedKeyInfo& info = key_info.second;
+ uint32_t num_reads = info.num_reads;
+ uint32_t num_writes = info.num_writes;
+
+ auto current_key_info = current_keys.find(key);
+ assert(current_key_info != current_keys.end());
+ assert(current_key_info->second.num_reads >= num_reads);
+ assert(current_key_info->second.num_writes >= num_writes);
+
+ if (current_key_info->second.num_reads == num_reads &&
+ current_key_info->second.num_writes == num_writes) {
+ // All the reads/writes to this key were done in the last savepoint.
+ PointLockRequest r;
+ r.column_family_id = cf;
+ r.key = key;
+ r.seq = info.seq;
+ r.read_only = (num_writes == 0);
+ r.exclusive = info.exclusive;
+ t->Track(r);
+ }
+ }
+ }
+ return t;
+}
+
+PointLockStatus PointLockTracker::GetPointLockStatus(
+ ColumnFamilyId column_family_id, const std::string& key) const {
+ assert(IsPointLockSupported());
+ PointLockStatus status;
+ auto it = tracked_keys_.find(column_family_id);
+ if (it == tracked_keys_.end()) {
+ return status;
+ }
+
+ const auto& keys = it->second;
+ auto key_it = keys.find(key);
+ if (key_it == keys.end()) {
+ return status;
+ }
+
+ const TrackedKeyInfo& key_info = key_it->second;
+ status.locked = true;
+ status.exclusive = key_info.exclusive;
+ status.seq = key_info.seq;
+ return status;
+}
+
+uint64_t PointLockTracker::GetNumPointLocks() const {
+ uint64_t num_keys = 0;
+ for (const auto& cf_keys : tracked_keys_) {
+ num_keys += cf_keys.second.size();
+ }
+ return num_keys;
+}
+
+LockTracker::ColumnFamilyIterator* PointLockTracker::GetColumnFamilyIterator()
+ const {
+ return new TrackedKeysColumnFamilyIterator(tracked_keys_);
+}
+
+LockTracker::KeyIterator* PointLockTracker::GetKeyIterator(
+ ColumnFamilyId column_family_id) const {
+ assert(tracked_keys_.find(column_family_id) != tracked_keys_.end());
+ return new TrackedKeysIterator(tracked_keys_, column_family_id);
+}
+
+void PointLockTracker::Clear() { tracked_keys_.clear(); }
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.h b/src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.h
new file mode 100644
index 000000000..daf6f9aa2
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.h
@@ -0,0 +1,99 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// This source code is licensed under both the GPLv2 (found in the
+// COPYING file in the root directory) and Apache 2.0 License
+// (found in the LICENSE.Apache file in the root directory).
+
+#pragma once
+#ifndef ROCKSDB_LITE
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include "utilities/transactions/lock/lock_tracker.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct TrackedKeyInfo {
+ // Earliest sequence number that is relevant to this transaction for this key
+ SequenceNumber seq;
+
+ uint32_t num_writes;
+ uint32_t num_reads;
+
+ bool exclusive;
+
+ explicit TrackedKeyInfo(SequenceNumber seq_no)
+ : seq(seq_no), num_writes(0), num_reads(0), exclusive(false) {}
+
+ void Merge(const TrackedKeyInfo& info) {
+ assert(seq <= info.seq);
+ num_reads += info.num_reads;
+ num_writes += info.num_writes;
+ exclusive = exclusive || info.exclusive;
+ }
+};
+
+using TrackedKeyInfos = std::unordered_map<std::string, TrackedKeyInfo>;
+
+using TrackedKeys = std::unordered_map<ColumnFamilyId, TrackedKeyInfos>;
+
+// Tracks point locks on single keys.
+class PointLockTracker : public LockTracker {
+ public:
+ PointLockTracker() = default;
+
+ PointLockTracker(const PointLockTracker&) = delete;
+ PointLockTracker& operator=(const PointLockTracker&) = delete;
+
+ bool IsPointLockSupported() const override { return true; }
+
+ bool IsRangeLockSupported() const override { return false; }
+
+ void Track(const PointLockRequest& lock_request) override;
+
+ UntrackStatus Untrack(const PointLockRequest& lock_request) override;
+
+ void Track(const RangeLockRequest& /*lock_request*/) override {}
+
+ UntrackStatus Untrack(const RangeLockRequest& /*lock_request*/) override {
+ return UntrackStatus::NOT_TRACKED;
+ }
+
+ void Merge(const LockTracker& tracker) override;
+
+ void Subtract(const LockTracker& tracker) override;
+
+ void Clear() override;
+
+ virtual LockTracker* GetTrackedLocksSinceSavePoint(
+ const LockTracker& save_point_tracker) const override;
+
+ PointLockStatus GetPointLockStatus(ColumnFamilyId column_family_id,
+ const std::string& key) const override;
+
+ uint64_t GetNumPointLocks() const override;
+
+ ColumnFamilyIterator* GetColumnFamilyIterator() const override;
+
+ KeyIterator* GetKeyIterator(ColumnFamilyId column_family_id) const override;
+
+ private:
+ TrackedKeys tracked_keys_;
+};
+
+class PointLockTrackerFactory : public LockTrackerFactory {
+ public:
+ static const PointLockTrackerFactory& Get() {
+ static const PointLockTrackerFactory instance;
+ return instance;
+ }
+
+ LockTracker* Create() const override { return new PointLockTracker(); }
+
+ private:
+ PointLockTrackerFactory() {}
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_lock_manager.h b/src/rocksdb/utilities/transactions/lock/range/range_lock_manager.h
new file mode 100644
index 000000000..01899542e
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_lock_manager.h
@@ -0,0 +1,36 @@
+// Copyright (c) Meta Platforms, Inc. and affiliates.
+//
+// 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).
+
+//
+// Generic definitions for a Range-based Lock Manager
+//
+#pragma once
+#ifndef ROCKSDB_LITE
+
+#include "utilities/transactions/lock/lock_manager.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+/*
+ A base class for all Range-based lock managers
+
+ See also class RangeLockManagerHandle in
+ include/rocksdb/utilities/transaction_db.h
+*/
+class RangeLockManagerBase : public LockManager {
+ public:
+ // Geting a point lock is reduced to getting a range lock on a single-point
+ // range
+ using LockManager::TryLock;
+ Status TryLock(PessimisticTransaction* txn, ColumnFamilyId column_family_id,
+ const std::string& key, Env* env, bool exclusive) override {
+ Endpoint endp(key.data(), key.size(), false);
+ return TryLock(txn, column_family_id, endp, endp, env, exclusive);
+ }
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_locking_test.cc b/src/rocksdb/utilities/transactions/lock/range/range_locking_test.cc
new file mode 100644
index 000000000..bce66c1f3
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_locking_test.cc
@@ -0,0 +1,459 @@
+// 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");
+
+ EXPECT_OK(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.
+ EXPECT_OK(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, UpgradeLockAndGetConflict) {
+ WriteOptions write_options;
+ TransactionOptions txn_options;
+ auto cf = db->DefaultColumnFamily();
+ Status s;
+ std::string value;
+ txn_options.lock_timeout = 10;
+
+ Transaction* txn0 = db->BeginTransaction(write_options, txn_options);
+ Transaction* txn1 = db->BeginTransaction(write_options, txn_options);
+
+ // Get the shared lock in txn0
+ s = txn0->GetForUpdate(ReadOptions(), cf, Slice("a"), &value,
+ false /*exclusive*/);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Get the shared lock on the same key in txn1
+ s = txn1->GetForUpdate(ReadOptions(), cf, Slice("a"), &value,
+ false /*exclusive*/);
+ ASSERT_TRUE(s.IsNotFound());
+
+ // Now, try getting an exclusive lock that overlaps with the above
+ s = txn0->GetRangeLock(cf, Endpoint("a"), Endpoint("b"));
+ ASSERT_TRUE(s.IsTimedOut());
+
+ txn0->Rollback();
+ txn1->Rollback();
+
+ delete txn0;
+ delete txn1;
+}
+
+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
+#else
+#define SKIP_LOCK_ESCALATION_TEST 1
+#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++) {
+ std::ostringstream buf;
+ buf << std::setw(8) << std::setfill('0') << i;
+ std::string buf_str = buf.str();
+ ASSERT_OK(txn->GetRangeLock(cf, Endpoint(buf_str), Endpoint(buf_str)));
+ }
+ counters = range_lock_mgr->GetStatus();
+ ASSERT_GT(counters.escalation_count, 0);
+ ASSERT_LE(counters.current_lock_memory, 2000);
+
+ delete txn;
+}
+
+// An escalation barrier function. Allow escalation iff the first two bytes are
+// identical.
+static bool escalation_barrier(const Endpoint& a, const Endpoint& b) {
+ assert(a.slice.size() > 2);
+ assert(b.slice.size() > 2);
+ if (memcmp(a.slice.data(), b.slice.data(), 2)) {
+ return true; // This is a barrier
+ } else {
+ return false; // No barrier
+ }
+}
+
+TEST_F(RangeLockingTest, LockEscalationBarrier) {
+ auto cf = db->DefaultColumnFamily();
+
+ auto counters = range_lock_mgr->GetStatus();
+
+ // Initially not using any lock memory
+ ASSERT_EQ(counters.escalation_count, 0);
+
+ range_lock_mgr->SetMaxLockMemory(8000);
+ range_lock_mgr->SetEscalationBarrierFunc(escalation_barrier);
+
+ // Insert enough locks to cause lock escalations to happen
+ auto txn = NewTxn();
+ const int N = 2000;
+ for (int i = 0; i < N; i++) {
+ std::ostringstream buf;
+ buf << std::setw(4) << std::setfill('0') << i;
+ std::string buf_str = buf.str();
+ ASSERT_OK(txn->GetRangeLock(cf, Endpoint(buf_str), Endpoint(buf_str)));
+ }
+ counters = range_lock_mgr->GetStatus();
+ ASSERT_GT(counters.escalation_count, 0);
+
+ // Check that lock escalation was not performed across escalation barriers:
+ // Use another txn to acquire locks near the barriers.
+ auto txn2 = NewTxn();
+ range_lock_mgr->SetMaxLockMemory(500000);
+ for (int i = 100; i < N; i += 100) {
+ std::ostringstream buf;
+ buf << std::setw(4) << std::setfill('0') << i - 1 << "-a";
+ std::string buf_str = buf.str();
+ // Check that we CAN get a lock near the escalation barrier
+ ASSERT_OK(txn2->GetRangeLock(cf, Endpoint(buf_str), Endpoint(buf_str)));
+ }
+
+ txn->Rollback();
+ txn2->Rollback();
+ delete txn;
+ delete txn2;
+}
+
+#endif
+
+TEST_F(RangeLockingTest, LockWaitCount) {
+ TransactionOptions txn_options;
+ auto cf = db->DefaultColumnFamily();
+ txn_options.lock_timeout = 50;
+ Transaction* txn0 = db->BeginTransaction(WriteOptions(), txn_options);
+ Transaction* txn1 = db->BeginTransaction(WriteOptions(), txn_options);
+
+ // Get a range lock
+ ASSERT_OK(txn0->GetRangeLock(cf, Endpoint("a"), Endpoint("c")));
+
+ uint64_t lock_waits1 = range_lock_mgr->GetStatus().lock_wait_count;
+ // Attempt to get a conflicting lock
+ auto s = txn1->GetRangeLock(cf, Endpoint("b"), Endpoint("z"));
+ ASSERT_TRUE(s.IsTimedOut());
+
+ // Check that the counter was incremented
+ uint64_t lock_waits2 = range_lock_mgr->GetStatus().lock_wait_count;
+ ASSERT_EQ(lock_waits1 + 1, lock_waits2);
+
+ txn0->Rollback();
+ txn1->Rollback();
+
+ delete txn0;
+ delete txn1;
+}
+
+TEST_F(RangeLockingTest, LockWaiteeAccess) {
+ TransactionOptions txn_options;
+ auto cf = db->DefaultColumnFamily();
+ txn_options.lock_timeout = 60;
+ Transaction* txn0 = db->BeginTransaction(WriteOptions(), txn_options);
+ Transaction* txn1 = db->BeginTransaction(WriteOptions(), txn_options);
+
+ // Get a range lock
+ ASSERT_OK(txn0->GetRangeLock(cf, Endpoint("a"), Endpoint("c")));
+
+ std::atomic<bool> reached(false);
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "RangeTreeLockManager::TryRangeLock:EnterWaitingTxn", [&](void* /*arg*/) {
+ reached.store(true);
+ std::this_thread::sleep_for(std::chrono::milliseconds(2000));
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+
+ port::Thread t([&]() {
+ // Attempt to get a conflicting lock
+ auto s = txn1->GetRangeLock(cf, Endpoint("b"), Endpoint("z"));
+ ASSERT_TRUE(s.ok());
+ txn1->Rollback();
+ });
+
+ while (!reached.load()) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(100));
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
+
+ // Release locks and free the transaction
+ txn0->Rollback();
+ delete txn0;
+
+ t.join();
+
+ delete txn1;
+}
+
+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
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.AGPLv3 b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.AGPLv3
new file mode 100644
index 000000000..dba13ed2d
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.AGPLv3
@@ -0,0 +1,661 @@
+ GNU AFFERO GENERAL PUBLIC LICENSE
+ Version 3, 19 November 2007
+
+ Copyright (C) 2007 Free Software Foundation, Inc. <http://fsf.org/>
+ Everyone is permitted to copy and distribute verbatim copies
+ of this license document, but changing it is not allowed.
+
+ Preamble
+
+ The GNU Affero General Public License is a free, copyleft license for
+software and other kinds of works, specifically designed to ensure
+cooperation with the community in the case of network server software.
+
+ The licenses for most software and other practical works are designed
+to take away your freedom to share and change the works. By contrast,
+our General Public Licenses are intended to guarantee your freedom to
+share and change all versions of a program--to make sure it remains free
+software for all its users.
+
+ When we speak of free software, we are referring to freedom, not
+price. Our General Public Licenses are designed to make sure that you
+have the freedom to distribute copies of free software (and charge for
+them if you wish), that you receive source code or can get it if you
+want it, that you can change the software or use pieces of it in new
+free programs, and that you know you can do these things.
+
+ Developers that use our General Public Licenses protect your rights
+with two steps: (1) assert copyright on the software, and (2) offer
+you this License which gives you legal permission to copy, distribute
+and/or modify the software.
+
+ A secondary benefit of defending all users' freedom is that
+improvements made in alternate versions of the program, if they
+receive widespread use, become available for other developers to
+incorporate. Many developers of free software are heartened and
+encouraged by the resulting cooperation. However, in the case of
+software used on network servers, this result may fail to come about.
+The GNU General Public License permits making a modified version and
+letting the public access it on a server without ever releasing its
+source code to the public.
+
+ The GNU Affero General Public License is designed specifically to
+ensure that, in such cases, the modified source code becomes available
+to the community. It requires the operator of a network server to
+provide the source code of the modified version running there to the
+users of that server. Therefore, public use of a modified version, on
+a publicly accessible server, gives the public access to the source
+code of the modified version.
+
+ An older license, called the Affero General Public License and
+published by Affero, was designed to accomplish similar goals. This is
+a different license, not a version of the Affero GPL, but Affero has
+released a new version of the Affero GPL which permits relicensing under
+this license.
+
+ The precise terms and conditions for copying, distribution and
+modification follow.
+
+ TERMS AND CONDITIONS
+
+ 0. Definitions.
+
+ "This License" refers to version 3 of the GNU Affero General Public License.
+
+ "Copyright" also means copyright-like laws that apply to other kinds of
+works, such as semiconductor masks.
+
+ "The Program" refers to any copyrightable work licensed under this
+License. Each licensee is addressed as "you". "Licensees" and
+"recipients" may be individuals or organizations.
+
+ To "modify" a work means to copy from or adapt all or part of the work
+in a fashion requiring copyright permission, other than the making of an
+exact copy. The resulting work is called a "modified version" of the
+earlier work or a work "based on" the earlier work.
+
+ A "covered work" means either the unmodified Program or a work based
+on the Program.
+
+ To "propagate" a work means to do anything with it that, without
+permission, would make you directly or secondarily liable for
+infringement under applicable copyright law, except executing it on a
+computer or modifying a private copy. Propagation includes copying,
+distribution (with or without modification), making available to the
+public, and in some countries other activities as well.
+
+ To "convey" a work means any kind of propagation that enables other
+parties to make or receive copies. Mere interaction with a user through
+a computer network, with no transfer of a copy, is not conveying.
+
+ An interactive user interface displays "Appropriate Legal Notices"
+to the extent that it includes a convenient and prominently visible
+feature that (1) displays an appropriate copyright notice, and (2)
+tells the user that there is no warranty for the work (except to the
+extent that warranties are provided), that licensees may convey the
+work under this License, and how to view a copy of this License. If
+the interface presents a list of user commands or options, such as a
+menu, a prominent item in the list meets this criterion.
+
+ 1. Source Code.
+
+ The "source code" for a work means the preferred form of the work
+for making modifications to it. "Object code" means any non-source
+form of a work.
+
+ A "Standard Interface" means an interface that either is an official
+standard defined by a recognized standards body, or, in the case of
+interfaces specified for a particular programming language, one that
+is widely used among developers working in that language.
+
+ The "System Libraries" of an executable work include anything, other
+than the work as a whole, that (a) is included in the normal form of
+packaging a Major Component, but which is not part of that Major
+Component, and (b) serves only to enable use of the work with that
+Major Component, or to implement a Standard Interface for which an
+implementation is available to the public in source code form. A
+"Major Component", in this context, means a major essential component
+(kernel, window system, and so on) of the specific operating system
+(if any) on which the executable work runs, or a compiler used to
+produce the work, or an object code interpreter used to run it.
+
+ The "Corresponding Source" for a work in object code form means all
+the source code needed to generate, install, and (for an executable
+work) run the object code and to modify the work, including scripts to
+control those activities. However, it does not include the work's
+System Libraries, or general-purpose tools or generally available free
+programs which are used unmodified in performing those activities but
+which are not part of the work. For example, Corresponding Source
+includes interface definition files associated with source files for
+the work, and the source code for shared libraries and dynamically
+linked subprograms that the work is specifically designed to require,
+such as by intimate data communication or control flow between those
+subprograms and other parts of the work.
+
+ The Corresponding Source need not include anything that users
+can regenerate automatically from other parts of the Corresponding
+Source.
+
+ The Corresponding Source for a work in source code form is that
+same work.
+
+ 2. Basic Permissions.
+
+ All rights granted under this License are granted for the term of
+copyright on the Program, and are irrevocable provided the stated
+conditions are met. This License explicitly affirms your unlimited
+permission to run the unmodified Program. The output from running a
+covered work is covered by this License only if the output, given its
+content, constitutes a covered work. This License acknowledges your
+rights of fair use or other equivalent, as provided by copyright law.
+
+ You may make, run and propagate covered works that you do not
+convey, without conditions so long as your license otherwise remains
+in force. You may convey covered works to others for the sole purpose
+of having them make modifications exclusively for you, or provide you
+with facilities for running those works, provided that you comply with
+the terms of this License in conveying all material for which you do
+not control copyright. Those thus making or running the covered works
+for you must do so exclusively on your behalf, under your direction
+and control, on terms that prohibit them from making any copies of
+your copyrighted material outside their relationship with you.
+
+ Conveying under any other circumstances is permitted solely under
+the conditions stated below. Sublicensing is not allowed; section 10
+makes it unnecessary.
+
+ 3. Protecting Users' Legal Rights From Anti-Circumvention Law.
+
+ No covered work shall be deemed part of an effective technological
+measure under any applicable law fulfilling obligations under article
+11 of the WIPO copyright treaty adopted on 20 December 1996, or
+similar laws prohibiting or restricting circumvention of such
+measures.
+
+ When you convey a covered work, you waive any legal power to forbid
+circumvention of technological measures to the extent such circumvention
+is effected by exercising rights under this License with respect to
+the covered work, and you disclaim any intention to limit operation or
+modification of the work as a means of enforcing, against the work's
+users, your or third parties' legal rights to forbid circumvention of
+technological measures.
+
+ 4. Conveying Verbatim Copies.
+
+ You may convey verbatim copies of the Program's source code as you
+receive it, in any medium, provided that you conspicuously and
+appropriately publish on each copy an appropriate copyright notice;
+keep intact all notices stating that this License and any
+non-permissive terms added in accord with section 7 apply to the code;
+keep intact all notices of the absence of any warranty; and give all
+recipients a copy of this License along with the Program.
+
+ You may charge any price or no price for each copy that you convey,
+and you may offer support or warranty protection for a fee.
+
+ 5. Conveying Modified Source Versions.
+
+ You may convey a work based on the Program, or the modifications to
+produce it from the Program, in the form of source code under the
+terms of section 4, provided that you also meet all of these conditions:
+
+ a) The work must carry prominent notices stating that you modified
+ it, and giving a relevant date.
+
+ b) The work must carry prominent notices stating that it is
+ released under this License and any conditions added under section
+ 7. This requirement modifies the requirement in section 4 to
+ "keep intact all notices".
+
+ c) You must license the entire work, as a whole, under this
+ License to anyone who comes into possession of a copy. This
+ License will therefore apply, along with any applicable section 7
+ additional terms, to the whole of the work, and all its parts,
+ regardless of how they are packaged. This License gives no
+ permission to license the work in any other way, but it does not
+ invalidate such permission if you have separately received it.
+
+ d) If the work has interactive user interfaces, each must display
+ Appropriate Legal Notices; however, if the Program has interactive
+ interfaces that do not display Appropriate Legal Notices, your
+ work need not make them do so.
+
+ A compilation of a covered work with other separate and independent
+works, which are not by their nature extensions of the covered work,
+and which are not combined with it such as to form a larger program,
+in or on a volume of a storage or distribution medium, is called an
+"aggregate" if the compilation and its resulting copyright are not
+used to limit the access or legal rights of the compilation's users
+beyond what the individual works permit. Inclusion of a covered work
+in an aggregate does not cause this License to apply to the other
+parts of the aggregate.
+
+ 6. Conveying Non-Source Forms.
+
+ You may convey a covered work in object code form under the terms
+of sections 4 and 5, provided that you also convey the
+machine-readable Corresponding Source under the terms of this License,
+in one of these ways:
+
+ a) Convey the object code in, or embodied in, a physical product
+ (including a physical distribution medium), accompanied by the
+ Corresponding Source fixed on a durable physical medium
+ customarily used for software interchange.
+
+ b) Convey the object code in, or embodied in, a physical product
+ (including a physical distribution medium), accompanied by a
+ written offer, valid for at least three years and valid for as
+ long as you offer spare parts or customer support for that product
+ model, to give anyone who possesses the object code either (1) a
+ copy of the Corresponding Source for all the software in the
+ product that is covered by this License, on a durable physical
+ medium customarily used for software interchange, for a price no
+ more than your reasonable cost of physically performing this
+ conveying of source, or (2) access to copy the
+ Corresponding Source from a network server at no charge.
+
+ c) Convey individual copies of the object code with a copy of the
+ written offer to provide the Corresponding Source. This
+ alternative is allowed only occasionally and noncommercially, and
+ only if you received the object code with such an offer, in accord
+ with subsection 6b.
+
+ d) Convey the object code by offering access from a designated
+ place (gratis or for a charge), and offer equivalent access to the
+ Corresponding Source in the same way through the same place at no
+ further charge. You need not require recipients to copy the
+ Corresponding Source along with the object code. If the place to
+ copy the object code is a network server, the Corresponding Source
+ may be on a different server (operated by you or a third party)
+ that supports equivalent copying facilities, provided you maintain
+ clear directions next to the object code saying where to find the
+ Corresponding Source. Regardless of what server hosts the
+ Corresponding Source, you remain obligated to ensure that it is
+ available for as long as needed to satisfy these requirements.
+
+ e) Convey the object code using peer-to-peer transmission, provided
+ you inform other peers where the object code and Corresponding
+ Source of the work are being offered to the general public at no
+ charge under subsection 6d.
+
+ A separable portion of the object code, whose source code is excluded
+from the Corresponding Source as a System Library, need not be
+included in conveying the object code work.
+
+ A "User Product" is either (1) a "consumer product", which means any
+tangible personal property which is normally used for personal, family,
+or household purposes, or (2) anything designed or sold for incorporation
+into a dwelling. In determining whether a product is a consumer product,
+doubtful cases shall be resolved in favor of coverage. For a particular
+product received by a particular user, "normally used" refers to a
+typical or common use of that class of product, regardless of the status
+of the particular user or of the way in which the particular user
+actually uses, or expects or is expected to use, the product. A product
+is a consumer product regardless of whether the product has substantial
+commercial, industrial or non-consumer uses, unless such uses represent
+the only significant mode of use of the product.
+
+ "Installation Information" for a User Product means any methods,
+procedures, authorization keys, or other information required to install
+and execute modified versions of a covered work in that User Product from
+a modified version of its Corresponding Source. The information must
+suffice to ensure that the continued functioning of the modified object
+code is in no case prevented or interfered with solely because
+modification has been made.
+
+ If you convey an object code work under this section in, or with, or
+specifically for use in, a User Product, and the conveying occurs as
+part of a transaction in which the right of possession and use of the
+User Product is transferred to the recipient in perpetuity or for a
+fixed term (regardless of how the transaction is characterized), the
+Corresponding Source conveyed under this section must be accompanied
+by the Installation Information. But this requirement does not apply
+if neither you nor any third party retains the ability to install
+modified object code on the User Product (for example, the work has
+been installed in ROM).
+
+ The requirement to provide Installation Information does not include a
+requirement to continue to provide support service, warranty, or updates
+for a work that has been modified or installed by the recipient, or for
+the User Product in which it has been modified or installed. Access to a
+network may be denied when the modification itself materially and
+adversely affects the operation of the network or violates the rules and
+protocols for communication across the network.
+
+ Corresponding Source conveyed, and Installation Information provided,
+in accord with this section must be in a format that is publicly
+documented (and with an implementation available to the public in
+source code form), and must require no special password or key for
+unpacking, reading or copying.
+
+ 7. Additional Terms.
+
+ "Additional permissions" are terms that supplement the terms of this
+License by making exceptions from one or more of its conditions.
+Additional permissions that are applicable to the entire Program shall
+be treated as though they were included in this License, to the extent
+that they are valid under applicable law. If additional permissions
+apply only to part of the Program, that part may be used separately
+under those permissions, but the entire Program remains governed by
+this License without regard to the additional permissions.
+
+ When you convey a copy of a covered work, you may at your option
+remove any additional permissions from that copy, or from any part of
+it. (Additional permissions may be written to require their own
+removal in certain cases when you modify the work.) You may place
+additional permissions on material, added by you to a covered work,
+for which you have or can give appropriate copyright permission.
+
+ Notwithstanding any other provision of this License, for material you
+add to a covered work, you may (if authorized by the copyright holders of
+that material) supplement the terms of this License with terms:
+
+ a) Disclaiming warranty or limiting liability differently from the
+ terms of sections 15 and 16 of this License; or
+
+ b) Requiring preservation of specified reasonable legal notices or
+ author attributions in that material or in the Appropriate Legal
+ Notices displayed by works containing it; or
+
+ c) Prohibiting misrepresentation of the origin of that material, or
+ requiring that modified versions of such material be marked in
+ reasonable ways as different from the original version; or
+
+ d) Limiting the use for publicity purposes of names of licensors or
+ authors of the material; or
+
+ e) Declining to grant rights under trademark law for use of some
+ trade names, trademarks, or service marks; or
+
+ f) Requiring indemnification of licensors and authors of that
+ material by anyone who conveys the material (or modified versions of
+ it) with contractual assumptions of liability to the recipient, for
+ any liability that these contractual assumptions directly impose on
+ those licensors and authors.
+
+ All other non-permissive additional terms are considered "further
+restrictions" within the meaning of section 10. If the Program as you
+received it, or any part of it, contains a notice stating that it is
+governed by this License along with a term that is a further
+restriction, you may remove that term. If a license document contains
+a further restriction but permits relicensing or conveying under this
+License, you may add to a covered work material governed by the terms
+of that license document, provided that the further restriction does
+not survive such relicensing or conveying.
+
+ If you add terms to a covered work in accord with this section, you
+must place, in the relevant source files, a statement of the
+additional terms that apply to those files, or a notice indicating
+where to find the applicable terms.
+
+ Additional terms, permissive or non-permissive, may be stated in the
+form of a separately written license, or stated as exceptions;
+the above requirements apply either way.
+
+ 8. Termination.
+
+ You may not propagate or modify a covered work except as expressly
+provided under this License. Any attempt otherwise to propagate or
+modify it is void, and will automatically terminate your rights under
+this License (including any patent licenses granted under the third
+paragraph of section 11).
+
+ However, if you cease all violation of this License, then your
+license from a particular copyright holder is reinstated (a)
+provisionally, unless and until the copyright holder explicitly and
+finally terminates your license, and (b) permanently, if the copyright
+holder fails to notify you of the violation by some reasonable means
+prior to 60 days after the cessation.
+
+ Moreover, your license from a particular copyright holder is
+reinstated permanently if the copyright holder notifies you of the
+violation by some reasonable means, this is the first time you have
+received notice of violation of this License (for any work) from that
+copyright holder, and you cure the violation prior to 30 days after
+your receipt of the notice.
+
+ Termination of your rights under this section does not terminate the
+licenses of parties who have received copies or rights from you under
+this License. If your rights have been terminated and not permanently
+reinstated, you do not qualify to receive new licenses for the same
+material under section 10.
+
+ 9. Acceptance Not Required for Having Copies.
+
+ You are not required to accept this License in order to receive or
+run a copy of the Program. Ancillary propagation of a covered work
+occurring solely as a consequence of using peer-to-peer transmission
+to receive a copy likewise does not require acceptance. However,
+nothing other than this License grants you permission to propagate or
+modify any covered work. These actions infringe copyright if you do
+not accept this License. Therefore, by modifying or propagating a
+covered work, you indicate your acceptance of this License to do so.
+
+ 10. Automatic Licensing of Downstream Recipients.
+
+ Each time you convey a covered work, the recipient automatically
+receives a license from the original licensors, to run, modify and
+propagate that work, subject to this License. You are not responsible
+for enforcing compliance by third parties with this License.
+
+ An "entity transaction" is a transaction transferring control of an
+organization, or substantially all assets of one, or subdividing an
+organization, or merging organizations. If propagation of a covered
+work results from an entity transaction, each party to that
+transaction who receives a copy of the work also receives whatever
+licenses to the work the party's predecessor in interest had or could
+give under the previous paragraph, plus a right to possession of the
+Corresponding Source of the work from the predecessor in interest, if
+the predecessor has it or can get it with reasonable efforts.
+
+ You may not impose any further restrictions on the exercise of the
+rights granted or affirmed under this License. For example, you may
+not impose a license fee, royalty, or other charge for exercise of
+rights granted under this License, and you may not initiate litigation
+(including a cross-claim or counterclaim in a lawsuit) alleging that
+any patent claim is infringed by making, using, selling, offering for
+sale, or importing the Program or any portion of it.
+
+ 11. Patents.
+
+ A "contributor" is a copyright holder who authorizes use under this
+License of the Program or a work on which the Program is based. The
+work thus licensed is called the contributor's "contributor version".
+
+ A contributor's "essential patent claims" are all patent claims
+owned or controlled by the contributor, whether already acquired or
+hereafter acquired, that would be infringed by some manner, permitted
+by this License, of making, using, or selling its contributor version,
+but do not include claims that would be infringed only as a
+consequence of further modification of the contributor version. For
+purposes of this definition, "control" includes the right to grant
+patent sublicenses in a manner consistent with the requirements of
+this License.
+
+ Each contributor grants you a non-exclusive, worldwide, royalty-free
+patent license under the contributor's essential patent claims, to
+make, use, sell, offer for sale, import and otherwise run, modify and
+propagate the contents of its contributor version.
+
+ In the following three paragraphs, a "patent license" is any express
+agreement or commitment, however denominated, not to enforce a patent
+(such as an express permission to practice a patent or covenant not to
+sue for patent infringement). To "grant" such a patent license to a
+party means to make such an agreement or commitment not to enforce a
+patent against the party.
+
+ If you convey a covered work, knowingly relying on a patent license,
+and the Corresponding Source of the work is not available for anyone
+to copy, free of charge and under the terms of this License, through a
+publicly available network server or other readily accessible means,
+then you must either (1) cause the Corresponding Source to be so
+available, or (2) arrange to deprive yourself of the benefit of the
+patent license for this particular work, or (3) arrange, in a manner
+consistent with the requirements of this License, to extend the patent
+license to downstream recipients. "Knowingly relying" means you have
+actual knowledge that, but for the patent license, your conveying the
+covered work in a country, or your recipient's use of the covered work
+in a country, would infringe one or more identifiable patents in that
+country that you have reason to believe are valid.
+
+ If, pursuant to or in connection with a single transaction or
+arrangement, you convey, or propagate by procuring conveyance of, a
+covered work, and grant a patent license to some of the parties
+receiving the covered work authorizing them to use, propagate, modify
+or convey a specific copy of the covered work, then the patent license
+you grant is automatically extended to all recipients of the covered
+work and works based on it.
+
+ A patent license is "discriminatory" if it does not include within
+the scope of its coverage, prohibits the exercise of, or is
+conditioned on the non-exercise of one or more of the rights that are
+specifically granted under this License. You may not convey a covered
+work if you are a party to an arrangement with a third party that is
+in the business of distributing software, under which you make payment
+to the third party based on the extent of your activity of conveying
+the work, and under which the third party grants, to any of the
+parties who would receive the covered work from you, a discriminatory
+patent license (a) in connection with copies of the covered work
+conveyed by you (or copies made from those copies), or (b) primarily
+for and in connection with specific products or compilations that
+contain the covered work, unless you entered into that arrangement,
+or that patent license was granted, prior to 28 March 2007.
+
+ Nothing in this License shall be construed as excluding or limiting
+any implied license or other defenses to infringement that may
+otherwise be available to you under applicable patent law.
+
+ 12. No Surrender of Others' Freedom.
+
+ If conditions are imposed on you (whether by court order, agreement or
+otherwise) that contradict the conditions of this License, they do not
+excuse you from the conditions of this License. If you cannot convey a
+covered work so as to satisfy simultaneously your obligations under this
+License and any other pertinent obligations, then as a consequence you may
+not convey it at all. For example, if you agree to terms that obligate you
+to collect a royalty for further conveying from those to whom you convey
+the Program, the only way you could satisfy both those terms and this
+License would be to refrain entirely from conveying the Program.
+
+ 13. Remote Network Interaction; Use with the GNU General Public License.
+
+ Notwithstanding any other provision of this License, if you modify the
+Program, your modified version must prominently offer all users
+interacting with it remotely through a computer network (if your version
+supports such interaction) an opportunity to receive the Corresponding
+Source of your version by providing access to the Corresponding Source
+from a network server at no charge, through some standard or customary
+means of facilitating copying of software. This Corresponding Source
+shall include the Corresponding Source for any work covered by version 3
+of the GNU General Public License that is incorporated pursuant to the
+following paragraph.
+
+ Notwithstanding any other provision of this License, you have
+permission to link or combine any covered work with a work licensed
+under version 3 of the GNU General Public License into a single
+combined work, and to convey the resulting work. The terms of this
+License will continue to apply to the part which is the covered work,
+but the work with which it is combined will remain governed by version
+3 of the GNU General Public License.
+
+ 14. Revised Versions of this License.
+
+ The Free Software Foundation may publish revised and/or new versions of
+the GNU Affero General Public License from time to time. Such new versions
+will be similar in spirit to the present version, but may differ in detail to
+address new problems or concerns.
+
+ Each version is given a distinguishing version number. If the
+Program specifies that a certain numbered version of the GNU Affero General
+Public License "or any later version" applies to it, you have the
+option of following the terms and conditions either of that numbered
+version or of any later version published by the Free Software
+Foundation. If the Program does not specify a version number of the
+GNU Affero General Public License, you may choose any version ever published
+by the Free Software Foundation.
+
+ If the Program specifies that a proxy can decide which future
+versions of the GNU Affero General Public License can be used, that proxy's
+public statement of acceptance of a version permanently authorizes you
+to choose that version for the Program.
+
+ Later license versions may give you additional or different
+permissions. However, no additional obligations are imposed on any
+author or copyright holder as a result of your choosing to follow a
+later version.
+
+ 15. Disclaimer of Warranty.
+
+ THERE IS NO WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY
+APPLICABLE LAW. EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT
+HOLDERS AND/OR OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY
+OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO,
+THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM
+IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF
+ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+ 16. Limitation of Liability.
+
+ IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING
+WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MODIFIES AND/OR CONVEYS
+THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY
+GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE
+USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF
+DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD
+PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS),
+EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF
+SUCH DAMAGES.
+
+ 17. Interpretation of Sections 15 and 16.
+
+ If the disclaimer of warranty and limitation of liability provided
+above cannot be given local legal effect according to their terms,
+reviewing courts shall apply local law that most closely approximates
+an absolute waiver of all civil liability in connection with the
+Program, unless a warranty or assumption of liability accompanies a
+copy of the Program in return for a fee.
+
+ END OF TERMS AND CONDITIONS
+
+ How to Apply These Terms to Your New Programs
+
+ If you develop a new program, and you want it to be of the greatest
+possible use to the public, the best way to achieve this is to make it
+free software which everyone can redistribute and change under these terms.
+
+ To do so, attach the following notices to the program. It is safest
+to attach them to the start of each source file to most effectively
+state the exclusion of warranty; and each file should have at least
+the "copyright" line and a pointer to where the full notice is found.
+
+ <one line to give the program's name and a brief idea of what it does.>
+ Copyright (C) <year> <name of author>
+
+ This program is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License as published by
+ the Free Software Foundation, either version 3 of the License, or
+ (at your option) any later version.
+
+ This program is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with this program. If not, see <http://www.gnu.org/licenses/>.
+
+Also add information on how to contact you by electronic and paper mail.
+
+ If your software can interact with users remotely through a computer
+network, you should also make sure that it provides a way for users to
+get its source. For example, if your program is a web application, its
+interface could display a "Source" link that leads users to an archive
+of the code. There are many ways you could offer source, and different
+solutions will be better for different programs; see section 13 for the
+specific requirements.
+
+ You should also get your employer (if you work as a programmer) or school,
+if any, to sign a "copyright disclaimer" for the program, if necessary.
+For more information on this, and how to apply and follow the GNU AGPL, see
+<http://www.gnu.org/licenses/>.
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.APACHEv2 b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.APACHEv2
new file mode 100644
index 000000000..ecbfc770f
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.APACHEv2
@@ -0,0 +1,174 @@
+Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.GPLv2 b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.GPLv2
new file mode 100644
index 000000000..d511905c1
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.GPLv2
@@ -0,0 +1,339 @@
+ GNU GENERAL PUBLIC LICENSE
+ Version 2, June 1991
+
+ Copyright (C) 1989, 1991 Free Software Foundation, Inc.,
+ 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
+ Everyone is permitted to copy and distribute verbatim copies
+ of this license document, but changing it is not allowed.
+
+ Preamble
+
+ The licenses for most software are designed to take away your
+freedom to share and change it. By contrast, the GNU General Public
+License is intended to guarantee your freedom to share and change free
+software--to make sure the software is free for all its users. This
+General Public License applies to most of the Free Software
+Foundation's software and to any other program whose authors commit to
+using it. (Some other Free Software Foundation software is covered by
+the GNU Lesser General Public License instead.) You can apply it to
+your programs, too.
+
+ When we speak of free software, we are referring to freedom, not
+price. Our General Public Licenses are designed to make sure that you
+have the freedom to distribute copies of free software (and charge for
+this service if you wish), that you receive source code or can get it
+if you want it, that you can change the software or use pieces of it
+in new free programs; and that you know you can do these things.
+
+ To protect your rights, we need to make restrictions that forbid
+anyone to deny you these rights or to ask you to surrender the rights.
+These restrictions translate to certain responsibilities for you if you
+distribute copies of the software, or if you modify it.
+
+ For example, if you distribute copies of such a program, whether
+gratis or for a fee, you must give the recipients all the rights that
+you have. You must make sure that they, too, receive or can get the
+source code. And you must show them these terms so they know their
+rights.
+
+ We protect your rights with two steps: (1) copyright the software, and
+(2) offer you this license which gives you legal permission to copy,
+distribute and/or modify the software.
+
+ Also, for each author's protection and ours, we want to make certain
+that everyone understands that there is no warranty for this free
+software. If the software is modified by someone else and passed on, we
+want its recipients to know that what they have is not the original, so
+that any problems introduced by others will not reflect on the original
+authors' reputations.
+
+ Finally, any free program is threatened constantly by software
+patents. We wish to avoid the danger that redistributors of a free
+program will individually obtain patent licenses, in effect making the
+program proprietary. To prevent this, we have made it clear that any
+patent must be licensed for everyone's free use or not licensed at all.
+
+ The precise terms and conditions for copying, distribution and
+modification follow.
+
+ GNU GENERAL PUBLIC LICENSE
+ TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+ 0. This License applies to any program or other work which contains
+a notice placed by the copyright holder saying it may be distributed
+under the terms of this General Public License. The "Program", below,
+refers to any such program or work, and a "work based on the Program"
+means either the Program or any derivative work under copyright law:
+that is to say, a work containing the Program or a portion of it,
+either verbatim or with modifications and/or translated into another
+language. (Hereinafter, translation is included without limitation in
+the term "modification".) Each licensee is addressed as "you".
+
+Activities other than copying, distribution and modification are not
+covered by this License; they are outside its scope. The act of
+running the Program is not restricted, and the output from the Program
+is covered only if its contents constitute a work based on the
+Program (independent of having been made by running the Program).
+Whether that is true depends on what the Program does.
+
+ 1. You may copy and distribute verbatim copies of the Program's
+source code as you receive it, in any medium, provided that you
+conspicuously and appropriately publish on each copy an appropriate
+copyright notice and disclaimer of warranty; keep intact all the
+notices that refer to this License and to the absence of any warranty;
+and give any other recipients of the Program a copy of this License
+along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and
+you may at your option offer warranty protection in exchange for a fee.
+
+ 2. You may modify your copy or copies of the Program or any portion
+of it, thus forming a work based on the Program, and copy and
+distribute such modifications or work under the terms of Section 1
+above, provided that you also meet all of these conditions:
+
+ a) You must cause the modified files to carry prominent notices
+ stating that you changed the files and the date of any change.
+
+ b) You must cause any work that you distribute or publish, that in
+ whole or in part contains or is derived from the Program or any
+ part thereof, to be licensed as a whole at no charge to all third
+ parties under the terms of this License.
+
+ c) If the modified program normally reads commands interactively
+ when run, you must cause it, when started running for such
+ interactive use in the most ordinary way, to print or display an
+ announcement including an appropriate copyright notice and a
+ notice that there is no warranty (or else, saying that you provide
+ a warranty) and that users may redistribute the program under
+ these conditions, and telling the user how to view a copy of this
+ License. (Exception: if the Program itself is interactive but
+ does not normally print such an announcement, your work based on
+ the Program is not required to print an announcement.)
+
+These requirements apply to the modified work as a whole. If
+identifiable sections of that work are not derived from the Program,
+and can be reasonably considered independent and separate works in
+themselves, then this License, and its terms, do not apply to those
+sections when you distribute them as separate works. But when you
+distribute the same sections as part of a whole which is a work based
+on the Program, the distribution of the whole must be on the terms of
+this License, whose permissions for other licensees extend to the
+entire whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest
+your rights to work written entirely by you; rather, the intent is to
+exercise the right to control the distribution of derivative or
+collective works based on the Program.
+
+In addition, mere aggregation of another work not based on the Program
+with the Program (or with a work based on the Program) on a volume of
+a storage or distribution medium does not bring the other work under
+the scope of this License.
+
+ 3. You may copy and distribute the Program (or a work based on it,
+under Section 2) in object code or executable form under the terms of
+Sections 1 and 2 above provided that you also do one of the following:
+
+ a) Accompany it with the complete corresponding machine-readable
+ source code, which must be distributed under the terms of Sections
+ 1 and 2 above on a medium customarily used for software interchange; or,
+
+ b) Accompany it with a written offer, valid for at least three
+ years, to give any third party, for a charge no more than your
+ cost of physically performing source distribution, a complete
+ machine-readable copy of the corresponding source code, to be
+ distributed under the terms of Sections 1 and 2 above on a medium
+ customarily used for software interchange; or,
+
+ c) Accompany it with the information you received as to the offer
+ to distribute corresponding source code. (This alternative is
+ allowed only for noncommercial distribution and only if you
+ received the program in object code or executable form with such
+ an offer, in accord with Subsection b above.)
+
+The source code for a work means the preferred form of the work for
+making modifications to it. For an executable work, complete source
+code means all the source code for all modules it contains, plus any
+associated interface definition files, plus the scripts used to
+control compilation and installation of the executable. However, as a
+special exception, the source code distributed need not include
+anything that is normally distributed (in either source or binary
+form) with the major components (compiler, kernel, and so on) of the
+operating system on which the executable runs, unless that component
+itself accompanies the executable.
+
+If distribution of executable or object code is made by offering
+access to copy from a designated place, then offering equivalent
+access to copy the source code from the same place counts as
+distribution of the source code, even though third parties are not
+compelled to copy the source along with the object code.
+
+ 4. You may not copy, modify, sublicense, or distribute the Program
+except as expressly provided under this License. Any attempt
+otherwise to copy, modify, sublicense or distribute the Program is
+void, and will automatically terminate your rights under this License.
+However, parties who have received copies, or rights, from you under
+this License will not have their licenses terminated so long as such
+parties remain in full compliance.
+
+ 5. You are not required to accept this License, since you have not
+signed it. However, nothing else grants you permission to modify or
+distribute the Program or its derivative works. These actions are
+prohibited by law if you do not accept this License. Therefore, by
+modifying or distributing the Program (or any work based on the
+Program), you indicate your acceptance of this License to do so, and
+all its terms and conditions for copying, distributing or modifying
+the Program or works based on it.
+
+ 6. Each time you redistribute the Program (or any work based on the
+Program), the recipient automatically receives a license from the
+original licensor to copy, distribute or modify the Program subject to
+these terms and conditions. You may not impose any further
+restrictions on the recipients' exercise of the rights granted herein.
+You are not responsible for enforcing compliance by third parties to
+this License.
+
+ 7. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues),
+conditions are imposed on you (whether by court order, agreement or
+otherwise) that contradict the conditions of this License, they do not
+excuse you from the conditions of this License. If you cannot
+distribute so as to satisfy simultaneously your obligations under this
+License and any other pertinent obligations, then as a consequence you
+may not distribute the Program at all. For example, if a patent
+license would not permit royalty-free redistribution of the Program by
+all those who receive copies directly or indirectly through you, then
+the only way you could satisfy both it and this License would be to
+refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under
+any particular circumstance, the balance of the section is intended to
+apply and the section as a whole is intended to apply in other
+circumstances.
+
+It is not the purpose of this section to induce you to infringe any
+patents or other property right claims or to contest validity of any
+such claims; this section has the sole purpose of protecting the
+integrity of the free software distribution system, which is
+implemented by public license practices. Many people have made
+generous contributions to the wide range of software distributed
+through that system in reliance on consistent application of that
+system; it is up to the author/donor to decide if he or she is willing
+to distribute software through any other system and a licensee cannot
+impose that choice.
+
+This section is intended to make thoroughly clear what is believed to
+be a consequence of the rest of this License.
+
+ 8. If the distribution and/or use of the Program is restricted in
+certain countries either by patents or by copyrighted interfaces, the
+original copyright holder who places the Program under this License
+may add an explicit geographical distribution limitation excluding
+those countries, so that distribution is permitted only in or among
+countries not thus excluded. In such case, this License incorporates
+the limitation as if written in the body of this License.
+
+ 9. The Free Software Foundation may publish revised and/or new versions
+of the General Public License from time to time. Such new versions will
+be similar in spirit to the present version, but may differ in detail to
+address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Program
+specifies a version number of this License which applies to it and "any
+later version", you have the option of following the terms and conditions
+either of that version or of any later version published by the Free
+Software Foundation. If the Program does not specify a version number of
+this License, you may choose any version ever published by the Free Software
+Foundation.
+
+ 10. If you wish to incorporate parts of the Program into other free
+programs whose distribution conditions are different, write to the author
+to ask for permission. For software which is copyrighted by the Free
+Software Foundation, write to the Free Software Foundation; we sometimes
+make exceptions for this. Our decision will be guided by the two goals
+of preserving the free status of all derivatives of our free software and
+of promoting the sharing and reuse of software generally.
+
+ NO WARRANTY
+
+ 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY
+FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN
+OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES
+PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED
+OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE
+PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING,
+REPAIR OR CORRECTION.
+
+ 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING
+WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR
+REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES,
+INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING
+OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED
+TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY
+YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER
+PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE
+POSSIBILITY OF SUCH DAMAGES.
+
+ END OF TERMS AND CONDITIONS
+
+ How to Apply These Terms to Your New Programs
+
+ If you develop a new program, and you want it to be of the greatest
+possible use to the public, the best way to achieve this is to make it
+free software which everyone can redistribute and change under these terms.
+
+ To do so, attach the following notices to the program. It is safest
+to attach them to the start of each source file to most effectively
+convey the exclusion of warranty; and each file should have at least
+the "copyright" line and a pointer to where the full notice is found.
+
+ <one line to give the program's name and a brief idea of what it does.>
+ Copyright (C) <year> <name of author>
+
+ This program is free software; you can redistribute it and/or modify
+ it under the terms of the GNU General Public License as published by
+ the Free Software Foundation; either version 2 of the License, or
+ (at your option) any later version.
+
+ This program is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License along
+ with this program; if not, write to the Free Software Foundation, Inc.,
+ 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this
+when it starts in an interactive mode:
+
+ Gnomovision version 69, Copyright (C) year name of author
+ Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'.
+ This is free software, and you are welcome to redistribute it
+ under certain conditions; type `show c' for details.
+
+The hypothetical commands `show w' and `show c' should show the appropriate
+parts of the General Public License. Of course, the commands you use may
+be called something other than `show w' and `show c'; they could even be
+mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your
+school, if any, to sign a "copyright disclaimer" for the program, if
+necessary. Here is a sample; alter the names:
+
+ Yoyodyne, Inc., hereby disclaims all copyright interest in the program
+ `Gnomovision' (which makes passes at compilers) written by James Hacker.
+
+ <signature of Ty Coon>, 1 April 1989
+ Ty Coon, President of Vice
+
+This General Public License does not permit incorporating your program into
+proprietary programs. If your program is a subroutine library, you may
+consider it more useful to permit linking proprietary applications with the
+library. If this is what you want to do, use the GNU Lesser General
+Public License instead of this License.
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/README b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/README
new file mode 100644
index 000000000..2ea86bf46
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/README
@@ -0,0 +1,13 @@
+The files in this directory originally come from
+https://github.com/percona/PerconaFT/.
+
+This directory only includes the "locktree" part of PerconaFT, and its
+dependencies.
+
+The following modifications were made:
+- Make locktree usable outside of PerconaFT library
+- Add shared read-only lock support
+
+The files named *_subst.* are substitutes of the PerconaFT's files, they
+contain replacements of PerconaFT's functionality.
+
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/db.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/db.h
new file mode 100644
index 000000000..5aa826c8e
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/db.h
@@ -0,0 +1,76 @@
+#ifndef _DB_H
+#define _DB_H
+
+#include <stdint.h>
+#include <sys/types.h>
+
+typedef struct __toku_dbt DBT;
+
+// port: this is currently not used
+struct simple_dbt {
+ uint32_t len;
+ void *data;
+};
+
+// engine status info
+// engine status is passed to handlerton as an array of
+// TOKU_ENGINE_STATUS_ROW_S[]
+typedef enum {
+ STATUS_FS_STATE = 0, // interpret as file system state (redzone) enum
+ STATUS_UINT64, // interpret as uint64_t
+ STATUS_CHARSTR, // interpret as char *
+ STATUS_UNIXTIME, // interpret as time_t
+ STATUS_TOKUTIME, // interpret as tokutime_t
+ STATUS_PARCOUNT, // interpret as PARTITIONED_COUNTER
+ STATUS_DOUBLE // interpret as double
+} toku_engine_status_display_type;
+
+typedef enum {
+ TOKU_ENGINE_STATUS = (1ULL << 0), // Include when asking for engine status
+ TOKU_GLOBAL_STATUS =
+ (1ULL << 1), // Include when asking for information_schema.global_status
+} toku_engine_status_include_type;
+
+typedef struct __toku_engine_status_row {
+ const char *keyname; // info schema key, should not change across revisions
+ // without good reason
+ const char
+ *columnname; // column for mysql, e.g. information_schema.global_status.
+ // TOKUDB_ will automatically be prefixed.
+ const char *legend; // the text that will appear at user interface
+ toku_engine_status_display_type type; // how to interpret the value
+ toku_engine_status_include_type
+ include; // which kinds of callers should get read this row?
+ union {
+ double dnum;
+ uint64_t num;
+ const char *str;
+ char datebuf[26];
+ struct partitioned_counter *parcount;
+ } value;
+} * TOKU_ENGINE_STATUS_ROW, TOKU_ENGINE_STATUS_ROW_S;
+
+#define DB_BUFFER_SMALL -30999
+#define DB_LOCK_DEADLOCK -30995
+#define DB_LOCK_NOTGRANTED -30994
+#define DB_NOTFOUND -30989
+#define DB_KEYEXIST -30996
+#define DB_DBT_MALLOC 8
+#define DB_DBT_REALLOC 64
+#define DB_DBT_USERMEM 256
+
+/* PerconaFT specific error codes */
+#define TOKUDB_OUT_OF_LOCKS -100000
+
+typedef void (*lock_wait_callback)(void *arg, uint64_t requesting_txnid,
+ uint64_t blocking_txnid);
+
+struct __toku_dbt {
+ void *data;
+ size_t size;
+ size_t ulen;
+ // One of DB_DBT_XXX flags
+ uint32_t flags;
+};
+
+#endif
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/comparator.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/comparator.h
new file mode 100644
index 000000000..718efc623
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/comparator.h
@@ -0,0 +1,138 @@
+/* -*- mode: C; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <string.h>
+
+#include "../db.h"
+#include "../portability/memory.h"
+#include "../util/dbt.h"
+
+typedef int (*ft_compare_func)(void *arg, const DBT *a, const DBT *b);
+
+int toku_keycompare(const void *key1, size_t key1len, const void *key2,
+ size_t key2len);
+
+int toku_builtin_compare_fun(const DBT *, const DBT *)
+ __attribute__((__visibility__("default")));
+
+namespace toku {
+
+// a comparator object encapsulates the data necessary for
+// comparing two keys in a fractal tree. it further understands
+// that points may be positive or negative infinity.
+
+class comparator {
+ void init(ft_compare_func cmp, void *cmp_arg, uint8_t memcmp_magic) {
+ _cmp = cmp;
+ _cmp_arg = cmp_arg;
+ _memcmp_magic = memcmp_magic;
+ }
+
+ public:
+ // This magic value is reserved to mean that the magic has not been set.
+ static const uint8_t MEMCMP_MAGIC_NONE = 0;
+
+ void create(ft_compare_func cmp, void *cmp_arg,
+ uint8_t memcmp_magic = MEMCMP_MAGIC_NONE) {
+ init(cmp, cmp_arg, memcmp_magic);
+ }
+
+ // inherit the attributes of another comparator, but keep our own
+ // copy of fake_db that is owned separately from the one given.
+ void inherit(const comparator &cmp) {
+ invariant_notnull(cmp._cmp);
+ init(cmp._cmp, cmp._cmp_arg, cmp._memcmp_magic);
+ }
+
+ // like inherit, but doesn't require that the this comparator
+ // was already created
+ void create_from(const comparator &cmp) { inherit(cmp); }
+
+ void destroy() {}
+
+ ft_compare_func get_compare_func() const { return _cmp; }
+
+ uint8_t get_memcmp_magic() const { return _memcmp_magic; }
+
+ bool valid() const { return _cmp != nullptr; }
+
+ inline bool dbt_has_memcmp_magic(const DBT *dbt) const {
+ return *reinterpret_cast<const char *>(dbt->data) == _memcmp_magic;
+ }
+
+ int operator()(const DBT *a, const DBT *b) const {
+ if (__builtin_expect(toku_dbt_is_infinite(a) || toku_dbt_is_infinite(b),
+ 0)) {
+ return toku_dbt_infinite_compare(a, b);
+ } else if (_memcmp_magic != MEMCMP_MAGIC_NONE
+ // If `a' has the memcmp magic..
+ && dbt_has_memcmp_magic(a)
+ // ..then we expect `b' to also have the memcmp magic
+ && __builtin_expect(dbt_has_memcmp_magic(b), 1)) {
+ assert(0); // psergey: this branch should not be taken.
+ return toku_builtin_compare_fun(a, b);
+ } else {
+ // yikes, const sadness here
+ return _cmp(_cmp_arg, a, b);
+ }
+ }
+
+ private:
+ ft_compare_func _cmp;
+ void *_cmp_arg;
+
+ uint8_t _memcmp_magic;
+};
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/ft-status.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/ft-status.h
new file mode 100644
index 000000000..1b4511172
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/ft-status.h
@@ -0,0 +1,102 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include "../db.h"
+#include "../portability/toku_race_tools.h"
+#include "../util/status.h"
+
+//
+// Lock Tree Manager statistics
+//
+class LTM_STATUS_S {
+ public:
+ enum {
+ LTM_SIZE_CURRENT = 0,
+ LTM_SIZE_LIMIT,
+ LTM_ESCALATION_COUNT,
+ LTM_ESCALATION_TIME,
+ LTM_ESCALATION_LATEST_RESULT,
+ LTM_NUM_LOCKTREES,
+ LTM_LOCK_REQUESTS_PENDING,
+ LTM_STO_NUM_ELIGIBLE,
+ LTM_STO_END_EARLY_COUNT,
+ LTM_STO_END_EARLY_TIME,
+ LTM_WAIT_COUNT,
+ LTM_WAIT_TIME,
+ LTM_LONG_WAIT_COUNT,
+ LTM_LONG_WAIT_TIME,
+ LTM_TIMEOUT_COUNT,
+ LTM_WAIT_ESCALATION_COUNT,
+ LTM_WAIT_ESCALATION_TIME,
+ LTM_LONG_WAIT_ESCALATION_COUNT,
+ LTM_LONG_WAIT_ESCALATION_TIME,
+ LTM_STATUS_NUM_ROWS // must be last
+ };
+
+ void init(void);
+ void destroy(void);
+
+ TOKU_ENGINE_STATUS_ROW_S status[LTM_STATUS_NUM_ROWS];
+
+ private:
+ bool m_initialized = false;
+};
+typedef LTM_STATUS_S* LTM_STATUS;
+extern LTM_STATUS_S ltm_status;
+
+#define LTM_STATUS_VAL(x) ltm_status.status[LTM_STATUS_S::x].value.num
+
+void toku_status_init(void); // just call ltm_status.init();
+void toku_status_destroy(void); // just call ltm_status.destroy();
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc
new file mode 100644
index 000000000..5110cd482
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc
@@ -0,0 +1,139 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "concurrent_tree.h"
+
+// PORT #include <toku_assert.h>
+namespace toku {
+
+void concurrent_tree::create(const comparator *cmp) {
+ // start with an empty root node. we do this instead of
+ // setting m_root to null so there's always a root to lock
+ m_root.create_root(cmp);
+}
+
+void concurrent_tree::destroy(void) { m_root.destroy_root(); }
+
+bool concurrent_tree::is_empty(void) { return m_root.is_empty(); }
+
+uint64_t concurrent_tree::get_insertion_memory_overhead(void) {
+ return sizeof(treenode);
+}
+
+void concurrent_tree::locked_keyrange::prepare(concurrent_tree *tree) {
+ // the first step in acquiring a locked keyrange is locking the root
+ treenode *const root = &tree->m_root;
+ m_tree = tree;
+ m_subtree = root;
+ m_range = keyrange::get_infinite_range();
+ root->mutex_lock();
+}
+
+void concurrent_tree::locked_keyrange::acquire(const keyrange &range) {
+ treenode *const root = &m_tree->m_root;
+
+ treenode *subtree;
+ if (root->is_empty() || root->range_overlaps(range)) {
+ subtree = root;
+ } else {
+ // we do not have a precomputed comparison hint, so pass null
+ const keyrange::comparison *cmp_hint = nullptr;
+ subtree = root->find_node_with_overlapping_child(range, cmp_hint);
+ }
+
+ // subtree is locked. it will be unlocked when this is release()'d
+ invariant_notnull(subtree);
+ m_range = range;
+ m_subtree = subtree;
+}
+
+bool concurrent_tree::locked_keyrange::add_shared_owner(const keyrange &range,
+ TXNID new_owner) {
+ return m_subtree->insert(range, new_owner, /*is_shared*/ true);
+}
+
+void concurrent_tree::locked_keyrange::release(void) {
+ m_subtree->mutex_unlock();
+}
+
+void concurrent_tree::locked_keyrange::insert(const keyrange &range,
+ TXNID txnid, bool is_shared) {
+ // empty means no children, and only the root should ever be empty
+ if (m_subtree->is_empty()) {
+ m_subtree->set_range_and_txnid(range, txnid, is_shared);
+ } else {
+ m_subtree->insert(range, txnid, is_shared);
+ }
+}
+
+void concurrent_tree::locked_keyrange::remove(const keyrange &range,
+ TXNID txnid) {
+ invariant(!m_subtree->is_empty());
+ treenode *new_subtree = m_subtree->remove(range, txnid);
+ // if removing range changed the root of the subtree,
+ // then the subtree must be the root of the entire tree.
+ if (new_subtree == nullptr) {
+ invariant(m_subtree->is_root());
+ invariant(m_subtree->is_empty());
+ }
+}
+
+void concurrent_tree::locked_keyrange::remove_all(void) {
+ m_subtree->recursive_remove();
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.h
new file mode 100644
index 000000000..e1bfb86c5
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.h
@@ -0,0 +1,174 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=2:softtabstop=2:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include "../ft/comparator.h"
+#include "keyrange.h"
+#include "treenode.h"
+
+namespace toku {
+
+// A concurrent_tree stores non-overlapping ranges.
+// Access to disjoint parts of the tree usually occurs concurrently.
+
+class concurrent_tree {
+ public:
+ // A locked_keyrange gives you exclusive access to read and write
+ // operations that occur on any keys in that range. You only have
+ // the right to operate on keys in that range or keys that were read
+ // from the keyrange using iterate()
+ //
+ // Access model:
+ // - user prepares a locked keyrange. all threads serialize behind prepare().
+ // - user breaks the serialzation point by acquiring a range, or releasing.
+ // - one thread operates on a certain locked_keyrange object at a time.
+ // - when the thread is finished, it releases
+
+ class locked_keyrange {
+ public:
+ // effect: prepare to acquire a locked keyrange over the given
+ // concurrent_tree, preventing other threads from preparing
+ // until this thread either does acquire() or release().
+ // note: operations performed on a prepared keyrange are equivalent
+ // to ones performed on an acquired keyrange over -inf, +inf.
+ // rationale: this provides the user with a serialization point for
+ // descending
+ // or modifying the the tree. it also proives a convenient way of
+ // doing serializable operations on the tree.
+ // There are two valid sequences of calls:
+ // - prepare, acquire, [operations], release
+ // - prepare, [operations],release
+ void prepare(concurrent_tree *tree);
+
+ // requires: the locked keyrange was prepare()'d
+ // effect: acquire a locked keyrange over the given concurrent_tree.
+ // the locked keyrange represents the range of keys overlapped
+ // by the given range
+ void acquire(const keyrange &range);
+
+ // effect: releases a locked keyrange and the mutex it holds
+ void release(void);
+
+ // effect: iterate over each range this locked_keyrange represents,
+ // calling function->fn() on each node's keyrange and txnid
+ // until there are no more or the function returns false
+ template <class F>
+ void iterate(F *function) const {
+ // if the subtree is non-empty, traverse it by calling the given
+ // function on each range, txnid pair found that overlaps.
+ if (!m_subtree->is_empty()) {
+ m_subtree->traverse_overlaps(m_range, function);
+ }
+ }
+
+ // Adds another owner to the lock on the specified keyrange.
+ // requires: the keyrange contains one treenode whose bounds are
+ // exactly equal to the specifed range (no sub/supersets)
+ bool add_shared_owner(const keyrange &range, TXNID new_owner);
+
+ // inserts the given range into the tree, with an associated txnid.
+ // requires: range does not overlap with anything in this locked_keyrange
+ // rationale: caller is responsible for only inserting unique ranges
+ void insert(const keyrange &range, TXNID txnid, bool is_shared);
+
+ // effect: removes the given range from the tree.
+ // - txnid=TXNID_ANY means remove the range no matter what its
+ // owners are
+ // - Other value means remove the specified txnid from
+ // ownership (if the range has other owners, it will remain
+ // in the tree)
+ // requires: range exists exactly in this locked_keyrange
+ // rationale: caller is responsible for only removing existing ranges
+ void remove(const keyrange &range, TXNID txnid);
+
+ // effect: removes all of the keys represented by this locked keyrange
+ // rationale: we'd like a fast way to empty out a tree
+ void remove_all(void);
+
+ private:
+ // the concurrent tree this locked keyrange is for
+ concurrent_tree *m_tree;
+
+ // the range of keys this locked keyrange represents
+ keyrange m_range;
+
+ // the subtree under which all overlapping ranges exist
+ treenode *m_subtree;
+
+ friend class concurrent_tree_unit_test;
+ };
+
+ // effect: initialize the tree to an empty state
+ void create(const comparator *cmp);
+
+ // effect: destroy the tree.
+ // requires: tree is empty
+ void destroy(void);
+
+ // returns: true iff the tree is empty
+ bool is_empty(void);
+
+ // returns: the memory overhead of a single insertion into the tree
+ static uint64_t get_insertion_memory_overhead(void);
+
+ private:
+ // the root needs to always exist so there's a lock to grab
+ // even if the tree is empty. that's why we store a treenode
+ // here and not a pointer to one.
+ treenode m_root;
+
+ friend class concurrent_tree_unit_test;
+};
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.cc
new file mode 100644
index 000000000..e50ace5a9
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.cc
@@ -0,0 +1,222 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "keyrange.h"
+
+#include "../util/dbt.h"
+
+namespace toku {
+
+// create a keyrange by borrowing the left and right dbt
+// pointers. no memory is copied. no checks for infinity needed.
+void keyrange::create(const DBT *left, const DBT *right) {
+ init_empty();
+ m_left_key = left;
+ m_right_key = right;
+}
+
+// destroy the key copies. if they were never set, then destroy does nothing.
+void keyrange::destroy(void) {
+ toku_destroy_dbt(&m_left_key_copy);
+ toku_destroy_dbt(&m_right_key_copy);
+}
+
+// create a keyrange by copying the keys from the given range.
+void keyrange::create_copy(const keyrange &range) {
+ // start with an initialized, empty range
+ init_empty();
+
+ // optimize the case where the left and right keys are the same.
+ // we'd like to only have one copy of the data.
+ if (toku_dbt_equals(range.get_left_key(), range.get_right_key())) {
+ set_both_keys(range.get_left_key());
+ } else {
+ // replace our empty left and right keys with
+ // copies of the range's left and right keys
+ replace_left_key(range.get_left_key());
+ replace_right_key(range.get_right_key());
+ }
+}
+
+// extend this keyrange by choosing the leftmost and rightmost
+// endpoints between this range and the given. replaced keys
+// in this range are freed and inherited keys are copied.
+void keyrange::extend(const comparator &cmp, const keyrange &range) {
+ const DBT *range_left = range.get_left_key();
+ const DBT *range_right = range.get_right_key();
+ if (cmp(range_left, get_left_key()) < 0) {
+ replace_left_key(range_left);
+ }
+ if (cmp(range_right, get_right_key()) > 0) {
+ replace_right_key(range_right);
+ }
+}
+
+// how much memory does this keyrange take?
+// - the size of the left and right keys
+// --- ignore the fact that we may have optimized the point case.
+// it complicates things for little gain.
+// - the size of the keyrange class itself
+uint64_t keyrange::get_memory_size(void) const {
+ const DBT *left_key = get_left_key();
+ const DBT *right_key = get_right_key();
+ return left_key->size + right_key->size + sizeof(keyrange);
+}
+
+// compare ranges.
+keyrange::comparison keyrange::compare(const comparator &cmp,
+ const keyrange &range) const {
+ if (cmp(get_right_key(), range.get_left_key()) < 0) {
+ return comparison::LESS_THAN;
+ } else if (cmp(get_left_key(), range.get_right_key()) > 0) {
+ return comparison::GREATER_THAN;
+ } else if (cmp(get_left_key(), range.get_left_key()) == 0 &&
+ cmp(get_right_key(), range.get_right_key()) == 0) {
+ return comparison::EQUALS;
+ } else {
+ return comparison::OVERLAPS;
+ }
+}
+
+bool keyrange::overlaps(const comparator &cmp, const keyrange &range) const {
+ // equality is a stronger form of overlapping.
+ // so two ranges "overlap" if they're either equal or just overlapping.
+ comparison c = compare(cmp, range);
+ return c == comparison::EQUALS || c == comparison::OVERLAPS;
+}
+
+keyrange keyrange::get_infinite_range(void) {
+ keyrange range;
+ range.create(toku_dbt_negative_infinity(), toku_dbt_positive_infinity());
+ return range;
+}
+
+void keyrange::init_empty(void) {
+ m_left_key = nullptr;
+ m_right_key = nullptr;
+ toku_init_dbt(&m_left_key_copy);
+ toku_init_dbt(&m_right_key_copy);
+ m_point_range = false;
+}
+
+const DBT *keyrange::get_left_key(void) const {
+ if (m_left_key) {
+ return m_left_key;
+ } else {
+ return &m_left_key_copy;
+ }
+}
+
+const DBT *keyrange::get_right_key(void) const {
+ if (m_right_key) {
+ return m_right_key;
+ } else {
+ return &m_right_key_copy;
+ }
+}
+
+// copy the given once and set both the left and right pointers.
+// optimization for point ranges, so the left and right ranges
+// are not copied twice.
+void keyrange::set_both_keys(const DBT *key) {
+ if (toku_dbt_is_infinite(key)) {
+ m_left_key = key;
+ m_right_key = key;
+ } else {
+ toku_clone_dbt(&m_left_key_copy, *key);
+ toku_copyref_dbt(&m_right_key_copy, m_left_key_copy);
+ }
+ m_point_range = true;
+}
+
+// destroy the current left key. set and possibly copy the new one
+void keyrange::replace_left_key(const DBT *key) {
+ // a little magic:
+ //
+ // if this is a point range, then the left and right keys share
+ // one copy of the data, and it lives in the left key copy. so
+ // if we're replacing the left key, move the real data to the
+ // right key copy instead of destroying it. now, the memory is
+ // owned by the right key and the left key may be replaced.
+ if (m_point_range) {
+ m_right_key_copy = m_left_key_copy;
+ } else {
+ toku_destroy_dbt(&m_left_key_copy);
+ }
+
+ if (toku_dbt_is_infinite(key)) {
+ m_left_key = key;
+ } else {
+ toku_clone_dbt(&m_left_key_copy, *key);
+ m_left_key = nullptr;
+ }
+ m_point_range = false;
+}
+
+// destroy the current right key. set and possibly copy the new one
+void keyrange::replace_right_key(const DBT *key) {
+ toku_destroy_dbt(&m_right_key_copy);
+ if (toku_dbt_is_infinite(key)) {
+ m_right_key = key;
+ } else {
+ toku_clone_dbt(&m_right_key_copy, *key);
+ m_right_key = nullptr;
+ }
+ m_point_range = false;
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.h
new file mode 100644
index 000000000..f9aeea0c4
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.h
@@ -0,0 +1,141 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include "../ft/comparator.h"
+
+namespace toku {
+
+// A keyrange has a left and right key as endpoints.
+//
+// When a keyrange is created it owns no memory, but when it copies
+// or extends another keyrange, it copies memory as necessary. This
+// means it is cheap in the common case.
+
+class keyrange {
+ public:
+ // effect: constructor that borrows left and right key pointers.
+ // no memory is allocated or copied.
+ void create(const DBT *left_key, const DBT *right_key);
+
+ // effect: constructor that allocates and copies another keyrange's points.
+ void create_copy(const keyrange &range);
+
+ // effect: destroys the keyrange, freeing any allocated memory
+ void destroy(void);
+
+ // effect: extends the keyrange by choosing the leftmost and rightmost
+ // endpoints from this range and the given range.
+ // replaced keys in this range are freed, new keys are copied.
+ void extend(const comparator &cmp, const keyrange &range);
+
+ // returns: the amount of memory this keyrange takes. does not account
+ // for point optimizations or malloc overhead.
+ uint64_t get_memory_size(void) const;
+
+ // returns: pointer to the left key of this range
+ const DBT *get_left_key(void) const;
+
+ // returns: pointer to the right key of this range
+ const DBT *get_right_key(void) const;
+
+ // two ranges are either equal, lt, gt, or overlapping
+ enum comparison { EQUALS, LESS_THAN, GREATER_THAN, OVERLAPS };
+
+ // effect: compares this range to the given range
+ // returns: LESS_THAN if given range is strictly to the left
+ // GREATER_THAN if given range is strictly to the right
+ // EQUALS if given range has the same left and right endpoints
+ // OVERLAPS if at least one of the given range's endpoints falls
+ // between this range's endpoints
+ comparison compare(const comparator &cmp, const keyrange &range) const;
+
+ // returns: true if the range and the given range are equal or overlapping
+ bool overlaps(const comparator &cmp, const keyrange &range) const;
+
+ // returns: a keyrange representing -inf, +inf
+ static keyrange get_infinite_range(void);
+
+ private:
+ // some keys should be copied, some keys should not be.
+ //
+ // to support both, we use two DBTs for copies and two pointers
+ // for temporaries. the access rule is:
+ // - if a pointer is non-null, then it reprsents the key.
+ // - otherwise the pointer is null, and the key is in the copy.
+ DBT m_left_key_copy;
+ DBT m_right_key_copy;
+ const DBT *m_left_key;
+ const DBT *m_right_key;
+
+ // if this range is a point range, then m_left_key == m_right_key
+ // and the actual data is stored exactly once in m_left_key_copy.
+ bool m_point_range;
+
+ // effect: initializes a keyrange to be empty
+ void init_empty(void);
+
+ // effect: copies the given key once into the left key copy
+ // and sets the right key copy to share the left.
+ // rationale: optimization for point ranges to only do one malloc
+ void set_both_keys(const DBT *key);
+
+ // effect: destroys the current left key. sets and copies the new one.
+ void replace_left_key(const DBT *key);
+
+ // effect: destroys the current right key. sets and copies the new one.
+ void replace_right_key(const DBT *key);
+};
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.cc
new file mode 100644
index 000000000..3d217be70
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.cc
@@ -0,0 +1,527 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=2:softtabstop=2:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "lock_request.h"
+
+#include "../portability/toku_race_tools.h"
+#include "../portability/txn_subst.h"
+#include "../util/dbt.h"
+#include "locktree.h"
+
+namespace toku {
+
+// initialize a lock request's internals
+void lock_request::create(toku_external_mutex_factory_t mutex_factory) {
+ m_txnid = TXNID_NONE;
+ m_conflicting_txnid = TXNID_NONE;
+ m_start_time = 0;
+ m_left_key = nullptr;
+ m_right_key = nullptr;
+ toku_init_dbt(&m_left_key_copy);
+ toku_init_dbt(&m_right_key_copy);
+
+ m_type = type::UNKNOWN;
+ m_lt = nullptr;
+
+ m_complete_r = 0;
+ m_state = state::UNINITIALIZED;
+ m_info = nullptr;
+
+ // psergey-todo: this condition is for interruptible wait
+ // note: moved to here from lock_request::create:
+ toku_external_cond_init(mutex_factory, &m_wait_cond);
+
+ m_start_test_callback = nullptr;
+ m_start_before_pending_test_callback = nullptr;
+ m_retry_test_callback = nullptr;
+}
+
+// destroy a lock request.
+void lock_request::destroy(void) {
+ invariant(m_state != state::PENDING);
+ invariant(m_state != state::DESTROYED);
+ m_state = state::DESTROYED;
+ toku_destroy_dbt(&m_left_key_copy);
+ toku_destroy_dbt(&m_right_key_copy);
+ toku_external_cond_destroy(&m_wait_cond);
+}
+
+// set the lock request parameters. this API allows a lock request to be reused.
+void lock_request::set(locktree *lt, TXNID txnid, const DBT *left_key,
+ const DBT *right_key, lock_request::type lock_type,
+ bool big_txn, void *extra) {
+ invariant(m_state != state::PENDING);
+ m_lt = lt;
+
+ m_txnid = txnid;
+ m_left_key = left_key;
+ m_right_key = right_key;
+ toku_destroy_dbt(&m_left_key_copy);
+ toku_destroy_dbt(&m_right_key_copy);
+ m_type = lock_type;
+ m_state = state::INITIALIZED;
+ m_info = lt ? lt->get_lock_request_info() : nullptr;
+ m_big_txn = big_txn;
+ m_extra = extra;
+}
+
+// get rid of any stored left and right key copies and
+// replace them with copies of the given left and right key
+void lock_request::copy_keys() {
+ if (!toku_dbt_is_infinite(m_left_key)) {
+ toku_clone_dbt(&m_left_key_copy, *m_left_key);
+ m_left_key = &m_left_key_copy;
+ }
+ if (!toku_dbt_is_infinite(m_right_key)) {
+ toku_clone_dbt(&m_right_key_copy, *m_right_key);
+ m_right_key = &m_right_key_copy;
+ }
+}
+
+// what are the conflicts for this pending lock request?
+void lock_request::get_conflicts(txnid_set *conflicts) {
+ invariant(m_state == state::PENDING);
+ const bool is_write_request = m_type == type::WRITE;
+ m_lt->get_conflicts(is_write_request, m_txnid, m_left_key, m_right_key,
+ conflicts);
+}
+
+// build a wait-for-graph for this lock request and the given conflict set
+// for each transaction B that blocks A's lock request
+// if B is blocked then
+// add (A,T) to the WFG and if B is new, fill in the WFG from B
+void lock_request::build_wait_graph(wfg *wait_graph,
+ const txnid_set &conflicts) {
+ uint32_t num_conflicts = conflicts.size();
+ for (uint32_t i = 0; i < num_conflicts; i++) {
+ TXNID conflicting_txnid = conflicts.get(i);
+ lock_request *conflicting_request = find_lock_request(conflicting_txnid);
+ invariant(conflicting_txnid != m_txnid);
+ invariant(conflicting_request != this);
+ if (conflicting_request) {
+ bool already_exists = wait_graph->node_exists(conflicting_txnid);
+ wait_graph->add_edge(m_txnid, conflicting_txnid);
+ if (!already_exists) {
+ // recursively build the wait for graph rooted at the conflicting
+ // request, given its set of lock conflicts.
+ txnid_set other_conflicts;
+ other_conflicts.create();
+ conflicting_request->get_conflicts(&other_conflicts);
+ conflicting_request->build_wait_graph(wait_graph, other_conflicts);
+ other_conflicts.destroy();
+ }
+ }
+ }
+}
+
+// returns: true if the current set of lock requests contains
+// a deadlock, false otherwise.
+bool lock_request::deadlock_exists(const txnid_set &conflicts) {
+ wfg wait_graph;
+ wait_graph.create();
+
+ build_wait_graph(&wait_graph, conflicts);
+
+ std::function<void(TXNID)> reporter;
+ if (m_deadlock_cb) {
+ reporter = [this](TXNID a) {
+ lock_request *req = find_lock_request(a);
+ if (req) {
+ m_deadlock_cb(req->m_txnid, (req->m_type == lock_request::WRITE),
+ req->m_left_key, req->m_right_key);
+ }
+ };
+ }
+
+ bool deadlock = wait_graph.cycle_exists_from_txnid(m_txnid, reporter);
+ wait_graph.destroy();
+ return deadlock;
+}
+
+// try to acquire a lock described by this lock request.
+int lock_request::start(void) {
+ int r;
+
+ txnid_set conflicts;
+ conflicts.create();
+ if (m_type == type::WRITE) {
+ r = m_lt->acquire_write_lock(m_txnid, m_left_key, m_right_key, &conflicts,
+ m_big_txn);
+ } else {
+ invariant(m_type == type::READ);
+ r = m_lt->acquire_read_lock(m_txnid, m_left_key, m_right_key, &conflicts,
+ m_big_txn);
+ }
+
+ // if the lock is not granted, save it to the set of lock requests
+ // and check for a deadlock. if there is one, complete it as failed
+ if (r == DB_LOCK_NOTGRANTED) {
+ copy_keys();
+ m_state = state::PENDING;
+ m_start_time = toku_current_time_microsec() / 1000;
+ m_conflicting_txnid = conflicts.get(0);
+ if (m_start_before_pending_test_callback)
+ m_start_before_pending_test_callback();
+ toku_external_mutex_lock(&m_info->mutex);
+ insert_into_lock_requests();
+ if (deadlock_exists(conflicts)) {
+ remove_from_lock_requests();
+ r = DB_LOCK_DEADLOCK;
+ }
+ toku_external_mutex_unlock(&m_info->mutex);
+ if (m_start_test_callback) m_start_test_callback(); // test callback
+ }
+
+ if (r != DB_LOCK_NOTGRANTED) {
+ complete(r);
+ }
+
+ conflicts.destroy();
+ return r;
+}
+
+// sleep on the lock request until it becomes resolved or the wait time has
+// elapsed.
+int lock_request::wait(uint64_t wait_time_ms) {
+ return wait(wait_time_ms, 0, nullptr);
+}
+
+int lock_request::wait(uint64_t wait_time_ms, uint64_t killed_time_ms,
+ int (*killed_callback)(void),
+ void (*lock_wait_callback)(void *, lock_wait_infos *),
+ void *callback_arg) {
+ uint64_t t_now = toku_current_time_microsec();
+ uint64_t t_start = t_now;
+ uint64_t t_end = t_start + wait_time_ms * 1000;
+
+ toku_external_mutex_lock(&m_info->mutex);
+
+ // check again, this time locking out other retry calls
+ if (m_state == state::PENDING) {
+ lock_wait_infos conflicts_collector;
+ retry(&conflicts_collector);
+ if (m_state == state::PENDING) {
+ report_waits(&conflicts_collector, lock_wait_callback, callback_arg);
+ }
+ }
+
+ while (m_state == state::PENDING) {
+ // check if this thread is killed
+ if (killed_callback && killed_callback()) {
+ remove_from_lock_requests();
+ complete(DB_LOCK_NOTGRANTED);
+ continue;
+ }
+
+ // compute the time until we should wait
+ uint64_t t_wait;
+ if (killed_time_ms == 0) {
+ t_wait = t_end;
+ } else {
+ t_wait = t_now + killed_time_ms * 1000;
+ if (t_wait > t_end) t_wait = t_end;
+ }
+
+ int r = toku_external_cond_timedwait(&m_wait_cond, &m_info->mutex,
+ (int64_t)(t_wait - t_now));
+ invariant(r == 0 || r == ETIMEDOUT);
+
+ t_now = toku_current_time_microsec();
+ if (m_state == state::PENDING && (t_now >= t_end)) {
+ m_info->counters.timeout_count += 1;
+
+ // if we're still pending and we timed out, then remove our
+ // request from the set of lock requests and fail.
+ remove_from_lock_requests();
+
+ // complete sets m_state to COMPLETE, breaking us out of the loop
+ complete(DB_LOCK_NOTGRANTED);
+ }
+ }
+
+ uint64_t t_real_end = toku_current_time_microsec();
+ uint64_t duration = t_real_end - t_start;
+ m_info->counters.wait_count += 1;
+ m_info->counters.wait_time += duration;
+ if (duration >= 1000000) {
+ m_info->counters.long_wait_count += 1;
+ m_info->counters.long_wait_time += duration;
+ }
+ toku_external_mutex_unlock(&m_info->mutex);
+
+ invariant(m_state == state::COMPLETE);
+ return m_complete_r;
+}
+
+// complete this lock request with the given return value
+void lock_request::complete(int complete_r) {
+ m_complete_r = complete_r;
+ m_state = state::COMPLETE;
+}
+
+const DBT *lock_request::get_left_key(void) const { return m_left_key; }
+
+const DBT *lock_request::get_right_key(void) const { return m_right_key; }
+
+TXNID lock_request::get_txnid(void) const { return m_txnid; }
+
+uint64_t lock_request::get_start_time(void) const { return m_start_time; }
+
+TXNID lock_request::get_conflicting_txnid(void) const {
+ return m_conflicting_txnid;
+}
+
+int lock_request::retry(lock_wait_infos *conflicts_collector) {
+ invariant(m_state == state::PENDING);
+ int r;
+ txnid_set conflicts;
+ conflicts.create();
+
+ if (m_type == type::WRITE) {
+ r = m_lt->acquire_write_lock(m_txnid, m_left_key, m_right_key, &conflicts,
+ m_big_txn);
+ } else {
+ r = m_lt->acquire_read_lock(m_txnid, m_left_key, m_right_key, &conflicts,
+ m_big_txn);
+ }
+
+ // if the acquisition succeeded then remove ourselves from the
+ // set of lock requests, complete, and signal the waiting thread.
+ if (r == 0) {
+ remove_from_lock_requests();
+ complete(r);
+ if (m_retry_test_callback) m_retry_test_callback(); // test callback
+ toku_external_cond_broadcast(&m_wait_cond);
+ } else {
+ m_conflicting_txnid = conflicts.get(0);
+ add_conflicts_to_waits(&conflicts, conflicts_collector);
+ }
+ conflicts.destroy();
+
+ return r;
+}
+
+void lock_request::retry_all_lock_requests(
+ locktree *lt, void (*lock_wait_callback)(void *, lock_wait_infos *),
+ void *callback_arg, void (*after_retry_all_test_callback)(void)) {
+ lt_lock_request_info *info = lt->get_lock_request_info();
+
+ // if there are no pending lock requests than there is nothing to do
+ // the unlocked data race on pending_is_empty is OK since lock requests
+ // are retried after added to the pending set.
+ if (info->pending_is_empty) return;
+
+ // get my retry generation (post increment of retry_want)
+ unsigned long long my_retry_want = (info->retry_want += 1);
+
+ toku_mutex_lock(&info->retry_mutex);
+
+ // here is the group retry algorithm.
+ // get the latest retry_want count and use it as the generation number of
+ // this retry operation. if this retry generation is > the last retry
+ // generation, then do the lock retries. otherwise, no lock retries
+ // are needed.
+ if ((my_retry_want - 1) == info->retry_done) {
+ for (;;) {
+ if (!info->running_retry) {
+ info->running_retry = true;
+ info->retry_done = info->retry_want;
+ toku_mutex_unlock(&info->retry_mutex);
+ retry_all_lock_requests_info(info, lock_wait_callback, callback_arg);
+ if (after_retry_all_test_callback) after_retry_all_test_callback();
+ toku_mutex_lock(&info->retry_mutex);
+ info->running_retry = false;
+ toku_cond_broadcast(&info->retry_cv);
+ break;
+ } else {
+ toku_cond_wait(&info->retry_cv, &info->retry_mutex);
+ }
+ }
+ }
+ toku_mutex_unlock(&info->retry_mutex);
+}
+
+void lock_request::retry_all_lock_requests_info(
+ lt_lock_request_info *info,
+ void (*lock_wait_callback)(void *, lock_wait_infos *), void *callback_arg) {
+ toku_external_mutex_lock(&info->mutex);
+ // retry all of the pending lock requests.
+ lock_wait_infos conflicts_collector;
+ for (uint32_t i = 0; i < info->pending_lock_requests.size();) {
+ lock_request *request;
+ int r = info->pending_lock_requests.fetch(i, &request);
+ invariant_zero(r);
+
+ // retry the lock request. if it didn't succeed,
+ // move on to the next lock request. otherwise
+ // the request is gone from the list so we may
+ // read the i'th entry for the next one.
+ r = request->retry(&conflicts_collector);
+ if (r != 0) {
+ i++;
+ }
+ }
+
+ // call report_waits while holding the pending queue lock since
+ // the waiter object is still valid while it's in the queue
+ report_waits(&conflicts_collector, lock_wait_callback, callback_arg);
+
+ // future threads should only retry lock requests if some still exist
+ info->should_retry_lock_requests = info->pending_lock_requests.size() > 0;
+ toku_external_mutex_unlock(&info->mutex);
+}
+
+void lock_request::add_conflicts_to_waits(txnid_set *conflicts,
+ lock_wait_infos *wait_conflicts) {
+ wait_conflicts->push_back({m_lt, get_txnid(), m_extra, {}});
+ uint32_t num_conflicts = conflicts->size();
+ for (uint32_t i = 0; i < num_conflicts; i++) {
+ wait_conflicts->back().waitees.push_back(conflicts->get(i));
+ }
+}
+
+void lock_request::report_waits(lock_wait_infos *wait_conflicts,
+ void (*lock_wait_callback)(void *,
+ lock_wait_infos *),
+ void *callback_arg) {
+ if (lock_wait_callback) (*lock_wait_callback)(callback_arg, wait_conflicts);
+}
+
+void *lock_request::get_extra(void) const { return m_extra; }
+
+void lock_request::kill_waiter(void) {
+ remove_from_lock_requests();
+ complete(DB_LOCK_NOTGRANTED);
+ toku_external_cond_broadcast(&m_wait_cond);
+}
+
+void lock_request::kill_waiter(locktree *lt, void *extra) {
+ lt_lock_request_info *info = lt->get_lock_request_info();
+ toku_external_mutex_lock(&info->mutex);
+ for (uint32_t i = 0; i < info->pending_lock_requests.size(); i++) {
+ lock_request *request;
+ int r = info->pending_lock_requests.fetch(i, &request);
+ if (r == 0 && request->get_extra() == extra) {
+ request->kill_waiter();
+ break;
+ }
+ }
+ toku_external_mutex_unlock(&info->mutex);
+}
+
+// find another lock request by txnid. must hold the mutex.
+lock_request *lock_request::find_lock_request(const TXNID &txnid) {
+ lock_request *request;
+ int r = m_info->pending_lock_requests.find_zero<TXNID, find_by_txnid>(
+ txnid, &request, nullptr);
+ if (r != 0) {
+ request = nullptr;
+ }
+ return request;
+}
+
+// insert this lock request into the locktree's set. must hold the mutex.
+void lock_request::insert_into_lock_requests(void) {
+ uint32_t idx;
+ lock_request *request;
+ int r = m_info->pending_lock_requests.find_zero<TXNID, find_by_txnid>(
+ m_txnid, &request, &idx);
+ invariant(r == DB_NOTFOUND);
+ r = m_info->pending_lock_requests.insert_at(this, idx);
+ invariant_zero(r);
+ m_info->pending_is_empty = false;
+}
+
+// remove this lock request from the locktree's set. must hold the mutex.
+void lock_request::remove_from_lock_requests(void) {
+ uint32_t idx;
+ lock_request *request;
+ int r = m_info->pending_lock_requests.find_zero<TXNID, find_by_txnid>(
+ m_txnid, &request, &idx);
+ invariant_zero(r);
+ invariant(request == this);
+ r = m_info->pending_lock_requests.delete_at(idx);
+ invariant_zero(r);
+ if (m_info->pending_lock_requests.size() == 0)
+ m_info->pending_is_empty = true;
+}
+
+int lock_request::find_by_txnid(lock_request *const &request,
+ const TXNID &txnid) {
+ TXNID request_txnid = request->m_txnid;
+ if (request_txnid < txnid) {
+ return -1;
+ } else if (request_txnid == txnid) {
+ return 0;
+ } else {
+ return 1;
+ }
+}
+
+void lock_request::set_start_test_callback(void (*f)(void)) {
+ m_start_test_callback = f;
+}
+
+void lock_request::set_start_before_pending_test_callback(void (*f)(void)) {
+ m_start_before_pending_test_callback = f;
+}
+
+void lock_request::set_retry_test_callback(void (*f)(void)) {
+ m_retry_test_callback = f;
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.h
new file mode 100644
index 000000000..d30e1e2ca
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.h
@@ -0,0 +1,255 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=2:softtabstop=2:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include "../db.h"
+#include "../ft/comparator.h"
+#include "../portability/toku_pthread.h"
+#include "locktree.h"
+#include "txnid_set.h"
+#include "wfg.h"
+
+namespace toku {
+
+// Information about a lock wait
+struct lock_wait_info {
+ locktree *ltree; // the tree where wait happens
+ TXNID waiter; // the waiting transaction
+ void *m_extra; // lock_request's m_extra
+
+ // The transactions that are waited for.
+ std::vector<TXNID> waitees;
+};
+
+typedef std::vector<lock_wait_info> lock_wait_infos;
+
+// A lock request contains the db, the key range, the lock type, and
+// the transaction id that describes a potential row range lock.
+//
+// the typical use case is:
+// - initialize a lock request
+// - start to try to acquire the lock
+// - do something else
+// - wait for the lock request to be resolved on a timed condition
+// - destroy the lock request
+// a lock request is resolved when its state is no longer pending, or
+// when it becomes granted, or timedout, or deadlocked. when resolved, the
+// state of the lock request is changed and any waiting threads are awakened.
+
+class lock_request {
+ public:
+ enum type { UNKNOWN, READ, WRITE };
+
+ // effect: Initializes a lock request.
+ void create(toku_external_mutex_factory_t mutex_factory);
+
+ // effect: Destroys a lock request.
+ void destroy(void);
+
+ // effect: Resets the lock request parameters, allowing it to be reused.
+ // requires: Lock request was already created at some point
+ void set(locktree *lt, TXNID txnid, const DBT *left_key, const DBT *right_key,
+ type lock_type, bool big_txn, void *extra = nullptr);
+
+ // effect: Tries to acquire a lock described by this lock request.
+ // returns: The return code of locktree::acquire_[write,read]_lock()
+ // or DB_LOCK_DEADLOCK if this request would end up deadlocked.
+ int start(void);
+
+ // effect: Sleeps until either the request is granted or the wait time
+ // expires. returns: The return code of locktree::acquire_[write,read]_lock()
+ // or simply DB_LOCK_NOTGRANTED if the wait time expired.
+ int wait(uint64_t wait_time_ms);
+ int wait(uint64_t wait_time_ms, uint64_t killed_time_ms,
+ int (*killed_callback)(void),
+ void (*lock_wait_callback)(void *, lock_wait_infos *) = nullptr,
+ void *callback_arg = nullptr);
+
+ // return: left end-point of the lock range
+ const DBT *get_left_key(void) const;
+
+ // return: right end-point of the lock range
+ const DBT *get_right_key(void) const;
+
+ // return: the txnid waiting for a lock
+ TXNID get_txnid(void) const;
+
+ // return: when this lock request started, as milliseconds from epoch
+ uint64_t get_start_time(void) const;
+
+ // return: which txnid is blocking this request (there may be more, though)
+ TXNID get_conflicting_txnid(void) const;
+
+ // effect: Retries all of the lock requests for the given locktree.
+ // Any lock requests successfully restarted is completed and woken
+ // up.
+ // The rest remain pending.
+ static void retry_all_lock_requests(
+ locktree *lt,
+ void (*lock_wait_callback)(void *, lock_wait_infos *) = nullptr,
+ void *callback_arg = nullptr,
+ void (*after_retry_test_callback)(void) = nullptr);
+ static void retry_all_lock_requests_info(
+ lt_lock_request_info *info,
+ void (*lock_wait_callback)(void *, lock_wait_infos *),
+ void *callback_arg);
+
+ void set_start_test_callback(void (*f)(void));
+ void set_start_before_pending_test_callback(void (*f)(void));
+ void set_retry_test_callback(void (*f)(void));
+
+ void *get_extra(void) const;
+
+ void kill_waiter(void);
+ static void kill_waiter(locktree *lt, void *extra);
+
+ private:
+ enum state {
+ UNINITIALIZED,
+ INITIALIZED,
+ PENDING,
+ COMPLETE,
+ DESTROYED,
+ };
+
+ // The keys for a lock request are stored "unowned" in m_left_key
+ // and m_right_key. When the request is about to go to sleep, it
+ // copies these keys and stores them in m_left_key_copy etc and
+ // sets the temporary pointers to null.
+ TXNID m_txnid;
+ TXNID m_conflicting_txnid;
+ uint64_t m_start_time;
+ const DBT *m_left_key;
+ const DBT *m_right_key;
+ DBT m_left_key_copy;
+ DBT m_right_key_copy;
+
+ // The lock request type and associated locktree
+ type m_type;
+ locktree *m_lt;
+
+ // If the lock request is in the completed state, then its
+ // final return value is stored in m_complete_r
+ int m_complete_r;
+ state m_state;
+
+ toku_external_cond_t m_wait_cond;
+
+ bool m_big_txn;
+
+ // the lock request info state stored in the
+ // locktree that this lock request is for.
+ struct lt_lock_request_info *m_info;
+
+ void *m_extra;
+
+ // effect: tries again to acquire the lock described by this lock request
+ // returns: 0 if retrying the request succeeded and is now complete
+ int retry(lock_wait_infos *collector);
+
+ void complete(int complete_r);
+
+ // effect: Finds another lock request by txnid.
+ // requires: The lock request info mutex is held
+ lock_request *find_lock_request(const TXNID &txnid);
+
+ // effect: Insert this lock request into the locktree's set.
+ // requires: the locktree's mutex is held
+ void insert_into_lock_requests(void);
+
+ // effect: Removes this lock request from the locktree's set.
+ // requires: The lock request info mutex is held
+ void remove_from_lock_requests(void);
+
+ // effect: Asks this request's locktree which txnids are preventing
+ // us from getting the lock described by this request.
+ // returns: conflicts is populated with the txnid's that this request
+ // is blocked on
+ void get_conflicts(txnid_set *conflicts);
+
+ // effect: Builds a wait-for-graph for this lock request and the given
+ // conflict set
+ void build_wait_graph(wfg *wait_graph, const txnid_set &conflicts);
+
+ // returns: True if this lock request is in deadlock with the given conflicts
+ // set
+ bool deadlock_exists(const txnid_set &conflicts);
+
+ void copy_keys(void);
+
+ static int find_by_txnid(lock_request *const &request, const TXNID &txnid);
+
+ // Report list of conflicts to lock wait callback.
+ static void report_waits(lock_wait_infos *wait_conflicts,
+ void (*lock_wait_callback)(void *,
+ lock_wait_infos *),
+ void *callback_arg);
+ void add_conflicts_to_waits(txnid_set *conflicts,
+ lock_wait_infos *wait_conflicts);
+
+ void (*m_start_test_callback)(void);
+ void (*m_start_before_pending_test_callback)(void);
+ void (*m_retry_test_callback)(void);
+
+ public:
+ std::function<void(TXNID, bool, const DBT *, const DBT *)> m_deadlock_cb;
+
+ friend class lock_request_unit_test;
+};
+// PORT: lock_request is not a POD anymore due to use of toku_external_cond_t
+// This is ok as the PODness is not really required: lock_request objects are
+// not moved in memory or anything.
+// ENSURE_POD(lock_request);
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.cc
new file mode 100644
index 000000000..3d6a590c7
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.cc
@@ -0,0 +1,1023 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=2:softtabstop=2:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "locktree.h"
+
+#include <memory.h>
+
+#include "../portability/toku_pthread.h"
+#include "../portability/toku_time.h"
+#include "../util/growable_array.h"
+#include "range_buffer.h"
+
+// including the concurrent_tree here expands the templates
+// and "defines" the implementation, so we do it here in
+// the locktree source file instead of the header.
+#include "concurrent_tree.h"
+
+namespace toku {
+// A locktree represents the set of row locks owned by all transactions
+// over an open dictionary. Read and write ranges are represented as
+// a left and right key which are compared with the given descriptor
+// and comparison fn.
+//
+// Each locktree has a reference count which it manages
+// but does nothing based on the value of the reference count - it is
+// up to the user of the locktree to destroy it when it sees fit.
+
+void locktree::create(locktree_manager *mgr, DICTIONARY_ID dict_id,
+ const comparator &cmp,
+ toku_external_mutex_factory_t mutex_factory) {
+ m_mgr = mgr;
+ m_dict_id = dict_id;
+
+ m_cmp.create_from(cmp);
+ m_reference_count = 1;
+ m_userdata = nullptr;
+
+ XCALLOC(m_rangetree);
+ m_rangetree->create(&m_cmp);
+
+ m_sto_txnid = TXNID_NONE;
+ m_sto_buffer.create();
+ m_sto_score = STO_SCORE_THRESHOLD;
+ m_sto_end_early_count = 0;
+ m_sto_end_early_time = 0;
+
+ m_escalation_barrier = [](const DBT *, const DBT *, void *) -> bool {
+ return false;
+ };
+
+ m_lock_request_info.init(mutex_factory);
+}
+
+void locktree::set_escalation_barrier_func(
+ lt_escalation_barrier_check_func func, void *extra) {
+ m_escalation_barrier = func;
+ m_escalation_barrier_arg = extra;
+}
+
+void lt_lock_request_info::init(toku_external_mutex_factory_t mutex_factory) {
+ pending_lock_requests.create();
+ pending_is_empty = true;
+ toku_external_mutex_init(mutex_factory, &mutex);
+ retry_want = retry_done = 0;
+ ZERO_STRUCT(counters);
+ ZERO_STRUCT(retry_mutex);
+ toku_mutex_init(locktree_request_info_retry_mutex_key, &retry_mutex, nullptr);
+ toku_cond_init(locktree_request_info_retry_cv_key, &retry_cv, nullptr);
+ running_retry = false;
+
+ TOKU_VALGRIND_HG_DISABLE_CHECKING(&pending_is_empty,
+ sizeof(pending_is_empty));
+ TOKU_DRD_IGNORE_VAR(pending_is_empty);
+}
+
+void locktree::destroy(void) {
+ invariant(m_reference_count == 0);
+ invariant(m_lock_request_info.pending_lock_requests.size() == 0);
+ m_cmp.destroy();
+ m_rangetree->destroy();
+ toku_free(m_rangetree);
+ m_sto_buffer.destroy();
+ m_lock_request_info.destroy();
+}
+
+void lt_lock_request_info::destroy(void) {
+ pending_lock_requests.destroy();
+ toku_external_mutex_destroy(&mutex);
+ toku_mutex_destroy(&retry_mutex);
+ toku_cond_destroy(&retry_cv);
+}
+
+void locktree::add_reference(void) {
+ (void)toku_sync_add_and_fetch(&m_reference_count, 1);
+}
+
+uint32_t locktree::release_reference(void) {
+ return toku_sync_sub_and_fetch(&m_reference_count, 1);
+}
+
+uint32_t locktree::get_reference_count(void) { return m_reference_count; }
+
+// a container for a range/txnid pair
+struct row_lock {
+ keyrange range;
+ TXNID txnid;
+ bool is_shared;
+ TxnidVector *owners;
+};
+
+// iterate over a locked keyrange and copy out all of the data,
+// storing each row lock into the given growable array. the
+// caller does not own the range inside the returned row locks,
+// so remove from the tree with care using them as keys.
+static void iterate_and_get_overlapping_row_locks(
+ const concurrent_tree::locked_keyrange *lkr,
+ GrowableArray<row_lock> *row_locks) {
+ struct copy_fn_obj {
+ GrowableArray<row_lock> *row_locks;
+ bool fn(const keyrange &range, TXNID txnid, bool is_shared,
+ TxnidVector *owners) {
+ row_lock lock = {.range = range,
+ .txnid = txnid,
+ .is_shared = is_shared,
+ .owners = owners};
+ row_locks->push(lock);
+ return true;
+ }
+ } copy_fn;
+ copy_fn.row_locks = row_locks;
+ lkr->iterate(&copy_fn);
+}
+
+// given a txnid and a set of overlapping row locks, determine
+// which txnids are conflicting, and store them in the conflicts
+// set, if given.
+static bool determine_conflicting_txnids(
+ const GrowableArray<row_lock> &row_locks, const TXNID &txnid,
+ txnid_set *conflicts) {
+ bool conflicts_exist = false;
+ const size_t num_overlaps = row_locks.get_size();
+ for (size_t i = 0; i < num_overlaps; i++) {
+ const row_lock lock = row_locks.fetch_unchecked(i);
+ const TXNID other_txnid = lock.txnid;
+ if (other_txnid != txnid) {
+ if (conflicts) {
+ if (other_txnid == TXNID_SHARED) {
+ // Add all shared lock owners, except this transaction.
+ for (TXNID shared_id : *lock.owners) {
+ if (shared_id != txnid) conflicts->add(shared_id);
+ }
+ } else {
+ conflicts->add(other_txnid);
+ }
+ }
+ conflicts_exist = true;
+ }
+ }
+ return conflicts_exist;
+}
+
+// how much memory does a row lock take up in a concurrent tree?
+static uint64_t row_lock_size_in_tree(const row_lock &lock) {
+ const uint64_t overhead = concurrent_tree::get_insertion_memory_overhead();
+ return lock.range.get_memory_size() + overhead;
+}
+
+// remove and destroy the given row lock from the locked keyrange,
+// then notify the memory tracker of the newly freed lock.
+static void remove_row_lock_from_tree(concurrent_tree::locked_keyrange *lkr,
+ const row_lock &lock, TXNID txnid,
+ locktree_manager *mgr) {
+ const uint64_t mem_released = row_lock_size_in_tree(lock);
+ lkr->remove(lock.range, txnid);
+ if (mgr != nullptr) {
+ mgr->note_mem_released(mem_released);
+ }
+}
+
+// insert a row lock into the locked keyrange, then notify
+// the memory tracker of this newly acquired lock.
+static void insert_row_lock_into_tree(concurrent_tree::locked_keyrange *lkr,
+ const row_lock &lock,
+ locktree_manager *mgr) {
+ uint64_t mem_used = row_lock_size_in_tree(lock);
+ lkr->insert(lock.range, lock.txnid, lock.is_shared);
+ if (mgr != nullptr) {
+ mgr->note_mem_used(mem_used);
+ }
+}
+
+void locktree::sto_begin(TXNID txnid) {
+ invariant(m_sto_txnid == TXNID_NONE);
+ invariant(m_sto_buffer.is_empty());
+ m_sto_txnid = txnid;
+}
+
+void locktree::sto_append(const DBT *left_key, const DBT *right_key,
+ bool is_write_request) {
+ uint64_t buffer_mem, delta;
+
+ // psergey: the below two lines do not make any sense
+ // (and it's the same in upstream TokuDB)
+ keyrange range;
+ range.create(left_key, right_key);
+
+ buffer_mem = m_sto_buffer.total_memory_size();
+ m_sto_buffer.append(left_key, right_key, is_write_request);
+ delta = m_sto_buffer.total_memory_size() - buffer_mem;
+ if (m_mgr != nullptr) {
+ m_mgr->note_mem_used(delta);
+ }
+}
+
+void locktree::sto_end(void) {
+ uint64_t mem_size = m_sto_buffer.total_memory_size();
+ if (m_mgr != nullptr) {
+ m_mgr->note_mem_released(mem_size);
+ }
+ m_sto_buffer.destroy();
+ m_sto_buffer.create();
+ m_sto_txnid = TXNID_NONE;
+}
+
+void locktree::sto_end_early_no_accounting(void *prepared_lkr) {
+ sto_migrate_buffer_ranges_to_tree(prepared_lkr);
+ sto_end();
+ toku_unsafe_set(m_sto_score, 0);
+}
+
+void locktree::sto_end_early(void *prepared_lkr) {
+ m_sto_end_early_count++;
+
+ tokutime_t t0 = toku_time_now();
+ sto_end_early_no_accounting(prepared_lkr);
+ tokutime_t t1 = toku_time_now();
+
+ m_sto_end_early_time += (t1 - t0);
+}
+
+void locktree::sto_migrate_buffer_ranges_to_tree(void *prepared_lkr) {
+ // There should be something to migrate, and nothing in the rangetree.
+ invariant(!m_sto_buffer.is_empty());
+ invariant(m_rangetree->is_empty());
+
+ concurrent_tree sto_rangetree;
+ concurrent_tree::locked_keyrange sto_lkr;
+ sto_rangetree.create(&m_cmp);
+
+ // insert all of the ranges from the single txnid buffer into a new rangtree
+ range_buffer::iterator iter(&m_sto_buffer);
+ range_buffer::iterator::record rec;
+ while (iter.current(&rec)) {
+ sto_lkr.prepare(&sto_rangetree);
+ int r = acquire_lock_consolidated(&sto_lkr, m_sto_txnid, rec.get_left_key(),
+ rec.get_right_key(),
+ rec.get_exclusive_flag(), nullptr);
+ invariant_zero(r);
+ sto_lkr.release();
+ iter.next();
+ }
+
+ // Iterate the newly created rangetree and insert each range into the
+ // locktree's rangetree, on behalf of the old single txnid.
+ struct migrate_fn_obj {
+ concurrent_tree::locked_keyrange *dst_lkr;
+ bool fn(const keyrange &range, TXNID txnid, bool is_shared,
+ TxnidVector *owners) {
+ // There can't be multiple owners in STO mode
+ invariant_zero(owners);
+ dst_lkr->insert(range, txnid, is_shared);
+ return true;
+ }
+ } migrate_fn;
+ migrate_fn.dst_lkr =
+ static_cast<concurrent_tree::locked_keyrange *>(prepared_lkr);
+ sto_lkr.prepare(&sto_rangetree);
+ sto_lkr.iterate(&migrate_fn);
+ sto_lkr.remove_all();
+ sto_lkr.release();
+ sto_rangetree.destroy();
+ invariant(!m_rangetree->is_empty());
+}
+
+bool locktree::sto_try_acquire(void *prepared_lkr, TXNID txnid,
+ const DBT *left_key, const DBT *right_key,
+ bool is_write_request) {
+ if (m_rangetree->is_empty() && m_sto_buffer.is_empty() &&
+ toku_unsafe_fetch(m_sto_score) >= STO_SCORE_THRESHOLD) {
+ // We can do the optimization because the rangetree is empty, and
+ // we know its worth trying because the sto score is big enough.
+ sto_begin(txnid);
+ } else if (m_sto_txnid != TXNID_NONE) {
+ // We are currently doing the optimization. Check if we need to cancel
+ // it because a new txnid appeared, or if the current single txnid has
+ // taken too many locks already.
+ if (m_sto_txnid != txnid ||
+ m_sto_buffer.get_num_ranges() > STO_BUFFER_MAX_SIZE) {
+ sto_end_early(prepared_lkr);
+ }
+ }
+
+ // At this point the sto txnid is properly set. If it is valid, then
+ // this txnid can append its lock to the sto buffer successfully.
+ if (m_sto_txnid != TXNID_NONE) {
+ invariant(m_sto_txnid == txnid);
+ sto_append(left_key, right_key, is_write_request);
+ return true;
+ } else {
+ invariant(m_sto_buffer.is_empty());
+ return false;
+ }
+}
+
+/*
+ Do the same as iterate_and_get_overlapping_row_locks does, but also check for
+ this:
+ The set of overlapping rows locks consists of just one read-only shared
+ lock with the same endpoints as specified (in that case, we can just add
+ ourselves into that list)
+
+ @return true - One compatible shared lock
+ false - Otherwise
+*/
+static bool iterate_and_get_overlapping_row_locks2(
+ const concurrent_tree::locked_keyrange *lkr, const DBT *left_key,
+ const DBT *right_key, comparator *cmp, TXNID,
+ GrowableArray<row_lock> *row_locks) {
+ struct copy_fn_obj {
+ GrowableArray<row_lock> *row_locks;
+ bool first_call = true;
+ bool matching_lock_found = false;
+ const DBT *left_key, *right_key;
+ comparator *cmp;
+
+ bool fn(const keyrange &range, TXNID txnid, bool is_shared,
+ TxnidVector *owners) {
+ if (first_call) {
+ first_call = false;
+ if (is_shared && !(*cmp)(left_key, range.get_left_key()) &&
+ !(*cmp)(right_key, range.get_right_key())) {
+ matching_lock_found = true;
+ }
+ } else {
+ // if we see multiple matching locks, it doesn't matter whether
+ // the first one was matching.
+ matching_lock_found = false;
+ }
+ row_lock lock = {.range = range,
+ .txnid = txnid,
+ .is_shared = is_shared,
+ .owners = owners};
+ row_locks->push(lock);
+ return true;
+ }
+ } copy_fn;
+ copy_fn.row_locks = row_locks;
+ copy_fn.left_key = left_key;
+ copy_fn.right_key = right_key;
+ copy_fn.cmp = cmp;
+ lkr->iterate(&copy_fn);
+ return copy_fn.matching_lock_found;
+}
+
+// try to acquire a lock and consolidate it with existing locks if possible
+// param: lkr, a prepared locked keyrange
+// return: 0 on success, DB_LOCK_NOTGRANTED if conflicting locks exist.
+int locktree::acquire_lock_consolidated(void *prepared_lkr, TXNID txnid,
+ const DBT *left_key,
+ const DBT *right_key,
+ bool is_write_request,
+ txnid_set *conflicts) {
+ int r = 0;
+ concurrent_tree::locked_keyrange *lkr;
+
+ keyrange requested_range;
+ requested_range.create(left_key, right_key);
+ lkr = static_cast<concurrent_tree::locked_keyrange *>(prepared_lkr);
+ lkr->acquire(requested_range);
+
+ // copy out the set of overlapping row locks.
+ GrowableArray<row_lock> overlapping_row_locks;
+ overlapping_row_locks.init();
+ bool matching_shared_lock_found = false;
+
+ if (is_write_request)
+ iterate_and_get_overlapping_row_locks(lkr, &overlapping_row_locks);
+ else {
+ matching_shared_lock_found = iterate_and_get_overlapping_row_locks2(
+ lkr, left_key, right_key, &m_cmp, txnid, &overlapping_row_locks);
+ // psergey-todo: what to do now? So, we have figured we have just one
+ // shareable lock. Need to add us into it as an owner but the lock
+ // pointer cannot be kept?
+ // A: use find_node_with_overlapping_child(key_range, nullptr);
+ // then, add ourselves to the owner list.
+ // Dont' foreget to release the subtree after that.
+ }
+
+ if (matching_shared_lock_found) {
+ // there is just one non-confliting matching shared lock.
+ // we are hilding a lock on it (see acquire() call above).
+ // we need to modify it to indicate there is another locker...
+ if (lkr->add_shared_owner(requested_range, txnid)) {
+ // Pretend shared lock uses as much memory.
+ row_lock new_lock = {.range = requested_range,
+ .txnid = txnid,
+ .is_shared = false,
+ .owners = nullptr};
+ uint64_t mem_used = row_lock_size_in_tree(new_lock);
+ if (m_mgr) {
+ m_mgr->note_mem_used(mem_used);
+ }
+ }
+ requested_range.destroy();
+ overlapping_row_locks.deinit();
+ return 0;
+ }
+
+ size_t num_overlapping_row_locks = overlapping_row_locks.get_size();
+
+ // if any overlapping row locks conflict with this request, bail out.
+
+ bool conflicts_exist =
+ determine_conflicting_txnids(overlapping_row_locks, txnid, conflicts);
+ if (!conflicts_exist) {
+ // there are no conflicts, so all of the overlaps are for the requesting
+ // txnid. so, we must consolidate all existing overlapping ranges and the
+ // requested range into one dominating range. then we insert the dominating
+ // range.
+ bool all_shared = !is_write_request;
+ for (size_t i = 0; i < num_overlapping_row_locks; i++) {
+ row_lock overlapping_lock = overlapping_row_locks.fetch_unchecked(i);
+ invariant(overlapping_lock.txnid == txnid);
+ requested_range.extend(m_cmp, overlapping_lock.range);
+ remove_row_lock_from_tree(lkr, overlapping_lock, TXNID_ANY, m_mgr);
+ all_shared = all_shared && overlapping_lock.is_shared;
+ }
+
+ row_lock new_lock = {.range = requested_range,
+ .txnid = txnid,
+ .is_shared = all_shared,
+ .owners = nullptr};
+ insert_row_lock_into_tree(lkr, new_lock, m_mgr);
+ } else {
+ r = DB_LOCK_NOTGRANTED;
+ }
+
+ requested_range.destroy();
+ overlapping_row_locks.deinit();
+ return r;
+}
+
+// acquire a lock in the given key range, inclusive. if successful,
+// return 0. otherwise, populate the conflicts txnid_set with the set of
+// transactions that conflict with this request.
+int locktree::acquire_lock(bool is_write_request, TXNID txnid,
+ const DBT *left_key, const DBT *right_key,
+ txnid_set *conflicts) {
+ int r = 0;
+
+ // we are only supporting write locks for simplicity
+ // invariant(is_write_request);
+
+ // acquire and prepare a locked keyrange over the requested range.
+ // prepare is a serialzation point, so we take the opportunity to
+ // try the single txnid optimization first.
+ concurrent_tree::locked_keyrange lkr;
+ lkr.prepare(m_rangetree);
+
+ bool acquired =
+ sto_try_acquire(&lkr, txnid, left_key, right_key, is_write_request);
+ if (!acquired) {
+ r = acquire_lock_consolidated(&lkr, txnid, left_key, right_key,
+ is_write_request, conflicts);
+ }
+
+ lkr.release();
+ return r;
+}
+
+int locktree::try_acquire_lock(bool is_write_request, TXNID txnid,
+ const DBT *left_key, const DBT *right_key,
+ txnid_set *conflicts, bool big_txn) {
+ // All ranges in the locktree must have left endpoints <= right endpoints.
+ // Range comparisons rely on this fact, so we make a paranoid invariant here.
+ paranoid_invariant(m_cmp(left_key, right_key) <= 0);
+ int r = m_mgr == nullptr ? 0 : m_mgr->check_current_lock_constraints(big_txn);
+ if (r == 0) {
+ r = acquire_lock(is_write_request, txnid, left_key, right_key, conflicts);
+ }
+ return r;
+}
+
+// the locktree silently upgrades read locks to write locks for simplicity
+int locktree::acquire_read_lock(TXNID txnid, const DBT *left_key,
+ const DBT *right_key, txnid_set *conflicts,
+ bool big_txn) {
+ return try_acquire_lock(false, txnid, left_key, right_key, conflicts,
+ big_txn);
+}
+
+int locktree::acquire_write_lock(TXNID txnid, const DBT *left_key,
+ const DBT *right_key, txnid_set *conflicts,
+ bool big_txn) {
+ return try_acquire_lock(true, txnid, left_key, right_key, conflicts, big_txn);
+}
+
+// typedef void (*dump_callback)(void *cdata, const DBT *left, const DBT *right,
+// TXNID txnid);
+void locktree::dump_locks(void *cdata, dump_callback cb) {
+ concurrent_tree::locked_keyrange lkr;
+ keyrange range;
+ range.create(toku_dbt_negative_infinity(), toku_dbt_positive_infinity());
+
+ lkr.prepare(m_rangetree);
+ lkr.acquire(range);
+
+ TXNID sto_txn;
+ if ((sto_txn = toku_unsafe_fetch(m_sto_txnid)) != TXNID_NONE) {
+ // insert all of the ranges from the single txnid buffer into a new rangtree
+ range_buffer::iterator iter(&m_sto_buffer);
+ range_buffer::iterator::record rec;
+ while (iter.current(&rec)) {
+ (*cb)(cdata, rec.get_left_key(), rec.get_right_key(), sto_txn,
+ !rec.get_exclusive_flag(), nullptr);
+ iter.next();
+ }
+ } else {
+ GrowableArray<row_lock> all_locks;
+ all_locks.init();
+ iterate_and_get_overlapping_row_locks(&lkr, &all_locks);
+
+ const size_t n_locks = all_locks.get_size();
+ for (size_t i = 0; i < n_locks; i++) {
+ const row_lock lock = all_locks.fetch_unchecked(i);
+ (*cb)(cdata, lock.range.get_left_key(), lock.range.get_right_key(),
+ lock.txnid, lock.is_shared, lock.owners);
+ }
+ all_locks.deinit();
+ }
+ lkr.release();
+ range.destroy();
+}
+
+void locktree::get_conflicts(bool is_write_request, TXNID txnid,
+ const DBT *left_key, const DBT *right_key,
+ txnid_set *conflicts) {
+ // because we only support write locks, ignore this bit for now.
+ (void)is_write_request;
+
+ // preparing and acquire a locked keyrange over the range
+ keyrange range;
+ range.create(left_key, right_key);
+ concurrent_tree::locked_keyrange lkr;
+ lkr.prepare(m_rangetree);
+ lkr.acquire(range);
+
+ // copy out the set of overlapping row locks and determine the conflicts
+ GrowableArray<row_lock> overlapping_row_locks;
+ overlapping_row_locks.init();
+ iterate_and_get_overlapping_row_locks(&lkr, &overlapping_row_locks);
+
+ // we don't care if conflicts exist. we just want the conflicts set populated.
+ (void)determine_conflicting_txnids(overlapping_row_locks, txnid, conflicts);
+
+ lkr.release();
+ overlapping_row_locks.deinit();
+ range.destroy();
+}
+
+// Effect:
+// For each range in the lock tree that overlaps the given range and has
+// the given txnid, remove it.
+// Rationale:
+// In the common case, there is only the range [left_key, right_key] and
+// it is associated with txnid, so this is a single tree delete.
+//
+// However, consolidation and escalation change the objects in the tree
+// without telling the txn anything. In this case, the txn may own a
+// large range lock that represents its ownership of many smaller range
+// locks. For example, the txn may think it owns point locks on keys 1,
+// 2, and 3, but due to escalation, only the object [1,3] exists in the
+// tree.
+//
+// The first call for a small lock will remove the large range lock, and
+// the rest of the calls should do nothing. After the first release,
+// another thread can acquire one of the locks that the txn thinks it
+// still owns. That's ok, because the txn doesn't want it anymore (it
+// unlocks everything at once), but it may find a lock that it does not
+// own.
+//
+// In our example, the txn unlocks key 1, which actually removes the
+// whole lock [1,3]. Now, someone else can lock 2 before our txn gets
+// around to unlocking 2, so we should not remove that lock.
+void locktree::remove_overlapping_locks_for_txnid(TXNID txnid,
+ const DBT *left_key,
+ const DBT *right_key) {
+ keyrange release_range;
+ release_range.create(left_key, right_key);
+
+ // acquire and prepare a locked keyrange over the release range
+ concurrent_tree::locked_keyrange lkr;
+ lkr.prepare(m_rangetree);
+ lkr.acquire(release_range);
+
+ // copy out the set of overlapping row locks.
+ GrowableArray<row_lock> overlapping_row_locks;
+ overlapping_row_locks.init();
+ iterate_and_get_overlapping_row_locks(&lkr, &overlapping_row_locks);
+ size_t num_overlapping_row_locks = overlapping_row_locks.get_size();
+
+ for (size_t i = 0; i < num_overlapping_row_locks; i++) {
+ row_lock lock = overlapping_row_locks.fetch_unchecked(i);
+ // If this isn't our lock, that's ok, just don't remove it.
+ // See rationale above.
+ // psergey-todo: for shared locks, just remove ourselves from the
+ // owners.
+ if (lock.txnid == txnid || (lock.owners && lock.owners->contains(txnid))) {
+ remove_row_lock_from_tree(&lkr, lock, txnid, m_mgr);
+ }
+ }
+
+ lkr.release();
+ overlapping_row_locks.deinit();
+ release_range.destroy();
+}
+
+bool locktree::sto_txnid_is_valid_unsafe(void) const {
+ return toku_unsafe_fetch(m_sto_txnid) != TXNID_NONE;
+}
+
+int locktree::sto_get_score_unsafe(void) const {
+ return toku_unsafe_fetch(m_sto_score);
+}
+
+bool locktree::sto_try_release(TXNID txnid) {
+ bool released = false;
+ if (toku_unsafe_fetch(m_sto_txnid) != TXNID_NONE) {
+ // check the bit again with a prepared locked keyrange,
+ // which protects the optimization bits and rangetree data
+ concurrent_tree::locked_keyrange lkr;
+ lkr.prepare(m_rangetree);
+ if (m_sto_txnid != TXNID_NONE) {
+ // this txnid better be the single txnid on this locktree,
+ // or else we are in big trouble (meaning the logic is broken)
+ invariant(m_sto_txnid == txnid);
+ invariant(m_rangetree->is_empty());
+ sto_end();
+ released = true;
+ }
+ lkr.release();
+ }
+ return released;
+}
+
+// release all of the locks for a txnid whose endpoints are pairs
+// in the given range buffer.
+void locktree::release_locks(TXNID txnid, const range_buffer *ranges,
+ bool all_trx_locks_hint) {
+ // try the single txn optimization. if it worked, then all of the
+ // locks are already released, otherwise we need to do it here.
+ bool released;
+ if (all_trx_locks_hint) {
+ // This will release all of the locks the transaction is holding
+ released = sto_try_release(txnid);
+ } else {
+ /*
+ psergey: we are asked to release *Some* of the locks the transaction
+ is holding.
+ We could try doing that without leaving the STO mode, but right now,
+ the easiest way is to exit the STO mode and let the non-STO code path
+ handle it.
+ */
+ if (toku_unsafe_fetch(m_sto_txnid) != TXNID_NONE) {
+ // check the bit again with a prepared locked keyrange,
+ // which protects the optimization bits and rangetree data
+ concurrent_tree::locked_keyrange lkr;
+ lkr.prepare(m_rangetree);
+ if (m_sto_txnid != TXNID_NONE) {
+ sto_end_early(&lkr);
+ }
+ lkr.release();
+ }
+ released = false;
+ }
+ if (!released) {
+ range_buffer::iterator iter(ranges);
+ range_buffer::iterator::record rec;
+ while (iter.current(&rec)) {
+ const DBT *left_key = rec.get_left_key();
+ const DBT *right_key = rec.get_right_key();
+ // All ranges in the locktree must have left endpoints <= right endpoints.
+ // Range comparisons rely on this fact, so we make a paranoid invariant
+ // here.
+ paranoid_invariant(m_cmp(left_key, right_key) <= 0);
+ remove_overlapping_locks_for_txnid(txnid, left_key, right_key);
+ iter.next();
+ }
+ // Increase the sto score slightly. Eventually it will hit
+ // the threshold and we'll try the optimization again. This
+ // is how a previously multithreaded system transitions into
+ // a single threaded system that benefits from the optimization.
+ if (toku_unsafe_fetch(m_sto_score) < STO_SCORE_THRESHOLD) {
+ toku_sync_fetch_and_add(&m_sto_score, 1);
+ }
+ }
+}
+
+// iterate over a locked keyrange and extract copies of the first N
+// row locks, storing each one into the given array of size N,
+// then removing each extracted lock from the locked keyrange.
+static int extract_first_n_row_locks(concurrent_tree::locked_keyrange *lkr,
+ locktree_manager *mgr, row_lock *row_locks,
+ int num_to_extract) {
+ struct extract_fn_obj {
+ int num_extracted;
+ int num_to_extract;
+ row_lock *row_locks;
+ bool fn(const keyrange &range, TXNID txnid, bool is_shared,
+ TxnidVector *owners) {
+ if (num_extracted < num_to_extract) {
+ row_lock lock;
+ lock.range.create_copy(range);
+ lock.txnid = txnid;
+ lock.is_shared = is_shared;
+ // deep-copy the set of owners:
+ if (owners)
+ lock.owners = new TxnidVector(*owners);
+ else
+ lock.owners = nullptr;
+ row_locks[num_extracted++] = lock;
+ return true;
+ } else {
+ return false;
+ }
+ }
+ } extract_fn;
+
+ extract_fn.row_locks = row_locks;
+ extract_fn.num_to_extract = num_to_extract;
+ extract_fn.num_extracted = 0;
+ lkr->iterate(&extract_fn);
+
+ // now that the ranges have been copied out, complete
+ // the extraction by removing the ranges from the tree.
+ // use remove_row_lock_from_tree() so we properly track the
+ // amount of memory and number of locks freed.
+ int num_extracted = extract_fn.num_extracted;
+ invariant(num_extracted <= num_to_extract);
+ for (int i = 0; i < num_extracted; i++) {
+ remove_row_lock_from_tree(lkr, row_locks[i], TXNID_ANY, mgr);
+ }
+
+ return num_extracted;
+}
+
+// Store each newly escalated lock in a range buffer for appropriate txnid.
+// We'll rebuild the locktree by iterating over these ranges, and then we
+// can pass back each txnid/buffer pair individually through a callback
+// to notify higher layers that locks have changed.
+struct txnid_range_buffer {
+ TXNID txnid;
+ range_buffer buffer;
+
+ static int find_by_txnid(struct txnid_range_buffer *const &other_buffer,
+ const TXNID &txnid) {
+ if (txnid < other_buffer->txnid) {
+ return -1;
+ } else if (other_buffer->txnid == txnid) {
+ return 0;
+ } else {
+ return 1;
+ }
+ }
+};
+
+// escalate the locks in the locktree by merging adjacent
+// locks that have the same txnid into one larger lock.
+//
+// if there's only one txnid in the locktree then this
+// approach works well. if there are many txnids and each
+// has locks in a random/alternating order, then this does
+// not work so well.
+void locktree::escalate(lt_escalate_cb after_escalate_callback,
+ void *after_escalate_callback_extra) {
+ omt<struct txnid_range_buffer *, struct txnid_range_buffer *> range_buffers;
+ range_buffers.create();
+
+ // prepare and acquire a locked keyrange on the entire locktree
+ concurrent_tree::locked_keyrange lkr;
+ keyrange infinite_range = keyrange::get_infinite_range();
+ lkr.prepare(m_rangetree);
+ lkr.acquire(infinite_range);
+
+ // if we're in the single txnid optimization, simply call it off.
+ // if you have to run escalation, you probably don't care about
+ // the optimization anyway, and this makes things easier.
+ if (m_sto_txnid != TXNID_NONE) {
+ // We are already accounting for this escalation time and
+ // count, so don't do it for sto_end_early too.
+ sto_end_early_no_accounting(&lkr);
+ }
+
+ // extract and remove batches of row locks from the locktree
+ int num_extracted;
+ const int num_row_locks_per_batch = 128;
+ row_lock *XCALLOC_N(num_row_locks_per_batch, extracted_buf);
+
+ // we always remove the "first" n because we are removing n
+ // each time we do an extraction. so this loops until its empty.
+ while ((num_extracted = extract_first_n_row_locks(
+ &lkr, m_mgr, extracted_buf, num_row_locks_per_batch)) > 0) {
+ int current_index = 0;
+ while (current_index < num_extracted) {
+ // every batch of extracted locks is in range-sorted order. search
+ // through them and merge adjacent locks with the same txnid into
+ // one dominating lock and save it to a set of escalated locks.
+ //
+ // first, find the index of the next row lock that
+ // - belongs to a different txnid, or
+ // - belongs to several txnids, or
+ // - is a shared lock (we could potentially merge those but
+ // currently we don't), or
+ // - is across a lock escalation barrier.
+ int next_txnid_index = current_index + 1;
+
+ while (next_txnid_index < num_extracted &&
+ (extracted_buf[current_index].txnid ==
+ extracted_buf[next_txnid_index].txnid) &&
+ !extracted_buf[next_txnid_index].is_shared &&
+ !extracted_buf[next_txnid_index].owners &&
+ !m_escalation_barrier(
+ extracted_buf[current_index].range.get_right_key(),
+ extracted_buf[next_txnid_index].range.get_left_key(),
+ m_escalation_barrier_arg)) {
+ next_txnid_index++;
+ }
+
+ // Create an escalated range for the current txnid that dominates
+ // each range between the current indext and the next txnid's index.
+ // const TXNID current_txnid = extracted_buf[current_index].txnid;
+ const DBT *escalated_left_key =
+ extracted_buf[current_index].range.get_left_key();
+ const DBT *escalated_right_key =
+ extracted_buf[next_txnid_index - 1].range.get_right_key();
+
+ // Try to find a range buffer for the current txnid. Create one if it
+ // doesn't exist. Then, append the new escalated range to the buffer. (If
+ // a lock is shared by multiple txnids, append it each of txnid's lists)
+ TxnidVector *owners_ptr;
+ TxnidVector singleton_owner;
+ if (extracted_buf[current_index].owners)
+ owners_ptr = extracted_buf[current_index].owners;
+ else {
+ singleton_owner.insert(extracted_buf[current_index].txnid);
+ owners_ptr = &singleton_owner;
+ }
+
+ for (auto cur_txnid : *owners_ptr) {
+ uint32_t idx;
+ struct txnid_range_buffer *existing_range_buffer;
+ int r =
+ range_buffers.find_zero<TXNID, txnid_range_buffer::find_by_txnid>(
+ cur_txnid, &existing_range_buffer, &idx);
+ if (r == DB_NOTFOUND) {
+ struct txnid_range_buffer *XMALLOC(new_range_buffer);
+ new_range_buffer->txnid = cur_txnid;
+ new_range_buffer->buffer.create();
+ new_range_buffer->buffer.append(
+ escalated_left_key, escalated_right_key,
+ !extracted_buf[current_index].is_shared);
+ range_buffers.insert_at(new_range_buffer, idx);
+ } else {
+ invariant_zero(r);
+ invariant(existing_range_buffer->txnid == cur_txnid);
+ existing_range_buffer->buffer.append(
+ escalated_left_key, escalated_right_key,
+ !extracted_buf[current_index].is_shared);
+ }
+ }
+
+ current_index = next_txnid_index;
+ }
+
+ // destroy the ranges copied during the extraction
+ for (int i = 0; i < num_extracted; i++) {
+ delete extracted_buf[i].owners;
+ extracted_buf[i].range.destroy();
+ }
+ }
+ toku_free(extracted_buf);
+
+ // Rebuild the locktree from each range in each range buffer,
+ // then notify higher layers that the txnid's locks have changed.
+ //
+ // (shared locks: if a lock was initially shared between transactions TRX1,
+ // TRX2, etc, we will now try to acquire it acting on behalf on TRX1, on
+ // TRX2, etc. This will succeed and an identical shared lock will be
+ // constructed)
+
+ invariant(m_rangetree->is_empty());
+ const uint32_t num_range_buffers = range_buffers.size();
+ for (uint32_t i = 0; i < num_range_buffers; i++) {
+ struct txnid_range_buffer *current_range_buffer;
+ int r = range_buffers.fetch(i, &current_range_buffer);
+ invariant_zero(r);
+ if (r == EINVAL) // Shouldn't happen, avoid compiler warning
+ continue;
+
+ const TXNID current_txnid = current_range_buffer->txnid;
+ range_buffer::iterator iter(&current_range_buffer->buffer);
+ range_buffer::iterator::record rec;
+ while (iter.current(&rec)) {
+ keyrange range;
+ range.create(rec.get_left_key(), rec.get_right_key());
+ row_lock lock = {.range = range,
+ .txnid = current_txnid,
+ .is_shared = !rec.get_exclusive_flag(),
+ .owners = nullptr};
+ insert_row_lock_into_tree(&lkr, lock, m_mgr);
+ iter.next();
+ }
+
+ // Notify higher layers that locks have changed for the current txnid
+ if (after_escalate_callback) {
+ after_escalate_callback(current_txnid, this, current_range_buffer->buffer,
+ after_escalate_callback_extra);
+ }
+ current_range_buffer->buffer.destroy();
+ }
+
+ while (range_buffers.size() > 0) {
+ struct txnid_range_buffer *buffer;
+ int r = range_buffers.fetch(0, &buffer);
+ invariant_zero(r);
+ r = range_buffers.delete_at(0);
+ invariant_zero(r);
+ toku_free(buffer);
+ }
+ range_buffers.destroy();
+
+ lkr.release();
+}
+
+void *locktree::get_userdata(void) const { return m_userdata; }
+
+void locktree::set_userdata(void *userdata) { m_userdata = userdata; }
+
+struct lt_lock_request_info *locktree::get_lock_request_info(void) {
+ return &m_lock_request_info;
+}
+
+void locktree::set_comparator(const comparator &cmp) { m_cmp.inherit(cmp); }
+
+locktree_manager *locktree::get_manager(void) const { return m_mgr; }
+
+int locktree::compare(const locktree *lt) const {
+ if (m_dict_id.dictid < lt->m_dict_id.dictid) {
+ return -1;
+ } else if (m_dict_id.dictid == lt->m_dict_id.dictid) {
+ return 0;
+ } else {
+ return 1;
+ }
+}
+
+DICTIONARY_ID locktree::get_dict_id() const { return m_dict_id; }
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.h
new file mode 100644
index 000000000..f0f4b042d
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.h
@@ -0,0 +1,580 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <atomic>
+
+#include "../db.h"
+#include "../ft/comparator.h"
+#include "../portability/toku_external_pthread.h"
+#include "../portability/toku_pthread.h"
+#include "../portability/toku_time.h"
+// PORT #include <ft/ft-ops.h> // just for DICTIONARY_ID..
+// PORT: ft-status for LTM_STATUS:
+#include "../ft/ft-status.h"
+
+struct DICTIONARY_ID {
+ uint64_t dictid;
+};
+
+#include "../util/omt.h"
+#include "range_buffer.h"
+#include "txnid_set.h"
+#include "wfg.h"
+
+namespace toku {
+
+class locktree;
+class locktree_manager;
+class lock_request;
+class concurrent_tree;
+
+typedef int (*lt_create_cb)(locktree *lt, void *extra);
+typedef void (*lt_destroy_cb)(locktree *lt);
+typedef void (*lt_escalate_cb)(TXNID txnid, const locktree *lt,
+ const range_buffer &buffer, void *extra);
+
+typedef bool (*lt_escalation_barrier_check_func)(const DBT *a, const DBT *b,
+ void *extra);
+
+struct lt_counters {
+ uint64_t wait_count, wait_time;
+ uint64_t long_wait_count, long_wait_time;
+ uint64_t timeout_count;
+
+ void add(const lt_counters &rhs) {
+ wait_count += rhs.wait_count;
+ wait_time += rhs.wait_time;
+ long_wait_count += rhs.long_wait_count;
+ long_wait_time += rhs.long_wait_time;
+ timeout_count += rhs.timeout_count;
+ }
+};
+
+// Lock request state for some locktree
+struct lt_lock_request_info {
+ omt<lock_request *> pending_lock_requests;
+ std::atomic_bool pending_is_empty;
+ toku_external_mutex_t mutex;
+ bool should_retry_lock_requests;
+ lt_counters counters;
+ std::atomic_ullong retry_want;
+ unsigned long long retry_done;
+ toku_mutex_t retry_mutex;
+ toku_cond_t retry_cv;
+ bool running_retry;
+
+ void init(toku_external_mutex_factory_t mutex_factory);
+ void destroy(void);
+};
+
+// The locktree manager manages a set of locktrees, one for each open
+// dictionary. Locktrees are retrieved from the manager. When they are no
+// longer needed, they are be released by the user.
+class locktree_manager {
+ public:
+ // param: create_cb, called just after a locktree is first created.
+ // destroy_cb, called just before a locktree is destroyed.
+ // escalate_cb, called after a locktree is escalated (with extra
+ // param)
+ void create(lt_create_cb create_cb, lt_destroy_cb destroy_cb,
+ lt_escalate_cb escalate_cb, void *extra,
+ toku_external_mutex_factory_t mutex_factory_arg);
+
+ void destroy(void);
+
+ size_t get_max_lock_memory(void);
+
+ int set_max_lock_memory(size_t max_lock_memory);
+
+ // effect: Get a locktree from the manager. If a locktree exists with the
+ // given
+ // dict_id, it is referenced and then returned. If one did not exist,
+ // it is created. It will use the comparator for comparing keys. The
+ // on_create callback (passed to locktree_manager::create()) will be
+ // called with the given extra parameter.
+ locktree *get_lt(DICTIONARY_ID dict_id, const comparator &cmp,
+ void *on_create_extra);
+
+ void reference_lt(locktree *lt);
+
+ // effect: Releases one reference on a locktree. If the reference count
+ // transitions
+ // to zero, the on_destroy callback is called before it gets
+ // destroyed.
+ void release_lt(locktree *lt);
+
+ void get_status(LTM_STATUS status);
+
+ // effect: calls the iterate function on each pending lock request
+ // note: holds the manager's mutex
+ typedef int (*lock_request_iterate_callback)(DICTIONARY_ID dict_id,
+ TXNID txnid, const DBT *left_key,
+ const DBT *right_key,
+ TXNID blocking_txnid,
+ uint64_t start_time,
+ void *extra);
+ int iterate_pending_lock_requests(lock_request_iterate_callback cb,
+ void *extra);
+
+ // effect: Determines if too many locks or too much memory is being used,
+ // Runs escalation on the manager if so.
+ // param: big_txn, if the current transaction is 'big' (has spilled rollback
+ // logs) returns: 0 if there enough resources to create a new lock, or
+ // TOKUDB_OUT_OF_LOCKS
+ // if there are not enough resources and lock escalation failed to
+ // free up enough resources for a new lock.
+ int check_current_lock_constraints(bool big_txn);
+
+ bool over_big_threshold(void);
+
+ void note_mem_used(uint64_t mem_used);
+
+ void note_mem_released(uint64_t mem_freed);
+
+ bool out_of_locks(void) const;
+
+ // Escalate all locktrees
+ void escalate_all_locktrees(void);
+
+ // Escalate a set of locktrees
+ void escalate_locktrees(locktree **locktrees, int num_locktrees);
+
+ // effect: calls the private function run_escalation(), only ok to
+ // do for tests.
+ // rationale: to get better stress test coverage, we want a way to
+ // deterministicly trigger lock escalation.
+ void run_escalation_for_test(void);
+ void run_escalation(void);
+
+ // Add time t to the escalator's wait time statistics
+ void add_escalator_wait_time(uint64_t t);
+
+ void kill_waiter(void *extra);
+
+ private:
+ static const uint64_t DEFAULT_MAX_LOCK_MEMORY = 64L * 1024 * 1024;
+
+ // tracks the current number of locks and lock memory
+ uint64_t m_max_lock_memory;
+ uint64_t m_current_lock_memory;
+
+ struct lt_counters m_lt_counters;
+
+ // the create and destroy callbacks for the locktrees
+ lt_create_cb m_lt_create_callback;
+ lt_destroy_cb m_lt_destroy_callback;
+ lt_escalate_cb m_lt_escalate_callback;
+ void *m_lt_escalate_callback_extra;
+
+ omt<locktree *> m_locktree_map;
+
+ toku_external_mutex_factory_t mutex_factory;
+
+ // the manager's mutex protects the locktree map
+ toku_mutex_t m_mutex;
+
+ void mutex_lock(void);
+
+ void mutex_unlock(void);
+
+ // Manage the set of open locktrees
+ locktree *locktree_map_find(const DICTIONARY_ID &dict_id);
+ void locktree_map_put(locktree *lt);
+ void locktree_map_remove(locktree *lt);
+
+ static int find_by_dict_id(locktree *const &lt, const DICTIONARY_ID &dict_id);
+
+ void escalator_init(void);
+ void escalator_destroy(void);
+
+ // statistics about lock escalation.
+ toku_mutex_t m_escalation_mutex;
+ uint64_t m_escalation_count;
+ tokutime_t m_escalation_time;
+ uint64_t m_escalation_latest_result;
+ uint64_t m_wait_escalation_count;
+ uint64_t m_wait_escalation_time;
+ uint64_t m_long_wait_escalation_count;
+ uint64_t m_long_wait_escalation_time;
+
+ // the escalator coordinates escalation on a set of locktrees for a bunch of
+ // threads
+ class locktree_escalator {
+ public:
+ void create(void);
+ void destroy(void);
+ void run(locktree_manager *mgr, void (*escalate_locktrees_fun)(void *extra),
+ void *extra);
+
+ private:
+ toku_mutex_t m_escalator_mutex;
+ toku_cond_t m_escalator_done;
+ bool m_escalator_running;
+ };
+
+ locktree_escalator m_escalator;
+
+ friend class manager_unit_test;
+};
+
+// A locktree represents the set of row locks owned by all transactions
+// over an open dictionary. Read and write ranges are represented as
+// a left and right key which are compared with the given comparator
+//
+// Locktrees are not created and destroyed by the user. Instead, they are
+// referenced and released using the locktree manager.
+//
+// A sample workflow looks like this:
+// - Create a manager.
+// - Get a locktree by dictionaroy id from the manager.
+// - Perform read/write lock acquision on the locktree, add references to
+// the locktree using the manager, release locks, release references, etc.
+// - ...
+// - Release the final reference to the locktree. It will be destroyed.
+// - Destroy the manager.
+class locktree {
+ public:
+ // effect: Creates a locktree
+ void create(locktree_manager *mgr, DICTIONARY_ID dict_id,
+ const comparator &cmp,
+ toku_external_mutex_factory_t mutex_factory);
+
+ void destroy(void);
+
+ // For thread-safe, external reference counting
+ void add_reference(void);
+
+ // requires: the reference count is > 0
+ // returns: the reference count, after decrementing it by one
+ uint32_t release_reference(void);
+
+ // returns: the current reference count
+ uint32_t get_reference_count(void);
+
+ // effect: Attempts to grant a read lock for the range of keys between
+ // [left_key, right_key]. returns: If the lock cannot be granted, return
+ // DB_LOCK_NOTGRANTED, and populate the
+ // given conflicts set with the txnids that hold conflicting locks in
+ // the range. If the locktree cannot create more locks, return
+ // TOKUDB_OUT_OF_LOCKS.
+ // note: Read locks cannot be shared between txnids, as one would expect.
+ // This is for simplicity since read locks are rare in MySQL.
+ int acquire_read_lock(TXNID txnid, const DBT *left_key, const DBT *right_key,
+ txnid_set *conflicts, bool big_txn);
+
+ // effect: Attempts to grant a write lock for the range of keys between
+ // [left_key, right_key]. returns: If the lock cannot be granted, return
+ // DB_LOCK_NOTGRANTED, and populate the
+ // given conflicts set with the txnids that hold conflicting locks in
+ // the range. If the locktree cannot create more locks, return
+ // TOKUDB_OUT_OF_LOCKS.
+ int acquire_write_lock(TXNID txnid, const DBT *left_key, const DBT *right_key,
+ txnid_set *conflicts, bool big_txn);
+
+ // effect: populate the conflicts set with the txnids that would preventing
+ // the given txnid from getting a lock on [left_key, right_key]
+ void get_conflicts(bool is_write_request, TXNID txnid, const DBT *left_key,
+ const DBT *right_key, txnid_set *conflicts);
+
+ // effect: Release all of the lock ranges represented by the range buffer for
+ // a txnid.
+ void release_locks(TXNID txnid, const range_buffer *ranges,
+ bool all_trx_locks_hint = false);
+
+ // effect: Runs escalation on this locktree
+ void escalate(lt_escalate_cb after_escalate_callback, void *extra);
+
+ // returns: The userdata associated with this locktree, or null if it has not
+ // been set.
+ void *get_userdata(void) const;
+
+ void set_userdata(void *userdata);
+
+ locktree_manager *get_manager(void) const;
+
+ void set_comparator(const comparator &cmp);
+
+ // Set the user-provided Lock Escalation Barrier check function and its
+ // argument
+ //
+ // Lock Escalation Barrier limits the scope of Lock Escalation.
+ // For two keys A and B (such that A < B),
+ // escalation_barrier_check_func(A, B)==true means that there's a lock
+ // escalation barrier between A and B, and lock escalation is not allowed to
+ // bridge the gap between A and B.
+ //
+ // This method sets the user-provided barrier check function and its
+ // parameter.
+ void set_escalation_barrier_func(lt_escalation_barrier_check_func func,
+ void *extra);
+
+ int compare(const locktree *lt) const;
+
+ DICTIONARY_ID get_dict_id() const;
+
+ // Private info struct for storing pending lock request state.
+ // Only to be used by lock requests. We store it here as
+ // something less opaque than usual to strike a tradeoff between
+ // abstraction and code complexity. It is still fairly abstract
+ // since the lock_request object is opaque
+ struct lt_lock_request_info *get_lock_request_info(void);
+
+ typedef void (*dump_callback)(void *cdata, const DBT *left, const DBT *right,
+ TXNID txnid, bool is_shared,
+ TxnidVector *owners);
+ void dump_locks(void *cdata, dump_callback cb);
+
+ private:
+ locktree_manager *m_mgr;
+ DICTIONARY_ID m_dict_id;
+ uint32_t m_reference_count;
+
+ // Since the memory referenced by this comparator is not owned by the
+ // locktree, the user must guarantee it will outlive the locktree.
+ //
+ // The ydb API accomplishes this by opening an ft_handle in the on_create
+ // callback, which will keep the underlying FT (and its descriptor) in memory
+ // for as long as the handle is open. The ft_handle is stored opaquely in the
+ // userdata pointer below. see locktree_manager::get_lt w/ on_create_extra
+ comparator m_cmp;
+
+ lt_escalation_barrier_check_func m_escalation_barrier;
+ void *m_escalation_barrier_arg;
+
+ concurrent_tree *m_rangetree;
+
+ void *m_userdata;
+ struct lt_lock_request_info m_lock_request_info;
+
+ // psergey-todo:
+ // Each transaction also keeps a list of ranges it has locked.
+ // So, when a transaction is running in STO mode, two identical
+ // lists are kept: the STO lock list and transaction's owned locks
+ // list. Why can't we do with just one list?
+
+ // The following fields and members prefixed with "sto_" are for
+ // the single txnid optimization, intended to speed up the case
+ // when only one transaction is using the locktree. If we know
+ // the locktree has only one transaction, then acquiring locks
+ // takes O(1) work and releasing all locks takes O(1) work.
+ //
+ // How do we know that the locktree only has a single txnid?
+ // What do we do if it does?
+ //
+ // When a txn with txnid T requests a lock:
+ // - If the tree is empty, the optimization is possible. Set the single
+ // txnid to T, and insert the lock range into the buffer.
+ // - If the tree is not empty, check if the single txnid is T. If so,
+ // append the lock range to the buffer. Otherwise, migrate all of
+ // the locks in the buffer into the rangetree on behalf of txnid T,
+ // and invalid the single txnid.
+ //
+ // When a txn with txnid T releases its locks:
+ // - If the single txnid is valid, it must be for T. Destroy the buffer.
+ // - If it's not valid, release locks the normal way in the rangetree.
+ //
+ // To carry out the optimization we need to record a single txnid
+ // and a range buffer for each locktree, each protected by the root
+ // lock of the locktree's rangetree. The root lock for a rangetree
+ // is grabbed by preparing a locked keyrange on the rangetree.
+ TXNID m_sto_txnid;
+ range_buffer m_sto_buffer;
+
+ // The single txnid optimization speeds up the case when only one
+ // transaction is using the locktree. But it has the potential to
+ // hurt the case when more than one txnid exists.
+ //
+ // There are two things we need to do to make the optimization only
+ // optimize the case we care about, and not hurt the general case.
+ //
+ // Bound the worst-case latency for lock migration when the
+ // optimization stops working:
+ // - Idea: Stop the optimization and migrate immediate if we notice
+ // the single txnid has takes many locks in the range buffer.
+ // - Implementation: Enforce a max size on the single txnid range buffer.
+ // - Analysis: Choosing the perfect max value, M, is difficult to do
+ // without some feedback from the field. Intuition tells us that M should
+ // not be so small that the optimization is worthless, and it should not
+ // be so big that it's unreasonable to have to wait behind a thread doing
+ // the work of converting M buffer locks into rangetree locks.
+ //
+ // Prevent concurrent-transaction workloads from trying the optimization
+ // in vain:
+ // - Idea: Don't even bother trying the optimization if we think the
+ // system is in a concurrent-transaction state.
+ // - Implementation: Do something even simpler than detecting whether the
+ // system is in a concurent-transaction state. Just keep a "score" value
+ // and some threshold. If at any time the locktree is eligible for the
+ // optimization, only do it if the score is at this threshold. When you
+ // actually do the optimization but someone has to migrate locks in the buffer
+ // (expensive), then reset the score back to zero. Each time a txn
+ // releases locks, the score is incremented by 1.
+ // - Analysis: If you let the threshold be "C", then at most 1 / C txns will
+ // do the optimization in a concurrent-transaction system. Similarly, it
+ // takes at most C txns to start using the single txnid optimzation, which
+ // is good when the system transitions from multithreaded to single threaded.
+ //
+ // STO_BUFFER_MAX_SIZE:
+ //
+ // We choose the max value to be 1 million since most transactions are smaller
+ // than 1 million and we can create a rangetree of 1 million elements in
+ // less than a second. So we can be pretty confident that this threshold
+ // enables the optimization almost always, and prevents super pathological
+ // latency issues for the first lock taken by a second thread.
+ //
+ // STO_SCORE_THRESHOLD:
+ //
+ // A simple first guess at a good value for the score threshold is 100.
+ // By our analysis, we'd end up doing the optimization in vain for
+ // around 1% of all transactions, which seems reasonable. Further,
+ // if the system goes single threaded, it ought to be pretty quick
+ // for 100 transactions to go by, so we won't have to wait long before
+ // we start doing the single txind optimzation again.
+ static const int STO_BUFFER_MAX_SIZE = 50 * 1024;
+ static const int STO_SCORE_THRESHOLD = 100;
+ int m_sto_score;
+
+ // statistics about time spent ending the STO early
+ uint64_t m_sto_end_early_count;
+ tokutime_t m_sto_end_early_time;
+
+ // effect: begins the single txnid optimizaiton, setting m_sto_txnid
+ // to the given txnid.
+ // requires: m_sto_txnid is invalid
+ void sto_begin(TXNID txnid);
+
+ // effect: append a range to the sto buffer
+ // requires: m_sto_txnid is valid
+ void sto_append(const DBT *left_key, const DBT *right_key,
+ bool is_write_request);
+
+ // effect: ends the single txnid optimization, releaseing any memory
+ // stored in the sto buffer, notifying the tracker, and
+ // invalidating m_sto_txnid.
+ // requires: m_sto_txnid is valid
+ void sto_end(void);
+
+ // params: prepared_lkr is a void * to a prepared locked keyrange. see below.
+ // effect: ends the single txnid optimization early, migrating buffer locks
+ // into the rangetree, calling sto_end(), and then setting the
+ // sto_score back to zero.
+ // requires: m_sto_txnid is valid
+ void sto_end_early(void *prepared_lkr);
+ void sto_end_early_no_accounting(void *prepared_lkr);
+
+ // params: prepared_lkr is a void * to a prepared locked keyrange. we can't
+ // use
+ // the real type because the compiler won't allow us to forward
+ // declare concurrent_tree::locked_keyrange without including
+ // concurrent_tree.h, which we cannot do here because it is a template
+ // implementation.
+ // requires: the prepared locked keyrange is for the locktree's rangetree
+ // requires: m_sto_txnid is valid
+ // effect: migrates each lock in the single txnid buffer into the locktree's
+ // rangetree, notifying the memory tracker as necessary.
+ void sto_migrate_buffer_ranges_to_tree(void *prepared_lkr);
+
+ // effect: If m_sto_txnid is valid, then release the txnid's locks
+ // by ending the optimization.
+ // requires: If m_sto_txnid is valid, it is equal to the given txnid
+ // returns: True if locks were released for this txnid
+ bool sto_try_release(TXNID txnid);
+
+ // params: prepared_lkr is a void * to a prepared locked keyrange. see above.
+ // requires: the prepared locked keyrange is for the locktree's rangetree
+ // effect: If m_sto_txnid is valid and equal to the given txnid, then
+ // append a range onto the buffer. Otherwise, if m_sto_txnid is valid
+ // but not equal to this txnid, then migrate the buffer's locks
+ // into the rangetree and end the optimization, setting the score
+ // back to zero.
+ // returns: true if the lock was acquired for this txnid
+ bool sto_try_acquire(void *prepared_lkr, TXNID txnid, const DBT *left_key,
+ const DBT *right_key, bool is_write_request);
+
+ // Effect:
+ // Provides a hook for a helgrind suppression.
+ // Returns:
+ // true if m_sto_txnid is not TXNID_NONE
+ bool sto_txnid_is_valid_unsafe(void) const;
+
+ // Effect:
+ // Provides a hook for a helgrind suppression.
+ // Returns:
+ // m_sto_score
+ int sto_get_score_unsafe(void) const;
+
+ void remove_overlapping_locks_for_txnid(TXNID txnid, const DBT *left_key,
+ const DBT *right_key);
+
+ int acquire_lock_consolidated(void *prepared_lkr, TXNID txnid,
+ const DBT *left_key, const DBT *right_key,
+ bool is_write_request, txnid_set *conflicts);
+
+ int acquire_lock(bool is_write_request, TXNID txnid, const DBT *left_key,
+ const DBT *right_key, txnid_set *conflicts);
+
+ int try_acquire_lock(bool is_write_request, TXNID txnid, const DBT *left_key,
+ const DBT *right_key, txnid_set *conflicts,
+ bool big_txn);
+
+ friend class locktree_unit_test;
+ friend class manager_unit_test;
+ friend class lock_request_unit_test;
+
+ // engine status reaches into the locktree to read some stats
+ friend void locktree_manager::get_status(LTM_STATUS status);
+};
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/manager.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/manager.cc
new file mode 100644
index 000000000..4186182be
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/manager.cc
@@ -0,0 +1,527 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include <stdlib.h>
+#include <string.h>
+
+#include "../portability/toku_pthread.h"
+#include "../util/status.h"
+#include "lock_request.h"
+#include "locktree.h"
+
+namespace toku {
+
+void locktree_manager::create(lt_create_cb create_cb, lt_destroy_cb destroy_cb,
+ lt_escalate_cb escalate_cb, void *escalate_extra,
+ toku_external_mutex_factory_t mutex_factory_arg) {
+ mutex_factory = mutex_factory_arg;
+ m_max_lock_memory = DEFAULT_MAX_LOCK_MEMORY;
+ m_current_lock_memory = 0;
+
+ m_locktree_map.create();
+ m_lt_create_callback = create_cb;
+ m_lt_destroy_callback = destroy_cb;
+ m_lt_escalate_callback = escalate_cb;
+ m_lt_escalate_callback_extra = escalate_extra;
+ ZERO_STRUCT(m_mutex);
+ toku_mutex_init(manager_mutex_key, &m_mutex, nullptr);
+
+ ZERO_STRUCT(m_lt_counters);
+
+ escalator_init();
+}
+
+void locktree_manager::destroy(void) {
+ escalator_destroy();
+ invariant(m_current_lock_memory == 0);
+ invariant(m_locktree_map.size() == 0);
+ m_locktree_map.destroy();
+ toku_mutex_destroy(&m_mutex);
+}
+
+void locktree_manager::mutex_lock(void) { toku_mutex_lock(&m_mutex); }
+
+void locktree_manager::mutex_unlock(void) { toku_mutex_unlock(&m_mutex); }
+
+size_t locktree_manager::get_max_lock_memory(void) { return m_max_lock_memory; }
+
+int locktree_manager::set_max_lock_memory(size_t max_lock_memory) {
+ int r = 0;
+ mutex_lock();
+ if (max_lock_memory < m_current_lock_memory) {
+ r = EDOM;
+ } else {
+ m_max_lock_memory = max_lock_memory;
+ }
+ mutex_unlock();
+ return r;
+}
+
+int locktree_manager::find_by_dict_id(locktree *const &lt,
+ const DICTIONARY_ID &dict_id) {
+ if (lt->get_dict_id().dictid < dict_id.dictid) {
+ return -1;
+ } else if (lt->get_dict_id().dictid == dict_id.dictid) {
+ return 0;
+ } else {
+ return 1;
+ }
+}
+
+locktree *locktree_manager::locktree_map_find(const DICTIONARY_ID &dict_id) {
+ locktree *lt;
+ int r = m_locktree_map.find_zero<DICTIONARY_ID, find_by_dict_id>(dict_id, &lt,
+ nullptr);
+ return r == 0 ? lt : nullptr;
+}
+
+void locktree_manager::locktree_map_put(locktree *lt) {
+ int r = m_locktree_map.insert<DICTIONARY_ID, find_by_dict_id>(
+ lt, lt->get_dict_id(), nullptr);
+ invariant_zero(r);
+}
+
+void locktree_manager::locktree_map_remove(locktree *lt) {
+ uint32_t idx;
+ locktree *found_lt;
+ int r = m_locktree_map.find_zero<DICTIONARY_ID, find_by_dict_id>(
+ lt->get_dict_id(), &found_lt, &idx);
+ invariant_zero(r);
+ invariant(found_lt == lt);
+ r = m_locktree_map.delete_at(idx);
+ invariant_zero(r);
+}
+
+locktree *locktree_manager::get_lt(DICTIONARY_ID dict_id, const comparator &cmp,
+ void *on_create_extra) {
+ // hold the mutex around searching and maybe
+ // inserting into the locktree map
+ mutex_lock();
+
+ locktree *lt = locktree_map_find(dict_id);
+ if (lt == nullptr) {
+ XCALLOC(lt);
+ lt->create(this, dict_id, cmp, mutex_factory);
+
+ // new locktree created - call the on_create callback
+ // and put it in the locktree map
+ if (m_lt_create_callback) {
+ int r = m_lt_create_callback(lt, on_create_extra);
+ if (r != 0) {
+ lt->release_reference();
+ lt->destroy();
+ toku_free(lt);
+ lt = nullptr;
+ }
+ }
+ if (lt) {
+ locktree_map_put(lt);
+ }
+ } else {
+ reference_lt(lt);
+ }
+
+ mutex_unlock();
+
+ return lt;
+}
+
+void locktree_manager::reference_lt(locktree *lt) {
+ // increment using a sync fetch and add.
+ // the caller guarantees that the lt won't be
+ // destroyed while we increment the count here.
+ //
+ // the caller can do this by already having an lt
+ // reference or by holding the manager mutex.
+ //
+ // if the manager's mutex is held, it is ok for the
+ // reference count to transition from 0 to 1 (no race),
+ // since we're serialized with other opens and closes.
+ lt->add_reference();
+}
+
+void locktree_manager::release_lt(locktree *lt) {
+ bool do_destroy = false;
+ DICTIONARY_ID dict_id = lt->get_dict_id();
+
+ // Release a reference on the locktree. If the count transitions to zero,
+ // then we *may* need to do the cleanup.
+ //
+ // Grab the manager's mutex and look for a locktree with this locktree's
+ // dictionary id. Since dictionary id's never get reused, any locktree
+ // found must be the one we just released a reference on.
+ //
+ // At least two things could have happened since we got the mutex:
+ // - Another thread gets a locktree with the same dict_id, increments
+ // the reference count. In this case, we shouldn't destroy it.
+ // - Another thread gets a locktree with the same dict_id and then
+ // releases it quickly, transitioning the reference count from zero to
+ // one and back to zero. In this case, only one of us should destroy it.
+ // It doesn't matter which. We originally missed this case, see #5776.
+ //
+ // After 5776, the high level rule for release is described below.
+ //
+ // If a thread releases a locktree and notices the reference count transition
+ // to zero, then that thread must immediately:
+ // - assume the locktree object is invalid
+ // - grab the manager's mutex
+ // - search the locktree map for a locktree with the same dict_id and remove
+ // it, if it exists. the destroy may be deferred.
+ // - release the manager's mutex
+ //
+ // This way, if many threads transition the same locktree's reference count
+ // from 1 to zero and wait behind the manager's mutex, only one of them will
+ // do the actual destroy and the others will happily do nothing.
+ uint32_t refs = lt->release_reference();
+ if (refs == 0) {
+ mutex_lock();
+ // lt may not have already been destroyed, so look it up.
+ locktree *find_lt = locktree_map_find(dict_id);
+ if (find_lt != nullptr) {
+ // A locktree is still in the map with that dict_id, so it must be
+ // equal to lt. This is true because dictionary ids are never reused.
+ // If the reference count is zero, it's our responsibility to remove
+ // it and do the destroy. Otherwise, someone still wants it.
+ // If the locktree is still valid then check if it should be deleted.
+ if (find_lt == lt) {
+ if (lt->get_reference_count() == 0) {
+ locktree_map_remove(lt);
+ do_destroy = true;
+ }
+ m_lt_counters.add(lt->get_lock_request_info()->counters);
+ }
+ }
+ mutex_unlock();
+ }
+
+ // if necessary, do the destroy without holding the mutex
+ if (do_destroy) {
+ if (m_lt_destroy_callback) {
+ m_lt_destroy_callback(lt);
+ }
+ lt->destroy();
+ toku_free(lt);
+ }
+}
+
+void locktree_manager::run_escalation(void) {
+ struct escalation_fn {
+ static void run(void *extra) {
+ locktree_manager *mgr = (locktree_manager *)extra;
+ mgr->escalate_all_locktrees();
+ };
+ };
+ m_escalator.run(this, escalation_fn::run, this);
+}
+
+// test-only version of lock escalation
+void locktree_manager::run_escalation_for_test(void) { run_escalation(); }
+
+void locktree_manager::escalate_all_locktrees(void) {
+ uint64_t t0 = toku_current_time_microsec();
+
+ // get all locktrees
+ mutex_lock();
+ int num_locktrees = m_locktree_map.size();
+ locktree **locktrees = new locktree *[num_locktrees];
+ for (int i = 0; i < num_locktrees; i++) {
+ int r = m_locktree_map.fetch(i, &locktrees[i]);
+ invariant_zero(r);
+ reference_lt(locktrees[i]);
+ }
+ mutex_unlock();
+
+ // escalate them
+ escalate_locktrees(locktrees, num_locktrees);
+
+ delete[] locktrees;
+
+ uint64_t t1 = toku_current_time_microsec();
+ add_escalator_wait_time(t1 - t0);
+}
+
+void locktree_manager::note_mem_used(uint64_t mem_used) {
+ (void)toku_sync_fetch_and_add(&m_current_lock_memory, mem_used);
+}
+
+void locktree_manager::note_mem_released(uint64_t mem_released) {
+ uint64_t old_mem_used =
+ toku_sync_fetch_and_sub(&m_current_lock_memory, mem_released);
+ invariant(old_mem_used >= mem_released);
+}
+
+bool locktree_manager::out_of_locks(void) const {
+ return m_current_lock_memory >= m_max_lock_memory;
+}
+
+bool locktree_manager::over_big_threshold(void) {
+ return m_current_lock_memory >= m_max_lock_memory / 2;
+}
+
+int locktree_manager::iterate_pending_lock_requests(
+ lock_request_iterate_callback callback, void *extra) {
+ mutex_lock();
+ int r = 0;
+ uint32_t num_locktrees = m_locktree_map.size();
+ for (uint32_t i = 0; i < num_locktrees && r == 0; i++) {
+ locktree *lt;
+ r = m_locktree_map.fetch(i, &lt);
+ invariant_zero(r);
+ if (r == EINVAL) // Shouldn't happen, avoid compiler warning
+ continue;
+
+ struct lt_lock_request_info *info = lt->get_lock_request_info();
+ toku_external_mutex_lock(&info->mutex);
+
+ uint32_t num_requests = info->pending_lock_requests.size();
+ for (uint32_t k = 0; k < num_requests && r == 0; k++) {
+ lock_request *req;
+ r = info->pending_lock_requests.fetch(k, &req);
+ invariant_zero(r);
+ if (r == EINVAL) /* Shouldn't happen, avoid compiler warning */
+ continue;
+ r = callback(lt->get_dict_id(), req->get_txnid(), req->get_left_key(),
+ req->get_right_key(), req->get_conflicting_txnid(),
+ req->get_start_time(), extra);
+ }
+
+ toku_external_mutex_unlock(&info->mutex);
+ }
+ mutex_unlock();
+ return r;
+}
+
+int locktree_manager::check_current_lock_constraints(bool big_txn) {
+ int r = 0;
+ if (big_txn && over_big_threshold()) {
+ run_escalation();
+ if (over_big_threshold()) {
+ r = TOKUDB_OUT_OF_LOCKS;
+ }
+ }
+ if (r == 0 && out_of_locks()) {
+ run_escalation();
+ if (out_of_locks()) {
+ // return an error if we're still out of locks after escalation.
+ r = TOKUDB_OUT_OF_LOCKS;
+ }
+ }
+ return r;
+}
+
+void locktree_manager::escalator_init(void) {
+ ZERO_STRUCT(m_escalation_mutex);
+ toku_mutex_init(manager_escalation_mutex_key, &m_escalation_mutex, nullptr);
+ m_escalation_count = 0;
+ m_escalation_time = 0;
+ m_wait_escalation_count = 0;
+ m_wait_escalation_time = 0;
+ m_long_wait_escalation_count = 0;
+ m_long_wait_escalation_time = 0;
+ m_escalation_latest_result = 0;
+ m_escalator.create();
+}
+
+void locktree_manager::escalator_destroy(void) {
+ m_escalator.destroy();
+ toku_mutex_destroy(&m_escalation_mutex);
+}
+
+void locktree_manager::add_escalator_wait_time(uint64_t t) {
+ toku_mutex_lock(&m_escalation_mutex);
+ m_wait_escalation_count += 1;
+ m_wait_escalation_time += t;
+ if (t >= 1000000) {
+ m_long_wait_escalation_count += 1;
+ m_long_wait_escalation_time += t;
+ }
+ toku_mutex_unlock(&m_escalation_mutex);
+}
+
+void locktree_manager::escalate_locktrees(locktree **locktrees,
+ int num_locktrees) {
+ // there are too many row locks in the system and we need to tidy up.
+ //
+ // a simple implementation of escalation does not attempt
+ // to reduce the memory foot print of each txn's range buffer.
+ // doing so would require some layering hackery (or a callback)
+ // and more complicated locking. for now, just escalate each
+ // locktree individually, in-place.
+ tokutime_t t0 = toku_time_now();
+ for (int i = 0; i < num_locktrees; i++) {
+ locktrees[i]->escalate(m_lt_escalate_callback,
+ m_lt_escalate_callback_extra);
+ release_lt(locktrees[i]);
+ }
+ tokutime_t t1 = toku_time_now();
+
+ toku_mutex_lock(&m_escalation_mutex);
+ m_escalation_count++;
+ m_escalation_time += (t1 - t0);
+ m_escalation_latest_result = m_current_lock_memory;
+ toku_mutex_unlock(&m_escalation_mutex);
+}
+
+struct escalate_args {
+ locktree_manager *mgr;
+ locktree **locktrees;
+ int num_locktrees;
+};
+
+void locktree_manager::locktree_escalator::create(void) {
+ ZERO_STRUCT(m_escalator_mutex);
+ toku_mutex_init(manager_escalator_mutex_key, &m_escalator_mutex, nullptr);
+ toku_cond_init(manager_m_escalator_done_key, &m_escalator_done, nullptr);
+ m_escalator_running = false;
+}
+
+void locktree_manager::locktree_escalator::destroy(void) {
+ toku_cond_destroy(&m_escalator_done);
+ toku_mutex_destroy(&m_escalator_mutex);
+}
+
+void locktree_manager::locktree_escalator::run(
+ locktree_manager *mgr, void (*escalate_locktrees_fun)(void *extra),
+ void *extra) {
+ uint64_t t0 = toku_current_time_microsec();
+ toku_mutex_lock(&m_escalator_mutex);
+ if (!m_escalator_running) {
+ // run escalation on this thread
+ m_escalator_running = true;
+ toku_mutex_unlock(&m_escalator_mutex);
+ escalate_locktrees_fun(extra);
+ toku_mutex_lock(&m_escalator_mutex);
+ m_escalator_running = false;
+ toku_cond_broadcast(&m_escalator_done);
+ } else {
+ toku_cond_wait(&m_escalator_done, &m_escalator_mutex);
+ }
+ toku_mutex_unlock(&m_escalator_mutex);
+ uint64_t t1 = toku_current_time_microsec();
+ mgr->add_escalator_wait_time(t1 - t0);
+}
+
+void locktree_manager::get_status(LTM_STATUS statp) {
+ ltm_status.init();
+ LTM_STATUS_VAL(LTM_SIZE_CURRENT) = m_current_lock_memory;
+ LTM_STATUS_VAL(LTM_SIZE_LIMIT) = m_max_lock_memory;
+ LTM_STATUS_VAL(LTM_ESCALATION_COUNT) = m_escalation_count;
+ LTM_STATUS_VAL(LTM_ESCALATION_TIME) = m_escalation_time;
+ LTM_STATUS_VAL(LTM_ESCALATION_LATEST_RESULT) = m_escalation_latest_result;
+ LTM_STATUS_VAL(LTM_WAIT_ESCALATION_COUNT) = m_wait_escalation_count;
+ LTM_STATUS_VAL(LTM_WAIT_ESCALATION_TIME) = m_wait_escalation_time;
+ LTM_STATUS_VAL(LTM_LONG_WAIT_ESCALATION_COUNT) = m_long_wait_escalation_count;
+ LTM_STATUS_VAL(LTM_LONG_WAIT_ESCALATION_TIME) = m_long_wait_escalation_time;
+
+ uint64_t lock_requests_pending = 0;
+ uint64_t sto_num_eligible = 0;
+ uint64_t sto_end_early_count = 0;
+ tokutime_t sto_end_early_time = 0;
+ uint32_t num_locktrees = 0;
+ struct lt_counters lt_counters;
+ ZERO_STRUCT(lt_counters); // PORT: instead of ={}.
+
+ if (toku_mutex_trylock(&m_mutex) == 0) {
+ lt_counters = m_lt_counters;
+ num_locktrees = m_locktree_map.size();
+ for (uint32_t i = 0; i < num_locktrees; i++) {
+ locktree *lt;
+ int r = m_locktree_map.fetch(i, &lt);
+ invariant_zero(r);
+ if (r == EINVAL) // Shouldn't happen, avoid compiler warning
+ continue;
+ if (toku_external_mutex_trylock(&lt->m_lock_request_info.mutex) == 0) {
+ lock_requests_pending +=
+ lt->m_lock_request_info.pending_lock_requests.size();
+ lt_counters.add(lt->get_lock_request_info()->counters);
+ toku_external_mutex_unlock(&lt->m_lock_request_info.mutex);
+ }
+ sto_num_eligible += lt->sto_txnid_is_valid_unsafe() ? 1 : 0;
+ sto_end_early_count += lt->m_sto_end_early_count;
+ sto_end_early_time += lt->m_sto_end_early_time;
+ }
+ mutex_unlock();
+ }
+
+ LTM_STATUS_VAL(LTM_NUM_LOCKTREES) = num_locktrees;
+ LTM_STATUS_VAL(LTM_LOCK_REQUESTS_PENDING) = lock_requests_pending;
+ LTM_STATUS_VAL(LTM_STO_NUM_ELIGIBLE) = sto_num_eligible;
+ LTM_STATUS_VAL(LTM_STO_END_EARLY_COUNT) = sto_end_early_count;
+ LTM_STATUS_VAL(LTM_STO_END_EARLY_TIME) = sto_end_early_time;
+ LTM_STATUS_VAL(LTM_WAIT_COUNT) = lt_counters.wait_count;
+ LTM_STATUS_VAL(LTM_WAIT_TIME) = lt_counters.wait_time;
+ LTM_STATUS_VAL(LTM_LONG_WAIT_COUNT) = lt_counters.long_wait_count;
+ LTM_STATUS_VAL(LTM_LONG_WAIT_TIME) = lt_counters.long_wait_time;
+ LTM_STATUS_VAL(LTM_TIMEOUT_COUNT) = lt_counters.timeout_count;
+ *statp = ltm_status;
+}
+
+void locktree_manager::kill_waiter(void *extra) {
+ mutex_lock();
+ int r = 0;
+ uint32_t num_locktrees = m_locktree_map.size();
+ for (uint32_t i = 0; i < num_locktrees; i++) {
+ locktree *lt;
+ r = m_locktree_map.fetch(i, &lt);
+ invariant_zero(r);
+ if (r) continue; // Get rid of "may be used uninitialized" warning
+ lock_request::kill_waiter(lt, extra);
+ }
+ mutex_unlock();
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.cc
new file mode 100644
index 000000000..1e1d23ef8
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.cc
@@ -0,0 +1,265 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "range_buffer.h"
+
+#include <string.h>
+
+#include "../portability/memory.h"
+#include "../util/dbt.h"
+
+namespace toku {
+
+bool range_buffer::record_header::left_is_infinite(void) const {
+ return left_neg_inf || left_pos_inf;
+}
+
+bool range_buffer::record_header::right_is_infinite(void) const {
+ return right_neg_inf || right_pos_inf;
+}
+
+void range_buffer::record_header::init(const DBT *left_key,
+ const DBT *right_key,
+ bool is_exclusive) {
+ is_exclusive_lock = is_exclusive;
+ left_neg_inf = left_key == toku_dbt_negative_infinity();
+ left_pos_inf = left_key == toku_dbt_positive_infinity();
+ left_key_size = toku_dbt_is_infinite(left_key) ? 0 : left_key->size;
+ if (right_key) {
+ right_neg_inf = right_key == toku_dbt_negative_infinity();
+ right_pos_inf = right_key == toku_dbt_positive_infinity();
+ right_key_size = toku_dbt_is_infinite(right_key) ? 0 : right_key->size;
+ } else {
+ right_neg_inf = left_neg_inf;
+ right_pos_inf = left_pos_inf;
+ right_key_size = 0;
+ }
+}
+
+const DBT *range_buffer::iterator::record::get_left_key(void) const {
+ if (_header.left_neg_inf) {
+ return toku_dbt_negative_infinity();
+ } else if (_header.left_pos_inf) {
+ return toku_dbt_positive_infinity();
+ } else {
+ return &_left_key;
+ }
+}
+
+const DBT *range_buffer::iterator::record::get_right_key(void) const {
+ if (_header.right_neg_inf) {
+ return toku_dbt_negative_infinity();
+ } else if (_header.right_pos_inf) {
+ return toku_dbt_positive_infinity();
+ } else {
+ return &_right_key;
+ }
+}
+
+size_t range_buffer::iterator::record::size(void) const {
+ return sizeof(record_header) + _header.left_key_size + _header.right_key_size;
+}
+
+void range_buffer::iterator::record::deserialize(const char *buf) {
+ size_t current = 0;
+
+ // deserialize the header
+ memcpy(&_header, buf, sizeof(record_header));
+ current += sizeof(record_header);
+
+ // deserialize the left key if necessary
+ if (!_header.left_is_infinite()) {
+ // point the left DBT's buffer into ours
+ toku_fill_dbt(&_left_key, buf + current, _header.left_key_size);
+ current += _header.left_key_size;
+ }
+
+ // deserialize the right key if necessary
+ if (!_header.right_is_infinite()) {
+ if (_header.right_key_size == 0) {
+ toku_copyref_dbt(&_right_key, _left_key);
+ } else {
+ toku_fill_dbt(&_right_key, buf + current, _header.right_key_size);
+ }
+ }
+}
+
+toku::range_buffer::iterator::iterator()
+ : _ma_chunk_iterator(nullptr),
+ _current_chunk_base(nullptr),
+ _current_chunk_offset(0),
+ _current_chunk_max(0),
+ _current_rec_size(0) {}
+
+toku::range_buffer::iterator::iterator(const range_buffer *buffer)
+ : _ma_chunk_iterator(&buffer->_arena),
+ _current_chunk_base(nullptr),
+ _current_chunk_offset(0),
+ _current_chunk_max(0),
+ _current_rec_size(0) {
+ reset_current_chunk();
+}
+
+void range_buffer::iterator::reset_current_chunk() {
+ _current_chunk_base = _ma_chunk_iterator.current(&_current_chunk_max);
+ _current_chunk_offset = 0;
+}
+
+bool range_buffer::iterator::current(record *rec) {
+ if (_current_chunk_offset < _current_chunk_max) {
+ const char *buf = reinterpret_cast<const char *>(_current_chunk_base);
+ rec->deserialize(buf + _current_chunk_offset);
+ _current_rec_size = rec->size();
+ return true;
+ } else {
+ return false;
+ }
+}
+
+// move the iterator to the next record in the buffer
+void range_buffer::iterator::next(void) {
+ invariant(_current_chunk_offset < _current_chunk_max);
+ invariant(_current_rec_size > 0);
+
+ // the next record is _current_rec_size bytes forward
+ _current_chunk_offset += _current_rec_size;
+ // now, we don't know how big the current is, set it to 0.
+ _current_rec_size = 0;
+
+ if (_current_chunk_offset >= _current_chunk_max) {
+ // current chunk is exhausted, try moving to the next one
+ if (_ma_chunk_iterator.more()) {
+ _ma_chunk_iterator.next();
+ reset_current_chunk();
+ }
+ }
+}
+
+void range_buffer::create(void) {
+ // allocate buffer space lazily instead of on creation. this way,
+ // no malloc/free is done if the transaction ends up taking no locks.
+ _arena.create(0);
+ _num_ranges = 0;
+}
+
+void range_buffer::append(const DBT *left_key, const DBT *right_key,
+ bool is_write_request) {
+ // if the keys are equal, then only one copy is stored.
+ if (toku_dbt_equals(left_key, right_key)) {
+ invariant(left_key->size <= MAX_KEY_SIZE);
+ append_point(left_key, is_write_request);
+ } else {
+ invariant(left_key->size <= MAX_KEY_SIZE);
+ invariant(right_key->size <= MAX_KEY_SIZE);
+ append_range(left_key, right_key, is_write_request);
+ }
+ _num_ranges++;
+}
+
+bool range_buffer::is_empty(void) const { return total_memory_size() == 0; }
+
+uint64_t range_buffer::total_memory_size(void) const {
+ return _arena.total_size_in_use();
+}
+
+int range_buffer::get_num_ranges(void) const { return _num_ranges; }
+
+void range_buffer::destroy(void) { _arena.destroy(); }
+
+void range_buffer::append_range(const DBT *left_key, const DBT *right_key,
+ bool is_exclusive) {
+ size_t record_length =
+ sizeof(record_header) + left_key->size + right_key->size;
+ char *buf = reinterpret_cast<char *>(_arena.malloc_from_arena(record_length));
+
+ record_header h;
+ h.init(left_key, right_key, is_exclusive);
+
+ // serialize the header
+ memcpy(buf, &h, sizeof(record_header));
+ buf += sizeof(record_header);
+
+ // serialize the left key if necessary
+ if (!h.left_is_infinite()) {
+ memcpy(buf, left_key->data, left_key->size);
+ buf += left_key->size;
+ }
+
+ // serialize the right key if necessary
+ if (!h.right_is_infinite()) {
+ memcpy(buf, right_key->data, right_key->size);
+ }
+}
+
+void range_buffer::append_point(const DBT *key, bool is_exclusive) {
+ size_t record_length = sizeof(record_header) + key->size;
+ char *buf = reinterpret_cast<char *>(_arena.malloc_from_arena(record_length));
+
+ record_header h;
+ h.init(key, nullptr, is_exclusive);
+
+ // serialize the header
+ memcpy(buf, &h, sizeof(record_header));
+ buf += sizeof(record_header);
+
+ // serialize the key if necessary
+ if (!h.left_is_infinite()) {
+ memcpy(buf, key->data, key->size);
+ }
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.h
new file mode 100644
index 000000000..76e28d747
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.h
@@ -0,0 +1,178 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <inttypes.h>
+#include <stdint.h>
+
+#include "../util/dbt.h"
+#include "../util/memarena.h"
+
+namespace toku {
+
+// a key range buffer represents a set of key ranges that can
+// be stored, iterated over, and then destroyed all at once.
+class range_buffer {
+ private:
+ // the key range buffer is a bunch of records in a row.
+ // each record has the following header, followed by the
+ // left key and right key data payload, if applicable.
+ // we limit keys to be 2^16, since we store lengths as 2 bytes.
+ static const size_t MAX_KEY_SIZE = 1 << 16;
+
+ struct record_header {
+ bool left_neg_inf;
+ bool left_pos_inf;
+ bool right_pos_inf;
+ bool right_neg_inf;
+ uint16_t left_key_size;
+ uint16_t right_key_size;
+ bool is_exclusive_lock;
+
+ bool left_is_infinite(void) const;
+
+ bool right_is_infinite(void) const;
+
+ void init(const DBT *left_key, const DBT *right_key, bool is_exclusive);
+ };
+ // PORT static_assert(sizeof(record_header) == 8, "record header format is
+ // off");
+
+ public:
+ // the iterator abstracts reading over a buffer of variable length
+ // records one by one until there are no more left.
+ class iterator {
+ public:
+ iterator();
+ iterator(const range_buffer *buffer);
+
+ // a record represents the user-view of a serialized key range.
+ // it handles positive and negative infinity and the optimized
+ // point range case, where left and right points share memory.
+ class record {
+ public:
+ // get a read-only pointer to the left key of this record's range
+ const DBT *get_left_key(void) const;
+
+ // get a read-only pointer to the right key of this record's range
+ const DBT *get_right_key(void) const;
+
+ // how big is this record? this tells us where the next record is
+ size_t size(void) const;
+
+ bool get_exclusive_flag() const { return _header.is_exclusive_lock; }
+
+ // populate a record header and point our DBT's
+ // buffers into ours if they are not infinite.
+ void deserialize(const char *buf);
+
+ private:
+ record_header _header;
+ DBT _left_key;
+ DBT _right_key;
+ };
+
+ // populate the given record object with the current
+ // the memory referred to by record is valid for only
+ // as long as the record exists.
+ bool current(record *rec);
+
+ // move the iterator to the next record in the buffer
+ void next(void);
+
+ private:
+ void reset_current_chunk();
+
+ // the key range buffer we are iterating over, the current
+ // offset in that buffer, and the size of the current record.
+ memarena::chunk_iterator _ma_chunk_iterator;
+ const void *_current_chunk_base;
+ size_t _current_chunk_offset;
+ size_t _current_chunk_max;
+ size_t _current_rec_size;
+ };
+
+ // allocate buffer space lazily instead of on creation. this way,
+ // no malloc/free is done if the transaction ends up taking no locks.
+ void create(void);
+
+ // append a left/right key range to the buffer.
+ // if the keys are equal, then only one copy is stored.
+ void append(const DBT *left_key, const DBT *right_key,
+ bool is_write_request = false);
+
+ // is this range buffer empty?
+ bool is_empty(void) const;
+
+ // how much memory is being used by this range buffer?
+ uint64_t total_memory_size(void) const;
+
+ // how many ranges are stored in this range buffer?
+ int get_num_ranges(void) const;
+
+ void destroy(void);
+
+ private:
+ memarena _arena;
+ int _num_ranges;
+
+ void append_range(const DBT *left_key, const DBT *right_key,
+ bool is_write_request);
+
+ // append a point to the buffer. this is the space/time saving
+ // optimization for key ranges where left == right.
+ void append_point(const DBT *key, bool is_write_request);
+};
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.cc
new file mode 100644
index 000000000..8997f634b
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.cc
@@ -0,0 +1,520 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "treenode.h"
+
+#include "../portability/toku_race_tools.h"
+
+namespace toku {
+
+// TODO: source location info might have to be pulled up one caller
+// to be useful
+void treenode::mutex_lock(void) { toku_mutex_lock(&m_mutex); }
+
+void treenode::mutex_unlock(void) { toku_mutex_unlock(&m_mutex); }
+
+void treenode::init(const comparator *cmp) {
+ m_txnid = TXNID_NONE;
+ m_is_root = false;
+ m_is_empty = true;
+ m_cmp = cmp;
+
+ m_is_shared = false;
+ m_owners = nullptr;
+
+ // use an adaptive mutex at each node since we expect the time the
+ // lock is held to be relatively short compared to a context switch.
+ // indeed, this improves performance at high thread counts considerably.
+ memset(&m_mutex, 0, sizeof(toku_mutex_t));
+ toku_pthread_mutexattr_t attr;
+ toku_mutexattr_init(&attr);
+ toku_mutexattr_settype(&attr, TOKU_MUTEX_ADAPTIVE);
+ toku_mutex_init(treenode_mutex_key, &m_mutex, &attr);
+ toku_mutexattr_destroy(&attr);
+ m_left_child.set(nullptr);
+ m_right_child.set(nullptr);
+}
+
+void treenode::create_root(const comparator *cmp) {
+ init(cmp);
+ m_is_root = true;
+}
+
+void treenode::destroy_root(void) {
+ invariant(is_root());
+ invariant(is_empty());
+ toku_mutex_destroy(&m_mutex);
+ m_cmp = nullptr;
+}
+
+void treenode::set_range_and_txnid(const keyrange &range, TXNID txnid,
+ bool is_shared) {
+ // allocates a new copy of the range for this node
+ m_range.create_copy(range);
+ m_txnid = txnid;
+ m_is_shared = is_shared;
+ m_is_empty = false;
+}
+
+bool treenode::is_root(void) { return m_is_root; }
+
+bool treenode::is_empty(void) { return m_is_empty; }
+
+bool treenode::range_overlaps(const keyrange &range) {
+ return m_range.overlaps(*m_cmp, range);
+}
+
+treenode *treenode::alloc(const comparator *cmp, const keyrange &range,
+ TXNID txnid, bool is_shared) {
+ treenode *XCALLOC(node);
+ node->init(cmp);
+ node->set_range_and_txnid(range, txnid, is_shared);
+ return node;
+}
+
+void treenode::swap_in_place(treenode *node1, treenode *node2) {
+ keyrange tmp_range = node1->m_range;
+ TXNID tmp_txnid = node1->m_txnid;
+ node1->m_range = node2->m_range;
+ node1->m_txnid = node2->m_txnid;
+ node2->m_range = tmp_range;
+ node2->m_txnid = tmp_txnid;
+
+ bool tmp_is_shared = node1->m_is_shared;
+ node1->m_is_shared = node2->m_is_shared;
+ node2->m_is_shared = tmp_is_shared;
+
+ auto tmp_m_owners = node1->m_owners;
+ node1->m_owners = node2->m_owners;
+ node2->m_owners = tmp_m_owners;
+}
+
+bool treenode::add_shared_owner(TXNID txnid) {
+ assert(m_is_shared);
+ if (txnid == m_txnid)
+ return false; // acquiring a lock on the same range by the same trx
+
+ if (m_txnid != TXNID_SHARED) {
+ m_owners = new TxnidVector;
+ m_owners->insert(m_txnid);
+ m_txnid = TXNID_SHARED;
+ }
+ m_owners->insert(txnid);
+ return true;
+}
+
+void treenode::free(treenode *node) {
+ // destroy the range, freeing any copied keys
+ node->m_range.destroy();
+
+ if (node->m_owners) {
+ delete node->m_owners;
+ node->m_owners = nullptr; // need this?
+ }
+
+ // the root is simply marked as empty.
+ if (node->is_root()) {
+ // PORT toku_mutex_assert_locked(&node->m_mutex);
+ node->m_is_empty = true;
+ } else {
+ // PORT toku_mutex_assert_unlocked(&node->m_mutex);
+ toku_mutex_destroy(&node->m_mutex);
+ toku_free(node);
+ }
+}
+
+uint32_t treenode::get_depth_estimate(void) const {
+ const uint32_t left_est = m_left_child.depth_est;
+ const uint32_t right_est = m_right_child.depth_est;
+ return (left_est > right_est ? left_est : right_est) + 1;
+}
+
+treenode *treenode::find_node_with_overlapping_child(
+ const keyrange &range, const keyrange::comparison *cmp_hint) {
+ // determine which child to look at based on a comparison. if we were
+ // given a comparison hint, use that. otherwise, compare them now.
+ keyrange::comparison c =
+ cmp_hint ? *cmp_hint : range.compare(*m_cmp, m_range);
+
+ treenode *child;
+ if (c == keyrange::comparison::LESS_THAN) {
+ child = lock_and_rebalance_left();
+ } else {
+ // The caller (locked_keyrange::acquire) handles the case where
+ // the root of the locked_keyrange is the node that overlaps.
+ // range is guaranteed not to overlap this node.
+ invariant(c == keyrange::comparison::GREATER_THAN);
+ child = lock_and_rebalance_right();
+ }
+
+ // if the search would lead us to an empty subtree (child == nullptr),
+ // or the child overlaps, then we know this node is the parent we want.
+ // otherwise we need to recur into that child.
+ if (child == nullptr) {
+ return this;
+ } else {
+ c = range.compare(*m_cmp, child->m_range);
+ if (c == keyrange::comparison::EQUALS ||
+ c == keyrange::comparison::OVERLAPS) {
+ child->mutex_unlock();
+ return this;
+ } else {
+ // unlock this node before recurring into the locked child,
+ // passing in a comparison hint since we just comapred range
+ // to the child's range.
+ mutex_unlock();
+ return child->find_node_with_overlapping_child(range, &c);
+ }
+ }
+}
+
+bool treenode::insert(const keyrange &range, TXNID txnid, bool is_shared) {
+ int rc = true;
+ // choose a child to check. if that child is null, then insert the new node
+ // there. otherwise recur down that child's subtree
+ keyrange::comparison c = range.compare(*m_cmp, m_range);
+ if (c == keyrange::comparison::LESS_THAN) {
+ treenode *left_child = lock_and_rebalance_left();
+ if (left_child == nullptr) {
+ left_child = treenode::alloc(m_cmp, range, txnid, is_shared);
+ m_left_child.set(left_child);
+ } else {
+ left_child->insert(range, txnid, is_shared);
+ left_child->mutex_unlock();
+ }
+ } else if (c == keyrange::comparison::GREATER_THAN) {
+ // invariant(c == keyrange::comparison::GREATER_THAN);
+ treenode *right_child = lock_and_rebalance_right();
+ if (right_child == nullptr) {
+ right_child = treenode::alloc(m_cmp, range, txnid, is_shared);
+ m_right_child.set(right_child);
+ } else {
+ right_child->insert(range, txnid, is_shared);
+ right_child->mutex_unlock();
+ }
+ } else if (c == keyrange::comparison::EQUALS) {
+ invariant(is_shared);
+ invariant(m_is_shared);
+ rc = add_shared_owner(txnid);
+ } else {
+ invariant(0);
+ }
+ return rc;
+}
+
+treenode *treenode::find_child_at_extreme(int direction, treenode **parent) {
+ treenode *child =
+ direction > 0 ? m_right_child.get_locked() : m_left_child.get_locked();
+
+ if (child) {
+ *parent = this;
+ treenode *child_extreme = child->find_child_at_extreme(direction, parent);
+ child->mutex_unlock();
+ return child_extreme;
+ } else {
+ return this;
+ }
+}
+
+treenode *treenode::find_leftmost_child(treenode **parent) {
+ return find_child_at_extreme(-1, parent);
+}
+
+treenode *treenode::find_rightmost_child(treenode **parent) {
+ return find_child_at_extreme(1, parent);
+}
+
+treenode *treenode::remove_root_of_subtree() {
+ // if this node has no children, just free it and return null
+ if (m_left_child.ptr == nullptr && m_right_child.ptr == nullptr) {
+ // treenode::free requires that non-root nodes are unlocked
+ if (!is_root()) {
+ mutex_unlock();
+ }
+ treenode::free(this);
+ return nullptr;
+ }
+
+ // we have a child, so get either the in-order successor or
+ // predecessor of this node to be our replacement.
+ // replacement_parent is updated by the find functions as
+ // they recur down the tree, so initialize it to this.
+ treenode *child, *replacement;
+ treenode *replacement_parent = this;
+ if (m_left_child.ptr != nullptr) {
+ child = m_left_child.get_locked();
+ replacement = child->find_rightmost_child(&replacement_parent);
+ invariant(replacement == child || replacement_parent != this);
+
+ // detach the replacement from its parent
+ if (replacement_parent == this) {
+ m_left_child = replacement->m_left_child;
+ } else {
+ replacement_parent->m_right_child = replacement->m_left_child;
+ }
+ } else {
+ child = m_right_child.get_locked();
+ replacement = child->find_leftmost_child(&replacement_parent);
+ invariant(replacement == child || replacement_parent != this);
+
+ // detach the replacement from its parent
+ if (replacement_parent == this) {
+ m_right_child = replacement->m_right_child;
+ } else {
+ replacement_parent->m_left_child = replacement->m_right_child;
+ }
+ }
+ child->mutex_unlock();
+
+ // swap in place with the detached replacement, then destroy it
+ treenode::swap_in_place(replacement, this);
+ treenode::free(replacement);
+
+ return this;
+}
+
+void treenode::recursive_remove(void) {
+ treenode *left = m_left_child.ptr;
+ if (left) {
+ left->recursive_remove();
+ }
+ m_left_child.set(nullptr);
+
+ treenode *right = m_right_child.ptr;
+ if (right) {
+ right->recursive_remove();
+ }
+ m_right_child.set(nullptr);
+
+ // we do not take locks on the way down, so we know non-root nodes
+ // are unlocked here and the caller is required to pass a locked
+ // root, so this free is correct.
+ treenode::free(this);
+}
+
+void treenode::remove_shared_owner(TXNID txnid) {
+ assert(m_owners->size() > 1);
+ m_owners->erase(txnid);
+ assert(m_owners->size() > 0);
+ /* if there is just one owner left, move it to m_txnid */
+ if (m_owners->size() == 1) {
+ m_txnid = *m_owners->begin();
+ delete m_owners;
+ m_owners = nullptr;
+ }
+}
+
+treenode *treenode::remove(const keyrange &range, TXNID txnid) {
+ treenode *child;
+ // if the range is equal to this node's range, then just remove
+ // the root of this subtree. otherwise search down the tree
+ // in either the left or right children.
+ keyrange::comparison c = range.compare(*m_cmp, m_range);
+ switch (c) {
+ case keyrange::comparison::EQUALS: {
+ // if we are the only owners, remove. Otherwise, just remove
+ // us from the owners list.
+ if (txnid != TXNID_ANY && has_multiple_owners()) {
+ remove_shared_owner(txnid);
+ return this;
+ } else {
+ return remove_root_of_subtree();
+ }
+ }
+ case keyrange::comparison::LESS_THAN:
+ child = m_left_child.get_locked();
+ invariant_notnull(child);
+ child = child->remove(range, txnid);
+
+ // unlock the child if there still is one.
+ // regardless, set the right child pointer
+ if (child) {
+ child->mutex_unlock();
+ }
+ m_left_child.set(child);
+ break;
+ case keyrange::comparison::GREATER_THAN:
+ child = m_right_child.get_locked();
+ invariant_notnull(child);
+ child = child->remove(range, txnid);
+
+ // unlock the child if there still is one.
+ // regardless, set the right child pointer
+ if (child) {
+ child->mutex_unlock();
+ }
+ m_right_child.set(child);
+ break;
+ case keyrange::comparison::OVERLAPS:
+ // shouldn't be overlapping, since the tree is
+ // non-overlapping and this range must exist
+ abort();
+ }
+
+ return this;
+}
+
+bool treenode::left_imbalanced(int threshold) const {
+ uint32_t left_depth = m_left_child.depth_est;
+ uint32_t right_depth = m_right_child.depth_est;
+ return m_left_child.ptr != nullptr && left_depth > threshold + right_depth;
+}
+
+bool treenode::right_imbalanced(int threshold) const {
+ uint32_t left_depth = m_left_child.depth_est;
+ uint32_t right_depth = m_right_child.depth_est;
+ return m_right_child.ptr != nullptr && right_depth > threshold + left_depth;
+}
+
+// effect: rebalances the subtree rooted at this node
+// using AVL style O(1) rotations. unlocks this
+// node if it is not the new root of the subtree.
+// requires: node is locked by this thread, children are not
+// returns: locked root node of the rebalanced tree
+treenode *treenode::maybe_rebalance(void) {
+ // if we end up not rotating at all, the new root is this
+ treenode *new_root = this;
+ treenode *child = nullptr;
+
+ if (left_imbalanced(IMBALANCE_THRESHOLD)) {
+ child = m_left_child.get_locked();
+ if (child->right_imbalanced(0)) {
+ treenode *grandchild = child->m_right_child.get_locked();
+
+ child->m_right_child = grandchild->m_left_child;
+ grandchild->m_left_child.set(child);
+
+ m_left_child = grandchild->m_right_child;
+ grandchild->m_right_child.set(this);
+
+ new_root = grandchild;
+ } else {
+ m_left_child = child->m_right_child;
+ child->m_right_child.set(this);
+ new_root = child;
+ }
+ } else if (right_imbalanced(IMBALANCE_THRESHOLD)) {
+ child = m_right_child.get_locked();
+ if (child->left_imbalanced(0)) {
+ treenode *grandchild = child->m_left_child.get_locked();
+
+ child->m_left_child = grandchild->m_right_child;
+ grandchild->m_right_child.set(child);
+
+ m_right_child = grandchild->m_left_child;
+ grandchild->m_left_child.set(this);
+
+ new_root = grandchild;
+ } else {
+ m_right_child = child->m_left_child;
+ child->m_left_child.set(this);
+ new_root = child;
+ }
+ }
+
+ // up to three nodes may be locked.
+ // - this
+ // - child
+ // - grandchild (but if it is locked, its the new root)
+ //
+ // one of them is the new root. we unlock everything except the new root.
+ if (child && child != new_root) {
+ TOKU_VALGRIND_RESET_MUTEX_ORDERING_INFO(&child->m_mutex);
+ child->mutex_unlock();
+ }
+ if (this != new_root) {
+ TOKU_VALGRIND_RESET_MUTEX_ORDERING_INFO(&m_mutex);
+ mutex_unlock();
+ }
+ TOKU_VALGRIND_RESET_MUTEX_ORDERING_INFO(&new_root->m_mutex);
+ return new_root;
+}
+
+treenode *treenode::lock_and_rebalance_left(void) {
+ treenode *child = m_left_child.get_locked();
+ if (child) {
+ treenode *new_root = child->maybe_rebalance();
+ m_left_child.set(new_root);
+ child = new_root;
+ }
+ return child;
+}
+
+treenode *treenode::lock_and_rebalance_right(void) {
+ treenode *child = m_right_child.get_locked();
+ if (child) {
+ treenode *new_root = child->maybe_rebalance();
+ m_right_child.set(new_root);
+ child = new_root;
+ }
+ return child;
+}
+
+void treenode::child_ptr::set(treenode *node) {
+ ptr = node;
+ depth_est = ptr ? ptr->get_depth_estimate() : 0;
+}
+
+treenode *treenode::child_ptr::get_locked(void) {
+ if (ptr) {
+ ptr->mutex_lock();
+ depth_est = ptr->get_depth_estimate();
+ }
+ return ptr;
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.h
new file mode 100644
index 000000000..ec25a8c58
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.h
@@ -0,0 +1,302 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=2:softtabstop=2:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <string.h>
+
+#include "../ft/comparator.h"
+#include "../portability/memory.h"
+#include "../portability/toku_pthread.h"
+// PORT: we need LTM_STATUS
+#include "../ft/ft-status.h"
+#include "../portability/txn_subst.h"
+#include "keyrange.h"
+
+namespace toku {
+
+// a node in a tree with its own mutex
+// - range is the "key" of this node
+// - txnid is the single txnid associated with this node
+// - left and right children may be null
+//
+// to build a tree on top of this abstraction, the user:
+// - provides memory for a root node, initializes it via create_root()
+// - performs tree operations on the root node. memory management
+// below the root node is handled by the abstraction, not the user.
+// this pattern:
+// - guaruntees a root node always exists.
+// - does not allow for rebalances on the root node
+
+class treenode {
+ public:
+ // every treenode function has some common requirements:
+ // - node is locked and children are never locked
+ // - node may be unlocked if no other thread has visibility
+
+ // effect: create the root node
+ void create_root(const comparator *cmp);
+
+ // effect: destroys the root node
+ void destroy_root(void);
+
+ // effect: sets the txnid and copies the given range for this node
+ void set_range_and_txnid(const keyrange &range, TXNID txnid, bool is_shared);
+
+ // returns: true iff this node is marked as empty
+ bool is_empty(void);
+
+ // returns: true if this is the root node, denoted by a null parent
+ bool is_root(void);
+
+ // returns: true if the given range overlaps with this node's range
+ bool range_overlaps(const keyrange &range);
+
+ // effect: locks the node
+ void mutex_lock(void);
+
+ // effect: unlocks the node
+ void mutex_unlock(void);
+
+ // return: node whose child overlaps, or a child that is empty
+ // and would contain range if it existed
+ // given: if cmp_hint is non-null, then it is a precomputed
+ // comparison of this node's range to the given range.
+ treenode *find_node_with_overlapping_child(
+ const keyrange &range, const keyrange::comparison *cmp_hint);
+
+ // effect: performs an in-order traversal of the ranges that overlap the
+ // given range, calling function->fn() on each node that does
+ // requires: function signature is: bool fn(const keyrange &range, TXNID
+ // txnid) requires: fn returns true to keep iterating, false to stop iterating
+ // requires: fn does not attempt to use any ranges read out by value
+ // after removing a node with an overlapping range from the tree.
+ template <class F>
+ void traverse_overlaps(const keyrange &range, F *function) {
+ keyrange::comparison c = range.compare(*m_cmp, m_range);
+ if (c == keyrange::comparison::EQUALS) {
+ // Doesn't matter if fn wants to keep going, there
+ // is nothing left, so return.
+ function->fn(m_range, m_txnid, m_is_shared, m_owners);
+ return;
+ }
+
+ treenode *left = m_left_child.get_locked();
+ if (left) {
+ if (c != keyrange::comparison::GREATER_THAN) {
+ // Target range is less than this node, or it overlaps this
+ // node. There may be something on the left.
+ left->traverse_overlaps(range, function);
+ }
+ left->mutex_unlock();
+ }
+
+ if (c == keyrange::comparison::OVERLAPS) {
+ bool keep_going = function->fn(m_range, m_txnid, m_is_shared, m_owners);
+ if (!keep_going) {
+ return;
+ }
+ }
+
+ treenode *right = m_right_child.get_locked();
+ if (right) {
+ if (c != keyrange::comparison::LESS_THAN) {
+ // Target range is greater than this node, or it overlaps this
+ // node. There may be something on the right.
+ right->traverse_overlaps(range, function);
+ }
+ right->mutex_unlock();
+ }
+ }
+
+ // effect: inserts the given range and txnid into a subtree, recursively
+ // requires: range does not overlap with any node below the subtree
+ bool insert(const keyrange &range, TXNID txnid, bool is_shared);
+
+ // effect: removes the given range from the subtree
+ // requires: range exists in the subtree
+ // returns: the root of the resulting subtree
+ treenode *remove(const keyrange &range, TXNID txnid);
+
+ // effect: removes this node and all of its children, recursively
+ // requires: every node at and below this node is unlocked
+ void recursive_remove(void);
+
+ private:
+ // the child_ptr is a light abstraction for the locking of
+ // a child and the maintenence of its depth estimate.
+
+ struct child_ptr {
+ // set the child pointer
+ void set(treenode *node);
+
+ // get and lock this child if it exists
+ treenode *get_locked(void);
+
+ treenode *ptr;
+ uint32_t depth_est;
+ };
+
+ // the balance factor at which a node is considered imbalanced
+ static const int32_t IMBALANCE_THRESHOLD = 2;
+
+ // node-level mutex
+ toku_mutex_t m_mutex;
+
+ // the range and txnid for this node. the range contains a copy
+ // of the keys originally inserted into the tree. nodes may
+ // swap ranges. but at the end of the day, when a node is
+ // destroyed, it frees the memory associated with whatever range
+ // it has at the time of destruction.
+ keyrange m_range;
+
+ void remove_shared_owner(TXNID txnid);
+
+ bool has_multiple_owners() { return (m_txnid == TXNID_SHARED); }
+
+ private:
+ // Owner transaction id.
+ // A value of TXNID_SHARED means this node has multiple owners
+ TXNID m_txnid;
+
+ // If true, this lock is a non-exclusive lock, and it can have either
+ // one or several owners.
+ bool m_is_shared;
+
+ // List of the owners, or nullptr if there's just one owner.
+ TxnidVector *m_owners;
+
+ // two child pointers
+ child_ptr m_left_child;
+ child_ptr m_right_child;
+
+ // comparator for ranges
+ // psergey-todo: Is there any sense to store the comparator in each tree
+ // node?
+ const comparator *m_cmp;
+
+ // marked for the root node. the root node is never free()'d
+ // when removed, but instead marked as empty.
+ bool m_is_root;
+
+ // marked for an empty node. only valid for the root.
+ bool m_is_empty;
+
+ // effect: initializes an empty node with the given comparator
+ void init(const comparator *cmp);
+
+ // requires: this is a shared node (m_is_shared==true)
+ // effect: another transaction is added as an owner.
+ // returns: true <=> added another owner
+ // false <=> this transaction is already an owner
+ bool add_shared_owner(TXNID txnid);
+
+ // requires: *parent is initialized to something meaningful.
+ // requires: subtree is non-empty
+ // returns: the leftmost child of the given subtree
+ // returns: a pointer to the parent of said child in *parent, only
+ // if this function recurred, otherwise it is untouched.
+ treenode *find_leftmost_child(treenode **parent);
+
+ // requires: *parent is initialized to something meaningful.
+ // requires: subtree is non-empty
+ // returns: the rightmost child of the given subtree
+ // returns: a pointer to the parent of said child in *parent, only
+ // if this function recurred, otherwise it is untouched.
+ treenode *find_rightmost_child(treenode **parent);
+
+ // effect: remove the root of this subtree, destroying the old root
+ // returns: the new root of the subtree
+ treenode *remove_root_of_subtree(void);
+
+ // requires: subtree is non-empty, direction is not 0
+ // returns: the child of the subtree at either the left or rightmost extreme
+ treenode *find_child_at_extreme(int direction, treenode **parent);
+
+ // effect: retrieves and possibly rebalances the left child
+ // returns: a locked left child, if it exists
+ treenode *lock_and_rebalance_left(void);
+
+ // effect: retrieves and possibly rebalances the right child
+ // returns: a locked right child, if it exists
+ treenode *lock_and_rebalance_right(void);
+
+ // returns: the estimated depth of this subtree
+ uint32_t get_depth_estimate(void) const;
+
+ // returns: true iff left subtree depth is sufficiently less than the right
+ bool left_imbalanced(int threshold) const;
+
+ // returns: true iff right subtree depth is sufficiently greater than the left
+ bool right_imbalanced(int threshold) const;
+
+ // effect: performs an O(1) rebalance, which will "heal" an imbalance by at
+ // most 1. effect: if the new root is not this node, then this node is
+ // unlocked. returns: locked node representing the new root of the rebalanced
+ // subtree
+ treenode *maybe_rebalance(void);
+
+ // returns: allocated treenode populated with a copy of the range and txnid
+ static treenode *alloc(const comparator *cmp, const keyrange &range,
+ TXNID txnid, bool is_shared);
+
+ // requires: node is a locked root node, or an unlocked non-root node
+ static void free(treenode *node);
+
+ // effect: swaps the range/txnid pairs for node1 and node2.
+ static void swap_in_place(treenode *node1, treenode *node2);
+
+ friend class concurrent_tree_unit_test;
+};
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.cc
new file mode 100644
index 000000000..4caf1e26f
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.cc
@@ -0,0 +1,120 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "txnid_set.h"
+
+#include "../db.h"
+
+namespace toku {
+
+int find_by_txnid(const TXNID &txnid_a, const TXNID &txnid_b);
+int find_by_txnid(const TXNID &txnid_a, const TXNID &txnid_b) {
+ if (txnid_a < txnid_b) {
+ return -1;
+ } else if (txnid_a == txnid_b) {
+ return 0;
+ } else {
+ return 1;
+ }
+}
+
+void txnid_set::create(void) {
+ // lazily allocate the underlying omt, since it is common
+ // to create a txnid set and never put anything in it.
+ m_txnids.create_no_array();
+}
+
+void txnid_set::destroy(void) { m_txnids.destroy(); }
+
+// Return true if the given transaction id is a member of the set.
+// Otherwise, return false.
+bool txnid_set::contains(TXNID txnid) const {
+ TXNID find_txnid;
+ int r = m_txnids.find_zero<TXNID, find_by_txnid>(txnid, &find_txnid, nullptr);
+ return r == 0 ? true : false;
+}
+
+// Add a given txnid to the set
+void txnid_set::add(TXNID txnid) {
+ int r = m_txnids.insert<TXNID, find_by_txnid>(txnid, txnid, nullptr);
+ invariant(r == 0 || r == DB_KEYEXIST);
+}
+
+// Delete a given txnid from the set.
+void txnid_set::remove(TXNID txnid) {
+ uint32_t idx;
+ int r = m_txnids.find_zero<TXNID, find_by_txnid>(txnid, nullptr, &idx);
+ if (r == 0) {
+ r = m_txnids.delete_at(idx);
+ invariant_zero(r);
+ }
+}
+
+// Return the size of the set
+uint32_t txnid_set::size(void) const { return m_txnids.size(); }
+
+// Get the ith id in the set, assuming that the set is sorted.
+TXNID txnid_set::get(uint32_t i) const {
+ TXNID txnid;
+ int r = m_txnids.fetch(i, &txnid);
+ if (r == EINVAL) /* Shouldn't happen, avoid compiler warning */
+ return TXNID_NONE;
+ invariant_zero(r);
+ return txnid;
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.h
new file mode 100644
index 000000000..d79c24fb0
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.h
@@ -0,0 +1,92 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include "../portability/txn_subst.h"
+#include "../util/omt.h"
+
+namespace toku {
+
+class txnid_set {
+ public:
+ // effect: Creates an empty set. Does not malloc space for
+ // any entries yet. That is done lazily on add().
+ void create(void);
+
+ // effect: Destroy the set's internals.
+ void destroy(void);
+
+ // returns: True if the given txnid is a member of the set.
+ bool contains(TXNID id) const;
+
+ // effect: Adds a given txnid to the set if it did not exist
+ void add(TXNID txnid);
+
+ // effect: Deletes a txnid from the set if it exists.
+ void remove(TXNID txnid);
+
+ // returns: Size of the set
+ uint32_t size(void) const;
+
+ // returns: The "i'th" id in the set, as if it were sorted.
+ TXNID get(uint32_t i) const;
+
+ private:
+ toku::omt<TXNID> m_txnids;
+
+ friend class txnid_set_unit_test;
+};
+ENSURE_POD(txnid_set);
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.cc
new file mode 100644
index 000000000..24536c88e
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.cc
@@ -0,0 +1,213 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "../db.h"
+#include "../portability/memory.h"
+// PORT #include <toku_assert.h>
+#include <memory.h>
+#include <string.h>
+
+#include "txnid_set.h"
+#include "wfg.h"
+
+namespace toku {
+
+// Create a lock request graph
+void wfg::create(void) { m_nodes.create(); }
+
+// Destroy the internals of the lock request graph
+void wfg::destroy(void) {
+ uint32_t n_nodes = m_nodes.size();
+ for (uint32_t i = 0; i < n_nodes; i++) {
+ node *n;
+ int r = m_nodes.fetch(i, &n);
+ invariant_zero(r);
+ invariant_notnull(n);
+ if (r) continue; // Get rid of "may be used uninitialized" warning
+ node::free(n);
+ }
+ m_nodes.destroy();
+}
+
+// Add an edge (a_id, b_id) to the graph
+void wfg::add_edge(TXNID a_txnid, TXNID b_txnid) {
+ node *a_node = find_create_node(a_txnid);
+ node *b_node = find_create_node(b_txnid);
+ a_node->edges.add(b_node->txnid);
+}
+
+// Return true if a node with the given transaction id exists in the graph.
+// Return false otherwise.
+bool wfg::node_exists(TXNID txnid) {
+ node *n = find_node(txnid);
+ return n != NULL;
+}
+
+bool wfg::cycle_exists_from_node(node *target, node *head,
+ std::function<void(TXNID)> reporter) {
+ bool cycle_found = false;
+ head->visited = true;
+ uint32_t n_edges = head->edges.size();
+ for (uint32_t i = 0; i < n_edges && !cycle_found; i++) {
+ TXNID edge_id = head->edges.get(i);
+ if (target->txnid == edge_id) {
+ cycle_found = true;
+ if (reporter) reporter(edge_id);
+ } else {
+ node *new_head = find_node(edge_id);
+ if (new_head && !new_head->visited) {
+ cycle_found = cycle_exists_from_node(target, new_head, reporter);
+ if (cycle_found && reporter) reporter(edge_id);
+ }
+ }
+ }
+ head->visited = false;
+ return cycle_found;
+}
+
+// Return true if there exists a cycle from a given transaction id in the graph.
+// Return false otherwise.
+bool wfg::cycle_exists_from_txnid(TXNID txnid,
+ std::function<void(TXNID)> reporter) {
+ node *a_node = find_node(txnid);
+ bool cycles_found = false;
+ if (a_node) {
+ cycles_found = cycle_exists_from_node(a_node, a_node, reporter);
+ }
+ return cycles_found;
+}
+
+// Apply a given function f to all of the nodes in the graph. The apply
+// function returns when the function f is called for all of the nodes in the
+// graph, or the function f returns non-zero.
+void wfg::apply_nodes(int (*fn)(TXNID id, void *extra), void *extra) {
+ int r = 0;
+ uint32_t n_nodes = m_nodes.size();
+ for (uint32_t i = 0; i < n_nodes && r == 0; i++) {
+ node *n;
+ r = m_nodes.fetch(i, &n);
+ invariant_zero(r);
+ if (r) continue; // Get rid of "may be used uninitialized" warning
+ r = fn(n->txnid, extra);
+ }
+}
+
+// Apply a given function f to all of the edges whose origin is a given node id.
+// The apply function returns when the function f is called for all edges in the
+// graph rooted at node id, or the function f returns non-zero.
+void wfg::apply_edges(TXNID txnid,
+ int (*fn)(TXNID txnid, TXNID edge_txnid, void *extra),
+ void *extra) {
+ node *n = find_node(txnid);
+ if (n) {
+ int r = 0;
+ uint32_t n_edges = n->edges.size();
+ for (uint32_t i = 0; i < n_edges && r == 0; i++) {
+ r = fn(txnid, n->edges.get(i), extra);
+ }
+ }
+}
+
+// find node by id
+wfg::node *wfg::find_node(TXNID txnid) {
+ node *n = nullptr;
+ int r = m_nodes.find_zero<TXNID, find_by_txnid>(txnid, &n, nullptr);
+ invariant(r == 0 || r == DB_NOTFOUND);
+ return n;
+}
+
+// this is the omt comparison function
+// nodes are compared by their txnid.
+int wfg::find_by_txnid(node *const &node_a, const TXNID &txnid_b) {
+ TXNID txnid_a = node_a->txnid;
+ if (txnid_a < txnid_b) {
+ return -1;
+ } else if (txnid_a == txnid_b) {
+ return 0;
+ } else {
+ return 1;
+ }
+}
+
+// insert a new node
+wfg::node *wfg::find_create_node(TXNID txnid) {
+ node *n;
+ uint32_t idx;
+ int r = m_nodes.find_zero<TXNID, find_by_txnid>(txnid, &n, &idx);
+ if (r == DB_NOTFOUND) {
+ n = node::alloc(txnid);
+ r = m_nodes.insert_at(n, idx);
+ invariant_zero(r);
+ }
+ invariant_notnull(n);
+ return n;
+}
+
+wfg::node *wfg::node::alloc(TXNID txnid) {
+ node *XCALLOC(n);
+ n->txnid = txnid;
+ n->visited = false;
+ n->edges.create();
+ return n;
+}
+
+void wfg::node::free(wfg::node *n) {
+ n->edges.destroy();
+ toku_free(n);
+}
+
+} /* namespace toku */
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.h
new file mode 100644
index 000000000..804202170
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.h
@@ -0,0 +1,124 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <functional>
+
+#include "../util/omt.h"
+#include "txnid_set.h"
+
+namespace toku {
+
+// A wfg is a 'wait-for' graph. A directed edge in represents one
+// txn waiting for another to finish before it can acquire a lock.
+
+class wfg {
+ public:
+ // Create a lock request graph
+ void create(void);
+
+ // Destroy the internals of the lock request graph
+ void destroy(void);
+
+ // Add an edge (a_id, b_id) to the graph
+ void add_edge(TXNID a_txnid, TXNID b_txnid);
+
+ // Return true if a node with the given transaction id exists in the graph.
+ // Return false otherwise.
+ bool node_exists(TXNID txnid);
+
+ // Return true if there exists a cycle from a given transaction id in the
+ // graph. Return false otherwise.
+ bool cycle_exists_from_txnid(TXNID txnid,
+ std::function<void(TXNID)> reporter);
+
+ // Apply a given function f to all of the nodes in the graph. The apply
+ // function returns when the function f is called for all of the nodes in the
+ // graph, or the function f returns non-zero.
+ void apply_nodes(int (*fn)(TXNID txnid, void *extra), void *extra);
+
+ // Apply a given function f to all of the edges whose origin is a given node
+ // id. The apply function returns when the function f is called for all edges
+ // in the graph rooted at node id, or the function f returns non-zero.
+ void apply_edges(TXNID txnid,
+ int (*fn)(TXNID txnid, TXNID edge_txnid, void *extra),
+ void *extra);
+
+ private:
+ struct node {
+ // txnid for this node and the associated set of edges
+ TXNID txnid;
+ txnid_set edges;
+ bool visited;
+
+ static node *alloc(TXNID txnid);
+
+ static void free(node *n);
+ };
+ ENSURE_POD(node);
+
+ toku::omt<node *> m_nodes;
+
+ node *find_node(TXNID txnid);
+
+ node *find_create_node(TXNID txnid);
+
+ bool cycle_exists_from_node(node *target, node *head,
+ std::function<void(TXNID)> reporter);
+
+ static int find_by_txnid(node *const &node_a, const TXNID &txnid_b);
+};
+ENSURE_POD(wfg);
+
+} /* namespace toku */
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/memory.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/memory.h
new file mode 100644
index 000000000..0a621f8e0
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/memory.h
@@ -0,0 +1,215 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <stdlib.h>
+
+#include "toku_portability.h"
+
+/* Percona memory allocation functions and macros.
+ * These are functions for malloc and free */
+
+int toku_memory_startup(void) __attribute__((constructor));
+void toku_memory_shutdown(void) __attribute__((destructor));
+
+/* Generally: errno is set to 0 or a value to indicate problems. */
+
+// Everything should call toku_malloc() instead of malloc(), and toku_calloc()
+// instead of calloc() That way the tests can can, e.g., replace the malloc
+// function using toku_set_func_malloc().
+void *toku_calloc(size_t nmemb, size_t size)
+ __attribute__((__visibility__("default")));
+void *toku_xcalloc(size_t nmemb, size_t size)
+ __attribute__((__visibility__("default")));
+void *toku_malloc(size_t size) __attribute__((__visibility__("default")));
+void *toku_malloc_aligned(size_t alignment, size_t size)
+ __attribute__((__visibility__("default")));
+
+// xmalloc aborts instead of return NULL if we run out of memory
+void *toku_xmalloc(size_t size) __attribute__((__visibility__("default")));
+void *toku_xrealloc(void *, size_t size)
+ __attribute__((__visibility__("default")));
+void *toku_xmalloc_aligned(size_t alignment, size_t size)
+ __attribute__((__visibility__("default")));
+// Effect: Perform a os_malloc_aligned(size) with the additional property that
+// the returned pointer is a multiple of ALIGNMENT.
+// Fail with a resource_assert if the allocation fails (don't return an error
+// code). If the alloc_aligned function has been set then call it instead.
+// Requires: alignment is a power of two.
+
+void toku_free(void *) __attribute__((__visibility__("default")));
+
+size_t toku_malloc_usable_size(void *p)
+ __attribute__((__visibility__("default")));
+
+/* MALLOC is a macro that helps avoid a common error:
+ * Suppose I write
+ * struct foo *x = malloc(sizeof(struct foo));
+ * That works fine. But if I change it to this, I've probably made an mistake:
+ * struct foo *x = malloc(sizeof(struct bar));
+ * It can get worse, since one might have something like
+ * struct foo *x = malloc(sizeof(struct foo *))
+ * which looks reasonable, but it allocoates enough to hold a pointer instead of
+ * the amount needed for the struct. So instead, write struct foo *MALLOC(x);
+ * and you cannot go wrong.
+ */
+#define MALLOC(v) CAST_FROM_VOIDP(v, toku_malloc(sizeof(*v)))
+/* MALLOC_N is like calloc(Except no 0ing of data): It makes an array. Write
+ * int *MALLOC_N(5,x);
+ * to make an array of 5 integers.
+ */
+#define MALLOC_N(n, v) CAST_FROM_VOIDP(v, toku_malloc((n) * sizeof(*v)))
+#define MALLOC_N_ALIGNED(align, n, v) \
+ CAST_FROM_VOIDP(v, toku_malloc_aligned((align), (n) * sizeof(*v)))
+
+// CALLOC_N is like calloc with auto-figuring out size of members
+#define CALLOC_N(n, v) CAST_FROM_VOIDP(v, toku_calloc((n), sizeof(*v)))
+
+#define CALLOC(v) CALLOC_N(1, v)
+
+// XMALLOC macros are like MALLOC except they abort if the operation fails
+#define XMALLOC(v) CAST_FROM_VOIDP(v, toku_xmalloc(sizeof(*v)))
+#define XMALLOC_N(n, v) CAST_FROM_VOIDP(v, toku_xmalloc((n) * sizeof(*v)))
+#define XCALLOC_N(n, v) CAST_FROM_VOIDP(v, toku_xcalloc((n), (sizeof(*v))))
+#define XCALLOC(v) XCALLOC_N(1, v)
+#define XREALLOC(v, s) CAST_FROM_VOIDP(v, toku_xrealloc(v, s))
+#define XREALLOC_N(n, v) CAST_FROM_VOIDP(v, toku_xrealloc(v, (n) * sizeof(*v)))
+
+#define XMALLOC_N_ALIGNED(align, n, v) \
+ CAST_FROM_VOIDP(v, toku_xmalloc_aligned((align), (n) * sizeof(*v)))
+
+#define XMEMDUP(dst, src) CAST_FROM_VOIDP(dst, toku_xmemdup(src, sizeof(*src)))
+#define XMEMDUP_N(dst, src, len) CAST_FROM_VOIDP(dst, toku_xmemdup(src, len))
+
+// ZERO_ARRAY writes zeroes to a stack-allocated array
+#define ZERO_ARRAY(o) \
+ do { \
+ memset((o), 0, sizeof(o)); \
+ } while (0)
+// ZERO_STRUCT writes zeroes to a stack-allocated struct
+#define ZERO_STRUCT(o) \
+ do { \
+ memset(&(o), 0, sizeof(o)); \
+ } while (0)
+
+/* Copy memory. Analogous to strdup() */
+void *toku_memdup(const void *v, size_t len);
+/* Toku-version of strdup. Use this so that it calls toku_malloc() */
+char *toku_strdup(const char *s) __attribute__((__visibility__("default")));
+/* Toku-version of strndup. Use this so that it calls toku_malloc() */
+char *toku_strndup(const char *s, size_t n)
+ __attribute__((__visibility__("default")));
+/* Copy memory. Analogous to strdup() Crashes instead of returning NULL */
+void *toku_xmemdup(const void *v, size_t len)
+ __attribute__((__visibility__("default")));
+/* Toku-version of strdup. Use this so that it calls toku_xmalloc() Crashes
+ * instead of returning NULL */
+char *toku_xstrdup(const char *s) __attribute__((__visibility__("default")));
+
+void toku_malloc_cleanup(
+ void); /* Before exiting, call this function to free up any internal data
+ structures from toku_malloc. Otherwise valgrind will complain of
+ memory leaks. */
+
+/* Check to see if everything malloc'd was free. Might be a no-op depending on
+ * how memory.c is configured. */
+void toku_memory_check_all_free(void);
+/* Check to see if memory is "sane". Might be a no-op. Probably better to
+ * simply use valgrind. */
+void toku_do_memory_check(void);
+
+typedef void *(*malloc_fun_t)(size_t);
+typedef void (*free_fun_t)(void *);
+typedef void *(*realloc_fun_t)(void *, size_t);
+typedef void *(*malloc_aligned_fun_t)(size_t /*alignment*/, size_t /*size*/);
+typedef void *(*realloc_aligned_fun_t)(size_t /*alignment*/, void * /*pointer*/,
+ size_t /*size*/);
+
+void toku_set_func_malloc(malloc_fun_t f);
+void toku_set_func_xmalloc_only(malloc_fun_t f);
+void toku_set_func_malloc_only(malloc_fun_t f);
+void toku_set_func_realloc(realloc_fun_t f);
+void toku_set_func_xrealloc_only(realloc_fun_t f);
+void toku_set_func_realloc_only(realloc_fun_t f);
+void toku_set_func_free(free_fun_t f);
+
+typedef struct memory_status {
+ uint64_t malloc_count; // number of malloc operations
+ uint64_t free_count; // number of free operations
+ uint64_t realloc_count; // number of realloc operations
+ uint64_t malloc_fail; // number of malloc operations that failed
+ uint64_t realloc_fail; // number of realloc operations that failed
+ uint64_t requested; // number of bytes requested
+ uint64_t used; // number of bytes used (requested + overhead), obtained from
+ // malloc_usable_size()
+ uint64_t freed; // number of bytes freed;
+ uint64_t max_requested_size; // largest attempted allocation size
+ uint64_t last_failed_size; // size of the last failed allocation attempt
+ volatile uint64_t
+ max_in_use; // maximum memory footprint (used - freed), approximate (not
+ // worth threadsafety overhead for exact)
+ const char *mallocator_version;
+ uint64_t mmap_threshold;
+} LOCAL_MEMORY_STATUS_S, *LOCAL_MEMORY_STATUS;
+
+void toku_memory_get_status(LOCAL_MEMORY_STATUS s);
+
+// Effect: Like toku_memory_footprint, except instead of passing p,
+// we pass toku_malloc_usable_size(p).
+size_t toku_memory_footprint_given_usable_size(size_t touched, size_t usable);
+
+// Effect: Return an estimate how how much space an object is using, possibly by
+// using toku_malloc_usable_size(p).
+// If p is NULL then returns 0.
+size_t toku_memory_footprint(void *p, size_t touched);
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_assert_subst.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_assert_subst.h
new file mode 100644
index 000000000..af47800fb
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_assert_subst.h
@@ -0,0 +1,39 @@
+//
+// A replacement for toku_assert.h
+//
+#pragma once
+
+#include <assert.h>
+#include <errno.h>
+
+#ifdef NDEBUG
+
+#define assert_zero(a) ((void)(a))
+#define invariant(a) ((void)(a))
+#define invariant_notnull(a) ((void)(a))
+#define invariant_zero(a) ((void)(a))
+
+#else
+
+#define assert_zero(a) assert((a) == 0)
+#define invariant(a) assert(a)
+#define invariant_notnull(a) assert(a)
+#define invariant_zero(a) assert_zero(a)
+
+#endif
+
+#define lazy_assert_zero(a) assert_zero(a)
+
+#define paranoid_invariant_zero(a) assert_zero(a)
+#define paranoid_invariant_notnull(a) assert(a)
+#define paranoid_invariant(a) assert(a)
+
+#define ENSURE_POD(type) \
+ static_assert( \
+ std::is_standard_layout<type>::value && std::is_trivial<type>::value, \
+ #type "isn't POD")
+
+inline int get_error_errno(void) {
+ invariant(errno);
+ return errno;
+}
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_atomic.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_atomic.h
new file mode 100644
index 000000000..aaa2298fa
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_atomic.h
@@ -0,0 +1,130 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+// PORT2: #include <portability/toku_config.h>
+#include <stdbool.h>
+#include <stddef.h>
+#include <stdint.h>
+
+#include "toku_assert_subst.h"
+
+__attribute__((const, always_inline)) static inline intptr_t which_cache_line(
+ intptr_t addr) {
+ static const size_t assumed_cache_line_size = 64;
+ return addr / assumed_cache_line_size;
+}
+template <typename T>
+__attribute__((const, always_inline)) static inline bool crosses_boundary(
+ T *addr, size_t width) {
+ const intptr_t int_addr = reinterpret_cast<intptr_t>(addr);
+ const intptr_t last_byte = int_addr + width - 1;
+ return which_cache_line(int_addr) != which_cache_line(last_byte);
+}
+
+template <typename T, typename U>
+__attribute__((always_inline)) static inline T toku_sync_fetch_and_add(T *addr,
+ U diff) {
+ paranoid_invariant(!crosses_boundary(addr, sizeof *addr));
+ return __sync_fetch_and_add(addr, diff);
+}
+template <typename T, typename U>
+__attribute__((always_inline)) static inline T toku_sync_add_and_fetch(T *addr,
+ U diff) {
+ paranoid_invariant(!crosses_boundary(addr, sizeof *addr));
+ return __sync_add_and_fetch(addr, diff);
+}
+template <typename T, typename U>
+__attribute__((always_inline)) static inline T toku_sync_fetch_and_sub(T *addr,
+ U diff) {
+ paranoid_invariant(!crosses_boundary(addr, sizeof *addr));
+ return __sync_fetch_and_sub(addr, diff);
+}
+template <typename T, typename U>
+__attribute__((always_inline)) static inline T toku_sync_sub_and_fetch(T *addr,
+ U diff) {
+ paranoid_invariant(!crosses_boundary(addr, sizeof *addr));
+ return __sync_sub_and_fetch(addr, diff);
+}
+template <typename T, typename U, typename V>
+__attribute__((always_inline)) static inline T toku_sync_val_compare_and_swap(
+ T *addr, U oldval, V newval) {
+ paranoid_invariant(!crosses_boundary(addr, sizeof *addr));
+ return __sync_val_compare_and_swap(addr, oldval, newval);
+}
+template <typename T, typename U, typename V>
+__attribute__((always_inline)) static inline bool
+toku_sync_bool_compare_and_swap(T *addr, U oldval, V newval) {
+ paranoid_invariant(!crosses_boundary(addr, sizeof *addr));
+ return __sync_bool_compare_and_swap(addr, oldval, newval);
+}
+
+// in case you include this but not toku_portability.h
+#pragma GCC poison __sync_fetch_and_add
+#pragma GCC poison __sync_fetch_and_sub
+#pragma GCC poison __sync_fetch_and_or
+#pragma GCC poison __sync_fetch_and_and
+#pragma GCC poison __sync_fetch_and_xor
+#pragma GCC poison __sync_fetch_and_nand
+#pragma GCC poison __sync_add_and_fetch
+#pragma GCC poison __sync_sub_and_fetch
+#pragma GCC poison __sync_or_and_fetch
+#pragma GCC poison __sync_and_and_fetch
+#pragma GCC poison __sync_xor_and_fetch
+#pragma GCC poison __sync_nand_and_fetch
+#pragma GCC poison __sync_bool_compare_and_swap
+#pragma GCC poison __sync_val_compare_and_swap
+#pragma GCC poison __sync_synchronize
+#pragma GCC poison __sync_lock_test_and_set
+#pragma GCC poison __sync_release
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_external_pthread.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_external_pthread.h
new file mode 100644
index 000000000..eb8291c1d
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_external_pthread.h
@@ -0,0 +1,83 @@
+/*
+ A wrapper around ROCKSDB_NAMESPACE::TransactionDBMutexFactory-provided
+ condition and mutex that provides toku_pthread_*-like interface. The functions
+ are named
+
+ toku_external_{mutex|cond}_XXX
+
+ Lock Tree uses this mutex and condition for interruptible (long) lock waits.
+
+ (It also still uses toku_pthread_XXX calls for mutexes/conditions for
+ shorter waits on internal objects)
+*/
+
+#pragma once
+
+#include <pthread.h>
+#include <stdint.h>
+#include <time.h>
+
+#include "rocksdb/utilities/transaction_db.h"
+#include "rocksdb/utilities/transaction_db_mutex.h"
+#include "toku_portability.h"
+
+using ROCKSDB_NAMESPACE::TransactionDBCondVar;
+using ROCKSDB_NAMESPACE::TransactionDBMutex;
+
+typedef std::shared_ptr<ROCKSDB_NAMESPACE::TransactionDBMutexFactory>
+ toku_external_mutex_factory_t;
+
+typedef std::shared_ptr<TransactionDBMutex> toku_external_mutex_t;
+typedef std::shared_ptr<TransactionDBCondVar> toku_external_cond_t;
+
+static inline void toku_external_cond_init(
+ toku_external_mutex_factory_t mutex_factory, toku_external_cond_t *cond) {
+ *cond = mutex_factory->AllocateCondVar();
+}
+
+inline void toku_external_cond_destroy(toku_external_cond_t *cond) {
+ cond->reset(); // this will destroy the managed object
+}
+
+inline void toku_external_cond_signal(toku_external_cond_t *cond) {
+ (*cond)->Notify();
+}
+
+inline void toku_external_cond_broadcast(toku_external_cond_t *cond) {
+ (*cond)->NotifyAll();
+}
+
+inline int toku_external_cond_timedwait(toku_external_cond_t *cond,
+ toku_external_mutex_t *mutex,
+ int64_t timeout_microsec) {
+ auto res = (*cond)->WaitFor(*mutex, timeout_microsec);
+ if (res.ok())
+ return 0;
+ else
+ return ETIMEDOUT;
+}
+
+inline void toku_external_mutex_init(toku_external_mutex_factory_t factory,
+ toku_external_mutex_t *mutex) {
+ // Use placement new: the memory has been allocated but constructor wasn't
+ // called
+ new (mutex) toku_external_mutex_t;
+ *mutex = factory->AllocateMutex();
+}
+
+inline void toku_external_mutex_lock(toku_external_mutex_t *mutex) {
+ (*mutex)->Lock();
+}
+
+inline int toku_external_mutex_trylock(toku_external_mutex_t *mutex) {
+ (*mutex)->Lock();
+ return 0;
+}
+
+inline void toku_external_mutex_unlock(toku_external_mutex_t *mutex) {
+ (*mutex)->UnLock();
+}
+
+inline void toku_external_mutex_destroy(toku_external_mutex_t *mutex) {
+ mutex->reset(); // this will destroy the managed object
+}
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_instrumentation.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_instrumentation.h
new file mode 100644
index 000000000..c967e7177
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_instrumentation.h
@@ -0,0 +1,286 @@
+/*======
+This file is part of PerconaFT.
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#pragma once
+
+#include <stdio.h> // FILE
+
+// Performance instrumentation object identifier type
+typedef unsigned int pfs_key_t;
+
+enum class toku_instr_object_type { mutex, rwlock, cond, thread, file };
+
+struct PSI_file;
+
+struct TOKU_FILE {
+ /** The real file. */
+ FILE *file;
+ struct PSI_file *key;
+ TOKU_FILE() : file(nullptr), key(nullptr) {}
+};
+
+struct PSI_mutex;
+struct PSI_cond;
+struct PSI_rwlock;
+
+struct toku_mutex_t;
+struct toku_cond_t;
+struct toku_pthread_rwlock_t;
+
+class toku_instr_key;
+
+class toku_instr_probe_empty {
+ public:
+ explicit toku_instr_probe_empty(UU(const toku_instr_key &key)) {}
+
+ void start_with_source_location(UU(const char *src_file), UU(int src_line)) {}
+
+ void stop() {}
+};
+
+#define TOKU_PROBE_START(p) p->start_with_source_location(__FILE__, __LINE__)
+#define TOKU_PROBE_STOP(p) p->stop
+
+extern toku_instr_key toku_uninstrumented;
+
+#ifndef MYSQL_TOKUDB_ENGINE
+
+#include <pthread.h>
+
+class toku_instr_key {
+ public:
+ toku_instr_key(UU(toku_instr_object_type type), UU(const char *group),
+ UU(const char *name)) {}
+
+ explicit toku_instr_key(UU(pfs_key_t key_id)) {}
+ // No-instrumentation constructor:
+ toku_instr_key() {}
+ ~toku_instr_key() {}
+};
+
+typedef toku_instr_probe_empty toku_instr_probe;
+
+enum class toku_instr_file_op {
+ file_stream_open,
+ file_create,
+ file_open,
+ file_delete,
+ file_rename,
+ file_read,
+ file_write,
+ file_sync,
+ file_stream_close,
+ file_close,
+ file_stat
+};
+
+struct PSI_file {};
+struct PSI_mutex {};
+
+struct toku_io_instrumentation {};
+
+inline int toku_pthread_create(UU(const toku_instr_key &key), pthread_t *thread,
+ const pthread_attr_t *attr,
+ void *(*start_routine)(void *), void *arg) {
+ return pthread_create(thread, attr, start_routine, arg);
+}
+
+inline void toku_instr_register_current_thread() {}
+
+inline void toku_instr_delete_current_thread() {}
+
+// Instrument file creation, opening, closing, and renaming
+inline void toku_instr_file_open_begin(UU(toku_io_instrumentation &io_instr),
+ UU(const toku_instr_key &key),
+ UU(toku_instr_file_op op),
+ UU(const char *name),
+ UU(const char *src_file),
+ UU(int src_line)) {}
+
+inline void toku_instr_file_stream_open_end(
+ UU(toku_io_instrumentation &io_instr), UU(TOKU_FILE &file)) {}
+
+inline void toku_instr_file_open_end(UU(toku_io_instrumentation &io_instr),
+ UU(int fd)) {}
+
+inline void toku_instr_file_name_close_begin(
+ UU(toku_io_instrumentation &io_instr), UU(const toku_instr_key &key),
+ UU(toku_instr_file_op op), UU(const char *name), UU(const char *src_file),
+ UU(int src_line)) {}
+
+inline void toku_instr_file_stream_close_begin(
+ UU(toku_io_instrumentation &io_instr), UU(toku_instr_file_op op),
+ UU(TOKU_FILE &file), UU(const char *src_file), UU(int src_line)) {}
+
+inline void toku_instr_file_fd_close_begin(
+ UU(toku_io_instrumentation &io_instr), UU(toku_instr_file_op op),
+ UU(int fd), UU(const char *src_file), UU(int src_line)) {}
+
+inline void toku_instr_file_close_end(UU(toku_io_instrumentation &io_instr),
+ UU(int result)) {}
+
+inline void toku_instr_file_io_begin(UU(toku_io_instrumentation &io_instr),
+ UU(toku_instr_file_op op), UU(int fd),
+ UU(unsigned int count),
+ UU(const char *src_file),
+ UU(int src_line)) {}
+
+inline void toku_instr_file_name_io_begin(
+ UU(toku_io_instrumentation &io_instr), UU(const toku_instr_key &key),
+ UU(toku_instr_file_op op), UU(const char *name), UU(unsigned int count),
+ UU(const char *src_file), UU(int src_line)) {}
+
+inline void toku_instr_file_stream_io_begin(
+ UU(toku_io_instrumentation &io_instr), UU(toku_instr_file_op op),
+ UU(TOKU_FILE &file), UU(unsigned int count), UU(const char *src_file),
+ UU(int src_line)) {}
+
+inline void toku_instr_file_io_end(UU(toku_io_instrumentation &io_instr),
+ UU(unsigned int count)) {}
+
+struct toku_mutex_t;
+
+struct toku_mutex_instrumentation {};
+
+inline PSI_mutex *toku_instr_mutex_init(UU(const toku_instr_key &key),
+ UU(toku_mutex_t &mutex)) {
+ return nullptr;
+}
+
+inline void toku_instr_mutex_destroy(UU(PSI_mutex *&mutex_instr)) {}
+
+inline void toku_instr_mutex_lock_start(
+ UU(toku_mutex_instrumentation &mutex_instr), UU(toku_mutex_t &mutex),
+ UU(const char *src_file), UU(int src_line)) {}
+
+inline void toku_instr_mutex_trylock_start(
+ UU(toku_mutex_instrumentation &mutex_instr), UU(toku_mutex_t &mutex),
+ UU(const char *src_file), UU(int src_line)) {}
+
+inline void toku_instr_mutex_lock_end(
+ UU(toku_mutex_instrumentation &mutex_instr),
+ UU(int pthread_mutex_lock_result)) {}
+
+inline void toku_instr_mutex_unlock(UU(PSI_mutex *mutex_instr)) {}
+
+struct toku_cond_instrumentation {};
+
+enum class toku_instr_cond_op {
+ cond_wait,
+ cond_timedwait,
+};
+
+inline PSI_cond *toku_instr_cond_init(UU(const toku_instr_key &key),
+ UU(toku_cond_t &cond)) {
+ return nullptr;
+}
+
+inline void toku_instr_cond_destroy(UU(PSI_cond *&cond_instr)) {}
+
+inline void toku_instr_cond_wait_start(
+ UU(toku_cond_instrumentation &cond_instr), UU(toku_instr_cond_op op),
+ UU(toku_cond_t &cond), UU(toku_mutex_t &mutex), UU(const char *src_file),
+ UU(int src_line)) {}
+
+inline void toku_instr_cond_wait_end(UU(toku_cond_instrumentation &cond_instr),
+ UU(int pthread_cond_wait_result)) {}
+
+inline void toku_instr_cond_signal(UU(toku_cond_t &cond)) {}
+
+inline void toku_instr_cond_broadcast(UU(toku_cond_t &cond)) {}
+
+#if 0
+// rw locks are not used
+// rwlock instrumentation
+struct toku_rwlock_instrumentation {};
+
+inline PSI_rwlock *toku_instr_rwlock_init(UU(const toku_instr_key &key),
+ UU(toku_pthread_rwlock_t &rwlock)) {
+ return nullptr;
+}
+
+inline void toku_instr_rwlock_destroy(UU(PSI_rwlock *&rwlock_instr)) {}
+
+inline void toku_instr_rwlock_rdlock_wait_start(
+ UU(toku_rwlock_instrumentation &rwlock_instr),
+ UU(toku_pthread_rwlock_t &rwlock),
+ UU(const char *src_file),
+ UU(int src_line)) {}
+
+inline void toku_instr_rwlock_wrlock_wait_start(
+ UU(toku_rwlock_instrumentation &rwlock_instr),
+ UU(toku_pthread_rwlock_t &rwlock),
+ UU(const char *src_file),
+ UU(int src_line)) {}
+
+inline void toku_instr_rwlock_rdlock_wait_end(
+ UU(toku_rwlock_instrumentation &rwlock_instr),
+ UU(int pthread_rwlock_wait_result)) {}
+
+inline void toku_instr_rwlock_wrlock_wait_end(
+ UU(toku_rwlock_instrumentation &rwlock_instr),
+ UU(int pthread_rwlock_wait_result)) {}
+
+inline void toku_instr_rwlock_unlock(UU(toku_pthread_rwlock_t &rwlock)) {}
+#endif
+
+#else // MYSQL_TOKUDB_ENGINE
+// There can be not only mysql but also mongodb or any other PFS stuff
+#include <toku_instr_mysql.h>
+#endif // MYSQL_TOKUDB_ENGINE
+
+// Mutexes
+extern toku_instr_key manager_escalation_mutex_key;
+extern toku_instr_key manager_escalator_mutex_key;
+extern toku_instr_key manager_mutex_key;
+extern toku_instr_key treenode_mutex_key;
+extern toku_instr_key locktree_request_info_mutex_key;
+extern toku_instr_key locktree_request_info_retry_mutex_key;
+
+// condition vars
+extern toku_instr_key lock_request_m_wait_cond_key;
+extern toku_instr_key locktree_request_info_retry_cv_key;
+extern toku_instr_key manager_m_escalator_done_key; // unused
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_portability.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_portability.h
new file mode 100644
index 000000000..9a95b38bd
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_portability.h
@@ -0,0 +1,87 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#if defined(__clang__)
+#define constexpr_static_assert(a, b)
+#else
+#define constexpr_static_assert(a, b) static_assert(a, b)
+#endif
+
+// include here, before they get deprecated
+#include <inttypes.h>
+#include <stdint.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "toku_atomic.h"
+
+#if defined(__cplusplus)
+#include <type_traits>
+#endif
+
+#if defined(__cplusplus)
+// decltype() here gives a reference-to-pointer instead of just a pointer,
+// just use __typeof__
+#define CAST_FROM_VOIDP(name, value) name = static_cast<__typeof__(name)>(value)
+#else
+#define CAST_FROM_VOIDP(name, value) name = cast_to_typeof(name)(value)
+#endif
+
+#define UU(x) x __attribute__((__unused__))
+
+#include "toku_instrumentation.h"
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_pthread.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_pthread.h
new file mode 100644
index 000000000..571b950e1
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_pthread.h
@@ -0,0 +1,520 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <pthread.h>
+#include <stdint.h>
+#include <time.h>
+
+#include "toku_portability.h"
+// PORT2: #include "toku_assert.h"
+
+// TODO: some things moved toku_instrumentation.h, not necessarily the best
+// place
+typedef pthread_attr_t toku_pthread_attr_t;
+typedef pthread_t toku_pthread_t;
+typedef pthread_mutex_t toku_pthread_mutex_t;
+typedef pthread_condattr_t toku_pthread_condattr_t;
+typedef pthread_cond_t toku_pthread_cond_t;
+typedef pthread_rwlockattr_t toku_pthread_rwlockattr_t;
+typedef pthread_key_t toku_pthread_key_t;
+typedef struct timespec toku_timespec_t;
+
+// TODO: break this include loop
+#include <pthread.h>
+typedef pthread_mutexattr_t toku_pthread_mutexattr_t;
+
+struct toku_mutex_t {
+ pthread_mutex_t pmutex;
+ struct PSI_mutex *psi_mutex; /* The performance schema instrumentation hook */
+#if defined(TOKU_PTHREAD_DEBUG)
+ pthread_t owner; // = pthread_self(); // for debugging
+ bool locked;
+ bool valid;
+ pfs_key_t instr_key_id;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+};
+
+struct toku_cond_t {
+ pthread_cond_t pcond;
+ struct PSI_cond *psi_cond;
+#if defined(TOKU_PTHREAD_DEBUG)
+ pfs_key_t instr_key_id;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+};
+
+#if defined(TOKU_PTHREAD_DEBUG)
+#define TOKU_COND_INITIALIZER \
+ { .pcond = PTHREAD_COND_INITIALIZER, .psi_cond = nullptr, .instr_key_id = 0 }
+#else
+#define TOKU_COND_INITIALIZER \
+ { .pcond = PTHREAD_COND_INITIALIZER, .psi_cond = nullptr }
+#endif // defined(TOKU_PTHREAD_DEBUG)
+
+struct toku_pthread_rwlock_t {
+ pthread_rwlock_t rwlock;
+ struct PSI_rwlock *psi_rwlock;
+#if defined(TOKU_PTHREAD_DEBUG)
+ pfs_key_t instr_key_id;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+};
+
+typedef struct toku_mutex_aligned {
+ toku_mutex_t aligned_mutex __attribute__((__aligned__(64)));
+} toku_mutex_aligned_t;
+
+// Initializing with {} will fill in a struct with all zeros.
+// But you may also need a pragma to suppress the warnings, as follows
+//
+// #pragma GCC diagnostic push
+// #pragma GCC diagnostic ignored "-Wmissing-field-initializers"
+// toku_mutex_t foo = ZERO_MUTEX_INITIALIZER;
+// #pragma GCC diagnostic pop
+//
+// In general it will be a lot of busy work to make this codebase compile
+// cleanly with -Wmissing-field-initializers
+
+#define ZERO_MUTEX_INITIALIZER \
+ {}
+
+#if defined(TOKU_PTHREAD_DEBUG)
+#define TOKU_MUTEX_INITIALIZER \
+ { \
+ .pmutex = PTHREAD_MUTEX_INITIALIZER, .psi_mutex = nullptr, .owner = 0, \
+ .locked = false, .valid = true, .instr_key_id = 0 \
+ }
+#else
+#define TOKU_MUTEX_INITIALIZER \
+ { .pmutex = PTHREAD_MUTEX_INITIALIZER, .psi_mutex = nullptr }
+#endif // defined(TOKU_PTHREAD_DEBUG)
+
+// Darwin doesn't provide adaptive mutexes
+#if defined(__APPLE__)
+#define TOKU_MUTEX_ADAPTIVE PTHREAD_MUTEX_DEFAULT
+#if defined(TOKU_PTHREAD_DEBUG)
+#define TOKU_ADAPTIVE_MUTEX_INITIALIZER \
+ { \
+ .pmutex = PTHREAD_MUTEX_INITIALIZER, .psi_mutex = nullptr, .owner = 0, \
+ .locked = false, .valid = true, .instr_key_id = 0 \
+ }
+#else
+#define TOKU_ADAPTIVE_MUTEX_INITIALIZER \
+ { .pmutex = PTHREAD_MUTEX_INITIALIZER, .psi_mutex = nullptr }
+#endif // defined(TOKU_PTHREAD_DEBUG)
+#else // __FreeBSD__, __linux__, at least
+#if defined(__GLIBC__)
+#define TOKU_MUTEX_ADAPTIVE PTHREAD_MUTEX_ADAPTIVE_NP
+#else
+// not all libc (e.g. musl) implement NP (Non-POSIX) attributes
+#define TOKU_MUTEX_ADAPTIVE PTHREAD_MUTEX_DEFAULT
+#endif
+#if defined(TOKU_PTHREAD_DEBUG)
+#define TOKU_ADAPTIVE_MUTEX_INITIALIZER \
+ { \
+ .pmutex = PTHREAD_ADAPTIVE_MUTEX_INITIALIZER_NP, .psi_mutex = nullptr, \
+ .owner = 0, .locked = false, .valid = true, .instr_key_id = 0 \
+ }
+#else
+#define TOKU_ADAPTIVE_MUTEX_INITIALIZER \
+ { .pmutex = PTHREAD_ADAPTIVE_MUTEX_INITIALIZER_NP, .psi_mutex = nullptr }
+#endif // defined(TOKU_PTHREAD_DEBUG)
+#endif // defined(__APPLE__)
+
+// Different OSes implement mutexes as different amounts of nested structs.
+// C++ will fill out all missing values with zeroes if you provide at least one
+// zero, but it needs the right amount of nesting.
+#if defined(__FreeBSD__)
+#define ZERO_COND_INITIALIZER \
+ { 0 }
+#elif defined(__APPLE__)
+#define ZERO_COND_INITIALIZER \
+ { \
+ { 0 } \
+ }
+#else // __linux__, at least
+#define ZERO_COND_INITIALIZER \
+ {}
+#endif
+
+static inline void toku_mutexattr_init(toku_pthread_mutexattr_t *attr) {
+ int r = pthread_mutexattr_init(attr);
+ assert_zero(r);
+}
+
+static inline void toku_mutexattr_settype(toku_pthread_mutexattr_t *attr,
+ int type) {
+ int r = pthread_mutexattr_settype(attr, type);
+ assert_zero(r);
+}
+
+static inline void toku_mutexattr_destroy(toku_pthread_mutexattr_t *attr) {
+ int r = pthread_mutexattr_destroy(attr);
+ assert_zero(r);
+}
+
+#if defined(TOKU_PTHREAD_DEBUG)
+static inline void toku_mutex_assert_locked(const toku_mutex_t *mutex) {
+ invariant(mutex->locked);
+ invariant(mutex->owner == pthread_self());
+}
+#else
+static inline void toku_mutex_assert_locked(const toku_mutex_t *mutex
+ __attribute__((unused))) {}
+#endif // defined(TOKU_PTHREAD_DEBUG)
+
+// asserting that a mutex is unlocked only makes sense
+// if the calling thread can guaruntee that no other threads
+// are trying to lock this mutex at the time of the assertion
+//
+// a good example of this is a tree with mutexes on each node.
+// when a node is locked the caller knows that no other threads
+// can be trying to lock its childrens' mutexes. the children
+// are in one of two fixed states: locked or unlocked.
+#if defined(TOKU_PTHREAD_DEBUG)
+static inline void toku_mutex_assert_unlocked(toku_mutex_t *mutex) {
+ invariant(mutex->owner == 0);
+ invariant(!mutex->locked);
+}
+#else
+static inline void toku_mutex_assert_unlocked(toku_mutex_t *mutex
+ __attribute__((unused))) {}
+#endif // defined(TOKU_PTHREAD_DEBUG)
+
+#define toku_mutex_lock(M) \
+ toku_mutex_lock_with_source_location(M, __FILE__, __LINE__)
+
+static inline void toku_cond_init(toku_cond_t *cond,
+ const toku_pthread_condattr_t *attr) {
+ int r = pthread_cond_init(&cond->pcond, attr);
+ assert_zero(r);
+}
+
+#define toku_mutex_trylock(M) \
+ toku_mutex_trylock_with_source_location(M, __FILE__, __LINE__)
+
+inline void toku_mutex_unlock(toku_mutex_t *mutex) {
+#if defined(TOKU_PTHREAD_DEBUG)
+ invariant(mutex->owner == pthread_self());
+ invariant(mutex->valid);
+ invariant(mutex->locked);
+ mutex->locked = false;
+ mutex->owner = 0;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+ toku_instr_mutex_unlock(mutex->psi_mutex);
+ int r = pthread_mutex_unlock(&mutex->pmutex);
+ assert_zero(r);
+}
+
+inline void toku_mutex_lock_with_source_location(toku_mutex_t *mutex,
+ const char *src_file,
+ int src_line) {
+ toku_mutex_instrumentation mutex_instr;
+ toku_instr_mutex_lock_start(mutex_instr, *mutex, src_file, src_line);
+
+ const int r = pthread_mutex_lock(&mutex->pmutex);
+ toku_instr_mutex_lock_end(mutex_instr, r);
+
+ assert_zero(r);
+#if defined(TOKU_PTHREAD_DEBUG)
+ invariant(mutex->valid);
+ invariant(!mutex->locked);
+ invariant(mutex->owner == 0);
+ mutex->locked = true;
+ mutex->owner = pthread_self();
+#endif // defined(TOKU_PTHREAD_DEBUG)
+}
+
+inline int toku_mutex_trylock_with_source_location(toku_mutex_t *mutex,
+ const char *src_file,
+ int src_line) {
+ toku_mutex_instrumentation mutex_instr;
+ toku_instr_mutex_trylock_start(mutex_instr, *mutex, src_file, src_line);
+
+ const int r = pthread_mutex_lock(&mutex->pmutex);
+ toku_instr_mutex_lock_end(mutex_instr, r);
+
+#if defined(TOKU_PTHREAD_DEBUG)
+ if (r == 0) {
+ invariant(mutex->valid);
+ invariant(!mutex->locked);
+ invariant(mutex->owner == 0);
+ mutex->locked = true;
+ mutex->owner = pthread_self();
+ }
+#endif // defined(TOKU_PTHREAD_DEBUG)
+ return r;
+}
+
+#define toku_cond_wait(C, M) \
+ toku_cond_wait_with_source_location(C, M, __FILE__, __LINE__)
+
+#define toku_cond_timedwait(C, M, W) \
+ toku_cond_timedwait_with_source_location(C, M, W, __FILE__, __LINE__)
+
+inline void toku_cond_init(const toku_instr_key &key, toku_cond_t *cond,
+ const pthread_condattr_t *attr) {
+ toku_instr_cond_init(key, *cond);
+ int r = pthread_cond_init(&cond->pcond, attr);
+ assert_zero(r);
+}
+
+inline void toku_cond_destroy(toku_cond_t *cond) {
+ toku_instr_cond_destroy(cond->psi_cond);
+ int r = pthread_cond_destroy(&cond->pcond);
+ assert_zero(r);
+}
+
+inline void toku_cond_wait_with_source_location(toku_cond_t *cond,
+ toku_mutex_t *mutex,
+ const char *src_file,
+ int src_line) {
+#if defined(TOKU_PTHREAD_DEBUG)
+ invariant(mutex->locked);
+ mutex->locked = false;
+ mutex->owner = 0;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+
+ /* Instrumentation start */
+ toku_cond_instrumentation cond_instr;
+ toku_instr_cond_wait_start(cond_instr, toku_instr_cond_op::cond_wait, *cond,
+ *mutex, src_file, src_line);
+
+ /* Instrumented code */
+ const int r = pthread_cond_wait(&cond->pcond, &mutex->pmutex);
+
+ /* Instrumentation end */
+ toku_instr_cond_wait_end(cond_instr, r);
+
+ assert_zero(r);
+#if defined(TOKU_PTHREAD_DEBUG)
+ invariant(!mutex->locked);
+ mutex->locked = true;
+ mutex->owner = pthread_self();
+#endif // defined(TOKU_PTHREAD_DEBUG)
+}
+
+inline int toku_cond_timedwait_with_source_location(toku_cond_t *cond,
+ toku_mutex_t *mutex,
+ toku_timespec_t *wakeup_at,
+ const char *src_file,
+ int src_line) {
+#if defined(TOKU_PTHREAD_DEBUG)
+ invariant(mutex->locked);
+ mutex->locked = false;
+ mutex->owner = 0;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+
+ /* Instrumentation start */
+ toku_cond_instrumentation cond_instr;
+ toku_instr_cond_wait_start(cond_instr, toku_instr_cond_op::cond_timedwait,
+ *cond, *mutex, src_file, src_line);
+
+ /* Instrumented code */
+ const int r = pthread_cond_timedwait(&cond->pcond, &mutex->pmutex, wakeup_at);
+
+ /* Instrumentation end */
+ toku_instr_cond_wait_end(cond_instr, r);
+
+#if defined(TOKU_PTHREAD_DEBUG)
+ invariant(!mutex->locked);
+ mutex->locked = true;
+ mutex->owner = pthread_self();
+#endif // defined(TOKU_PTHREAD_DEBUG)
+ return r;
+}
+
+inline void toku_cond_signal(toku_cond_t *cond) {
+ toku_instr_cond_signal(*cond);
+ const int r = pthread_cond_signal(&cond->pcond);
+ assert_zero(r);
+}
+
+inline void toku_cond_broadcast(toku_cond_t *cond) {
+ toku_instr_cond_broadcast(*cond);
+ const int r = pthread_cond_broadcast(&cond->pcond);
+ assert_zero(r);
+}
+
+inline void toku_mutex_init(const toku_instr_key &key, toku_mutex_t *mutex,
+ const toku_pthread_mutexattr_t *attr) {
+#if defined(TOKU_PTHREAD_DEBUG)
+ mutex->valid = true;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+ toku_instr_mutex_init(key, *mutex);
+ const int r = pthread_mutex_init(&mutex->pmutex, attr);
+ assert_zero(r);
+#if defined(TOKU_PTHREAD_DEBUG)
+ mutex->locked = false;
+ invariant(mutex->valid);
+ mutex->valid = true;
+ mutex->owner = 0;
+#endif // defined(TOKU_PTHREAD_DEBUG)
+}
+
+inline void toku_mutex_destroy(toku_mutex_t *mutex) {
+#if defined(TOKU_PTHREAD_DEBUG)
+ invariant(mutex->valid);
+ mutex->valid = false;
+ invariant(!mutex->locked);
+#endif // defined(TOKU_PTHREAD_DEBUG)
+ toku_instr_mutex_destroy(mutex->psi_mutex);
+ int r = pthread_mutex_destroy(&mutex->pmutex);
+ assert_zero(r);
+}
+
+#define toku_pthread_rwlock_rdlock(RW) \
+ toku_pthread_rwlock_rdlock_with_source_location(RW, __FILE__, __LINE__)
+
+#define toku_pthread_rwlock_wrlock(RW) \
+ toku_pthread_rwlock_wrlock_with_source_location(RW, __FILE__, __LINE__)
+
+#if 0
+inline void toku_pthread_rwlock_init(
+ const toku_instr_key &key,
+ toku_pthread_rwlock_t *__restrict rwlock,
+ const toku_pthread_rwlockattr_t *__restrict attr) {
+ toku_instr_rwlock_init(key, *rwlock);
+ int r = pthread_rwlock_init(&rwlock->rwlock, attr);
+ assert_zero(r);
+}
+
+inline void toku_pthread_rwlock_destroy(toku_pthread_rwlock_t *rwlock) {
+ toku_instr_rwlock_destroy(rwlock->psi_rwlock);
+ int r = pthread_rwlock_destroy(&rwlock->rwlock);
+ assert_zero(r);
+}
+
+inline void toku_pthread_rwlock_rdlock_with_source_location(
+ toku_pthread_rwlock_t *rwlock,
+ const char *src_file,
+ uint src_line) {
+
+ /* Instrumentation start */
+ toku_rwlock_instrumentation rwlock_instr;
+ toku_instr_rwlock_rdlock_wait_start(
+ rwlock_instr, *rwlock, src_file, src_line);
+ /* Instrumented code */
+ const int r = pthread_rwlock_rdlock(&rwlock->rwlock);
+
+ /* Instrumentation end */
+ toku_instr_rwlock_rdlock_wait_end(rwlock_instr, r);
+
+ assert_zero(r);
+}
+
+inline void toku_pthread_rwlock_wrlock_with_source_location(
+ toku_pthread_rwlock_t *rwlock,
+ const char *src_file,
+ uint src_line) {
+
+ /* Instrumentation start */
+ toku_rwlock_instrumentation rwlock_instr;
+ toku_instr_rwlock_wrlock_wait_start(
+ rwlock_instr, *rwlock, src_file, src_line);
+ /* Instrumented code */
+ const int r = pthread_rwlock_wrlock(&rwlock->rwlock);
+
+ /* Instrumentation end */
+ toku_instr_rwlock_wrlock_wait_end(rwlock_instr, r);
+
+ assert_zero(r);
+}
+
+inline void toku_pthread_rwlock_rdunlock(toku_pthread_rwlock_t *rwlock) {
+ toku_instr_rwlock_unlock(*rwlock);
+ const int r = pthread_rwlock_unlock(&rwlock->rwlock);
+ assert_zero(r);
+}
+
+inline void toku_pthread_rwlock_wrunlock(toku_pthread_rwlock_t *rwlock) {
+ toku_instr_rwlock_unlock(*rwlock);
+ const int r = pthread_rwlock_unlock(&rwlock->rwlock);
+ assert_zero(r);
+}
+#endif
+
+static inline int toku_pthread_join(toku_pthread_t thread, void **value_ptr) {
+ return pthread_join(thread, value_ptr);
+}
+
+static inline int toku_pthread_detach(toku_pthread_t thread) {
+ return pthread_detach(thread);
+}
+
+static inline int toku_pthread_key_create(toku_pthread_key_t *key,
+ void (*destroyf)(void *)) {
+ return pthread_key_create(key, destroyf);
+}
+
+static inline int toku_pthread_key_delete(toku_pthread_key_t key) {
+ return pthread_key_delete(key);
+}
+
+static inline void *toku_pthread_getspecific(toku_pthread_key_t key) {
+ return pthread_getspecific(key);
+}
+
+static inline int toku_pthread_setspecific(toku_pthread_key_t key, void *data) {
+ return pthread_setspecific(key, data);
+}
+
+int toku_pthread_yield(void) __attribute__((__visibility__("default")));
+
+static inline toku_pthread_t toku_pthread_self(void) { return pthread_self(); }
+
+static inline void *toku_pthread_done(void *exit_value) {
+ toku_instr_delete_current_thread();
+ pthread_exit(exit_value);
+ return nullptr; // Avoid compiler warning
+}
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_race_tools.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_race_tools.h
new file mode 100644
index 000000000..3cb5b5790
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_race_tools.h
@@ -0,0 +1,179 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+// PORT2: #include <portability/toku_config.h>
+
+#ifdef HAVE_valgrind
+#undef USE_VALGRIND
+#define USE_VALGRIND 1
+#endif
+
+#if defined(__linux__) && USE_VALGRIND
+
+#include <valgrind/drd.h>
+#include <valgrind/helgrind.h>
+
+#define TOKU_ANNOTATE_NEW_MEMORY(p, size) ANNOTATE_NEW_MEMORY(p, size)
+#define TOKU_VALGRIND_HG_ENABLE_CHECKING(p, size) \
+ VALGRIND_HG_ENABLE_CHECKING(p, size)
+#define TOKU_VALGRIND_HG_DISABLE_CHECKING(p, size) \
+ VALGRIND_HG_DISABLE_CHECKING(p, size)
+#define TOKU_DRD_IGNORE_VAR(v) DRD_IGNORE_VAR(v)
+#define TOKU_DRD_STOP_IGNORING_VAR(v) DRD_STOP_IGNORING_VAR(v)
+#define TOKU_ANNOTATE_IGNORE_READS_BEGIN() ANNOTATE_IGNORE_READS_BEGIN()
+#define TOKU_ANNOTATE_IGNORE_READS_END() ANNOTATE_IGNORE_READS_END()
+#define TOKU_ANNOTATE_IGNORE_WRITES_BEGIN() ANNOTATE_IGNORE_WRITES_BEGIN()
+#define TOKU_ANNOTATE_IGNORE_WRITES_END() ANNOTATE_IGNORE_WRITES_END()
+
+/*
+ * How to make helgrind happy about tree rotations and new mutex orderings:
+ *
+ * // Tell helgrind that we unlocked it so that the next call doesn't get a
+ * "destroyed a locked mutex" error.
+ * // Tell helgrind that we destroyed the mutex.
+ * VALGRIND_HG_MUTEX_UNLOCK_PRE(&locka);
+ * VALGRIND_HG_MUTEX_DESTROY_PRE(&locka);
+ *
+ * // And recreate it. It would be better to simply be able to say that the
+ * order on these two can now be reversed, because this code forgets all the
+ * ordering information for this mutex.
+ * // Then tell helgrind that we have locked it again.
+ * VALGRIND_HG_MUTEX_INIT_POST(&locka, 0);
+ * VALGRIND_HG_MUTEX_LOCK_POST(&locka);
+ *
+ * When the ordering of two locks changes, we don't need tell Helgrind about do
+ * both locks. Just one is good enough.
+ */
+
+#define TOKU_VALGRIND_RESET_MUTEX_ORDERING_INFO(mutex) \
+ VALGRIND_HG_MUTEX_UNLOCK_PRE(mutex); \
+ VALGRIND_HG_MUTEX_DESTROY_PRE(mutex); \
+ VALGRIND_HG_MUTEX_INIT_POST(mutex, 0); \
+ VALGRIND_HG_MUTEX_LOCK_POST(mutex);
+
+#else // !defined(__linux__) || !USE_VALGRIND
+
+#define NVALGRIND 1
+#define TOKU_ANNOTATE_NEW_MEMORY(p, size) ((void)0)
+#define TOKU_VALGRIND_HG_ENABLE_CHECKING(p, size) ((void)0)
+#define TOKU_VALGRIND_HG_DISABLE_CHECKING(p, size) ((void)0)
+#define TOKU_DRD_IGNORE_VAR(v)
+#define TOKU_DRD_STOP_IGNORING_VAR(v)
+#define TOKU_ANNOTATE_IGNORE_READS_BEGIN() ((void)0)
+#define TOKU_ANNOTATE_IGNORE_READS_END() ((void)0)
+#define TOKU_ANNOTATE_IGNORE_WRITES_BEGIN() ((void)0)
+#define TOKU_ANNOTATE_IGNORE_WRITES_END() ((void)0)
+#define TOKU_VALGRIND_RESET_MUTEX_ORDERING_INFO(mutex)
+#undef RUNNING_ON_VALGRIND
+#define RUNNING_ON_VALGRIND (0U)
+#endif
+
+// Valgrind 3.10.1 (and previous versions).
+// Problems with VALGRIND_HG_DISABLE_CHECKING and VALGRIND_HG_ENABLE_CHECKING.
+// Helgrind's implementation of disable and enable checking causes false races
+// to be reported. In addition, the race report does not include ANY
+// information about the code that uses the helgrind disable and enable
+// functions. Therefore, it is very difficult to figure out the cause of the
+// race. DRD does implement the disable and enable functions.
+
+// Problems with ANNOTATE_IGNORE_READS.
+// Helgrind does not implement ignore reads.
+// Annotate ignore reads is the way to inform DRD to ignore racy reads.
+
+// FT code uses unsafe reads in several places. These unsafe reads have been
+// noted as valid since they use the toku_unsafe_fetch function. Unfortunately,
+// this causes helgrind to report erroneous data races which makes use of
+// helgrind problematic.
+
+// Unsafely fetch and return a `T' from src, telling drd to ignore
+// racey access to src for the next sizeof(*src) bytes
+template <typename T>
+T toku_unsafe_fetch(T *src) {
+ if (0)
+ TOKU_VALGRIND_HG_DISABLE_CHECKING(src,
+ sizeof *src); // disabled, see comment
+ TOKU_ANNOTATE_IGNORE_READS_BEGIN();
+ T r = *src;
+ TOKU_ANNOTATE_IGNORE_READS_END();
+ if (0)
+ TOKU_VALGRIND_HG_ENABLE_CHECKING(src,
+ sizeof *src); // disabled, see comment
+ return r;
+}
+
+template <typename T>
+T toku_unsafe_fetch(T &src) {
+ return toku_unsafe_fetch(&src);
+}
+
+// Unsafely set a `T' value into *dest from src, telling drd to ignore
+// racey access to dest for the next sizeof(*dest) bytes
+template <typename T>
+void toku_unsafe_set(T *dest, const T src) {
+ if (0)
+ TOKU_VALGRIND_HG_DISABLE_CHECKING(dest,
+ sizeof *dest); // disabled, see comment
+ TOKU_ANNOTATE_IGNORE_WRITES_BEGIN();
+ *dest = src;
+ TOKU_ANNOTATE_IGNORE_WRITES_END();
+ if (0)
+ TOKU_VALGRIND_HG_ENABLE_CHECKING(dest,
+ sizeof *dest); // disabled, see comment
+}
+
+template <typename T>
+void toku_unsafe_set(T &dest, const T src) {
+ toku_unsafe_set(&dest, src);
+}
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h
new file mode 100644
index 000000000..46111e7f0
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h
@@ -0,0 +1,193 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+// PORT2: #include "toku_config.h"
+
+#include <stdint.h>
+#include <sys/time.h>
+#include <time.h>
+#if defined(__powerpc__)
+#include <sys/platform/ppc.h>
+#endif
+
+#if 0
+static inline float toku_tdiff (struct timeval *a, struct timeval *b) {
+ return (float)((a->tv_sec - b->tv_sec) + 1e-6 * (a->tv_usec - b->tv_usec));
+}
+// PORT2: temporary:
+#define HAVE_CLOCK_REALTIME
+#if !defined(HAVE_CLOCK_REALTIME)
+// OS X does not have clock_gettime, we fake clockid_t for the interface, and we'll implement it with clock_get_time.
+typedef int clockid_t;
+// just something bogus, it doesn't matter, we just want to make sure we're
+// only supporting this mode because we're not sure we can support other modes
+// without a real clock_gettime()
+#define CLOCK_REALTIME 0x01867234
+#endif
+int toku_clock_gettime(clockid_t clk_id, struct timespec *ts) __attribute__((__visibility__("default")));
+#endif
+
+// *************** Performance timers ************************
+// What do you really want from a performance timer:
+// (1) Can determine actual time of day from the performance time.
+// (2) Time goes forward, never backward.
+// (3) Same time on different processors (or even different machines).
+// (4) Time goes forward at a constant rate (doesn't get faster and slower)
+// (5) Portable.
+// (6) Getting the time is cheap.
+// Unfortuately it seems tough to get Properties 1-5. So we go for Property 6,,
+// but we abstract it. We offer a type tokutime_t which can hold the time. This
+// type can be subtracted to get a time difference. We can get the present time
+// cheaply. We can convert this type to seconds (but that can be expensive). The
+// implementation is to use RDTSC (hence we lose property 3: not portable).
+// Recent machines have constant_tsc in which case we get property (4).
+// Recent OSs on recent machines (that have RDTSCP) fix the per-processor clock
+// skew, so we get property (3). We get property 2 with RDTSC (as long as
+// there's not any skew). We don't even try to get propety 1, since we don't
+// need it. The decision here is that these times are really accurate only on
+// modern machines with modern OSs.
+typedef uint64_t tokutime_t; // Time type used in by tokutek timers.
+
+#if 0
+// The value of tokutime_t is not specified here.
+// It might be microseconds since 1/1/1970 (if gettimeofday() is
+// used), or clock cycles since boot (if rdtsc is used). Or something
+// else.
+// Two tokutime_t values can be subtracted to get a time difference.
+// Use tokutime_to_seconds to that convert difference to seconds.
+// We want get_tokutime() to be fast, but don't care so much about tokutime_to_seconds();
+//
+// For accurate time calculations do the subtraction in the right order:
+// Right: tokutime_to_seconds(t1-t2);
+// Wrong tokutime_to_seconds(t1)-toku_time_to_seconds(t2);
+// Doing it the wrong way is likely to result in loss of precision.
+// A double can hold numbers up to about 53 bits. RDTSC which uses about 33 bits every second, so that leaves
+// 2^20 seconds from booting (about 2 weeks) before the RDTSC value cannot be represented accurately as a double.
+//
+double tokutime_to_seconds(tokutime_t) __attribute__((__visibility__("default"))); // Convert tokutime to seconds.
+
+#endif
+
+// Get the value of tokutime for right now. We want this to be fast, so we
+// expose the implementation as RDTSC.
+static inline tokutime_t toku_time_now(void) {
+#if defined(__x86_64__) || defined(__i386__)
+ uint32_t lo, hi;
+ __asm__ __volatile__("rdtsc" : "=a"(lo), "=d"(hi));
+ return (uint64_t)hi << 32 | lo;
+#elif defined(__aarch64__)
+ uint64_t result;
+ __asm __volatile__("mrs %[rt], cntvct_el0" : [rt] "=r"(result));
+ return result;
+#elif defined(__powerpc__)
+ return __ppc_get_timebase();
+#elif defined(__s390x__)
+ uint64_t result;
+ asm volatile("stckf %0" : "=Q"(result) : : "cc");
+ return result;
+#elif defined(__riscv) && __riscv_xlen == 32
+ uint32_t cycles_lo, cycles_hi0, cycles_hi1;
+ // Implemented in assembly because Clang insisted on branching.
+ asm volatile(
+ "rdcycleh %0\n"
+ "rdcycle %1\n"
+ "rdcycleh %2\n"
+ "sub %0, %0, %2\n"
+ "seqz %0, %0\n"
+ "sub %0, zero, %0\n"
+ "and %1, %1, %0\n"
+ : "=r"(cycles_hi0), "=r"(cycles_lo), "=r"(cycles_hi1));
+ return (static_cast<uint64_t>(cycles_hi1) << 32) | cycles_lo;
+#elif defined(__riscv) && __riscv_xlen == 64
+ uint64_t cycles;
+ asm volatile("rdcycle %0" : "=r"(cycles));
+ return cycles;
+#else
+#error No timer implementation for this platform
+#endif
+}
+
+static inline uint64_t toku_current_time_microsec(void) {
+ struct timeval t;
+ gettimeofday(&t, NULL);
+ return t.tv_sec * (1UL * 1000 * 1000) + t.tv_usec;
+}
+
+#if 0
+// sleep microseconds
+static inline void toku_sleep_microsec(uint64_t ms) {
+ struct timeval t;
+
+ t.tv_sec = ms / 1000000;
+ t.tv_usec = ms % 1000000;
+
+ select(0, NULL, NULL, NULL, &t);
+}
+#endif
+
+/*
+ PORT: Usage of this file:
+
+ uint64_t toku_current_time_microsec() // uses gettimeoday
+ is used to track how much time various operations took (for example, lock
+ escalation). (TODO: it is not clear why these operations are tracked with
+ microsecond precision while others use nanoseconds)
+
+ tokutime_t toku_time_now() // uses rdtsc
+ seems to be used for a very similar purpose. This has greater precision
+
+ RocksDB environment provides Env::Default()->NowMicros() and NowNanos() which
+ should be adequate substitutes.
+*/
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/txn_subst.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/txn_subst.h
new file mode 100644
index 000000000..803914862
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/txn_subst.h
@@ -0,0 +1,27 @@
+//
+// A substitute for ft/txn/txn.h
+//
+#pragma once
+
+#include <set>
+
+#include "../util/omt.h"
+
+typedef uint64_t TXNID;
+#define TXNID_NONE ((TXNID)0)
+
+// A set of transactions
+// (TODO: consider using class toku::txnid_set. The reason for using STL
+// container was that its API is easier)
+class TxnidVector : public std::set<TXNID> {
+ public:
+ bool contains(TXNID txnid) { return find(txnid) != end(); }
+};
+
+// A value for lock structures with a meaning "the lock is owned by multiple
+// transactions (and one has to check the TxnidVector to get their ids)
+#define TXNID_SHARED (TXNID(-1))
+
+// Auxiliary value meaning "any transaction id will do". No real transaction
+// may have this is as id.
+#define TXNID_ANY (TXNID(-2))
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc
new file mode 100644
index 000000000..50dc879ce
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc
@@ -0,0 +1,132 @@
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+/*
+ This is a dump ground to make Lock Tree work without the rest of TokuDB.
+*/
+#include <string.h>
+
+#include "db.h"
+#include "ft/ft-status.h"
+#include "portability/memory.h"
+#include "util/dbt.h"
+
+// portability/os_malloc.cc
+
+void toku_free(void *p) { free(p); }
+
+void *toku_xmalloc(size_t size) { return malloc(size); }
+
+void *toku_xrealloc(void *v, size_t size) { return realloc(v, size); }
+
+void *toku_xmemdup(const void *v, size_t len) {
+ void *p = toku_xmalloc(len);
+ memcpy(p, v, len);
+ return p;
+}
+
+// TODO: what are the X-functions? Xcalloc, Xrealloc?
+void *toku_xcalloc(size_t nmemb, size_t size) { return calloc(nmemb, size); }
+
+// ft-ft-opts.cc:
+
+// locktree
+toku_instr_key lock_request_m_wait_cond_key;
+toku_instr_key manager_m_escalator_done_key;
+toku_instr_key locktree_request_info_mutex_key;
+toku_instr_key locktree_request_info_retry_mutex_key;
+toku_instr_key locktree_request_info_retry_cv_key;
+
+toku_instr_key treenode_mutex_key;
+toku_instr_key manager_mutex_key;
+toku_instr_key manager_escalation_mutex_key;
+toku_instr_key manager_escalator_mutex_key;
+
+// portability/memory.cc
+size_t toku_memory_footprint(void *, size_t touched) { return touched; }
+
+// ft/ft-status.c
+// PORT2: note: the @c parameter to TOKUFT_STATUS_INIT must not start with
+// "TOKU"
+LTM_STATUS_S ltm_status;
+void LTM_STATUS_S::init() {
+ if (m_initialized) return;
+#define LTM_STATUS_INIT(k, c, t, l) \
+ TOKUFT_STATUS_INIT((*this), k, c, t, "locktree: " l, \
+ TOKU_ENGINE_STATUS | TOKU_GLOBAL_STATUS)
+ LTM_STATUS_INIT(LTM_SIZE_CURRENT, LOCKTREE_MEMORY_SIZE, STATUS_UINT64,
+ "memory size");
+ LTM_STATUS_INIT(LTM_SIZE_LIMIT, LOCKTREE_MEMORY_SIZE_LIMIT, STATUS_UINT64,
+ "memory size limit");
+ LTM_STATUS_INIT(LTM_ESCALATION_COUNT, LOCKTREE_ESCALATION_NUM, STATUS_UINT64,
+ "number of times lock escalation ran");
+ LTM_STATUS_INIT(LTM_ESCALATION_TIME, LOCKTREE_ESCALATION_SECONDS,
+ STATUS_TOKUTIME, "time spent running escalation (seconds)");
+ LTM_STATUS_INIT(LTM_ESCALATION_LATEST_RESULT,
+ LOCKTREE_LATEST_POST_ESCALATION_MEMORY_SIZE, STATUS_UINT64,
+ "latest post-escalation memory size");
+ LTM_STATUS_INIT(LTM_NUM_LOCKTREES, LOCKTREE_OPEN_CURRENT, STATUS_UINT64,
+ "number of locktrees open now");
+ LTM_STATUS_INIT(LTM_LOCK_REQUESTS_PENDING, LOCKTREE_PENDING_LOCK_REQUESTS,
+ STATUS_UINT64, "number of pending lock requests");
+ LTM_STATUS_INIT(LTM_STO_NUM_ELIGIBLE, LOCKTREE_STO_ELIGIBLE_NUM,
+ STATUS_UINT64, "number of locktrees eligible for the STO");
+ LTM_STATUS_INIT(LTM_STO_END_EARLY_COUNT, LOCKTREE_STO_ENDED_NUM,
+ STATUS_UINT64,
+ "number of times a locktree ended the STO early");
+ LTM_STATUS_INIT(LTM_STO_END_EARLY_TIME, LOCKTREE_STO_ENDED_SECONDS,
+ STATUS_TOKUTIME, "time spent ending the STO early (seconds)");
+ LTM_STATUS_INIT(LTM_WAIT_COUNT, LOCKTREE_WAIT_COUNT, STATUS_UINT64,
+ "number of wait locks");
+ LTM_STATUS_INIT(LTM_WAIT_TIME, LOCKTREE_WAIT_TIME, STATUS_UINT64,
+ "time waiting for locks");
+ LTM_STATUS_INIT(LTM_LONG_WAIT_COUNT, LOCKTREE_LONG_WAIT_COUNT, STATUS_UINT64,
+ "number of long wait locks");
+ LTM_STATUS_INIT(LTM_LONG_WAIT_TIME, LOCKTREE_LONG_WAIT_TIME, STATUS_UINT64,
+ "long time waiting for locks");
+ LTM_STATUS_INIT(LTM_TIMEOUT_COUNT, LOCKTREE_TIMEOUT_COUNT, STATUS_UINT64,
+ "number of lock timeouts");
+ LTM_STATUS_INIT(LTM_WAIT_ESCALATION_COUNT, LOCKTREE_WAIT_ESCALATION_COUNT,
+ STATUS_UINT64, "number of waits on lock escalation");
+ LTM_STATUS_INIT(LTM_WAIT_ESCALATION_TIME, LOCKTREE_WAIT_ESCALATION_TIME,
+ STATUS_UINT64, "time waiting on lock escalation");
+ LTM_STATUS_INIT(LTM_LONG_WAIT_ESCALATION_COUNT,
+ LOCKTREE_LONG_WAIT_ESCALATION_COUNT, STATUS_UINT64,
+ "number of long waits on lock escalation");
+ LTM_STATUS_INIT(LTM_LONG_WAIT_ESCALATION_TIME,
+ LOCKTREE_LONG_WAIT_ESCALATION_TIME, STATUS_UINT64,
+ "long time waiting on lock escalation");
+
+ m_initialized = true;
+#undef LTM_STATUS_INIT
+}
+void LTM_STATUS_S::destroy() {
+ if (!m_initialized) return;
+ for (int i = 0; i < LTM_STATUS_NUM_ROWS; ++i) {
+ if (status[i].type == STATUS_PARCOUNT) {
+ // PORT: TODO?? destroy_partitioned_counter(status[i].value.parcount);
+ }
+ }
+}
+
+int toku_keycompare(const void *key1, size_t key1len, const void *key2,
+ size_t key2len) {
+ size_t comparelen = key1len < key2len ? key1len : key2len;
+ int c = memcmp(key1, key2, comparelen);
+ if (__builtin_expect(c != 0, 1)) {
+ return c;
+ } else {
+ if (key1len < key2len) {
+ return -1;
+ } else if (key1len > key2len) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+}
+
+int toku_builtin_compare_fun(const DBT *a, const DBT *b) {
+ return toku_keycompare(a->data, a->size, b->data, b->size);
+}
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc
new file mode 100644
index 000000000..63cc3a267
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc
@@ -0,0 +1,153 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "dbt.h"
+
+#include <string.h>
+
+#include "../db.h"
+#include "../portability/memory.h"
+
+DBT *toku_init_dbt(DBT *dbt) {
+ memset(dbt, 0, sizeof(*dbt));
+ return dbt;
+}
+
+DBT toku_empty_dbt(void) {
+ static const DBT empty_dbt = {.data = 0, .size = 0, .ulen = 0, .flags = 0};
+ return empty_dbt;
+}
+
+DBT *toku_init_dbt_flags(DBT *dbt, uint32_t flags) {
+ toku_init_dbt(dbt);
+ dbt->flags = flags;
+ return dbt;
+}
+
+void toku_destroy_dbt(DBT *dbt) {
+ switch (dbt->flags) {
+ case DB_DBT_MALLOC:
+ case DB_DBT_REALLOC:
+ toku_free(dbt->data);
+ toku_init_dbt(dbt);
+ break;
+ }
+}
+
+DBT *toku_fill_dbt(DBT *dbt, const void *k, size_t len) {
+ toku_init_dbt(dbt);
+ dbt->size = len;
+ dbt->data = (char *)k;
+ return dbt;
+}
+
+DBT *toku_memdup_dbt(DBT *dbt, const void *k, size_t len) {
+ toku_init_dbt_flags(dbt, DB_DBT_MALLOC);
+ dbt->size = len;
+ dbt->data = toku_xmemdup(k, len);
+ return dbt;
+}
+
+DBT *toku_copyref_dbt(DBT *dst, const DBT src) {
+ dst->flags = 0;
+ dst->ulen = 0;
+ dst->size = src.size;
+ dst->data = src.data;
+ return dst;
+}
+
+DBT *toku_clone_dbt(DBT *dst, const DBT &src) {
+ return toku_memdup_dbt(dst, src.data, src.size);
+}
+
+void toku_sdbt_cleanup(struct simple_dbt *sdbt) {
+ if (sdbt->data) toku_free(sdbt->data);
+ memset(sdbt, 0, sizeof(*sdbt));
+}
+
+const DBT *toku_dbt_positive_infinity(void) {
+ static DBT positive_infinity_dbt = {
+ .data = 0, .size = 0, .ulen = 0, .flags = 0}; // port
+ return &positive_infinity_dbt;
+}
+
+const DBT *toku_dbt_negative_infinity(void) {
+ static DBT negative_infinity_dbt = {
+ .data = 0, .size = 0, .ulen = 0, .flags = 0}; // port
+ return &negative_infinity_dbt;
+}
+
+bool toku_dbt_is_infinite(const DBT *dbt) {
+ return dbt == toku_dbt_positive_infinity() ||
+ dbt == toku_dbt_negative_infinity();
+}
+
+bool toku_dbt_is_empty(const DBT *dbt) {
+ // can't have a null data field with a non-zero size
+ paranoid_invariant(dbt->data != nullptr || dbt->size == 0);
+ return dbt->data == nullptr;
+}
+
+int toku_dbt_infinite_compare(const DBT *a, const DBT *b) {
+ if (a == b) {
+ return 0;
+ } else if (a == toku_dbt_positive_infinity()) {
+ return 1;
+ } else if (b == toku_dbt_positive_infinity()) {
+ return -1;
+ } else if (a == toku_dbt_negative_infinity()) {
+ return -1;
+ } else {
+ invariant(b == toku_dbt_negative_infinity());
+ return 1;
+ }
+}
+
+bool toku_dbt_equals(const DBT *a, const DBT *b) {
+ if (!toku_dbt_is_infinite(a) && !toku_dbt_is_infinite(b)) {
+ return a->data == b->data && a->size == b->size;
+ } else {
+ // a or b is infinite, so they're equal if they are the same infinite
+ return a == b ? true : false;
+ }
+}
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.h
new file mode 100644
index 000000000..d86c440f8
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.h
@@ -0,0 +1,98 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include "../db.h"
+
+// TODO: John
+// Document this API a little better so that DBT
+// memory management can be morm widely understood.
+
+DBT *toku_init_dbt(DBT *);
+
+// returns: an initialized but empty dbt (for which toku_dbt_is_empty() is true)
+DBT toku_empty_dbt(void);
+
+DBT *toku_init_dbt_flags(DBT *, uint32_t flags);
+
+void toku_destroy_dbt(DBT *);
+
+DBT *toku_fill_dbt(DBT *dbt, const void *k, size_t len);
+
+DBT *toku_memdup_dbt(DBT *dbt, const void *k, size_t len);
+
+DBT *toku_copyref_dbt(DBT *dst, const DBT src);
+
+DBT *toku_clone_dbt(DBT *dst, const DBT &src);
+
+void toku_sdbt_cleanup(struct simple_dbt *sdbt);
+
+// returns: special DBT pointer representing positive infinity
+const DBT *toku_dbt_positive_infinity(void);
+
+// returns: special DBT pointer representing negative infinity
+const DBT *toku_dbt_negative_infinity(void);
+
+// returns: true if the given dbt is either positive or negative infinity
+bool toku_dbt_is_infinite(const DBT *dbt);
+
+// returns: true if the given dbt has no data (ie: dbt->data == nullptr)
+bool toku_dbt_is_empty(const DBT *dbt);
+
+// effect: compares two potentially infinity-valued dbts
+// requires: at least one is infinite (assert otherwise)
+int toku_dbt_infinite_compare(const DBT *a, const DBT *b);
+
+// returns: true if the given dbts have the same data pointer and size
+bool toku_dbt_equals(const DBT *a, const DBT *b);
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h
new file mode 100644
index 000000000..158750fdb
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h
@@ -0,0 +1,144 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <memory.h>
+
+//******************************************************************************
+//
+// Overview: A growable array is a little bit like std::vector except that
+// it doesn't have constructors (hence can be used in static constructs, since
+// the google style guide says no constructors), and it's a little simpler.
+// Operations:
+// init and deinit (we don't have constructors and destructors).
+// fetch_unchecked to get values out.
+// store_unchecked to put values in.
+// push to add an element at the end
+// get_size to find out the size
+// get_memory_size to find out how much memory the data stucture is using.
+//
+//******************************************************************************
+
+namespace toku {
+
+template <typename T>
+class GrowableArray {
+ public:
+ void init(void)
+ // Effect: Initialize the array to contain no elements.
+ {
+ m_array = NULL;
+ m_size = 0;
+ m_size_limit = 0;
+ }
+
+ void deinit(void)
+ // Effect: Deinitialize the array (freeing any memory it uses, for example).
+ {
+ toku_free(m_array);
+ m_array = NULL;
+ m_size = 0;
+ m_size_limit = 0;
+ }
+
+ T fetch_unchecked(size_t i) const
+ // Effect: Fetch the ith element. If i is out of range, the system asserts.
+ {
+ return m_array[i];
+ }
+
+ void store_unchecked(size_t i, T v)
+ // Effect: Store v in the ith element. If i is out of range, the system
+ // asserts.
+ {
+ paranoid_invariant(i < m_size);
+ m_array[i] = v;
+ }
+
+ void push(T v)
+ // Effect: Add v to the end of the array (increasing the size). The amortized
+ // cost of this operation is constant. Implementation hint: Double the size
+ // of the array when it gets too big so that the amortized cost stays
+ // constant.
+ {
+ if (m_size >= m_size_limit) {
+ if (m_array == NULL) {
+ m_size_limit = 1;
+ } else {
+ m_size_limit *= 2;
+ }
+ XREALLOC_N(m_size_limit, m_array);
+ }
+ m_array[m_size++] = v;
+ }
+
+ size_t get_size(void) const
+ // Effect: Return the number of elements in the array.
+ {
+ return m_size;
+ }
+ size_t memory_size(void) const
+ // Effect: Return the size (in bytes) that the array occupies in memory. This
+ // is really only an estimate.
+ {
+ return sizeof(*this) + sizeof(T) * m_size_limit;
+ }
+
+ private:
+ T *m_array;
+ size_t m_size;
+ size_t m_size_limit; // How much space is allocated in array.
+};
+
+} // namespace toku
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc
new file mode 100644
index 000000000..0e7a9880b
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc
@@ -0,0 +1,201 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ifndef ROCKSDB_LITE
+#ifndef OS_WIN
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include "memarena.h"
+
+#include <string.h>
+
+#include <algorithm>
+
+#include "../portability/memory.h"
+
+void memarena::create(size_t initial_size) {
+ _current_chunk = arena_chunk();
+ _other_chunks = nullptr;
+ _size_of_other_chunks = 0;
+ _footprint_of_other_chunks = 0;
+ _n_other_chunks = 0;
+
+ _current_chunk.size = initial_size;
+ if (_current_chunk.size > 0) {
+ XMALLOC_N(_current_chunk.size, _current_chunk.buf);
+ }
+}
+
+void memarena::destroy(void) {
+ if (_current_chunk.buf) {
+ toku_free(_current_chunk.buf);
+ }
+ for (int i = 0; i < _n_other_chunks; i++) {
+ toku_free(_other_chunks[i].buf);
+ }
+ if (_other_chunks) {
+ toku_free(_other_chunks);
+ }
+ _current_chunk = arena_chunk();
+ _other_chunks = nullptr;
+ _n_other_chunks = 0;
+}
+
+static size_t round_to_page(size_t size) {
+ const size_t page_size = 4096;
+ const size_t r = page_size + ((size - 1) & ~(page_size - 1));
+ assert((r & (page_size - 1)) == 0); // make sure it's aligned
+ assert(r >= size); // make sure it's not too small
+ assert(r <
+ size + page_size); // make sure we didn't grow by more than a page.
+ return r;
+}
+
+static const size_t MEMARENA_MAX_CHUNK_SIZE = 64 * 1024 * 1024;
+
+void *memarena::malloc_from_arena(size_t size) {
+ if (_current_chunk.buf == nullptr ||
+ _current_chunk.size < _current_chunk.used + size) {
+ // The existing block isn't big enough.
+ // Add the block to the vector of blocks.
+ if (_current_chunk.buf) {
+ invariant(_current_chunk.size > 0);
+ int old_n = _n_other_chunks;
+ XREALLOC_N(old_n + 1, _other_chunks);
+ _other_chunks[old_n] = _current_chunk;
+ _n_other_chunks = old_n + 1;
+ _size_of_other_chunks += _current_chunk.size;
+ _footprint_of_other_chunks +=
+ toku_memory_footprint(_current_chunk.buf, _current_chunk.used);
+ }
+
+ // Make a new one. Grow the buffer size exponentially until we hit
+ // the max chunk size, but make it at least `size' bytes so the
+ // current allocation always fit.
+ size_t new_size =
+ std::min(MEMARENA_MAX_CHUNK_SIZE, 2 * _current_chunk.size);
+ if (new_size < size) {
+ new_size = size;
+ }
+ new_size = round_to_page(
+ new_size); // at least size, but round to the next page size
+ XMALLOC_N(new_size, _current_chunk.buf);
+ _current_chunk.used = 0;
+ _current_chunk.size = new_size;
+ }
+ invariant(_current_chunk.buf != nullptr);
+
+ // allocate in the existing block.
+ char *p = _current_chunk.buf + _current_chunk.used;
+ _current_chunk.used += size;
+ return p;
+}
+
+void memarena::move_memory(memarena *dest) {
+ // Move memory to dest
+ XREALLOC_N(dest->_n_other_chunks + _n_other_chunks + 1, dest->_other_chunks);
+ dest->_size_of_other_chunks += _size_of_other_chunks + _current_chunk.size;
+ dest->_footprint_of_other_chunks +=
+ _footprint_of_other_chunks +
+ toku_memory_footprint(_current_chunk.buf, _current_chunk.used);
+ for (int i = 0; i < _n_other_chunks; i++) {
+ dest->_other_chunks[dest->_n_other_chunks++] = _other_chunks[i];
+ }
+ dest->_other_chunks[dest->_n_other_chunks++] = _current_chunk;
+
+ // Clear out this memarena's memory
+ toku_free(_other_chunks);
+ _current_chunk = arena_chunk();
+ _other_chunks = nullptr;
+ _size_of_other_chunks = 0;
+ _footprint_of_other_chunks = 0;
+ _n_other_chunks = 0;
+}
+
+size_t memarena::total_memory_size(void) const {
+ return sizeof(*this) + total_size_in_use() +
+ _n_other_chunks * sizeof(*_other_chunks);
+}
+
+size_t memarena::total_size_in_use(void) const {
+ return _size_of_other_chunks + _current_chunk.used;
+}
+
+size_t memarena::total_footprint(void) const {
+ return sizeof(*this) + _footprint_of_other_chunks +
+ toku_memory_footprint(_current_chunk.buf, _current_chunk.used) +
+ _n_other_chunks * sizeof(*_other_chunks);
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+const void *memarena::chunk_iterator::current(size_t *used) const {
+ if (_chunk_idx < 0) {
+ *used = _ma->_current_chunk.used;
+ return _ma->_current_chunk.buf;
+ } else if (_chunk_idx < _ma->_n_other_chunks) {
+ *used = _ma->_other_chunks[_chunk_idx].used;
+ return _ma->_other_chunks[_chunk_idx].buf;
+ }
+ *used = 0;
+ return nullptr;
+}
+
+void memarena::chunk_iterator::next() { _chunk_idx++; }
+
+bool memarena::chunk_iterator::more() const {
+ if (_chunk_idx < 0) {
+ return _ma->_current_chunk.buf != nullptr;
+ }
+ return _chunk_idx < _ma->_n_other_chunks;
+}
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.h
new file mode 100644
index 000000000..ddcc1144f
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.h
@@ -0,0 +1,141 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <stdlib.h>
+
+/*
+ * A memarena is used to efficiently store a collection of objects that never
+ * move The pattern is allocate more and more stuff and free all of the items at
+ * once. The underlying memory will store 1 or more objects per chunk. Each
+ * chunk is contiguously laid out in memory but chunks are not necessarily
+ * contiguous with each other.
+ */
+class memarena {
+ public:
+ memarena()
+ : _current_chunk(arena_chunk()),
+ _other_chunks(nullptr),
+ _n_other_chunks(0),
+ _size_of_other_chunks(0),
+ _footprint_of_other_chunks(0) {}
+
+ // Effect: Create a memarena with the specified initial size
+ void create(size_t initial_size);
+
+ void destroy(void);
+
+ // Effect: Allocate some memory. The returned value remains valid until the
+ // memarena is cleared or closed.
+ // In case of ENOMEM, aborts.
+ void *malloc_from_arena(size_t size);
+
+ // Effect: Move all the memory from this memarena into DEST.
+ // When SOURCE is closed the memory won't be freed.
+ // When DEST is closed, the memory will be freed, unless DEST moves
+ // its memory to another memarena...
+ void move_memory(memarena *dest);
+
+ // Effect: Calculate the amount of memory used by a memory arena.
+ size_t total_memory_size(void) const;
+
+ // Effect: Calculate the used space of the memory arena (ie: excludes unused
+ // space)
+ size_t total_size_in_use(void) const;
+
+ // Effect: Calculate the amount of memory used, according to
+ // toku_memory_footprint(),
+ // which is a more expensive but more accurate count of memory used.
+ size_t total_footprint(void) const;
+
+ // iterator over the underlying chunks that store objects in the memarena.
+ // a chunk is represented by a pointer to const memory and a usable byte
+ // count.
+ class chunk_iterator {
+ public:
+ chunk_iterator(const memarena *ma) : _ma(ma), _chunk_idx(-1) {}
+
+ // returns: base pointer to the current chunk
+ // *used set to the number of usable bytes
+ // if more() is false, returns nullptr and *used = 0
+ const void *current(size_t *used) const;
+
+ // requires: more() is true
+ void next();
+
+ bool more() const;
+
+ private:
+ // -1 represents the 'initial' chunk in a memarena, ie: ma->_current_chunk
+ // >= 0 represents the i'th chunk in the ma->_other_chunks array
+ const memarena *_ma;
+ int _chunk_idx;
+ };
+
+ private:
+ struct arena_chunk {
+ arena_chunk() : buf(nullptr), used(0), size(0) {}
+ char *buf;
+ size_t used;
+ size_t size;
+ };
+
+ struct arena_chunk _current_chunk;
+ struct arena_chunk *_other_chunks;
+ int _n_other_chunks;
+ size_t _size_of_other_chunks; // the buf_size of all the other chunks.
+ size_t _footprint_of_other_chunks; // the footprint of all the other chunks.
+
+ friend class memarena_unit_test;
+};
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt.h
new file mode 100644
index 000000000..f208002d3
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt.h
@@ -0,0 +1,794 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include <memory.h>
+#include <stdint.h>
+
+#include "../portability/toku_portability.h"
+#include "../portability/toku_race_tools.h"
+#include "growable_array.h"
+
+namespace toku {
+
+/**
+ * Order Maintenance Tree (OMT)
+ *
+ * Maintains a collection of totally ordered values, where each value has an
+ * integer weight. The OMT is a mutable datatype.
+ *
+ * The Abstraction:
+ *
+ * An OMT is a vector of values, $V$, where $|V|$ is the length of the vector.
+ * The vector is numbered from $0$ to $|V|-1$.
+ * Each value has a weight. The weight of the $i$th element is denoted
+ * $w(V_i)$.
+ *
+ * We can create a new OMT, which is the empty vector.
+ *
+ * We can insert a new element $x$ into slot $i$, changing $V$ into $V'$ where
+ * $|V'|=1+|V|$ and
+ *
+ * V'_j = V_j if $j<i$
+ * x if $j=i$
+ * V_{j-1} if $j>i$.
+ *
+ * We can specify $i$ using a kind of function instead of as an integer.
+ * Let $b$ be a function mapping from values to nonzero integers, such that
+ * the signum of $b$ is monotically increasing.
+ * We can specify $i$ as the minimum integer such that $b(V_i)>0$.
+ *
+ * We look up a value using its index, or using a Heaviside function.
+ * For lookups, we allow $b$ to be zero for some values, and again the signum of
+ * $b$ must be monotonically increasing. When lookup up values, we can look up
+ * $V_i$ where $i$ is the minimum integer such that $b(V_i)=0$. (With a
+ * special return code if no such value exists.) (Rationale: Ordinarily we want
+ * $i$ to be unique. But for various reasons we want to allow multiple zeros,
+ * and we want the smallest $i$ in that case.) $V_i$ where $i$ is the minimum
+ * integer such that $b(V_i)>0$. (Or an indication that no such value exists.)
+ * $V_i$ where $i$ is the maximum integer such that $b(V_i)<0$. (Or an
+ * indication that no such value exists.)
+ *
+ * When looking up a value using a Heaviside function, we get the value and its
+ * index.
+ *
+ * We can also split an OMT into two OMTs, splitting the weight of the values
+ * evenly. Find a value $j$ such that the values to the left of $j$ have about
+ * the same total weight as the values to the right of $j$. The resulting two
+ * OMTs contain the values to the left of $j$ and the values to the right of $j$
+ * respectively. All of the values from the original OMT go into one of the new
+ * OMTs. If the weights of the values don't split exactly evenly, then the
+ * implementation has the freedom to choose whether the new left OMT or the new
+ * right OMT is larger.
+ *
+ * Performance:
+ * Insertion and deletion should run with $O(\log |V|)$ time and $O(\log |V|)$
+ * calls to the Heaviside function. The memory required is O(|V|).
+ *
+ * Usage:
+ * The omt is templated by two parameters:
+ * - omtdata_t is what will be stored within the omt. These could be pointers
+ * or real data types (ints, structs).
+ * - omtdataout_t is what will be returned by find and related functions. By
+ * default, it is the same as omtdata_t, but you can set it to (omtdata_t *). To
+ * create an omt which will store "TXNID"s, for example, it is a good idea to
+ * typedef the template: typedef omt<TXNID> txnid_omt_t; If you are storing
+ * structs, you may want to be able to get a pointer to the data actually stored
+ * in the omt (see find_zero). To do this, use the second template parameter:
+ * typedef omt<struct foo, struct foo *> foo_omt_t;
+ */
+
+namespace omt_internal {
+
+template <bool subtree_supports_marks>
+class subtree_templated {
+ private:
+ uint32_t m_index;
+
+ public:
+ static const uint32_t NODE_NULL = UINT32_MAX;
+ inline void set_to_null(void) { m_index = NODE_NULL; }
+
+ inline bool is_null(void) const { return NODE_NULL == this->get_index(); }
+
+ inline uint32_t get_index(void) const { return m_index; }
+
+ inline void set_index(uint32_t index) {
+ paranoid_invariant(index != NODE_NULL);
+ m_index = index;
+ }
+} __attribute__((__packed__, aligned(4)));
+
+template <>
+class subtree_templated<true> {
+ private:
+ uint32_t m_bitfield;
+ static const uint32_t MASK_INDEX = ~(((uint32_t)1) << 31);
+ static const uint32_t MASK_BIT = ((uint32_t)1) << 31;
+
+ inline void set_index_internal(uint32_t new_index) {
+ m_bitfield = (m_bitfield & MASK_BIT) | new_index;
+ }
+
+ public:
+ static const uint32_t NODE_NULL = INT32_MAX;
+ inline void set_to_null(void) { this->set_index_internal(NODE_NULL); }
+
+ inline bool is_null(void) const { return NODE_NULL == this->get_index(); }
+
+ inline uint32_t get_index(void) const {
+ TOKU_DRD_IGNORE_VAR(m_bitfield);
+ const uint32_t bits = m_bitfield;
+ TOKU_DRD_STOP_IGNORING_VAR(m_bitfield);
+ return bits & MASK_INDEX;
+ }
+
+ inline void set_index(uint32_t index) {
+ paranoid_invariant(index < NODE_NULL);
+ this->set_index_internal(index);
+ }
+
+ inline bool get_bit(void) const {
+ TOKU_DRD_IGNORE_VAR(m_bitfield);
+ const uint32_t bits = m_bitfield;
+ TOKU_DRD_STOP_IGNORING_VAR(m_bitfield);
+ return (bits & MASK_BIT) != 0;
+ }
+
+ inline void enable_bit(void) {
+ // These bits may be set by a thread with a write lock on some
+ // leaf, and the index can be read by another thread with a (read
+ // or write) lock on another thread. Also, the has_marks_below
+ // bit can be set by two threads simultaneously. Neither of these
+ // are real races, so if we are using DRD we should tell it to
+ // ignore these bits just while we set this bit. If there were a
+ // race in setting the index, that would be a real race.
+ TOKU_DRD_IGNORE_VAR(m_bitfield);
+ m_bitfield |= MASK_BIT;
+ TOKU_DRD_STOP_IGNORING_VAR(m_bitfield);
+ }
+
+ inline void disable_bit(void) { m_bitfield &= MASK_INDEX; }
+} __attribute__((__packed__));
+
+template <typename omtdata_t, bool subtree_supports_marks>
+class omt_node_templated {
+ public:
+ omtdata_t value;
+ uint32_t weight;
+ subtree_templated<subtree_supports_marks> left;
+ subtree_templated<subtree_supports_marks> right;
+
+ // this needs to be in both implementations because we don't have
+ // a "static if" the caller can use
+ inline void clear_stolen_bits(void) {}
+}; // note: originally this class had __attribute__((__packed__, aligned(4)))
+
+template <typename omtdata_t>
+class omt_node_templated<omtdata_t, true> {
+ public:
+ omtdata_t value;
+ uint32_t weight;
+ subtree_templated<true> left;
+ subtree_templated<true> right;
+ inline bool get_marked(void) const { return left.get_bit(); }
+ inline void set_marked_bit(void) { return left.enable_bit(); }
+ inline void unset_marked_bit(void) { return left.disable_bit(); }
+
+ inline bool get_marks_below(void) const { return right.get_bit(); }
+ inline void set_marks_below_bit(void) {
+ // This function can be called by multiple threads.
+ // Checking first reduces cache invalidation.
+ if (!this->get_marks_below()) {
+ right.enable_bit();
+ }
+ }
+ inline void unset_marks_below_bit(void) { right.disable_bit(); }
+
+ inline void clear_stolen_bits(void) {
+ this->unset_marked_bit();
+ this->unset_marks_below_bit();
+ }
+}; // note: originally this class had __attribute__((__packed__, aligned(4)))
+
+} // namespace omt_internal
+
+template <typename omtdata_t, typename omtdataout_t = omtdata_t,
+ bool supports_marks = false>
+class omt {
+ public:
+ /**
+ * Effect: Create an empty OMT.
+ * Performance: constant time.
+ */
+ void create(void);
+
+ /**
+ * Effect: Create an empty OMT with no internal allocated space.
+ * Performance: constant time.
+ * Rationale: In some cases we need a valid omt but don't want to malloc.
+ */
+ void create_no_array(void);
+
+ /**
+ * Effect: Create a OMT containing values. The number of values is in
+ * numvalues. Stores the new OMT in *omtp. Requires: this has not been created
+ * yet Requires: values != NULL Requires: values is sorted Performance:
+ * time=O(numvalues) Rationale: Normally to insert N values takes O(N lg N)
+ * amortized time. If the N values are known in advance, are sorted, and the
+ * structure is empty, we can batch insert them much faster.
+ */
+ __attribute__((nonnull)) void create_from_sorted_array(
+ const omtdata_t *const values, const uint32_t numvalues);
+
+ /**
+ * Effect: Create an OMT containing values. The number of values is in
+ * numvalues. On success the OMT takes ownership of *values array, and sets
+ * values=NULL. Requires: this has not been created yet Requires: values !=
+ * NULL Requires: *values is sorted Requires: *values was allocated with
+ * toku_malloc Requires: Capacity of the *values array is <= new_capacity
+ * Requires: On success, *values may not be accessed again by the caller.
+ * Performance: time=O(1)
+ * Rational: create_from_sorted_array takes O(numvalues) time.
+ * By taking ownership of the array, we save a malloc and
+ * memcpy, and possibly a free (if the caller is done with the array).
+ */
+ void create_steal_sorted_array(omtdata_t **const values,
+ const uint32_t numvalues,
+ const uint32_t new_capacity);
+
+ /**
+ * Effect: Create a new OMT, storing it in *newomt.
+ * The values to the right of index (starting at index) are moved to *newomt.
+ * Requires: newomt != NULL
+ * Returns
+ * 0 success,
+ * EINVAL if index > toku_omt_size(omt)
+ * On nonzero return, omt and *newomt are unmodified.
+ * Performance: time=O(n)
+ * Rationale: We don't need a split-evenly operation. We need to split items
+ * so that their total sizes are even, and other similar splitting criteria.
+ * It's easy to split evenly by calling size(), and dividing by two.
+ */
+ __attribute__((nonnull)) int split_at(omt *const newomt, const uint32_t idx);
+
+ /**
+ * Effect: Appends leftomt and rightomt to produce a new omt.
+ * Creates this as the new omt.
+ * leftomt and rightomt are destroyed.
+ * Performance: time=O(n) is acceptable, but one can imagine implementations
+ * that are O(\log n) worst-case.
+ */
+ __attribute__((nonnull)) void merge(omt *const leftomt, omt *const rightomt);
+
+ /**
+ * Effect: Creates a copy of an omt.
+ * Creates this as the clone.
+ * Each element is copied directly. If they are pointers, the underlying
+ * data is not duplicated. Performance: O(n) or the running time of
+ * fill_array_with_subtree_values()
+ */
+ void clone(const omt &src);
+
+ /**
+ * Effect: Set the tree to be empty.
+ * Note: Will not reallocate or resize any memory.
+ * Performance: time=O(1)
+ */
+ void clear(void);
+
+ /**
+ * Effect: Destroy an OMT, freeing all its memory.
+ * If the values being stored are pointers, their underlying data is not
+ * freed. See free_items() Those values may be freed before or after calling
+ * toku_omt_destroy. Rationale: Returns no values since free() cannot fail.
+ * Rationale: Does not free the underlying pointers to reduce complexity.
+ * Performance: time=O(1)
+ */
+ void destroy(void);
+
+ /**
+ * Effect: return |this|.
+ * Performance: time=O(1)
+ */
+ uint32_t size(void) const;
+
+ /**
+ * Effect: Insert value into the OMT.
+ * If there is some i such that $h(V_i, v)=0$ then returns DB_KEYEXIST.
+ * Otherwise, let i be the minimum value such that $h(V_i, v)>0$.
+ * If no such i exists, then let i be |V|
+ * Then this has the same effect as
+ * insert_at(tree, value, i);
+ * If idx!=NULL then i is stored in *idx
+ * Requires: The signum of h must be monotonically increasing.
+ * Returns:
+ * 0 success
+ * DB_KEYEXIST the key is present (h was equal to zero for some value)
+ * On nonzero return, omt is unchanged.
+ * Performance: time=O(\log N) amortized.
+ * Rationale: Some future implementation may be O(\log N) worst-case time, but
+ * O(\log N) amortized is good enough for now.
+ */
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int insert(const omtdata_t &value, const omtcmp_t &v, uint32_t *const idx);
+
+ /**
+ * Effect: Increases indexes of all items at slot >= idx by 1.
+ * Insert value into the position at idx.
+ * Returns:
+ * 0 success
+ * EINVAL if idx > this->size()
+ * On error, omt is unchanged.
+ * Performance: time=O(\log N) amortized time.
+ * Rationale: Some future implementation may be O(\log N) worst-case time, but
+ * O(\log N) amortized is good enough for now.
+ */
+ int insert_at(const omtdata_t &value, const uint32_t idx);
+
+ /**
+ * Effect: Replaces the item at idx with value.
+ * Returns:
+ * 0 success
+ * EINVAL if idx>=this->size()
+ * On error, omt is unchanged.
+ * Performance: time=O(\log N)
+ * Rationale: The FT needs to be able to replace a value with another copy of
+ * the same value (allocated in a different location)
+ *
+ */
+ int set_at(const omtdata_t &value, const uint32_t idx);
+
+ /**
+ * Effect: Delete the item in slot idx.
+ * Decreases indexes of all items at slot > idx by 1.
+ * Returns
+ * 0 success
+ * EINVAL if idx>=this->size()
+ * On error, omt is unchanged.
+ * Rationale: To delete an item, first find its index using find or find_zero,
+ * then delete it. Performance: time=O(\log N) amortized.
+ */
+ int delete_at(const uint32_t idx);
+
+ /**
+ * Effect: Iterate over the values of the omt, from left to right, calling f
+ * on each value. The first argument passed to f is a ref-to-const of the
+ * value stored in the omt. The second argument passed to f is the index of
+ * the value. The third argument passed to f is iterate_extra. The indices run
+ * from 0 (inclusive) to this->size() (exclusive). Requires: f != NULL
+ * Returns:
+ * If f ever returns nonzero, then the iteration stops, and the value
+ * returned by f is returned by iterate. If f always returns zero, then
+ * iterate returns 0. Requires: Don't modify the omt while running. (E.g., f
+ * may not insert or delete values from the omt.) Performance: time=O(i+\log
+ * N) where i is the number of times f is called, and N is the number of
+ * elements in the omt. Rationale: Although the functional iterator requires
+ * defining another function (as opposed to C++ style iterator), it is much
+ * easier to read. Rationale: We may at some point use functors, but for now
+ * this is a smaller change from the old OMT.
+ */
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate(iterate_extra_t *const iterate_extra) const;
+
+ /**
+ * Effect: Iterate over the values of the omt, from left to right, calling f
+ * on each value. The first argument passed to f is a ref-to-const of the
+ * value stored in the omt. The second argument passed to f is the index of
+ * the value. The third argument passed to f is iterate_extra. The indices run
+ * from 0 (inclusive) to this->size() (exclusive). We will iterate only over
+ * [left,right)
+ *
+ * Requires: left <= right
+ * Requires: f != NULL
+ * Returns:
+ * EINVAL if right > this->size()
+ * If f ever returns nonzero, then the iteration stops, and the value
+ * returned by f is returned by iterate_on_range. If f always returns zero,
+ * then iterate_on_range returns 0. Requires: Don't modify the omt while
+ * running. (E.g., f may not insert or delete values from the omt.)
+ * Performance: time=O(i+\log N) where i is the number of times f is called,
+ * and N is the number of elements in the omt. Rational: Although the
+ * functional iterator requires defining another function (as opposed to C++
+ * style iterator), it is much easier to read.
+ */
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate_on_range(const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra) const;
+
+ /**
+ * Effect: Iterate over the values of the omt, and mark the nodes that are
+ * visited. Other than the marks, this behaves the same as iterate_on_range.
+ * Requires: supports_marks == true
+ * Performance: time=O(i+\log N) where i is the number of times f is called,
+ * and N is the number of elements in the omt. Notes: This function MAY be
+ * called concurrently by multiple threads, but not concurrently with any
+ * other non-const function.
+ */
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate_and_mark_range(const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra);
+
+ /**
+ * Effect: Iterate over the values of the omt, from left to right, calling f
+ * on each value whose node has been marked. Other than the marks, this
+ * behaves the same as iterate. Requires: supports_marks == true Performance:
+ * time=O(i+\log N) where i is the number of times f is called, and N is the
+ * number of elements in the omt.
+ */
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate_over_marked(iterate_extra_t *const iterate_extra) const;
+
+ /**
+ * Effect: Delete all elements from the omt, whose nodes have been marked.
+ * Requires: supports_marks == true
+ * Performance: time=O(N + i\log N) where i is the number of marked elements,
+ * {c,sh}ould be faster
+ */
+ void delete_all_marked(void);
+
+ /**
+ * Effect: Verify that the internal state of the marks in the tree are
+ * self-consistent. Crashes the system if the marks are in a bad state.
+ * Requires: supports_marks == true
+ * Performance: time=O(N)
+ * Notes:
+ * Even though this is a const function, it requires exclusive access.
+ * Rationale:
+ * The current implementation of the marks relies on a sort of
+ * "cache" bit representing the state of bits below it in the tree.
+ * This allows glass-box testing that these bits are correct.
+ */
+ void verify_marks_consistent(void) const;
+
+ /**
+ * Effect: None
+ * Returns whether there are any marks in the tree.
+ */
+ bool has_marks(void) const;
+
+ /**
+ * Effect: Iterate over the values of the omt, from left to right, calling f
+ * on each value. The first argument passed to f is a pointer to the value
+ * stored in the omt. The second argument passed to f is the index of the
+ * value. The third argument passed to f is iterate_extra. The indices run
+ * from 0 (inclusive) to this->size() (exclusive). Requires: same as for
+ * iterate() Returns: same as for iterate() Performance: same as for iterate()
+ * Rationale: In general, most iterators should use iterate() since they
+ * should not modify the data stored in the omt. This function is for
+ * iterators which need to modify values (for example, free_items). Rationale:
+ * We assume if you are transforming the data in place, you want to do it to
+ * everything at once, so there is not yet an iterate_on_range_ptr (but there
+ * could be).
+ */
+ template <typename iterate_extra_t,
+ int (*f)(omtdata_t *, const uint32_t, iterate_extra_t *const)>
+ void iterate_ptr(iterate_extra_t *const iterate_extra);
+
+ /**
+ * Effect: Set *value=V_idx
+ * Returns
+ * 0 success
+ * EINVAL if index>=toku_omt_size(omt)
+ * On nonzero return, *value is unchanged
+ * Performance: time=O(\log N)
+ */
+ int fetch(const uint32_t idx, omtdataout_t *const value) const;
+
+ /**
+ * Effect: Find the smallest i such that h(V_i, extra)>=0
+ * If there is such an i and h(V_i,extra)==0 then set *idxp=i, set *value =
+ * V_i, and return 0. If there is such an i and h(V_i,extra)>0 then set
+ * *idxp=i and return DB_NOTFOUND. If there is no such i then set
+ * *idx=this->size() and return DB_NOTFOUND. Note: value is of type
+ * omtdataout_t, which may be of type (omtdata_t) or (omtdata_t *) but is
+ * fixed by the instantiation. If it is the value type, then the value is
+ * copied out (even if the value type is a pointer to something else) If it is
+ * the pointer type, then *value is set to a pointer to the data within the
+ * omt. This is determined by the type of the omt as initially declared. If
+ * the omt is declared as omt<foo_t>, then foo_t's will be stored and foo_t's
+ * will be returned by find and related functions. If the omt is declared as
+ * omt<foo_t, foo_t *>, then foo_t's will be stored, and pointers to the
+ * stored items will be returned by find and related functions. Rationale:
+ * Structs too small for malloc should be stored directly in the omt.
+ * These structs may need to be edited as they exist inside the omt, so we
+ * need a way to get a pointer within the omt. Using separate functions for
+ * returning pointers and values increases code duplication and reduces
+ * type-checking. That also reduces the ability of the creator of a data
+ * structure to give advice to its future users. Slight overloading in this
+ * case seemed to provide a better API and better type checking.
+ */
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find_zero(const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const;
+
+ /**
+ * Effect:
+ * If direction >0 then find the smallest i such that h(V_i,extra)>0.
+ * If direction <0 then find the largest i such that h(V_i,extra)<0.
+ * (Direction may not be equal to zero.)
+ * If value!=NULL then store V_i in *value
+ * If idxp!=NULL then store i in *idxp.
+ * Requires: The signum of h is monotically increasing.
+ * Returns
+ * 0 success
+ * DB_NOTFOUND no such value is found.
+ * On nonzero return, *value and *idxp are unchanged
+ * Performance: time=O(\log N)
+ * Rationale:
+ * Here's how to use the find function to find various things
+ * Cases for find:
+ * find first value: ( h(v)=+1, direction=+1 )
+ * find last value ( h(v)=-1, direction=-1 )
+ * find first X ( h(v)=(v< x) ? -1 : 1 direction=+1 )
+ * find last X ( h(v)=(v<=x) ? -1 : 1 direction=-1 )
+ * find X or successor to X ( same as find first X. )
+ *
+ * Rationale: To help understand heaviside functions and behavor of find:
+ * There are 7 kinds of heaviside functions.
+ * The signus of the h must be monotonically increasing.
+ * Given a function of the following form, A is the element
+ * returned for direction>0, B is the element returned
+ * for direction<0, C is the element returned for
+ * direction==0 (see find_zero) (with a return of 0), and D is the element
+ * returned for direction==0 (see find_zero) with a return of DB_NOTFOUND.
+ * If any of A, B, or C are not found, then asking for the
+ * associated direction will return DB_NOTFOUND.
+ * See find_zero for more information.
+ *
+ * Let the following represent the signus of the heaviside function.
+ *
+ * -...-
+ * A
+ * D
+ *
+ * +...+
+ * B
+ * D
+ *
+ * 0...0
+ * C
+ *
+ * -...-0...0
+ * AC
+ *
+ * 0...0+...+
+ * C B
+ *
+ * -...-+...+
+ * AB
+ * D
+ *
+ * -...-0...0+...+
+ * AC B
+ */
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find(const omtcmp_t &extra, int direction, omtdataout_t *const value,
+ uint32_t *const idxp) const;
+
+ /**
+ * Effect: Return the size (in bytes) of the omt, as it resides in main
+ * memory. If the data stored are pointers, don't include the size of what
+ * they all point to.
+ */
+ size_t memory_size(void);
+
+ private:
+ typedef uint32_t node_idx;
+ typedef omt_internal::subtree_templated<supports_marks> subtree;
+ typedef omt_internal::omt_node_templated<omtdata_t, supports_marks> omt_node;
+ ENSURE_POD(subtree);
+
+ struct omt_array {
+ uint32_t start_idx;
+ uint32_t num_values;
+ omtdata_t *values;
+ };
+
+ struct omt_tree {
+ subtree root;
+ uint32_t free_idx;
+ omt_node *nodes;
+ };
+
+ bool is_array;
+ uint32_t capacity;
+ union {
+ struct omt_array a;
+ struct omt_tree t;
+ } d;
+
+ __attribute__((nonnull)) void unmark(const subtree &subtree,
+ const uint32_t index,
+ GrowableArray<node_idx> *const indexes);
+
+ void create_internal_no_array(const uint32_t new_capacity);
+
+ void create_internal(const uint32_t new_capacity);
+
+ uint32_t nweight(const subtree &subtree) const;
+
+ node_idx node_malloc(void);
+
+ void node_free(const node_idx idx);
+
+ void maybe_resize_array(const uint32_t n);
+
+ __attribute__((nonnull)) void fill_array_with_subtree_values(
+ omtdata_t *const array, const subtree &subtree) const;
+
+ void convert_to_array(void);
+
+ __attribute__((nonnull)) void rebuild_from_sorted_array(
+ subtree *const subtree, const omtdata_t *const values,
+ const uint32_t numvalues);
+
+ void convert_to_tree(void);
+
+ void maybe_resize_or_convert(const uint32_t n);
+
+ bool will_need_rebalance(const subtree &subtree, const int leftmod,
+ const int rightmod) const;
+
+ __attribute__((nonnull)) void insert_internal(
+ subtree *const subtreep, const omtdata_t &value, const uint32_t idx,
+ subtree **const rebalance_subtree);
+
+ void set_at_internal_array(const omtdata_t &value, const uint32_t idx);
+
+ void set_at_internal(const subtree &subtree, const omtdata_t &value,
+ const uint32_t idx);
+
+ void delete_internal(subtree *const subtreep, const uint32_t idx,
+ omt_node *const copyn,
+ subtree **const rebalance_subtree);
+
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate_internal_array(const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra) const;
+
+ template <typename iterate_extra_t,
+ int (*f)(omtdata_t *, const uint32_t, iterate_extra_t *const)>
+ void iterate_ptr_internal(const uint32_t left, const uint32_t right,
+ const subtree &subtree, const uint32_t idx,
+ iterate_extra_t *const iterate_extra);
+
+ template <typename iterate_extra_t,
+ int (*f)(omtdata_t *, const uint32_t, iterate_extra_t *const)>
+ void iterate_ptr_internal_array(const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra);
+
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate_internal(const uint32_t left, const uint32_t right,
+ const subtree &subtree, const uint32_t idx,
+ iterate_extra_t *const iterate_extra) const;
+
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate_and_mark_range_internal(const uint32_t left, const uint32_t right,
+ const subtree &subtree,
+ const uint32_t idx,
+ iterate_extra_t *const iterate_extra);
+
+ template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+ int iterate_over_marked_internal(const subtree &subtree, const uint32_t idx,
+ iterate_extra_t *const iterate_extra) const;
+
+ uint32_t verify_marks_consistent_internal(const subtree &subtree,
+ const bool allow_marks) const;
+
+ void fetch_internal_array(const uint32_t i, omtdataout_t *const value) const;
+
+ void fetch_internal(const subtree &subtree, const uint32_t i,
+ omtdataout_t *const value) const;
+
+ __attribute__((nonnull)) void fill_array_with_subtree_idxs(
+ node_idx *const array, const subtree &subtree) const;
+
+ __attribute__((nonnull)) void rebuild_subtree_from_idxs(
+ subtree *const subtree, const node_idx *const idxs,
+ const uint32_t numvalues);
+
+ __attribute__((nonnull)) void rebalance(subtree *const subtree);
+
+ __attribute__((nonnull)) static void copyout(omtdata_t *const out,
+ const omt_node *const n);
+
+ __attribute__((nonnull)) static void copyout(omtdata_t **const out,
+ omt_node *const n);
+
+ __attribute__((nonnull)) static void copyout(
+ omtdata_t *const out, const omtdata_t *const stored_value_ptr);
+
+ __attribute__((nonnull)) static void copyout(
+ omtdata_t **const out, omtdata_t *const stored_value_ptr);
+
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find_internal_zero_array(const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const;
+
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find_internal_zero(const subtree &subtree, const omtcmp_t &extra,
+ omtdataout_t *const value, uint32_t *const idxp) const;
+
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find_internal_plus_array(const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const;
+
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find_internal_plus(const subtree &subtree, const omtcmp_t &extra,
+ omtdataout_t *const value, uint32_t *const idxp) const;
+
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find_internal_minus_array(const omtcmp_t &extra,
+ omtdataout_t *const value,
+ uint32_t *const idxp) const;
+
+ template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+ int find_internal_minus(const subtree &subtree, const omtcmp_t &extra,
+ omtdataout_t *const value,
+ uint32_t *const idxp) const;
+};
+
+} // namespace toku
+
+// include the implementation here
+#include "omt_impl.h"
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt_impl.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt_impl.h
new file mode 100644
index 000000000..e77986716
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt_impl.h
@@ -0,0 +1,1295 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#include <string.h>
+
+#include "../db.h"
+#include "../portability/memory.h"
+
+namespace toku {
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::create(void) {
+ this->create_internal(2);
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::create_no_array(void) {
+ if (!supports_marks) {
+ this->create_internal_no_array(0);
+ } else {
+ this->is_array = false;
+ this->capacity = 0;
+ this->d.t.nodes = nullptr;
+ this->d.t.root.set_to_null();
+ this->d.t.free_idx = 0;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::create_from_sorted_array(
+ const omtdata_t *const values, const uint32_t numvalues) {
+ this->create_internal(numvalues);
+ memcpy(this->d.a.values, values, numvalues * (sizeof values[0]));
+ this->d.a.num_values = numvalues;
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::create_steal_sorted_array(
+ omtdata_t **const values, const uint32_t numvalues,
+ const uint32_t new_capacity) {
+ paranoid_invariant_notnull(values);
+ this->create_internal_no_array(new_capacity);
+ this->d.a.num_values = numvalues;
+ this->d.a.values = *values;
+ *values = nullptr;
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+int omt<omtdata_t, omtdataout_t, supports_marks>::split_at(omt *const newomt,
+ const uint32_t idx) {
+ barf_if_marked(*this);
+ paranoid_invariant_notnull(newomt);
+ if (idx > this->size()) {
+ return EINVAL;
+ }
+ this->convert_to_array();
+ const uint32_t newsize = this->size() - idx;
+ newomt->create_from_sorted_array(&this->d.a.values[this->d.a.start_idx + idx],
+ newsize);
+ this->d.a.num_values = idx;
+ this->maybe_resize_array(idx);
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::merge(omt *const leftomt,
+ omt *const rightomt) {
+ barf_if_marked(*this);
+ paranoid_invariant_notnull(leftomt);
+ paranoid_invariant_notnull(rightomt);
+ const uint32_t leftsize = leftomt->size();
+ const uint32_t rightsize = rightomt->size();
+ const uint32_t newsize = leftsize + rightsize;
+
+ if (leftomt->is_array) {
+ if (leftomt->capacity -
+ (leftomt->d.a.start_idx + leftomt->d.a.num_values) >=
+ rightsize) {
+ this->create_steal_sorted_array(
+ &leftomt->d.a.values, leftomt->d.a.num_values, leftomt->capacity);
+ this->d.a.start_idx = leftomt->d.a.start_idx;
+ } else {
+ this->create_internal(newsize);
+ memcpy(&this->d.a.values[0], &leftomt->d.a.values[leftomt->d.a.start_idx],
+ leftomt->d.a.num_values * (sizeof this->d.a.values[0]));
+ }
+ } else {
+ this->create_internal(newsize);
+ leftomt->fill_array_with_subtree_values(&this->d.a.values[0],
+ leftomt->d.t.root);
+ }
+ leftomt->destroy();
+ this->d.a.num_values = leftsize;
+
+ if (rightomt->is_array) {
+ memcpy(&this->d.a.values[this->d.a.start_idx + this->d.a.num_values],
+ &rightomt->d.a.values[rightomt->d.a.start_idx],
+ rightomt->d.a.num_values * (sizeof this->d.a.values[0]));
+ } else {
+ rightomt->fill_array_with_subtree_values(
+ &this->d.a.values[this->d.a.start_idx + this->d.a.num_values],
+ rightomt->d.t.root);
+ }
+ rightomt->destroy();
+ this->d.a.num_values += rightsize;
+ paranoid_invariant(this->size() == newsize);
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::clone(const omt &src) {
+ barf_if_marked(*this);
+ this->create_internal(src.size());
+ if (src.is_array) {
+ memcpy(&this->d.a.values[0], &src.d.a.values[src.d.a.start_idx],
+ src.d.a.num_values * (sizeof this->d.a.values[0]));
+ } else {
+ src.fill_array_with_subtree_values(&this->d.a.values[0], src.d.t.root);
+ }
+ this->d.a.num_values = src.size();
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::clear(void) {
+ if (this->is_array) {
+ this->d.a.start_idx = 0;
+ this->d.a.num_values = 0;
+ } else {
+ this->d.t.root.set_to_null();
+ this->d.t.free_idx = 0;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::destroy(void) {
+ this->clear();
+ this->capacity = 0;
+ if (this->is_array) {
+ if (this->d.a.values != nullptr) {
+ toku_free(this->d.a.values);
+ }
+ this->d.a.values = nullptr;
+ } else {
+ if (this->d.t.nodes != nullptr) {
+ toku_free(this->d.t.nodes);
+ }
+ this->d.t.nodes = nullptr;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+uint32_t omt<omtdata_t, omtdataout_t, supports_marks>::size(void) const {
+ if (this->is_array) {
+ return this->d.a.num_values;
+ } else {
+ return this->nweight(this->d.t.root);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::insert(const omtdata_t &value,
+ const omtcmp_t &v,
+ uint32_t *const idx) {
+ int r;
+ uint32_t insert_idx;
+
+ r = this->find_zero<omtcmp_t, h>(v, nullptr, &insert_idx);
+ if (r == 0) {
+ if (idx) *idx = insert_idx;
+ return DB_KEYEXIST;
+ }
+ if (r != DB_NOTFOUND) return r;
+
+ if ((r = this->insert_at(value, insert_idx))) return r;
+ if (idx) *idx = insert_idx;
+
+ return 0;
+}
+
+// The following 3 functions implement a static if for us.
+template <typename omtdata_t, typename omtdataout_t>
+static void barf_if_marked(const omt<omtdata_t, omtdataout_t, false> &UU(omt)) {
+}
+
+template <typename omtdata_t, typename omtdataout_t>
+static void barf_if_marked(const omt<omtdata_t, omtdataout_t, true> &omt) {
+ invariant(!omt.has_marks());
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+bool omt<omtdata_t, omtdataout_t, supports_marks>::has_marks(void) const {
+ static_assert(supports_marks, "Does not support marks");
+ if (this->d.t.root.is_null()) {
+ return false;
+ }
+ const omt_node &node = this->d.t.nodes[this->d.t.root.get_index()];
+ return node.get_marks_below() || node.get_marked();
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+int omt<omtdata_t, omtdataout_t, supports_marks>::insert_at(
+ const omtdata_t &value, const uint32_t idx) {
+ barf_if_marked(*this);
+ if (idx > this->size()) {
+ return EINVAL;
+ }
+
+ this->maybe_resize_or_convert(this->size() + 1);
+ if (this->is_array && idx != this->d.a.num_values &&
+ (idx != 0 || this->d.a.start_idx == 0)) {
+ this->convert_to_tree();
+ }
+ if (this->is_array) {
+ if (idx == this->d.a.num_values) {
+ this->d.a.values[this->d.a.start_idx + this->d.a.num_values] = value;
+ } else {
+ this->d.a.values[--this->d.a.start_idx] = value;
+ }
+ this->d.a.num_values++;
+ } else {
+ subtree *rebalance_subtree = nullptr;
+ this->insert_internal(&this->d.t.root, value, idx, &rebalance_subtree);
+ if (rebalance_subtree != nullptr) {
+ this->rebalance(rebalance_subtree);
+ }
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+int omt<omtdata_t, omtdataout_t, supports_marks>::set_at(const omtdata_t &value,
+ const uint32_t idx) {
+ barf_if_marked(*this);
+ if (idx >= this->size()) {
+ return EINVAL;
+ }
+
+ if (this->is_array) {
+ this->set_at_internal_array(value, idx);
+ } else {
+ this->set_at_internal(this->d.t.root, value, idx);
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+int omt<omtdata_t, omtdataout_t, supports_marks>::delete_at(
+ const uint32_t idx) {
+ barf_if_marked(*this);
+ if (idx >= this->size()) {
+ return EINVAL;
+ }
+
+ this->maybe_resize_or_convert(this->size() - 1);
+ if (this->is_array && idx != 0 && idx != this->d.a.num_values - 1) {
+ this->convert_to_tree();
+ }
+ if (this->is_array) {
+ // Testing for 0 does not rule out it being the last entry.
+ // Test explicitly for num_values-1
+ if (idx != this->d.a.num_values - 1) {
+ this->d.a.start_idx++;
+ }
+ this->d.a.num_values--;
+ } else {
+ subtree *rebalance_subtree = nullptr;
+ this->delete_internal(&this->d.t.root, idx, nullptr, &rebalance_subtree);
+ if (rebalance_subtree != nullptr) {
+ this->rebalance(rebalance_subtree);
+ }
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::iterate(
+ iterate_extra_t *const iterate_extra) const {
+ return this->iterate_on_range<iterate_extra_t, f>(0, this->size(),
+ iterate_extra);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::iterate_on_range(
+ const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra) const {
+ if (right > this->size()) {
+ return EINVAL;
+ }
+ if (left == right) {
+ return 0;
+ }
+ if (this->is_array) {
+ return this->iterate_internal_array<iterate_extra_t, f>(left, right,
+ iterate_extra);
+ }
+ return this->iterate_internal<iterate_extra_t, f>(left, right, this->d.t.root,
+ 0, iterate_extra);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::iterate_and_mark_range(
+ const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra) {
+ static_assert(supports_marks, "does not support marks");
+ if (right > this->size()) {
+ return EINVAL;
+ }
+ if (left == right) {
+ return 0;
+ }
+ paranoid_invariant(!this->is_array);
+ return this->iterate_and_mark_range_internal<iterate_extra_t, f>(
+ left, right, this->d.t.root, 0, iterate_extra);
+}
+
+// TODO: We can optimize this if we steal 3 bits. 1 bit: this node is
+// marked. 1 bit: left subtree has marks. 1 bit: right subtree has marks.
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::iterate_over_marked(
+ iterate_extra_t *const iterate_extra) const {
+ static_assert(supports_marks, "does not support marks");
+ paranoid_invariant(!this->is_array);
+ return this->iterate_over_marked_internal<iterate_extra_t, f>(
+ this->d.t.root, 0, iterate_extra);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::unmark(
+ const subtree &st, const uint32_t index,
+ GrowableArray<node_idx> *const indexes) {
+ if (st.is_null()) {
+ return;
+ }
+ omt_node &n = this->d.t.nodes[st.get_index()];
+ const uint32_t index_root = index + this->nweight(n.left);
+
+ const bool below = n.get_marks_below();
+ if (below) {
+ this->unmark(n.left, index, indexes);
+ }
+ if (n.get_marked()) {
+ indexes->push(index_root);
+ }
+ n.clear_stolen_bits();
+ if (below) {
+ this->unmark(n.right, index_root + 1, indexes);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::delete_all_marked(void) {
+ static_assert(supports_marks, "does not support marks");
+ if (!this->has_marks()) {
+ return;
+ }
+ paranoid_invariant(!this->is_array);
+ GrowableArray<node_idx> marked_indexes;
+ marked_indexes.init();
+
+ // Remove all marks.
+ // We need to delete all the stolen bits before calling delete_at to
+ // prevent barfing.
+ this->unmark(this->d.t.root, 0, &marked_indexes);
+
+ for (uint32_t i = 0; i < marked_indexes.get_size(); i++) {
+ // Delete from left to right, shift by number already deleted.
+ // Alternative is delete from right to left.
+ int r = this->delete_at(marked_indexes.fetch_unchecked(i) - i);
+ lazy_assert_zero(r);
+ }
+ marked_indexes.deinit();
+ barf_if_marked(*this);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+uint32_t
+omt<omtdata_t, omtdataout_t, supports_marks>::verify_marks_consistent_internal(
+ const subtree &st, const bool UU(allow_marks)) const {
+ if (st.is_null()) {
+ return 0;
+ }
+ const omt_node &node = this->d.t.nodes[st.get_index()];
+ uint32_t num_marks =
+ verify_marks_consistent_internal(node.left, node.get_marks_below());
+ num_marks +=
+ verify_marks_consistent_internal(node.right, node.get_marks_below());
+ if (node.get_marks_below()) {
+ paranoid_invariant(allow_marks);
+ paranoid_invariant(num_marks > 0);
+ } else {
+ // redundant with invariant below, but nice to have explicitly
+ paranoid_invariant(num_marks == 0);
+ }
+ if (node.get_marked()) {
+ paranoid_invariant(allow_marks);
+ ++num_marks;
+ }
+ return num_marks;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::verify_marks_consistent(
+ void) const {
+ static_assert(supports_marks, "does not support marks");
+ paranoid_invariant(!this->is_array);
+ this->verify_marks_consistent_internal(this->d.t.root, true);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(omtdata_t *, const uint32_t, iterate_extra_t *const)>
+void omt<omtdata_t, omtdataout_t, supports_marks>::iterate_ptr(
+ iterate_extra_t *const iterate_extra) {
+ if (this->is_array) {
+ this->iterate_ptr_internal_array<iterate_extra_t, f>(0, this->size(),
+ iterate_extra);
+ } else {
+ this->iterate_ptr_internal<iterate_extra_t, f>(
+ 0, this->size(), this->d.t.root, 0, iterate_extra);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+int omt<omtdata_t, omtdataout_t, supports_marks>::fetch(
+ const uint32_t idx, omtdataout_t *const value) const {
+ if (idx >= this->size()) {
+ return EINVAL;
+ }
+ if (this->is_array) {
+ this->fetch_internal_array(idx, value);
+ } else {
+ this->fetch_internal(this->d.t.root, idx, value);
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find_zero(
+ const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ uint32_t tmp_index;
+ uint32_t *const child_idxp = (idxp != nullptr) ? idxp : &tmp_index;
+ int r;
+ if (this->is_array) {
+ r = this->find_internal_zero_array<omtcmp_t, h>(extra, value, child_idxp);
+ } else {
+ r = this->find_internal_zero<omtcmp_t, h>(this->d.t.root, extra, value,
+ child_idxp);
+ }
+ return r;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find(
+ const omtcmp_t &extra, int direction, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ uint32_t tmp_index;
+ uint32_t *const child_idxp = (idxp != nullptr) ? idxp : &tmp_index;
+ paranoid_invariant(direction != 0);
+ if (direction < 0) {
+ if (this->is_array) {
+ return this->find_internal_minus_array<omtcmp_t, h>(extra, value,
+ child_idxp);
+ } else {
+ return this->find_internal_minus<omtcmp_t, h>(this->d.t.root, extra,
+ value, child_idxp);
+ }
+ } else {
+ if (this->is_array) {
+ return this->find_internal_plus_array<omtcmp_t, h>(extra, value,
+ child_idxp);
+ } else {
+ return this->find_internal_plus<omtcmp_t, h>(this->d.t.root, extra, value,
+ child_idxp);
+ }
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+size_t omt<omtdata_t, omtdataout_t, supports_marks>::memory_size(void) {
+ if (this->is_array) {
+ return (sizeof *this) + this->capacity * (sizeof this->d.a.values[0]);
+ }
+ return (sizeof *this) + this->capacity * (sizeof this->d.t.nodes[0]);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::create_internal_no_array(
+ const uint32_t new_capacity) {
+ this->is_array = true;
+ this->d.a.start_idx = 0;
+ this->d.a.num_values = 0;
+ this->d.a.values = nullptr;
+ this->capacity = new_capacity;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::create_internal(
+ const uint32_t new_capacity) {
+ this->create_internal_no_array(new_capacity);
+ XMALLOC_N(this->capacity, this->d.a.values);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+uint32_t omt<omtdata_t, omtdataout_t, supports_marks>::nweight(
+ const subtree &st) const {
+ if (st.is_null()) {
+ return 0;
+ } else {
+ return this->d.t.nodes[st.get_index()].weight;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+typename omt<omtdata_t, omtdataout_t, supports_marks>::node_idx
+omt<omtdata_t, omtdataout_t, supports_marks>::node_malloc(void) {
+ paranoid_invariant(this->d.t.free_idx < this->capacity);
+ omt_node &n = this->d.t.nodes[this->d.t.free_idx];
+ n.clear_stolen_bits();
+ return this->d.t.free_idx++;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::node_free(
+ const node_idx UU(idx)) {
+ paranoid_invariant(idx < this->capacity);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::maybe_resize_array(
+ const uint32_t n) {
+ const uint32_t new_size = n <= 2 ? 4 : 2 * n;
+ const uint32_t room = this->capacity - this->d.a.start_idx;
+
+ if (room < n || this->capacity / 2 >= new_size) {
+ omtdata_t *XMALLOC_N(new_size, tmp_values);
+ if (this->d.a.num_values) {
+ memcpy(tmp_values, &this->d.a.values[this->d.a.start_idx],
+ this->d.a.num_values * (sizeof tmp_values[0]));
+ }
+ this->d.a.start_idx = 0;
+ this->capacity = new_size;
+ toku_free(this->d.a.values);
+ this->d.a.values = tmp_values;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t,
+ supports_marks>::fill_array_with_subtree_values(omtdata_t *const array,
+ const subtree &st)
+ const {
+ if (st.is_null()) return;
+ const omt_node &tree = this->d.t.nodes[st.get_index()];
+ this->fill_array_with_subtree_values(&array[0], tree.left);
+ array[this->nweight(tree.left)] = tree.value;
+ this->fill_array_with_subtree_values(&array[this->nweight(tree.left) + 1],
+ tree.right);
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::convert_to_array(void) {
+ if (!this->is_array) {
+ const uint32_t num_values = this->size();
+ uint32_t new_size = 2 * num_values;
+ new_size = new_size < 4 ? 4 : new_size;
+
+ omtdata_t *XMALLOC_N(new_size, tmp_values);
+ this->fill_array_with_subtree_values(tmp_values, this->d.t.root);
+ toku_free(this->d.t.nodes);
+ this->is_array = true;
+ this->capacity = new_size;
+ this->d.a.num_values = num_values;
+ this->d.a.values = tmp_values;
+ this->d.a.start_idx = 0;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::rebuild_from_sorted_array(
+ subtree *const st, const omtdata_t *const values,
+ const uint32_t numvalues) {
+ if (numvalues == 0) {
+ st->set_to_null();
+ } else {
+ const uint32_t halfway = numvalues / 2;
+ const node_idx newidx = this->node_malloc();
+ omt_node *const newnode = &this->d.t.nodes[newidx];
+ newnode->weight = numvalues;
+ newnode->value = values[halfway];
+ st->set_index(newidx);
+ // update everything before the recursive calls so the second call
+ // can be a tail call.
+ this->rebuild_from_sorted_array(&newnode->left, &values[0], halfway);
+ this->rebuild_from_sorted_array(&newnode->right, &values[halfway + 1],
+ numvalues - (halfway + 1));
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::convert_to_tree(void) {
+ if (this->is_array) {
+ const uint32_t num_nodes = this->size();
+ uint32_t new_size = num_nodes * 2;
+ new_size = new_size < 4 ? 4 : new_size;
+
+ omt_node *XMALLOC_N(new_size, new_nodes);
+ omtdata_t *const values = this->d.a.values;
+ omtdata_t *const tmp_values = &values[this->d.a.start_idx];
+ this->is_array = false;
+ this->d.t.nodes = new_nodes;
+ this->capacity = new_size;
+ this->d.t.free_idx = 0;
+ this->d.t.root.set_to_null();
+ this->rebuild_from_sorted_array(&this->d.t.root, tmp_values, num_nodes);
+ toku_free(values);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::maybe_resize_or_convert(
+ const uint32_t n) {
+ if (this->is_array) {
+ this->maybe_resize_array(n);
+ } else {
+ const uint32_t new_size = n <= 2 ? 4 : 2 * n;
+ const uint32_t num_nodes = this->nweight(this->d.t.root);
+ if ((this->capacity / 2 >= new_size) ||
+ (this->d.t.free_idx >= this->capacity && num_nodes < n) ||
+ (this->capacity < n)) {
+ this->convert_to_array();
+ // if we had a free list, the "supports_marks" version could
+ // just resize, as it is now, we have to convert to and back
+ // from an array.
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+ }
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+bool omt<omtdata_t, omtdataout_t, supports_marks>::will_need_rebalance(
+ const subtree &st, const int leftmod, const int rightmod) const {
+ if (st.is_null()) {
+ return false;
+ }
+ const omt_node &n = this->d.t.nodes[st.get_index()];
+ // one of the 1's is for the root.
+ // the other is to take ceil(n/2)
+ const uint32_t weight_left = this->nweight(n.left) + leftmod;
+ const uint32_t weight_right = this->nweight(n.right) + rightmod;
+ return ((1 + weight_left < (1 + 1 + weight_right) / 2) ||
+ (1 + weight_right < (1 + 1 + weight_left) / 2));
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::insert_internal(
+ subtree *const subtreep, const omtdata_t &value, const uint32_t idx,
+ subtree **const rebalance_subtree) {
+ if (subtreep->is_null()) {
+ paranoid_invariant_zero(idx);
+ const node_idx newidx = this->node_malloc();
+ omt_node *const newnode = &this->d.t.nodes[newidx];
+ newnode->weight = 1;
+ newnode->left.set_to_null();
+ newnode->right.set_to_null();
+ newnode->value = value;
+ subtreep->set_index(newidx);
+ } else {
+ omt_node &n = this->d.t.nodes[subtreep->get_index()];
+ n.weight++;
+ if (idx <= this->nweight(n.left)) {
+ if (*rebalance_subtree == nullptr &&
+ this->will_need_rebalance(*subtreep, 1, 0)) {
+ *rebalance_subtree = subtreep;
+ }
+ this->insert_internal(&n.left, value, idx, rebalance_subtree);
+ } else {
+ if (*rebalance_subtree == nullptr &&
+ this->will_need_rebalance(*subtreep, 0, 1)) {
+ *rebalance_subtree = subtreep;
+ }
+ const uint32_t sub_index = idx - this->nweight(n.left) - 1;
+ this->insert_internal(&n.right, value, sub_index, rebalance_subtree);
+ }
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::set_at_internal_array(
+ const omtdata_t &value, const uint32_t idx) {
+ this->d.a.values[this->d.a.start_idx + idx] = value;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::set_at_internal(
+ const subtree &st, const omtdata_t &value, const uint32_t idx) {
+ paranoid_invariant(!st.is_null());
+ omt_node &n = this->d.t.nodes[st.get_index()];
+ const uint32_t leftweight = this->nweight(n.left);
+ if (idx < leftweight) {
+ this->set_at_internal(n.left, value, idx);
+ } else if (idx == leftweight) {
+ n.value = value;
+ } else {
+ this->set_at_internal(n.right, value, idx - leftweight - 1);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::delete_internal(
+ subtree *const subtreep, const uint32_t idx, omt_node *const copyn,
+ subtree **const rebalance_subtree) {
+ paranoid_invariant_notnull(subtreep);
+ paranoid_invariant_notnull(rebalance_subtree);
+ paranoid_invariant(!subtreep->is_null());
+ omt_node &n = this->d.t.nodes[subtreep->get_index()];
+ const uint32_t leftweight = this->nweight(n.left);
+ if (idx < leftweight) {
+ n.weight--;
+ if (*rebalance_subtree == nullptr &&
+ this->will_need_rebalance(*subtreep, -1, 0)) {
+ *rebalance_subtree = subtreep;
+ }
+ this->delete_internal(&n.left, idx, copyn, rebalance_subtree);
+ } else if (idx == leftweight) {
+ if (n.left.is_null()) {
+ const uint32_t oldidx = subtreep->get_index();
+ *subtreep = n.right;
+ if (copyn != nullptr) {
+ copyn->value = n.value;
+ }
+ this->node_free(oldidx);
+ } else if (n.right.is_null()) {
+ const uint32_t oldidx = subtreep->get_index();
+ *subtreep = n.left;
+ if (copyn != nullptr) {
+ copyn->value = n.value;
+ }
+ this->node_free(oldidx);
+ } else {
+ if (*rebalance_subtree == nullptr &&
+ this->will_need_rebalance(*subtreep, 0, -1)) {
+ *rebalance_subtree = subtreep;
+ }
+ // don't need to copy up value, it's only used by this
+ // next call, and when that gets to the bottom there
+ // won't be any more recursion
+ n.weight--;
+ this->delete_internal(&n.right, 0, &n, rebalance_subtree);
+ }
+ } else {
+ n.weight--;
+ if (*rebalance_subtree == nullptr &&
+ this->will_need_rebalance(*subtreep, 0, -1)) {
+ *rebalance_subtree = subtreep;
+ }
+ this->delete_internal(&n.right, idx - leftweight - 1, copyn,
+ rebalance_subtree);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::iterate_internal_array(
+ const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra) const {
+ int r;
+ for (uint32_t i = left; i < right; ++i) {
+ r = f(this->d.a.values[this->d.a.start_idx + i], i, iterate_extra);
+ if (r != 0) {
+ return r;
+ }
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(omtdata_t *, const uint32_t, iterate_extra_t *const)>
+void omt<omtdata_t, omtdataout_t, supports_marks>::iterate_ptr_internal(
+ const uint32_t left, const uint32_t right, const subtree &st,
+ const uint32_t idx, iterate_extra_t *const iterate_extra) {
+ if (!st.is_null()) {
+ omt_node &n = this->d.t.nodes[st.get_index()];
+ const uint32_t idx_root = idx + this->nweight(n.left);
+ if (left < idx_root) {
+ this->iterate_ptr_internal<iterate_extra_t, f>(left, right, n.left, idx,
+ iterate_extra);
+ }
+ if (left <= idx_root && idx_root < right) {
+ int r = f(&n.value, idx_root, iterate_extra);
+ lazy_assert_zero(r);
+ }
+ if (idx_root + 1 < right) {
+ this->iterate_ptr_internal<iterate_extra_t, f>(
+ left, right, n.right, idx_root + 1, iterate_extra);
+ }
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(omtdata_t *, const uint32_t, iterate_extra_t *const)>
+void omt<omtdata_t, omtdataout_t, supports_marks>::iterate_ptr_internal_array(
+ const uint32_t left, const uint32_t right,
+ iterate_extra_t *const iterate_extra) {
+ for (uint32_t i = left; i < right; ++i) {
+ int r = f(&this->d.a.values[this->d.a.start_idx + i], i, iterate_extra);
+ lazy_assert_zero(r);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::iterate_internal(
+ const uint32_t left, const uint32_t right, const subtree &st,
+ const uint32_t idx, iterate_extra_t *const iterate_extra) const {
+ if (st.is_null()) {
+ return 0;
+ }
+ int r;
+ const omt_node &n = this->d.t.nodes[st.get_index()];
+ const uint32_t idx_root = idx + this->nweight(n.left);
+ if (left < idx_root) {
+ r = this->iterate_internal<iterate_extra_t, f>(left, right, n.left, idx,
+ iterate_extra);
+ if (r != 0) {
+ return r;
+ }
+ }
+ if (left <= idx_root && idx_root < right) {
+ r = f(n.value, idx_root, iterate_extra);
+ if (r != 0) {
+ return r;
+ }
+ }
+ if (idx_root + 1 < right) {
+ return this->iterate_internal<iterate_extra_t, f>(
+ left, right, n.right, idx_root + 1, iterate_extra);
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::
+ iterate_and_mark_range_internal(const uint32_t left, const uint32_t right,
+ const subtree &st, const uint32_t idx,
+ iterate_extra_t *const iterate_extra) {
+ paranoid_invariant(!st.is_null());
+ int r;
+ omt_node &n = this->d.t.nodes[st.get_index()];
+ const uint32_t idx_root = idx + this->nweight(n.left);
+ if (left < idx_root && !n.left.is_null()) {
+ n.set_marks_below_bit();
+ r = this->iterate_and_mark_range_internal<iterate_extra_t, f>(
+ left, right, n.left, idx, iterate_extra);
+ if (r != 0) {
+ return r;
+ }
+ }
+ if (left <= idx_root && idx_root < right) {
+ n.set_marked_bit();
+ r = f(n.value, idx_root, iterate_extra);
+ if (r != 0) {
+ return r;
+ }
+ }
+ if (idx_root + 1 < right && !n.right.is_null()) {
+ n.set_marks_below_bit();
+ return this->iterate_and_mark_range_internal<iterate_extra_t, f>(
+ left, right, n.right, idx_root + 1, iterate_extra);
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename iterate_extra_t,
+ int (*f)(const omtdata_t &, const uint32_t, iterate_extra_t *const)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::iterate_over_marked_internal(
+ const subtree &st, const uint32_t idx,
+ iterate_extra_t *const iterate_extra) const {
+ if (st.is_null()) {
+ return 0;
+ }
+ int r;
+ const omt_node &n = this->d.t.nodes[st.get_index()];
+ const uint32_t idx_root = idx + this->nweight(n.left);
+ if (n.get_marks_below()) {
+ r = this->iterate_over_marked_internal<iterate_extra_t, f>(n.left, idx,
+ iterate_extra);
+ if (r != 0) {
+ return r;
+ }
+ }
+ if (n.get_marked()) {
+ r = f(n.value, idx_root, iterate_extra);
+ if (r != 0) {
+ return r;
+ }
+ }
+ if (n.get_marks_below()) {
+ return this->iterate_over_marked_internal<iterate_extra_t, f>(
+ n.right, idx_root + 1, iterate_extra);
+ }
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::fetch_internal_array(
+ const uint32_t i, omtdataout_t *const value) const {
+ if (value != nullptr) {
+ copyout(value, &this->d.a.values[this->d.a.start_idx + i]);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::fetch_internal(
+ const subtree &st, const uint32_t i, omtdataout_t *const value) const {
+ omt_node &n = this->d.t.nodes[st.get_index()];
+ const uint32_t leftweight = this->nweight(n.left);
+ if (i < leftweight) {
+ this->fetch_internal(n.left, i, value);
+ } else if (i == leftweight) {
+ if (value != nullptr) {
+ copyout(value, &n);
+ }
+ } else {
+ this->fetch_internal(n.right, i - leftweight - 1, value);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::fill_array_with_subtree_idxs(
+ node_idx *const array, const subtree &st) const {
+ if (!st.is_null()) {
+ const omt_node &tree = this->d.t.nodes[st.get_index()];
+ this->fill_array_with_subtree_idxs(&array[0], tree.left);
+ array[this->nweight(tree.left)] = st.get_index();
+ this->fill_array_with_subtree_idxs(&array[this->nweight(tree.left) + 1],
+ tree.right);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::rebuild_subtree_from_idxs(
+ subtree *const st, const node_idx *const idxs, const uint32_t numvalues) {
+ if (numvalues == 0) {
+ st->set_to_null();
+ } else {
+ uint32_t halfway = numvalues / 2;
+ st->set_index(idxs[halfway]);
+ // node_idx newidx = idxs[halfway];
+ omt_node &newnode = this->d.t.nodes[st->get_index()];
+ newnode.weight = numvalues;
+ // value is already in there.
+ this->rebuild_subtree_from_idxs(&newnode.left, &idxs[0], halfway);
+ this->rebuild_subtree_from_idxs(&newnode.right, &idxs[halfway + 1],
+ numvalues - (halfway + 1));
+ // n_idx = newidx;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::rebalance(
+ subtree *const st) {
+ node_idx idx = st->get_index();
+ if (idx == this->d.t.root.get_index()) {
+ // Try to convert to an array.
+ // If this fails, (malloc) nothing will have changed.
+ // In the failure case we continue on to the standard rebalance
+ // algorithm.
+ this->convert_to_array();
+ if (supports_marks) {
+ this->convert_to_tree();
+ }
+ } else {
+ const omt_node &n = this->d.t.nodes[idx];
+ node_idx *tmp_array;
+ size_t mem_needed = n.weight * (sizeof tmp_array[0]);
+ size_t mem_free =
+ (this->capacity - this->d.t.free_idx) * (sizeof this->d.t.nodes[0]);
+ bool malloced;
+ if (mem_needed <= mem_free) {
+ // There is sufficient free space at the end of the nodes array
+ // to hold enough node indexes to rebalance.
+ malloced = false;
+ tmp_array =
+ reinterpret_cast<node_idx *>(&this->d.t.nodes[this->d.t.free_idx]);
+ } else {
+ malloced = true;
+ XMALLOC_N(n.weight, tmp_array);
+ }
+ this->fill_array_with_subtree_idxs(tmp_array, *st);
+ this->rebuild_subtree_from_idxs(st, tmp_array, n.weight);
+ if (malloced) toku_free(tmp_array);
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::copyout(
+ omtdata_t *const out, const omt_node *const n) {
+ *out = n->value;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::copyout(
+ omtdata_t **const out, omt_node *const n) {
+ *out = &n->value;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::copyout(
+ omtdata_t *const out, const omtdata_t *const stored_value_ptr) {
+ *out = *stored_value_ptr;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+void omt<omtdata_t, omtdataout_t, supports_marks>::copyout(
+ omtdata_t **const out, omtdata_t *const stored_value_ptr) {
+ *out = stored_value_ptr;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find_internal_zero_array(
+ const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ paranoid_invariant_notnull(idxp);
+ uint32_t min = this->d.a.start_idx;
+ uint32_t limit = this->d.a.start_idx + this->d.a.num_values;
+ uint32_t best_pos = subtree::NODE_NULL;
+ uint32_t best_zero = subtree::NODE_NULL;
+
+ while (min != limit) {
+ uint32_t mid = (min + limit) / 2;
+ int hv = h(this->d.a.values[mid], extra);
+ if (hv < 0) {
+ min = mid + 1;
+ } else if (hv > 0) {
+ best_pos = mid;
+ limit = mid;
+ } else {
+ best_zero = mid;
+ limit = mid;
+ }
+ }
+ if (best_zero != subtree::NODE_NULL) {
+ // Found a zero
+ if (value != nullptr) {
+ copyout(value, &this->d.a.values[best_zero]);
+ }
+ *idxp = best_zero - this->d.a.start_idx;
+ return 0;
+ }
+ if (best_pos != subtree::NODE_NULL)
+ *idxp = best_pos - this->d.a.start_idx;
+ else
+ *idxp = this->d.a.num_values;
+ return DB_NOTFOUND;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find_internal_zero(
+ const subtree &st, const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ paranoid_invariant_notnull(idxp);
+ if (st.is_null()) {
+ *idxp = 0;
+ return DB_NOTFOUND;
+ }
+ omt_node &n = this->d.t.nodes[st.get_index()];
+ int hv = h(n.value, extra);
+ if (hv < 0) {
+ int r = this->find_internal_zero<omtcmp_t, h>(n.right, extra, value, idxp);
+ *idxp += this->nweight(n.left) + 1;
+ return r;
+ } else if (hv > 0) {
+ return this->find_internal_zero<omtcmp_t, h>(n.left, extra, value, idxp);
+ } else {
+ int r = this->find_internal_zero<omtcmp_t, h>(n.left, extra, value, idxp);
+ if (r == DB_NOTFOUND) {
+ *idxp = this->nweight(n.left);
+ if (value != nullptr) {
+ copyout(value, &n);
+ }
+ r = 0;
+ }
+ return r;
+ }
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find_internal_plus_array(
+ const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ paranoid_invariant_notnull(idxp);
+ uint32_t min = this->d.a.start_idx;
+ uint32_t limit = this->d.a.start_idx + this->d.a.num_values;
+ uint32_t best = subtree::NODE_NULL;
+
+ while (min != limit) {
+ const uint32_t mid = (min + limit) / 2;
+ const int hv = h(this->d.a.values[mid], extra);
+ if (hv > 0) {
+ best = mid;
+ limit = mid;
+ } else {
+ min = mid + 1;
+ }
+ }
+ if (best == subtree::NODE_NULL) {
+ return DB_NOTFOUND;
+ }
+ if (value != nullptr) {
+ copyout(value, &this->d.a.values[best]);
+ }
+ *idxp = best - this->d.a.start_idx;
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find_internal_plus(
+ const subtree &st, const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ paranoid_invariant_notnull(idxp);
+ if (st.is_null()) {
+ return DB_NOTFOUND;
+ }
+ omt_node *const n = &this->d.t.nodes[st.get_index()];
+ int hv = h(n->value, extra);
+ int r;
+ if (hv > 0) {
+ r = this->find_internal_plus<omtcmp_t, h>(n->left, extra, value, idxp);
+ if (r == DB_NOTFOUND) {
+ *idxp = this->nweight(n->left);
+ if (value != nullptr) {
+ copyout(value, n);
+ }
+ r = 0;
+ }
+ } else {
+ r = this->find_internal_plus<omtcmp_t, h>(n->right, extra, value, idxp);
+ if (r == 0) {
+ *idxp += this->nweight(n->left) + 1;
+ }
+ }
+ return r;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find_internal_minus_array(
+ const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ paranoid_invariant_notnull(idxp);
+ uint32_t min = this->d.a.start_idx;
+ uint32_t limit = this->d.a.start_idx + this->d.a.num_values;
+ uint32_t best = subtree::NODE_NULL;
+
+ while (min != limit) {
+ const uint32_t mid = (min + limit) / 2;
+ const int hv = h(this->d.a.values[mid], extra);
+ if (hv < 0) {
+ best = mid;
+ min = mid + 1;
+ } else {
+ limit = mid;
+ }
+ }
+ if (best == subtree::NODE_NULL) {
+ return DB_NOTFOUND;
+ }
+ if (value != nullptr) {
+ copyout(value, &this->d.a.values[best]);
+ }
+ *idxp = best - this->d.a.start_idx;
+ return 0;
+}
+
+template <typename omtdata_t, typename omtdataout_t, bool supports_marks>
+template <typename omtcmp_t, int (*h)(const omtdata_t &, const omtcmp_t &)>
+int omt<omtdata_t, omtdataout_t, supports_marks>::find_internal_minus(
+ const subtree &st, const omtcmp_t &extra, omtdataout_t *const value,
+ uint32_t *const idxp) const {
+ paranoid_invariant_notnull(idxp);
+ if (st.is_null()) {
+ return DB_NOTFOUND;
+ }
+ omt_node *const n = &this->d.t.nodes[st.get_index()];
+ int hv = h(n->value, extra);
+ if (hv < 0) {
+ int r =
+ this->find_internal_minus<omtcmp_t, h>(n->right, extra, value, idxp);
+ if (r == 0) {
+ *idxp += this->nweight(n->left) + 1;
+ } else if (r == DB_NOTFOUND) {
+ *idxp = this->nweight(n->left);
+ if (value != nullptr) {
+ copyout(value, n);
+ }
+ r = 0;
+ }
+ return r;
+ } else {
+ return this->find_internal_minus<omtcmp_t, h>(n->left, extra, value, idxp);
+ }
+}
+} // namespace toku
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/partitioned_counter.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/partitioned_counter.h
new file mode 100644
index 000000000..f20eeedf2
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/partitioned_counter.h
@@ -0,0 +1,165 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+// Overview: A partitioned_counter provides a counter that can be incremented
+// and the running sum can be read at any time.
+// We assume that increments are frequent, whereas reading is infrequent.
+// Implementation hint: Use thread-local storage so each thread increments its
+// own data. The increment does not require a lock or atomic operation.
+// Reading the data can be performed by iterating over the thread-local
+// versions, summing them up. The data structure also includes a sum for all
+// the threads that have died. Use a pthread_key to create the thread-local
+// versions. When a thread finishes, the system calls pthread_key destructor
+// which can add that thread's copy into the sum_of_dead counter.
+// Rationale: For statistics such as are found in engine status, we need a
+// counter that requires no cache misses to increment. We've seen significant
+// performance speedups by removing certain counters. Rather than removing
+// those statistics, we would like to just make the counter fast. We generally
+// increment the counters frequently, and want to fetch the values
+// infrequently. The counters are monotonic. The counters can be split into
+// many counters, which can be summed up at the end. We don't care if we get
+// slightly out-of-date counter sums when we read the counter. We don't care
+// if there is a race on reading the a counter
+// variable and incrementing.
+// See tests/test_partitioned_counter.c for some performance measurements.
+// Operations:
+// create_partitioned_counter Create a counter initialized to zero.
+// destroy_partitioned_counter Destroy it.
+// increment_partitioned_counter Increment it. This is the frequent
+// operation. read_partitioned_counter Get the current value. This is
+// infrequent.
+// See partitioned_counter.cc for the abstraction function and representation
+// invariant.
+//
+// The google style guide says to avoid using constructors, and it appears that
+// constructors may have broken all the tests, because they called
+// pthread_key_create before the key was actually created. So the google style
+// guide may have some wisdom there...
+//
+// This version does not use constructors, essentially reverrting to the google
+// C++ style guide.
+//
+
+// The old C interface. This required a bunch of explicit
+// ___attribute__((__destructor__)) functions to remember to destroy counters at
+// the end.
+#if defined(__cplusplus)
+extern "C" {
+#endif
+
+typedef struct partitioned_counter *PARTITIONED_COUNTER;
+PARTITIONED_COUNTER create_partitioned_counter(void);
+// Effect: Create a counter, initialized to zero.
+
+void destroy_partitioned_counter(PARTITIONED_COUNTER);
+// Effect: Destroy the counter. No operations on that counter are permitted
+// after this.
+
+void increment_partitioned_counter(PARTITIONED_COUNTER, uint64_t amount);
+// Effect: Increment the counter by amount.
+// Requires: No overflows. This is a 64-bit unsigned counter.
+
+uint64_t read_partitioned_counter(PARTITIONED_COUNTER)
+ __attribute__((__visibility__("default")));
+// Effect: Return the current value of the counter.
+
+void partitioned_counters_init(void);
+// Effect: Initialize any partitioned counters data structures that must be set
+// up before any partitioned counters run.
+
+void partitioned_counters_destroy(void);
+// Effect: Destroy any partitioned counters data structures.
+
+#if defined(__cplusplus)
+};
+#endif
+
+#if 0
+#include <pthread.h>
+
+#include "fttypes.h"
+
+// Used inside the PARTITIONED_COUNTER.
+struct linked_list_head {
+ struct linked_list_element *first;
+};
+
+
+class PARTITIONED_COUNTER {
+public:
+ PARTITIONED_COUNTER(void);
+ // Effect: Construct a counter, initialized to zero.
+
+ ~PARTITIONED_COUNTER(void);
+ // Effect: Destruct the counter.
+
+ void increment(uint64_t amount);
+ // Effect: Increment the counter by amount. This is a 64-bit unsigned counter, and if you overflow it, you will get overflowed results (that is mod 2^64).
+ // Requires: Don't use this from a static constructor or destructor.
+
+ uint64_t read(void);
+ // Effect: Read the sum.
+ // Requires: Don't use this from a static constructor or destructor.
+
+private:
+ uint64_t _sum_of_dead; // The sum of all thread-local counts from threads that have terminated.
+ pthread_key_t _key; // The pthread_key which gives us the hook to construct and destruct thread-local storage.
+ struct linked_list_head _ll_counter_head; // A linked list of all the thread-local information for this counter.
+
+ // This function is used to destroy the thread-local part of the state when a thread terminates.
+ // But it's not the destructor for the local part of the counter, it's a destructor on a "dummy" key just so that we get a notification when a thread ends.
+ friend void destroy_thread_local_part_of_partitioned_counters (void *);
+};
+#endif
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/status.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/status.h
new file mode 100644
index 000000000..3fd0095d0
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/status.h
@@ -0,0 +1,76 @@
+/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
+// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
+#ident "$Id$"
+/*======
+This file is part of PerconaFT.
+
+
+Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved.
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU General Public License, version 2,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ PerconaFT is free software: you can redistribute it and/or modify
+ it under the terms of the GNU Affero General Public License, version 3,
+ as published by the Free Software Foundation.
+
+ PerconaFT is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU Affero General Public License for more details.
+
+ You should have received a copy of the GNU Affero General Public License
+ along with PerconaFT. If not, see <http://www.gnu.org/licenses/>.
+
+----------------------------------------
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+======= */
+
+#ident \
+ "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved."
+
+#pragma once
+
+#include "partitioned_counter.h"
+// PORT2: #include <util/constexpr.h>
+
+#define TOKUFT_STATUS_INIT(array, k, c, t, l, inc) \
+ do { \
+ array.status[k].keyname = #k; \
+ array.status[k].columnname = #c; \
+ array.status[k].type = t; \
+ array.status[k].legend = l; \
+ constexpr_static_assert( \
+ strcmp(#c, "NULL") && strcmp(#c, "0"), \
+ "Use nullptr for no column name instead of NULL, 0, etc..."); \
+ constexpr_static_assert( \
+ (inc) == TOKU_ENGINE_STATUS || strcmp(#c, "nullptr"), \
+ "Missing column name."); \
+ array.status[k].include = \
+ static_cast<toku_engine_status_include_type>(inc); \
+ if (t == STATUS_PARCOUNT) { \
+ array.status[k].value.parcount = create_partitioned_counter(); \
+ } \
+ } while (0)
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.cc
new file mode 100644
index 000000000..531165dea
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.cc
@@ -0,0 +1,503 @@
+// 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
+// the one used outside: either Endpoint or EndpointWithString
+template <typename EndpointStruct>
+void deserialize_endpoint(const DBT* dbt, EndpointStruct* 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 = decltype(EndpointStruct::slice)(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({txnid, 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*, toku::lock_wait_infos* infos) {
+ TEST_SYNC_POINT("RangeTreeLockManager::TryRangeLock:EnterWaitingTxn");
+ for (auto wait_info : *infos) {
+ // As long as we hold the lock on the locktree's pending request queue
+ // this should be safe.
+ 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(waitee);
+ }
+ 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<toku::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_);
+}
+
+int RangeTreeLockManager::on_create(toku::locktree* lt, void* arg) {
+ // arg is a pointer to RangeTreeLockManager
+ lt->set_escalation_barrier_func(&OnEscalationBarrierCheck, arg);
+ return 0;
+}
+
+bool RangeTreeLockManager::OnEscalationBarrierCheck(const DBT* a, const DBT* b,
+ void* extra) {
+ Endpoint a_endp, b_endp;
+ deserialize_endpoint(a, &a_endp);
+ deserialize_endpoint(b, &b_endp);
+ auto self = static_cast<RangeTreeLockManager*>(extra);
+ return self->barrier_func_(a_endp, b_endp);
+}
+
+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 toku::locktree* lt,
+ const toku::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(&ltm_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 = &ltm_status_test.status[i];
+ if (strcmp(status->keyname, "LTM_ESCALATION_COUNT") == 0) {
+ res.escalation_count = status->value.num;
+ continue;
+ }
+ if (strcmp(status->keyname, "LTM_WAIT_COUNT") == 0) {
+ res.lock_wait_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<toku::locktree> RangeTreeLockManager::MakeLockTreePtr(
+ toku::locktree* lt) {
+ toku::locktree_manager* ltm = &ltm_;
+ return std::shared_ptr<toku::locktree>(
+ lt, [ltm](toku::locktree* p) { ltm->release_lt(p); });
+}
+
+void RangeTreeLockManager::AddColumnFamily(const ColumnFamilyHandle* cfh) {
+ uint32_t column_family_id = cfh->GetID();
+
+ InstrumentedMutexLock l(&ltree_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*/ static_cast<void*>(this));
+ // 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(&ltree_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<toku::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(&ltree_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) {
+ info.ids.push_back(txnid_arg);
+ } else {
+ for (auto it : *owners) {
+ info.ids.push_back(it);
+ }
+ }
+ ctx->data->insert({ctx->cfh_id, info});
+}
+
+LockManager::RangeLockStatus RangeTreeLockManager::GetRangeLockStatus() {
+ LockManager::RangeLockStatus data;
+ {
+ InstrumentedMutexLock l(&ltree_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
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.h
new file mode 100644
index 000000000..e4236d600
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.h
@@ -0,0 +1,137 @@
+// 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 {
+
+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<toku::locktree> GetLockTreeForCF(ColumnFamilyId cf_id);
+
+ void SetEscalationBarrierFunc(EscalationBarrierFunc func) override {
+ barrier_func_ = func;
+ }
+
+ private:
+ toku::locktree_manager ltm_;
+
+ EscalationBarrierFunc barrier_func_ =
+ [](const Endpoint&, const Endpoint&) -> bool { return false; };
+
+ 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<toku::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<toku::locktree> MakeLockTreePtr(toku::locktree* lt);
+ static int CompareDbtEndpoints(void* arg, const DBT* a_key, const DBT* b_key);
+
+ // Callbacks
+ static int on_create(toku::locktree*, void*);
+ static void on_destroy(toku::locktree*) {}
+ static void on_escalate(TXNID txnid, const toku::locktree* lt,
+ const toku::range_buffer& buffer, void* extra);
+
+ static bool OnEscalationBarrierCheck(const DBT* a, const DBT* b, void* extra);
+};
+
+void serialize_endpoint(const Endpoint& endp, std::string* buf);
+void wait_callback_for_locktree(void* cdata, toku::lock_wait_infos* infos);
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // OS_WIN
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.cc b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.cc
new file mode 100644
index 000000000..be1e1478b
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.cc
@@ -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
diff --git a/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.h b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.h
new file mode 100644
index 000000000..4ef48d252
--- /dev/null
+++ b/src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.h
@@ -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