diff --git a/src/database/distributed_graph_db.cpp b/src/database/distributed_graph_db.cpp
index b52eab96e..d9ef38d81 100644
--- a/src/database/distributed_graph_db.cpp
+++ b/src/database/distributed_graph_db.cpp
@@ -31,6 +31,188 @@ using namespace std::literals::chrono_literals;
 
 namespace database {
 
+// Accessors
+namespace {
+
+class DistributedAccessor : public GraphDbAccessor {
+  distributed::UpdatesRpcClients *updates_clients_{nullptr};
+  distributed::DataManager *data_manager_{nullptr};
+
+ protected:
+  DistributedAccessor(DistributedGraphDb *db, tx::TransactionId tx_id)
+      : GraphDbAccessor(*db, tx_id),
+        updates_clients_(&db->updates_clients()),
+        data_manager_(&db->data_manager()) {}
+
+  explicit DistributedAccessor(DistributedGraphDb *db)
+      : GraphDbAccessor(*db),
+        updates_clients_(&db->updates_clients()),
+        data_manager_(&db->data_manager()) {}
+
+ public:
+  bool RemoveVertex(VertexAccessor &vertex_accessor,
+                    bool check_empty = true) override {
+    if (!vertex_accessor.is_local()) {
+      auto address = vertex_accessor.address();
+      updates_clients_->RemoveVertex(address.worker_id(), transaction_id(),
+                                     address.gid(), check_empty);
+      // We can't know if we are going to be able to remove vertex until
+      // deferred updates on a remote worker are executed
+      return true;
+    }
+    return GraphDbAccessor::RemoveVertex(vertex_accessor, check_empty);
+  }
+
+  void RemoveEdge(EdgeAccessor &edge, bool remove_out_edge = true,
+                  bool remove_in_edge = true) override {
+    if (edge.is_local()) {
+      return GraphDbAccessor::RemoveEdge(edge, remove_out_edge, remove_in_edge);
+    }
+    auto edge_addr = edge.GlobalAddress();
+    auto from_addr = db().storage().GlobalizedAddress(edge.from_addr());
+    CHECK(edge_addr.worker_id() == from_addr.worker_id())
+        << "Edge and it's 'from' vertex not on the same worker";
+    auto to_addr = db().storage().GlobalizedAddress(edge.to_addr());
+    updates_clients_->RemoveEdge(transaction_id(), edge_addr.worker_id(),
+                                 edge_addr.gid(), from_addr.gid(), to_addr);
+    // Another RPC is necessary only if the first did not handle vertices on
+    // both sides.
+    if (edge_addr.worker_id() != to_addr.worker_id()) {
+      updates_clients_->RemoveInEdge(transaction_id(), to_addr.worker_id(),
+                                     to_addr.gid(), edge_addr);
+    }
+  }
+
+  storage::EdgeAddress InsertEdgeOnFrom(
+      VertexAccessor *from, VertexAccessor *to,
+      const storage::EdgeType &edge_type,
+      const std::experimental::optional<gid::Gid> &requested_gid,
+      const std::experimental::optional<int64_t> &cypher_id) override {
+    if (from->is_local()) {
+      return GraphDbAccessor::InsertEdgeOnFrom(from, to, edge_type,
+                                               requested_gid, cypher_id);
+    }
+    auto edge_address =
+        updates_clients_->CreateEdge(transaction_id(), *from, *to, edge_type);
+    auto *from_updated =
+        data_manager_->Elements<Vertex>(transaction_id()).FindNew(from->gid());
+    // Create an Edge and insert it into the Cache so we see it locally.
+    data_manager_->Elements<Edge>(transaction_id())
+        .emplace(
+            edge_address.gid(), nullptr,
+            std::make_unique<Edge>(from->address(), to->address(), edge_type));
+    from_updated->out_.emplace(
+        db().storage().LocalizedAddressIfPossible(to->address()), edge_address,
+        edge_type);
+    return edge_address;
+  }
+
+  void InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to,
+                      const storage::EdgeType &edge_type,
+                      const storage::EdgeAddress &edge_address) override {
+    if (to->is_local()) {
+      return GraphDbAccessor::InsertEdgeOnTo(from, to, edge_type, edge_address);
+    }
+    // The RPC call for the `to` side is already handled if `from` is not
+    // local.
+    if (from->is_local() ||
+        from->address().worker_id() != to->address().worker_id()) {
+      updates_clients_->AddInEdge(
+          transaction_id(), *from,
+          db().storage().GlobalizedAddress(edge_address), *to, edge_type);
+    }
+    auto *to_updated =
+        data_manager_->Elements<Vertex>(transaction_id()).FindNew(to->gid());
+    to_updated->in_.emplace(
+        db().storage().LocalizedAddressIfPossible(from->address()),
+        edge_address, edge_type);
+  }
+};
+
+class MasterAccessor final : public DistributedAccessor {
+  distributed::IndexRpcClients *index_rpc_clients_{nullptr};
+  int worker_id_{0};
+
+ public:
+  explicit MasterAccessor(Master *db,
+                          distributed::IndexRpcClients *index_rpc_clients)
+      : DistributedAccessor(db),
+        index_rpc_clients_(index_rpc_clients),
+        worker_id_(db->WorkerId()) {}
+  MasterAccessor(Master *db, tx::TransactionId tx_id,
+                 distributed::IndexRpcClients *index_rpc_clients)
+      : DistributedAccessor(db, tx_id),
+        index_rpc_clients_(index_rpc_clients),
+        worker_id_(db->WorkerId()) {}
+
+  void PostCreateIndex(const LabelPropertyIndex::Key &key) override {
+    std::experimental::optional<std::vector<utils::Future<bool>>>
+        index_rpc_completions;
+
+    // Notify all workers to create the index
+    index_rpc_completions.emplace(index_rpc_clients_->GetCreateIndexFutures(
+        key.label_, key.property_, worker_id_));
+
+    if (index_rpc_completions) {
+      // Wait first, check later - so that every thread finishes and none
+      // terminates - this can probably be optimized in case we fail early so
+      // that we notify other workers to stop building indexes
+      for (auto &index_built : *index_rpc_completions) index_built.wait();
+      for (auto &index_built : *index_rpc_completions) {
+        if (!index_built.get()) {
+          db().storage().label_property_index().DeleteIndex(key);
+          throw IndexCreationOnWorkerException("Index exists on a worker");
+        }
+      }
+    }
+  }
+
+  void PopulateIndexFromBuildIndex(
+      const LabelPropertyIndex::Key &key) override {
+    // Notify all workers to start populating an index if we are the master
+    // since they don't have to wait anymore
+    std::experimental::optional<std::vector<utils::Future<bool>>>
+        index_rpc_completions;
+    index_rpc_completions.emplace(index_rpc_clients_->GetPopulateIndexFutures(
+        key.label_, key.property_, transaction_id(), worker_id_));
+
+    // Populate our own storage
+    GraphDbAccessor::PopulateIndexFromBuildIndex(key);
+
+    // Check if all workers successfully built their indexes and after this we
+    // can set the index as built
+    if (index_rpc_completions) {
+      // Wait first, check later - so that every thread finishes and none
+      // terminates - this can probably be optimized in case we fail early so
+      // that we notify other workers to stop building indexes
+      for (auto &index_built : *index_rpc_completions) index_built.wait();
+      for (auto &index_built : *index_rpc_completions) {
+        if (!index_built.get()) {
+          db().storage().label_property_index().DeleteIndex(key);
+          throw IndexCreationOnWorkerException("Index exists on a worker");
+        }
+      }
+    }
+  }
+};
+
+class WorkerAccessor final : public DistributedAccessor {
+ public:
+  explicit WorkerAccessor(Worker *db) : DistributedAccessor(db) {}
+  WorkerAccessor(Worker *db, tx::TransactionId tx_id)
+      : DistributedAccessor(db, tx_id) {}
+
+  void BuildIndex(storage::Label, storage::Property) override {
+    // TODO: Rethink BuildIndex API or inheritance. It's rather strange that a
+    // derived type blocks this functionality.
+    LOG(FATAL) << "BuildIndex invoked on worker.";
+  }
+};
+
+}  // namespace
+
+// GraphDb implementations
+
 namespace impl {
 
 template <template <typename TId> class TMapper>
@@ -80,12 +262,12 @@ class Master {
   distributed::BfsRpcClients bfs_subcursor_clients_{
       self_, &subcursor_storage_, &rpc_worker_clients_, &data_manager_};
   distributed::DurabilityRpcMaster durability_rpc_{rpc_worker_clients_};
-  distributed::DataRpcServer data_server_{*self_, server_};
+  distributed::DataRpcServer data_server_{self_, &server_};
   distributed::DataRpcClients data_clients_{rpc_worker_clients_};
   distributed::PlanDispatcher plan_dispatcher_{rpc_worker_clients_};
-  distributed::PullRpcClients pull_clients_{rpc_worker_clients_};
+  distributed::PullRpcClients pull_clients_{&rpc_worker_clients_, &data_manager_};
   distributed::IndexRpcClients index_rpc_clients_{rpc_worker_clients_};
-  distributed::UpdatesRpcServer updates_server_{*self_, server_};
+  distributed::UpdatesRpcServer updates_server_{self_, &server_};
   distributed::UpdatesRpcClients updates_clients_{rpc_worker_clients_};
   distributed::DataManager data_manager_{*self_, data_clients_};
   distributed::TransactionalCacheCleaner cache_cleaner_{
@@ -159,8 +341,8 @@ Master::Master(Config config)
     snapshot_creator_->Run(
         "Snapshot", std::chrono::seconds(impl_->config_.snapshot_cycle_sec),
         [this] {
-          GraphDbAccessor dba(*this);
-          MakeSnapshot(dba);
+          auto dba = this->Access();
+          MakeSnapshot(*dba);
         });
   }
 
@@ -194,11 +376,20 @@ Master::~Master() {
   // We are not a worker, so we can do a snapshot on exit if it's enabled. Doing
   // this on the master forces workers to do the same through rpcs
   if (impl_->config_.snapshot_on_exit) {
-    GraphDbAccessor dba(*this);
-    MakeSnapshot(dba);
+    auto dba = Access();
+    MakeSnapshot(*dba);
   }
 }
 
+std::unique_ptr<GraphDbAccessor> Master::Access() {
+  return std::make_unique<MasterAccessor>(this, &impl_->index_rpc_clients_);
+}
+
+std::unique_ptr<GraphDbAccessor> Master::Access(tx::TransactionId tx_id) {
+  return std::make_unique<MasterAccessor>(this, tx_id,
+                                          &impl_->index_rpc_clients_);
+}
+
 Storage &Master::storage() { return *impl_->storage_; }
 
 durability::WriteAheadLog &Master::wal() { return impl_->wal_; }
@@ -299,6 +490,30 @@ distributed::IndexRpcClients &Master::index_rpc_clients() {
   return impl_->index_rpc_clients_;
 }
 
+VertexAccessor InsertVertexIntoRemote(
+    GraphDbAccessor *dba, int worker_id,
+    const std::vector<storage::Label> &labels,
+    const std::unordered_map<storage::Property, query::TypedValue>
+        &properties) {
+  // TODO: Replace this with virtual call or some other mechanism.
+  auto *distributed_db =
+      dynamic_cast<database::DistributedGraphDb *>(&dba->db());
+  CHECK(distributed_db);
+  CHECK(worker_id != distributed_db->WorkerId())
+      << "Not allowed to call InsertVertexIntoRemote for local worker";
+  auto *updates_clients = &distributed_db->updates_clients();
+  auto *data_manager = &distributed_db->data_manager();
+  CHECK(updates_clients && data_manager);
+  gid::Gid gid = updates_clients->CreateVertex(worker_id, dba->transaction_id(),
+                                               labels, properties);
+  auto vertex = std::make_unique<Vertex>();
+  vertex->labels_ = labels;
+  for (auto &kv : properties) vertex->properties_.set(kv.first, kv.second);
+  data_manager->Elements<Vertex>(dba->transaction_id())
+      .emplace(gid, nullptr, std::move(vertex));
+  return VertexAccessor({gid, worker_id}, *dba);
+}
+
 // Worker
 
 namespace impl {
@@ -341,13 +556,13 @@ class Worker {
                                                   &subcursor_storage_};
   distributed::BfsRpcClients bfs_subcursor_clients_{
       self_, &subcursor_storage_, &rpc_worker_clients_, &data_manager_};
-  distributed::DataRpcServer data_server_{*self_, server_};
+  distributed::DataRpcServer data_server_{self_, &server_};
   distributed::DataRpcClients data_clients_{rpc_worker_clients_};
   distributed::PlanConsumer plan_consumer_{server_};
-  distributed::ProduceRpcServer produce_server_{*self_, tx_engine_, server_,
+  distributed::ProduceRpcServer produce_server_{self_, &tx_engine_, server_,
                                                 plan_consumer_, &data_manager_};
   distributed::IndexRpcServer index_rpc_server_{*self_, server_};
-  distributed::UpdatesRpcServer updates_server_{*self_, server_};
+  distributed::UpdatesRpcServer updates_server_{self_, &server_};
   distributed::UpdatesRpcClients updates_clients_{rpc_worker_clients_};
   distributed::DataManager data_manager_{*self_, data_clients_};
   distributed::WorkerTransactionalCacheCleaner cache_cleaner_{
@@ -373,8 +588,7 @@ Worker::Worker(Config config)
   // Durability recovery.
   {
     // What we should recover.
-    std::experimental::optional<tx::TransactionId> snapshot_to_recover;
-    snapshot_to_recover = impl_->cluster_discovery_.snapshot_to_recover();
+    auto snapshot_to_recover = impl_->cluster_discovery_.snapshot_to_recover();
 
     // What we recover.
     std::experimental::optional<durability::RecoveryInfo> recovery_info;
@@ -426,6 +640,14 @@ Worker::~Worker() {
       [](auto &t) { t.set_should_abort(); });
 }
 
+std::unique_ptr<GraphDbAccessor> Worker::Access() {
+  return std::make_unique<WorkerAccessor>(this);
+}
+
+std::unique_ptr<GraphDbAccessor> Worker::Access(tx::TransactionId tx_id) {
+  return std::make_unique<WorkerAccessor>(this, tx_id);
+}
+
 Storage &Worker::storage() { return *impl_->storage_; }
 
 durability::WriteAheadLog &Worker::wal() { return impl_->wal_; }
diff --git a/src/database/distributed_graph_db.hpp b/src/database/distributed_graph_db.hpp
index 3a3aef635..8671cad6a 100644
--- a/src/database/distributed_graph_db.hpp
+++ b/src/database/distributed_graph_db.hpp
@@ -1,3 +1,5 @@
+/// @file
+
 #pragma once
 
 #include "database/graph_db.hpp"
@@ -28,6 +30,9 @@ class Worker;
 /// Abstract base class for concrete distributed versions of GraphDb
 class DistributedGraphDb : public GraphDb {
  public:
+  virtual int WorkerId() const = 0;
+  virtual std::vector<int> GetWorkerIds() const = 0;
+
   virtual distributed::BfsRpcClients &bfs_subcursor_clients() = 0;
   virtual distributed::DataRpcClients &data_clients() = 0;
   virtual distributed::UpdatesRpcServer &updates_server() = 0;
@@ -44,6 +49,9 @@ class Master final : public DistributedGraphDb {
     return GraphDb::Type::DISTRIBUTED_MASTER;
   }
 
+  std::unique_ptr<GraphDbAccessor> Access() override;
+  std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) override;
+
   Storage &storage() override;
   durability::WriteAheadLog &wal() override;
   tx::Engine &tx_engine() override;
@@ -89,6 +97,9 @@ class Worker final : public DistributedGraphDb {
     return GraphDb::Type::DISTRIBUTED_WORKER;
   }
 
+  std::unique_ptr<GraphDbAccessor> Access() override;
+  std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) override;
+
   Storage &storage() override;
   durability::WriteAheadLog &wal() override;
   tx::Engine &tx_engine() override;
@@ -124,4 +135,11 @@ class Worker final : public DistributedGraphDb {
   utils::Scheduler transaction_killer_;
 };
 
+/// Creates a new Vertex on the given worker.
+/// It is NOT allowed to call this function with this worker's id.
+VertexAccessor InsertVertexIntoRemote(
+    GraphDbAccessor *dba, int worker_id,
+    const std::vector<storage::Label> &labels,
+    const std::unordered_map<storage::Property, query::TypedValue> &properties);
+
 }  // namespace database
diff --git a/src/database/graph_db.cpp b/src/database/graph_db.cpp
index 7b8a714f3..552bc18fb 100644
--- a/src/database/graph_db.cpp
+++ b/src/database/graph_db.cpp
@@ -81,8 +81,8 @@ SingleNode::SingleNode(Config config)
     snapshot_creator_->Run(
         "Snapshot", std::chrono::seconds(impl_->config_.snapshot_cycle_sec),
         [this] {
-          GraphDbAccessor dba(*this);
-          this->MakeSnapshot(dba);
+          auto dba = this->Access();
+          this->MakeSnapshot(*dba);
         });
   }
 
@@ -114,11 +114,30 @@ SingleNode::~SingleNode() {
       [](auto &t) { t.set_should_abort(); });
 
   if (impl_->config_.snapshot_on_exit) {
-    GraphDbAccessor dba(*this);
-    MakeSnapshot(dba);
+    auto dba = this->Access();
+    MakeSnapshot(*dba);
   }
 }
 
+class SingleNodeAccessor : public GraphDbAccessor {
+ public:
+  explicit SingleNodeAccessor(GraphDb &db) : GraphDbAccessor(db) {}
+  SingleNodeAccessor(GraphDb &db, tx::TransactionId tx_id)
+      : GraphDbAccessor(db, tx_id) {}
+};
+
+std::unique_ptr<GraphDbAccessor> SingleNode::Access() {
+  // NOTE: We are doing a heap allocation to allow polymorphism. If this poses
+  // performance issues, we may want to have a stack allocated GraphDbAccessor
+  // which is constructed with a pointer to some global implementation struct
+  // which contains only pure functions (without any state).
+  return std::make_unique<SingleNodeAccessor>(*this);
+}
+
+std::unique_ptr<GraphDbAccessor> SingleNode::Access(tx::TransactionId tx_id) {
+  return std::make_unique<SingleNodeAccessor>(*this, tx_id);
+}
+
 Storage &SingleNode::storage() { return *impl_->storage_; }
 
 durability::WriteAheadLog &SingleNode::wal() { return impl_->wal_; }
@@ -141,10 +160,6 @@ database::Counters &SingleNode::counters() { return impl_->counters_; }
 
 void SingleNode::CollectGarbage() { impl_->storage_gc_->CollectGarbage(); }
 
-int SingleNode::WorkerId() const { return impl_->config_.worker_id; }
-
-std::vector<int> SingleNode::GetWorkerIds() const { return {0}; }
-
 bool SingleNode::MakeSnapshot(GraphDbAccessor &accessor) {
   const bool status = durability::MakeSnapshot(
       *this, accessor, fs::path(impl_->config_.durability_directory),
diff --git a/src/database/graph_db.hpp b/src/database/graph_db.hpp
index 411a5f912..d329c686c 100644
--- a/src/database/graph_db.hpp
+++ b/src/database/graph_db.hpp
@@ -44,29 +44,29 @@ struct Config {
   int recovering_cluster_size{0};
 };
 
-/**
- * An abstract base class for a SingleNode/Master/Worker graph db.
- *
- * Always be sure that GraphDb object is destructed before main exits, i. e.
- * GraphDb object shouldn't be part of global/static variable, except if its
- * destructor is explicitly called before main exits. Consider code:
- *
- * GraphDb db;  // KeyIndex is created as a part of database::Storage
- * int main() {
- *   GraphDbAccessor dba(db);
- *   auto v = dba.InsertVertex();
- *   v.add_label(dba.Label(
- *       "Start"));  // New SkipList is created in KeyIndex for LabelIndex.
- *                   // That SkipList creates SkipListGc which
- *                   // initialises static Executor object.
- *   return 0;
- * }
- *
- * After main exits: 1. Executor is destructed, 2. KeyIndex is destructed.
- * Destructor of KeyIndex calls delete on created SkipLists which destroy
- * SkipListGc that tries to use Excutioner object that doesn't exist anymore.
- * -> CRASH
- */
+class GraphDbAccessor;
+
+/// An abstract base class for a SingleNode/Master/Worker graph db.
+///
+/// Always be sure that GraphDb object is destructed before main exits, i. e.
+/// GraphDb object shouldn't be part of global/static variable, except if its
+/// destructor is explicitly called before main exits. Consider code:
+///
+/// GraphDb db;  // KeyIndex is created as a part of database::Storage
+/// int main() {
+///   GraphDbAccessor dba(db);
+///   auto v = dba.InsertVertex();
+///   v.add_label(dba.Label(
+///       "Start"));  // New SkipList is created in KeyIndex for LabelIndex.
+///                   // That SkipList creates SkipListGc which
+///                   // initialises static Executor object.
+///   return 0;
+/// }
+///
+/// After main exits: 1. Executor is destructed, 2. KeyIndex is destructed.
+/// Destructor of KeyIndex calls delete on created SkipLists which destroy
+/// SkipListGc that tries to use Excutioner object that doesn't exist anymore.
+/// -> CRASH
 class GraphDb {
  public:
   enum class Type { SINGLE_NODE, DISTRIBUTED_MASTER, DISTRIBUTED_WORKER };
@@ -80,6 +80,12 @@ class GraphDb {
   virtual ~GraphDb() {}
 
   virtual Type type() const = 0;
+
+  /// Create a new accessor by starting a new transaction.
+  virtual std::unique_ptr<GraphDbAccessor> Access() = 0;
+  /// Create an accessor for a running transaction.
+  virtual std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) = 0;
+
   virtual Storage &storage() = 0;
   virtual durability::WriteAheadLog &wal() = 0;
   virtual tx::Engine &tx_engine() = 0;
@@ -89,19 +95,17 @@ class GraphDb {
   virtual storage::ConcurrentIdMapper<storage::Property> &property_mapper() = 0;
   virtual database::Counters &counters() = 0;
   virtual void CollectGarbage() = 0;
-  virtual int WorkerId() const = 0;
-  virtual std::vector<int> GetWorkerIds() const = 0;
 
-  // Makes a snapshot from the visibility of the given accessor
+  /// Makes a snapshot from the visibility of the given accessor
   virtual bool MakeSnapshot(GraphDbAccessor &accessor) = 0;
 
-  // Releases the storage object safely and creates a new object.
-  // This is needed because of recovery, otherwise we might try to recover into
-  // a storage which has already been polluted because of a failed previous
-  // recovery
+  /// Releases the storage object safely and creates a new object.
+  /// This is needed because of recovery, otherwise we might try to recover into
+  /// a storage which has already been polluted because of a failed previous
+  /// recovery
   virtual void ReinitializeStorage() = 0;
 
-  /** When this is false, no new transactions should be created. */
+  /// When this is false, no new transactions should be created.
   bool is_accepting_transactions() const { return is_accepting_transactions_; }
 
  protected:
@@ -118,6 +122,10 @@ class SingleNode final : public GraphDb {
   ~SingleNode();
 
   Type type() const override { return GraphDb::Type::SINGLE_NODE; }
+
+  std::unique_ptr<GraphDbAccessor> Access() override;
+  std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId) override;
+
   Storage &storage() override;
   durability::WriteAheadLog &wal() override;
   tx::Engine &tx_engine() override;
@@ -126,8 +134,6 @@ class SingleNode final : public GraphDb {
   storage::ConcurrentIdMapper<storage::Property> &property_mapper() override;
   database::Counters &counters() override;
   void CollectGarbage() override;
-  int WorkerId() const override;
-  std::vector<int> GetWorkerIds() const override;
 
   bool MakeSnapshot(GraphDbAccessor &accessor) override;
   void ReinitializeStorage() override;
diff --git a/src/database/graph_db_accessor.cpp b/src/database/graph_db_accessor.cpp
index 8dd670bae..db48bc4df 100644
--- a/src/database/graph_db_accessor.cpp
+++ b/src/database/graph_db_accessor.cpp
@@ -1,20 +1,17 @@
 #include "database/graph_db_accessor.hpp"
 
-#include <functional>
+#include <chrono>
+#include <thread>
 
-#include "glog/logging.h"
+#include <glog/logging.h>
 
-#include "database/distributed_graph_db.hpp"
 #include "database/state_delta.hpp"
-#include "distributed/data_manager.hpp"
-#include "distributed/rpc_worker_clients.hpp"
-#include "distributed/updates_rpc_clients.hpp"
 #include "storage/address_types.hpp"
 #include "storage/edge.hpp"
 #include "storage/edge_accessor.hpp"
 #include "storage/vertex.hpp"
 #include "storage/vertex_accessor.hpp"
-#include "utils/atomic.hpp"
+#include "utils/cast.hpp"
 #include "utils/on_scope_exit.hpp"
 
 namespace database {
@@ -83,34 +80,6 @@ VertexAccessor GraphDbAccessor::InsertVertex(
   return va;
 }
 
-VertexAccessor GraphDbAccessor::InsertVertexIntoRemote(
-    int worker_id, const std::vector<storage::Label> &labels,
-    const std::unordered_map<storage::Property, query::TypedValue>
-        &properties) {
-  CHECK(worker_id != db().WorkerId())
-      << "Not allowed to call InsertVertexIntoRemote for local worker";
-
-  distributed::UpdatesRpcClients *updates_clients = nullptr;
-  distributed::DataManager *data_manager = nullptr;
-  // TODO: Replace this with virtual call or some other mechanism.
-  if (auto *distributed_db =
-          dynamic_cast<database::DistributedGraphDb *>(&db())) {
-    updates_clients = &distributed_db->updates_clients();
-    data_manager = &distributed_db->data_manager();
-  }
-  CHECK(updates_clients && data_manager);
-  gid::Gid gid = updates_clients->CreateVertex(worker_id, transaction_id(),
-                                               labels, properties);
-
-  auto vertex = std::make_unique<Vertex>();
-  vertex->labels_ = labels;
-  for (auto &kv : properties) vertex->properties_.set(kv.first, kv.second);
-
-  data_manager->Elements<Vertex>(transaction_id())
-      .emplace(gid, nullptr, std::move(vertex));
-  return VertexAccessor({gid, worker_id}, *this);
-}
-
 std::experimental::optional<VertexAccessor> GraphDbAccessor::FindVertexOptional(
     gid::Gid gid, bool current_state) {
   VertexAccessor record_accessor(
@@ -144,8 +113,6 @@ EdgeAccessor GraphDbAccessor::FindEdge(gid::Gid gid, bool current_state) {
 void GraphDbAccessor::BuildIndex(storage::Label label,
                                  storage::Property property) {
   DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
-  DCHECK(db_.type() != GraphDb::Type::DISTRIBUTED_WORKER)
-      << "BuildIndex invoked on worker";
 
   db_.storage().index_build_tx_in_progress_.access().insert(transaction_.id_);
 
@@ -157,37 +124,15 @@ void GraphDbAccessor::BuildIndex(storage::Label label,
     DCHECK(removed) << "Index creation transaction should be inside set";
   });
 
+  // Create the index
   const LabelPropertyIndex::Key key(label, property);
   if (db_.storage().label_property_index_.CreateIndex(key) == false) {
     throw IndexExistsException(
         "Index is either being created by another transaction or already "
         "exists.");
   }
-
-  std::experimental::optional<std::vector<utils::Future<bool>>>
-      index_rpc_completions;
-
-  // Notify all workers to create the index
-  if (db_.type() == GraphDb::Type::DISTRIBUTED_MASTER) {
-    // TODO: Replace this with virtual call or some other mechanism.
-    database::Master *master_db = dynamic_cast<database::Master *>(&db_);
-    index_rpc_completions.emplace(
-        master_db->index_rpc_clients().GetCreateIndexFutures(
-            label, property, this->db_.WorkerId()));
-  }
-
-  if (index_rpc_completions) {
-    // Wait first, check later - so that every thread finishes and none
-    // terminates - this can probably be optimized in case we fail early so that
-    // we notify other workers to stop building indexes
-    for (auto &index_built : *index_rpc_completions) index_built.wait();
-    for (auto &index_built : *index_rpc_completions) {
-      if (!index_built.get()) {
-        db_.storage().label_property_index_.DeleteIndex(key);
-        throw IndexCreationOnWorkerException("Index exists on a worker");
-      }
-    }
-  }
+  // Call the hook for inherited classes.
+  PostCreateIndex(key);
 
   // Everything that happens after the line above ended will be added to the
   // index automatically, but we still have to add to index everything that
@@ -212,21 +157,11 @@ void GraphDbAccessor::BuildIndex(storage::Label label,
 
   // This accessor's transaction surely sees everything that happened before
   // CreateIndex.
-  GraphDbAccessor dba(db_);
-
-  // Notify all workers to start populating an index if we are the master since
-  // they don't have to wait anymore
-  if (db_.type() == GraphDb::Type::DISTRIBUTED_MASTER) {
-    // TODO: Replace this with virtual call or some other mechanism.
-    database::Master *master_db = dynamic_cast<database::Master *>(&db_);
-    index_rpc_completions.emplace(
-        master_db->index_rpc_clients().GetPopulateIndexFutures(
-            label, property, dba.transaction_id(), this->db_.WorkerId()));
-  }
+  auto dba = db_.Access();
 
   // Add transaction to the build_tx_in_progress as this transaction doesn't
   // change data and shouldn't block other parallel index creations
-  auto read_transaction_id = dba.transaction().id_;
+  auto read_transaction_id = dba->transaction().id_;
   db_.storage().index_build_tx_in_progress_.access().insert(
       read_transaction_id);
   // on function exit remove the read transaction from build_tx_in_progress
@@ -236,25 +171,10 @@ void GraphDbAccessor::BuildIndex(storage::Label label,
     DCHECK(removed) << "Index building (read) transaction should be inside set";
   });
 
-  dba.PopulateIndex(key);
+  dba->PopulateIndexFromBuildIndex(key);
 
-  // Check if all workers sucesfully built their indexes and after this we can
-  // set the index as built
-  if (index_rpc_completions) {
-    // Wait first, check later - so that every thread finishes and none
-    // terminates - this can probably be optimized in case we fail early so that
-    // we notify other workers to stop building indexes
-    for (auto &index_built : *index_rpc_completions) index_built.wait();
-    for (auto &index_built : *index_rpc_completions) {
-      if (!index_built.get()) {
-        db_.storage().label_property_index_.DeleteIndex(key);
-        throw IndexCreationOnWorkerException("Index exists on a worker");
-      }
-    }
-  }
-
-  dba.EnableIndex(key);
-  dba.Commit();
+  dba->EnableIndex(key);
+  dba->Commit();
 }
 
 void GraphDbAccessor::EnableIndex(const LabelPropertyIndex::Key &key) {
@@ -376,22 +296,6 @@ int64_t GraphDbAccessor::VerticesCount(
 bool GraphDbAccessor::RemoveVertex(VertexAccessor &vertex_accessor,
                                    bool check_empty) {
   DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
-
-  if (!vertex_accessor.is_local()) {
-    auto address = vertex_accessor.address();
-    distributed::UpdatesRpcClients *updates_clients = nullptr;
-    // TODO: Replace this with virtual call or some other mechanism.
-    if (auto *distributed_db =
-            dynamic_cast<database::DistributedGraphDb *>(&db())) {
-      updates_clients = &distributed_db->updates_clients();
-    }
-    CHECK(updates_clients);
-    updates_clients->RemoveVertex(address.worker_id(), transaction_id(),
-                                  address.gid(), check_empty);
-    // We can't know if we are going to be able to remove vertex until deferred
-    // updates on a remote worker are executed
-    return true;
-  }
   vertex_accessor.SwitchNew();
   // it's possible the vertex was removed already in this transaction
   // due to it getting matched multiple times by some patterns
@@ -431,86 +335,52 @@ EdgeAccessor GraphDbAccessor::InsertEdge(
     std::experimental::optional<int64_t> cypher_id) {
   DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
 
-  // The address of an edge we'll create.
-  storage::EdgeAddress edge_address;
-
-  Vertex *from_updated;
-  if (from.is_local()) {
-    auto edge_accessor = InsertOnlyEdge(from.address(), to.address(), edge_type,
-                                        requested_gid, cypher_id);
-    edge_address = edge_accessor.address();
-
-    from.SwitchNew();
-    from_updated = &from.update();
-
-    // TODO when preparing WAL for distributed, most likely never use
-    // `CREATE_EDGE`, but always have it split into 3 parts (edge insertion,
-    // in/out modification).
-    wal().Emplace(database::StateDelta::CreateEdge(
-        transaction_.id_, edge_accessor.gid(), edge_accessor.cypher_id(),
-        from.gid(), to.gid(), edge_type, EdgeTypeName(edge_type)));
-
-  } else {
-    distributed::UpdatesRpcClients *updates_clients = nullptr;
-    distributed::DataManager *data_manager = nullptr;
-    // TODO: Replace this with virtual call or some other mechanism.
-    if (auto *distributed_db =
-            dynamic_cast<database::DistributedGraphDb *>(&db())) {
-      updates_clients = &distributed_db->updates_clients();
-      data_manager = &distributed_db->data_manager();
-    }
-    CHECK(updates_clients && data_manager);
-    edge_address =
-        updates_clients->CreateEdge(transaction_id(), from, to, edge_type);
-
-    from_updated =
-        data_manager->Elements<Vertex>(transaction_id()).FindNew(from.gid());
-
-    // Create an Edge and insert it into the Cache so we see it locally.
-    data_manager->Elements<Edge>(transaction_id())
-        .emplace(
-            edge_address.gid(), nullptr,
-            std::make_unique<Edge>(from.address(), to.address(), edge_type));
-  }
-  from_updated->out_.emplace(
-      db_.storage().LocalizedAddressIfPossible(to.address()), edge_address,
-      edge_type);
-
-  Vertex *to_updated;
-  if (to.is_local()) {
-    // ensure that the "to" accessor has the latest version (Switch new)
-    // WARNING: must do that after the above "from.update()" for cases when
-    // we are creating a cycle and "from" and "to" are the same vlist
-    to.SwitchNew();
-    to_updated = &to.update();
-  } else {
-    distributed::UpdatesRpcClients *updates_clients = nullptr;
-    distributed::DataManager *data_manager = nullptr;
-    // TODO: Replace this with virtual call or some other mechanism.
-    if (auto *distributed_db =
-            dynamic_cast<database::DistributedGraphDb *>(&db())) {
-      updates_clients = &distributed_db->updates_clients();
-      data_manager = &distributed_db->data_manager();
-    }
-    CHECK(updates_clients && data_manager);
-    // The RPC call for the `to` side is already handled if `from` is not local.
-    if (from.is_local() ||
-        from.address().worker_id() != to.address().worker_id()) {
-      updates_clients->AddInEdge(transaction_id(), from,
-                                 db().storage().GlobalizedAddress(edge_address),
-                                 to, edge_type);
-    }
-    to_updated =
-        data_manager->Elements<Vertex>(transaction_id()).FindNew(to.gid());
-  }
-  to_updated->in_.emplace(
-      db_.storage().LocalizedAddressIfPossible(from.address()), edge_address,
-      edge_type);
+  auto edge_address =
+      InsertEdgeOnFrom(&from, &to, edge_type, requested_gid, cypher_id);
 
+  InsertEdgeOnTo(&from, &to, edge_type, edge_address);
   return EdgeAccessor(edge_address, *this, from.address(), to.address(),
                       edge_type);
 }
 
+storage::EdgeAddress GraphDbAccessor::InsertEdgeOnFrom(
+    VertexAccessor *from, VertexAccessor *to,
+    const storage::EdgeType &edge_type,
+    const std::experimental::optional<gid::Gid> &requested_gid,
+    const std::experimental::optional<int64_t> &cypher_id) {
+  auto edge_accessor = InsertOnlyEdge(from->address(), to->address(), edge_type,
+                                      requested_gid, cypher_id);
+  auto edge_address = edge_accessor.address();
+
+  from->SwitchNew();
+  auto from_updated = &from->update();
+
+  // TODO when preparing WAL for distributed, most likely never use
+  // `CREATE_EDGE`, but always have it split into 3 parts (edge insertion,
+  // in/out modification).
+  wal().Emplace(database::StateDelta::CreateEdge(
+      transaction_.id_, edge_accessor.gid(), edge_accessor.cypher_id(),
+      from->gid(), to->gid(), edge_type, EdgeTypeName(edge_type)));
+
+  from_updated->out_.emplace(
+      db_.storage().LocalizedAddressIfPossible(to->address()), edge_address,
+      edge_type);
+  return edge_address;
+}
+
+void GraphDbAccessor::InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to,
+                                     const storage::EdgeType &edge_type,
+                                     const storage::EdgeAddress &edge_address) {
+  // ensure that the "to" accessor has the latest version (Switch new)
+  // WARNING: must do that after the above "from->update()" for cases when
+  // we are creating a cycle and "from" and "to" are the same vlist
+  to->SwitchNew();
+  auto *to_updated = &to->update();
+  to_updated->in_.emplace(
+      db_.storage().LocalizedAddressIfPossible(from->address()), edge_address,
+      edge_type);
+}
+
 EdgeAccessor GraphDbAccessor::InsertOnlyEdge(
     storage::VertexAddress from, storage::VertexAddress to,
     storage::EdgeType edge_type,
@@ -541,41 +411,16 @@ int64_t GraphDbAccessor::EdgesCount() const {
 void GraphDbAccessor::RemoveEdge(EdgeAccessor &edge, bool remove_out_edge,
                                  bool remove_in_edge) {
   DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
-  if (edge.is_local()) {
-    // it's possible the edge was removed already in this transaction
-    // due to it getting matched multiple times by some patterns
-    // we can only delete it once, so check if it's already deleted
-    edge.SwitchNew();
-    if (edge.current().is_expired_by(transaction_)) return;
-    if (remove_out_edge) edge.from().RemoveOutEdge(edge.address());
-    if (remove_in_edge) edge.to().RemoveInEdge(edge.address());
+  // it's possible the edge was removed already in this transaction
+  // due to it getting matched multiple times by some patterns
+  // we can only delete it once, so check if it's already deleted
+  edge.SwitchNew();
+  if (edge.current().is_expired_by(transaction_)) return;
+  if (remove_out_edge) edge.from().RemoveOutEdge(edge.address());
+  if (remove_in_edge) edge.to().RemoveInEdge(edge.address());
 
-    edge.address().local()->remove(edge.current_, transaction_);
-    wal().Emplace(
-        database::StateDelta::RemoveEdge(transaction_.id_, edge.gid()));
-  } else {
-    auto edge_addr = edge.GlobalAddress();
-    auto from_addr = db().storage().GlobalizedAddress(edge.from_addr());
-    CHECK(edge_addr.worker_id() == from_addr.worker_id())
-        << "Edge and it's 'from' vertex not on the same worker";
-    auto to_addr = db().storage().GlobalizedAddress(edge.to_addr());
-    distributed::UpdatesRpcClients *updates_clients = nullptr;
-    // TODO: Replace this with virtual call or some other mechanism.
-    if (auto *distributed_db =
-            dynamic_cast<database::DistributedGraphDb *>(&db())) {
-      updates_clients = &distributed_db->updates_clients();
-    }
-    CHECK(updates_clients);
-    updates_clients->RemoveEdge(transaction_id(), edge_addr.worker_id(),
-                                edge_addr.gid(), from_addr.gid(), to_addr);
-
-    // Another RPC is necessary only if the first did not handle vertices on
-    // both sides.
-    if (edge_addr.worker_id() != to_addr.worker_id()) {
-      updates_clients->RemoveInEdge(transaction_id(), to_addr.worker_id(),
-                                    to_addr.gid(), edge_addr);
-    }
-  }
+  edge.address().local()->remove(edge.current_, transaction_);
+  wal().Emplace(database::StateDelta::RemoveEdge(transaction_.id_, edge.gid()));
 }
 
 storage::Label GraphDbAccessor::Label(const std::string &label_name) {
diff --git a/src/database/graph_db_accessor.hpp b/src/database/graph_db_accessor.hpp
index 9de9f1b4d..098eedd8f 100644
--- a/src/database/graph_db_accessor.hpp
+++ b/src/database/graph_db_accessor.hpp
@@ -1,24 +1,22 @@
 #pragma once
 
 #include <experimental/optional>
-#include <unordered_map>
+#include <string>
 #include <vector>
 
-#include "cppitertools/filter.hpp"
-#include "cppitertools/imap.hpp"
-#include "glog/logging.h"
+#include <cppitertools/filter.hpp>
+#include <cppitertools/imap.hpp>
+#include <glog/logging.h>
 
 #include "database/graph_db.hpp"
-#include "distributed/cache.hpp"
-#include "query/typed_value.hpp"
 #include "storage/address_types.hpp"
 #include "storage/edge_accessor.hpp"
 #include "storage/types.hpp"
 #include "storage/vertex_accessor.hpp"
-#include "transactions/engine_single_node.hpp"
 #include "transactions/transaction.hpp"
 #include "transactions/type.hpp"
 #include "utils/bound.hpp"
+#include "utils/exceptions.hpp"
 
 namespace database {
 
@@ -33,27 +31,28 @@ class IndexCreationOnWorkerException : public utils::BasicException {
 };
 
 /**
- * An accessor for the database object: exposes functions for operating on the
+ * Base accessor for the database object: exposes functions for operating on the
  * database. All the functions in this class should be self-sufficient: for
  * example the function for creating a new Vertex should take care of all the
  * book-keeping around the creation.
  */
-
 class GraphDbAccessor {
   // We need to make friends with this guys since they need to access private
   // methods for updating indices.
   friend class ::RecordAccessor<Vertex>;
-  friend class ::RecordAccessor<Edge>;
   friend class ::VertexAccessor;
-  friend class ::EdgeAccessor;
 
- public:
+ protected:
+  // Construction should only be done through GraphDb::Access function and
+  // concrete GraphDbAccessor type.
+
   /// Creates a new accessor by starting a new transaction.
   explicit GraphDbAccessor(GraphDb &db);
-
   /// Creates an accessor for a running transaction.
   GraphDbAccessor(GraphDb &db, tx::TransactionId tx_id);
-  ~GraphDbAccessor();
+
+ public:
+  virtual ~GraphDbAccessor();
 
   GraphDbAccessor(const GraphDbAccessor &other) = delete;
   GraphDbAccessor(GraphDbAccessor &&other) = delete;
@@ -82,13 +81,6 @@ class GraphDbAccessor {
                               std::experimental::optional<int64_t> cypher_id =
                                   std::experimental::nullopt);
 
-  /** Creates a new Vertex on the given worker. It is NOT allowed to call this
-   * function with this worker's id. */
-  VertexAccessor InsertVertexIntoRemote(
-      int worker_id, const std::vector<storage::Label> &labels,
-      const std::unordered_map<storage::Property, query::TypedValue>
-          &properties);
-
   /**
    * Removes the vertex of the given accessor. If the vertex has any outgoing or
    * incoming edges, it is not deleted. See `DetachRemoveVertex` if you want to
@@ -102,7 +94,8 @@ class GraphDbAccessor {
    * before deletion.
    * @return  If or not the vertex was deleted.
    */
-  bool RemoveVertex(VertexAccessor &vertex_accessor, bool check_empty = true);
+  virtual bool RemoveVertex(VertexAccessor &vertex_accessor,
+                            bool check_empty = true);
 
   /**
    * Removes the vertex of the given accessor along with all it's outgoing
@@ -340,8 +333,8 @@ class GraphDbAccessor {
    * @param remove_in_edge If the edge should be removed from the its
    * destination side.
    */
-  void RemoveEdge(EdgeAccessor &edge, bool remove_out_edge = true,
-                  bool remove_in_edge = true);
+  virtual void RemoveEdge(EdgeAccessor &edge, bool remove_out_edge = true,
+                          bool remove_in_edge = true);
 
   /**
    * Obtains the edge for the given ID. If there is no edge for the given
@@ -444,7 +437,7 @@ class GraphDbAccessor {
    * @param label - label to build for
    * @param property - property to build for
    */
-  void BuildIndex(storage::Label label, storage::Property property);
+  virtual void BuildIndex(storage::Label label, storage::Property property);
 
   /// Populates index with vertices containing the key
   void PopulateIndex(const LabelPropertyIndex::Key &key);
@@ -607,6 +600,34 @@ class GraphDbAccessor {
   /* Returns a list of index names present in the database. */
   std::vector<std::string> IndexInfo() const;
 
+ protected:
+  /** Called in `BuildIndex` after creating an index, but before populating. */
+  virtual void PostCreateIndex(const LabelPropertyIndex::Key &key) {}
+
+  /** Populates the index from a *new* transaction after creating the index. */
+  virtual void PopulateIndexFromBuildIndex(const LabelPropertyIndex::Key &key) {
+    PopulateIndex(key);
+  }
+
+  /**
+   * Insert a new edge to `from` vertex and return the address.
+   * Called from `InsertEdge` as the first step in edge insertion.
+   * */
+  virtual storage::EdgeAddress InsertEdgeOnFrom(
+      VertexAccessor *from, VertexAccessor *to,
+      const storage::EdgeType &edge_type,
+      const std::experimental::optional<gid::Gid> &requested_gid,
+      const std::experimental::optional<int64_t> &cypher_id);
+
+  /**
+   * Set the newly created edge on `to` vertex.
+   * Called after `InsertEdgeOnFrom` in `InsertEdge`. The given `edge_address`
+   * is from the created edge, returned by `InsertEdgeOnFrom`.
+   */
+  virtual void InsertEdgeOnTo(VertexAccessor *from, VertexAccessor *to,
+                              const storage::EdgeType &edge_type,
+                              const storage::EdgeAddress &edge_address);
+
  private:
   GraphDb &db_;
   tx::Transaction &transaction_;
@@ -640,4 +661,5 @@ class GraphDbAccessor {
                            const RecordAccessor<Vertex> &vertex_accessor,
                            const Vertex *const vertex);
 };
+
 }  // namespace database
diff --git a/src/distributed/bfs_rpc_clients.cpp b/src/distributed/bfs_rpc_clients.cpp
index 1b49c1c46..5f013f6da 100644
--- a/src/distributed/bfs_rpc_clients.cpp
+++ b/src/distributed/bfs_rpc_clients.cpp
@@ -6,7 +6,7 @@
 
 namespace distributed {
 
-BfsRpcClients::BfsRpcClients(database::GraphDb *db,
+BfsRpcClients::BfsRpcClients(database::DistributedGraphDb *db,
                              BfsSubcursorStorage *subcursor_storage,
                              RpcWorkerClients *clients,
                              DataManager *data_manager)
diff --git a/src/distributed/bfs_rpc_clients.hpp b/src/distributed/bfs_rpc_clients.hpp
index a86fbe78b..6baf71bd8 100644
--- a/src/distributed/bfs_rpc_clients.hpp
+++ b/src/distributed/bfs_rpc_clients.hpp
@@ -5,6 +5,10 @@
 #include "distributed/rpc_worker_clients.hpp"
 #include "transactions/transaction.hpp"
 
+namespace database {
+class DistributedGraphDb;
+}
+
 namespace distributed {
 
 class DataManager;
@@ -17,7 +21,7 @@ class DataManager;
 /// directly.
 class BfsRpcClients {
  public:
-  BfsRpcClients(database::GraphDb *db,
+  BfsRpcClients(database::DistributedGraphDb *db,
                 BfsSubcursorStorage *subcursor_storage,
                 RpcWorkerClients *clients,
                 DataManager *data_manager);
@@ -60,7 +64,7 @@ class BfsRpcClients {
       const std::unordered_map<int16_t, int64_t> &subcursor_ids, bool clear);
 
  private:
-  database::GraphDb *db_{nullptr};
+  database::DistributedGraphDb *db_{nullptr};
   distributed::BfsSubcursorStorage *subcursor_storage_{nullptr};
   distributed::RpcWorkerClients *clients_{nullptr};
   distributed::DataManager *data_manager_{nullptr};
diff --git a/src/distributed/bfs_rpc_messages.lcp b/src/distributed/bfs_rpc_messages.lcp
index 2971772d3..e96ad1359 100644
--- a/src/distributed/bfs_rpc_messages.lcp
+++ b/src/distributed/bfs_rpc_messages.lcp
@@ -120,10 +120,9 @@ cpp<#
           << "Only global addresses should be used with SerializedGraphElement";
     }
 
-    SerializedGraphElement(const RecordAccessor<TElement> &accessor)
+    SerializedGraphElement(const RecordAccessor<TElement> &accessor, int16_t worker_id)
         : SerializedGraphElement(accessor.GlobalAddress(), accessor.GetOld(),
-                                 accessor.GetNew(),
-                                 accessor.db_accessor().db().WorkerId()) {}
+                                 accessor.GetNew(), worker_id) {}
 
     SerializedGraphElement() {}
    cpp<#)
@@ -265,12 +264,13 @@ cpp<#
     ReconstructPathRes(
         const std::vector<EdgeAccessor> &edge_accessors,
         std::experimental::optional<storage::VertexAddress> next_vertex,
-        std::experimental::optional<storage::EdgeAddress> next_edge)
+        std::experimental::optional<storage::EdgeAddress> next_edge,
+        int16_t worker_id)
         : next_vertex(std::move(next_vertex)), next_edge(std::move(next_edge)) {
       CHECK(!static_cast<bool>(next_vertex) || !static_cast<bool>(next_edge))
           << "At most one of `next_vertex` and `next_edge` should be set";
       for (const auto &edge : edge_accessors) {
-        edges.emplace_back(edge);
+        edges.emplace_back(edge, worker_id);
       }
     }
     cpp<#)))
diff --git a/src/distributed/bfs_rpc_server.hpp b/src/distributed/bfs_rpc_server.hpp
index 8ce85bdd1..597ce923d 100644
--- a/src/distributed/bfs_rpc_server.hpp
+++ b/src/distributed/bfs_rpc_server.hpp
@@ -16,7 +16,8 @@ namespace distributed {
 /// subcursor storage.
 class BfsRpcServer {
  public:
-  BfsRpcServer(database::GraphDb *db, communication::rpc::Server *server,
+  BfsRpcServer(database::DistributedGraphDb *db,
+               communication::rpc::Server *server,
                BfsSubcursorStorage *subcursor_storage)
       : db_(db), server_(server), subcursor_storage_(subcursor_storage) {
     server_->Register<CreateBfsSubcursorRpc>(
@@ -38,14 +39,14 @@ class BfsRpcServer {
           res.Save(res_builder);
         });
 
-    server_->Register<ResetSubcursorRpc>([this](const auto &req_reader,
-                                                auto *res_builder) {
-      ResetSubcursorReq req;
-      req.Load(req_reader);
-      subcursor_storage_->Get(req.subcursor_id)->Reset();
-      ResetSubcursorRes res;
-      res.Save(res_builder);
-    });
+    server_->Register<ResetSubcursorRpc>(
+        [this](const auto &req_reader, auto *res_builder) {
+          ResetSubcursorReq req;
+          req.Load(req_reader);
+          subcursor_storage_->Get(req.subcursor_id)->Reset();
+          ResetSubcursorRes res;
+          res.Save(res_builder);
+        });
 
     server_->Register<RemoveBfsSubcursorRpc>(
         [this](const auto &req_reader, auto *res_builder) {
@@ -83,7 +84,7 @@ class BfsRpcServer {
             res.Save(res_builder);
             return;
           }
-          SubcursorPullRes res(*vertex);
+          SubcursorPullRes res(SerializedVertex(*vertex, db_->WorkerId()));
           res.Save(res_builder);
         });
 
@@ -111,7 +112,7 @@ class BfsRpcServer {
         LOG(FATAL) << "`edge` or `vertex` should be set in ReconstructPathReq";
       }
       ReconstructPathRes res(result.edges, result.next_vertex,
-                             result.next_edge);
+                             result.next_edge, db_->WorkerId());
       res.Save(res_builder);
     });
 
@@ -126,7 +127,7 @@ class BfsRpcServer {
   }
 
  private:
-  database::GraphDb *db_;
+  database::DistributedGraphDb *db_;
 
   communication::rpc::Server *server_;
   BfsSubcursorStorage *subcursor_storage_;
diff --git a/src/distributed/bfs_subcursor.cpp b/src/distributed/bfs_subcursor.cpp
index c376bd04a..ccb3fba98 100644
--- a/src/distributed/bfs_subcursor.cpp
+++ b/src/distributed/bfs_subcursor.cpp
@@ -18,7 +18,7 @@ ExpandBfsSubcursor::ExpandBfsSubcursor(
     std::vector<storage::EdgeType> edge_types, query::GraphView graph_view,
     BfsRpcClients *bfs_subcursor_clients)
     : bfs_subcursor_clients_(bfs_subcursor_clients),
-      dba_(*db, tx_id),
+      dba_(db->Access(tx_id)),
       direction_(direction),
       edge_types_(std::move(edge_types)),
       graph_view_(graph_view) {
@@ -34,7 +34,7 @@ void ExpandBfsSubcursor::Reset() {
 
 void ExpandBfsSubcursor::SetSource(storage::VertexAddress source_address) {
   Reset();
-  auto source = VertexAccessor(source_address, dba_);
+  auto source = VertexAccessor(source_address, *dba_);
   SwitchAccessor(source, graph_view_);
   processed_.emplace(source, std::experimental::nullopt);
   ExpandFromVertex(source);
@@ -70,7 +70,7 @@ bool ExpandBfsSubcursor::ExpandToLocalVertex(storage::EdgeAddress edge,
   CHECK(vertex.address().is_local())
       << "ExpandToLocalVertex called with remote vertex";
 
-  edge = dba_.db().storage().LocalizedAddressIfPossible(edge);
+  edge = dba_->db().storage().LocalizedAddressIfPossible(edge);
   SwitchAccessor(vertex, graph_view_);
 
   std::lock_guard<std::mutex> lock(mutex_);
@@ -83,13 +83,13 @@ bool ExpandBfsSubcursor::ExpandToLocalVertex(storage::EdgeAddress edge,
 
 bool ExpandBfsSubcursor::ExpandToLocalVertex(storage::EdgeAddress edge,
                                              storage::VertexAddress vertex) {
-  auto vertex_accessor = VertexAccessor(vertex, dba_);
-  return ExpandToLocalVertex(edge, VertexAccessor(vertex, dba_));
+  auto vertex_accessor = VertexAccessor(vertex, *dba_);
+  return ExpandToLocalVertex(edge, VertexAccessor(vertex, *dba_));
 }
 
 PathSegment ExpandBfsSubcursor::ReconstructPath(
     storage::EdgeAddress edge_address) {
-  EdgeAccessor edge(edge_address, dba_);
+  EdgeAccessor edge(edge_address, *dba_);
   CHECK(edge.address().is_local()) << "ReconstructPath called with remote edge";
   DCHECK(edge.from_addr().is_local()) << "`from` vertex should always be local";
   DCHECK(!edge.to_addr().is_local()) << "`to` vertex should be remote when "
@@ -103,7 +103,7 @@ PathSegment ExpandBfsSubcursor::ReconstructPath(
 
 PathSegment ExpandBfsSubcursor::ReconstructPath(
     storage::VertexAddress vertex_addr) {
-  VertexAccessor vertex(vertex_addr, dba_);
+  VertexAccessor vertex(vertex_addr, *dba_);
   CHECK(vertex.address().is_local())
       << "ReconstructPath called with remote vertex";
   PathSegment result;
@@ -126,7 +126,7 @@ void ExpandBfsSubcursor::ReconstructPathHelper(VertexAccessor vertex,
       break;
     }
 
-    result->edges.emplace_back(*in_edge_address, dba_);
+    result->edges.emplace_back(*in_edge_address, *dba_);
 
     auto &in_edge = result->edges.back();
     auto next_vertex_address =
@@ -139,7 +139,7 @@ void ExpandBfsSubcursor::ReconstructPathHelper(VertexAccessor vertex,
       break;
     }
 
-    vertex = VertexAccessor(next_vertex_address, dba_);
+    vertex = VertexAccessor(next_vertex_address, *dba_);
     in_edge_address = processed_[vertex];
   }
 }
diff --git a/src/distributed/bfs_subcursor.hpp b/src/distributed/bfs_subcursor.hpp
index 08cbfcc1a..3cdd81214 100644
--- a/src/distributed/bfs_subcursor.hpp
+++ b/src/distributed/bfs_subcursor.hpp
@@ -94,7 +94,7 @@ class ExpandBfsSubcursor {
 
   BfsRpcClients *bfs_subcursor_clients_{nullptr};
 
-  database::GraphDbAccessor dba_;
+  std::unique_ptr<database::GraphDbAccessor> dba_;
 
   /// IDs of subcursors on other workers, used when sending RPCs.
   std::unordered_map<int16_t, int64_t> subcursor_ids_;
diff --git a/src/distributed/data_rpc_server.cpp b/src/distributed/data_rpc_server.cpp
index 62c09ce4c..038e108ed 100644
--- a/src/distributed/data_rpc_server.cpp
+++ b/src/distributed/data_rpc_server.cpp
@@ -1,40 +1,42 @@
+#include "distributed/data_rpc_server.hpp"
+
 #include <memory>
 
-#include "data_rpc_server.hpp"
+#include "database/distributed_graph_db.hpp"
 #include "database/graph_db_accessor.hpp"
 #include "distributed/data_rpc_messages.hpp"
 
 namespace distributed {
 
-DataRpcServer::DataRpcServer(database::GraphDb &db,
-                             communication::rpc::Server &server)
+DataRpcServer::DataRpcServer(database::DistributedGraphDb *db,
+                             communication::rpc::Server *server)
     : db_(db), rpc_server_(server) {
-  rpc_server_.Register<VertexRpc>(
+  rpc_server_->Register<VertexRpc>(
       [this](const auto &req_reader, auto *res_builder) {
-        database::GraphDbAccessor dba(db_, req_reader.getMember().getTxId());
-        auto vertex = dba.FindVertex(req_reader.getMember().getGid(), false);
+        auto dba = db_->Access(req_reader.getMember().getTxId());
+        auto vertex = dba->FindVertex(req_reader.getMember().getGid(), false);
         CHECK(vertex.GetOld())
             << "Old record must exist when sending vertex by RPC";
-        VertexRes response(vertex.GetOld(), db_.WorkerId());
+        VertexRes response(vertex.GetOld(), db_->WorkerId());
         response.Save(res_builder);
       });
 
-  rpc_server_.Register<EdgeRpc>([this](const auto &req_reader,
-                                       auto *res_builder) {
-    database::GraphDbAccessor dba(db_, req_reader.getMember().getTxId());
-    auto edge = dba.FindEdge(req_reader.getMember().getGid(), false);
+  rpc_server_->Register<EdgeRpc>([this](const auto &req_reader,
+                                        auto *res_builder) {
+    auto dba = db_->Access(req_reader.getMember().getTxId());
+    auto edge = dba->FindEdge(req_reader.getMember().getGid(), false);
     CHECK(edge.GetOld()) << "Old record must exist when sending edge by RPC";
-    EdgeRes response(edge.GetOld(), db_.WorkerId());
+    EdgeRes response(edge.GetOld(), db_->WorkerId());
     response.Save(res_builder);
   });
 
-  rpc_server_.Register<VertexCountRpc>(
+  rpc_server_->Register<VertexCountRpc>(
       [this](const auto &req_reader, auto *res_builder) {
         VertexCountReq req;
         req.Load(req_reader);
-        database::GraphDbAccessor dba(db_, req.member);
+        auto dba = db_->Access(req.member);
         int64_t size = 0;
-        for (auto vertex : dba.Vertices(false)) ++size;
+        for (auto vertex : dba->Vertices(false)) ++size;
         VertexCountRes res(size);
         res.Save(res_builder);
       });
diff --git a/src/distributed/data_rpc_server.hpp b/src/distributed/data_rpc_server.hpp
index 91612a5cc..9fb47a336 100644
--- a/src/distributed/data_rpc_server.hpp
+++ b/src/distributed/data_rpc_server.hpp
@@ -3,15 +3,21 @@
 #include "communication/rpc/server.hpp"
 #include "database/graph_db.hpp"
 
+namespace database {
+class DistributedGraphDb;
+}
+
 namespace distributed {
 
 /// Serves this worker's data to others.
 class DataRpcServer {
  public:
-  DataRpcServer(database::GraphDb &db, communication::rpc::Server &server);
+  DataRpcServer(database::DistributedGraphDb *db,
+                communication::rpc::Server *server);
 
  private:
-  database::GraphDb &db_;
-  communication::rpc::Server &rpc_server_;
+  database::DistributedGraphDb *db_;
+  communication::rpc::Server *rpc_server_;
 };
+
 }  // namespace distributed
diff --git a/src/distributed/durability_rpc_worker.cpp b/src/distributed/durability_rpc_worker.cpp
index 3b2d0a48a..a9b976b7f 100644
--- a/src/distributed/durability_rpc_worker.cpp
+++ b/src/distributed/durability_rpc_worker.cpp
@@ -11,8 +11,8 @@ DurabilityRpcWorker::DurabilityRpcWorker(database::Worker *db,
     : db_(db), rpc_server_(server) {
   rpc_server_->Register<MakeSnapshotRpc>(
       [this](const auto &req_reader, auto *res_builder) {
-        database::GraphDbAccessor dba(*this->db_, req_reader.getMember());
-        MakeSnapshotRes res(this->db_->MakeSnapshot(dba));
+        auto dba = db_->Access(req_reader.getMember());
+        MakeSnapshotRes res(db_->MakeSnapshot(*dba));
         res.Save(res_builder);
       });
 
diff --git a/src/distributed/index_rpc_server.cpp b/src/distributed/index_rpc_server.cpp
index ea47b6373..9a782c480 100644
--- a/src/distributed/index_rpc_server.cpp
+++ b/src/distributed/index_rpc_server.cpp
@@ -1,5 +1,6 @@
 #include "database/graph_db.hpp"
 #include "database/graph_db_accessor.hpp"
+#include "distributed/index_rpc_messages.hpp"
 #include "distributed/index_rpc_server.hpp"
 
 namespace distributed {
@@ -20,9 +21,9 @@ IndexRpcServer::IndexRpcServer(database::GraphDb &db,
         PopulateIndexReq req;
         req.Load(req_reader);
         database::LabelPropertyIndex::Key key{req.label, req.property};
-        database::GraphDbAccessor dba(db_, req.tx_id);
-        dba.PopulateIndex(key);
-        dba.EnableIndex(key);
+        auto dba = db_.Access(req.tx_id);
+        dba->PopulateIndex(key);
+        dba->EnableIndex(key);
       });
 }
 
diff --git a/src/distributed/produce_rpc_server.cpp b/src/distributed/produce_rpc_server.cpp
index 456e65881..642ed1d4f 100644
--- a/src/distributed/produce_rpc_server.cpp
+++ b/src/distributed/produce_rpc_server.cpp
@@ -1,4 +1,6 @@
 #include "distributed/produce_rpc_server.hpp"
+
+#include "database/distributed_graph_db.hpp"
 #include "distributed/data_manager.hpp"
 #include "distributed/pull_produce_rpc_messages.hpp"
 #include "query/common.hpp"
@@ -8,15 +10,15 @@
 namespace distributed {
 
 ProduceRpcServer::OngoingProduce::OngoingProduce(
-    database::GraphDb &db, tx::TransactionId tx_id,
+    database::Worker *db, tx::TransactionId tx_id,
     std::shared_ptr<query::plan::LogicalOperator> op,
     query::SymbolTable symbol_table, Parameters parameters, int64_t timestamp,
     std::vector<query::Symbol> pull_symbols)
-    : dba_{db, tx_id},
-      context_(dba_),
+    : dba_(db->Access(tx_id)),
+      context_(*dba_),
       pull_symbols_(std::move(pull_symbols)),
       frame_(symbol_table.max_position()),
-      cursor_(op->MakeCursor(dba_)) {
+      cursor_(op->MakeCursor(*dba_)) {
   context_.symbol_table_ = std::move(symbol_table);
   context_.parameters_ = std::move(parameters);
   context_.timestamp_ = timestamp;
@@ -95,11 +97,11 @@ ProduceRpcServer::OngoingProduce::PullOneFromCursor() {
   return std::make_pair(std::move(results), cursor_state_);
 }
 
-ProduceRpcServer::ProduceRpcServer(
-    database::GraphDb &db, tx::Engine &tx_engine,
-    communication::rpc::Server &server,
-    const PlanConsumer &plan_consumer,
-    DataManager *data_manager)
+ProduceRpcServer::ProduceRpcServer(database::Worker *db,
+                                   tx::WorkerEngine *tx_engine,
+                                   communication::rpc::Server &server,
+                                   const PlanConsumer &plan_consumer,
+                                   DataManager *data_manager)
     : db_(db),
       produce_rpc_server_(server),
       plan_consumer_(plan_consumer),
@@ -127,7 +129,7 @@ ProduceRpcServer::ProduceRpcServer(
       [this, data_manager](const auto &req_reader, auto *res_builder) {
         TransactionCommandAdvancedReq req;
         req.Load(req_reader);
-        tx_engine_.UpdateCommand(req.member);
+        tx_engine_->UpdateCommand(req.member);
         data_manager->ClearCacheForSingleTransaction(req.member);
         TransactionCommandAdvancedRes res;
         res.Save(res_builder);
@@ -154,11 +156,8 @@ ProduceRpcServer::OngoingProduce &ProduceRpcServer::GetOngoingProduce(
   if (found != ongoing_produces_.end()) {
     return found->second;
   }
-  if (db_.type() == database::GraphDb::Type::DISTRIBUTED_WORKER) {
-    // On the worker cache the snapshot to have one RPC less.
-    dynamic_cast<tx::WorkerEngine &>(tx_engine_)
-        .RunningTransaction(req.tx_id, req.tx_snapshot);
-  }
+  // On the worker cache the snapshot to have one RPC less.
+  tx_engine_->RunningTransaction(req.tx_id, req.tx_snapshot);
   auto &plan_pack = plan_consumer_.PlanForId(req.plan_id);
   return ongoing_produces_
       .emplace(std::piecewise_construct, std::forward_as_tuple(key_tuple),
@@ -171,7 +170,7 @@ ProduceRpcServer::OngoingProduce &ProduceRpcServer::GetOngoingProduce(
 PullResData ProduceRpcServer::Pull(const PullReq &req) {
   auto &ongoing_produce = GetOngoingProduce(req);
 
-  PullResData result(db_.WorkerId(), req.send_old, req.send_new);
+  PullResData result(db_->WorkerId(), req.send_old, req.send_new);
   result.pull_state = PullState::CURSOR_IN_PROGRESS;
 
   if (req.accumulate) {
diff --git a/src/distributed/produce_rpc_server.hpp b/src/distributed/produce_rpc_server.hpp
index 60570cfeb..0927d889d 100644
--- a/src/distributed/produce_rpc_server.hpp
+++ b/src/distributed/produce_rpc_server.hpp
@@ -16,9 +16,16 @@
 #include "query/parameters.hpp"
 #include "query/plan/operator.hpp"
 #include "query/typed_value.hpp"
-#include "transactions/engine.hpp"
 #include "transactions/type.hpp"
 
+namespace database {
+class Worker;
+}
+
+namespace tx {
+class WorkerEngine;
+}
+
 namespace distributed {
 
 class DataManager;
@@ -34,7 +41,7 @@ class ProduceRpcServer {
   /// MG (see query::plan::Synchronize).
   class OngoingProduce {
    public:
-    OngoingProduce(database::GraphDb &db, tx::TransactionId tx_id,
+    OngoingProduce(database::Worker *db, tx::TransactionId tx_id,
                    std::shared_ptr<query::plan::LogicalOperator> op,
                    query::SymbolTable symbol_table, Parameters parameters,
                    int64_t timestamp, std::vector<query::Symbol> pull_symbols);
@@ -51,7 +58,7 @@ class ProduceRpcServer {
     void Reset();
 
    private:
-    database::GraphDbAccessor dba_;
+    std::unique_ptr<database::GraphDbAccessor> dba_;
     query::Context context_;
     std::vector<query::Symbol> pull_symbols_;
     query::Frame frame_;
@@ -64,7 +71,7 @@ class ProduceRpcServer {
   };
 
  public:
-  ProduceRpcServer(database::GraphDb &db, tx::Engine &tx_engine,
+  ProduceRpcServer(database::Worker *db, tx::WorkerEngine *tx_engine,
                    communication::rpc::Server &server,
                    const PlanConsumer &plan_consumer,
                    DataManager *data_manager);
@@ -81,10 +88,10 @@ class ProduceRpcServer {
   std::map<std::tuple<tx::TransactionId, tx::CommandId, int64_t>,
            OngoingProduce>
       ongoing_produces_;
-  database::GraphDb &db_;
+  database::Worker *db_;
   communication::rpc::Server &produce_rpc_server_;
   const distributed::PlanConsumer &plan_consumer_;
-  tx::Engine &tx_engine_;
+  tx::WorkerEngine *tx_engine_;
 
   /// Gets an ongoing produce for the given pull request. Creates a new one if
   /// there is none currently existing.
diff --git a/src/distributed/pull_produce_rpc_messages.lcp b/src/distributed/pull_produce_rpc_messages.lcp
index 2847a654d..a58bb6cd8 100644
--- a/src/distributed/pull_produce_rpc_messages.lcp
+++ b/src/distributed/pull_produce_rpc_messages.lcp
@@ -36,6 +36,9 @@ cpp<#
 (lcp:capnp-type-conversion "tx::TransactionId" "UInt64")
 
 #>cpp
+// Forward declare for LoadGraphElement.
+class DataManager;
+
 /// The default number of results returned via RPC from remote execution to the
 /// master that requested it.
 constexpr int kDefaultBatchSize = 20;
@@ -97,8 +100,8 @@ the relevant parts of the response, ready for use."))
                  query::TypedValue value;
                  utils::LoadCapnpTypedValue(
                      value_reader, &value,
-                     [this, dba](const auto &reader, auto *value) {
-                       this->LoadGraphElement(dba, reader, value);
+                     [this, dba, data_manager](const auto &reader, auto *value) {
+                       this->LoadGraphElement(dba, reader, value, data_manager);
                      });
                  current_frame.emplace_back(value);
                }
@@ -302,9 +305,10 @@ to the appropriate value. Not used on side that generates the response.")
                          distributed::capnp::TypedValue::Builder *) const;
    void LoadGraphElement(database::GraphDbAccessor *,
                          const distributed::capnp::TypedValue::Reader &,
-                         query::TypedValue *);
+                         query::TypedValue *, distributed::DataManager *);
    cpp<#)
-  (:serialize :capnp :load-args '((dba "database::GraphDbAccessor *"))))
+  (:serialize :capnp :load-args '((dba "database::GraphDbAccessor *")
+                                  (data-manager "distributed::DataManager *"))))
 
 (lcp:in-impl
  #>cpp
@@ -369,14 +373,7 @@ to the appropriate value. Not used on side that generates the response.")
 void PullResData::LoadGraphElement(
     database::GraphDbAccessor *dba,
     const distributed::capnp::TypedValue::Reader &reader,
-    query::TypedValue *value) {
-  distributed::DataManager *data_manager = nullptr;
-  // TODO: Pass in a DistributedGraphDb or data_manager.
-  if (auto *distributed_db =
-          dynamic_cast<database::DistributedGraphDb *>(&dba->db())) {
-    data_manager = &distributed_db->data_manager();
-  }
-  CHECK(data_manager);
+    query::TypedValue *value, distributed::DataManager *data_manager) {
   auto load_vertex = [dba, data_manager](const auto &vertex_reader) {
     storage::VertexAddress global_address(vertex_reader.getAddress());
     auto old_record =
@@ -537,7 +534,8 @@ cpp<#)
             }
           }
           "))
-   (:serialize :capnp :base t :load-args '((dba "database::GraphDbAccessor *")))))
+   (:serialize :capnp :base t :load-args '((dba "database::GraphDbAccessor *")
+                                           (data-manager "distributed::DataManager *")))))
 
 ;; TODO make a separate RPC for the continuation of an existing pull, as an
 ;; optimization not to have to send the full PullReqData pack every time.
diff --git a/src/distributed/pull_rpc_clients.cpp b/src/distributed/pull_rpc_clients.cpp
index d03f82e72..24b7f7b72 100644
--- a/src/distributed/pull_rpc_clients.cpp
+++ b/src/distributed/pull_rpc_clients.cpp
@@ -1,6 +1,5 @@
 #include <functional>
 
-#include "distributed/data_manager.hpp"
 #include "distributed/pull_rpc_clients.hpp"
 #include "storage/edge.hpp"
 #include "storage/vertex.hpp"
@@ -12,13 +11,14 @@ utils::Future<PullData> PullRpcClients::Pull(
     tx::CommandId command_id, const Parameters &params,
     const std::vector<query::Symbol> &symbols, int64_t timestamp,
     bool accumulate, int batch_size) {
-  return clients_.ExecuteOnWorker<
-      PullData>(worker_id, [dba, plan_id, command_id, params, symbols,
-                            timestamp, accumulate, batch_size](
-                               int worker_id, ClientPool &client_pool) {
-    auto load_pull_res = [dba](const auto &res_reader) {
+  return clients_->ExecuteOnWorker<
+      PullData>(worker_id, [data_manager = data_manager_, dba, plan_id,
+                            command_id, params, symbols, timestamp, accumulate,
+                            batch_size](int worker_id,
+                                        ClientPool &client_pool) {
+    auto load_pull_res = [data_manager, dba](const auto &res_reader) {
       PullRes res;
-      res.Load(res_reader, dba);
+      res.Load(res_reader, dba, data_manager);
       return res;
     };
     auto result = client_pool.CallWithLoad<PullRpc>(
@@ -32,7 +32,7 @@ utils::Future<PullData> PullRpcClients::Pull(
 utils::Future<void> PullRpcClients::ResetCursor(database::GraphDbAccessor *dba,
                                                 int worker_id, int64_t plan_id,
                                                 tx::CommandId command_id) {
-  return clients_.ExecuteOnWorker<void>(
+  return clients_->ExecuteOnWorker<void>(
       worker_id, [dba, plan_id, command_id](int worker_id, auto &client) {
         auto res = client.template Call<ResetCursorRpc>(dba->transaction_id(),
                                                         plan_id, command_id);
@@ -42,7 +42,7 @@ utils::Future<void> PullRpcClients::ResetCursor(database::GraphDbAccessor *dba,
 
 std::vector<utils::Future<void>>
 PullRpcClients::NotifyAllTransactionCommandAdvanced(tx::TransactionId tx_id) {
-  return clients_.ExecuteOnWorkers<void>(
+  return clients_->ExecuteOnWorkers<void>(
       0, [tx_id](int worker_id, auto &client) {
         auto res = client.template Call<TransactionCommandAdvancedRpc>(tx_id);
         CHECK(res) << "TransactionCommandAdvanceRpc failed";
diff --git a/src/distributed/pull_rpc_clients.hpp b/src/distributed/pull_rpc_clients.hpp
index 030bcecb7..e1c8a9a3f 100644
--- a/src/distributed/pull_rpc_clients.hpp
+++ b/src/distributed/pull_rpc_clients.hpp
@@ -12,6 +12,8 @@
 
 namespace distributed {
 
+class DataManager;
+
 /// Provides means of calling for the execution of a plan on some remote worker,
 /// and getting the results of that execution. The results are returned in
 /// batches and are therefore accompanied with an enum indicator of the state of
@@ -20,7 +22,8 @@ class PullRpcClients {
   using ClientPool = communication::rpc::ClientPool;
 
  public:
-  PullRpcClients(RpcWorkerClients &clients) : clients_(clients) {}
+  PullRpcClients(RpcWorkerClients *clients, DataManager *data_manager)
+      : clients_(clients), data_manager_(data_manager) {}
 
   /// Calls a remote pull asynchroniously. IMPORTANT: take care not to call this
   /// function for the same (tx_id, worker_id, plan_id, command_id) before the
@@ -39,13 +42,14 @@ class PullRpcClients {
   utils::Future<void> ResetCursor(database::GraphDbAccessor *dba, int worker_id,
                                   int64_t plan_id, tx::CommandId command_id);
 
-  auto GetWorkerIds() { return clients_.GetWorkerIds(); }
+  auto GetWorkerIds() { return clients_->GetWorkerIds(); }
 
   std::vector<utils::Future<void>> NotifyAllTransactionCommandAdvanced(
       tx::TransactionId tx_id);
 
  private:
-  RpcWorkerClients &clients_;
+  RpcWorkerClients *clients_{nullptr};
+  DataManager *data_manager_{nullptr};
 };
 
 }  // namespace distributed
diff --git a/src/distributed/updates_rpc_server.cpp b/src/distributed/updates_rpc_server.cpp
index 106d2d8f5..d3f967029 100644
--- a/src/distributed/updates_rpc_server.cpp
+++ b/src/distributed/updates_rpc_server.cpp
@@ -1,8 +1,9 @@
+#include "distributed/updates_rpc_server.hpp"
+
 #include <utility>
 
-#include "glog/logging.h"
+#include <glog/logging.h>
 
-#include "distributed/updates_rpc_server.hpp"
 #include "utils/thread/sync.hpp"
 
 namespace distributed {
@@ -63,7 +64,7 @@ gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateVertex(
     const std::vector<storage::Label> &labels,
     const std::unordered_map<storage::Property, query::TypedValue>
         &properties) {
-  auto result = db_accessor_.InsertVertex();
+  auto result = db_accessor_->InsertVertex();
   for (auto &label : labels) result.add_label(label);
   for (auto &kv : properties) result.PropsSet(kv.first, kv.second);
   std::lock_guard<utils::SpinLock> guard{lock_};
@@ -74,12 +75,13 @@ gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateVertex(
 
 template <typename TRecordAccessor>
 gid::Gid UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::CreateEdge(
-    gid::Gid from, storage::VertexAddress to, storage::EdgeType edge_type) {
-  auto &db = db_accessor_.db();
+    gid::Gid from, storage::VertexAddress to, storage::EdgeType edge_type,
+    int worker_id) {
+  auto &db = db_accessor_->db();
   auto from_addr = db.storage().LocalizedAddressIfPossible(
-      storage::VertexAddress(from, db.WorkerId()));
+      storage::VertexAddress(from, worker_id));
   auto to_addr = db.storage().LocalizedAddressIfPossible(to);
-  auto edge = db_accessor_.InsertOnlyEdge(from_addr, to_addr, edge_type);
+  auto edge = db_accessor_->InsertOnlyEdge(from_addr, to_addr, edge_type);
   std::lock_guard<utils::SpinLock> guard{lock_};
   deltas_.emplace(edge.gid(),
                   std::make_pair(edge, std::vector<database::StateDelta>{}));
@@ -96,7 +98,7 @@ UpdateResult UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::Apply() {
     record_accessor.Reconstruct();
     for (database::StateDelta &delta : kv.second.second) {
       try {
-        auto &dba = db_accessor_;
+        auto &dba = *db_accessor_;
         switch (delta.type) {
           case database::StateDelta::Type::TRANSACTION_BEGIN:
           case database::StateDelta::Type::TRANSACTION_COMMIT:
@@ -147,7 +149,7 @@ UpdateResult UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::Apply() {
             // We only remove the edge as a result of this StateDelta,
             // because the removal of edge from vertex in/out is performed
             // in REMOVE_[IN/OUT]_EDGE deltas.
-            db_accessor_.RemoveEdge(
+            db_accessor_->RemoveEdge(
                 reinterpret_cast<EdgeAccessor &>(record_accessor), false,
                 false);
             break;
@@ -172,10 +174,11 @@ UpdateResult UpdatesRpcServer::TransactionUpdates<TRecordAccessor>::Apply() {
   return UpdateResult::DONE;
 }
 
-UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
-                                   communication::rpc::Server &server)
+UpdatesRpcServer::UpdatesRpcServer(database::DistributedGraphDb *db,
+                                   communication::rpc::Server *server)
     : db_(db) {
-  server.Register<UpdateRpc>([this](const auto &req_reader, auto *res_builder) {
+  server->Register<UpdateRpc>([this](const auto &req_reader,
+                                     auto *res_builder) {
     UpdateReq req;
     req.Load(req_reader);
     using DeltaType = database::StateDelta::Type;
@@ -203,7 +206,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
     }
   });
 
-  server.Register<UpdateApplyRpc>(
+  server->Register<UpdateApplyRpc>(
       [this](const auto &req_reader, auto *res_builder) {
         UpdateApplyReq req;
         req.Load(req_reader);
@@ -211,8 +214,8 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
         res.Save(res_builder);
       });
 
-  server.Register<CreateVertexRpc>([this](const auto &req_reader,
-                                          auto *res_builder) {
+  server->Register<CreateVertexRpc>([this](const auto &req_reader,
+                                           auto *res_builder) {
     CreateVertexReq req;
     req.Load(req_reader);
     gid::Gid gid = GetUpdates(vertex_updates_, req.member.tx_id)
@@ -221,7 +224,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
     res.Save(res_builder);
   });
 
-  server.Register<CreateEdgeRpc>(
+  server->Register<CreateEdgeRpc>(
       [this](const auto &req_reader, auto *res_builder) {
         CreateEdgeReq req;
         req.Load(req_reader);
@@ -231,10 +234,10 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
         // If `from` and `to` are both on this worker, we handle it in this
         // RPC call. Do it only if CreateEdge succeeded.
         if (creation_result.result == UpdateResult::DONE &&
-            data.to.worker_id() == db_.WorkerId()) {
+            data.to.worker_id() == db_->WorkerId()) {
           auto to_delta = database::StateDelta::AddInEdge(
-              data.tx_id, data.to.gid(), {data.from, db_.WorkerId()},
-              {creation_result.gid, db_.WorkerId()}, data.edge_type);
+              data.tx_id, data.to.gid(), {data.from, db_->WorkerId()},
+              {creation_result.gid, db_->WorkerId()}, data.edge_type);
           creation_result.result =
               GetUpdates(vertex_updates_, data.tx_id).Emplace(to_delta);
         }
@@ -243,7 +246,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
         res.Save(res_builder);
       });
 
-  server.Register<AddInEdgeRpc>(
+  server->Register<AddInEdgeRpc>(
       [this](const auto &req_reader, auto *res_builder) {
         AddInEdgeReq req;
         req.Load(req_reader);
@@ -256,7 +259,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
         res.Save(res_builder);
       });
 
-  server.Register<RemoveVertexRpc>(
+  server->Register<RemoveVertexRpc>(
       [this](const auto &req_reader, auto *res_builder) {
         RemoveVertexReq req;
         req.Load(req_reader);
@@ -268,7 +271,7 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
         res.Save(res_builder);
       });
 
-  server.Register<RemoveEdgeRpc>(
+  server->Register<RemoveEdgeRpc>(
       [this](const auto &req_reader, auto *res_builder) {
         RemoveEdgeReq req;
         req.Load(req_reader);
@@ -276,8 +279,8 @@ UpdatesRpcServer::UpdatesRpcServer(database::GraphDb &db,
         res.Save(res_builder);
       });
 
-  server.Register<RemoveInEdgeRpc>([this](const auto &req_reader,
-                                          auto *res_builder) {
+  server->Register<RemoveInEdgeRpc>([this](const auto &req_reader,
+                                           auto *res_builder) {
     RemoveInEdgeReq req;
     req.Load(req_reader);
     auto data = req.member;
@@ -335,10 +338,10 @@ UpdatesRpcServer::TransactionUpdates<TAccessor> &UpdatesRpcServer::GetUpdates(
 
 CreateResult UpdatesRpcServer::CreateEdge(const CreateEdgeReqData &req) {
   auto gid = GetUpdates(edge_updates_, req.tx_id)
-                 .CreateEdge(req.from, req.to, req.edge_type);
+                 .CreateEdge(req.from, req.to, req.edge_type, db_->WorkerId());
 
   auto from_delta = database::StateDelta::AddOutEdge(
-      req.tx_id, req.from, req.to, {gid, db_.WorkerId()}, req.edge_type);
+      req.tx_id, req.from, req.to, {gid, db_->WorkerId()}, req.edge_type);
 
   auto result = GetUpdates(vertex_updates_, req.tx_id).Emplace(from_delta);
   return {result, gid};
@@ -353,16 +356,16 @@ UpdateResult UpdatesRpcServer::RemoveEdge(const RemoveEdgeData &data) {
   // Out-edge removal, for sure is local.
   if (result == UpdateResult::DONE) {
     auto remove_out_delta = database::StateDelta::RemoveOutEdge(
-        data.tx_id, data.vertex_from_id, {data.edge_id, db_.WorkerId()});
+        data.tx_id, data.vertex_from_id, {data.edge_id, db_->WorkerId()});
     result = GetUpdates(vertex_updates_, data.tx_id).Emplace(remove_out_delta);
   }
 
   // In-edge removal, might not be local.
   if (result == UpdateResult::DONE &&
-      data.vertex_to_address.worker_id() == db_.WorkerId()) {
+      data.vertex_to_address.worker_id() == db_->WorkerId()) {
     auto remove_in_delta = database::StateDelta::RemoveInEdge(
         data.tx_id, data.vertex_to_address.gid(),
-        {data.edge_id, db_.WorkerId()});
+        {data.edge_id, db_->WorkerId()});
     result = GetUpdates(vertex_updates_, data.tx_id).Emplace(remove_in_delta);
   }
 
@@ -373,13 +376,13 @@ template <>
 VertexAccessor
 UpdatesRpcServer::TransactionUpdates<VertexAccessor>::FindAccessor(
     gid::Gid gid) {
-  return db_accessor_.FindVertex(gid, false);
+  return db_accessor_->FindVertex(gid, false);
 }
 
 template <>
 EdgeAccessor UpdatesRpcServer::TransactionUpdates<EdgeAccessor>::FindAccessor(
     gid::Gid gid) {
-  return db_accessor_.FindEdge(gid, false);
+  return db_accessor_->FindEdge(gid, false);
 }
 
 }  // namespace distributed
diff --git a/src/distributed/updates_rpc_server.hpp b/src/distributed/updates_rpc_server.hpp
index dc81d9eb6..9052597a4 100644
--- a/src/distributed/updates_rpc_server.hpp
+++ b/src/distributed/updates_rpc_server.hpp
@@ -7,7 +7,7 @@
 
 #include "communication/rpc/server.hpp"
 #include "data_structures/concurrent/concurrent_map.hpp"
-#include "database/graph_db.hpp"
+#include "database/distributed_graph_db.hpp"
 #include "database/graph_db_accessor.hpp"
 #include "database/state_delta.hpp"
 #include "distributed/updates_rpc_messages.hpp"
@@ -32,8 +32,9 @@ class UpdatesRpcServer {
   template <typename TRecordAccessor>
   class TransactionUpdates {
    public:
-    TransactionUpdates(database::GraphDb &db, tx::TransactionId tx_id)
-        : db_accessor_(db, tx_id) {}
+    TransactionUpdates(database::DistributedGraphDb *db,
+                       tx::TransactionId tx_id)
+        : db_accessor_(db->Access(tx_id)) {}
 
     /// Adds a delta and returns the result. Does not modify the state (data) of
     /// the graph element the update is for, but calls the `update` method to
@@ -49,15 +50,15 @@ class UpdatesRpcServer {
     /// Creates a new edge and returns it's gid. Does not update vertices at the
     /// end of the edge.
     gid::Gid CreateEdge(gid::Gid from, storage::VertexAddress to,
-                        storage::EdgeType edge_type);
+                        storage::EdgeType edge_type, int worker_id);
 
     /// Applies all the deltas on the record.
     UpdateResult Apply();
 
-    auto &db_accessor() { return db_accessor_; }
+    auto &db_accessor() { return *db_accessor_; }
 
    private:
-    database::GraphDbAccessor db_accessor_;
+    std::unique_ptr<database::GraphDbAccessor> db_accessor_;
     std::unordered_map<
         gid::Gid, std::pair<TRecordAccessor, std::vector<database::StateDelta>>>
         deltas_;
@@ -69,7 +70,8 @@ class UpdatesRpcServer {
   };
 
  public:
-  UpdatesRpcServer(database::GraphDb &db, communication::rpc::Server &server);
+  UpdatesRpcServer(database::DistributedGraphDb *db,
+                   communication::rpc::Server *server);
 
   /// Applies all existsing updates for the given transaction ID. If there are
   /// no updates for that transaction, nothing happens. Clears the updates cache
@@ -81,11 +83,10 @@ class UpdatesRpcServer {
   void ClearTransactionalCache(tx::TransactionId oldest_active);
 
  private:
-  database::GraphDb &db_;
+  database::DistributedGraphDb *db_;
 
   template <typename TAccessor>
-  using MapT =
-      ConcurrentMap<tx::TransactionId, TransactionUpdates<TAccessor>>;
+  using MapT = ConcurrentMap<tx::TransactionId, TransactionUpdates<TAccessor>>;
   MapT<VertexAccessor> vertex_updates_;
   MapT<EdgeAccessor> edge_updates_;
 
diff --git a/src/durability/recovery.cpp b/src/durability/recovery.cpp
index 8c99abaf0..11e4b4766 100644
--- a/src/durability/recovery.cpp
+++ b/src/durability/recovery.cpp
@@ -4,6 +4,7 @@
 #include <limits>
 #include <unordered_map>
 
+#include "database/distributed_graph_db.hpp"
 #include "database/graph_db_accessor.hpp"
 #include "database/indexes/label_property_index.hpp"
 #include "durability/hashed_file_reader.hpp"
@@ -66,9 +67,18 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db,
   RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int) &&
                 dv.ValueInt() == durability::kVersion);
 
+  int worker_id = 0;
+  // TODO: Figure out a better solution for SingleNode recovery vs
+  // DistributedGraphDb.
+  if (auto *distributed_db =
+          dynamic_cast<database::DistributedGraphDb *>(db)) {
+    worker_id = distributed_db->WorkerId();
+  } else {
+    CHECK(dynamic_cast<database::SingleNode *>(db));
+  }
   // Checks worker id was set correctly
   RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int) &&
-                dv.ValueInt() == db->WorkerId());
+                dv.ValueInt() == worker_id);
 
   // Vertex and edge generator ids
   RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int));
@@ -101,7 +111,7 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db,
                                         property.ValueString());
   }
 
-  database::GraphDbAccessor dba(*db);
+  auto dba = db->Access();
   std::unordered_map<gid::Gid,
                      std::pair<storage::VertexAddress, storage::VertexAddress>>
       edge_gid_endpoints_mapping;
@@ -110,47 +120,47 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db,
     auto vertex = decoder.ReadSnapshotVertex();
     RETURN_IF_NOT(vertex);
 
-    auto vertex_accessor = dba.InsertVertex(vertex->gid, vertex->cypher_id);
+    auto vertex_accessor = dba->InsertVertex(vertex->gid, vertex->cypher_id);
     for (const auto &label : vertex->labels) {
-      vertex_accessor.add_label(dba.Label(label));
+      vertex_accessor.add_label(dba->Label(label));
     }
     for (const auto &property_pair : vertex->properties) {
-      vertex_accessor.PropsSet(dba.Property(property_pair.first),
+      vertex_accessor.PropsSet(dba->Property(property_pair.first),
                                glue::ToTypedValue(property_pair.second));
     }
     auto vertex_record = vertex_accessor.GetNew();
     for (const auto &edge : vertex->in) {
       vertex_record->in_.emplace(edge.vertex, edge.address,
-                                 dba.EdgeType(edge.type));
+                                 dba->EdgeType(edge.type));
       edge_gid_endpoints_mapping[edge.address.gid()] = {
           edge.vertex, vertex_accessor.GlobalAddress()};
     }
     for (const auto &edge : vertex->out) {
       vertex_record->out_.emplace(edge.vertex, edge.address,
-                                  dba.EdgeType(edge.type));
+                                  dba->EdgeType(edge.type));
       edge_gid_endpoints_mapping[edge.address.gid()] = {
           vertex_accessor.GlobalAddress(), edge.vertex};
     }
   }
 
   auto vertex_transform_to_local_if_possible =
-      [&db, &dba](storage::VertexAddress &address) {
+      [&dba, worker_id](storage::VertexAddress &address) {
         if (address.is_local()) return;
         // If the worker id matches it should be a local apperance
-        if (address.worker_id() == db->WorkerId()) {
+        if (address.worker_id() == worker_id) {
           address = storage::VertexAddress(
-              dba.db().storage().LocalAddress<Vertex>(address.gid()));
+              dba->db().storage().LocalAddress<Vertex>(address.gid()));
           CHECK(address.is_local()) << "Address should be local but isn't";
         }
       };
 
   auto edge_transform_to_local_if_possible =
-      [&db, &dba](storage::EdgeAddress &address) {
+      [&dba, worker_id](storage::EdgeAddress &address) {
         if (address.is_local()) return;
         // If the worker id matches it should be a local apperance
-        if (address.worker_id() == db->WorkerId()) {
+        if (address.worker_id() == worker_id) {
           address = storage::EdgeAddress(
-              dba.db().storage().LocalAddress<Edge>(address.gid()));
+              dba->db().storage().LocalAddress<Edge>(address.gid()));
           CHECK(address.is_local()) << "Address should be local but isn't";
         }
       };
@@ -181,11 +191,11 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db,
     vertex_transform_to_local_if_possible(from);
     vertex_transform_to_local_if_possible(to);
 
-    auto edge_accessor = dba.InsertOnlyEdge(from, to, dba.EdgeType(edge.type),
-                                            edge.id.AsUint(), cypher_id);
+    auto edge_accessor = dba->InsertOnlyEdge(from, to, dba->EdgeType(edge.type),
+                                             edge.id.AsUint(), cypher_id);
 
     for (const auto &property_pair : edge.properties)
-      edge_accessor.PropsSet(dba.Property(property_pair.first),
+      edge_accessor.PropsSet(dba->Property(property_pair.first),
                              glue::ToTypedValue(property_pair.second));
   }
 
@@ -194,14 +204,14 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db,
   reader.ReadType(vertex_count);
   reader.ReadType(edge_count);
   if (!reader.Close() || reader.hash() != hash) {
-    dba.Abort();
+    dba->Abort();
     return false;
   }
 
   // We have to replace global_ids with local ids where possible for all edges
   // in every vertex and this can only be done after we inserted the edges; this
   // is to speedup execution
-  for (auto &vertex_accessor : dba.Vertices(true)) {
+  for (auto &vertex_accessor : dba->Vertices(true)) {
     auto vertex = vertex_accessor.GetNew();
     auto iterate_and_transform =
         [vertex_transform_to_local_if_possible,
@@ -231,8 +241,8 @@ bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db,
   tx::TransactionId max_id = recovery_data->snapshooter_tx_id;
   auto &snap = recovery_data->snapshooter_tx_snapshot;
   if (!snap.empty()) max_id = *std::max_element(snap.begin(), snap.end());
-  dba.db().tx_engine().EnsureNextIdGreater(max_id);
-  dba.Commit();
+  dba->db().tx_engine().EnsureNextIdGreater(max_id);
+  dba->Commit();
   return true;
 }
 
@@ -319,7 +329,9 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db,
             !utils::Contains(common_wal_tx, tx_id);
   };
 
-  std::unordered_map<tx::TransactionId, database::GraphDbAccessor> accessors;
+  std::unordered_map<tx::TransactionId,
+                     std::unique_ptr<database::GraphDbAccessor>>
+      accessors;
   auto get_accessor =
       [db, &accessors](tx::TransactionId tx_id) -> database::GraphDbAccessor & {
     auto found = accessors.find(tx_id);
@@ -328,12 +340,13 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db,
     // don't have a transaction begin, the accessors are not created.
     if (db->type() == database::GraphDb::Type::DISTRIBUTED_WORKER &&
         found == accessors.end()) {
-      std::tie(found, std::ignore) = accessors.emplace(tx_id, *db);
+      // TODO: Do we want to call db->Access with tx_id?
+      std::tie(found, std::ignore) = accessors.emplace(tx_id, db->Access());
     }
 
     CHECK(found != accessors.end())
         << "Accessor does not exist for transaction: " << tx_id;
-    return found->second;
+    return *found->second;
   };
 
   // Ensure that the next transaction ID in the recovered DB will be greater
@@ -353,7 +366,7 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db,
           case database::StateDelta::Type::TRANSACTION_BEGIN:
             CHECK(accessors.find(delta.transaction_id) == accessors.end())
                 << "Double transaction start";
-            accessors.emplace(delta.transaction_id, *db);
+            accessors.emplace(delta.transaction_id, db->Access());
             break;
           case database::StateDelta::Type::TRANSACTION_ABORT:
             get_accessor(delta.transaction_id).Abort();
@@ -381,6 +394,7 @@ bool RecoverWal(const fs::path &wal_dir, database::GraphDb *db,
   db->tx_engine().EnsureNextIdGreater(max_observed_tx_id);
   return true;
 }
+
 }  // anonymous namespace
 
 RecoveryInfo RecoverOnlySnapshot(
@@ -439,15 +453,16 @@ void RecoverWalAndIndexes(const fs::path &durability_dir, database::GraphDb *db,
   RecoverWal(durability_dir / kWalDir, db, recovery_data);
 
   // Index recovery.
-  database::GraphDbAccessor db_accessor_indices{*db};
+  auto db_accessor_indices = db->Access();
   for (const auto &label_prop : recovery_data->indexes) {
     const database::LabelPropertyIndex::Key key{
-        db_accessor_indices.Label(label_prop.first),
-        db_accessor_indices.Property(label_prop.second)};
-    db_accessor_indices.db().storage().label_property_index().CreateIndex(key);
-    db_accessor_indices.PopulateIndex(key);
-    db_accessor_indices.EnableIndex(key);
+        db_accessor_indices->Label(label_prop.first),
+        db_accessor_indices->Property(label_prop.second)};
+    db_accessor_indices->db().storage().label_property_index().CreateIndex(key);
+    db_accessor_indices->PopulateIndex(key);
+    db_accessor_indices->EnableIndex(key);
   }
-  db_accessor_indices.Commit();
+  db_accessor_indices->Commit();
 }
+
 }  // namespace durability
diff --git a/src/durability/snapshooter.cpp b/src/durability/snapshooter.cpp
index 6c4402424..31deeae66 100644
--- a/src/durability/snapshooter.cpp
+++ b/src/durability/snapshooter.cpp
@@ -1,9 +1,10 @@
+#include "durability/snapshooter.hpp"
+
 #include <algorithm>
 
 #include <glog/logging.h>
 
-#include "durability/snapshooter.hpp"
-
+#include "database/distributed_graph_db.hpp"
 #include "database/graph_db_accessor.hpp"
 #include "durability/hashed_file_writer.hpp"
 #include "durability/paths.hpp"
@@ -31,9 +32,18 @@ bool Encode(const fs::path &snapshot_file, database::GraphDb &db,
                      durability::kMagicNumber.size());
     encoder.WriteInt(durability::kVersion);
 
+    int worker_id = 0;
+    // TODO: Figure out a better solution for SingleNode recovery vs
+    // DistributedGraphDb.
+    if (auto *distributed_db =
+            dynamic_cast<database::DistributedGraphDb *>(&dba.db())) {
+      worker_id = distributed_db->WorkerId();
+    } else {
+      CHECK(dynamic_cast<database::SingleNode *>(&dba.db()));
+    }
     // Writes the worker id to snapshot, used to guarantee consistent cluster
     // state after recovery
-    encoder.WriteInt(db.WorkerId());
+    encoder.WriteInt(worker_id);
 
     // Write the number of generated vertex and edges, used to recover
     // generators internal states
@@ -125,8 +135,17 @@ bool MakeSnapshot(database::GraphDb &db, database::GraphDbAccessor &dba,
                   const fs::path &durability_dir,
                   const int snapshot_max_retained) {
   if (!utils::EnsureDir(durability_dir / kSnapshotDir)) return false;
+  int worker_id = 0;
+  // TODO: Figure out a better solution for SingleNode recovery vs
+  // DistributedGraphDb.
+  if (auto *distributed_db =
+          dynamic_cast<database::DistributedGraphDb *>(&db)) {
+    worker_id = distributed_db->WorkerId();
+  } else {
+    CHECK(dynamic_cast<database::SingleNode *>(&db));
+  }
   const auto snapshot_file =
-      MakeSnapshotPath(durability_dir, db.WorkerId(), dba.transaction_id());
+      MakeSnapshotPath(durability_dir, worker_id, dba.transaction_id());
   if (fs::exists(snapshot_file)) return false;
   if (Encode(snapshot_file, db, dba)) {
     RemoveOldSnapshots(durability_dir / kSnapshotDir, snapshot_max_retained);
diff --git a/src/memgraph_bolt.cpp b/src/memgraph_bolt.cpp
index 667d8b85b..dce8ef68b 100644
--- a/src/memgraph_bolt.cpp
+++ b/src/memgraph_bolt.cpp
@@ -268,14 +268,14 @@ void SingleNodeMain() {
   auto stream_writer =
       [&session_data](const std::vector<std::string> &queries) {
         for (auto &query : queries) {
-          database::GraphDbAccessor dba(session_data.db);
+          auto dba = session_data.db.Access();
           KafkaResultStream stream;
           try {
-            session_data.interpreter(query, dba, {}, false).PullAll(stream);
-            dba.Commit();
+            session_data.interpreter(query, *dba, {}, false).PullAll(stream);
+            dba->Commit();
           } catch (const query::QueryException &e) {
             LOG(ERROR) << e.what();
-            dba.Abort();
+            dba->Abort();
           }
         }
       };
@@ -314,8 +314,8 @@ void SingleNodeMain() {
             "telemetry",
         std::chrono::minutes(10));
     telemetry->AddCollector("db", [&db]() -> nlohmann::json {
-      database::GraphDbAccessor dba(db);
-      return {{"vertices", dba.VerticesCount()}, {"edges", dba.EdgesCount()}};
+      auto dba = db.Access();
+      return {{"vertices", dba->VerticesCount()}, {"edges", dba->EdgesCount()}};
     });
   }
 
@@ -358,14 +358,14 @@ void MasterMain() {
   auto stream_writer =
       [&session_data](const std::vector<std::string> &queries) {
         for (auto &query : queries) {
-          database::GraphDbAccessor dba(session_data.db);
+          auto dba = session_data.db.Access();
           KafkaResultStream stream;
           try {
-            session_data.interpreter(query, dba, {}, false).PullAll(stream);
-            dba.Commit();
+            session_data.interpreter(query, *dba, {}, false).PullAll(stream);
+            dba->Commit();
           } catch (const query::QueryException &e) {
             LOG(ERROR) << e.what();
-            dba.Abort();
+            dba->Abort();
           }
         }
       };
diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp
index 591dfca24..82e928b54 100644
--- a/src/query/plan/operator.cpp
+++ b/src/query/plan/operator.cpp
@@ -126,7 +126,7 @@ CreateNode::CreateNode(const std::shared_ptr<LogicalOperator> &input,
 namespace {
 
 // Returns a random worker id. Worker ID is obtained from the Db.
-int RandomWorkerId(database::GraphDb &db) {
+int RandomWorkerId(const database::DistributedGraphDb &db) {
   thread_local std::mt19937 gen_{std::random_device{}()};
   thread_local std::uniform_int_distribution<int> rand_;
 
@@ -156,7 +156,16 @@ VertexAccessor &CreateVertexOnWorker(int worker_id, NodeAtom *node_atom,
                                      Frame &frame, Context &context) {
   auto &dba = context.db_accessor_;
 
-  if (worker_id == dba.db().WorkerId())
+  int current_worker_id = 0;
+  // TODO: Figure out a better solution.
+  if (auto *distributed_db =
+      dynamic_cast<database::DistributedGraphDb *>(&dba.db())) {
+    current_worker_id = distributed_db->WorkerId();
+  } else {
+    CHECK(dynamic_cast<database::SingleNode *>(&dba.db()));
+  }
+
+  if (worker_id == current_worker_id)
     return CreateLocalVertex(node_atom, frame, context);
 
   std::unordered_map<storage::Property, query::TypedValue> properties;
@@ -173,11 +182,12 @@ VertexAccessor &CreateVertexOnWorker(int worker_id, NodeAtom *node_atom,
     properties.emplace(kv.first.second, std::move(value));
   }
 
-  auto new_node =
-      dba.InsertVertexIntoRemote(worker_id, node_atom->labels_, properties);
+  auto new_node = database::InsertVertexIntoRemote(
+      &dba, worker_id, node_atom->labels_, properties);
   frame[context.symbol_table_.at(*node_atom->identifier_)] = new_node;
   return frame[context.symbol_table_.at(*node_atom->identifier_)].ValueVertex();
 }
+
 }  // namespace
 
 ACCEPT_WITH_INPUT(CreateNode)
@@ -201,8 +211,12 @@ CreateNode::CreateNodeCursor::CreateNodeCursor(const CreateNode &self,
 bool CreateNode::CreateNodeCursor::Pull(Frame &frame, Context &context) {
   if (input_cursor_->Pull(frame, context)) {
     if (self_.on_random_worker_) {
-      CreateVertexOnWorker(RandomWorkerId(db_.db()), self_.node_atom_, frame,
-                           context);
+      // TODO: Replace this with some other mechanism
+      auto *distributed_db =
+          dynamic_cast<database::DistributedGraphDb *>(&db_.db());
+      CHECK(distributed_db);
+      CreateVertexOnWorker(RandomWorkerId(*distributed_db), self_.node_atom_,
+                           frame, context);
     } else {
       CreateLocalVertex(self_.node_atom_, frame, context);
     }
@@ -3475,7 +3489,10 @@ class SynchronizeCursor : public Cursor {
         input_cursor_(self.input()->MakeCursor(db)),
         pull_remote_cursor_(
             self.pull_remote() ? self.pull_remote()->MakeCursor(db) : nullptr),
-        command_id_(db.transaction().cid()) {}
+        command_id_(db.transaction().cid()),
+        master_id_(
+            // TODO: Pass in a Master GraphDb.
+            dynamic_cast<database::Master *>(&db.db())->WorkerId()) {}
 
   bool Pull(Frame &frame, Context &context) override {
     if (!initial_pull_done_) {
@@ -3526,17 +3543,17 @@ class SynchronizeCursor : public Cursor {
   bool initial_pull_done_{false};
   std::vector<std::vector<TypedValue>> local_frames_;
   tx::CommandId command_id_;
+  int master_id_;
 
   void InitialPull(Frame &frame, Context &context) {
     VLOG(10) << "[SynchronizeCursor] [" << context.db_accessor_.transaction_id()
              << "] initial pull";
-    auto &db = context.db_accessor_.db();
 
     // Tell all workers to accumulate, only if there is a remote pull.
     std::vector<utils::Future<distributed::PullData>> worker_accumulations;
     if (pull_remote_cursor_) {
       for (auto worker_id : pull_clients_->GetWorkerIds()) {
-        if (worker_id == db.WorkerId()) continue;
+        if (worker_id == master_id_) continue;
         worker_accumulations.emplace_back(pull_clients_->Pull(
             &context.db_accessor_, worker_id, self_.pull_remote()->plan_id(),
             command_id_, context.parameters_, self_.pull_remote()->symbols(),
@@ -3595,7 +3612,7 @@ class SynchronizeCursor : public Cursor {
 
     // Make all the workers apply their deltas.
     auto tx_id = context.db_accessor_.transaction_id();
-    auto apply_futures = updates_clients_->UpdateApplyAll(db.WorkerId(), tx_id);
+    auto apply_futures = updates_clients_->UpdateApplyAll(master_id_, tx_id);
     updates_server_->Apply(tx_id);
     for (auto &future : apply_futures) {
       switch (future.get()) {
diff --git a/src/query/repl.cpp b/src/query/repl.cpp
index c43f115e0..9ad528abc 100644
--- a/src/query/repl.cpp
+++ b/src/query/repl.cpp
@@ -63,14 +63,14 @@ void query::Repl(database::GraphDb &db) {
 
     // regular cypher queries
     try {
-      database::GraphDbAccessor dba(db);
+      auto dba = db.Access();
       ResultStreamFaker<query::TypedValue> stream;
-      auto results = interpeter(command, dba, {}, false);
+      auto results = interpeter(command, *dba, {}, false);
       stream.Header(results.header());
       results.PullAll(stream);
       stream.Summary(results.summary());
       std::cout << stream;
-      dba.Commit();
+      dba->Commit();
     } catch (const query::SyntaxException &e) {
       std::cout << "SYNTAX EXCEPTION: " << e.what() << std::endl;
     } catch (const query::LexingException &e) {
diff --git a/src/query/transaction_engine.hpp b/src/query/transaction_engine.hpp
index d8d0a3a7b..fde1bc1b4 100644
--- a/src/query/transaction_engine.hpp
+++ b/src/query/transaction_engine.hpp
@@ -60,7 +60,7 @@ class TransactionEngine final {
 
     // Create a DB accessor if we don't yet have one.
     if (!db_accessor_)
-      db_accessor_ = std::make_unique<database::GraphDbAccessor>(db_);
+      db_accessor_ = db_.Access();
 
     // Interpret the query and return the headers.
     try {
diff --git a/src/storage/dynamic_graph_partitioner/dgp.cpp b/src/storage/dynamic_graph_partitioner/dgp.cpp
index 637640a07..479080215 100644
--- a/src/storage/dynamic_graph_partitioner/dgp.cpp
+++ b/src/storage/dynamic_graph_partitioner/dgp.cpp
@@ -29,20 +29,20 @@ DynamicGraphPartitioner::DynamicGraphPartitioner(
     : db_(db) {}
 
 void DynamicGraphPartitioner::Run() {
-  database::GraphDbAccessor dba(*db_);
+  auto dba = db_->Access();
   VLOG(21) << "Starting DynamicGraphPartitioner in tx: "
-           << dba.transaction().id_;
+           << dba->transaction().id_;
 
-  auto migrations = FindMigrations(dba);
+  auto migrations = FindMigrations(*dba);
 
   try {
-    VertexMigrator migrator(&dba);
+    VertexMigrator migrator(dba.get());
     for (auto &migration : migrations) {
       migrator.MigrateVertex(migration.first, migration.second);
     }
 
     auto apply_futures = db_->updates_clients().UpdateApplyAll(
-        db_->WorkerId(), dba.transaction().id_);
+        db_->WorkerId(), dba->transaction().id_);
 
     for (auto &future : apply_futures) {
       switch (future.get()) {
@@ -62,11 +62,11 @@ void DynamicGraphPartitioner::Run() {
       }
     }
 
-    dba.Commit();
+    dba->Commit();
     VLOG(21) << "Sucesfully migrated " << migrations.size() << " vertices..";
   } catch (const utils::BasicException &e) {
     VLOG(21) << "Didn't succeed in relocating; " << e.what();
-    dba.Abort();
+    dba->Abort();
   }
 }
 
diff --git a/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp b/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp
index 7b4ec59d4..0fa958aca 100644
--- a/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp
+++ b/src/storage/dynamic_graph_partitioner/vertex_migrator.cpp
@@ -1,5 +1,6 @@
 #include "storage/dynamic_graph_partitioner/vertex_migrator.hpp"
 
+#include "database/distributed_graph_db.hpp"
 #include "database/graph_db_accessor.hpp"
 #include "query/typed_value.hpp"
 
@@ -20,8 +21,8 @@ void VertexMigrator::MigrateVertex(VertexAccessor &vertex, int destination) {
     }
   };
 
-  auto relocated_vertex = dba_->InsertVertexIntoRemote(
-      destination, vertex.labels(), get_props(vertex));
+  auto relocated_vertex = database::InsertVertexIntoRemote(
+      dba_, destination, vertex.labels(), get_props(vertex));
 
   vertex_migrated_to_[vertex.gid()] = relocated_vertex.address();
 
diff --git a/src/storage/record_accessor.cpp b/src/storage/record_accessor.cpp
index 4245b81d6..e7f2b2c70 100644
--- a/src/storage/record_accessor.cpp
+++ b/src/storage/record_accessor.cpp
@@ -109,9 +109,17 @@ typename RecordAccessor<TRecord>::AddressT RecordAccessor<TRecord>::address()
 template <typename TRecord>
 typename RecordAccessor<TRecord>::AddressT
 RecordAccessor<TRecord>::GlobalAddress() const {
-  return is_local() ? storage::Address<mvcc::VersionList<TRecord>>(
-                          gid(), db_accessor_->db_.WorkerId())
-                    : address_;
+  // TODO: Replace this with some other mechanism, i.e. virtual call.
+  int worker_id = 0;
+  if (auto *distributed_db =
+          dynamic_cast<database::DistributedGraphDb *>(&db_accessor_->db())) {
+    worker_id = distributed_db->WorkerId();
+  } else {
+    CHECK(dynamic_cast<database::SingleNode *>(&db_accessor_->db()));
+  }
+  return is_local()
+             ? storage::Address<mvcc::VersionList<TRecord>>(gid(), worker_id)
+             : address_;
 }
 
 template <typename TRecord>
diff --git a/src/utils/random_graph_generator.hpp b/src/utils/random_graph_generator.hpp
index 41bbe307b..8b7263d80 100644
--- a/src/utils/random_graph_generator.hpp
+++ b/src/utils/random_graph_generator.hpp
@@ -60,10 +60,10 @@ class RandomGraphGenerator {
    */
   void AddVertices(int count, const std::vector<std::string> &label_names,
                    int thread_count, int batch_size = 2000) {
-    database::GraphDbAccessor dba(db_);
+    auto dba = db_.Access();
     std::vector<storage::Label> labels;
     for (const auto &label_name : label_names)
-      labels.push_back(dba.Label(label_name));
+      labels.push_back(dba->Label(label_name));
 
     Map(
         [&labels, this](database::GraphDbAccessor &dba) {
@@ -80,8 +80,8 @@ class RandomGraphGenerator {
    * regardless of their labels.
    */
   int64_t VertexCount() const {
-    database::GraphDbAccessor accessor(db_);
-    return CountIterable(accessor.Vertices(true));
+    auto accessor = db_.Access();
+    return CountIterable(accessor->Vertices(true));
   }
 
   /**
@@ -105,12 +105,12 @@ class RandomGraphGenerator {
     auto vertices_from = FilterVertices(from_filter);
     auto vertices_to = FilterVertices(to_filter);
 
-    database::GraphDbAccessor dba(db_);
-    auto edge_type = dba.EdgeType(edge_type_name);
+    auto dba = db_.Access();
+    auto edge_type = dba->EdgeType(edge_type_name);
 
     // for small vertex counts reduce the batch size
     batch_size =
-        std::min(batch_size, static_cast<int>(dba.VerticesCount() / 1000 + 1));
+        std::min(batch_size, static_cast<int>(dba->VerticesCount() / 1000 + 1));
 
     Map(
         [&vertices_from, &vertices_to, edge_type,
@@ -132,8 +132,8 @@ class RandomGraphGenerator {
    * regardless of their types and origin/destination labels.
    */
   int64_t EdgeCount() const {
-    database::GraphDbAccessor accessor(db_);
-    return CountIterable(accessor.Edges(true));
+    auto accessor = db_.Access();
+    return CountIterable(accessor->Edges(true));
   }
 
   /**
@@ -150,11 +150,11 @@ class RandomGraphGenerator {
       const std::string &prop_name, std::function<TValue()> value_generator,
       std::function<bool(VertexAccessor &va)> predicate = {}) {
     if (!predicate) predicate = [](VertexAccessor &) { return true; };
-    database::GraphDbAccessor dba(db_);
-    auto property = dba.Property(prop_name);
-    for (VertexAccessor va : dba.Vertices(false))
+    auto dba = db_.Access();
+    auto property = dba->Property(prop_name);
+    for (VertexAccessor va : dba->Vertices(false))
       if (predicate(va)) va.PropsSet(property, value_generator());
-    dba.Commit();
+    dba->Commit();
   }
 
  private:
@@ -179,8 +179,8 @@ class RandomGraphGenerator {
       std::function<bool(VertexAccessor &item)> predicate = {}) {
     if (!predicate) predicate = [](VertexAccessor &) { return true; };
     std::vector<VertexAccessor> r_val;
-    database::GraphDbAccessor dba(db_);
-    for (VertexAccessor &item : dba.Vertices(false))
+    auto dba = db_.Access();
+    for (VertexAccessor &item : dba->Vertices(false))
       if (predicate(item)) r_val.emplace_back(item);
 
     return r_val;
@@ -210,14 +210,14 @@ class RandomGraphGenerator {
       threads.emplace_back([count_per_thread, &f, this, elements_per_commit]() {
         for (int i = 0; i < count_per_thread; i += elements_per_commit) {
           while (true) {
-            database::GraphDbAccessor dba(db_);
+            auto dba = db_.Access();
             try {
               int apply_count =
                   std::min(elements_per_commit, count_per_thread - i);
               while (apply_count--) {
-                f(dba);
+                f(*dba);
               }
-              dba.Commit();
+              dba->Commit();
               break;
             } catch (...) {
             }
diff --git a/tests/benchmark/expansion.cpp b/tests/benchmark/expansion.cpp
index 369b75437..abce0359d 100644
--- a/tests/benchmark/expansion.cpp
+++ b/tests/benchmark/expansion.cpp
@@ -18,24 +18,24 @@ class ExpansionBenchFixture : public benchmark::Fixture {
   void SetUp(const benchmark::State &state) override {
     db_.emplace();
     interpreter_.emplace(db_.value());
-    database::GraphDbAccessor dba(*db_);
-    for (int i = 0; i < state.range(0); i++) dba.InsertVertex();
+    auto dba = db_->Access();
+    for (int i = 0; i < state.range(0); i++) dba->InsertVertex();
 
     // the fixed part is one vertex expanding to 1000 others
-    auto start = dba.InsertVertex();
-    start.add_label(dba.Label("Starting"));
-    auto edge_type = dba.EdgeType("edge_type");
+    auto start = dba->InsertVertex();
+    start.add_label(dba->Label("Starting"));
+    auto edge_type = dba->EdgeType("edge_type");
     for (int i = 0; i < 1000; i++) {
-      auto dest = dba.InsertVertex();
-      dba.InsertEdge(start, dest, edge_type);
+      auto dest = dba->InsertVertex();
+      dba->InsertEdge(start, dest, edge_type);
     }
-    dba.Commit();
+    dba->Commit();
   }
 
   void TearDown(const benchmark::State &) override {
-    database::GraphDbAccessor dba(*db_);
-    for (auto vertex : dba.Vertices(false)) dba.DetachRemoveVertex(vertex);
-    dba.Commit();
+    auto dba = db_->Access();
+    for (auto vertex : dba->Vertices(false)) dba->DetachRemoveVertex(vertex);
+    dba->Commit();
     interpreter_ = std::experimental::nullopt;
     db_ = std::experimental::nullopt;
   }
@@ -45,10 +45,10 @@ class ExpansionBenchFixture : public benchmark::Fixture {
 
 BENCHMARK_DEFINE_F(ExpansionBenchFixture, Match)(benchmark::State &state) {
   auto query = "MATCH (s:Starting) return s";
-  database::GraphDbAccessor dba(*db_);
+  auto dba = db_->Access();
   while (state.KeepRunning()) {
     ResultStreamFaker<query::TypedValue> results;
-    interpreter()(query, dba, {}, false).PullAll(results);
+    interpreter()(query, *dba, {}, false).PullAll(results);
   }
 }
 
@@ -59,10 +59,10 @@ BENCHMARK_REGISTER_F(ExpansionBenchFixture, Match)
 
 BENCHMARK_DEFINE_F(ExpansionBenchFixture, Expand)(benchmark::State &state) {
   auto query = "MATCH (s:Starting) WITH s MATCH (s)--(d) RETURN count(d)";
-  database::GraphDbAccessor dba(*db_);
+  auto dba = db_->Access();
   while (state.KeepRunning()) {
     ResultStreamFaker<query::TypedValue> results;
-    interpreter()(query, dba, {}, false).PullAll(results);
+    interpreter()(query, *dba, {}, false).PullAll(results);
   }
 }
 
diff --git a/tests/benchmark/query/planner.cpp b/tests/benchmark/query/planner.cpp
index 052630f9c..203a77e84 100644
--- a/tests/benchmark/query/planner.cpp
+++ b/tests/benchmark/query/planner.cpp
@@ -32,7 +32,7 @@ static void AddChainedMatches(int num_matches, query::AstStorage &storage) {
 
 static void BM_PlanChainedMatches(benchmark::State &state) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   while (state.KeepRunning()) {
     state.PauseTiming();
     query::AstStorage storage;
@@ -41,7 +41,7 @@ static void BM_PlanChainedMatches(benchmark::State &state) {
     query::SymbolTable symbol_table;
     query::SymbolGenerator symbol_generator(symbol_table);
     storage.query()->Accept(symbol_generator);
-    auto ctx = query::plan::MakePlanningContext(storage, symbol_table, dba);
+    auto ctx = query::plan::MakePlanningContext(storage, symbol_table, *dba);
     state.ResumeTiming();
     auto query_parts = query::plan::CollectQueryParts(symbol_table, storage);
     if (query_parts.query_parts.size() == 0) {
@@ -85,18 +85,18 @@ static void AddIndexedMatches(
 
 static auto CreateIndexedVertices(int index_count, int vertex_count,
                                   database::GraphDb &db) {
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
-  database::GraphDbAccessor(db).BuildIndex(label, prop);
-  database::GraphDbAccessor dba(db);
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
+  db.Access()->BuildIndex(label, prop);
+  auto dba = db.Access();
   for (int vi = 0; vi < vertex_count; ++vi) {
     for (int index = 0; index < index_count; ++index) {
-      auto vertex = dba.InsertVertex();
+      auto vertex = dba->InsertVertex();
       vertex.add_label(label);
       vertex.PropsSet(prop, index);
     }
   }
-  dba.Commit();
+  dba->Commit();
   return std::make_pair(label, prop);
 }
 
@@ -107,7 +107,7 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
   int index_count = state.range(0);
   int vertex_count = state.range(1);
   std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   Parameters parameters;
   while (state.KeepRunning()) {
     state.PauseTiming();
@@ -118,7 +118,7 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
     query::SymbolGenerator symbol_generator(symbol_table);
     storage.query()->Accept(symbol_generator);
     state.ResumeTiming();
-    auto ctx = query::plan::MakePlanningContext(storage, symbol_table, dba);
+    auto ctx = query::plan::MakePlanningContext(storage, symbol_table, *dba);
     auto query_parts = query::plan::CollectQueryParts(symbol_table, storage);
     if (query_parts.query_parts.size() == 0) {
       std::exit(EXIT_FAILURE);
@@ -127,7 +127,7 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
     auto plans = query::plan::MakeLogicalPlanForSingleQuery<
         query::plan::VariableStartPlanner>(single_query_parts, ctx);
     for (auto plan : plans) {
-      query::plan::EstimatePlanCost(dba, parameters, *plan);
+      query::plan::EstimatePlanCost(*dba, parameters, *plan);
     }
   }
 }
@@ -140,8 +140,8 @@ static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(
   int index_count = state.range(0);
   int vertex_count = state.range(1);
   std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
-  database::GraphDbAccessor dba(db);
-  auto vertex_counts = query::plan::MakeVertexCountCache(dba);
+  auto dba = db.Access();
+  auto vertex_counts = query::plan::MakeVertexCountCache(*dba);
   Parameters parameters;
   while (state.KeepRunning()) {
     state.PauseTiming();
diff --git a/tests/manual/distributed_common.hpp b/tests/manual/distributed_common.hpp
index e70f3ef4a..5923660f1 100644
--- a/tests/manual/distributed_common.hpp
+++ b/tests/manual/distributed_common.hpp
@@ -64,10 +64,10 @@ class Cluster {
 
   auto Execute(const std::string &query,
                std::map<std::string, query::TypedValue> params = {}) {
-    database::GraphDbAccessor dba(*master_);
+    auto dba = master_->Access();
     ResultStreamFaker<query::TypedValue> result;
-    interpreter_->operator()(query, dba, params, false).PullAll(result);
-    dba.Commit();
+    interpreter_->operator()(query, *dba, params, false).PullAll(result);
+    dba->Commit();
     return result.GetResults();
   };
 
diff --git a/tests/manual/query_planner.cpp b/tests/manual/query_planner.cpp
index 20c775478..855a242e6 100644
--- a/tests/manual/query_planner.cpp
+++ b/tests/manual/query_planner.cpp
@@ -832,10 +832,10 @@ int main(int argc, char *argv[]) {
     std::exit(EXIT_FAILURE);
   }
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   Timer planning_timer;
   InteractiveDbAccessor interactive_db(
-      dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0,
+      *dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0,
       planning_timer);
   if (!in_db_filename.empty()) {
     std::ifstream db_file(in_db_filename);
@@ -846,7 +846,7 @@ int main(int argc, char *argv[]) {
     if (!line || *line == "quit") break;
     if (line->empty()) continue;
     try {
-      auto ast = MakeAst(*line, dba);
+      auto ast = MakeAst(*line, *dba);
       auto symbol_table = MakeSymbolTable(ast);
       planning_timer.Start();
       auto plans = MakeLogicalPlans(ast, symbol_table, interactive_db);
@@ -856,7 +856,7 @@ int main(int argc, char *argv[]) {
           << std::chrono::duration<double, std::milli>(planning_time).count()
           << "ms" << std::endl;
       std::cout << "Generated " << plans.size() << " plans" << std::endl;
-      ExaminePlans(dba, symbol_table, plans);
+      ExaminePlans(*dba, symbol_table, plans);
     } catch (const utils::BasicException &e) {
       std::cout << "Error: " << e.what() << std::endl;
     }
diff --git a/tests/manual/single_query.cpp b/tests/manual/single_query.cpp
index 48d186514..21b7397bc 100644
--- a/tests/manual/single_query.cpp
+++ b/tests/manual/single_query.cpp
@@ -12,9 +12,9 @@ int main(int argc, char *argv[]) {
     exit(1);
   }
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   ResultStreamFaker<query::TypedValue> stream;
-  auto results = query::Interpreter{db}(argv[1], dba, {}, false);
+  auto results = query::Interpreter{db}(argv[1], *dba, {}, false);
   stream.Header(results.header());
   results.PullAll(stream);
   stream.Summary(results.summary());
diff --git a/tests/property_based/random_graph.cpp b/tests/property_based/random_graph.cpp
index fe11e6854..38f3bb502 100644
--- a/tests/property_based/random_graph.cpp
+++ b/tests/property_based/random_graph.cpp
@@ -29,11 +29,11 @@ RC_GTEST_PROP(RandomGraph, RandomGraph, (std::vector<std::string> vertex_labels,
   std::unordered_map<VertexAccessor, std::string> vertex_label_map;
   std::unordered_map<EdgeAccessor, std::string> edge_type_map;
 
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   for (auto label : vertex_labels) {
-    auto vertex_accessor = dba.InsertVertex();
-    vertex_accessor.add_label(dba.Label(label));
+    auto vertex_accessor = dba->InsertVertex();
+    vertex_accessor.add_label(dba->Label(label));
     vertex_label_map.insert({vertex_accessor, label});
     vertices.push_back(vertex_accessor);
   }
@@ -41,23 +41,23 @@ RC_GTEST_PROP(RandomGraph, RandomGraph, (std::vector<std::string> vertex_labels,
   for (auto type : edge_types) {
     auto from = vertices[*rc::gen::inRange(0, vertices_num)];
     auto to = vertices[*rc::gen::inRange(0, vertices_num)];
-    auto edge_accessor = dba.InsertEdge(from, to, dba.EdgeType(type));
+    auto edge_accessor = dba->InsertEdge(from, to, dba->EdgeType(type));
     edge_type_map.insert({edge_accessor, type});
   }
 
-  dba.AdvanceCommand();
+  dba->AdvanceCommand();
 
   int edges_num_check = 0;
   int vertices_num_check = 0;
-  for (const auto &vertex : dba.Vertices(false)) {
+  for (const auto &vertex : dba->Vertices(false)) {
     auto label = vertex_label_map.at(vertex);
     RC_ASSERT(vertex.labels().size() == 1);
-    RC_ASSERT(dba.LabelName(vertex.labels()[0]) == label);
+    RC_ASSERT(dba->LabelName(vertex.labels()[0]) == label);
     vertices_num_check++;
   }
-  for (const auto &edge : dba.Edges(false)) {
+  for (const auto &edge : dba->Edges(false)) {
     auto type = edge_type_map.at(edge);
-    RC_ASSERT(dba.EdgeTypeName(edge.EdgeType()) == type);
+    RC_ASSERT(dba->EdgeTypeName(edge.EdgeType()) == type);
     edges_num_check++;
   }
   RC_ASSERT(vertices_num_check == vertices_num);
diff --git a/tests/unit/bolt_encoder.cpp b/tests/unit/bolt_encoder.cpp
index c19505c99..1c69794ae 100644
--- a/tests/unit/bolt_encoder.cpp
+++ b/tests/unit/bolt_encoder.cpp
@@ -165,24 +165,24 @@ TEST(BoltEncoder, VertexAndEdge) {
 
   // create vertex
   database::SingleNode db;
-  database::GraphDbAccessor db_accessor(db);
-  auto va1 = db_accessor.InsertVertex();
-  auto va2 = db_accessor.InsertVertex();
-  auto l1 = db_accessor.Label("label1");
-  auto l2 = db_accessor.Label("label2");
+  auto db_accessor = db.Access();
+  auto va1 = db_accessor->InsertVertex();
+  auto va2 = db_accessor->InsertVertex();
+  auto l1 = db_accessor->Label("label1");
+  auto l2 = db_accessor->Label("label2");
   va1.add_label(l1);
   va1.add_label(l2);
-  auto p1 = db_accessor.Property("prop1");
-  auto p2 = db_accessor.Property("prop2");
+  auto p1 = db_accessor->Property("prop1");
+  auto p2 = db_accessor->Property("prop2");
   PropertyValue pv1(12), pv2(200);
   va1.PropsSet(p1, pv1);
   va1.PropsSet(p2, pv2);
 
   // create edge
-  auto et = db_accessor.EdgeType("edgetype");
-  auto ea = db_accessor.InsertEdge(va1, va2, et);
-  auto p3 = db_accessor.Property("prop3");
-  auto p4 = db_accessor.Property("prop4");
+  auto et = db_accessor->EdgeType("edgetype");
+  auto ea = db_accessor->InsertEdge(va1, va2, et);
+  auto p3 = db_accessor->Property("prop3");
+  auto p4 = db_accessor->Property("prop4");
   PropertyValue pv3(42), pv4(1234);
   ea.PropsSet(p3, pv3);
   ea.PropsSet(p4, pv4);
diff --git a/tests/unit/cypher_main_visitor.cpp b/tests/unit/cypher_main_visitor.cpp
index 24ba8aec7..d4e77e0bf 100644
--- a/tests/unit/cypher_main_visitor.cpp
+++ b/tests/unit/cypher_main_visitor.cpp
@@ -35,12 +35,12 @@ class Base {
  public:
   explicit Base(const std::string &query) : query_string_(query) {}
   database::SingleNode db_;
-  database::GraphDbAccessor db_accessor_{db_};
-  Context context_{db_accessor_};
+  std::unique_ptr<database::GraphDbAccessor> db_accessor_{db_.Access()};
+  Context context_{*db_accessor_};
   std::string query_string_;
 
   auto Prop(const std::string &prop_name) {
-    return db_accessor_.Property(prop_name);
+    return db_accessor_->Property(prop_name);
   }
 
   auto PropPair(const std::string &prop_name) {
@@ -203,7 +203,7 @@ TYPED_TEST(CypherMainVisitorTest, PropertyLookup) {
   ASSERT_TRUE(identifier);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_EQ(property_lookup->property_,
-            ast_generator.db_accessor_.Property("x"));
+            ast_generator.db_accessor_->Property("x"));
 }
 
 TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
@@ -220,8 +220,8 @@ TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
   ASSERT_TRUE(identifier);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_.Label("x"),
-                          ast_generator.db_accessor_.Label("y")));
+              ElementsAre(ast_generator.db_accessor_->Label("x"),
+                          ast_generator.db_accessor_->Label("y")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
@@ -236,7 +236,7 @@ TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
   auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_.Label("l-$\"'ab`e``l")));
+              ElementsAre(ast_generator.db_accessor_->Label("l-$\"'ab`e``l")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
@@ -251,7 +251,7 @@ TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
   auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_.Label("DEletE")));
+              ElementsAre(ast_generator.db_accessor_->Label("DEletE")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) {
@@ -266,7 +266,7 @@ TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) {
   auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
   EXPECT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_.Label("a")));
+              ElementsAre(ast_generator.db_accessor_->Label("a")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, ReturnNoDistinctNoBagSemantics) {
@@ -946,10 +946,10 @@ TYPED_TEST(CypherMainVisitorTest, NodePattern) {
   EXPECT_EQ(node->identifier_->name_,
             CypherMainVisitor::kAnonPrefix + std::to_string(1));
   EXPECT_FALSE(node->identifier_->user_declared_);
-  EXPECT_THAT(node->labels_,
-              UnorderedElementsAre(ast_generator.db_accessor_.Label("label1"),
-                                   ast_generator.db_accessor_.Label("label2"),
-                                   ast_generator.db_accessor_.Label("label3")));
+  EXPECT_THAT(node->labels_, UnorderedElementsAre(
+                                 ast_generator.db_accessor_->Label("label1"),
+                                 ast_generator.db_accessor_->Label("label2"),
+                                 ast_generator.db_accessor_->Label("label3")));
   std::map<std::pair<std::string, storage::Property>, int64_t> properties;
   for (auto x : node->properties_) {
     TypedValue value = LiteralValue(ast_generator.context_, x.second);
@@ -1048,8 +1048,8 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternDetails) {
   EXPECT_EQ(edge->direction_, EdgeAtom::Direction::IN);
   EXPECT_THAT(
       edge->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"),
-                           ast_generator.db_accessor_.EdgeType("type2")));
+      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
+                           ast_generator.db_accessor_->EdgeType("type2")));
   std::map<std::pair<std::string, storage::Property>, int64_t> properties;
   for (auto x : edge->properties_) {
     TypedValue value = LiteralValue(ast_generator.context_, x.second);
@@ -1205,7 +1205,7 @@ TYPED_TEST(CypherMainVisitorTest,
   CheckLiteral(ast_generator.context_,
                edge->properties_[ast_generator.PropPair("prop")], 42);
   ASSERT_EQ(edge->edge_types_.size(), 1U);
-  auto edge_type = ast_generator.db_accessor_.EdgeType("edge_type");
+  auto edge_type = ast_generator.db_accessor_->EdgeType("edge_type");
   EXPECT_EQ(edge->edge_types_[0], edge_type);
 }
 
@@ -1341,7 +1341,7 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
     ASSERT_TRUE(identifier1);
     ASSERT_EQ(identifier1->name_, "a");
     ASSERT_EQ(set_property->property_lookup_->property_,
-              ast_generator.db_accessor_.Property("x"));
+              ast_generator.db_accessor_->Property("x"));
     auto *identifier2 = dynamic_cast<Identifier *>(set_property->expression_);
     ASSERT_EQ(identifier2->name_, "b");
   }
@@ -1376,8 +1376,8 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
     ASSERT_TRUE(set_labels->identifier_);
     ASSERT_EQ(set_labels->identifier_->name_, "g");
     ASSERT_THAT(set_labels->labels_,
-                UnorderedElementsAre(ast_generator.db_accessor_.Label("h"),
-                                     ast_generator.db_accessor_.Label("i")));
+                UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
+                                     ast_generator.db_accessor_->Label("i")));
   }
 }
 
@@ -1398,7 +1398,7 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
     ASSERT_TRUE(identifier1);
     ASSERT_EQ(identifier1->name_, "a");
     ASSERT_EQ(remove_property->property_lookup_->property_,
-              ast_generator.db_accessor_.Property("x"));
+              ast_generator.db_accessor_->Property("x"));
   }
   {
     auto *remove_labels =
@@ -1407,8 +1407,8 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
     ASSERT_TRUE(remove_labels->identifier_);
     ASSERT_EQ(remove_labels->identifier_->name_, "g");
     ASSERT_THAT(remove_labels->labels_,
-                UnorderedElementsAre(ast_generator.db_accessor_.Label("h"),
-                                     ast_generator.db_accessor_.Label("i")));
+                UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
+                                     ast_generator.db_accessor_->Label("i")));
   }
 }
 
@@ -1601,9 +1601,9 @@ TYPED_TEST(CypherMainVisitorTest, CreateIndex) {
   ASSERT_EQ(single_query->clauses_.size(), 1U);
   auto *create_index = dynamic_cast<CreateIndex *>(single_query->clauses_[0]);
   ASSERT_TRUE(create_index);
-  ASSERT_EQ(create_index->label_, ast_generator.db_accessor_.Label("mirko"));
+  ASSERT_EQ(create_index->label_, ast_generator.db_accessor_->Label("mirko"));
   ASSERT_EQ(create_index->property_,
-            ast_generator.db_accessor_.Property("slavko"));
+            ast_generator.db_accessor_->Property("slavko"));
 }
 
 TYPED_TEST(CypherMainVisitorTest, ReturnAll) {
@@ -1700,8 +1700,8 @@ TYPED_TEST(CypherMainVisitorTest, MatchBfsReturn) {
   EXPECT_EQ(bfs->direction_, EdgeAtom::Direction::OUT);
   EXPECT_THAT(
       bfs->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"),
-                           ast_generator.db_accessor_.EdgeType("type2")));
+      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
+                           ast_generator.db_accessor_->EdgeType("type2")));
   EXPECT_EQ(bfs->identifier_->name_, "r");
   EXPECT_EQ(bfs->filter_lambda_.inner_edge->name_, "e");
   EXPECT_TRUE(bfs->filter_lambda_.inner_edge->user_declared_);
@@ -1748,8 +1748,8 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestReturn) {
   EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
   EXPECT_THAT(
       shortest->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"),
-                           ast_generator.db_accessor_.EdgeType("type2")));
+      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
+                           ast_generator.db_accessor_->EdgeType("type2")));
   CheckLiteral(ast_generator.context_, shortest->upper_bound_, 10);
   EXPECT_FALSE(shortest->lower_bound_);
   EXPECT_EQ(shortest->identifier_->name_, "r");
@@ -1788,8 +1788,8 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestNoFilterReturn) {
   EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
   EXPECT_THAT(
       shortest->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_.EdgeType("type1"),
-                           ast_generator.db_accessor_.EdgeType("type2")));
+      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
+                           ast_generator.db_accessor_->EdgeType("type2")));
   CheckLiteral(ast_generator.context_, shortest->upper_bound_, 10);
   EXPECT_FALSE(shortest->lower_bound_);
   EXPECT_EQ(shortest->identifier_->name_, "r");
diff --git a/tests/unit/database_key_index.cpp b/tests/unit/database_key_index.cpp
index c87b452a5..394329377 100644
--- a/tests/unit/database_key_index.cpp
+++ b/tests/unit/database_key_index.cpp
@@ -15,7 +15,7 @@ using testing::UnorderedElementsAreArray;
 TEST(LabelsIndex, UniqueInsert) {
   database::KeyIndex<storage::Label, Vertex> index;
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   tx::SingleNodeEngine engine;
 
   auto t1 = engine.Begin();
@@ -23,28 +23,28 @@ TEST(LabelsIndex, UniqueInsert) {
   engine.Commit(*t1);
   auto t2 = engine.Begin();
 
-  vlist.find(*t2)->labels_.push_back(dba.Label("1"));
-  index.Update(dba.Label("1"), &vlist, vlist.find(*t2));
+  vlist.find(*t2)->labels_.push_back(dba->Label("1"));
+  index.Update(dba->Label("1"), &vlist, vlist.find(*t2));
   // Try multiple inserts
-  index.Update(dba.Label("1"), &vlist, vlist.find(*t2));
+  index.Update(dba->Label("1"), &vlist, vlist.find(*t2));
 
-  vlist.find(*t2)->labels_.push_back(dba.Label("2"));
-  index.Update(dba.Label("2"), &vlist, vlist.find(*t2));
+  vlist.find(*t2)->labels_.push_back(dba->Label("2"));
+  index.Update(dba->Label("2"), &vlist, vlist.find(*t2));
 
-  vlist.find(*t2)->labels_.push_back(dba.Label("3"));
-  index.Update(dba.Label("3"), &vlist, vlist.find(*t2));
+  vlist.find(*t2)->labels_.push_back(dba->Label("3"));
+  index.Update(dba->Label("3"), &vlist, vlist.find(*t2));
   engine.Commit(*t2);
 
-  EXPECT_EQ(index.Count(dba.Label("1")), 1);
-  EXPECT_EQ(index.Count(dba.Label("2")), 1);
-  EXPECT_EQ(index.Count(dba.Label("3")), 1);
+  EXPECT_EQ(index.Count(dba->Label("1")), 1);
+  EXPECT_EQ(index.Count(dba->Label("2")), 1);
+  EXPECT_EQ(index.Count(dba->Label("3")), 1);
 }
 
 // Check if index filters duplicates.
 TEST(LabelsIndex, UniqueFilter) {
   database::SingleNode db;
   database::KeyIndex<storage::Label, Vertex> index;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   tx::SingleNodeEngine engine;
 
   auto t1 = engine.Begin();
@@ -55,7 +55,7 @@ TEST(LabelsIndex, UniqueFilter) {
   auto r1v2 = vlist2.find(*t1);
   EXPECT_NE(vlist1.find(*t1), nullptr);
 
-  auto label1 = dba.Label("1");
+  auto label1 = dba->Label("1");
   vlist1.find(*t1)->labels_.push_back(label1);
   vlist2.find(*t1)->labels_.push_back(label1);
   index.Update(label1, &vlist1, r1v1);
@@ -84,7 +84,7 @@ TEST(LabelsIndex, UniqueFilter) {
 TEST(LabelsIndex, Refresh) {
   database::KeyIndex<storage::Label, Vertex> index;
   database::SingleNode db;
-  database::GraphDbAccessor access(db);
+  auto access = db.Access();
   tx::SingleNodeEngine engine;
 
   // add two vertices to  database
@@ -98,7 +98,7 @@ TEST(LabelsIndex, Refresh) {
   EXPECT_NE(v1r1, nullptr);
   EXPECT_NE(v2r1, nullptr);
 
-  auto label = access.Label("label");
+  auto label = access->Label("label");
   v1r1->labels_.push_back(label);
   v2r1->labels_.push_back(label);
   index.Update(label, &vlist1, v1r1);
@@ -123,10 +123,10 @@ TEST(LabelsIndex, Refresh) {
 // Transaction hasn't ended and so the vertex is not visible.
 TEST(LabelsIndexDb, AddGetZeroLabels) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto vertex = dba.InsertVertex();
-  vertex.add_label(dba.Label("test"));
-  auto collection = dba.Vertices(dba.Label("test"), false);
+  auto dba = db.Access();
+  auto vertex = dba->InsertVertex();
+  vertex.add_label(dba->Label("test"));
+  auto collection = dba->Vertices(dba->Label("test"), false);
   std::vector<VertexAccessor> collection_vector(collection.begin(),
                                                 collection.end());
   EXPECT_EQ(collection_vector.size(), (size_t)0);
@@ -137,61 +137,61 @@ TEST(LabelsIndexDb, AddGetZeroLabels) {
 TEST(LabelsIndexDb, AddGetRemoveLabel) {
   database::SingleNode db;
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
 
-    auto vertex1 = dba.InsertVertex();
-    vertex1.add_label(dba.Label("test"));
+    auto vertex1 = dba->InsertVertex();
+    vertex1.add_label(dba->Label("test"));
 
-    auto vertex2 = dba.InsertVertex();
-    vertex2.add_label(dba.Label("test2"));
+    auto vertex2 = dba->InsertVertex();
+    vertex2.add_label(dba->Label("test2"));
 
-    auto vertex3 = dba.InsertVertex();
-    vertex3.add_label(dba.Label("test"));
+    auto vertex3 = dba->InsertVertex();
+    vertex3.add_label(dba->Label("test"));
 
-    dba.Commit();
+    dba->Commit();
   }  // Finish transaction.
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
 
-    auto filtered = dba.Vertices(dba.Label("test"), false);
+    auto filtered = dba->Vertices(dba->Label("test"), false);
     std::vector<VertexAccessor> collection(filtered.begin(), filtered.end());
-    auto vertices = dba.Vertices(false);
+    auto vertices = dba->Vertices(false);
 
     std::vector<VertexAccessor> expected_collection;
     for (auto vertex : vertices) {
-      if (vertex.has_label(dba.Label("test"))) {
+      if (vertex.has_label(dba->Label("test"))) {
         expected_collection.push_back(vertex);
       } else {
-        EXPECT_TRUE(vertex.has_label(dba.Label("test2")));
+        EXPECT_TRUE(vertex.has_label(dba->Label("test2")));
       }
     }
 
     EXPECT_EQ(expected_collection.size(), collection.size());
-    EXPECT_TRUE(collection[0].has_label(dba.Label("test")));
-    EXPECT_TRUE(collection[1].has_label(dba.Label("test")));
-    EXPECT_FALSE(collection[0].has_label(dba.Label("test2")));
-    EXPECT_FALSE(collection[1].has_label(dba.Label("test2")));
-    dba.RemoveVertex(collection[0]);  // Remove from database and test if
-                                      // index won't return it.
+    EXPECT_TRUE(collection[0].has_label(dba->Label("test")));
+    EXPECT_TRUE(collection[1].has_label(dba->Label("test")));
+    EXPECT_FALSE(collection[0].has_label(dba->Label("test2")));
+    EXPECT_FALSE(collection[1].has_label(dba->Label("test2")));
+    dba->RemoveVertex(collection[0]);  // Remove from database and test if
+                                       // index won't return it.
 
     // Remove label from the vertex and add new label.
-    collection[1].remove_label(dba.Label("test"));
-    collection[1].add_label(dba.Label("test2"));
-    dba.Commit();
+    collection[1].remove_label(dba->Label("test"));
+    collection[1].add_label(dba->Label("test2"));
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
 
-    auto filtered = dba.Vertices(dba.Label("test"), false);
+    auto filtered = dba->Vertices(dba->Label("test"), false);
     std::vector<VertexAccessor> collection(filtered.begin(), filtered.end());
-    auto vertices = dba.Vertices(false);
+    auto vertices = dba->Vertices(false);
 
     std::vector<VertexAccessor> expected_collection;
     for (auto vertex : vertices) {
-      if (vertex.has_label(dba.Label("test"))) {
+      if (vertex.has_label(dba->Label("test"))) {
         expected_collection.push_back(vertex);
       } else {
-        EXPECT_TRUE(vertex.has_label(dba.Label("test2")));
+        EXPECT_TRUE(vertex.has_label(dba->Label("test2")));
       }
     }
 
diff --git a/tests/unit/database_label_property_index.cpp b/tests/unit/database_label_property_index.cpp
index a22c9e1c9..b6400e370 100644
--- a/tests/unit/database_label_property_index.cpp
+++ b/tests/unit/database_label_property_index.cpp
@@ -12,12 +12,12 @@ using namespace database;
 class LabelPropertyIndexComplexTest : public ::testing::Test {
  protected:
   virtual void SetUp() {
-    GraphDbAccessor accessor(db_);
+    auto accessor = db_.Access();
 
-    label = accessor.Label("label");
-    property = accessor.Property("property");
-    label2 = accessor.Label("label2");
-    property2 = accessor.Property("property2");
+    label = accessor->Label("label");
+    property = accessor->Property("property");
+    label2 = accessor->Label("label2");
+    property2 = accessor->Property("property2");
 
     key = new LabelPropertyIndex::Key(label, property);
     EXPECT_EQ(index.CreateIndex(*key), true);
@@ -59,9 +59,9 @@ class LabelPropertyIndexComplexTest : public ::testing::Test {
 
 TEST(LabelPropertyIndex, CreateIndex) {
   SingleNode db;
-  GraphDbAccessor accessor(db);
-  LabelPropertyIndex::Key key(accessor.Label("test"),
-                              accessor.Property("test2"));
+  auto accessor = db.Access();
+  LabelPropertyIndex::Key key(accessor->Label("test"),
+                              accessor->Property("test2"));
   LabelPropertyIndex index;
   EXPECT_EQ(index.CreateIndex(key), true);
   EXPECT_EQ(index.CreateIndex(key), false);
@@ -69,9 +69,9 @@ TEST(LabelPropertyIndex, CreateIndex) {
 
 TEST(LabelPropertyIndex, DeleteIndex) {
   SingleNode db;
-  GraphDbAccessor accessor(db);
-  LabelPropertyIndex::Key key(accessor.Label("test"),
-                              accessor.Property("test2"));
+  auto accessor = db.Access();
+  LabelPropertyIndex::Key key(accessor->Label("test"),
+                              accessor->Property("test2"));
   LabelPropertyIndex index;
   EXPECT_EQ(index.CreateIndex(key), true);
   EXPECT_EQ(index.CreateIndex(key), false);
@@ -81,9 +81,9 @@ TEST(LabelPropertyIndex, DeleteIndex) {
 
 TEST(LabelPropertyIndex, IndexExistance) {
   SingleNode db;
-  GraphDbAccessor accessor(db);
-  LabelPropertyIndex::Key key(accessor.Label("test"),
-                              accessor.Property("test2"));
+  auto accessor = db.Access();
+  LabelPropertyIndex::Key key(accessor->Label("test"),
+                              accessor->Property("test2"));
   LabelPropertyIndex index;
   EXPECT_EQ(index.CreateIndex(key), true);
   // Index doesn't exist - and can't be used untill it's been notified as built.
@@ -94,9 +94,9 @@ TEST(LabelPropertyIndex, IndexExistance) {
 
 TEST(LabelPropertyIndex, Count) {
   SingleNode db;
-  GraphDbAccessor accessor(db);
-  auto label = accessor.Label("label");
-  auto property = accessor.Property("property");
+  auto accessor = db.Access();
+  auto label = accessor->Label("label");
+  auto property = accessor->Property("property");
   LabelPropertyIndex::Key key(label, property);
   LabelPropertyIndex index;
   ::testing::FLAGS_gtest_death_test_style = "threadsafe";
diff --git a/tests/unit/database_transaction_timeout.cpp b/tests/unit/database_transaction_timeout.cpp
index 90c7cbf8e..f1ec5ef15 100644
--- a/tests/unit/database_transaction_timeout.cpp
+++ b/tests/unit/database_transaction_timeout.cpp
@@ -14,19 +14,19 @@ TEST(TransactionTimeout, TransactionTimeout) {
   auto interpret = [&](auto &dba, const std::string &query) {
     ResultStreamFaker<query::TypedValue> stream;
     interpreter(query, dba, {}, false).PullAll(stream);
-
   };
   {
-    database::GraphDbAccessor dba(db);
-    interpret(dba, "MATCH (n) RETURN n");
+    auto dba = db.Access();
+    interpret(*dba, "MATCH (n) RETURN n");
   }
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     std::this_thread::sleep_for(std::chrono::seconds(5));
-    ASSERT_THROW(interpret(dba, "MATCH (n) RETURN n"), query::HintedAbortError);
+    ASSERT_THROW(interpret(*dba, "MATCH (n) RETURN n"),
+                 query::HintedAbortError);
   }
   {
-    database::GraphDbAccessor dba(db);
-    interpret(dba, "MATCH (n) RETURN n");
+    auto dba = db.Access();
+    interpret(*dba, "MATCH (n) RETURN n");
   }
 }
diff --git a/tests/unit/distributed_bfs.cpp b/tests/unit/distributed_bfs.cpp
index 9ab941aef..861258b3b 100644
--- a/tests/unit/distributed_bfs.cpp
+++ b/tests/unit/distributed_bfs.cpp
@@ -33,18 +33,18 @@ class BfsTest : public DistributedGraphDbTest {
 };
 
 TEST_F(BfsTest, Expansion) {
-  GraphDbAccessor dba{master()};
+  auto dba = master().Access();
 
   auto &clients = master().bfs_subcursor_clients();
   auto subcursor_ids = clients.CreateBfsSubcursors(
-      dba.transaction_id(), query::EdgeAtom::Direction::BOTH,
-      {dba.EdgeType("Edge")}, query::GraphView::OLD);
+      dba->transaction_id(), query::EdgeAtom::Direction::BOTH,
+      {dba->EdgeType("Edge")}, query::GraphView::OLD);
   clients.RegisterSubcursors(subcursor_ids);
 
   clients.SetSource(subcursor_ids, vertices[0]);
 
   auto pull = [&clients, &subcursor_ids, &dba](int worker_id) {
-    return clients.Pull(worker_id, subcursor_ids[worker_id], &dba);
+    return clients.Pull(worker_id, subcursor_ids[worker_id], dba.get());
   };
 
   EXPECT_EQ(pull(0), std::experimental::nullopt);
@@ -87,17 +87,17 @@ TEST_F(BfsTest, Expansion) {
 
   distributed::PathSegment ps;
 
-  ps = clients.ReconstructPath(subcursor_ids, vertices[3], &dba);
+  ps = clients.ReconstructPath(subcursor_ids, vertices[3], dba.get());
   ASSERT_EQ(ps.next_vertex, vertices[4]);
   ASSERT_EQ(ps.next_edge, std::experimental::nullopt);
   compare(ps.edges, {{3, 4}});
 
-  ps = clients.ReconstructPath(subcursor_ids, vertices[4], &dba);
+  ps = clients.ReconstructPath(subcursor_ids, vertices[4], dba.get());
   EXPECT_EQ(ps.next_vertex, std::experimental::nullopt);
   EXPECT_EQ(ps.next_edge, (edges[{0, 1}]));
   compare(ps.edges, {{2, 4}, {1, 2}});
 
-  ps = clients.ReconstructPath(subcursor_ids, edges[{0, 1}], &dba);
+  ps = clients.ReconstructPath(subcursor_ids, edges[{0, 1}], dba.get());
   EXPECT_EQ(ps.next_vertex, std::experimental::nullopt);
   EXPECT_EQ(ps.next_edge, std::experimental::nullopt);
   compare(ps.edges, {{0, 1}});
diff --git a/tests/unit/distributed_common.hpp b/tests/unit/distributed_common.hpp
index 1109d1af9..553feb3c1 100644
--- a/tests/unit/distributed_common.hpp
+++ b/tests/unit/distributed_common.hpp
@@ -98,9 +98,6 @@ class DistributedGraphDbTest : public ::testing::Test {
   }
 
   database::Master &master() { return *master_; }
-  auto &master_tx_engine() {
-    return dynamic_cast<tx::MasterEngine &>(master_->tx_engine());
-  }
 
   database::Worker &worker(int worker_id) {
     return workers_[worker_id - 1]->worker_;
@@ -109,9 +106,9 @@ class DistributedGraphDbTest : public ::testing::Test {
   /// Inserts a vertex and returns it's global address. Does it in a new
   /// transaction.
   storage::VertexAddress InsertVertex(database::GraphDb &db) {
-    database::GraphDbAccessor dba{db};
-    auto r_val = dba.InsertVertex().GlobalAddress();
-    dba.Commit();
+    auto dba = db.Access();
+    auto r_val = dba->InsertVertex().GlobalAddress();
+    dba->Commit();
     return r_val;
   }
 
@@ -121,27 +118,27 @@ class DistributedGraphDbTest : public ::testing::Test {
                   const std::string &edge_type_name) {
     CHECK(from_addr.is_remote() && to_addr.is_remote())
         << "Distributed test InsertEdge only takes global addresses";
-    database::GraphDbAccessor dba{master()};
-    VertexAccessor from{from_addr, dba};
-    VertexAccessor to{to_addr, dba};
+    auto dba = master().Access();
+    VertexAccessor from{from_addr, *dba};
+    VertexAccessor to{to_addr, *dba};
     auto r_val =
-        dba.InsertEdge(from, to, dba.EdgeType(edge_type_name)).GlobalAddress();
-    master().updates_server().Apply(dba.transaction_id());
-    worker(1).updates_server().Apply(dba.transaction_id());
-    worker(2).updates_server().Apply(dba.transaction_id());
-    dba.Commit();
+        dba->InsertEdge(from, to, dba->EdgeType(edge_type_name)).GlobalAddress();
+    master().updates_server().Apply(dba->transaction_id());
+    worker(1).updates_server().Apply(dba->transaction_id());
+    worker(2).updates_server().Apply(dba->transaction_id());
+    dba->Commit();
     return r_val;
   }
 
   auto VertexCount(database::GraphDb &db) {
-    database::GraphDbAccessor dba{db};
-    auto vertices = dba.Vertices(false);
+    auto dba = db.Access();
+    auto vertices = dba->Vertices(false);
     return std::distance(vertices.begin(), vertices.end());
   };
 
   auto EdgeCount(database::GraphDb &db) {
-    database::GraphDbAccessor dba(db);
-    auto edges = dba.Edges(false);
+    auto dba = db.Access();
+    auto edges = dba->Edges(false);
     return std::distance(edges.begin(), edges.end());
   };
 
diff --git a/tests/unit/distributed_data_exchange.cpp b/tests/unit/distributed_data_exchange.cpp
index 6cc136786..86155b322 100644
--- a/tests/unit/distributed_data_exchange.cpp
+++ b/tests/unit/distributed_data_exchange.cpp
@@ -21,51 +21,51 @@ TEST_F(DistributedDataExchangeTest, RemoteDataGetting) {
   gid::Gid v1_id, v2_id, e1_id;
 
   {
-    GraphDbAccessor dba{master()};
-    auto v1 = dba.InsertVertex();
-    auto v2 = dba.InsertVertex();
-    auto e1 = dba.InsertEdge(v1, v2, dba.EdgeType("et"));
+    auto dba = master().Access();
+    auto v1 = dba->InsertVertex();
+    auto v2 = dba->InsertVertex();
+    auto e1 = dba->InsertEdge(v1, v2, dba->EdgeType("et"));
 
     // Set some data so we see we're getting the right stuff.
-    v1.PropsSet(dba.Property("p1"), 42);
-    v1.add_label(dba.Label("label"));
-    v2.PropsSet(dba.Property("p2"), "value");
-    e1.PropsSet(dba.Property("p3"), true);
+    v1.PropsSet(dba->Property("p1"), 42);
+    v1.add_label(dba->Label("label"));
+    v2.PropsSet(dba->Property("p2"), "value");
+    e1.PropsSet(dba->Property("p3"), true);
 
     v1_id = v1.gid();
     v2_id = v2.gid();
     e1_id = e1.gid();
 
-    dba.Commit();
+    dba->Commit();
   }
 
   // The master must start a transaction before workers can work in it.
-  GraphDbAccessor master_dba{master()};
+  auto master_dba = master().Access();
 
   {
-    GraphDbAccessor w1_dba{worker(1), master_dba.transaction_id()};
-    VertexAccessor v1_in_w1{{v1_id, 0}, w1_dba};
+    auto w1_dba = worker(1).Access(master_dba->transaction_id());
+    VertexAccessor v1_in_w1{{v1_id, 0}, *w1_dba};
     EXPECT_NE(v1_in_w1.GetOld(), nullptr);
     EXPECT_EQ(v1_in_w1.GetNew(), nullptr);
-    EXPECT_EQ(v1_in_w1.PropsAt(w1_dba.Property("p1")).Value<int64_t>(), 42);
-    EXPECT_TRUE(v1_in_w1.has_label(w1_dba.Label("label")));
+    EXPECT_EQ(v1_in_w1.PropsAt(w1_dba->Property("p1")).Value<int64_t>(), 42);
+    EXPECT_TRUE(v1_in_w1.has_label(w1_dba->Label("label")));
   }
 
   {
-    GraphDbAccessor w2_dba{worker(2), master_dba.transaction_id()};
-    VertexAccessor v2_in_w2{{v2_id, 0}, w2_dba};
+    auto w2_dba = worker(2).Access(master_dba->transaction_id());
+    VertexAccessor v2_in_w2{{v2_id, 0}, *w2_dba};
     EXPECT_NE(v2_in_w2.GetOld(), nullptr);
     EXPECT_EQ(v2_in_w2.GetNew(), nullptr);
-    EXPECT_EQ(v2_in_w2.PropsAt(w2_dba.Property("p2")).Value<std::string>(),
+    EXPECT_EQ(v2_in_w2.PropsAt(w2_dba->Property("p2")).Value<std::string>(),
               "value");
-    EXPECT_FALSE(v2_in_w2.has_label(w2_dba.Label("label")));
+    EXPECT_FALSE(v2_in_w2.has_label(w2_dba->Label("label")));
 
-    VertexAccessor v1_in_w2{{v1_id, 0}, w2_dba};
-    EdgeAccessor e1_in_w2{{e1_id, 0}, w2_dba};
+    VertexAccessor v1_in_w2{{v1_id, 0}, *w2_dba};
+    EdgeAccessor e1_in_w2{{e1_id, 0}, *w2_dba};
     EXPECT_EQ(e1_in_w2.from(), v1_in_w2);
     EXPECT_EQ(e1_in_w2.to(), v2_in_w2);
-    EXPECT_EQ(e1_in_w2.EdgeType(), w2_dba.EdgeType("et"));
-    EXPECT_EQ(e1_in_w2.PropsAt(w2_dba.Property("p3")).Value<bool>(), true);
+    EXPECT_EQ(e1_in_w2.EdgeType(), w2_dba->EdgeType("et"));
+    EXPECT_EQ(e1_in_w2.PropsAt(w2_dba->Property("p3")).Value<bool>(), true);
   }
 }
 
@@ -76,7 +76,7 @@ TEST_F(DistributedDataExchangeTest, RemoteExpansion) {
   InsertEdge(from, to, "et");
   {
     // Expand on the master for three hops. Collect vertex gids.
-    GraphDbAccessor dba{master()};
+    auto dba = master().Access();
     std::vector<VertexAccessor> visited;
 
     auto expand = [](auto &v) {
@@ -86,7 +86,7 @@ TEST_F(DistributedDataExchangeTest, RemoteExpansion) {
     };
 
     // Do a few hops back and forth, all on the master.
-    VertexAccessor v{from, dba};
+    VertexAccessor v{from, *dba};
     for (int i = 0; i < 5; ++i) {
       v = expand(v);
       EXPECT_FALSE(v.address().is_local());
@@ -101,13 +101,13 @@ TEST_F(DistributedDataExchangeTest, VertexCountsEqual) {
   for (int i = 0; i < 9; ++i) InsertVertex(worker(2));
 
   {
-    GraphDbAccessor accessor(master());
+    auto accessor = master().Access();
     auto m_cnt =
-        master().data_clients().VertexCounts(accessor.transaction().id_);
+        master().data_clients().VertexCounts(accessor->transaction().id_);
     auto w1_cnt =
-        worker(1).data_clients().VertexCounts(accessor.transaction().id_);
+        worker(1).data_clients().VertexCounts(accessor->transaction().id_);
     auto w2_cnt =
-        worker(2).data_clients().VertexCounts(accessor.transaction().id_);
+        worker(2).data_clients().VertexCounts(accessor->transaction().id_);
 
     auto check = [&m_cnt, &w1_cnt, &w2_cnt](int key, int value) {
       return m_cnt[key] == w1_cnt[key] && w1_cnt[key] == w2_cnt[key] &&
@@ -122,17 +122,17 @@ TEST_F(DistributedDataExchangeTest, VertexCountsEqual) {
 
 TEST_F(DistributedDataExchangeTest, VertexCountsTransactional) {
   {
-    GraphDbAccessor accessor(master());
+    auto accessor = master().Access();
     InsertVertex(master());
     EXPECT_EQ(master().data_clients().VertexCounts(
-                  accessor.transaction().id_)[master().WorkerId()],
+                  accessor->transaction().id_)[master().WorkerId()],
               0);
   }
   // Transaction after insert which should now see the insertion
   {
-    GraphDbAccessor accessor(master());
+    auto accessor = master().Access();
     EXPECT_EQ(master().data_clients().VertexCounts(
-                  accessor.transaction().id_)[master().WorkerId()],
+                  accessor->transaction().id_)[master().WorkerId()],
               1);
   }
 }
diff --git a/tests/unit/distributed_durability.cpp b/tests/unit/distributed_durability.cpp
index d25edb1ed..6025e99de 100644
--- a/tests/unit/distributed_durability.cpp
+++ b/tests/unit/distributed_durability.cpp
@@ -47,21 +47,21 @@ class DistributedDurability : public DistributedGraphDbTest {
 
  private:
   void AddVertex(database::GraphDb &db, const std::string &label) {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.InsertVertex();
-    vertex.add_label(dba.Label(label));
-    dba.Commit();
+    auto dba = db.Access();
+    auto vertex = dba->InsertVertex();
+    vertex.add_label(dba->Label(label));
+    dba->Commit();
   }
 
   void CheckVertex(database::GraphDb &db, int expected_count,
                    const std::string &label) {
-    database::GraphDbAccessor dba(db);
-    auto it = dba.Vertices(false);
+    auto dba = db.Access();
+    auto it = dba->Vertices(false);
     std::vector<VertexAccessor> vertices{it.begin(), it.end()};
     EXPECT_EQ(vertices.size(), expected_count);
     for (auto &vertex : vertices) {
       ASSERT_EQ(vertex.labels().size(), 1);
-      EXPECT_EQ(vertex.labels()[0], dba.Label(label));
+      EXPECT_EQ(vertex.labels()[0], dba->Label(label));
     }
   }
 };
@@ -71,8 +71,8 @@ TEST_F(DistributedDurability, MakeSnapshot) {
   // of it
   {
     AddVertices();
-    database::GraphDbAccessor dba(master());
-    master().MakeSnapshot(dba);
+    auto dba = master().Access();
+    master().MakeSnapshot(*dba);
   }
   // Recover the graph and check if it's the same as before
   {
@@ -101,15 +101,15 @@ TEST_F(DistributedDurability, RecoveryFromSameSnapshot) {
   {
     AddVertices();
     // Make snapshot on one worker, expect it won't recover from that.
-    database::GraphDbAccessor dba(worker(1));
-    worker(1).MakeSnapshot(dba);
+    auto dba = worker(1).Access();
+    worker(1).MakeSnapshot(*dba);
   }
   {
     RestartWithRecovery();
     CheckVertices(0);
     AddVertices();
-    database::GraphDbAccessor dba(master());
-    master().MakeSnapshot(dba);
+    auto dba = master().Access();
+    master().MakeSnapshot(*dba);
   }
   {
     RestartWithRecovery();
@@ -117,8 +117,8 @@ TEST_F(DistributedDurability, RecoveryFromSameSnapshot) {
     AddVertices();
     CheckVertices(2);
     // Make snapshot on one worker, expect it won't recover from that.
-    database::GraphDbAccessor dba(worker(1));
-    worker(1).MakeSnapshot(dba);
+    auto dba = worker(1).Access();
+    worker(1).MakeSnapshot(*dba);
   }
   {
     RestartWithRecovery();
@@ -130,8 +130,8 @@ TEST_F(DistributedDurability, RecoveryFailure) {
   {
     AddVertices();
     // Make a snapshot on the master without the right snapshots on workers.
-    database::GraphDbAccessor dba(master());
-    bool status = durability::MakeSnapshot(master(), dba, tmp_dir_, 100);
+    auto dba = master().Access();
+    bool status = durability::MakeSnapshot(master(), *dba, tmp_dir_, 100);
     ASSERT_TRUE(status);
   }
   ::testing::FLAGS_gtest_death_test_style = "threadsafe";
@@ -171,8 +171,8 @@ void CheckDeltas(fs::path wal_dir, database::StateDelta::Type op) {
 
 TEST_F(DistributedDurability, WriteCommittedTx) {
   RestartWithWal();
-  database::GraphDbAccessor dba(master());
-  dba.Commit();
+  auto dba = master().Access();
+  dba->Commit();
   FlushAllWal();
   CheckDeltas(tmp_dir_ / durability::kWalDir,
               database::StateDelta::Type::TRANSACTION_COMMIT);
@@ -180,8 +180,8 @@ TEST_F(DistributedDurability, WriteCommittedTx) {
 
 TEST_F(DistributedDurability, WriteAbortedTx) {
   RestartWithWal();
-  database::GraphDbAccessor dba(master());
-  dba.Abort();
+  auto dba = master().Access();
+  dba->Abort();
   FlushAllWal();
   CheckDeltas(tmp_dir_ / durability::kWalDir,
               database::StateDelta::Type::TRANSACTION_ABORT);
diff --git a/tests/unit/distributed_dynamic_graph_partitioner.cpp b/tests/unit/distributed_dynamic_graph_partitioner.cpp
index c5a8f865e..962252380 100644
--- a/tests/unit/distributed_dynamic_graph_partitioner.cpp
+++ b/tests/unit/distributed_dynamic_graph_partitioner.cpp
@@ -32,8 +32,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, CountLabels) {
   for (int i = 0; i < 6; ++i) InsertEdge(vc, va, "edge");
 
   DynamicGraphPartitioner dgp(&master());
-  GraphDbAccessor dba(master());
-  VertexAccessor v(va, dba);
+  auto dba = master().Access();
+  VertexAccessor v(va, *dba);
   auto count_labels = dgp.CountLabels(v);
 
   // Self loops counted twice
@@ -53,8 +53,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMoveVertex) {
 
   for (int i = 0; i < 100; ++i) InsertEdge(va, vb, "edge");
   DynamicGraphPartitioner dgp(&master());
-  GraphDbAccessor dba(master());
-  auto migrations = dgp.FindMigrations(dba);
+  auto dba = master().Access();
+  auto migrations = dgp.FindMigrations(*dba);
   // Expect `va` to try to move to another worker, the one connected to it
   ASSERT_EQ(migrations.size(), 1);
   EXPECT_EQ(migrations[0].second, worker(1).WorkerId());
@@ -68,8 +68,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsNoChange) {
   // Everything is balanced, there should be no movement
 
   DynamicGraphPartitioner dgp(&master());
-  GraphDbAccessor dba(master());
-  auto migrations = dgp.FindMigrations(dba);
+  auto dba = master().Access();
+  auto migrations = dgp.FindMigrations(*dba);
   EXPECT_EQ(migrations.size(), 0);
 }
 
@@ -86,9 +86,9 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMultipleAndLimit) {
   for (int i = 0; i < 100; ++i) InsertEdge(va, vc, "edge");
   for (int i = 0; i < 100; ++i) InsertEdge(vb, vc, "edge");
   DynamicGraphPartitioner dgp(&master());
-  GraphDbAccessor dba(master());
+  auto dba = master().Access();
   {
-    auto migrations = dgp.FindMigrations(dba);
+    auto migrations = dgp.FindMigrations(*dba);
     // Expect vertices to try to move to another worker
     ASSERT_EQ(migrations.size(), 2);
   }
@@ -96,7 +96,7 @@ TEST_F(DistributedDynamicGraphPartitionerTest, FindMigrationsMultipleAndLimit) {
   // See if flag affects number of returned results
   {
     FLAGS_dgp_max_batch_size = 1;
-    auto migrations = dgp.FindMigrations(dba);
+    auto migrations = dgp.FindMigrations(*dba);
     // Expect vertices to try to move to another worker
     ASSERT_EQ(migrations.size(), 1);
   }
@@ -151,8 +151,8 @@ TEST_F(DistributedDynamicGraphPartitionerTest, Run) {
     return cnt;
   };
 
-  GraphDbAccessor dba_m(master());
-  GraphDbAccessor dba_w1(worker(1));
-  EXPECT_EQ(CountRemotes(dba_m), 50);
-  EXPECT_EQ(CountRemotes(dba_w1), 50);
+  auto dba_m = master().Access();
+  auto dba_w1 = worker(1).Access();
+  EXPECT_EQ(CountRemotes(*dba_m), 50);
+  EXPECT_EQ(CountRemotes(*dba_w1), 50);
 }
diff --git a/tests/unit/distributed_gc.cpp b/tests/unit/distributed_gc.cpp
index 44c742727..7d0e4188b 100644
--- a/tests/unit/distributed_gc.cpp
+++ b/tests/unit/distributed_gc.cpp
@@ -8,13 +8,13 @@ class DistributedGcTest : public DistributedGraphDbTest {
 };
 
 TEST_F(DistributedGcTest, GarbageCollect) {
-  database::GraphDbAccessor dba{master()};
-  auto tx = dba.transaction_id();
-  dba.Commit();
+  auto dba = master().Access();
+  auto tx = dba->transaction_id();
+  dba->Commit();
 
   // Create multiple transactions so that the commit log can be cleared
   for (int i = 0; i < tx::CommitLog::kBitsetBlockSize; ++i) {
-    database::GraphDbAccessor dba{master()};
+    auto dba = master().Access();
   }
 
   master().CollectGarbage();
@@ -22,9 +22,9 @@ TEST_F(DistributedGcTest, GarbageCollect) {
   worker(2).CollectGarbage();
   EXPECT_EQ(master().tx_engine().Info(tx).is_committed(), true);
 
-  database::GraphDbAccessor dba2{master()};
-  auto tx_last = dba2.transaction_id();
-  dba2.Commit();
+  auto dba2 = master().Access();
+  auto tx_last = dba2->transaction_id();
+  dba2->Commit();
 
   worker(1).CollectGarbage();
   worker(2).CollectGarbage();
@@ -39,17 +39,17 @@ TEST_F(DistributedGcTest, GarbageCollect) {
 }
 
 TEST_F(DistributedGcTest, GarbageCollectBlocked) {
-  database::GraphDbAccessor dba{master()};
-  auto tx = dba.transaction_id();
-  dba.Commit();
+  auto dba = master().Access();
+  auto tx = dba->transaction_id();
+  dba->Commit();
 
   // Block garbage collection because this is a still alive transaction on the
   // worker
-  database::GraphDbAccessor dba3{worker(1)};
+  auto dba3 = worker(1).Access();
 
   // Create multiple transactions so that the commit log can be cleared
   for (int i = 0; i < tx::CommitLog::kBitsetBlockSize; ++i) {
-    database::GraphDbAccessor dba{master()};
+    auto dba = master().Access();
   }
 
   // Query for a large id so that the commit log new block is created
@@ -60,9 +60,9 @@ TEST_F(DistributedGcTest, GarbageCollectBlocked) {
   worker(2).CollectGarbage();
   EXPECT_EQ(master().tx_engine().Info(tx).is_committed(), true);
 
-  database::GraphDbAccessor dba2{master()};
-  auto tx_last = dba2.transaction_id();
-  dba2.Commit();
+  auto dba2 = master().Access();
+  auto tx_last = dba2->transaction_id();
+  dba2->Commit();
 
   worker(1).CollectGarbage();
   worker(2).CollectGarbage();
diff --git a/tests/unit/distributed_graph_db.cpp b/tests/unit/distributed_graph_db.cpp
index 039df4290..464119e45 100644
--- a/tests/unit/distributed_graph_db.cpp
+++ b/tests/unit/distributed_graph_db.cpp
@@ -49,8 +49,8 @@ TEST_F(DistributedGraphDb, Coordination) {
 }
 
 TEST_F(DistributedGraphDb, TxEngine) {
-  auto *tx1 = master_tx_engine().Begin();
-  auto *tx2 = master_tx_engine().Begin();
+  auto *tx1 = master().tx_engine().Begin();
+  auto *tx2 = master().tx_engine().Begin();
   EXPECT_EQ(tx2->snapshot().size(), 1);
   EXPECT_EQ(
       worker(1).tx_engine().RunningTransaction(tx1->id_)->snapshot().size(), 0);
@@ -134,43 +134,43 @@ TEST_F(DistributedGraphDb, BuildIndexDistributed) {
   storage::Property property;
 
   {
-    GraphDbAccessor dba0{master()};
-    label = dba0.Label("label");
-    property = dba0.Property("property");
-    auto tx_id = dba0.transaction_id();
+    auto dba0 = master().Access();
+    label = dba0->Label("label");
+    property = dba0->Property("property");
+    auto tx_id = dba0->transaction_id();
 
-    GraphDbAccessor dba1{worker(1), tx_id};
-    GraphDbAccessor dba2{worker(2), tx_id};
+    auto dba1 = worker(1).Access(tx_id);
+    auto dba2 = worker(2).Access(tx_id);
     auto add_vertex = [label, property](GraphDbAccessor &dba) {
       auto vertex = dba.InsertVertex();
       vertex.add_label(label);
       vertex.PropsSet(property, 1);
     };
-    for (int i = 0; i < 100; ++i) add_vertex(dba0);
-    for (int i = 0; i < 50; ++i) add_vertex(dba1);
-    for (int i = 0; i < 300; ++i) add_vertex(dba2);
-    dba0.Commit();
+    for (int i = 0; i < 100; ++i) add_vertex(*dba0);
+    for (int i = 0; i < 50; ++i) add_vertex(*dba1);
+    for (int i = 0; i < 300; ++i) add_vertex(*dba2);
+    dba0->Commit();
   }
 
   {
-    GraphDbAccessor dba{master()};
-    dba.BuildIndex(label, property);
-    EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property));
-    EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 100);
+    auto dba = master().Access();
+    dba->BuildIndex(label, property);
+    EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property));
+    EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 100);
   }
 
-  GraphDbAccessor dba_master{master()};
+  auto dba_master = master().Access();
 
   {
-    GraphDbAccessor dba{worker(1), dba_master.transaction_id()};
-    EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property));
-    EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 50);
+    auto dba = worker(1).Access(dba_master->transaction_id());
+    EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property));
+    EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 50);
   }
 
   {
-    GraphDbAccessor dba{worker(2), dba_master.transaction_id()};
-    EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property));
-    EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 300);
+    auto dba = worker(2).Access(dba_master->transaction_id());
+    EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property));
+    EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 300);
   }
 }
 
@@ -178,9 +178,9 @@ TEST_F(DistributedGraphDb, BuildIndexConcurrentInsert) {
   storage::Label label;
   storage::Property property;
 
-  GraphDbAccessor dba0{master()};
-  label = dba0.Label("label");
-  property = dba0.Property("property");
+  auto dba0 = master().Access();
+  label = dba0->Label("label");
+  property = dba0->Property("property");
 
   int cnt = 0;
   auto add_vertex = [label, property, &cnt](GraphDbAccessor &dba) {
@@ -188,40 +188,40 @@ TEST_F(DistributedGraphDb, BuildIndexConcurrentInsert) {
     vertex.add_label(label);
     vertex.PropsSet(property, ++cnt);
   };
-  dba0.Commit();
+  dba0->Commit();
 
   auto worker_insert = std::thread([this, &add_vertex]() {
     for (int i = 0; i < 10000; ++i) {
-      GraphDbAccessor dba1{worker(1)};
-      add_vertex(dba1);
-      dba1.Commit();
+      auto dba1 = worker(1).Access();
+      add_vertex(*dba1);
+      dba1->Commit();
     }
   });
 
   std::this_thread::sleep_for(0.5s);
   {
-    GraphDbAccessor dba{master()};
-    dba.BuildIndex(label, property);
-    EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property));
+    auto dba = master().Access();
+    dba->BuildIndex(label, property);
+    EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property));
   }
 
   worker_insert.join();
   {
-    GraphDbAccessor dba{worker(1)};
-    EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property));
-    EXPECT_EQ(CountIterable(dba.Vertices(label, property, false)), 10000);
+    auto dba = worker(1).Access();
+    EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property));
+    EXPECT_EQ(CountIterable(dba->Vertices(label, property, false)), 10000);
   }
 }
 
 TEST_F(DistributedGraphDb, WorkerOwnedDbAccessors) {
-  GraphDbAccessor dba_w1(worker(1));
-  auto v = dba_w1.InsertVertex();
-  auto prop = dba_w1.Property("p");
+  auto dba_w1 = worker(1).Access();
+  auto v = dba_w1->InsertVertex();
+  auto prop = dba_w1->Property("p");
   v.PropsSet(prop, 42);
   auto v_ga = v.GlobalAddress();
-  dba_w1.Commit();
+  dba_w1->Commit();
 
-  GraphDbAccessor dba_w2(worker(2));
-  VertexAccessor v_in_w2{v_ga, dba_w2};
+  auto dba_w2 = worker(2).Access();
+  VertexAccessor v_in_w2{v_ga, *dba_w2};
   EXPECT_EQ(v_in_w2.PropsAt(prop).Value<int64_t>(), 42);
 }
diff --git a/tests/unit/distributed_interpretation.cpp b/tests/unit/distributed_interpretation.cpp
index beea9794a..dabac4d39 100644
--- a/tests/unit/distributed_interpretation.cpp
+++ b/tests/unit/distributed_interpretation.cpp
@@ -48,9 +48,9 @@ class DistributedInterpretationTest : public DistributedGraphDbTest {
   }
 
   auto Run(const std::string &query) {
-    GraphDbAccessor dba(master());
-    auto results = RunWithDba(query, dba);
-    dba.Commit();
+    auto dba = master().Access();
+    auto results = RunWithDba(query, *dba);
+    dba->Commit();
     return results;
   }
 
@@ -164,20 +164,20 @@ TEST_F(DistributedInterpretationTest, Cartesian) {
   // Create some data on the master and both workers.
   storage::Property prop;
   {
-    GraphDbAccessor dba{master()};
-    auto tx_id = dba.transaction_id();
-    GraphDbAccessor dba1{worker(1), tx_id};
-    GraphDbAccessor dba2{worker(2), tx_id};
-    prop = dba.Property("prop");
+    auto dba = master().Access();
+    auto tx_id = dba->transaction_id();
+    auto dba1 = worker(1).Access(tx_id);
+    auto dba2 = worker(2).Access(tx_id);
+    prop = dba->Property("prop");
     auto add_data = [prop](GraphDbAccessor &dba, int value) {
       dba.InsertVertex().PropsSet(prop, value);
     };
 
-    for (int i = 0; i < 10; ++i) add_data(dba, i);
-    for (int i = 10; i < 20; ++i) add_data(dba1, i);
-    for (int i = 20; i < 30; ++i) add_data(dba2, i);
+    for (int i = 0; i < 10; ++i) add_data(*dba, i);
+    for (int i = 10; i < 20; ++i) add_data(*dba1, i);
+    for (int i = 20; i < 30; ++i) add_data(*dba2, i);
 
-    dba.Commit();
+    dba->Commit();
   }
 
   std::vector<std::vector<int64_t>> expected;
@@ -214,14 +214,14 @@ class TestQueryWaitsOnFutures : public DistributedInterpretationTest {
 TEST_F(TestQueryWaitsOnFutures, Test) {
   const int kVertexCount = 10;
   auto make_fully_connected = [](database::GraphDb &db) {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     std::vector<VertexAccessor> vertices;
     for (int i = 0; i < kVertexCount; ++i)
-      vertices.emplace_back(dba.InsertVertex());
-    auto et = dba.EdgeType("et");
+      vertices.emplace_back(dba->InsertVertex());
+    auto et = dba->EdgeType("et");
     for (auto &from : vertices)
-      for (auto &to : vertices) dba.InsertEdge(from, to, et);
-    dba.Commit();
+      for (auto &to : vertices) dba->InsertEdge(from, to, et);
+    dba->Commit();
   };
 
   make_fully_connected(worker(1));
@@ -292,25 +292,25 @@ TEST_F(DistributedInterpretationTest, OngoingProduceKeyTest) {
     InsertVertex(worker(2));
   }
 
-  GraphDbAccessor dba(master());
-  auto count1 = RunWithDba("MATCH (n) RETURN count(n)", dba);
-  dba.AdvanceCommand();
-  auto count2 = RunWithDba("MATCH (n) RETURN count(n)", dba);
+  auto dba = master().Access();
+  auto count1 = RunWithDba("MATCH (n) RETURN count(n)", *dba);
+  dba->AdvanceCommand();
+  auto count2 = RunWithDba("MATCH (n) RETURN count(n)", *dba);
 
   ASSERT_EQ(count1[0][0].ValueInt(), 3 * worker_count);
   ASSERT_EQ(count2[0][0].ValueInt(), 3 * worker_count);
 }
 
 TEST_F(DistributedInterpretationTest, AdvanceCommandOnWorkers) {
-  GraphDbAccessor dba(master());
-  RunWithDba("UNWIND RANGE(1, 10) as x CREATE (:A {id: x})", dba);
-  dba.AdvanceCommand();
+  auto dba = master().Access();
+  RunWithDba("UNWIND RANGE(1, 10) as x CREATE (:A {id: x})", *dba);
+  dba->AdvanceCommand();
   // Advance commands on workers also.
   auto futures = master().pull_clients().NotifyAllTransactionCommandAdvanced(
-      dba.transaction_id());
+      dba->transaction_id());
   for (auto &future : futures) future.wait();
 
-  auto count = RunWithDba("MATCH (n) RETURN count(n)", dba);
+  auto count = RunWithDba("MATCH (n) RETURN count(n)", *dba);
   ASSERT_EQ(count[0][0].ValueInt(), 10);
 }
 
diff --git a/tests/unit/distributed_query_plan.cpp b/tests/unit/distributed_query_plan.cpp
index 0ca1da1d2..8cf355c70 100644
--- a/tests/unit/distributed_query_plan.cpp
+++ b/tests/unit/distributed_query_plan.cpp
@@ -38,8 +38,8 @@ class DistributedQueryPlan : public DistributedGraphDbTest {
 };
 
 TEST_F(DistributedQueryPlan, PullProduceRpc) {
-  GraphDbAccessor dba{master()};
-  Context ctx{dba};
+  auto dba = master().Access();
+  Context ctx{*dba};
   SymbolGenerator symbol_generator{ctx.symbol_table_};
   AstStorage storage;
 
@@ -56,13 +56,13 @@ TEST_F(DistributedQueryPlan, PullProduceRpc) {
   auto produce = MakeProduce(unwind, x_ne);
 
   // Test that the plan works locally.
-  auto results = CollectProduce(produce.get(), ctx.symbol_table_, dba);
+  auto results = CollectProduce(produce.get(), ctx.symbol_table_, *dba);
   ASSERT_EQ(results.size(), 5);
 
   const int plan_id = 42;
   master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_);
 
-  tx::CommandId command_id = dba.transaction().cid();
+  tx::CommandId command_id = dba->transaction().cid();
   Parameters params;
   std::vector<query::Symbol> symbols{ctx.symbol_table_[*x_ne]};
   auto remote_pull = [this, &command_id, &params, &symbols](
@@ -86,17 +86,17 @@ TEST_F(DistributedQueryPlan, PullProduceRpc) {
     EXPECT_EQ(batch.frames[1][0].ValueInt(), 2);
   };
 
-  GraphDbAccessor dba_1{master()};
-  GraphDbAccessor dba_2{master()};
+  auto dba_1 = master().Access();
+  auto dba_2 = master().Access();
   for (int worker_id : {1, 2}) {
     // TODO flor, proper test async here.
-    auto tx1_batch1 = remote_pull(dba_1, worker_id).get();
+    auto tx1_batch1 = remote_pull(*dba_1, worker_id).get();
     expect_first_batch(tx1_batch1);
-    auto tx2_batch1 = remote_pull(dba_2, worker_id).get();
+    auto tx2_batch1 = remote_pull(*dba_2, worker_id).get();
     expect_first_batch(tx2_batch1);
-    auto tx2_batch2 = remote_pull(dba_2, worker_id).get();
+    auto tx2_batch2 = remote_pull(*dba_2, worker_id).get();
     expect_second_batch(tx2_batch2);
-    auto tx1_batch2 = remote_pull(dba_1, worker_id).get();
+    auto tx1_batch2 = remote_pull(*dba_1, worker_id).get();
     expect_second_batch(tx1_batch2);
   }
 }
@@ -107,8 +107,8 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) {
   // sequence ID, so we can check we retrieved all.
   storage::Property prop;
   {
-    GraphDbAccessor dba{master()};
-    prop = dba.Property("prop");
+    auto dba = master().Access();
+    prop = dba->Property("prop");
     auto create_data = [prop](GraphDbAccessor &dba, int worker_id) {
       auto v1 = dba.InsertVertex();
       v1.PropsSet(prop, worker_id * 10);
@@ -117,16 +117,16 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) {
       auto e12 = dba.InsertEdge(v1, v2, dba.EdgeType("et"));
       e12.PropsSet(prop, worker_id * 10 + 2);
     };
-    create_data(dba, 0);
-    GraphDbAccessor dba_w1{worker(1), dba.transaction_id()};
-    create_data(dba_w1, 1);
-    GraphDbAccessor dba_w2{worker(2), dba.transaction_id()};
-    create_data(dba_w2, 2);
-    dba.Commit();
+    create_data(*dba, 0);
+    auto dba_w1 = worker(1).Access(dba->transaction_id());
+    create_data(*dba_w1, 1);
+    auto dba_w2 = worker(2).Access(dba->transaction_id());
+    create_data(*dba_w2, 2);
+    dba->Commit();
   }
 
-  GraphDbAccessor dba{master()};
-  Context ctx{dba};
+  auto dba = master().Access();
+  Context ctx{*dba};
   SymbolGenerator symbol_generator{ctx.symbol_table_};
   AstStorage storage;
 
@@ -175,13 +175,13 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) {
   };
 
   // Test that the plan works locally.
-  auto results = CollectProduce(produce.get(), ctx.symbol_table_, dba);
+  auto results = CollectProduce(produce.get(), ctx.symbol_table_, *dba);
   check_result(0, results);
 
   const int plan_id = 42;
   master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_);
 
-  tx::CommandId command_id = dba.transaction().cid();
+  tx::CommandId command_id = dba->transaction().cid();
   Parameters params;
   std::vector<query::Symbol> symbols{ctx.symbol_table_[*return_n_r],
                                      ctx.symbol_table_[*return_m], p_sym};
@@ -190,8 +190,8 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) {
     return master().pull_clients().Pull(&dba, worker_id, plan_id, command_id,
                                         params, symbols, 0, false, 3);
   };
-  auto future_w1_results = remote_pull(dba, 1);
-  auto future_w2_results = remote_pull(dba, 2);
+  auto future_w1_results = remote_pull(*dba, 1);
+  auto future_w2_results = remote_pull(*dba, 2);
   check_result(1, future_w1_results.get().frames);
   check_result(2, future_w2_results.get().frames);
 }
@@ -204,8 +204,8 @@ TEST_F(DistributedQueryPlan, Synchronize) {
   // Query: MATCH (n)--(m) SET m.prop = 2 RETURN n.prop
   // This query ensures that a remote update gets applied and the local stuff
   // gets reconstructed.
-  auto &db = master();
-  GraphDbAccessor dba{db};
+  auto dba_ptr = master().Access();
+  auto &dba = *dba_ptr;
   Context ctx{dba};
   SymbolGenerator symbol_generator{ctx.symbol_table_};
   AstStorage storage;
@@ -252,9 +252,8 @@ TEST_F(DistributedQueryPlan, Synchronize) {
 
 TEST_F(DistributedQueryPlan, Create) {
   // Query: UNWIND range(0, 1000) as x CREATE ()
-  auto &db = master();
-  GraphDbAccessor dba{db};
-  Context ctx{dba};
+  auto dba = master().Access();
+  Context ctx{*dba};
   SymbolGenerator symbol_generator{ctx.symbol_table_};
   AstStorage storage;
   auto range = FN("range", LITERAL(0), LITERAL(1000));
@@ -264,8 +263,8 @@ TEST_F(DistributedQueryPlan, Create) {
   ctx.symbol_table_[*node->identifier_] =
       ctx.symbol_table_.CreateSymbol("n", true);
   auto create = std::make_shared<query::plan::CreateNode>(unwind, node, true);
-  PullAll(create, dba, ctx.symbol_table_);
-  dba.Commit();
+  PullAll(create, *dba, ctx.symbol_table_);
+  dba->Commit();
 
   EXPECT_GT(VertexCount(master()), 200);
   EXPECT_GT(VertexCount(worker(1)), 200);
@@ -276,11 +275,11 @@ TEST_F(DistributedQueryPlan, PullRemoteOrderBy) {
   // Create some data on the master and both workers.
   storage::Property prop;
   {
-    GraphDbAccessor dba{master()};
-    auto tx_id = dba.transaction_id();
-    GraphDbAccessor dba1{worker(1), tx_id};
-    GraphDbAccessor dba2{worker(2), tx_id};
-    prop = dba.Property("prop");
+    auto dba = master().Access();
+    auto tx_id = dba->transaction_id();
+    auto dba1 = worker(1).Access(tx_id);
+    auto dba2 = worker(2).Access(tx_id);
+    prop = dba->Property("prop");
     auto add_data = [prop](GraphDbAccessor &dba, int value) {
       dba.InsertVertex().PropsSet(prop, value);
     };
@@ -289,15 +288,15 @@ TEST_F(DistributedQueryPlan, PullRemoteOrderBy) {
     for (int i = 0; i < 300; ++i) data.push_back(i);
     std::random_shuffle(data.begin(), data.end());
 
-    for (int i = 0; i < 100; ++i) add_data(dba, data[i]);
-    for (int i = 100; i < 200; ++i) add_data(dba1, data[i]);
-    for (int i = 200; i < 300; ++i) add_data(dba2, data[i]);
+    for (int i = 0; i < 100; ++i) add_data(*dba, data[i]);
+    for (int i = 100; i < 200; ++i) add_data(*dba1, data[i]);
+    for (int i = 200; i < 300; ++i) add_data(*dba2, data[i]);
 
-    dba.Commit();
+    dba->Commit();
   }
 
-  auto &db = master();
-  GraphDbAccessor dba{db};
+  auto dba_ptr = master().Access();
+  auto &dba = *dba_ptr;
   Context ctx{dba};
   SymbolGenerator symbol_generator{ctx.symbol_table_};
   AstStorage storage;
@@ -341,8 +340,8 @@ TEST_F(DistributedTransactionTimeout, Timeout) {
   InsertVertex(worker(1));
   InsertVertex(worker(1));
 
-  GraphDbAccessor dba{master()};
-  Context ctx{dba};
+  auto dba = master().Access();
+  Context ctx{*dba};
   SymbolGenerator symbol_generator{ctx.symbol_table_};
   AstStorage storage;
 
@@ -356,14 +355,14 @@ TEST_F(DistributedTransactionTimeout, Timeout) {
 
   const int plan_id = 42;
   master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_);
-  tx::CommandId command_id = dba.transaction().cid();
+  tx::CommandId command_id = dba->transaction().cid();
 
   Parameters params;
   std::vector<query::Symbol> symbols{ctx.symbol_table_[*output]};
   auto remote_pull = [this, &command_id, &params, &symbols, &dba]() {
     return master()
         .pull_clients()
-        .Pull(&dba, 1, plan_id, command_id, params, symbols, 0, false, 1)
+        .Pull(dba.get(), 1, plan_id, command_id, params, symbols, 0, false, 1)
         .get()
         .pull_state;
   };
diff --git a/tests/unit/distributed_reset.cpp b/tests/unit/distributed_reset.cpp
index ba93b4e77..0a3dbaf1b 100644
--- a/tests/unit/distributed_reset.cpp
+++ b/tests/unit/distributed_reset.cpp
@@ -16,10 +16,10 @@ TEST_F(DistributedReset, ResetTest) {
   auto pull_remote = std::make_shared<query::plan::PullRemote>(
       once, 42, std::vector<query::Symbol>());
   master().plan_dispatcher().DispatchPlan(42, once, symbol_table);
-  database::GraphDbAccessor dba{master()};
+  auto dba = master().Access();
   query::Frame frame(0);
-  query::Context context(dba);
-  auto pull_remote_cursor = pull_remote->MakeCursor(dba);
+  query::Context context(*dba);
+  auto pull_remote_cursor = pull_remote->MakeCursor(*dba);
 
   for (int i = 0; i < 3; ++i) {
     EXPECT_TRUE(pull_remote_cursor->Pull(frame, context));
diff --git a/tests/unit/distributed_updates.cpp b/tests/unit/distributed_updates.cpp
index 00a5e314a..423b2b628 100644
--- a/tests/unit/distributed_updates.cpp
+++ b/tests/unit/distributed_updates.cpp
@@ -24,14 +24,13 @@ class DistributedUpdateTest : public DistributedGraphDbTest {
   void SetUp() override {
     DistributedGraphDbTest::SetUp();
 
-    database::GraphDbAccessor dba_tx1{worker(1)};
-    auto v = dba_tx1.InsertVertex();
+    auto dba_tx1 = worker(1).Access();
+    auto v = dba_tx1->InsertVertex();
     auto v_ga = v.GlobalAddress();
-    dba_tx1.Commit();
+    dba_tx1->Commit();
 
-    dba1 = std::make_unique<database::GraphDbAccessor>(worker(1));
-    dba2 = std::make_unique<database::GraphDbAccessor>(worker(2),
-                                                       dba1->transaction_id());
+    dba1 = worker(1).Access();
+    dba2 = worker(2).Access(dba1->transaction_id());
 
     v1_dba1 = std::make_unique<VertexAccessor>(v_ga, *dba1);
     v1_dba2 = std::make_unique<VertexAccessor>(v_ga, *dba2);
@@ -77,14 +76,14 @@ class DistributedGraphDbSimpleUpdatesTest : public DistributedGraphDbTest {
 TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertex) {
   gid::Gid gid;
   {
-    database::GraphDbAccessor dba{worker(1)};
-    auto v = dba.InsertVertexIntoRemote(2, {}, {});
+    auto dba = worker(1).Access();
+    auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {});
     gid = v.gid();
-    dba.Commit();
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba{worker(2)};
-    auto v = dba.FindVertexOptional(gid, false);
+    auto dba = worker(2).Access();
+    auto v = dba->FindVertexOptional(gid, false);
     ASSERT_TRUE(v);
   }
 }
@@ -93,17 +92,17 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithUpdate) {
   gid::Gid gid;
   storage::Property prop;
   {
-    database::GraphDbAccessor dba{worker(1)};
-    auto v = dba.InsertVertexIntoRemote(2, {}, {});
+    auto dba = worker(1).Access();
+    auto v = database::InsertVertexIntoRemote(dba.get(), 2, {}, {});
     gid = v.gid();
-    prop = dba.Property("prop");
+    prop = dba->Property("prop");
     v.PropsSet(prop, 42);
-    worker(2).updates_server().Apply(dba.transaction_id());
-    dba.Commit();
+    worker(2).updates_server().Apply(dba->transaction_id());
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba{worker(2)};
-    auto v = dba.FindVertexOptional(gid, false);
+    auto dba = worker(2).Access();
+    auto v = dba->FindVertexOptional(gid, false);
     ASSERT_TRUE(v);
     EXPECT_EQ(v->PropsAt(prop).Value<int64_t>(), 42);
   }
@@ -115,11 +114,12 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithData) {
   storage::Label l2;
   storage::Property prop;
   {
-    database::GraphDbAccessor dba{worker(1)};
-    l1 = dba.Label("l1");
-    l2 = dba.Label("l2");
-    prop = dba.Property("prop");
-    auto v = dba.InsertVertexIntoRemote(2, {l1, l2}, {{prop, 42}});
+    auto dba = worker(1).Access();
+    l1 = dba->Label("l1");
+    l2 = dba->Label("l2");
+    prop = dba->Property("prop");
+    auto v =
+        database::InsertVertexIntoRemote(dba.get(), 2, {l1, l2}, {{prop, 42}});
     gid = v.gid();
 
     // Check local visibility before commit.
@@ -127,12 +127,12 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, CreateVertexWithData) {
     EXPECT_TRUE(v.has_label(l2));
     EXPECT_EQ(v.PropsAt(prop).Value<int64_t>(), 42);
 
-    worker(2).updates_server().Apply(dba.transaction_id());
-    dba.Commit();
+    worker(2).updates_server().Apply(dba->transaction_id());
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba{worker(2)};
-    auto v = dba.FindVertexOptional(gid, false);
+    auto dba = worker(2).Access();
+    auto v = dba->FindVertexOptional(gid, false);
     ASSERT_TRUE(v);
     // Check remote data after commit.
     EXPECT_TRUE(v->has_label(l1));
@@ -148,23 +148,23 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, UpdateVertexRemoteAndLocal) {
   storage::Label l1;
   storage::Label l2;
   {
-    database::GraphDbAccessor dba{worker(1)};
-    auto v = dba.InsertVertex();
+    auto dba = worker(1).Access();
+    auto v = dba->InsertVertex();
     gid = v.gid();
-    l1 = dba.Label("label1");
-    l2 = dba.Label("label2");
-    dba.Commit();
+    l1 = dba->Label("label1");
+    l2 = dba->Label("label2");
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba0{master()};
-    database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-    auto v_local = dba1.FindVertex(gid, false);
-    auto v_remote = VertexAccessor(storage::VertexAddress(gid, 1), dba0);
+    auto dba0 = master().Access();
+    auto dba1 = worker(1).Access(dba0->transaction_id());
+    auto v_local = dba1->FindVertex(gid, false);
+    auto v_remote = VertexAccessor(storage::VertexAddress(gid, 1), *dba0);
 
     v_remote.add_label(l2);
     v_local.add_label(l1);
 
-    auto result = worker(1).updates_server().Apply(dba0.transaction_id());
+    auto result = worker(1).updates_server().Apply(dba0->transaction_id());
     EXPECT_EQ(result, distributed::UpdateResult::DONE);
   }
 }
@@ -172,20 +172,20 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, UpdateVertexRemoteAndLocal) {
 TEST_F(DistributedGraphDbSimpleUpdatesTest, AddSameLabelRemoteAndLocal) {
   auto v_address = InsertVertex(worker(1));
   {
-    database::GraphDbAccessor dba0{master()};
-    database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-    auto v_local = dba1.FindVertex(v_address.gid(), false);
-    auto v_remote = VertexAccessor(v_address, dba0);
-    auto l1 = dba1.Label("label");
+    auto dba0 = master().Access();
+    auto dba1 = worker(1).Access(dba0->transaction_id());
+    auto v_local = dba1->FindVertex(v_address.gid(), false);
+    auto v_remote = VertexAccessor(v_address, *dba0);
+    auto l1 = dba1->Label("label");
     v_remote.add_label(l1);
     v_local.add_label(l1);
-    worker(1).updates_server().Apply(dba0.transaction_id());
-    dba0.Commit();
+    worker(1).updates_server().Apply(dba0->transaction_id());
+    dba0->Commit();
   }
   {
-    database::GraphDbAccessor dba0{master()};
-    database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-    auto v = dba1.FindVertex(v_address.gid(), false);
+    auto dba0 = master().Access();
+    auto dba1 = worker(1).Access(dba0->transaction_id());
+    auto v = dba1->FindVertex(v_address.gid(), false);
     EXPECT_EQ(v.labels().size(), 1);
   }
 }
@@ -194,44 +194,44 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, IndexGetsUpdatedRemotely) {
   storage::VertexAddress v_remote = InsertVertex(worker(1));
   storage::Label label;
   {
-    database::GraphDbAccessor dba0{master()};
-    label = dba0.Label("label");
-    VertexAccessor va(v_remote, dba0);
+    auto dba0 = master().Access();
+    label = dba0->Label("label");
+    VertexAccessor va(v_remote, *dba0);
     va.add_label(label);
-    worker(1).updates_server().Apply(dba0.transaction_id());
-    dba0.Commit();
+    worker(1).updates_server().Apply(dba0->transaction_id());
+    dba0->Commit();
   }
   {
-    database::GraphDbAccessor dba1{worker(1)};
-    auto vertices = dba1.Vertices(label, false);
+    auto dba1 = worker(1).Access();
+    auto vertices = dba1->Vertices(label, false);
     EXPECT_EQ(std::distance(vertices.begin(), vertices.end()), 1);
   }
 }
 
 TEST_F(DistributedGraphDbSimpleUpdatesTest, DeleteVertexRemoteCommit) {
   auto v_address = InsertVertex(worker(1));
-  database::GraphDbAccessor dba0{master()};
-  database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-  auto v_remote = VertexAccessor(v_address, dba0);
-  dba0.RemoveVertex(v_remote);
-  EXPECT_TRUE(dba1.FindVertexOptional(v_address.gid(), true));
-  EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()),
+  auto dba0 = master().Access();
+  auto dba1 = worker(1).Access(dba0->transaction_id());
+  auto v_remote = VertexAccessor(v_address, *dba0);
+  dba0->RemoveVertex(v_remote);
+  EXPECT_TRUE(dba1->FindVertexOptional(v_address.gid(), true));
+  EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()),
             distributed::UpdateResult::DONE);
-  EXPECT_FALSE(dba1.FindVertexOptional(v_address.gid(), true));
+  EXPECT_FALSE(dba1->FindVertexOptional(v_address.gid(), true));
 }
 
 TEST_F(DistributedGraphDbSimpleUpdatesTest, DeleteVertexRemoteBothDelete) {
   auto v_address = InsertVertex(worker(1));
   {
-    database::GraphDbAccessor dba0{master()};
-    database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-    auto v_local = dba1.FindVertex(v_address.gid(), false);
-    auto v_remote = VertexAccessor(v_address, dba0);
-    EXPECT_TRUE(dba1.RemoveVertex(v_local));
-    EXPECT_TRUE(dba0.RemoveVertex(v_remote));
-    EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()),
+    auto dba0 = master().Access();
+    auto dba1 = worker(1).Access(dba0->transaction_id());
+    auto v_local = dba1->FindVertex(v_address.gid(), false);
+    auto v_remote = VertexAccessor(v_address, *dba0);
+    EXPECT_TRUE(dba1->RemoveVertex(v_local));
+    EXPECT_TRUE(dba0->RemoveVertex(v_remote));
+    EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()),
               distributed::UpdateResult::DONE);
-    EXPECT_FALSE(dba1.FindVertexOptional(v_address.gid(), true));
+    EXPECT_FALSE(dba1->FindVertexOptional(v_address.gid(), true));
   }
 }
 
@@ -240,27 +240,27 @@ TEST_F(DistributedGraphDbSimpleUpdatesTest, DeleteVertexRemoteStillConnected) {
   auto e_address = InsertEdge(v_address, v_address, "edge");
 
   {
-    database::GraphDbAccessor dba0{master()};
-    database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-    auto v_remote = VertexAccessor(v_address, dba0);
-    dba0.RemoveVertex(v_remote);
-    EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()),
+    auto dba0 = master().Access();
+    auto dba1 = worker(1).Access(dba0->transaction_id());
+    auto v_remote = VertexAccessor(v_address, *dba0);
+    dba0->RemoveVertex(v_remote);
+    EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()),
               distributed::UpdateResult::UNABLE_TO_DELETE_VERTEX_ERROR);
-    EXPECT_TRUE(dba1.FindVertexOptional(v_address.gid(), true));
+    EXPECT_TRUE(dba1->FindVertexOptional(v_address.gid(), true));
   }
   {
-    database::GraphDbAccessor dba0{master()};
-    database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-    auto e_local = dba1.FindEdge(e_address.gid(), false);
-    auto v_local = dba1.FindVertex(v_address.gid(), false);
-    auto v_remote = VertexAccessor(v_address, dba0);
+    auto dba0 = master().Access();
+    auto dba1 = worker(1).Access(dba0->transaction_id());
+    auto e_local = dba1->FindEdge(e_address.gid(), false);
+    auto v_local = dba1->FindVertex(v_address.gid(), false);
+    auto v_remote = VertexAccessor(v_address, *dba0);
 
-    dba1.RemoveEdge(e_local);
-    dba0.RemoveVertex(v_remote);
+    dba1->RemoveEdge(e_local);
+    dba0->RemoveVertex(v_remote);
 
-    EXPECT_EQ(worker(1).updates_server().Apply(dba0.transaction_id()),
+    EXPECT_EQ(worker(1).updates_server().Apply(dba0->transaction_id()),
               distributed::UpdateResult::DONE);
-    EXPECT_FALSE(dba1.FindVertexOptional(v_address.gid(), true));
+    EXPECT_FALSE(dba1->FindVertexOptional(v_address.gid(), true));
   }
 }
 
@@ -282,28 +282,21 @@ class DistributedDetachDeleteTest : public DistributedGraphDbTest {
   template <typename TF>
   void Run(storage::VertexAddress v_address, TF check_func) {
     for (int i : {0, 1, 2}) {
-      database::GraphDbAccessor dba0{master()};
-      database::GraphDbAccessor dba1{worker(1), dba0.transaction_id()};
-      database::GraphDbAccessor dba2{worker(2), dba0.transaction_id()};
+      auto dba0 = master().Access();
+      auto dba1 = worker(1).Access(dba0->transaction_id());
+      auto dba2 = worker(2).Access(dba0->transaction_id());
 
-      std::vector<std::reference_wrapper<database::GraphDbAccessor>> dba;
-      dba.emplace_back(dba0);
-      dba.emplace_back(dba1);
-      dba.emplace_back(dba2);
+      std::vector<std::reference_wrapper<database::GraphDbAccessor>> dba{
+          *dba0, *dba1, *dba2};
+      std::vector<database::DistributedGraphDb *> dbs{&master(), &worker(1),
+                                                      &worker(2)};
 
       auto &accessor = dba[i].get();
       auto v_accessor = VertexAccessor(v_address, accessor);
       accessor.DetachRemoveVertex(v_accessor);
 
-      for (auto db_accessor : dba) {
-        distributed::UpdatesRpcServer *updates_server = nullptr;
-        auto *db = &db_accessor.get().db();
-        if (auto *distributed_db =
-                dynamic_cast<database::DistributedGraphDb *>(db)) {
-          updates_server = &distributed_db->updates_server();
-        }
-        ASSERT_TRUE(updates_server);
-        ASSERT_EQ(updates_server->Apply(dba[0].get().transaction_id()),
+      for (auto *db : dbs) {
+        ASSERT_EQ(db->updates_server().Apply(dba[0].get().transaction_id()),
                   distributed::UpdateResult::DONE);
       }
 
@@ -388,29 +381,29 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest {
                   storage::VertexAddress to_addr) {
     CHECK(from_addr.is_remote() && to_addr.is_remote())
         << "Local address given to CreateEdge";
-    database::GraphDbAccessor dba{creator};
-    auto edge_type = dba.EdgeType("et");
-    VertexAccessor v1{from_addr, dba};
-    VertexAccessor v2{to_addr, dba};
-    auto edge = dba.InsertEdge(v1, v2, edge_type);
+    auto dba = creator.Access();
+    auto edge_type = dba->EdgeType("et");
+    VertexAccessor v1{from_addr, *dba};
+    VertexAccessor v2{to_addr, *dba};
+    auto edge = dba->InsertEdge(v1, v2, edge_type);
     e_ga = edge.GlobalAddress();
 
-    for (auto &kv : props) edge.PropsSet(dba.Property(kv.first), kv.second);
+    for (auto &kv : props) edge.PropsSet(dba->Property(kv.first), kv.second);
 
-    master().updates_server().Apply(dba.transaction_id());
-    worker(1).updates_server().Apply(dba.transaction_id());
-    worker(2).updates_server().Apply(dba.transaction_id());
-    dba.Commit();
+    master().updates_server().Apply(dba->transaction_id());
+    worker(1).updates_server().Apply(dba->transaction_id());
+    worker(2).updates_server().Apply(dba->transaction_id());
+    dba->Commit();
   }
 
   void CheckState(database::GraphDb &db, bool edge_is_local,
                   storage::VertexAddress from_addr,
                   storage::VertexAddress to_addr) {
-    database::GraphDbAccessor dba{db};
+    auto dba = db.Access();
 
     // Check edge data.
     {
-      EdgeAccessor edge{e_ga, dba};
+      EdgeAccessor edge{e_ga, *dba};
       EXPECT_EQ(edge.address().is_local(), edge_is_local);
       EXPECT_EQ(edge.GlobalAddress(), e_ga);
       auto from = edge.from();
@@ -422,7 +415,7 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest {
 
       EXPECT_EQ(edge.Properties().size(), props.size());
       for (auto &kv : props) {
-        auto equality = edge.PropsAt(dba.Property(kv.first)) ==
+        auto equality = edge.PropsAt(dba->Property(kv.first)) ==
                         query::TypedValue(kv.second);
         EXPECT_TRUE(equality.IsBool() && equality.ValueBool());
       }
@@ -436,7 +429,7 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest {
 
     // Check `from` data.
     {
-      VertexAccessor from{from_addr, dba};
+      VertexAccessor from{from_addr, *dba};
       ASSERT_EQ(edges(from.out()).size(), 1);
       EXPECT_EQ(edges(from.out())[0].GlobalAddress(), e_ga);
       // In case of cycles we have 1 in the `in` edges.
@@ -445,7 +438,7 @@ class DistributedEdgeCreateTest : public DistributedGraphDbTest {
 
     // Check `to` data.
     {
-      VertexAccessor to{to_addr, dba};
+      VertexAccessor to{to_addr, *dba};
       // In case of cycles we have 1 in the `out` edges.
       EXPECT_EQ(edges(to.out()).size(), from_addr == to_addr);
       ASSERT_EQ(edges(to.in()).size(), 1);
@@ -505,13 +498,13 @@ class DistributedEdgeRemoveTest : public DistributedGraphDbTest {
   }
 
   void Delete(database::GraphDb &db) {
-    database::GraphDbAccessor dba{db};
-    EdgeAccessor edge{edge_addr, dba};
-    dba.RemoveEdge(edge);
-    master().updates_server().Apply(dba.transaction_id());
-    worker(1).updates_server().Apply(dba.transaction_id());
-    worker(2).updates_server().Apply(dba.transaction_id());
-    dba.Commit();
+    auto dba = db.Access();
+    EdgeAccessor edge{edge_addr, *dba};
+    dba->RemoveEdge(edge);
+    master().updates_server().Apply(dba->transaction_id());
+    worker(1).updates_server().Apply(dba->transaction_id());
+    worker(2).updates_server().Apply(dba->transaction_id());
+    dba->Commit();
   }
 
   template <typename TIterable>
@@ -526,12 +519,12 @@ class DistributedEdgeRemoveTest : public DistributedGraphDbTest {
     ASSERT_EQ(EdgeCount(worker(1)), wid == 1);
     ASSERT_EQ(EdgeCount(worker(2)), wid == 2);
 
-    database::GraphDbAccessor dba{master()};
-    VertexAccessor from{from_addr, dba};
+    auto dba = master().Access();
+    VertexAccessor from{from_addr, *dba};
     EXPECT_EQ(Size(from.out()), 1);
     EXPECT_EQ(Size(from.in()), 0);
 
-    VertexAccessor to{to_addr, dba};
+    VertexAccessor to{to_addr, *dba};
     EXPECT_EQ(Size(to.out()), 0);
     EXPECT_EQ(Size(to.in()), 1);
   }
@@ -541,13 +534,13 @@ class DistributedEdgeRemoveTest : public DistributedGraphDbTest {
     EXPECT_EQ(EdgeCount(worker(1)), 0);
     EXPECT_EQ(EdgeCount(worker(2)), 0);
 
-    database::GraphDbAccessor dba{master()};
+    auto dba = master().Access();
 
-    VertexAccessor from{from_addr, dba};
+    VertexAccessor from{from_addr, *dba};
     EXPECT_EQ(Size(from.out()), 0);
     EXPECT_EQ(Size(from.in()), 0);
 
-    VertexAccessor to{to_addr, dba};
+    VertexAccessor to{to_addr, *dba};
     EXPECT_EQ(Size(to.out()), 0);
     EXPECT_EQ(Size(to.in()), 0);
   }
diff --git a/tests/unit/durability.cpp b/tests/unit/durability.cpp
index b342bb652..3ce05d941 100644
--- a/tests/unit/durability.cpp
+++ b/tests/unit/durability.cpp
@@ -11,6 +11,7 @@
 #include "glog/logging.h"
 #include "gtest/gtest.h"
 
+#include "database/distributed_graph_db.hpp"
 #include "database/graph_db.hpp"
 #include "database/graph_db_accessor.hpp"
 #include "database/state_delta.hpp"
@@ -165,12 +166,12 @@ class DbGenerator {
 /** Checks if the given databases have the same contents (indices,
  * vertices and edges). */
 void CompareDbs(database::GraphDb &a, database::GraphDb &b) {
-  database::GraphDbAccessor dba_a(a);
-  database::GraphDbAccessor dba_b(b);
+  auto dba_a = a.Access();
+  auto dba_b = b.Access();
 
   {
-    auto index_a = dba_a.IndexInfo();
-    auto index_b = dba_b.IndexInfo();
+    auto index_a = dba_a->IndexInfo();
+    auto index_b = dba_b->IndexInfo();
     EXPECT_TRUE(
         index_a.size() == index_b.size() &&
         std::is_permutation(index_a.begin(), index_a.end(), index_b.begin()))
@@ -183,8 +184,8 @@ void CompareDbs(database::GraphDb &a, database::GraphDb &b) {
     std::vector<std::pair<std::string, query::TypedValue>> p1;
     std::vector<std::pair<std::string, query::TypedValue>> p2;
 
-    for (auto x : p1_id) p1.push_back({dba_a.PropertyName(x.first), x.second});
-    for (auto x : p2_id) p2.push_back({dba_b.PropertyName(x.first), x.second});
+    for (auto x : p1_id) p1.push_back({dba_a->PropertyName(x.first), x.second});
+    for (auto x : p2_id) p2.push_back({dba_b->PropertyName(x.first), x.second});
 
     // Don't use a binary predicate which depends on different value getters
     // semantics for two containers because is_permutation might call the
@@ -200,37 +201,37 @@ void CompareDbs(database::GraphDb &a, database::GraphDb &b) {
 
   {
     int vertices_a_count = 0;
-    for (auto v_a : dba_a.Vertices(false)) {
+    for (auto v_a : dba_a->Vertices(false)) {
       vertices_a_count++;
-      auto v_b = dba_b.FindVertexOptional(v_a.gid(), false);
+      auto v_b = dba_b->FindVertexOptional(v_a.gid(), false);
       ASSERT_TRUE(v_b) << "Vertex not found, id: " << v_a.gid();
       ASSERT_EQ(v_a.labels().size(), v_b->labels().size());
       std::vector<std::string> v_a_labels;
       std::vector<std::string> v_b_labels;
-      for (auto x : v_a.labels()) v_a_labels.push_back(dba_a.LabelName(x));
-      for (auto x : v_b->labels()) v_b_labels.push_back(dba_b.LabelName(x));
+      for (auto x : v_a.labels()) v_a_labels.push_back(dba_a->LabelName(x));
+      for (auto x : v_b->labels()) v_b_labels.push_back(dba_b->LabelName(x));
       EXPECT_TRUE(std::is_permutation(v_a_labels.begin(), v_a_labels.end(),
                                       v_b_labels.begin()));
       EXPECT_TRUE(is_permutation_props(v_a.Properties(), v_b->Properties()));
     }
-    auto vertices_b = dba_b.Vertices(false);
+    auto vertices_b = dba_b->Vertices(false);
     EXPECT_EQ(std::distance(vertices_b.begin(), vertices_b.end()),
               vertices_a_count);
   }
   {
     int edges_a_count = 0;
-    for (auto e_a : dba_a.Edges(false)) {
+    for (auto e_a : dba_a->Edges(false)) {
       edges_a_count++;
-      auto e_b = dba_b.FindEdgeOptional(e_a.gid(), false);
+      auto e_b = dba_b->FindEdgeOptional(e_a.gid(), false);
       ASSERT_TRUE(e_b);
       ASSERT_TRUE(e_b) << "Edge not found, id: " << e_a.gid();
-      EXPECT_EQ(dba_a.EdgeTypeName(e_a.EdgeType()),
-                dba_b.EdgeTypeName(e_b->EdgeType()));
+      EXPECT_EQ(dba_a->EdgeTypeName(e_a.EdgeType()),
+                dba_b->EdgeTypeName(e_b->EdgeType()));
       EXPECT_EQ(e_a.from().gid(), e_b->from().gid());
       EXPECT_EQ(e_a.to().gid(), e_b->to().gid());
       EXPECT_TRUE(is_permutation_props(e_a.Properties(), e_b->Properties()));
     }
-    auto edges_b = dba_b.Edges(false);
+    auto edges_b = dba_b->Edges(false);
     EXPECT_EQ(std::distance(edges_b.begin(), edges_b.end()), edges_a_count);
   }
 }
@@ -276,9 +277,9 @@ void MakeDb(durability::WriteAheadLog &wal, database::GraphDbAccessor &dba,
 }
 
 void MakeDb(database::GraphDb &db, int scale, std::vector<int> indices = {}) {
-  database::GraphDbAccessor dba{db};
-  MakeDb(db.wal(), dba, scale, indices);
-  dba.Commit();
+  auto dba = db.Access();
+  MakeDb(db.wal(), *dba, scale, indices);
+  dba->Commit();
 }
 
 class Durability : public ::testing::Test {
@@ -303,8 +304,8 @@ class Durability : public ::testing::Test {
   }
 
   void MakeSnapshot(database::GraphDb &db, int snapshot_max_retained = -1) {
-    database::GraphDbAccessor dba(db);
-    ASSERT_TRUE(durability::MakeSnapshot(db, dba, durability_dir_,
+    auto dba = db.Access();
+    ASSERT_TRUE(durability::MakeSnapshot(db, *dba, durability_dir_,
                                          snapshot_max_retained));
   }
 
@@ -330,18 +331,18 @@ TEST_F(Durability, WalEncoding) {
     auto config = DbConfig();
     config.durability_enabled = true;
     database::SingleNode db{config};
-    database::GraphDbAccessor dba(db);
-    auto v0 = dba.InsertVertex();
+    auto dba = db.Access();
+    auto v0 = dba->InsertVertex();
     ASSERT_EQ(v0.gid(), gid0);
-    v0.add_label(dba.Label("l0"));
-    v0.PropsSet(dba.Property("p0"), 42);
-    auto v1 = dba.InsertVertex();
+    v0.add_label(dba->Label("l0"));
+    v0.PropsSet(dba->Property("p0"), 42);
+    auto v1 = dba->InsertVertex();
     ASSERT_EQ(v1.gid(), gid1);
-    auto e0 = dba.InsertEdge(v0, v1, dba.EdgeType("et0"));
+    auto e0 = dba->InsertEdge(v0, v1, dba->EdgeType("et0"));
     ASSERT_EQ(e0.gid(), gid0);
-    e0.PropsSet(dba.Property("p0"), std::vector<PropertyValue>{1, 2, 3});
-    dba.BuildIndex(dba.Label("l1"), dba.Property("p1"));
-    dba.Commit();
+    e0.PropsSet(dba->Property("p0"), std::vector<PropertyValue>{1, 2, 3});
+    dba->BuildIndex(dba->Label("l1"), dba->Property("p1"));
+    dba->Commit();
 
     db.wal().Flush();
   }
@@ -394,26 +395,26 @@ TEST_F(Durability, SnapshotEncoding) {
   auto gid2 = generator.Next();
   {
     database::SingleNode db{DbConfig()};
-    database::GraphDbAccessor dba(db);
-    auto v0 = dba.InsertVertex();
+    auto dba = db.Access();
+    auto v0 = dba->InsertVertex();
     ASSERT_EQ(v0.gid(), gid0);
-    v0.add_label(dba.Label("l0"));
-    v0.PropsSet(dba.Property("p0"), 42);
-    auto v1 = dba.InsertVertex();
+    v0.add_label(dba->Label("l0"));
+    v0.PropsSet(dba->Property("p0"), 42);
+    auto v1 = dba->InsertVertex();
     ASSERT_EQ(v1.gid(), gid1);
-    v1.add_label(dba.Label("l0"));
-    v1.add_label(dba.Label("l1"));
-    auto v2 = dba.InsertVertex();
+    v1.add_label(dba->Label("l0"));
+    v1.add_label(dba->Label("l1"));
+    auto v2 = dba->InsertVertex();
     ASSERT_EQ(v2.gid(), gid2);
-    v2.PropsSet(dba.Property("p0"), true);
-    v2.PropsSet(dba.Property("p1"), "Johnny");
-    auto e0 = dba.InsertEdge(v0, v1, dba.EdgeType("et0"));
+    v2.PropsSet(dba->Property("p0"), true);
+    v2.PropsSet(dba->Property("p1"), "Johnny");
+    auto e0 = dba->InsertEdge(v0, v1, dba->EdgeType("et0"));
     ASSERT_EQ(e0.gid(), gid0);
-    e0.PropsSet(dba.Property("p0"), std::vector<PropertyValue>{1, 2, 3});
-    auto e1 = dba.InsertEdge(v2, v1, dba.EdgeType("et1"));
+    e0.PropsSet(dba->Property("p0"), std::vector<PropertyValue>{1, 2, 3});
+    auto e1 = dba->InsertEdge(v2, v1, dba->EdgeType("et1"));
     ASSERT_EQ(e1.gid(), gid1);
-    dba.BuildIndex(dba.Label("l1"), dba.Property("p1"));
-    dba.Commit();
+    dba->BuildIndex(dba->Label("l1"), dba->Property("p1"));
+    dba->Commit();
     MakeSnapshot(db);
   }
 
@@ -528,9 +529,9 @@ TEST_F(Durability, SnapshotNoVerticesIdRecovery) {
   // vertices which should make it not change any id after snapshot recovery,
   // but we still have to make sure that the id for generators is recovered
   {
-    database::GraphDbAccessor dba(db);
-    for (auto vertex : dba.Vertices(false)) dba.RemoveVertex(vertex);
-    dba.Commit();
+    auto dba = db.Access();
+    for (auto vertex : dba->Vertices(false)) dba->RemoveVertex(vertex);
+    dba->Commit();
   }
 
   MakeSnapshot(db);
@@ -634,34 +635,34 @@ TEST_F(Durability, SnapshotAndWalRecoveryAfterComplexTxSituation) {
   database::SingleNode db{config};
 
   // The first transaction modifies and commits.
-  database::GraphDbAccessor dba_1{db};
-  MakeDb(db.wal(), dba_1, 100);
-  dba_1.Commit();
+  auto dba_1 = db.Access();
+  MakeDb(db.wal(), *dba_1, 100);
+  dba_1->Commit();
 
   // The second transaction will commit after snapshot.
-  database::GraphDbAccessor dba_2{db};
-  MakeDb(db.wal(), dba_2, 100);
+  auto dba_2 = db.Access();
+  MakeDb(db.wal(), *dba_2, 100);
 
   // The third transaction modifies and commits.
-  database::GraphDbAccessor dba_3{db};
-  MakeDb(db.wal(), dba_3, 100);
-  dba_3.Commit();
+  auto dba_3 = db.Access();
+  MakeDb(db.wal(), *dba_3, 100);
+  dba_3->Commit();
 
   MakeSnapshot(db);  // Snapshooter takes the fourth transaction.
-  dba_2.Commit();
+  dba_2->Commit();
 
   // The fifth transaction starts and commits after snapshot.
-  database::GraphDbAccessor dba_5{db};
-  MakeDb(db.wal(), dba_5, 100);
-  dba_5.Commit();
+  auto dba_5 = db.Access();
+  MakeDb(db.wal(), *dba_5, 100);
+  dba_5->Commit();
 
   // The sixth transaction will not commit at all.
-  database::GraphDbAccessor dba_6{db};
-  MakeDb(db.wal(), dba_6, 100);
+  auto dba_6 = db.Access();
+  MakeDb(db.wal(), *dba_6, 100);
 
   auto VisibleVertexCount = [](database::GraphDb &db) {
-    database::GraphDbAccessor dba{db};
-    auto vertices = dba.Vertices(false);
+    auto dba = db.Access();
+    auto vertices = dba->Vertices(false);
     return std::distance(vertices.begin(), vertices.end());
   };
   ASSERT_EQ(VisibleVertexCount(db), 400);
@@ -758,7 +759,8 @@ TEST_F(Durability, SnapshotOnExit) {
 TEST_F(Durability, WorkerIdRecovery) {
   auto config = DbConfig();
   config.worker_id = 5;
-  database::SingleNode db{config};
+  config.recovering_cluster_size = 1;
+  database::Master db{config};
   MakeDb(db, 100);
   MakeSnapshot(db);
   EXPECT_EQ(DirFiles(snapshot_dir_).size(), 1);
@@ -768,12 +770,13 @@ TEST_F(Durability, WorkerIdRecovery) {
     auto config = DbConfig();
     config.worker_id = 5;
     config.db_recover_on_startup = true;
-    database::SingleNode recovered{config};
+    config.recovering_cluster_size = 1;
+    database::Master recovered{config};
     EXPECT_EQ(recovered.WorkerId(), config.worker_id);
     CompareDbs(db, recovered);
-    database::GraphDbAccessor dba(recovered);
-    EXPECT_NE(dba.VerticesCount(), 0);
-    EXPECT_NE(dba.EdgesCount(), 0);
+    auto dba = recovered.Access();
+    EXPECT_NE(dba->VerticesCount(), 0);
+    EXPECT_NE(dba->EdgesCount(), 0);
   }
 
   // WorkerIds are not equal and recovery should fail
@@ -781,11 +784,12 @@ TEST_F(Durability, WorkerIdRecovery) {
     auto config = DbConfig();
     config.worker_id = 10;
     config.db_recover_on_startup = true;
-    database::SingleNode recovered{config};
+    config.recovering_cluster_size = 1;
+    database::Master recovered{config};
     EXPECT_NE(recovered.WorkerId(), db.WorkerId());
-    database::GraphDbAccessor dba(recovered);
-    EXPECT_EQ(dba.VerticesCount(), 0);
-    EXPECT_EQ(dba.EdgesCount(), 0);
+    auto dba = recovered.Access();
+    EXPECT_EQ(dba->VerticesCount(), 0);
+    EXPECT_EQ(dba->EdgesCount(), 0);
   }
 }
 
@@ -800,9 +804,9 @@ TEST_F(Durability, SequentialRecovery) {
   };
 
   auto init_db = [](database::GraphDb &db) {
-    database::GraphDbAccessor dba{db};
-    for (int i = 0; i < kNumVertices; ++i) dba.InsertVertex(i);
-    dba.Commit();
+    auto dba = db.Access();
+    for (int i = 0; i < kNumVertices; ++i) dba->InsertVertex(i);
+    dba->Commit();
   };
 
   auto run_updates = [&random_int](database::GraphDb &db,
@@ -811,15 +815,15 @@ TEST_F(Durability, SequentialRecovery) {
     for (int i = 0; i < kNumWorkers; ++i) {
       threads.emplace_back([&random_int, &db, &keep_running]() {
         while (keep_running) {
-          database::GraphDbAccessor dba{db};
-          auto v = dba.FindVertex(random_int(kNumVertices), false);
+          auto dba = db.Access();
+          auto v = dba->FindVertex(random_int(kNumVertices), false);
           try {
-            v.PropsSet(dba.Property("prop"), random_int(100));
+            v.PropsSet(dba->Property("prop"), random_int(100));
           } catch (utils::LockTimeoutException &) {
           } catch (mvcc::SerializationError &) {
           }
-          dba.InsertVertex();
-          dba.Commit();
+          dba->InsertVertex();
+          dba->Commit();
         }
       });
     }
diff --git a/tests/unit/graph_db.cpp b/tests/unit/graph_db.cpp
index 085b072e1..25dbadd23 100644
--- a/tests/unit/graph_db.cpp
+++ b/tests/unit/graph_db.cpp
@@ -11,12 +11,11 @@ TEST(GraphDbTest, GarbageCollectIndices) {
   database::Config config;
   config.gc_cycle_sec = -1;
   database::SingleNode graph_db{config};
-  std::unique_ptr<database::GraphDbAccessor> dba =
-      std::make_unique<database::GraphDbAccessor>(graph_db);
+  std::unique_ptr<database::GraphDbAccessor> dba = graph_db.Access();
 
   auto commit = [&] {
     dba->Commit();
-    dba = std::make_unique<database::GraphDbAccessor>(graph_db);
+    dba = graph_db.Access();
   };
   auto label = dba->Label("label");
   auto property = dba->Property("property");
diff --git a/tests/unit/graph_db_accessor.cpp b/tests/unit/graph_db_accessor.cpp
index 151ae6ef2..f11dcce45 100644
--- a/tests/unit/graph_db_accessor.cpp
+++ b/tests/unit/graph_db_accessor.cpp
@@ -17,22 +17,22 @@ auto Count(TIterable iterable) {
 
 TEST(GraphDbAccessorTest, InsertVertex) {
   SingleNode db;
-  GraphDbAccessor accessor(db);
+  auto accessor = db.Access();
   gid::Generator generator(0);
 
-  EXPECT_EQ(Count(accessor.Vertices(false)), 0);
+  EXPECT_EQ(Count(accessor->Vertices(false)), 0);
 
-  EXPECT_EQ(accessor.InsertVertex().gid(), generator.Next());
-  EXPECT_EQ(Count(accessor.Vertices(false)), 0);
-  EXPECT_EQ(Count(accessor.Vertices(true)), 1);
-  accessor.AdvanceCommand();
-  EXPECT_EQ(Count(accessor.Vertices(false)), 1);
+  EXPECT_EQ(accessor->InsertVertex().gid(), generator.Next());
+  EXPECT_EQ(Count(accessor->Vertices(false)), 0);
+  EXPECT_EQ(Count(accessor->Vertices(true)), 1);
+  accessor->AdvanceCommand();
+  EXPECT_EQ(Count(accessor->Vertices(false)), 1);
 
-  EXPECT_EQ(accessor.InsertVertex().gid(), generator.Next());
-  EXPECT_EQ(Count(accessor.Vertices(false)), 1);
-  EXPECT_EQ(Count(accessor.Vertices(true)), 2);
-  accessor.AdvanceCommand();
-  EXPECT_EQ(Count(accessor.Vertices(false)), 2);
+  EXPECT_EQ(accessor->InsertVertex().gid(), generator.Next());
+  EXPECT_EQ(Count(accessor->Vertices(false)), 1);
+  EXPECT_EQ(Count(accessor->Vertices(true)), 2);
+  accessor->AdvanceCommand();
+  EXPECT_EQ(Count(accessor->Vertices(false)), 2);
 }
 
 TEST(GraphDbAccessorTest, UniqueVertexId) {
@@ -42,9 +42,9 @@ TEST(GraphDbAccessorTest, UniqueVertexId) {
   std::vector<std::thread> threads;
   for (int i = 0; i < 50; i++) {
     threads.emplace_back([&db, &ids]() {
-      GraphDbAccessor dba(db);
+      auto dba = db.Access();
       auto access = ids.access();
-      for (int i = 0; i < 200; i++) access.insert(dba.InsertVertex().gid());
+      for (int i = 0; i < 200; i++) access.insert(dba->InsertVertex().gid());
     });
   }
 
@@ -54,66 +54,66 @@ TEST(GraphDbAccessorTest, UniqueVertexId) {
 
 TEST(GraphDbAccessorTest, RemoveVertexSameTransaction) {
   SingleNode db;
-  GraphDbAccessor accessor(db);
+  auto accessor = db.Access();
 
-  EXPECT_EQ(Count(accessor.Vertices(false)), 0);
+  EXPECT_EQ(Count(accessor->Vertices(false)), 0);
 
-  auto va1 = accessor.InsertVertex();
-  accessor.AdvanceCommand();
-  EXPECT_EQ(Count(accessor.Vertices(false)), 1);
+  auto va1 = accessor->InsertVertex();
+  accessor->AdvanceCommand();
+  EXPECT_EQ(Count(accessor->Vertices(false)), 1);
 
-  EXPECT_TRUE(accessor.RemoveVertex(va1));
-  EXPECT_EQ(Count(accessor.Vertices(false)), 1);
-  EXPECT_EQ(Count(accessor.Vertices(true)), 0);
-  accessor.AdvanceCommand();
-  EXPECT_EQ(Count(accessor.Vertices(false)), 0);
-  EXPECT_EQ(Count(accessor.Vertices(true)), 0);
+  EXPECT_TRUE(accessor->RemoveVertex(va1));
+  EXPECT_EQ(Count(accessor->Vertices(false)), 1);
+  EXPECT_EQ(Count(accessor->Vertices(true)), 0);
+  accessor->AdvanceCommand();
+  EXPECT_EQ(Count(accessor->Vertices(false)), 0);
+  EXPECT_EQ(Count(accessor->Vertices(true)), 0);
 }
 
 TEST(GraphDbAccessorTest, RemoveVertexDifferentTransaction) {
   SingleNode db;
   // first transaction creates a vertex
   {
-    GraphDbAccessor accessor(db);
-    accessor.InsertVertex();
-    accessor.Commit();
+    auto accessor = db.Access();
+    accessor->InsertVertex();
+    accessor->Commit();
   }
   // second transaction checks that it sees it, and deletes it
   {
-    GraphDbAccessor accessor(db);
-    EXPECT_EQ(Count(accessor.Vertices(false)), 1);
-    EXPECT_EQ(Count(accessor.Vertices(true)), 1);
-    for (auto vertex_accessor : accessor.Vertices(false))
-      accessor.RemoveVertex(vertex_accessor);
-    accessor.Commit();
+    auto accessor = db.Access();
+    EXPECT_EQ(Count(accessor->Vertices(false)), 1);
+    EXPECT_EQ(Count(accessor->Vertices(true)), 1);
+    for (auto vertex_accessor : accessor->Vertices(false))
+      accessor->RemoveVertex(vertex_accessor);
+    accessor->Commit();
   }
   // third transaction checks that it does not see the vertex
   {
-    GraphDbAccessor accessor(db);
-    EXPECT_EQ(Count(accessor.Vertices(false)), 0);
-    EXPECT_EQ(Count(accessor.Vertices(true)), 0);
+    auto accessor = db.Access();
+    EXPECT_EQ(Count(accessor->Vertices(false)), 0);
+    EXPECT_EQ(Count(accessor->Vertices(true)), 0);
   }
 }
 
 TEST(GraphDbAccessorTest, InsertEdge) {
   SingleNode db;
-  GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto va1 = dba.InsertVertex();
-  auto va2 = dba.InsertVertex();
-  dba.AdvanceCommand();
+  auto va1 = dba->InsertVertex();
+  auto va2 = dba->InsertVertex();
+  dba->AdvanceCommand();
   EXPECT_EQ(va1.in_degree(), 0);
   EXPECT_EQ(va1.out_degree(), 0);
   EXPECT_EQ(va2.in_degree(), 0);
   EXPECT_EQ(va2.out_degree(), 0);
 
   // setup (v1) - [:likes] -> (v2)
-  dba.InsertEdge(va1, va2, dba.EdgeType("likes"));
-  EXPECT_EQ(Count(dba.Edges(false)), 0);
-  EXPECT_EQ(Count(dba.Edges(true)), 1);
-  dba.AdvanceCommand();
-  EXPECT_EQ(Count(dba.Edges(false)), 1);
-  EXPECT_EQ(Count(dba.Edges(true)), 1);
+  dba->InsertEdge(va1, va2, dba->EdgeType("likes"));
+  EXPECT_EQ(Count(dba->Edges(false)), 0);
+  EXPECT_EQ(Count(dba->Edges(true)), 1);
+  dba->AdvanceCommand();
+  EXPECT_EQ(Count(dba->Edges(false)), 1);
+  EXPECT_EQ(Count(dba->Edges(true)), 1);
   EXPECT_EQ(va1.out().begin()->to(), va2);
   EXPECT_EQ(va2.in().begin()->from(), va1);
   EXPECT_EQ(va1.in_degree(), 0);
@@ -122,12 +122,12 @@ TEST(GraphDbAccessorTest, InsertEdge) {
   EXPECT_EQ(va2.out_degree(), 0);
 
   // setup (v1) - [:likes] -> (v2) <- [:hates] - (v3)
-  auto va3 = dba.InsertVertex();
-  dba.InsertEdge(va3, va2, dba.EdgeType("hates"));
-  EXPECT_EQ(Count(dba.Edges(false)), 1);
-  EXPECT_EQ(Count(dba.Edges(true)), 2);
-  dba.AdvanceCommand();
-  EXPECT_EQ(Count(dba.Edges(false)), 2);
+  auto va3 = dba->InsertVertex();
+  dba->InsertEdge(va3, va2, dba->EdgeType("hates"));
+  EXPECT_EQ(Count(dba->Edges(false)), 1);
+  EXPECT_EQ(Count(dba->Edges(true)), 2);
+  dba->AdvanceCommand();
+  EXPECT_EQ(Count(dba->Edges(false)), 2);
   EXPECT_EQ(va3.out().begin()->to(), va2);
   EXPECT_EQ(va1.in_degree(), 0);
   EXPECT_EQ(va1.out_degree(), 1);
@@ -144,13 +144,13 @@ TEST(GraphDbAccessorTest, UniqueEdgeId) {
   std::vector<std::thread> threads;
   for (int i = 0; i < 50; i++) {
     threads.emplace_back([&db, &ids]() {
-      GraphDbAccessor dba(db);
-      auto v1 = dba.InsertVertex();
-      auto v2 = dba.InsertVertex();
-      auto edge_type = dba.EdgeType("edge_type");
+      auto dba = db.Access();
+      auto v1 = dba->InsertVertex();
+      auto v2 = dba->InsertVertex();
+      auto edge_type = dba->EdgeType("edge_type");
       auto access = ids.access();
       for (int i = 0; i < 200; i++)
-        access.insert(dba.InsertEdge(v1, v2, edge_type).gid());
+        access.insert(dba->InsertEdge(v1, v2, edge_type).gid());
     });
   }
 
@@ -160,37 +160,37 @@ TEST(GraphDbAccessorTest, UniqueEdgeId) {
 
 TEST(GraphDbAccessorTest, RemoveEdge) {
   SingleNode db;
-  GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // setup (v1) - [:likes] -> (v2) <- [:hates] - (v3)
-  auto va1 = dba.InsertVertex();
-  auto va2 = dba.InsertVertex();
-  auto va3 = dba.InsertVertex();
-  dba.InsertEdge(va1, va2, dba.EdgeType("likes"));
-  dba.InsertEdge(va3, va2, dba.EdgeType("hates"));
-  dba.AdvanceCommand();
-  EXPECT_EQ(Count(dba.Edges(false)), 2);
-  EXPECT_EQ(Count(dba.Edges(true)), 2);
+  auto va1 = dba->InsertVertex();
+  auto va2 = dba->InsertVertex();
+  auto va3 = dba->InsertVertex();
+  dba->InsertEdge(va1, va2, dba->EdgeType("likes"));
+  dba->InsertEdge(va3, va2, dba->EdgeType("hates"));
+  dba->AdvanceCommand();
+  EXPECT_EQ(Count(dba->Edges(false)), 2);
+  EXPECT_EQ(Count(dba->Edges(true)), 2);
 
   // remove all [:hates] edges
-  for (auto edge : dba.Edges(false))
-    if (edge.EdgeType() == dba.EdgeType("hates")) dba.RemoveEdge(edge);
-  EXPECT_EQ(Count(dba.Edges(false)), 2);
-  EXPECT_EQ(Count(dba.Edges(true)), 1);
+  for (auto edge : dba->Edges(false))
+    if (edge.EdgeType() == dba->EdgeType("hates")) dba->RemoveEdge(edge);
+  EXPECT_EQ(Count(dba->Edges(false)), 2);
+  EXPECT_EQ(Count(dba->Edges(true)), 1);
 
   // current state: (v1) - [:likes] -> (v2), (v3)
-  dba.AdvanceCommand();
-  EXPECT_EQ(Count(dba.Edges(false)), 1);
-  EXPECT_EQ(Count(dba.Edges(true)), 1);
-  EXPECT_EQ(Count(dba.Vertices(false)), 3);
-  EXPECT_EQ(Count(dba.Vertices(true)), 3);
-  for (auto edge : dba.Edges(false)) {
-    EXPECT_EQ(edge.EdgeType(), dba.EdgeType("likes"));
+  dba->AdvanceCommand();
+  EXPECT_EQ(Count(dba->Edges(false)), 1);
+  EXPECT_EQ(Count(dba->Edges(true)), 1);
+  EXPECT_EQ(Count(dba->Vertices(false)), 3);
+  EXPECT_EQ(Count(dba->Vertices(true)), 3);
+  for (auto edge : dba->Edges(false)) {
+    EXPECT_EQ(edge.EdgeType(), dba->EdgeType("likes"));
     auto v1 = edge.from();
     auto v2 = edge.to();
 
     // ensure correct connectivity for all the vertices
-    for (auto vertex : dba.Vertices(false)) {
+    for (auto vertex : dba->Vertices(false)) {
       if (vertex == v1) {
         EXPECT_EQ(vertex.in_degree(), 0);
         EXPECT_EQ(vertex.out_degree(), 1);
@@ -207,73 +207,73 @@ TEST(GraphDbAccessorTest, RemoveEdge) {
 
 TEST(GraphDbAccessorTest, DetachRemoveVertex) {
   SingleNode db;
-  GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // setup (v0)- []->(v1)<-[]-(v2)<-[]-(v3)
   std::vector<VertexAccessor> vertices;
-  for (int i = 0; i < 4; ++i) vertices.emplace_back(dba.InsertVertex());
+  for (int i = 0; i < 4; ++i) vertices.emplace_back(dba->InsertVertex());
 
-  auto edge_type = dba.EdgeType("type");
-  dba.InsertEdge(vertices[0], vertices[1], edge_type);
-  dba.InsertEdge(vertices[2], vertices[1], edge_type);
-  dba.InsertEdge(vertices[3], vertices[2], edge_type);
+  auto edge_type = dba->EdgeType("type");
+  dba->InsertEdge(vertices[0], vertices[1], edge_type);
+  dba->InsertEdge(vertices[2], vertices[1], edge_type);
+  dba->InsertEdge(vertices[3], vertices[2], edge_type);
 
-  dba.AdvanceCommand();
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
 
   // ensure that plain remove does NOT work
-  EXPECT_EQ(Count(dba.Vertices(false)), 4);
-  EXPECT_EQ(Count(dba.Edges(false)), 3);
-  EXPECT_FALSE(dba.RemoveVertex(vertices[0]));
-  EXPECT_FALSE(dba.RemoveVertex(vertices[1]));
-  EXPECT_FALSE(dba.RemoveVertex(vertices[2]));
-  EXPECT_EQ(Count(dba.Vertices(false)), 4);
-  EXPECT_EQ(Count(dba.Edges(false)), 3);
+  EXPECT_EQ(Count(dba->Vertices(false)), 4);
+  EXPECT_EQ(Count(dba->Edges(false)), 3);
+  EXPECT_FALSE(dba->RemoveVertex(vertices[0]));
+  EXPECT_FALSE(dba->RemoveVertex(vertices[1]));
+  EXPECT_FALSE(dba->RemoveVertex(vertices[2]));
+  EXPECT_EQ(Count(dba->Vertices(false)), 4);
+  EXPECT_EQ(Count(dba->Edges(false)), 3);
 
-  dba.DetachRemoveVertex(vertices[2]);
-  EXPECT_EQ(Count(dba.Vertices(false)), 4);
-  EXPECT_EQ(Count(dba.Vertices(true)), 3);
-  EXPECT_EQ(Count(dba.Edges(false)), 3);
-  EXPECT_EQ(Count(dba.Edges(true)), 1);
-  dba.AdvanceCommand();
+  dba->DetachRemoveVertex(vertices[2]);
+  EXPECT_EQ(Count(dba->Vertices(false)), 4);
+  EXPECT_EQ(Count(dba->Vertices(true)), 3);
+  EXPECT_EQ(Count(dba->Edges(false)), 3);
+  EXPECT_EQ(Count(dba->Edges(true)), 1);
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
 
-  EXPECT_EQ(Count(dba.Vertices(false)), 3);
-  EXPECT_EQ(Count(dba.Edges(false)), 1);
-  EXPECT_TRUE(dba.RemoveVertex(vertices[3]));
-  EXPECT_EQ(Count(dba.Vertices(true)), 2);
-  EXPECT_EQ(Count(dba.Vertices(false)), 3);
-  dba.AdvanceCommand();
+  EXPECT_EQ(Count(dba->Vertices(false)), 3);
+  EXPECT_EQ(Count(dba->Edges(false)), 1);
+  EXPECT_TRUE(dba->RemoveVertex(vertices[3]));
+  EXPECT_EQ(Count(dba->Vertices(true)), 2);
+  EXPECT_EQ(Count(dba->Vertices(false)), 3);
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
 
-  EXPECT_EQ(Count(dba.Vertices(false)), 2);
-  EXPECT_EQ(Count(dba.Edges(false)), 1);
-  for (auto va : dba.Vertices(false)) EXPECT_FALSE(dba.RemoveVertex(va));
-  dba.AdvanceCommand();
+  EXPECT_EQ(Count(dba->Vertices(false)), 2);
+  EXPECT_EQ(Count(dba->Edges(false)), 1);
+  for (auto va : dba->Vertices(false)) EXPECT_FALSE(dba->RemoveVertex(va));
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
 
-  EXPECT_EQ(Count(dba.Vertices(false)), 2);
-  EXPECT_EQ(Count(dba.Edges(false)), 1);
-  for (auto va : dba.Vertices(false)) {
-    EXPECT_FALSE(dba.RemoveVertex(va));
-    dba.DetachRemoveVertex(va);
+  EXPECT_EQ(Count(dba->Vertices(false)), 2);
+  EXPECT_EQ(Count(dba->Edges(false)), 1);
+  for (auto va : dba->Vertices(false)) {
+    EXPECT_FALSE(dba->RemoveVertex(va));
+    dba->DetachRemoveVertex(va);
     break;
   }
-  EXPECT_EQ(Count(dba.Vertices(true)), 1);
-  EXPECT_EQ(Count(dba.Vertices(false)), 2);
-  dba.AdvanceCommand();
+  EXPECT_EQ(Count(dba->Vertices(true)), 1);
+  EXPECT_EQ(Count(dba->Vertices(false)), 2);
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
 
-  EXPECT_EQ(Count(dba.Vertices(false)), 1);
-  EXPECT_EQ(Count(dba.Edges(false)), 0);
+  EXPECT_EQ(Count(dba->Vertices(false)), 1);
+  EXPECT_EQ(Count(dba->Edges(false)), 0);
 
   // remove the last vertex, it has no connections
   // so that should work
-  for (auto va : dba.Vertices(false)) EXPECT_TRUE(dba.RemoveVertex(va));
-  dba.AdvanceCommand();
+  for (auto va : dba->Vertices(false)) EXPECT_TRUE(dba->RemoveVertex(va));
+  dba->AdvanceCommand();
 
-  EXPECT_EQ(Count(dba.Vertices(false)), 0);
-  EXPECT_EQ(Count(dba.Edges(false)), 0);
+  EXPECT_EQ(Count(dba->Vertices(false)), 0);
+  EXPECT_EQ(Count(dba->Edges(false)), 0);
 }
 
 TEST(GraphDbAccessorTest, DetachRemoveVertexMultiple) {
@@ -281,113 +281,113 @@ TEST(GraphDbAccessorTest, DetachRemoveVertexMultiple) {
   // same vertex / edge multiple times
 
   SingleNode db;
-  GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // setup: make a fully connected N graph
   // with cycles too!
   int N = 7;
   std::vector<VertexAccessor> vertices;
-  auto edge_type = dba.EdgeType("edge");
-  for (int i = 0; i < N; ++i) vertices.emplace_back(dba.InsertVertex());
+  auto edge_type = dba->EdgeType("edge");
+  for (int i = 0; i < N; ++i) vertices.emplace_back(dba->InsertVertex());
   for (int j = 0; j < N; ++j)
     for (int k = 0; k < N; ++k)
-      dba.InsertEdge(vertices[j], vertices[k], edge_type);
+      dba->InsertEdge(vertices[j], vertices[k], edge_type);
 
-  dba.AdvanceCommand();
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
 
-  EXPECT_EQ(Count(dba.Vertices(false)), N);
-  EXPECT_EQ(Count(dba.Edges(false)), N * N);
+  EXPECT_EQ(Count(dba->Vertices(false)), N);
+  EXPECT_EQ(Count(dba->Edges(false)), N * N);
 
   // detach delete one edge
-  dba.DetachRemoveVertex(vertices[0]);
-  dba.AdvanceCommand();
+  dba->DetachRemoveVertex(vertices[0]);
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
-  EXPECT_EQ(Count(dba.Vertices(false)), N - 1);
-  EXPECT_EQ(Count(dba.Edges(false)), (N - 1) * (N - 1));
+  EXPECT_EQ(Count(dba->Vertices(false)), N - 1);
+  EXPECT_EQ(Count(dba->Edges(false)), (N - 1) * (N - 1));
 
   // detach delete two neighboring edges
-  dba.DetachRemoveVertex(vertices[1]);
-  dba.DetachRemoveVertex(vertices[2]);
-  dba.AdvanceCommand();
+  dba->DetachRemoveVertex(vertices[1]);
+  dba->DetachRemoveVertex(vertices[2]);
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
-  EXPECT_EQ(Count(dba.Vertices(false)), N - 3);
-  EXPECT_EQ(Count(dba.Edges(false)), (N - 3) * (N - 3));
+  EXPECT_EQ(Count(dba->Vertices(false)), N - 3);
+  EXPECT_EQ(Count(dba->Edges(false)), (N - 3) * (N - 3));
 
   // detach delete everything, buwahahahaha
-  for (int l = 3; l < N; ++l) dba.DetachRemoveVertex(vertices[l]);
-  dba.AdvanceCommand();
+  for (int l = 3; l < N; ++l) dba->DetachRemoveVertex(vertices[l]);
+  dba->AdvanceCommand();
   for (auto &vertex : vertices) vertex.Reconstruct();
-  EXPECT_EQ(Count(dba.Vertices(false)), 0);
-  EXPECT_EQ(Count(dba.Edges(false)), 0);
+  EXPECT_EQ(Count(dba->Vertices(false)), 0);
+  EXPECT_EQ(Count(dba->Edges(false)), 0);
 }
 
 TEST(GraphDbAccessorTest, Labels) {
   SingleNode db;
-  GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  Label label_friend = dba.Label("friend");
-  EXPECT_EQ(label_friend, dba.Label("friend"));
-  EXPECT_NE(label_friend, dba.Label("friend2"));
-  EXPECT_EQ(dba.LabelName(label_friend), "friend");
+  Label label_friend = dba->Label("friend");
+  EXPECT_EQ(label_friend, dba->Label("friend"));
+  EXPECT_NE(label_friend, dba->Label("friend2"));
+  EXPECT_EQ(dba->LabelName(label_friend), "friend");
 
   // test that getting labels through a different accessor works
-  EXPECT_EQ(label_friend, GraphDbAccessor(db).Label("friend"));
-  EXPECT_NE(label_friend, GraphDbAccessor(db).Label("friend2"));
+  EXPECT_EQ(label_friend, db.Access()->Label("friend"));
+  EXPECT_NE(label_friend, db.Access()->Label("friend2"));
 }
 
 TEST(GraphDbAccessorTest, EdgeTypes) {
   SingleNode db;
-  GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  EdgeType edge_type = dba.EdgeType("likes");
-  EXPECT_EQ(edge_type, dba.EdgeType("likes"));
-  EXPECT_NE(edge_type, dba.EdgeType("hates"));
-  EXPECT_EQ(dba.EdgeTypeName(edge_type), "likes");
+  EdgeType edge_type = dba->EdgeType("likes");
+  EXPECT_EQ(edge_type, dba->EdgeType("likes"));
+  EXPECT_NE(edge_type, dba->EdgeType("hates"));
+  EXPECT_EQ(dba->EdgeTypeName(edge_type), "likes");
 
   // test that getting labels through a different accessor works
-  EXPECT_EQ(edge_type, GraphDbAccessor(db).EdgeType("likes"));
-  EXPECT_NE(edge_type, GraphDbAccessor(db).EdgeType("hates"));
+  EXPECT_EQ(edge_type, db.Access()->EdgeType("likes"));
+  EXPECT_NE(edge_type, db.Access()->EdgeType("hates"));
 }
 
 TEST(GraphDbAccessorTest, Properties) {
   SingleNode db;
-  GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  Property prop = dba.Property("name");
-  EXPECT_EQ(prop, dba.Property("name"));
-  EXPECT_NE(prop, dba.Property("surname"));
-  EXPECT_EQ(dba.PropertyName(prop), "name");
+  Property prop = dba->Property("name");
+  EXPECT_EQ(prop, dba->Property("name"));
+  EXPECT_NE(prop, dba->Property("surname"));
+  EXPECT_EQ(dba->PropertyName(prop), "name");
 
   // test that getting labels through a different accessor works
-  EXPECT_EQ(prop, GraphDbAccessor(db).Property("name"));
-  EXPECT_NE(prop, GraphDbAccessor(db).Property("surname"));
+  EXPECT_EQ(prop, db.Access()->Property("name"));
+  EXPECT_NE(prop, db.Access()->Property("surname"));
 }
 
 TEST(GraphDbAccessorTest, Transfer) {
   SingleNode db;
 
-  GraphDbAccessor dba1(db);
-  auto prop = dba1.Property("property");
-  VertexAccessor v1 = dba1.InsertVertex();
+  auto dba1 = db.Access();
+  auto prop = dba1->Property("property");
+  VertexAccessor v1 = dba1->InsertVertex();
   v1.PropsSet(prop, 1);
-  VertexAccessor v2 = dba1.InsertVertex();
+  VertexAccessor v2 = dba1->InsertVertex();
   v2.PropsSet(prop, 2);
-  EdgeAccessor e12 = dba1.InsertEdge(v1, v2, dba1.EdgeType("et"));
+  EdgeAccessor e12 = dba1->InsertEdge(v1, v2, dba1->EdgeType("et"));
   e12.PropsSet(prop, 12);
 
   // make dba2 that has dba1 in it's snapshot, so data isn't visible
-  GraphDbAccessor dba2(db);
-  EXPECT_EQ(dba2.Transfer(v1), std::experimental::nullopt);
-  EXPECT_EQ(dba2.Transfer(e12), std::experimental::nullopt);
+  auto dba2 = db.Access();
+  EXPECT_EQ(dba2->Transfer(v1), std::experimental::nullopt);
+  EXPECT_EQ(dba2->Transfer(e12), std::experimental::nullopt);
 
   // make dba3 that does not have dba1 in it's snapshot
-  dba1.Commit();
-  GraphDbAccessor dba3(db);
+  dba1->Commit();
+  auto dba3 = db.Access();
   // we can transfer accessors even though the GraphDbAccessor they
   // belong to is not alive anymore
-  EXPECT_EQ(dba3.Transfer(v1)->PropsAt(prop).Value<int64_t>(), 1);
-  EXPECT_EQ(dba3.Transfer(e12)->PropsAt(prop).Value<int64_t>(), 12);
+  EXPECT_EQ(dba3->Transfer(v1)->PropsAt(prop).Value<int64_t>(), 1);
+  EXPECT_EQ(dba3->Transfer(e12)->PropsAt(prop).Value<int64_t>(), 12);
 }
 
 int main(int argc, char **argv) {
diff --git a/tests/unit/graph_db_accessor_index_api.cpp b/tests/unit/graph_db_accessor_index_api.cpp
index bb910865b..8619f57c3 100644
--- a/tests/unit/graph_db_accessor_index_api.cpp
+++ b/tests/unit/graph_db_accessor_index_api.cpp
@@ -23,7 +23,7 @@ auto Count(TIterable iterable) {
 class GraphDbAccessorIndex : public testing::Test {
  protected:
   database::SingleNode db;
-  std::experimental::optional<database::GraphDbAccessor> dba{db};
+  std::unique_ptr<database::GraphDbAccessor> dba{db.Access()};
   storage::Property property = dba->Property("property");
   storage::Label label = dba->Label("label");
   storage::EdgeType edge_type = dba->EdgeType("edge_type");
@@ -44,7 +44,7 @@ class GraphDbAccessorIndex : public testing::Test {
   // commits the current dba, and replaces it with a new one
   void Commit() {
     dba->Commit();
-    dba.emplace(db);
+    dba = db.Access();
   }
 };
 
@@ -142,9 +142,9 @@ TEST(GraphDbAccessorIndexApi, LabelPropertyBuildIndexConcurrent) {
     std::vector<std::thread> threads;
     for (int index = 0; index < THREAD_COUNT; ++index) {
       threads.emplace_back([&db, index]() {
-        database::GraphDbAccessor dba(db);
-        dba.BuildIndex(dba.Label("l" + std::to_string(index)),
-                       dba.Property("p" + std::to_string(index)));
+        auto dba = db.Access();
+        dba->BuildIndex(dba->Label("l" + std::to_string(index)),
+                        dba->Property("p" + std::to_string(index)));
 
       });
     }
diff --git a/tests/unit/interpreter.cpp b/tests/unit/interpreter.cpp
index 678ababcd..b342aa2ec 100644
--- a/tests/unit/interpreter.cpp
+++ b/tests/unit/interpreter.cpp
@@ -19,9 +19,9 @@ class InterpreterTest : public ::testing::Test {
 
   auto Interpret(const std::string &query,
                  const std::map<std::string, query::TypedValue> &params = {}) {
-    database::GraphDbAccessor dba(db_);
+    auto dba = db_.Access();
     ResultStreamFaker<query::TypedValue> stream;
-    auto results = interpreter_(query, dba, params, false);
+    auto results = interpreter_(query, *dba, params, false);
     stream.Header(results.header());
     results.PullAll(stream);
     stream.Summary(results.summary());
@@ -146,19 +146,19 @@ TEST_F(InterpreterTest, Bfs) {
 
   // Set up.
   {
-    database::GraphDbAccessor dba(db_);
+    auto dba = db_.Access();
     auto add_node = [&](int level, bool reachable) {
-      auto node = dba.InsertVertex();
-      node.PropsSet(dba.Property(kId), id++);
-      node.PropsSet(dba.Property(kReachable), reachable);
+      auto node = dba->InsertVertex();
+      node.PropsSet(dba->Property(kId), id++);
+      node.PropsSet(dba->Property(kReachable), reachable);
       levels[level].push_back(node);
       return node;
     };
 
     auto add_edge = [&](VertexAccessor &v1, VertexAccessor &v2,
                         bool reachable) {
-      auto edge = dba.InsertEdge(v1, v2, dba.EdgeType("edge"));
-      edge.PropsSet(dba.Property(kReachable), reachable);
+      auto edge = dba->InsertEdge(v1, v2, dba->EdgeType("edge"));
+      edge.PropsSet(dba->Property(kReachable), reachable);
     };
 
     // Add source node.
@@ -196,15 +196,15 @@ TEST_F(InterpreterTest, Bfs) {
       add_edge(node1, node2, false);
     }
 
-    dba.Commit();
+    dba->Commit();
   }
 
-  database::GraphDbAccessor dba(db_);
+  auto dba = db_.Access();
   ResultStreamFaker<query::TypedValue> stream;
   auto results = interpreter_(
       "MATCH (n {id: 0})-[r *bfs..5 (e, n | n.reachable and "
       "e.reachable)]->(m) RETURN r",
-      dba, {}, false);
+      *dba, {}, false);
   stream.Header(results.header());
   results.PullAll(stream);
   stream.Summary(results.summary());
@@ -225,14 +225,14 @@ TEST_F(InterpreterTest, Bfs) {
     EXPECT_EQ(edges.size(), expected_level);
     // Check that starting node is correct.
     EXPECT_EQ(
-        edges[0].from().PropsAt(dba.Property(kId)).template Value<int64_t>(),
+        edges[0].from().PropsAt(dba->Property(kId)).template Value<int64_t>(),
         0);
     for (int i = 1; i < static_cast<int>(edges.size()); ++i) {
       // Check that edges form a connected path.
       EXPECT_EQ(edges[i - 1].to(), edges[i].from());
     }
     auto matched_id =
-        edges.back().to().PropsAt(dba.Property(kId)).Value<int64_t>();
+        edges.back().to().PropsAt(dba->Property(kId)).Value<int64_t>();
     // Check that we didn't match that node already.
     EXPECT_TRUE(matched_ids.insert(matched_id).second);
     // Check that shortest path was found.
@@ -247,9 +247,9 @@ TEST_F(InterpreterTest, Bfs) {
 
 TEST_F(InterpreterTest, CreateIndexInMulticommandTransaction) {
   ResultStreamFaker<query::TypedValue> stream;
-  database::GraphDbAccessor dba(db_);
+  auto dba = db_.Access();
   ASSERT_THROW(
-      interpreter_("CREATE INDEX ON :X(y)", dba, {}, true).PullAll(stream),
+      interpreter_("CREATE INDEX ON :X(y)", *dba, {}, true).PullAll(stream),
       query::IndexInMulticommandTxException);
 }
 
@@ -257,20 +257,21 @@ TEST_F(InterpreterTest, CreateIndexInMulticommandTransaction) {
 TEST_F(InterpreterTest, ShortestPath) {
   {
     ResultStreamFaker<query::TypedValue> stream;
-    database::GraphDbAccessor dba(db_);
+    auto dba = db_.Access();
     interpreter_(
         "CREATE (n:A {x: 1}), (m:B {x: 2}), (l:C {x: 1}), (n)-[:r1 {w: 1 "
         "}]->(m)-[:r2 {w: 2}]->(l), (n)-[:r3 {w: 4}]->(l)",
-        dba, {}, true)
+        *dba, {}, true)
         .PullAll(stream);
 
-    dba.Commit();
+    dba->Commit();
   }
 
   ResultStreamFaker<query::TypedValue> stream;
-  database::GraphDbAccessor dba(db_);
-  auto results = interpreter_(
-      "MATCH (n)-[e *wshortest 5 (e, n | e.w) ]->(m) return e", dba, {}, false);
+  auto dba = db_.Access();
+  auto results =
+      interpreter_("MATCH (n)-[e *wshortest 5 (e, n | e.w) ]->(m) return e",
+                   *dba, {}, false);
   stream.Header(results.header());
   results.PullAll(stream);
   stream.Summary(results.summary());
@@ -288,7 +289,7 @@ TEST_F(InterpreterTest, ShortestPath) {
 
     std::vector<std::string> datum;
     for (const auto &edge : edges) {
-      datum.push_back(dba.EdgeTypeName(edge.EdgeType()));
+      datum.push_back(dba->EdgeTypeName(edge.EdgeType()));
     }
 
     bool any_match = false;
diff --git a/tests/unit/query_common.hpp b/tests/unit/query_common.hpp
index 497774a37..277dc77f5 100644
--- a/tests/unit/query_common.hpp
+++ b/tests/unit/query_common.hpp
@@ -5,9 +5,10 @@
 ///
 ///     AstStorage storage;  // Macros rely on storage being in scope.
 ///     // PROPERTY_LOOKUP and PROPERTY_PAIR macros
-///     // rely on a DbAccessor named dba.
+///     // rely on a DbAccessor *reference* named dba.
 ///     database::SingleNode db;
-///     database::GraphDbAccessor dba(db);
+///     auto dba_ptr = db.Access();
+///     auto &dba = *dba_ptr;
 ///
 ///     QUERY(MATCH(PATTERN(NODE("n"), EDGE("e"), NODE("m"))),
 ///           WHERE(LESS(PROPERTY_LOOKUP("e", edge_prop), LITERAL(3))),
diff --git a/tests/unit/query_cost_estimator.cpp b/tests/unit/query_cost_estimator.cpp
index 82b7d896f..910315b30 100644
--- a/tests/unit/query_cost_estimator.cpp
+++ b/tests/unit/query_cost_estimator.cpp
@@ -24,7 +24,7 @@ using MiscParam = CostEstimator<database::GraphDbAccessor>::MiscParam;
 class QueryCostEstimator : public ::testing::Test {
  protected:
   database::SingleNode db;
-  std::experimental::optional<database::GraphDbAccessor> dba{db};
+  std::unique_ptr<database::GraphDbAccessor> dba{db.Access()};
   storage::Label label = dba->Label("label");
   storage::Property property = dba->Property("property");
 
@@ -40,7 +40,7 @@ class QueryCostEstimator : public ::testing::Test {
   void SetUp() {
     // create the index in the current db accessor and then swap it to a new one
     dba->BuildIndex(label, property);
-    dba.emplace(db);
+    dba = db.Access();
   }
 
   Symbol NextSymbol() {
diff --git a/tests/unit/query_expression_evaluator.cpp b/tests/unit/query_expression_evaluator.cpp
index 2999df82b..efc69b08f 100644
--- a/tests/unit/query_expression_evaluator.cpp
+++ b/tests/unit/query_expression_evaluator.cpp
@@ -31,8 +31,8 @@ struct NoContextExpressionEvaluator {
   NoContextExpressionEvaluator() {}
   Frame frame{128};
   database::SingleNode db;
-  database::GraphDbAccessor dba{db};
-  Context ctx{dba};
+  std::unique_ptr<database::GraphDbAccessor> dba{db.Access()};
+  Context ctx{*dba};
   ExpressionEvaluator eval{frame, &ctx, GraphView::OLD};
 };
 
@@ -54,8 +54,8 @@ TypedValue EvaluateFunction(const std::string &function_name,
 TypedValue EvaluateFunction(const std::string &function_name,
                             const std::vector<TypedValue> &args) {
   database::SingleNode db;
-  database::GraphDbAccessor dba{db};
-  Context ctx{dba};
+  auto dba = db.Access();
+  Context ctx{*dba};
   return EvaluateFunction(function_name, args, &ctx);
 }
 
@@ -415,7 +415,8 @@ TEST(ExpressionEvaluator, MapIndexing) {
   AstStorage storage;
   NoContextExpressionEvaluator eval;
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   auto *map_literal = storage.Create<MapLiteral>(
       std::unordered_map<std::pair<std::string, storage::Property>,
                          Expression *>{
@@ -454,7 +455,7 @@ TEST(ExpressionEvaluator, MapIndexing) {
 TEST(ExpressionEvaluator, VertexAndEdgeIndexing) {
   AstStorage storage;
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
 
   auto edge_type = dba.EdgeType("edge_type");
   auto prop = dba.Property("prop");
@@ -684,7 +685,8 @@ class ExpressionEvaluatorPropertyLookup : public testing::Test {
   AstStorage storage;
   NoContextExpressionEvaluator eval;
   database::SingleNode db;
-  database::GraphDbAccessor dba{db};
+  std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()};
+  database::GraphDbAccessor &dba{*dba_ptr};
   std::pair<std::string, storage::Property> prop_age = PROPERTY_PAIR("age");
   std::pair<std::string, storage::Property> prop_height =
       PROPERTY_PAIR("height");
@@ -732,11 +734,11 @@ TEST(ExpressionEvaluator, LabelsTest) {
   AstStorage storage;
   NoContextExpressionEvaluator eval;
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  v1.add_label(dba.Label("ANIMAL"));
-  v1.add_label(dba.Label("DOG"));
-  v1.add_label(dba.Label("NICE_DOG"));
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  v1.add_label(dba->Label("ANIMAL"));
+  v1.add_label(dba->Label("DOG"));
+  v1.add_label(dba->Label("NICE_DOG"));
   auto *identifier = storage.Create<Identifier>("n");
   auto node_symbol = eval.ctx.symbol_table_.CreateSymbol("n", true);
   eval.ctx.symbol_table_[*identifier] = node_symbol;
@@ -744,15 +746,15 @@ TEST(ExpressionEvaluator, LabelsTest) {
   {
     auto *op = storage.Create<LabelsTest>(
         identifier,
-        std::vector<storage::Label>{dba.Label("DOG"), dba.Label("ANIMAL")});
+        std::vector<storage::Label>{dba->Label("DOG"), dba->Label("ANIMAL")});
     auto value = op->Accept(eval.eval);
     EXPECT_EQ(value.Value<bool>(), true);
   }
   {
     auto *op = storage.Create<LabelsTest>(
         identifier,
-        std::vector<storage::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"),
-                                    dba.Label("ANIMAL")});
+        std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"),
+                                    dba->Label("ANIMAL")});
     auto value = op->Accept(eval.eval);
     EXPECT_EQ(value.Value<bool>(), false);
   }
@@ -760,8 +762,8 @@ TEST(ExpressionEvaluator, LabelsTest) {
     eval.frame[node_symbol] = TypedValue::Null;
     auto *op = storage.Create<LabelsTest>(
         identifier,
-        std::vector<storage::Label>{dba.Label("DOG"), dba.Label("BAD_DOG"),
-                                    dba.Label("ANIMAL")});
+        std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"),
+                                    dba->Label("ANIMAL")});
     auto value = op->Accept(eval.eval);
     EXPECT_TRUE(value.IsNull());
   }
@@ -772,8 +774,8 @@ TEST(ExpressionEvaluator, Aggregation) {
   auto aggr = storage.Create<Aggregation>(storage.Create<PrimitiveLiteral>(42),
                                           nullptr, Aggregation::Op::COUNT);
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  Context ctx(dba);
+  auto dba = db.Access();
+  Context ctx(*dba);
   auto aggr_sym = ctx.symbol_table_.CreateSymbol("aggr", true);
   ctx.symbol_table_[*aggr] = aggr_sym;
   Frame frame{ctx.symbol_table_.max_position()};
@@ -815,15 +817,15 @@ TEST(ExpressionEvaluator, FunctionEndNode) {
   ASSERT_EQ(EvaluateFunction("ENDNODE", {TypedValue::Null}).type(),
             TypedValue::Type::Null);
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  v1.add_label(dba.Label("label1"));
-  auto v2 = dba.InsertVertex();
-  v2.add_label(dba.Label("label2"));
-  auto e = dba.InsertEdge(v1, v2, dba.EdgeType("t"));
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  v1.add_label(dba->Label("label1"));
+  auto v2 = dba->InsertVertex();
+  v2.add_label(dba->Label("label2"));
+  auto e = dba->InsertEdge(v1, v2, dba->EdgeType("t"));
   ASSERT_TRUE(EvaluateFunction("ENDNODE", {e})
                   .Value<VertexAccessor>()
-                  .has_label(dba.Label("label2")));
+                  .has_label(dba->Label("label2")));
   ASSERT_THROW(EvaluateFunction("ENDNODE", {2}), QueryRuntimeException);
 }
 
@@ -844,7 +846,7 @@ TEST(ExpressionEvaluator, FunctionProperties) {
   ASSERT_EQ(EvaluateFunction("PROPERTIES", {TypedValue::Null}).type(),
             TypedValue::Type::Null);
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   auto v1 = dba.InsertVertex();
   v1.PropsSet(dba.Property("height"), 5);
   v1.PropsSet(dba.Property("age"), 10);
@@ -900,12 +902,12 @@ TEST(ExpressionEvaluator, FunctionSize) {
   ASSERT_THROW(EvaluateFunction("SIZE", {5}), QueryRuntimeException);
 
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v0 = dba.InsertVertex();
+  auto dba = db.Access();
+  auto v0 = dba->InsertVertex();
   query::Path path(v0);
   EXPECT_EQ(EvaluateFunction("SIZE", {path}).ValueInt(), 0);
-  auto v1 = dba.InsertVertex();
-  path.Expand(dba.InsertEdge(v0, v1, dba.EdgeType("type")));
+  auto v1 = dba->InsertVertex();
+  path.Expand(dba->InsertEdge(v0, v1, dba->EdgeType("type")));
   path.Expand(v1);
   EXPECT_EQ(EvaluateFunction("SIZE", {path}).ValueInt(), 1);
 }
@@ -915,15 +917,15 @@ TEST(ExpressionEvaluator, FunctionStartNode) {
   ASSERT_EQ(EvaluateFunction("STARTNODE", {TypedValue::Null}).type(),
             TypedValue::Type::Null);
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  v1.add_label(dba.Label("label1"));
-  auto v2 = dba.InsertVertex();
-  v2.add_label(dba.Label("label2"));
-  auto e = dba.InsertEdge(v1, v2, dba.EdgeType("t"));
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  v1.add_label(dba->Label("label1"));
+  auto v2 = dba->InsertVertex();
+  v2.add_label(dba->Label("label2"));
+  auto e = dba->InsertEdge(v1, v2, dba->EdgeType("t"));
   ASSERT_TRUE(EvaluateFunction("STARTNODE", {e})
                   .Value<VertexAccessor>()
-                  .has_label(dba.Label("label1")));
+                  .has_label(dba->Label("label1")));
   ASSERT_THROW(EvaluateFunction("STARTNODE", {2}), QueryRuntimeException);
 }
 
@@ -932,12 +934,12 @@ TEST(ExpressionEvaluator, FunctionDegree) {
   ASSERT_EQ(EvaluateFunction("DEGREE", {TypedValue::Null}).type(),
             TypedValue::Type::Null);
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto v3 = dba.InsertVertex();
-  auto e12 = dba.InsertEdge(v1, v2, dba.EdgeType("t"));
-  dba.InsertEdge(v3, v2, dba.EdgeType("t"));
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto v3 = dba->InsertVertex();
+  auto e12 = dba->InsertEdge(v1, v2, dba->EdgeType("t"));
+  dba->InsertEdge(v3, v2, dba->EdgeType("t"));
   ASSERT_EQ(EvaluateFunction("DEGREE", {v1}).Value<int64_t>(), 1);
   ASSERT_EQ(EvaluateFunction("DEGREE", {v2}).Value<int64_t>(), 2);
   ASSERT_EQ(EvaluateFunction("DEGREE", {v3}).Value<int64_t>(), 1);
@@ -992,7 +994,7 @@ TEST(ExpressionEvaluator, FunctionType) {
   ASSERT_EQ(EvaluateFunction("TYPE", {TypedValue::Null}).type(),
             TypedValue::Type::Null);
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   auto v1 = dba.InsertVertex();
   v1.add_label(dba.Label("label1"));
   auto v2 = dba.InsertVertex();
@@ -1008,7 +1010,7 @@ TEST(ExpressionEvaluator, FunctionLabels) {
   ASSERT_EQ(EvaluateFunction("LABELS", {TypedValue::Null}).type(),
             TypedValue::Type::Null);
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   auto v = dba.InsertVertex();
   v.add_label(dba.Label("label1"));
   v.add_label(dba.Label("label2"));
@@ -1031,7 +1033,7 @@ TEST(ExpressionEvaluator, FunctionNodesRelationships) {
 
   {
     NoContextExpressionEvaluator eval;
-    auto &dba = eval.dba;
+    auto &dba = *eval.dba;
     auto v1 = dba.InsertVertex();
     auto v2 = dba.InsertVertex();
     auto v3 = dba.InsertVertex();
@@ -1089,7 +1091,7 @@ TEST(ExpressionEvaluator, FunctionKeys) {
   ASSERT_EQ(EvaluateFunction("KEYS", {TypedValue::Null}).type(),
             TypedValue::Type::Null);
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   auto v1 = dba.InsertVertex();
   v1.PropsSet(dba.Property("height"), 5);
   v1.PropsSet(dba.Property("age"), 10);
@@ -1456,7 +1458,7 @@ TEST(ExpressionEvaluator, FunctionIndexInfo) {
   EXPECT_THROW(EvaluateFunction("INDEXINFO", {1}, &eval.ctx),
                QueryRuntimeException);
   EXPECT_EQ(EvaluateFunction("INDEXINFO", {}, &eval.ctx).ValueList().size(), 0);
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   dba.InsertVertex().add_label(dba.Label("l1"));
   {
     auto info =
@@ -1475,7 +1477,7 @@ TEST(ExpressionEvaluator, FunctionIndexInfo) {
 
 TEST(ExpressionEvaluator, FunctionId) {
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   auto va = dba.InsertVertex();
   auto ea = dba.InsertEdge(va, va, dba.EdgeType("edge"));
   auto vb = dba.InsertVertex();
@@ -1491,7 +1493,7 @@ TEST(ExpressionEvaluator, FunctionId) {
 TEST(ExpressionEvaluator, FunctionWorkerIdException) {
   database::SingleNode db;
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   auto va = dba.InsertVertex();
   EXPECT_THROW(EvaluateFunction("WORKERID", {}, &eval.ctx),
                QueryRuntimeException);
@@ -1501,10 +1503,9 @@ TEST(ExpressionEvaluator, FunctionWorkerIdException) {
 
 TEST(ExpressionEvaluator, FunctionWorkerIdSingleNode) {
   NoContextExpressionEvaluator eval;
-  auto &dba = eval.dba;
+  auto &dba = *eval.dba;
   auto va = dba.InsertVertex();
-  EXPECT_EQ(EvaluateFunction("WORKERID", {va}, &eval.ctx).Value<int64_t>(),
-            eval.db.WorkerId());
+  EXPECT_EQ(EvaluateFunction("WORKERID", {va}, &eval.ctx).Value<int64_t>(), 0);
 }
 
 TEST(ExpressionEvaluator, FunctionToStringNull) {
diff --git a/tests/unit/query_plan_accumulate_aggregate.cpp b/tests/unit/query_plan_accumulate_aggregate.cpp
index 36caaae7a..f3158ab12 100644
--- a/tests/unit/query_plan_accumulate_aggregate.cpp
+++ b/tests/unit/query_plan_accumulate_aggregate.cpp
@@ -28,7 +28,8 @@ TEST(QueryPlan, Accumulate) {
 
   auto check = [&](bool accumulate) {
     database::SingleNode db;
-    database::GraphDbAccessor dba(db);
+    auto dba_ptr = db.Access();
+    auto &dba = *dba_ptr;
     auto prop = dba.Property("x");
 
     auto v1 = dba.InsertVertex();
@@ -88,7 +89,7 @@ TEST(QueryPlan, AccumulateAdvance) {
 
   auto check = [&](bool advance) {
     database::SingleNode db;
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     AstStorage storage;
     SymbolTable symbol_table;
 
@@ -99,7 +100,7 @@ TEST(QueryPlan, AccumulateAdvance) {
     auto accumulate = std::make_shared<Accumulate>(
         create, std::vector<Symbol>{sym_n}, advance);
     auto match = MakeScanAll(storage, symbol_table, "m", accumulate);
-    EXPECT_EQ(advance ? 1 : 0, PullAll(match.op_, dba, symbol_table));
+    EXPECT_EQ(advance ? 1 : 0, PullAll(match.op_, *dba, symbol_table));
   };
   check(false);
   check(true);
@@ -150,7 +151,8 @@ std::shared_ptr<Produce> MakeAggregationProduce(
 class QueryPlanAggregateOps : public ::testing::Test {
  protected:
   database::SingleNode db;
-  database::GraphDbAccessor dba{db};
+  std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()};
+  database::GraphDbAccessor &dba{*dba_ptr};
   storage::Property prop = dba.Property("prop");
 
   AstStorage storage;
@@ -289,7 +291,8 @@ TEST(QueryPlan, AggregateGroupByValues) {
   // Also test the "remember" part of the Aggregation API as final results are
   // obtained via a property lookup of a remembered node.
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   // a vector of TypedValue to be set as property values on vertices
   // most of them should result in a distinct group (commented where not)
@@ -348,7 +351,8 @@ TEST(QueryPlan, AggregateMultipleGroupBy) {
   // for different records and assert that we get the correct combination
   // of values in our groups
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   auto prop1 = dba.Property("prop1");
   auto prop2 = dba.Property("prop2");
@@ -383,7 +387,7 @@ TEST(QueryPlan, AggregateMultipleGroupBy) {
 
 TEST(QueryPlan, AggregateNoInput) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   AstStorage storage;
   SymbolTable symbol_table;
 
@@ -393,7 +397,7 @@ TEST(QueryPlan, AggregateNoInput) {
 
   auto produce = MakeAggregationProduce(nullptr, symbol_table, storage, {two},
                                         {Aggregation::Op::COUNT}, {}, {});
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(1, results.size());
   EXPECT_EQ(1, results[0].size());
   EXPECT_EQ(TypedValue::Type::Int, results[0][0].type());
@@ -410,7 +414,8 @@ TEST(QueryPlan, AggregateCountEdgeCases) {
   //  - 2 vertices in database, property set on both
 
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   auto prop = dba.Property("prop");
 
   AstStorage storage;
@@ -462,7 +467,8 @@ TEST(QueryPlan, AggregateFirstValueTypes) {
   // type check
 
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   auto v1 = dba.InsertVertex();
   auto prop_string = dba.Property("string");
@@ -519,7 +525,8 @@ TEST(QueryPlan, AggregateTypes) {
   // (that logic is defined and tested by TypedValue)
 
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   auto p1 = dba.Property("p1");  // has only string props
   dba.InsertVertex().PropsSet(p1, "string");
@@ -575,7 +582,7 @@ TEST(QueryPlan, AggregateTypes) {
 
 TEST(QueryPlan, Unwind) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   AstStorage storage;
   SymbolTable symbol_table;
 
@@ -598,7 +605,7 @@ TEST(QueryPlan, Unwind) {
   symbol_table[*y_ne] = symbol_table.CreateSymbol("y_ne", true);
   auto produce = MakeProduce(unwind_1, x_ne, y_ne);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   ASSERT_EQ(4, results.size());
   const std::vector<int> expected_x_card{3, 3, 3, 1};
   auto expected_x_card_it = expected_x_card.begin();
diff --git a/tests/unit/query_plan_bag_semantics.cpp b/tests/unit/query_plan_bag_semantics.cpp
index bf0082e67..78e354179 100644
--- a/tests/unit/query_plan_bag_semantics.cpp
+++ b/tests/unit/query_plan_bag_semantics.cpp
@@ -23,7 +23,7 @@ using namespace query::plan;
 
 TEST(QueryPlan, Skip) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -31,28 +31,28 @@ TEST(QueryPlan, Skip) {
   auto n = MakeScanAll(storage, symbol_table, "n1");
   auto skip = std::make_shared<plan::Skip>(n.op_, LITERAL(2));
 
-  EXPECT_EQ(0, PullAll(skip, dba, symbol_table));
+  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
 
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(0, PullAll(skip, dba, symbol_table));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
 
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(0, PullAll(skip, dba, symbol_table));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
 
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(1, PullAll(skip, dba, symbol_table));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
 
-  for (int i = 0; i < 10; ++i) dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(11, PullAll(skip, dba, symbol_table));
+  for (int i = 0; i < 10; ++i) dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(11, PullAll(skip, *dba, symbol_table));
 }
 
 TEST(QueryPlan, Limit) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -60,23 +60,23 @@ TEST(QueryPlan, Limit) {
   auto n = MakeScanAll(storage, symbol_table, "n1");
   auto skip = std::make_shared<plan::Limit>(n.op_, LITERAL(2));
 
-  EXPECT_EQ(0, PullAll(skip, dba, symbol_table));
+  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
 
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(1, PullAll(skip, dba, symbol_table));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
 
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(2, PullAll(skip, dba, symbol_table));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
 
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(2, PullAll(skip, dba, symbol_table));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
 
-  for (int i = 0; i < 10; ++i) dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(2, PullAll(skip, dba, symbol_table));
+  for (int i = 0; i < 10; ++i) dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
 }
 
 TEST(QueryPlan, CreateLimit) {
@@ -84,10 +84,10 @@ TEST(QueryPlan, CreateLimit) {
   // MATCH (n) CREATE (m) LIMIT 1
   // in the end we need to have 3 vertices in the db
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  dba.InsertVertex();
-  dba.InsertVertex();
-  dba.AdvanceCommand();
+  auto dba = db.Access();
+  dba->InsertVertex();
+  dba->InsertVertex();
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -98,14 +98,15 @@ TEST(QueryPlan, CreateLimit) {
   auto c = std::make_shared<CreateNode>(n.op_, m, false);
   auto skip = std::make_shared<plan::Limit>(c, LITERAL(1));
 
-  EXPECT_EQ(1, PullAll(skip, dba, symbol_table));
-  dba.AdvanceCommand();
-  EXPECT_EQ(3, CountIterable(dba.Vertices(false)));
+  EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
+  dba->AdvanceCommand();
+  EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
 }
 
 TEST(QueryPlan, OrderBy) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   AstStorage storage;
   SymbolTable symbol_table;
   auto prop = dba.Property("prop");
@@ -166,7 +167,8 @@ TEST(QueryPlan, OrderBy) {
 
 TEST(QueryPlan, OrderByMultiple) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   AstStorage storage;
   SymbolTable symbol_table;
 
@@ -202,7 +204,8 @@ TEST(QueryPlan, OrderByMultiple) {
   auto order_by = std::make_shared<plan::OrderBy>(
       n.op_,
       std::vector<std::pair<Ordering, Expression *>>{
-          {Ordering::ASC, n_p1}, {Ordering::DESC, n_p2},
+          {Ordering::ASC, n_p1},
+          {Ordering::DESC, n_p2},
       },
       std::vector<Symbol>{n.sym_});
   auto n_p1_ne = NEXPR("n.p1", n_p1);
@@ -222,7 +225,8 @@ TEST(QueryPlan, OrderByMultiple) {
 
 TEST(QueryPlan, OrderByExceptions) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   AstStorage storage;
   SymbolTable symbol_table;
   auto prop = dba.Property("prop");
diff --git a/tests/unit/query_plan_create_set_remove_delete.cpp b/tests/unit/query_plan_create_set_remove_delete.cpp
index 619503258..206753df1 100644
--- a/tests/unit/query_plan_create_set_remove_delete.cpp
+++ b/tests/unit/query_plan_create_set_remove_delete.cpp
@@ -18,7 +18,8 @@ using namespace query::plan;
 
 TEST(QueryPlan, CreateNodeWithAttributes) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   storage::Label label = dba.Label("Person");
   auto property = PROPERTY_PAIR("prop");
@@ -52,7 +53,8 @@ TEST(QueryPlan, CreateNodeWithAttributes) {
 TEST(QueryPlan, CreateReturn) {
   // test CREATE (n:Person {age: 42}) RETURN n, n.age
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   storage::Label label = dba.Label("Person");
   auto property = PROPERTY_PAIR("property");
@@ -93,7 +95,8 @@ TEST(QueryPlan, CreateReturn) {
 
 TEST(QueryPlan, CreateExpand) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   storage::Label label_node_1 = dba.Label("Node1");
   storage::Label label_node_2 = dba.Label("Node2");
@@ -167,13 +170,13 @@ TEST(QueryPlan, CreateExpand) {
 
 TEST(QueryPlan, MatchCreateNode) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // add three nodes we'll match and expand-create from
-  dba.InsertVertex();
-  dba.InsertVertex();
-  dba.InsertVertex();
-  dba.AdvanceCommand();
+  dba->InsertVertex();
+  dba->InsertVertex();
+  dba->InsertVertex();
+  dba->AdvanceCommand();
 
   SymbolTable symbol_table;
   AstStorage storage;
@@ -186,34 +189,34 @@ TEST(QueryPlan, MatchCreateNode) {
   // creation op
   auto create_node = std::make_shared<CreateNode>(n_scan_all.op_, m, false);
 
-  EXPECT_EQ(CountIterable(dba.Vertices(false)), 3);
-  PullAll(create_node, dba, symbol_table);
-  dba.AdvanceCommand();
-  EXPECT_EQ(CountIterable(dba.Vertices(false)), 6);
+  EXPECT_EQ(CountIterable(dba->Vertices(false)), 3);
+  PullAll(create_node, *dba, symbol_table);
+  dba->AdvanceCommand();
+  EXPECT_EQ(CountIterable(dba->Vertices(false)), 6);
 }
 
 TEST(QueryPlan, MatchCreateExpand) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // add three nodes we'll match and expand-create from
-  dba.InsertVertex();
-  dba.InsertVertex();
-  dba.InsertVertex();
-  dba.AdvanceCommand();
+  dba->InsertVertex();
+  dba->InsertVertex();
+  dba->InsertVertex();
+  dba->AdvanceCommand();
 
-  //  storage::Label label_node_1 = dba.Label("Node1");
-  //  storage::Label label_node_2 = dba.Label("Node2");
-  //  storage::Property property = dba.Label("prop");
-  storage::EdgeType edge_type = dba.EdgeType("edge_type");
+  //  storage::Label label_node_1 = dba->Label("Node1");
+  //  storage::Label label_node_2 = dba->Label("Node2");
+  //  storage::Property property = dba->Label("prop");
+  storage::EdgeType edge_type = dba->EdgeType("edge_type");
 
   SymbolTable symbol_table;
   AstStorage storage;
 
   auto test_create_path = [&](bool cycle, int expected_nodes_created,
                               int expected_edges_created) {
-    int before_v = CountIterable(dba.Vertices(false));
-    int before_e = CountIterable(dba.Edges(false));
+    int before_v = CountIterable(dba->Vertices(false));
+    int before_e = CountIterable(dba->Edges(false));
 
     // data for the first node
     auto n_scan_all = MakeScanAll(storage, symbol_table, "n");
@@ -231,12 +234,12 @@ TEST(QueryPlan, MatchCreateExpand) {
 
     auto create_expand = std::make_shared<CreateExpand>(m, r, n_scan_all.op_,
                                                         n_scan_all.sym_, cycle);
-    PullAll(create_expand, dba, symbol_table);
-    dba.AdvanceCommand();
+    PullAll(create_expand, *dba, symbol_table);
+    dba->AdvanceCommand();
 
-    EXPECT_EQ(CountIterable(dba.Vertices(false)) - before_v,
+    EXPECT_EQ(CountIterable(dba->Vertices(false)) - before_v,
               expected_nodes_created);
-    EXPECT_EQ(CountIterable(dba.Edges(false)) - before_e,
+    EXPECT_EQ(CountIterable(dba->Edges(false)) - before_e,
               expected_edges_created);
   };
 
@@ -246,19 +249,19 @@ TEST(QueryPlan, MatchCreateExpand) {
 
 TEST(QueryPlan, Delete) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // make a fully-connected (one-direction, no cycles) with 4 nodes
   std::vector<VertexAccessor> vertices;
-  for (int i = 0; i < 4; ++i) vertices.push_back(dba.InsertVertex());
-  auto type = dba.EdgeType("type");
+  for (int i = 0; i < 4; ++i) vertices.push_back(dba->InsertVertex());
+  auto type = dba->EdgeType("type");
   for (int j = 0; j < 4; ++j)
     for (int k = j + 1; k < 4; ++k)
-      dba.InsertEdge(vertices[j], vertices[k], type);
+      dba->InsertEdge(vertices[j], vertices[k], type);
 
-  dba.AdvanceCommand();
-  EXPECT_EQ(4, CountIterable(dba.Vertices(false)));
-  EXPECT_EQ(6, CountIterable(dba.Edges(false)));
+  dba->AdvanceCommand();
+  EXPECT_EQ(4, CountIterable(dba->Vertices(false)));
+  EXPECT_EQ(6, CountIterable(dba->Edges(false)));
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -270,10 +273,10 @@ TEST(QueryPlan, Delete) {
     symbol_table[*n_get] = n.sym_;
     auto delete_op = std::make_shared<plan::Delete>(
         n.op_, std::vector<Expression *>{n_get}, false);
-    EXPECT_THROW(PullAll(delete_op, dba, symbol_table), QueryRuntimeException);
-    dba.AdvanceCommand();
-    EXPECT_EQ(4, CountIterable(dba.Vertices(false)));
-    EXPECT_EQ(6, CountIterable(dba.Edges(false)));
+    EXPECT_THROW(PullAll(delete_op, *dba, symbol_table), QueryRuntimeException);
+    dba->AdvanceCommand();
+    EXPECT_EQ(4, CountIterable(dba->Vertices(false)));
+    EXPECT_EQ(6, CountIterable(dba->Edges(false)));
   }
 
   // detach delete a single vertex
@@ -284,12 +287,12 @@ TEST(QueryPlan, Delete) {
     auto delete_op = std::make_shared<plan::Delete>(
         n.op_, std::vector<Expression *>{n_get}, true);
     Frame frame(symbol_table.max_position());
-    Context context(dba);
+    Context context(*dba);
     context.symbol_table_ = symbol_table;
-    delete_op->MakeCursor(dba)->Pull(frame, context);
-    dba.AdvanceCommand();
-    EXPECT_EQ(3, CountIterable(dba.Vertices(false)));
-    EXPECT_EQ(3, CountIterable(dba.Edges(false)));
+    delete_op->MakeCursor(*dba)->Pull(frame, context);
+    dba->AdvanceCommand();
+    EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
+    EXPECT_EQ(3, CountIterable(dba->Edges(false)));
   }
 
   // delete all remaining edges
@@ -302,10 +305,10 @@ TEST(QueryPlan, Delete) {
     symbol_table[*r_get] = r_m.edge_sym_;
     auto delete_op = std::make_shared<plan::Delete>(
         r_m.op_, std::vector<Expression *>{r_get}, false);
-    PullAll(delete_op, dba, symbol_table);
-    dba.AdvanceCommand();
-    EXPECT_EQ(3, CountIterable(dba.Vertices(false)));
-    EXPECT_EQ(0, CountIterable(dba.Edges(false)));
+    PullAll(delete_op, *dba, symbol_table);
+    dba->AdvanceCommand();
+    EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
+    EXPECT_EQ(0, CountIterable(dba->Edges(false)));
   }
 
   // delete all remaining vertices
@@ -315,10 +318,10 @@ TEST(QueryPlan, Delete) {
     symbol_table[*n_get] = n.sym_;
     auto delete_op = std::make_shared<plan::Delete>(
         n.op_, std::vector<Expression *>{n_get}, false);
-    PullAll(delete_op, dba, symbol_table);
-    dba.AdvanceCommand();
-    EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
-    EXPECT_EQ(0, CountIterable(dba.Edges(false)));
+    PullAll(delete_op, *dba, symbol_table);
+    dba->AdvanceCommand();
+    EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
+    EXPECT_EQ(0, CountIterable(dba->Edges(false)));
   }
 }
 
@@ -336,14 +339,14 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
 
   auto test_delete = [](bool detach) {
     database::SingleNode db;
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
 
-    auto v1 = dba.InsertVertex();
-    auto v2 = dba.InsertVertex();
-    dba.InsertEdge(v1, v2, dba.EdgeType("T"));
-    dba.AdvanceCommand();
-    EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
-    EXPECT_EQ(1, CountIterable(dba.Edges(false)));
+    auto v1 = dba->InsertVertex();
+    auto v2 = dba->InsertVertex();
+    dba->InsertEdge(v1, v2, dba->EdgeType("T"));
+    dba->AdvanceCommand();
+    EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
+    EXPECT_EQ(1, CountIterable(dba->Edges(false)));
 
     AstStorage storage;
     SymbolTable symbol_table;
@@ -363,10 +366,10 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
 
     auto delete_op = std::make_shared<plan::Delete>(
         r_m.op_, std::vector<Expression *>{n_get, r_get, m_get}, detach);
-    EXPECT_EQ(2, PullAll(delete_op, dba, symbol_table));
-    dba.AdvanceCommand();
-    EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
-    EXPECT_EQ(0, CountIterable(dba.Edges(false)));
+    EXPECT_EQ(2, PullAll(delete_op, *dba, symbol_table));
+    dba->AdvanceCommand();
+    EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
+    EXPECT_EQ(0, CountIterable(dba->Edges(false)));
   };
 
   test_delete(true);
@@ -375,7 +378,8 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
 
 TEST(QueryPlan, DeleteReturn) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   // make a fully-connected (one-direction, no cycles) with 4 nodes
   auto prop = PROPERTY_PAIR("property");
@@ -414,14 +418,14 @@ TEST(QueryPlan, DeleteReturn) {
 TEST(QueryPlan, DeleteNull) {
   // test (simplified) WITH Null as x delete x
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   AstStorage storage;
   SymbolTable symbol_table;
 
   auto once = std::make_shared<Once>();
   auto delete_op = std::make_shared<plan::Delete>(
       once, std::vector<Expression *>{LITERAL(TypedValue::Null)}, false);
-  EXPECT_EQ(1, PullAll(delete_op, dba, symbol_table));
+  EXPECT_EQ(1, PullAll(delete_op, *dba, symbol_table));
 }
 
 TEST(QueryPlan, DeleteAdvance) {
@@ -435,9 +439,9 @@ TEST(QueryPlan, DeleteAdvance) {
   // record is not used in subsequent clauses, but
   // we are not yet compatible with that
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  dba.InsertVertex();
-  dba.AdvanceCommand();
+  auto dba = db.Access();
+  dba->InsertVertex();
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -449,12 +453,13 @@ TEST(QueryPlan, DeleteAdvance) {
       n.op_, std::vector<Expression *>{n_get}, false);
   auto advance = std::make_shared<Accumulate>(
       delete_op, std::vector<Symbol>{n.sym_}, true);
-  EXPECT_THROW(PullAll(advance, dba, symbol_table), ReconstructionException);
+  EXPECT_THROW(PullAll(advance, *dba, symbol_table), ReconstructionException);
 }
 
 TEST(QueryPlan, SetProperty) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   // graph with 4 vertices in connected pairs
   // the origin vertex in each par and both edges
@@ -506,19 +511,19 @@ TEST(QueryPlan, SetProperty) {
 TEST(QueryPlan, SetProperties) {
   auto test_set_properties = [](bool update) {
     database::SingleNode db;
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
 
     // graph: ({a: 0})-[:R {b:1}]->({c:2})
-    auto prop_a = dba.Property("a");
-    auto prop_b = dba.Property("b");
-    auto prop_c = dba.Property("c");
-    auto v1 = dba.InsertVertex();
-    auto v2 = dba.InsertVertex();
-    auto e = dba.InsertEdge(v1, v2, dba.EdgeType("R"));
+    auto prop_a = dba->Property("a");
+    auto prop_b = dba->Property("b");
+    auto prop_c = dba->Property("c");
+    auto v1 = dba->InsertVertex();
+    auto v2 = dba->InsertVertex();
+    auto e = dba->InsertEdge(v1, v2, dba->EdgeType("R"));
     v1.PropsSet(prop_a, 0);
     e.PropsSet(prop_b, 1);
     v2.PropsSet(prop_c, 2);
-    dba.AdvanceCommand();
+    dba->AdvanceCommand();
 
     AstStorage storage;
     SymbolTable symbol_table;
@@ -541,11 +546,11 @@ TEST(QueryPlan, SetProperties) {
         std::make_shared<plan::SetProperties>(r_m.op_, n.sym_, r_ident, op);
     auto set_m_to_r = std::make_shared<plan::SetProperties>(
         set_r_to_n, r_m.edge_sym_, m_ident, op);
-    EXPECT_EQ(1, PullAll(set_m_to_r, dba, symbol_table));
-    dba.AdvanceCommand();
+    EXPECT_EQ(1, PullAll(set_m_to_r, *dba, symbol_table));
+    dba->AdvanceCommand();
 
-    EXPECT_EQ(CountIterable(dba.Edges(false)), 1);
-    for (EdgeAccessor edge : dba.Edges(false)) {
+    EXPECT_EQ(CountIterable(dba->Edges(false)), 1);
+    for (EdgeAccessor edge : dba->Edges(false)) {
       VertexAccessor from = edge.from();
       EXPECT_EQ(from.Properties().size(), update ? 2 : 1);
       if (update) {
@@ -576,14 +581,14 @@ TEST(QueryPlan, SetProperties) {
 
 TEST(QueryPlan, SetLabels) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto label1 = dba.Label("label1");
-  auto label2 = dba.Label("label2");
-  auto label3 = dba.Label("label3");
-  dba.InsertVertex().add_label(label1);
-  dba.InsertVertex().add_label(label1);
-  dba.AdvanceCommand();
+  auto label1 = dba->Label("label1");
+  auto label2 = dba->Label("label2");
+  auto label3 = dba->Label("label3");
+  dba->InsertVertex().add_label(label1);
+  dba->InsertVertex().add_label(label1);
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -591,9 +596,9 @@ TEST(QueryPlan, SetLabels) {
   auto n = MakeScanAll(storage, symbol_table, "n");
   auto label_set = std::make_shared<plan::SetLabels>(
       n.op_, n.sym_, std::vector<storage::Label>{label2, label3});
-  EXPECT_EQ(2, PullAll(label_set, dba, symbol_table));
+  EXPECT_EQ(2, PullAll(label_set, *dba, symbol_table));
 
-  for (VertexAccessor vertex : dba.Vertices(false)) {
+  for (VertexAccessor vertex : dba->Vertices(false)) {
     vertex.SwitchNew();
     EXPECT_EQ(3, vertex.labels().size());
     EXPECT_TRUE(vertex.has_label(label2));
@@ -603,7 +608,8 @@ TEST(QueryPlan, SetLabels) {
 
 TEST(QueryPlan, RemoveProperty) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   // graph with 4 vertices in connected pairs
   // the origin vertex in each par and both edges
@@ -656,19 +662,19 @@ TEST(QueryPlan, RemoveProperty) {
 
 TEST(QueryPlan, RemoveLabels) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto label1 = dba.Label("label1");
-  auto label2 = dba.Label("label2");
-  auto label3 = dba.Label("label3");
-  auto v1 = dba.InsertVertex();
+  auto label1 = dba->Label("label1");
+  auto label2 = dba->Label("label2");
+  auto label3 = dba->Label("label3");
+  auto v1 = dba->InsertVertex();
   v1.add_label(label1);
   v1.add_label(label2);
   v1.add_label(label3);
-  auto v2 = dba.InsertVertex();
+  auto v2 = dba->InsertVertex();
   v2.add_label(label1);
   v2.add_label(label3);
-  dba.AdvanceCommand();
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -676,9 +682,9 @@ TEST(QueryPlan, RemoveLabels) {
   auto n = MakeScanAll(storage, symbol_table, "n");
   auto label_remove = std::make_shared<plan::RemoveLabels>(
       n.op_, n.sym_, std::vector<storage::Label>{label1, label2});
-  EXPECT_EQ(2, PullAll(label_remove, dba, symbol_table));
+  EXPECT_EQ(2, PullAll(label_remove, *dba, symbol_table));
 
-  for (VertexAccessor vertex : dba.Vertices(false)) {
+  for (VertexAccessor vertex : dba->Vertices(false)) {
     vertex.SwitchNew();
     EXPECT_EQ(1, vertex.labels().size());
     EXPECT_FALSE(vertex.has_label(label1));
@@ -688,7 +694,8 @@ TEST(QueryPlan, RemoveLabels) {
 
 TEST(QueryPlan, NodeFilterSet) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   // Create a graph such that (v1 {prop: 42}) is connected to v2 and v3.
   auto v1 = dba.InsertVertex();
   auto prop = PROPERTY_PAIR("property");
@@ -729,7 +736,8 @@ TEST(QueryPlan, NodeFilterSet) {
 
 TEST(QueryPlan, FilterRemove) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   // Create a graph such that (v1 {prop: 42}) is connected to v2 and v3.
   auto v1 = dba.InsertVertex();
   auto prop = PROPERTY_PAIR("property");
@@ -766,11 +774,11 @@ TEST(QueryPlan, FilterRemove) {
 
 TEST(QueryPlan, SetRemove) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v = dba.InsertVertex();
-  auto label1 = dba.Label("label1");
-  auto label2 = dba.Label("label2");
-  dba.AdvanceCommand();
+  auto dba = db.Access();
+  auto v = dba->InsertVertex();
+  auto label1 = dba->Label("label1");
+  auto label2 = dba->Label("label2");
+  dba->AdvanceCommand();
   // Create operations which match (v) and set and remove v :label.
   // The expected result is single (v) as it was at the start.
   AstStorage storage;
@@ -781,8 +789,8 @@ TEST(QueryPlan, SetRemove) {
       scan_all.op_, scan_all.sym_, std::vector<storage::Label>{label1, label2});
   auto rem = std::make_shared<plan::RemoveLabels>(
       set, scan_all.sym_, std::vector<storage::Label>{label1, label2});
-  EXPECT_EQ(1, PullAll(rem, dba, symbol_table));
-  dba.AdvanceCommand();
+  EXPECT_EQ(1, PullAll(rem, *dba, symbol_table));
+  dba->AdvanceCommand();
   v.Reconstruct();
   EXPECT_FALSE(v.has_label(label1));
   EXPECT_FALSE(v.has_label(label2));
@@ -796,7 +804,8 @@ TEST(QueryPlan, Merge) {
   //    and sets some property (for result validation)
   //  - merge_create branch just sets some other property
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   auto v1 = dba.InsertVertex();
   auto v2 = dba.InsertVertex();
   dba.InsertEdge(v1, v2, dba.EdgeType("Type"));
@@ -842,7 +851,7 @@ TEST(QueryPlan, MergeNoInput) {
   // merge with no input, creates a single node
 
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   AstStorage storage;
   SymbolTable symbol_table;
 
@@ -852,16 +861,17 @@ TEST(QueryPlan, MergeNoInput) {
   auto create = std::make_shared<CreateNode>(nullptr, node, false);
   auto merge = std::make_shared<plan::Merge>(nullptr, create, create);
 
-  EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
-  EXPECT_EQ(1, PullAll(merge, dba, symbol_table));
-  dba.AdvanceCommand();
-  EXPECT_EQ(1, CountIterable(dba.Vertices(false)));
+  EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
+  EXPECT_EQ(1, PullAll(merge, *dba, symbol_table));
+  dba->AdvanceCommand();
+  EXPECT_EQ(1, CountIterable(dba->Vertices(false)));
 }
 
 TEST(QueryPlan, SetPropertyOnNull) {
   // SET (Null).prop = 42
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   AstStorage storage;
   SymbolTable symbol_table;
   auto prop = PROPERTY_PAIR("property");
@@ -876,7 +886,7 @@ TEST(QueryPlan, SetPropertyOnNull) {
 TEST(QueryPlan, SetPropertiesOnNull) {
   // OPTIONAL MATCH (n) SET n = n
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   AstStorage storage;
   SymbolTable symbol_table;
   auto n = MakeScanAll(storage, symbol_table, "n");
@@ -886,15 +896,15 @@ TEST(QueryPlan, SetPropertiesOnNull) {
                                                    std::vector<Symbol>{n.sym_});
   auto set_op = std::make_shared<plan::SetProperties>(
       optional, n.sym_, n_ident, plan::SetProperties::Op::REPLACE);
-  EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
-  EXPECT_EQ(1, PullAll(set_op, dba, symbol_table));
+  EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
+  EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table));
 }
 
 TEST(QueryPlan, SetLabelsOnNull) {
   // OPTIONAL MATCH (n) SET n :label
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto label = dba.Label("label");
+  auto dba = db.Access();
+  auto label = dba->Label("label");
   AstStorage storage;
   SymbolTable symbol_table;
   auto n = MakeScanAll(storage, symbol_table, "n");
@@ -904,14 +914,15 @@ TEST(QueryPlan, SetLabelsOnNull) {
                                                    std::vector<Symbol>{n.sym_});
   auto set_op = std::make_shared<plan::SetLabels>(
       optional, n.sym_, std::vector<storage::Label>{label});
-  EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
-  EXPECT_EQ(1, PullAll(set_op, dba, symbol_table));
+  EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
+  EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table));
 }
 
 TEST(QueryPlan, RemovePropertyOnNull) {
   // REMOVE (Null).prop
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   AstStorage storage;
   SymbolTable symbol_table;
   auto prop = PROPERTY_PAIR("property");
@@ -925,8 +936,8 @@ TEST(QueryPlan, RemovePropertyOnNull) {
 TEST(QueryPlan, RemoveLabelsOnNull) {
   // OPTIONAL MATCH (n) REMOVE n :label
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto label = dba.Label("label");
+  auto dba = db.Access();
+  auto label = dba->Label("label");
   AstStorage storage;
   SymbolTable symbol_table;
   auto n = MakeScanAll(storage, symbol_table, "n");
@@ -936,26 +947,27 @@ TEST(QueryPlan, RemoveLabelsOnNull) {
                                                    std::vector<Symbol>{n.sym_});
   auto remove_op = std::make_shared<plan::RemoveLabels>(
       optional, n.sym_, std::vector<storage::Label>{label});
-  EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
-  EXPECT_EQ(1, PullAll(remove_op, dba, symbol_table));
+  EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
+  EXPECT_EQ(1, PullAll(remove_op, *dba, symbol_table));
 }
 
 TEST(QueryPlan, CreateIndex) {
   // CREATE INDEX ON :Label(property)
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto label = dba.Label("label");
-  auto property = dba.Property("property");
-  EXPECT_FALSE(dba.LabelPropertyIndexExists(label, property));
+  auto dba = db.Access();
+  auto label = dba->Label("label");
+  auto property = dba->Property("property");
+  EXPECT_FALSE(dba->LabelPropertyIndexExists(label, property));
   auto create_index = std::make_shared<plan::CreateIndex>(label, property);
   SymbolTable symbol_table;
-  EXPECT_EQ(PullAll(create_index, dba, symbol_table), 1);
-  EXPECT_TRUE(dba.LabelPropertyIndexExists(label, property));
+  EXPECT_EQ(PullAll(create_index, *dba, symbol_table), 1);
+  EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property));
 }
 
 TEST(QueryPlan, DeleteSetProperty) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   // Add a single vertex.
   dba.InsertVertex();
   dba.AdvanceCommand();
@@ -978,7 +990,8 @@ TEST(QueryPlan, DeleteSetProperty) {
 
 TEST(QueryPlan, DeleteSetPropertiesFromMap) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   // Add a single vertex.
   dba.InsertVertex();
   dba.AdvanceCommand();
@@ -1009,7 +1022,8 @@ TEST(QueryPlan, DeleteSetPropertiesFromMap) {
 
 TEST(QueryPlan, DeleteSetPropertiesFromVertex) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   // Add a single vertex.
   {
     auto v = dba.InsertVertex();
@@ -1040,11 +1054,11 @@ TEST(QueryPlan, DeleteSetPropertiesFromVertex) {
 
 TEST(QueryPlan, DeleteRemoveLabels) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Add a single vertex.
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(1, CountIterable(dba.Vertices(false)));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(1, CountIterable(dba->Vertices(false)));
   AstStorage storage;
   SymbolTable symbol_table;
   // MATCH (n) DELETE n REMOVE n :label
@@ -1053,14 +1067,15 @@ TEST(QueryPlan, DeleteRemoveLabels) {
   symbol_table[*n_get] = n.sym_;
   auto delete_op = std::make_shared<plan::Delete>(
       n.op_, std::vector<Expression *>{n_get}, false);
-  std::vector<storage::Label> labels{dba.Label("label")};
+  std::vector<storage::Label> labels{dba->Label("label")};
   auto rem_op = std::make_shared<plan::RemoveLabels>(delete_op, n.sym_, labels);
-  EXPECT_THROW(PullAll(rem_op, dba, symbol_table), QueryRuntimeException);
+  EXPECT_THROW(PullAll(rem_op, *dba, symbol_table), QueryRuntimeException);
 }
 
 TEST(QueryPlan, DeleteRemoveProperty) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   // Add a single vertex.
   dba.InsertVertex();
   dba.AdvanceCommand();
diff --git a/tests/unit/query_plan_edge_cases.cpp b/tests/unit/query_plan_edge_cases.cpp
index b0b964a78..41c1aa9da 100644
--- a/tests/unit/query_plan_edge_cases.cpp
+++ b/tests/unit/query_plan_edge_cases.cpp
@@ -17,15 +17,15 @@ DECLARE_bool(query_cost_planner);
 class QueryExecution : public testing::Test {
  protected:
   std::experimental::optional<database::SingleNode> db_;
-  std::experimental::optional<database::GraphDbAccessor> dba_;
+  std::unique_ptr<database::GraphDbAccessor> dba_;
 
   void SetUp() {
     db_.emplace();
-    dba_.emplace(*db_);
+    dba_ = db_->Access();
   }
 
   void TearDown() {
-    dba_ = std::experimental::nullopt;
+    dba_ = nullptr;
     db_ = std::experimental::nullopt;
   }
 
@@ -33,7 +33,7 @@ class QueryExecution : public testing::Test {
    * variable to hold a new accessor with a new transaction */
   void Commit() {
     dba_->Commit();
-    dba_.emplace(*db_);
+    dba_ = db_->Access();
   }
 
   /** Executes the query and returns the results.
diff --git a/tests/unit/query_plan_match_filter_return.cpp b/tests/unit/query_plan_match_filter_return.cpp
index bac579336..5d3e80bd0 100644
--- a/tests/unit/query_plan_match_filter_return.cpp
+++ b/tests/unit/query_plan_match_filter_return.cpp
@@ -28,18 +28,18 @@ using namespace query::plan;
 class MatchReturnFixture : public testing::Test {
  protected:
   database::SingleNode db_;
-  database::GraphDbAccessor dba_{db_};
+  std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()};
   AstStorage storage;
   SymbolTable symbol_table;
 
   void AddVertices(int count) {
-    for (int i = 0; i < count; i++) dba_.InsertVertex();
+    for (int i = 0; i < count; i++) dba_->InsertVertex();
   }
 
   template <typename TResult>
   std::vector<TResult> Results(std::shared_ptr<Produce> &op) {
     std::vector<TResult> res;
-    for (const auto &row : CollectProduce(op.get(), symbol_table, dba_))
+    for (const auto &row : CollectProduce(op.get(), symbol_table, *dba_))
       res.emplace_back(row[0].Value<TResult>());
     return res;
   }
@@ -47,7 +47,7 @@ class MatchReturnFixture : public testing::Test {
 
 TEST_F(MatchReturnFixture, MatchReturn) {
   AddVertices(2);
-  dba_.AdvanceCommand();
+  dba_->AdvanceCommand();
 
   auto test_pull_count = [&](GraphView graph_view) {
     auto scan_all =
@@ -57,21 +57,21 @@ TEST_F(MatchReturnFixture, MatchReturn) {
     symbol_table[*output->expression_] = scan_all.sym_;
     symbol_table[*output] =
         symbol_table.CreateSymbol("named_expression_1", true);
-    return PullAll(produce, dba_, symbol_table);
+    return PullAll(produce, *dba_, symbol_table);
   };
 
   EXPECT_EQ(2, test_pull_count(GraphView::NEW));
   EXPECT_EQ(2, test_pull_count(GraphView::OLD));
-  dba_.InsertVertex();
+  dba_->InsertVertex();
   EXPECT_EQ(3, test_pull_count(GraphView::NEW));
   EXPECT_EQ(2, test_pull_count(GraphView::OLD));
-  dba_.AdvanceCommand();
+  dba_->AdvanceCommand();
   EXPECT_EQ(3, test_pull_count(GraphView::OLD));
 }
 
 TEST_F(MatchReturnFixture, MatchReturnPath) {
   AddVertices(2);
-  dba_.AdvanceCommand();
+  dba_->AdvanceCommand();
 
   auto scan_all = MakeScanAll(storage, symbol_table, "n", nullptr);
   Symbol path_sym = symbol_table.CreateSymbol("path", true);
@@ -84,7 +84,7 @@ TEST_F(MatchReturnFixture, MatchReturnPath) {
   auto results = Results<query::Path>(produce);
   ASSERT_EQ(results.size(), 2);
   std::vector<query::Path> expected_paths;
-  for (const auto &v : dba_.Vertices(false)) expected_paths.emplace_back(v);
+  for (const auto &v : dba_->Vertices(false)) expected_paths.emplace_back(v);
   ASSERT_EQ(expected_paths.size(), 2);
   EXPECT_TRUE(std::is_permutation(expected_paths.begin(), expected_paths.end(),
                                   results.begin()));
@@ -92,11 +92,11 @@ TEST_F(MatchReturnFixture, MatchReturnPath) {
 
 TEST(QueryPlan, MatchReturnCartesian) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  dba.InsertVertex().add_label(dba.Label("l1"));
-  dba.InsertVertex().add_label(dba.Label("l2"));
-  dba.AdvanceCommand();
+  dba->InsertVertex().add_label(dba->Label("l1"));
+  dba->InsertVertex().add_label(dba->Label("l2"));
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -113,7 +113,7 @@ TEST(QueryPlan, MatchReturnCartesian) {
       symbol_table.CreateSymbol("named_expression_2", true);
   auto produce = MakeProduce(m.op_, return_n, return_m);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 4);
   // ensure the result ordering is OK:
   // "n" from the results is the same for the first two rows, while "m" isn't
@@ -125,12 +125,12 @@ TEST(QueryPlan, MatchReturnCartesian) {
 
 TEST(QueryPlan, StandaloneReturn) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // add a few nodes to the database
-  dba.InsertVertex();
-  dba.InsertVertex();
-  dba.AdvanceCommand();
+  dba->InsertVertex();
+  dba->InsertVertex();
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -139,7 +139,7 @@ TEST(QueryPlan, StandaloneReturn) {
   auto produce = MakeProduce(std::shared_ptr<LogicalOperator>(nullptr), output);
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 1);
   EXPECT_EQ(results[0].size(), 1);
   EXPECT_EQ(results[0][0].Value<int64_t>(), 42);
@@ -147,7 +147,8 @@ TEST(QueryPlan, StandaloneReturn) {
 
 TEST(QueryPlan, NodeFilterLabelsAndProperties) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   // add a few nodes to the database
   storage::Label label = dba.Label("Label");
@@ -203,28 +204,28 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
 
 TEST(QueryPlan, NodeFilterMultipleLabels) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // add a few nodes to the database
-  storage::Label label1 = dba.Label("label1");
-  storage::Label label2 = dba.Label("label2");
-  storage::Label label3 = dba.Label("label3");
+  storage::Label label1 = dba->Label("label1");
+  storage::Label label2 = dba->Label("label2");
+  storage::Label label3 = dba->Label("label3");
   // the test will look for nodes that have label1 and label2
-  dba.InsertVertex();                    // NOT accepted
-  dba.InsertVertex().add_label(label1);  // NOT accepted
-  dba.InsertVertex().add_label(label2);  // NOT accepted
-  dba.InsertVertex().add_label(label3);  // NOT accepted
-  auto v1 = dba.InsertVertex();          // YES accepted
+  dba->InsertVertex();                    // NOT accepted
+  dba->InsertVertex().add_label(label1);  // NOT accepted
+  dba->InsertVertex().add_label(label2);  // NOT accepted
+  dba->InsertVertex().add_label(label3);  // NOT accepted
+  auto v1 = dba->InsertVertex();          // YES accepted
   v1.add_label(label1);
   v1.add_label(label2);
-  auto v2 = dba.InsertVertex();  // NOT accepted
+  auto v2 = dba->InsertVertex();  // NOT accepted
   v2.add_label(label1);
   v2.add_label(label3);
-  auto v3 = dba.InsertVertex();  // YES accepted
+  auto v3 = dba->InsertVertex();  // YES accepted
   v3.add_label(label1);
   v3.add_label(label2);
   v3.add_label(label3);
-  dba.AdvanceCommand();
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -247,23 +248,23 @@ TEST(QueryPlan, NodeFilterMultipleLabels) {
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
   symbol_table[*output->expression_] = n.sym_;
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 2);
 }
 
 TEST(QueryPlan, Cartesian) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   auto add_vertex = [&dba](std::string label) {
-    auto vertex = dba.InsertVertex();
-    vertex.add_label(dba.Label(label));
+    auto vertex = dba->InsertVertex();
+    vertex.add_label(dba->Label(label));
     return vertex;
   };
 
   std::vector<VertexAccessor> vertices{add_vertex("v1"), add_vertex("v2"),
                                        add_vertex("v3")};
-  dba.AdvanceCommand();
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -286,7 +287,7 @@ TEST(QueryPlan, Cartesian) {
 
   auto produce = MakeProduce(cartesian_op, return_n, return_m);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 9);
   for (int i = 0; i < 3; ++i) {
     for (int j = 0; j < 3; ++j) {
@@ -298,7 +299,7 @@ TEST(QueryPlan, Cartesian) {
 
 TEST(QueryPlan, CartesianEmptySet) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -321,22 +322,22 @@ TEST(QueryPlan, CartesianEmptySet) {
 
   auto produce = MakeProduce(cartesian_op, return_n, return_m);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 0);
 }
 
 TEST(QueryPlan, CartesianThreeWay) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   auto add_vertex = [&dba](std::string label) {
-    auto vertex = dba.InsertVertex();
-    vertex.add_label(dba.Label(label));
+    auto vertex = dba->InsertVertex();
+    vertex.add_label(dba->Label(label));
     return vertex;
   };
 
   std::vector<VertexAccessor> vertices{add_vertex("v1"), add_vertex("v2"),
                                        add_vertex("v3")};
-  dba.AdvanceCommand();
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -369,7 +370,7 @@ TEST(QueryPlan, CartesianThreeWay) {
 
   auto produce = MakeProduce(cartesian_op_2, return_n, return_m, return_l);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 27);
   int id = 0;
   for (int i = 0; i < 3; ++i) {
@@ -387,23 +388,23 @@ TEST(QueryPlan, CartesianThreeWay) {
 class ExpandFixture : public testing::Test {
  protected:
   database::SingleNode db_;
-  database::GraphDbAccessor dba_{db_};
+  std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()};
   AstStorage storage;
   SymbolTable symbol_table;
 
   // make a V-graph (v3)<-[r2]-(v1)-[r1]->(v2)
-  VertexAccessor v1 = dba_.InsertVertex();
-  VertexAccessor v2 = dba_.InsertVertex();
-  VertexAccessor v3 = dba_.InsertVertex();
-  storage::EdgeType edge_type = dba_.EdgeType("Edge");
-  EdgeAccessor r1 = dba_.InsertEdge(v1, v2, edge_type);
-  EdgeAccessor r2 = dba_.InsertEdge(v1, v3, edge_type);
+  VertexAccessor v1 = dba_->InsertVertex();
+  VertexAccessor v2 = dba_->InsertVertex();
+  VertexAccessor v3 = dba_->InsertVertex();
+  storage::EdgeType edge_type = dba_->EdgeType("Edge");
+  EdgeAccessor r1 = dba_->InsertEdge(v1, v2, edge_type);
+  EdgeAccessor r2 = dba_->InsertEdge(v1, v3, edge_type);
 
   void SetUp() override {
-    v1.add_label(dba_.Label("l1"));
-    v2.add_label(dba_.Label("l2"));
-    v3.add_label(dba_.Label("l3"));
-    dba_.AdvanceCommand();
+    v1.add_label(dba_->Label("l1"));
+    v2.add_label(dba_->Label("l2"));
+    v3.add_label(dba_->Label("l3"));
+    dba_->AdvanceCommand();
   }
 };
 
@@ -420,22 +421,22 @@ TEST_F(ExpandFixture, Expand) {
         symbol_table.CreateSymbol("named_expression_1", true);
     auto produce = MakeProduce(r_m.op_, output);
 
-    return PullAll(produce, dba_, symbol_table);
+    return PullAll(produce, *dba_, symbol_table);
   };
 
   // test that expand works well for both old and new graph state
   v1.Reconstruct();
   v2.Reconstruct();
   v3.Reconstruct();
-  dba_.InsertEdge(v1, v2, edge_type);
-  dba_.InsertEdge(v1, v3, edge_type);
+  dba_->InsertEdge(v1, v2, edge_type);
+  dba_->InsertEdge(v1, v3, edge_type);
   EXPECT_EQ(2, test_expand(EdgeAtom::Direction::OUT, GraphView::OLD));
   EXPECT_EQ(2, test_expand(EdgeAtom::Direction::IN, GraphView::OLD));
   EXPECT_EQ(4, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD));
   EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, GraphView::NEW));
   EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, GraphView::NEW));
   EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::NEW));
-  dba_.AdvanceCommand();
+  dba_->AdvanceCommand();
   EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, GraphView::OLD));
   EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, GraphView::OLD));
   EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD));
@@ -456,7 +457,7 @@ TEST_F(ExpandFixture, ExpandPath) {
   auto produce = MakeProduce(path, output);
 
   std::vector<query::Path> expected_paths{{v1, r2, v3}, {v1, r1, v2}};
-  auto results = CollectProduce(produce.get(), symbol_table, dba_);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba_);
   ASSERT_EQ(results.size(), 2);
   std::vector<query::Path> results_paths;
   for (const auto &result : results)
@@ -484,11 +485,11 @@ class QueryPlanExpandVariable : public testing::Test {
   using map_int = std::unordered_map<int, int>;
 
   database::SingleNode db_;
-  database::GraphDbAccessor dba_{db_};
+  std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()};
   // labels for layers in the double chain
   std::vector<storage::Label> labels;
   // for all the edges
-  storage::EdgeType edge_type = dba_.EdgeType("edge_type");
+  storage::EdgeType edge_type = dba_->EdgeType("edge_type");
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -502,26 +503,26 @@ class QueryPlanExpandVariable : public testing::Test {
     std::vector<VertexAccessor> layer;
     for (int from_layer_ind = -1; from_layer_ind < chain_length - 1;
          from_layer_ind++) {
-      std::vector<VertexAccessor> new_layer{dba_.InsertVertex(),
-                                            dba_.InsertVertex()};
-      auto label = dba_.Label(std::to_string(from_layer_ind + 1));
+      std::vector<VertexAccessor> new_layer{dba_->InsertVertex(),
+                                            dba_->InsertVertex()};
+      auto label = dba_->Label(std::to_string(from_layer_ind + 1));
       labels.push_back(label);
       for (size_t v_to_ind = 0; v_to_ind < new_layer.size(); v_to_ind++) {
         auto &v_to = new_layer[v_to_ind];
         v_to.add_label(label);
         for (size_t v_from_ind = 0; v_from_ind < layer.size(); v_from_ind++) {
           auto &v_from = layer[v_from_ind];
-          auto edge = dba_.InsertEdge(v_from, v_to, edge_type);
-          edge.PropsSet(dba_.Property("p"),
+          auto edge = dba_->InsertEdge(v_from, v_to, edge_type);
+          edge.PropsSet(dba_->Property("p"),
                         fmt::format("V{}{}->V{}{}", from_layer_ind, v_from_ind,
                                     from_layer_ind + 1, v_to_ind));
         }
       }
       layer = new_layer;
     }
-    dba_.AdvanceCommand();
-    ASSERT_EQ(CountIterable(dba_.Vertices(false)), 2 * chain_length);
-    ASSERT_EQ(CountIterable(dba_.Edges(false)), 4 * (chain_length - 1));
+    dba_->AdvanceCommand();
+    ASSERT_EQ(CountIterable(dba_->Vertices(false)), 2 * chain_length);
+    ASSERT_EQ(CountIterable(dba_->Edges(false)), 4 * (chain_length - 1));
   }
 
   /**
@@ -591,8 +592,8 @@ class QueryPlanExpandVariable : public testing::Test {
   template <typename TResult>
   auto GetResults(std::shared_ptr<LogicalOperator> input_op, Symbol symbol) {
     Frame frame(symbol_table.max_position());
-    auto cursor = input_op->MakeCursor(dba_);
-    Context context(dba_);
+    auto cursor = input_op->MakeCursor(*dba_);
+    Context context(*dba_);
     context.symbol_table_ = symbol_table;
     std::vector<TResult> results;
     while (cursor->Pull(frame, context))
@@ -686,7 +687,6 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessSingleAndVariableExpansion) {
                          std::experimental::optional<size_t> upper,
                          bool single_expansion_before,
                          bool add_uniqueness_check) {
-
     std::shared_ptr<LogicalOperator> last_op{nullptr};
     std::vector<Symbol> symbols;
 
@@ -765,22 +765,22 @@ TEST_F(QueryPlanExpandVariable, GraphState) {
         e);
   };
 
-  auto new_edge_type = dba_.EdgeType("some_type");
+  auto new_edge_type = dba_->EdgeType("some_type");
   // add two vertices branching out from the second layer
-  for (VertexAccessor &vertex : dba_.Vertices(true))
+  for (VertexAccessor &vertex : dba_->Vertices(true))
     if (vertex.has_label(labels[1])) {
-      auto new_vertex = dba_.InsertVertex();
-      dba_.InsertEdge(vertex, new_vertex, new_edge_type);
+      auto new_vertex = dba_->InsertVertex();
+      dba_->InsertEdge(vertex, new_vertex, new_edge_type);
     }
-  ASSERT_EQ(CountIterable(dba_.Vertices(false)), 6);
-  ASSERT_EQ(CountIterable(dba_.Vertices(true)), 8);
+  ASSERT_EQ(CountIterable(dba_->Vertices(false)), 6);
+  ASSERT_EQ(CountIterable(dba_->Vertices(true)), 8);
 
   EXPECT_EQ(test_expand(GraphView::OLD, {}), (map_int{{2, 8}}));
   EXPECT_EQ(test_expand(GraphView::OLD, {new_edge_type}), (map_int{}));
   EXPECT_EQ(test_expand(GraphView::NEW, {}), (map_int{{2, 12}}));
   EXPECT_EQ(test_expand(GraphView::NEW, {edge_type}), (map_int{{2, 8}}));
   EXPECT_EQ(test_expand(GraphView::NEW, {new_edge_type}), (map_int{}));
-  dba_.AdvanceCommand();
+  dba_->AdvanceCommand();
   for (const auto graph_view : {GraphView::OLD, GraphView::NEW}) {
     EXPECT_EQ(test_expand(graph_view, {}), (map_int{{2, 12}}));
     EXPECT_EQ(test_expand(graph_view, {edge_type}), (map_int{{2, 8}}));
@@ -806,7 +806,7 @@ TEST_F(QueryPlanExpandVariable, NamedPath) {
       std::vector<Symbol>{find_symbol("n"), e, find_symbol("m")});
 
   std::vector<query::Path> expected_paths;
-  for (const auto &v : dba_.Vertices(labels[0], false))
+  for (const auto &v : dba_->Vertices(labels[0], false))
     for (const auto &e1 : v.out())
       for (const auto &e2 : e1.to().out())
         expected_paths.emplace_back(v, e1, e1.to(), e2, e2.to());
@@ -851,7 +851,8 @@ class QueryPlanExpandBfs
           if (cluster_) return cluster_->master();
           return single_node_.get();
         }()),
-        dba(*db_) {}
+        dba_ptr(db_->Access()),
+        dba(*dba_ptr) {}
 
   // Worker IDs where vertices are located.
   const std::vector<int> vertices = {0, 1, 1, 0, 1, 2};
@@ -861,7 +862,8 @@ class QueryPlanExpandBfs
 
   // Style-guide non-conformant name due to PROPERTY_PAIR and PROPERTY_LOOKUP
   // macro requirements.
-  database::GraphDbAccessor dba;
+  std::unique_ptr<database::GraphDbAccessor> dba_ptr;
+  database::GraphDbAccessor &dba;
   std::vector<storage::VertexAddress> v;
 
   AstStorage storage;
@@ -883,8 +885,8 @@ class QueryPlanExpandBfs
         vertex.PropsSet(prop.second, id);
         v.push_back(vertex.GlobalAddress());
       } else {
-        auto vertex =
-            dba.InsertVertexIntoRemote(worker, {}, {{prop.second, id}});
+        auto vertex = database::InsertVertexIntoRemote(&dba, worker, {},
+                                                       {{prop.second, id}});
         v.push_back(vertex.GlobalAddress());
       }
     }
@@ -973,7 +975,7 @@ class QueryPlanExpandBfs
     if (GetParam().first == TestType::DISTRIBUTED)
       cluster_->AdvanceCommand(tx_id);
     else
-      database::GraphDbAccessor(*db_, tx_id).AdvanceCommand();
+      db_->Access(tx_id)->AdvanceCommand();
   }
 };
 
@@ -1240,7 +1242,8 @@ class QueryPlanExpandWeightedShortestPath : public testing::Test {
   // style-guide non-conformant name due to PROPERTY_PAIR and
   // PROPERTY_LOOKUP macro requirements
   database::SingleNode db;
-  database::GraphDbAccessor dba{db};
+  std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()};
+  database::GraphDbAccessor &dba{*dba_ptr};
   std::pair<std::string, storage::Property> prop = PROPERTY_PAIR("property");
   storage::EdgeType edge_type = dba.EdgeType("edge_type");
 
@@ -1592,23 +1595,23 @@ TEST_F(QueryPlanExpandWeightedShortestPath, Exceptions) {
 
 TEST(QueryPlan, ExpandOptional) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   AstStorage storage;
   SymbolTable symbol_table;
 
   // graph (v2 {p: 2})<-[:T]-(v1 {p: 1})-[:T]->(v3 {p: 2})
-  auto prop = dba.Property("p");
-  auto edge_type = dba.EdgeType("T");
-  auto v1 = dba.InsertVertex();
+  auto prop = dba->Property("p");
+  auto edge_type = dba->EdgeType("T");
+  auto v1 = dba->InsertVertex();
   v1.PropsSet(prop, 1);
-  auto v2 = dba.InsertVertex();
+  auto v2 = dba->InsertVertex();
   v2.PropsSet(prop, 2);
-  dba.InsertEdge(v1, v2, edge_type);
-  auto v3 = dba.InsertVertex();
+  dba->InsertEdge(v1, v2, edge_type);
+  auto v3 = dba->InsertVertex();
   v3.PropsSet(prop, 2);
-  dba.InsertEdge(v1, v3, edge_type);
-  dba.AdvanceCommand();
+  dba->InsertEdge(v1, v3, edge_type);
+  dba->AdvanceCommand();
 
   // MATCH (n) OPTIONAL MATCH (n)-[r]->(m)
   auto n = MakeScanAll(storage, symbol_table, "n");
@@ -1630,7 +1633,7 @@ TEST(QueryPlan, ExpandOptional) {
   symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
   auto produce = MakeProduce(optional, n_ne, r_ne, m_ne);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   ASSERT_EQ(4, results.size());
   int v1_is_n_count = 0;
   for (auto &row : results) {
@@ -1652,7 +1655,7 @@ TEST(QueryPlan, ExpandOptional) {
 
 TEST(QueryPlan, OptionalMatchEmptyDB) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -1667,14 +1670,14 @@ TEST(QueryPlan, OptionalMatchEmptyDB) {
                                                    std::vector<Symbol>{n.sym_});
   auto produce = MakeProduce(optional, n_ne);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   ASSERT_EQ(1, results.size());
   EXPECT_EQ(results[0][0].type(), TypedValue::Type::Null);
 }
 
 TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   AstStorage storage;
   SymbolTable symbol_table;
   // OPTIONAL MATCH (n)
@@ -1696,26 +1699,26 @@ TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
   symbol_table[*m_ne->expression_] = r_m.node_sym_;
   symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
   auto produce = MakeProduce(r_m.op_, m_ne);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(0, results.size());
 }
 
 TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Make a graph with 2 connected, unlabeled nodes.
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto edge_type = dba.EdgeType("edge_type");
-  dba.InsertEdge(v1, v2, edge_type);
-  dba.AdvanceCommand();
-  EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
-  EXPECT_EQ(1, CountIterable(dba.Edges(false)));
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto edge_type = dba->EdgeType("edge_type");
+  dba->InsertEdge(v1, v2, edge_type);
+  dba->AdvanceCommand();
+  EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
+  EXPECT_EQ(1, CountIterable(dba->Edges(false)));
   AstStorage storage;
   SymbolTable symbol_table;
   // OPTIONAL MATCH (n :missing)
   auto n = MakeScanAll(storage, symbol_table, "n");
-  auto label_missing = dba.Label("missing");
+  auto label_missing = dba->Label("missing");
   n.node_->labels_.emplace_back(label_missing);
 
   auto *filter_expr =
@@ -1745,22 +1748,22 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
   symbol_table[*m_ne->expression_] = m.sym_;
   symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
   auto produce = MakeProduce(expand, m_ne);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(0, results.size());
 }
 
 TEST(QueryPlan, ExpandExistingNode) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // make a graph (v1)->(v2) that
   // has a recursive edge (v1)->(v1)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto edge_type = dba.EdgeType("Edge");
-  dba.InsertEdge(v1, v1, edge_type);
-  dba.InsertEdge(v1, v2, edge_type);
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto edge_type = dba->EdgeType("Edge");
+  dba->InsertEdge(v1, v1, edge_type);
+  dba->InsertEdge(v1, v2, edge_type);
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -1783,7 +1786,7 @@ TEST(QueryPlan, ExpandExistingNode) {
         symbol_table.CreateSymbol("named_expression_1", true);
     auto produce = MakeProduce(r_n.op_, output);
 
-    auto results = CollectProduce(produce.get(), symbol_table, dba);
+    auto results = CollectProduce(produce.get(), symbol_table, *dba);
     EXPECT_EQ(results.size(), expected_result_count);
   };
 
@@ -1795,11 +1798,11 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) {
   // we're testing that expanding on BOTH
   // does only one expansion for a cycle
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto v = dba.InsertVertex();
-  dba.InsertEdge(v, v, dba.EdgeType("et"));
-  dba.AdvanceCommand();
+  auto v = dba->InsertVertex();
+  dba->InsertEdge(v, v, dba->EdgeType("et"));
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -1808,12 +1811,13 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) {
   auto r_ =
       MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
                  EdgeAtom::Direction::BOTH, {}, "_", false, GraphView::OLD);
-  EXPECT_EQ(1, PullAll(r_.op_, dba, symbol_table));
+  EXPECT_EQ(1, PullAll(r_.op_, *dba, symbol_table));
 }
 
 TEST(QueryPlan, EdgeFilter) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   // make an N-star expanding from (v1)
   // where only one edge will qualify
@@ -1882,17 +1886,17 @@ TEST(QueryPlan, EdgeFilter) {
 
 TEST(QueryPlan, EdgeFilterMultipleTypes) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto type_1 = dba.EdgeType("type_1");
-  auto type_2 = dba.EdgeType("type_2");
-  auto type_3 = dba.EdgeType("type_3");
-  dba.InsertEdge(v1, v2, type_1);
-  dba.InsertEdge(v1, v2, type_2);
-  dba.InsertEdge(v1, v2, type_3);
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto type_1 = dba->EdgeType("type_1");
+  auto type_2 = dba->EdgeType("type_2");
+  auto type_3 = dba->EdgeType("type_3");
+  dba->InsertEdge(v1, v2, type_1);
+  dba->InsertEdge(v1, v2, type_2);
+  dba->InsertEdge(v1, v2, type_3);
+  dba->AdvanceCommand();
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -1911,13 +1915,14 @@ TEST(QueryPlan, EdgeFilterMultipleTypes) {
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
   symbol_table[*output->expression_] = r_m.node_sym_;
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 2);
 }
 
 TEST(QueryPlan, Filter) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
 
   // add a 6 nodes with property 'prop', 2 have true as value
   auto property = PROPERTY_PAIR("property");
@@ -1946,15 +1951,15 @@ TEST(QueryPlan, Filter) {
 
 TEST(QueryPlan, ExpandUniquenessFilter) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
   // make a graph that has (v1)->(v2) and a recursive edge (v1)->(v1)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto edge_type = dba.EdgeType("edge_type");
-  dba.InsertEdge(v1, v2, edge_type);
-  dba.InsertEdge(v1, v1, edge_type);
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto edge_type = dba->EdgeType("edge_type");
+  dba->InsertEdge(v1, v2, edge_type);
+  dba->InsertEdge(v1, v1, edge_type);
+  dba->AdvanceCommand();
 
   auto check_expand_results = [&](bool vertex_uniqueness,
                                   bool edge_uniqueness) {
@@ -1981,7 +1986,7 @@ TEST(QueryPlan, ExpandUniquenessFilter) {
           last_op, r2_n3.node_sym_,
           std::vector<Symbol>{n1.sym_, r1_n2.node_sym_});
 
-    return PullAll(last_op, dba, symbol_table);
+    return PullAll(last_op, *dba, symbol_table);
   };
 
   EXPECT_EQ(2, check_expand_results(false, false));
@@ -1994,14 +1999,13 @@ TEST(QueryPlan, Distinct) {
   // UNWIND [1, 2, 3, 3] AS x RETURN DISTINCT x
 
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   AstStorage storage;
   SymbolTable symbol_table;
 
   auto check_distinct = [&](const std::vector<TypedValue> input,
                             const std::vector<TypedValue> output,
                             bool assume_int_value) {
-
     auto input_expr = LITERAL(TypedValue(input));
 
     auto x = symbol_table.CreateSymbol("x", true);
@@ -2016,7 +2020,7 @@ TEST(QueryPlan, Distinct) {
     symbol_table[*x_ne] = symbol_table.CreateSymbol("x_ne", true);
     auto produce = MakeProduce(distinct, x_ne);
 
-    auto results = CollectProduce(produce.get(), symbol_table, dba);
+    auto results = CollectProduce(produce.get(), symbol_table, *dba);
     ASSERT_EQ(output.size(), results.size());
     auto output_it = output.begin();
     for (const auto &row : results) {
@@ -2037,14 +2041,14 @@ TEST(QueryPlan, Distinct) {
 
 TEST(QueryPlan, ScanAllByLabel) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Add a vertex with a label and one without.
-  auto label = dba.Label("label");
-  auto labeled_vertex = dba.InsertVertex();
+  auto label = dba->Label("label");
+  auto labeled_vertex = dba->InsertVertex();
   labeled_vertex.add_label(label);
-  dba.InsertVertex();
-  dba.AdvanceCommand();
-  EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
+  dba->InsertVertex();
+  dba->AdvanceCommand();
+  EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
   // MATCH (n :label)
   AstStorage storage;
   SymbolTable symbol_table;
@@ -2055,7 +2059,7 @@ TEST(QueryPlan, ScanAllByLabel) {
   auto produce = MakeProduce(scan_all_by_label.op_, output);
   symbol_table[*output->expression_] = scan_all_by_label.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   ASSERT_EQ(results.size(), 1);
   auto result_row = results[0];
   ASSERT_EQ(result_row.size(), 1);
@@ -2065,8 +2069,8 @@ TEST(QueryPlan, ScanAllByLabel) {
 TEST(QueryPlan, ScanAllByLabelProperty) {
   database::SingleNode db;
   // Add 5 vertices with same label, but with different property values.
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
   // vertex property values that will be stored into the DB
   // clang-format off
   std::vector<TypedValue> values{
@@ -2075,17 +2079,17 @@ TEST(QueryPlan, ScanAllByLabelProperty) {
       std::vector<TypedValue>{2}};
   // clang-format on
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     for (const auto &value : values) {
-      auto vertex = dba.InsertVertex();
+      auto vertex = dba->InsertVertex();
       vertex.add_label(label);
       vertex.PropsSet(prop, value);
     }
-    dba.Commit();
-    database::GraphDbAccessor(db).BuildIndex(label, prop);
+    dba->Commit();
+    db.Access()->BuildIndex(label, prop);
   }
-  database::GraphDbAccessor dba(db);
-  ASSERT_EQ(14, CountIterable(dba.Vertices(false)));
+  auto dba = db.Access();
+  ASSERT_EQ(14, CountIterable(dba->Vertices(false)));
 
   auto check = [&dba, label, prop](TypedValue lower, Bound::Type lower_type,
                                    TypedValue upper, Bound::Type upper_type,
@@ -2100,7 +2104,7 @@ TEST(QueryPlan, ScanAllByLabelProperty) {
     auto produce = MakeProduce(scan_all.op_, output);
     symbol_table[*output->expression_] = scan_all.sym_;
     symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-    auto results = CollectProduce(produce.get(), symbol_table, dba);
+    auto results = CollectProduce(produce.get(), symbol_table, *dba);
     ASSERT_EQ(results.size(), expected.size());
     for (size_t i = 0; i < expected.size(); i++) {
       TypedValue equal =
@@ -2136,21 +2140,21 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
   database::SingleNode db;
   // Add 2 vertices with same label, but with property values that cannot be
   // compared. On the other hand, equality works fine.
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
   {
-    database::GraphDbAccessor dba(db);
-    auto number_vertex = dba.InsertVertex();
+    auto dba = db.Access();
+    auto number_vertex = dba->InsertVertex();
     number_vertex.add_label(label);
     number_vertex.PropsSet(prop, 42);
-    auto string_vertex = dba.InsertVertex();
+    auto string_vertex = dba->InsertVertex();
     string_vertex.add_label(label);
     string_vertex.PropsSet(prop, "string");
-    dba.Commit();
-    database::GraphDbAccessor(db).BuildIndex(label, prop);
+    dba->Commit();
+    db.Access()->BuildIndex(label, prop);
   }
-  database::GraphDbAccessor dba(db);
-  EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
+  auto dba = db.Access();
+  EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
   // MATCH (n :label {prop: 42})
   AstStorage storage;
   SymbolTable symbol_table;
@@ -2161,7 +2165,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
   auto produce = MakeProduce(scan_all.op_, output);
   symbol_table[*output->expression_] = scan_all.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   ASSERT_EQ(results.size(), 1);
   const auto &row = results[0];
   ASSERT_EQ(row.size(), 1);
@@ -2173,20 +2177,20 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
 
 TEST(QueryPlan, ScanAllByLabelPropertyValueError) {
   database::SingleNode db;
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     for (int i = 0; i < 2; ++i) {
-      auto vertex = dba.InsertVertex();
+      auto vertex = dba->InsertVertex();
       vertex.add_label(label);
       vertex.PropsSet(prop, i);
     }
-    dba.Commit();
+    dba->Commit();
   }
-  database::GraphDbAccessor(db).BuildIndex(label, prop);
-  database::GraphDbAccessor dba(db);
-  EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
+  db.Access()->BuildIndex(label, prop);
+  auto dba = db.Access();
+  EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
   // MATCH (m), (n :label {prop: m})
   AstStorage storage;
   SymbolTable symbol_table;
@@ -2195,26 +2199,26 @@ TEST(QueryPlan, ScanAllByLabelPropertyValueError) {
   symbol_table[*ident_m] = scan_all.sym_;
   auto scan_index = MakeScanAllByLabelPropertyValue(
       storage, symbol_table, "n", label, prop, ident_m, scan_all.op_);
-  EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table),
+  EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
                QueryRuntimeException);
 }
 
 TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
   database::SingleNode db;
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     for (int i = 0; i < 2; ++i) {
-      auto vertex = dba.InsertVertex();
+      auto vertex = dba->InsertVertex();
       vertex.add_label(label);
       vertex.PropsSet(prop, i);
     }
-    dba.Commit();
+    dba->Commit();
   }
-  database::GraphDbAccessor(db).BuildIndex(label, prop);
-  database::GraphDbAccessor dba(db);
-  EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
+  db.Access()->BuildIndex(label, prop);
+  auto dba = db.Access();
+  EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
   // MATCH (m), (n :label {prop: m})
   AstStorage storage;
   SymbolTable symbol_table;
@@ -2227,7 +2231,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
         storage, symbol_table, "n", label, prop,
         Bound{ident_m, Bound::Type::INCLUSIVE}, std::experimental::nullopt,
         scan_all.op_);
-    EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table),
+    EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
                  QueryRuntimeException);
   }
   {
@@ -2235,7 +2239,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
     auto scan_index = MakeScanAllByLabelPropertyRange(
         storage, symbol_table, "n", label, prop, std::experimental::nullopt,
         Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_);
-    EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table),
+    EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
                  QueryRuntimeException);
   }
   {
@@ -2244,7 +2248,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
         storage, symbol_table, "n", label, prop,
         Bound{ident_m, Bound::Type::INCLUSIVE},
         Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_);
-    EXPECT_THROW(PullAll(scan_index.op_, dba, symbol_table),
+    EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
                  QueryRuntimeException);
   }
 }
@@ -2254,20 +2258,20 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) {
   // Add 2 vertices with the same label, but one has a property value while
   // the other does not. Checking if the value is equal to null, should
   // yield no results.
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.InsertVertex();
+    auto dba = db.Access();
+    auto vertex = dba->InsertVertex();
     vertex.add_label(label);
-    auto vertex_with_prop = dba.InsertVertex();
+    auto vertex_with_prop = dba->InsertVertex();
     vertex_with_prop.add_label(label);
     vertex_with_prop.PropsSet(prop, 42);
-    dba.Commit();
-    database::GraphDbAccessor(db).BuildIndex(label, prop);
+    dba->Commit();
+    db.Access()->BuildIndex(label, prop);
   }
-  database::GraphDbAccessor dba(db);
-  EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
+  auto dba = db.Access();
+  EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
   // MATCH (n :label {prop: 42})
   AstStorage storage;
   SymbolTable symbol_table;
@@ -2278,7 +2282,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) {
   auto produce = MakeProduce(scan_all.op_, output);
   symbol_table[*output->expression_] = scan_all.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 0);
 }
 
@@ -2287,20 +2291,20 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
   // Add 2 vertices with the same label, but one has a property value while
   // the other does not. Checking if the value is between nulls, should
   // yield no results.
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.InsertVertex();
+    auto dba = db.Access();
+    auto vertex = dba->InsertVertex();
     vertex.add_label(label);
-    auto vertex_with_prop = dba.InsertVertex();
+    auto vertex_with_prop = dba->InsertVertex();
     vertex_with_prop.add_label(label);
     vertex_with_prop.PropsSet(prop, 42);
-    dba.Commit();
-    database::GraphDbAccessor(db).BuildIndex(label, prop);
+    dba->Commit();
+    db.Access()->BuildIndex(label, prop);
   }
-  database::GraphDbAccessor dba(db);
-  EXPECT_EQ(2, CountIterable(dba.Vertices(false)));
+  auto dba = db.Access();
+  EXPECT_EQ(2, CountIterable(dba->Vertices(false)));
   // MATCH (n :label) WHERE null <= n.prop < null
   AstStorage storage;
   SymbolTable symbol_table;
@@ -2313,24 +2317,24 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
   auto produce = MakeProduce(scan_all.op_, output);
   symbol_table[*output->expression_] = scan_all.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  auto results = CollectProduce(produce.get(), symbol_table, *dba);
   EXPECT_EQ(results.size(), 0);
 }
 
 TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) {
   database::SingleNode db;
-  auto label = database::GraphDbAccessor(db).Label("label");
-  auto prop = database::GraphDbAccessor(db).Property("prop");
+  auto label = db.Access()->Label("label");
+  auto prop = db.Access()->Property("prop");
   {
-    database::GraphDbAccessor dba(db);
-    auto v = dba.InsertVertex();
+    auto dba = db.Access();
+    auto v = dba->InsertVertex();
     v.add_label(label);
     v.PropsSet(prop, 2);
-    dba.Commit();
-    database::GraphDbAccessor(db).BuildIndex(label, prop);
+    dba->Commit();
+    db.Access()->BuildIndex(label, prop);
   }
-  database::GraphDbAccessor dba(db);
-  EXPECT_EQ(1, CountIterable(dba.Vertices(false)));
+  auto dba = db.Access();
+  EXPECT_EQ(1, CountIterable(dba->Vertices(false)));
 
   AstStorage storage;
   SymbolTable symbol_table;
@@ -2346,7 +2350,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) {
   auto scan_all = MakeScanAllByLabelPropertyValue(storage, symbol_table, "n",
                                                   label, prop, x_expr, unwind);
 
-  EXPECT_EQ(PullAll(scan_all.op_, dba, symbol_table), 1);
+  EXPECT_EQ(PullAll(scan_all.op_, *dba, symbol_table), 1);
 }
 
 int main(int argc, char **argv) {
diff --git a/tests/unit/query_semantic.cpp b/tests/unit/query_semantic.cpp
index 6469a52ba..46df5c252 100644
--- a/tests/unit/query_semantic.cpp
+++ b/tests/unit/query_semantic.cpp
@@ -18,7 +18,8 @@ using namespace query;
 class TestSymbolGenerator : public ::testing::Test {
  protected:
   database::SingleNode db;
-  database::GraphDbAccessor dba{db};
+  std::unique_ptr<database::GraphDbAccessor> dba_ptr{db.Access()};
+  database::GraphDbAccessor &dba{*dba_ptr};
   SymbolTable symbol_table;
   SymbolGenerator symbol_generator{symbol_table};
   AstStorage storage;
diff --git a/tests/unit/query_variable_start_planner.cpp b/tests/unit/query_variable_start_planner.cpp
index 949c51bf3..525a52ba1 100644
--- a/tests/unit/query_variable_start_planner.cpp
+++ b/tests/unit/query_variable_start_planner.cpp
@@ -84,19 +84,19 @@ void CheckPlansProduce(
 
 TEST(TestVariableStartPlanner, MatchReturn) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Make a graph (v1) -[:r]-> (v2)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  dba.InsertEdge(v1, v2, dba.EdgeType("r"));
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  dba->InsertEdge(v1, v2, dba->EdgeType("r"));
+  dba->AdvanceCommand();
   // Test MATCH (n) -[r]-> (m) RETURN n
   AstStorage storage;
   QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m"))),
       RETURN("n")));
   // We have 2 nodes `n` and `m` from which we could start, so expect 2 plans.
-  CheckPlansProduce(2, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(2, storage, *dba, [&](const auto &results) {
     // We expect to produce only a single (v1) node.
     AssertRows(results, {{v1}});
   });
@@ -104,14 +104,14 @@ TEST(TestVariableStartPlanner, MatchReturn) {
 
 TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto v3 = dba.InsertVertex();
-  dba.InsertEdge(v1, v2, dba.EdgeType("r"));
-  dba.InsertEdge(v2, v3, dba.EdgeType("r"));
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto v3 = dba->InsertVertex();
+  dba->InsertEdge(v1, v2, dba->EdgeType("r"));
+  dba->InsertEdge(v2, v3, dba->EdgeType("r"));
+  dba->AdvanceCommand();
   {
     // Test `MATCH (n) -[r]-> (m) -[e]-> (l) RETURN n`
     AstStorage storage;
@@ -120,7 +120,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
                       EDGE("e", Direction::OUT), NODE("l"))),
         RETURN("n")));
     // We have 3 nodes: `n`, `m` and `l` from which we could start.
-    CheckPlansProduce(3, storage, dba, [&](const auto &results) {
+    CheckPlansProduce(3, storage, *dba, [&](const auto &results) {
       // We expect to produce only a single (v1) node.
       AssertRows(results, {{v1}});
     });
@@ -132,7 +132,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
         MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m")),
               PATTERN(NODE("m"), EDGE("e", Direction::OUT), NODE("l"))),
         RETURN("n")));
-    CheckPlansProduce(3, storage, dba, [&](const auto &results) {
+    CheckPlansProduce(3, storage, *dba, [&](const auto &results) {
       AssertRows(results, {{v1}});
     });
   }
@@ -140,14 +140,14 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
 
 TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Make a graph (v1) -[:r]-> (v2) -[:r]-> (v3)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto v3 = dba.InsertVertex();
-  dba.InsertEdge(v1, v2, dba.EdgeType("r"));
-  dba.InsertEdge(v2, v3, dba.EdgeType("r"));
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto v3 = dba->InsertVertex();
+  dba->InsertEdge(v1, v2, dba->EdgeType("r"));
+  dba->InsertEdge(v2, v3, dba->EdgeType("r"));
+  dba->AdvanceCommand();
   // Test MATCH (n) -[r]-> (m) OPTIONAL MATCH (m) -[e]-> (l) RETURN n, l
   AstStorage storage;
   QUERY(SINGLE_QUERY(
@@ -156,7 +156,7 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
       RETURN("n", "l")));
   // We have 2 nodes `n` and `m` from which we could start the MATCH, and 2
   // nodes for OPTIONAL MATCH. This should produce 2 * 2 plans.
-  CheckPlansProduce(4, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(4, storage, *dba, [&](const auto &results) {
     // We expect to produce 2 rows:
     //   * (v1), (v3)
     //   * (v2), null
@@ -166,13 +166,13 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
 
 TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Graph (v1) -[:r]-> (v2)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto r_type = dba.EdgeType("r");
-  dba.InsertEdge(v1, v2, r_type);
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto r_type = dba->EdgeType("r");
+  dba->InsertEdge(v1, v2, r_type);
+  dba->AdvanceCommand();
   // Test MATCH (n) -[r]-> (m) OPTIONAL MATCH (m) -[e]-> (l)
   //      MERGE (u) -[q:r]-> (v) RETURN n, m, l, u, v
   AstStorage storage;
@@ -183,7 +183,7 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
       RETURN("n", "m", "l", "u", "v")));
   // Since MATCH, OPTIONAL MATCH and MERGE each have 2 nodes from which we can
   // start, we generate 2 * 2 * 2 plans.
-  CheckPlansProduce(8, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(8, storage, *dba, [&](const auto &results) {
     // We expect to produce a single row: (v1), (v2), null, (v1), (v2)
     AssertRows(results, {{v1, v2, TypedValue::Null, v1, v2}});
   });
@@ -191,12 +191,12 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
 
 TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Graph (v1) -[:r]-> (v2)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  dba.InsertEdge(v1, v2, dba.EdgeType("r"));
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  dba->InsertEdge(v1, v2, dba->EdgeType("r"));
+  dba->AdvanceCommand();
   // Test MATCH (n) -[r]-> (m) WITH n MATCH (m) -[r]-> (l) RETURN n, m, l
   AstStorage storage;
   QUERY(SINGLE_QUERY(
@@ -206,7 +206,7 @@ TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
       RETURN("n", "m", "l")));
   // We can start from 2 nodes in each match. Since WITH separates query parts,
   // we expect to get 2 plans for each, which totals 2 * 2.
-  CheckPlansProduce(4, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(4, storage, *dba, [&](const auto &results) {
     // We expect to produce a single row: (v1), (v1), (v2)
     AssertRows(results, {{v1, v1, v2}});
   });
@@ -214,14 +214,14 @@ TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
 
 TEST(TestVariableStartPlanner, MatchVariableExpand) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
   // Graph (v1) -[:r1]-> (v2) -[:r2]-> (v3)
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto v3 = dba.InsertVertex();
-  auto r1 = dba.InsertEdge(v1, v2, dba.EdgeType("r1"));
-  auto r2 = dba.InsertEdge(v2, v3, dba.EdgeType("r2"));
-  dba.AdvanceCommand();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto v3 = dba->InsertVertex();
+  auto r1 = dba->InsertEdge(v1, v2, dba->EdgeType("r1"));
+  auto r2 = dba->InsertEdge(v2, v3, dba->EdgeType("r2"));
+  dba->AdvanceCommand();
   // Test MATCH (n) -[r*]-> (m) RETURN r
   AstStorage storage;
   auto edge = EDGE_VARIABLE("r", Direction::OUT);
@@ -230,14 +230,15 @@ TEST(TestVariableStartPlanner, MatchVariableExpand) {
   TypedValue r1_list(std::vector<TypedValue>{r1});         // [r1]
   TypedValue r2_list(std::vector<TypedValue>{r2});         // [r2]
   TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2});  // [r1, r2]
-  CheckPlansProduce(2, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(2, storage, *dba, [&](const auto &results) {
     AssertRows(results, {{r1_list}, {r2_list}, {r1_r2_list}});
   });
 }
 
 TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   auto id = dba.Property("id");
   // Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3})
   auto v1 = dba.InsertVertex();
@@ -264,16 +265,16 @@ TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
 
 TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto id = dba.Property("id");
+  auto dba = db.Access();
+  auto id = dba->Property("id");
   // Graph (v1 {id:1}) -[:r1]-> (v2) -[:r2]-> (v3)
-  auto v1 = dba.InsertVertex();
+  auto v1 = dba->InsertVertex();
   v1.PropsSet(id, 1);
-  auto v2 = dba.InsertVertex();
-  auto v3 = dba.InsertVertex();
-  auto r1 = dba.InsertEdge(v1, v2, dba.EdgeType("r1"));
-  auto r2 = dba.InsertEdge(v2, v3, dba.EdgeType("r2"));
-  dba.AdvanceCommand();
+  auto v2 = dba->InsertVertex();
+  auto v3 = dba->InsertVertex();
+  auto r1 = dba->InsertEdge(v1, v2, dba->EdgeType("r1"));
+  auto r2 = dba->InsertEdge(v2, v3, dba->EdgeType("r2"));
+  dba->AdvanceCommand();
   // Test MATCH (n {id:1}) -[r*]- (m) RETURN r
   AstStorage storage;
   auto edge = EDGE_VARIABLE("r", Direction::BOTH);
@@ -283,14 +284,15 @@ TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
   // We expect to get a single column with the following rows:
   TypedValue r1_list(std::vector<TypedValue>{r1});         // [r1]
   TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2});  // [r1, r2]
-  CheckPlansProduce(2, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(2, storage, *dba, [&](const auto &results) {
     AssertRows(results, {{r1_list}, {r1_r2_list}});
   });
 }
 
 TEST(TestVariableStartPlanner, MatchBfs) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba_ptr = db.Access();
+  auto &dba = *dba_ptr;
   auto id = dba.Property("id");
   // Graph (v1 {id:1}) -[:r1]-> (v2 {id: 2}) -[:r2]-> (v3 {id: 3})
   auto v1 = dba.InsertVertex();
diff --git a/tests/unit/record_edge_vertex_accessor.cpp b/tests/unit/record_edge_vertex_accessor.cpp
index 6956a995c..b1f27b742 100644
--- a/tests/unit/record_edge_vertex_accessor.cpp
+++ b/tests/unit/record_edge_vertex_accessor.cpp
@@ -14,13 +14,13 @@
 
 TEST(RecordAccessor, Properties) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto vertex = dba.InsertVertex();
+  auto vertex = dba->InsertVertex();
   auto &properties = vertex.Properties();
 
-  auto property = dba.Property("PropName");
-  auto property_other = dba.Property("Other");
+  auto property = dba->Property("PropName");
+  auto property_other = dba->Property("Other");
   EXPECT_EQ(vertex.PropsAt(property).type(), PropertyValue::Type::Null);
 
   vertex.PropsSet(property, 42);
@@ -36,39 +36,38 @@ TEST(RecordAccessor, Properties) {
 
 TEST(RecordAccessor, DbAccessor) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto vertex = dba.InsertVertex();
+  auto vertex = dba->InsertVertex();
   const auto &const_vertex_dba = vertex.db_accessor();
-  EXPECT_EQ(&dba, &const_vertex_dba);
+  EXPECT_EQ(dba.get(), &const_vertex_dba);
   auto &vertex_dba = vertex.db_accessor();
-  EXPECT_EQ(&dba, &vertex_dba);
+  EXPECT_EQ(dba.get(), &vertex_dba);
 }
 
 TEST(RecordAccessor, RecordEquality) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
   EXPECT_EQ(v1, v1);
   EXPECT_NE(v1, v2);
 
-  auto e1 = dba.InsertEdge(v1, v2, dba.EdgeType("type"));
-  auto e2 = dba.InsertEdge(v1, v2, dba.EdgeType("type"));
+  auto e1 = dba->InsertEdge(v1, v2, dba->EdgeType("type"));
+  auto e2 = dba->InsertEdge(v1, v2, dba->EdgeType("type"));
   EXPECT_EQ(e1, e1);
   EXPECT_NE(e1, e2);
 }
 
 TEST(RecordAccessor, GlobalToLocalAddressConversion) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
+  auto dba = db.Access();
 
-  auto v1 = dba.InsertVertex();
-  storage::Address<mvcc::VersionList<Vertex>> global_address{v1.gid(),
-                                                             db.WorkerId()};
+  auto v1 = dba->InsertVertex();
+  storage::Address<mvcc::VersionList<Vertex>> global_address{v1.gid(), 0};
   EXPECT_FALSE(global_address.is_local());
-  auto v1_from_global = VertexAccessor(global_address, dba);
+  auto v1_from_global = VertexAccessor(global_address, *dba);
   EXPECT_TRUE(v1_from_global.address().is_local());
   EXPECT_EQ(v1_from_global.address(), v1.address());
 }
@@ -78,26 +77,26 @@ TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) {
 
   // test both Switches work on new record
   {
-    database::GraphDbAccessor dba(db);
-    auto v1 = dba.InsertVertex();
+    auto dba = db.Access();
+    auto v1 = dba->InsertVertex();
     v1.SwitchOld();
     v1.SwitchNew();
-    dba.Commit();
+    dba->Commit();
   }
 
   // test both Switches work on existing record
   {
-    database::GraphDbAccessor dba(db);
-    auto v1 = *dba.Vertices(false).begin();
+    auto dba = db.Access();
+    auto v1 = *dba->Vertices(false).begin();
     v1.SwitchOld();
     v1.SwitchNew();
   }
 
   // ensure switch exposes the right data
   {
-    database::GraphDbAccessor dba(db);
-    auto label = dba.Label("label");
-    auto v1 = *dba.Vertices(false).begin();
+    auto dba = db.Access();
+    auto label = dba->Label("label");
+    auto v1 = *dba->Vertices(false).begin();
 
     EXPECT_FALSE(v1.has_label(label));  // old record
     v1.add_label(label);                // modifying data does not switch to new
@@ -111,26 +110,26 @@ TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) {
 
 TEST(RecordAccessor, Reconstruct) {
   database::SingleNode db;
-  auto label = database::GraphDbAccessor(db).Label("label");
+  auto label = db.Access()->Label("label");
 
   {
     // we must operate on an old vertex
     // because otherwise we only have new
     // so create a vertex and commit it
-    database::GraphDbAccessor dba(db);
-    dba.InsertVertex();
-    dba.Commit();
+    auto dba = db.Access();
+    dba->InsertVertex();
+    dba->Commit();
   }
 
   // ensure we don't have label set
-  database::GraphDbAccessor dba(db);
-  auto v1 = *dba.Vertices(false).begin();
+  auto dba = db.Access();
+  auto v1 = *dba->Vertices(false).begin();
   v1.SwitchNew();
   EXPECT_FALSE(v1.has_label(label));
 
   {
     // update the record through a different accessor
-    auto v1_other_accessor = *dba.Vertices(false).begin();
+    auto v1_other_accessor = *dba->Vertices(false).begin();
     v1_other_accessor.add_label(label);
     EXPECT_FALSE(v1.has_label(label));
     v1_other_accessor.SwitchNew();
@@ -145,14 +144,14 @@ TEST(RecordAccessor, Reconstruct) {
 
 TEST(RecordAccessor, VertexLabels) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
   auto &labels = v1.labels();
 
   EXPECT_EQ(v1.labels().size(), 0);
 
-  storage::Label l1 = dba.Label("label1");
-  storage::Label l2 = dba.Label("label2");
+  storage::Label l1 = dba->Label("label1");
+  storage::Label l2 = dba->Label("label2");
 
   // adding labels
   EXPECT_FALSE(v1.has_label(l1));
@@ -172,7 +171,7 @@ TEST(RecordAccessor, VertexLabels) {
   EXPECT_EQ(labels.size(), 2);
 
   // removing labels
-  storage::Label l3 = dba.Label("label3");
+  storage::Label l3 = dba->Label("label3");
   v1.remove_label(l3);
   EXPECT_EQ(labels.size(), 2);
 
@@ -186,38 +185,38 @@ TEST(RecordAccessor, VertexLabels) {
 
 TEST(RecordAccessor, EdgeType) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
 
-  storage::EdgeType likes = dba.EdgeType("likes");
-  storage::EdgeType hates = dba.EdgeType("hates");
+  storage::EdgeType likes = dba->EdgeType("likes");
+  storage::EdgeType hates = dba->EdgeType("hates");
 
-  auto edge = dba.InsertEdge(v1, v2, likes);
+  auto edge = dba->InsertEdge(v1, v2, likes);
   EXPECT_EQ(edge.EdgeType(), likes);
   EXPECT_NE(edge.EdgeType(), hates);
 }
 
 TEST(RecordAccessor, EdgeIsCycle) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto likes = dba.EdgeType("edge_type");
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto likes = dba->EdgeType("edge_type");
 
-  EXPECT_TRUE(dba.InsertEdge(v1, v1, likes).is_cycle());
-  EXPECT_TRUE(dba.InsertEdge(v2, v2, likes).is_cycle());
-  EXPECT_FALSE(dba.InsertEdge(v1, v2, likes).is_cycle());
-  EXPECT_FALSE(dba.InsertEdge(v2, v1, likes).is_cycle());
+  EXPECT_TRUE(dba->InsertEdge(v1, v1, likes).is_cycle());
+  EXPECT_TRUE(dba->InsertEdge(v2, v2, likes).is_cycle());
+  EXPECT_FALSE(dba->InsertEdge(v1, v2, likes).is_cycle());
+  EXPECT_FALSE(dba->InsertEdge(v2, v1, likes).is_cycle());
 }
 
 TEST(RecordAccessor, VertexEdgeConnections) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto edge = dba.InsertEdge(v1, v2, dba.EdgeType("likes"));
-  dba.AdvanceCommand();
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto edge = dba->InsertEdge(v1, v2, dba->EdgeType("likes"));
+  dba->AdvanceCommand();
 
   EXPECT_EQ(edge.from(), v1);
   EXPECT_NE(edge.from(), v2);
@@ -245,17 +244,17 @@ TEST(RecordAccessor, VertexEdgeConnections) {
 
 TEST(RecordAccessor, VertexEdgeConnectionsWithExistingVertex) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto v3 = dba.InsertVertex();
-  auto edge_type = dba.EdgeType("edge type");
-  auto e12 = dba.InsertEdge(v1, v2, edge_type);
-  auto e22 = dba.InsertEdge(v2, v2, edge_type);
-  auto e23a = dba.InsertEdge(v2, v3, edge_type);
-  auto e23b = dba.InsertEdge(v2, v3, edge_type);
-  auto e32 = dba.InsertEdge(v3, v2, edge_type);
-  dba.AdvanceCommand();
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto v3 = dba->InsertVertex();
+  auto edge_type = dba->EdgeType("edge type");
+  auto e12 = dba->InsertEdge(v1, v2, edge_type);
+  auto e22 = dba->InsertEdge(v2, v2, edge_type);
+  auto e23a = dba->InsertEdge(v2, v3, edge_type);
+  auto e23b = dba->InsertEdge(v2, v3, edge_type);
+  auto e32 = dba->InsertEdge(v3, v2, edge_type);
+  dba->AdvanceCommand();
 
   TEST_EDGE_ITERABLE(v1.out(v1));
   TEST_EDGE_ITERABLE(v1.out(v2), {e12});
@@ -280,17 +279,17 @@ TEST(RecordAccessor, VertexEdgeConnectionsWithExistingVertex) {
 
 TEST(RecordAccessor, VertexEdgeConnectionsWithEdgeType) {
   database::SingleNode db;
-  database::GraphDbAccessor dba(db);
-  auto v1 = dba.InsertVertex();
-  auto v2 = dba.InsertVertex();
-  auto a = dba.EdgeType("a");
-  auto b = dba.EdgeType("b");
-  auto c = dba.EdgeType("c");
-  auto ea = dba.InsertEdge(v1, v2, a);
-  auto eb_1 = dba.InsertEdge(v2, v1, b);
-  auto eb_2 = dba.InsertEdge(v2, v1, b);
-  auto ec = dba.InsertEdge(v1, v2, c);
-  dba.AdvanceCommand();
+  auto dba = db.Access();
+  auto v1 = dba->InsertVertex();
+  auto v2 = dba->InsertVertex();
+  auto a = dba->EdgeType("a");
+  auto b = dba->EdgeType("b");
+  auto c = dba->EdgeType("c");
+  auto ea = dba->InsertEdge(v1, v2, a);
+  auto eb_1 = dba->InsertEdge(v2, v1, b);
+  auto eb_2 = dba->InsertEdge(v2, v1, b);
+  auto ec = dba->InsertEdge(v1, v2, c);
+  dba->AdvanceCommand();
 
   TEST_EDGE_ITERABLE(v1.in(), {eb_1, eb_2});
   TEST_EDGE_ITERABLE(v2.in(), {ea, ec});
diff --git a/tests/unit/state_delta.cpp b/tests/unit/state_delta.cpp
index f7e86e6e5..7d64189b7 100644
--- a/tests/unit/state_delta.cpp
+++ b/tests/unit/state_delta.cpp
@@ -9,15 +9,15 @@ TEST(StateDelta, CreateVertex) {
   gid::Generator generator(0);
   auto gid0 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     auto delta =
-        database::StateDelta::CreateVertex(dba.transaction_id(), gid0, 0);
-    delta.Apply(dba);
-    dba.Commit();
+        database::StateDelta::CreateVertex(dba->transaction_id(), gid0, 0);
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.FindVertexOptional(gid0, false);
+    auto dba = db.Access();
+    auto vertex = dba->FindVertexOptional(gid0, false);
     EXPECT_TRUE(vertex);
     EXPECT_EQ(vertex->cypher_id(), 0);
   }
@@ -28,20 +28,20 @@ TEST(StateDelta, RemoveVertex) {
   gid::Generator generator(0);
   auto gid0 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
-    dba.InsertVertex(gid0);
-    dba.Commit();
+    auto dba = db.Access();
+    dba->InsertVertex(gid0);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     auto delta =
-        database::StateDelta::RemoveVertex(dba.transaction_id(), gid0, true);
-    delta.Apply(dba);
-    dba.Commit();
+        database::StateDelta::RemoveVertex(dba->transaction_id(), gid0, true);
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.FindVertexOptional(gid0, false);
+    auto dba = db.Access();
+    auto vertex = dba->FindVertexOptional(gid0, false);
     EXPECT_FALSE(vertex);
   }
 }
@@ -53,22 +53,22 @@ TEST(StateDelta, CreateEdge) {
   auto gid1 = generator.Next();
   auto gid2 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
-    dba.InsertVertex(gid0);
-    dba.InsertVertex(gid1);
-    dba.Commit();
+    auto dba = db.Access();
+    dba->InsertVertex(gid0);
+    dba->InsertVertex(gid1);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     auto delta =
-        database::StateDelta::CreateEdge(dba.transaction_id(), gid2, 0, gid0,
-                                         gid1, dba.EdgeType("edge"), "edge");
-    delta.Apply(dba);
-    dba.Commit();
+        database::StateDelta::CreateEdge(dba->transaction_id(), gid2, 0, gid0,
+                                         gid1, dba->EdgeType("edge"), "edge");
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto edge = dba.FindEdgeOptional(gid2, false);
+    auto dba = db.Access();
+    auto edge = dba->FindEdgeOptional(gid2, false);
     EXPECT_TRUE(edge);
   }
 }
@@ -80,21 +80,21 @@ TEST(StateDelta, RemoveEdge) {
   auto gid1 = generator.Next();
   auto gid2 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
-    auto v0 = dba.InsertVertex(gid0);
-    auto v1 = dba.InsertVertex(gid1);
-    dba.InsertEdge(v0, v1, dba.EdgeType("edge"), gid2);
-    dba.Commit();
+    auto dba = db.Access();
+    auto v0 = dba->InsertVertex(gid0);
+    auto v1 = dba->InsertVertex(gid1);
+    dba->InsertEdge(v0, v1, dba->EdgeType("edge"), gid2);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto delta = database::StateDelta::RemoveEdge(dba.transaction_id(), gid2);
-    delta.Apply(dba);
-    dba.Commit();
+    auto dba = db.Access();
+    auto delta = database::StateDelta::RemoveEdge(dba->transaction_id(), gid2);
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto edge = dba.FindEdgeOptional(gid2, false);
+    auto dba = db.Access();
+    auto edge = dba->FindEdgeOptional(gid2, false);
     EXPECT_FALSE(edge);
   }
 }
@@ -104,24 +104,24 @@ TEST(StateDelta, AddLabel) {
   gid::Generator generator(0);
   auto gid0 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
-    dba.InsertVertex(gid0);
-    dba.Commit();
+    auto dba = db.Access();
+    dba->InsertVertex(gid0);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto delta = database::StateDelta::AddLabel(dba.transaction_id(), gid0,
-                                                dba.Label("label"), "label");
-    delta.Apply(dba);
-    dba.Commit();
+    auto dba = db.Access();
+    auto delta = database::StateDelta::AddLabel(dba->transaction_id(), gid0,
+                                                dba->Label("label"), "label");
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.FindVertexOptional(gid0, false);
+    auto dba = db.Access();
+    auto vertex = dba->FindVertexOptional(gid0, false);
     EXPECT_TRUE(vertex);
     auto labels = vertex->labels();
     EXPECT_EQ(labels.size(), 1);
-    EXPECT_EQ(labels[0], dba.Label("label"));
+    EXPECT_EQ(labels[0], dba->Label("label"));
   }
 }
 
@@ -130,21 +130,21 @@ TEST(StateDelta, RemoveLabel) {
   gid::Generator generator(0);
   auto gid0 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.InsertVertex(gid0);
-    vertex.add_label(dba.Label("label"));
-    dba.Commit();
+    auto dba = db.Access();
+    auto vertex = dba->InsertVertex(gid0);
+    vertex.add_label(dba->Label("label"));
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto delta = database::StateDelta::RemoveLabel(dba.transaction_id(), gid0,
-                                                   dba.Label("label"), "label");
-    delta.Apply(dba);
-    dba.Commit();
+    auto dba = db.Access();
+    auto delta = database::StateDelta::RemoveLabel(dba->transaction_id(), gid0,
+                                                   dba->Label("label"), "label");
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.FindVertexOptional(gid0, false);
+    auto dba = db.Access();
+    auto vertex = dba->FindVertexOptional(gid0, false);
     EXPECT_TRUE(vertex);
     auto labels = vertex->labels();
     EXPECT_EQ(labels.size(), 0);
@@ -156,23 +156,23 @@ TEST(StateDelta, SetPropertyVertex) {
   gid::Generator generator(0);
   auto gid0 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
-    dba.InsertVertex(gid0);
-    dba.Commit();
+    auto dba = db.Access();
+    dba->InsertVertex(gid0);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     auto delta = database::StateDelta::PropsSetVertex(
-        dba.transaction_id(), gid0, dba.Property("property"), "property",
+        dba->transaction_id(), gid0, dba->Property("property"), "property",
         PropertyValue(2212));
-    delta.Apply(dba);
-    dba.Commit();
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto vertex = dba.FindVertexOptional(gid0, false);
+    auto dba = db.Access();
+    auto vertex = dba->FindVertexOptional(gid0, false);
     EXPECT_TRUE(vertex);
-    auto prop = vertex->PropsAt(dba.Property("property"));
+    auto prop = vertex->PropsAt(dba->Property("property"));
     EXPECT_EQ(prop.Value<int64_t>(), 2212);
   }
 }
@@ -184,25 +184,25 @@ TEST(StateDelta, SetPropertyEdge) {
   auto gid1 = generator.Next();
   auto gid2 = generator.Next();
   {
-    database::GraphDbAccessor dba(db);
-    auto v0 = dba.InsertVertex(gid0);
-    auto v1 = dba.InsertVertex(gid1);
-    dba.InsertEdge(v0, v1, dba.EdgeType("edge"), gid2);
-    dba.Commit();
+    auto dba = db.Access();
+    auto v0 = dba->InsertVertex(gid0);
+    auto v1 = dba->InsertVertex(gid1);
+    dba->InsertEdge(v0, v1, dba->EdgeType("edge"), gid2);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
+    auto dba = db.Access();
     auto delta = database::StateDelta::PropsSetEdge(
-        dba.transaction_id(), gid2, dba.Property("property"), "property",
+        dba->transaction_id(), gid2, dba->Property("property"), "property",
         PropertyValue(2212));
-    delta.Apply(dba);
-    dba.Commit();
+    delta.Apply(*dba);
+    dba->Commit();
   }
   {
-    database::GraphDbAccessor dba(db);
-    auto edge = dba.FindEdgeOptional(gid2, false);
+    auto dba = db.Access();
+    auto edge = dba->FindEdgeOptional(gid2, false);
     EXPECT_TRUE(edge);
-    auto prop = edge->PropsAt(dba.Property("property"));
+    auto prop = edge->PropsAt(dba->Property("property"));
     EXPECT_EQ(prop.Value<int64_t>(), 2212);
   }
 }
diff --git a/tests/unit/typed_value.cpp b/tests/unit/typed_value.cpp
index 7d8780443..49297b4b7 100644
--- a/tests/unit/typed_value.cpp
+++ b/tests/unit/typed_value.cpp
@@ -19,7 +19,7 @@ class AllTypesFixture : public testing::Test {
  protected:
   std::vector<TypedValue> values_;
   database::SingleNode db_;
-  database::GraphDbAccessor dba_{db_};
+  std::unique_ptr<database::GraphDbAccessor> dba_{db_.Access()};
 
   void SetUp() override {
     values_.emplace_back(TypedValue::Null);
@@ -35,10 +35,11 @@ class AllTypesFixture : public testing::Test {
                                           {"c", 42},
                                           {"d", 0.5},
                                           {"e", TypedValue::Null}});
-    auto vertex = dba_.InsertVertex();
+    auto vertex = dba_->InsertVertex();
     values_.emplace_back(vertex);
-    values_.emplace_back(dba_.InsertEdge(vertex, vertex, dba_.EdgeType("et")));
-    values_.emplace_back(query::Path(dba_.InsertVertex()));
+    values_.emplace_back(
+        dba_->InsertEdge(vertex, vertex, dba_->EdgeType("et")));
+    values_.emplace_back(query::Path(dba_->InsertVertex()));
   }
 };
 
diff --git a/tools/tests/mg_recovery_check.cpp b/tools/tests/mg_recovery_check.cpp
index 1ae6959e1..34326202e 100644
--- a/tools/tests/mg_recovery_check.cpp
+++ b/tools/tests/mg_recovery_check.cpp
@@ -31,21 +31,21 @@ class RecoveryTest : public ::testing::Test {
 };
 
 TEST_F(RecoveryTest, TestVerticesRecovered) {
-  database::GraphDbAccessor dba(db_);
-  EXPECT_EQ(dba.VerticesCount(), 10);
-  EXPECT_EQ(dba.VerticesCount(dba.Label("Comment")), 5);
-  for (const auto &vertex : dba.Vertices(dba.Label("Comment"), false)) {
-    EXPECT_TRUE(vertex.has_label(dba.Label("Message")));
+  auto dba = db_.Access();
+  EXPECT_EQ(dba->VerticesCount(), 10);
+  EXPECT_EQ(dba->VerticesCount(dba->Label("Comment")), 5);
+  for (const auto &vertex : dba->Vertices(dba->Label("Comment"), false)) {
+    EXPECT_TRUE(vertex.has_label(dba->Label("Message")));
   }
-  EXPECT_EQ(dba.VerticesCount(dba.Label("Forum")), 5);
+  EXPECT_EQ(dba->VerticesCount(dba->Label("Forum")), 5);
 }
 
 TEST_F(RecoveryTest, TestPropertyNull) {
-  database::GraphDbAccessor dba(db_);
+  auto dba = db_.Access();
   bool found = false;
-  for (const auto &vertex : dba.Vertices(dba.Label("Comment"), false)) {
-    auto id_prop = query::TypedValue(vertex.PropsAt(dba.Property("id")));
-    auto browser = query::TypedValue(vertex.PropsAt(dba.Property("browser")));
+  for (const auto &vertex : dba->Vertices(dba->Label("Comment"), false)) {
+    auto id_prop = query::TypedValue(vertex.PropsAt(dba->Property("id")));
+    auto browser = query::TypedValue(vertex.PropsAt(dba->Property("browser")));
     if (id_prop.IsString() && id_prop.Value<std::string>() == "2") {
       EXPECT_FALSE(found);
       found = true;
@@ -58,10 +58,10 @@ TEST_F(RecoveryTest, TestPropertyNull) {
 }
 
 TEST_F(RecoveryTest, TestEdgesRecovered) {
-  database::GraphDbAccessor dba(db_);
-  EXPECT_EQ(dba.EdgesCount(), 5);
-  for (const auto &edge : dba.Edges(false)) {
-    EXPECT_TRUE(edge.EdgeType() == dba.EdgeType("POSTED_ON"));
+  auto dba = db_.Access();
+  EXPECT_EQ(dba->EdgesCount(), 5);
+  for (const auto &edge : dba->Edges(false)) {
+    EXPECT_TRUE(edge.EdgeType() == dba->EdgeType("POSTED_ON"));
   }
 }