From e6918187568dbd01842d8d1d2c808ce16a894239 Mon Sep 17 00:00:00 2001 From: Daniel Baumann Date: Sun, 21 Apr 2024 13:54:28 +0200 Subject: Adding upstream version 18.2.2. Signed-off-by: Daniel Baumann --- .../utilities/transactions/lock/lock_manager.cc | 29 + .../utilities/transactions/lock/lock_manager.h | 82 ++ .../utilities/transactions/lock/lock_tracker.h | 209 ++++ .../transactions/lock/point/point_lock_manager.cc | 721 +++++++++++ .../transactions/lock/point/point_lock_manager.h | 224 ++++ .../lock/point/point_lock_manager_test.cc | 181 +++ .../lock/point/point_lock_manager_test.h | 324 +++++ .../transactions/lock/point/point_lock_tracker.cc | 257 ++++ .../transactions/lock/point/point_lock_tracker.h | 99 ++ .../transactions/lock/range/range_lock_manager.h | 36 + .../transactions/lock/range/range_locking_test.cc | 459 +++++++ .../lock/range/range_tree/lib/COPYING.AGPLv3 | 661 ++++++++++ .../lock/range/range_tree/lib/COPYING.APACHEv2 | 174 +++ .../lock/range/range_tree/lib/COPYING.GPLv2 | 339 +++++ .../transactions/lock/range/range_tree/lib/README | 13 + .../transactions/lock/range/range_tree/lib/db.h | 76 ++ .../lock/range/range_tree/lib/ft/comparator.h | 138 +++ .../lock/range/range_tree/lib/ft/ft-status.h | 102 ++ .../range_tree/lib/locktree/concurrent_tree.cc | 139 +++ .../range_tree/lib/locktree/concurrent_tree.h | 174 +++ .../lock/range/range_tree/lib/locktree/keyrange.cc | 222 ++++ .../lock/range/range_tree/lib/locktree/keyrange.h | 141 +++ .../range/range_tree/lib/locktree/lock_request.cc | 527 ++++++++ .../range/range_tree/lib/locktree/lock_request.h | 255 ++++ .../lock/range/range_tree/lib/locktree/locktree.cc | 1023 ++++++++++++++++ .../lock/range/range_tree/lib/locktree/locktree.h | 580 +++++++++ .../lock/range/range_tree/lib/locktree/manager.cc | 527 ++++++++ .../range/range_tree/lib/locktree/range_buffer.cc | 265 ++++ .../range/range_tree/lib/locktree/range_buffer.h | 178 +++ .../lock/range/range_tree/lib/locktree/treenode.cc | 520 ++++++++ .../lock/range/range_tree/lib/locktree/treenode.h | 302 +++++ .../range/range_tree/lib/locktree/txnid_set.cc | 120 ++ .../lock/range/range_tree/lib/locktree/txnid_set.h | 92 ++ .../lock/range/range_tree/lib/locktree/wfg.cc | 213 ++++ .../lock/range/range_tree/lib/locktree/wfg.h | 124 ++ .../lock/range/range_tree/lib/portability/memory.h | 215 ++++ .../range_tree/lib/portability/toku_assert_subst.h | 39 + .../range/range_tree/lib/portability/toku_atomic.h | 130 ++ .../lib/portability/toku_external_pthread.h | 83 ++ .../lib/portability/toku_instrumentation.h | 286 +++++ .../range_tree/lib/portability/toku_portability.h | 87 ++ .../range_tree/lib/portability/toku_pthread.h | 520 ++++++++ .../range_tree/lib/portability/toku_race_tools.h | 179 +++ .../range/range_tree/lib/portability/toku_time.h | 193 +++ .../range/range_tree/lib/portability/txn_subst.h | 27 + .../lock/range/range_tree/lib/standalone_port.cc | 132 ++ .../lock/range/range_tree/lib/util/dbt.cc | 153 +++ .../lock/range/range_tree/lib/util/dbt.h | 98 ++ .../range/range_tree/lib/util/growable_array.h | 144 +++ .../lock/range/range_tree/lib/util/memarena.cc | 201 +++ .../lock/range/range_tree/lib/util/memarena.h | 141 +++ .../lock/range/range_tree/lib/util/omt.h | 794 ++++++++++++ .../lock/range/range_tree/lib/util/omt_impl.h | 1295 ++++++++++++++++++++ .../range_tree/lib/util/partitioned_counter.h | 165 +++ .../lock/range/range_tree/lib/util/status.h | 76 ++ .../range/range_tree/range_tree_lock_manager.cc | 503 ++++++++ .../range/range_tree/range_tree_lock_manager.h | 137 +++ .../range/range_tree/range_tree_lock_tracker.cc | 156 +++ .../range/range_tree/range_tree_lock_tracker.h | 146 +++ 59 files changed, 15426 insertions(+) create mode 100644 src/rocksdb/utilities/transactions/lock/lock_manager.cc create mode 100644 src/rocksdb/utilities/transactions/lock/lock_manager.h create mode 100644 src/rocksdb/utilities/transactions/lock/lock_tracker.h create mode 100644 src/rocksdb/utilities/transactions/lock/point/point_lock_manager.cc create mode 100644 src/rocksdb/utilities/transactions/lock/point/point_lock_manager.h create mode 100644 src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.cc create mode 100644 src/rocksdb/utilities/transactions/lock/point/point_lock_manager_test.h create mode 100644 src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.cc create mode 100644 src/rocksdb/utilities/transactions/lock/point/point_lock_tracker.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_lock_manager.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_locking_test.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.AGPLv3 create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.APACHEv2 create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/COPYING.GPLv2 create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/README create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/db.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/comparator.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/ft/ft-status.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/manager.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/memory.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_assert_subst.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_atomic.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_external_pthread.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_instrumentation.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_portability.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_pthread.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_race_tools.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/portability/txn_subst.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/dbt.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/memarena.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/omt_impl.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/partitioned_counter.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/lib/util/status.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.h create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.cc create mode 100644 src/rocksdb/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.h (limited to 'src/rocksdb/utilities/transactions/lock') 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 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(opt.lock_mgr_handle, mgr); + } else { + // Use a point lock manager by default + return std::shared_ptr(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; + virtual PointLockStatus GetPointLockStatus() = 0; + + using RangeLockStatus = + std::unordered_multimap; + virtual RangeLockStatus GetRangeLockStatus() = 0; + + virtual std::vector 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 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 + +#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 +#include +#include + +#include "monitoring/perf_context_imp.h" +#include "rocksdb/slice.h" +#include "rocksdb/utilities/transaction_db_mutex.h" +#include "test_util/sync_point.h" +#include "util/cast_util.h" +#include "util/hash.h" +#include "util/thread_local.h" +#include "utilities/transactions/pessimistic_transaction_db.h" +#include "utilities/transactions/transaction_db_mutex_impl.h" + +namespace ROCKSDB_NAMESPACE { + +struct LockInfo { + bool exclusive; + autovector txn_ids; + + // Transaction locks are not valid after this time in us + uint64_t expiration_time; + + LockInfo(TransactionID id, uint64_t time, bool ex) + : exclusive(ex), expiration_time(time) { + txn_ids.push_back(id); + } + LockInfo(const LockInfo& lock_info) + : exclusive(lock_info.exclusive), + txn_ids(lock_info.txn_ids), + expiration_time(lock_info.expiration_time) {} + 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 factory) { + stripe_mutex = factory->AllocateMutex(); + stripe_cv = factory->AllocateCondVar(); + assert(stripe_mutex); + assert(stripe_cv); + } + + // Mutex must be held before modifying keys map + std::shared_ptr stripe_mutex; + + // Condition Variable per stripe for waiting on a lock + std::shared_ptr stripe_cv; + + // Locked keys mapped to the info about the transactions that locked them. + // TODO(agiardullo): Explore performance of other data structures. + UnorderedMap keys; +}; + +// Map of #num_stripes LockMapStripes +struct LockMap { + explicit LockMap(size_t num_stripes, + std::shared_ptr factory) + : num_stripes_(num_stripes) { + lock_map_stripes_.reserve(num_stripes); + for (size_t i = 0; i < num_stripes; i++) { + LockMapStripe* stripe = new LockMapStripe(factory); + lock_map_stripes_.push_back(stripe); + } + } + + ~LockMap() { + for (auto stripe : lock_map_stripes_) { + delete stripe; + } + } + + // Number of sepearate LockMapStripes to create, each with their own Mutex + const size_t num_stripes_; + + // Count of keys that are currently locked in this column family. + // (Only maintained if PointLockManager::max_num_locks_ is positive.) + std::atomic lock_cnt{0}; + + std::vector 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>*>(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()) {} + +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( + 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 local_caches; + lock_maps_cache_->Scrape(&local_caches, nullptr); + for (auto cache : local_caches) { + delete static_cast(cache); + } +} + +// Look up the LockMap std::shared_ptr for a given column_family_id. +// Note: The LockMap is only valid as long as the caller is still holding on +// to the returned std::shared_ptr. +std::shared_ptr 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(lock_maps_cache_->Get()); + + auto lock_map_iter = lock_maps_cache->find(column_family_id); + if (lock_map_iter != lock_maps_cache->end()) { + // Found lock map for this column family. + return lock_map_iter->second; + } + + // Not found in local cache, grab mutex and check shared LockMaps + InstrumentedMutexLock l(&lock_map_mutex_); + + lock_map_iter = lock_maps_.find(column_family_id); + if (lock_map_iter == lock_maps_.end()) { + return std::shared_ptr(nullptr); + } else { + // Found lock map. Store in thread-local cache and return. + std::shared_ptr& lock_map = lock_map_iter->second; + lock_maps_cache->insert({column_family_id, lock_map}); + + return lock_map; + } +} + +// Returns true if this lock has expired and can be acquired by another +// transaction. +// If false, sets *expire_time to the expiration time of the lock according +// to Env->GetMicros() or 0 if no expiration. +bool 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 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 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(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& wait_ids) { + std::lock_guard lock(wait_txn_map_mutex_); + DecrementWaitersImpl(txn, wait_ids); +} + +void PointLockManager::DecrementWaitersImpl( + const PessimisticTransaction* txn, + const autovector& wait_ids) { + auto id = txn->GetID(); + assert(wait_txn_map_.Contains(id)); + wait_txn_map_.Delete(id); + + for (auto wait_id : wait_ids) { + rev_wait_txn_map_.Get(wait_id)--; + if (rev_wait_txn_map_.Get(wait_id) == 0) { + rev_wait_txn_map_.Delete(wait_id); + } + } +} + +bool PointLockManager::IncrementWaiters( + const PessimisticTransaction* txn, + const autovector& wait_ids, const std::string& key, + const uint32_t& cf_id, const bool& exclusive, Env* const env) { + auto id = txn->GetID(); + std::vector queue_parents( + static_cast(txn->GetDeadlockDetectDepth())); + std::vector queue_values( + static_cast(txn->GetDeadlockDetectDepth())); + std::lock_guard lock(wait_txn_map_mutex_); + assert(!wait_txn_map_.Contains(id)); + + wait_txn_map_.Insert(id, {wait_ids, cf_id, exclusive, key}); + + for (auto wait_id : wait_ids) { + if (rev_wait_txn_map_.Contains(wait_id)) { + rev_wait_txn_map_.Get(wait_id)++; + } else { + rev_wait_txn_map_.Insert(wait_id, 1); + } + } + + // No deadlock if nobody is waiting on self. + if (!rev_wait_txn_map_.Contains(id)) { + return false; + } + + const auto* next_ids = &wait_ids; + int parent = -1; + int64_t deadlock_time = 0; + for (int tail = 0, head = 0; head < txn->GetDeadlockDetectDepth(); head++) { + int i = 0; + if (next_ids) { + for (; i < static_cast(next_ids->size()) && + tail + i < txn->GetDeadlockDetectDepth(); + i++) { + queue_values[tail + i] = (*next_ids)[i]; + queue_parents[tail + i] = parent; + } + tail += i; + } + + // No more items in the list, meaning no deadlock. + if (tail == head) { + return false; + } + + auto next = queue_values[head]; + if (next == id) { + std::vector path; + while (head != -1) { + assert(wait_txn_map_.Contains(queue_values[head])); + + auto extracted_info = wait_txn_map_.Get(queue_values[head]); + path.push_back({queue_values[head], extracted_info.m_cf_id, + extracted_info.m_exclusive, + extracted_info.m_waiting_key}); + head = queue_parents[head]; + } + 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* 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 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 cf_it( + tracker.GetColumnFamilyIterator()); + assert(cf_it != nullptr); + while (cf_it->HasNext()) { + ColumnFamilyId cf = cf_it->Next(); + std::shared_ptr 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> keys_by_stripe( + lock_map->num_stripes_); + std::unique_ptr 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 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 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 +#include +#include +#include +#include + +#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 DeadlockInfoBufferTempl { + private: + std::vector paths_buffer_; + uint32_t buffer_idx_; + std::mutex paths_buffer_mutex_; + + std::vector 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 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 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 PrepareBuffer() { + std::lock_guard lock(paths_buffer_mutex_); + + // Reversing the normalized vector returns the latest deadlocks first + auto working = Normalize(); + std::reverse(working.begin(), working.end()); + + return working; + } +}; + +using DeadlockInfoBuffer = DeadlockInfoBufferTempl; + +struct TrackedTrxInfo { + autovector 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 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>; + LockMaps lock_maps_; + + // Thread-local cache of entries in lock_maps_. This is an optimization + // to avoid acquiring a mutex in order to look up a LockMap + std::unique_ptr lock_maps_cache_; + + // Must be held when modifying wait_txn_map_ and rev_wait_txn_map_. + std::mutex wait_txn_map_mutex_; + + // Maps from waitee -> number of waiters. + HashMap rev_wait_txn_map_; + // Maps from waiter -> waitee. + HashMap wait_txn_map_; + DeadlockInfoBuffer dlock_buffer_; + + // Used to allocate mutexes/condvars to use when locking keys + std::shared_ptr mutex_factory_; + + bool IsLockExpired(TransactionID txn_id, const LockInfo& lock_info, Env* env, + uint64_t* wait_time); + + std::shared_ptr GetLockMap(uint32_t column_family_id); + + Status AcquireWithTimeout(PessimisticTransaction* txn, LockMap* lock_map, + LockMapStripe* stripe, uint32_t column_family_id, + const std::string& key, Env* env, int64_t timeout, + 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* txn_ids); + + void UnLockKey(PessimisticTransaction* txn, const std::string& key, + LockMapStripe* stripe, LockMap* lock_map, Env* env); + + bool IncrementWaiters(const PessimisticTransaction* txn, + const autovector& wait_ids, + const std::string& key, const uint32_t& cf_id, + const bool& exclusive, Env* const env); + void DecrementWaiters(const PessimisticTransaction* txn, + const autovector& wait_ids); + void DecrementWaitersImpl(const PessimisticTransaction* txn, + const autovector& wait_ids); +}; + +} // namespace ROCKSDB_NAMESPACE +#endif // ROCKSDB_LITE diff --git a/src/rocksdb/utilities/transactions/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 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 + +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(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(txn); + } + + protected: + Env* env_; + std::shared_ptr 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 { + 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 f) { + std::atomic 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 deadlock_paths = locker_->GetDeadlockInfoBuffer(); + ASSERT_EQ(deadlock_paths.size(), 1u); + ASSERT_FALSE(deadlock_paths[0].limit_exceeded); + + std::vector 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(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(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(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 +#include +#include + +#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; + +using TrackedKeys = std::unordered_map; + +// 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 +#include +#include +#include + +#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 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(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 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(); + self->locker_.reset(NewRangeLockManager(mutex_factory)->getLockManager()); + std::shared_ptr range_lock_mgr = + std::dynamic_pointer_cast(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 +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 + +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. + 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. + + + Copyright (C) + + 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 . + +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 +. 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. + + + Copyright (C) + + 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. + + , 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 +#include + +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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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(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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 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, &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( + 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( + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 waitees; +}; + +typedef std::vector 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 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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_locks) { + struct copy_fn_obj { + GrowableArray *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(©_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_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(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_locks) { + struct copy_fn_obj { + GrowableArray *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(©_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(prepared_lkr); + lkr->acquire(requested_range); + + // copy out the set of overlapping row locks. + GrowableArray 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 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 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 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 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( + 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, ¤t_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(¤t_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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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 // 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 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 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 <, 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include + +#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 <, + 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(dict_id, <, + nullptr); + return r == 0 ? lt : nullptr; +} + +void locktree_manager::locktree_map_put(locktree *lt) { + int r = m_locktree_map.insert( + 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( + 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, <); + 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, <); + invariant_zero(r); + if (r == EINVAL) // Shouldn't happen, avoid compiler warning + continue; + if (toku_external_mutex_trylock(<->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(<->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, <); + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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(_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(_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(_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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include + +#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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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 + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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_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, 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, 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include +#include + +#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 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 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, &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, &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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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 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 m_nodes; + + node *find_node(TXNID txnid); + + node *find_create_node(TXNID txnid); + + bool cycle_exists_from_node(node *target, node *head, + std::function 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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 +#include + +#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::value && std::is_trivial::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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include +#include +#include + +#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 +__attribute__((const, always_inline)) static inline bool crosses_boundary( + T *addr, size_t width) { + const intptr_t int_addr = reinterpret_cast(addr); + const intptr_t last_byte = int_addr + width - 1; + return which_cache_line(int_addr) != which_cache_line(last_byte); +} + +template +__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 +__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 +__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 +__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 +__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 +__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 +#include +#include + +#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 + toku_external_mutex_factory_t; + +typedef std::shared_ptr toku_external_mutex_t; +typedef std::shared_ptr 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 // 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 + +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 +#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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include +#include +#include +#include +#include +#include + +#include "toku_atomic.h" + +#if defined(__cplusplus) +#include +#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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include +#include + +#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 +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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#ifdef HAVE_valgrind +#undef USE_VALGRIND +#define USE_VALGRIND 1 +#endif + +#if defined(__linux__) && USE_VALGRIND + +#include +#include + +#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 +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 +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 +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 +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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include +#include +#if defined(__powerpc__) +#include +#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(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 + +#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 { + 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 + +#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 . + +---------------------------------------- + + 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 . +======= */ + +#ident \ + "Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved." + +#include "dbt.h" + +#include + +#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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +//****************************************************************************** +// +// 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 +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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#include + +#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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +/* + * 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 +#include + +#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 $ji$. + * + * 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_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 foo_omt_t; + */ + +namespace omt_internal { + +template +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 { + 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 +class omt_node_templated { + public: + omtdata_t value; + uint32_t weight; + subtree_templated left; + subtree_templated 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 +class omt_node_templated { + public: + omtdata_t value; + uint32_t weight; + subtree_templated left; + subtree_templated 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 +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 + 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 + 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 + 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 + 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 + 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 + 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, 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, 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 + 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 + 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 subtree; + typedef omt_internal::omt_node_templated 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 *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 + int iterate_internal_array(const uint32_t left, const uint32_t right, + iterate_extra_t *const iterate_extra) const; + + template + 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 + void iterate_ptr_internal_array(const uint32_t left, const uint32_t right, + iterate_extra_t *const iterate_extra); + + template + 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 + 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 + 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 + int find_internal_zero_array(const omtcmp_t &extra, omtdataout_t *const value, + uint32_t *const idxp) const; + + template + int find_internal_zero(const subtree &subtree, const omtcmp_t &extra, + omtdataout_t *const value, uint32_t *const idxp) const; + + template + int find_internal_plus_array(const omtcmp_t &extra, omtdataout_t *const value, + uint32_t *const idxp) const; + + template + int find_internal_plus(const subtree &subtree, const omtcmp_t &extra, + omtdataout_t *const value, uint32_t *const idxp) const; + + template + int find_internal_minus_array(const omtcmp_t &extra, + omtdataout_t *const value, + uint32_t *const idxp) const; + + template + 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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#include "../db.h" +#include "../portability/memory.h" + +namespace toku { + +template +void omt::create(void) { + this->create_internal(2); + if (supports_marks) { + this->convert_to_tree(); + } +} + +template +void omt::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 +void omt::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 +void omt::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 +int omt::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 +void omt::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 +void omt::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 +void omt::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 +void omt::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 +uint32_t omt::size(void) const { + if (this->is_array) { + return this->d.a.num_values; + } else { + return this->nweight(this->d.t.root); + } +} + +template +template +int omt::insert(const omtdata_t &value, + const omtcmp_t &v, + uint32_t *const idx) { + int r; + uint32_t insert_idx; + + r = this->find_zero(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 +static void barf_if_marked(const omt &UU(omt)) { +} + +template +static void barf_if_marked(const omt &omt) { + invariant(!omt.has_marks()); +} + +template +bool omt::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 +int omt::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 +int omt::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 +int omt::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 +template +int omt::iterate( + iterate_extra_t *const iterate_extra) const { + return this->iterate_on_range(0, this->size(), + iterate_extra); +} + +template +template +int omt::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(left, right, + iterate_extra); + } + return this->iterate_internal(left, right, this->d.t.root, + 0, iterate_extra); +} + +template +template +int omt::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( + 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 +template +int omt::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( + this->d.t.root, 0, iterate_extra); +} + +template +void omt::unmark( + const subtree &st, const uint32_t index, + GrowableArray *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 +void omt::delete_all_marked(void) { + static_assert(supports_marks, "does not support marks"); + if (!this->has_marks()) { + return; + } + paranoid_invariant(!this->is_array); + GrowableArray 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 +uint32_t +omt::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 +void omt::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 +template +void omt::iterate_ptr( + iterate_extra_t *const iterate_extra) { + if (this->is_array) { + this->iterate_ptr_internal_array(0, this->size(), + iterate_extra); + } else { + this->iterate_ptr_internal( + 0, this->size(), this->d.t.root, 0, iterate_extra); + } +} + +template +int omt::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 +template +int omt::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(extra, value, child_idxp); + } else { + r = this->find_internal_zero(this->d.t.root, extra, value, + child_idxp); + } + return r; +} + +template +template +int omt::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(extra, value, + child_idxp); + } else { + return this->find_internal_minus(this->d.t.root, extra, + value, child_idxp); + } + } else { + if (this->is_array) { + return this->find_internal_plus_array(extra, value, + child_idxp); + } else { + return this->find_internal_plus(this->d.t.root, extra, value, + child_idxp); + } + } +} + +template +size_t omt::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 +void omt::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 +void omt::create_internal( + const uint32_t new_capacity) { + this->create_internal_no_array(new_capacity); + XMALLOC_N(this->capacity, this->d.a.values); +} + +template +uint32_t omt::nweight( + const subtree &st) const { + if (st.is_null()) { + return 0; + } else { + return this->d.t.nodes[st.get_index()].weight; + } +} + +template +typename omt::node_idx +omt::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 +void omt::node_free( + const node_idx UU(idx)) { + paranoid_invariant(idx < this->capacity); +} + +template +void omt::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 +void omt::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 +void omt::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 +void omt::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 +void omt::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 +void omt::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 +bool omt::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 +void omt::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 +void omt::set_at_internal_array( + const omtdata_t &value, const uint32_t idx) { + this->d.a.values[this->d.a.start_idx + idx] = value; +} + +template +void omt::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 +void omt::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 +template +int omt::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 +template +void omt::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(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( + left, right, n.right, idx_root + 1, iterate_extra); + } + } +} + +template +template +void omt::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 +template +int omt::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(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( + left, right, n.right, idx_root + 1, iterate_extra); + } + return 0; +} + +template +template +int omt:: + 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( + 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( + left, right, n.right, idx_root + 1, iterate_extra); + } + return 0; +} + +template +template +int omt::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(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( + n.right, idx_root + 1, iterate_extra); + } + return 0; +} + +template +void omt::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 +void omt::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 +void omt::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 +void omt::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 +void omt::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(&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 +void omt::copyout( + omtdata_t *const out, const omt_node *const n) { + *out = n->value; +} + +template +void omt::copyout( + omtdata_t **const out, omt_node *const n) { + *out = &n->value; +} + +template +void omt::copyout( + omtdata_t *const out, const omtdata_t *const stored_value_ptr) { + *out = *stored_value_ptr; +} + +template +void omt::copyout( + omtdata_t **const out, omtdata_t *const stored_value_ptr) { + *out = stored_value_ptr; +} + +template +template +int omt::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 +template +int omt::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(n.right, extra, value, idxp); + *idxp += this->nweight(n.left) + 1; + return r; + } else if (hv > 0) { + return this->find_internal_zero(n.left, extra, value, idxp); + } else { + int r = this->find_internal_zero(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 +template +int omt::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 +template +int omt::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(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(n->right, extra, value, idxp); + if (r == 0) { + *idxp += this->nweight(n->left) + 1; + } + } + return r; +} + +template +template +int omt::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 +template +int omt::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(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(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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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 . + +---------------------------------------- + + 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 . + +---------------------------------------- + + 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 + +#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(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 +#include +#include + +#include "monitoring/perf_context_imp.h" +#include "rocksdb/slice.h" +#include "rocksdb/utilities/transaction_db_mutex.h" +#include "test_util/sync_point.h" +#include "util/cast_util.h" +#include "util/hash.h" +#include "util/thread_local.h" +#include "utilities/transactions/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 mutex_factory) { + std::shared_ptr 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 +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(-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 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 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(&tracker); + + RangeTreeLockTracker* range_trx_tracker = + static_cast(&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>*>( + ptr); + delete lock_tree_map_cache; +} +} // anonymous namespace + +RangeTreeLockManager::RangeTreeLockManager( + std::shared_ptr 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(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 +RangeTreeLockManager::GetRangeDeadlockInfoBuffer() { + return dlock_buffer_.PrepareBuffer(); +} + +std::vector RangeTreeLockManager::GetDeadlockInfoBuffer() { + std::vector res; + std::vector data = GetRangeDeadlockInfoBuffer(); + // report left endpoints + for (auto it = data.begin(); it != data.end(); ++it) { + std::vector 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 local_caches; + ltree_lookup_cache_->Scrape(&local_caches, nullptr); + for (auto cache : local_caches) { + delete static_cast(cache); + } + ltree_map_.clear(); // this will call release_lt() for all locktrees + ltm_.destroy(); +} + +RangeLockManagerHandle::Counters RangeTreeLockManager::GetStatus() { + LTM_STATUS_S ltm_status_test; + ltm_.get_status(<m_status_test); + Counters res; + + // Searching status variable by its string name is how Toku's unit tests + // do it (why didn't they make LTM_ESCALATION_COUNT constant visible?) + // lookup keyname in status + for (int i = 0; i < LTM_STATUS_S::LTM_STATUS_NUM_ROWS; i++) { + TOKU_ENGINE_STATUS_ROW status = <m_status_test.status[i]; + if (strcmp(status->keyname, "LTM_ESCALATION_COUNT") == 0) { + res.escalation_count = status->value.num; + continue; + } + if (strcmp(status->keyname, "LTM_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 RangeTreeLockManager::MakeLockTreePtr( + toku::locktree* lt) { + toku::locktree_manager* ltm = <m_; + return std::shared_ptr( + lt, [ltm](toku::locktree* p) { ltm->release_lt(p); }); +} + +void RangeTreeLockManager::AddColumnFamily(const ColumnFamilyHandle* cfh) { + uint32_t column_family_id = cfh->GetID(); + + InstrumentedMutexLock l(<ree_map_mutex_); + if (ltree_map_.find(column_family_id) == ltree_map_.end()) { + DICTIONARY_ID dict_id = {.dictid = column_family_id}; + toku::comparator cmp; + cmp.create(CompareDbtEndpoints, (void*)cfh->GetComparator()); + toku::locktree* ltree = + ltm_.get_lt(dict_id, cmp, + /* on_create_extra*/ static_cast(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(<ree_map_mutex_); + + auto lock_maps_iter = ltree_map_.find(column_family_id); + assert(lock_maps_iter != ltree_map_.end()); + ltree_map_.erase(lock_maps_iter); + } // lock_map_mutex_ + + autovector local_caches; + ltree_lookup_cache_->Scrape(&local_caches, nullptr); + for (auto cache : local_caches) { + delete static_cast(cache); + } +} + +std::shared_ptr 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(ltree_lookup_cache_->Get()); + + auto it = ltree_map_cache->find(column_family_id); + if (it != ltree_map_cache->end()) { + // Found lock map for this column family. + return it->second; + } + + // Not found in local cache, grab mutex and check shared LockMaps + InstrumentedMutexLock l(<ree_map_mutex_); + + it = ltree_map_.find(column_family_id); + if (it == ltree_map_.end()) { + return nullptr; + } else { + // Found lock map. Store in thread-local cache and return. + ltree_map_cache->insert({column_family_id, it->second}); + return it->second; + } +} + +struct LOCK_PRINT_CONTEXT { + RangeLockManagerHandle::RangeLockStatus* data; // Save locks here + uint32_t cfh_id; // Column Family whose tree we are traversing +}; + +// Report left endpoints of the acquired locks +LockManager::PointLockStatus RangeTreeLockManager::GetPointLockStatus() { + PointLockStatus res; + LockManager::RangeLockStatus data = GetRangeLockStatus(); + // report left endpoints + for (auto it = data.begin(); it != data.end(); ++it) { + auto& val = it->second; + res.insert({it->first, {val.start.slice, val.ids, val.exclusive}}); + } + return res; +} + +static void push_into_lock_status_data(void* param, const DBT* left, + const DBT* right, TXNID txnid_arg, + bool is_shared, TxnidVector* owners) { + struct LOCK_PRINT_CONTEXT* ctx = (LOCK_PRINT_CONTEXT*)param; + struct RangeLockInfo info; + + info.exclusive = !is_shared; + + deserialize_endpoint(left, &info.start); + deserialize_endpoint(right, &info.end); + + if (txnid_arg != TXNID_SHARED) { + 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(<ree_map_mutex_); + for (auto it : ltree_map_) { + LOCK_PRINT_CONTEXT ctx = {&data, it.first}; + it.second->dump_locks((void*)&ctx, push_into_lock_status_data); + } + } + return data; +} + +} // namespace ROCKSDB_NAMESPACE +#endif // OS_WIN +#endif // ROCKSDB_LITE 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 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 GetDeadlockInfoBuffer() override; + + std::vector 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 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 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 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>; + 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 ltree_lookup_cache_; + + RangeDeadlockInfoBuffer dlock_buffer_; + + std::shared_ptr 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()).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 +#include +#include + +#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> + buffers_; + port::Mutex mutex_; + std::atomic 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 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 -- cgit v1.2.3