You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by zu...@apache.org on 2016/05/16 20:56:07 UTC

[30/46] incubator-quickstep git commit: Transaction Part 4: LockManager, CycleDetector and DeadLockDetector. (#187)

Transaction Part 4: LockManager, CycleDetector and DeadLockDetector. (#187)

Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/bbaff7a2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/bbaff7a2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/bbaff7a2

Branch: refs/heads/master
Commit: bbaff7a2aa84b0792a33ca652da701c1b3498008
Parents: a3889a3
Author: Hakan Memisoglu <ha...@gmail.com>
Authored: Wed May 4 23:30:51 2016 -0500
Committer: Zuyu ZHANG <zu...@users.noreply.github.com>
Committed: Wed May 4 21:30:51 2016 -0700

----------------------------------------------------------------------
 transaction/AccessMode.hpp                      |  76 +++++-
 transaction/CMakeLists.txt                      |  59 ++++-
 transaction/CycleDetector.cpp                   | 120 ++++++++++
 transaction/CycleDetector.hpp                   |  83 +++++++
 transaction/DeadLockDetector.cpp                | 177 ++++++++++++++
 transaction/DeadLockDetector.hpp                | 156 ++++++++++++
 transaction/DirectedGraph.hpp                   |  56 ++---
 transaction/LockManager.cpp                     | 237 +++++++++++++++++++
 transaction/LockManager.hpp                     | 128 ++++++++++
 transaction/LockTable.cpp                       |  22 +-
 transaction/LockTable.hpp                       |  36 +--
 transaction/ResourceId.hpp                      |   9 +-
 transaction/StronglyConnectedComponents.cpp     |   1 -
 transaction/Transaction.cpp                     |  48 ----
 transaction/TransactionTable.cpp                |   8 +-
 transaction/TransactionTable.hpp                |  11 +-
 transaction/tests/AccessMode_unittest.cpp       |  12 +-
 transaction/tests/CycleDetector_unittest.cpp    | 157 ++++++++++++
 transaction/tests/DeadLockDetector_unittest.cpp |  96 ++++++++
 transaction/tests/DirectedGraph_unittest.cpp    | 113 +++++----
 transaction/tests/LockRequest_unittest.cpp      |   4 +-
 transaction/tests/LockTable_unittest.cpp        |  63 ++---
 transaction/tests/Lock_unittest.cpp             |  13 +-
 .../StronglyConnectedComponents_unittest.cpp    |  15 +-
 transaction/tests/TransactionTable_unittest.cpp | 102 ++++----
 25 files changed, 1512 insertions(+), 290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/AccessMode.hpp
----------------------------------------------------------------------
diff --git a/transaction/AccessMode.hpp b/transaction/AccessMode.hpp
index 34ace36..bb06689 100644
--- a/transaction/AccessMode.hpp
+++ b/transaction/AccessMode.hpp
@@ -32,12 +32,12 @@ namespace transaction {
  * @brief Represents mode type. Possible options are NL, IS, IX, S, SIX, X.
  **/
 enum class AccessModeType : std::uint8_t {
-  kNoLock = 0,
-  kIsLock,
-  kIxLock,
-  kSLock,
-  kSixLock,
-  kXLock,
+  kNoLockMode = 0,
+  kIsLockMode,
+  kIxLockMode,
+  kSLockMode,
+  kSixLockMode,
+  kXLockMode,
   kNumAccessModeTypes,
 };
 
@@ -55,6 +55,60 @@ class AccessMode {
       : access_mode_(access_mode) {}
 
   /**
+   * @brief Factory method for NoLockMode.
+   *
+   * @return NoLockMode instance.
+   **/
+  static AccessMode NoLockMode() {
+    return AccessMode(AccessModeType::kNoLockMode);
+  }
+
+  /**
+   * @brief Factory method for IsLockMode.
+   *
+   * @return IsLockMode instance.
+   **/
+  static AccessMode IsLockMode() {
+    return AccessMode(AccessModeType::kIsLockMode);
+  }
+
+  /**
+   * @brief Factory method for IxLockMode.
+   *
+   * @return IxLockMode instance.
+   **/
+  static AccessMode IxLockMode() {
+    return AccessMode(AccessModeType::kIxLockMode);
+  }
+
+  /**
+   * @brief Factory method for SixLockMode.
+   *
+   * @return SixLockMode instance.
+   **/
+  static AccessMode SixLockMode() {
+    return AccessMode(AccessModeType::kSixLockMode);
+  }
+
+  /**
+   * @brief Factory method for SLockMode.
+   *
+   * @return SLockMode instance.
+   **/
+  static AccessMode SLockMode() {
+    return AccessMode(AccessModeType::kSLockMode);
+  }
+
+  /**
+   * @brief Factory method for XLockMode.
+   *
+   * @return XLockMode instance.
+   **/
+  static AccessMode XLockMode() {
+    return AccessMode(AccessModeType::kXLockMode);
+  }
+
+  /**
    * @brief Checks whether this access mode is compatible with the other.
    *
    * @param other Other access mode that will be checked against to this one.
@@ -74,7 +128,7 @@ class AccessMode {
    * @return True if it is IS mode, false otherwise.
    **/
   inline bool isIntentionShareLock() const {
-    return access_mode_ == AccessModeType::kIsLock;
+    return access_mode_ == AccessModeType::kIsLockMode;
   }
 
   /**
@@ -83,7 +137,7 @@ class AccessMode {
    * @return True if it is IX mode, false otherwise.
    **/
   inline bool isIntentionExclusiveLock() const {
-    return access_mode_ == AccessModeType::kIxLock;
+    return access_mode_ == AccessModeType::kIxLockMode;
   }
 
   /**
@@ -92,7 +146,7 @@ class AccessMode {
    * @return True if it is SIX mode, false otherwise.
    **/
   inline bool isShareAndIntentionExclusiveLock() const {
-    return access_mode_ == AccessModeType::kSixLock;
+    return access_mode_ == AccessModeType::kSixLockMode;
   }
 
   /**
@@ -101,7 +155,7 @@ class AccessMode {
    * @return True if it is S mode, false otherwise.
    **/
   inline bool isShareLock() const {
-    return access_mode_ == AccessModeType::kSLock;
+    return access_mode_ == AccessModeType::kSLockMode;
   }
 
   /**
@@ -110,7 +164,7 @@ class AccessMode {
    * @return True if it is X mode, false otherwise.
    **/
   inline bool isExclusiveLock() const {
-    return access_mode_ == AccessModeType::kXLock;
+    return access_mode_ == AccessModeType::kXLockMode;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/transaction/CMakeLists.txt b/transaction/CMakeLists.txt
index 05fc96a..c6c87b6 100644
--- a/transaction/CMakeLists.txt
+++ b/transaction/CMakeLists.txt
@@ -16,12 +16,21 @@
 add_library(quickstep_transaction_AccessMode
             AccessMode.cpp
             AccessMode.hpp)
+add_library(quickstep_transaction_CycleDetector
+            CycleDetector.cpp
+            CycleDetector.hpp)
+add_library(quickstep_transaction_DeadLockDetector
+            DeadLockDetector.cpp
+            DeadLockDetector.cpp)
 add_library(quickstep_transaction_DirectedGraph
             ../empty_src.cpp
             DirectedGraph.hpp)
 add_library(quickstep_transaction_Lock
             ../empty_src.cpp
             Lock.hpp)
+add_library(quickstep_transaction_LockManager
+            LockManager.hpp
+            LockManager.cpp)
 add_library(quickstep_transaction_LockRequest
             ../empty_src.cpp
             LockRequest.hpp)
@@ -40,7 +49,17 @@ add_library(quickstep_transaction_Transaction
 add_library(quickstep_transaction_TransactionTable
             TransactionTable.cpp
             TransactionTable.hpp)
-          
+
+target_link_libraries(quickstep_transaction_CycleDetector
+                      quickstep_transaction_DirectedGraph
+                      quickstep_transaction_StronglyConnectedComponents
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_transaction_DeadLockDetector
+                      glog
+                      quickstep_transaction_CycleDetector
+                      quickstep_transaction_DirectedGraph
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_Transaction)
 target_link_libraries(quickstep_transaction_DirectedGraph
                       glog
                       quickstep_transaction_Transaction
@@ -48,6 +67,18 @@ target_link_libraries(quickstep_transaction_DirectedGraph
 target_link_libraries(quickstep_transaction_Lock
                       quickstep_transaction_AccessMode
                       quickstep_transaction_ResourceId)
+target_link_libraries(quickstep_transaction_LockManager
+                      gflags_nothreads-static
+                      glog
+                      quickstep_utility_ThreadSafeQueue
+                      quickstep_threading_Thread
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_DeadLockDetector
+                      quickstep_transaction_LockRequest
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction
+                      quickstep_transaction_TransactionTable)
 target_link_libraries(quickstep_transaction_LockRequest
                       quickstep_transaction_AccessMode
                       quickstep_transaction_ResourceId
@@ -80,8 +111,11 @@ add_library(quickstep_transaction
             TransactionModule.hpp)
 target_link_libraries(quickstep_transaction
                       quickstep_transaction_AccessMode
+                      quickstep_transaction_CycleDetector
+                      quickstep_transaction_DeadLockDetector
                       quickstep_transaction_DirectedGraph
                       quickstep_transaction_Lock
+                      quickstep_transaction_LockManager
                       quickstep_transaction_LockRequest
                       quickstep_transaction_LockTable
                       quickstep_transaction_ResourceId
@@ -97,6 +131,29 @@ target_link_libraries(AccessMode_unittest
                       quickstep_transaction_AccessMode)
 add_test(AccessMode_unittest AccessMode_unittest)
 
+add_executable(CycleDetector_unittest
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/CycleDetector_unittest.cpp")
+target_link_libraries(CycleDetector_unittest
+                      gtest
+                      gtest_main
+                      quickstep_transaction_CycleDetector
+                      quickstep_transaction_DirectedGraph
+                      quickstep_transaction_StronglyConnectedComponents)
+add_test(CycleDetector_unittest CycleDetector_unittest)
+
+add_executable(DeadLockDetector_unittest
+  "${CMAKE_CURRENT_SOURCE_DIR}/tests/DeadLockDetector_unittest.cpp")
+target_link_libraries(DeadLockDetector_unittest
+                      gtest
+                      gtest_main
+                      quickstep_threading_Thread
+                      quickstep_transaction_AccessMode
+                      quickstep_transaction_DeadLockDetector
+                      quickstep_transaction_LockTable
+                      quickstep_transaction_ResourceId
+                      quickstep_transaction_Transaction)
+add_test(DeadLockDetector_unittest DeadLockDetector_unittest)
+
 add_executable(DirectedGraph_unittest
                "${CMAKE_CURRENT_SOURCE_DIR}/tests/DirectedGraph_unittest.cpp")
 target_link_libraries(DirectedGraph_unittest

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/CycleDetector.cpp
----------------------------------------------------------------------
diff --git a/transaction/CycleDetector.cpp b/transaction/CycleDetector.cpp
new file mode 100644
index 0000000..b12897f
--- /dev/null
+++ b/transaction/CycleDetector.cpp
@@ -0,0 +1,120 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#include "transaction/CycleDetector.hpp"
+
+#include <cstdint>
+#include <memory>
+#include <stack>
+#include <unordered_map>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/StronglyConnectedComponents.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+CycleDetector::CycleDetector(DirectedGraph *wait_for_graph)
+    : wait_for_graph_(wait_for_graph),
+      strongly_connected_components_(
+          std::make_unique<StronglyConnectedComponents>(*wait_for_graph)) {
+}
+
+std::vector<DirectedGraph::node_id>
+CycleDetector::chooseVictimsToBreakCycle() const {
+  std::vector<DirectedGraph::node_id> nodes_to_kill;
+  const std::unordered_map<std::uint64_t, std::vector<DirectedGraph::node_id>>
+      component_mapping = strongly_connected_components_->getComponentMapping();
+  for (const auto &entry : component_mapping) {
+    // One node means no cycle.
+    if (entry.second.size() == 1) {
+      continue;
+    }
+    const std::vector<DirectedGraph::node_id> nodes =
+        chooseVictimsInComponent(entry.second);
+    nodes_to_kill.insert(nodes_to_kill.end(), nodes.begin(), nodes.end());
+  }
+  return nodes_to_kill;
+}
+
+std::vector<DirectedGraph::node_id> CycleDetector::chooseVictimsInComponent(
+    const std::vector<DirectedGraph::node_id> &nodes) const {
+  std::vector<DirectedGraph::node_id> targets;
+  // Convert it to set to ensure defensively that the elements are unique.
+  std::unordered_set<DirectedGraph::node_id> nodes_set(nodes.begin(),
+                                                       nodes.end());
+
+  while (true) {
+    if (!hasCycle(nodes_set)) {
+      break;
+    }
+    // Connected component still has a cycle, therefore choose a
+    // victim and keep trying to remove nodes until there is no cycle.
+    const DirectedGraph::node_id victim = chooseVictim(nodes_set);
+    // Remove the victim node from the connected component.
+    nodes_set.erase(victim);
+    // Removed node is a victim now.
+    targets.push_back(victim);
+  }
+  return targets;
+}
+
+bool CycleDetector::hasCycle(
+    const std::unordered_set<DirectedGraph::node_id> &nodes) const {
+  // Keeps track of the nodes the algorithms visited.
+  std::unordered_set<DirectedGraph::node_id> visited;
+  for (const DirectedGraph::node_id node_id : nodes) {
+    // If it is visited, then pass to the next one.
+    if (visited.count(node_id) == 1) {
+      continue;
+    }
+    // Save the backtracking information.
+    std::stack<DirectedGraph::node_id> to_visit;
+    // Mark this id as "to be visited".
+    to_visit.push(node_id);
+    // Start to visit nodes until it is done.
+    while (!to_visit.empty()) {
+      const DirectedGraph::node_id current_node = to_visit.top();
+      to_visit.pop();
+      // Mark the node coming from stack as "visited".
+      visited.insert(current_node);
+      // For all adjacent nodes of this "visited" node,
+      const std::vector<DirectedGraph::node_id> adjacents
+          = wait_for_graph_->getAdjacentNodes(current_node);
+      for (const DirectedGraph::node_id adjacent : adjacents) {
+        if (visited.count(adjacent) == 1) {
+          // If this adjacent node is a node we already visited, then
+          // there is a cycle.
+          return true;
+        } else if (nodes.count(adjacent) == 1 && visited.count(adjacent) == 0) {
+          // Otherwise, if it is a node that we did not visit before
+          // mark this nodes as "to be visited".
+          to_visit.push(adjacent);
+        }
+      }
+    }
+  }
+  // If we have already visited all nodes and could not find a cycle,
+  // then we should return "no cycle" result.
+  return false;
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/CycleDetector.hpp
----------------------------------------------------------------------
diff --git a/transaction/CycleDetector.hpp b/transaction/CycleDetector.hpp
new file mode 100644
index 0000000..6865e2d
--- /dev/null
+++ b/transaction/CycleDetector.hpp
@@ -0,0 +1,83 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_TRANSACTION_CYCLE_DETECTOR_HPP_
+#define QUICKSTEP_TRANSACTION_CYCLE_DETECTOR_HPP_
+
+#include <memory>
+#include <unordered_set>
+#include <vector>
+
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/StronglyConnectedComponents.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Class for running cycle detection algorithm on directed graph.
+ */
+class CycleDetector {
+ public:
+  /**
+   * @brief Constructor for DirectedGraph.
+   *
+   * @param wait_for_graph Pointer to a directed wait-for graph.
+   */
+  explicit CycleDetector(DirectedGraph *wait_for_graph);
+
+  /**
+   * @brief Calculate which nodes should be killed to eliminate all cycles
+   *        in the graph.
+   *
+   * @return Vector of node ids that should be killed to break all cycles.
+   */
+  std::vector<DirectedGraph::node_id> chooseVictimsToBreakCycle() const;
+
+ private:
+  std::vector<DirectedGraph::node_id> chooseVictimsInComponent(
+      const std::vector<DirectedGraph::node_id> &nodes) const;
+
+  inline DirectedGraph::node_id chooseVictim(
+      const std::unordered_set<DirectedGraph::node_id> &nodes_set) const {
+    // TODO(Hakan): This is very inefficient scheme, however in the
+    //              future, we can use the transaction's priority
+    //              as the victim selection parameter.
+    return *(nodes_set.begin());
+  }
+
+  // Checks whether the nodes in the set make a cycle.
+  bool hasCycle(const std::unordered_set<DirectedGraph::node_id> &within) const;
+
+  DirectedGraph *wait_for_graph_;
+
+  std::unique_ptr<StronglyConnectedComponents> strongly_connected_components_;
+
+  DISALLOW_COPY_AND_ASSIGN(CycleDetector);
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_CYCLE_DETECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/DeadLockDetector.cpp
----------------------------------------------------------------------
diff --git a/transaction/DeadLockDetector.cpp b/transaction/DeadLockDetector.cpp
new file mode 100644
index 0000000..26ab115
--- /dev/null
+++ b/transaction/DeadLockDetector.cpp
@@ -0,0 +1,177 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#include "transaction/DeadLockDetector.hpp"
+
+#include <algorithm>
+#include <atomic>
+#include <chrono>
+#include <memory>
+#include <thread>  // NOLINT(build/c++11)
+#include <utility>
+#include <vector>
+
+#include "transaction/CycleDetector.hpp"
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/LockTable.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace transaction {
+
+constexpr std::int64_t DeadLockDetector::kSleepDurationInSeconds;
+
+DeadLockDetector::DeadLockDetector(LockTable *lock_table,
+                                   std::atomic<DeadLockDetectorStatus> *status,
+                                   std::vector<DirectedGraph::node_id> *victims)
+    : tid_node_mapping_(std::make_unique<transaction_id_node_map>()),
+      lock_table_(lock_table),
+      status_(status),
+      victims_(victims) {
+}
+
+void DeadLockDetector::run() {
+  while (true) {
+    if (status_->load() == DeadLockDetectorStatus::kQuit) {
+      // DeadLockDetector should stop.
+      return;
+    }
+    while (status_->load() == DeadLockDetectorStatus::kDone) {
+      // LockTable has not process the previous batch yet.
+    }
+
+    // TODO(Hakan): Implement logging mechanism for deadlock detection
+    //              start and end times.
+    std::vector<DirectedGraph::node_id> victim_new_batch = getAllVictims();
+
+    // Swap new batch with old batch to make LockTable to see new victims.
+    std::swap(victim_new_batch, *victims_);
+
+    // Signal LockTable that new batch is ready.
+    status_->store(DeadLockDetectorStatus::kDone);
+
+    // DeadLockDetector should run once in a predefined interval.
+    std::this_thread::sleep_for(
+        std::chrono::seconds(kSleepDurationInSeconds));
+  }
+}
+
+void DeadLockDetector::addPendingInfo(const transaction_id pending,
+                                      const transaction_id owner) {
+  const DirectedGraph::node_id pending_node_id = getNodeId(pending);
+  const DirectedGraph::node_id owner_node_id = getNodeId(owner);
+
+  // TODO(Hakan): Check first whether link is already created. Use checked
+  //              version for adding an edge.
+  wait_for_graph_->addEdgeUnchecked(pending_node_id, owner_node_id);
+}
+
+void DeadLockDetector::deletePendingInfo(const transaction_id pending,
+                                         const transaction_id owner) {
+  LOG(FATAL) << "Not implemented";
+}
+
+bool DeadLockDetector::isDependent(const transaction_id pending,
+                                   const transaction_id owner) const {
+  LOG(FATAL) << "Not implemented";
+}
+
+std::vector<transaction_id>
+DeadLockDetector::getAllDependents(const transaction_id owner) const {
+  LOG(FATAL) << "Not implemented";
+}
+
+std::vector<transaction_id>
+DeadLockDetector::getAllDependees(transaction_id pending) {
+  const DirectedGraph::node_id pending_node_id = getNodeId(pending);
+  const std::vector<DirectedGraph::node_id> nodes
+      = wait_for_graph_->getAdjacentNodes(pending_node_id);
+  std::vector<transaction_id> transactions;
+  transactions.reserve(nodes.size());
+  for (const DirectedGraph::node_id node_id : nodes) {
+    const transaction_id tid = wait_for_graph_->getDataFromNode(node_id);
+    transactions.push_back(tid);
+  }
+  return transactions;
+}
+
+DirectedGraph::node_id DeadLockDetector::getNodeId(const transaction_id tid) {
+  DirectedGraph::node_id node_id;
+  if (tid_node_mapping_->count(tid) == 0) {
+    // If it is not created, create it.
+    node_id = addNode(tid);
+  } else {
+    // Otherwise find it in the map.
+    node_id = (*tid_node_mapping_)[tid];
+  }
+  return node_id;
+}
+
+
+DirectedGraph::node_id DeadLockDetector::addNode(const transaction_id tid) {
+  const DirectedGraph::node_id node_id =
+      wait_for_graph_->addNodeUnchecked(tid);
+  tid_node_mapping_->emplace(tid, node_id);
+  return node_id;
+}
+
+std::vector<transaction_id> DeadLockDetector::getAllVictims()  {
+  std::vector<transaction_id> result_victims;
+  wait_for_graph_.reset(new DirectedGraph());
+
+  // Critical region on LockTable starts here.
+  lock_table_->latchShared();
+  for (const auto &lock_control_block : *lock_table_) {
+    const LockTable::lock_own_list &own_list = lock_control_block.second.first;
+    const LockTable::lock_pending_list &pending_list =
+        lock_control_block.second.second;
+
+    for (const auto &owned_lock_info : own_list) {
+      const transaction_id owned_transaction = owned_lock_info.first;
+      const DirectedGraph::node_id owned_node = getNodeId(owned_transaction);
+
+      for (const auto &pending_lock_info : pending_list) {
+        const transaction_id pending_transaction = pending_lock_info.first;
+        const DirectedGraph::node_id pending_node = getNodeId(pending_transaction);
+
+        wait_for_graph_->addEdgeUnchecked(pending_node, owned_node);
+      }
+    }
+  }
+
+  lock_table_->unlatchShared();
+  // Critical region on LockTable ends here.
+
+  const CycleDetector cycle_detector(wait_for_graph_.get());
+  const std::vector<DirectedGraph::node_id> victim_nodes =
+      cycle_detector.chooseVictimsToBreakCycle();
+  for (const DirectedGraph::node_id node_id : victim_nodes) {
+    const transaction_id victim_tid = wait_for_graph_->getDataFromNode(node_id);
+    result_victims.push_back(victim_tid);
+  }
+
+  // Destroy the wait graph. It will be reconstructed kSleepDurationSeconds
+  // seconds later.
+  wait_for_graph_.reset();
+
+  return result_victims;
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/DeadLockDetector.hpp
----------------------------------------------------------------------
diff --git a/transaction/DeadLockDetector.hpp b/transaction/DeadLockDetector.hpp
new file mode 100644
index 0000000..6897afb
--- /dev/null
+++ b/transaction/DeadLockDetector.hpp
@@ -0,0 +1,156 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_TRANSACTION_DEAD_LOCK_DETECTOR_HPP_
+#define QUICKSTEP_TRANSACTION_DEAD_LOCK_DETECTOR_HPP_
+
+#include <atomic>
+#include <cstdint>
+#include <memory>
+#include <unordered_map>
+#include <vector>
+
+#include "threading/Thread.hpp"
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/Transaction.hpp"
+
+namespace quickstep {
+namespace transaction {
+
+class LockTable;
+
+/** \addtogroup Transaction
+ *  @{
+ */
+
+/**
+ * @brief Notification mechanism between LockManager and DeadLockDetector.
+ **/
+enum class DeadLockDetectorStatus {
+  kNotReady = 0,
+  kDone,
+  kQuit,
+};
+
+/**
+ * @brief Class for deadlock detection on wait-for graph.
+ **/
+class DeadLockDetector : public Thread {
+ public:
+  typedef std::unordered_map<transaction_id, DirectedGraph::node_id>
+      transaction_id_node_map;
+
+  /**
+   * @brief Constructor for DeadLockDetector.
+   *
+   * @param lock_table Pointer to lock table, which this class gets the
+   *        necessary information.
+   * @param status Pointer to status object which will act as a message
+   *        passing algorithm between LockManager.
+   * @param victims Message passing buffer betwen DeadLockDetector and
+   *        LockManager.
+   **/
+  DeadLockDetector(LockTable *lock_table,
+                   std::atomic<DeadLockDetectorStatus> *status,
+                   std::vector<DirectedGraph::node_id> *victims);
+
+  void run() override;
+
+  /**
+   * @brief Adds pending information based on a resource conflict.
+   *
+   * @param pending Id of the transaction that waits for the resource lock.
+   * @param owner Id of the transaction that owns the resource lock.
+   */
+  void addPendingInfo(const transaction_id pending,
+                      const transaction_id owner);
+
+  /**
+   * @brief Deletes pending information on a resource.
+   *
+   * @param pending Id of the transaction that waits for the resource lock.
+   * @param owner Id of the transaction that owns the resource lock.
+   *
+   * @warning This method is not implemented yet.
+   */
+  void deletePendingInfo(const transaction_id pending,
+                         const transaction_id owner);
+
+  /**
+   * @brief Check whether first transaction waits for the latter.
+   *
+   * @param pending Id of the transaction which will be checked whether
+   *        it waits for the other.
+   * @param owner Id of the transaction which will be checked whether
+   *        it is waited by the first.
+   *
+   * @warning This method is not implemented yet.
+   */
+  bool isDependent(const transaction_id pending,
+                   const transaction_id owner) const;
+
+  /**
+   * @brief Gives the ids of transactions that wait for the owner transaction.
+   *
+   * @param owner Id of the transaction whose the penders will be returned.
+   * @return Vector of transaction ids that wait for owner.
+   */
+  std::vector<transaction_id> getAllDependents(const transaction_id owner) const;
+
+  /**
+   * @brief Gives the ids of transaction that the pending transaction waits for.
+   * @warning This method is not implemented yet.
+
+   * @param pending Id of the transaction that is pending.
+   * @return Vector of transaction ids which the pending transaction waits for.
+   */
+  std::vector<transaction_id> getAllDependees(transaction_id pending);
+
+  /**
+   * @brief Gives the list of victims whose the terminations will end the cycle.
+   *
+   * @return Vector of victim transaction ids.
+   */
+  std::vector<transaction_id> getAllVictims();
+
+ private:
+  static constexpr std::int64_t kSleepDurationInSeconds = 5;
+
+  DirectedGraph::node_id getNodeId(const transaction_id tid);
+
+  DirectedGraph::node_id addNode(const transaction_id tid);
+
+  // Owned pointer to wait-for graph.
+  std::unique_ptr<DirectedGraph> wait_for_graph_;
+
+  // Mapping from TransactioId to NodeId in graph.
+  std::unique_ptr<transaction_id_node_map> tid_node_mapping_;
+
+  // Pointer to lock table. Dependence edges will be created
+  // by the information got from lock table.
+  LockTable *lock_table_;
+
+  std::atomic<DeadLockDetectorStatus> *status_;
+  std::vector<DirectedGraph::node_id> *victims_;
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_DEAD_LOCK_DETECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/DirectedGraph.hpp
----------------------------------------------------------------------
diff --git a/transaction/DirectedGraph.hpp b/transaction/DirectedGraph.hpp
index 89ce9c6..16b551a 100644
--- a/transaction/DirectedGraph.hpp
+++ b/transaction/DirectedGraph.hpp
@@ -21,8 +21,8 @@
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <iterator>
 #include <memory>
-#include <stack>
 #include <unordered_set>
 #include <vector>
 
@@ -39,7 +39,7 @@ namespace transaction {
  */
 
 /**
- * @brief Class for representing a directed graph. Vertices are transaction 
+ * @brief Class for representing a directed graph. Vertices are transaction
  *        ids, edges are wait-for relations.
  **/
 class DirectedGraph {
@@ -54,34 +54,31 @@ class DirectedGraph {
   /**
    * @brief Adds a new node to the graph with the given transaction id.
    *        It does not check whether the transaction id is valid or not.
-   * @warning Pointer ownership will pass to the graph, therefore it
-   *          should not be deleted.
    *
-   * @param data Pointer to the transaction id that will be contained
+   * @param transaction_id_payload Transaction id that will be contained
    *        in the node.
    * @return Id of the newly created node.
    **/
-  inline node_id addNodeUnchecked(transaction_id *data) {
-    nodes_.emplace_back(data);
+  inline
+  node_id addNodeUnchecked(const transaction_id transaction_id_payload) {
+    nodes_.emplace_back(transaction_id_payload);
     return nodes_.size() - 1;
   }
 
   /**
    * @brief Adds a new node to the graph with the given transaction id.
    *        It checks whether the transaction id is valid or not.
-   * @warning Pointer ownership will pass to the graph, therefore it
-   *          should not be deleted.
    *
-   * @param data Pointer to the transaction id that will be contained
+   * @param transaction_id_payload Transaction id that will be contained
    *        in the node.
    * @return Id of the newly created node.
    **/
-  inline node_id addNodeCheckExists(transaction_id *data) {
-    for (std::vector<DirectedGraphNode>::const_iterator
-           it = nodes_.cbegin(); it != nodes_.cend(); ++it) {
-      CHECK(*data != it->getData());
+  node_id addNodeCheckExists(const transaction_id transaction_id_payload) {
+    for (const auto &node : nodes_) {
+      CHECK(transaction_id_payload != node.getData());
     }
-    nodes_.emplace_back(data);
+
+    nodes_.emplace_back(transaction_id_payload);
     return nodes_.size() - 1;
   }
 
@@ -91,10 +88,10 @@ class DirectedGraph {
    * @warning Does not check arguments are legit. It may cause
    *          out of range errors.
    *
-   * @param fromNode The node that edge is orginated.
+   * @param fromNode The node that edge is originated.
    * @param toNode The node that edge is ended.
    **/
-  inline void addEdgeUnchecked(node_id from_node, node_id to_node) {
+  void addEdgeUnchecked(node_id from_node, node_id to_node) {
     nodes_[from_node].addOutgoingEdge(to_node);
   }
 
@@ -105,7 +102,7 @@ class DirectedGraph {
    * @param fromNode The node that edge is orginated.
    * @param toNode The node that edge is ended.
    **/
-  inline void addEdgeCheckExists(node_id from_node, node_id to_node) {
+  void addEdgeCheckExists(node_id from_node, node_id to_node) {
     CHECK(from_node < getNumNodes() && to_node < getNumNodes());
     nodes_[from_node].addOutgoingEdge(to_node);
   }
@@ -119,7 +116,7 @@ class DirectedGraph {
    * @param toNode Id of the node that edge is ended.
    * @return True if there is an edge, false otherwise.
    **/
-  inline bool hasEdge(node_id from_node, node_id to_node) const {
+  bool hasEdge(node_id from_node, node_id to_node) const {
     DCHECK(from_node < getNumNodes() && to_node < getNumNodes());
     return nodes_[from_node].hasOutgoingEdge(to_node);
   }
@@ -130,7 +127,7 @@ class DirectedGraph {
    * @param node Id of the node that the data is got from.
    * @return Id of the transaction that this node contains.
    **/
-  inline transaction_id getDataFromNode(node_id node) const {
+  transaction_id getDataFromNode(node_id node) const {
     DCHECK(node < getNumNodes());
     return nodes_[node].getData();
   }
@@ -140,7 +137,7 @@ class DirectedGraph {
    *
    * @return The number of nodes the graph has.
    **/
-  inline std::size_t getNumNodes() const {
+  std::size_t getNumNodes() const {
     return nodes_.size();
   }
 
@@ -158,18 +155,18 @@ class DirectedGraph {
   // Class for representing a graph node.
   class DirectedGraphNode {
    public:
-    explicit DirectedGraphNode(transaction_id *data)
-      : data_(data) {}
+    explicit DirectedGraphNode(const transaction_id payload)
+      : transaction_id_payload_(payload) {}
 
-    inline void addOutgoingEdge(node_id to_node) {
+    void addOutgoingEdge(node_id to_node) {
       outgoing_edges_.insert(to_node);
     }
 
-    inline bool hasOutgoingEdge(node_id to_node) const {
+    bool hasOutgoingEdge(node_id to_node) const {
       return outgoing_edges_.count(to_node) == 1;
     }
 
-    inline std::vector<node_id> getOutgoingEdges() const {
+    std::vector<node_id> getOutgoingEdges() const {
       // TODO(hakan): Benchmark this version and the alternative which the
       //              function returns const reference and the uniqueness
       //              is imposed in the outgoing_edges_ as a vector.
@@ -179,13 +176,12 @@ class DirectedGraph {
       return result;
     }
 
-    inline transaction_id getData() const {
-      return *(data_.get());
+    transaction_id getData() const {
+      return transaction_id_payload_;
     }
 
    private:
-    // Owner pointer to transaction id.
-    std::unique_ptr<transaction_id> data_;
+    const transaction_id transaction_id_payload_;
 
     // Endpoint nodes of outgoing edges originated from this node.
     std::unordered_set<node_id> outgoing_edges_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/LockManager.cpp
----------------------------------------------------------------------
diff --git a/transaction/LockManager.cpp b/transaction/LockManager.cpp
new file mode 100644
index 0000000..da6181a
--- /dev/null
+++ b/transaction/LockManager.cpp
@@ -0,0 +1,237 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#include "transaction/LockManager.hpp"
+
+#include <cstdint>
+#include <stack>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/DeadLockDetector.hpp"
+#include "transaction/LockRequest.hpp"
+#include "transaction/LockTable.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+#include "transaction/TransactionTable.hpp"
+#include "utility/ThreadSafeQueue.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace transaction {
+
+DEFINE_uint64(max_try_incoming, 10000,
+              "The maximum number of tries that lock manager checks incoming "
+              "request buffer until the buffer is empty.");
+
+DEFINE_uint64(max_try_inner, 6000,
+              "The maximum number of tries that lock manager checks inner "
+              "request buffer until the buffer is empty.");
+
+LockManager::LockManager(ThreadSafeQueue<LockRequest> *incoming_requests,
+                         ThreadSafeQueue<LockRequest> *permitted_requests)
+    : lock_table_(std::make_unique<LockTable>()),
+      transaction_table_(std::make_unique<TransactionTable>()),
+      detector_status_(DeadLockDetectorStatus::kNotReady),
+      deadlock_detector_(std::make_unique<DeadLockDetector>(lock_table_.get(),
+                                                            &detector_status_,
+                                                            &victim_result_)),
+      incoming_requests_(*incoming_requests),
+      permitted_requests_(*permitted_requests),
+      inner_pending_requests_() {
+}
+
+LockManager::~LockManager() {
+  deadlock_detector_->join();
+}
+
+void LockManager::run() {
+  deadlock_detector_->start();
+
+  const std::uint64_t kMaxTryIncoming =
+      static_cast<std::uint64_t>(FLAGS_max_try_incoming);
+  const std::uint64_t kMaxTryInner =
+      static_cast<std::uint64_t>(FLAGS_max_try_incoming);
+
+  while (true) {
+    for (std::uint64_t tries = 0; tries < kMaxTryIncoming; ++tries) {
+      if (!incoming_requests_.empty()) {
+        const LockRequest request = incoming_requests_.popOne();
+        if (request.getRequestType() == RequestType::kReleaseLocks) {
+          CHECK(releaseAllLocks(request.getTransactionId()))
+              << "Unexpected condition occured.";
+
+        } else if (acquireLock(request.getTransactionId(),
+                               request.getResourceId(),
+                               request.getAccessMode())) {
+          LOG(INFO) << "Transaction "
+                    << std::to_string(request.getTransactionId())
+                    << " is waiting " + request.getResourceId().toString();
+
+            inner_pending_requests_.push(request);
+        } else {
+            LOG(INFO) << "Transaction "
+                      << std::to_string(request.getTransactionId())
+                      << " acquired " + request.getResourceId().toString();
+
+            permitted_requests_.push(request);
+        }
+      }
+    }
+
+    for (std::uint64_t tries = 0; tries < kMaxTryInner; ++tries) {
+      if (!inner_pending_requests_.empty()) {
+        const LockRequest request = inner_pending_requests_.front();
+
+        if (acquireLock(request.getTransactionId(), request.getResourceId(),
+                        request.getAccessMode())) {
+          inner_pending_requests_.pop();
+          permitted_requests_.push(request);
+        }
+      }
+    }
+
+    // Resolve deadlocks.
+    killVictims();
+  }
+}
+
+bool LockManager::acquireLock(const transaction_id tid,
+                              const ResourceId &rid,
+                              const AccessMode &access_mode) {
+  std::stack<std::pair<ResourceId, AccessMode>> stack;
+  ResourceId current_rid = rid;
+  AccessMode current_access_mode = access_mode;
+  stack.push(std::make_pair(current_rid, current_access_mode));
+
+  while (current_rid.hasParent()) {
+    current_rid = current_rid.getParentResourceId();
+    current_access_mode = (current_access_mode.isShareLock() ||
+                           current_access_mode.isIntentionShareLock())
+                              ? AccessMode(AccessMode::IsLockMode())
+                              : AccessMode(AccessMode::IxLockMode());
+
+    stack.push(std::make_pair(current_rid, current_access_mode));
+  }
+
+  lock_table_->latchExclusive();
+
+  while (!stack.empty()) {
+    const std::pair<ResourceId, AccessMode> pair_to_pick = stack.top();
+    const ResourceId rid_to_pick = pair_to_pick.first;
+    const AccessMode access_mode_to_pick = pair_to_pick.second;
+
+    if (!acquireLockInternal(tid, rid_to_pick, access_mode_to_pick)) {
+      lock_table_->unlatchExclusive();
+      return false;
+    }
+    stack.pop();
+  }
+  lock_table_->unlatchExclusive();
+  return true;
+}
+
+bool LockManager::releaseAllLocks(const transaction_id tid,
+                                  const bool latch_table) {
+  const std::vector<ResourceId> resource_ids
+      = transaction_table_->getResourceIdList(tid);
+  const TransactionTableResult transaction_deleted
+      = transaction_table_->deleteTransaction(tid);
+
+  CHECK(transaction_deleted != TransactionTableResult::kTransactionDeleteError)
+      << "In LockManager.releaseAllLocks: Transaction could not be deleted!";
+
+  if (latch_table) {
+    lock_table_->latchExclusive();
+  }
+  for (const auto &resource_id : resource_ids) {
+    const LockTableResult lock_deleted = lock_table_->deleteLock(tid, resource_id);
+
+    LOG(INFO) << "Transaction "
+              << std::to_string(tid)
+              << " released lock:"
+              << resource_id.toString();
+    CHECK(lock_deleted != LockTableResult::kDeleteError)
+        << "In LockManager.releaseAllLock lock could not be deleted from "
+           "LockTable";
+  }
+  if (latch_table) {
+    lock_table_->unlatchExclusive();
+  }
+  return true;
+}
+
+bool LockManager::acquireLockInternal(const transaction_id tid,
+                                      const ResourceId &rid,
+                                      const AccessMode &access_mode) {
+  const LockTableResult lock_result = lock_table_->putLock(tid, rid, access_mode);
+  CHECK(lock_result != LockTableResult::kPutError)
+      << "Unexpected result in LockManager.acquireLockInternal";
+
+  switch (lock_result) {
+  case LockTableResult::kAlreadyInOwned: {
+    return true;
+  }
+  case LockTableResult::kPlacedInOwned: {
+    const TransactionTableResult transaction_result
+        = transaction_table_->putOwnEntry(tid, rid, access_mode);
+    CHECK(transaction_result == TransactionTableResult::kPlacedInOwned)
+        << "Unexpected result in LockManager.acquireLockInternal: "
+           "Mismatch of table results: LockTable entry is owned, "
+           "whereas TransactionTable entry is not owned.";
+    return true;
+  }
+  case LockTableResult::kAlreadyInPending: {
+    return false;
+  }
+  case LockTableResult::kPlacedInPending: {
+    const TransactionTableResult transaction_result =
+      transaction_table_->putPendingEntry(tid, rid, access_mode);
+    CHECK(transaction_result == TransactionTableResult::kPlacedInPending)
+        << "Unexpected result in LockManager.acquireLockInternal: "
+           "Mismatch of table results: LockTable entry is pending, "
+           "whereas TransactionTable entry is not pending";
+    return false;
+  }
+  default: {
+    return false;
+  }
+  }
+}
+
+void LockManager::killVictims() {
+  if (detector_status_.load() == DeadLockDetectorStatus::kDone) {
+    lock_table_->latchExclusive();
+    for (const auto victim_transaction_id : victim_result_) {
+      releaseAllLocks(victim_transaction_id, false);
+      // TODO(Hakan): Find a way to kill transaction, so that requests with this
+      //              tid should be ignored.
+      LOG(INFO) << "Killed transaction "
+                << std::to_string(victim_transaction_id);
+    }
+    lock_table_->unlatchExclusive();
+  }
+  victim_result_.clear();
+  detector_status_.store(DeadLockDetectorStatus::kNotReady);
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/LockManager.hpp
----------------------------------------------------------------------
diff --git a/transaction/LockManager.hpp b/transaction/LockManager.hpp
new file mode 100644
index 0000000..40ee6c8
--- /dev/null
+++ b/transaction/LockManager.hpp
@@ -0,0 +1,128 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#ifndef QUICKSTEP_TRANSACTION_LOCK_MANAGER_HPP_
+#define QUICKSTEP_TRANSACTION_LOCK_MANAGER_HPP_
+
+#include <atomic>
+#include <memory>
+#include <queue>
+#include <vector>
+
+#include "threading/Thread.hpp"
+#include "transaction/DeadLockDetector.hpp"
+#include "transaction/LockRequest.hpp"
+#include "transaction/Transaction.hpp"
+
+namespace quickstep {
+
+template <typename T> class ThreadSafeQueue;
+
+namespace transaction {
+
+class AccessMode;
+class LockTable;
+class ResourceId;
+class TransactionTable;
+
+/** \addtogroup Transaction
+ *  @{
+ **/
+
+/**
+ * @brief Class for centralized location of acquisition and releasing
+ *        of resource locks.
+ **/
+class LockManager : public Thread {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param incoming_requests Queue for the lock requests that are waiting
+   *        for the permission.
+   * @param permitted_requests Queue for the lock requests that are granted
+   *        permission.
+   **/
+  LockManager(ThreadSafeQueue<LockRequest> *incoming_requests,
+              ThreadSafeQueue<LockRequest> *permitted_requests);
+
+  /**
+   * @brief Destructor for LockManager. It handles the thread
+   *        joins that it owns.
+   **/
+  ~LockManager();
+
+  /**
+   * @brief Method for defining the LockManager's thread main logic.
+   *
+   * @warning Users must not use this function directly. Instead use
+   *          start() method inherited from the Thread class.
+   **/
+  void run() override;
+
+  /**
+   * @brief Acquires the lock on resource with specified access mode.
+   *
+   * @param tid Id of the transaction which the resource lock is acquired for.
+   * @param rid Id of the resource on which the resource lock is acquired.
+   * @param access_mode Permissible access mode on resource.
+   *
+   * @return True if it can acquire the lock from root to leaf lock hierarchy,
+   *         false otherwise.
+   **/
+  bool acquireLock(const transaction_id tid,
+                   const ResourceId &rid,
+                   const AccessMode &access_mode);
+
+  /**
+   * @brief Releases all locks hold by the transaction.
+   *
+   * @param tid Id of the transaction whose locks will be released.
+   * @param latch_table If it is true, the method latch the whole
+   *        lock table, which is default.
+   * @return True if tid releases all of its locks.
+   **/
+  bool releaseAllLocks(const transaction_id tid,
+                       const bool latch_table = true);
+
+  /**
+   * @brief Release the locks acquired by the transactions contained
+   *        in victim buffer to break the deadlock.
+   **/
+  void killVictims();
+
+ private:
+  bool acquireLockInternal(const transaction_id tid,
+                           const ResourceId &rid,
+                           const AccessMode &access_mode);
+
+  std::unique_ptr<LockTable> lock_table_;
+  std::unique_ptr<TransactionTable> transaction_table_;
+  std::atomic<DeadLockDetectorStatus> detector_status_;
+  std::vector<transaction_id> victim_result_;
+  std::unique_ptr<DeadLockDetector> deadlock_detector_;
+  ThreadSafeQueue<LockRequest> &incoming_requests_;
+  ThreadSafeQueue<LockRequest> &permitted_requests_;
+  std::queue<LockRequest> inner_pending_requests_;
+};
+
+/** @} */
+
+}  // namespace transaction
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TRANSACTION_LOCK_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/LockTable.cpp
----------------------------------------------------------------------
diff --git a/transaction/LockTable.cpp b/transaction/LockTable.cpp
index 77986f6..7b568fc 100644
--- a/transaction/LockTable.cpp
+++ b/transaction/LockTable.cpp
@@ -18,15 +18,11 @@
 #include "transaction/LockTable.hpp"
 
 #include <list>
-#include <unordered_map>
 #include <utility>
 
-#include "threading/SharedMutex.hpp"
 #include "transaction/AccessMode.hpp"
 #include "transaction/Lock.hpp"
-#include "transaction/ResourceId.hpp"
 #include "transaction/Transaction.hpp"
-#include "utility/Macros.hpp"
 
 namespace quickstep {
 namespace transaction {
@@ -34,7 +30,7 @@ namespace transaction {
 LockTableResult
 LockTable::putLock(const transaction_id tid,
                    const ResourceId &rid,
-                   const AccessMode access_mode) {
+                   const AccessMode &access_mode) {
   // TODO(hakan): Lock upgrade is not supported.
   lock_list_pair &lock_list_pair = internal_map_[rid];
 
@@ -47,7 +43,7 @@ LockTable::putLock(const transaction_id tid,
   for (lock_own_list::const_iterator it = lock_own_list.cbegin();
        it != lock_own_list.cend(); ++it) {
     if (it->first == tid && it->second.getAccessMode() == access_mode) {
-      return LockTableResult::kALREADY_IN_OWNED;
+      return LockTableResult::kAlreadyInOwned;
     }
   }
 
@@ -56,7 +52,7 @@ LockTable::putLock(const transaction_id tid,
   for (lock_pending_list::const_iterator it = lock_pending_list.cbegin();
        it != lock_pending_list.cend(); ++it) {
     if (it->first == tid && it->second.getAccessMode() == access_mode) {
-      return LockTableResult::kALREADY_IN_PENDING;
+      return LockTableResult::kAlreadyInPending;
     }
   }
 
@@ -68,18 +64,18 @@ LockTable::putLock(const transaction_id tid,
       if (!access_mode.isCompatible(it->second.getAccessMode())) {
         lock_pending_list.push_back(std::make_pair(tid,
                                                    Lock(rid, access_mode)));
-        return LockTableResult::kPLACED_IN_PENDING;
+        return LockTableResult::kPlacedInPending;
       }
     }
 
     lock_own_list.push_back(std::make_pair(tid, Lock(rid, access_mode)));
-    return LockTableResult::kPLACED_IN_OWNED;
+    return LockTableResult::kPlacedInOwned;
   } else {
     // If the pending list is not empty, even if the lock request is compatible
     // with other owned lock entries, we put the new request into the pending
     // list to eliminate starvation.
     lock_pending_list.push_back(std::make_pair(tid, Lock(rid, access_mode)));
-    return LockTableResult::kPLACED_IN_PENDING;
+    return LockTableResult::kPlacedInPending;
   }
 }
 
@@ -105,7 +101,7 @@ LockTable::deleteLock(const transaction_id tid,
       // compatible with the remaining owned entries.
       movePendingToOwned(rid);
 
-      return LockTableResult::kDEL_FROM_OWNED;
+      return LockTableResult::kDeleteFromOwned;
     }
   }
 
@@ -116,13 +112,13 @@ LockTable::deleteLock(const transaction_id tid,
     if (it->first == tid) {
       // If it exists, erase it from pending list.
       lock_pending_list.erase(it);
-      return LockTableResult::kDEL_FROM_PENDING;
+      return LockTableResult::kDeleteFromPending;
     }
   }
 
   // Execution reaches here, if we cannot find the corresponding lock entry
   // in the both list.
-  return LockTableResult::kDEL_ERROR;
+  return LockTableResult::kDeleteError;
 }
 
 void LockTable::movePendingToOwned(const ResourceId &rid) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/LockTable.hpp
----------------------------------------------------------------------
diff --git a/transaction/LockTable.hpp b/transaction/LockTable.hpp
index 5a0612e..529db12 100644
--- a/transaction/LockTable.hpp
+++ b/transaction/LockTable.hpp
@@ -23,7 +23,6 @@
 #include <utility>
 
 #include "threading/SharedMutex.hpp"
-#include "transaction/AccessMode.hpp"
 #include "transaction/Lock.hpp"
 #include "transaction/ResourceId.hpp"
 #include "transaction/Transaction.hpp"
@@ -32,6 +31,8 @@
 namespace quickstep {
 namespace transaction {
 
+class AccessMode;
+
 /** \addtogroup Transaction
  * @{
  */
@@ -40,14 +41,14 @@ namespace transaction {
  * @brief Represents different results for LockTable's methods.
  **/
 enum class LockTableResult {
-  kPLACED_IN_OWNED = 0,
-  kPLACED_IN_PENDING,
-  kALREADY_IN_OWNED,
-  kALREADY_IN_PENDING,
-  kDEL_FROM_OWNED,
-  kDEL_FROM_PENDING,
-  kDEL_ERROR,
-  kPUT_ERROR,
+  kPlacedInOwned = 0,
+  kPlacedInPending,
+  kAlreadyInOwned,
+  kAlreadyInPending,
+  kDeleteFromOwned,
+  kDeleteFromPending,
+  kDeleteError,
+  kPutError,
 };
 
 /**
@@ -79,32 +80,31 @@ class LockTable {
    * @param rid Id of the resource to be locked.
    * @param access_mode Access mode of the lock.
    *
-   * @return LockTableResult::kPLACED_IN_OWNED if lock is granted,
-   *         LockTableResult::kPLACED_IN_PENDING if lock is not granted,
-   *         LockTableResult::kALREADY_IN_OWNED if lock has been
+   * @return LockTableResult::kPlacedInOwned if lock is granted,
+   *         LockTableResult::kPlacedInPending if lock is not granted,
+   *         LockTableResult::kAlreadyInOwned if lock has been
    *         already granted,
-   *         LockTableResult::kALREADY_IN_PENDING if lock has been
+   *         LockTableResult::kAlreadyInPending if lock has been
    *         already pending.
    **/
   LockTableResult putLock(const transaction_id tid,
                           const ResourceId &rid,
-                          const AccessMode access_mode);
+                          const AccessMode &access_mode);
   /**
    * @brief Deletes the lock entry.
    *
    * @param tid Id of the transaction that owns or awaits.
    * @param rid Id of resource that the lock covers.
    *
-   * @return LockTableResult::kDEL_FROM_OWNED if the lock is deleted from
+   * @return LockTableResult::kDelFromOwned if the lock is deleted from
    *         owned list,
-   *         LockTableResult::kDEL_FROM_PENDING if the lock is deleted from
+   *         LockTableResult::kDelFromPending if the lock is deleted from
    *         pending list,
-   *         LockTableResult::kDEL_ERROR if the lock cannot be found
+   *         LockTableResult::kDelError if the lock cannot be found
    **/
   LockTableResult deleteLock(const transaction_id tid,
                              const ResourceId &rid);
 
-
   /**
    * @brief Iterator for begin position.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/ResourceId.hpp
----------------------------------------------------------------------
diff --git a/transaction/ResourceId.hpp b/transaction/ResourceId.hpp
index b9d1cdf..3a770dd 100644
--- a/transaction/ResourceId.hpp
+++ b/transaction/ResourceId.hpp
@@ -19,7 +19,6 @@
 #define QUICKSTEP_TRANSACTION_RESOURCE_ID_HPP_
 
 #include <cstddef>
-#include <limits>
 #include <string>
 
 #include "catalog/CatalogTypedefs.hpp"
@@ -202,10 +201,10 @@ class ResourceId {
     return tuple_id_ == kTupleIdPlaceholder;
   }
 
-  const database_id db_id_;
-  const relation_id rel_id_;
-  const block_id block_id_;
-  const tuple_id tuple_id_;
+  database_id db_id_;
+  relation_id rel_id_;
+  block_id block_id_;
+  tuple_id tuple_id_;
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/StronglyConnectedComponents.cpp
----------------------------------------------------------------------
diff --git a/transaction/StronglyConnectedComponents.cpp b/transaction/StronglyConnectedComponents.cpp
index 89daf46..f50ed85 100644
--- a/transaction/StronglyConnectedComponents.cpp
+++ b/transaction/StronglyConnectedComponents.cpp
@@ -17,7 +17,6 @@
 
 #include "transaction/StronglyConnectedComponents.hpp"
 
-#include <cstddef>
 #include <cstdint>
 #include <stack>
 #include <unordered_map>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/Transaction.cpp
----------------------------------------------------------------------
diff --git a/transaction/Transaction.cpp b/transaction/Transaction.cpp
deleted file mode 100644
index 3478d01..0000000
--- a/transaction/Transaction.cpp
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *     University of Wisconsin\u2014Madison.
- *
- *   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.
- **/
-
-#include "transaction/Transaction.hpp"
-
-#include <functional>
-
-namespace quickstep {
-
-namespace transaction {
-
-TransactionId Transaction::getTransactionId() const {
-  return tid_;
-}
-
-void Transaction::setStatus(TransactionStatus status) {
-  status_ = status;
-}
-
-TransactionStatus Transaction::getStatus() const {
-  return status_;
-}
-
-bool Transaction::operator==(const Transaction &other) const {
-  return tid_ == other.tid_;
-}
-
-std::size_t Transaction::TransactionHasher::operator()(const Transaction &transaction) const {
-  return std::hash<TransactionId>()(transaction.tid_);
-}
-
-}  // namespace transaction
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/TransactionTable.cpp
----------------------------------------------------------------------
diff --git a/transaction/TransactionTable.cpp b/transaction/TransactionTable.cpp
index 993703a..3e37439 100644
--- a/transaction/TransactionTable.cpp
+++ b/transaction/TransactionTable.cpp
@@ -33,7 +33,7 @@ namespace transaction {
 TransactionTableResult
 TransactionTable::putOwnEntry(const transaction_id tid,
                               const ResourceId &rid,
-                              const AccessMode access_mode) {
+                              const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_own_list &transaction_own_list = transaction_list_pair.first;
 
@@ -45,7 +45,7 @@ TransactionTable::putOwnEntry(const transaction_id tid,
 TransactionTableResult
 TransactionTable::putPendingEntry(const transaction_id tid,
                                   const ResourceId &rid,
-                                  const AccessMode access_mode) {
+                                  const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_pending_list &transaction_pending_list
       = transaction_list_pair.second;
@@ -59,7 +59,7 @@ TransactionTable::putPendingEntry(const transaction_id tid,
 TransactionTableResult
 TransactionTable::deleteOwnEntry(const transaction_id tid,
                                  const ResourceId &rid,
-                                 const AccessMode access_mode) {
+                                 const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_own_list &transaction_own_list = transaction_list_pair.first;
 
@@ -79,7 +79,7 @@ TransactionTable::deleteOwnEntry(const transaction_id tid,
 TransactionTableResult
 TransactionTable::deletePendingEntry(const transaction_id tid,
                                      const ResourceId &rid,
-                                     const AccessMode access_mode) {
+                                     const AccessMode &access_mode) {
   transaction_list_pair &transaction_list_pair = internal_map_[tid];
   transaction_pending_list &transaction_pending_list
       = transaction_list_pair.second;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/TransactionTable.hpp
----------------------------------------------------------------------
diff --git a/transaction/TransactionTable.hpp b/transaction/TransactionTable.hpp
index a5e1da4..29df536 100644
--- a/transaction/TransactionTable.hpp
+++ b/transaction/TransactionTable.hpp
@@ -23,7 +23,6 @@
 #include <utility>
 #include <vector>
 
-#include "transaction/AccessMode.hpp"
 #include "transaction/Lock.hpp"
 #include "transaction/ResourceId.hpp"
 #include "transaction/Transaction.hpp"
@@ -32,6 +31,8 @@
 namespace quickstep {
 namespace transaction {
 
+class AccessMode;
+
 /** \addtogroup Transaction
  *  @{
  */
@@ -81,7 +82,7 @@ class TransactionTable {
    **/
   TransactionTableResult putOwnEntry(const transaction_id tid,
                                      const ResourceId &rid,
-                                     const AccessMode access_mode);
+                                     const AccessMode &access_mode);
 
   /**
    * @brief Puts a pending entry of the given resource id in the given
@@ -95,7 +96,7 @@ class TransactionTable {
    **/
   TransactionTableResult putPendingEntry(const transaction_id tid,
                                          const ResourceId &rid,
-                                         const AccessMode access_mode);
+                                         const AccessMode &access_mode);
 
   /**
    * @brief Deletes the owned entry corresponding to the resource id
@@ -110,7 +111,7 @@ class TransactionTable {
    **/
   TransactionTableResult deleteOwnEntry(const transaction_id tid,
                                         const ResourceId &rid,
-                                        const AccessMode access_mode);
+                                        const AccessMode &access_mode);
 
   /**
    * @brief Deletes the pending entry corresponding to the resource id
@@ -124,7 +125,7 @@ class TransactionTable {
    **/
   TransactionTableResult deletePendingEntry(const transaction_id tid,
                                             const ResourceId &rid,
-                                            const AccessMode access_mode);
+                                            const AccessMode &access_mode);
 
   /**
    * @brief Returns a vector of resource ids which the corresponding transaction

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/tests/AccessMode_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/AccessMode_unittest.cpp b/transaction/tests/AccessMode_unittest.cpp
index fa51525..3287fb0 100644
--- a/transaction/tests/AccessMode_unittest.cpp
+++ b/transaction/tests/AccessMode_unittest.cpp
@@ -25,12 +25,12 @@ namespace transaction {
 class AccessModeTest : public ::testing::Test {
  protected:
   AccessModeTest()
-      : nl_mode_(AccessModeType::kNoLock),
-        is_mode_(AccessModeType::kIsLock),
-        ix_mode_(AccessModeType::kIxLock),
-        s_mode_(AccessModeType::kSLock),
-        six_mode_(AccessModeType::kSixLock),
-        x_mode_(AccessModeType::kXLock) {
+      : nl_mode_(AccessMode::NoLockMode()),
+        is_mode_(AccessMode::IsLockMode()),
+        ix_mode_(AccessMode::IxLockMode()),
+        s_mode_(AccessMode::SLockMode()),
+        six_mode_(AccessMode::SixLockMode()),
+        x_mode_(AccessMode::XLockMode()) {
   }
 
   const AccessMode nl_mode_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/tests/CycleDetector_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/CycleDetector_unittest.cpp b/transaction/tests/CycleDetector_unittest.cpp
new file mode 100644
index 0000000..6edaa63
--- /dev/null
+++ b/transaction/tests/CycleDetector_unittest.cpp
@@ -0,0 +1,157 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#include "transaction/CycleDetector.hpp"
+
+#include <cstdint>
+#include <memory>
+#include <stack>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class CycleDetectorTest : public testing::Test {
+ protected:
+  const std::uint64_t kNumberOfTransactions = 12;
+
+  CycleDetectorTest()
+      : wait_for_graph_(std::make_unique<DirectedGraph>()) {
+  }
+
+  virtual void SetUp() {
+    std::vector<transaction_id> transactions(kNumberOfTransactions);
+    for (std::uint64_t i = 0; i < kNumberOfTransactions; ++i) {
+      transactions.push_back(transaction_id(i));
+    }
+
+    std::vector<DirectedGraph::node_id> node_ids;
+    for (std::uint64_t i = 0; i < kNumberOfTransactions; ++i) {
+      node_ids.push_back(wait_for_graph_->addNodeUnchecked(transactions[i]));
+    }
+  }
+
+  void initializeCycleDetector() {
+    for (const auto &edge : edges_) {
+      wait_for_graph_->addEdgeUnchecked(edge.first, edge.second);
+    }
+
+    cycle_detector_.reset(new CycleDetector(wait_for_graph_.get()));
+  }
+
+  void checkVictims(
+      const std::unordered_set<DirectedGraph::node_id> &expected_victims) {
+    const std::vector<DirectedGraph::node_id> victims =
+        cycle_detector_->chooseVictimsToBreakCycle();
+
+    std::unordered_set<DirectedGraph::node_id> remaining_nodes;
+
+    for (DirectedGraph::node_id node = 0; node < wait_for_graph_->getNumNodes();
+         ++node) {
+      if (std::find(victims.begin(), victims.end(), node) == victims.end()) {
+        // Node is not in victims, then insert it to remaining set.
+        remaining_nodes.insert(node);
+      }
+    }
+
+    for (const auto node : remaining_nodes) {
+      ASSERT_FALSE(isSelfReachableNode(node, remaining_nodes));
+    }
+  }
+
+  bool isSelfReachableNode(
+      const DirectedGraph::node_id start_node,
+      const std::unordered_set<DirectedGraph::node_id> &node_set) {
+    std::unordered_set<DirectedGraph::node_id> marked_nodes;
+    std::stack<DirectedGraph::node_id> to_be_visied_nodes;
+
+    const std::vector<DirectedGraph::node_id> neighbors_of_start_node =
+        wait_for_graph_->getAdjacentNodes(start_node);
+    for (const auto node : neighbors_of_start_node) {
+      marked_nodes.insert(node);
+      to_be_visied_nodes.push(node);
+    }
+
+    while (!to_be_visied_nodes.empty()) {
+      const DirectedGraph::node_id current_node = to_be_visied_nodes.top();
+      to_be_visied_nodes.pop();
+      if (current_node == start_node) {
+        return true;
+      }
+      if (node_set.count(current_node) == 1 &&
+          marked_nodes.count(current_node) == 0) {
+        // Means, we did not visited this node yet, and it is in the node set,
+        // so we should process it (mark it and push all of its neighbors
+        // into stack).
+        marked_nodes.insert(current_node);
+        const auto neighbors = wait_for_graph_->getAdjacentNodes(current_node);
+        for (const auto neighbor : neighbors) {
+          to_be_visied_nodes.push(neighbor);
+        }
+      }
+    }
+    return false;
+  }
+
+  std::vector<std::pair<DirectedGraph::node_id, DirectedGraph::node_id>> edges_;
+  std::unique_ptr<DirectedGraph> wait_for_graph_;
+  std::unique_ptr<CycleDetector> cycle_detector_;
+};
+
+TEST_F(CycleDetectorTest, Interleaving) {
+  edges_ = {{0, 1},
+            {1, 0}};
+
+  initializeCycleDetector();
+
+  std::unordered_set<DirectedGraph::node_id> expected_victims = {1};
+
+  checkVictims(expected_victims);
+}
+
+TEST_F(CycleDetectorTest, MultipleCycle) {
+  // This edge contains lots of cycles of degree 1, 2 and 3.
+  edges_ = {{0, 1},
+            {1, 2}, {1, 3}, {1, 4},
+            {2, 5},
+            {3, 4}, {3, 6},
+            {4, 1}, {4, 5}, {4, 6},
+            {5, 2}, {5, 7},
+            {6, 7}, {6, 9},
+            {7, 6},
+            {8, 6},
+            {9, 8}, {9, 10},
+            {10, 11},
+            {11, 9}};
+
+  initializeCycleDetector();
+
+  std::unordered_set<DirectedGraph::node_id> expected_victims
+      = {4, 5, 7, 8, 9, 10, 11};
+
+  checkVictims(expected_victims);
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/tests/DeadLockDetector_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/DeadLockDetector_unittest.cpp b/transaction/tests/DeadLockDetector_unittest.cpp
new file mode 100644
index 0000000..bc65ef5
--- /dev/null
+++ b/transaction/tests/DeadLockDetector_unittest.cpp
@@ -0,0 +1,96 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   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.
+ **/
+
+#include "transaction/DeadLockDetector.hpp"
+
+#include <atomic>
+#include <memory>
+#include <vector>
+
+#include "transaction/AccessMode.hpp"
+#include "transaction/DirectedGraph.hpp"
+#include "transaction/LockTable.hpp"
+#include "transaction/ResourceId.hpp"
+#include "transaction/Transaction.hpp"
+
+#include "gtest/gtest.h"
+
+namespace quickstep {
+namespace transaction {
+
+class DeadLockDetectorTest : public ::testing::Test {
+ protected:
+  DeadLockDetectorTest()
+      : lock_table_(std::make_unique<LockTable>()),
+        status_(DeadLockDetectorStatus::kDone) {
+  }
+
+  std::unique_ptr<LockTable> lock_table_;
+  std::atomic<DeadLockDetectorStatus> status_;
+  std::vector<DirectedGraph::node_id> victims_;
+};
+
+TEST_F(DeadLockDetectorTest, SimpleCycle) {
+  const transaction_id transaction_one(1), transaction_two(2);
+  const ResourceId resource_one(1, 2), resource_two(4, 5);
+
+  const AccessMode x_lock_mode(AccessMode::XLockMode());
+
+  // Produce a conflicting schedule.
+  // Transaction 1 will acquire X lock on resource 1.
+  lock_table_->putLock(transaction_one,
+                       resource_one,
+                       x_lock_mode);
+
+  // Transaction 2 will acquire X lock on resource 2.
+  lock_table_->putLock(transaction_two,
+                       resource_two,
+                       x_lock_mode);
+
+  // Transaction 1 will try to acquire X lock on resource 2,
+  // but it will fail since Transaction 2 has already acquired
+  // X lock on resource 2.
+  lock_table_->putLock(transaction_one,
+                       resource_two,
+                       x_lock_mode);
+
+  // Transaction 2 will try to acquire X lock on resource 1,
+  // but it will fail since Transaction 1 has already acquired
+  // X lock on resource 2.
+  lock_table_->putLock(transaction_two,
+                       resource_one,
+                       x_lock_mode);
+
+  // Run deadlock detector.
+  DeadLockDetector deadlock_detector(lock_table_.get(), &status_, &victims_);
+  status_.store(DeadLockDetectorStatus::kNotReady);
+
+  deadlock_detector.start();
+
+  // Signal deadlock detector.
+  while (status_.load() == DeadLockDetectorStatus::kNotReady) {
+  }
+
+  status_.store(DeadLockDetectorStatus::kQuit);
+  deadlock_detector.join();
+
+  // Victim size must be 1.
+  ASSERT_EQ(1u, victims_.size());
+}
+
+}  // namespace transaction
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/tests/DirectedGraph_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/DirectedGraph_unittest.cpp b/transaction/tests/DirectedGraph_unittest.cpp
index 43ad972..00fe276 100644
--- a/transaction/tests/DirectedGraph_unittest.cpp
+++ b/transaction/tests/DirectedGraph_unittest.cpp
@@ -26,101 +26,96 @@
 namespace quickstep {
 namespace transaction {
 
-TEST(DirectedGraphTest, AddNode) {
-  // Prepare the data, but do not include in the graph.
-  DirectedGraph wait_for_graph;
-  transaction_id *tid3 = new transaction_id(3);
-  transaction_id *tid4 = new transaction_id(4);
-  transaction_id *tid5 = new transaction_id(5);
-  transaction_id *tid6 = new transaction_id(6);
-
+class DirectedGraphTest : public ::testing::Test {
+ protected:
+  DirectedGraphTest()
+      : tid3_(3),
+        tid4_(4),
+        tid5_(5),
+        tid6_(6) {
+  }
+
+  DirectedGraph wait_for_graph_;
+  transaction_id tid3_;
+  transaction_id tid4_;
+  transaction_id tid5_;
+  transaction_id tid6_;
+};
+
+TEST_F(DirectedGraphTest, AddNode) {
   // The nodes are not added yet, total no of nodesshould be zero.
-  EXPECT_EQ(0u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(0u, wait_for_graph_.getNumNodes());
 
-  wait_for_graph.addNodeUnchecked(tid3);
+  wait_for_graph_.addNodeUnchecked(tid3_);
 
   // One node is added.
-  EXPECT_EQ(1u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(1u, wait_for_graph_.getNumNodes());
 
-  wait_for_graph.addNodeUnchecked(tid4);
+  wait_for_graph_.addNodeUnchecked(tid4_);
 
   // Another node is added.
-  EXPECT_EQ(2u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(2u, wait_for_graph_.getNumNodes());
 
-  wait_for_graph.addNodeUnchecked(tid5);
-  wait_for_graph.addNodeUnchecked(tid6);
+  wait_for_graph_.addNodeUnchecked(tid5_);
+  wait_for_graph_.addNodeUnchecked(tid6_);
 
   // Total no of nodes should be 4 right now.
-  EXPECT_EQ(4u, wait_for_graph.getNumNodes());
+  EXPECT_EQ(4u, wait_for_graph_.getNumNodes());
 }
 
-TEST(DirectedGraphTest, AddEdge) {
-  // Prepare the graph.
-  DirectedGraph wait_for_graph;
-  transaction_id *tid3 = new transaction_id(3);
-  transaction_id *tid4 = new transaction_id(4);
-  transaction_id *tid5 = new transaction_id(5);
-  transaction_id *tid6 = new transaction_id(6);
-
-  DirectedGraph::node_id nid3 = wait_for_graph.addNodeUnchecked(tid3);
-  DirectedGraph::node_id nid6 = wait_for_graph.addNodeUnchecked(tid6);
-  DirectedGraph::node_id nid4 = wait_for_graph.addNodeUnchecked(tid4);
-  DirectedGraph::node_id nid5 = wait_for_graph.addNodeUnchecked(tid5);
+TEST_F(DirectedGraphTest, AddEdge) {
+  DirectedGraph::node_id nid3 = wait_for_graph_.addNodeUnchecked(tid3_);
+  DirectedGraph::node_id nid6 = wait_for_graph_.addNodeUnchecked(tid6_);
+  DirectedGraph::node_id nid4 = wait_for_graph_.addNodeUnchecked(tid4_);
+  DirectedGraph::node_id nid5 = wait_for_graph_.addNodeUnchecked(tid5_);
 
   // Add edges.
-  wait_for_graph.addEdgeUnchecked(nid3, nid5);
-  wait_for_graph.addEdgeUnchecked(nid6, nid4);
-  wait_for_graph.addEdgeUnchecked(nid3, nid6);
-  wait_for_graph.addEdgeUnchecked(nid4, nid6);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid5);
+  wait_for_graph_.addEdgeUnchecked(nid6, nid4);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid6);
+  wait_for_graph_.addEdgeUnchecked(nid4, nid6);
 
   // Check whether the edges are already there.
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid3, nid5));
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid6, nid4));
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid4, nid6));
-  EXPECT_TRUE(wait_for_graph.hasEdge(nid3, nid6));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid3, nid5));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid6, nid4));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid4, nid6));
+  EXPECT_TRUE(wait_for_graph_.hasEdge(nid3, nid6));
 
   // Check non-existent edges.
-  EXPECT_FALSE(wait_for_graph.hasEdge(nid5, nid3));
-  EXPECT_FALSE(wait_for_graph.hasEdge(nid6, nid3));
-  EXPECT_FALSE(wait_for_graph.hasEdge(nid4, nid5));
+  EXPECT_FALSE(wait_for_graph_.hasEdge(nid5, nid3));
+  EXPECT_FALSE(wait_for_graph_.hasEdge(nid6, nid3));
+  EXPECT_FALSE(wait_for_graph_.hasEdge(nid4, nid5));
 }
 
-TEST(DirectedGraphTest, GetAdjacentNodes) {
-  // Prepare the graph.
-  DirectedGraph wait_for_graph;
-  transaction_id *tid3 = new transaction_id(3);
-  transaction_id *tid4 = new transaction_id(4);
-  transaction_id *tid5 = new transaction_id(5);
-  transaction_id *tid6 = new transaction_id(6);
-
+TEST_F(DirectedGraphTest, GetAdjacentNodes) {
   // Add 4 disconnected nodes to the graph.
-  DirectedGraph::node_id nid3 = wait_for_graph.addNodeUnchecked(tid3);
-  DirectedGraph::node_id nid6 = wait_for_graph.addNodeUnchecked(tid6);
-  DirectedGraph::node_id nid4 = wait_for_graph.addNodeUnchecked(tid4);
-  DirectedGraph::node_id nid5 = wait_for_graph.addNodeUnchecked(tid5);
+  DirectedGraph::node_id nid3 = wait_for_graph_.addNodeUnchecked(tid3_);
+  DirectedGraph::node_id nid6 = wait_for_graph_.addNodeUnchecked(tid6_);
+  DirectedGraph::node_id nid4 = wait_for_graph_.addNodeUnchecked(tid4_);
+  DirectedGraph::node_id nid5 = wait_for_graph_.addNodeUnchecked(tid5_);
 
-  std::vector<DirectedGraph::node_id> result1 = wait_for_graph.getAdjacentNodes(nid3);
+  std::vector<DirectedGraph::node_id> result1 = wait_for_graph_.getAdjacentNodes(nid3);
   // nid3 has no edge to other nodes.
   EXPECT_EQ(0u, result1.size());
 
   // Now nid3 has outgoing edge to nid4 and nid5.
-  wait_for_graph.addEdgeUnchecked(nid3, nid4);
-  wait_for_graph.addEdgeUnchecked(nid3, nid5);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid4);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid5);
 
-  std::vector<DirectedGraph::node_id> result2 = wait_for_graph.getAdjacentNodes(nid3);
+  std::vector<DirectedGraph::node_id> result2 = wait_for_graph_.getAdjacentNodes(nid3);
   // Therefore, number of outgoing edges from nid3 is 2.
   EXPECT_EQ(2u, result2.size());
 
   // Add an edge from nid3 to nid6.
-  wait_for_graph.addEdgeUnchecked(nid3, nid6);
-  std::vector<DirectedGraph::node_id> result3 = wait_for_graph.getAdjacentNodes(nid3);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid6);
+  std::vector<DirectedGraph::node_id> result3 = wait_for_graph_.getAdjacentNodes(nid3);
 
   // Now there are 3 outgoing edges.
   EXPECT_EQ(3u, result3.size());
 
   // Again add edge from nid3 to nid6.
-  wait_for_graph.addEdgeUnchecked(nid3, nid6);
-  std::vector<DirectedGraph::node_id> result4 = wait_for_graph.getAdjacentNodes(nid3);
+  wait_for_graph_.addEdgeUnchecked(nid3, nid6);
+  std::vector<DirectedGraph::node_id> result4 = wait_for_graph_.getAdjacentNodes(nid3);
   // Since we have already add same edge before, number of edges are still 3.
   EXPECT_EQ(3u, result4.size());
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/tests/LockRequest_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/LockRequest_unittest.cpp b/transaction/tests/LockRequest_unittest.cpp
index 0e4138a..77047d9 100644
--- a/transaction/tests/LockRequest_unittest.cpp
+++ b/transaction/tests/LockRequest_unittest.cpp
@@ -31,7 +31,7 @@ class LockRequestTest : public ::testing::Test {
   LockRequestTest()
       : lock_request_(transaction_id(3),
                       ResourceId(5),
-                      AccessMode(AccessModeType::kSLock),
+                      AccessMode::SLockMode(),
                       RequestType::kAcquireLock) {
   }
 
@@ -41,7 +41,7 @@ class LockRequestTest : public ::testing::Test {
 TEST_F(LockRequestTest, CheckGetters) {
   EXPECT_EQ(transaction_id(3), lock_request_.getTransactionId());
   EXPECT_EQ(ResourceId(5), lock_request_.getResourceId());
-  EXPECT_EQ(AccessMode(AccessModeType::kSLock), lock_request_.getAccessMode());
+  EXPECT_EQ(AccessMode::SLockMode(), lock_request_.getAccessMode());
   EXPECT_EQ(RequestType::kAcquireLock, lock_request_.getRequestType());
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/tests/LockTable_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/LockTable_unittest.cpp b/transaction/tests/LockTable_unittest.cpp
index 577cb79..1aed0b8 100644
--- a/transaction/tests/LockTable_unittest.cpp
+++ b/transaction/tests/LockTable_unittest.cpp
@@ -41,64 +41,73 @@ class LockTableTest : public ::testing::Test {
 };
 
 TEST_F(LockTableTest, CompatibleRequestsFromDifferentTransactions) {
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  const AccessMode is_lock_mode = AccessMode::IsLockMode();
+  const AccessMode s_lock_mode = AccessMode::SLockMode();
+
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                is_lock_mode));
 
   // Acquire the same lock mode on same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  EXPECT_EQ(LockTableResult::kAlreadyInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kALREADY_IN_OWNED);
+                                is_lock_mode));
 
   // Another transaction acquires compatible lock on the same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_2_,
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_2_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kSLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                s_lock_mode));
 }
 
 TEST_F(LockTableTest, IncompatibleRequestsFromDifferentTransactions) {
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  const AccessMode is_lock_mode = AccessMode::IsLockMode();
+  const AccessMode x_lock_mode = AccessMode::XLockMode();
+
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                is_lock_mode));
 
   // Acquire the same lock mode on same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  EXPECT_EQ(LockTableResult::kAlreadyInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kALREADY_IN_OWNED);
+                                is_lock_mode));
 
   // Another transaction acquires incompatible lock on the same resource.
-  EXPECT_EQ(lock_table_.putLock(tid_2_,
+  EXPECT_EQ(LockTableResult::kPlacedInPending,
+            lock_table_.putLock(tid_2_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kXLock)),
-            LockTableResult::kPLACED_IN_PENDING);
+                                x_lock_mode));
 }
 
 TEST_F(LockTableTest, StarvationProtection) {
-  EXPECT_EQ(lock_table_.putLock(tid_1_,
+  const AccessMode is_lock_mode = AccessMode::IsLockMode();
+  const AccessMode x_lock_mode = AccessMode::XLockMode();
+
+  EXPECT_EQ(LockTableResult::kPlacedInOwned,
+            lock_table_.putLock(tid_1_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_OWNED);
+                                is_lock_mode));
 
   // Another transaction requests incompatible lock on the same resource.
   // It should wait for the previous transaction.
-  EXPECT_EQ(lock_table_.putLock(tid_2_,
+  EXPECT_EQ(LockTableResult::kPlacedInPending,
+            lock_table_.putLock(tid_2_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kXLock)),
-            LockTableResult::kPLACED_IN_PENDING);
+                                x_lock_mode));
 
   // Another third transaction requests a compatible lock on the same resource.
   // Normally, it should acquire the lock, however, there is a pending
   // transaction waiting on the same resource. To prevent starvation, we should
   // put in the pending list.
-  EXPECT_EQ(lock_table_.putLock(tid_3_,
+  EXPECT_EQ(LockTableResult::kPlacedInPending,
+            lock_table_.putLock(tid_3_,
                                 ResourceId(2),
-                                AccessMode(AccessModeType::kIsLock)),
-            LockTableResult::kPLACED_IN_PENDING);
+                                is_lock_mode));
 }
 
 }  // namespace transaction

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bbaff7a2/transaction/tests/Lock_unittest.cpp
----------------------------------------------------------------------
diff --git a/transaction/tests/Lock_unittest.cpp b/transaction/tests/Lock_unittest.cpp
index 2ab8b3e..59a5e7a 100644
--- a/transaction/tests/Lock_unittest.cpp
+++ b/transaction/tests/Lock_unittest.cpp
@@ -18,6 +18,7 @@
 #include "transaction/Lock.hpp"
 
 #include <cstddef>
+#include <memory>
 #include <vector>
 
 #include "transaction/AccessMode.hpp"
@@ -31,12 +32,12 @@ namespace transaction {
 class LockTest : public ::testing::Test {
  protected:
   LockTest()
-      : modes_({AccessMode(AccessModeType::kNoLock),
-                AccessMode(AccessModeType::kIsLock),
-                AccessMode(AccessModeType::kIxLock),
-                AccessMode(AccessModeType::kSLock),
-                AccessMode(AccessModeType::kSixLock),
-                AccessMode(AccessModeType::kXLock)}),
+      : modes_({AccessMode::NoLockMode(),
+                AccessMode::IsLockMode(),
+                AccessMode::IxLockMode(),
+                AccessMode::SLockMode(),
+                AccessMode::SixLockMode(),
+                AccessMode::XLockMode()}),
         resource_a_(3, 10, 2, 5),
         resource_b_(4, 5, 3, 2),
         locks_on_resource_a_({Lock(resource_a_, modes_[0]),