From 7e84744d0754e9b4658c383f897a091dc3807cc4 Mon Sep 17 00:00:00 2001
From: Jure Bajic <jure.bajic@memgraph.com>
Date: Thu, 1 Sep 2022 09:10:40 +0200
Subject: [PATCH] Split storage and shards (#519)

- Rename storage to shard
- Add primary label and range for shard
- Remove id_mapper functionality from shard
- Adapt tests
---
 src/glue/v2/communication.cpp                 |   14 +-
 src/glue/v2/communication.hpp                 |   16 +-
 src/query/v2/db_accessor.hpp                  |   19 +-
 src/query/v2/interpreter.cpp                  |   37 +-
 src/query/v2/interpreter.hpp                  |   23 +-
 src/query/v2/stream/streams.cpp               |    2 +-
 src/storage/v3/CMakeLists.txt                 |    1 +
 src/storage/v3/edge_accessor.hpp              |    2 +-
 .../v3/replication/replication_client.cpp     |  122 +-
 .../v3/replication/replication_client.hpp     |    8 +-
 .../v3/replication/replication_server.cpp     |  186 +-
 .../v3/replication/replication_server.hpp     |    8 +-
 src/storage/v3/shard.cpp                      | 1849 +++++++++++++++++
 src/storage/v3/shard.hpp                      |  617 ++++++
 src/storage/v3/storage.cpp                    | 1847 +---------------
 src/storage/v3/storage.hpp                    |  601 +-----
 src/storage/v3/vertex_accessor.hpp            |    4 +-
 tests/unit/CMakeLists.txt                     |    6 +-
 tests/unit/query_v2_interpreter.cpp           |   50 +-
 tests/unit/result_stream_faker.hpp            |    4 +-
 tests/unit/storage_v3.cpp                     |  180 +-
 tests/unit/storage_v3_test_utils.cpp          |    2 +-
 tests/unit/storage_v3_test_utils.hpp          |    2 +-
 tests/unit/storage_v3_vertex_accessors.cpp    |   44 +-
 24 files changed, 2869 insertions(+), 2775 deletions(-)
 create mode 100644 src/storage/v3/shard.cpp
 create mode 100644 src/storage/v3/shard.hpp

diff --git a/src/glue/v2/communication.cpp b/src/glue/v2/communication.cpp
index 55dfd8838..d05b521fe 100644
--- a/src/glue/v2/communication.cpp
+++ b/src/glue/v2/communication.cpp
@@ -16,7 +16,7 @@
 #include <vector>
 
 #include "storage/v3/edge_accessor.hpp"
-#include "storage/v3/storage.hpp"
+#include "storage/v3/shard.hpp"
 #include "storage/v3/vertex_accessor.hpp"
 #include "utils/temporal.hpp"
 
@@ -64,16 +64,16 @@ query::v2::TypedValue ToTypedValue(const Value &value) {
 }
 
 storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const query::v2::VertexAccessor &vertex,
-                                                              const storage::v3::Storage &db, storage::v3::View view) {
+                                                              const storage::v3::Shard &db, storage::v3::View view) {
   return ToBoltVertex(vertex.impl_, db, view);
 }
 
 storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const query::v2::EdgeAccessor &edge,
-                                                          const storage::v3::Storage &db, storage::v3::View view) {
+                                                          const storage::v3::Shard &db, storage::v3::View view) {
   return ToBoltEdge(edge.impl_, db, view);
 }
 
-storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const storage::v3::Storage &db,
+storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const storage::v3::Shard &db,
                                        storage::v3::View view) {
   switch (value.type()) {
     case query::v2::TypedValue::Type::Null:
@@ -132,7 +132,7 @@ storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const
 }
 
 storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const storage::v3::VertexAccessor &vertex,
-                                                              const storage::v3::Storage &db, storage::v3::View view) {
+                                                              const storage::v3::Shard &db, storage::v3::View view) {
   // TODO(jbajic) Fix bolt communication
   auto id = communication::bolt::Id::FromUint(0);
   auto maybe_labels = vertex.Labels(view);
@@ -152,7 +152,7 @@ storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const storage::v3:
 }
 
 storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const storage::v3::EdgeAccessor &edge,
-                                                          const storage::v3::Storage &db, storage::v3::View view) {
+                                                          const storage::v3::Shard &db, storage::v3::View view) {
   // TODO(jbajic) Fix bolt communication
   auto id = communication::bolt::Id::FromUint(0);
   auto from = communication::bolt::Id::FromUint(0);
@@ -167,7 +167,7 @@ storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const storage::v3::Edg
   return communication::bolt::Edge{id, from, to, type, properties};
 }
 
-storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Storage &db,
+storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Shard &db,
                                                           storage::v3::View view) {
   std::vector<communication::bolt::Vertex> vertices;
   vertices.reserve(path.vertices().size());
diff --git a/src/glue/v2/communication.hpp b/src/glue/v2/communication.hpp
index 13bf96fca..794912724 100644
--- a/src/glue/v2/communication.hpp
+++ b/src/glue/v2/communication.hpp
@@ -28,36 +28,36 @@ namespace memgraph::glue::v2 {
 
 /// @param storage::v3::VertexAccessor for converting to
 ///        communication::bolt::Vertex.
-/// @param storage::v3::Storage for getting label and property names.
+/// @param storage::v3::Shard for getting label and property names.
 /// @param storage::v3::View for deciding which vertex attributes are visible.
 ///
 /// @throw std::bad_alloc
 storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const storage::v3::VertexAccessor &vertex,
-                                                              const storage::v3::Storage &db, storage::v3::View view);
+                                                              const storage::v3::Shard &db, storage::v3::View view);
 
 /// @param storage::v3::EdgeAccessor for converting to communication::bolt::Edge.
-/// @param storage::v3::Storage for getting edge type and property names.
+/// @param storage::v3::Shard for getting edge type and property names.
 /// @param storage::v3::View for deciding which edge attributes are visible.
 ///
 /// @throw std::bad_alloc
 storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const storage::v3::EdgeAccessor &edge,
-                                                          const storage::v3::Storage &db, storage::v3::View view);
+                                                          const storage::v3::Shard &db, storage::v3::View view);
 
 /// @param query::v2::Path for converting to communication::bolt::Path.
-/// @param storage::v3::Storage for ToBoltVertex and ToBoltEdge.
+/// @param storage::v3::Shard for ToBoltVertex and ToBoltEdge.
 /// @param storage::v3::View for ToBoltVertex and ToBoltEdge.
 ///
 /// @throw std::bad_alloc
-storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Storage &db,
+storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Shard &db,
                                                           storage::v3::View view);
 
 /// @param query::v2::TypedValue for converting to communication::bolt::Value.
-/// @param storage::v3::Storage for ToBoltVertex and ToBoltEdge.
+/// @param storage::v3::Shard for ToBoltVertex and ToBoltEdge.
 /// @param storage::v3::View for ToBoltVertex and ToBoltEdge.
 ///
 /// @throw std::bad_alloc
 storage::v3::Result<communication::bolt::Value> ToBoltValue(const query::v2::TypedValue &value,
-                                                            const storage::v3::Storage &db, storage::v3::View view);
+                                                            const storage::v3::Shard &db, storage::v3::View view);
 
 query::v2::TypedValue ToTypedValue(const communication::bolt::Value &value);
 
diff --git a/src/query/v2/db_accessor.hpp b/src/query/v2/db_accessor.hpp
index beacf2138..a652ca656 100644
--- a/src/query/v2/db_accessor.hpp
+++ b/src/query/v2/db_accessor.hpp
@@ -212,7 +212,7 @@ inline VertexAccessor EdgeAccessor::From() const { return VertexAccessor(impl_.F
 inline bool EdgeAccessor::IsCycle() const { return To() == From(); }
 
 class DbAccessor final {
-  storage::v3::Storage::Accessor *accessor_;
+  storage::v3::Shard::Accessor *accessor_;
 
   class VerticesIterable final {
     storage::v3::VerticesIterable iterable_;
@@ -244,7 +244,7 @@ class DbAccessor final {
   };
 
  public:
-  explicit DbAccessor(storage::v3::Storage::Accessor *accessor) : accessor_(accessor) {}
+  explicit DbAccessor(storage::v3::Shard::Accessor *accessor) : accessor_(accessor) {}
 
   // TODO(jbajic) Fix Remove Gid
   // NOLINTNEXTLINE(readability-convert-member-functions-to-static)
@@ -348,11 +348,20 @@ class DbAccessor final {
     return {std::make_optional<VertexAccessor>(*value)};
   }
 
-  storage::v3::PropertyId NameToProperty(const std::string_view name) { return accessor_->NameToProperty(name); }
+  // TODO(jbajic) Query engine should have a map of labels, properties and edge
+  // types
+  // NOLINTNEXTLINE(readability-convert-member-functions-to-static)
+  storage::v3::PropertyId NameToProperty(const std::string_view /*name*/) {
+    return storage::v3::PropertyId::FromUint(0);
+  }
 
-  storage::v3::LabelId NameToLabel(const std::string_view name) { return accessor_->NameToLabel(name); }
+  // NOLINTNEXTLINE(readability-convert-member-functions-to-static)
+  storage::v3::LabelId NameToLabel(const std::string_view /*name*/) { return storage::v3::LabelId::FromUint(0); }
 
-  storage::v3::EdgeTypeId NameToEdgeType(const std::string_view name) { return accessor_->NameToEdgeType(name); }
+  // NOLINTNEXTLINE(readability-convert-member-functions-to-static)
+  storage::v3::EdgeTypeId NameToEdgeType(const std::string_view /*name*/) {
+    return storage::v3::EdgeTypeId::FromUint(0);
+  }
 
   const std::string &PropertyToName(storage::v3::PropertyId prop) const { return accessor_->PropertyToName(prop); }
 
diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index c65b5e971..86825dd5e 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -43,6 +43,7 @@
 #include "query/v2/trigger.hpp"
 #include "query/v2/typed_value.hpp"
 #include "storage/v3/property_value.hpp"
+#include "storage/v3/shard.hpp"
 #include "storage/v3/storage.hpp"
 #include "utils/algorithm.hpp"
 #include "utils/csv_parsing.hpp"
@@ -127,7 +128,7 @@ std::optional<std::string> GetOptionalStringValue(query::v2::Expression *express
 
 class ReplQueryHandler final : public query::v2::ReplicationQueryHandler {
  public:
-  explicit ReplQueryHandler(storage::v3::Storage *db) : db_(db) {}
+  explicit ReplQueryHandler(storage::v3::Shard *db) : db_(db) {}
 
   /// @throw QueryRuntimeException if an error ocurred.
   void SetReplicationRole(ReplicationQuery::ReplicationRole replication_role, std::optional<int64_t> port) override {
@@ -255,7 +256,7 @@ class ReplQueryHandler final : public query::v2::ReplicationQueryHandler {
   }
 
  private:
-  storage::v3::Storage *db_;
+  storage::v3::Shard *db_;
 };
 /// returns false if the replication role can't be set
 /// @throw QueryRuntimeException if an error ocurred.
@@ -913,7 +914,7 @@ Callback HandleSchemaQuery(SchemaQuery *schema_query, InterpreterContext *interp
       callback.header = {"property_name", "property_type"};
       callback.fn = [interpreter_context, primary_label = schema_query->label_]() {
         auto *db = interpreter_context->db;
-        const auto label = db->NameToLabel(primary_label.name);
+        const auto label = interpreter_context->NameToLabelId(primary_label.name);
         const auto *schema = db->GetSchema(label);
         std::vector<std::vector<TypedValue>> results;
         if (schema) {
@@ -938,11 +939,11 @@ Callback HandleSchemaQuery(SchemaQuery *schema_query, InterpreterContext *interp
       callback.fn = [interpreter_context, primary_label = schema_query->label_,
                      schema_type_map = std::move(schema_type_map)]() {
         auto *db = interpreter_context->db;
-        const auto label = db->NameToLabel(primary_label.name);
+        const auto label = interpreter_context->NameToLabelId(primary_label.name);
         std::vector<storage::v3::SchemaProperty> schemas_types;
         schemas_types.reserve(schema_type_map.size());
         for (const auto &schema_type : schema_type_map) {
-          auto property_id = db->NameToProperty(schema_type.first.name);
+          auto property_id = interpreter_context->NameToPropertyId(schema_type.first.name);
           schemas_types.push_back({property_id, schema_type.second});
         }
         if (!db->CreateSchema(label, schemas_types)) {
@@ -957,7 +958,7 @@ Callback HandleSchemaQuery(SchemaQuery *schema_query, InterpreterContext *interp
     case SchemaQuery::Action::DROP_SCHEMA: {
       callback.fn = [interpreter_context, primary_label = schema_query->label_]() {
         auto *db = interpreter_context->db;
-        const auto label = db->NameToLabel(primary_label.name);
+        const auto label = interpreter_context->NameToLabelId(primary_label.name);
 
         if (!db->DropSchema(label)) {
           throw QueryException(fmt::format("Schema on label :{} does not exist!", primary_label.name));
@@ -1138,7 +1139,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *strea
 using RWType = plan::ReadWriteTypeChecker::RWType;
 }  // namespace
 
-InterpreterContext::InterpreterContext(storage::v3::Storage *db, const InterpreterConfig config,
+InterpreterContext::InterpreterContext(storage::v3::Shard *db, const InterpreterConfig config,
                                        const std::filesystem::path &data_directory)
     : db(db), trigger_store(data_directory / "triggers"), config(config), streams{this, data_directory / "streams"} {}
 
@@ -1157,8 +1158,8 @@ PreparedQuery Interpreter::PrepareTransactionQuery(std::string_view query_upper)
       in_explicit_transaction_ = true;
       expect_rollback_ = false;
 
-      db_accessor_ = std::make_unique<storage::v3::Storage::Accessor>(
-          interpreter_context_->db->Access(GetIsolationLevelOverride()));
+      db_accessor_ =
+          std::make_unique<storage::v3::Shard::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride()));
       execution_db_accessor_.emplace(db_accessor_.get());
 
       if (interpreter_context_->trigger_store.HasTriggers()) {
@@ -1427,14 +1428,14 @@ PreparedQuery PrepareIndexQuery(ParsedQuery parsed_query, bool in_explicit_trans
     }
   };
 
-  auto label = interpreter_context->db->NameToLabel(index_query->label_.name);
+  auto label = interpreter_context->NameToLabelId(index_query->label_.name);
 
   std::vector<storage::v3::PropertyId> properties;
   std::vector<std::string> properties_string;
   properties.reserve(index_query->properties_.size());
   properties_string.reserve(index_query->properties_.size());
   for (const auto &prop : index_query->properties_) {
-    properties.push_back(interpreter_context->db->NameToProperty(prop.name));
+    properties.push_back(interpreter_context->NameToPropertyId(prop.name));
     properties_string.push_back(prop.name);
   }
   auto properties_stringified = utils::Join(properties_string, ", ");
@@ -1842,7 +1843,7 @@ PreparedQuery PrepareCreateSnapshotQuery(ParsedQuery parsed_query, bool in_expli
       [interpreter_context](AnyStream *stream, std::optional<int> n) -> std::optional<QueryHandlerResult> {
         if (auto maybe_error = interpreter_context->db->CreateSnapshot(); maybe_error.HasError()) {
           switch (maybe_error.GetError()) {
-            case storage::v3::Storage::CreateSnapshotError::DisabledForReplica:
+            case storage::v3::Shard::CreateSnapshotError::DisabledForReplica:
               throw utils::BasicException(
                   "Failed to create a snapshot. Replica instances are not allowed to create them.");
           }
@@ -1897,8 +1898,8 @@ PreparedQuery PrepareVersionQuery(ParsedQuery parsed_query, const bool in_explic
 }
 
 PreparedQuery PrepareInfoQuery(ParsedQuery parsed_query, bool in_explicit_transaction,
-                               std::map<std::string, TypedValue> *summary, InterpreterContext *interpreter_context,
-                               storage::v3::Storage *db, utils::MemoryResource *execution_memory) {
+                               std::map<std::string, TypedValue> * /*summary*/, InterpreterContext *interpreter_context,
+                               storage::v3::Shard *db, utils::MemoryResource * /*execution_memory*/) {
   if (in_explicit_transaction) {
     throw InfoInMulticommandTxException();
   }
@@ -1994,13 +1995,13 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
   auto *constraint_query = utils::Downcast<ConstraintQuery>(parsed_query.query);
   std::function<void(Notification &)> handler;
 
-  auto label = interpreter_context->db->NameToLabel(constraint_query->constraint_.label.name);
+  auto label = interpreter_context->NameToLabelId(constraint_query->constraint_.label.name);
   std::vector<storage::v3::PropertyId> properties;
   std::vector<std::string> properties_string;
   properties.reserve(constraint_query->constraint_.properties.size());
   properties_string.reserve(constraint_query->constraint_.properties.size());
   for (const auto &prop : constraint_query->constraint_.properties) {
-    properties.push_back(interpreter_context->db->NameToProperty(prop.name));
+    properties.push_back(interpreter_context->NameToPropertyId(prop.name));
     properties_string.push_back(prop.name);
   }
   auto properties_stringified = utils::Join(properties_string, ", ");
@@ -2259,8 +2260,8 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
         (utils::Downcast<CypherQuery>(parsed_query.query) || utils::Downcast<ExplainQuery>(parsed_query.query) ||
          utils::Downcast<ProfileQuery>(parsed_query.query) || utils::Downcast<DumpQuery>(parsed_query.query) ||
          utils::Downcast<TriggerQuery>(parsed_query.query))) {
-      db_accessor_ = std::make_unique<storage::v3::Storage::Accessor>(
-          interpreter_context_->db->Access(GetIsolationLevelOverride()));
+      db_accessor_ =
+          std::make_unique<storage::v3::Shard::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride()));
       execution_db_accessor_.emplace(db_accessor_.get());
 
       if (utils::Downcast<CypherQuery>(parsed_query.query) && interpreter_context_->trigger_store.HasTriggers()) {
diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index d3da9c724..75267ae88 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -31,6 +31,7 @@
 #include "query/v2/trigger.hpp"
 #include "query/v2/typed_value.hpp"
 #include "storage/v3/isolation_level.hpp"
+#include "storage/v3/name_id_mapper.hpp"
 #include "utils/event_counter.hpp"
 #include "utils/logging.hpp"
 #include "utils/memory.hpp"
@@ -166,10 +167,10 @@ struct PreparedQuery {
  * been passed to an `Interpreter` instance.
  */
 struct InterpreterContext {
-  explicit InterpreterContext(storage::v3::Storage *db, InterpreterConfig config,
+  explicit InterpreterContext(storage::v3::Shard *db, InterpreterConfig config,
                               const std::filesystem::path &data_directory);
 
-  storage::v3::Storage *db;
+  storage::v3::Shard *db;
 
   std::optional<double> tsc_frequency{utils::GetTSCFrequency()};
   std::atomic<bool> is_shutting_down{false};
@@ -186,6 +187,22 @@ struct InterpreterContext {
   const InterpreterConfig config;
 
   query::v2::stream::Streams streams;
+
+  storage::v3::LabelId NameToLabelId(std::string_view label_name) {
+    return storage::v3::LabelId::FromUint(query_id_mapper.NameToId(label_name));
+  }
+
+  storage::v3::PropertyId NameToPropertyId(std::string_view property_name) {
+    return storage::v3::PropertyId::FromUint(query_id_mapper.NameToId(property_name));
+  }
+
+  storage::v3::EdgeTypeId NameToEdgeTypeId(std::string_view edge_type_name) {
+    return storage::v3::EdgeTypeId::FromUint(query_id_mapper.NameToId(edge_type_name));
+  }
+
+ private:
+  // TODO Replace with local map of labels, properties and edge type ids
+  storage::v3::NameIdMapper query_id_mapper;
 };
 
 /// Function that is used to tell all active interpreters that they should stop
@@ -316,7 +333,7 @@ class Interpreter final {
   // This cannot be std::optional because we need to move this accessor later on into a lambda capture
   // which is assigned to std::function. std::function requires every object to be copyable, so we
   // move this unique_ptr into a shrared_ptr.
-  std::unique_ptr<storage::v3::Storage::Accessor> db_accessor_;
+  std::unique_ptr<storage::v3::Shard::Accessor> db_accessor_;
   std::optional<DbAccessor> execution_db_accessor_;
   std::optional<TriggerContextCollector> trigger_context_collector_;
   bool in_explicit_transaction_{false};
diff --git a/src/query/v2/stream/streams.cpp b/src/query/v2/stream/streams.cpp
index 563e1401f..026976127 100644
--- a/src/query/v2/stream/streams.cpp
+++ b/src/query/v2/stream/streams.cpp
@@ -83,7 +83,7 @@ std::pair<TypedValue /*query*/, TypedValue /*parameters*/> ExtractTransformation
 
 template <typename TMessage>
 void CallCustomTransformation(const std::string &transformation_name, const std::vector<TMessage> &messages,
-                              mgp_result &result, storage::v3::Storage::Accessor &storage_accessor,
+                              mgp_result &result, storage::v3::Shard::Accessor &storage_accessor,
                               utils::MemoryResource &memory_resource, const std::string &stream_name) {
   DbAccessor db_accessor{&storage_accessor};
   {
diff --git a/src/storage/v3/CMakeLists.txt b/src/storage/v3/CMakeLists.txt
index 272d2cd03..bd93ba607 100644
--- a/src/storage/v3/CMakeLists.txt
+++ b/src/storage/v3/CMakeLists.txt
@@ -14,6 +14,7 @@ set(storage_v3_src_files
     vertex_accessor.cpp
     schemas.cpp
     schema_validator.cpp
+    shard.cpp
     storage.cpp)
 
 # #### Replication #####
diff --git a/src/storage/v3/edge_accessor.hpp b/src/storage/v3/edge_accessor.hpp
index cf8b658d8..8a17b163c 100644
--- a/src/storage/v3/edge_accessor.hpp
+++ b/src/storage/v3/edge_accessor.hpp
@@ -31,7 +31,7 @@ struct Constraints;
 
 class EdgeAccessor final {
  private:
-  friend class Storage;
+  friend class Shard;
 
  public:
   EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex, Transaction *transaction,
diff --git a/src/storage/v3/replication/replication_client.cpp b/src/storage/v3/replication/replication_client.cpp
index ecc3ffbf4..9a5594c5b 100644
--- a/src/storage/v3/replication/replication_client.cpp
+++ b/src/storage/v3/replication/replication_client.cpp
@@ -30,10 +30,10 @@ template <typename>
 }  // namespace
 
 ////// ReplicationClient //////
-Storage::ReplicationClient::ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint,
-                                              const replication::ReplicationMode mode,
-                                              const replication::ReplicationClientConfig &config)
-    : name_(std::move(name)), storage_(storage), mode_(mode) {
+Shard::ReplicationClient::ReplicationClient(std::string name, Shard *shard, const io::network::Endpoint &endpoint,
+                                            const replication::ReplicationMode mode,
+                                            const replication::ReplicationClientConfig &config)
+    : name_(std::move(name)), shard_(shard), mode_(mode) {
   if (config.ssl) {
     rpc_context_.emplace(config.ssl->key_file, config.ssl->cert_file);
   } else {
@@ -54,14 +54,14 @@ Storage::ReplicationClient::ReplicationClient(std::string name, Storage *storage
   }
 }
 
-void Storage::ReplicationClient::TryInitializeClientAsync() {
+void Shard::ReplicationClient::TryInitializeClientAsync() {
   thread_pool_.AddTask([this] {
     rpc_client_->Abort();
     this->TryInitializeClientSync();
   });
 }
 
-void Storage::ReplicationClient::FrequentCheck() {
+void Shard::ReplicationClient::FrequentCheck() {
   const auto is_success = std::invoke([this]() {
     try {
       auto stream{rpc_client_->Stream<replication::FrequentHeartbeatRpc>()};
@@ -87,15 +87,15 @@ void Storage::ReplicationClient::FrequentCheck() {
 }
 
 /// @throws rpc::RpcFailedException
-void Storage::ReplicationClient::InitializeClient() {
+void Shard::ReplicationClient::InitializeClient() {
   uint64_t current_commit_timestamp{kTimestampInitialId};
 
-  auto stream{rpc_client_->Stream<replication::HeartbeatRpc>(storage_->last_commit_timestamp_, storage_->epoch_id_)};
+  auto stream{rpc_client_->Stream<replication::HeartbeatRpc>(shard_->last_commit_timestamp_, shard_->epoch_id_)};
 
   const auto response = stream.AwaitResponse();
   std::optional<uint64_t> branching_point;
-  if (response.epoch_id != storage_->epoch_id_ && response.current_commit_timestamp != kTimestampInitialId) {
-    const auto &epoch_history = storage_->epoch_history_;
+  if (response.epoch_id != shard_->epoch_id_ && response.current_commit_timestamp != kTimestampInitialId) {
+    const auto &epoch_history = shard_->epoch_history_;
     const auto epoch_info_iter =
         std::find_if(epoch_history.crbegin(), epoch_history.crend(),
                      [&](const auto &epoch_info) { return epoch_info.first == response.epoch_id; });
@@ -115,8 +115,8 @@ void Storage::ReplicationClient::InitializeClient() {
 
   current_commit_timestamp = response.current_commit_timestamp;
   spdlog::trace("Current timestamp on replica: {}", current_commit_timestamp);
-  spdlog::trace("Current timestamp on main: {}", storage_->last_commit_timestamp_);
-  if (current_commit_timestamp == storage_->last_commit_timestamp_) {
+  spdlog::trace("Current timestamp on main: {}", shard_->last_commit_timestamp_);
+  if (current_commit_timestamp == shard_->last_commit_timestamp_) {
     spdlog::debug("Replica '{}' up to date", name_);
     std::unique_lock client_guard{client_lock_};
     replica_state_.store(replication::ReplicaState::READY);
@@ -130,7 +130,7 @@ void Storage::ReplicationClient::InitializeClient() {
   }
 }
 
-void Storage::ReplicationClient::TryInitializeClientSync() {
+void Shard::ReplicationClient::TryInitializeClientSync() {
   try {
     InitializeClient();
   } catch (const rpc::RpcFailedException &) {
@@ -141,19 +141,19 @@ void Storage::ReplicationClient::TryInitializeClientSync() {
   }
 }
 
-void Storage::ReplicationClient::HandleRpcFailure() {
+void Shard::ReplicationClient::HandleRpcFailure() {
   spdlog::error(utils::MessageWithLink("Couldn't replicate data to {}.", name_, "https://memgr.ph/replication"));
   TryInitializeClientAsync();
 }
 
-replication::SnapshotRes Storage::ReplicationClient::TransferSnapshot(const std::filesystem::path &path) {
+replication::SnapshotRes Shard::ReplicationClient::TransferSnapshot(const std::filesystem::path &path) {
   auto stream{rpc_client_->Stream<replication::SnapshotRpc>()};
   replication::Encoder encoder(stream.GetBuilder());
   encoder.WriteFile(path);
   return stream.AwaitResponse();
 }
 
-replication::WalFilesRes Storage::ReplicationClient::TransferWalFiles(
+replication::WalFilesRes Shard::ReplicationClient::TransferWalFiles(
     const std::vector<std::filesystem::path> &wal_files) {
   MG_ASSERT(!wal_files.empty(), "Wal files list is empty!");
   auto stream{rpc_client_->Stream<replication::WalFilesRpc>(wal_files.size())};
@@ -166,7 +166,7 @@ replication::WalFilesRes Storage::ReplicationClient::TransferWalFiles(
   return stream.AwaitResponse();
 }
 
-void Storage::ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) {
+void Shard::ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) {
   std::unique_lock guard(client_lock_);
   const auto status = replica_state_.load();
   switch (status) {
@@ -190,7 +190,7 @@ void Storage::ReplicationClient::StartTransactionReplication(const uint64_t curr
     case replication::ReplicaState::READY:
       MG_ASSERT(!replica_stream_);
       try {
-        replica_stream_.emplace(ReplicaStream{this, storage_->last_commit_timestamp_, current_wal_seq_num});
+        replica_stream_.emplace(ReplicaStream{this, shard_->last_commit_timestamp_, current_wal_seq_num});
         replica_state_.store(replication::ReplicaState::REPLICATING);
       } catch (const rpc::RpcFailedException &) {
         replica_state_.store(replication::ReplicaState::INVALID);
@@ -200,7 +200,7 @@ void Storage::ReplicationClient::StartTransactionReplication(const uint64_t curr
   }
 }
 
-void Storage::ReplicationClient::IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback) {
+void Shard::ReplicationClient::IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback) {
   // We can only check the state because it guarantees to be only
   // valid during a single transaction replication (if the assumption
   // that this and other transaction replication functions can only be
@@ -220,7 +220,7 @@ void Storage::ReplicationClient::IfStreamingTransaction(const std::function<void
   }
 }
 
-void Storage::ReplicationClient::FinalizeTransactionReplication() {
+void Shard::ReplicationClient::FinalizeTransactionReplication() {
   // We can only check the state because it guarantees to be only
   // valid during a single transaction replication (if the assumption
   // that this and other transaction replication functions can only be
@@ -271,7 +271,7 @@ void Storage::ReplicationClient::FinalizeTransactionReplication() {
   }
 }
 
-void Storage::ReplicationClient::FinalizeTransactionReplicationInternal() {
+void Shard::ReplicationClient::FinalizeTransactionReplicationInternal() {
   MG_ASSERT(replica_stream_, "Missing stream for transaction deltas");
   try {
     auto response = replica_stream_->Finalize();
@@ -293,9 +293,9 @@ void Storage::ReplicationClient::FinalizeTransactionReplicationInternal() {
   }
 }
 
-void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
+void Shard::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
   while (true) {
-    auto file_locker = storage_->file_retainer_.AddLocker();
+    auto file_locker = shard_->file_retainer_.AddLocker();
 
     const auto steps = GetRecoverySteps(replica_commit, &file_locker);
     for (const auto &recovery_step : steps) {
@@ -312,11 +312,11 @@ void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
                 auto response = TransferWalFiles(arg);
                 replica_commit = response.current_commit_timestamp;
               } else if constexpr (std::is_same_v<StepType, RecoveryCurrentWal>) {
-                if (storage_->wal_file_ && storage_->wal_file_->SequenceNumber() == arg.current_wal_seq_num) {
-                  storage_->wal_file_->DisableFlushing();
+                if (shard_->wal_file_ && shard_->wal_file_->SequenceNumber() == arg.current_wal_seq_num) {
+                  shard_->wal_file_->DisableFlushing();
                   spdlog::debug("Sending current wal file");
                   replica_commit = ReplicateCurrentWal();
-                  storage_->wal_file_->EnableFlushing();
+                  shard_->wal_file_->EnableFlushing();
                 }
               } else {
                 static_assert(always_false_v<T>, "Missing type from variant visitor");
@@ -345,20 +345,20 @@ void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
     // By adding this lock, we can avoid that, and go to RECOVERY immediately.
     std::unique_lock client_guard{client_lock_};
     SPDLOG_INFO("Replica timestamp: {}", replica_commit);
-    SPDLOG_INFO("Last commit: {}", storage_->last_commit_timestamp_);
-    if (storage_->last_commit_timestamp_ == replica_commit) {
+    SPDLOG_INFO("Last commit: {}", shard_->last_commit_timestamp_);
+    if (shard_->last_commit_timestamp_ == replica_commit) {
       replica_state_.store(replication::ReplicaState::READY);
       return;
     }
   }
 }
 
-uint64_t Storage::ReplicationClient::ReplicateCurrentWal() {
-  const auto &wal_file = storage_->wal_file_;
+uint64_t Shard::ReplicationClient::ReplicateCurrentWal() {
+  const auto &wal_file = shard_->wal_file_;
   auto stream = TransferCurrentWalFile();
   stream.AppendFilename(wal_file->Path().filename());
   utils::InputFile file;
-  MG_ASSERT(file.Open(storage_->wal_file_->Path()), "Failed to open current WAL file!");
+  MG_ASSERT(file.Open(shard_->wal_file_->Path()), "Failed to open current WAL file!");
   const auto [buffer, buffer_size] = wal_file->CurrentFileBuffer();
   stream.AppendSize(file.GetSize() + buffer_size);
   stream.AppendFileData(&file);
@@ -387,23 +387,23 @@ uint64_t Storage::ReplicationClient::ReplicateCurrentWal() {
 /// recovery steps, so we can safely send it to the replica.
 /// We assume that the property of preserving at least 1 WAL before the snapshot
 /// is satisfied as we extract the timestamp information from it.
-std::vector<Storage::ReplicationClient::RecoveryStep> Storage::ReplicationClient::GetRecoverySteps(
+std::vector<Shard::ReplicationClient::RecoveryStep> Shard::ReplicationClient::GetRecoverySteps(
     const uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker) {
   // First check if we can recover using the current wal file only
   // otherwise save the seq_num of the current wal file
   // This lock is also necessary to force the missed transaction to finish.
   std::optional<uint64_t> current_wal_seq_num;
   std::optional<uint64_t> current_wal_from_timestamp;
-  if (storage_->wal_file_) {
-    current_wal_seq_num.emplace(storage_->wal_file_->SequenceNumber());
-    current_wal_from_timestamp.emplace(storage_->wal_file_->FromTimestamp());
+  if (shard_->wal_file_) {
+    current_wal_seq_num.emplace(shard_->wal_file_->SequenceNumber());
+    current_wal_from_timestamp.emplace(shard_->wal_file_->FromTimestamp());
   }
 
   auto locker_acc = file_locker->Access();
-  auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_, current_wal_seq_num);
+  auto wal_files = durability::GetWalFiles(shard_->wal_directory_, shard_->uuid_, current_wal_seq_num);
   MG_ASSERT(wal_files, "Wal files could not be loaded");
 
-  auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
+  auto snapshot_files = durability::GetSnapshotFiles(shard_->snapshot_directory_, shard_->uuid_);
   std::optional<durability::SnapshotDurabilityInfo> latest_snapshot;
   if (!snapshot_files.empty()) {
     std::sort(snapshot_files.begin(), snapshot_files.end());
@@ -529,13 +529,13 @@ std::vector<Storage::ReplicationClient::RecoveryStep> Storage::ReplicationClient
 }
 
 ////// TimeoutDispatcher //////
-void Storage::ReplicationClient::TimeoutDispatcher::WaitForTaskToFinish() {
+void Shard::ReplicationClient::TimeoutDispatcher::WaitForTaskToFinish() {
   // Wait for the previous timeout task to finish
   std::unique_lock main_guard(main_lock);
   main_cv.wait(main_guard, [&] { return finished; });
 }
 
-void Storage::ReplicationClient::TimeoutDispatcher::StartTimeoutTask(const double timeout) {
+void Shard::ReplicationClient::TimeoutDispatcher::StartTimeoutTask(const double timeout) {
   timeout_pool.AddTask([timeout, this] {
     finished = false;
     using std::chrono::steady_clock;
@@ -553,65 +553,65 @@ void Storage::ReplicationClient::TimeoutDispatcher::StartTimeoutTask(const doubl
   });
 }
 ////// ReplicaStream //////
-Storage::ReplicationClient::ReplicaStream::ReplicaStream(ReplicationClient *self,
-                                                         const uint64_t previous_commit_timestamp,
-                                                         const uint64_t current_seq_num)
+Shard::ReplicationClient::ReplicaStream::ReplicaStream(ReplicationClient *self,
+                                                       const uint64_t previous_commit_timestamp,
+                                                       const uint64_t current_seq_num)
     : self_(self),
       stream_(self_->rpc_client_->Stream<replication::AppendDeltasRpc>(previous_commit_timestamp, current_seq_num)) {
   replication::Encoder encoder{stream_.GetBuilder()};
-  encoder.WriteString(self_->storage_->epoch_id_);
+  encoder.WriteString(self_->shard_->epoch_id_);
 }
 
-void Storage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex,
-                                                            uint64_t final_commit_timestamp) {
+void Shard::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex,
+                                                          uint64_t final_commit_timestamp) {
   replication::Encoder encoder(stream_.GetBuilder());
-  EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, self_->storage_->config_.items, delta, vertex,
+  EncodeDelta(&encoder, &self_->shard_->name_id_mapper_, self_->shard_->config_.items, delta, vertex,
               final_commit_timestamp);
 }
 
-void Storage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge,
-                                                            uint64_t final_commit_timestamp) {
+void Shard::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge,
+                                                          uint64_t final_commit_timestamp) {
   replication::Encoder encoder(stream_.GetBuilder());
-  EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, delta, edge, final_commit_timestamp);
+  EncodeDelta(&encoder, &self_->shard_->name_id_mapper_, delta, edge, final_commit_timestamp);
 }
 
-void Storage::ReplicationClient::ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) {
+void Shard::ReplicationClient::ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) {
   replication::Encoder encoder(stream_.GetBuilder());
   EncodeTransactionEnd(&encoder, final_commit_timestamp);
 }
 
-void Storage::ReplicationClient::ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation,
-                                                                LabelId label, const std::set<PropertyId> &properties,
-                                                                uint64_t timestamp) {
+void Shard::ReplicationClient::ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation,
+                                                              LabelId label, const std::set<PropertyId> &properties,
+                                                              uint64_t timestamp) {
   replication::Encoder encoder(stream_.GetBuilder());
-  EncodeOperation(&encoder, &self_->storage_->name_id_mapper_, operation, label, properties, timestamp);
+  EncodeOperation(&encoder, &self_->shard_->name_id_mapper_, operation, label, properties, timestamp);
 }
 
-replication::AppendDeltasRes Storage::ReplicationClient::ReplicaStream::Finalize() { return stream_.AwaitResponse(); }
+replication::AppendDeltasRes Shard::ReplicationClient::ReplicaStream::Finalize() { return stream_.AwaitResponse(); }
 
 ////// CurrentWalHandler //////
-Storage::ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self)
+Shard::ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self)
     : self_(self), stream_(self_->rpc_client_->Stream<replication::CurrentWalRpc>()) {}
 
-void Storage::ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) {
+void Shard::ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) {
   replication::Encoder encoder(stream_.GetBuilder());
   encoder.WriteString(filename);
 }
 
-void Storage::ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) {
+void Shard::ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) {
   replication::Encoder encoder(stream_.GetBuilder());
   encoder.WriteUint(size);
 }
 
-void Storage::ReplicationClient::CurrentWalHandler::AppendFileData(utils::InputFile *file) {
+void Shard::ReplicationClient::CurrentWalHandler::AppendFileData(utils::InputFile *file) {
   replication::Encoder encoder(stream_.GetBuilder());
   encoder.WriteFileData(file);
 }
 
-void Storage::ReplicationClient::CurrentWalHandler::AppendBufferData(const uint8_t *buffer, const size_t buffer_size) {
+void Shard::ReplicationClient::CurrentWalHandler::AppendBufferData(const uint8_t *buffer, const size_t buffer_size) {
   replication::Encoder encoder(stream_.GetBuilder());
   encoder.WriteBuffer(buffer, buffer_size);
 }
 
-replication::CurrentWalRes Storage::ReplicationClient::CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); }
+replication::CurrentWalRes Shard::ReplicationClient::CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); }
 }  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/replication/replication_client.hpp b/src/storage/v3/replication/replication_client.hpp
index f0c47f2e0..7a1731f5f 100644
--- a/src/storage/v3/replication/replication_client.hpp
+++ b/src/storage/v3/replication/replication_client.hpp
@@ -28,7 +28,7 @@
 #include "storage/v3/replication/enums.hpp"
 #include "storage/v3/replication/rpc.hpp"
 #include "storage/v3/replication/serialization.hpp"
-#include "storage/v3/storage.hpp"
+#include "storage/v3/shard.hpp"
 #include "utils/file.hpp"
 #include "utils/file_locker.hpp"
 #include "utils/spin_lock.hpp"
@@ -37,9 +37,9 @@
 
 namespace memgraph::storage::v3 {
 
-class Storage::ReplicationClient {
+class Shard::ReplicationClient {
  public:
-  ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint,
+  ReplicationClient(std::string name, Shard *shard, const io::network::Endpoint &endpoint,
                     replication::ReplicationMode mode, const replication::ReplicationClientConfig &config = {});
 
   // Handler used for transfering the current transaction.
@@ -149,7 +149,7 @@ class Storage::ReplicationClient {
   void HandleRpcFailure();
 
   std::string name_;
-  Storage *storage_;
+  Shard *shard_;
   std::optional<communication::ClientContext> rpc_context_;
   std::optional<rpc::Client> rpc_client_;
 
diff --git a/src/storage/v3/replication/replication_server.cpp b/src/storage/v3/replication/replication_server.cpp
index 552214b9a..1905071ce 100644
--- a/src/storage/v3/replication/replication_server.cpp
+++ b/src/storage/v3/replication/replication_server.cpp
@@ -40,9 +40,9 @@ std::pair<uint64_t, durability::WalDeltaData> ReadDelta(durability::BaseDecoder
 };
 }  // namespace
 
-Storage::ReplicationServer::ReplicationServer(Storage *storage, io::network::Endpoint endpoint,
-                                              const replication::ReplicationServerConfig &config)
-    : storage_(storage) {
+Shard::ReplicationServer::ReplicationServer(Shard *shard, io::network::Endpoint endpoint,
+                                            const replication::ReplicationServerConfig &config)
+    : shard_(shard) {
   // Create RPC server.
   if (config.ssl) {
     rpc_server_context_.emplace(config.ssl->key_file, config.ssl->cert_file, config.ssl->ca_file,
@@ -84,21 +84,21 @@ Storage::ReplicationServer::ReplicationServer(Storage *storage, io::network::End
   rpc_server_->Start();
 }
 
-void Storage::ReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
+void Shard::ReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
   replication::HeartbeatReq req;
   slk::Load(&req, req_reader);
-  replication::HeartbeatRes res{true, storage_->last_commit_timestamp_, storage_->epoch_id_};
+  replication::HeartbeatRes res{true, shard_->last_commit_timestamp_, shard_->epoch_id_};
   slk::Save(res, res_builder);
 }
 
-void Storage::ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
+void Shard::ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
   replication::FrequentHeartbeatReq req;
   slk::Load(&req, req_reader);
   replication::FrequentHeartbeatRes res{true};
   slk::Save(res, res_builder);
 }
 
-void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
+void Shard::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
   replication::AppendDeltasReq req;
   slk::Load(&req, req_reader);
 
@@ -107,25 +107,25 @@ void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, sl
   auto maybe_epoch_id = decoder.ReadString();
   MG_ASSERT(maybe_epoch_id, "Invalid replication message");
 
-  if (*maybe_epoch_id != storage_->epoch_id_) {
-    storage_->epoch_history_.emplace_back(std::move(storage_->epoch_id_), storage_->last_commit_timestamp_);
-    storage_->epoch_id_ = std::move(*maybe_epoch_id);
+  if (*maybe_epoch_id != shard_->epoch_id_) {
+    shard_->epoch_history_.emplace_back(std::move(shard_->epoch_id_), shard_->last_commit_timestamp_);
+    shard_->epoch_id_ = std::move(*maybe_epoch_id);
   }
 
-  if (storage_->wal_file_) {
-    if (req.seq_num > storage_->wal_file_->SequenceNumber() || *maybe_epoch_id != storage_->epoch_id_) {
-      storage_->wal_file_->FinalizeWal();
-      storage_->wal_file_.reset();
-      storage_->wal_seq_num_ = req.seq_num;
+  if (shard_->wal_file_) {
+    if (req.seq_num > shard_->wal_file_->SequenceNumber() || *maybe_epoch_id != shard_->epoch_id_) {
+      shard_->wal_file_->FinalizeWal();
+      shard_->wal_file_.reset();
+      shard_->wal_seq_num_ = req.seq_num;
     } else {
-      MG_ASSERT(storage_->wal_file_->SequenceNumber() == req.seq_num, "Invalid sequence number of current wal file");
-      storage_->wal_seq_num_ = req.seq_num + 1;
+      MG_ASSERT(shard_->wal_file_->SequenceNumber() == req.seq_num, "Invalid sequence number of current wal file");
+      shard_->wal_seq_num_ = req.seq_num + 1;
     }
   } else {
-    storage_->wal_seq_num_ = req.seq_num;
+    shard_->wal_seq_num_ = req.seq_num;
   }
 
-  if (req.previous_commit_timestamp != storage_->last_commit_timestamp_) {
+  if (req.previous_commit_timestamp != shard_->last_commit_timestamp_) {
     // Empty the stream
     bool transaction_complete = false;
     while (!transaction_complete) {
@@ -134,82 +134,82 @@ void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, sl
       transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
     }
 
-    replication::AppendDeltasRes res{false, storage_->last_commit_timestamp_};
+    replication::AppendDeltasRes res{false, shard_->last_commit_timestamp_};
     slk::Save(res, res_builder);
     return;
   }
 
   ReadAndApplyDelta(&decoder);
 
-  replication::AppendDeltasRes res{true, storage_->last_commit_timestamp_};
+  replication::AppendDeltasRes res{true, shard_->last_commit_timestamp_};
   slk::Save(res, res_builder);
 }
 
-void Storage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
+void Shard::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
   replication::SnapshotReq req;
   slk::Load(&req, req_reader);
 
   replication::Decoder decoder(req_reader);
 
-  utils::EnsureDirOrDie(storage_->snapshot_directory_);
+  utils::EnsureDirOrDie(shard_->snapshot_directory_);
 
-  const auto maybe_snapshot_path = decoder.ReadFile(storage_->snapshot_directory_);
+  const auto maybe_snapshot_path = decoder.ReadFile(shard_->snapshot_directory_);
   MG_ASSERT(maybe_snapshot_path, "Failed to load snapshot!");
   spdlog::info("Received snapshot saved to {}", *maybe_snapshot_path);
 
   // Clear the database
-  storage_->vertices_.clear();
-  storage_->edges_.clear();
+  shard_->vertices_.clear();
+  shard_->edges_.clear();
 
-  storage_->constraints_ = Constraints();
-  storage_->indices_.label_index =
-      LabelIndex(&storage_->indices_, &storage_->constraints_, storage_->config_.items, storage_->schema_validator_);
-  storage_->indices_.label_property_index = LabelPropertyIndex(&storage_->indices_, &storage_->constraints_,
-                                                               storage_->config_.items, storage_->schema_validator_);
+  shard_->constraints_ = Constraints();
+  shard_->indices_.label_index =
+      LabelIndex(&shard_->indices_, &shard_->constraints_, shard_->config_.items, shard_->schema_validator_);
+  shard_->indices_.label_property_index =
+      LabelPropertyIndex(&shard_->indices_, &shard_->constraints_, shard_->config_.items, shard_->schema_validator_);
   try {
     spdlog::debug("Loading snapshot");
     auto recovered_snapshot = durability::RecoveredSnapshot{};
 
-    // durability::LoadSnapshot(*maybe_snapshot_path, &storage_->vertices_, &storage_->edges_,
-    // &storage_->epoch_history_,
-    //                          &storage_->name_id_mapper_, &storage_->edge_count_, storage_->config_.items);
+    // durability::LoadSnapshot(*maybe_snapshot_path, &shard_->vertices_, &shard_->edges_,
+    // &shard_->epoch_history_,
+    //                          &shard_->name_id_mapper_, &shard_->edge_count_, shard_->config_.items);
     spdlog::debug("Snapshot loaded successfully");
     // If this step is present it should always be the first step of
     // the recovery so we use the UUID we read from snasphost
-    storage_->uuid_ = std::move(recovered_snapshot.snapshot_info.uuid);
-    storage_->epoch_id_ = std::move(recovered_snapshot.snapshot_info.epoch_id);
+    shard_->uuid_ = std::move(recovered_snapshot.snapshot_info.uuid);
+    shard_->epoch_id_ = std::move(recovered_snapshot.snapshot_info.epoch_id);
     const auto &recovery_info = recovered_snapshot.recovery_info;
-    storage_->edge_id_ = recovery_info.next_edge_id;
-    storage_->timestamp_ = std::max(storage_->timestamp_, recovery_info.next_timestamp);
+    shard_->edge_id_ = recovery_info.next_edge_id;
+    shard_->timestamp_ = std::max(shard_->timestamp_, recovery_info.next_timestamp);
 
-    // durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &storage_->indices_,
-    //                                          &storage_->constraints_, &storage_->vertices_);
+    // durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &shard_->indices_,
+    //                                          &shard_->constraints_, &shard_->vertices_);
   } catch (const durability::RecoveryFailure &e) {
     LOG_FATAL("Couldn't load the snapshot because of: {}", e.what());
   }
 
-  replication::SnapshotRes res{true, storage_->last_commit_timestamp_};
+  replication::SnapshotRes res{true, shard_->last_commit_timestamp_};
   slk::Save(res, res_builder);
 
   // Delete other durability files
-  auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
+  auto snapshot_files = durability::GetSnapshotFiles(shard_->snapshot_directory_, shard_->uuid_);
   for (const auto &[path, uuid, _] : snapshot_files) {
     if (path != *maybe_snapshot_path) {
-      storage_->file_retainer_.DeleteFile(path);
+      shard_->file_retainer_.DeleteFile(path);
     }
   }
 
-  auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_);
+  auto wal_files = durability::GetWalFiles(shard_->wal_directory_, shard_->uuid_);
   if (wal_files) {
     for (const auto &wal_file : *wal_files) {
-      storage_->file_retainer_.DeleteFile(wal_file.path);
+      shard_->file_retainer_.DeleteFile(wal_file.path);
     }
 
-    storage_->wal_file_.reset();
+    shard_->wal_file_.reset();
   }
 }
 
-void Storage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
+void Shard::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
   replication::WalFilesReq req;
   slk::Load(&req, req_reader);
 
@@ -218,31 +218,31 @@ void Storage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::B
 
   replication::Decoder decoder(req_reader);
 
-  utils::EnsureDirOrDie(storage_->wal_directory_);
+  utils::EnsureDirOrDie(shard_->wal_directory_);
 
   for (auto i = 0; i < wal_file_number; ++i) {
     LoadWal(&decoder);
   }
 
-  replication::WalFilesRes res{true, storage_->last_commit_timestamp_};
+  replication::WalFilesRes res{true, shard_->last_commit_timestamp_};
   slk::Save(res, res_builder);
 }
 
-void Storage::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
+void Shard::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
   replication::CurrentWalReq req;
   slk::Load(&req, req_reader);
 
   replication::Decoder decoder(req_reader);
 
-  utils::EnsureDirOrDie(storage_->wal_directory_);
+  utils::EnsureDirOrDie(shard_->wal_directory_);
 
   LoadWal(&decoder);
 
-  replication::CurrentWalRes res{true, storage_->last_commit_timestamp_};
+  replication::CurrentWalRes res{true, shard_->last_commit_timestamp_};
   slk::Save(res, res_builder);
 }
 
-void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) {
+void Shard::ReplicationServer::LoadWal(replication::Decoder *decoder) {
   const auto temp_wal_directory = std::filesystem::temp_directory_path() / "memgraph" / durability::kWalDirectory;
   utils::EnsureDir(temp_wal_directory);
   auto maybe_wal_path = decoder->ReadFile(temp_wal_directory);
@@ -251,22 +251,22 @@ void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) {
   try {
     auto wal_info = durability::ReadWalInfo(*maybe_wal_path);
     if (wal_info.seq_num == 0) {
-      storage_->uuid_ = wal_info.uuid;
+      shard_->uuid_ = wal_info.uuid;
     }
 
-    if (wal_info.epoch_id != storage_->epoch_id_) {
-      storage_->epoch_history_.emplace_back(wal_info.epoch_id, storage_->last_commit_timestamp_);
-      storage_->epoch_id_ = std::move(wal_info.epoch_id);
+    if (wal_info.epoch_id != shard_->epoch_id_) {
+      shard_->epoch_history_.emplace_back(wal_info.epoch_id, shard_->last_commit_timestamp_);
+      shard_->epoch_id_ = std::move(wal_info.epoch_id);
     }
 
-    if (storage_->wal_file_) {
-      if (storage_->wal_file_->SequenceNumber() != wal_info.seq_num) {
-        storage_->wal_file_->FinalizeWal();
-        storage_->wal_seq_num_ = wal_info.seq_num;
-        storage_->wal_file_.reset();
+    if (shard_->wal_file_) {
+      if (shard_->wal_file_->SequenceNumber() != wal_info.seq_num) {
+        shard_->wal_file_->FinalizeWal();
+        shard_->wal_seq_num_ = wal_info.seq_num;
+        shard_->wal_file_.reset();
       }
     } else {
-      storage_->wal_seq_num_ = wal_info.seq_num;
+      shard_->wal_seq_num_ = wal_info.seq_num;
     }
 
     durability::Decoder wal;
@@ -285,20 +285,20 @@ void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) {
   }
 }
 
-Storage::ReplicationServer::~ReplicationServer() {
+Shard::ReplicationServer::~ReplicationServer() {
   if (rpc_server_) {
     rpc_server_->Shutdown();
     rpc_server_->AwaitShutdown();
   }
 }
-uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) {
-  auto edge_acc = storage_->edges_.access();
-  // auto vertex_acc = storage_->vertices_.access();
+uint64_t Shard::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) {
+  auto edge_acc = shard_->edges_.access();
+  // auto vertex_acc = shard_->vertices_.access();
 
-  std::optional<std::pair<uint64_t, Storage::Accessor>> commit_timestamp_and_accessor;
+  std::optional<std::pair<uint64_t, Shard::Accessor>> commit_timestamp_and_accessor;
   // auto get_transaction = [this, &commit_timestamp_and_accessor](uint64_t commit_timestamp) {
   //   if (!commit_timestamp_and_accessor) {
-  //     commit_timestamp_and_accessor.emplace(commit_timestamp, storage_->Access());
+  //     commit_timestamp_and_accessor.emplace(commit_timestamp, shard_->Access());
   //   } else if (commit_timestamp_and_accessor->first != commit_timestamp) {
   //     throw utils::BasicException("Received more than one transaction!");
   //   }
@@ -306,7 +306,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
   // };
 
   uint64_t applied_deltas = 0;
-  auto max_commit_timestamp = storage_->last_commit_timestamp_;
+  auto max_commit_timestamp = shard_->last_commit_timestamp_;
 
   for (bool transaction_complete = false; !transaction_complete; ++applied_deltas) {
     const auto [timestamp, delta] = ReadDelta(decoder);
@@ -316,7 +316,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
 
     transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
 
-    if (timestamp < storage_->timestamp_) {
+    if (timestamp < shard_->timestamp_) {
       continue;
     }
 
@@ -406,12 +406,12 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       spdlog::trace("       Edge {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(),
     //                     delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value);
 
-    //       if (!storage_->config_.items.properties_on_edges)
+    //       if (!shard_->config_.items.properties_on_edges)
     //         throw utils::BasicException(
     //             "Can't set properties on edges because properties on edges "
     //             "are disabled!");
 
-    //       auto *transaction = get_transaction(timestamp);
+    //   //       auto *transaction = get_transaction(timestamp);
 
     //       // The following block of code effectively implements `FindEdge` and
     //       // yields an accessor that is only valid for managing the edge's
@@ -456,10 +456,10 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //                              nullptr,
     //                              nullptr,
     //                              &transaction->transaction_,
-    //                              &storage_->indices_,
-    //                              &storage_->constraints_,
-    //                              storage_->config_.items,
-    //                              storage_->schema_validator_};
+    //                              &shard_->indices_,
+    //                              &shard_->constraints_,
+    //                              shard_->config_.items,
+    //                              shard_->schema_validator_};
 
     //       auto ret = ea.SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property),
     //                                 delta.vertex_edge_set_property.value);
@@ -481,14 +481,14 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       spdlog::trace("       Create label index on :{}", delta.operation_label.label);
     //       // Need to send the timestamp
     //       if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
-    //       if (!storage_->CreateIndex(storage_->NameToLabel(delta.operation_label.label), timestamp))
+    //       if (!shard_->CreateIndex(shard_->NameToLabel(delta.operation_label.label), timestamp))
     //         throw utils::BasicException("Invalid transaction!");
     //       break;
     //     }
     //     case durability::WalDeltaData::Type::LABEL_INDEX_DROP: {
     //       spdlog::trace("       Drop label index on :{}", delta.operation_label.label);
     //       if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
-    //       if (!storage_->DropIndex(storage_->NameToLabel(delta.operation_label.label), timestamp))
+    //       if (!shard_->DropIndex(shard_->NameToLabel(delta.operation_label.label), timestamp))
     //         throw utils::BasicException("Invalid transaction!");
     //       break;
     //     }
@@ -496,8 +496,8 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       spdlog::trace("       Create label+property index on :{} ({})", delta.operation_label_property.label,
     //                     delta.operation_label_property.property);
     //       if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
-    //       if (!storage_->CreateIndex(storage_->NameToLabel(delta.operation_label_property.label),
-    //                                  storage_->NameToProperty(delta.operation_label_property.property), timestamp))
+    //       if (!shard_->CreateIndex(shard_->NameToLabel(delta.operation_label_property.label),
+    //                                  shard_->NameToProperty(delta.operation_label_property.property), timestamp))
     //         throw utils::BasicException("Invalid transaction!");
     //       break;
     //     }
@@ -505,8 +505,8 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       spdlog::trace("       Drop label+property index on :{} ({})", delta.operation_label_property.label,
     //                     delta.operation_label_property.property);
     //       if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
-    //       if (!storage_->DropIndex(storage_->NameToLabel(delta.operation_label_property.label),
-    //                                storage_->NameToProperty(delta.operation_label_property.property), timestamp))
+    //       if (!shard_->DropIndex(shard_->NameToLabel(delta.operation_label_property.label),
+    //                                shard_->NameToProperty(delta.operation_label_property.property), timestamp))
     //         throw utils::BasicException("Invalid transaction!");
     //       break;
     //     }
@@ -514,9 +514,9 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       spdlog::trace("       Create existence constraint on :{} ({})", delta.operation_label_property.label,
     //                     delta.operation_label_property.property);
     //       if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
-    //       auto ret = storage_->CreateExistenceConstraint(
-    //           storage_->NameToLabel(delta.operation_label_property.label),
-    //           storage_->NameToProperty(delta.operation_label_property.property), timestamp);
+    //       auto ret = shard_->CreateExistenceConstraint(
+    //           shard_->NameToLabel(delta.operation_label_property.label),
+    //           shard_->NameToProperty(delta.operation_label_property.property), timestamp);
     //       if (!ret.HasValue() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
     //       break;
     //     }
@@ -524,8 +524,8 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       spdlog::trace("       Drop existence constraint on :{} ({})", delta.operation_label_property.label,
     //                     delta.operation_label_property.property);
     //       if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
-    //       if (!storage_->DropExistenceConstraint(storage_->NameToLabel(delta.operation_label_property.label),
-    //                                              storage_->NameToProperty(delta.operation_label_property.property),
+    //       if (!shard_->DropExistenceConstraint(shard_->NameToLabel(delta.operation_label_property.label),
+    //                                              shard_->NameToProperty(delta.operation_label_property.property),
     //                                              timestamp))
     //         throw utils::BasicException("Invalid transaction!");
     //       break;
@@ -537,9 +537,9 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       ss.str()); if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
     //       std::set<PropertyId> properties;
     //       for (const auto &prop : delta.operation_label_properties.properties) {
-    //         properties.emplace(storage_->NameToProperty(prop));
+    //         properties.emplace(shard_->NameToProperty(prop));
     //       }
-    //       auto ret = storage_->CreateUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label),
+    //       auto ret = shard_->CreateUniqueConstraint(shard_->NameToLabel(delta.operation_label_properties.label),
     //                                                   properties, timestamp);
     //       if (!ret.HasValue() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS)
     //         throw utils::BasicException("Invalid transaction!");
@@ -552,9 +552,9 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
     //       ss.str()); if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
     //       std::set<PropertyId> properties;
     //       for (const auto &prop : delta.operation_label_properties.properties) {
-    //         properties.emplace(storage_->NameToProperty(prop));
+    //         properties.emplace(shard_->NameToProperty(prop));
     //       }
-    //       auto ret = storage_->DropUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label),
+    //       auto ret = shard_->DropUniqueConstraint(shard_->NameToLabel(delta.operation_label_properties.label),
     //                                                 properties, timestamp);
     //       if (ret != UniqueConstraints::DeletionStatus::SUCCESS) throw utils::BasicException("Invalid transaction!");
     //       break;
@@ -564,7 +564,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
 
   if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid data!");
 
-  storage_->last_commit_timestamp_ = max_commit_timestamp;
+  shard_->last_commit_timestamp_ = max_commit_timestamp;
 
   return applied_deltas;
 }
diff --git a/src/storage/v3/replication/replication_server.hpp b/src/storage/v3/replication/replication_server.hpp
index 6d6cc256f..6b278a103 100644
--- a/src/storage/v3/replication/replication_server.hpp
+++ b/src/storage/v3/replication/replication_server.hpp
@@ -11,13 +11,13 @@
 
 #pragma once
 
-#include "storage/v3/storage.hpp"
+#include "storage/v3/shard.hpp"
 
 namespace memgraph::storage::v3 {
 
-class Storage::ReplicationServer {
+class Shard::ReplicationServer {
  public:
-  explicit ReplicationServer(Storage *storage, io::network::Endpoint endpoint,
+  explicit ReplicationServer(Shard *shard, io::network::Endpoint endpoint,
                              const replication::ReplicationServerConfig &config);
   ReplicationServer(const ReplicationServer &) = delete;
   ReplicationServer(ReplicationServer &&) = delete;
@@ -41,7 +41,7 @@ class Storage::ReplicationServer {
   std::optional<communication::ServerContext> rpc_server_context_;
   std::optional<rpc::Server> rpc_server_;
 
-  Storage *storage_;
+  Shard *shard_;
 };
 
 }  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/shard.cpp b/src/storage/v3/shard.cpp
new file mode 100644
index 000000000..48040a80c
--- /dev/null
+++ b/src/storage/v3/shard.cpp
@@ -0,0 +1,1849 @@
+// Copyright 2022 Memgraph Ltd.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
+// License, and you may not use this file except in compliance with the Business Source License.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+#include "storage/v3/shard.hpp"
+
+#include <algorithm>
+#include <atomic>
+#include <cstdint>
+#include <iterator>
+#include <memory>
+#include <mutex>
+#include <optional>
+#include <variant>
+
+#include <bits/ranges_algo.h>
+#include <gflags/gflags.h>
+#include <spdlog/spdlog.h>
+
+#include "io/network/endpoint.hpp"
+#include "storage/v3/constraints.hpp"
+#include "storage/v3/durability/durability.hpp"
+#include "storage/v3/durability/metadata.hpp"
+#include "storage/v3/durability/paths.hpp"
+#include "storage/v3/durability/snapshot.hpp"
+#include "storage/v3/durability/wal.hpp"
+#include "storage/v3/edge_accessor.hpp"
+#include "storage/v3/id_types.hpp"
+#include "storage/v3/indices.hpp"
+#include "storage/v3/key_store.hpp"
+#include "storage/v3/mvcc.hpp"
+#include "storage/v3/property_value.hpp"
+#include "storage/v3/replication/config.hpp"
+#include "storage/v3/replication/replication_client.hpp"
+#include "storage/v3/replication/replication_server.hpp"
+#include "storage/v3/replication/rpc.hpp"
+#include "storage/v3/schema_validator.hpp"
+#include "storage/v3/transaction.hpp"
+#include "storage/v3/vertex.hpp"
+#include "storage/v3/vertex_accessor.hpp"
+#include "storage/v3/vertices_skip_list.hpp"
+#include "utils/exceptions.hpp"
+#include "utils/file.hpp"
+#include "utils/logging.hpp"
+#include "utils/memory_tracker.hpp"
+#include "utils/message.hpp"
+#include "utils/result.hpp"
+#include "utils/rw_lock.hpp"
+#include "utils/skip_list.hpp"
+#include "utils/spin_lock.hpp"
+#include "utils/stat.hpp"
+#include "utils/uuid.hpp"
+
+namespace memgraph::storage::v3 {
+
+using OOMExceptionEnabler = utils::MemoryTracker::OutOfMemoryExceptionEnabler;
+
+namespace {
+inline constexpr uint16_t kEpochHistoryRetention = 1000;
+
+void InsertVertexPKIntoList(auto &container, const PrimaryKey &primary_key) { container.push_back(primary_key); }
+}  // namespace
+
+auto AdvanceToVisibleVertex(VerticesSkipList::Iterator it, VerticesSkipList::Iterator end,
+                            std::optional<VertexAccessor> *vertex, Transaction *tx, View view, Indices *indices,
+                            Constraints *constraints, Config::Items config, const SchemaValidator &schema_validator) {
+  while (it != end) {
+    *vertex = VertexAccessor::Create(&it->vertex, tx, indices, constraints, config, schema_validator, view);
+    if (!*vertex) {
+      ++it;
+      continue;
+    }
+    break;
+  }
+  return it;
+}
+
+AllVerticesIterable::Iterator::Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it)
+    : self_(self),
+      it_(AdvanceToVisibleVertex(it, self->vertices_accessor_.end(), &self->vertex_, self->transaction_, self->view_,
+                                 self->indices_, self_->constraints_, self->config_, *self_->schema_validator_)) {}
+
+VertexAccessor AllVerticesIterable::Iterator::operator*() const { return *self_->vertex_; }
+
+AllVerticesIterable::Iterator &AllVerticesIterable::Iterator::operator++() {
+  ++it_;
+  it_ = AdvanceToVisibleVertex(it_, self_->vertices_accessor_.end(), &self_->vertex_, self_->transaction_, self_->view_,
+                               self_->indices_, self_->constraints_, self_->config_, *self_->schema_validator_);
+  return *this;
+}
+
+VerticesIterable::VerticesIterable(AllVerticesIterable vertices) : type_(Type::ALL) {
+  new (&all_vertices_) AllVerticesIterable(std::move(vertices));
+}
+
+VerticesIterable::VerticesIterable(LabelIndex::Iterable vertices) : type_(Type::BY_LABEL) {
+  new (&vertices_by_label_) LabelIndex::Iterable(std::move(vertices));
+}
+
+VerticesIterable::VerticesIterable(LabelPropertyIndex::Iterable vertices) : type_(Type::BY_LABEL_PROPERTY) {
+  new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(vertices));
+}
+
+VerticesIterable::VerticesIterable(VerticesIterable &&other) noexcept : type_(other.type_) {
+  switch (other.type_) {
+    case Type::ALL:
+      new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_));
+      break;
+    case Type::BY_LABEL:
+      new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_));
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_));
+      break;
+  }
+}
+
+VerticesIterable &VerticesIterable::operator=(VerticesIterable &&other) noexcept {
+  switch (type_) {
+    case Type::ALL:
+      all_vertices_.AllVerticesIterable::~AllVerticesIterable();
+      break;
+    case Type::BY_LABEL:
+      vertices_by_label_.LabelIndex::Iterable::~Iterable();
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable();
+      break;
+  }
+  type_ = other.type_;
+  switch (other.type_) {
+    case Type::ALL:
+      new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_));
+      break;
+    case Type::BY_LABEL:
+      new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_));
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_));
+      break;
+  }
+  return *this;
+}
+
+VerticesIterable::~VerticesIterable() {
+  switch (type_) {
+    case Type::ALL:
+      all_vertices_.AllVerticesIterable::~AllVerticesIterable();
+      break;
+    case Type::BY_LABEL:
+      vertices_by_label_.LabelIndex::Iterable::~Iterable();
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable();
+      break;
+  }
+}
+
+VerticesIterable::Iterator VerticesIterable::begin() {
+  switch (type_) {
+    case Type::ALL:
+      return Iterator(all_vertices_.begin());
+    case Type::BY_LABEL:
+      return Iterator(vertices_by_label_.begin());
+    case Type::BY_LABEL_PROPERTY:
+      return Iterator(vertices_by_label_property_.begin());
+  }
+}
+
+VerticesIterable::Iterator VerticesIterable::end() {
+  switch (type_) {
+    case Type::ALL:
+      return Iterator(all_vertices_.end());
+    case Type::BY_LABEL:
+      return Iterator(vertices_by_label_.end());
+    case Type::BY_LABEL_PROPERTY:
+      return Iterator(vertices_by_label_property_.end());
+  }
+}
+
+VerticesIterable::Iterator::Iterator(AllVerticesIterable::Iterator it) : type_(Type::ALL) {
+  new (&all_it_) AllVerticesIterable::Iterator(it);
+}
+
+VerticesIterable::Iterator::Iterator(LabelIndex::Iterable::Iterator it) : type_(Type::BY_LABEL) {
+  new (&by_label_it_) LabelIndex::Iterable::Iterator(it);
+}
+
+VerticesIterable::Iterator::Iterator(LabelPropertyIndex::Iterable::Iterator it) : type_(Type::BY_LABEL_PROPERTY) {
+  new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(it);
+}
+
+VerticesIterable::Iterator::Iterator(const VerticesIterable::Iterator &other) : type_(other.type_) {
+  switch (other.type_) {
+    case Type::ALL:
+      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
+      break;
+    case Type::BY_LABEL:
+      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
+      break;
+  }
+}
+
+VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(const VerticesIterable::Iterator &other) {
+  if (this == &other) {
+    return *this;
+  }
+  Destroy();
+  type_ = other.type_;
+  switch (other.type_) {
+    case Type::ALL:
+      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
+      break;
+    case Type::BY_LABEL:
+      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
+      break;
+  }
+  return *this;
+}
+
+VerticesIterable::Iterator::Iterator(VerticesIterable::Iterator &&other) noexcept : type_(other.type_) {
+  switch (other.type_) {
+    case Type::ALL:
+      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
+      break;
+    case Type::BY_LABEL:
+      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
+      break;
+  }
+}
+
+VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(VerticesIterable::Iterator &&other) noexcept {
+  Destroy();
+  type_ = other.type_;
+  switch (other.type_) {
+    case Type::ALL:
+      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
+      break;
+    case Type::BY_LABEL:
+      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
+      break;
+  }
+  return *this;
+}
+
+VerticesIterable::Iterator::~Iterator() { Destroy(); }
+
+void VerticesIterable::Iterator::Destroy() noexcept {
+  switch (type_) {
+    case Type::ALL:
+      all_it_.AllVerticesIterable::Iterator::~Iterator();
+      break;
+    case Type::BY_LABEL:
+      by_label_it_.LabelIndex::Iterable::Iterator::~Iterator();
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      by_label_property_it_.LabelPropertyIndex::Iterable::Iterator::~Iterator();
+      break;
+  }
+}
+
+VertexAccessor VerticesIterable::Iterator::operator*() const {
+  switch (type_) {
+    case Type::ALL:
+      return *all_it_;
+    case Type::BY_LABEL:
+      return *by_label_it_;
+    case Type::BY_LABEL_PROPERTY:
+      return *by_label_property_it_;
+  }
+}
+
+VerticesIterable::Iterator &VerticesIterable::Iterator::operator++() {
+  switch (type_) {
+    case Type::ALL:
+      ++all_it_;
+      break;
+    case Type::BY_LABEL:
+      ++by_label_it_;
+      break;
+    case Type::BY_LABEL_PROPERTY:
+      ++by_label_property_it_;
+      break;
+  }
+  return *this;
+}
+
+bool VerticesIterable::Iterator::operator==(const Iterator &other) const {
+  switch (type_) {
+    case Type::ALL:
+      return all_it_ == other.all_it_;
+    case Type::BY_LABEL:
+      return by_label_it_ == other.by_label_it_;
+    case Type::BY_LABEL_PROPERTY:
+      return by_label_property_it_ == other.by_label_property_it_;
+  }
+}
+
+Shard::Shard(const LabelId primary_label, const PrimaryKey min_primary_key,
+             const std::optional<PrimaryKey> max_primary_key, Config config)
+    : primary_label_{primary_label},
+      min_primary_key_{min_primary_key},
+      max_primary_key_{max_primary_key},
+      schema_validator_{schemas_},
+      indices_{&constraints_, config.items, schema_validator_},
+      isolation_level_{config.transaction.isolation_level},
+      config_{config},
+      snapshot_directory_{config_.durability.storage_directory / durability::kSnapshotDirectory},
+      wal_directory_{config_.durability.storage_directory / durability::kWalDirectory},
+      lock_file_path_{config_.durability.storage_directory / durability::kLockFile},
+      uuid_{utils::GenerateUUID()},
+      epoch_id_{utils::GenerateUUID()},
+      global_locker_{file_retainer_.AddLocker()} {
+  if (config_.durability.snapshot_wal_mode == Config::Durability::SnapshotWalMode::DISABLED &&
+      replication_role_ == ReplicationRole::MAIN) {
+    spdlog::warn(
+        "The instance has the MAIN replication role, but durability logs and snapshots are disabled. Please consider "
+        "enabling durability by using --storage-snapshot-interval-sec and --storage-wal-enabled flags because "
+        "without write-ahead logs this instance is not replicating any data.");
+  }
+  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED ||
+      config_.durability.snapshot_on_exit || config_.durability.recover_on_startup) {
+    // Create the directory initially to crash the database in case of
+    // permission errors. This is done early to crash the database on startup
+    // instead of crashing the database for the first time during runtime (which
+    // could be an unpleasant surprise).
+    utils::EnsureDirOrDie(snapshot_directory_);
+    // Same reasoning as above.
+    utils::EnsureDirOrDie(wal_directory_);
+
+    // Verify that the user that started the process is the same user that is
+    // the owner of the storage directory.
+    durability::VerifyStorageDirectoryOwnerAndProcessUserOrDie(config_.durability.storage_directory);
+
+    // Create the lock file and open a handle to it. This will crash the
+    // database if it can't open the file for writing or if any other process is
+    // holding the file opened.
+    lock_file_handle_.Open(lock_file_path_, utils::OutputFile::Mode::OVERWRITE_EXISTING);
+    MG_ASSERT(lock_file_handle_.AcquireLock(),
+              "Couldn't acquire lock on the storage directory {}!\n"
+              "Another Memgraph process is currently running with the same "
+              "storage directory, please stop it first before starting this "
+              "process!",
+              config_.durability.storage_directory);
+  }
+  if (config_.durability.recover_on_startup) {
+    auto info = std::optional<durability::RecoveryInfo>{};
+
+    // durability::RecoverData(snapshot_directory_, wal_directory_, &uuid_, &epoch_id_, &epoch_history_, &vertices_,
+    //                         &edges_, &edge_count_, &name_id_mapper_, &indices_, &constraints_, config_.items,
+    //                         &wal_seq_num_);
+    if (info) {
+      edge_id_ = info->next_edge_id;
+      timestamp_ = std::max(timestamp_, info->next_timestamp);
+      if (info->last_commit_timestamp) {
+        last_commit_timestamp_ = *info->last_commit_timestamp;
+      }
+    }
+  } else if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED ||
+             config_.durability.snapshot_on_exit) {
+    bool files_moved = false;
+    auto backup_root = config_.durability.storage_directory / durability::kBackupDirectory;
+    for (const auto &[path, dirname, what] :
+         {std::make_tuple(snapshot_directory_, durability::kSnapshotDirectory, "snapshot"),
+          std::make_tuple(wal_directory_, durability::kWalDirectory, "WAL")}) {
+      if (!utils::DirExists(path)) continue;
+      auto backup_curr = backup_root / dirname;
+      std::error_code error_code;
+      for (const auto &item : std::filesystem::directory_iterator(path, error_code)) {
+        utils::EnsureDirOrDie(backup_root);
+        utils::EnsureDirOrDie(backup_curr);
+        std::error_code item_error_code;
+        std::filesystem::rename(item.path(), backup_curr / item.path().filename(), item_error_code);
+        MG_ASSERT(!item_error_code, "Couldn't move {} file {} because of: {}", what, item.path(),
+                  item_error_code.message());
+        files_moved = true;
+      }
+      MG_ASSERT(!error_code, "Couldn't backup {} files because of: {}", what, error_code.message());
+    }
+    if (files_moved) {
+      spdlog::warn(
+          "Since Memgraph was not supposed to recover on startup and "
+          "durability is enabled, your current durability files will likely "
+          "be overridden. To prevent important data loss, Memgraph has stored "
+          "those files into a .backup directory inside the storage directory.");
+    }
+  }
+  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) {
+    // TODO(antaljanosbenjamin): handle snapshots
+    // snapshot_runner_.Run("Snapshot", config_.durability.snapshot_interval, [this] {
+    //   if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) {
+    //     switch (maybe_error.GetError()) {
+    //       case CreateSnapshotError::DisabledForReplica:
+    //         spdlog::warn(
+    //             utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication"));
+    //         break;
+    //     }
+    //   }
+    // });
+  }
+
+  if (timestamp_ == kTimestampInitialId) {
+    commit_log_.emplace();
+  } else {
+    commit_log_.emplace(timestamp_);
+  }
+}
+
+Shard::~Shard() {
+  {
+    // Clear replication data
+    replication_server_.reset();
+    replication_clients_.WithLock([&](auto &clients) { clients.clear(); });
+  }
+  if (wal_file_) {
+    wal_file_->FinalizeWal();
+    wal_file_ = std::nullopt;
+  }
+  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) {
+    // TODO(antaljanosbenjamin): stop snapshot creation
+  }
+  if (config_.durability.snapshot_on_exit) {
+    if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) {
+      switch (maybe_error.GetError()) {
+        case CreateSnapshotError::DisabledForReplica:
+          spdlog::warn(utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication"));
+          break;
+      }
+    }
+  }
+}
+
+Shard::Accessor::Accessor(Shard *shard, IsolationLevel isolation_level)
+    : shard_(shard),
+      transaction_(shard->CreateTransaction(isolation_level)),
+      is_transaction_active_(true),
+      config_(shard->config_.items) {}
+
+Shard::Accessor::Accessor(Accessor &&other) noexcept
+    : shard_(other.shard_),
+      transaction_(std::move(other.transaction_)),
+      commit_timestamp_(other.commit_timestamp_),
+      is_transaction_active_(other.is_transaction_active_),
+      config_(other.config_) {
+  // Don't allow the other accessor to abort our transaction in destructor.
+  other.is_transaction_active_ = false;
+  other.commit_timestamp_.reset();
+}
+
+Shard::Accessor::~Accessor() {
+  if (is_transaction_active_) {
+    Abort();
+  }
+
+  FinalizeTransaction();
+}
+
+ResultSchema<VertexAccessor> Shard::Accessor::CreateVertexAndValidate(
+    LabelId primary_label, const std::vector<LabelId> &labels,
+    const std::vector<std::pair<PropertyId, PropertyValue>> &properties) {
+  if (primary_label != shard_->primary_label_) {
+    throw utils::BasicException("Cannot add vertex to shard which does not hold the given primary label!");
+  }
+  auto maybe_schema_violation = GetSchemaValidator().ValidateVertexCreate(primary_label, labels, properties);
+  if (maybe_schema_violation) {
+    return {std::move(*maybe_schema_violation)};
+  }
+  OOMExceptionEnabler oom_exception;
+  // Extract key properties
+  std::vector<PropertyValue> primary_properties;
+  for ([[maybe_unused]] const auto &[property_id, property_type] : shard_->GetSchema(primary_label)->second) {
+    // We know there definitely is key in properties since we have validated
+    primary_properties.push_back(
+        std::ranges::find_if(properties, [property_id = property_id](const auto &property_pair) {
+          return property_pair.first == property_id;
+        })->second);
+  }
+
+  // Get secondary properties
+  std::vector<std::pair<PropertyId, PropertyValue>> secondary_properties;
+  for (const auto &[property_id, property_value] : properties) {
+    if (!shard_->schemas_.IsPropertyKey(primary_label, property_id)) {
+      secondary_properties.emplace_back(property_id, property_value);
+    }
+  }
+
+  auto acc = shard_->vertices_.access();
+  auto *delta = CreateDeleteObjectDelta(&transaction_);
+  auto [it, inserted] = acc.insert({Vertex{delta, primary_label, primary_properties, labels, secondary_properties}});
+  MG_ASSERT(inserted, "The vertex must be inserted here!");
+  MG_ASSERT(it != acc.end(), "Invalid Vertex accessor!");
+  delta->prev.Set(&it->vertex);
+  return VertexAccessor{&it->vertex,           &transaction_, &shard_->indices_,
+                        &shard_->constraints_, config_,       shard_->schema_validator_};
+}
+
+std::optional<VertexAccessor> Shard::Accessor::FindVertex(std::vector<PropertyValue> primary_key, View view) {
+  auto acc = shard_->vertices_.access();
+  // Later on use label space
+  auto it = acc.find(primary_key);
+  if (it == acc.end()) {
+    return std::nullopt;
+  }
+  return VertexAccessor::Create(&it->vertex, &transaction_, &shard_->indices_, &shard_->constraints_, config_,
+                                shard_->schema_validator_, view);
+}
+
+Result<std::optional<VertexAccessor>> Shard::Accessor::DeleteVertex(VertexAccessor *vertex) {
+  MG_ASSERT(vertex->transaction_ == &transaction_,
+            "VertexAccessor must be from the same transaction as the storage "
+            "accessor when deleting a vertex!");
+  auto *vertex_ptr = vertex->vertex_;
+
+  if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
+
+  if (vertex_ptr->deleted) {
+    return std::optional<VertexAccessor>{};
+  }
+
+  if (!vertex_ptr->in_edges.empty() || !vertex_ptr->out_edges.empty()) return Error::VERTEX_HAS_EDGES;
+
+  CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag());
+  vertex_ptr->deleted = true;
+
+  return std::make_optional<VertexAccessor>(vertex_ptr, &transaction_, &shard_->indices_, &shard_->constraints_,
+                                            config_, shard_->schema_validator_, true);
+}
+
+Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shard::Accessor::DetachDeleteVertex(
+    VertexAccessor *vertex) {
+  using ReturnType = std::pair<VertexAccessor, std::vector<EdgeAccessor>>;
+
+  MG_ASSERT(vertex->transaction_ == &transaction_,
+            "VertexAccessor must be from the same transaction as the storage "
+            "accessor when deleting a vertex!");
+  auto *vertex_ptr = vertex->vertex_;
+
+  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
+  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
+
+  {
+    if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
+
+    if (vertex_ptr->deleted) return std::optional<ReturnType>{};
+
+    in_edges = vertex_ptr->in_edges;
+    out_edges = vertex_ptr->out_edges;
+  }
+
+  std::vector<EdgeAccessor> deleted_edges;
+  for (const auto &item : in_edges) {
+    auto [edge_type, from_vertex, edge] = item;
+    EdgeAccessor e(edge, edge_type, from_vertex, vertex_ptr, &transaction_, &shard_->indices_, &shard_->constraints_,
+                   config_, shard_->schema_validator_);
+    auto ret = DeleteEdge(&e);
+    if (ret.HasError()) {
+      MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
+      return ret.GetError();
+    }
+
+    if (ret.GetValue()) {
+      deleted_edges.push_back(*ret.GetValue());
+    }
+  }
+  for (const auto &item : out_edges) {
+    auto [edge_type, to_vertex, edge] = item;
+    EdgeAccessor e(edge, edge_type, vertex_ptr, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
+                   config_, shard_->schema_validator_);
+    auto ret = DeleteEdge(&e);
+    if (ret.HasError()) {
+      MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
+      return ret.GetError();
+    }
+
+    if (ret.GetValue()) {
+      deleted_edges.push_back(*ret.GetValue());
+    }
+  }
+
+  // We need to check again for serialization errors because we unlocked the
+  // vertex. Some other transaction could have modified the vertex in the
+  // meantime if we didn't have any edges to delete.
+
+  if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
+
+  MG_ASSERT(!vertex_ptr->deleted, "Invalid database state!");
+
+  CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag());
+  vertex_ptr->deleted = true;
+
+  return std::make_optional<ReturnType>(VertexAccessor{vertex_ptr, &transaction_, &shard_->indices_,
+                                                       &shard_->constraints_, config_, shard_->schema_validator_, true},
+                                        std::move(deleted_edges));
+}
+
+Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) {
+  OOMExceptionEnabler oom_exception;
+  MG_ASSERT(from->transaction_ == to->transaction_,
+            "VertexAccessors must be from the same transaction when creating "
+            "an edge!");
+  MG_ASSERT(from->transaction_ == &transaction_,
+            "VertexAccessors must be from the same transaction in when "
+            "creating an edge!");
+
+  auto *from_vertex = from->vertex_;
+  auto *to_vertex = to->vertex_;
+
+  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
+  if (from_vertex->deleted) return Error::DELETED_OBJECT;
+
+  if (to_vertex != from_vertex) {
+    if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
+    if (to_vertex->deleted) return Error::DELETED_OBJECT;
+  }
+
+  auto gid = Gid::FromUint(shard_->edge_id_++);
+  EdgeRef edge(gid);
+  if (config_.properties_on_edges) {
+    auto acc = shard_->edges_.access();
+    auto *delta = CreateDeleteObjectDelta(&transaction_);
+    auto [it, inserted] = acc.insert(Edge(gid, delta));
+    MG_ASSERT(inserted, "The edge must be inserted here!");
+    MG_ASSERT(it != acc.end(), "Invalid Edge accessor!");
+    edge = EdgeRef(&*it);
+    delta->prev.Set(&*it);
+  }
+
+  CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex, edge);
+  from_vertex->out_edges.emplace_back(edge_type, to_vertex, edge);
+
+  CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex, edge);
+  to_vertex->in_edges.emplace_back(edge_type, from_vertex, edge);
+
+  // Increment edge count.
+  ++shard_->edge_count_;
+
+  return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
+                      config_, shard_->schema_validator_);
+}
+
+Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type,
+                                                 Gid gid) {
+  OOMExceptionEnabler oom_exception;
+  MG_ASSERT(from->transaction_ == to->transaction_,
+            "VertexAccessors must be from the same transaction when creating "
+            "an edge!");
+  MG_ASSERT(from->transaction_ == &transaction_,
+            "VertexAccessors must be from the same transaction in when "
+            "creating an edge!");
+
+  auto *from_vertex = from->vertex_;
+  auto *to_vertex = to->vertex_;
+
+  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
+  if (from_vertex->deleted) return Error::DELETED_OBJECT;
+
+  if (to_vertex != from_vertex) {
+    if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
+    if (to_vertex->deleted) return Error::DELETED_OBJECT;
+  }
+
+  // NOTE: When we update the next `edge_id_` here we perform a RMW
+  // (read-modify-write) operation that ISN'T atomic! But, that isn't an issue
+  // because this function is only called from the replication delta applier
+  // that runs single-threadedly and while this instance is set-up to apply
+  // threads (it is the replica), it is guaranteed that no other writes are
+  // possible.
+  shard_->edge_id_ = std::max(shard_->edge_id_, gid.AsUint() + 1);
+
+  EdgeRef edge(gid);
+  if (config_.properties_on_edges) {
+    auto acc = shard_->edges_.access();
+    auto *delta = CreateDeleteObjectDelta(&transaction_);
+    auto [it, inserted] = acc.insert(Edge(gid, delta));
+    MG_ASSERT(inserted, "The edge must be inserted here!");
+    MG_ASSERT(it != acc.end(), "Invalid Edge accessor!");
+    edge = EdgeRef(&*it);
+    delta->prev.Set(&*it);
+  }
+
+  CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex, edge);
+  from_vertex->out_edges.emplace_back(edge_type, to_vertex, edge);
+
+  CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex, edge);
+  to_vertex->in_edges.emplace_back(edge_type, from_vertex, edge);
+
+  // Increment edge count.
+  ++shard_->edge_count_;
+
+  return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
+                      config_, shard_->schema_validator_);
+}
+
+Result<std::optional<EdgeAccessor>> Shard::Accessor::DeleteEdge(EdgeAccessor *edge) {
+  MG_ASSERT(edge->transaction_ == &transaction_,
+            "EdgeAccessor must be from the same transaction as the storage "
+            "accessor when deleting an edge!");
+  auto edge_ref = edge->edge_;
+  auto edge_type = edge->edge_type_;
+
+  if (config_.properties_on_edges) {
+    auto *edge_ptr = edge_ref.ptr;
+
+    if (!PrepareForWrite(&transaction_, edge_ptr)) return Error::SERIALIZATION_ERROR;
+
+    if (edge_ptr->deleted) return std::optional<EdgeAccessor>{};
+  }
+
+  auto *from_vertex = edge->from_vertex_;
+  auto *to_vertex = edge->to_vertex_;
+
+  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
+  MG_ASSERT(!from_vertex->deleted, "Invalid database state!");
+
+  if (to_vertex != from_vertex) {
+    if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
+    MG_ASSERT(!to_vertex->deleted, "Invalid database state!");
+  }
+
+  auto delete_edge_from_storage = [&edge_type, &edge_ref, this](auto *vertex, auto *edges) {
+    std::tuple<EdgeTypeId, Vertex *, EdgeRef> link(edge_type, vertex, edge_ref);
+    auto it = std::find(edges->begin(), edges->end(), link);
+    if (config_.properties_on_edges) {
+      MG_ASSERT(it != edges->end(), "Invalid database state!");
+    } else if (it == edges->end()) {
+      return false;
+    }
+    std::swap(*it, *edges->rbegin());
+    edges->pop_back();
+    return true;
+  };
+
+  auto op1 = delete_edge_from_storage(to_vertex, &from_vertex->out_edges);
+  auto op2 = delete_edge_from_storage(from_vertex, &to_vertex->in_edges);
+
+  if (config_.properties_on_edges) {
+    MG_ASSERT((op1 && op2), "Invalid database state!");
+  } else {
+    MG_ASSERT((op1 && op2) || (!op1 && !op2), "Invalid database state!");
+    if (!op1 && !op2) {
+      // The edge is already deleted.
+      return std::optional<EdgeAccessor>{};
+    }
+  }
+
+  if (config_.properties_on_edges) {
+    auto *edge_ptr = edge_ref.ptr;
+    CreateAndLinkDelta(&transaction_, edge_ptr, Delta::RecreateObjectTag());
+    edge_ptr->deleted = true;
+  }
+
+  CreateAndLinkDelta(&transaction_, from_vertex, Delta::AddOutEdgeTag(), edge_type, to_vertex, edge_ref);
+  CreateAndLinkDelta(&transaction_, to_vertex, Delta::AddInEdgeTag(), edge_type, from_vertex, edge_ref);
+
+  // Decrement edge count.
+  --shard_->edge_count_;
+
+  return std::make_optional<EdgeAccessor>(edge_ref, edge_type, from_vertex, to_vertex, &transaction_, &shard_->indices_,
+                                          &shard_->constraints_, config_, shard_->schema_validator_, true);
+}
+
+const std::string &Shard::Accessor::LabelToName(LabelId label) const { return shard_->LabelToName(label); }
+
+const std::string &Shard::Accessor::PropertyToName(PropertyId property) const {
+  return shard_->PropertyToName(property);
+}
+
+const std::string &Shard::Accessor::EdgeTypeToName(EdgeTypeId edge_type) const {
+  return shard_->EdgeTypeToName(edge_type);
+}
+
+void Shard::Accessor::AdvanceCommand() { ++transaction_.command_id; }
+
+utils::BasicResult<ConstraintViolation, void> Shard::Accessor::Commit(
+    const std::optional<uint64_t> desired_commit_timestamp) {
+  MG_ASSERT(is_transaction_active_, "The transaction is already terminated!");
+  MG_ASSERT(!transaction_.must_abort, "The transaction can't be committed!");
+
+  if (transaction_.deltas.empty()) {
+    // We don't have to update the commit timestamp here because no one reads
+    // it.
+    shard_->commit_log_->MarkFinished(transaction_.start_timestamp);
+  } else {
+    // Validate that existence constraints are satisfied for all modified
+    // vertices.
+    for (const auto &delta : transaction_.deltas) {
+      auto prev = delta.prev.Get();
+      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+      if (prev.type != PreviousPtr::Type::VERTEX) {
+        continue;
+      }
+      // No need to take any locks here because we modified this vertex and no
+      // one else can touch it until we commit.
+      auto validation_result = ValidateExistenceConstraints(*prev.vertex, shard_->constraints_);
+      if (validation_result) {
+        Abort();
+        return {*validation_result};
+      }
+    }
+
+    // Result of validating the vertex against unique constraints. It has to be
+    // declared outside of the critical section scope because its value is
+    // tested for Abort call which has to be done out of the scope.
+    std::optional<ConstraintViolation> unique_constraint_violation;
+
+    // Save these so we can mark them used in the commit log.
+    uint64_t start_timestamp = transaction_.start_timestamp;
+
+    commit_timestamp_.emplace(shard_->CommitTimestamp(desired_commit_timestamp));
+
+    // Before committing and validating vertices against unique constraints,
+    // we have to update unique constraints with the vertices that are going
+    // to be validated/committed.
+    for (const auto &delta : transaction_.deltas) {
+      auto prev = delta.prev.Get();
+      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+      if (prev.type != PreviousPtr::Type::VERTEX) {
+        continue;
+      }
+      shard_->constraints_.unique_constraints.UpdateBeforeCommit(prev.vertex, transaction_);
+    }
+
+    // Validate that unique constraints are satisfied for all modified
+    // vertices.
+    for (const auto &delta : transaction_.deltas) {
+      auto prev = delta.prev.Get();
+      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+      if (prev.type != PreviousPtr::Type::VERTEX) {
+        continue;
+      }
+
+      // No need to take any locks here because we modified this vertex and no
+      // one else can touch it until we commit.
+      unique_constraint_violation =
+          shard_->constraints_.unique_constraints.Validate(*prev.vertex, transaction_, *commit_timestamp_);
+      if (unique_constraint_violation) {
+        break;
+      }
+    }
+
+    if (!unique_constraint_violation) {
+      // Write transaction to WAL while holding the engine lock to make sure
+      // that committed transactions are sorted by the commit timestamp in the
+      // WAL files. We supply the new commit timestamp to the function so that
+      // it knows what will be the final commit timestamp. The WAL must be
+      // written before actually committing the transaction (before setting
+      // the commit timestamp) so that no other transaction can see the
+      // modifications before they are written to disk.
+      // Replica can log only the write transaction received from Main
+      // so the Wal files are consistent
+      if (shard_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) {
+        shard_->AppendToWal(transaction_, *commit_timestamp_);
+      }
+
+      // TODO(antaljanosbenjamin): Figure out:
+      //   1. How the committed transactions are sorted in `committed_transactions_`
+      //   2. Why it was necessary to lock `committed_transactions_` when it was not accessed at all
+      // TODO: Update all deltas to have a local copy of the commit timestamp
+      MG_ASSERT(transaction_.commit_timestamp != nullptr, "Invalid database state!");
+      transaction_.commit_timestamp->store(*commit_timestamp_, std::memory_order_release);
+      // Replica can only update the last commit timestamp with
+      // the commits received from main.
+      if (shard_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) {
+        // Update the last commit timestamp
+        shard_->last_commit_timestamp_ = *commit_timestamp_;
+      }
+
+      shard_->commit_log_->MarkFinished(start_timestamp);
+    }
+
+    if (unique_constraint_violation) {
+      Abort();
+      return {*unique_constraint_violation};
+    }
+  }
+  is_transaction_active_ = false;
+
+  return {};
+}
+
+void Shard::Accessor::Abort() {
+  MG_ASSERT(is_transaction_active_, "The transaction is already terminated!");
+
+  for (const auto &delta : transaction_.deltas) {
+    auto prev = delta.prev.Get();
+    switch (prev.type) {
+      case PreviousPtr::Type::VERTEX: {
+        auto *vertex = prev.vertex;
+        Delta *current = vertex->delta;
+        while (current != nullptr &&
+               current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) {
+          switch (current->action) {
+            case Delta::Action::REMOVE_LABEL: {
+              auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label);
+              MG_ASSERT(it != vertex->labels.end(), "Invalid database state!");
+              std::swap(*it, *vertex->labels.rbegin());
+              vertex->labels.pop_back();
+              break;
+            }
+            case Delta::Action::ADD_LABEL: {
+              auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label);
+              MG_ASSERT(it == vertex->labels.end(), "Invalid database state!");
+              vertex->labels.push_back(current->label);
+              break;
+            }
+            case Delta::Action::SET_PROPERTY: {
+              vertex->properties.SetProperty(current->property.key, current->property.value);
+              break;
+            }
+            case Delta::Action::ADD_IN_EDGE: {
+              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
+                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
+              auto it = std::find(vertex->in_edges.begin(), vertex->in_edges.end(), link);
+              MG_ASSERT(it == vertex->in_edges.end(), "Invalid database state!");
+              vertex->in_edges.push_back(link);
+              break;
+            }
+            case Delta::Action::ADD_OUT_EDGE: {
+              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
+                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
+              auto it = std::find(vertex->out_edges.begin(), vertex->out_edges.end(), link);
+              MG_ASSERT(it == vertex->out_edges.end(), "Invalid database state!");
+              vertex->out_edges.push_back(link);
+              // Increment edge count. We only increment the count here because
+              // the information in `ADD_IN_EDGE` and `Edge/RECREATE_OBJECT` is
+              // redundant. Also, `Edge/RECREATE_OBJECT` isn't available when
+              // edge properties are disabled.
+              ++shard_->edge_count_;
+              break;
+            }
+            case Delta::Action::REMOVE_IN_EDGE: {
+              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
+                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
+              auto it = std::find(vertex->in_edges.begin(), vertex->in_edges.end(), link);
+              MG_ASSERT(it != vertex->in_edges.end(), "Invalid database state!");
+              std::swap(*it, *vertex->in_edges.rbegin());
+              vertex->in_edges.pop_back();
+              break;
+            }
+            case Delta::Action::REMOVE_OUT_EDGE: {
+              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
+                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
+              auto it = std::find(vertex->out_edges.begin(), vertex->out_edges.end(), link);
+              MG_ASSERT(it != vertex->out_edges.end(), "Invalid database state!");
+              std::swap(*it, *vertex->out_edges.rbegin());
+              vertex->out_edges.pop_back();
+              // Decrement edge count. We only decrement the count here because
+              // the information in `REMOVE_IN_EDGE` and `Edge/DELETE_OBJECT` is
+              // redundant. Also, `Edge/DELETE_OBJECT` isn't available when edge
+              // properties are disabled.
+              --shard_->edge_count_;
+              break;
+            }
+            case Delta::Action::DELETE_OBJECT: {
+              vertex->deleted = true;
+              InsertVertexPKIntoList(shard_->deleted_vertices_, vertex->keys.Keys());
+              break;
+            }
+            case Delta::Action::RECREATE_OBJECT: {
+              vertex->deleted = false;
+              break;
+            }
+          }
+          current = current->next.load(std::memory_order_acquire);
+        }
+        vertex->delta = current;
+        if (current != nullptr) {
+          current->prev.Set(vertex);
+        }
+
+        break;
+      }
+      case PreviousPtr::Type::EDGE: {
+        auto *edge = prev.edge;
+        Delta *current = edge->delta;
+        while (current != nullptr &&
+               current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) {
+          switch (current->action) {
+            case Delta::Action::SET_PROPERTY: {
+              edge->properties.SetProperty(current->property.key, current->property.value);
+              break;
+            }
+            case Delta::Action::DELETE_OBJECT: {
+              edge->deleted = true;
+              shard_->deleted_edges_.push_back(edge->gid);
+              break;
+            }
+            case Delta::Action::RECREATE_OBJECT: {
+              edge->deleted = false;
+              break;
+            }
+            case Delta::Action::REMOVE_LABEL:
+            case Delta::Action::ADD_LABEL:
+            case Delta::Action::ADD_IN_EDGE:
+            case Delta::Action::ADD_OUT_EDGE:
+            case Delta::Action::REMOVE_IN_EDGE:
+            case Delta::Action::REMOVE_OUT_EDGE: {
+              LOG_FATAL("Invalid database state!");
+              break;
+            }
+          }
+          current = current->next.load(std::memory_order_acquire);
+        }
+        edge->delta = current;
+        if (current != nullptr) {
+          current->prev.Set(edge);
+        }
+
+        break;
+      }
+      case PreviousPtr::Type::DELTA:
+      // pointer probably couldn't be set because allocation failed
+      case PreviousPtr::Type::NULLPTR:
+        break;
+    }
+  }
+
+  {
+    uint64_t mark_timestamp = shard_->timestamp_;
+
+    // Release engine lock because we don't have to hold it anymore and
+    // emplace back could take a long time.
+    shard_->garbage_undo_buffers_.emplace_back(mark_timestamp, std::move(transaction_.deltas));
+  }
+
+  shard_->commit_log_->MarkFinished(transaction_.start_timestamp);
+  is_transaction_active_ = false;
+}
+
+void Shard::Accessor::FinalizeTransaction() {
+  if (commit_timestamp_) {
+    shard_->commit_log_->MarkFinished(*commit_timestamp_);
+    shard_->committed_transactions_.emplace_back(std::move(transaction_));
+    commit_timestamp_.reset();
+  }
+}
+
+const std::string &Shard::LabelToName(LabelId label) const { return name_id_mapper_.IdToName(label.AsUint()); }
+
+const std::string &Shard::PropertyToName(PropertyId property) const {
+  return name_id_mapper_.IdToName(property.AsUint());
+}
+
+const std::string &Shard::EdgeTypeToName(EdgeTypeId edge_type) const {
+  return name_id_mapper_.IdToName(edge_type.AsUint());
+}
+
+bool Shard::CreateIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
+  // TODO Fix Index
+  return false;
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_CREATE, label, {}, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return true;
+}
+
+bool Shard::CreateIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
+  // TODO Fix Index
+  // if (!indices_.label_property_index.CreateIndex(label, property, labelspace.access())) return false;
+  return false;
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE, label, {property}, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return true;
+}
+
+bool Shard::DropIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
+  if (!indices_.label_index.DropIndex(label)) return false;
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_DROP, label, {}, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return true;
+}
+
+bool Shard::DropIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
+  if (!indices_.label_property_index.DropIndex(label, property)) return false;
+  // For a description why using `timestamp_` is correct, see
+  // `CreateIndex(LabelId label)`.
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP, label, {property}, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return true;
+}
+
+IndicesInfo Shard::ListAllIndices() const {
+  return {indices_.label_index.ListIndices(), indices_.label_property_index.ListIndices()};
+}
+
+utils::BasicResult<ConstraintViolation, bool> Shard::CreateExistenceConstraint(
+    LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
+  // TODO Fix constraints
+  // auto ret = ::memgraph::storage::v3::CreateExistenceConstraint(&constraints_, label, property, vertices_.access());
+  // if (ret.HasError() || !ret.GetValue()) return ret;
+  return false;
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE, label, {property}, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return true;
+}
+
+bool Shard::DropExistenceConstraint(LabelId label, PropertyId property,
+                                    const std::optional<uint64_t> desired_commit_timestamp) {
+  if (!memgraph::storage::v3::DropExistenceConstraint(&constraints_, label, property)) return false;
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP, label, {property}, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return true;
+}
+
+utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> Shard::CreateUniqueConstraint(
+    LabelId label, const std::set<PropertyId> &properties, const std::optional<uint64_t> desired_commit_timestamp) {
+  // TODO Fix constraints
+  // auto ret = constraints_.unique_constraints.CreateConstraint(label, properties, vertices_.access());
+  // if (ret.HasError() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS) {
+  //   return ret;
+  // }
+  return UniqueConstraints::CreationStatus::ALREADY_EXISTS;
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE, label, properties, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return UniqueConstraints::CreationStatus::SUCCESS;
+}
+
+UniqueConstraints::DeletionStatus Shard::DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
+                                                              const std::optional<uint64_t> desired_commit_timestamp) {
+  auto ret = constraints_.unique_constraints.DropConstraint(label, properties);
+  if (ret != UniqueConstraints::DeletionStatus::SUCCESS) {
+    return ret;
+  }
+  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
+  AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP, label, properties, commit_timestamp);
+  commit_log_->MarkFinished(commit_timestamp);
+  last_commit_timestamp_ = commit_timestamp;
+  return UniqueConstraints::DeletionStatus::SUCCESS;
+}
+
+const SchemaValidator &Shard::Accessor::GetSchemaValidator() const { return shard_->schema_validator_; }
+
+ConstraintsInfo Shard::ListAllConstraints() const {
+  return {ListExistenceConstraints(constraints_), constraints_.unique_constraints.ListConstraints()};
+}
+
+SchemasInfo Shard::ListAllSchemas() const { return {schemas_.ListSchemas()}; }
+
+const Schemas::Schema *Shard::GetSchema(const LabelId primary_label) const { return schemas_.GetSchema(primary_label); }
+
+bool Shard::CreateSchema(const LabelId primary_label, const std::vector<SchemaProperty> &schemas_types) {
+  return schemas_.CreateSchema(primary_label, schemas_types);
+}
+
+bool Shard::DropSchema(const LabelId primary_label) { return schemas_.DropSchema(primary_label); }
+
+StorageInfo Shard::GetInfo() const {
+  auto vertex_count = vertices_.size();
+  double average_degree = 0.0;
+  if (vertex_count) {
+    average_degree = 2.0 * static_cast<double>(edge_count_) / static_cast<double>(vertex_count);
+  }
+  return {vertex_count, edge_count_, average_degree, utils::GetMemoryUsage(),
+          utils::GetDirDiskUsage(config_.durability.storage_directory)};
+}
+
+VerticesIterable Shard::Accessor::Vertices(LabelId label, View view) {
+  return VerticesIterable(shard_->indices_.label_index.Vertices(label, view, &transaction_));
+}
+
+VerticesIterable Shard::Accessor::Vertices(LabelId label, PropertyId property, View view) {
+  return VerticesIterable(shard_->indices_.label_property_index.Vertices(
+      label, property, std::nullopt, std::nullopt, view, &transaction_, shard_->schema_validator_));
+}
+
+VerticesIterable Shard::Accessor::Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) {
+  return VerticesIterable(shard_->indices_.label_property_index.Vertices(
+      label, property, utils::MakeBoundInclusive(value), utils::MakeBoundInclusive(value), view, &transaction_,
+      shard_->schema_validator_));
+}
+
+VerticesIterable Shard::Accessor::Vertices(LabelId label, PropertyId property,
+                                           const std::optional<utils::Bound<PropertyValue>> &lower_bound,
+                                           const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) {
+  return VerticesIterable(shard_->indices_.label_property_index.Vertices(
+      label, property, lower_bound, upper_bound, view, &transaction_, shard_->schema_validator_));
+}
+
+Transaction Shard::CreateTransaction(IsolationLevel isolation_level) {
+  // We acquire the transaction engine lock here because we access (and
+  // modify) the transaction engine variables (`transaction_id` and
+  // `timestamp`) below.
+  uint64_t transaction_id{0};
+  uint64_t start_timestamp{0};
+
+  transaction_id = transaction_id_++;
+  // Replica should have only read queries and the write queries
+  // can come from main instance with any past timestamp.
+  // To preserve snapshot isolation we set the start timestamp
+  // of any query on replica to the last commited transaction
+  // which is timestamp_ as only commit of transaction with writes
+  // can change the value of it.
+  if (replication_role_ == ReplicationRole::REPLICA) {
+    start_timestamp = timestamp_;
+  } else {
+    start_timestamp = timestamp_++;
+  }
+
+  return {transaction_id, start_timestamp, isolation_level};
+}
+
+// `force` means there are no active transactions, so everything can be deleted without worrying about removing some
+// data that is used by an active transaction
+template <bool force>
+void Shard::CollectGarbage() {
+  if constexpr (force) {
+    // TODO(antaljanosbenjamin): figure out whether is there any active transaction or not (probably accessors should
+    // increment/decrement a counter). If there are no transactions, then garbage collection can be forced
+    CollectGarbage<false>();
+    return;
+  }
+
+  // Garbage collection must be performed in two phases. In the first phase,
+  // deltas that won't be applied by any transaction anymore are unlinked from
+  // the version chains. They cannot be deleted immediately, because there
+  // might be a transaction that still needs them to terminate the version
+  // chain traversal. They are instead marked for deletion and will be deleted
+  // in the second GC phase in this GC iteration or some of the following
+  // ones.
+
+  uint64_t oldest_active_start_timestamp = commit_log_->OldestActive();
+  // We don't move undo buffers of unlinked transactions to garbage_undo_buffers
+  // list immediately, because we would have to repeatedly take
+  // garbage_undo_buffers lock.
+  std::list<std::pair<uint64_t, std::list<Delta>>> unlinked_undo_buffers;
+
+  // We will only free vertices deleted up until now in this GC cycle, and we
+  // will do it after cleaning-up the indices. That way we are sure that all
+  // vertices that appear in an index also exist in main storage.
+
+  // Flag that will be used to determine whether the Index GC should be run. It
+  // should be run when there were any items that were cleaned up (there were
+  // updates between this run of the GC and the previous run of the GC). This
+  // eliminates high CPU usage when the GC doesn't have to clean up anything.
+  bool run_index_cleanup = !committed_transactions_.empty() || !garbage_undo_buffers_.empty();
+
+  while (true) {
+    // We don't want to hold the lock on commited transactions for too long,
+    // because that prevents other transactions from committing.
+    Transaction *transaction{nullptr};
+    {
+      if (committed_transactions_.empty()) {
+        break;
+      }
+      transaction = &committed_transactions_.front();
+    }
+
+    auto commit_timestamp = transaction->commit_timestamp->load(std::memory_order_acquire);
+    if (commit_timestamp >= oldest_active_start_timestamp) {
+      break;
+    }
+
+    // When unlinking a delta which is the first delta in its version chain,
+    // special care has to be taken to avoid the following race condition:
+    //
+    // [Vertex] --> [Delta A]
+    //
+    //    GC thread: Delta A is the first in its chain, it must be unlinked from
+    //               vertex and marked for deletion
+    //    TX thread: Update vertex and add Delta B with Delta A as next
+    //
+    // [Vertex] --> [Delta B] <--> [Delta A]
+    //
+    //    GC thread: Unlink delta from Vertex
+    //
+    // [Vertex] --> (nullptr)
+    //
+    // When processing a delta that is the first one in its chain, we
+    // obtain the corresponding vertex or edge lock, and then verify that this
+    // delta still is the first in its chain.
+    // When processing a delta that is in the middle of the chain we only
+    // process the final delta of the given transaction in that chain. We
+    // determine the owner of the chain (either a vertex or an edge), obtain the
+    // corresponding lock, and then verify that this delta is still in the same
+    // position as it was before taking the lock.
+    //
+    // Even though the delta chain is lock-free (both `next` and `prev`) the
+    // chain should not be modified without taking the lock from the object that
+    // owns the chain (either a vertex or an edge). Modifying the chain without
+    // taking the lock will cause subtle race conditions that will leave the
+    // chain in a broken state.
+    // The chain can be only read without taking any locks.
+
+    for (Delta &delta : transaction->deltas) {
+      while (true) {
+        auto prev = delta.prev.Get();
+        switch (prev.type) {
+          case PreviousPtr::Type::VERTEX: {
+            Vertex *vertex = prev.vertex;
+            if (vertex->delta != &delta) {
+              // Something changed, we're not the first delta in the chain
+              // anymore.
+              continue;
+            }
+            vertex->delta = nullptr;
+            if (vertex->deleted) {
+              InsertVertexPKIntoList(deleted_vertices_, vertex->keys.Keys());
+            }
+            break;
+          }
+          case PreviousPtr::Type::EDGE: {
+            Edge *edge = prev.edge;
+            if (edge->delta != &delta) {
+              // Something changed, we're not the first delta in the chain
+              // anymore.
+              continue;
+            }
+            edge->delta = nullptr;
+            if (edge->deleted) {
+              deleted_edges_.push_back(edge->gid);
+            }
+            break;
+          }
+          case PreviousPtr::Type::DELTA: {
+            if (prev.delta->timestamp->load(std::memory_order_acquire) == commit_timestamp) {
+              // The delta that is newer than this one is also a delta from this
+              // transaction. We skip the current delta and will remove it as a
+              // part of the suffix later.
+              break;
+            }
+            {
+              // We need to find the parent object in order to be able to use
+              // its lock.
+              auto parent = prev;
+              while (parent.type == PreviousPtr::Type::DELTA) {
+                parent = parent.delta->prev.Get();
+              }
+              switch (parent.type) {
+                case PreviousPtr::Type::VERTEX:
+                case PreviousPtr::Type::EDGE:
+                  break;
+                case PreviousPtr::Type::DELTA:
+                case PreviousPtr::Type::NULLPTR:
+                  LOG_FATAL("Invalid database state!");
+              }
+            }
+            Delta *prev_delta = prev.delta;
+            prev_delta->next.store(nullptr, std::memory_order_release);
+            break;
+          }
+          case PreviousPtr::Type::NULLPTR: {
+            LOG_FATAL("Invalid pointer!");
+          }
+        }
+        break;
+      }
+    }
+
+    unlinked_undo_buffers.emplace_back(0, std::move(transaction->deltas));
+    committed_transactions_.pop_front();
+  }
+
+  // After unlinking deltas from vertices, we refresh the indices. That way
+  // we're sure that none of the vertices from `current_deleted_vertices`
+  // appears in an index, and we can safely remove the from the main storage
+  // after the last currently active transaction is finished.
+  if (run_index_cleanup) {
+    // This operation is very expensive as it traverses through all of the items
+    // in every index every time.
+    RemoveObsoleteEntries(&indices_, oldest_active_start_timestamp);
+    constraints_.unique_constraints.RemoveObsoleteEntries(oldest_active_start_timestamp);
+  }
+
+  {
+    uint64_t mark_timestamp = timestamp_;
+    for (auto &[timestamp, undo_buffer] : unlinked_undo_buffers) {
+      timestamp = mark_timestamp;
+    }
+    garbage_undo_buffers_.splice(garbage_undo_buffers_.end(), unlinked_undo_buffers);
+
+    for (const auto &vertex : deleted_vertices_) {
+      garbage_vertices_.emplace_back(mark_timestamp, vertex);
+    }
+  }
+
+  // if force is set to true we can simply delete all the leftover undos because
+  // no transaction is active
+  if constexpr (force) {
+    garbage_undo_buffers_.clear();
+  } else {
+    while (!garbage_undo_buffers_.empty() && garbage_undo_buffers_.front().first <= oldest_active_start_timestamp) {
+      garbage_undo_buffers_.pop_front();
+    }
+  }
+
+  {
+    auto vertex_acc = vertices_.access();
+    if constexpr (force) {
+      // if force is set to true, then we have unique_lock and no transactions are active
+      // so we can clean all of the deleted vertices
+      while (!garbage_vertices_.empty()) {
+        MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!");
+        garbage_vertices_.pop_front();
+      }
+    } else {
+      while (!garbage_vertices_.empty() && garbage_vertices_.front().first < oldest_active_start_timestamp) {
+        MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!");
+        garbage_vertices_.pop_front();
+      }
+    }
+  }
+  {
+    auto edge_acc = edges_.access();
+    for (auto edge : deleted_edges_) {
+      MG_ASSERT(edge_acc.remove(edge), "Invalid database state!");
+    }
+  }
+}
+
+// tell the linker he can find the CollectGarbage definitions here
+template void Shard::CollectGarbage<true>();
+template void Shard::CollectGarbage<false>();
+
+bool Shard::InitializeWalFile() {
+  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL)
+    return false;
+  if (!wal_file_) {
+    wal_file_.emplace(wal_directory_, uuid_, epoch_id_, config_.items, &name_id_mapper_, wal_seq_num_++,
+                      &file_retainer_);
+  }
+  return true;
+}
+
+void Shard::FinalizeWalFile() {
+  ++wal_unsynced_transactions_;
+  if (wal_unsynced_transactions_ >= config_.durability.wal_file_flush_every_n_tx) {
+    wal_file_->Sync();
+    wal_unsynced_transactions_ = 0;
+  }
+  if (wal_file_->GetSize() / 1024 >= config_.durability.wal_file_size_kibibytes) {
+    wal_file_->FinalizeWal();
+    wal_file_ = std::nullopt;
+    wal_unsynced_transactions_ = 0;
+  } else {
+    // Try writing the internal buffer if possible, if not
+    // the data should be written as soon as it's possible
+    // (triggered by the new transaction commit, or some
+    // reading thread EnabledFlushing)
+    wal_file_->TryFlushing();
+  }
+}
+
+void Shard::AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp) {
+  if (!InitializeWalFile()) return;
+  // Traverse deltas and append them to the WAL file.
+  // A single transaction will always be contained in a single WAL file.
+  auto current_commit_timestamp = transaction.commit_timestamp->load(std::memory_order_acquire);
+
+  if (replication_role_ == ReplicationRole::MAIN) {
+    replication_clients_.WithLock([&](auto &clients) {
+      for (auto &client : clients) {
+        client->StartTransactionReplication(wal_file_->SequenceNumber());
+      }
+    });
+  }
+
+  // Helper lambda that traverses the delta chain on order to find the first
+  // delta that should be processed and then appends all discovered deltas.
+  auto find_and_apply_deltas = [&](const auto *delta, const auto &parent, auto filter) {
+    while (true) {
+      auto *older = delta->next.load(std::memory_order_acquire);
+      if (older == nullptr || older->timestamp->load(std::memory_order_acquire) != current_commit_timestamp) break;
+      delta = older;
+    }
+    while (true) {
+      if (filter(delta->action)) {
+        wal_file_->AppendDelta(*delta, parent, final_commit_timestamp);
+        replication_clients_.WithLock([&](auto &clients) {
+          for (auto &client : clients) {
+            client->IfStreamingTransaction(
+                [&](auto &stream) { stream.AppendDelta(*delta, parent, final_commit_timestamp); });
+          }
+        });
+      }
+      auto prev = delta->prev.Get();
+      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+      if (prev.type != PreviousPtr::Type::DELTA) break;
+      delta = prev.delta;
+    }
+  };
+
+  // The deltas are ordered correctly in the `transaction.deltas` buffer, but we
+  // don't traverse them in that order. That is because for each delta we need
+  // information about the vertex or edge they belong to and that information
+  // isn't stored in the deltas themselves. In order to find out information
+  // about the corresponding vertex or edge it is necessary to traverse the
+  // delta chain for each delta until a vertex or edge is encountered. This
+  // operation is very expensive as the chain grows.
+  // Instead, we traverse the edges until we find a vertex or edge and traverse
+  // their delta chains. This approach has a drawback because we lose the
+  // correct order of the operations. Because of that, we need to traverse the
+  // deltas several times and we have to manually ensure that the stored deltas
+  // will be ordered correctly.
+
+  // 1. Process all Vertex deltas and store all operations that create vertices
+  // and modify vertex data.
+  for (const auto &delta : transaction.deltas) {
+    auto prev = delta.prev.Get();
+    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+    if (prev.type != PreviousPtr::Type::VERTEX) continue;
+    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
+      switch (action) {
+        case Delta::Action::DELETE_OBJECT:
+        case Delta::Action::SET_PROPERTY:
+        case Delta::Action::ADD_LABEL:
+        case Delta::Action::REMOVE_LABEL:
+          return true;
+
+        case Delta::Action::RECREATE_OBJECT:
+        case Delta::Action::ADD_IN_EDGE:
+        case Delta::Action::ADD_OUT_EDGE:
+        case Delta::Action::REMOVE_IN_EDGE:
+        case Delta::Action::REMOVE_OUT_EDGE:
+          return false;
+      }
+    });
+  }
+  // 2. Process all Vertex deltas and store all operations that create edges.
+  for (const auto &delta : transaction.deltas) {
+    auto prev = delta.prev.Get();
+    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+    if (prev.type != PreviousPtr::Type::VERTEX) continue;
+    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
+      switch (action) {
+        case Delta::Action::REMOVE_OUT_EDGE:
+          return true;
+
+        case Delta::Action::DELETE_OBJECT:
+        case Delta::Action::RECREATE_OBJECT:
+        case Delta::Action::SET_PROPERTY:
+        case Delta::Action::ADD_LABEL:
+        case Delta::Action::REMOVE_LABEL:
+        case Delta::Action::ADD_IN_EDGE:
+        case Delta::Action::ADD_OUT_EDGE:
+        case Delta::Action::REMOVE_IN_EDGE:
+          return false;
+      }
+    });
+  }
+  // 3. Process all Edge deltas and store all operations that modify edge data.
+  for (const auto &delta : transaction.deltas) {
+    auto prev = delta.prev.Get();
+    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+    if (prev.type != PreviousPtr::Type::EDGE) continue;
+    find_and_apply_deltas(&delta, *prev.edge, [](auto action) {
+      switch (action) {
+        case Delta::Action::SET_PROPERTY:
+          return true;
+
+        case Delta::Action::DELETE_OBJECT:
+        case Delta::Action::RECREATE_OBJECT:
+        case Delta::Action::ADD_LABEL:
+        case Delta::Action::REMOVE_LABEL:
+        case Delta::Action::ADD_IN_EDGE:
+        case Delta::Action::ADD_OUT_EDGE:
+        case Delta::Action::REMOVE_IN_EDGE:
+        case Delta::Action::REMOVE_OUT_EDGE:
+          return false;
+      }
+    });
+  }
+  // 4. Process all Vertex deltas and store all operations that delete edges.
+  for (const auto &delta : transaction.deltas) {
+    auto prev = delta.prev.Get();
+    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+    if (prev.type != PreviousPtr::Type::VERTEX) continue;
+    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
+      switch (action) {
+        case Delta::Action::ADD_OUT_EDGE:
+          return true;
+
+        case Delta::Action::DELETE_OBJECT:
+        case Delta::Action::RECREATE_OBJECT:
+        case Delta::Action::SET_PROPERTY:
+        case Delta::Action::ADD_LABEL:
+        case Delta::Action::REMOVE_LABEL:
+        case Delta::Action::ADD_IN_EDGE:
+        case Delta::Action::REMOVE_IN_EDGE:
+        case Delta::Action::REMOVE_OUT_EDGE:
+          return false;
+      }
+    });
+  }
+  // 5. Process all Vertex deltas and store all operations that delete vertices.
+  for (const auto &delta : transaction.deltas) {
+    auto prev = delta.prev.Get();
+    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
+    if (prev.type != PreviousPtr::Type::VERTEX) continue;
+    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
+      switch (action) {
+        case Delta::Action::RECREATE_OBJECT:
+          return true;
+
+        case Delta::Action::DELETE_OBJECT:
+        case Delta::Action::SET_PROPERTY:
+        case Delta::Action::ADD_LABEL:
+        case Delta::Action::REMOVE_LABEL:
+        case Delta::Action::ADD_IN_EDGE:
+        case Delta::Action::ADD_OUT_EDGE:
+        case Delta::Action::REMOVE_IN_EDGE:
+        case Delta::Action::REMOVE_OUT_EDGE:
+          return false;
+      }
+    });
+  }
+
+  // Add a delta that indicates that the transaction is fully written to the WAL
+  // file.
+  wal_file_->AppendTransactionEnd(final_commit_timestamp);
+
+  FinalizeWalFile();
+
+  replication_clients_.WithLock([&](auto &clients) {
+    for (auto &client : clients) {
+      client->IfStreamingTransaction([&](auto &stream) { stream.AppendTransactionEnd(final_commit_timestamp); });
+      client->FinalizeTransactionReplication();
+    }
+  });
+}
+
+void Shard::AppendToWal(durability::StorageGlobalOperation operation, LabelId label,
+                        const std::set<PropertyId> &properties, uint64_t final_commit_timestamp) {
+  if (!InitializeWalFile()) return;
+  wal_file_->AppendOperation(operation, label, properties, final_commit_timestamp);
+  {
+    if (replication_role_ == ReplicationRole::MAIN) {
+      replication_clients_.WithLock([&](auto &clients) {
+        for (auto &client : clients) {
+          client->StartTransactionReplication(wal_file_->SequenceNumber());
+          client->IfStreamingTransaction(
+              [&](auto &stream) { stream.AppendOperation(operation, label, properties, final_commit_timestamp); });
+          client->FinalizeTransactionReplication();
+        }
+      });
+    }
+  }
+  FinalizeWalFile();
+}
+
+utils::BasicResult<Shard::CreateSnapshotError> Shard::CreateSnapshot() {
+  if (replication_role_ != ReplicationRole::MAIN) {
+    return CreateSnapshotError::DisabledForReplica;
+  }
+
+  // Create the transaction used to create the snapshot.
+  auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION);
+
+  // Create snapshot.
+  // durability::CreateSnapshot(&transaction, snapshot_directory_, wal_directory_,
+  //                            config_.durability.snapshot_retention_count, &vertices_, &edges_,
+  //                            &name_id_mapper_, &indices_, &constraints_, config_.items, schema_validator_,
+  //                            uuid_, epoch_id_, epoch_history_, &file_retainer_);
+
+  // Finalize snapshot transaction.
+  commit_log_->MarkFinished(transaction.start_timestamp);
+  return {};
+}
+
+bool Shard::LockPath() {
+  auto locker_accessor = global_locker_.Access();
+  return locker_accessor.AddPath(config_.durability.storage_directory);
+}
+
+bool Shard::UnlockPath() {
+  {
+    auto locker_accessor = global_locker_.Access();
+    if (!locker_accessor.RemovePath(config_.durability.storage_directory)) {
+      return false;
+    }
+  }
+
+  // We use locker accessor in seperate scope so we don't produce deadlock
+  // after we call clean queue.
+  file_retainer_.CleanQueue();
+  return true;
+}
+
+void Shard::FreeMemory() {
+  CollectGarbage<true>();
+
+  // SkipList is already threadsafe
+  vertices_.run_gc();
+  edges_.run_gc();
+  indices_.label_index.RunGC();
+  indices_.label_property_index.RunGC();
+}
+
+uint64_t Shard::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) {
+  if (!desired_commit_timestamp) {
+    return timestamp_++;
+  }
+  timestamp_ = std::max(timestamp_, *desired_commit_timestamp + 1);
+  return *desired_commit_timestamp;
+}
+
+bool Shard::SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) {
+  // We don't want to restart the server if we're already a REPLICA
+  if (replication_role_ == ReplicationRole::REPLICA) {
+    return false;
+  }
+
+  replication_server_ = std::make_unique<ReplicationServer>(this, std::move(endpoint), config);
+
+  replication_role_ = ReplicationRole::REPLICA;
+  return true;
+}
+
+bool Shard::SetMainReplicationRole() {
+  // We don't want to generate new epoch_id and do the
+  // cleanup if we're already a MAIN
+  if (replication_role_ == ReplicationRole::MAIN) {
+    return false;
+  }
+
+  // Main instance does not need replication server
+  // This should be always called first so we finalize everything
+  replication_server_.reset(nullptr);
+
+  if (wal_file_) {
+    wal_file_->FinalizeWal();
+    wal_file_.reset();
+  }
+
+  // Generate new epoch id and save the last one to the history.
+  if (epoch_history_.size() == kEpochHistoryRetention) {
+    epoch_history_.pop_front();
+  }
+  epoch_history_.emplace_back(std::move(epoch_id_), last_commit_timestamp_);
+  epoch_id_ = utils::GenerateUUID();
+
+  replication_role_ = ReplicationRole::MAIN;
+  return true;
+}
+
+utils::BasicResult<Shard::RegisterReplicaError> Shard::RegisterReplica(
+    std::string name, io::network::Endpoint endpoint, const replication::ReplicationMode replication_mode,
+    const replication::ReplicationClientConfig &config) {
+  MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can register a replica!");
+
+  const bool name_exists = replication_clients_.WithLock([&](auto &clients) {
+    return std::any_of(clients.begin(), clients.end(), [&name](const auto &client) { return client->Name() == name; });
+  });
+
+  if (name_exists) {
+    return RegisterReplicaError::NAME_EXISTS;
+  }
+
+  const auto end_point_exists = replication_clients_.WithLock([&endpoint](auto &clients) {
+    return std::any_of(clients.begin(), clients.end(),
+                       [&endpoint](const auto &client) { return client->Endpoint() == endpoint; });
+  });
+
+  if (end_point_exists) {
+    return RegisterReplicaError::END_POINT_EXISTS;
+  }
+
+  MG_ASSERT(replication_mode == replication::ReplicationMode::SYNC || !config.timeout,
+            "Only SYNC mode can have a timeout set");
+
+  auto client = std::make_unique<ReplicationClient>(std::move(name), this, endpoint, replication_mode, config);
+  if (client->State() == replication::ReplicaState::INVALID) {
+    return RegisterReplicaError::CONNECTION_FAILED;
+  }
+
+  return replication_clients_.WithLock([&](auto &clients) -> utils::BasicResult<Shard::RegisterReplicaError> {
+    // Another thread could have added a client with same name while
+    // we were connecting to this client.
+    if (std::any_of(clients.begin(), clients.end(),
+                    [&](const auto &other_client) { return client->Name() == other_client->Name(); })) {
+      return RegisterReplicaError::NAME_EXISTS;
+    }
+
+    if (std::any_of(clients.begin(), clients.end(),
+                    [&client](const auto &other_client) { return client->Endpoint() == other_client->Endpoint(); })) {
+      return RegisterReplicaError::END_POINT_EXISTS;
+    }
+
+    clients.push_back(std::move(client));
+    return {};
+  });
+}
+
+bool Shard::UnregisterReplica(const std::string_view name) {
+  MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can unregister a replica!");
+  return replication_clients_.WithLock([&](auto &clients) {
+    return std::erase_if(clients, [&](const auto &client) { return client->Name() == name; });
+  });
+}
+
+std::optional<replication::ReplicaState> Shard::GetReplicaState(const std::string_view name) {
+  return replication_clients_.WithLock([&](auto &clients) -> std::optional<replication::ReplicaState> {
+    const auto client_it =
+        std::find_if(clients.cbegin(), clients.cend(), [name](auto &client) { return client->Name() == name; });
+    if (client_it == clients.cend()) {
+      return std::nullopt;
+    }
+    return (*client_it)->State();
+  });
+}
+
+ReplicationRole Shard::GetReplicationRole() const { return replication_role_; }
+
+std::vector<Shard::ReplicaInfo> Shard::ReplicasInfo() {
+  return replication_clients_.WithLock([](auto &clients) {
+    std::vector<Shard::ReplicaInfo> replica_info;
+    replica_info.reserve(clients.size());
+    std::transform(clients.begin(), clients.end(), std::back_inserter(replica_info),
+                   [](const auto &client) -> ReplicaInfo {
+                     return {client->Name(), client->Mode(), client->Timeout(), client->Endpoint(), client->State()};
+                   });
+    return replica_info;
+  });
+}
+
+void Shard::SetIsolationLevel(IsolationLevel isolation_level) { isolation_level_ = isolation_level; }
+
+}  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/shard.hpp b/src/storage/v3/shard.hpp
new file mode 100644
index 000000000..8a42e542d
--- /dev/null
+++ b/src/storage/v3/shard.hpp
@@ -0,0 +1,617 @@
+// Copyright 2022 Memgraph Ltd.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
+// License, and you may not use this file except in compliance with the Business Source License.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+#pragma once
+
+#include <atomic>
+#include <cstdint>
+#include <filesystem>
+#include <map>
+#include <numeric>
+#include <optional>
+#include <shared_mutex>
+#include <variant>
+#include <vector>
+
+#include "io/network/endpoint.hpp"
+#include "kvstore/kvstore.hpp"
+#include "storage/v3/commit_log.hpp"
+#include "storage/v3/config.hpp"
+#include "storage/v3/constraints.hpp"
+#include "storage/v3/durability/metadata.hpp"
+#include "storage/v3/durability/wal.hpp"
+#include "storage/v3/edge.hpp"
+#include "storage/v3/edge_accessor.hpp"
+#include "storage/v3/id_types.hpp"
+#include "storage/v3/indices.hpp"
+#include "storage/v3/isolation_level.hpp"
+#include "storage/v3/key_store.hpp"
+#include "storage/v3/lexicographically_ordered_vertex.hpp"
+#include "storage/v3/mvcc.hpp"
+#include "storage/v3/name_id_mapper.hpp"
+#include "storage/v3/property_value.hpp"
+#include "storage/v3/result.hpp"
+#include "storage/v3/schema_validator.hpp"
+#include "storage/v3/schemas.hpp"
+#include "storage/v3/transaction.hpp"
+#include "storage/v3/vertex.hpp"
+#include "storage/v3/vertex_accessor.hpp"
+#include "storage/v3/vertices_skip_list.hpp"
+#include "utils/exceptions.hpp"
+#include "utils/file_locker.hpp"
+#include "utils/on_scope_exit.hpp"
+#include "utils/rw_lock.hpp"
+#include "utils/scheduler.hpp"
+#include "utils/skip_list.hpp"
+#include "utils/synchronized.hpp"
+#include "utils/uuid.hpp"
+
+/// REPLICATION ///
+#include "rpc/server.hpp"
+#include "storage/v3/replication/config.hpp"
+#include "storage/v3/replication/enums.hpp"
+#include "storage/v3/replication/rpc.hpp"
+#include "storage/v3/replication/serialization.hpp"
+
+namespace memgraph::storage::v3 {
+
+// The storage is based on this paper:
+// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
+// The paper implements a fully serializable storage, in our implementation we
+// only implement snapshot isolation for transactions.
+
+/// Iterable for iterating through all vertices of a Storage.
+///
+/// An instance of this will be usually be wrapped inside VerticesIterable for
+/// generic, public use.
+class AllVerticesIterable final {
+  VerticesSkipList::Accessor vertices_accessor_;
+  Transaction *transaction_;
+  View view_;
+  Indices *indices_;
+  Constraints *constraints_;
+  Config::Items config_;
+  const SchemaValidator *schema_validator_;
+  const Schemas *schemas_;
+  std::optional<VertexAccessor> vertex_;
+
+ public:
+  class Iterator final {
+    AllVerticesIterable *self_;
+    VerticesSkipList::Iterator it_;
+
+   public:
+    Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it);
+
+    VertexAccessor operator*() const;
+
+    Iterator &operator++();
+
+    bool operator==(const Iterator &other) const { return self_ == other.self_ && it_ == other.it_; }
+
+    bool operator!=(const Iterator &other) const { return !(*this == other); }
+  };
+
+  AllVerticesIterable(VerticesSkipList::Accessor vertices_accessor, Transaction *transaction, View view,
+                      Indices *indices, Constraints *constraints, Config::Items config,
+                      const SchemaValidator &schema_validator)
+      : vertices_accessor_(std::move(vertices_accessor)),
+        transaction_(transaction),
+        view_(view),
+        indices_(indices),
+        constraints_(constraints),
+        config_(config),
+        schema_validator_{&schema_validator} {}
+
+  Iterator begin() { return {this, vertices_accessor_.begin()}; }
+  Iterator end() { return {this, vertices_accessor_.end()}; }
+};
+
+/// Generic access to different kinds of vertex iterations.
+///
+/// This class should be the primary type used by the client code to iterate
+/// over vertices inside a Storage instance.
+class VerticesIterable final {
+  enum class Type { ALL, BY_LABEL, BY_LABEL_PROPERTY };
+
+  Type type_;
+  union {
+    AllVerticesIterable all_vertices_;
+    LabelIndex::Iterable vertices_by_label_;
+    LabelPropertyIndex::Iterable vertices_by_label_property_;
+  };
+
+ public:
+  explicit VerticesIterable(AllVerticesIterable);
+  explicit VerticesIterable(LabelIndex::Iterable);
+  explicit VerticesIterable(LabelPropertyIndex::Iterable);
+
+  VerticesIterable(const VerticesIterable &) = delete;
+  VerticesIterable &operator=(const VerticesIterable &) = delete;
+
+  VerticesIterable(VerticesIterable &&) noexcept;
+  VerticesIterable &operator=(VerticesIterable &&) noexcept;
+
+  ~VerticesIterable();
+
+  class Iterator final {
+    Type type_;
+    union {
+      AllVerticesIterable::Iterator all_it_;
+      LabelIndex::Iterable::Iterator by_label_it_;
+      LabelPropertyIndex::Iterable::Iterator by_label_property_it_;
+    };
+
+    void Destroy() noexcept;
+
+   public:
+    explicit Iterator(AllVerticesIterable::Iterator);
+    explicit Iterator(LabelIndex::Iterable::Iterator);
+    explicit Iterator(LabelPropertyIndex::Iterable::Iterator);
+
+    Iterator(const Iterator &);
+    Iterator &operator=(const Iterator &);
+
+    Iterator(Iterator &&) noexcept;
+    Iterator &operator=(Iterator &&) noexcept;
+
+    ~Iterator();
+
+    VertexAccessor operator*() const;
+
+    Iterator &operator++();
+
+    bool operator==(const Iterator &other) const;
+    bool operator!=(const Iterator &other) const { return !(*this == other); }
+  };
+
+  Iterator begin();
+  Iterator end();
+};
+
+/// Structure used to return information about existing indices in the storage.
+struct IndicesInfo {
+  std::vector<LabelId> label;
+  std::vector<std::pair<LabelId, PropertyId>> label_property;
+};
+
+/// Structure used to return information about existing constraints in the
+/// storage.
+struct ConstraintsInfo {
+  std::vector<std::pair<LabelId, PropertyId>> existence;
+  std::vector<std::pair<LabelId, std::set<PropertyId>>> unique;
+};
+
+/// Structure used to return information about existing schemas in the storage
+struct SchemasInfo {
+  Schemas::SchemasList schemas;
+};
+
+/// Structure used to return information about the storage.
+struct StorageInfo {
+  uint64_t vertex_count;
+  uint64_t edge_count;
+  double average_degree;
+  uint64_t memory_usage;
+  uint64_t disk_usage;
+};
+
+enum class ReplicationRole : uint8_t { MAIN, REPLICA };
+
+class Shard final {
+ public:
+  /// @throw std::system_error
+  /// @throw std::bad_alloc
+  explicit Shard(LabelId primary_label, PrimaryKey min_primary_key, std::optional<PrimaryKey> max_primary_key,
+                 Config config = Config());
+
+  Shard(const Shard &) = delete;
+  Shard(Shard &&) noexcept = delete;
+  Shard &operator=(const Shard &) = delete;
+  Shard operator=(Shard &&) noexcept = delete;
+  ~Shard();
+
+  class Accessor final {
+   private:
+    friend class Shard;
+
+    explicit Accessor(Shard *shard, IsolationLevel isolation_level);
+
+   public:
+    Accessor(const Accessor &) = delete;
+    Accessor &operator=(const Accessor &) = delete;
+    Accessor &operator=(Accessor &&other) = delete;
+
+    // NOTE: After the accessor is moved, all objects derived from it (accessors
+    // and iterators) are *invalid*. You have to get all derived objects again.
+    Accessor(Accessor &&other) noexcept;
+
+    ~Accessor();
+
+    /// @throw std::bad_alloc
+    ResultSchema<VertexAccessor> CreateVertexAndValidate(
+        LabelId primary_label, const std::vector<LabelId> &labels,
+        const std::vector<std::pair<PropertyId, PropertyValue>> &properties);
+
+    std::optional<VertexAccessor> FindVertex(std::vector<PropertyValue> primary_key, View view);
+
+    VerticesIterable Vertices(View view) {
+      return VerticesIterable(AllVerticesIterable(shard_->vertices_.access(), &transaction_, view, &shard_->indices_,
+                                                  &shard_->constraints_, shard_->config_.items,
+                                                  shard_->schema_validator_));
+    }
+
+    VerticesIterable Vertices(LabelId label, View view);
+
+    VerticesIterable Vertices(LabelId label, PropertyId property, View view);
+
+    VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view);
+
+    VerticesIterable Vertices(LabelId label, PropertyId property,
+                              const std::optional<utils::Bound<PropertyValue>> &lower_bound,
+                              const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view);
+
+    /// Return approximate number of all vertices in the database.
+    /// Note that this is always an over-estimate and never an under-estimate.
+    int64_t ApproximateVertexCount() const { return static_cast<int64_t>(shard_->vertices_.size()); }
+
+    /// Return approximate number of vertices with the given label.
+    /// Note that this is always an over-estimate and never an under-estimate.
+    int64_t ApproximateVertexCount(LabelId label) const {
+      return shard_->indices_.label_index.ApproximateVertexCount(label);
+    }
+
+    /// Return approximate number of vertices with the given label and property.
+    /// Note that this is always an over-estimate and never an under-estimate.
+    int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
+      return shard_->indices_.label_property_index.ApproximateVertexCount(label, property);
+    }
+
+    /// Return approximate number of vertices with the given label and the given
+    /// value for the given property. Note that this is always an over-estimate
+    /// and never an under-estimate.
+    int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const {
+      return shard_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
+    }
+
+    /// Return approximate number of vertices with the given label and value for
+    /// the given property in the range defined by provided upper and lower
+    /// bounds.
+    int64_t ApproximateVertexCount(LabelId label, PropertyId property,
+                                   const std::optional<utils::Bound<PropertyValue>> &lower,
+                                   const std::optional<utils::Bound<PropertyValue>> &upper) const {
+      return shard_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
+    }
+
+    /// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
+    /// @throw std::bad_alloc
+    Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex);
+
+    /// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
+    /// @throw std::bad_alloc
+    Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
+        VertexAccessor *vertex);
+
+    /// @throw std::bad_alloc
+    Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type);
+
+    /// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
+    /// @throw std::bad_alloc
+    Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge);
+
+    const std::string &LabelToName(LabelId label) const;
+    const std::string &PropertyToName(PropertyId property) const;
+    const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
+
+    bool LabelIndexExists(LabelId label) const { return shard_->indices_.label_index.IndexExists(label); }
+
+    bool LabelPropertyIndexExists(LabelId label, PropertyId property) const {
+      return shard_->indices_.label_property_index.IndexExists(label, property);
+    }
+
+    IndicesInfo ListAllIndices() const {
+      return {shard_->indices_.label_index.ListIndices(), shard_->indices_.label_property_index.ListIndices()};
+    }
+
+    ConstraintsInfo ListAllConstraints() const {
+      return {ListExistenceConstraints(shard_->constraints_),
+              shard_->constraints_.unique_constraints.ListConstraints()};
+    }
+
+    const SchemaValidator &GetSchemaValidator() const;
+
+    SchemasInfo ListAllSchemas() const { return {shard_->schemas_.ListSchemas()}; }
+
+    void AdvanceCommand();
+
+    /// Commit returns `ConstraintViolation` if the changes made by this
+    /// transaction violate an existence or unique constraint. In that case the
+    /// transaction is automatically aborted. Otherwise, void is returned.
+    /// @throw std::bad_alloc
+    utils::BasicResult<ConstraintViolation, void> Commit(std::optional<uint64_t> desired_commit_timestamp = {});
+
+    /// @throw std::bad_alloc
+    void Abort();
+
+    void FinalizeTransaction();
+
+   private:
+    /// @throw std::bad_alloc
+    VertexAccessor CreateVertex(Gid gid, LabelId primary_label);
+
+    /// @throw std::bad_alloc
+    Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, Gid gid);
+
+    Shard *shard_;
+    Transaction transaction_;
+    std::optional<uint64_t> commit_timestamp_;
+    bool is_transaction_active_;
+    Config::Items config_;
+  };
+
+  Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) {
+    return Accessor{this, override_isolation_level.value_or(isolation_level_)};
+  }
+
+  const std::string &LabelToName(LabelId label) const;
+  const std::string &PropertyToName(PropertyId property) const;
+  const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
+
+  /// @throw std::bad_alloc if unable to insert a new mapping
+  LabelId NameToLabel(std::string_view name);
+
+  /// @throw std::bad_alloc if unable to insert a new mapping
+  PropertyId NameToProperty(std::string_view name);
+
+  /// @throw std::bad_alloc if unable to insert a new mapping
+  EdgeTypeId NameToEdgeType(std::string_view name);
+
+  /// @throw std::bad_alloc
+  bool CreateIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
+
+  /// @throw std::bad_alloc
+  bool CreateIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
+
+  bool DropIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
+
+  bool DropIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
+
+  IndicesInfo ListAllIndices() const;
+
+  /// Creates an existence constraint. Returns true if the constraint was
+  /// successfully added, false if it already exists and a `ConstraintViolation`
+  /// if there is an existing vertex violating the constraint.
+  ///
+  /// @throw std::bad_alloc
+  /// @throw std::length_error
+  utils::BasicResult<ConstraintViolation, bool> CreateExistenceConstraint(
+      LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
+
+  /// Removes an existence constraint. Returns true if the constraint was
+  /// removed, and false if it doesn't exist.
+  bool DropExistenceConstraint(LabelId label, PropertyId property,
+                               std::optional<uint64_t> desired_commit_timestamp = {});
+
+  /// Creates a unique constraint. In the case of two vertices violating the
+  /// constraint, it returns `ConstraintViolation`. Otherwise returns a
+  /// `UniqueConstraints::CreationStatus` enum with the following possibilities:
+  ///     * `SUCCESS` if the constraint was successfully created,
+  ///     * `ALREADY_EXISTS` if the constraint already existed,
+  ///     * `EMPTY_PROPERTIES` if the property set is empty, or
+  //      * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the
+  //        limit of maximum number of properties.
+  ///
+  /// @throw std::bad_alloc
+  utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
+      LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp = {});
+
+  /// Removes a unique constraint. Returns `UniqueConstraints::DeletionStatus`
+  /// enum with the following possibilities:
+  ///     * `SUCCESS` if constraint was successfully removed,
+  ///     * `NOT_FOUND` if the specified constraint was not found,
+  ///     * `EMPTY_PROPERTIES` if the property set is empty, or
+  ///     * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the
+  //        limit of maximum number of properties.
+  UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
+                                                         std::optional<uint64_t> desired_commit_timestamp = {});
+
+  ConstraintsInfo ListAllConstraints() const;
+
+  SchemasInfo ListAllSchemas() const;
+
+  const Schemas::Schema *GetSchema(LabelId primary_label) const;
+
+  bool CreateSchema(LabelId primary_label, const std::vector<SchemaProperty> &schemas_types);
+
+  bool DropSchema(LabelId primary_label);
+
+  StorageInfo GetInfo() const;
+
+  bool LockPath();
+  bool UnlockPath();
+
+  bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {});
+
+  bool SetMainReplicationRole();
+
+  enum class RegisterReplicaError : uint8_t {
+    NAME_EXISTS,
+    END_POINT_EXISTS,
+    CONNECTION_FAILED,
+    COULD_NOT_BE_PERSISTED
+  };
+
+  /// @pre The instance should have a MAIN role
+  /// @pre Timeout can only be set for SYNC replication
+  utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
+      std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
+      const replication::ReplicationClientConfig &config = {});
+  /// @pre The instance should have a MAIN role
+  bool UnregisterReplica(std::string_view name);
+
+  std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
+
+  ReplicationRole GetReplicationRole() const;
+
+  struct ReplicaInfo {
+    std::string name;
+    replication::ReplicationMode mode;
+    std::optional<double> timeout;
+    io::network::Endpoint endpoint;
+    replication::ReplicaState state;
+  };
+
+  std::vector<ReplicaInfo> ReplicasInfo();
+
+  void FreeMemory();
+
+  void SetIsolationLevel(IsolationLevel isolation_level);
+
+  enum class CreateSnapshotError : uint8_t { DisabledForReplica };
+
+  utils::BasicResult<CreateSnapshotError> CreateSnapshot();
+
+ private:
+  Transaction CreateTransaction(IsolationLevel isolation_level);
+
+  /// The force parameter determines the behaviour of the garbage collector.
+  /// If it's set to true, it will behave as a global operation, i.e. it can't
+  /// be part of a transaction, and no other transaction can be active at the same time.
+  /// This allows it to delete immediately vertices without worrying that some other
+  /// transaction is possibly using it. If there are active transactions when this method
+  /// is called with force set to true, it will fallback to the same method with the force
+  /// set to false.
+  /// If it's set to false, it will execute in parallel with other transactions, ensuring
+  /// that no object in use can be deleted.
+  /// @throw std::system_error
+  /// @throw std::bad_alloc
+  template <bool force>
+  void CollectGarbage();
+
+  bool InitializeWalFile();
+  void FinalizeWalFile();
+
+  void AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp);
+  void AppendToWal(durability::StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties,
+                   uint64_t final_commit_timestamp);
+
+  uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
+
+  // Main object storage
+  NameIdMapper name_id_mapper_;
+  LabelId primary_label_;
+  PrimaryKey min_primary_key_;
+  std::optional<PrimaryKey> max_primary_key_;
+  VerticesSkipList vertices_;
+  utils::SkipList<Edge> edges_;
+  uint64_t edge_id_{0};
+  // Even though the edge count is already kept in the `edges_` SkipList, the
+  // list is used only when properties are enabled for edges. Because of that we
+  // keep a separate count of edges that is always updated.
+  uint64_t edge_count_{0};
+
+  SchemaValidator schema_validator_;
+  Constraints constraints_;
+  Indices indices_;
+  Schemas schemas_;
+
+  // Transaction engine
+  uint64_t timestamp_{kTimestampInitialId};
+  uint64_t transaction_id_{kTransactionInitialId};
+  // TODO: This isn't really a commit log, it doesn't even care if a
+  // transaction commited or aborted. We could probably combine this with
+  // `timestamp_` in a sensible unit, something like TransactionClock or
+  // whatever.
+  std::optional<CommitLog> commit_log_;
+
+  std::list<Transaction> committed_transactions_;
+  IsolationLevel isolation_level_;
+
+  Config config_;
+
+  // Undo buffers that were unlinked and now are waiting to be freed.
+  std::list<std::pair<uint64_t, std::list<Delta>>> garbage_undo_buffers_;
+
+  // Vertices that are logically deleted but still have to be removed from
+  // indices before removing them from the main storage.
+  std::list<PrimaryKey> deleted_vertices_;
+
+  // Vertices that are logically deleted and removed from indices and now wait
+  // to be removed from the main storage.
+  std::list<std::pair<uint64_t, PrimaryKey>> garbage_vertices_;
+
+  // Edges that are logically deleted and wait to be removed from the main
+  // storage.
+  std::list<Gid> deleted_edges_;
+
+  // Durability
+  std::filesystem::path snapshot_directory_;
+  std::filesystem::path wal_directory_;
+  std::filesystem::path lock_file_path_;
+  utils::OutputFile lock_file_handle_;
+
+  // UUID used to distinguish snapshots and to link snapshots to WALs
+  std::string uuid_;
+  // Sequence number used to keep track of the chain of WALs.
+  uint64_t wal_seq_num_{0};
+
+  // UUID to distinguish different main instance runs for replication process
+  // on SAME storage.
+  // Multiple instances can have same storage UUID and be MAIN at the same time.
+  // We cannot compare commit timestamps of those instances if one of them
+  // becomes the replica of the other so we use epoch_id_ as additional
+  // discriminating property.
+  // Example of this:
+  // We have 2 instances of the same storage, S1 and S2.
+  // S1 and S2 are MAIN and accept their own commits and write them to the WAL.
+  // At the moment when S1 commited a transaction with timestamp 20, and S2
+  // a different transaction with timestamp 15, we change S2's role to REPLICA
+  // and register it on S1.
+  // Without using the epoch_id, we don't know that S1 and S2 have completely
+  // different transactions, we think that the S2 is behind only by 5 commits.
+  std::string epoch_id_;
+  // History of the previous epoch ids.
+  // Each value consists of the epoch id along the last commit belonging to that
+  // epoch.
+  std::deque<std::pair<std::string, uint64_t>> epoch_history_;
+
+  std::optional<durability::WalFile> wal_file_;
+  uint64_t wal_unsynced_transactions_{0};
+
+  utils::FileRetainer file_retainer_;
+
+  // Global locker that is used for clients file locking
+  utils::FileRetainer::FileLocker global_locker_;
+
+  // Last commited timestamp
+  uint64_t last_commit_timestamp_{kTimestampInitialId};
+
+  class ReplicationServer;
+  std::unique_ptr<ReplicationServer> replication_server_{nullptr};
+
+  class ReplicationClient;
+  // We create ReplicationClient using unique_ptr so we can move
+  // newly created client into the vector.
+  // We cannot move the client directly because it contains ThreadPool
+  // which cannot be moved. Also, the move is necessary because
+  // we don't want to create the client directly inside the vector
+  // because that would require the lock on the list putting all
+  // commits (they iterate list of clients) to halt.
+  // This way we can initialize client in main thread which means
+  // that we can immediately notify the user if the initialization
+  // failed.
+  using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
+  ReplicationClientList replication_clients_;
+
+  ReplicationRole replication_role_{ReplicationRole::MAIN};
+};
+
+}  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/storage.cpp b/src/storage/v3/storage.cpp
index 23dd45427..28fd8b330 100644
--- a/src/storage/v3/storage.cpp
+++ b/src/storage/v3/storage.cpp
@@ -11,1853 +11,10 @@
 
 #include "storage/v3/storage.hpp"
 
-#include <algorithm>
-#include <atomic>
-#include <cstdint>
-#include <iterator>
-#include <memory>
-#include <mutex>
-#include <optional>
-#include <variant>
-
-#include <bits/ranges_algo.h>
-#include <gflags/gflags.h>
-#include <spdlog/spdlog.h>
-
-#include "io/network/endpoint.hpp"
-#include "storage/v3/constraints.hpp"
-#include "storage/v3/durability/durability.hpp"
-#include "storage/v3/durability/metadata.hpp"
-#include "storage/v3/durability/paths.hpp"
-#include "storage/v3/durability/snapshot.hpp"
-#include "storage/v3/durability/wal.hpp"
-#include "storage/v3/edge_accessor.hpp"
-#include "storage/v3/id_types.hpp"
-#include "storage/v3/indices.hpp"
-#include "storage/v3/key_store.hpp"
-#include "storage/v3/mvcc.hpp"
-#include "storage/v3/property_value.hpp"
-#include "storage/v3/replication/config.hpp"
-#include "storage/v3/replication/replication_client.hpp"
-#include "storage/v3/replication/replication_server.hpp"
-#include "storage/v3/replication/rpc.hpp"
-#include "storage/v3/schema_validator.hpp"
-#include "storage/v3/transaction.hpp"
-#include "storage/v3/vertex.hpp"
-#include "storage/v3/vertex_accessor.hpp"
-#include "storage/v3/vertices_skip_list.hpp"
-#include "utils/exceptions.hpp"
-#include "utils/file.hpp"
-#include "utils/logging.hpp"
-#include "utils/memory_tracker.hpp"
-#include "utils/message.hpp"
-#include "utils/result.hpp"
-#include "utils/rw_lock.hpp"
-#include "utils/skip_list.hpp"
-#include "utils/spin_lock.hpp"
-#include "utils/stat.hpp"
-#include "utils/uuid.hpp"
+#include "storage/v3/config.hpp"
 
 namespace memgraph::storage::v3 {
 
-using OOMExceptionEnabler = utils::MemoryTracker::OutOfMemoryExceptionEnabler;
-
-namespace {
-inline constexpr uint16_t kEpochHistoryRetention = 1000;
-
-void InsertVertexPKIntoList(auto &container, const PrimaryKey &primary_key) { container.push_back(primary_key); }
-}  // namespace
-
-auto AdvanceToVisibleVertex(VerticesSkipList::Iterator it, VerticesSkipList::Iterator end,
-                            std::optional<VertexAccessor> *vertex, Transaction *tx, View view, Indices *indices,
-                            Constraints *constraints, Config::Items config, const SchemaValidator &schema_validator) {
-  while (it != end) {
-    *vertex = VertexAccessor::Create(&it->vertex, tx, indices, constraints, config, schema_validator, view);
-    if (!*vertex) {
-      ++it;
-      continue;
-    }
-    break;
-  }
-  return it;
-}
-
-AllVerticesIterable::Iterator::Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it)
-    : self_(self),
-      it_(AdvanceToVisibleVertex(it, self->vertices_accessor_.end(), &self->vertex_, self->transaction_, self->view_,
-                                 self->indices_, self_->constraints_, self->config_, *self_->schema_validator_)) {}
-
-VertexAccessor AllVerticesIterable::Iterator::operator*() const { return *self_->vertex_; }
-
-AllVerticesIterable::Iterator &AllVerticesIterable::Iterator::operator++() {
-  ++it_;
-  it_ = AdvanceToVisibleVertex(it_, self_->vertices_accessor_.end(), &self_->vertex_, self_->transaction_, self_->view_,
-                               self_->indices_, self_->constraints_, self_->config_, *self_->schema_validator_);
-  return *this;
-}
-
-VerticesIterable::VerticesIterable(AllVerticesIterable vertices) : type_(Type::ALL) {
-  new (&all_vertices_) AllVerticesIterable(std::move(vertices));
-}
-
-VerticesIterable::VerticesIterable(LabelIndex::Iterable vertices) : type_(Type::BY_LABEL) {
-  new (&vertices_by_label_) LabelIndex::Iterable(std::move(vertices));
-}
-
-VerticesIterable::VerticesIterable(LabelPropertyIndex::Iterable vertices) : type_(Type::BY_LABEL_PROPERTY) {
-  new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(vertices));
-}
-
-VerticesIterable::VerticesIterable(VerticesIterable &&other) noexcept : type_(other.type_) {
-  switch (other.type_) {
-    case Type::ALL:
-      new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_));
-      break;
-    case Type::BY_LABEL:
-      new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_));
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_));
-      break;
-  }
-}
-
-VerticesIterable &VerticesIterable::operator=(VerticesIterable &&other) noexcept {
-  switch (type_) {
-    case Type::ALL:
-      all_vertices_.AllVerticesIterable::~AllVerticesIterable();
-      break;
-    case Type::BY_LABEL:
-      vertices_by_label_.LabelIndex::Iterable::~Iterable();
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable();
-      break;
-  }
-  type_ = other.type_;
-  switch (other.type_) {
-    case Type::ALL:
-      new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_));
-      break;
-    case Type::BY_LABEL:
-      new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_));
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_));
-      break;
-  }
-  return *this;
-}
-
-VerticesIterable::~VerticesIterable() {
-  switch (type_) {
-    case Type::ALL:
-      all_vertices_.AllVerticesIterable::~AllVerticesIterable();
-      break;
-    case Type::BY_LABEL:
-      vertices_by_label_.LabelIndex::Iterable::~Iterable();
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable();
-      break;
-  }
-}
-
-VerticesIterable::Iterator VerticesIterable::begin() {
-  switch (type_) {
-    case Type::ALL:
-      return Iterator(all_vertices_.begin());
-    case Type::BY_LABEL:
-      return Iterator(vertices_by_label_.begin());
-    case Type::BY_LABEL_PROPERTY:
-      return Iterator(vertices_by_label_property_.begin());
-  }
-}
-
-VerticesIterable::Iterator VerticesIterable::end() {
-  switch (type_) {
-    case Type::ALL:
-      return Iterator(all_vertices_.end());
-    case Type::BY_LABEL:
-      return Iterator(vertices_by_label_.end());
-    case Type::BY_LABEL_PROPERTY:
-      return Iterator(vertices_by_label_property_.end());
-  }
-}
-
-VerticesIterable::Iterator::Iterator(AllVerticesIterable::Iterator it) : type_(Type::ALL) {
-  new (&all_it_) AllVerticesIterable::Iterator(it);
-}
-
-VerticesIterable::Iterator::Iterator(LabelIndex::Iterable::Iterator it) : type_(Type::BY_LABEL) {
-  new (&by_label_it_) LabelIndex::Iterable::Iterator(it);
-}
-
-VerticesIterable::Iterator::Iterator(LabelPropertyIndex::Iterable::Iterator it) : type_(Type::BY_LABEL_PROPERTY) {
-  new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(it);
-}
-
-VerticesIterable::Iterator::Iterator(const VerticesIterable::Iterator &other) : type_(other.type_) {
-  switch (other.type_) {
-    case Type::ALL:
-      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
-      break;
-    case Type::BY_LABEL:
-      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
-      break;
-  }
-}
-
-VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(const VerticesIterable::Iterator &other) {
-  if (this == &other) {
-    return *this;
-  }
-  Destroy();
-  type_ = other.type_;
-  switch (other.type_) {
-    case Type::ALL:
-      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
-      break;
-    case Type::BY_LABEL:
-      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
-      break;
-  }
-  return *this;
-}
-
-VerticesIterable::Iterator::Iterator(VerticesIterable::Iterator &&other) noexcept : type_(other.type_) {
-  switch (other.type_) {
-    case Type::ALL:
-      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
-      break;
-    case Type::BY_LABEL:
-      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
-      break;
-  }
-}
-
-VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(VerticesIterable::Iterator &&other) noexcept {
-  Destroy();
-  type_ = other.type_;
-  switch (other.type_) {
-    case Type::ALL:
-      new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
-      break;
-    case Type::BY_LABEL:
-      new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_);
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_);
-      break;
-  }
-  return *this;
-}
-
-VerticesIterable::Iterator::~Iterator() { Destroy(); }
-
-void VerticesIterable::Iterator::Destroy() noexcept {
-  switch (type_) {
-    case Type::ALL:
-      all_it_.AllVerticesIterable::Iterator::~Iterator();
-      break;
-    case Type::BY_LABEL:
-      by_label_it_.LabelIndex::Iterable::Iterator::~Iterator();
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      by_label_property_it_.LabelPropertyIndex::Iterable::Iterator::~Iterator();
-      break;
-  }
-}
-
-VertexAccessor VerticesIterable::Iterator::operator*() const {
-  switch (type_) {
-    case Type::ALL:
-      return *all_it_;
-    case Type::BY_LABEL:
-      return *by_label_it_;
-    case Type::BY_LABEL_PROPERTY:
-      return *by_label_property_it_;
-  }
-}
-
-VerticesIterable::Iterator &VerticesIterable::Iterator::operator++() {
-  switch (type_) {
-    case Type::ALL:
-      ++all_it_;
-      break;
-    case Type::BY_LABEL:
-      ++by_label_it_;
-      break;
-    case Type::BY_LABEL_PROPERTY:
-      ++by_label_property_it_;
-      break;
-  }
-  return *this;
-}
-
-bool VerticesIterable::Iterator::operator==(const Iterator &other) const {
-  switch (type_) {
-    case Type::ALL:
-      return all_it_ == other.all_it_;
-    case Type::BY_LABEL:
-      return by_label_it_ == other.by_label_it_;
-    case Type::BY_LABEL_PROPERTY:
-      return by_label_property_it_ == other.by_label_property_it_;
-  }
-}
-
-Storage::Storage(Config config)
-    : schema_validator_(schemas_),
-      indices_(&constraints_, config.items, schema_validator_),
-      isolation_level_(config.transaction.isolation_level),
-      config_(config),
-      snapshot_directory_(config_.durability.storage_directory / durability::kSnapshotDirectory),
-      wal_directory_(config_.durability.storage_directory / durability::kWalDirectory),
-      lock_file_path_(config_.durability.storage_directory / durability::kLockFile),
-      uuid_(utils::GenerateUUID()),
-      epoch_id_(utils::GenerateUUID()),
-      global_locker_(file_retainer_.AddLocker()) {
-  if (config_.durability.snapshot_wal_mode == Config::Durability::SnapshotWalMode::DISABLED &&
-      replication_role_ == ReplicationRole::MAIN) {
-    spdlog::warn(
-        "The instance has the MAIN replication role, but durability logs and snapshots are disabled. Please consider "
-        "enabling durability by using --storage-snapshot-interval-sec and --storage-wal-enabled flags because "
-        "without write-ahead logs this instance is not replicating any data.");
-  }
-  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED ||
-      config_.durability.snapshot_on_exit || config_.durability.recover_on_startup) {
-    // Create the directory initially to crash the database in case of
-    // permission errors. This is done early to crash the database on startup
-    // instead of crashing the database for the first time during runtime (which
-    // could be an unpleasant surprise).
-    utils::EnsureDirOrDie(snapshot_directory_);
-    // Same reasoning as above.
-    utils::EnsureDirOrDie(wal_directory_);
-
-    // Verify that the user that started the process is the same user that is
-    // the owner of the storage directory.
-    durability::VerifyStorageDirectoryOwnerAndProcessUserOrDie(config_.durability.storage_directory);
-
-    // Create the lock file and open a handle to it. This will crash the
-    // database if it can't open the file for writing or if any other process is
-    // holding the file opened.
-    lock_file_handle_.Open(lock_file_path_, utils::OutputFile::Mode::OVERWRITE_EXISTING);
-    MG_ASSERT(lock_file_handle_.AcquireLock(),
-              "Couldn't acquire lock on the storage directory {}"
-              "!\nAnother Memgraph process is currently running with the same "
-              "storage directory, please stop it first before starting this "
-              "process!",
-              config_.durability.storage_directory);
-  }
-  if (config_.durability.recover_on_startup) {
-    auto info = std::optional<durability::RecoveryInfo>{};
-
-    // durability::RecoverData(snapshot_directory_, wal_directory_, &uuid_, &epoch_id_, &epoch_history_, &vertices_,
-    //                         &edges_, &edge_count_, &name_id_mapper_, &indices_, &constraints_, config_.items,
-    //                         &wal_seq_num_);
-    if (info) {
-      edge_id_ = info->next_edge_id;
-      timestamp_ = std::max(timestamp_, info->next_timestamp);
-      if (info->last_commit_timestamp) {
-        last_commit_timestamp_ = *info->last_commit_timestamp;
-      }
-    }
-  } else if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED ||
-             config_.durability.snapshot_on_exit) {
-    bool files_moved = false;
-    auto backup_root = config_.durability.storage_directory / durability::kBackupDirectory;
-    for (const auto &[path, dirname, what] :
-         {std::make_tuple(snapshot_directory_, durability::kSnapshotDirectory, "snapshot"),
-          std::make_tuple(wal_directory_, durability::kWalDirectory, "WAL")}) {
-      if (!utils::DirExists(path)) continue;
-      auto backup_curr = backup_root / dirname;
-      std::error_code error_code;
-      for (const auto &item : std::filesystem::directory_iterator(path, error_code)) {
-        utils::EnsureDirOrDie(backup_root);
-        utils::EnsureDirOrDie(backup_curr);
-        std::error_code item_error_code;
-        std::filesystem::rename(item.path(), backup_curr / item.path().filename(), item_error_code);
-        MG_ASSERT(!item_error_code, "Couldn't move {} file {} because of: {}", what, item.path(),
-                  item_error_code.message());
-        files_moved = true;
-      }
-      MG_ASSERT(!error_code, "Couldn't backup {} files because of: {}", what, error_code.message());
-    }
-    if (files_moved) {
-      spdlog::warn(
-          "Since Memgraph was not supposed to recover on startup and "
-          "durability is enabled, your current durability files will likely "
-          "be overridden. To prevent important data loss, Memgraph has stored "
-          "those files into a .backup directory inside the storage directory.");
-    }
-  }
-  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) {
-    // TODO(antaljanosbenjamin): handle snapshots
-    // snapshot_runner_.Run("Snapshot", config_.durability.snapshot_interval, [this] {
-    //   if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) {
-    //     switch (maybe_error.GetError()) {
-    //       case CreateSnapshotError::DisabledForReplica:
-    //         spdlog::warn(
-    //             utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication"));
-    //         break;
-    //     }
-    //   }
-    // });
-  }
-
-  if (timestamp_ == kTimestampInitialId) {
-    commit_log_.emplace();
-  } else {
-    commit_log_.emplace(timestamp_);
-  }
-}
-
-Storage::~Storage() {
-  {
-    // Clear replication data
-    replication_server_.reset();
-    replication_clients_.WithLock([&](auto &clients) { clients.clear(); });
-  }
-  if (wal_file_) {
-    wal_file_->FinalizeWal();
-    wal_file_ = std::nullopt;
-  }
-  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) {
-    // TODO(antaljanosbenjamin): stop snapshot creation
-  }
-  if (config_.durability.snapshot_on_exit) {
-    if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) {
-      switch (maybe_error.GetError()) {
-        case CreateSnapshotError::DisabledForReplica:
-          spdlog::warn(utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication"));
-          break;
-      }
-    }
-  }
-}
-
-Storage::Accessor::Accessor(Storage *storage, IsolationLevel isolation_level)
-    : storage_(storage),
-      transaction_(storage->CreateTransaction(isolation_level)),
-      is_transaction_active_(true),
-      config_(storage->config_.items) {}
-
-Storage::Accessor::Accessor(Accessor &&other) noexcept
-    : storage_(other.storage_),
-      transaction_(std::move(other.transaction_)),
-      commit_timestamp_(other.commit_timestamp_),
-      is_transaction_active_(other.is_transaction_active_),
-      config_(other.config_) {
-  // Don't allow the other accessor to abort our transaction in destructor.
-  other.is_transaction_active_ = false;
-  other.commit_timestamp_.reset();
-}
-
-Storage::Accessor::~Accessor() {
-  if (is_transaction_active_) {
-    Abort();
-  }
-
-  FinalizeTransaction();
-}
-
-ResultSchema<VertexAccessor> Storage::Accessor::CreateVertexAndValidate(
-    LabelId primary_label, const std::vector<LabelId> &labels,
-    const std::vector<std::pair<PropertyId, PropertyValue>> &properties) {
-  auto maybe_schema_violation = GetSchemaValidator().ValidateVertexCreate(primary_label, labels, properties);
-  if (maybe_schema_violation) {
-    return {std::move(*maybe_schema_violation)};
-  }
-  OOMExceptionEnabler oom_exception;
-  // Extract key properties
-  std::vector<PropertyValue> primary_properties;
-  for ([[maybe_unused]] const auto &[property_id, property_type] : storage_->GetSchema(primary_label)->second) {
-    // We know there definitely is key in properties since we have validated
-    primary_properties.push_back(
-        std::ranges::find_if(properties, [property_id = property_id](const auto &property_pair) {
-          return property_pair.first == property_id;
-        })->second);
-  }
-
-  // Get secondary properties
-  std::vector<std::pair<PropertyId, PropertyValue>> secondary_properties;
-  for (const auto &[property_id, property_value] : properties) {
-    if (!storage_->schemas_.IsPropertyKey(primary_label, property_id)) {
-      secondary_properties.emplace_back(property_id, property_value);
-    }
-  }
-
-  auto acc = storage_->vertices_.access();
-  auto *delta = CreateDeleteObjectDelta(&transaction_);
-  auto [it, inserted] = acc.insert({Vertex{delta, primary_label, primary_properties, labels, secondary_properties}});
-  MG_ASSERT(inserted, "The vertex must be inserted here!");
-  MG_ASSERT(it != acc.end(), "Invalid Vertex accessor!");
-  delta->prev.Set(&it->vertex);
-  return VertexAccessor{
-      &it->vertex, &transaction_, &storage_->indices_, &storage_->constraints_, config_, storage_->schema_validator_};
-}
-
-std::optional<VertexAccessor> Storage::Accessor::FindVertex(std::vector<PropertyValue> primary_key, View view) {
-  auto acc = storage_->vertices_.access();
-  // Later on use label space
-  auto it = acc.find(primary_key);
-  if (it == acc.end()) {
-    return std::nullopt;
-  }
-  return VertexAccessor::Create(&it->vertex, &transaction_, &storage_->indices_, &storage_->constraints_, config_,
-                                storage_->schema_validator_, view);
-}
-
-Result<std::optional<VertexAccessor>> Storage::Accessor::DeleteVertex(VertexAccessor *vertex) {
-  MG_ASSERT(vertex->transaction_ == &transaction_,
-            "VertexAccessor must be from the same transaction as the storage "
-            "accessor when deleting a vertex!");
-  auto *vertex_ptr = vertex->vertex_;
-
-  if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
-
-  if (vertex_ptr->deleted) {
-    return std::optional<VertexAccessor>{};
-  }
-
-  if (!vertex_ptr->in_edges.empty() || !vertex_ptr->out_edges.empty()) return Error::VERTEX_HAS_EDGES;
-
-  CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag());
-  vertex_ptr->deleted = true;
-
-  return std::make_optional<VertexAccessor>(vertex_ptr, &transaction_, &storage_->indices_, &storage_->constraints_,
-                                            config_, storage_->schema_validator_, true);
-}
-
-Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Storage::Accessor::DetachDeleteVertex(
-    VertexAccessor *vertex) {
-  using ReturnType = std::pair<VertexAccessor, std::vector<EdgeAccessor>>;
-
-  MG_ASSERT(vertex->transaction_ == &transaction_,
-            "VertexAccessor must be from the same transaction as the storage "
-            "accessor when deleting a vertex!");
-  auto *vertex_ptr = vertex->vertex_;
-
-  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
-  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
-
-  {
-    if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
-
-    if (vertex_ptr->deleted) return std::optional<ReturnType>{};
-
-    in_edges = vertex_ptr->in_edges;
-    out_edges = vertex_ptr->out_edges;
-  }
-
-  std::vector<EdgeAccessor> deleted_edges;
-  for (const auto &item : in_edges) {
-    auto [edge_type, from_vertex, edge] = item;
-    EdgeAccessor e(edge, edge_type, from_vertex, vertex_ptr, &transaction_, &storage_->indices_,
-                   &storage_->constraints_, config_, storage_->schema_validator_);
-    auto ret = DeleteEdge(&e);
-    if (ret.HasError()) {
-      MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
-      return ret.GetError();
-    }
-
-    if (ret.GetValue()) {
-      deleted_edges.push_back(*ret.GetValue());
-    }
-  }
-  for (const auto &item : out_edges) {
-    auto [edge_type, to_vertex, edge] = item;
-    EdgeAccessor e(edge, edge_type, vertex_ptr, to_vertex, &transaction_, &storage_->indices_, &storage_->constraints_,
-                   config_, storage_->schema_validator_);
-    auto ret = DeleteEdge(&e);
-    if (ret.HasError()) {
-      MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
-      return ret.GetError();
-    }
-
-    if (ret.GetValue()) {
-      deleted_edges.push_back(*ret.GetValue());
-    }
-  }
-
-  // We need to check again for serialization errors because we unlocked the
-  // vertex. Some other transaction could have modified the vertex in the
-  // meantime if we didn't have any edges to delete.
-
-  if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
-
-  MG_ASSERT(!vertex_ptr->deleted, "Invalid database state!");
-
-  CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag());
-  vertex_ptr->deleted = true;
-
-  return std::make_optional<ReturnType>(
-      VertexAccessor{vertex_ptr, &transaction_, &storage_->indices_, &storage_->constraints_, config_,
-                     storage_->schema_validator_, true},
-      std::move(deleted_edges));
-}
-
-Result<EdgeAccessor> Storage::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) {
-  OOMExceptionEnabler oom_exception;
-  MG_ASSERT(from->transaction_ == to->transaction_,
-            "VertexAccessors must be from the same transaction when creating "
-            "an edge!");
-  MG_ASSERT(from->transaction_ == &transaction_,
-            "VertexAccessors must be from the same transaction in when "
-            "creating an edge!");
-
-  auto *from_vertex = from->vertex_;
-  auto *to_vertex = to->vertex_;
-
-  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
-  if (from_vertex->deleted) return Error::DELETED_OBJECT;
-
-  if (to_vertex != from_vertex) {
-    if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
-    if (to_vertex->deleted) return Error::DELETED_OBJECT;
-  }
-
-  auto gid = Gid::FromUint(storage_->edge_id_++);
-  EdgeRef edge(gid);
-  if (config_.properties_on_edges) {
-    auto acc = storage_->edges_.access();
-    auto *delta = CreateDeleteObjectDelta(&transaction_);
-    auto [it, inserted] = acc.insert(Edge(gid, delta));
-    MG_ASSERT(inserted, "The edge must be inserted here!");
-    MG_ASSERT(it != acc.end(), "Invalid Edge accessor!");
-    edge = EdgeRef(&*it);
-    delta->prev.Set(&*it);
-  }
-
-  CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex, edge);
-  from_vertex->out_edges.emplace_back(edge_type, to_vertex, edge);
-
-  CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex, edge);
-  to_vertex->in_edges.emplace_back(edge_type, from_vertex, edge);
-
-  // Increment edge count.
-  ++storage_->edge_count_;
-
-  return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &storage_->indices_,
-                      &storage_->constraints_, config_, storage_->schema_validator_);
-}
-
-Result<EdgeAccessor> Storage::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type,
-                                                   Gid gid) {
-  OOMExceptionEnabler oom_exception;
-  MG_ASSERT(from->transaction_ == to->transaction_,
-            "VertexAccessors must be from the same transaction when creating "
-            "an edge!");
-  MG_ASSERT(from->transaction_ == &transaction_,
-            "VertexAccessors must be from the same transaction in when "
-            "creating an edge!");
-
-  auto *from_vertex = from->vertex_;
-  auto *to_vertex = to->vertex_;
-
-  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
-  if (from_vertex->deleted) return Error::DELETED_OBJECT;
-
-  if (to_vertex != from_vertex) {
-    if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
-    if (to_vertex->deleted) return Error::DELETED_OBJECT;
-  }
-
-  // NOTE: When we update the next `edge_id_` here we perform a RMW
-  // (read-modify-write) operation that ISN'T atomic! But, that isn't an issue
-  // because this function is only called from the replication delta applier
-  // that runs single-threadedly and while this instance is set-up to apply
-  // threads (it is the replica), it is guaranteed that no other writes are
-  // possible.
-  storage_->edge_id_ = std::max(storage_->edge_id_, gid.AsUint() + 1);
-
-  EdgeRef edge(gid);
-  if (config_.properties_on_edges) {
-    auto acc = storage_->edges_.access();
-    auto *delta = CreateDeleteObjectDelta(&transaction_);
-    auto [it, inserted] = acc.insert(Edge(gid, delta));
-    MG_ASSERT(inserted, "The edge must be inserted here!");
-    MG_ASSERT(it != acc.end(), "Invalid Edge accessor!");
-    edge = EdgeRef(&*it);
-    delta->prev.Set(&*it);
-  }
-
-  CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex, edge);
-  from_vertex->out_edges.emplace_back(edge_type, to_vertex, edge);
-
-  CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex, edge);
-  to_vertex->in_edges.emplace_back(edge_type, from_vertex, edge);
-
-  // Increment edge count.
-  ++storage_->edge_count_;
-
-  return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &storage_->indices_,
-                      &storage_->constraints_, config_, storage_->schema_validator_);
-}
-
-Result<std::optional<EdgeAccessor>> Storage::Accessor::DeleteEdge(EdgeAccessor *edge) {
-  MG_ASSERT(edge->transaction_ == &transaction_,
-            "EdgeAccessor must be from the same transaction as the storage "
-            "accessor when deleting an edge!");
-  auto edge_ref = edge->edge_;
-  auto edge_type = edge->edge_type_;
-
-  if (config_.properties_on_edges) {
-    auto *edge_ptr = edge_ref.ptr;
-
-    if (!PrepareForWrite(&transaction_, edge_ptr)) return Error::SERIALIZATION_ERROR;
-
-    if (edge_ptr->deleted) return std::optional<EdgeAccessor>{};
-  }
-
-  auto *from_vertex = edge->from_vertex_;
-  auto *to_vertex = edge->to_vertex_;
-
-  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
-  MG_ASSERT(!from_vertex->deleted, "Invalid database state!");
-
-  if (to_vertex != from_vertex) {
-    if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
-    MG_ASSERT(!to_vertex->deleted, "Invalid database state!");
-  }
-
-  auto delete_edge_from_storage = [&edge_type, &edge_ref, this](auto *vertex, auto *edges) {
-    std::tuple<EdgeTypeId, Vertex *, EdgeRef> link(edge_type, vertex, edge_ref);
-    auto it = std::find(edges->begin(), edges->end(), link);
-    if (config_.properties_on_edges) {
-      MG_ASSERT(it != edges->end(), "Invalid database state!");
-    } else if (it == edges->end()) {
-      return false;
-    }
-    std::swap(*it, *edges->rbegin());
-    edges->pop_back();
-    return true;
-  };
-
-  auto op1 = delete_edge_from_storage(to_vertex, &from_vertex->out_edges);
-  auto op2 = delete_edge_from_storage(from_vertex, &to_vertex->in_edges);
-
-  if (config_.properties_on_edges) {
-    MG_ASSERT((op1 && op2), "Invalid database state!");
-  } else {
-    MG_ASSERT((op1 && op2) || (!op1 && !op2), "Invalid database state!");
-    if (!op1 && !op2) {
-      // The edge is already deleted.
-      return std::optional<EdgeAccessor>{};
-    }
-  }
-
-  if (config_.properties_on_edges) {
-    auto *edge_ptr = edge_ref.ptr;
-    CreateAndLinkDelta(&transaction_, edge_ptr, Delta::RecreateObjectTag());
-    edge_ptr->deleted = true;
-  }
-
-  CreateAndLinkDelta(&transaction_, from_vertex, Delta::AddOutEdgeTag(), edge_type, to_vertex, edge_ref);
-  CreateAndLinkDelta(&transaction_, to_vertex, Delta::AddInEdgeTag(), edge_type, from_vertex, edge_ref);
-
-  // Decrement edge count.
-  --storage_->edge_count_;
-
-  return std::make_optional<EdgeAccessor>(edge_ref, edge_type, from_vertex, to_vertex, &transaction_,
-                                          &storage_->indices_, &storage_->constraints_, config_,
-                                          storage_->schema_validator_, true);
-}
-
-const std::string &Storage::Accessor::LabelToName(LabelId label) const { return storage_->LabelToName(label); }
-
-const std::string &Storage::Accessor::PropertyToName(PropertyId property) const {
-  return storage_->PropertyToName(property);
-}
-
-const std::string &Storage::Accessor::EdgeTypeToName(EdgeTypeId edge_type) const {
-  return storage_->EdgeTypeToName(edge_type);
-}
-
-LabelId Storage::Accessor::NameToLabel(const std::string_view name) { return storage_->NameToLabel(name); }
-
-PropertyId Storage::Accessor::NameToProperty(const std::string_view name) { return storage_->NameToProperty(name); }
-
-EdgeTypeId Storage::Accessor::NameToEdgeType(const std::string_view name) { return storage_->NameToEdgeType(name); }
-
-void Storage::Accessor::AdvanceCommand() { ++transaction_.command_id; }
-
-utils::BasicResult<ConstraintViolation, void> Storage::Accessor::Commit(
-    const std::optional<uint64_t> desired_commit_timestamp) {
-  MG_ASSERT(is_transaction_active_, "The transaction is already terminated!");
-  MG_ASSERT(!transaction_.must_abort, "The transaction can't be committed!");
-
-  if (transaction_.deltas.empty()) {
-    // We don't have to update the commit timestamp here because no one reads
-    // it.
-    storage_->commit_log_->MarkFinished(transaction_.start_timestamp);
-  } else {
-    // Validate that existence constraints are satisfied for all modified
-    // vertices.
-    for (const auto &delta : transaction_.deltas) {
-      auto prev = delta.prev.Get();
-      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-      if (prev.type != PreviousPtr::Type::VERTEX) {
-        continue;
-      }
-      // No need to take any locks here because we modified this vertex and no
-      // one else can touch it until we commit.
-      auto validation_result = ValidateExistenceConstraints(*prev.vertex, storage_->constraints_);
-      if (validation_result) {
-        Abort();
-        return {*validation_result};
-      }
-    }
-
-    // Result of validating the vertex against unique constraints. It has to be
-    // declared outside of the critical section scope because its value is
-    // tested for Abort call which has to be done out of the scope.
-    std::optional<ConstraintViolation> unique_constraint_violation;
-
-    // Save these so we can mark them used in the commit log.
-    uint64_t start_timestamp = transaction_.start_timestamp;
-
-    commit_timestamp_.emplace(storage_->CommitTimestamp(desired_commit_timestamp));
-
-    // Before committing and validating vertices against unique constraints,
-    // we have to update unique constraints with the vertices that are going
-    // to be validated/committed.
-    for (const auto &delta : transaction_.deltas) {
-      auto prev = delta.prev.Get();
-      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-      if (prev.type != PreviousPtr::Type::VERTEX) {
-        continue;
-      }
-      storage_->constraints_.unique_constraints.UpdateBeforeCommit(prev.vertex, transaction_);
-    }
-
-    // Validate that unique constraints are satisfied for all modified
-    // vertices.
-    for (const auto &delta : transaction_.deltas) {
-      auto prev = delta.prev.Get();
-      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-      if (prev.type != PreviousPtr::Type::VERTEX) {
-        continue;
-      }
-
-      // No need to take any locks here because we modified this vertex and no
-      // one else can touch it until we commit.
-      unique_constraint_violation =
-          storage_->constraints_.unique_constraints.Validate(*prev.vertex, transaction_, *commit_timestamp_);
-      if (unique_constraint_violation) {
-        break;
-      }
-    }
-
-    if (!unique_constraint_violation) {
-      // Write transaction to WAL while holding the engine lock to make sure
-      // that committed transactions are sorted by the commit timestamp in the
-      // WAL files. We supply the new commit timestamp to the function so that
-      // it knows what will be the final commit timestamp. The WAL must be
-      // written before actually committing the transaction (before setting
-      // the commit timestamp) so that no other transaction can see the
-      // modifications before they are written to disk.
-      // Replica can log only the write transaction received from Main
-      // so the Wal files are consistent
-      if (storage_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) {
-        storage_->AppendToWal(transaction_, *commit_timestamp_);
-      }
-
-      // TODO(antaljanosbenjamin): Figure out:
-      //   1. How the committed transactions are sorted in `committed_transactions_`
-      //   2. Why it was necessary to lock `committed_transactions_` when it was not accessed at all
-      // TODO: Update all deltas to have a local copy of the commit timestamp
-      MG_ASSERT(transaction_.commit_timestamp != nullptr, "Invalid database state!");
-      transaction_.commit_timestamp->store(*commit_timestamp_, std::memory_order_release);
-      // Replica can only update the last commit timestamp with
-      // the commits received from main.
-      if (storage_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) {
-        // Update the last commit timestamp
-        storage_->last_commit_timestamp_ = *commit_timestamp_;
-      }
-
-      storage_->commit_log_->MarkFinished(start_timestamp);
-    }
-
-    if (unique_constraint_violation) {
-      Abort();
-      return {*unique_constraint_violation};
-    }
-  }
-  is_transaction_active_ = false;
-
-  return {};
-}
-
-void Storage::Accessor::Abort() {
-  MG_ASSERT(is_transaction_active_, "The transaction is already terminated!");
-
-  for (const auto &delta : transaction_.deltas) {
-    auto prev = delta.prev.Get();
-    switch (prev.type) {
-      case PreviousPtr::Type::VERTEX: {
-        auto *vertex = prev.vertex;
-        Delta *current = vertex->delta;
-        while (current != nullptr &&
-               current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) {
-          switch (current->action) {
-            case Delta::Action::REMOVE_LABEL: {
-              auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label);
-              MG_ASSERT(it != vertex->labels.end(), "Invalid database state!");
-              std::swap(*it, *vertex->labels.rbegin());
-              vertex->labels.pop_back();
-              break;
-            }
-            case Delta::Action::ADD_LABEL: {
-              auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label);
-              MG_ASSERT(it == vertex->labels.end(), "Invalid database state!");
-              vertex->labels.push_back(current->label);
-              break;
-            }
-            case Delta::Action::SET_PROPERTY: {
-              vertex->properties.SetProperty(current->property.key, current->property.value);
-              break;
-            }
-            case Delta::Action::ADD_IN_EDGE: {
-              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
-                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
-              auto it = std::find(vertex->in_edges.begin(), vertex->in_edges.end(), link);
-              MG_ASSERT(it == vertex->in_edges.end(), "Invalid database state!");
-              vertex->in_edges.push_back(link);
-              break;
-            }
-            case Delta::Action::ADD_OUT_EDGE: {
-              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
-                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
-              auto it = std::find(vertex->out_edges.begin(), vertex->out_edges.end(), link);
-              MG_ASSERT(it == vertex->out_edges.end(), "Invalid database state!");
-              vertex->out_edges.push_back(link);
-              // Increment edge count. We only increment the count here because
-              // the information in `ADD_IN_EDGE` and `Edge/RECREATE_OBJECT` is
-              // redundant. Also, `Edge/RECREATE_OBJECT` isn't available when
-              // edge properties are disabled.
-              ++storage_->edge_count_;
-              break;
-            }
-            case Delta::Action::REMOVE_IN_EDGE: {
-              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
-                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
-              auto it = std::find(vertex->in_edges.begin(), vertex->in_edges.end(), link);
-              MG_ASSERT(it != vertex->in_edges.end(), "Invalid database state!");
-              std::swap(*it, *vertex->in_edges.rbegin());
-              vertex->in_edges.pop_back();
-              break;
-            }
-            case Delta::Action::REMOVE_OUT_EDGE: {
-              std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
-                                                             current->vertex_edge.vertex, current->vertex_edge.edge};
-              auto it = std::find(vertex->out_edges.begin(), vertex->out_edges.end(), link);
-              MG_ASSERT(it != vertex->out_edges.end(), "Invalid database state!");
-              std::swap(*it, *vertex->out_edges.rbegin());
-              vertex->out_edges.pop_back();
-              // Decrement edge count. We only decrement the count here because
-              // the information in `REMOVE_IN_EDGE` and `Edge/DELETE_OBJECT` is
-              // redundant. Also, `Edge/DELETE_OBJECT` isn't available when edge
-              // properties are disabled.
-              --storage_->edge_count_;
-              break;
-            }
-            case Delta::Action::DELETE_OBJECT: {
-              vertex->deleted = true;
-              InsertVertexPKIntoList(storage_->deleted_vertices_, vertex->keys.Keys());
-              break;
-            }
-            case Delta::Action::RECREATE_OBJECT: {
-              vertex->deleted = false;
-              break;
-            }
-          }
-          current = current->next.load(std::memory_order_acquire);
-        }
-        vertex->delta = current;
-        if (current != nullptr) {
-          current->prev.Set(vertex);
-        }
-
-        break;
-      }
-      case PreviousPtr::Type::EDGE: {
-        auto *edge = prev.edge;
-        Delta *current = edge->delta;
-        while (current != nullptr &&
-               current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) {
-          switch (current->action) {
-            case Delta::Action::SET_PROPERTY: {
-              edge->properties.SetProperty(current->property.key, current->property.value);
-              break;
-            }
-            case Delta::Action::DELETE_OBJECT: {
-              edge->deleted = true;
-              storage_->deleted_edges_.push_back(edge->gid);
-              break;
-            }
-            case Delta::Action::RECREATE_OBJECT: {
-              edge->deleted = false;
-              break;
-            }
-            case Delta::Action::REMOVE_LABEL:
-            case Delta::Action::ADD_LABEL:
-            case Delta::Action::ADD_IN_EDGE:
-            case Delta::Action::ADD_OUT_EDGE:
-            case Delta::Action::REMOVE_IN_EDGE:
-            case Delta::Action::REMOVE_OUT_EDGE: {
-              LOG_FATAL("Invalid database state!");
-              break;
-            }
-          }
-          current = current->next.load(std::memory_order_acquire);
-        }
-        edge->delta = current;
-        if (current != nullptr) {
-          current->prev.Set(edge);
-        }
-
-        break;
-      }
-      case PreviousPtr::Type::DELTA:
-      // pointer probably couldn't be set because allocation failed
-      case PreviousPtr::Type::NULLPTR:
-        break;
-    }
-  }
-
-  {
-    uint64_t mark_timestamp = storage_->timestamp_;
-
-    // Release engine lock because we don't have to hold it anymore and
-    // emplace back could take a long time.
-    storage_->garbage_undo_buffers_.emplace_back(mark_timestamp, std::move(transaction_.deltas));
-  }
-
-  storage_->commit_log_->MarkFinished(transaction_.start_timestamp);
-  is_transaction_active_ = false;
-}
-
-void Storage::Accessor::FinalizeTransaction() {
-  if (commit_timestamp_) {
-    storage_->commit_log_->MarkFinished(*commit_timestamp_);
-    storage_->committed_transactions_.emplace_back(std::move(transaction_));
-    commit_timestamp_.reset();
-  }
-}
-
-const std::string &Storage::LabelToName(LabelId label) const { return name_id_mapper_.IdToName(label.AsUint()); }
-
-const std::string &Storage::PropertyToName(PropertyId property) const {
-  return name_id_mapper_.IdToName(property.AsUint());
-}
-
-const std::string &Storage::EdgeTypeToName(EdgeTypeId edge_type) const {
-  return name_id_mapper_.IdToName(edge_type.AsUint());
-}
-
-LabelId Storage::NameToLabel(const std::string_view name) { return LabelId::FromUint(name_id_mapper_.NameToId(name)); }
-
-PropertyId Storage::NameToProperty(const std::string_view name) {
-  return PropertyId::FromUint(name_id_mapper_.NameToId(name));
-}
-
-EdgeTypeId Storage::NameToEdgeType(const std::string_view name) {
-  return EdgeTypeId::FromUint(name_id_mapper_.NameToId(name));
-}
-
-bool Storage::CreateIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
-  // TODO Fix Index
-  return false;
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_CREATE, label, {}, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return true;
-}
-
-bool Storage::CreateIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
-  // TODO Fix Index
-  // if (!indices_.label_property_index.CreateIndex(label, property, labelspace.access())) return false;
-  return false;
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE, label, {property}, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return true;
-}
-
-bool Storage::DropIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
-  if (!indices_.label_index.DropIndex(label)) return false;
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_DROP, label, {}, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return true;
-}
-
-bool Storage::DropIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
-  if (!indices_.label_property_index.DropIndex(label, property)) return false;
-  // For a description why using `timestamp_` is correct, see
-  // `CreateIndex(LabelId label)`.
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP, label, {property}, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return true;
-}
-
-IndicesInfo Storage::ListAllIndices() const {
-  return {indices_.label_index.ListIndices(), indices_.label_property_index.ListIndices()};
-}
-
-utils::BasicResult<ConstraintViolation, bool> Storage::CreateExistenceConstraint(
-    LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
-  // TODO Fix constraints
-  // auto ret = ::memgraph::storage::v3::CreateExistenceConstraint(&constraints_, label, property, vertices_.access());
-  // if (ret.HasError() || !ret.GetValue()) return ret;
-  return false;
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE, label, {property}, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return true;
-}
-
-bool Storage::DropExistenceConstraint(LabelId label, PropertyId property,
-                                      const std::optional<uint64_t> desired_commit_timestamp) {
-  if (!::memgraph::storage::v3::DropExistenceConstraint(&constraints_, label, property)) return false;
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP, label, {property}, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return true;
-}
-
-utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> Storage::CreateUniqueConstraint(
-    LabelId label, const std::set<PropertyId> &properties, const std::optional<uint64_t> desired_commit_timestamp) {
-  // TODO Fix constraints
-  // auto ret = constraints_.unique_constraints.CreateConstraint(label, properties, vertices_.access());
-  // if (ret.HasError() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS) {
-  //   return ret;
-  // }
-  return UniqueConstraints::CreationStatus::ALREADY_EXISTS;
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE, label, properties, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return UniqueConstraints::CreationStatus::SUCCESS;
-}
-
-UniqueConstraints::DeletionStatus Storage::DropUniqueConstraint(
-    LabelId label, const std::set<PropertyId> &properties, const std::optional<uint64_t> desired_commit_timestamp) {
-  auto ret = constraints_.unique_constraints.DropConstraint(label, properties);
-  if (ret != UniqueConstraints::DeletionStatus::SUCCESS) {
-    return ret;
-  }
-  const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
-  AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP, label, properties, commit_timestamp);
-  commit_log_->MarkFinished(commit_timestamp);
-  last_commit_timestamp_ = commit_timestamp;
-  return UniqueConstraints::DeletionStatus::SUCCESS;
-}
-
-const SchemaValidator &Storage::Accessor::GetSchemaValidator() const { return storage_->schema_validator_; }
-
-ConstraintsInfo Storage::ListAllConstraints() const {
-  return {ListExistenceConstraints(constraints_), constraints_.unique_constraints.ListConstraints()};
-}
-
-SchemasInfo Storage::ListAllSchemas() const { return {schemas_.ListSchemas()}; }
-
-const Schemas::Schema *Storage::GetSchema(const LabelId primary_label) const {
-  return schemas_.GetSchema(primary_label);
-}
-
-bool Storage::CreateSchema(const LabelId primary_label, const std::vector<SchemaProperty> &schemas_types) {
-  return schemas_.CreateSchema(primary_label, schemas_types);
-}
-
-bool Storage::DropSchema(const LabelId primary_label) { return schemas_.DropSchema(primary_label); }
-
-StorageInfo Storage::GetInfo() const {
-  auto vertex_count = vertices_.size();
-  double average_degree = 0.0;
-  if (vertex_count) {
-    average_degree = 2.0 * static_cast<double>(edge_count_) / static_cast<double>(vertex_count);
-  }
-  return {vertex_count, edge_count_, average_degree, utils::GetMemoryUsage(),
-          utils::GetDirDiskUsage(config_.durability.storage_directory)};
-}
-
-VerticesIterable Storage::Accessor::Vertices(LabelId label, View view) {
-  return VerticesIterable(storage_->indices_.label_index.Vertices(label, view, &transaction_));
-}
-
-VerticesIterable Storage::Accessor::Vertices(LabelId label, PropertyId property, View view) {
-  return VerticesIterable(storage_->indices_.label_property_index.Vertices(
-      label, property, std::nullopt, std::nullopt, view, &transaction_, storage_->schema_validator_));
-}
-
-VerticesIterable Storage::Accessor::Vertices(LabelId label, PropertyId property, const PropertyValue &value,
-                                             View view) {
-  return VerticesIterable(storage_->indices_.label_property_index.Vertices(
-      label, property, utils::MakeBoundInclusive(value), utils::MakeBoundInclusive(value), view, &transaction_,
-      storage_->schema_validator_));
-}
-
-VerticesIterable Storage::Accessor::Vertices(LabelId label, PropertyId property,
-                                             const std::optional<utils::Bound<PropertyValue>> &lower_bound,
-                                             const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) {
-  return VerticesIterable(storage_->indices_.label_property_index.Vertices(
-      label, property, lower_bound, upper_bound, view, &transaction_, storage_->schema_validator_));
-}
-
-Transaction Storage::CreateTransaction(IsolationLevel isolation_level) {
-  // We acquire the transaction engine lock here because we access (and
-  // modify) the transaction engine variables (`transaction_id` and
-  // `timestamp`) below.
-  uint64_t transaction_id{0};
-  uint64_t start_timestamp{0};
-
-  transaction_id = transaction_id_++;
-  // Replica should have only read queries and the write queries
-  // can come from main instance with any past timestamp.
-  // To preserve snapshot isolation we set the start timestamp
-  // of any query on replica to the last commited transaction
-  // which is timestamp_ as only commit of transaction with writes
-  // can change the value of it.
-  if (replication_role_ == ReplicationRole::REPLICA) {
-    start_timestamp = timestamp_;
-  } else {
-    start_timestamp = timestamp_++;
-  }
-
-  return {transaction_id, start_timestamp, isolation_level};
-}
-
-// `force` means there are no active transactions, so everything can be deleted without worrying about removing some
-// data that is used by an active transaction
-template <bool force>
-void Storage::CollectGarbage() {
-  if constexpr (force) {
-    // TODO(antaljanosbenjamin): figure out whether is there any active transaction or not (probably accessors should
-    // increment/decrement a counter). If there are no transactions, then garbage collection can be forced
-    CollectGarbage<false>();
-    return;
-  }
-
-  // Garbage collection must be performed in two phases. In the first phase,
-  // deltas that won't be applied by any transaction anymore are unlinked from
-  // the version chains. They cannot be deleted immediately, because there
-  // might be a transaction that still needs them to terminate the version
-  // chain traversal. They are instead marked for deletion and will be deleted
-  // in the second GC phase in this GC iteration or some of the following
-  // ones.
-
-  uint64_t oldest_active_start_timestamp = commit_log_->OldestActive();
-  // We don't move undo buffers of unlinked transactions to garbage_undo_buffers
-  // list immediately, because we would have to repeatedly take
-  // garbage_undo_buffers lock.
-  std::list<std::pair<uint64_t, std::list<Delta>>> unlinked_undo_buffers;
-
-  // We will only free vertices deleted up until now in this GC cycle, and we
-  // will do it after cleaning-up the indices. That way we are sure that all
-  // vertices that appear in an index also exist in main storage.
-
-  // Flag that will be used to determine whether the Index GC should be run. It
-  // should be run when there were any items that were cleaned up (there were
-  // updates between this run of the GC and the previous run of the GC). This
-  // eliminates high CPU usage when the GC doesn't have to clean up anything.
-  bool run_index_cleanup = !committed_transactions_.empty() || !garbage_undo_buffers_.empty();
-
-  while (true) {
-    // We don't want to hold the lock on commited transactions for too long,
-    // because that prevents other transactions from committing.
-    Transaction *transaction{nullptr};
-    {
-      if (committed_transactions_.empty()) {
-        break;
-      }
-      transaction = &committed_transactions_.front();
-    }
-
-    auto commit_timestamp = transaction->commit_timestamp->load(std::memory_order_acquire);
-    if (commit_timestamp >= oldest_active_start_timestamp) {
-      break;
-    }
-
-    // When unlinking a delta which is the first delta in its version chain,
-    // special care has to be taken to avoid the following race condition:
-    //
-    // [Vertex] --> [Delta A]
-    //
-    //    GC thread: Delta A is the first in its chain, it must be unlinked from
-    //               vertex and marked for deletion
-    //    TX thread: Update vertex and add Delta B with Delta A as next
-    //
-    // [Vertex] --> [Delta B] <--> [Delta A]
-    //
-    //    GC thread: Unlink delta from Vertex
-    //
-    // [Vertex] --> (nullptr)
-    //
-    // When processing a delta that is the first one in its chain, we
-    // obtain the corresponding vertex or edge lock, and then verify that this
-    // delta still is the first in its chain.
-    // When processing a delta that is in the middle of the chain we only
-    // process the final delta of the given transaction in that chain. We
-    // determine the owner of the chain (either a vertex or an edge), obtain the
-    // corresponding lock, and then verify that this delta is still in the same
-    // position as it was before taking the lock.
-    //
-    // Even though the delta chain is lock-free (both `next` and `prev`) the
-    // chain should not be modified without taking the lock from the object that
-    // owns the chain (either a vertex or an edge). Modifying the chain without
-    // taking the lock will cause subtle race conditions that will leave the
-    // chain in a broken state.
-    // The chain can be only read without taking any locks.
-
-    for (Delta &delta : transaction->deltas) {
-      while (true) {
-        auto prev = delta.prev.Get();
-        switch (prev.type) {
-          case PreviousPtr::Type::VERTEX: {
-            Vertex *vertex = prev.vertex;
-            if (vertex->delta != &delta) {
-              // Something changed, we're not the first delta in the chain
-              // anymore.
-              continue;
-            }
-            vertex->delta = nullptr;
-            if (vertex->deleted) {
-              InsertVertexPKIntoList(deleted_vertices_, vertex->keys.Keys());
-            }
-            break;
-          }
-          case PreviousPtr::Type::EDGE: {
-            Edge *edge = prev.edge;
-            if (edge->delta != &delta) {
-              // Something changed, we're not the first delta in the chain
-              // anymore.
-              continue;
-            }
-            edge->delta = nullptr;
-            if (edge->deleted) {
-              deleted_edges_.push_back(edge->gid);
-            }
-            break;
-          }
-          case PreviousPtr::Type::DELTA: {
-            if (prev.delta->timestamp->load(std::memory_order_acquire) == commit_timestamp) {
-              // The delta that is newer than this one is also a delta from this
-              // transaction. We skip the current delta and will remove it as a
-              // part of the suffix later.
-              break;
-            }
-            {
-              // We need to find the parent object in order to be able to use
-              // its lock.
-              auto parent = prev;
-              while (parent.type == PreviousPtr::Type::DELTA) {
-                parent = parent.delta->prev.Get();
-              }
-              switch (parent.type) {
-                case PreviousPtr::Type::VERTEX:
-                case PreviousPtr::Type::EDGE:
-                  break;
-                case PreviousPtr::Type::DELTA:
-                case PreviousPtr::Type::NULLPTR:
-                  LOG_FATAL("Invalid database state!");
-              }
-            }
-            Delta *prev_delta = prev.delta;
-            prev_delta->next.store(nullptr, std::memory_order_release);
-            break;
-          }
-          case PreviousPtr::Type::NULLPTR: {
-            LOG_FATAL("Invalid pointer!");
-          }
-        }
-        break;
-      }
-    }
-
-    unlinked_undo_buffers.emplace_back(0, std::move(transaction->deltas));
-    committed_transactions_.pop_front();
-  }
-
-  // After unlinking deltas from vertices, we refresh the indices. That way
-  // we're sure that none of the vertices from `current_deleted_vertices`
-  // appears in an index, and we can safely remove the from the main storage
-  // after the last currently active transaction is finished.
-  if (run_index_cleanup) {
-    // This operation is very expensive as it traverses through all of the items
-    // in every index every time.
-    RemoveObsoleteEntries(&indices_, oldest_active_start_timestamp);
-    constraints_.unique_constraints.RemoveObsoleteEntries(oldest_active_start_timestamp);
-  }
-
-  {
-    uint64_t mark_timestamp = timestamp_;
-    for (auto &[timestamp, undo_buffer] : unlinked_undo_buffers) {
-      timestamp = mark_timestamp;
-    }
-    garbage_undo_buffers_.splice(garbage_undo_buffers_.end(), unlinked_undo_buffers);
-
-    for (const auto &vertex : deleted_vertices_) {
-      garbage_vertices_.emplace_back(mark_timestamp, vertex);
-    }
-  }
-
-  // if force is set to true we can simply delete all the leftover undos because
-  // no transaction is active
-  if constexpr (force) {
-    garbage_undo_buffers_.clear();
-  } else {
-    while (!garbage_undo_buffers_.empty() && garbage_undo_buffers_.front().first <= oldest_active_start_timestamp) {
-      garbage_undo_buffers_.pop_front();
-    }
-  }
-
-  {
-    auto vertex_acc = vertices_.access();
-    if constexpr (force) {
-      // if force is set to true, then we have unique_lock and no transactions are active
-      // so we can clean all of the deleted vertices
-      while (!garbage_vertices_.empty()) {
-        MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!");
-        garbage_vertices_.pop_front();
-      }
-    } else {
-      while (!garbage_vertices_.empty() && garbage_vertices_.front().first < oldest_active_start_timestamp) {
-        MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!");
-        garbage_vertices_.pop_front();
-      }
-    }
-  }
-  {
-    auto edge_acc = edges_.access();
-    for (auto edge : deleted_edges_) {
-      MG_ASSERT(edge_acc.remove(edge), "Invalid database state!");
-    }
-  }
-}
-
-// tell the linker he can find the CollectGarbage definitions here
-template void Storage::CollectGarbage<true>();
-template void Storage::CollectGarbage<false>();
-
-bool Storage::InitializeWalFile() {
-  if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL)
-    return false;
-  if (!wal_file_) {
-    wal_file_.emplace(wal_directory_, uuid_, epoch_id_, config_.items, &name_id_mapper_, wal_seq_num_++,
-                      &file_retainer_);
-  }
-  return true;
-}
-
-void Storage::FinalizeWalFile() {
-  ++wal_unsynced_transactions_;
-  if (wal_unsynced_transactions_ >= config_.durability.wal_file_flush_every_n_tx) {
-    wal_file_->Sync();
-    wal_unsynced_transactions_ = 0;
-  }
-  if (wal_file_->GetSize() / 1024 >= config_.durability.wal_file_size_kibibytes) {
-    wal_file_->FinalizeWal();
-    wal_file_ = std::nullopt;
-    wal_unsynced_transactions_ = 0;
-  } else {
-    // Try writing the internal buffer if possible, if not
-    // the data should be written as soon as it's possible
-    // (triggered by the new transaction commit, or some
-    // reading thread EnabledFlushing)
-    wal_file_->TryFlushing();
-  }
-}
-
-void Storage::AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp) {
-  if (!InitializeWalFile()) return;
-  // Traverse deltas and append them to the WAL file.
-  // A single transaction will always be contained in a single WAL file.
-  auto current_commit_timestamp = transaction.commit_timestamp->load(std::memory_order_acquire);
-
-  if (replication_role_ == ReplicationRole::MAIN) {
-    replication_clients_.WithLock([&](auto &clients) {
-      for (auto &client : clients) {
-        client->StartTransactionReplication(wal_file_->SequenceNumber());
-      }
-    });
-  }
-
-  // Helper lambda that traverses the delta chain on order to find the first
-  // delta that should be processed and then appends all discovered deltas.
-  auto find_and_apply_deltas = [&](const auto *delta, const auto &parent, auto filter) {
-    while (true) {
-      auto *older = delta->next.load(std::memory_order_acquire);
-      if (older == nullptr || older->timestamp->load(std::memory_order_acquire) != current_commit_timestamp) break;
-      delta = older;
-    }
-    while (true) {
-      if (filter(delta->action)) {
-        wal_file_->AppendDelta(*delta, parent, final_commit_timestamp);
-        replication_clients_.WithLock([&](auto &clients) {
-          for (auto &client : clients) {
-            client->IfStreamingTransaction(
-                [&](auto &stream) { stream.AppendDelta(*delta, parent, final_commit_timestamp); });
-          }
-        });
-      }
-      auto prev = delta->prev.Get();
-      MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-      if (prev.type != PreviousPtr::Type::DELTA) break;
-      delta = prev.delta;
-    }
-  };
-
-  // The deltas are ordered correctly in the `transaction.deltas` buffer, but we
-  // don't traverse them in that order. That is because for each delta we need
-  // information about the vertex or edge they belong to and that information
-  // isn't stored in the deltas themselves. In order to find out information
-  // about the corresponding vertex or edge it is necessary to traverse the
-  // delta chain for each delta until a vertex or edge is encountered. This
-  // operation is very expensive as the chain grows.
-  // Instead, we traverse the edges until we find a vertex or edge and traverse
-  // their delta chains. This approach has a drawback because we lose the
-  // correct order of the operations. Because of that, we need to traverse the
-  // deltas several times and we have to manually ensure that the stored deltas
-  // will be ordered correctly.
-
-  // 1. Process all Vertex deltas and store all operations that create vertices
-  // and modify vertex data.
-  for (const auto &delta : transaction.deltas) {
-    auto prev = delta.prev.Get();
-    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-    if (prev.type != PreviousPtr::Type::VERTEX) continue;
-    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
-      switch (action) {
-        case Delta::Action::DELETE_OBJECT:
-        case Delta::Action::SET_PROPERTY:
-        case Delta::Action::ADD_LABEL:
-        case Delta::Action::REMOVE_LABEL:
-          return true;
-
-        case Delta::Action::RECREATE_OBJECT:
-        case Delta::Action::ADD_IN_EDGE:
-        case Delta::Action::ADD_OUT_EDGE:
-        case Delta::Action::REMOVE_IN_EDGE:
-        case Delta::Action::REMOVE_OUT_EDGE:
-          return false;
-      }
-    });
-  }
-  // 2. Process all Vertex deltas and store all operations that create edges.
-  for (const auto &delta : transaction.deltas) {
-    auto prev = delta.prev.Get();
-    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-    if (prev.type != PreviousPtr::Type::VERTEX) continue;
-    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
-      switch (action) {
-        case Delta::Action::REMOVE_OUT_EDGE:
-          return true;
-
-        case Delta::Action::DELETE_OBJECT:
-        case Delta::Action::RECREATE_OBJECT:
-        case Delta::Action::SET_PROPERTY:
-        case Delta::Action::ADD_LABEL:
-        case Delta::Action::REMOVE_LABEL:
-        case Delta::Action::ADD_IN_EDGE:
-        case Delta::Action::ADD_OUT_EDGE:
-        case Delta::Action::REMOVE_IN_EDGE:
-          return false;
-      }
-    });
-  }
-  // 3. Process all Edge deltas and store all operations that modify edge data.
-  for (const auto &delta : transaction.deltas) {
-    auto prev = delta.prev.Get();
-    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-    if (prev.type != PreviousPtr::Type::EDGE) continue;
-    find_and_apply_deltas(&delta, *prev.edge, [](auto action) {
-      switch (action) {
-        case Delta::Action::SET_PROPERTY:
-          return true;
-
-        case Delta::Action::DELETE_OBJECT:
-        case Delta::Action::RECREATE_OBJECT:
-        case Delta::Action::ADD_LABEL:
-        case Delta::Action::REMOVE_LABEL:
-        case Delta::Action::ADD_IN_EDGE:
-        case Delta::Action::ADD_OUT_EDGE:
-        case Delta::Action::REMOVE_IN_EDGE:
-        case Delta::Action::REMOVE_OUT_EDGE:
-          return false;
-      }
-    });
-  }
-  // 4. Process all Vertex deltas and store all operations that delete edges.
-  for (const auto &delta : transaction.deltas) {
-    auto prev = delta.prev.Get();
-    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-    if (prev.type != PreviousPtr::Type::VERTEX) continue;
-    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
-      switch (action) {
-        case Delta::Action::ADD_OUT_EDGE:
-          return true;
-
-        case Delta::Action::DELETE_OBJECT:
-        case Delta::Action::RECREATE_OBJECT:
-        case Delta::Action::SET_PROPERTY:
-        case Delta::Action::ADD_LABEL:
-        case Delta::Action::REMOVE_LABEL:
-        case Delta::Action::ADD_IN_EDGE:
-        case Delta::Action::REMOVE_IN_EDGE:
-        case Delta::Action::REMOVE_OUT_EDGE:
-          return false;
-      }
-    });
-  }
-  // 5. Process all Vertex deltas and store all operations that delete vertices.
-  for (const auto &delta : transaction.deltas) {
-    auto prev = delta.prev.Get();
-    MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!");
-    if (prev.type != PreviousPtr::Type::VERTEX) continue;
-    find_and_apply_deltas(&delta, *prev.vertex, [](auto action) {
-      switch (action) {
-        case Delta::Action::RECREATE_OBJECT:
-          return true;
-
-        case Delta::Action::DELETE_OBJECT:
-        case Delta::Action::SET_PROPERTY:
-        case Delta::Action::ADD_LABEL:
-        case Delta::Action::REMOVE_LABEL:
-        case Delta::Action::ADD_IN_EDGE:
-        case Delta::Action::ADD_OUT_EDGE:
-        case Delta::Action::REMOVE_IN_EDGE:
-        case Delta::Action::REMOVE_OUT_EDGE:
-          return false;
-      }
-    });
-  }
-
-  // Add a delta that indicates that the transaction is fully written to the WAL
-  // file.
-  wal_file_->AppendTransactionEnd(final_commit_timestamp);
-
-  FinalizeWalFile();
-
-  replication_clients_.WithLock([&](auto &clients) {
-    for (auto &client : clients) {
-      client->IfStreamingTransaction([&](auto &stream) { stream.AppendTransactionEnd(final_commit_timestamp); });
-      client->FinalizeTransactionReplication();
-    }
-  });
-}
-
-void Storage::AppendToWal(durability::StorageGlobalOperation operation, LabelId label,
-                          const std::set<PropertyId> &properties, uint64_t final_commit_timestamp) {
-  if (!InitializeWalFile()) return;
-  wal_file_->AppendOperation(operation, label, properties, final_commit_timestamp);
-  {
-    if (replication_role_ == ReplicationRole::MAIN) {
-      replication_clients_.WithLock([&](auto &clients) {
-        for (auto &client : clients) {
-          client->StartTransactionReplication(wal_file_->SequenceNumber());
-          client->IfStreamingTransaction(
-              [&](auto &stream) { stream.AppendOperation(operation, label, properties, final_commit_timestamp); });
-          client->FinalizeTransactionReplication();
-        }
-      });
-    }
-  }
-  FinalizeWalFile();
-}
-
-utils::BasicResult<Storage::CreateSnapshotError> Storage::CreateSnapshot() {
-  if (replication_role_ != ReplicationRole::MAIN) {
-    return CreateSnapshotError::DisabledForReplica;
-  }
-
-  // Create the transaction used to create the snapshot.
-  auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION);
-
-  // Create snapshot.
-  // durability::CreateSnapshot(&transaction, snapshot_directory_, wal_directory_,
-  //                            config_.durability.snapshot_retention_count, &vertices_, &edges_,
-  //                            &name_id_mapper_, &indices_, &constraints_, config_.items, schema_validator_,
-  //                            uuid_, epoch_id_, epoch_history_, &file_retainer_);
-
-  // Finalize snapshot transaction.
-  commit_log_->MarkFinished(transaction.start_timestamp);
-  return {};
-}
-
-bool Storage::LockPath() {
-  auto locker_accessor = global_locker_.Access();
-  return locker_accessor.AddPath(config_.durability.storage_directory);
-}
-
-bool Storage::UnlockPath() {
-  {
-    auto locker_accessor = global_locker_.Access();
-    if (!locker_accessor.RemovePath(config_.durability.storage_directory)) {
-      return false;
-    }
-  }
-
-  // We use locker accessor in seperate scope so we don't produce deadlock
-  // after we call clean queue.
-  file_retainer_.CleanQueue();
-  return true;
-}
-
-void Storage::FreeMemory() {
-  CollectGarbage<true>();
-
-  // SkipList is already threadsafe
-  vertices_.run_gc();
-  edges_.run_gc();
-  indices_.label_index.RunGC();
-  indices_.label_property_index.RunGC();
-}
-
-uint64_t Storage::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) {
-  if (!desired_commit_timestamp) {
-    return timestamp_++;
-  }
-  timestamp_ = std::max(timestamp_, *desired_commit_timestamp + 1);
-  return *desired_commit_timestamp;
-}
-
-bool Storage::SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) {
-  // We don't want to restart the server if we're already a REPLICA
-  if (replication_role_ == ReplicationRole::REPLICA) {
-    return false;
-  }
-
-  replication_server_ = std::make_unique<ReplicationServer>(this, std::move(endpoint), config);
-
-  replication_role_ = ReplicationRole::REPLICA;
-  return true;
-}
-
-bool Storage::SetMainReplicationRole() {
-  // We don't want to generate new epoch_id and do the
-  // cleanup if we're already a MAIN
-  if (replication_role_ == ReplicationRole::MAIN) {
-    return false;
-  }
-
-  // Main instance does not need replication server
-  // This should be always called first so we finalize everything
-  replication_server_.reset(nullptr);
-
-  if (wal_file_) {
-    wal_file_->FinalizeWal();
-    wal_file_.reset();
-  }
-
-  // Generate new epoch id and save the last one to the history.
-  if (epoch_history_.size() == kEpochHistoryRetention) {
-    epoch_history_.pop_front();
-  }
-  epoch_history_.emplace_back(std::move(epoch_id_), last_commit_timestamp_);
-  epoch_id_ = utils::GenerateUUID();
-
-  replication_role_ = ReplicationRole::MAIN;
-  return true;
-}
-
-utils::BasicResult<Storage::RegisterReplicaError> Storage::RegisterReplica(
-    std::string name, io::network::Endpoint endpoint, const replication::ReplicationMode replication_mode,
-    const replication::ReplicationClientConfig &config) {
-  MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can register a replica!");
-
-  const bool name_exists = replication_clients_.WithLock([&](auto &clients) {
-    return std::any_of(clients.begin(), clients.end(), [&name](const auto &client) { return client->Name() == name; });
-  });
-
-  if (name_exists) {
-    return RegisterReplicaError::NAME_EXISTS;
-  }
-
-  const auto end_point_exists = replication_clients_.WithLock([&endpoint](auto &clients) {
-    return std::any_of(clients.begin(), clients.end(),
-                       [&endpoint](const auto &client) { return client->Endpoint() == endpoint; });
-  });
-
-  if (end_point_exists) {
-    return RegisterReplicaError::END_POINT_EXISTS;
-  }
-
-  MG_ASSERT(replication_mode == replication::ReplicationMode::SYNC || !config.timeout,
-            "Only SYNC mode can have a timeout set");
-
-  auto client = std::make_unique<ReplicationClient>(std::move(name), this, endpoint, replication_mode, config);
-  if (client->State() == replication::ReplicaState::INVALID) {
-    return RegisterReplicaError::CONNECTION_FAILED;
-  }
-
-  return replication_clients_.WithLock([&](auto &clients) -> utils::BasicResult<Storage::RegisterReplicaError> {
-    // Another thread could have added a client with same name while
-    // we were connecting to this client.
-    if (std::any_of(clients.begin(), clients.end(),
-                    [&](const auto &other_client) { return client->Name() == other_client->Name(); })) {
-      return RegisterReplicaError::NAME_EXISTS;
-    }
-
-    if (std::any_of(clients.begin(), clients.end(),
-                    [&client](const auto &other_client) { return client->Endpoint() == other_client->Endpoint(); })) {
-      return RegisterReplicaError::END_POINT_EXISTS;
-    }
-
-    clients.push_back(std::move(client));
-    return {};
-  });
-}
-
-bool Storage::UnregisterReplica(const std::string_view name) {
-  MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can unregister a replica!");
-  return replication_clients_.WithLock([&](auto &clients) {
-    return std::erase_if(clients, [&](const auto &client) { return client->Name() == name; });
-  });
-}
-
-std::optional<replication::ReplicaState> Storage::GetReplicaState(const std::string_view name) {
-  return replication_clients_.WithLock([&](auto &clients) -> std::optional<replication::ReplicaState> {
-    const auto client_it =
-        std::find_if(clients.cbegin(), clients.cend(), [name](auto &client) { return client->Name() == name; });
-    if (client_it == clients.cend()) {
-      return std::nullopt;
-    }
-    return (*client_it)->State();
-  });
-}
-
-ReplicationRole Storage::GetReplicationRole() const { return replication_role_; }
-
-std::vector<Storage::ReplicaInfo> Storage::ReplicasInfo() {
-  return replication_clients_.WithLock([](auto &clients) {
-    std::vector<Storage::ReplicaInfo> replica_info;
-    replica_info.reserve(clients.size());
-    std::transform(clients.begin(), clients.end(), std::back_inserter(replica_info),
-                   [](const auto &client) -> ReplicaInfo {
-                     return {client->Name(), client->Mode(), client->Timeout(), client->Endpoint(), client->State()};
-                   });
-    return replica_info;
-  });
-}
-
-void Storage::SetIsolationLevel(IsolationLevel isolation_level) { isolation_level_ = isolation_level; }
+Storage::Storage(Config config) : config_{config} {}
 
 }  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/storage.hpp b/src/storage/v3/storage.hpp
index ed884c86e..9305ba707 100644
--- a/src/storage/v3/storage.hpp
+++ b/src/storage/v3/storage.hpp
@@ -11,609 +11,24 @@
 
 #pragma once
 
-#include <atomic>
-#include <cstdint>
-#include <filesystem>
-#include <map>
-#include <numeric>
-#include <optional>
-#include <shared_mutex>
-#include <variant>
 #include <vector>
 
-#include "io/network/endpoint.hpp"
-#include "kvstore/kvstore.hpp"
-#include "storage/v3/commit_log.hpp"
-#include "storage/v3/config.hpp"
-#include "storage/v3/constraints.hpp"
-#include "storage/v3/durability/metadata.hpp"
-#include "storage/v3/durability/wal.hpp"
-#include "storage/v3/edge.hpp"
-#include "storage/v3/edge_accessor.hpp"
-#include "storage/v3/id_types.hpp"
-#include "storage/v3/indices.hpp"
-#include "storage/v3/isolation_level.hpp"
-#include "storage/v3/key_store.hpp"
-#include "storage/v3/lexicographically_ordered_vertex.hpp"
-#include "storage/v3/mvcc.hpp"
-#include "storage/v3/name_id_mapper.hpp"
-#include "storage/v3/property_value.hpp"
-#include "storage/v3/result.hpp"
-#include "storage/v3/schema_validator.hpp"
-#include "storage/v3/schemas.hpp"
-#include "storage/v3/transaction.hpp"
-#include "storage/v3/vertex.hpp"
-#include "storage/v3/vertex_accessor.hpp"
-#include "storage/v3/vertices_skip_list.hpp"
-#include "utils/exceptions.hpp"
-#include "utils/file_locker.hpp"
-#include "utils/on_scope_exit.hpp"
-#include "utils/rw_lock.hpp"
-#include "utils/scheduler.hpp"
-#include "utils/skip_list.hpp"
-#include "utils/synchronized.hpp"
-#include "utils/uuid.hpp"
+#include <boost/asio/thread_pool.hpp>
 
-/// REPLICATION ///
-#include "rpc/server.hpp"
-#include "storage/v3/replication/config.hpp"
-#include "storage/v3/replication/enums.hpp"
-#include "storage/v3/replication/rpc.hpp"
-#include "storage/v3/replication/serialization.hpp"
+#include "storage/v3/shard.hpp"
 
 namespace memgraph::storage::v3 {
 
-// The storage is based on this paper:
-// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
-// The paper implements a fully serializable storage, in our implementation we
-// only implement snapshot isolation for transactions.
-
-/// Iterable for iterating through all vertices of a Storage.
-///
-/// An instance of this will be usually be wrapped inside VerticesIterable for
-/// generic, public use.
-class AllVerticesIterable final {
-  VerticesSkipList::Accessor vertices_accessor_;
-  Transaction *transaction_;
-  View view_;
-  Indices *indices_;
-  Constraints *constraints_;
-  Config::Items config_;
-  const SchemaValidator *schema_validator_;
-  const Schemas *schemas_;
-  std::optional<VertexAccessor> vertex_;
-
+class Storage {
  public:
-  class Iterator final {
-    AllVerticesIterable *self_;
-    VerticesSkipList::Iterator it_;
-
-   public:
-    Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it);
-
-    VertexAccessor operator*() const;
-
-    Iterator &operator++();
-
-    bool operator==(const Iterator &other) const { return self_ == other.self_ && it_ == other.it_; }
-
-    bool operator!=(const Iterator &other) const { return !(*this == other); }
-  };
-
-  AllVerticesIterable(VerticesSkipList::Accessor vertices_accessor, Transaction *transaction, View view,
-                      Indices *indices, Constraints *constraints, Config::Items config,
-                      const SchemaValidator &schema_validator)
-      : vertices_accessor_(std::move(vertices_accessor)),
-        transaction_(transaction),
-        view_(view),
-        indices_(indices),
-        constraints_(constraints),
-        config_(config),
-        schema_validator_{&schema_validator} {}
-
-  Iterator begin() { return {this, vertices_accessor_.begin()}; }
-  Iterator end() { return {this, vertices_accessor_.end()}; }
-};
-
-/// Generic access to different kinds of vertex iterations.
-///
-/// This class should be the primary type used by the client code to iterate
-/// over vertices inside a Storage instance.
-class VerticesIterable final {
-  enum class Type { ALL, BY_LABEL, BY_LABEL_PROPERTY };
-
-  Type type_;
-  union {
-    AllVerticesIterable all_vertices_;
-    LabelIndex::Iterable vertices_by_label_;
-    LabelPropertyIndex::Iterable vertices_by_label_property_;
-  };
-
- public:
-  explicit VerticesIterable(AllVerticesIterable);
-  explicit VerticesIterable(LabelIndex::Iterable);
-  explicit VerticesIterable(LabelPropertyIndex::Iterable);
-
-  VerticesIterable(const VerticesIterable &) = delete;
-  VerticesIterable &operator=(const VerticesIterable &) = delete;
-
-  VerticesIterable(VerticesIterable &&) noexcept;
-  VerticesIterable &operator=(VerticesIterable &&) noexcept;
-
-  ~VerticesIterable();
-
-  class Iterator final {
-    Type type_;
-    union {
-      AllVerticesIterable::Iterator all_it_;
-      LabelIndex::Iterable::Iterator by_label_it_;
-      LabelPropertyIndex::Iterable::Iterator by_label_property_it_;
-    };
-
-    void Destroy() noexcept;
-
-   public:
-    explicit Iterator(AllVerticesIterable::Iterator);
-    explicit Iterator(LabelIndex::Iterable::Iterator);
-    explicit Iterator(LabelPropertyIndex::Iterable::Iterator);
-
-    Iterator(const Iterator &);
-    Iterator &operator=(const Iterator &);
-
-    Iterator(Iterator &&) noexcept;
-    Iterator &operator=(Iterator &&) noexcept;
-
-    ~Iterator();
-
-    VertexAccessor operator*() const;
-
-    Iterator &operator++();
-
-    bool operator==(const Iterator &other) const;
-    bool operator!=(const Iterator &other) const { return !(*this == other); }
-  };
-
-  Iterator begin();
-  Iterator end();
-};
-
-/// Structure used to return information about existing indices in the storage.
-struct IndicesInfo {
-  std::vector<LabelId> label;
-  std::vector<std::pair<LabelId, PropertyId>> label_property;
-};
-
-/// Structure used to return information about existing constraints in the
-/// storage.
-struct ConstraintsInfo {
-  std::vector<std::pair<LabelId, PropertyId>> existence;
-  std::vector<std::pair<LabelId, std::set<PropertyId>>> unique;
-};
-
-/// Structure used to return information about existing schemas in the storage
-struct SchemasInfo {
-  Schemas::SchemasList schemas;
-};
-
-/// Structure used to return information about the storage.
-struct StorageInfo {
-  uint64_t vertex_count;
-  uint64_t edge_count;
-  double average_degree;
-  uint64_t memory_usage;
-  uint64_t disk_usage;
-};
-
-enum class ReplicationRole : uint8_t { MAIN, REPLICA };
-
-class Storage final {
- public:
-  /// @throw std::system_error
-  /// @throw std::bad_alloc
-  explicit Storage(Config config = Config());
-
-  ~Storage();
-
-  class Accessor final {
-   private:
-    friend class Storage;
-
-    explicit Accessor(Storage *storage, IsolationLevel isolation_level);
-
-   public:
-    Accessor(const Accessor &) = delete;
-    Accessor &operator=(const Accessor &) = delete;
-    Accessor &operator=(Accessor &&other) = delete;
-
-    // NOTE: After the accessor is moved, all objects derived from it (accessors
-    // and iterators) are *invalid*. You have to get all derived objects again.
-    Accessor(Accessor &&other) noexcept;
-
-    ~Accessor();
-
-    /// @throw std::bad_alloc
-    ResultSchema<VertexAccessor> CreateVertexAndValidate(
-        LabelId primary_label, const std::vector<LabelId> &labels,
-        const std::vector<std::pair<PropertyId, PropertyValue>> &properties);
-
-    std::optional<VertexAccessor> FindVertex(std::vector<PropertyValue> primary_key, View view);
-
-    VerticesIterable Vertices(View view) {
-      return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view,
-                                                  &storage_->indices_, &storage_->constraints_, storage_->config_.items,
-                                                  storage_->schema_validator_));
-    }
-
-    VerticesIterable Vertices(LabelId label, View view);
-
-    VerticesIterable Vertices(LabelId label, PropertyId property, View view);
-
-    VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view);
-
-    VerticesIterable Vertices(LabelId label, PropertyId property,
-                              const std::optional<utils::Bound<PropertyValue>> &lower_bound,
-                              const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view);
-
-    /// Return approximate number of all vertices in the database.
-    /// Note that this is always an over-estimate and never an under-estimate.
-    int64_t ApproximateVertexCount() const { return static_cast<int64_t>(storage_->vertices_.size()); }
-
-    /// Return approximate number of vertices with the given label.
-    /// Note that this is always an over-estimate and never an under-estimate.
-    int64_t ApproximateVertexCount(LabelId label) const {
-      return storage_->indices_.label_index.ApproximateVertexCount(label);
-    }
-
-    /// Return approximate number of vertices with the given label and property.
-    /// Note that this is always an over-estimate and never an under-estimate.
-    int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
-      return storage_->indices_.label_property_index.ApproximateVertexCount(label, property);
-    }
-
-    /// Return approximate number of vertices with the given label and the given
-    /// value for the given property. Note that this is always an over-estimate
-    /// and never an under-estimate.
-    int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const {
-      return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
-    }
-
-    /// Return approximate number of vertices with the given label and value for
-    /// the given property in the range defined by provided upper and lower
-    /// bounds.
-    int64_t ApproximateVertexCount(LabelId label, PropertyId property,
-                                   const std::optional<utils::Bound<PropertyValue>> &lower,
-                                   const std::optional<utils::Bound<PropertyValue>> &upper) const {
-      return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
-    }
-
-    /// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
-    /// @throw std::bad_alloc
-    Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex);
-
-    /// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
-    /// @throw std::bad_alloc
-    Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
-        VertexAccessor *vertex);
-
-    /// @throw std::bad_alloc
-    Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type);
-
-    /// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
-    /// @throw std::bad_alloc
-    Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge);
-
-    const std::string &LabelToName(LabelId label) const;
-    const std::string &PropertyToName(PropertyId property) const;
-    const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
-
-    /// @throw std::bad_alloc if unable to insert a new mapping
-    LabelId NameToLabel(std::string_view name);
-
-    /// @throw std::bad_alloc if unable to insert a new mapping
-    PropertyId NameToProperty(std::string_view name);
-
-    /// @throw std::bad_alloc if unable to insert a new mapping
-    EdgeTypeId NameToEdgeType(std::string_view name);
-
-    bool LabelIndexExists(LabelId label) const { return storage_->indices_.label_index.IndexExists(label); }
-
-    bool LabelPropertyIndexExists(LabelId label, PropertyId property) const {
-      return storage_->indices_.label_property_index.IndexExists(label, property);
-    }
-
-    IndicesInfo ListAllIndices() const {
-      return {storage_->indices_.label_index.ListIndices(), storage_->indices_.label_property_index.ListIndices()};
-    }
-
-    ConstraintsInfo ListAllConstraints() const {
-      return {ListExistenceConstraints(storage_->constraints_),
-              storage_->constraints_.unique_constraints.ListConstraints()};
-    }
-
-    const SchemaValidator &GetSchemaValidator() const;
-
-    SchemasInfo ListAllSchemas() const { return {storage_->schemas_.ListSchemas()}; }
-
-    void AdvanceCommand();
-
-    /// Commit returns `ConstraintViolation` if the changes made by this
-    /// transaction violate an existence or unique constraint. In that case the
-    /// transaction is automatically aborted. Otherwise, void is returned.
-    /// @throw std::bad_alloc
-    utils::BasicResult<ConstraintViolation, void> Commit(std::optional<uint64_t> desired_commit_timestamp = {});
-
-    /// @throw std::bad_alloc
-    void Abort();
-
-    void FinalizeTransaction();
-
-   private:
-    /// @throw std::bad_alloc
-    VertexAccessor CreateVertex(Gid gid, LabelId primary_label);
-
-    /// @throw std::bad_alloc
-    Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, Gid gid);
-
-    Storage *storage_;
-    Transaction transaction_;
-    std::optional<uint64_t> commit_timestamp_;
-    bool is_transaction_active_;
-    Config::Items config_;
-  };
-
-  Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) {
-    return Accessor{this, override_isolation_level.value_or(isolation_level_)};
-  }
-
-  const std::string &LabelToName(LabelId label) const;
-  const std::string &PropertyToName(PropertyId property) const;
-  const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
-
-  /// @throw std::bad_alloc if unable to insert a new mapping
-  LabelId NameToLabel(std::string_view name);
-
-  /// @throw std::bad_alloc if unable to insert a new mapping
-  PropertyId NameToProperty(std::string_view name);
-
-  /// @throw std::bad_alloc if unable to insert a new mapping
-  EdgeTypeId NameToEdgeType(std::string_view name);
-
-  /// @throw std::bad_alloc
-  bool CreateIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
-
-  /// @throw std::bad_alloc
-  bool CreateIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
-
-  bool DropIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
-
-  bool DropIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
-
-  IndicesInfo ListAllIndices() const;
-
-  /// Creates an existence constraint. Returns true if the constraint was
-  /// successfully added, false if it already exists and a `ConstraintViolation`
-  /// if there is an existing vertex violating the constraint.
-  ///
-  /// @throw std::bad_alloc
-  /// @throw std::length_error
-  utils::BasicResult<ConstraintViolation, bool> CreateExistenceConstraint(
-      LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
-
-  /// Removes an existence constraint. Returns true if the constraint was
-  /// removed, and false if it doesn't exist.
-  bool DropExistenceConstraint(LabelId label, PropertyId property,
-                               std::optional<uint64_t> desired_commit_timestamp = {});
-
-  /// Creates a unique constraint. In the case of two vertices violating the
-  /// constraint, it returns `ConstraintViolation`. Otherwise returns a
-  /// `UniqueConstraints::CreationStatus` enum with the following possibilities:
-  ///     * `SUCCESS` if the constraint was successfully created,
-  ///     * `ALREADY_EXISTS` if the constraint already existed,
-  ///     * `EMPTY_PROPERTIES` if the property set is empty, or
-  //      * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the
-  //        limit of maximum number of properties.
-  ///
-  /// @throw std::bad_alloc
-  utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
-      LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp = {});
-
-  /// Removes a unique constraint. Returns `UniqueConstraints::DeletionStatus`
-  /// enum with the following possibilities:
-  ///     * `SUCCESS` if constraint was successfully removed,
-  ///     * `NOT_FOUND` if the specified constraint was not found,
-  ///     * `EMPTY_PROPERTIES` if the property set is empty, or
-  ///     * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the
-  //        limit of maximum number of properties.
-  UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
-                                                         std::optional<uint64_t> desired_commit_timestamp = {});
-
-  ConstraintsInfo ListAllConstraints() const;
-
-  SchemasInfo ListAllSchemas() const;
-
-  const Schemas::Schema *GetSchema(LabelId primary_label) const;
-
-  bool CreateSchema(LabelId primary_label, const std::vector<SchemaProperty> &schemas_types);
-
-  bool DropSchema(LabelId primary_label);
-
-  StorageInfo GetInfo() const;
-
-  bool LockPath();
-  bool UnlockPath();
-
-  bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {});
-
-  bool SetMainReplicationRole();
-
-  enum class RegisterReplicaError : uint8_t {
-    NAME_EXISTS,
-    END_POINT_EXISTS,
-    CONNECTION_FAILED,
-    COULD_NOT_BE_PERSISTED
-  };
-
-  /// @pre The instance should have a MAIN role
-  /// @pre Timeout can only be set for SYNC replication
-  utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
-      std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
-      const replication::ReplicationClientConfig &config = {});
-  /// @pre The instance should have a MAIN role
-  bool UnregisterReplica(std::string_view name);
-
-  std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
-
-  ReplicationRole GetReplicationRole() const;
-
-  struct ReplicaInfo {
-    std::string name;
-    replication::ReplicationMode mode;
-    std::optional<double> timeout;
-    io::network::Endpoint endpoint;
-    replication::ReplicaState state;
-  };
-
-  std::vector<ReplicaInfo> ReplicasInfo();
-
-  void FreeMemory();
-
-  void SetIsolationLevel(IsolationLevel isolation_level);
-
-  enum class CreateSnapshotError : uint8_t { DisabledForReplica };
-
-  utils::BasicResult<CreateSnapshotError> CreateSnapshot();
+  explicit Storage(Config config);
+  // Interface toward shard manipulation
+  // Shard handler -> will use rsm client
 
  private:
-  Transaction CreateTransaction(IsolationLevel isolation_level);
-
-  /// The force parameter determines the behaviour of the garbage collector.
-  /// If it's set to true, it will behave as a global operation, i.e. it can't
-  /// be part of a transaction, and no other transaction can be active at the same time.
-  /// This allows it to delete immediately vertices without worrying that some other
-  /// transaction is possibly using it. If there are active transactions when this method
-  /// is called with force set to true, it will fallback to the same method with the force
-  /// set to false.
-  /// If it's set to false, it will execute in parallel with other transactions, ensuring
-  /// that no object in use can be deleted.
-  /// @throw std::system_error
-  /// @throw std::bad_alloc
-  template <bool force>
-  void CollectGarbage();
-
-  bool InitializeWalFile();
-  void FinalizeWalFile();
-
-  void AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp);
-  void AppendToWal(durability::StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties,
-                   uint64_t final_commit_timestamp);
-
-  uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
-
-  // Main object storage
-  VerticesSkipList vertices_;
-  utils::SkipList<Edge> edges_;
-  uint64_t edge_id_{0};
-  // Even though the edge count is already kept in the `edges_` SkipList, the
-  // list is used only when properties are enabled for edges. Because of that we
-  // keep a separate count of edges that is always updated.
-  uint64_t edge_count_{0};
-
-  NameIdMapper name_id_mapper_;
-
-  SchemaValidator schema_validator_;
-  Constraints constraints_;
-  Indices indices_;
-  Schemas schemas_;
-
-  // Transaction engine
-  uint64_t timestamp_{kTimestampInitialId};
-  uint64_t transaction_id_{kTransactionInitialId};
-  // TODO: This isn't really a commit log, it doesn't even care if a
-  // transaction commited or aborted. We could probably combine this with
-  // `timestamp_` in a sensible unit, something like TransactionClock or
-  // whatever.
-  std::optional<CommitLog> commit_log_;
-
-  std::list<Transaction> committed_transactions_;
-  IsolationLevel isolation_level_;
-
+  std::vector<Shard> shards_;
+  boost::asio::thread_pool shard_handlers_;
   Config config_;
-
-  // Undo buffers that were unlinked and now are waiting to be freed.
-  std::list<std::pair<uint64_t, std::list<Delta>>> garbage_undo_buffers_;
-
-  // Vertices that are logically deleted but still have to be removed from
-  // indices before removing them from the main storage.
-  std::list<PrimaryKey> deleted_vertices_;
-
-  // Vertices that are logically deleted and removed from indices and now wait
-  // to be removed from the main storage.
-  std::list<std::pair<uint64_t, PrimaryKey>> garbage_vertices_;
-
-  // Edges that are logically deleted and wait to be removed from the main
-  // storage.
-  std::list<Gid> deleted_edges_;
-
-  // Durability
-  std::filesystem::path snapshot_directory_;
-  std::filesystem::path wal_directory_;
-  std::filesystem::path lock_file_path_;
-  utils::OutputFile lock_file_handle_;
-
-  // UUID used to distinguish snapshots and to link snapshots to WALs
-  std::string uuid_;
-  // Sequence number used to keep track of the chain of WALs.
-  uint64_t wal_seq_num_{0};
-
-  // UUID to distinguish different main instance runs for replication process
-  // on SAME storage.
-  // Multiple instances can have same storage UUID and be MAIN at the same time.
-  // We cannot compare commit timestamps of those instances if one of them
-  // becomes the replica of the other so we use epoch_id_ as additional
-  // discriminating property.
-  // Example of this:
-  // We have 2 instances of the same storage, S1 and S2.
-  // S1 and S2 are MAIN and accept their own commits and write them to the WAL.
-  // At the moment when S1 commited a transaction with timestamp 20, and S2
-  // a different transaction with timestamp 15, we change S2's role to REPLICA
-  // and register it on S1.
-  // Without using the epoch_id, we don't know that S1 and S2 have completely
-  // different transactions, we think that the S2 is behind only by 5 commits.
-  std::string epoch_id_;
-  // History of the previous epoch ids.
-  // Each value consists of the epoch id along the last commit belonging to that
-  // epoch.
-  std::deque<std::pair<std::string, uint64_t>> epoch_history_;
-
-  std::optional<durability::WalFile> wal_file_;
-  uint64_t wal_unsynced_transactions_{0};
-
-  utils::FileRetainer file_retainer_;
-
-  // Global locker that is used for clients file locking
-  utils::FileRetainer::FileLocker global_locker_;
-
-  // Last commited timestamp
-  uint64_t last_commit_timestamp_{kTimestampInitialId};
-
-  class ReplicationServer;
-  std::unique_ptr<ReplicationServer> replication_server_{nullptr};
-
-  class ReplicationClient;
-  // We create ReplicationClient using unique_ptr so we can move
-  // newly created client into the vector.
-  // We cannot move the client directly because it contains ThreadPool
-  // which cannot be moved. Also, the move is necessary because
-  // we don't want to create the client directly inside the vector
-  // because that would require the lock on the list putting all
-  // commits (they iterate list of clients) to halt.
-  // This way we can initialize client in main thread which means
-  // that we can immediately notify the user if the initialization
-  // failed.
-  using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
-  ReplicationClientList replication_clients_;
-
-  ReplicationRole replication_role_{ReplicationRole::MAIN};
 };
 
 }  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/vertex_accessor.hpp b/src/storage/v3/vertex_accessor.hpp
index 061cf399d..71e297478 100644
--- a/src/storage/v3/vertex_accessor.hpp
+++ b/src/storage/v3/vertex_accessor.hpp
@@ -25,7 +25,7 @@
 namespace memgraph::storage::v3 {
 
 class EdgeAccessor;
-class Storage;
+class Shard;
 struct Indices;
 struct Constraints;
 
@@ -46,7 +46,7 @@ class VertexAccessor final {
    private:
     const Vertex *vertex_;
   };
-  friend class Storage;
+  friend class Shard;
 
  public:
   // Be careful when using VertexAccessor since it can be instantiated with
diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt
index afd1f8dd9..cceade081 100644
--- a/tests/unit/CMakeLists.txt
+++ b/tests/unit/CMakeLists.txt
@@ -337,15 +337,15 @@ target_link_libraries(${test_prefix}storage_v3_key_store mg-storage-v3 rapidchec
 add_unit_test(storage_v3_vertex_accessors.cpp)
 target_link_libraries(${test_prefix}storage_v3_vertex_accessors mg-storage-v3)
 
-# Test mg-query-v3
+# Test mg-query-v2
 add_unit_test(query_v2_interpreter.cpp ${CMAKE_SOURCE_DIR}/src/glue/v2/communication.cpp)
 target_link_libraries(${test_prefix}query_v2_interpreter mg-storage-v3 mg-query-v2 mg-communication)
 
 # add_unit_test(query_v2_query_plan_accumulate_aggregate.cpp)
 # target_link_libraries(${test_prefix}query_v2_query_plan_accumulate_aggregate mg-query-v2)
 
-# add_unit_test(query_v2_query_plan_create_set_remove_delete.cpp)
-# target_link_libraries(${test_prefix}query_v2_query_plan_create_set_remove_delete mg-query-v2)
+# # add_unit_test(query_v2_query_plan_create_set_remove_delete.cpp)
+# # target_link_libraries(${test_prefix}query_v2_query_plan_create_set_remove_delete mg-query-v2)
 
 # add_unit_test(query_v2_query_plan_bag_semantics.cpp)
 # target_link_libraries(${test_prefix}query_v2_query_plan_bag_semantics mg-query-v2)
diff --git a/tests/unit/query_v2_interpreter.cpp b/tests/unit/query_v2_interpreter.cpp
index fdc15b032..c24f2b995 100644
--- a/tests/unit/query_v2_interpreter.cpp
+++ b/tests/unit/query_v2_interpreter.cpp
@@ -29,27 +29,29 @@
 #include "query_v2_query_common.hpp"
 #include "result_stream_faker.hpp"
 #include "storage/v3/isolation_level.hpp"
+#include "storage/v3/key_store.hpp"
+#include "storage/v3/name_id_mapper.hpp"
 #include "storage/v3/property_value.hpp"
 #include "utils/csv_parsing.hpp"
 #include "utils/logging.hpp"
 
 namespace memgraph::query::v2::tests {
 
-// auto ToEdgeList(const memgraph::communication::bolt::Value &v) {
-//   std::vector<memgraph::communication::bolt::Edge> list;
-//   for (auto x : v.ValueList()) {
-//     list.push_back(x.ValueEdge());
-//   }
-//   return list;
-// }
+auto ToEdgeList(const memgraph::communication::bolt::Value &v) {
+  std::vector<memgraph::communication::bolt::Edge> list;
+  for (auto x : v.ValueList()) {
+    list.push_back(x.ValueEdge());
+  }
+  return list;
+}
 
-// auto StringToUnorderedSet(const std::string &element) {
-//   const auto element_split = memgraph::utils::Split(element, ", ");
-//   return std::unordered_set<std::string>(element_split.begin(), element_split.end());
-// };
+auto StringToUnorderedSet(const std::string &element) {
+  const auto element_split = memgraph::utils::Split(element, ", ");
+  return std::unordered_set<std::string>(element_split.begin(), element_split.end());
+};
 
 struct InterpreterFaker {
-  InterpreterFaker(memgraph::storage::v3::Storage *db, const memgraph::query::v2::InterpreterConfig config,
+  InterpreterFaker(memgraph::storage::v3::Shard *db, const memgraph::query::v2::InterpreterConfig config,
                    const std::filesystem::path &data_directory)
       : interpreter_context(db, config, data_directory), interpreter(&interpreter_context) {
     interpreter_context.auth_checker = &auth_checker;
@@ -113,10 +115,20 @@ class InterpreterTest : public ::testing::Test {
     return default_interpreter.Interpret(query, params);
   }
 
-  memgraph::storage::v3::Storage db_;
+  storage::v3::LabelId NameToLabelId(std::string_view label_name) {
+    return storage::v3::LabelId::FromUint(id_mapper.NameToId(label_name));
+  }
+
+  storage::v3::PropertyId NameToPropertyId(std::string_view property_name) {
+    return storage::v3::PropertyId::FromUint(id_mapper.NameToId(property_name));
+  }
+
+  storage::v3::PrimaryKey pk{storage::v3::PropertyValue(0)};
+  memgraph::storage::v3::NameIdMapper id_mapper;
+  const storage::v3::LabelId label{NameToLabelId("label")};
+  storage::v3::Shard db_{label, pk, std::nullopt};
   std::filesystem::path data_directory{std::filesystem::temp_directory_path() / "MG_tests_unit_query_v2_interpreter"};
-  const storage::v3::LabelId label{db_.NameToLabel("label")};
-  const storage::v3::PropertyId property{db_.NameToProperty("property")};
+  const storage::v3::PropertyId property{NameToPropertyId("property")};
   InterpreterFaker default_interpreter{&db_, {}, data_directory};
 };
 
@@ -150,8 +162,8 @@ TEST_F(InterpreterTest, DummyTestToForceQueryV2Compilation) {
 //   }
 // }
 
-// // Run query with different ast twice to see if query executes correctly when
-// // ast is read from cache.
+// Run query with different ast twice to see if query executes correctly when
+// ast is read from cache.
 // TEST_F(InterpreterTest, AstCache) {
 //   {
 //     auto stream = Interpret("RETURN 2 + 3");
@@ -1447,8 +1459,8 @@ TEST_F(InterpreterTest, DummyTestToForceQueryV2Compilation) {
 //     auto notification = notifications[0].ValueMap();
 //     ASSERT_EQ(notification["severity"].ValueString(), "INFO");
 //     ASSERT_EQ(notification["code"].ValueString(), "ConstraintDoesNotExist");
-//     ASSERT_EQ(notification["title"].ValueString(), "Constraint EXISTS on label L1 on properties name doesn't
-//     exist."); ASSERT_EQ(notification["description"].ValueString(), "");
+//     ASSERT_EQ(notification["title"].ValueString(), "Constraint EXISTS on label L1 on properties name doesn'texist.");
+//     ASSERT_EQ(notification["description"].ValueString(), "");
 //   }
 // }
 
diff --git a/tests/unit/result_stream_faker.hpp b/tests/unit/result_stream_faker.hpp
index 60c5884e7..6e7aab261 100644
--- a/tests/unit/result_stream_faker.hpp
+++ b/tests/unit/result_stream_faker.hpp
@@ -26,7 +26,7 @@
  */
 class ResultStreamFaker {
  public:
-  explicit ResultStreamFaker(memgraph::storage::v3::Storage *store) : store_(store) {}
+  explicit ResultStreamFaker(memgraph::storage::v3::Shard *store) : store_(store) {}
 
   ResultStreamFaker(const ResultStreamFaker &) = delete;
   ResultStreamFaker &operator=(const ResultStreamFaker &) = delete;
@@ -124,7 +124,7 @@ class ResultStreamFaker {
   }
 
  private:
-  memgraph::storage::v3::Storage *store_;
+  memgraph::storage::v3::Shard *store_;
   // the data that the record stream can accept
   std::vector<std::string> header_;
   std::vector<std::vector<memgraph::communication::bolt::Value>> results_;
diff --git a/tests/unit/storage_v3.cpp b/tests/unit/storage_v3.cpp
index fbcd85704..4163c96a2 100644
--- a/tests/unit/storage_v3.cpp
+++ b/tests/unit/storage_v3.cpp
@@ -19,12 +19,14 @@
 #include "storage/v3/delta.hpp"
 #include "storage/v3/id_types.hpp"
 #include "storage/v3/key_store.hpp"
+#include "storage/v3/name_id_mapper.hpp"
 #include "storage/v3/property_value.hpp"
 #include "storage/v3/result.hpp"
 #include "storage/v3/schema_validator.hpp"
-#include "storage/v3/storage.hpp"
+#include "storage/v3/shard.hpp"
 #include "storage/v3/vertex_accessor.hpp"
 #include "storage_v3_test_utils.hpp"
+#include "utils/exceptions.hpp"
 
 using testing::UnorderedElementsAre;
 
@@ -37,7 +39,7 @@ class StorageV3 : public ::testing::Test {
         store.CreateSchema(primary_label, {storage::v3::SchemaProperty{primary_property, common::SchemaType::INT}}));
   }
 
-  VertexAccessor CreateVertexAndValidate(Storage::Accessor &acc, LabelId primary_label,
+  VertexAccessor CreateVertexAndValidate(Shard::Accessor &acc, LabelId primary_label,
                                          const std::vector<LabelId> &labels,
                                          const std::vector<std::pair<PropertyId, PropertyValue>> &properties) {
     auto vtx = acc.CreateVertexAndValidate(primary_label, labels, properties);
@@ -45,10 +47,21 @@ class StorageV3 : public ::testing::Test {
     return *vtx;
   }
 
-  Storage store;
-  const LabelId primary_label{store.NameToLabel("label")};
-  const PropertyId primary_property{store.NameToProperty("property")};
+  LabelId NameToLabelId(std::string_view label_name) { return LabelId::FromUint(id_mapper.NameToId(label_name)); }
+
+  PropertyId NameToPropertyId(std::string_view property_name) {
+    return PropertyId::FromUint(id_mapper.NameToId(property_name));
+  }
+
+  EdgeTypeId NameToEdgeTypeId(std::string_view edge_type_name) {
+    return EdgeTypeId::FromUint(id_mapper.NameToId(edge_type_name));
+  }
+
+  NameIdMapper id_mapper;
   const std::vector<PropertyValue> pk{PropertyValue{0}};
+  const LabelId primary_label{NameToLabelId("label")};
+  Shard store{primary_label, pk, std::nullopt};
+  const PropertyId primary_property{NameToPropertyId("property")};
 };
 
 // NOLINTNEXTLINE(hicpp-special-member-functions)
@@ -236,7 +249,7 @@ TEST_F(StorageV3, AccessorMove) {
     ASSERT_TRUE(acc.FindVertex(pk, View::NEW).has_value());
     EXPECT_EQ(CountVertices(acc, View::NEW), 1U);
 
-    Storage::Accessor moved(std::move(acc));
+    Shard::Accessor moved(std::move(acc));
 
     ASSERT_FALSE(moved.FindVertex(pk, View::OLD).has_value());
     EXPECT_EQ(CountVertices(moved, View::OLD), 0U);
@@ -600,7 +613,7 @@ TEST_F(StorageV3, VertexDeleteLabel) {
     auto vertex = acc.FindVertex(pk, View::NEW);
     ASSERT_TRUE(vertex);
 
-    auto label5 = acc.NameToLabel("label5");
+    auto label5 = NameToLabelId("label5");
 
     // Check whether label 5 exists
     ASSERT_FALSE(vertex->HasLabel(label5, View::OLD).GetValue());
@@ -651,7 +664,7 @@ TEST_F(StorageV3, VertexDeleteLabel) {
     auto vertex = acc.FindVertex(pk, View::NEW);
     ASSERT_TRUE(vertex);
 
-    auto label5 = acc.NameToLabel("label5");
+    auto label5 = NameToLabelId("label5");
 
     // Check whether label 5 exists
     ASSERT_FALSE(vertex->HasLabel(label5, View::OLD).GetValue());
@@ -744,7 +757,7 @@ TEST_F(StorageV3, VertexDeleteProperty) {
     auto vertex = acc.FindVertex(pk, View::NEW);
     ASSERT_TRUE(vertex);
 
-    auto property5 = acc.NameToProperty("property5");
+    auto property5 = NameToPropertyId("property5");
 
     // Check whether property 5 exists
     ASSERT_TRUE(vertex->GetProperty(property5, View::OLD)->IsNull());
@@ -790,7 +803,7 @@ TEST_F(StorageV3, VertexDeleteProperty) {
     auto vertex = acc.FindVertex(pk, View::NEW);
     ASSERT_TRUE(vertex);
 
-    auto property5 = acc.NameToProperty("property5");
+    auto property5 = NameToPropertyId("property5");
 
     // Check whether property 5 exists
     ASSERT_TRUE(vertex->GetProperty(property5, View::OLD)->IsNull());
@@ -866,7 +879,7 @@ TEST_F(StorageV3, VertexLabelCommit) {
     auto acc = store.Access();
     auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}});
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_FALSE(vertex.HasLabel(label, View::NEW).GetValue());
     ASSERT_EQ(vertex.Labels(View::NEW)->size(), 0);
@@ -897,7 +910,7 @@ TEST_F(StorageV3, VertexLabelCommit) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_TRUE(vertex->HasLabel(label, View::OLD).GetValue());
     {
@@ -913,7 +926,7 @@ TEST_F(StorageV3, VertexLabelCommit) {
       ASSERT_EQ(labels[0], label);
     }
 
-    auto other_label = acc.NameToLabel("other");
+    auto other_label = NameToLabelId("other");
 
     ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue());
@@ -925,7 +938,7 @@ TEST_F(StorageV3, VertexLabelCommit) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     {
       auto res = vertex->RemoveLabelAndValidate(label);
@@ -956,14 +969,14 @@ TEST_F(StorageV3, VertexLabelCommit) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_FALSE(vertex->HasLabel(label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue());
     ASSERT_EQ(vertex->Labels(View::OLD)->size(), 0);
     ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0);
 
-    auto other_label = acc.NameToLabel("other");
+    auto other_label = NameToLabelId("other");
 
     ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue());
@@ -987,7 +1000,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue());
     ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0);
@@ -1020,14 +1033,14 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_FALSE(vertex->HasLabel(label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue());
     ASSERT_EQ(vertex->Labels(View::OLD)->size(), 0);
     ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0);
 
-    auto other_label = acc.NameToLabel("other");
+    auto other_label = NameToLabelId("other");
 
     ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue());
@@ -1041,7 +1054,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue());
     ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0);
@@ -1074,7 +1087,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_TRUE(vertex->HasLabel(label, View::OLD).GetValue());
     {
@@ -1090,7 +1103,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
       ASSERT_EQ(labels[0], label);
     }
 
-    auto other_label = acc.NameToLabel("other");
+    auto other_label = NameToLabelId("other");
 
     ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue());
@@ -1104,7 +1117,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     {
       auto res = vertex->RemoveLabelAndValidate(label);
@@ -1137,7 +1150,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_TRUE(vertex->HasLabel(label, View::OLD).GetValue());
     {
@@ -1153,7 +1166,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
       ASSERT_EQ(labels[0], label);
     }
 
-    auto other_label = acc.NameToLabel("other");
+    auto other_label = NameToLabelId("other");
 
     ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue());
@@ -1167,7 +1180,7 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     {
       auto res = vertex->RemoveLabelAndValidate(label);
@@ -1200,14 +1213,14 @@ TEST_F(StorageV3, VertexLabelAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label = acc.NameToLabel("label5");
+    auto label = NameToLabelId("label5");
 
     ASSERT_FALSE(vertex->HasLabel(label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue());
     ASSERT_EQ(vertex->Labels(View::OLD)->size(), 0);
     ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0);
 
-    auto other_label = acc.NameToLabel("other");
+    auto other_label = NameToLabelId("other");
 
     ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue());
@@ -1232,8 +1245,8 @@ TEST_F(StorageV3, VertexLabelSerializationError) {
     auto vertex = acc1.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label1 = acc1.NameToLabel("label1");
-    auto label2 = acc1.NameToLabel("label2");
+    auto label1 = NameToLabelId("label1");
+    auto label2 = NameToLabelId("label2");
 
     ASSERT_FALSE(vertex->HasLabel(label1, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(label1, View::NEW).GetValue());
@@ -1271,8 +1284,8 @@ TEST_F(StorageV3, VertexLabelSerializationError) {
     auto vertex = acc2.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label1 = acc2.NameToLabel("label1");
-    auto label2 = acc2.NameToLabel("label2");
+    auto label1 = NameToLabelId("label1");
+    auto label2 = NameToLabelId("label2");
 
     ASSERT_FALSE(vertex->HasLabel(label1, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(label1, View::NEW).GetValue());
@@ -1297,8 +1310,8 @@ TEST_F(StorageV3, VertexLabelSerializationError) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto label1 = acc.NameToLabel("label1");
-    auto label2 = acc.NameToLabel("label2");
+    auto label1 = NameToLabelId("label1");
+    auto label2 = NameToLabelId("label2");
 
     ASSERT_TRUE(vertex->HasLabel(label1, View::OLD).GetValue());
     ASSERT_FALSE(vertex->HasLabel(label2, View::OLD).GetValue());
@@ -1326,7 +1339,7 @@ TEST_F(StorageV3, VertexPropertyCommit) {
     auto acc = store.Access();
     auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}});
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_TRUE(vertex.GetProperty(property, View::NEW)->IsNull());
     ASSERT_EQ(vertex.Properties(View::NEW)->size(), 0);
@@ -1364,7 +1377,7 @@ TEST_F(StorageV3, VertexPropertyCommit) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare");
     {
@@ -1380,7 +1393,7 @@ TEST_F(StorageV3, VertexPropertyCommit) {
       ASSERT_EQ(properties[property].ValueString(), "nandare");
     }
 
-    auto other_property = acc.NameToProperty("other");
+    auto other_property = NameToPropertyId("other");
 
     ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull());
@@ -1392,7 +1405,7 @@ TEST_F(StorageV3, VertexPropertyCommit) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     {
       auto old_value = vertex->SetPropertyAndValidate(property, PropertyValue());
@@ -1423,14 +1436,14 @@ TEST_F(StorageV3, VertexPropertyCommit) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_TRUE(vertex->GetProperty(property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull());
     ASSERT_EQ(vertex->Properties(View::OLD)->size(), 0);
     ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0);
 
-    auto other_property = acc.NameToProperty("other");
+    auto other_property = NameToPropertyId("other");
 
     ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull());
@@ -1454,7 +1467,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull());
     ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0);
@@ -1494,14 +1507,14 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_TRUE(vertex->GetProperty(property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull());
     ASSERT_EQ(vertex->Properties(View::OLD)->size(), 0);
     ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0);
 
-    auto other_property = acc.NameToProperty("other");
+    auto other_property = NameToPropertyId("other");
 
     ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull());
@@ -1515,7 +1528,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull());
     ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0);
@@ -1555,7 +1568,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare");
     {
@@ -1571,7 +1584,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
       ASSERT_EQ(properties[property].ValueString(), "nandare");
     }
 
-    auto other_property = acc.NameToProperty("other");
+    auto other_property = NameToPropertyId("other");
 
     ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull());
@@ -1585,7 +1598,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare");
     {
@@ -1626,7 +1639,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare");
     {
@@ -1642,7 +1655,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
       ASSERT_EQ(properties[property].ValueString(), "nandare");
     }
 
-    auto other_property = acc.NameToProperty("other");
+    auto other_property = NameToPropertyId("other");
 
     ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull());
@@ -1656,7 +1669,7 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare");
     {
@@ -1697,14 +1710,14 @@ TEST_F(StorageV3, VertexPropertyAbort) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property = acc.NameToProperty("property5");
+    auto property = NameToPropertyId("property5");
 
     ASSERT_TRUE(vertex->GetProperty(property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull());
     ASSERT_EQ(vertex->Properties(View::OLD)->size(), 0);
     ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0);
 
-    auto other_property = acc.NameToProperty("other");
+    auto other_property = NameToPropertyId("other");
 
     ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull());
@@ -1729,8 +1742,8 @@ TEST_F(StorageV3, VertexPropertySerializationError) {
     auto vertex = acc1.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property1 = acc1.NameToProperty("property1");
-    auto property2 = acc1.NameToProperty("property2");
+    auto property1 = NameToPropertyId("property1");
+    auto property2 = NameToPropertyId("property2");
 
     ASSERT_TRUE(vertex->GetProperty(property1, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(property1, View::NEW)->IsNull());
@@ -1762,8 +1775,8 @@ TEST_F(StorageV3, VertexPropertySerializationError) {
     auto vertex = acc2.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property1 = acc2.NameToProperty("property1");
-    auto property2 = acc2.NameToProperty("property2");
+    auto property1 = NameToPropertyId("property1");
+    auto property2 = NameToPropertyId("property2");
 
     ASSERT_TRUE(vertex->GetProperty(property1, View::OLD)->IsNull());
     ASSERT_TRUE(vertex->GetProperty(property1, View::NEW)->IsNull());
@@ -1788,8 +1801,8 @@ TEST_F(StorageV3, VertexPropertySerializationError) {
     auto vertex = acc.FindVertex(pk, View::OLD);
     ASSERT_TRUE(vertex);
 
-    auto property1 = acc.NameToProperty("property1");
-    auto property2 = acc.NameToProperty("property2");
+    auto property1 = NameToPropertyId("property1");
+    auto property2 = NameToPropertyId("property2");
 
     ASSERT_EQ(vertex->GetProperty(property1, View::OLD)->ValueInt(), 123);
     ASSERT_TRUE(vertex->GetProperty(property2, View::OLD)->IsNull());
@@ -1816,8 +1829,8 @@ TEST_F(StorageV3, VertexLabelPropertyMixed) {
   auto acc = store.Access();
   auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}});
 
-  auto label = acc.NameToLabel("label5");
-  auto property = acc.NameToProperty("property5");
+  auto label = NameToLabelId("label5");
+  auto property = NameToPropertyId("property5");
 
   // Check whether label 5 and property 5 exist
   ASSERT_FALSE(vertex.HasLabel(label, View::NEW).GetValue());
@@ -2051,8 +2064,8 @@ TEST_F(StorageV3, VertexLabelPropertyMixed) {
 }
 
 TEST_F(StorageV3, VertexPropertyClear) {
-  auto property1 = store.NameToProperty("property1");
-  auto property2 = store.NameToProperty("property2");
+  auto property1 = NameToPropertyId("property1");
+  auto property2 = NameToPropertyId("property2");
   {
     auto acc = store.Access();
     auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}});
@@ -2153,8 +2166,8 @@ TEST_F(StorageV3, VertexPropertyClear) {
 }
 
 TEST_F(StorageV3, VertexNonexistentLabelPropertyEdgeAPI) {
-  auto label1 = store.NameToLabel("label1");
-  auto property1 = store.NameToProperty("property1");
+  auto label1 = NameToLabelId("label1");
+  auto property1 = NameToPropertyId("property1");
 
   auto acc = store.Access();
   auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}});
@@ -2182,7 +2195,7 @@ TEST_F(StorageV3, VertexNonexistentLabelPropertyEdgeAPI) {
   // Modify vertex.
   ASSERT_TRUE(vertex.AddLabelAndValidate(label1).HasValue());
   ASSERT_TRUE(vertex.SetPropertyAndValidate(property1, PropertyValue("value")).HasValue());
-  ASSERT_TRUE(acc.CreateEdge(&vertex, &vertex, acc.NameToEdgeType("edge")).HasValue());
+  ASSERT_TRUE(acc.CreateEdge(&vertex, &vertex, NameToEdgeTypeId("edge")).HasValue());
 
   // Check state after (OLD view).
   ASSERT_EQ(vertex.Labels(View::OLD).GetError(), Error::NONEXISTENT_OBJECT);
@@ -2218,14 +2231,14 @@ TEST_F(StorageV3, VertexVisibilitySingleTransaction) {
   EXPECT_FALSE(acc2.FindVertex(pk, View::OLD));
   EXPECT_FALSE(acc2.FindVertex(pk, View::NEW));
 
-  ASSERT_TRUE(vertex.AddLabelAndValidate(acc1.NameToLabel("label1")).HasValue());
+  ASSERT_TRUE(vertex.AddLabelAndValidate(NameToLabelId("label1")).HasValue());
 
   EXPECT_FALSE(acc1.FindVertex(pk, View::OLD));
   EXPECT_TRUE(acc1.FindVertex(pk, View::NEW));
   EXPECT_FALSE(acc2.FindVertex(pk, View::OLD));
   EXPECT_FALSE(acc2.FindVertex(pk, View::NEW));
 
-  ASSERT_TRUE(vertex.SetPropertyAndValidate(acc1.NameToProperty("meaning"), PropertyValue(42)).HasValue());
+  ASSERT_TRUE(vertex.SetPropertyAndValidate(NameToPropertyId("meaning"), PropertyValue(42)).HasValue());
 
   auto acc3 = store.Access();
 
@@ -2302,7 +2315,7 @@ TEST_F(StorageV3, VertexVisibilityMultipleTransactions) {
     EXPECT_TRUE(acc2.FindVertex(pk, View::OLD));
     EXPECT_TRUE(acc2.FindVertex(pk, View::NEW));
 
-    ASSERT_TRUE(vertex->AddLabelAndValidate(acc1.NameToLabel("label1")).HasValue());
+    ASSERT_TRUE(vertex->AddLabelAndValidate(NameToLabelId("label1")).HasValue());
 
     EXPECT_TRUE(acc1.FindVertex(pk, View::OLD));
     EXPECT_TRUE(acc1.FindVertex(pk, View::NEW));
@@ -2323,7 +2336,7 @@ TEST_F(StorageV3, VertexVisibilityMultipleTransactions) {
     EXPECT_TRUE(acc2.FindVertex(pk, View::OLD));
     EXPECT_TRUE(acc2.FindVertex(pk, View::NEW));
 
-    ASSERT_TRUE(vertex->SetPropertyAndValidate(acc1.NameToProperty("meaning"), PropertyValue(42)).HasValue());
+    ASSERT_TRUE(vertex->SetPropertyAndValidate(NameToPropertyId("meaning"), PropertyValue(42)).HasValue());
 
     auto acc3 = store.Access();
 
@@ -2497,7 +2510,7 @@ TEST_F(StorageV3, VertexVisibilityMultipleTransactions) {
 
 // NOLINTNEXTLINE(hicpp-special-member-functions)
 TEST_F(StorageV3, DeletedVertexAccessor) {
-  const auto property1 = store.NameToProperty("property1");
+  const auto property1 = NameToPropertyId("property1");
   const PropertyValue property_value{"property_value"};
 
   // Create the vertex
@@ -2538,8 +2551,8 @@ TEST_F(StorageV3, DeletedVertexAccessor) {
 TEST_F(StorageV3, TestCreateVertexAndValidate) {
   {
     auto acc = store.Access();
-    const auto label1 = store.NameToLabel("label1");
-    const auto prop1 = store.NameToProperty("prop1");
+    const auto label1 = NameToLabelId("label1");
+    const auto prop1 = NameToPropertyId("prop1");
     auto vertex = acc.CreateVertexAndValidate(primary_label, {label1},
                                               {{primary_property, PropertyValue(0)}, {prop1, PropertyValue(111)}});
     ASSERT_TRUE(vertex.HasValue());
@@ -2552,25 +2565,20 @@ TEST_F(StorageV3, TestCreateVertexAndValidate) {
               (std::map<PropertyId, PropertyValue>{{prop1, PropertyValue(111)}}));
   }
   {
-    const auto label1 = store.NameToLabel("new_primary_label");
-    const auto prop1 = store.NameToProperty("key1");
-    const auto prop2 = store.NameToProperty("key2");
-    ASSERT_TRUE(store.CreateSchema(
-        label1, {SchemaProperty{prop1, common::SchemaType::INT}, SchemaProperty{prop2, common::SchemaType::STRING}}));
     auto acc = store.Access();
-    auto vertex = acc.CreateVertexAndValidate(label1, {}, {{prop1, PropertyValue(21)}, {prop2, PropertyValue("test")}});
-    ASSERT_TRUE(vertex.HasValue());
-    ASSERT_TRUE(vertex->PrimaryLabel(View::NEW).HasValue());
-    EXPECT_EQ(vertex->PrimaryLabel(View::NEW).GetValue(), label1);
-    ASSERT_TRUE(vertex->PrimaryKey(View::NEW).HasValue());
-    EXPECT_EQ(vertex->PrimaryKey(View::NEW).GetValue(), (PrimaryKey{{PropertyValue(21), PropertyValue("test")}}));
-    ASSERT_TRUE(vertex->Properties(View::NEW).HasValue());
-    EXPECT_TRUE(vertex->Properties(View::NEW).GetValue().empty());
+    const auto label1 = NameToLabelId("label1");
+    const auto prop1 = NameToPropertyId("prop1");
+    EXPECT_THROW(
+        {
+          auto vertex = acc.CreateVertexAndValidate(
+              label1, {}, {{primary_property, PropertyValue(0)}, {prop1, PropertyValue(111)}});
+        },
+        utils::BasicException);
   }
   {
     ASSERT_DEATH(
         {
-          Storage store;
+          Shard store(primary_label, pk, std::nullopt);
           ASSERT_TRUE(store.CreateSchema(primary_label,
                                          {storage::v3::SchemaProperty{primary_property, common::SchemaType::INT}}));
           auto acc = store.Access();
diff --git a/tests/unit/storage_v3_test_utils.cpp b/tests/unit/storage_v3_test_utils.cpp
index b1a59b7aa..4a8c7b310 100644
--- a/tests/unit/storage_v3_test_utils.cpp
+++ b/tests/unit/storage_v3_test_utils.cpp
@@ -13,7 +13,7 @@
 
 namespace memgraph::storage::v3::tests {
 
-size_t CountVertices(Storage::Accessor &storage_accessor, View view) {
+size_t CountVertices(Shard::Accessor &storage_accessor, View view) {
   auto vertices = storage_accessor.Vertices(view);
   size_t count = 0U;
   for (auto it = vertices.begin(); it != vertices.end(); ++it, ++count)
diff --git a/tests/unit/storage_v3_test_utils.hpp b/tests/unit/storage_v3_test_utils.hpp
index 1cc772036..7cbb7414b 100644
--- a/tests/unit/storage_v3_test_utils.hpp
+++ b/tests/unit/storage_v3_test_utils.hpp
@@ -16,6 +16,6 @@
 
 namespace memgraph::storage::v3::tests {
 
-size_t CountVertices(Storage::Accessor &storage_accessor, View view);
+size_t CountVertices(Shard::Accessor &storage_accessor, View view);
 
 }  // namespace memgraph::storage::v3::tests
diff --git a/tests/unit/storage_v3_vertex_accessors.cpp b/tests/unit/storage_v3_vertex_accessors.cpp
index dcf94788a..8dea70e59 100644
--- a/tests/unit/storage_v3_vertex_accessors.cpp
+++ b/tests/unit/storage_v3_vertex_accessors.cpp
@@ -21,7 +21,7 @@
 #include "storage/v3/property_value.hpp"
 #include "storage/v3/result.hpp"
 #include "storage/v3/schema_validator.hpp"
-#include "storage/v3/storage.hpp"
+#include "storage/v3/shard.hpp"
 #include "storage/v3/vertex_accessor.hpp"
 #include "storage_v3_test_utils.hpp"
 
@@ -35,7 +35,7 @@ class StorageV3Accessor : public ::testing::Test {
     ASSERT_TRUE(storage.CreateSchema(primary_label, {SchemaProperty{primary_property, common::SchemaType::INT}}));
   }
 
-  VertexAccessor CreateVertexAndValidate(Storage::Accessor &acc, LabelId primary_label,
+  VertexAccessor CreateVertexAndValidate(Shard::Accessor &acc, LabelId primary_label,
                                          const std::vector<LabelId> &labels,
                                          const std::vector<std::pair<PropertyId, PropertyValue>> &properties) {
     auto vtx = acc.CreateVertexAndValidate(primary_label, labels, properties);
@@ -43,9 +43,17 @@ class StorageV3Accessor : public ::testing::Test {
     return *vtx;
   }
 
-  Storage storage;
-  const LabelId primary_label{storage.NameToLabel("label")};
-  const PropertyId primary_property{storage.NameToProperty("property")};
+  LabelId NameToLabelId(std::string_view label_name) { return LabelId::FromUint(id_mapper.NameToId(label_name)); }
+
+  PropertyId NameToPropertyId(std::string_view property_name) {
+    return PropertyId::FromUint(id_mapper.NameToId(property_name));
+  }
+
+  const std::vector<PropertyValue> pk{PropertyValue{0}};
+  NameIdMapper id_mapper;
+  Shard storage{NameToLabelId("label"), pk, std::nullopt};
+  const LabelId primary_label{NameToLabelId("label")};
+  const PropertyId primary_property{NameToPropertyId("property")};
 };
 
 TEST_F(StorageV3Accessor, TestPrimaryLabel) {
@@ -85,9 +93,9 @@ TEST_F(StorageV3Accessor, TestPrimaryLabel) {
 TEST_F(StorageV3Accessor, TestAddLabels) {
   {
     auto acc = storage.Access();
-    const auto label1 = storage.NameToLabel("label1");
-    const auto label2 = storage.NameToLabel("label2");
-    const auto label3 = storage.NameToLabel("label3");
+    const auto label1 = NameToLabelId("label1");
+    const auto label2 = NameToLabelId("label2");
+    const auto label3 = NameToLabelId("label3");
     const auto vertex =
         CreateVertexAndValidate(acc, primary_label, {label1, label2, label3}, {{primary_property, PropertyValue(0)}});
     ASSERT_TRUE(vertex.Labels(View::NEW).HasValue());
@@ -96,9 +104,9 @@ TEST_F(StorageV3Accessor, TestAddLabels) {
   }
   {
     auto acc = storage.Access();
-    const auto label1 = storage.NameToLabel("label1");
-    const auto label2 = storage.NameToLabel("label2");
-    const auto label3 = storage.NameToLabel("label3");
+    const auto label1 = NameToLabelId("label1");
+    const auto label2 = NameToLabelId("label2");
+    const auto label3 = NameToLabelId("label3");
     auto vertex = CreateVertexAndValidate(acc, primary_label, {label1}, {{primary_property, PropertyValue(1)}});
     ASSERT_TRUE(vertex.Labels(View::NEW).HasValue());
     ASSERT_FALSE(vertex.Labels(View::OLD).HasValue());
@@ -111,7 +119,7 @@ TEST_F(StorageV3Accessor, TestAddLabels) {
   }
   {
     auto acc = storage.Access();
-    const auto label1 = storage.NameToLabel("label");
+    const auto label1 = NameToLabelId("label");
     auto vertex = acc.CreateVertexAndValidate(primary_label, {label1}, {{primary_property, PropertyValue(2)}});
     ASSERT_TRUE(vertex.HasError());
     ASSERT_TRUE(std::holds_alternative<SchemaViolation>(vertex.GetError()));
@@ -120,7 +128,7 @@ TEST_F(StorageV3Accessor, TestAddLabels) {
   }
   {
     auto acc = storage.Access();
-    const auto label1 = storage.NameToLabel("label");
+    const auto label1 = NameToLabelId("label");
     auto vertex = acc.CreateVertexAndValidate(primary_label, {}, {{primary_property, PropertyValue(3)}});
     ASSERT_TRUE(vertex.HasValue());
     const auto schema_violation = vertex->AddLabelAndValidate(label1);
@@ -134,9 +142,9 @@ TEST_F(StorageV3Accessor, TestAddLabels) {
 TEST_F(StorageV3Accessor, TestRemoveLabels) {
   {
     auto acc = storage.Access();
-    const auto label1 = storage.NameToLabel("label1");
-    const auto label2 = storage.NameToLabel("label2");
-    const auto label3 = storage.NameToLabel("label3");
+    const auto label1 = NameToLabelId("label1");
+    const auto label2 = NameToLabelId("label2");
+    const auto label3 = NameToLabelId("label3");
     auto vertex =
         CreateVertexAndValidate(acc, primary_label, {label1, label2, label3}, {{primary_property, PropertyValue(0)}});
     ASSERT_TRUE(vertex.Labels(View::NEW).HasValue());
@@ -157,7 +165,7 @@ TEST_F(StorageV3Accessor, TestRemoveLabels) {
   }
   {
     auto acc = storage.Access();
-    const auto label1 = storage.NameToLabel("label1");
+    const auto label1 = NameToLabelId("label1");
     auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue(1)}});
     ASSERT_TRUE(vertex.Labels(View::NEW).HasValue());
     EXPECT_TRUE(vertex.Labels(View::NEW).GetValue().empty());
@@ -179,7 +187,7 @@ TEST_F(StorageV3Accessor, TestRemoveLabels) {
 TEST_F(StorageV3Accessor, TestSetKeysAndProperties) {
   {
     auto acc = storage.Access();
-    const PropertyId prop1{storage.NameToProperty("prop1")};
+    const PropertyId prop1{NameToPropertyId("prop1")};
     auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue(0)}});
     const auto res = vertex.SetPropertyAndValidate(prop1, PropertyValue(1));
     ASSERT_TRUE(res.HasValue());