transaction_lock_mgr.h 5.6 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. #pragma once
  6. #ifndef ROCKSDB_LITE
  7. #include <chrono>
  8. #include <string>
  9. #include <unordered_map>
  10. #include <memory>
  11. #include <utility>
  12. #include <vector>
  13. #include "monitoring/instrumented_mutex.h"
  14. #include "rocksdb/utilities/transaction.h"
  15. #include "util/autovector.h"
  16. #include "util/hash_map.h"
  17. #include "util/thread_local.h"
  18. #include "utilities/transactions/pessimistic_transaction.h"
  19. namespace ROCKSDB_NAMESPACE {
  20. class ColumnFamilyHandle;
  21. struct LockInfo;
  22. struct LockMap;
  23. struct LockMapStripe;
  24. struct DeadlockInfoBuffer {
  25. private:
  26. std::vector<DeadlockPath> paths_buffer_;
  27. uint32_t buffer_idx_;
  28. std::mutex paths_buffer_mutex_;
  29. std::vector<DeadlockPath> Normalize();
  30. public:
  31. explicit DeadlockInfoBuffer(uint32_t n_latest_dlocks)
  32. : paths_buffer_(n_latest_dlocks), buffer_idx_(0) {}
  33. void AddNewPath(DeadlockPath path);
  34. void Resize(uint32_t target_size);
  35. std::vector<DeadlockPath> PrepareBuffer();
  36. };
  37. struct TrackedTrxInfo {
  38. autovector<TransactionID> m_neighbors;
  39. uint32_t m_cf_id;
  40. bool m_exclusive;
  41. std::string m_waiting_key;
  42. };
  43. class Slice;
  44. class PessimisticTransactionDB;
  45. class TransactionLockMgr {
  46. public:
  47. TransactionLockMgr(TransactionDB* txn_db, size_t default_num_stripes,
  48. int64_t max_num_locks, uint32_t max_num_deadlocks,
  49. std::shared_ptr<TransactionDBMutexFactory> factory);
  50. // No copying allowed
  51. TransactionLockMgr(const TransactionLockMgr&) = delete;
  52. void operator=(const TransactionLockMgr&) = delete;
  53. ~TransactionLockMgr();
  54. // Creates a new LockMap for this column family. Caller should guarantee
  55. // that this column family does not already exist.
  56. void AddColumnFamily(uint32_t column_family_id);
  57. // Deletes the LockMap for this column family. Caller should guarantee that
  58. // this column family is no longer in use.
  59. void RemoveColumnFamily(uint32_t column_family_id);
  60. // Attempt to lock key. If OK status is returned, the caller is responsible
  61. // for calling UnLock() on this key.
  62. Status TryLock(PessimisticTransaction* txn, uint32_t column_family_id,
  63. const std::string& key, Env* env, bool exclusive);
  64. // Unlock a key locked by TryLock(). txn must be the same Transaction that
  65. // locked this key.
  66. void UnLock(const PessimisticTransaction* txn, const TransactionKeyMap* keys,
  67. Env* env);
  68. void UnLock(PessimisticTransaction* txn, uint32_t column_family_id,
  69. const std::string& key, Env* env);
  70. using LockStatusData = std::unordered_multimap<uint32_t, KeyLockInfo>;
  71. LockStatusData GetLockStatusData();
  72. std::vector<DeadlockPath> GetDeadlockInfoBuffer();
  73. void Resize(uint32_t);
  74. private:
  75. PessimisticTransactionDB* txn_db_impl_;
  76. // Default number of lock map stripes per column family
  77. const size_t default_num_stripes_;
  78. // Limit on number of keys locked per column family
  79. const int64_t max_num_locks_;
  80. // The following lock order must be satisfied in order to avoid deadlocking
  81. // ourselves.
  82. // - lock_map_mutex_
  83. // - stripe mutexes in ascending cf id, ascending stripe order
  84. // - wait_txn_map_mutex_
  85. //
  86. // Must be held when accessing/modifying lock_maps_.
  87. InstrumentedMutex lock_map_mutex_;
  88. // Map of ColumnFamilyId to locked key info
  89. using LockMaps = std::unordered_map<uint32_t, std::shared_ptr<LockMap>>;
  90. LockMaps lock_maps_;
  91. // Thread-local cache of entries in lock_maps_. This is an optimization
  92. // to avoid acquiring a mutex in order to look up a LockMap
  93. std::unique_ptr<ThreadLocalPtr> lock_maps_cache_;
  94. // Must be held when modifying wait_txn_map_ and rev_wait_txn_map_.
  95. std::mutex wait_txn_map_mutex_;
  96. // Maps from waitee -> number of waiters.
  97. HashMap<TransactionID, int> rev_wait_txn_map_;
  98. // Maps from waiter -> waitee.
  99. HashMap<TransactionID, TrackedTrxInfo> wait_txn_map_;
  100. DeadlockInfoBuffer dlock_buffer_;
  101. // Used to allocate mutexes/condvars to use when locking keys
  102. std::shared_ptr<TransactionDBMutexFactory> mutex_factory_;
  103. bool IsLockExpired(TransactionID txn_id, const LockInfo& lock_info, Env* env,
  104. uint64_t* wait_time);
  105. std::shared_ptr<LockMap> GetLockMap(uint32_t column_family_id);
  106. Status AcquireWithTimeout(PessimisticTransaction* txn, LockMap* lock_map,
  107. LockMapStripe* stripe, uint32_t column_family_id,
  108. const std::string& key, Env* env, int64_t timeout,
  109. LockInfo&& lock_info);
  110. Status AcquireLocked(LockMap* lock_map, LockMapStripe* stripe,
  111. const std::string& key, Env* env,
  112. LockInfo&& lock_info, uint64_t* wait_time,
  113. autovector<TransactionID>* txn_ids);
  114. void UnLockKey(const PessimisticTransaction* txn, const std::string& key,
  115. LockMapStripe* stripe, LockMap* lock_map, Env* env);
  116. bool IncrementWaiters(const PessimisticTransaction* txn,
  117. const autovector<TransactionID>& wait_ids,
  118. const std::string& key, const uint32_t& cf_id,
  119. const bool& exclusive, Env* const env);
  120. void DecrementWaiters(const PessimisticTransaction* txn,
  121. const autovector<TransactionID>& wait_ids);
  122. void DecrementWaitersImpl(const PessimisticTransaction* txn,
  123. const autovector<TransactionID>& wait_ids);
  124. };
  125. } // namespace ROCKSDB_NAMESPACE
  126. #endif // ROCKSDB_LITE